From c72005fd2b5be05df1df40acd6a0a158b9355b9b Mon Sep 17 00:00:00 2001 From: Ethan Atkins Date: Sun, 13 Jan 2019 16:15:06 -0800 Subject: [PATCH] Support inputs in dynamic tasks Prior to this commit, it was necessary to add breadcrumbs for every input that is used within a dynamic task. In this commit, I rework the watch setup so that we can track the dynamic inputs that are used. To simplify the discussion, I'm going to ignore aggregation and multi-commands, but they are both supported. To implement this change, I update the GlobLister.all method to take a second implicit argument: DynamicInputs. This is effectively a mutable Set of Globs that is updated every time a task looks up files from a glob. The repository.get method should already register the glob with the repository. The set of globs are necessary because the repository may not do any file filtering so the file event monitor needs to check the input globs to ensure that the file event is for a file that actually requested by a task during evaluation. * Long term, I plan to add support for lifting tasks into a dynamic task in a way that records _all_ of the possible dependencies for the task through each of the dynamic code paths. We should revisit this change to determine if its still necessary after that change. --- .../util/appmacro/MacroDefaults.scala | 12 ++ main/src/main/scala/sbt/Defaults.scala | 1 + .../main/scala/sbt/internal/Continuous.scala | 184 +++++++++++------- .../scala/sbt/internal/ExternalHooks.scala | 15 +- .../scala/sbt/internal/FileManagement.scala | 1 + .../main/scala/sbt/internal/FileTree.scala | 13 +- .../main/scala/sbt/internal/GlobLister.scala | 83 +++++--- sbt/src/sbt-test/tests/glob-dsl/build.sbt | 5 +- sbt/src/sbt-test/tests/inputs/build.sbt | 2 - .../sbt-test/watch/dynamic-inputs/build.sbt | 3 + .../watch/dynamic-inputs/project/Build.scala | 41 ++++ sbt/src/sbt-test/watch/dynamic-inputs/test | 7 + 12 files changed, 253 insertions(+), 114 deletions(-) create mode 100644 sbt/src/sbt-test/watch/dynamic-inputs/build.sbt create mode 100644 sbt/src/sbt-test/watch/dynamic-inputs/project/Build.scala create mode 100644 sbt/src/sbt-test/watch/dynamic-inputs/test diff --git a/core-macros/src/main/scala/sbt/internal/util/appmacro/MacroDefaults.scala b/core-macros/src/main/scala/sbt/internal/util/appmacro/MacroDefaults.scala index 4de9b65e5..43526a373 100644 --- a/core-macros/src/main/scala/sbt/internal/util/appmacro/MacroDefaults.scala +++ b/core-macros/src/main/scala/sbt/internal/util/appmacro/MacroDefaults.scala @@ -22,4 +22,16 @@ object MacroDefaults { import c.universe._ q"sbt.Keys.fileTreeRepository.value: @sbtUnchecked" } + + /** + * Macro to generated default file tree repository. It must be defined as an untyped tree because + * sbt.Keys is not available in this project. This is meant for internal use only, but must be + * public because its a macro. + * @param c the macro context + * @return the tree expressing the default file tree repository. + */ + def dynamicInputs(c: blackbox.Context): c.Tree = { + import c.universe._ + q"sbt.internal.Continuous.dynamicInputs.value: @sbtUnchecked" + } } diff --git a/main/src/main/scala/sbt/Defaults.scala b/main/src/main/scala/sbt/Defaults.scala index 0d151743e..d368eb5d2 100755 --- a/main/src/main/scala/sbt/Defaults.scala +++ b/main/src/main/scala/sbt/Defaults.scala @@ -293,6 +293,7 @@ object Defaults extends BuildCommon { case Some(r) => r case None => FileTreeView.DEFAULT.asDataView(FileAttributes.default) }), + Continuous.dynamicInputs := Continuous.dynamicInputsImpl.value, externalHooks := { val repository = fileTreeRepository.value compileOptions => diff --git a/main/src/main/scala/sbt/internal/Continuous.scala b/main/src/main/scala/sbt/internal/Continuous.scala index d332bfb86..66601d19b 100644 --- a/main/src/main/scala/sbt/internal/Continuous.scala +++ b/main/src/main/scala/sbt/internal/Continuous.scala @@ -134,6 +134,18 @@ object Continuous extends DeprecatedContinuous { "Receives a copy of all of the bytes from System.in.", 10000 ) + val dynamicInputs = taskKey[FileTree.DynamicInputs]( + "The input globs found during task evaluation that are used in watch." + ) + def dynamicInputsImpl: Def.Initialize[Task[FileTree.DynamicInputs]] = Def.task { + Keys.state.value.get(DynamicInputs).getOrElse(FileTree.DynamicInputs.none) + } + private[sbt] val DynamicInputs = + AttributeKey[FileTree.DynamicInputs]( + "dynamic-inputs", + "Stores the inputs (dynamic and regular) for a task", + 10000 + ) private[this] val continuousParser: State => Parser[(Int, String)] = { def toInt(s: String): Int = Try(s.toInt).getOrElse(0) @@ -175,12 +187,14 @@ object Continuous extends DeprecatedContinuous { } val repository = getRepository(state) - (inputs ++ triggers).foreach(repository.register) + val registeringSet = state.get(DynamicInputs).get + registeringSet.value.foreach(_ ++= inputs) + (inputs ++ triggers).foreach(repository.register(_: Glob)) val watchSettings = new WatchSettings(scopedKey) new Config( scopedKey, repository, - inputs, + () => registeringSet.value.fold(Nil: Seq[Glob])(_.toSeq).sorted, triggers, watchSettings ) @@ -195,25 +209,13 @@ object Continuous extends DeprecatedContinuous { } private[sbt] def setup[R](state: State, command: String)( - f: (State, Seq[String], Seq[() => Boolean], Seq[String]) => R + f: (State, Seq[(String, State, () => Boolean)], Seq[String]) => R ): R = { // First set up the state so that we can capture whether or not a task completed successfully // or if it threw an Exception (we lose the actual exception, but that should still be printed // to the console anyway). val failureCommandName = "SbtContinuousWatchOnFail" val onFail = Command.command(failureCommandName)(identity) - /* - * Takes a task string and converts it to an EitherTask. We cannot preserve either - * the value returned by the task or any exception thrown by the task, but we can determine - * whether or not the task ran successfully using the onFail command defined above. - */ - def makeTask(cmd: String)(task: () => State): () => Boolean = { () => - MainLoop - .processCommand(Exec(cmd, None), state, task) - .remainingCommands - .forall(_.commandLine != failureCommandName) - } - // This adds the "SbtContinuousWatchOnFail" onFailure handler which allows us to determine // whether or not the last task successfully ran. It is used in the makeTask method below. val s = (FailureWall :: state).copy( @@ -221,6 +223,35 @@ object Continuous extends DeprecatedContinuous { definedCommands = state.definedCommands :+ onFail ) + /* + * Takes a task string and converts it to an EitherTask. We cannot preserve either + * the value returned by the task or any exception thrown by the task, but we can determine + * whether or not the task ran successfully using the onFail command defined above. Each + * task gets its own state with its own file tree repository. This is so that we can keep + * track of what globs are actually used by the task to ensure that we monitor them, even + * if they are not visible in the input graph due to the use of Def.taskDyn. + */ + def makeTask(cmd: String): (String, State, () => Boolean) = { + val newState = s.put(DynamicInputs, FileTree.DynamicInputs.empty) + val task = Parser + .parse(cmd, Command.combine(newState.definedCommands)(newState)) + .getOrElse( + throw new IllegalStateException( + "No longer able to parse command after transforming state" + ) + ) + ( + cmd, + newState, + () => { + MainLoop + .processCommand(Exec(cmd, None), newState, task) + .remainingCommands + .forall(_.commandLine != failureCommandName) + } + ) + } + // We support multiple commands in watch, so it's necessary to run the command string through // the multi parser. val trimmed = command.trim @@ -234,14 +265,15 @@ object Continuous extends DeprecatedContinuous { val taskParser = Command.combine(s.definedCommands)(s) // This specified either the task corresponding to a command or the command itself if the // the command cannot be converted to a task. - val (invalid, valid) = commands.foldLeft((Nil: Seq[String], Nil: Seq[() => Boolean])) { - case ((i, v), cmd) => - Parser.parse(cmd, taskParser) match { - case Right(task) => (i, v :+ makeTask(cmd)(task)) - case Left(c) => (i :+ c, v) - } - } - f(s, commands, valid, invalid) + val (invalid, valid) = + commands.foldLeft((Nil: Seq[String], Nil: Seq[(String, State, () => Boolean)])) { + case ((i, v), cmd) => + Parser.parse(cmd, taskParser) match { + case Right(_) => (i, v :+ makeTask(cmd)) + case Left(c) => (i :+ c, v) + } + } + f(s, valid, invalid) } private[sbt] def runToTermination( @@ -251,18 +283,18 @@ object Continuous extends DeprecatedContinuous { isCommand: Boolean ): State = Watch.withCharBufferedStdIn { in => val duped = new DupedInputStream(in) - setup(state.put(DupedSystemIn, duped), command) { (s, commands, valid, invalid) => + setup(state.put(DupedSystemIn, duped), command) { (s, valid, invalid) => implicit val extracted: Extracted = Project.extract(s) EvaluateTask.withStreams(extracted.structure, s)(_.use(Keys.streams in Global) { streams => implicit val logger: Logger = streams.log if (invalid.isEmpty) { val currentCount = new AtomicInteger(count) - val callbacks = - aggregate(getAllConfigs(s, commands), logger, in, state, currentCount, isCommand) + val configs = getAllConfigs(valid.map(v => v._1 -> v._2)) + val callbacks = aggregate(configs, logger, in, s, currentCount, isCommand) val task = () => { currentCount.getAndIncrement() // abort as soon as one of the tasks fails - valid.takeWhile(_.apply()) + valid.takeWhile(_._3.apply()) () } callbacks.onEnter() @@ -273,7 +305,10 @@ object Continuous extends DeprecatedContinuous { try { val terminationAction = Watch(task, callbacks.onStart, callbacks.nextEvent) callbacks.onTermination(terminationAction, command, currentCount.get(), state) - } finally callbacks.onExit() + } finally { + configs.foreach(_.repository.close()) + callbacks.onExit() + } } else { // At least one of the commands in the multi command string could not be parsed, so we // log an error and exit. @@ -285,28 +320,27 @@ object Continuous extends DeprecatedContinuous { } } - private def parseCommands(state: State, commands: Seq[String]): Seq[ScopedKey[_]] = { + private def parseCommand(command: String, state: State): Seq[ScopedKey[_]] = { // Collect all of the scoped keys that are used to delegate the multi commands. These are // necessary to extract all of the transitive globs that we need to monitor during watch. // We have to add the <~ Parsers.any.* to ensure that we're able to extract the input key // from input tasks. val scopedKeyParser: Parser[Seq[ScopedKey[_]]] = Act.aggregatedKeyParser(state) <~ Parsers.any.* - commands.flatMap { cmd: String => - Parser.parse(cmd, scopedKeyParser) match { - case Right(scopedKeys: Seq[ScopedKey[_]]) => scopedKeys - case Left(e) => - throw new IllegalStateException(s"Error attempting to extract scope from $cmd: $e.") - case _ => Nil: Seq[ScopedKey[_]] - } + Parser.parse(command, scopedKeyParser) match { + case Right(scopedKeys: Seq[ScopedKey[_]]) => scopedKeys + case Left(e) => + throw new IllegalStateException(s"Error attempting to extract scope from $command: $e.") + case _ => Nil: Seq[ScopedKey[_]] } } private def getAllConfigs( - state: State, - commands: Seq[String] + inputs: Seq[(String, State)] )(implicit extracted: Extracted, logger: Logger): Seq[Config] = { - val commandKeys = parseCommands(state, commands) + val commandKeys = inputs.map { case (c, s) => s -> parseCommand(c, s) } val compiledMap = InputGraph.compile(extracted.structure) - commandKeys.map((scopedKey: ScopedKey[_]) => getConfig(state, scopedKey, compiledMap)) + commandKeys.flatMap { + case (s, scopedKeys) => scopedKeys.map(getConfig(s, _, compiledMap)) + } } private class Callbacks( @@ -430,16 +464,16 @@ object Continuous extends DeprecatedContinuous { val ws = params.watchSettings ws.onTrigger .map(_.apply(params.arguments(logger))) - .getOrElse { - val globFilter = (params.inputs ++ params.triggers).toEntryFilter - event: Event => - if (globFilter(event.entry)) { - ws.triggerMessage match { - case Some(Left(tm)) => logger.info(tm(params.watchState(count.get()))) - case Some(Right(tm)) => tm(count.get(), event).foreach(logger.info(_)) - case None => // By default don't print anything - } + .getOrElse { event: Event => + val globFilter = + (params.inputs() ++ params.triggers).toEntryFilter + if (globFilter(event.entry)) { + ws.triggerMessage match { + case Some(Left(tm)) => logger.info(tm(params.watchState(count.get()))) + case Some(Right(tm)) => tm(count.get(), event).foreach(logger.info(_)) + case None => // By default don't print anything } + } } } event: Event => @@ -457,14 +491,16 @@ object Continuous extends DeprecatedContinuous { val onInputEvent = ws.onInputEvent.getOrElse(defaultTrigger) val onTriggerEvent = ws.onTriggerEvent.getOrElse(defaultTrigger) val onMetaBuildEvent = ws.onMetaBuildEvent.getOrElse(Watch.ifChanged(Watch.Reload)) - val inputFilter = params.inputs.toEntryFilter val triggerFilter = params.triggers.toEntryFilter + val excludedBuildFilter = buildFilter event: Event => + val inputFilter = params.inputs().toEntryFilter val c = count.get() + val entry = event.entry Seq[Watch.Action]( - if (inputFilter(event.entry)) onInputEvent(c, event) else Watch.Ignore, - if (triggerFilter(event.entry)) onTriggerEvent(c, event) else Watch.Ignore, - if (buildFilter(event.entry)) onMetaBuildEvent(c, event) else Watch.Ignore + if (inputFilter(entry)) onInputEvent(c, event) else Watch.Ignore, + if (triggerFilter(entry)) onTriggerEvent(c, event) else Watch.Ignore, + if (excludedBuildFilter(entry)) onMetaBuildEvent(c, event) else Watch.Ignore ).min } event: Event => @@ -477,18 +513,26 @@ object Continuous extends DeprecatedContinuous { f.view.map(_.apply(event)).minBy(_._2) } val monitor: FileEventMonitor[FileAttributes] = new FileEventMonitor[FileAttributes] { - private def setup( + + /** + * Create a filtered monitor that only accepts globs that have been registered for the + * task at hand. + * @param monitor the file event monitor to filter + * @param globs the globs to accept. This must be a function because we want to be able + * to accept globs that are added dynamically as part of task evaluation. + * @return the filtered FileEventMonitor. + */ + private def filter( monitor: FileEventMonitor[FileAttributes], - globs: Seq[Glob] + globs: () => Seq[Glob] ): FileEventMonitor[FileAttributes] = { - val globFilters = globs.toEntryFilter - val filter: Event => Boolean = (event: Event) => globFilters(event.entry) new FileEventMonitor[FileAttributes] { override def poll(duration: Duration): Seq[FileEventMonitor.Event[FileAttributes]] = - monitor.poll(duration).filter(filter) + monitor.poll(duration).filter(e => globs().toEntryFilter(e.entry)) override def close(): Unit = monitor.close() } } + // TODO make this a normal monitor private[this] val monitors: Seq[FileEventMonitor[FileAttributes]] = configs.map { config => // Create a logger with a scoped key prefix so that we can tell from which @@ -496,17 +540,20 @@ object Continuous extends DeprecatedContinuous { val l = logger.withPrefix(config.key.show) val monitor: FileEventMonitor[FileAttributes] = FileManagement.monitor(config.repository, config.watchSettings.antiEntropy, l) - val allGlobs = (config.inputs ++ config.triggers).distinct.sorted - setup(monitor, allGlobs) + val allGlobs: () => Seq[Glob] = () => (config.inputs() ++ config.triggers).distinct.sorted + filter(monitor, allGlobs) } ++ (if (trackMetaBuild) { val l = logger.withPrefix("meta-build") - val antiEntropy = configs.map(_.watchSettings.antiEntropy).min - setup(FileManagement.monitor(getRepository(state), antiEntropy, l), buildGlobs) :: Nil + val antiEntropy = configs.map(_.watchSettings.antiEntropy).max + val repo = getRepository(state) + buildGlobs.foreach(repo.register) + val monitor = FileManagement.monitor(repo, antiEntropy, l) + filter(monitor, () => buildGlobs) :: Nil } else Nil) override def poll(duration: Duration): Seq[FileEventMonitor.Event[FileAttributes]] = { - // The call to .par allows us to poll all of the monitors in parallel. - // This should be cheap because poll just blocks on a queue until an event is added. - monitors.par.flatMap(_.poll(duration)).toSet.toVector + val res = monitors.flatMap(_.poll(0.millis)).toSet.toVector + if (res.isEmpty) Thread.sleep(duration.toMillis) + res } override def close(): Unit = monitors.foreach(_.close()) } @@ -745,13 +792,13 @@ object Continuous extends DeprecatedContinuous { private final class Config private[internal] ( val key: ScopedKey[_], val repository: FileTreeRepository[FileAttributes], - val inputs: Seq[Glob], + val inputs: () => Seq[Glob], val triggers: Seq[Glob], val watchSettings: WatchSettings ) { private[sbt] def watchState(count: Int): DeprecatedWatchState = - WatchState.empty(inputs ++ triggers).withCount(count) - def arguments(logger: Logger): Arguments = new Arguments(logger, inputs, triggers) + WatchState.empty(inputs() ++ triggers).withCount(count) + def arguments(logger: Logger): Arguments = new Arguments(logger, inputs(), triggers) } private def getStartMessage(key: ScopedKey[_])(implicit e: Extracted): StartMessage = Some { lazy val default = key.get(Keys.watchStartMessage).getOrElse(Watch.defaultStartWatch) @@ -865,5 +912,4 @@ object Continuous extends DeprecatedContinuous { logger.log(level, s"$prefix - $message") } } - } diff --git a/main/src/main/scala/sbt/internal/ExternalHooks.scala b/main/src/main/scala/sbt/internal/ExternalHooks.scala index df30bcbd3..4ee753be5 100644 --- a/main/src/main/scala/sbt/internal/ExternalHooks.scala +++ b/main/src/main/scala/sbt/internal/ExternalHooks.scala @@ -7,13 +7,13 @@ package sbt.internal -import java.nio.file.{ Path, Paths } +import java.nio.file.Paths import java.util.Optional +import sbt.Stamped import sbt.internal.inc.ExternalLookup import sbt.io.syntax._ -import sbt.io.{ AllPassFilter, Glob, TypedPath } -import sbt.Stamped +import sbt.io.{ AllPassFilter, TypedPath } import xsbti.compile._ import xsbti.compile.analysis.Stamp @@ -22,7 +22,6 @@ import scala.collection.mutable private[sbt] object ExternalHooks { private val javaHome = Option(System.getProperty("java.home")).map(Paths.get(_)) def apply(options: CompileOptions, repo: FileTree.Repository): DefaultExternalHooks = { - def listEntries(glob: Glob): Seq[(Path, FileAttributes)] = repo.get(glob) import scala.collection.JavaConverters._ val sources = options.sources() val cachedSources = new java.util.HashMap[File, Stamp] @@ -34,13 +33,9 @@ private[sbt] object ExternalHooks { val allBinaries = new java.util.HashMap[File, Stamp] options.classpath.foreach { case f if f.getName.endsWith(".jar") => - // This gives us the entry for the path itself, which is necessary if the path is a jar file - // rather than a directory. - listEntries(f.toGlob) foreach { case (p, a) => allBinaries.put(p.toFile, a.stamp) } + repo.get(f.toGlob) foreach { case (p, a) => allBinaries.put(p.toFile, a.stamp) } case f => - listEntries(f ** AllPassFilter) foreach { - case (p, a) => allBinaries.put(p.toFile, a.stamp) - } + repo.get(f ** AllPassFilter) foreach { case (p, a) => allBinaries.put(p.toFile, a.stamp) } } val lookup = new ExternalLookup { diff --git a/main/src/main/scala/sbt/internal/FileManagement.scala b/main/src/main/scala/sbt/internal/FileManagement.scala index 88c61ed95..42a7d4a43 100644 --- a/main/src/main/scala/sbt/internal/FileManagement.scala +++ b/main/src/main/scala/sbt/internal/FileManagement.scala @@ -95,6 +95,7 @@ private[sbt] object FileManagement { override def close(): Unit = monitor.close() } } + private[sbt] class HybridMonitoringRepository[T]( underlying: HybridPollingFileTreeRepository[T], delay: FiniteDuration, diff --git a/main/src/main/scala/sbt/internal/FileTree.scala b/main/src/main/scala/sbt/internal/FileTree.scala index 7b0919056..14895b48c 100644 --- a/main/src/main/scala/sbt/internal/FileTree.scala +++ b/main/src/main/scala/sbt/internal/FileTree.scala @@ -14,12 +14,22 @@ import sbt.internal.util.appmacro.MacroDefaults import sbt.io.FileTreeDataView.Entry import sbt.io._ +import scala.collection.mutable import scala.language.experimental.macros object FileTree { private def toPair(e: Entry[FileAttributes]): Option[(Path, FileAttributes)] = e.value.toOption.map(a => e.typedPath.toPath -> a) trait Repository extends sbt.internal.Repository[Seq, Glob, (Path, FileAttributes)] + private[sbt] trait DynamicInputs { + def value: Option[mutable.Set[Glob]] + } + private[sbt] object DynamicInputs { + def empty: DynamicInputs = new impl(Some(mutable.Set.empty[Glob])) + final val none: DynamicInputs = new impl(None) + private final class impl(override val value: Option[mutable.Set[Glob]]) extends DynamicInputs + implicit def default: DynamicInputs = macro MacroDefaults.dynamicInputs + } private[sbt] object Repository { /** @@ -45,7 +55,8 @@ object FileTree { extends Repository { override def get(key: Glob): Seq[(Path, FileAttributes)] = { underlying.register(key) - underlying.listEntries(key).flatMap(toPair) + //underlying.listEntries(key).flatMap(toPair).distinct + Repository.polling.get(key) } override def close(): Unit = underlying.close() } diff --git a/main/src/main/scala/sbt/internal/GlobLister.scala b/main/src/main/scala/sbt/internal/GlobLister.scala index 03483b312..050d3affb 100644 --- a/main/src/main/scala/sbt/internal/GlobLister.scala +++ b/main/src/main/scala/sbt/internal/GlobLister.scala @@ -8,9 +8,14 @@ package sbt package internal +import java.io.File import java.nio.file.Path +import java.util.concurrent.ConcurrentSkipListMap -import sbt.io.Glob +import sbt.io.{ FileFilter, Glob, SimpleFileFilter } + +import scala.collection.JavaConverters._ +import scala.collection.mutable /** * Retrieve files from a repository. This should usually be an extension class for @@ -19,21 +24,21 @@ import sbt.io.Glob */ private[sbt] sealed trait GlobLister extends Any { - /** - * Get the sources described this `GlobLister`. - * - * @param repository the [[FileTree.Repository]] to delegate file i/o. - * @return the files described by this `GlobLister`. - */ - def all(implicit repository: FileTree.Repository): Seq[(Path, FileAttributes)] + final def all(repository: FileTree.Repository): Seq[(Path, FileAttributes)] = + all(repository, FileTree.DynamicInputs.empty) /** - * Get the unique sources described this `GlobLister`. + * Get the sources described this `GlobLister`. The results should not return any duplicate + * entries for each path in the result set. * - * @param repository the [[FileTree.Repository]] to delegate file i/o. - * @return the files described by this `GlobLister` with any duplicates removed. + * @param repository the file tree repository for retrieving the files for a given glob. + * @param dynamicInputs the task dynamic inputs to track for watch. + * @return the files described by this `GlobLister`. */ - def unique(implicit repository: FileTree.Repository): Seq[(Path, FileAttributes)] + def all( + implicit repository: FileTree.Repository, + dynamicInputs: FileTree.DynamicInputs + ): Seq[(Path, FileAttributes)] } /** @@ -57,10 +62,7 @@ private[sbt] trait GlobListers { implicit def fromGlob(source: Glob): GlobLister = new impl(source :: Nil) /** - * Generate a GlobLister given a collection of Globs. If the input collection type - * preserves uniqueness, e.g. `Set[Glob]`, then the output of `GlobLister.all` will be - * the unique source list. Otherwise duplicates are possible in all and it is necessary to call - * `GlobLister.unique` to de-duplicate the files. + * Generate a GlobLister given a collection of Globs. * * @param sources the collection of sources * @tparam T the source collection type @@ -69,6 +71,34 @@ private[sbt] trait GlobListers { new impl(sources) } private[internal] object GlobListers { + private def covers(left: Glob, right: Glob): Boolean = { + right.base.startsWith(left.base) && { + left.depth == Int.MaxValue || { + val depth = left.base.relativize(right.base).getNameCount + depth < left.depth - right.depth + } + } + } + private def aggregate(globs: Traversable[Glob]): Seq[(Glob, Traversable[Glob])] = { + val sorted = globs.toSeq.sorted + val map = new ConcurrentSkipListMap[Path, (Glob, mutable.Set[Glob])] + if (sorted.size > 1) { + sorted.foreach { glob => + map.subMap(glob.base.getRoot, glob.base.resolve(Char.MaxValue.toString)).asScala.find { + case (_, (g, _)) => covers(g, glob) + } match { + case Some((_, (_, globs))) => globs += glob + case None => + val globs = mutable.Set(glob) + val filter: FileFilter = new SimpleFileFilter((file: File) => { + globs.exists(_.toFileFilter.accept(file)) + }) + map.put(glob.base, (Glob(glob.base, filter, glob.depth), globs)) + } + } + map.asScala.values.toIndexedSeq + } else sorted.map(g => g -> (g :: Nil)) + } /** * Implements `GlobLister` given a collection of Globs. If the input collection type @@ -79,18 +109,15 @@ private[internal] object GlobListers { * @tparam T the collection type */ private class impl[T <: Traversable[Glob]](val globs: T) extends AnyVal with GlobLister { - private def get[T0 <: Traversable[Glob]]( - traversable: T0, - repository: FileTree.Repository - ): Seq[(Path, FileAttributes)] = - traversable.flatMap { glob => - val sourceFilter = glob.toFileFilter - repository.get(glob).filter { case (p, _) => sourceFilter.accept(p.toFile) } + override def all( + implicit repository: FileTree.Repository, + dynamicInputs: FileTree.DynamicInputs + ): Seq[(Path, FileAttributes)] = { + aggregate(globs).flatMap { + case (glob, allGlobs) => + dynamicInputs.value.foreach(_ ++= allGlobs) + repository.get(glob) }.toIndexedSeq - - override def all(implicit repository: FileTree.Repository): Seq[(Path, FileAttributes)] = - get(globs, repository) - override def unique(implicit repository: FileTree.Repository): Seq[(Path, FileAttributes)] = - get(globs.toSet[Glob], repository) + } } } diff --git a/sbt/src/sbt-test/tests/glob-dsl/build.sbt b/sbt/src/sbt-test/tests/glob-dsl/build.sbt index 16b161d89..c6452cd51 100644 --- a/sbt/src/sbt-test/tests/glob-dsl/build.sbt +++ b/sbt/src/sbt-test/tests/glob-dsl/build.sbt @@ -46,12 +46,9 @@ val checkSet = taskKey[Unit]("Verify that redundant sources are handled") checkSet := { val redundant = (set / fileInputs).value.all.map(_._1.toFile) - assert(redundant.size == 4) // It should get Foo.txt and Bar.md twice + assert(redundant.size == 2) val deduped = (set / fileInputs).value.toSet[Glob].all.map(_._1.toFile) val expected = Seq("Bar.md", "Foo.txt").map(baseDirectory.value / "base/subdir/nested-subdir" / _) assert(deduped.sorted == expected) - - val altDeduped = (set / fileInputs).value.unique.map(_._1.toFile) - assert(altDeduped.sorted == expected) } diff --git a/sbt/src/sbt-test/tests/inputs/build.sbt b/sbt/src/sbt-test/tests/inputs/build.sbt index 88cc5a636..54e1ce2fb 100644 --- a/sbt/src/sbt-test/tests/inputs/build.sbt +++ b/sbt/src/sbt-test/tests/inputs/build.sbt @@ -1,8 +1,6 @@ import java.nio.file.Path import sbt.internal.{FileAttributes, FileTree} -import sbt.io.FileTreeDataView -import xsbti.compile.analysis.Stamp val allInputs = taskKey[Seq[File]]("") val allInputsExplicit = taskKey[Seq[File]]("") diff --git a/sbt/src/sbt-test/watch/dynamic-inputs/build.sbt b/sbt/src/sbt-test/watch/dynamic-inputs/build.sbt new file mode 100644 index 000000000..3e1169f6d --- /dev/null +++ b/sbt/src/sbt-test/watch/dynamic-inputs/build.sbt @@ -0,0 +1,3 @@ +import sbt.watch.task.Build + +val root = Build.root diff --git a/sbt/src/sbt-test/watch/dynamic-inputs/project/Build.scala b/sbt/src/sbt-test/watch/dynamic-inputs/project/Build.scala new file mode 100644 index 000000000..474a6b715 --- /dev/null +++ b/sbt/src/sbt-test/watch/dynamic-inputs/project/Build.scala @@ -0,0 +1,41 @@ +package sbt.watch.task + +import sbt._ +import Keys._ +import sbt.internal.FileTree + +object Build { + val reloadFile = settingKey[File]("file to toggle whether or not to reload") + val setStringValue = taskKey[Unit]("set a global string to a value") + val checkStringValue = inputKey[Unit]("check the value of a global") + val foo = taskKey[Unit]("foo") + def setStringValueImpl: Def.Initialize[Task[Unit]] = Def.task { + val i = (setStringValue / fileInputs).value + val (stringFile, string) = ("foo.txt", "bar") + IO.write(file(stringFile), string) + } + def checkStringValueImpl: Def.Initialize[InputTask[Unit]] = Def.inputTask { + val Seq(stringFile, string) = Def.spaceDelimited().parsed + assert(IO.read(file(stringFile)) == string) + } + lazy val root = (project in file(".")).settings( + reloadFile := baseDirectory.value / "reload", + foo / fileInputs += baseDirectory.value * "foo.txt", + setStringValue := Def.taskDyn { + // This hides foo / fileInputs from the input graph + Def.taskDyn { + val _ = (foo / fileInputs).value.all + // By putting setStringValueImpl.value inside a Def.task, we ensure that + // (foo / fileInputs).value is registered with the file repository before modifying the file. + Def.task(setStringValueImpl.value) + } + }.value, + checkStringValue := checkStringValueImpl.evaluated, + watchOnInputEvent := { (_, _) => Watch.CancelWatch }, + watchOnTriggerEvent := { (_, _) => Watch.CancelWatch }, + watchTasks := Def.inputTask { + val prev = watchTasks.evaluated + new StateTransform(prev.state.fail) + }.evaluated + ) +} \ No newline at end of file diff --git a/sbt/src/sbt-test/watch/dynamic-inputs/test b/sbt/src/sbt-test/watch/dynamic-inputs/test new file mode 100644 index 000000000..57725e3a4 --- /dev/null +++ b/sbt/src/sbt-test/watch/dynamic-inputs/test @@ -0,0 +1,7 @@ +# This tests that we can override the state transformation in the watch task +# In the build, watchOnEvent should return CancelWatch which should be successful, but we +# override watchTasks to fail the state instead + +-> watch root / setStringValue + +> checkStringValue foo.txt bar