Skip to content

Commit

Permalink
Rearchitect diagnostics reporting in bloop
Browse files Browse the repository at this point in the history
Motivation: supporting BSP v2 diagnostics reporting.

The following commit introduces:

1. Clearer separation of diagnostics reporting for CLI and for BSP.
   (Both have opposing interests when it comes to reporting diagnostics)
2. Add in-depth tests that check the correct behaviour of BSP reporting
   for clients, including reporting whenever errors are cleared or
   whenever there is a no-op incremental cycle.
3. Find a better way of getting previous compilation problems that
   doesn't depend on the source infos of the Zinc analysis.
  • Loading branch information
jvican committed Nov 16, 2018
1 parent 2ba5537 commit fbfe1c5
Show file tree
Hide file tree
Showing 15 changed files with 449 additions and 207 deletions.
63 changes: 28 additions & 35 deletions backend/src/main/scala/bloop/Compiler.scala
Original file line number Diff line number Diff line change
@@ -1,15 +1,15 @@
package bloop

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

import bloop.internal.Ecosystem
import bloop.io.AbsolutePath
import bloop.reporter.Reporter
import sbt.internal.inc.bloop.BloopZincCompiler
import sbt.internal.inc.{FreshCompilerCache, JavaInterfaceUtil, Locate}
import sbt.internal.inc.{FreshCompilerCache, Locate}
import _root_.monix.eval.Task
import bloop.util.CacheHashCode
import sbt.internal.inc.bloop.internal.StopPipelining
Expand All @@ -28,6 +28,7 @@ case class CompileInputs(
compileOrder: CompileOrder,
classpathOptions: ClasspathOptions,
previousResult: PreviousResult,
previousCompilerResult: Compiler.Result,
reporter: Reporter,
mode: CompileMode,
dependentResults: Map[File, PreviousResult]
Expand All @@ -49,7 +50,6 @@ object Compiler {
object Result {
final case object Empty extends Result with CacheHashCode
final case class Blocked(on: List[String]) extends Result with CacheHashCode
final case class Cancelled(elapsed: Long) extends Result with CacheHashCode
final case class GlobalError(problem: String) extends Result with CacheHashCode

final case class Success(
Expand All @@ -66,6 +66,12 @@ object Compiler {
) extends Result
with CacheHashCode

final case class Cancelled(
problems: List[xsbti.Problem],
elapsed: Long
) extends Result
with CacheHashCode

object Ok {
def unapply(result: Result): Option[Result] = result match {
case s @ (Success(_, _, _) | Empty) => Some(s)
Expand All @@ -75,26 +81,12 @@ object Compiler {

object NotOk {
def unapply(result: Result): Option[Result] = result match {
case f @ (Failed(_, _, _) | Cancelled(_) | Blocked(_)) => Some(f)
case f @ (Failed(_, _, _) | Cancelled(_, _) | Blocked(_)) => Some(f)
case _ => None
}
}
}

def warningsFromPreviousRuns(
previous: CompileAnalysis,
current: CompileAnalysis
): List[xsbti.Problem] = {
import scala.collection.JavaConverters._
val previousSourceInfos = previous.readSourceInfos().getAllSourceInfos.asScala.toMap
val currentSourceInfos = current.readSourceInfos().getAllSourceInfos.asScala.toMap
val eligibleSourceInfos =
previousSourceInfos.filterKeys(f => !currentSourceInfos.contains(f)).values
eligibleSourceInfos.flatMap { i =>
i.getReportedProblems.filter(_.severity() == xsbti.Severity.Warn)
}.toList
}

def compile(compileInputs: CompileInputs): Task[Result] = {
val classesDir = compileInputs.classesDir.toFile
def getInputs(compilers: Compilers): Inputs = {
Expand Down Expand Up @@ -157,40 +149,41 @@ object Compiler {
import ch.epfl.scala.bsp
import scala.util.{Success, Failure}
val reporter = compileInputs.reporter
reporter.reportStartCompilation()

val previousAnalysis = InterfaceUtil.toOption(compileInputs.previousResult.analysis())
val previousProblems: List[Problem] = compileInputs.previousCompilerResult match {
case f: Compiler.Result.Failed => f.problems
case c: Compiler.Result.Cancelled => c.problems
case _: Compiler.Result.Success =>
import scala.collection.JavaConverters._
val infos =
previousAnalysis.toList.flatMap(_.readSourceInfos().getAllSourceInfos.asScala.values)
infos.flatMap(info => info.getReportedProblems.toList).toList
case _ => Nil
}

reporter.reportStartCompilation(previousProblems)
BloopZincCompiler
.compile(inputs, compileInputs.mode, reporter.logger)
.compile(inputs, compileInputs.mode, reporter)
.materialize
.map {
case Success(result) =>
// Report warnings that occurred in previous compilation cycles
previousAnalysis.foreach { previous =>
warningsFromPreviousRuns(previous, result.analysis()).foreach { p =>
// Note that buffered warnings are not added back to the current analysis on purpose
compileInputs.reporter.log(p)
}
}

// Report end of compilation only after we have reported all warnings from previous runs
reporter.reportEndCompilation(bsp.StatusCode.Ok)
reporter.reportEndCompilation(previousAnalysis, Some(result.analysis), bsp.StatusCode.Ok)
val res = PreviousResult.of(Optional.of(result.analysis()), Optional.of(result.setup()))
Result.Success(compileInputs.reporter, res, elapsed)
case Failure(_: xsbti.CompileCancelled) =>
reporter.reportEndCompilation(bsp.StatusCode.Cancelled)
Result.Cancelled(elapsed)
reporter.reportEndCompilation(previousAnalysis, None, bsp.StatusCode.Cancelled)
Result.Cancelled(reporter.allProblems.toList, elapsed)
case Failure(cause) =>
val result = cause match {
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 t: Throwable =>
t.printStackTrace()
Result.Failed(Nil, Some(t), elapsed)
}

reporter.reportEndCompilation(bsp.StatusCode.Error)
result
}
}
}
69 changes: 30 additions & 39 deletions backend/src/main/scala/bloop/reporter/Reporter.scala
Original file line number Diff line number Diff line change
@@ -1,11 +1,12 @@
package bloop.reporter

import java.io.File

import bloop.io.AbsolutePath
import bloop.logging.Logger
import xsbti.compile.CompileAnalysis
import xsbti.{Position, Severity}

import ch.epfl.scala.bsp
import xsbti.compile.CompileAnalysis

import scala.collection.mutable

Expand All @@ -14,6 +15,8 @@ import scala.collection.mutable
* This configuration indicated whether to use colors, how to format messages,
* etc.
*
* A reporter has internal state and must be instantiated per compilation.
*
* @param logger The logger that will receive the output of the reporter.
* @param cwd The current working directory of the user who started compilation.
* @param sourcePositionMapper A function that transforms positions.
Expand Down Expand Up @@ -62,47 +65,35 @@ abstract class Reporter(
private def hasWarnings(problems: Seq[Problem]): Boolean =
problems.exists(_.severity == Severity.Warn)

def reportStartCompilation(): Unit
def reportEndCompilation(code: bsp.StatusCode): Unit
}

final class LogReporter(
override val logger: Logger,
override val cwd: AbsolutePath,
sourcePositionMapper: Position => Position,
override val config: ReporterConfig,
override val _problems: mutable.Buffer[Problem] = mutable.ArrayBuffer.empty
) extends Reporter(logger, cwd, sourcePositionMapper, config, _problems) {

private final val format = config.format(this)
override def printSummary(): Unit = {
if (config.reverseOrder) { _problems.reverse.foreach(logFull) }
format.printSummary()
}
/** A function called *always* at the very beginning of compilation. */
def reportStartCompilation(previousProblems: List[xsbti.Problem]): Unit

/**
* Log the full error message for `problem`.
* A function called at the very end of compilation, before returning from Zinc to bloop.
*
* This method **is** called if the compilation is a no-op.
*
* @param problem The problem to log.
* @param previousAnalysis An instance of a previous compiler analysis, if any.
* @param analysis An instance of a new compiler analysis, if no error happened.
* @param code The status code for a given compilation.
*/
override protected def logFull(problem: Problem): Unit = {
val text = format.formatProblem(problem)
problem.severity match {
case Severity.Error => logger.error(text)
case Severity.Warn => logger.warn(text)
case Severity.Info => logger.info(text)
}
}
def reportEndCompilation(
previousAnalysis: Option[CompileAnalysis],
analysis: Option[CompileAnalysis],
code: bsp.StatusCode
): Unit

override def reportStartCompilation(): Unit = ()
override def reportEndCompilation(code: bsp.StatusCode): Unit = ()
}
/**
* A function called before every incremental cycle with the compilation inputs.
*
* This method is not called if the compilation is a no-op (e.g. same analysis as before).
*/
def reportStartIncrementalCycle(sources: Seq[File], outputDirs: Seq[File]): Unit

object Reporter {
def fromAnalysis(analysis: CompileAnalysis, cwd: AbsolutePath, logger: Logger): Reporter = {
import scala.collection.JavaConverters._
val sourceInfos = analysis.readSourceInfos.getAllSourceInfos.asScala.toBuffer
val ps = sourceInfos.flatMap(_._2.getReportedProblems).map(Problem.fromZincProblem(_))
new LogReporter(logger, cwd, identity, ReporterConfig.defaultFormat, ps)
}
/**
* A function called after every incremental cycle, even if any compilation errors happen.
*
* This method is not called if the compilation is a no-op (e.g. same analysis as before).
*/
def reportEndIncrementalCycle(): Unit
}
Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,13 @@
package sbt.internal.inc.bloop

import java.io.File
import java.net.URI
import java.util.Optional
import java.util.concurrent.CompletableFuture

import bloop.CompileMode
import bloop.reporter.Reporter
import monix.eval.Task
import sbt.internal.inc.{Analysis, CompileConfiguration, CompileOutput, Incremental, LookupImpl, MiniSetupUtil, MixedAnalyzingCompiler}
import xsbti.{AnalysisCallback, Logger, Reporter}
import xsbti.{AnalysisCallback, Logger}
import sbt.internal.inc.JavaInterfaceUtil.{EnrichOptional, EnrichSbtTuple}
import sbt.internal.inc.bloop.internal.{BloopHighLevelCompiler, BloopIncremental}
import sbt.util.InterfaceUtil
Expand Down Expand Up @@ -38,12 +37,12 @@ object BloopZincCompiler {
def compile(
in: Inputs,
compileMode: CompileMode,
logger: Logger
reporter: Reporter
): Task[CompileResult] = {
val config = in.options()
val setup = in.setup()
import config._
import setup._
import setup.{reporter => _, _}
val compilers = in.compilers
val javacChosen = compilers.javaTools.javac
val scalac = compilers.scalac
Expand All @@ -70,7 +69,7 @@ object BloopZincCompiler {
extraOptions,
irPromise,
compileMode
)(logger)
)(reporter.logger)
}

def compileIncrementally(
Expand Down Expand Up @@ -108,13 +107,13 @@ object BloopZincCompiler {
if (skip) Task.now(CompileResult.of(prev, config.currentSetup, false))
else {
val setOfSources = sources.toSet
val compiler = BloopHighLevelCompiler(config, logger)
val compiler = BloopHighLevelCompiler(config, reporter)
val lookup = new LookupImpl(config, previousSetup)
val analysis = invalidateAnalysisFromSetup(config.currentSetup, previousSetup, incrementalOptions.ignoredScalacOptions(), setOfSources, prev)

// Scala needs the explicit type signature to infer the function type arguments
val compile: (Set[File], DependencyChanges, AnalysisCallback, ClassFileManager) => Task[Unit] = compiler.compile(_, _, _, _, compileMode)
BloopIncremental.compile(setOfSources, lookup, compile, analysis, output, logger, config.incOptions, irPromise).map {
BloopIncremental.compile(setOfSources, lookup, compile, analysis, output, logger, reporter, config.incOptions, irPromise).map {
case (changed, analysis) => CompileResult.of(analysis, config.currentSetup, changed)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,12 +6,13 @@ import java.net.URI
import java.util.Optional
import java.util.concurrent.CompletableFuture

import bloop.reporter.Reporter
import bloop.{CompileMode, CompilerOracle, JavaSignal, SimpleIRStore}
import monix.eval.Task
import sbt.internal.inc.JavaInterfaceUtil.EnrichOption
import sbt.internal.inc.javac.AnalyzingJavaCompiler
import sbt.internal.inc.{Analysis, AnalyzingCompiler, CompileConfiguration, CompilerArguments, MixedAnalyzingCompiler, ScalaInstance, Stamper, Stamps}
import sbt.util.{InterfaceUtil, Logger}
import sbt.util.Logger
import xsbti.{AnalysisCallback, CompileFailed}
import xsbti.compile._

Expand All @@ -28,14 +29,15 @@ import scala.util.control.NonFatal
* @param scalac The Scala compiler (this one takes the concrete implementation, not an interface).
* @param javac The concrete Java compiler.
* @param config The compilation configuration.
* @param logger The logger.
* @param reporter The reporter to be used to compile.
*/
final class BloopHighLevelCompiler(
scalac: AnalyzingCompiler,
javac: AnalyzingJavaCompiler,
config: CompileConfiguration,
logger: Logger
reporter: Reporter
) {
import reporter.logger
private[this] final val setup = config.currentSetup
private[this] final val classpath = config.classpath.map(_.getAbsoluteFile)

Expand Down Expand Up @@ -84,7 +86,10 @@ final class BloopHighLevelCompiler(

val includedSources = config.sources.filter(sourcesToCompile)
val (javaSources, scalaSources) = includedSources.partition(_.getName.endsWith(".java"))
logInputs(logger, javaSources.size, scalaSources.size, outputDirs)
val existsCompilation = javaSources.size + scalaSources.size > 0
if (existsCompilation) {
reporter.reportStartIncrementalCycle(includedSources, outputDirs)
}

// Note `pickleURI` has already been used to create the analysis callback in `BloopZincCompiler`
val (pipeline: Boolean, batches: Option[Int], completeJava: CompletableFuture[Unit], fireJavaCompilation: Task[JavaSignal], transitiveJavaSources: List[File], separateJavaAndScala: Boolean) = {
Expand Down Expand Up @@ -250,27 +255,13 @@ final class BloopHighLevelCompiler(
}
}

combinedTasks.map { _ =>
// TODO(jvican): Fix https://github.com/scalacenter/bloop/issues/386 here
if (javaSources.size + scalaSources.size > 0)
logger.info("Done compiling.")
}
}
combinedTasks.materialize.map { r =>
if (existsCompilation) {
reporter.reportEndIncrementalCycle()
}

// TODO(jvican): Fix https://github.com/scalacenter/bloop/issues/386 here
private[this] def logInputs(
log: Logger,
javaCount: Int,
scalaCount: Int,
outputDirs: Seq[File]
): Unit = {
val scalaMsg = Analysis.counted("Scala source", "", "s", scalaCount)
val javaMsg = Analysis.counted("Java source", "", "s", javaCount)
val combined = scalaMsg ++ javaMsg
if (combined.nonEmpty) {
val targets = outputDirs.map(_.getAbsolutePath).mkString(",")
log.info(combined.mkString("Compiling ", " and ", s" to $targets ..."))
}
r
}.dematerialize
}
}

Expand All @@ -295,10 +286,10 @@ object BloopHighLevelCompiler {
new CallbackBuilder(_ => None, _ => Set.empty, (_, _) => None, stamps, output, options, promise).build()
}

def apply(config: CompileConfiguration, log: Logger): BloopHighLevelCompiler = {
def apply(config: CompileConfiguration, reporter: Reporter): BloopHighLevelCompiler = {
val (searchClasspath, entry) = MixedAnalyzingCompiler.searchClasspathAndLookup(config)
val scalaCompiler = config.compiler.asInstanceOf[AnalyzingCompiler]
val javaCompiler = new AnalyzingJavaCompiler(config.javac, config.classpath, config.compiler.scalaInstance, config.classpathOptions, entry, searchClasspath)
new BloopHighLevelCompiler(scalaCompiler, javaCompiler, config, log)
new BloopHighLevelCompiler(scalaCompiler, javaCompiler, config, reporter)
}
}
Loading

0 comments on commit fbfe1c5

Please sign in to comment.