From fbfe1c5a3396c242d6e65108bc451f5a1c75a0d7 Mon Sep 17 00:00:00 2001 From: Jorge Vicente Cantero Date: Thu, 15 Nov 2018 23:36:31 +0100 Subject: [PATCH] Rearchitect diagnostics reporting in bloop 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. --- backend/src/main/scala/bloop/Compiler.scala | 63 +++-- .../main/scala/bloop/reporter/Reporter.scala | 69 +++--- .../inc/bloop/BloopZincCompiler.scala | 15 +- .../internal/BloopHighLevelCompiler.scala | 43 ++-- .../inc/bloop/internal/BloopIncremental.scala | 20 +- .../inc/bloop/internal/BloopNameHashing.scala | 6 +- .../scala/bloop/bsp/BloopBspServices.scala | 3 +- .../bloop/engine/caches/ResultsCache.scala | 4 +- .../bloop/engine/tasks/CompilationTask.scala | 6 +- .../tasks/compilation/CompileResult.scala | 3 +- .../scala/bloop/logging/BspServerLogger.scala | 15 +- .../bloop/reporter/BspProjectReporter.scala | 66 ++++- .../scala/bloop/reporter/LogReporter.scala | 101 ++++++++ .../test/scala/bloop/bsp/BspClientTest.scala | 14 +- .../scala/bloop/bsp/BspProtocolSpec.scala | 228 +++++++++++++----- 15 files changed, 449 insertions(+), 207 deletions(-) create mode 100644 frontend/src/main/scala/bloop/reporter/LogReporter.scala diff --git a/backend/src/main/scala/bloop/Compiler.scala b/backend/src/main/scala/bloop/Compiler.scala index c887a533d4..022abf86c4 100644 --- a/backend/src/main/scala/bloop/Compiler.scala +++ b/backend/src/main/scala/bloop/Compiler.scala @@ -1,7 +1,7 @@ package bloop import xsbti.compile._ -import xsbti.T2 +import xsbti.{Problem, T2} import java.util.Optional import java.io.File @@ -9,7 +9,7 @@ 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 @@ -28,6 +28,7 @@ case class CompileInputs( compileOrder: CompileOrder, classpathOptions: ClasspathOptions, previousResult: PreviousResult, + previousCompilerResult: Compiler.Result, reporter: Reporter, mode: CompileMode, dependentResults: Map[File, PreviousResult] @@ -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( @@ -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) @@ -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 = { @@ -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 } } } diff --git a/backend/src/main/scala/bloop/reporter/Reporter.scala b/backend/src/main/scala/bloop/reporter/Reporter.scala index a7ea20ea26..6bdb9a2cfb 100644 --- a/backend/src/main/scala/bloop/reporter/Reporter.scala +++ b/backend/src/main/scala/bloop/reporter/Reporter.scala @@ -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 @@ -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. @@ -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 } diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/BloopZincCompiler.scala b/backend/src/main/scala/sbt/internal/inc/bloop/BloopZincCompiler.scala index deafe8d57a..465c8673d0 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/BloopZincCompiler.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/BloopZincCompiler.scala @@ -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 @@ -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 @@ -70,7 +69,7 @@ object BloopZincCompiler { extraOptions, irPromise, compileMode - )(logger) + )(reporter.logger) } def compileIncrementally( @@ -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) } } diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopHighLevelCompiler.scala b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopHighLevelCompiler.scala index c6fcf1739e..40507d8f3f 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopHighLevelCompiler.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopHighLevelCompiler.scala @@ -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._ @@ -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) @@ -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) = { @@ -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 } } @@ -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) } } diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopIncremental.scala b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopIncremental.scala index 4be52d1ed4..382c7a7d94 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopIncremental.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopIncremental.scala @@ -2,10 +2,9 @@ package sbt.internal.inc.bloop.internal import java.io.File -import java.net.URI -import java.util.Optional import java.util.concurrent.CompletableFuture +import bloop.reporter.Reporter import monix.eval.Task import sbt.internal.inc.{Analysis, InvalidationProfiler, Lookup, Stamper, Stamps, AnalysisCallback => AnalysisCallbackImpl} import sbt.util.Logger @@ -24,6 +23,7 @@ object BloopIncremental { previous0: CompileAnalysis, output: Output, log: Logger, + reporter: Reporter, options: IncOptions, irPromise: CompletableFuture[Array[IR]] ): Task[(Boolean, Analysis)] = { @@ -46,7 +46,7 @@ object BloopIncremental { val builder = new AnalysisCallbackImpl.Builder(internalBinaryToSourceClassName, internalSourceToClassNamesMap, externalAPI, current, output, options, irPromise) // We used to catch for `CompileCancelled`, but we prefer to propagate it so that Bloop catches it - compileIncremental(sources, lookup, previous, current, compile, builder, log, options) + compileIncremental(sources, lookup, previous, current, compile, builder, reporter, log, options) } def compileIncremental( @@ -56,6 +56,7 @@ object BloopIncremental { current: ReadStamps, compile: CompileFunction, callbackBuilder: AnalysisCallbackImpl.Builder, + reporter: Reporter, log: sbt.util.Logger, options: IncOptions, // TODO(jvican): Enable profiling of the invalidation algorithm down the road @@ -72,7 +73,7 @@ object BloopIncremental { } val setOfSources = sources.toSet - val incremental = new BloopNameHashing(log, options, profiler.profileRun) + val incremental = new BloopNameHashing(reporter, options, profiler.profileRun) val initialChanges = incremental.detectInitialChanges(setOfSources, previous, current, lookup) val binaryChanges = new DependencyChanges { val modifiedBinaries = initialChanges.binaryDeps.toArray @@ -101,16 +102,7 @@ object BloopIncremental { } yield callback.get } - /* Normal Zinc happens to add the source infos of the previous result to the infos - * of the new previous result. In constrast, we desire to only have the source infos - * of those files that we have indeed compiled so that we can know from the outside - * to which extent a new compilation overlaps with a previous compilation. This is - * important whenever we want to know which warnings were not reported in the new - * compilation but should be reported given that they are still present in the codebase. - */ - val previousWithNoSourceInfos = - previous.copy(infos = previous.infos -- previous.infos.allInfos.keys) - try incremental.entrypoint(initialInvClasses, initialInvSources, setOfSources, binaryChanges, lookup, previousWithNoSourceInfos, doCompile, classfileManager, 1) + try incremental.entrypoint(initialInvClasses, initialInvSources, setOfSources, binaryChanges, lookup, previous, doCompile, classfileManager, 1) catch { case e: Throwable => classfileManager.complete(false); throw e } } diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopNameHashing.scala b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopNameHashing.scala index 16f123d38a..ac08ac30ea 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopNameHashing.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopNameHashing.scala @@ -4,7 +4,7 @@ import java.io.File import monix.eval.Task import sbt.internal.inc._ -import sbt.util.Logger +import _root_.bloop.reporter.Reporter import xsbti.compile.{ClassFileManager, DependencyChanges, IncOptions} /** @@ -22,8 +22,8 @@ import xsbti.compile.{ClassFileManager, DependencyChanges, IncOptions} * @param options The incremental compiler options. * @param profiler The profiler used for the incremental invalidation algorithm. */ -private final class BloopNameHashing(log: Logger, options: IncOptions, profiler: RunProfiler) - extends IncrementalNameHashingCommon(log, options, profiler) { +private final class BloopNameHashing(reporter: Reporter, options: IncOptions, profiler: RunProfiler) + extends IncrementalNameHashingCommon(reporter.logger, options, profiler) { /** * Compile a project as many times as it is required incrementally. This logic is the start diff --git a/frontend/src/main/scala/bloop/bsp/BloopBspServices.scala b/frontend/src/main/scala/bloop/bsp/BloopBspServices.scala index 35c895bbd4..96689116dd 100644 --- a/frontend/src/main/scala/bloop/bsp/BloopBspServices.scala +++ b/frontend/src/main/scala/bloop/bsp/BloopBspServices.scala @@ -203,10 +203,11 @@ final class BloopBspServices( case (p, result) => result match { case Compiler.Result.Empty => Nil - case Compiler.Result.Cancelled(_) => Nil case Compiler.Result.Blocked(_) => Nil case Compiler.Result.Success(_, _, _) => Nil case Compiler.Result.GlobalError(problem) => List(problem) + case Compiler.Result.Cancelled(problems, elapsed) => Nil + List(reportError(p, problems, elapsed)) case Compiler.Result.Failed(problems, t, elapsed) => val acc = List(reportError(p, problems, elapsed)) t match { diff --git a/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala b/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala index bf0f9f5155..581a3a20de 100644 --- a/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala +++ b/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala @@ -9,7 +9,7 @@ import bloop.engine.tasks.compilation.FinalCompileResult import bloop.engine.{Build, ExecutionContext} import bloop.io.AbsolutePath import bloop.logging.{DebugFilter, Logger} -import bloop.reporter.Reporter +import bloop.reporter.LogReporter import monix.eval.Task import xsbti.compile.{CompileAnalysis, MiniSetup, PreviousResult} @@ -111,7 +111,7 @@ object ResultsCache { case Some(res) => logger.debug(s"Loading previous analysis for '${p.name}' from '$analysisFile'.") val r = PreviousResult.of(Optional.of(res.getAnalysis), Optional.of(res.getMiniSetup)) - val reporter = Reporter.fromAnalysis(res.getAnalysis, cwd, logger) + val reporter = LogReporter.fromAnalysis(res.getAnalysis, cwd, logger) Result.Success(reporter, r, 0L) case None => logger.debug(s"Analysis '$analysisFile' for '${p.name}' is empty.") diff --git a/frontend/src/main/scala/bloop/engine/tasks/CompilationTask.scala b/frontend/src/main/scala/bloop/engine/tasks/CompilationTask.scala index 8de3a2fa36..eef2d508f4 100644 --- a/frontend/src/main/scala/bloop/engine/tasks/CompilationTask.scala +++ b/frontend/src/main/scala/bloop/engine/tasks/CompilationTask.scala @@ -49,7 +49,8 @@ object CompilationTask { graphInputs.completeJava.complete(()) Task.now(earlyResult) case Right(CompileSourcesAndInstance(sources, instance, javaOnly)) => - val previous = state.results.lastSuccessfulResultOrEmpty(project) + val previousResult = state.results.latestResult(project) + val previousSuccesful = state.results.lastSuccessfulResultOrEmpty(project) val reporter = createCompilationReporter(project, cwd, reporterConfig, state.logger) val (scalacOptions, compileMode) = { @@ -92,7 +93,8 @@ object CompilationTask { project.javacOptions.toArray, project.compileOrder, project.classpathOptions, - previous, + previousSuccesful, + previousResult, reporter, compileMode, graphInputs.dependentResults diff --git a/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileResult.scala b/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileResult.scala index 5c1ff64587..6978984f92 100644 --- a/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileResult.scala +++ b/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileResult.scala @@ -77,7 +77,8 @@ object FinalCompileResult { val projectName = r.bundle.project.name r.result match { case Compiler.Result.Empty => s"${projectName} (empty)" - case Compiler.Result.Cancelled(ms) => s"${projectName} (cancelled, lasted ${ms}ms)" + case Compiler.Result.Cancelled(problems, ms) => + s"${projectName} (cancelled, failed with ${Problem.count(problems)}, ${ms}ms)" case Compiler.Result.Success(_, _, ms) => s"${projectName} (success ${ms}ms)" case Compiler.Result.Blocked(on) => s"${projectName} (blocked on ${on.mkString(", ")})" case Compiler.Result.GlobalError(problem) => diff --git a/frontend/src/main/scala/bloop/logging/BspServerLogger.scala b/frontend/src/main/scala/bloop/logging/BspServerLogger.scala index 249b9a8354..c95fa705d6 100644 --- a/frontend/src/main/scala/bloop/logging/BspServerLogger.scala +++ b/frontend/src/main/scala/bloop/logging/BspServerLogger.scala @@ -1,5 +1,6 @@ package bloop.logging +import java.io.File import java.util.concurrent.atomic.AtomicInteger import bloop.data.Project @@ -57,7 +58,7 @@ final class BspServerLogger private ( () } - def diagnostic(project: Project, problem: Problem): Unit = { + def diagnostic(project: Project, problem: Problem, clear: Boolean): Unit = { import sbt.util.InterfaceUtil.toOption val message = problem.message val problemPos = problem.position @@ -87,7 +88,7 @@ final class BspServerLogger private ( val textDocument = bsp.TextDocumentIdentifier(uri) val buildTargetId = bsp.BuildTargetIdentifier(project.bspUri) val diagnostics = - bsp.PublishDiagnosticsParams(textDocument, buildTargetId, None, List(diagnostic), true) + bsp.PublishDiagnosticsParams(textDocument, buildTargetId, None, List(diagnostic), clear) Build.publishDiagnostics.notify(diagnostics) case _ => problem.severity match { @@ -99,6 +100,16 @@ final class BspServerLogger private ( () } + def noDiagnostic(project: Project, file: File): Unit = { + val uri = bsp.Uri(file.toURI) + val textDocument = bsp.TextDocumentIdentifier(uri) + val buildTargetId = bsp.BuildTargetIdentifier(project.bspUri) + val diagnostics = + bsp.PublishDiagnosticsParams(textDocument, buildTargetId, None, Nil, true) + Build.publishDiagnostics.notify(diagnostics) + () + } + /** Return the next task id per bsp session. */ def nextTaskId: bsp.TaskId = { // TODO(jvican): Add parent information to the task id diff --git a/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala b/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala index 2c874ba70a..07676005a9 100644 --- a/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala +++ b/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala @@ -1,11 +1,15 @@ package bloop.reporter +import java.io.File +import java.util.concurrent.ConcurrentHashMap + import bloop.data.Project import bloop.io.AbsolutePath import bloop.logging.BspServerLogger import xsbti.Position - import ch.epfl.scala.bsp +import sbt.util.InterfaceUtil +import xsbti.compile.CompileAnalysis import scala.collection.mutable @@ -18,17 +22,69 @@ final class BspProjectReporter( override val _problems: mutable.Buffer[Problem] = mutable.ArrayBuffer.empty ) extends Reporter(logger, cwd, sourcePositionMapper, config, _problems) { private val taskId = logger.nextTaskId - override protected def logFull(problem: Problem): Unit = - logger.diagnostic(project, problem) + + /** A thread-safe map containing files with reported problems (from this cycle and + * previous ones, such as buffered warnings from previously compiled source files). */ + private val filesWithProblems = new ConcurrentHashMap[File, Boolean]() + + /** Log a problem in a thread-safe manner. */ + override protected def logFull(problem: Problem): Unit = { + sbt.util.InterfaceUtil.toOption(problem.position.sourceFile()) match { + case Some(file) => + // If it's the first diagnostic for this file, set clear to true + var sent: Boolean = false + filesWithProblems.computeIfAbsent(file, f => { + sent = true + logger.diagnostic(project, problem, true) + true + }) + + // If diagnostic wasn't sent in previous thunk, send it now as the clear already happened + if (sent) () else logger.diagnostic(project, problem, false) + case None => logger.diagnostic(project, problem, false) + } + } // Report summary manually via `reportEndCompilation` for BSP clients override def printSummary(): Unit = () - override def reportStartCompilation(): Unit = { + // Includes problems of both successful and failed compilations + private var previouslyReportedProblems: List[xsbti.Problem] = Nil + override def reportStartCompilation(previousProblems: List[xsbti.Problem]): Unit = { + previouslyReportedProblems = previousProblems logger.publishCompileStart(project, taskId) } - override def reportEndCompilation(code: bsp.StatusCode): Unit = { + override def reportStartIncrementalCycle(sources: Seq[File], outputDirs: Seq[File]): Unit = () + override def reportEndIncrementalCycle(): Unit = () + + override def reportEndCompilation( + previousAnalysis: Option[CompileAnalysis], + currentAnalysis: Option[CompileAnalysis], + code: bsp.StatusCode + ): Unit = { + val isNoOp = previousAnalysis match { + case Some(previous) => + currentAnalysis match { + case Some(current) => current == previous + case None => false + } + case None => false + } + + previouslyReportedProblems.foreach { problem => + InterfaceUtil.toOption(problem.position().sourceFile).foreach { source => + // Do nothing if problem maps to a file with problems, assume it's already reported + if (filesWithProblems.containsKey(source)) () + else { + // Log no diagnostic if there was an error in a file that now has no problems + if (problem.severity() == xsbti.Severity.Error) logger.noDiagnostic(project, source) + } + } + } + + // Clear the state of files with problems at the end of copmilation + filesWithProblems.clear() logger.publishCompileEnd(project, taskId, allProblems, code) } } diff --git a/frontend/src/main/scala/bloop/reporter/LogReporter.scala b/frontend/src/main/scala/bloop/reporter/LogReporter.scala new file mode 100644 index 0000000000..e0a3800b79 --- /dev/null +++ b/frontend/src/main/scala/bloop/reporter/LogReporter.scala @@ -0,0 +1,101 @@ +package bloop.reporter +import java.io.File + +import bloop.data.Project +import bloop.io.AbsolutePath +import bloop.logging.Logger +import xsbti.compile.CompileAnalysis +import xsbti.{Position, Severity} +import ch.epfl.scala.bsp +import sbt.internal.inc.Analysis + +import scala.collection.mutable + +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) { + + // Contains the files that are compiled in all incremental compiler cycles + private val compilingFiles = mutable.HashSet[File]() + + private final val format = config.format(this) + override def printSummary(): Unit = { + if (config.reverseOrder) { _problems.reverse.foreach(logFull) } + format.printSummary() + } + + /** + * Log the full error message for `problem`. + * + * @param problem The problem to log. + */ + 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) + } + } + + override def reportStartIncrementalCycle(sources: Seq[File], outputDirs: Seq[File]): Unit = { + // TODO(jvican): Fix https://github.com/scalacenter/bloop/issues/386 here + require(sources.size > 0) // This is an invariant enforced in the call-site + compilingFiles ++= sources + + val (javaSources, scalaSources) = sources.partition(_.getName.endsWith(".java")) + val scalaMsg = Analysis.counted("Scala source", "", "s", scalaSources.size) + val javaMsg = Analysis.counted("Java source", "", "s", javaSources.size) + val combined = scalaMsg ++ javaMsg + val targets = outputDirs.map(_.getAbsolutePath).mkString(",") + logger.info(combined.mkString("Compiling ", " and ", s" to $targets ...")) + } + + override def reportEndIncrementalCycle(): Unit = { + logger.info("Done compiling.") + } + + override def reportStartCompilation(previousProblems: List[xsbti.Problem]): Unit = () + override def reportEndCompilation( + previousAnalysis: Option[CompileAnalysis], + analysis: Option[CompileAnalysis], + code: bsp.StatusCode + ): Unit = { + def warningsFromPreviousRuns(previous: CompileAnalysis): List[xsbti.Problem] = { + import scala.collection.JavaConverters._ + val previousSourceInfos = previous.readSourceInfos().getAllSourceInfos.asScala.toMap + val eligibleSourceInfos = + previousSourceInfos.filterKeys(f => !compilingFiles.contains(f)).values + eligibleSourceInfos.flatMap { i => + i.getReportedProblems.filter(_.severity() == xsbti.Severity.Warn) + }.toList + } + + code match { + case bsp.StatusCode.Ok => + // Report warnings that occurred in previous compilation cycles only if + previousAnalysis.foreach { previous => + // Note that buffered warnings are not added back to the current analysis on purpose + warningsFromPreviousRuns(previous).foreach(p => log(p)) + } + case _ => () + } + } +} + +object LogReporter { + 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) + } +} diff --git a/frontend/src/test/scala/bloop/bsp/BspClientTest.scala b/frontend/src/test/scala/bloop/bsp/BspClientTest.scala index 43066f1447..b373886635 100644 --- a/frontend/src/test/scala/bloop/bsp/BspClientTest.scala +++ b/frontend/src/test/scala/bloop/bsp/BspClientTest.scala @@ -54,9 +54,9 @@ object BspClientTest { ExecutionModel.AlwaysAsyncExecution ) - def createServices(logger0: BspClientLogger[_]): Services = { + def createServices(addDiagnosticsHandler: Boolean, logger0: BspClientLogger[_]): Services = { val logger: bloop.logging.Logger = logger0 - Services + val rawServices = Services .empty(logger0) .notification(endpoints.Build.showMessage) { case bsp.ShowMessageParams(bsp.MessageType.Log, _, _, msg) => logger.debug(msg) @@ -70,7 +70,11 @@ object BspClientTest { case bsp.LogMessageParams(bsp.MessageType.Warning, _, _, msg) => logger.warn(msg) case bsp.LogMessageParams(bsp.MessageType.Error, _, _, msg) => logger.error(msg) } - .notification(endpoints.Build.publishDiagnostics) { + + // Lsp4s fails if we try to repeat a handler for a given notification + if (!addDiagnosticsHandler) rawServices + else { + rawServices.notification(endpoints.Build.publishDiagnostics) { case bsp.PublishDiagnosticsParams(uri, _, _, diagnostics, _) => // We prepend diagnostics so that tests can check they came from this notification def printDiagnostic(d: bsp.Diagnostic): String = s"[diagnostic] ${d.message} ${d.range}" @@ -84,6 +88,7 @@ object BspClientTest { } } } + } } type TestLogger = Slf4jAdapter[RecordingLogger] @@ -94,6 +99,7 @@ object BspClientTest { customServices: Services => Services = identity[Services], allowError: Boolean = false, reusePreviousState: Boolean = false, + addDiagnosticsHandler: Boolean = true )(runEndpoints: LanguageClient => me.Task[Either[Response.Error, T]]): Unit = { // Set an empty results cache and update the state globally val state = { @@ -116,7 +122,7 @@ object BspClientTest { implicit val lsClient = new LanguageClient(out, logger) val messages = BaseProtocolMessage.fromInputStream(in, logger) - val services = customServices(createServices(logger)) + val services = customServices(createServices(addDiagnosticsHandler, logger)) val lsServer = new LanguageServer(messages, lsClient, services, scheduler, logger) val runningClientServer = lsServer.startTask.runAsync(scheduler) diff --git a/frontend/src/test/scala/bloop/bsp/BspProtocolSpec.scala b/frontend/src/test/scala/bloop/bsp/BspProtocolSpec.scala index 623d8ef412..99c12af2d1 100644 --- a/frontend/src/test/scala/bloop/bsp/BspProtocolSpec.scala +++ b/frontend/src/test/scala/bloop/bsp/BspProtocolSpec.scala @@ -2,23 +2,22 @@ package bloop.bsp import java.nio.charset.StandardCharsets import java.nio.file.Files +import java.util.concurrent.ConcurrentHashMap import bloop.cli.validation.Validate import bloop.cli.{BspProtocol, CliOptions, Commands} import bloop.data.Project import bloop.engine.{BuildLoader, Run} -import bloop.io.{AbsolutePath, Paths} +import bloop.io.AbsolutePath import bloop.tasks.TestUtil -import bloop.logging.{BspClientLogger, DebugFilter, RecordingLogger} +import bloop.logging.{BspClientLogger, RecordingLogger} import org.junit.Test import ch.epfl.scala.bsp import ch.epfl.scala.bsp.{BuildTargetIdentifier, ScalaBuildTarget, endpoints} import junit.framework.Assert import monix.eval.Task -import scala.collection.mutable import scala.meta.jsonrpc.{LanguageClient, Response, Services} -import scala.util.control.NonFatal class BspProtocolSpec { private final val configDir = AbsolutePath(TestUtil.getBloopConfigDir("cross-test-build-0.6")) @@ -76,7 +75,7 @@ class BspProtocolSpec { def reportIfError(logger: BspClientLogger[RecordingLogger])(thunk: => Unit): Unit = { try thunk catch { - case NonFatal(t) => + case t: Throwable => val logs = logger.underlying.getMessages().map(t => s"${t._1}: ${t._2}") val errorMsg = s"BSP test failed with the following logs:\n${logs.mkString("\n")}" System.err.println(errorMsg) @@ -331,9 +330,12 @@ class BspProtocolSpec { projectBaseDir.resolve("src").resolve("test").resolve("scala") // The contents of the file (which is created during the test) contains a syntax error on purpose - private val TestIncrementalCompilationContents = + private val TestErrorIncrementalCompilationContents = "package example\n\nclass UserTest {\n List[String)](\"\")\n}" + private val TestSuccessIncrementalCompilationContents = + "package example\n\nclass UserTest {\n List[String](\"\")\n}" + // The contents of the file adding a new subclass to force a 2nd cycle of incremental compilation private val JUnitTestSubclassContents = "package hello\n\nclass JUnitTestSubclass extends JUnitTest" @@ -388,6 +390,7 @@ class BspProtocolSpec { var receivedReports: Int = 0 var receivedTestReports: Int = 0 val logger = new BspClientLogger(new RecordingLogger) + def exhaustiveTestCompile(target: bsp.BuildTarget)(implicit client: LanguageClient) = { def compileProject: Task[Either[Response.Error, bsp.CompileResult]] = endpoints.BuildTarget.compile.request(bsp.CompileParams(List(target.id), None, None)) @@ -404,12 +407,13 @@ class BspProtocolSpec { } else { // Harcode the source directory path to avoid extra BSP invocations val sourceDir = sourceDirectoryOf(AbsolutePath(ProjectUris.toPath(target.id.uri))) + Files.createDirectories(sourceDir.underlying) val testIncrementalCompilationFile = sourceDir.resolve("TestIncrementalCompilation.scala") // Write a new file in the directory so that incremental compilation picks it up Files.write( testIncrementalCompilationFile.underlying, - TestIncrementalCompilationContents.getBytes(StandardCharsets.UTF_8) + TestErrorIncrementalCompilationContents.getBytes(StandardCharsets.UTF_8) ) def deleteTestIncrementalCompilationFile(): Unit = { @@ -425,7 +429,7 @@ class BspProtocolSpec { Right(Assert.assertEquals(bsp.StatusCode.Error, incrementalReport.statusCode)) } - val nextSuccessfulCompile = failingIncrementalCompile + val noopCompilation = failingIncrementalCompile .doOnFinish(_ => Task(deleteTestIncrementalCompilationFile())) // Delete the file regardless of the result .flatMap { case Left(e) => Task.now(Left(e)) @@ -441,32 +445,92 @@ class BspProtocolSpec { } } + val failedCompile2 = noopCompilation.flatMap { + case Left(e) => Task.now(Left(e)) + case Right(_) => + // Write again the file with the error and get the same error! + Files.write( + testIncrementalCompilationFile.underlying, + TestErrorIncrementalCompilationContents.getBytes(StandardCharsets.UTF_8) + ) + + compileProject.map { + case Left(e) => Left(e) + case Right(result) => + if (receivedReports == 4) { + Right(Assert.assertEquals(bsp.StatusCode.Error, result.statusCode)) + } else { + Left(Response.internalError(s"Expected 4, got ${receivedReports} reports")) + } + } + } + + val nextSuccessfulCompile2 = failedCompile2.flatMap { + case Left(e) => Task.now(Left(e)) + case Right(_) => + // Write again the file with the error and get the same error! + Files.write( + testIncrementalCompilationFile.underlying, + TestSuccessIncrementalCompilationContents.getBytes(StandardCharsets.UTF_8) + ) + + compileProject.map { + case Left(e) => Left(e) + case Right(result) => + if (receivedReports == 5) { + Right(Assert.assertEquals(bsp.StatusCode.Ok, result.statusCode)) + } else { + Left(Response.internalError(s"Expected 5, got ${receivedReports} reports")) + } + } + } + + val noopCompilation2 = nextSuccessfulCompile2 + .doOnFinish(_ => Task(deleteTestIncrementalCompilationFile())) + .flatMap { + case Left(e) => Task.now(Left(e)) + case Right(_) => + deleteTestIncrementalCompilationFile() + + compileProject.map { + case Left(e) => Left(e) + case Right(result) => + if (receivedReports == 6) { + Right(Assert.assertEquals(bsp.StatusCode.Ok, result.statusCode)) + } else { + Left(Response.internalError(s"Expected 6, got ${receivedReports} reports")) + } + } + } + // Harcode the source directory path to avoid extra BSP invocations val testSourceDir = testSourceDirectoryOf(AbsolutePath(ProjectUris.toPath(target.id.uri))) val junitTestSubclassFile = testSourceDir.resolve("JUnitTestSubclass.scala") - // Write a new file in the directory so that incremental compilation picks it up - Files.write( - junitTestSubclassFile.underlying, - JUnitTestSubclassContents.getBytes(StandardCharsets.UTF_8) - ) def deleteJUnitTestSubclassFile(): Unit = { Files.deleteIfExists(junitTestSubclassFile.underlying) () } - val nextSuccessfulIncrementalCompile = nextSuccessfulCompile.flatMap { + val nextSuccessfulIncrementalCompile = noopCompilation2.flatMap { case Left(e) => Task.now(Left(e)) case Right(_) => + // Write a new file in the directory so that incremental compilation picks it up + Files.createDirectories(testSourceDir.underlying) + Files.write( + junitTestSubclassFile.underlying, + JUnitTestSubclassContents.getBytes(StandardCharsets.UTF_8) + ) + // The creation of new test should force an incremental compilation with 2 cycles compileProject.map { case Left(e) => Left(e) case Right(result) => // Make sure that we only get one report even if we do two cycles! - if (receivedReports == 4) Right(result) + if (receivedReports == 7) Right(result) else { - Left(Response.internalError(s"Expected 4, got ${receivedReports} reports")) + Left(Response.internalError(s"Expected 7, got ${receivedReports} reports")) } } } @@ -482,9 +546,9 @@ class BspProtocolSpec { case Left(e) => Left(e) case Right(result) => // After removing the new test class file, we compile everything correctly - if (receivedReports == 5) Right(result) + if (receivedReports == 8) Right(result) else { - Left(Response.internalError(s"Expected 5, got ${receivedReports} reports")) + Left(Response.internalError(s"Expected 8, got ${receivedReports} reports")) } } } @@ -507,6 +571,11 @@ class BspProtocolSpec { } val startedTask = scala.collection.mutable.HashSet[bsp.TaskId]() + type DiagnosticKey = (bsp.BuildTargetIdentifier, Int) // Int = Cycle starts at 0 + val stringifiedDiagnostics = new ConcurrentHashMap[DiagnosticKey, StringBuilder]() + def findDiagnosticsReport(btid: BuildTargetIdentifier, cycle: Int) = + stringifiedDiagnostics.get((btid, receivedReports)).mkString.stripLineEnd + val addServicesTest = { (s: Services) => s.notification(endpoints.Build.taskStart) { taskStart => taskStart.dataKind match { @@ -520,11 +589,28 @@ class BspProtocolSpec { bsp.CompileTask.decodeCompileTask(json.hcursor) match { case Left(failure) => Assert.fail(s"Decoding `$json` as a scala build target failed: $failure") - case Right(compileTask) => checkCompileStart(taskStart, compileTask) + case Right(compileTask) => + checkCompileStart(taskStart, compileTask) } case _ => Assert.fail(s"Got an unknown task start $taskStart") } } + .notification(endpoints.Build.publishDiagnostics) { + case p @ bsp.PublishDiagnosticsParams(tid, btid, _, diagnostics, reset) => + // Add the diagnostics to the stringified diagnostics map + val cycle = + if (tid.uri.value == testProject.bspUri) receivedTestReports else receivedReports + stringifiedDiagnostics.compute( + (btid, cycle), + (_: DiagnosticKey, builder0) => { + val builder = if (builder0 == null) new StringBuilder() else builder0 + builder + .++=(s"$tid $diagnostics (reset = $reset)") + .++=(System.lineSeparator()) + } + ) + () + } .notification(endpoints.Build.taskFinish) { taskFinish => taskFinish.dataKind match { case Some(bsp.TaskDataKind.CompileReport) => @@ -536,35 +622,62 @@ class BspProtocolSpec { val json = taskFinish.data.get bsp.CompileReport.decodeCompileReport(json.hcursor) match { case Right(report) => + if (isMainProject(report.target)) { if (receivedReports == 0) { - // This is the batch compilation which should have a warning and no errors + val received = findDiagnosticsReport(report.target, receivedReports) + val expected = + "TextDocumentIdentifier(Uri(file:/Users/jvican/Code/bloop/frontend/src/test/resources/cross-test-build-0.6/test-project/shared/src/main/scala/hello/App.scala)) List(Diagnostic(Range(Position(5,8),Position(5,8)),Some(Warning),None,None,local val in method main is never used,None)) (reset = true)" + Assert.assertEquals(expected, received) receivedReports += 1 - Assert.assertEquals(s"Warnings in $MainProject != 1", 1, report.warnings) - Assert.assertEquals(s"Errors in $MainProject != 0", 0, report.errors) - //Assert.assertTrue(s"Duration in $MainProject == 0", report.time != 0) } else if (receivedReports == 1) { - // This is the incremental compile which should have errors + // Check that we only receive one error + val received = findDiagnosticsReport(report.target, receivedReports) + // We don't expect a warning here because the client is supposed to reuse the previous diagnostic (we *didn't* send a notification for that file with reset=true) + val expected = + "TextDocumentIdentifier(Uri(file:/Users/jvican/Code/bloop/frontend/src/test/resources/cross-test-build-0.6/test-project/jvm/src/main/scala/TestIncrementalCompilation.scala)) List(Diagnostic(Range(Position(3,13),Position(3,13)),Some(Error),None,None,']' expected but ')' found.,None)) (reset = true)" + Assert.assertEquals(expected, received) receivedReports += 1 - Assert.assertTrue( - s"Expected errors in incremental cycle of $MainProject", - report.errors > 0 + } else if (receivedReports == 2) { + // The following operation is a no-op, so we should get an empty diagnostic with reset enabled to delete the previous error + val received = findDiagnosticsReport(report.target, receivedReports) + val expected = + "TextDocumentIdentifier(Uri(file:/Users/jvican/Code/bloop/frontend/src/test/resources/cross-test-build-0.6/test-project/jvm/src/main/scala/TestIncrementalCompilation.scala)) List() (reset = true)" + Assert.assertEquals(expected, received) + receivedReports += 1 + } else if (receivedReports == 3) { + // Check that we receive the error that we received early in inteartion 1 + val received = findDiagnosticsReport(report.target, receivedReports) + // We don't expect a warning here because the client is supposed to reuse the previous diagnostic (we *didn't* send a notification for that file with reset=true) + val expected = + "TextDocumentIdentifier(Uri(file:/Users/jvican/Code/bloop/frontend/src/test/resources/cross-test-build-0.6/test-project/jvm/src/main/scala/TestIncrementalCompilation.scala)) List(Diagnostic(Range(Position(3,13),Position(3,13)),Some(Error),None,None,']' expected but ')' found.,None)) (reset = true)" + Assert.assertEquals(expected, received) + receivedReports += 1 + } else if (receivedReports == 4) { + // This compilation is a successful operation after an incomplete failed compilation, so we should get an empty diagnostic with reset enabled to delete the previous error + val received = findDiagnosticsReport(report.target, receivedReports) + val expected = + "TextDocumentIdentifier(Uri(file:/Users/jvican/Code/bloop/frontend/src/test/resources/cross-test-build-0.6/test-project/jvm/src/main/scala/TestIncrementalCompilation.scala)) List() (reset = true)" + Assert.assertEquals(expected, received) + receivedReports += 1 + } else if (receivedReports == 5) { + // This compilation is a no-op that should give no diagnostic as the previous iteration already emptied + Assert.assertNull( + stringifiedDiagnostics.get((report.target, receivedReports)) ) - } else if (receivedReports >= 2 && receivedReports < 5) { - // This is the last compilation which should be successful receivedReports += 1 - Assert.assertEquals(s"Warnings in $MainProject != 1", 1, report.warnings) - Assert.assertEquals(s"Errors in $MainProject != 0", 0, report.errors) - () + } else if (receivedReports > 5 && receivedReports < 8) { + receivedReports += 1 } else { Assert.fail(s"Unexpected compilation report in $MainProject: $report") } } else if (isTestProject(report.target)) { - if (receivedTestReports >= 0 && receivedTestReports < 4) { - // All compilations of the test project must compile correctly + if (receivedTestReports >= 0 && receivedTestReports < 6) { + // Compilation is correct and there is no diagnostic sent whatsoever + Assert.assertNull( + stringifiedDiagnostics.get((report.target, receivedTestReports)) + ) receivedTestReports += 1 - Assert.assertEquals(s"Warnings in $MainProject != 0", 0, report.warnings) - Assert.assertEquals(s"Errors in $MainProject != 0", 0, report.errors) } else { Assert.fail( s"Unexpected compilation ${receivedTestReports + 1}th report in $TestProject: $report") @@ -580,47 +693,32 @@ class BspProtocolSpec { } } + BspClientTest.runTest( + bspCmd, + configDir, + logger, + addServicesTest, + addDiagnosticsHandler = false + )(c => clientWork(c)) + reportIfError(logger) { - BspClientTest.runTest(bspCmd, configDir, logger, addServicesTest)(c => clientWork(c)) - // Make sure that the compilation is logged back to the client via logs in stdout - val msgs = logger.underlying.getMessagesAt(Some("info")) - Assert.assertTrue( - "End of compilation is not reported.", - msgs.filter(_.startsWith("Done compiling.")).nonEmpty - ) - // Project, 1st , 2nd , 3rd , 4th , 5th - // main , FULL , INC FAILED , NO-OP , NONE , NONE - // test , FULL , NONE , NO-OP , INC , INC + // P , 1st , 2nd , 3rd , 4th , 5th , 6th , 7th , 8th + // main , FULL , INC FAILED , NOOP , INC FAILED , INC , NOOP , NOOP , NOOP + // test , FULL , NONE , NOOP , NONE , NOOP , NOOP , INC , NOOP Assert.assertEquals( s"Mismatch in total number of compilations for $MainProject", - 5, + 8, receivedReports ) - // One full compile, failed incremental compile, one no-op + // Two times more than main because main fails to compile twice Assert.assertEquals( s"Mismatch in total number of compilations for $TestProject", - 4, + 6, receivedTestReports ) - - // Both the start line and the start column have to be indexed by 0 - val expectedWarning = - "[diagnostic] local val in method main is never used Range(Position(5,8),Position(5,8))" - val warnings = logger.underlying.getMessagesAt(Some("warn")) - Assert.assertTrue( - s"Expected two times $expectedWarning, obtained $warnings.", - warnings == List(expectedWarning, expectedWarning) - ) - - // The syntax error has to be present as a diagnostic, not a normal log error - val errors = logger.underlying.getMessagesAt(Some("error")) - Assert.assertTrue( - "Syntax error is not reported as diagnostic.", - errors.exists(_.startsWith("[diagnostic] ']' expected but ')' found.")) - ) } }