Skip to content

Commit

Permalink
Clear previous problems as early as possible
Browse files Browse the repository at this point in the history
BSP requires that bloop clears problems that were present in previous
compiler iterations manually. Previously, Bloop would meet this contract
by clearing problems from previous compiler runs at the end of a
successful incremental cycle. For every incremental cycle, bloop would
go through the problems and clear those that were affected by changes
occurred during that cycle. You can see more information in this pull
request: #782

However, this approach was slower than desired and required that we
successfully finished an incremental compiler cycle before *starting* to
clean some diagnostics in the recompiled files.

Instead, this pull request takes a different approach to this problem.
We clear diagnostics as soon as we know they are safe to be cleared. For
example, if in a previous compiler run typer emitted an error, we have
bloop record that a diagnostic happened during that phase and, in
subsequent compiler runs, we clear the diagnostic as soon as we've
passed typer for the source file that contained the error.

That is, we clear diagnostics right after the phase that generated them
succeeded.

If a diagnostic could not be mapped to a phase, we fallback to the
previous behavior: we will clear it in `reportRemainingProblems` right
after every incremental cycle.
  • Loading branch information
jvican committed Jan 10, 2019
1 parent 302639c commit 2c3cc39
Show file tree
Hide file tree
Showing 10 changed files with 217 additions and 128 deletions.
43 changes: 27 additions & 16 deletions backend/src/main/scala/bloop/Compiler.scala
Original file line number Diff line number Diff line change
@@ -1,13 +1,13 @@
package bloop

import xsbti.compile._
import xsbti.{Problem, T2}
import xsbti.T2
import java.util.Optional
import java.io.File

