mirror of https://github.com/sbt/sbt.git
Introduce FileChanges
Prior to this commit, change tracking in sbt 1.3.0 was done via the changed(Input|Output)Files tasks which were tasks returning Option[ChangedFiles]. The ChangedFiles case class was defined in io as case class ChangedFiles(created: Seq[Path], deleted: Seq[Path], updated: Seq[Path]) When no changes were found, or if there were no previous stamps, the changed(Input|Output)Files tasks returned None. This made it impossible to tell whether nothing had changed or if it was the first time. Moreover, the api was awkward as it required pattern matching or folding the result into a default value. To address these limitations, I introduce the FileChanges class. It can be generated regardless of whether or not previous file stamps were available. The changes contains all of the created, deleted, modified and unmodified files so that the user can directly call these methods without having to pattern match.
This commit is contained in:
parent
8e9efbeaac
commit
8ce2578060
|
|
@ -69,7 +69,7 @@ import sbt.librarymanagement.CrossVersion.{ binarySbtVersion, binaryScalaVersion
|
|||
import sbt.librarymanagement._
|
||||
import sbt.librarymanagement.ivy._
|
||||
import sbt.librarymanagement.syntax._
|
||||
import sbt.nio.Watch
|
||||
import sbt.nio.{ FileChanges, Watch }
|
||||
import sbt.nio.Keys._
|
||||
import sbt.nio.file.{ FileTreeView, Glob, RecursiveGlob }
|
||||
import sbt.nio.file.syntax._
|
||||
|
|
@ -609,7 +609,10 @@ object Defaults extends BuildCommon {
|
|||
val current =
|
||||
(unmanagedSources / inputFileStamps).value ++ (managedSources / outputFileStamps).value
|
||||
val previous = (externalHooks / inputFileStamps).previous
|
||||
ExternalHooks.default.value(previous.flatMap(sbt.nio.Settings.changedFiles(_, current)))
|
||||
val changes = previous
|
||||
.map(sbt.nio.Settings.changedFiles(_, current))
|
||||
.getOrElse(FileChanges.noPrevious(current.map(_._1)))
|
||||
ExternalHooks.default.value(changes)
|
||||
},
|
||||
externalHooks / inputFileStamps := {
|
||||
compile.value // ensures the inputFileStamps previous value is only set if compile succeeds.
|
||||
|
|
|
|||
|
|
@ -16,9 +16,9 @@ import sbt.internal.inc.ExternalLookup
|
|||
import sbt.internal.inc.Stamp.equivStamp.equiv
|
||||
import sbt.io.syntax._
|
||||
import sbt.nio.Keys._
|
||||
import sbt.nio.file.RecursiveGlob
|
||||
import sbt.nio.file.syntax._
|
||||
import sbt.nio.file.{ ChangedFiles, RecursiveGlob }
|
||||
import sbt.nio.{ FileStamp, FileStamper }
|
||||
import sbt.nio.{ FileChanges, FileStamp, FileStamper }
|
||||
import xsbti.compile._
|
||||
import xsbti.compile.analysis.Stamp
|
||||
|
||||
|
|
@ -26,7 +26,7 @@ import scala.collection.JavaConverters._
|
|||
|
||||
private[sbt] object ExternalHooks {
|
||||
private val javaHome = Option(System.getProperty("java.home")).map(Paths.get(_))
|
||||
def default: Def.Initialize[sbt.Task[Option[ChangedFiles] => ExternalHooks]] = Def.task {
|
||||
def default: Def.Initialize[sbt.Task[FileChanges => ExternalHooks]] = Def.task {
|
||||
val unmanagedCache = unmanagedFileStampCache.value
|
||||
val managedCache = managedFileStampCache.value
|
||||
val cp = dependencyClasspath.value.map(_.data)
|
||||
|
|
@ -42,7 +42,7 @@ private[sbt] object ExternalHooks {
|
|||
apply(_, options, unmanagedCache, managedCache)
|
||||
}
|
||||
private def apply(
|
||||
changedFiles: Option[ChangedFiles],
|
||||
changedFiles: FileChanges,
|
||||
options: CompileOptions,
|
||||
unmanagedCache: FileStamp.Cache,
|
||||
managedCache: FileStamp.Cache
|
||||
|
|
@ -59,11 +59,12 @@ private[sbt] object ExternalHooks {
|
|||
}
|
||||
private def add(f: File, set: java.util.Set[File]): Unit = { set.add(f); () }
|
||||
val allChanges = new java.util.HashSet[File]
|
||||
changedFiles foreach {
|
||||
case ChangedFiles(c, d, u) =>
|
||||
changedFiles match {
|
||||
case FileChanges(c, d, m, _) =>
|
||||
c.foreach(add(_, getAdded, allChanges))
|
||||
d.foreach(add(_, getRemoved, allChanges))
|
||||
u.foreach(add(_, getChanged, allChanges))
|
||||
m.foreach(add(_, getChanged, allChanges))
|
||||
case _ =>
|
||||
}
|
||||
override def isEmpty: java.lang.Boolean =
|
||||
getAdded.isEmpty && getRemoved.isEmpty && getChanged.isEmpty
|
||||
|
|
|
|||
|
|
@ -10,9 +10,8 @@ package internal
|
|||
|
||||
import java.nio.file.{ Path => NioPath }
|
||||
|
||||
import sbt.nio.FileStamp
|
||||
import sbt.nio.Keys._
|
||||
import sbt.nio.file.ChangedFiles
|
||||
import sbt.nio.{ FileChanges, FileStamp }
|
||||
|
||||
import scala.annotation.compileTimeOnly
|
||||
import scala.language.experimental.macros
|
||||
|
|
@ -27,13 +26,13 @@ import scala.reflect.macros.blackbox
|
|||
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."
|
||||
"`inputFileChanges` can only be called on a task within a task definition macro, such as :=, +=, ++=, or Def.task."
|
||||
)
|
||||
def changedInputFiles: Option[ChangedFiles] = macro changedInputFilesImpl[T]
|
||||
def inputFileChanges: FileChanges = macro changedInputFilesImpl[T]
|
||||
@compileTimeOnly(
|
||||
"`changedOutputFiles` can only be called on a task within a task definition macro, such as :=, +=, ++=, or Def.task."
|
||||
"`outputFileChanges` 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 outputFileChanges: FileChanges = macro changedOutputFilesImpl[T]
|
||||
@compileTimeOnly(
|
||||
"`inputFiles` can only be called on a task within a task definition macro, such as :=, +=, ++=, or Def.task."
|
||||
)
|
||||
|
|
@ -43,43 +42,59 @@ object FileChangesMacro {
|
|||
)
|
||||
def outputFiles: Seq[NioPath] = macro outputFilesImpl[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 changedInputFilesImpl[T: c.WeakTypeTag](c: blackbox.Context): c.Expr[FileChanges] = {
|
||||
impl[T](c)(
|
||||
c.universe.reify(allInputFiles),
|
||||
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))
|
||||
): c.Expr[FileChanges] = {
|
||||
impl[T](c)(
|
||||
c.universe.reify(allOutputFiles),
|
||||
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]]],
|
||||
currentKey: c.Expr[TaskKey[Seq[NioPath]]],
|
||||
changeKey: c.Expr[TaskKey[Seq[(NioPath, FileStamp)] => FileChanges]],
|
||||
mapKey: c.Expr[TaskKey[Seq[(NioPath, FileStamp)]]]
|
||||
): c.Expr[Option[ChangedFiles]] = {
|
||||
): c.Expr[FileChanges] = {
|
||||
import c.universe._
|
||||
val taskKey = getTaskKey(c)
|
||||
val taskScope = getTaskScope(c)
|
||||
reify {
|
||||
val changes = (changeKey.splice in taskKey.splice).value
|
||||
val changes = (changeKey.splice in taskScope.splice).value
|
||||
val current = (currentKey.splice in taskScope.splice).value
|
||||
import sbt.nio.FileStamp.Formats._
|
||||
Previous.runtimeInEnclosingTask(mapKey.splice in taskKey.splice).value.flatMap(changes)
|
||||
val previous = Previous.runtimeInEnclosingTask(mapKey.splice in taskScope.splice).value
|
||||
previous.map(changes).getOrElse(FileChanges.noPrevious(current))
|
||||
}
|
||||
}
|
||||
def inputFilesImpl[T: c.WeakTypeTag](c: blackbox.Context): c.Expr[Seq[NioPath]] = {
|
||||
val taskKey = getTaskKey(c)
|
||||
val taskKey = getTaskScope(c)
|
||||
c.universe.reify((allInputFiles in taskKey.splice).value)
|
||||
}
|
||||
def outputFilesImpl[T: c.WeakTypeTag](c: blackbox.Context): c.Expr[Seq[NioPath]] = {
|
||||
val taskKey = getTaskKey(c)
|
||||
val taskKey = getTaskScope(c)
|
||||
c.universe.reify((allOutputFiles in taskKey.splice).value)
|
||||
}
|
||||
private def getTaskKey[T: c.WeakTypeTag](c: blackbox.Context): c.Expr[TaskKey[T]] = {
|
||||
private def getTaskScope[T: c.WeakTypeTag](c: blackbox.Context): c.Expr[sbt.Scope] = {
|
||||
import c.universe._
|
||||
val taskTpe = c.weakTypeOf[TaskKey[T]]
|
||||
lazy val err = "Couldn't expand file change macro."
|
||||
c.Expr[TaskKey[T]](c.macroApplication match {
|
||||
case Select(Apply(_, k :: Nil), _) if k.tpe <:< taskTpe => k
|
||||
case _ => c.abort(c.enclosingPosition, err)
|
||||
})
|
||||
c.macroApplication match {
|
||||
case Select(Apply(_, k :: Nil), _) if k.tpe <:< taskTpe =>
|
||||
val expr = c.Expr[TaskKey[T]](k)
|
||||
c.universe.reify {
|
||||
if (expr.splice.scope.task.toOption.isDefined) expr.splice.scope
|
||||
else expr.splice.scope in expr.splice.key
|
||||
}
|
||||
case _ => c.abort(c.enclosingPosition, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,8 +11,9 @@ package internal.nio
|
|||
import sbt.Keys.{ baseDirectory, state, streams }
|
||||
import sbt.SlashSyntax0._
|
||||
import sbt.io.syntax._
|
||||
import sbt.nio.FileChanges
|
||||
import sbt.nio.Keys._
|
||||
import sbt.nio.file.{ ChangedFiles, Glob, RecursiveGlob }
|
||||
import sbt.nio.file.{ Glob, RecursiveGlob }
|
||||
|
||||
private[sbt] object CheckBuildSources {
|
||||
private[sbt] def needReloadImpl: Def.Initialize[Task[StateTransform]] = Def.task {
|
||||
|
|
@ -26,16 +27,17 @@ private[sbt] object CheckBuildSources {
|
|||
logger.debug("Checking for meta build source updates")
|
||||
val previous = (inputFileStamps in checkBuildSources).previous
|
||||
val changes = (changedInputFiles in checkBuildSources).value
|
||||
previous.flatMap(changes) match {
|
||||
case Some(cf: ChangedFiles) if !firstTime =>
|
||||
previous.map(changes) match {
|
||||
case Some(fileChanges @ FileChanges(created, deleted, modified, _))
|
||||
if fileChanges.hasChanges && !firstTime =>
|
||||
val rawPrefix = s"build source files have changed\n" +
|
||||
(if (cf.created.nonEmpty) s"new files: ${cf.created.mkString("\n ", "\n ", "\n")}"
|
||||
(if (created.nonEmpty) s"new files: ${created.mkString("\n ", "\n ", "\n")}"
|
||||
else "") +
|
||||
(if (cf.deleted.nonEmpty)
|
||||
s"deleted files: ${cf.deleted.mkString("\n ", "\n ", "\n")}"
|
||||
(if (deleted.nonEmpty)
|
||||
s"deleted files: ${deleted.mkString("\n ", "\n ", "\n")}"
|
||||
else "") +
|
||||
(if (cf.updated.nonEmpty)
|
||||
s"updated files: ${cf.updated.mkString("\n ", "\n ", "\n")}"
|
||||
(if (modified.nonEmpty)
|
||||
s"modified files: ${modified.mkString("\n ", "\n ", "\n")}"
|
||||
else "")
|
||||
val prefix = rawPrefix.linesIterator.filterNot(_.trim.isEmpty).mkString("\n")
|
||||
if (o == ReloadOnSourceChanges) {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* sbt
|
||||
* Copyright 2011 - 2018, Lightbend, Inc.
|
||||
* Copyright 2008 - 2010, Mark Harrah
|
||||
* Licensed under Apache License 2.0 (see LICENSE)
|
||||
*/
|
||||
|
||||
package sbt.nio
|
||||
|
||||
import java.nio.file.Path
|
||||
|
||||
/**
|
||||
* A report on the changes of the input file dependencies or output files of a task compared to
|
||||
* some previous time. It also contains the complete list of current inputs or outputs.
|
||||
*
|
||||
* @param created the files that were not present previously. When this is non empty, it does not
|
||||
* necessarily mean that the files were recently created. It could just indicate
|
||||
* that there was no previous cache entry for the file stamps (
|
||||
* see [[FileChanges#noPrevious]]).
|
||||
* @param deleted the files that have been deleted. This should be empty when no previous list of
|
||||
* files is available.
|
||||
* @param modified the files that have been modified. This should be empty when no previous list of
|
||||
* files is available.
|
||||
* @param unmodified the files that have no changes. This should be empty when no previous list of
|
||||
* files is availab.e
|
||||
*/
|
||||
final case class FileChanges(
|
||||
created: Seq[Path],
|
||||
deleted: Seq[Path],
|
||||
modified: Seq[Path],
|
||||
unmodified: Seq[Path]
|
||||
) {
|
||||
|
||||
/**
|
||||
* Return true either if there is no previous information or
|
||||
* @return true if there are no changes.
|
||||
*/
|
||||
lazy val hasChanges: Boolean = created.nonEmpty || deleted.nonEmpty || modified.nonEmpty
|
||||
}
|
||||
|
||||
object FileChanges {
|
||||
|
||||
/**
|
||||
* Creates an instance of [[FileChanges]] for a collection of files for which there were no
|
||||
* previous file stamps available.
|
||||
* @param files all of the existing files.
|
||||
* @return the [[FileChanges]] with the [[FileChanges.created]] field set to the input, `files`.
|
||||
*/
|
||||
def noPrevious(files: Seq[Path]): FileChanges =
|
||||
FileChanges(created = files, deleted = Nil, modified = Nil, unmodified = Nil)
|
||||
|
||||
/**
|
||||
* Creates an instance of [[FileChanges]] for a collection of files for which there were no
|
||||
* changes when compared to the previous file stamps.
|
||||
* @param files all of the existing files.
|
||||
* @return the [[FileChanges]] with the [[FileChanges.unmodified]] field set to the input, `files`.
|
||||
*/
|
||||
def unmodified(files: Seq[Path]): FileChanges =
|
||||
FileChanges(created = Nil, deleted = Nil, modified = Nil, unmodified = files)
|
||||
}
|
||||
|
|
@ -17,8 +17,8 @@ import sbt.internal.DynamicInput
|
|||
import sbt.internal.nio.FileTreeRepository
|
||||
import sbt.internal.util.AttributeKey
|
||||
import sbt.internal.util.complete.Parser
|
||||
import sbt.nio.file.{ ChangedFiles, FileAttributes, FileTreeView, Glob }
|
||||
import sbt.{ Def, InputKey, ProjectRef, State, StateTransform }
|
||||
import sbt.nio.file.{ FileAttributes, FileTreeView, Glob }
|
||||
import sbt._
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
|
|
@ -30,7 +30,7 @@ object Keys {
|
|||
val allInputFiles =
|
||||
taskKey[Seq[Path]]("All of the file inputs for a task excluding directories and hidden files.")
|
||||
val changedInputFiles =
|
||||
taskKey[Seq[(Path, FileStamp)] => Option[ChangedFiles]]("The changed files for a task")
|
||||
taskKey[Seq[(Path, FileStamp)] => FileChanges]("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."
|
||||
)
|
||||
|
|
@ -42,7 +42,7 @@ object Keys {
|
|||
val allOutputFiles =
|
||||
taskKey[Seq[Path]]("All of the file outputs for a task excluding directories and hidden files.")
|
||||
val changedOutputFiles =
|
||||
taskKey[Seq[(Path, FileStamp)] => Option[ChangedFiles]](
|
||||
taskKey[Seq[(Path, FileStamp)] => FileChanges](
|
||||
"The files that have changed since the last task run."
|
||||
)
|
||||
val outputFileStamper = settingKey[FileStamper](
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ package nio
|
|||
|
||||
import java.io.File
|
||||
import java.nio.file.{ Files, Path }
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
import sbt.Project._
|
||||
import sbt.internal.Clean.ToSeqPath
|
||||
|
|
@ -19,7 +20,6 @@ import sbt.internal.{ Clean, Continuous, DynamicInput, SettingsGraph }
|
|||
import sbt.nio.FileStamp.Formats._
|
||||
import sbt.nio.FileStamper.{ Hash, LastModified }
|
||||
import sbt.nio.Keys._
|
||||
import sbt.nio.file.ChangedFiles
|
||||
import sbt.std.TaskExtra._
|
||||
import sjsonnew.JsonFormat
|
||||
|
||||
|
|
@ -64,7 +64,7 @@ private[sbt] object Settings {
|
|||
val scope = setting.key.scope.copy(task = Select(ak))
|
||||
if (fileOutputScopes.contains(scope)) {
|
||||
val sk = setting.asInstanceOf[Def.Setting[Task[Any]]].key
|
||||
val scopedKey = sk.scopedKey.copy(sk.scope in sk.key, Keys.dynamicFileOutputs.key)
|
||||
val scopedKey = Keys.dynamicFileOutputs in (sk.scope in sk.key)
|
||||
cleanScopes.add(scope)
|
||||
addTaskDefinition {
|
||||
val init: Def.Initialize[Task[Seq[Path]]] = sk(_.map(_ => Nil))
|
||||
|
|
@ -72,44 +72,28 @@ private[sbt] object Settings {
|
|||
} :: allOutputPathsImpl(scope) :: outputFileStampsImpl(scope) :: cleanImpl(scope) :: Nil
|
||||
} else Nil
|
||||
}
|
||||
def mkSetting[T: JsonFormat: ToSeqPath]: List[Def.Setting[_]] = {
|
||||
val sk = setting.asInstanceOf[Def.Setting[Task[T]]].key
|
||||
val taskKey = TaskKey(sk.key) in sk.scope
|
||||
// We create a previous reference so that clean automatically works without the
|
||||
// user having to explicitly call previous anywhere.
|
||||
val init = Previous.runtime(taskKey).zip(taskKey) {
|
||||
case (_, t) => t.map(implicitly[ToSeqPath[T]].apply)
|
||||
}
|
||||
val key = Def.ScopedKey(taskKey.scope in taskKey.key, Keys.dynamicFileOutputs.key)
|
||||
addTaskDefinition(Def.setting[Task[Seq[Path]]](key, init, setting.pos)) ::
|
||||
outputsAndStamps(taskKey, cleanScopes)
|
||||
}
|
||||
ak.manifest.typeArguments match {
|
||||
case t :: Nil if seqClass.isAssignableFrom(t.runtimeClass) =>
|
||||
t.typeArguments match {
|
||||
// Task[Seq[File]]
|
||||
case f :: Nil if fileClass.isAssignableFrom(f.runtimeClass) =>
|
||||
val sk = setting.asInstanceOf[Def.Setting[Task[Seq[File]]]].key
|
||||
val scopedKey = sk.scopedKey.copy(sk.scope in sk.key, Keys.dynamicFileOutputs.key)
|
||||
addTaskDefinition {
|
||||
val init: Def.Initialize[Task[Seq[Path]]] = sk(_.map(_.map(_.toPath)))
|
||||
Def.setting[Task[Seq[Path]]](scopedKey, init, setting.pos)
|
||||
} :: outputsAndStamps(TaskKey(sk.key) in sk.scope, cleanScopes)
|
||||
// Task[Seq[Path]]
|
||||
case p :: Nil if pathClass.isAssignableFrom(p.runtimeClass) =>
|
||||
val sk = setting.asInstanceOf[Def.Setting[Task[Seq[Path]]]].key
|
||||
val scopedKey = sk.scopedKey.copy(sk.scope in sk.key, Keys.dynamicFileOutputs.key)
|
||||
addTaskDefinition {
|
||||
val init: Def.Initialize[Task[Seq[Path]]] = sk(_.map(identity))
|
||||
Def.setting[Task[Seq[Path]]](scopedKey, init, setting.pos)
|
||||
} :: outputsAndStamps(TaskKey(sk.key) in sk.scope, cleanScopes)
|
||||
case _ => default
|
||||
case f :: Nil if fileClass.isAssignableFrom(f.runtimeClass) => mkSetting[Seq[File]]
|
||||
case p :: Nil if pathClass.isAssignableFrom(p.runtimeClass) => mkSetting[Seq[Path]]
|
||||
case _ => default
|
||||
}
|
||||
// Task[File]
|
||||
case t :: Nil if fileClass.isAssignableFrom(t.runtimeClass) =>
|
||||
val sk = setting.asInstanceOf[Def.Setting[Task[File]]].key
|
||||
val scopedKey = sk.scopedKey.copy(sk.scope in sk.key, Keys.dynamicFileOutputs.key)
|
||||
addTaskDefinition {
|
||||
val init: Def.Initialize[Task[Seq[Path]]] = sk(_.map(_.toPath :: Nil))
|
||||
Def.setting[Task[Seq[Path]]](scopedKey, init, setting.pos)
|
||||
} :: outputsAndStamps(TaskKey(sk.key) in sk.scope, cleanScopes)
|
||||
// Task[Path]
|
||||
case t :: Nil if pathClass.isAssignableFrom(t.runtimeClass) =>
|
||||
val sk = setting.asInstanceOf[Def.Setting[Task[Path]]].key
|
||||
val scopedKey = sk.scopedKey.copy(sk.scope in sk.key, Keys.dynamicFileOutputs.key)
|
||||
addTaskDefinition {
|
||||
val init: Def.Initialize[Task[Seq[Path]]] = sk(_.map(_ :: Nil))
|
||||
Def.setting[Task[Seq[Path]]](scopedKey, init, setting.pos)
|
||||
} :: outputsAndStamps(TaskKey(sk.key) in sk.scope, cleanScopes)
|
||||
case _ => default
|
||||
case t :: Nil if fileClass.isAssignableFrom(t.runtimeClass) => mkSetting[File]
|
||||
case t :: Nil if pathClass.isAssignableFrom(t.runtimeClass) => mkSetting[Path]
|
||||
case _ => default
|
||||
}
|
||||
case _ => Nil
|
||||
}
|
||||
|
|
@ -124,8 +108,6 @@ private[sbt] object Settings {
|
|||
case dynamicDependency.key => (dynamicDependency in scopedKey.scope := { () }) :: Nil
|
||||
case transitiveClasspathDependency.key =>
|
||||
(transitiveClasspathDependency in scopedKey.scope := { () }) :: Nil
|
||||
case changedOutputFiles.key =>
|
||||
changedFilesImpl(scopedKey, changedOutputFiles, outputFileStamps)
|
||||
case _ => Nil
|
||||
}
|
||||
|
||||
|
|
@ -186,7 +168,7 @@ private[sbt] object Settings {
|
|||
dynamicInputs.foreach(_ ++= inputs.map(g => DynamicInput(g, stamper, forceTrigger)))
|
||||
view.list(inputs)
|
||||
}) :: fileStamps(scopedKey) :: allFilesImpl(scopedKey) :: Nil ++
|
||||
changedInputFilesImpl(scopedKey)
|
||||
changedInputFilesImpl(scopedKey.scope)
|
||||
}
|
||||
|
||||
private[this] val taskClass = classOf[Task[_]]
|
||||
|
|
@ -214,55 +196,55 @@ private[sbt] object Settings {
|
|||
* files or files whose stamp has not changed since the previous run. Directories and hidden
|
||||
* files are excluded
|
||||
*
|
||||
* @param scopedKey the key whose fileInputs we are seeking
|
||||
* @param scope the scope corresponding to the task whose fileInputs we are seeking
|
||||
* @return a task definition that retrieves the changed input files scoped to the key.
|
||||
*/
|
||||
private[this] def changedInputFilesImpl(scopedKey: Def.ScopedKey[_]): Seq[Def.Setting[_]] =
|
||||
changedFilesImpl(scopedKey, changedInputFiles, inputFileStamps) ::
|
||||
(watchForceTriggerOnAnyChange in scopedKey.scope := {
|
||||
(watchForceTriggerOnAnyChange in scopedKey.scope).?.value match {
|
||||
private[this] def changedInputFilesImpl(scope: Scope): Seq[Def.Setting[_]] =
|
||||
changedFilesImpl(scope, changedInputFiles, inputFileStamps) ::
|
||||
(watchForceTriggerOnAnyChange in scope := {
|
||||
(watchForceTriggerOnAnyChange in scope).?.value match {
|
||||
case Some(t) => t
|
||||
case None => false
|
||||
}
|
||||
}) :: Nil
|
||||
private[this] def changedFilesImpl(
|
||||
scopedKey: Def.ScopedKey[_],
|
||||
changeKey: TaskKey[Seq[(Path, FileStamp)] => Option[ChangedFiles]],
|
||||
scope: Scope,
|
||||
changeKey: TaskKey[Seq[(Path, FileStamp)] => FileChanges],
|
||||
stampKey: TaskKey[Seq[(Path, FileStamp)]]
|
||||
): Def.Setting[_] =
|
||||
addTaskDefinition(changeKey in scopedKey.scope := {
|
||||
val current = (stampKey in scopedKey.scope).value
|
||||
previous => changedFiles(previous, current)
|
||||
addTaskDefinition(changeKey in scope := {
|
||||
val current = (stampKey in scope).value
|
||||
changedFiles(_, current)
|
||||
})
|
||||
|
||||
private[sbt] def changedFiles(
|
||||
previous: Seq[(Path, FileStamp)],
|
||||
current: Seq[(Path, FileStamp)]
|
||||
): Option[ChangedFiles] = {
|
||||
): FileChanges = {
|
||||
val createdBuilder = new VectorBuilder[Path]
|
||||
val deletedBuilder = new VectorBuilder[Path]
|
||||
val updatedBuilder = new VectorBuilder[Path]
|
||||
val currentMap = current.toMap
|
||||
val prevMap = previous.toMap
|
||||
val modifiedBuilder = new VectorBuilder[Path]
|
||||
val unmodifiedBuilder = new VectorBuilder[Path]
|
||||
val seen = ConcurrentHashMap.newKeySet[Path]
|
||||
val prevMap = new ConcurrentHashMap[Path, FileStamp]()
|
||||
previous.foreach { case (k, v) => prevMap.put(k, v); () }
|
||||
current.foreach {
|
||||
case (path, currentStamp) =>
|
||||
prevMap.get(path) match {
|
||||
case Some(oldStamp) => if (oldStamp != currentStamp) updatedBuilder += path
|
||||
case None => createdBuilder += path
|
||||
if (seen.add(path)) {
|
||||
prevMap.remove(path) match {
|
||||
case null => createdBuilder += path
|
||||
case old => (if (old != currentStamp) modifiedBuilder else unmodifiedBuilder) += path
|
||||
}
|
||||
}
|
||||
}
|
||||
previous.foreach {
|
||||
case (path, _) =>
|
||||
if (currentMap.get(path).isEmpty) deletedBuilder += path
|
||||
}
|
||||
val created = createdBuilder.result()
|
||||
val deleted = deletedBuilder.result()
|
||||
val updated = updatedBuilder.result()
|
||||
if (created.isEmpty && deleted.isEmpty && updated.isEmpty) {
|
||||
None
|
||||
prevMap.forEach((p, _) => deletedBuilder += p)
|
||||
val unmodified = unmodifiedBuilder.result()
|
||||
if (unmodified.size == current.size) {
|
||||
FileChanges.unmodified(unmodifiedBuilder.result)
|
||||
} else {
|
||||
val cf = ChangedFiles(created = created, deleted = deleted, updated = updated)
|
||||
Some(cf)
|
||||
val created = createdBuilder.result()
|
||||
val deleted = deletedBuilder.result()
|
||||
val modified = modifiedBuilder.result()
|
||||
FileChanges(created, deleted, modified, unmodified)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -328,7 +310,8 @@ private[sbt] object Settings {
|
|||
): List[Def.Setting[_]] = {
|
||||
val scope = taskKey.scope in taskKey.key
|
||||
cleanScopes.add(scope)
|
||||
allOutputPathsImpl(scope) :: outputFileStampsImpl(scope) :: cleanImpl(taskKey) :: Nil
|
||||
val changes = changedFilesImpl(scope, changedOutputFiles, outputFileStamps) :: Nil
|
||||
allOutputPathsImpl(scope) :: outputFileStampsImpl(scope) :: cleanImpl(taskKey) :: changes
|
||||
}
|
||||
private[this] def allOutputPathsImpl(scope: Scope): Def.Setting[_] =
|
||||
addTaskDefinition(allOutputFiles in scope := {
|
||||
|
|
|
|||
|
|
@ -66,6 +66,8 @@ trait Import {
|
|||
val AnyPath = sbt.nio.file.AnyPath
|
||||
type ChangedFiles = sbt.nio.file.ChangedFiles
|
||||
val ChangedFiles = sbt.nio.file.ChangedFiles
|
||||
type FileChanges = sbt.nio.FileChanges
|
||||
val FileChanges = sbt.nio.FileChanges
|
||||
type Glob = sbt.nio.file.Glob
|
||||
val Glob = sbt.nio.file.Glob
|
||||
type RelativeGlob = sbt.nio.file.RelativeGlob
|
||||
|
|
|
|||
|
|
@ -9,8 +9,9 @@ copyFile / target := baseDirectory.value / "out"
|
|||
|
||||
copyFile := Def.task {
|
||||
val prev = copyFile.previous
|
||||
val changes: Option[Seq[Path]] = copyFile.changedInputFiles.map {
|
||||
case ChangedFiles(c, _, u) => c ++ u
|
||||
val changes: Option[Seq[Path]] = copyFile.inputFileChanges match {
|
||||
case fc @ FileChanges(c, _, u, _) if fc.hasChanges => Some(c ++ u)
|
||||
case _ => None
|
||||
}
|
||||
prev match {
|
||||
case Some(v: Int) if changes.isEmpty => v
|
||||
|
|
|
|||
|
|
@ -5,8 +5,7 @@ val fileInputTask = taskKey[Unit]("task with file inputs")
|
|||
fileInputTask / fileInputs += Glob(baseDirectory.value / "base", "*.md")
|
||||
|
||||
fileInputTask := {
|
||||
if (fileInputTask.changedInputFiles.fold(false)(
|
||||
_.created.exists(_.getFileName.toString.startsWith("foo"))
|
||||
)) assert(false)
|
||||
val created = fileInputTask.inputFileChanges.created
|
||||
if (created.exists(_.getFileName.toString.startsWith("foo"))) assert(false)
|
||||
assert(true)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1 @@
|
|||
class Foo
|
||||
|
|
@ -2,4 +2,4 @@
|
|||
|
||||
-> test
|
||||
|
||||
> test
|
||||
> test
|
||||
|
|
|
|||
|
|
@ -8,9 +8,7 @@ foo / fileInputs := Seq(
|
|||
|
||||
val checkModified = taskKey[Unit]("check that modified files are returned")
|
||||
checkModified := {
|
||||
val changes = foo.changedInputFiles
|
||||
val modified = changes.map(_.updated).getOrElse(Nil)
|
||||
println(modified)
|
||||
val modified = foo.inputFileChanges.modified
|
||||
val allFiles = foo.inputFiles
|
||||
if (modified.isEmpty) assert(true)
|
||||
else {
|
||||
|
|
@ -22,7 +20,7 @@ checkModified := {
|
|||
val checkRemoved = taskKey[Unit]("check that removed files are returned")
|
||||
checkRemoved := Def.taskDyn {
|
||||
val files = foo.inputFiles
|
||||
val removed = foo.changedInputFiles.map(_.deleted).getOrElse(Nil)
|
||||
val removed = foo.inputFileChanges.deleted
|
||||
if (removed.isEmpty) Def.task(assert(true))
|
||||
else Def.task {
|
||||
assert(files == Seq((baseDirectory.value / "base" / "Foo.txt").toPath))
|
||||
|
|
@ -33,11 +31,11 @@ checkRemoved := Def.taskDyn {
|
|||
val checkAdded = taskKey[Unit]("check that modified files are returned")
|
||||
checkAdded := Def.taskDyn {
|
||||
val files = foo.inputFiles
|
||||
val added = foo.changedInputFiles.map(_.created).getOrElse(Nil)
|
||||
if (added.isEmpty || (files.toSet == added.toSet)) Def.task(assert(true))
|
||||
val created = foo.inputFileChanges.created
|
||||
if (created.isEmpty || (files.toSet == created.toSet)) Def.task(assert(true))
|
||||
else Def.task {
|
||||
val base = baseDirectory.value / "base"
|
||||
assert(files.toSet == Set("Bar.md", "Foo.txt").map(p => (base / p).toPath))
|
||||
assert(added == Seq((baseDirectory.value / "base" / "Bar.md").toPath))
|
||||
assert(created == Seq((baseDirectory.value / "base" / "Bar.md").toPath))
|
||||
}
|
||||
}.value
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ fileInputTask := {
|
|||
Try(IO.read(baseDirectory.value / "expected").split(" ").toSeq.filterNot(_.isEmpty))
|
||||
.getOrElse(Nil)
|
||||
.map(baseDirectory.value.toPath / "base" / _)
|
||||
val actual = fileInputTask.changedInputFiles.toSeq.flatMap(_.updated)
|
||||
val actual = fileInputTask.inputFileChanges.modified
|
||||
assert(actual.toSet == expectedChanges.toSet)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -11,66 +11,53 @@ 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.inputFiles
|
||||
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 outputDir = Files.createDirectories(streams.value.cacheDirectory.toPath)
|
||||
val logger = streams.value.log
|
||||
compileLib.previous match {
|
||||
case Some(outputs: Seq[Path]) if changedFiles.isEmpty =>
|
||||
logger.info("Not compiling libfoo: no inputs have changed.")
|
||||
outputs
|
||||
case _ =>
|
||||
Files.createDirectories(objectDir)
|
||||
def extensionFilter(ext: String): Path => Boolean = _.getFileName.toString.endsWith(s".$ext")
|
||||
val cFiles: Seq[Path] =
|
||||
if (changedFiles.fold(false)(_.exists(extensionFilter("h"))))
|
||||
allFiles.filter(extensionFilter("c"))
|
||||
else changedFiles.getOrElse(allFiles).filter(extensionFilter("c"))
|
||||
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 include = (compileLib / sourceDirectory).value / "include"
|
||||
def outputPath(path: Path): Path =
|
||||
outputDir / path.getFileName.toString.replaceAll(".c$", ".o")
|
||||
def compile(path: Path): Path = {
|
||||
val output = outputPath(path)
|
||||
logger.info(s"Compiling $path to $output")
|
||||
Seq("gcc", "-fPIC", "-std=gnu99", s"-I$include", "-c", s"$path", "-o", s"$output").!!
|
||||
output
|
||||
}
|
||||
val report = compileLib.inputFileChanges
|
||||
val sourceMap = compileLib.inputFiles.view.collect {
|
||||
case p: Path if p.getFileName.toString.endsWith(".c") => outputPath(p) -> p
|
||||
}.toMap
|
||||
val existingTargets = fileTreeView.value.list(outputDir.toGlob / **).flatMap { case (p, _) =>
|
||||
if (!sourceMap.contains(p)) {
|
||||
Files.deleteIfExists(p)
|
||||
None
|
||||
} else {
|
||||
Some(p)
|
||||
}
|
||||
}.toSet
|
||||
val updatedPaths = (report.created ++ report.modified).toSet
|
||||
val needCompile =
|
||||
if (updatedPaths.exists(_.getFileName.toString.endsWith(".h"))) sourceMap.values
|
||||
else updatedPaths ++ sourceMap.filterKeys(!existingTargets(_)).values
|
||||
needCompile.foreach(compile)
|
||||
sourceMap.keys.toVector
|
||||
}
|
||||
|
||||
val linkLib = taskKey[Path]("")
|
||||
linkLib / target := baseDirectory.value / "out" / "lib"
|
||||
linkLib := {
|
||||
val changedObjects = compileLib.changedOutputFiles
|
||||
val outPath = (linkLib / target).value.toPath
|
||||
val allObjects = compileLib.outputFiles.map(_.toString)
|
||||
val outputDir = Files.createDirectories(streams.value.cacheDirectory.toPath)
|
||||
val logger = streams.value.log
|
||||
linkLib.previous match {
|
||||
case Some(p: Path) if changedObjects.isEmpty =>
|
||||
logger.info("Not running linker: no outputs have changed.")
|
||||
p
|
||||
case _ =>
|
||||
val (linkOptions, libraryPath) = if (scala.util.Properties.isMac) {
|
||||
val path = outPath.resolve("libfoo.dylib")
|
||||
(Seq("-dynamiclib", "-o", path.toString), path)
|
||||
} else {
|
||||
val path = outPath.resolve("libfoo.so")
|
||||
(Seq("-shared", "-fPIC", "-o", path.toString), path)
|
||||
}
|
||||
logger.info(s"Linking $libraryPath")
|
||||
Files.createDirectories(outPath)
|
||||
("gcc" +: (linkOptions ++ allObjects)).!!
|
||||
libraryPath
|
||||
val isMac = scala.util.Properties.isMac
|
||||
val library = outputDir / s"libfoo.${if (isMac) "dylib" else "so"}"
|
||||
val (report, objects) = (compileLib.outputFileChanges, compileLib.outputFiles)
|
||||
val linkOpts = if (isMac) Seq("-dynamiclib") else Seq("-shared", "-fPIC")
|
||||
if (report.hasChanges || !Files.exists(library)) {
|
||||
logger.info(s"Linking $library")
|
||||
(Seq("gcc") ++ linkOpts ++ Seq("-o", s"$library") ++ objects.map(_.toString)).!!
|
||||
} else {
|
||||
logger.debug(s"Skipping linking of $library")
|
||||
}
|
||||
library
|
||||
}
|
||||
|
||||
val compileMain = taskKey[Path]("compile main")
|
||||
|
|
@ -79,34 +66,33 @@ compileMain / fileInputs := (compileMain / sourceDirectory).value.toGlob / "main
|
|||
compileMain / target := baseDirectory.value / "out" / "main"
|
||||
compileMain := {
|
||||
val library = linkLib.value
|
||||
val changed: Boolean = compileMain.changedInputFiles.nonEmpty ||
|
||||
linkLib.changedOutputFiles.nonEmpty
|
||||
val changed: Boolean = compileMain.inputFileChanges.hasChanges ||
|
||||
linkLib.outputFileChanges.hasChanges
|
||||
val include = (compileLib / sourceDirectory).value / "include"
|
||||
val logger = streams.value.log
|
||||
val outDir = (compileMain / target).value.toPath
|
||||
val outPath = outDir.resolve("main.out")
|
||||
compileMain.previous match {
|
||||
case Some(p: Path) if changed =>
|
||||
logger.info(s"Not building $outPath: no dependencies have changed")
|
||||
p
|
||||
case _ =>
|
||||
compileMain.inputFiles match {
|
||||
case Seq(main) =>
|
||||
Files.createDirectories(outDir)
|
||||
logger.info(s"Building executable $outPath")
|
||||
(Seq("gcc") ++ compileOpts.value ++ Seq(
|
||||
main.toString,
|
||||
s"-I$include",
|
||||
"-o",
|
||||
outPath.toString,
|
||||
s"-L${library.getParent}",
|
||||
"-lfoo"
|
||||
)).!!
|
||||
outPath
|
||||
case main =>
|
||||
throw new IllegalStateException(s"multiple main files detected: ${main.mkString(",")}")
|
||||
}
|
||||
val inputs = compileMain.inputFiles
|
||||
if (changed || !Files.exists(outPath)) {
|
||||
inputs match {
|
||||
case Seq(main) =>
|
||||
Files.createDirectories(outDir)
|
||||
logger.info(s"Building executable $outPath")
|
||||
(Seq("gcc") ++ compileOpts.value ++ Seq(
|
||||
main.toString,
|
||||
s"-I$include",
|
||||
"-o",
|
||||
outPath.toString,
|
||||
s"-L${library.getParent}",
|
||||
"-lfoo"
|
||||
)).!!
|
||||
case main =>
|
||||
throw new IllegalStateException(s"multiple main files detected: ${main.mkString(",")}")
|
||||
}
|
||||
} else {
|
||||
logger.info(s"Not building $outPath: no dependencies have changed")
|
||||
}
|
||||
outPath
|
||||
}
|
||||
|
||||
val executeMain = inputKey[Unit]("run the main method")
|
||||
|
|
@ -130,9 +116,9 @@ executeMain := {
|
|||
|
||||
val checkOutput = inputKey[Unit]("check the output value")
|
||||
checkOutput := {
|
||||
val args @ Seq(arg, res) = Def.spaceDelimited("").parsed
|
||||
val Seq(arg, res) = Def.spaceDelimited("").parsed
|
||||
val binary: Path = compileMain.outputFiles.head
|
||||
val output = RunBinary(binary, args, linkLib.value)
|
||||
val output = RunBinary(binary, arg :: Nil, linkLib.value)
|
||||
assert(output.contains(s"f($arg) = $res"))
|
||||
()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,8 +6,7 @@ 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 {
|
||||
(foo.inputFileChanges.modified ++ bar.inputFileChanges.modified) match {
|
||||
case Nil =>
|
||||
val contents = IO.read(baseDirectory.value / "foo" / "foo.md")
|
||||
assert(contents == "foo", s"expected 'foo', got '$contents")
|
||||
|
|
|
|||
|
|
@ -8,8 +8,7 @@ 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 {
|
||||
foo.outputFileChanges.modified ++ bar.outputFileChanges.modified match {
|
||||
case Nil =>
|
||||
val contents = IO.read(baseDirectory.value / "foo" / "foo.md")
|
||||
assert(contents == "foo", s"expected 'foo', got '$contents")
|
||||
|
|
|
|||
Loading…
Reference in New Issue