Merge pull request #4576 from eed3si9n/wip/trace

output Chrome traces on -Dsbt.traces=true
This commit is contained in:
eugene yokota 2019-03-25 17:21:59 -04:00 committed by GitHub
commit de5c0434df
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 226 additions and 96 deletions

View File

@ -62,6 +62,29 @@ object Def extends Init[Scope] with TaskMacroExtra {
)
)
private[sbt] def showShortKey(
keyNameColor: Option[String],
): Show[ScopedKey[_]] = {
def displayShort(
project: Reference
): String = {
val trailing = " /"
project match {
case BuildRef(_) => "ThisBuild" + trailing
case ProjectRef(_, x) => x + trailing
case _ => Reference.display(project) + trailing
}
}
Show[ScopedKey[_]](
key =>
Scope.display(
key.scope,
withColor(key.key.label, keyNameColor),
ref => displayShort(ref)
)
)
}
@deprecated("Use showBuildRelativeKey2 which doesn't take the unused multi param", "1.1.1")
def showBuildRelativeKey(
currentBuild: URI,

View File

@ -266,8 +266,9 @@ object Defaults extends BuildCommon {
useSuperShell :== sbt.internal.TaskProgress.isEnabled,
progressReports := { (s: State) =>
val progress = useSuperShell.value
val rs = EvaluateTask.taskTimingProgress.toVector ++ {
if (progress) Vector(EvaluateTask.taskProgress(s))
val rs = EvaluateTask.taskTimingProgress.toVector ++
EvaluateTask.taskTraceEvent.toVector ++ {
if (progress) Vector(EvaluateTask.taskProgress)
else Vector()
}
rs map { Keys.TaskProgress(_) }

View File

@ -7,7 +7,15 @@
package sbt
import sbt.internal.{ Load, BuildStructure, TaskTimings, TaskName, GCUtil, TaskProgress }
import sbt.internal.{
Load,
BuildStructure,
TaskTimings,
TaskName,
GCUtil,
TaskProgress,
TaskTraceEvent
}
import sbt.internal.util.{ Attributed, ConsoleAppender, ErrorHandling, HList, RMap, Signals, Types }
import sbt.util.{ Logger, Show }
import sbt.librarymanagement.{ Resolver, UpdateReport }
@ -170,9 +178,13 @@ object EvaluateTask {
Some(new TaskTimings(reportOnShutdown = false))
} else None
def taskProgress(state: State): ExecuteProgress[Task] = {
new TaskProgress(Project.extract(state).currentRef)
}
lazy private val sharedTraceEvent = new TaskTraceEvent()
def taskTraceEvent: Option[ExecuteProgress[Task]] =
if (java.lang.Boolean.getBoolean("sbt.traces")) {
Some(sharedTraceEvent)
} else None
def taskProgress: ExecuteProgress[Task] = new TaskProgress()
// sbt-pgp calls this
@deprecated("No longer used", "1.3.0")

View File

@ -0,0 +1,81 @@
/*
* 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.util.concurrent.ConcurrentHashMap
import scala.collection.concurrent.TrieMap
import scala.collection.JavaConverters._
private[sbt] abstract class AbstractTaskExecuteProgress extends ExecuteProgress[Task] {
import AbstractTaskExecuteProgress.Timer
private[this] val showScopedKey = Def.showShortKey(None)
private[this] val anonOwners = new ConcurrentHashMap[Task[_], Task[_]]
private[this] val calledBy = new ConcurrentHashMap[Task[_], Task[_]]
private[this] val activeTasksMap = new ConcurrentHashMap[Task[_], Unit]
protected val timings = new ConcurrentHashMap[Task[_], Timer]
def activeTasks: Set[Task[_]] = activeTasksMap.keySet.asScala.toSet
override def afterRegistered(
task: Task[_],
allDeps: Iterable[Task[_]],
pendingDeps: Iterable[Task[_]]
): Unit = {
// we need this to infer anonymous task names
pendingDeps foreach { t =>
if (TaskName.transformNode(t).isEmpty) {
anonOwners.put(t, task)
}
}
}
override def beforeWork(task: Task[_]): Unit = {
timings.put(task, new Timer)
activeTasksMap.put(task, ())
}
override def afterWork[A](task: Task[A], result: Either[Task[A], Result[A]]): Unit = {
timings.get(task).stop()
activeTasksMap.remove(task)
// we need this to infer anonymous task names
result.left.foreach { t =>
calledBy.put(t, task)
}
}
private[this] val taskNameCache = TrieMap.empty[Task[_], String]
protected def taskName(t: Task[_]): String =
taskNameCache.getOrElseUpdate(t, taskName0(t))
private[this] def taskName0(t: Task[_]): String = {
def definedName(node: Task[_]): Option[String] =
node.info.name orElse TaskName.transformNode(node).map(showScopedKey.show)
def inferredName(t: Task[_]): Option[String] = nameDelegate(t) map taskName
def nameDelegate(t: Task[_]): Option[Task[_]] =
Option(anonOwners.get(t)) orElse Option(calledBy.get(t))
definedName(t) orElse inferredName(t) getOrElse TaskName.anonymousName(t)
}
}
object AbstractTaskExecuteProgress {
private[sbt] class Timer() {
val startNanos: Long = System.nanoTime()
val threadId: Long = Thread.currentThread().getId
var endNanos: Long = 0L
def stop(): Unit = {
endNanos = System.nanoTime()
}
def durationNanos: Long = endNanos - startNanos
def startMicros: Long = (startNanos.toDouble / 1000).toLong
def durationMicros: Long = (durationNanos.toDouble / 1000).toLong
def currentElapsedSeconds: Long =
((System.nanoTime() - startNanos).toDouble / 1000000000L).toLong
}
}

View File

@ -9,23 +9,16 @@ package sbt
package internal
import sbt.internal.util.{ RMap, ConsoleOut, ConsoleAppender, LogOption, JLine }
import java.util.concurrent.ConcurrentHashMap
import scala.concurrent.{ blocking, Future, ExecutionContext }
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger }
import scala.collection.JavaConverters._
import scala.collection.concurrent.TrieMap
import TaskProgress._
/**
* implements task progress display on the shell.
*/
private[sbt] final class TaskProgress(currentRef: ProjectRef) extends ExecuteProgress[Task] {
private[this] val showScopedKey = Def.showRelativeKey2(currentRef)
// private[this] var start = 0L
private[this] val activeTasks = new ConcurrentHashMap[Task[_], Long]
private[this] val timings = new ConcurrentHashMap[Task[_], Long]
private[this] val calledBy = new ConcurrentHashMap[Task[_], Task[_]]
private[this] val anonOwners = new ConcurrentHashMap[Task[_], Task[_]]
private[sbt] final class TaskProgress
extends AbstractTaskExecuteProgress
with ExecuteProgress[Task] {
private[this] val isReady = new AtomicBoolean(false)
private[this] val lastTaskCount = new AtomicInteger(0)
private[this] val isAllCompleted = new AtomicBoolean(false)
@ -35,37 +28,10 @@ private[sbt] final class TaskProgress(currentRef: ProjectRef) extends ExecutePro
ConsoleAppender.setTerminalWidth(JLine.usingTerminal(_.getWidth))
}
override def afterRegistered(
task: Task[_],
allDeps: Iterable[Task[_]],
pendingDeps: Iterable[Task[_]]
): Unit = {
// we need this to infer anonymous task names
pendingDeps foreach { t =>
if (TaskName.transformNode(t).isEmpty) {
anonOwners.put(t, task)
}
}
}
override def afterReady(task: Task[_]): Unit = {
isReady.set(true)
}
override def beforeWork(task: Task[_]): Unit = {
activeTasks.put(task, System.nanoTime)
()
}
override def afterWork[A](task: Task[A], result: Either[Task[A], Result[A]]): Unit = {
val start = activeTasks.get(task)
timings.put(task, System.nanoTime - start)
activeTasks.remove(task)
// we need this to infer anonymous task names
result.left.foreach { t =>
calledBy.put(t, task)
}
}
override def afterCompleted[A](task: Task[A], result: Result[A]): Unit = ()
override def stop(): Unit = {
@ -95,15 +61,14 @@ private[sbt] final class TaskProgress(currentRef: ProjectRef) extends ExecutePro
private[this] val skipReportTasks =
Set("run", "bgRun", "fgRun", "scala", "console", "consoleProject")
private[this] def report(): Unit = console.lockObject.synchronized {
val currentTasks = activeTasks.asScala.toList
val currentTasks = activeTasks.toList
val ltc = lastTaskCount.get
val currentTasksCount = currentTasks.size
def report0(): Unit = {
console.print(s"$CursorDown1")
currentTasks foreach {
case (task, start) =>
val elapsed = (System.nanoTime - start) / 1000000000L
console.println(s"$DeleteLine | => ${taskName(task)} ${elapsed}s")
currentTasks foreach { task =>
val elapsed = timings.get(task).currentElapsedSeconds
console.println(s"$DeleteLine | => ${taskName(task)} ${elapsed}s")
}
if (ltc > currentTasksCount) deleteConsoleLines(ltc - currentTasksCount)
else ()
@ -138,9 +103,9 @@ private[sbt] final class TaskProgress(currentRef: ProjectRef) extends ExecutePro
// else report0()
// }
private[this] def containsSkipTasks(tasks: List[(Task[_], Long)]): Boolean =
private[this] def containsSkipTasks(tasks: List[Task[_]]): Boolean =
tasks
.map({ case (t, _) => taskName(t) })
.map(t => taskName(t))
.exists(n => skipReportTasks.exists(m => n.endsWith("/ " + m)))
private[this] def deleteConsoleLines(n: Int): Unit = {
@ -148,18 +113,6 @@ private[sbt] final class TaskProgress(currentRef: ProjectRef) extends ExecutePro
console.println(s"$DeleteLine")
}
}
private[this] val taskNameCache = TrieMap.empty[Task[_], String]
private[this] def taskName(t: Task[_]): String =
taskNameCache.getOrElseUpdate(t, taskName0(t))
private[this] def taskName0(t: Task[_]): String = {
def definedName(node: Task[_]): Option[String] =
node.info.name orElse TaskName.transformNode(node).map(showScopedKey.show)
def inferredName(t: Task[_]): Option[String] = nameDelegate(t) map taskName
def nameDelegate(t: Task[_]): Option[Task[_]] =
Option(anonOwners.get(t)) orElse Option(calledBy.get(t))
definedName(t) orElse inferredName(t) getOrElse TaskName.anonymousName(t)
}
}
private[sbt] object TaskProgress {

View File

@ -8,10 +8,7 @@
package sbt
package internal
import sbt.internal.util.RMap
import java.util.concurrent.ConcurrentHashMap
import TaskName._
import sbt.internal.util.{ RMap, ConsoleOut }
/**
* Measure the time elapsed for running tasks.
@ -22,10 +19,10 @@ import TaskName._
* - -Dsbt.task.timings.threshold=number
* @param reportOnShutdown Should the report be given when exiting the JVM (true) or immediately (false)?
*/
private[sbt] final class TaskTimings(reportOnShutdown: Boolean) extends ExecuteProgress[Task] {
private[this] val calledBy = new ConcurrentHashMap[Task[_], Task[_]]
private[this] val anonOwners = new ConcurrentHashMap[Task[_], Task[_]]
private[this] val timings = new ConcurrentHashMap[Task[_], Long]
private[sbt] final class TaskTimings(reportOnShutdown: Boolean)
extends AbstractTaskExecuteProgress
with ExecuteProgress[Task] {
import AbstractTaskExecuteProgress.Timer
private[this] var start = 0L
private[this] val threshold = java.lang.Long.getLong("sbt.task.timings.threshold", 0L)
private[this] val omitPaths = java.lang.Boolean.getBoolean("sbt.task.timings.omit.paths")
@ -50,23 +47,8 @@ private[sbt] final class TaskTimings(reportOnShutdown: Boolean) extends ExecuteP
if (!reportOnShutdown)
start = System.nanoTime
}
override def afterRegistered(
task: Task[_],
allDeps: Iterable[Task[_]],
pendingDeps: Iterable[Task[_]]
): Unit = {
pendingDeps foreach { t =>
if (transformNode(t).isEmpty) anonOwners.put(t, task)
}
}
override def afterReady(task: Task[_]): Unit = ()
override def beforeWork(task: Task[_]): Unit = { timings.put(task, System.nanoTime); () }
override def afterWork[T](task: Task[T], result: Either[Task[T], Result[T]]) = {
timings.put(task, System.nanoTime - timings.get(task))
result.left.foreach { t =>
calledBy.put(t, task)
}
}
override def afterCompleted[T](task: Task[T], result: Result[T]): Unit = ()
override def afterAllCompleted(results: RMap[Task, Result]): Unit =
if (!reportOnShutdown) {
@ -75,14 +57,15 @@ private[sbt] final class TaskTimings(reportOnShutdown: Boolean) extends ExecuteP
override def stop(): Unit = ()
private val reFilePath = raw"\{[^}]+\}".r
private[this] val reFilePath = raw"\{[^}]+\}".r
private[this] val console = ConsoleOut.systemOut
private[this] def report() = {
val total = divide(System.nanoTime - start)
println(s"Total time: $total $unit")
console.println(s"Total time: $total $unit")
import collection.JavaConverters._
def sumTimes(in: Seq[(Task[_], Long)]) = in.map(_._2).sum
val timingsByName = timings.asScala.toSeq.groupBy { case (t, _) => mappedName(t) } mapValues (sumTimes)
def sumTimes(in: Seq[(Task[_], Timer)]) = in.map(_._2.durationNanos).sum
val timingsByName = timings.asScala.toSeq.groupBy { case (t, _) => taskName(t) } mapValues (sumTimes)
val times = timingsByName.toSeq
.sortBy(_._2)
.reverse
@ -96,16 +79,12 @@ private[sbt] final class TaskTimings(reportOnShutdown: Boolean) extends ExecuteP
val maxTime = times.map { _._2 }.max.toString.length
times.foreach {
case (taskName, time) =>
println(s" ${taskName.padTo(maxTaskNameLength, ' ')}: ${""
console.println(s" ${taskName.padTo(maxTaskNameLength, ' ')}: ${""
.padTo(maxTime - time.toString.length, ' ')}$time $unit")
}
}
}
private[this] def inferredName(t: Task[_]): Option[String] = nameDelegate(t) map mappedName
private[this] def nameDelegate(t: Task[_]): Option[Task[_]] =
Option(anonOwners.get(t)) orElse Option(calledBy.get(t))
private[this] def mappedName(t: Task[_]): String =
definedName(t) orElse inferredName(t) getOrElse anonymousName(t)
private[this] def divide(time: Long) = (1L to divider.toLong).fold(time) { (a, b) =>
a / 10L
}

View File

@ -0,0 +1,81 @@
/*
* 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.io.File
import java.nio.file.Files
import sbt.internal.util.{ RMap, ConsoleOut }
import sbt.io.IO
import sbt.io.syntax._
import scala.collection.JavaConverters._
import sjsonnew.shaded.scalajson.ast.unsafe.JString
import sjsonnew.support.scalajson.unsafe.CompactPrinter
/**
* Measure the time elapsed for running tasks, and write the result out
* as Chrome Trace Event Format.
* This class is activated by adding -Dsbt.traces=true to the JVM options.
*/
private[sbt] final class TaskTraceEvent
extends AbstractTaskExecuteProgress
with ExecuteProgress[Task] {
import AbstractTaskExecuteProgress.Timer
private[this] var start = 0L
private[this] val console = ConsoleOut.systemOut
override def initial(): Unit = ()
override def afterReady(task: Task[_]): Unit = ()
override def afterCompleted[T](task: Task[T], result: Result[T]): Unit = ()
override def afterAllCompleted(results: RMap[Task, Result]): Unit = ()
override def stop(): Unit = ()
start = System.nanoTime
Runtime.getRuntime.addShutdownHook(new Thread {
override def run() = report()
})
private[this] def report() = {
if (timings.asScala.nonEmpty) {
writeTraceEvent()
}
}
private[this] def writeTraceEvent(): Unit = {
// import java.time.{ ZonedDateTime, ZoneOffset }
// import java.time.format.DateTimeFormatter
// val fileName = "build-" + ZonedDateTime
// .now(ZoneOffset.UTC)
// .format(DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HHmmss")) + ".trace"
val fileName = "build.trace"
val tracesDirectory = (new File("target").getAbsoluteFile) / "traces"
if (!tracesDirectory.exists) IO.createDirectory(tracesDirectory)
else ()
val outFile = tracesDirectory / fileName
val trace = Files.newBufferedWriter(outFile.toPath)
try {
trace.append("""{"traceEvents": [""")
def durationEvent(name: String, cat: String, t: Timer): String = {
val sb = new java.lang.StringBuilder(name.length + 2)
CompactPrinter.print(new JString(name), sb)
s"""{"name": ${sb.toString}, "cat": "$cat", "ph": "X", "ts": ${(t.startMicros)}, "dur": ${(t.durationMicros)}, "pid": 0, "tid": ${t.threadId}}"""
}
val entryIterator = timings.entrySet().iterator()
while (entryIterator.hasNext) {
val entry = entryIterator.next()
trace.append(durationEvent(taskName(entry.getKey), "task", entry.getValue))
if (entryIterator.hasNext) trace.append(",")
}
trace.append("]}")
()
} finally {
trace.close()
console.println(s"wrote $outFile")
}
}
}