import bloop.internal.Ecosystem
import bloop.io.AbsolutePath
import bloop.reporter.Reporter
import bloop.reporter.{ProblemPerPhase, Reporter}
import sbt.internal.inc.bloop.BloopZincCompiler
import sbt.internal.inc.{FreshCompilerCache, Locate}
import _root_.monix.eval.Task
Expand Down Expand Up @@ -53,17 +53,14 @@ object Compiler {
reporter: Reporter,
cancelPromise: Promise[Unit]
) extends CompileProgress {
private var lastPhase: String = ""
private var lastUnitPath: String = ""
override def startUnit(phase: String, unitPath: String): Unit = {
lastPhase = phase
lastUnitPath = unitPath
reporter.reportNextPhase(phase, new java.io.File(unitPath))
}

override def advance(current: Int, total: Int): Boolean = {
val isNotCancelled = !cancelPromise.isCompleted
if (isNotCancelled) {
reporter.reportCompilationProgress(current.toLong, total.toLong, lastPhase, lastUnitPath)
reporter.reportCompilationProgress(current.toLong, total.toLong)
}

isNotCancelled
Expand All @@ -84,14 +81,14 @@ object Compiler {
with CacheHashCode

final case class Failed(
problems: List[xsbti.Problem],
problems: List[ProblemPerPhase],
t: Option[Throwable],
elapsed: Long
) extends Result
with CacheHashCode

final case class Cancelled(
problems: List[xsbti.Problem],
problems: List[ProblemPerPhase],
elapsed: Long
) extends Result
with CacheHashCode
Expand Down Expand Up @@ -147,9 +144,10 @@ object Compiler {
val cacheFile = compileInputs.baseDirectory.resolve("cache").toFile
val incOptions = {
def withTransactional(opts: IncOptions): IncOptions = {
opts.withClassfileManagerType(Optional.of(
xsbti.compile.TransactionalManagerType.of(classesDirBak, reporter.logger)
))
opts.withClassfileManagerType(
Optional.of(
xsbti.compile.TransactionalManagerType.of(classesDirBak, reporter.logger)
))
}

val disableIncremental = java.lang.Boolean.getBoolean("bloop.zinc.disabled")
Expand Down Expand Up @@ -193,14 +191,18 @@ object Compiler {
}

val previousAnalysis = InterfaceUtil.toOption(compileInputs.previousResult.analysis())
val previousProblems: List[Problem] = compileInputs.previousCompilerResult match {
val previousProblems: List[ProblemPerPhase] = compileInputs.previousCompilerResult match {
case f: Compiler.Result.Failed => f.problems
case c: Compiler.Result.Cancelled => c.problems
case _: Compiler.Result.Success =>
// TODO(jvican): replace for problems already present in success reporter
import scala.collection.JavaConverters._
val infos =
previousAnalysis.toList.flatMap(_.readSourceInfos().getAllSourceInfos.asScala.values)
infos.flatMap(info => info.getReportedProblems.toList).toList
infos.iterator
.flatMap(info => info.getReportedProblems.toList)
.map(p => ProblemPerPhase(p, None))
.toList
case _ => Nil
}

Expand All @@ -217,12 +219,21 @@ object Compiler {
Result.Success(compileInputs.reporter, res, elapsed)
case Failure(_: xsbti.CompileCancelled) =>
reporter.reportEndCompilation(previousAnalysis, None, bsp.StatusCode.Cancelled)
Result.Cancelled(reporter.allProblems.toList, elapsed)
Result.Cancelled(reporter.allProblemsPerPhase.toList, elapsed)
case Failure(cause) =>
reporter.reportEndCompilation(previousAnalysis, None, bsp.StatusCode.Error)
cause match {
case f: StopPipelining => Result.Blocked(f.failedProjectNames)
case f: xsbti.CompileFailed => Result.Failed(f.problems().toList, None, elapsed)
case f: xsbti.CompileFailed =>
// We cannot assert reporter.problems == f.problems, so we aggregate them together
val reportedProblems = reporter.allProblemsPerPhase.toList
val rawProblemsFromReporter = reportedProblems.iterator.map(_.problem).toSet
val newProblems = f.problems().flatMap { p =>
if (rawProblemsFromReporter.contains(p)) Nil
else List(ProblemPerPhase(p, None))
}.toList
val failedProblems = reportedProblems ++ newProblems
Result.Failed(failedProblems, None, elapsed)
case t: Throwable =>
t.printStackTrace()
Result.Failed(Nil, Some(t), elapsed)
Expand Down
4 changes: 2 additions & 2 deletions backend/src/main/scala/bloop/reporter/Problem.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,12 @@ object Problem {
override def toString: String = s"$errors errors, $warnings warnings"
}

def count(problems: List[xsbti.Problem]): DiagnosticsCount = {
def count(problems: List[ProblemPerPhase]): DiagnosticsCount = {
// Compute the count manually because `count` returns an `Int`, not a `Long`
var errors = 0L
var warnings = 0L
problems.foreach { p =>
val severity = p.severity()
val severity = p.problem.severity()
if (severity == Severity.Error) errors += 1
if (severity == Severity.Warn) warnings += 1
}
Expand Down
9 changes: 9 additions & 0 deletions backend/src/main/scala/bloop/reporter/ProblemPerPhase.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,9 @@
package bloop.reporter

/**
* A problem that is mapped to the phase where it occurred.
*
* @param problem A problem reported by Zinc.
* @param phase An optional phase in case compiler reports a problem before a phase is registered.
*/
final case class ProblemPerPhase(problem: xsbti.Problem, phase: Option[String])
64 changes: 43 additions & 21 deletions backend/src/main/scala/bloop/reporter/Reporter.scala
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import xsbti.{Position, Severity}
import ch.epfl.scala.bsp
import xsbti.compile.CompileAnalysis

import scala.collection.concurrent.TrieMap
import scala.collection.mutable
import scala.util.Try

Expand All @@ -28,7 +29,7 @@ abstract class Reporter(
val cwd: AbsolutePath,
sourcePositionMapper: Position => Position,
val config: ReporterConfig,
val _problems: mutable.Buffer[Problem] = mutable.ArrayBuffer.empty
val _problems: mutable.Buffer[ProblemPerPhase] = mutable.ArrayBuffer.empty
) extends xsbti.Reporter
with ConfigurableReporter {

Expand All @@ -39,17 +40,33 @@ abstract class Reporter(
override def hasErrors(): Boolean = hasErrors(_problems)
override def hasWarnings(): Boolean = hasWarnings(_problems)

override def allProblems: Seq[Problem] = _problems
override def problems(): Array[xsbti.Problem] = _problems.toArray
override def problems(): Array[xsbti.Problem] = _problems.map(_.problem).toArray
override def allProblems: Seq[Problem] = _problems.map(p => liftProblem(p.problem)).toList

def allProblemsPerPhase: Seq[ProblemPerPhase] = _problems.toList

protected def logFull(problem: Problem): Unit

override def log(prob: xsbti.Problem): Unit = {
val mappedPos = sourcePositionMapper(prob.position)
val problemID = if (prob.position.sourceFile.isPresent) nextID() else -1
val problem =
Problem(problemID, prob.severity, prob.message, mappedPos, prob.category)
_problems += problem
protected def liftProblem(p: xsbti.Problem): Problem = {
p match {
case p: Problem => p
case _ =>
val mappedPos = sourcePositionMapper(p.position)
val problemID = if (p.position.sourceFile.isPresent) nextID() else -1
Problem(problemID, p.severity, p.message, mappedPos, p.category)
}
}

protected val phasesAtFile = TrieMap.empty[File, String]
protected val filesToPhaseStack = TrieMap.empty[File, List[String]]
override def log(xproblem: xsbti.Problem): Unit = {
val problem = liftProblem(xproblem)
val problemPerPhase = sbt.util.InterfaceUtil.toOption(problem.position.sourceFile()) match {
case Some(file) => ProblemPerPhase(problem, filesToPhaseStack.get(file).flatMap(_.headOption))
case None => ProblemPerPhase(problem, None)
}

_problems += problemPerPhase

// If we show errors in reverse order, they'll all be shown
// in `printSummary`.
Expand All @@ -58,27 +75,29 @@ abstract class Reporter(
}
}

/** Report when the compiler enters in a phase. */
def reportNextPhase(phase: String, sourceFile: File): Unit = {
// Update the phase that we have for every source file
val newPhaseStack = phase :: filesToPhaseStack.getOrElse(sourceFile, Nil)
filesToPhaseStack.update(sourceFile, newPhaseStack)
}

override def comment(pos: Position, msg: String): Unit = ()

private def hasErrors(problems: Seq[Problem]): Boolean =
problems.exists(_.severity == Severity.Error)
private def hasErrors(problems: Seq[ProblemPerPhase]): Boolean =
problems.exists(_.problem.severity == Severity.Error)

private def hasWarnings(problems: Seq[Problem]): Boolean =
problems.exists(_.severity == Severity.Warn)
private def hasWarnings(problems: Seq[ProblemPerPhase]): Boolean =
problems.exists(_.problem.severity == Severity.Warn)

/** Report the progress from the compiler. */
def reportCompilationProgress(
progress: Long,
total: Long,
phase: String,
sourceFile: String
): Unit
def reportCompilationProgress(progress: Long, total: Long): Unit

/** Report the compile cancellation of this project. */
def reportCancelledCompilation(): Unit

/** A function called *always* at the very beginning of compilation. */
def reportStartCompilation(previousProblems: List[xsbti.Problem]): Unit
def reportStartCompilation(previousProblems: List[ProblemPerPhase]): Unit

/**
* A function called at the very end of compilation, before returning from Zinc to bloop.
Expand All @@ -94,7 +113,10 @@ abstract class Reporter(
previousAnalysis: Option[CompileAnalysis],
analysis: Option[CompileAnalysis],
code: bsp.StatusCode
): Unit
): Unit = {
phasesAtFile.clear()
filesToPhaseStack.clear()
}

/**
* A function called before every incremental cycle with the compilation inputs.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ object ZincInternals {
*
* Example: "org.scala-sbt-compiler-bridge-1.0.0-bin_2.11.7__50.0".
*
*
* @param sources The moduleID representing the compiler bridge sources.
* @param scalaInstance The scala instance that sets the scala version for the id.
* @return The complete jar identifier for the bridge sources.
Expand Down
5 changes: 2 additions & 3 deletions frontend/src/main/scala/bloop/bsp/BloopBspServices.scala
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ import bloop.engine._
import bloop.internal.build.BuildInfo
import bloop.io.{AbsolutePath, RelativePath}
import bloop.logging.{BspServerLogger, DebugFilter}
import bloop.reporter.{BspProjectReporter, ReporterConfig}
import bloop.reporter.{BspProjectReporter, ProblemPerPhase, ReporterConfig}
import bloop.testing.{BspLoggingEventHandler, TestInternals}
import monix.eval.Task
import ch.epfl.scala.bsp.{
Expand All @@ -25,7 +25,6 @@ import ch.epfl.scala.bsp.{
}

import scala.meta.jsonrpc.{JsonRpcClient, Response => JsonRpcResponse, Services => JsonRpcServices}
import xsbti.Problem
import ch.epfl.scala.bsp
import ch.epfl.scala.bsp.ScalaBuildTarget.encodeScalaBuildTarget
import monix.execution.atomic.AtomicInt
Expand Down Expand Up @@ -204,7 +203,7 @@ final class BloopBspServices(
compileArgs: List[String]
): BspResult[bsp.CompileResult] = {
val cancelCompilation = Promise[Unit]()
def reportError(p: Project, problems: List[Problem], elapsedMs: Long): String = {
def reportError(p: Project, problems: List[ProblemPerPhase], elapsedMs: Long): String = {
// Don't show warnings in this "final report", we're handling them in the reporter
val count = bloop.reporter.Problem.count(problems)
s"${p.name} [${elapsedMs}ms] (errors ${count.errors})"
Expand Down
30 changes: 17 additions & 13 deletions frontend/src/main/scala/bloop/logging/BspServerLogger.scala
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import xsbti.Severity

import scala.meta.jsonrpc.JsonRpcClient
import ch.epfl.scala.bsp
import ch.epfl.scala.bsp.BuildTargetIdentifier
import ch.epfl.scala.bsp.endpoints.Build
import monix.execution.atomic.AtomicInt

Expand Down Expand Up @@ -118,31 +119,34 @@ final class BspServerLogger private (

private def now: Long = System.currentTimeMillis()

/**
* Publish a compile progress notification to the client via BSP.
*
* The following fields of the progress notification are not populated:
*
* 1. data: Option[Json] -- there is no additional metadata we want to share with the client.
*/
import io.circe.ObjectEncoder
private case class BloopProgress(
target: BuildTargetIdentifier
)

private implicit val bloopProgressEncoder: ObjectEncoder[BloopProgress] =
io.circe.derivation.deriveEncoder

/** Publish a compile progress notification to the client via BSP every 5% progress increments. */
def publishCompileProgress(
taskId: bsp.TaskId,
project: Project,
progress: Long,
total: Long,
phase: String,
sourceFile: String
percentage: Long
): Unit = {
val msg = s"Compiling ${sourceFile} (phase ${phase})"
val msg = s"Compiling ${project.name} (${percentage}%)"
val json = bloopProgressEncoder(BloopProgress(bsp.BuildTargetIdentifier(project.bspUri)))
Build.taskProgress.notify(
bsp.TaskProgressParams(
taskId,
Some(System.currentTimeMillis()),
Some(msg),
Some(total),
Some(progress),
Some("phase/file"),
Some("compile"),
None
None,
Some("bloop-progress"),
Some(json)
)
)
()
Expand Down
Loading

0 comments on commit 2c3cc39

Please sign in to comment.