mirror of https://github.com/sbt/sbt.git
Re-implement meta build source checking
The old implementation of checkBuildSources can easily take 20ms to run when called in MainLoop.processCommand. It is rarely faster than 4-5ms. To reduce this overhead, I stopped using the checkBuildSources task in processCommand. Instead, I manually cache the build source hashes in a global state variable and add a file monitor that invalidate the entire set of source hashes if any changes are detected. This could probably be more efficient, but I figure that build sources change infrequently enough that it's fine to just invalidate the entire list of source hashes. Because the CheckBuildSources instance is already watching the meta build, I reworked Continuous to use that FileTreeRepository for the build sources if it is available. Bonus: fixes https://github.com/sbt/sbt/issues/5482
This commit is contained in:
parent
b829ac9796
commit
f4c11a63ab
|
|
@ -413,7 +413,9 @@ object Defaults extends BuildCommon {
|
|||
sourceManaged := crossTarget.value / "src_managed",
|
||||
resourceManaged := crossTarget.value / "resource_managed",
|
||||
// Adds subproject build.sbt files to the global list of build files to monitor
|
||||
Scope.Global / checkBuildSources / fileInputs += baseDirectory.value.toGlob / "*.sbt"
|
||||
Scope.Global / checkBuildSources / pollInterval :==
|
||||
new FiniteDuration(Int.MinValue, TimeUnit.MILLISECONDS),
|
||||
Scope.Global / checkBuildSources / fileInputs += baseDirectory.value.toGlob / "*.sbt",
|
||||
)
|
||||
|
||||
lazy val configPaths = sourceConfigPaths ++ resourceConfigPaths ++ outputConfigPaths
|
||||
|
|
|
|||
|
|
@ -12,7 +12,6 @@ import java.net.URI
|
|||
import java.nio.file.{ FileAlreadyExistsException, FileSystems, Files }
|
||||
import java.util.Properties
|
||||
import java.util.concurrent.ForkJoinPool
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import sbt.BasicCommandStrings.{ Shell, TemplateCommand }
|
||||
import sbt.Project.LoadAction
|
||||
|
|
@ -21,6 +20,7 @@ import sbt.internal.Aggregation.AnyKeys
|
|||
import sbt.internal.CommandStrings.BootCommand
|
||||
import sbt.internal._
|
||||
import sbt.internal.inc.ScalaInstance
|
||||
import sbt.internal.nio.CheckBuildSources
|
||||
import sbt.internal.util.Types.{ const, idFun }
|
||||
import sbt.internal.util._
|
||||
import sbt.internal.util.complete.{ Parser, SizeParser }
|
||||
|
|
@ -854,12 +854,9 @@ object BuiltinCommands {
|
|||
|
||||
val session = Load.initialSession(structure, eval, s0)
|
||||
SessionSettings.checkSession(session, s2)
|
||||
val s3 = addCacheStoreFactoryFactory(
|
||||
Project
|
||||
.setProject(session, structure, s2)
|
||||
.put(sbt.nio.Keys.hasCheckedMetaBuild, new AtomicBoolean(false))
|
||||
)
|
||||
LintUnused.lintUnusedFunc(s3)
|
||||
val s3 = addCacheStoreFactoryFactory(Project.setProject(session, structure, s2))
|
||||
val s4 = LintUnused.lintUnusedFunc(s3)
|
||||
CheckBuildSources.init(s4)
|
||||
}
|
||||
|
||||
private val addCacheStoreFactoryFactory: State => State = (s: State) => {
|
||||
|
|
|
|||
|
|
@ -10,15 +10,14 @@ package sbt
|
|||
import java.io.PrintWriter
|
||||
import java.util.Properties
|
||||
|
||||
import sbt.internal.{ Aggregation, ShutdownHooks }
|
||||
import sbt.internal.ShutdownHooks
|
||||
import sbt.internal.langserver.ErrorCodes
|
||||
import sbt.internal.protocol.JsonRpcResponseError
|
||||
import sbt.internal.util.complete.Parser
|
||||
import sbt.internal.nio.CheckBuildSources.CheckBuildSourcesKey
|
||||
import sbt.internal.util.{ ErrorHandling, GlobalLogBacking, Terminal }
|
||||
import sbt.io.{ IO, Using }
|
||||
import sbt.protocol._
|
||||
import sbt.util.Logger
|
||||
import sbt.nio.Keys._
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.util.control.NonFatal
|
||||
|
|
@ -191,8 +190,6 @@ object MainLoop {
|
|||
} else state
|
||||
}
|
||||
val newState = Command.process(exec.commandLine, progressState)
|
||||
if (exec.commandLine.contains("session"))
|
||||
newState.get(hasCheckedMetaBuild).foreach(_.set(false))
|
||||
val doneEvent = ExecStatusEvent(
|
||||
"Done",
|
||||
channelName,
|
||||
|
|
@ -209,25 +206,11 @@ object MainLoop {
|
|||
newState.get(sbt.Keys.currentTaskProgress).foreach(_.progress.stop())
|
||||
newState.remove(sbt.Keys.currentTaskProgress)
|
||||
}
|
||||
// The split on space is to handle 'reboot full' and 'reboot'.
|
||||
state.currentCommand.flatMap(_.commandLine.trim.split(" ").headOption) match {
|
||||
case Some("reload") =>
|
||||
// Reset the hasCheckedMetaBuild parameter so that the next call to checkBuildSources
|
||||
// updates the previous cache for checkBuildSources / fileInputStamps but doesn't log.
|
||||
state.get(hasCheckedMetaBuild).foreach(_.set(false))
|
||||
process()
|
||||
case Some("exit") | Some("reboot") => process()
|
||||
case _ =>
|
||||
val emptyState = state.copy(remainingCommands = Nil).put(Aggregation.suppressShow, true)
|
||||
Parser.parse("checkBuildSources", emptyState.combinedParser) match {
|
||||
case Right(cmd) =>
|
||||
cmd() match {
|
||||
case s if s.remainingCommands.headOption.map(_.commandLine).contains("reload") =>
|
||||
Exec("reload", None, None) +: exec +: state
|
||||
case _ => process()
|
||||
}
|
||||
case _ => process()
|
||||
}
|
||||
state.get(CheckBuildSourcesKey) match {
|
||||
case Some(cbs) =>
|
||||
if (!cbs.needsReload(state, exec.commandLine)) process()
|
||||
else Exec("reload", None, None) +: exec +: state.remove(CheckBuildSourcesKey)
|
||||
case _ => process()
|
||||
}
|
||||
} catch {
|
||||
case err: JsonRpcResponseError =>
|
||||
|
|
|
|||
|
|
@ -588,7 +588,8 @@ private[sbt] object Continuous extends DeprecatedContinuous {
|
|||
|
||||
private[this] val observers: Observers[Event] = new Observers
|
||||
private[this] val repo = getRepository(state)
|
||||
private[this] val handle = repo.addObserver(observers)
|
||||
private[this] val handles = new java.util.ArrayList[AutoCloseable]
|
||||
handles.add(repo.addObserver(observers))
|
||||
private[this] val eventMonitorObservers = new Observers[Event]
|
||||
private[this] val configHandle: AutoCloseable =
|
||||
observers.addObserver { e =>
|
||||
|
|
@ -608,7 +609,12 @@ private[sbt] object Continuous extends DeprecatedContinuous {
|
|||
eventMonitorObservers.onNext(e)
|
||||
}
|
||||
}
|
||||
if (trackMetaBuild) buildGlobs.foreach(repo.register)
|
||||
if (trackMetaBuild) {
|
||||
state.get(CheckBuildSources.CheckBuildSourcesKey).flatMap(_.fileTreeRepository) match {
|
||||
case Some(r) => buildGlobs.foreach(r.register(_).foreach(observers.addObservable))
|
||||
case _ => buildGlobs.foreach(repo.register)
|
||||
}
|
||||
}
|
||||
|
||||
private[this] val monitor = FileEventMonitor.antiEntropy(
|
||||
eventMonitorObservers,
|
||||
|
|
@ -623,7 +629,7 @@ private[sbt] object Continuous extends DeprecatedContinuous {
|
|||
|
||||
override def close(): Unit = {
|
||||
configHandle.close()
|
||||
handle.close()
|
||||
handles.forEach(_.close())
|
||||
}
|
||||
}
|
||||
val watchLogger: WatchLogger = msg => logger.debug(msg.toString)
|
||||
|
|
|
|||
|
|
@ -8,56 +8,153 @@
|
|||
package sbt
|
||||
package internal.nio
|
||||
|
||||
import sbt.Keys.{ baseDirectory, state, streams }
|
||||
import java.nio.file.Path
|
||||
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference }
|
||||
import sbt.BasicCommandStrings.{ RebootCommand, TerminateAction }
|
||||
import sbt.Keys.{ baseDirectory, pollInterval, state }
|
||||
import sbt.Scope.Global
|
||||
import sbt.SlashSyntax0._
|
||||
import sbt.internal.CommandStrings.LoadProject
|
||||
import sbt.internal.SysProp
|
||||
import sbt.internal.util.AttributeKey
|
||||
import sbt.io.syntax._
|
||||
import sbt.nio.FileChanges
|
||||
import sbt.nio.FileStamp
|
||||
import sbt.nio.Keys._
|
||||
import sbt.nio.file.{ Glob, ** }
|
||||
import sbt.nio.file.{ FileAttributes, FileTreeView, Glob, ** }
|
||||
import sbt.nio.file.syntax._
|
||||
import sbt.nio.Settings
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration.{ Deadline => SDeadline, _ }
|
||||
|
||||
/**
|
||||
* This class is used to determine whether sbt needs to automatically reload
|
||||
* the build because its source files have changed. In general, it will use
|
||||
* a FileTreeRepository to monitor the build source directories and it will
|
||||
* only actually check whether any sources have changed if the monitor has
|
||||
* detected any events. Because it's using asynchronous monitoring by default,
|
||||
* the automatic reloading should not be relied upon in batch scripting. It is
|
||||
* possible to configure this feature by setting
|
||||
* `Global / onChangedBuildSource / pollInterval`. When this value is set to
|
||||
* 0.seconds, then it will poll every time. Otherwise, it will only repoll
|
||||
* the build files if the poll interval has elapsed.
|
||||
*/
|
||||
private[sbt] class CheckBuildSources extends AutoCloseable {
|
||||
private[this] val repository = new AtomicReference[FileTreeRepository[FileAttributes]]
|
||||
private[this] val pollingPeriod = new AtomicReference[FiniteDuration]
|
||||
private[this] val sources = new AtomicReference[Seq[Glob]](Nil)
|
||||
private[this] val needUpdate = new AtomicBoolean(true)
|
||||
private[this] val lastPolled = new AtomicReference[SDeadline](SDeadline.now)
|
||||
private[this] val previousStamps = new AtomicReference[Seq[(Path, FileStamp)]]
|
||||
private[sbt] def fileTreeRepository: Option[FileTreeRepository[FileAttributes]] =
|
||||
Option(repository.get)
|
||||
private def getStamps(force: Boolean) = {
|
||||
val now = SDeadline.now
|
||||
val lp = lastPolled.getAndSet(now)
|
||||
if (force || lp + pollingPeriod.get <= now) {
|
||||
FileTreeView.default.list(sources.get) flatMap {
|
||||
case (p, a) if a.isRegularFile => FileStamp.hash(p).map(p -> _)
|
||||
case _ => None
|
||||
}
|
||||
} else previousStamps.get
|
||||
}
|
||||
private def reset(state: State): Unit = {
|
||||
val extracted = Project.extract(state)
|
||||
val interval = extracted.get(checkBuildSources / pollInterval)
|
||||
val newSources = extracted.get(Global / checkBuildSources / fileInputs)
|
||||
if (interval >= 0.seconds || "polling" == SysProp.watchMode) {
|
||||
Option(repository.getAndSet(null)).foreach(_.close())
|
||||
pollingPeriod.set(interval)
|
||||
} else {
|
||||
pollingPeriod.set(0.seconds)
|
||||
repository.get match {
|
||||
case null =>
|
||||
val repo = FileTreeRepository.default
|
||||
repo.addObserver(_ => needUpdate.set(true))
|
||||
repository.set(repo)
|
||||
newSources.foreach(g => repo.register(g))
|
||||
case r =>
|
||||
}
|
||||
}
|
||||
val previousSources = sources.getAndSet(newSources)
|
||||
if (previousSources != newSources) {
|
||||
fileTreeRepository.foreach(r => newSources.foreach(g => r.register(g)))
|
||||
previousStamps.set(getStamps(force = true))
|
||||
}
|
||||
}
|
||||
private def needCheck(cmd: String): Boolean = {
|
||||
val commands = cmd.split(";").flatMap(_.trim.split(" ").headOption).filterNot(_.isEmpty)
|
||||
val res = !commands.exists { c =>
|
||||
c == LoadProject || c == RebootCommand || c == TerminateAction || c == "shutdown"
|
||||
}
|
||||
if (!res) {
|
||||
previousStamps.set(getStamps(force = true))
|
||||
needUpdate.set(false)
|
||||
}
|
||||
res
|
||||
}
|
||||
@inline private def forceCheck = fileTreeRepository.isEmpty
|
||||
private[sbt] def needsReload(state: State, cmd: String) = {
|
||||
(needCheck(cmd) && (forceCheck || needUpdate.compareAndSet(true, false))) && {
|
||||
val extracted = Project.extract(state)
|
||||
val onChanges = extracted.get(Global / onChangedBuildSource)
|
||||
val logger = state.globalLogging.full
|
||||
val current = getStamps(force = false)
|
||||
val previous = previousStamps.getAndSet(current)
|
||||
Settings.changedFiles(previous, current) match {
|
||||
case fileChanges @ FileChanges(created, deleted, modified, _) if fileChanges.hasChanges =>
|
||||
val rawPrefix = s"build source files have changed\n" +
|
||||
(if (created.nonEmpty) s"new files: ${created.mkString("\n ", "\n ", "\n")}"
|
||||
else "") +
|
||||
(if (deleted.nonEmpty)
|
||||
s"deleted files: ${deleted.mkString("\n ", "\n ", "\n")}"
|
||||
else "") +
|
||||
(if (modified.nonEmpty)
|
||||
s"modified files: ${modified.mkString("\n ", "\n ", "\n")}"
|
||||
else "")
|
||||
val prefix = rawPrefix.linesIterator.filterNot(_.trim.isEmpty).mkString("\n")
|
||||
if (onChanges == ReloadOnSourceChanges) {
|
||||
logger.info(s"$prefix\nReloading sbt...")
|
||||
true
|
||||
} else {
|
||||
val tail = "Apply these changes by running `reload`.\nAutomatically reload the " +
|
||||
"build when source changes are detected by setting " +
|
||||
"`Global / onChangedBuildSource := ReloadOnSourceChanges`.\nDisable this " +
|
||||
"warning by setting `Global / onChangedBuildSource := IgnoreSourceChanges`."
|
||||
logger.warn(s"$prefix\n$tail")
|
||||
false
|
||||
}
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
}
|
||||
override def close(): Unit = {}
|
||||
}
|
||||
|
||||
private[sbt] object CheckBuildSources {
|
||||
private[sbt] val CheckBuildSourcesKey =
|
||||
AttributeKey[CheckBuildSources]("check-build-source", "", KeyRanks.Invisible)
|
||||
/*
|
||||
* Reuse the same instance of CheckBuildSources across reloads but reset the state. This
|
||||
* should allow the `set` command to work with checkBuildSources / fileInputs and
|
||||
* checkBuildSources / pollInterval. The latter makes it possible to switch between
|
||||
* the asynchronous and polling implementations during the same sbt session.
|
||||
*/
|
||||
private[sbt] def init(state: State): State = state.get(CheckBuildSourcesKey) match {
|
||||
case Some(cbs) =>
|
||||
cbs.reset(state)
|
||||
state
|
||||
case _ =>
|
||||
val cbs = new CheckBuildSources
|
||||
cbs.reset(state)
|
||||
state.put(CheckBuildSourcesKey, cbs)
|
||||
}
|
||||
private[sbt] def needReloadImpl: Def.Initialize[Task[StateTransform]] = Def.task {
|
||||
val logger = streams.value.log
|
||||
val st: State = state.value
|
||||
val firstTime = st.get(hasCheckedMetaBuild).fold(true)(_.compareAndSet(false, true))
|
||||
(onChangedBuildSource in Scope.Global).value match {
|
||||
case IgnoreSourceChanges => StateTransform(identity)
|
||||
case o =>
|
||||
import sbt.nio.FileStamp.Formats._
|
||||
logger.debug("Checking for meta build source updates")
|
||||
val previous = (inputFileStamps in checkBuildSources).previous
|
||||
val changes = (changedInputFiles in checkBuildSources).value
|
||||
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 (created.nonEmpty) s"new files: ${created.mkString("\n ", "\n ", "\n")}"
|
||||
else "") +
|
||||
(if (deleted.nonEmpty)
|
||||
s"deleted files: ${deleted.mkString("\n ", "\n ", "\n")}"
|
||||
else "") +
|
||||
(if (modified.nonEmpty)
|
||||
s"modified files: ${modified.mkString("\n ", "\n ", "\n")}"
|
||||
else "")
|
||||
val prefix = rawPrefix.linesIterator.filterNot(_.trim.isEmpty).mkString("\n")
|
||||
if (o == ReloadOnSourceChanges) {
|
||||
logger.info(s"$prefix\nReloading sbt...")
|
||||
val remaining =
|
||||
Exec("reload", None, None) :: st.currentCommand.toList ::: st.remainingCommands
|
||||
StateTransform(_.copy(currentCommand = None, remainingCommands = remaining))
|
||||
} else {
|
||||
val tail = "Apply these changes by running `reload`.\nAutomatically reload the " +
|
||||
"build when source changes are detected by setting " +
|
||||
"`Global / onChangedBuildSource := ReloadOnSourceChanges`.\nDisable this " +
|
||||
"warning by setting `Global / onChangedBuildSource := IgnoreSourceChanges`."
|
||||
logger.warn(s"$prefix\n$tail")
|
||||
StateTransform(identity)
|
||||
}
|
||||
case _ => StateTransform(identity)
|
||||
}
|
||||
val st = state.value
|
||||
st.get(CheckBuildSourcesKey) match {
|
||||
case Some(cbs) if (cbs.needsReload(st, "")) => StateTransform("reload" :: (_: State))
|
||||
case _ => StateTransform(identity)
|
||||
}
|
||||
}
|
||||
private[sbt] def buildSourceFileInputs: Def.Initialize[Seq[Glob]] = Def.setting {
|
||||
|
|
|
|||
|
|
@ -1,3 +1,5 @@
|
|||
import scala.concurrent.duration._
|
||||
|
||||
val foo = inputKey[Unit]("working task")
|
||||
foo := {
|
||||
val filename = Def.spaceDelimited("").parsed.head
|
||||
|
|
@ -12,4 +14,6 @@ exists := {
|
|||
|
||||
Global / onChangedBuildSource := ReloadOnSourceChanges
|
||||
|
||||
ThisBuild / checkBuildSources / pollInterval := 0.seconds
|
||||
|
||||
val sub = project
|
||||
|
|
|
|||
|
|
@ -1,3 +1,5 @@
|
|||
import scala.concurrent.duration._
|
||||
|
||||
val foo = inputKey[Unit]("broken task")
|
||||
foo := { throw new IllegalStateException("foo") }
|
||||
|
||||
|
|
@ -9,4 +11,6 @@ exists := {
|
|||
|
||||
Global / onChangedBuildSource := ReloadOnSourceChanges
|
||||
|
||||
ThisBuild / checkBuildSources / pollInterval := 0.seconds
|
||||
|
||||
val sub = project
|
||||
|
|
|
|||
|
|
@ -1 +1,5 @@
|
|||
libraryDependencies += "org.scala-sbt" % "sbt" % "1.3.0"
|
||||
import scala.concurrent.duration._
|
||||
|
||||
libraryDependencies += "org.scala-sbt" % "sbt" % "1.3.0"
|
||||
|
||||
ThisBuild / checkBuildSources / pollInterval := 0.seconds
|
||||
|
|
|
|||
|
|
@ -1,3 +1,5 @@
|
|||
import scala.concurrent.duration._
|
||||
|
||||
val foo = inputKey[Unit]("working task")
|
||||
foo := {
|
||||
val filename = Def.spaceDelimited("").parsed.head
|
||||
|
|
@ -12,4 +14,6 @@ exists := {
|
|||
|
||||
Global / onChangedBuildSource := ReloadOnSourceChanges
|
||||
|
||||
ThisBuild / checkBuildSources / pollInterval := 0.seconds
|
||||
|
||||
val sub = project
|
||||
|
|
|
|||
|
|
@ -1,3 +1,5 @@
|
|||
> show Global / checkBuildSources / pollInterval
|
||||
|
||||
> foo bar
|
||||
|
||||
> exists bar
|
||||
|
|
|
|||
Loading…
Reference in New Issue