From dfd8e35473364e3d4caf4c910f7cd73754de5b2c Mon Sep 17 00:00:00 2001 From: Tomasz Godzik Date: Thu, 30 Dec 2021 19:54:19 +0100 Subject: [PATCH] Update Zinc to 1.6.0 Biggest changes required here involved the new VirtualFile interface as we needed to convert them to java.io.File in a number of places. I tried to keep the conversions to a minimum, so that we don't bump into unexpected things such as issues with `jrt` filesystem and tried using Path wherever possible. I am not 100% if everything is correct, but I double checked and everything seems in place. We also needed to adjust ScalaInstance to make it work with the new interface and add an additional setting for output in the Java compilers, since otherwise everything gets produced to source directories (this mirrors the change in zinc itself.) --- .../bloop/BloopClasspathEntryLookup.scala | 33 ++-- backend/src/main/scala/bloop/Compiler.scala | 45 +++-- .../src/main/scala/bloop/CompilerCache.scala | 35 +++- .../src/main/scala/bloop/ScalaInstance.scala | 30 ++- .../scala/bloop/UniqueCompileInputs.scala | 3 +- .../main/scala/bloop/io/ClasspathHasher.scala | 44 ++--- .../bloop/reporter/ObservedReporter.scala | 6 +- .../main/scala/bloop/reporter/Reporter.scala | 5 +- .../scala/bloop/reporter/ReporterAction.scala | 3 +- .../main/scala/bloop/util/AnalysisUtils.scala | 5 +- .../internal/inc/BloopComponentCompiler.scala | 34 ++-- .../inc/bloop/BloopZincCompiler.scala | 30 ++- .../internal/inc/bloop/ZincInternals.scala | 11 +- .../internal/BloopAnalysisCallback.scala | 171 ++++++++++++------ .../internal/BloopHighLevelCompiler.scala | 45 +++-- .../inc/bloop/internal/BloopIncremental.scala | 35 +++- .../inc/bloop/internal/BloopLookup.scala | 4 +- .../inc/bloop/internal/BloopNameHashing.scala | 70 ++++--- .../inc/bloop/internal/BloopStamps.scala | 19 +- .../internal/ConcurrentAnalysisCallback.scala | 169 ++++++++++++----- .../bloop/engine/caches/ResultsCache.scala | 5 +- .../main/scala/bloop/engine/tasks/Tasks.scala | 15 +- .../tasks/compilation/CompileBundle.scala | 5 +- .../main/scala/bloop/io/SourceHasher.scala | 3 +- .../bloop/reporter/BspProjectReporter.scala | 11 +- .../scala/bloop/reporter/LogReporter.scala | 12 +- .../test/scala/bloop/BaseCompileSpec.scala | 28 +-- .../scala/bloop/io/SourcesGlobsSpec.scala | 8 +- .../maven/MavenConfigGenerationSuite.scala | 3 +- project/Dependencies.scala | 2 +- 30 files changed, 612 insertions(+), 277 deletions(-) diff --git a/backend/src/main/scala/bloop/BloopClasspathEntryLookup.scala b/backend/src/main/scala/bloop/BloopClasspathEntryLookup.scala index 8b37ab2aac..4c8433954c 100644 --- a/backend/src/main/scala/bloop/BloopClasspathEntryLookup.scala +++ b/backend/src/main/scala/bloop/BloopClasspathEntryLookup.scala @@ -16,30 +16,38 @@ import java.util.concurrent.ConcurrentHashMap import xsbti.compile.FileHash import sbt.internal.inc.bloop.internal.BloopNameHashing import sbt.internal.inc.bloop.internal.BloopStamps +import sbt.internal.inc.PlainVirtualFile +import xsbti.VirtualFile +import bloop.util.AnalysisUtils +import sbt.internal.inc.classpath.ClasspathUtil +import xsbti.FileConverter final class BloopClasspathEntryLookup( results: Map[File, PreviousResult], - classpathHashes: Vector[FileHash] + classpathHashes: Vector[FileHash], + converter: FileConverter ) extends PerClasspathEntryLookup { - override def analysis(classpathEntry: File): ju.Optional[CompileAnalysis] = { - InterfaceUtil.toOptional(results.get(classpathEntry)).flatMap(_.analysis()) + override def analysis(classpathEntry: VirtualFile): ju.Optional[CompileAnalysis] = { + val file = converter.toPath(classpathEntry).toFile() + InterfaceUtil.toOptional(results.get(file)).flatMap(_.analysis()) } - override def definesClass(entry: File): DefinesClass = { - if (!entry.exists) FalseDefinesClass + override def definesClass(entry: VirtualFile): DefinesClass = { + val file = converter.toPath(entry).toFile() + if (!file.exists) FalseDefinesClass else { - classpathHashes.find(fh => fh.file() == entry) match { + classpathHashes.find(fh => fh.file() == file) match { case None => FalseDefinesClass case Some(entryHash) => def computeDefinesClassForJar = { - if (!ClasspathUtilities.isArchive(entry, contentFallback = true)) FalseDefinesClass - else new JarDefinesClass(entry) + if (!ClasspathUtil.isArchive(file.toPath(), contentFallback = true)) FalseDefinesClass + else new JarDefinesClass(file) } - if (BloopStamps.isDirectoryHash(entryHash)) new DirectoryDefinesClass(entry) + if (BloopStamps.isDirectoryHash(entryHash)) new DirectoryDefinesClass(file) else { val (_, cachedDefinesClass) = BloopClasspathEntryLookup.definedClasses.compute( - entry, + file, (entry, definesClass) => { definesClass match { case null => @@ -119,11 +127,12 @@ object BloopClasspathEntryLookup { ): Option[File] = { def findClassFile(t: (File, PreviousResult)): Option[File] = { val (classesDir, result) = t - val targetClassFile = new File(classesDir, relativeClassFile) + val targetFile = new File(classesDir.toPath().toFile(), relativeClassFile) + val targetClassFile = PlainVirtualFile(targetFile.toPath()) InterfaceUtil.toOption(result.analysis()).flatMap { analysis0 => val analysis = analysis0.asInstanceOf[sbt.internal.inc.Analysis] val definedClass = analysis.relations.allProducts.contains(targetClassFile) - if (definedClass) Some(targetClassFile) else None + if (definedClass) Some(targetFile) else None } } diff --git a/backend/src/main/scala/bloop/Compiler.scala b/backend/src/main/scala/bloop/Compiler.scala index 559d2f0159..45575d06fd 100644 --- a/backend/src/main/scala/bloop/Compiler.scala +++ b/backend/src/main/scala/bloop/Compiler.scala @@ -34,6 +34,10 @@ import sbt.internal.inc.bloop.internal.BloopStamps import sbt.internal.inc.bloop.internal.BloopLookup import bloop.reporter.Reporter import bloop.logging.CompilationEvent +import sbt.internal.inc.PlainVirtualFile +import xsbti.VirtualFile +import xsbti.VirtualFileRef +import sbt.internal.inc.PlainVirtualFileConverter case class CompileInputs( scalaInstance: ScalaInstance, @@ -160,6 +164,7 @@ object CompileOutPaths { object Compiler { private implicit val filter = bloop.logging.DebugFilter.Compilation + private val converter = PlainVirtualFileConverter.converter private final class BloopProgress( reporter: ZincReporter, cancelPromise: Promise[Unit] @@ -168,7 +173,12 @@ object Compiler { reporter.reportNextPhase(phase, new java.io.File(unitPath)) } - override def advance(current: Int, total: Int): Boolean = { + override def advance( + current: Int, + total: Int, + prevPhase: String, + nextPhase: String + ): Boolean = { val isNotCancelled = !cancelPromise.isCompleted if (isNotCancelled) { reporter.reportCompilationProgress(current.toLong, total.toLong) @@ -276,8 +286,10 @@ object Compiler { } def getCompilationOptions(inputs: CompileInputs): CompileOptions = { - val sources = inputs.sources // Sources are all files - val classpath = inputs.classpath.map(_.toFile) + // Sources are all files + val sources = inputs.sources.map(path => PlainVirtualFile(path.underlying): VirtualFile) + + val classpath = inputs.classpath.map(path => PlainVirtualFile(path.underlying): VirtualFile) val optionsWithoutFatalWarnings = inputs.scalacOptions.flatMap { option => if (option != "-Xfatal-warnings") List(option) else { @@ -292,8 +304,8 @@ object Compiler { CompileOptions .create() - .withClassesDirectory(newClassesDir.toFile) - .withSources(sources.map(_.toFile)) + .withClassesDirectory(newClassesDir) + .withSources(sources) .withClasspath(classpath) .withScalacOptions(optionsWithoutFatalWarnings) .withJavacOptions(inputs.javacOptions) @@ -308,7 +320,11 @@ object Compiler { newClassesDir.toFile -> compileInputs.previousResult ) - val lookup = new BloopClasspathEntryLookup(results, compileInputs.uniqueInputs.classpath) + val lookup = new BloopClasspathEntryLookup( + results, + compileInputs.uniqueInputs.classpath, + converter + ) val reporter = compileInputs.reporter val compilerCache = new FreshCompilerCache val cacheFile = compileInputs.baseDirectory.resolve("cache").toFile @@ -516,7 +532,7 @@ object Compiler { val resultForFutureCompilationRuns = { resultForDependentCompilationsInSameRun.withAnalysis( - Optional.of(analysisForFutureCompilationRuns) + Optional.of(analysisForFutureCompilationRuns): Optional[CompileAnalysis] ) } @@ -718,12 +734,14 @@ object Compiler { ): Analysis = { // Cast to the only internal analysis that we support val analysis = analysis0.asInstanceOf[Analysis] - def rebase(file: File): File = { - val filePath = file.toPath.toAbsolutePath + def rebase(file: VirtualFileRef): VirtualFileRef = { + + val filePath = PlainVirtualFileConverter.converter.toPath(file).toAbsolutePath() if (!filePath.startsWith(readOnlyClassesDir)) file else { // Hash for class file is the same because the copy duplicates metadata - newClassesDir.resolve(readOnlyClassesDir.relativize(filePath)).toFile + val path = newClassesDir.resolve(readOnlyClassesDir.relativize(filePath)) + PlainVirtualFile(path) } } @@ -732,11 +750,12 @@ object Compiler { val oldStamps = analysis.stamps // Use empty stamps for files that have fatal warnings so that next compile recompiles them val rebasedSources = oldStamps.sources.map { - case t @ (file, _) => + case t @ (virtualFile, _) => + val file = converter.toPath(virtualFile).toFile() // Assumes file in reported diagnostic matches path in here val fileHasFatalWarnings = sourceFilesWithFatalWarnings.contains(file) if (!fileHasFatalWarnings) t - else file -> BloopStamps.emptyStampFor(file) + else virtualFile -> BloopStamps.emptyStampFor(file) } val rebasedProducts = oldStamps.products.map { case t @ (file, _) => @@ -744,7 +763,7 @@ object Compiler { if (rebased == file) t else rebased -> t._2 } // Changes the paths associated with the class file paths - Stamps(rebasedProducts, rebasedSources, oldStamps.binaries) + Stamps(rebasedProducts, rebasedSources, oldStamps.libraries) } val newRelations = { diff --git a/backend/src/main/scala/bloop/CompilerCache.scala b/backend/src/main/scala/bloop/CompilerCache.scala index d7bbae4277..3949f9de8a 100644 --- a/backend/src/main/scala/bloop/CompilerCache.scala +++ b/backend/src/main/scala/bloop/CompilerCache.scala @@ -38,6 +38,11 @@ import javax.tools.JavaFileObject.Kind import javax.tools.{JavaCompiler => JavaxCompiler} import scala.collection.mutable.HashSet import scala.concurrent.ExecutionContext +import xsbti.VirtualFile +import bloop.util.AnalysisUtils +import xsbti.compile.{IncToolOptions, Output} +import sbt.internal.inc.CompilerArguments +import sbt.internal.inc.PlainVirtualFileConverter final class CompilerCache( componentProvider: ComponentProvider, @@ -145,9 +150,12 @@ final class CompilerCache( final class BloopForkedJavaCompiler(javaHome: Option[File]) extends JavaCompiler { import xsbti.compile.IncToolOptions + private val converter = PlainVirtualFileConverter.converter + def run( - sources: Array[File], + sources: Array[VirtualFile], options: Array[String], + output: Output, topts: IncToolOptions, reporter: XReporter, log: XLogger @@ -195,7 +203,14 @@ final class CompilerCache( try { import sbt.internal.inc.javac.BloopForkedJavaUtils - BloopForkedJavaUtils.launch(javaHome, "javac", sources, options, log, reporter) + BloopForkedJavaUtils.launch( + javaHome, + "javac", + sources.map(converter.toPath(_).toFile()), + options, + log, + reporter + ) } finally { Paths.delete(newInvalidatedEntry) } @@ -215,9 +230,11 @@ final class CompilerCache( import java.io.File import xsbti.compile.IncToolOptions import xsbti.Reporter + private val converter = PlainVirtualFileConverter.converter override def run( - sources: Array[File], + sources: Array[VirtualFile], options: Array[String], + output: Output, incToolOptions: IncToolOptions, reporter: Reporter, log0: xsbti.Logger @@ -241,8 +258,8 @@ final class CompilerCache( import sbt.internal.inc.javac.WriteReportingFileManager val zincFileManager = incToolOptions.classFileManager().get() val fileManager = new BloopInvalidatingFileManager(fileManager0, zincFileManager) - - val jfiles = fileManager0.getJavaFileObjectsFromFiles(sources.toList.asJava) + val sourceFiles: Array[File] = sources.map(converter.toPath(_).toFile()) + val jfiles = fileManager0.getJavaFileObjectsFromFiles(sourceFiles.toList.asJava) try { // Create directories of java args that trigger error if they don't exist def processJavaDirArgument(idx: Int): Unit = { @@ -264,7 +281,13 @@ final class CompilerCache( processJavaDirArgument(cleanedOptions.indexOf("-s")) processJavaDirArgument(cleanedOptions.indexOf("-h")) - val newJavacOptions = cleanedOptions.toList.asJava + output.getSingleOutputAsPath match { + case p if p.isPresent => java.nio.file.Files.createDirectories(p.get) + case _ => + } + + val outputOption = CompilerArguments.outputOption(output) + val newJavacOptions = (cleanedOptions.toList ++ outputOption).asJava log.debug(s"Invoking javac with ${newJavacOptions.asScala.mkString(" ")}") val success = compiler .getTask(logWriter, fileManager, diagnostics, newJavacOptions, null, jfiles) diff --git a/backend/src/main/scala/bloop/ScalaInstance.scala b/backend/src/main/scala/bloop/ScalaInstance.scala index 5d7f1775c6..aeba80e643 100644 --- a/backend/src/main/scala/bloop/ScalaInstance.scala +++ b/backend/src/main/scala/bloop/ScalaInstance.scala @@ -21,6 +21,16 @@ final class ScalaInstance private ( override val allJars: Array[File] ) extends xsbti.compile.ScalaInstance { + override lazy val loaderCompilerOnly: ClassLoader = + new URLClassLoader(compilerJars().map(_.toURI.toURL), ScalaInstance.topClassLoader) + + override def compilerJars(): Array[File] = { + val all = allJars + .filter(f => isJar(f.getName) && isCompilerJar(f)) + if (all.isEmpty) sys.error(s"Missing compiler jars in Scala jars ${allJars.mkString(", ")}") + all + } + override def libraryJars(): Array[File] = { allJars .filter(f => isJar(f.getName) && hasScalaLibraryName(f.getName)) @@ -58,7 +68,25 @@ final class ScalaInstance private ( filename.startsWith(ScalacCompilerName) || (isDotty && (filename.startsWith("dotty-compiler") || filename.startsWith("scala3-compiler"))) private def hasScalaLibraryName(filename: String): Boolean = - filename.startsWith("scala-library") + filename.startsWith("scala-library") || filename.startsWith("scala3-library") + + private def hasScalaReflectName(filename: String): Boolean = + filename.startsWith("scala-reflect") + + private def hasScalaXmlName(filename: String): Boolean = + filename.startsWith("scala-xml") + + private def hasScala3AdditionalLibraryName(filename: String): Boolean = + isDotty && + (filename.startsWith("scala3-interfaces") || filename.startsWith("tasty-core") || + filename.startsWith("scala-asm")) + + private def isCompilerJar(file: File) = { + val name = file.getName() + hasScalaReflectName(name) || hasScalaCompilerName(name) || + hasScalaLibraryName(name) || hasScalaXmlName(name) || + hasScala3AdditionalLibraryName(name) + } /** Tells us what the real version of the classloaded scalac compiler in this instance is. */ override def actualVersion(): String = { diff --git a/backend/src/main/scala/bloop/UniqueCompileInputs.scala b/backend/src/main/scala/bloop/UniqueCompileInputs.scala index e860d40954..9190d80c97 100644 --- a/backend/src/main/scala/bloop/UniqueCompileInputs.scala +++ b/backend/src/main/scala/bloop/UniqueCompileInputs.scala @@ -3,6 +3,7 @@ package bloop import bloop.io.AbsolutePath import bloop.util.CacheHashCode import xsbti.compile.FileHash +import xsbti.VirtualFileRef case class UniqueCompileInputs( sources: Vector[UniqueCompileInputs.HashedSource], @@ -36,7 +37,7 @@ case class UniqueCompileInputs( } object UniqueCompileInputs { - case class HashedSource(source: AbsolutePath, hash: Int) + case class HashedSource(source: VirtualFileRef, hash: Int) def emptyFor(originPath: String): UniqueCompileInputs = { UniqueCompileInputs(Vector.empty, Vector.empty, Vector.empty, Vector.empty, originPath) diff --git a/backend/src/main/scala/bloop/io/ClasspathHasher.scala b/backend/src/main/scala/bloop/io/ClasspathHasher.scala index 12dec74d9c..9e37d65555 100644 --- a/backend/src/main/scala/bloop/io/ClasspathHasher.scala +++ b/backend/src/main/scala/bloop/io/ClasspathHasher.scala @@ -27,13 +27,14 @@ import sbt.internal.inc.bloop.internal.BloopStamps import sbt.io.IO import java.util.concurrent.TimeUnit import java.io.PrintStream +import xsbti.VirtualFile object ClasspathHasher { // For more safety, store both the time and size private type JarMetadata = (FileTime, Long) - private[this] val hashingPromises = new ConcurrentHashMap[File, Promise[FileHash]]() - private[this] val cacheMetadataJar = new ConcurrentHashMap[File, (JarMetadata, FileHash)]() + private[this] val hashingPromises = new ConcurrentHashMap[Path, Promise[FileHash]]() + private[this] val cacheMetadataJar = new ConcurrentHashMap[Path, (JarMetadata, FileHash)]() /** * Hash the classpath in parallel with Monix's task. @@ -73,35 +74,35 @@ object ClasspathHasher { val timeoutSeconds: Long = 20L // We'll add the file hashes to the indices here and return it at the end val classpathHashes = new Array[FileHash](classpath.length) - case class AcquiredTask(file: File, idx: Int, p: Promise[FileHash]) + case class AcquiredTask(file: Path, idx: Int, p: Promise[FileHash]) val isCancelled = AtomicBoolean(false) val parallelConsumer = { Consumer.foreachParallelAsync[AcquiredTask](parallelUnits) { - case AcquiredTask(file, idx, p) => + case AcquiredTask(path, idx, p) => // Use task.now because Monix's load balancer already forces an async boundary val hashingTask = Task.now { val hash = try { if (cancelCompilation.isCompleted) { - BloopStamps.cancelledHash(file) + BloopStamps.cancelledHash(path) } else if (isCancelled.get) { cancelCompilation.trySuccess(()) - BloopStamps.cancelledHash(file) + BloopStamps.cancelledHash(path) } else { - val filePath = file.toPath - val attrs = Files.readAttributes(filePath, classOf[BasicFileAttributes]) - if (attrs.isDirectory) BloopStamps.directoryHash(file) + val attrs = Files.readAttributes(path, classOf[BasicFileAttributes]) + if (attrs.isDirectory) BloopStamps.directoryHash(path) else { val currentMetadata = - (FileTime.fromMillis(IO.getModifiedTimeOrZero(file)), attrs.size()) - Option(cacheMetadataJar.get(file)) match { + (FileTime.fromMillis(IO.getModifiedTimeOrZero(path.toFile)), attrs.size()) + Option(cacheMetadataJar.get(path)) match { case Some((metadata, hashHit)) if metadata == currentMetadata => hashHit case _ => - tracer.traceVerbose(s"computing hash ${filePath.toAbsolutePath.toString}") { + tracer.traceVerbose(s"computing hash ${path.toAbsolutePath.toString}") { _ => - val newHash = FileHash.of(file, ByteHasher.hashFileContents(file)) - cacheMetadataJar.put(file, (currentMetadata, newHash)) + val newHash = + FileHash.of(path, ByteHasher.hashFileContents(path.toFile)) + cacheMetadataJar.put(path, (currentMetadata, newHash)) newHash } } @@ -109,10 +110,10 @@ object ClasspathHasher { } } catch { // Can happen when a file doesn't exist, for example - case monix.execution.misc.NonFatal(t) => BloopStamps.emptyHash(file) + case monix.execution.misc.NonFatal(t) => BloopStamps.emptyHash(path) } classpathHashes(idx) = hash - hashingPromises.remove(file, p) + hashingPromises.remove(path, p) p.trySuccess(hash) () } @@ -131,12 +132,12 @@ object ClasspathHasher { TimeUnit.SECONDS, new Runnable { def run(): Unit = { - val hash = BloopStamps.cancelledHash(file) + val hash = BloopStamps.cancelledHash(path) // Complete if hashing for this entry hasn't finished in 15s, otherwise ignore - hashingPromises.remove(file, p) + hashingPromises.remove(path, p) if (p.trySuccess(hash)) { val msg = - s"Hashing ${file} is taking more than ${timeoutSeconds}s, detaching downstream clients to unblock them..." + s"Hashing ${path} is taking more than ${timeoutSeconds}s, detaching downstream clients to unblock them..." try { logger.warn(msg) serverOut.println(msg) @@ -157,7 +158,7 @@ object ClasspathHasher { val acquiredByOtherTasks = new mutable.ListBuffer[Task[Unit]]() val acquiredByThisHashingProcess = new mutable.ListBuffer[AcquiredTask]() - def acquireHashingEntry(entry: File, entryIdx: Int): Unit = { + def acquireHashingEntry(entry: Path, entryIdx: Int): Unit = { if (isCancelled.get) () else { val entryPromise = Promise[FileHash]() @@ -191,8 +192,7 @@ object ClasspathHasher { val initEntries = Task { classpath.zipWithIndex.foreach { case t @ (absoluteEntry, idx) => - val entry = absoluteEntry.toFile - acquireHashingEntry(entry, idx) + acquireHashingEntry(absoluteEntry.underlying, idx) } }.doOnCancel(Task { isCancelled.compareAndSet(false, true); () }) diff --git a/backend/src/main/scala/bloop/reporter/ObservedReporter.scala b/backend/src/main/scala/bloop/reporter/ObservedReporter.scala index 28dad4acf2..3237f3ec89 100644 --- a/backend/src/main/scala/bloop/reporter/ObservedReporter.scala +++ b/backend/src/main/scala/bloop/reporter/ObservedReporter.scala @@ -8,6 +8,7 @@ import ch.epfl.scala.bsp import bloop.logging.{ObservedLogger, Logger} import scala.concurrent.Promise import bloop.logging.CompilationEvent +import xsbti.VirtualFile final class ObservedReporter( val observedLogger: ObservedLogger[Logger], @@ -73,7 +74,10 @@ final class ObservedReporter( registerAction(ReporterAction.ProcessEndCompilation(code)) } - override def reportStartIncrementalCycle(sources: Seq[File], outputDirs: Seq[File]): Unit = { + override def reportStartIncrementalCycle( + sources: Seq[VirtualFile], + outputDirs: Seq[File] + ): Unit = { underlying.reportStartIncrementalCycle(sources, outputDirs) registerAction(ReporterAction.ReportStartIncrementalCycle(sources, outputDirs)) } diff --git a/backend/src/main/scala/bloop/reporter/Reporter.scala b/backend/src/main/scala/bloop/reporter/Reporter.scala index ad633842e9..1100aab355 100644 --- a/backend/src/main/scala/bloop/reporter/Reporter.scala +++ b/backend/src/main/scala/bloop/reporter/Reporter.scala @@ -15,6 +15,8 @@ import scala.util.Try import bloop.logging.CompilationEvent import scala.concurrent.Promise import monix.execution.atomic.AtomicInt +import xsbti.VirtualFile +import sbt.internal.inc.PlainVirtualFileConverter /** * A flexible reporter whose configuration is provided by a `ReporterConfig`. @@ -36,6 +38,7 @@ abstract class Reporter( override val config: ReporterConfig, val _problems: Reporter.Buffer[ProblemPerPhase] ) extends ZincReporter { + protected val converter = PlainVirtualFileConverter.converter private case class PositionId(sourcePath: String, offset: Int) private val _severities = TrieMap.empty[PositionId, Severity] private val _messages = TrieMap.empty[PositionId, List[String]] @@ -270,7 +273,7 @@ trait ZincReporter extends xsbti.Reporter with ConfigurableReporter { * 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 + def reportStartIncrementalCycle(sources: Seq[VirtualFile], outputDirs: Seq[File]): Unit /** Report when the compiler enters in a phase. */ def reportNextPhase(phase: String, sourceFile: File): Unit diff --git a/backend/src/main/scala/bloop/reporter/ReporterAction.scala b/backend/src/main/scala/bloop/reporter/ReporterAction.scala index c2ede91ef4..ae537364f2 100644 --- a/backend/src/main/scala/bloop/reporter/ReporterAction.scala +++ b/backend/src/main/scala/bloop/reporter/ReporterAction.scala @@ -3,6 +3,7 @@ package bloop.reporter import java.io.File import scala.util.Try import ch.epfl.scala.bsp +import xsbti.VirtualFile sealed trait ReporterAction object ReporterAction { @@ -10,7 +11,7 @@ object ReporterAction { final case object ReportStartCompilation extends ReporterAction final case class ReportStartIncrementalCycle( - sources: Seq[File], + sources: Seq[VirtualFile], outputDirs: Seq[File] ) extends ReporterAction diff --git a/backend/src/main/scala/bloop/util/AnalysisUtils.scala b/backend/src/main/scala/bloop/util/AnalysisUtils.scala index 5d66e1e02d..467b661ffe 100644 --- a/backend/src/main/scala/bloop/util/AnalysisUtils.scala +++ b/backend/src/main/scala/bloop/util/AnalysisUtils.scala @@ -5,10 +5,13 @@ import java.io.File import bloop.reporter.ProblemPerPhase import xsbti.compile.CompileAnalysis import xsbti.compile.analysis.SourceInfo +import xsbti.VirtualFileRef +import sbt.internal.inc.PlainVirtualFileConverter +import java.nio.file.Path object AnalysisUtils { import scala.collection.JavaConverters._ - def sourceInfosFrom(previousAnalysis: CompileAnalysis): Map[File, SourceInfo] = { + def sourceInfosFrom(previousAnalysis: CompileAnalysis): Map[VirtualFileRef, SourceInfo] = { previousAnalysis.readSourceInfos().getAllSourceInfos.asScala.toMap } diff --git a/backend/src/main/scala/sbt/internal/inc/BloopComponentCompiler.scala b/backend/src/main/scala/sbt/internal/inc/BloopComponentCompiler.scala index 833e5f2f12..87b1399eb2 100644 --- a/backend/src/main/scala/sbt/internal/inc/BloopComponentCompiler.scala +++ b/backend/src/main/scala/sbt/internal/inc/BloopComponentCompiler.scala @@ -31,6 +31,8 @@ import _root_.bloop.logging.{Logger => BloopLogger} import _root_.bloop.{DependencyResolution => BloopDependencyResolution} import _root_.bloop.logging.DebugFilter import scala.concurrent.ExecutionContext +import java.nio.file.Path +import sbt.internal.inc.classpath.ClasspathUtil object BloopComponentCompiler { import xsbti.compile.ScalaInstance @@ -159,18 +161,19 @@ object BloopComponentCompiler { val scalaLibrary = scalaArtifacts.library val jarsToLoad = (scalaCompiler +: scalaLibrary +: scalaArtifacts.others).toArray assert(jarsToLoad.forall(_.exists), "One or more jar(s) in the Scala instance do not exist.") - val loaderLibraryOnly = ClasspathUtilities.toLoader(Vector(scalaLibrary)) + val loaderLibraryOnly = ClasspathUtil.toLoader(Vector(scalaLibrary.toPath())) val jarsToLoad2 = jarsToLoad.toVector.filterNot(_ == scalaLibrary) - val loader = ClasspathUtilities.toLoader(jarsToLoad2, loaderLibraryOnly) + val loader = ClasspathUtil.toLoader(jarsToLoad2.map(_.toPath()), loaderLibraryOnly) val properties = ResourceLoader.getSafePropertiesFor("compiler.properties", loader) val loaderVersion = Option(properties.getProperty("version.number")) val scalaV = loaderVersion.getOrElse("unknown") new inc.ScalaInstance( scalaV, loader, + loader, loaderLibraryOnly, - scalaLibrary, - scalaCompiler, + Array(scalaLibrary), + jarsToLoad, jarsToLoad, loaderVersion ) @@ -264,7 +267,7 @@ private[inc] class BloopComponentCompiler( logger, resolveSources = shouldResolveSources )(scheduler) match { - case Right(paths) => paths.map(_.toFile).toVector + case Right(paths) => paths.map(_.underlying).toVector case Left(t) => val msg = s"Couldn't retrieve module $bridgeSources" throw new InvalidComponent(msg, t) @@ -272,9 +275,10 @@ private[inc] class BloopComponentCompiler( if (!shouldResolveSources) { // This is usually true in the Dotty case, that has a pre-compiled compiler - manager.define(compilerBridgeId, allArtifacts) + manager.define(compilerBridgeId, allArtifacts.map(_.toFile())) } else { - val (sources, xsbtiJars) = allArtifacts.partition(_.getName.endsWith("-sources.jar")) + val (sources, xsbtiJars) = + allArtifacts.partition(_.toFile.getName.endsWith("-sources.jar")) val (toCompileID, allSources) = { val instance = compiler.scalaInstance if (!HydraSupport.isEnabled(compiler.scalaInstance)) (bridgeSources.name, sources) @@ -296,7 +300,7 @@ private[inc] class BloopComponentCompiler( AnalyzingCompiler.compileSources( allSources, - target, + target.toPath(), xsbtiJars, toCompileID, compiler, @@ -311,9 +315,9 @@ private[inc] class BloopComponentCompiler( import xsbti.compile.ScalaInstance private def mergeBloopAndHydraBridges( - bloopBridgeSourceJars: Vector[File], + bloopBridgeSourceJars: Vector[Path], hydraBridgeModule: ModuleID - ): Either[InvalidComponent, Vector[File]] = { + ): Either[InvalidComponent, Vector[Path]] = { val hydraSourcesJars = BloopDependencyResolution.resolveWithErrors( List( BloopDependencyResolution @@ -327,7 +331,7 @@ private[inc] class BloopComponentCompiler( resolveSources = true, additionalRepositories = List(HydraSupport.resolver) )(scheduler) match { - case Right(paths) => Right(paths.map(_.toFile).toVector) + case Right(paths) => Right(paths.map(_.underlying).toVector) case Left(t) => val msg = s"Couldn't retrieve module $hydraBridgeModule" Left(new InvalidComponent(msg, t)) @@ -350,7 +354,7 @@ private[inc] class BloopComponentCompiler( } withTemporaryDirectory { tempDir => - val hydraSourceContents = unzip(sourceJar, tempDir) + val hydraSourceContents = unzip(sourceJar.toFile, tempDir) logger.debug(s"Sources from hydra bridge: $hydraSourceContents") // Unfortunately we can only use names to filter out, let's hope there's no clashes @@ -366,7 +370,7 @@ private[inc] class BloopComponentCompiler( // Extract bridge source contens in same folder with Hydra contents having preference val regularSourceContents = bloopBridgeSourceJars.foldLeft(Set.empty[File]) { case (extracted, sourceJar) => - extracted ++ unzip(sourceJar, tempDir, filter = filterOutConflicts) + extracted ++ unzip(sourceJar.toFile(), tempDir, filter = filterOutConflicts) } logger.debug(s"Sources from bloop bridge: $regularSourceContents") @@ -376,8 +380,8 @@ private[inc] class BloopComponentCompiler( val allSourceContents = (hydraSourceContents ++ regularSourceContents).map(s => s -> relativize(tempDir, s).get) - zip(allSourceContents.toSeq, mergedJar) - Right(Vector(mergedJar)) + zip(allSourceContents.toSeq, mergedJar, time = None) + Right(Vector(mergedJar.toPath())) } case Right(Seq()) => 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 711162eeff..aa05a7c8b1 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/BloopZincCompiler.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/BloopZincCompiler.scala @@ -18,6 +18,10 @@ import xsbti.compile._ import bloop.UniqueCompileInputs import scala.concurrent.Promise +import sbt.internal.inc.PlainVirtualFile +import sbt.internal.inc.PlainVirtualFileConverter +import bloop.util.AnalysisUtils +import xsbti.VirtualFile object BloopZincCompiler { import bloop.logging.DebugFilter @@ -90,8 +94,8 @@ object BloopZincCompiler { def compileIncrementally( scalaCompiler: xsbti.compile.ScalaCompiler, javaCompiler: xsbti.compile.JavaCompiler, - sources: Array[File], - classpath: Seq[File], + sources: Array[VirtualFile], + classpath: Seq[VirtualFile], uniqueInputs: UniqueCompileInputs, output: Output, cache: GlobalsCache, @@ -128,7 +132,7 @@ object BloopZincCompiler { val analysis = invalidateAnalysisFromSetup(config.currentSetup, previousSetup, incrementalOptions.ignoredScalacOptions(), setOfSources, prev, manager, logger) // Scala needs the explicit type signature to infer the function type arguments - val compile: (Set[File], DependencyChanges, AnalysisCallback, ClassFileManager) => Task[Unit] = compiler.compile(_, _, _, _, cancelPromise, classpathOptions) + val compile: (Set[VirtualFile], DependencyChanges, AnalysisCallback, ClassFileManager) => Task[Unit] = compiler.compile(_, _, _, _, cancelPromise, classpathOptions) BloopIncremental .compile( setOfSources, @@ -169,15 +173,18 @@ object BloopZincCompiler { setup: MiniSetup, previousSetup: Option[MiniSetup], ignoredScalacOptions: Array[String], - sources: Set[File], + sources: Set[VirtualFile], previousAnalysis: CompileAnalysis, manager: ClassFileManager, logger: ObservedLogger[_] ): CompileAnalysis = { // Copied from `Incremental` to pass in the class file manager we want - def prune(invalidatedSrcs: Set[File], previous0: CompileAnalysis, classfileManager: ClassFileManager): Analysis = { + def prune(invalidatedSrcs: Set[VirtualFile], previous0: CompileAnalysis, classfileManager: ClassFileManager): Analysis = { val previous = previous0 match { case a: Analysis => a } - classfileManager.delete(invalidatedSrcs.flatMap(previous.relations.products).toArray) + val toDelete = invalidatedSrcs.flatMap(previous.relations.products).toArray.collect { + case vf: VirtualFile => vf + } + classfileManager.delete(toDelete) previous -- invalidatedSrcs } @@ -222,8 +229,8 @@ object BloopZincCompiler { def configureAnalyzingCompiler( scalac: xsbti.compile.ScalaCompiler, javac: xsbti.compile.JavaCompiler, - sources: Seq[File], - classpath: Seq[File], + sources: Seq[VirtualFile], + classpath: Seq[VirtualFile], classpathHashes: Seq[FileHash], output: Output, cache: GlobalsCache, @@ -259,6 +266,7 @@ object BloopZincCompiler { val outputJar = JarUtils.createOutputJarContent(output) MixedAnalyzingCompiler.config( sources, + PlainVirtualFileConverter.converter, classpath, compileSetup, progress, @@ -271,7 +279,11 @@ object BloopZincCompiler { skip, cache, incrementalCompilerOptions, - outputJar + outputJar, + // deals with pipelining + earlyOutput = None, + earlyAnalysisStore = None, + stamper = BloopStamps.initial ) } } diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/ZincInternals.scala b/backend/src/main/scala/sbt/internal/inc/bloop/ZincInternals.scala index 204674da5a..64be4c221e 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/ZincInternals.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/ZincInternals.scala @@ -11,6 +11,9 @@ import sbt.internal.inc.javac.AnalyzingJavaCompiler import sbt.librarymanagement.{Configurations, ModuleID} import xsbti.compile.{ClasspathOptions, JavaCompiler} import xsbti.{ComponentProvider, Position} +import xsbti.VirtualFile +import bloop.util.AnalysisUtils +import xsbti.VirtualFileRef object ZincInternals { import sbt.internal.inc.JavaInterfaceUtil.EnrichOptional @@ -42,11 +45,11 @@ object ZincInternals { def instantiateJavaCompiler( javac: xsbti.compile.JavaCompiler, - classpath: Seq[File], + classpath: Seq[VirtualFile], instance: xsbti.compile.ScalaInstance, cpOptions: ClasspathOptions, - lookup: (String => Option[File]), - searchClasspath: Seq[File] + lookup: (String => Option[VirtualFile]), + searchClasspath: Seq[VirtualFile] ): JavaCompiler = { new AnalyzingJavaCompiler(javac, classpath, instance, cpOptions, lookup, searchClasspath) } @@ -55,7 +58,7 @@ object ZincInternals { import sbt.internal.util.Relation def copyRelations( relations: Relations, - rebase: File => File + rebase: VirtualFileRef => VirtualFileRef ): Relations = { val newSrcProd = Relation.empty ++ { relations.srcProd.all.map { diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopAnalysisCallback.scala b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopAnalysisCallback.scala index fb28441ae6..6413a87cb9 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopAnalysisCallback.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopAnalysisCallback.scala @@ -30,6 +30,16 @@ import sbt.internal.inc.UsedName import sbt.internal.inc.Analysis import sbt.internal.inc.Compilation import sbt.internal.inc.SourceInfos +import sbt.internal.inc.UsedNames +import sbt.internal.inc.PlainVirtualFile +import xsbti.VirtualFile +import bloop.util.AnalysisUtils +import java.nio.file.Path +import xsbti.VirtualFileRef +import xsbti.VirtualFileRef +import java.nio.file.Path +import xsbti.T2 +import sbt.internal.inc.PlainVirtualFileConverter trait IBloopAnalysisCallback extends xsbti.AnalysisCallback { def get: Analysis @@ -37,14 +47,15 @@ trait IBloopAnalysisCallback extends xsbti.AnalysisCallback { final class BloopAnalysisCallback( internalBinaryToSourceClassName: String => Option[String], - internalSourceToClassNamesMap: File => Set[String], - externalAPI: (File, String) => Option[AnalyzedClass], + internalSourceToClassNamesMap: VirtualFile => Set[String], + externalAPI: (Path, String) => Option[AnalyzedClass], stampReader: ReadStamps, output: Output, options: IncOptions, manager: ClassFileManager ) extends IBloopAnalysisCallback { - private[this] val compilation: Compilation = Compilation(output) + + private[this] val compilation: Compilation = Compilation(System.currentTimeMillis(), output) override def toString = (List("Class APIs", "Object APIs", "Binary deps", "Products", "Source deps") zip @@ -60,46 +71,54 @@ final class BloopAnalysisCallback( import collection.mutable - private[this] val srcs = mutable.HashSet[File]() + private[this] val srcs = mutable.HashSet[Path]() private[this] val classApis = new mutable.HashMap[String, ApiInfo] private[this] val objectApis = new mutable.HashMap[String, ApiInfo] private[this] val classPublicNameHashes = new mutable.HashMap[String, Array[NameHash]] private[this] val objectPublicNameHashes = new mutable.HashMap[String, Array[NameHash]] private[this] val usedNames = new mutable.HashMap[String, mutable.HashSet[UsedName]] - private[this] val unreportedProblems = new mutable.HashMap[File, mutable.ListBuffer[Problem]] - private[this] val reportedProblems = new mutable.HashMap[File, mutable.ListBuffer[Problem]] - private[this] val mainClasses = new mutable.HashMap[File, mutable.ListBuffer[String]] - private[this] val binaryDeps = new mutable.HashMap[File, mutable.HashSet[File]] + private[this] val unreportedProblems = new mutable.HashMap[Path, mutable.ListBuffer[Problem]] + private[this] val reportedProblems = new mutable.HashMap[Path, mutable.ListBuffer[Problem]] + private[this] val mainClasses = new mutable.HashMap[Path, mutable.ListBuffer[String]] + private[this] val binaryDeps = new mutable.HashMap[Path, mutable.HashSet[Path]] // source file to set of generated (class file, binary class name); only non local classes are stored here - private[this] val nonLocalClasses = new mutable.HashMap[File, mutable.HashSet[(File, String)]] - private[this] val localClasses = new mutable.HashMap[File, mutable.HashSet[File]] + private[this] val nonLocalClasses = new mutable.HashMap[Path, mutable.HashSet[(Path, String)]] + private[this] val localClasses = new mutable.HashMap[Path, mutable.HashSet[Path]] // mapping between src class name and binary (flat) class name for classes generated from src file - private[this] val classNames = new mutable.HashMap[File, mutable.HashSet[(String, String)]] + private[this] val classNames = new mutable.HashMap[Path, mutable.HashSet[(String, String)]] // generated class file to its source class name - private[this] val classToSource = new mutable.HashMap[File, String] + private[this] val classToSource = new mutable.HashMap[Path, String] // internal source dependencies private[this] val intSrcDeps = new mutable.HashMap[String, mutable.HashSet[InternalDependency]] // external source dependencies private[this] val extSrcDeps = new mutable.HashMap[String, mutable.HashSet[ExternalDependency]] - private[this] val binaryClassName = new mutable.HashMap[File, String] + private[this] val binaryClassName = new mutable.HashMap[Path, String] // source files containing a macro def. private[this] val macroClasses = mutable.HashSet[String]() + private[this] val converter = PlainVirtualFileConverter.converter + private def add[A, B](map: mutable.HashMap[A, mutable.HashSet[B]], a: A, b: B): Unit = { map.getOrElseUpdate(a, new mutable.HashSet[B]()).+=(b) () } - def startSource(source: File): Unit = { + def startSource(source: VirtualFile): Unit = { + val sourcePath = converter.toPath(source) if (options.strictMode()) { assert( - !srcs.contains(source), + !srcs.contains(sourcePath), s"The startSource can be called only once per source file: $source" ) } - srcs.add(source) + srcs.add(sourcePath) () + + } + + def startSource(source: File): Unit = { + startSource(PlainVirtualFile(source.toPath())) } def problem( @@ -112,7 +131,7 @@ final class BloopAnalysisCallback( for (source <- InterfaceUtil.jo2o(pos.sourceFile)) { val map = if (reported) reportedProblems else unreportedProblems map - .getOrElseUpdate(source, new mutable.ListBuffer()) + .getOrElseUpdate(source.toPath(), new mutable.ListBuffer()) .+=(InterfaceUtil.problem(category, pos, msg, severity, None)) } } @@ -123,13 +142,13 @@ final class BloopAnalysisCallback( } private[this] def externalBinaryDependency( - binary: File, + binary: Path, className: String, - source: File, + source: VirtualFileRef, context: DependencyContext ): Unit = { binaryClassName.put(binary, className) - add(binaryDeps, source, binary) + add(binaryDeps, converter.toPath(source), binary) } private[this] def externalSourceDependency( @@ -143,13 +162,13 @@ final class BloopAnalysisCallback( add(extSrcDeps, sourceClassName, dependency) } - def binaryDependency( - classFile: File, + override def binaryDependency( + classFile: Path, onBinaryClassName: String, fromClassName: String, - fromSourceFile: File, + fromSourceFile: VirtualFileRef, context: DependencyContext - ) = { + ): Unit = { internalBinaryToSourceClassName(onBinaryClassName) match { case Some(dependsOn) => // dependsOn is a source class name // dependency is a product of a source not included in this compilation @@ -165,12 +184,27 @@ final class BloopAnalysisCallback( } } } + def binaryDependency( + classFile: File, + onBinaryClassName: String, + fromClassName: String, + fromSourceFile: File, + context: DependencyContext + ) = { + binaryDependency( + classFile.toPath(), + onBinaryClassName, + fromClassName, + PlainVirtualFile(fromSourceFile.toPath()), + context + ) + } private[this] def externalDependency( - classFile: File, + classFile: Path, onBinaryName: String, sourceClassName: String, - sourceFile: File, + sourceFile: VirtualFileRef, context: DependencyContext ): Unit = { externalAPI(classFile, onBinaryName) match { @@ -184,29 +218,47 @@ final class BloopAnalysisCallback( } } + override def generatedNonLocalClass( + source: VirtualFileRef, + classFile: Path, + binaryClassName: String, + srcClassName: String + ): Unit = { + val sourcePath = converter.toPath(source) + add(nonLocalClasses, sourcePath, (classFile, binaryClassName)) + add(classNames, sourcePath, (srcClassName, binaryClassName)) + classToSource.put(classFile, srcClassName) + () + + } + def generatedNonLocalClass( source: File, classFile: File, binaryClassName: String, srcClassName: String ): Unit = { - //println(s"Generated non local class ${source}, ${classFile}, ${binaryClassName}, ${srcClassName}") - add(nonLocalClasses, source, (classFile, binaryClassName)) - add(classNames, source, (srcClassName, binaryClassName)) - classToSource.put(classFile, srcClassName) + generatedNonLocalClass( + PlainVirtualFile(source.toPath()), + classFile.toPath(), + binaryClassName, + srcClassName + ) + } + + override def generatedLocalClass(source: VirtualFileRef, classFile: Path): Unit = { + add(localClasses, converter.toPath(source), classFile) () } def generatedLocalClass(source: File, classFile: File): Unit = { - //println(s"Generated local class ${source}, ${classFile}") - add(localClasses, source, classFile) - () + generatedLocalClass(PlainVirtualFile(source.toPath()), classFile.toPath()) } - def api(sourceFile: File, classApi: ClassLike): Unit = { + def api(sourceFile: VirtualFileRef, classApi: ClassLike): Unit = { import xsbt.api.{APIUtil, HashAPI} val className = classApi.name - if (APIUtil.isScalaSourceName(sourceFile.getName) && APIUtil.hasMacro(classApi)) + if (APIUtil.isScalaSourceName(sourceFile.name()) && APIUtil.hasMacro(classApi)) macroClasses.add(className) val shouldMinimize = !Incremental.apiDebug(options) val savedClassApi = if (shouldMinimize) APIUtil.minimize(classApi) else classApi @@ -227,11 +279,19 @@ final class BloopAnalysisCallback( } } - def mainClass(sourceFile: File, className: String): Unit = { - mainClasses.getOrElseUpdate(sourceFile, new mutable.ListBuffer).+=(className) + override def api(sourceFile: File, classApi: ClassLike): Unit = { + api(converter.toVirtualFile(sourceFile.toPath()), classApi) + } + + override def mainClass(sourceFile: VirtualFileRef, className: String): Unit = { + mainClasses.getOrElseUpdate(converter.toPath(sourceFile), new mutable.ListBuffer).+=(className) () } + def mainClass(sourceFile: File, className: String): Unit = { + mainClass(PlainVirtualFile(sourceFile.toPath()), className) + } + def usedName(className: String, name: String, useScopes: ju.EnumSet[UseScope]) = add(usedNames, className, UsedName(name, useScopes)) @@ -241,14 +301,18 @@ final class BloopAnalysisCallback( addUsedNames(addCompilation(addProductsAndDeps(Analysis.empty))) } + // TODO make sure this is correct + override def isPickleJava(): Boolean = false + + // TODO make sure this is correct + override def getPickleJarPair(): ju.Optional[T2[Path, Path]] = ju.Optional.empty() + def getOrNil[A, B](m: collection.Map[A, Seq[B]], a: A): Seq[B] = m.get(a).toList.flatten def addCompilation(base: Analysis): Analysis = base.copy(compilations = base.compilations.add(compilation)) def addUsedNames(base: Analysis): Analysis = (base /: usedNames) { case (a, (className, names)) => - (a /: names) { - case (a, name) => a.copy(relations = a.relations.addUsedName(className, name)) - } + a.copy(relations = a.relations.addUsedNames(UsedNames.fromMultiMap(Map(className -> names)))) } private def companionsWithHash(className: String): (Companions, HashAPI.Hash, HashAPI.Hash) = { @@ -296,7 +360,7 @@ final class BloopAnalysisCallback( def addProductsAndDeps(base: Analysis): Analysis = { (base /: srcs) { case (a, src) => - val stamp = stampReader.source(src) + val stamp = stampReader.source(PlainVirtualFile(src)) val classesInSrc = classNames.getOrElse(src, new mutable.HashSet[(String, String)]()).map(_._1) val analyzedApis = classesInSrc.map(analyzeClass) @@ -305,12 +369,13 @@ final class BloopAnalysisCallback( getOrNil(unreportedProblems, src), getOrNil(mainClasses, src) ) - val binaries = binaryDeps.getOrElse(src, Nil: Iterable[File]) + val binaries = binaryDeps.getOrElse(src, Nil: Iterable[Path]) val localProds = localClasses - .getOrElse(src, new mutable.HashSet[File]()) + .getOrElse(src, new mutable.HashSet[Path]()) .map { classFile => - val classFileStamp = stampReader.product(classFile) - Analysis.LocalProduct(classFile, classFileStamp) + val virtualFile = PlainVirtualFile(classFile) + val classFileStamp = stampReader.product(virtualFile) + Analysis.LocalProduct(virtualFile, classFileStamp) } val binaryToSrcClassName = (classNames @@ -320,12 +385,13 @@ final class BloopAnalysisCallback( }) .toMap val nonLocalProds = nonLocalClasses - .getOrElse(src, Nil: Iterable[(File, String)]) + .getOrElse(src, Nil: Iterable[(Path, String)]) .map { case (classFile, binaryClassName) => + val virtualFile = PlainVirtualFile(classFile) val srcClassName = binaryToSrcClassName(binaryClassName) - val classFileStamp = stampReader.product(classFile) - Analysis.NonLocalProduct(srcClassName, binaryClassName, classFile, classFileStamp) + val classFileStamp = stampReader.product(virtualFile) + Analysis.NonLocalProduct(srcClassName, binaryClassName, virtualFile, classFileStamp) } val internalDeps = classesInSrc.flatMap(cls => @@ -334,10 +400,13 @@ final class BloopAnalysisCallback( val externalDeps = classesInSrc.flatMap(cls => extSrcDeps.getOrElse(cls, new mutable.HashSet[ExternalDependency]()) ) - val binDeps = binaries.map(d => (d, binaryClassName(d), stampReader binary d)) + val binDeps = binaries.map { d => + val virtual = PlainVirtualFile(d) + (virtual, binaryClassName(d), stampReader.library(virtual)) + } a.addSource( - src, + PlainVirtualFile(src), analyzedApis, stamp, info, @@ -356,7 +425,7 @@ final class BloopAnalysisCallback( * the Zinc API phase has run and collected them so that the class file * invalidation registers these files before compiling Java files incrementally. */ - manager.generated(classToSource.keysIterator.toArray) + manager.generated(classToSource.keysIterator.map(PlainVirtualFile(_): VirtualFile).toArray) } override def dependencyPhaseCompleted(): Unit = () override def classesInOutputJar(): java.util.Set[String] = ju.Collections.emptySet() 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 433de064a8..863e5cfa5d 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 @@ -22,6 +22,10 @@ import scala.util.control.NonFatal import sbt.internal.inc.JarUtils import scala.concurrent.Promise import xsbt.InterfaceCompileCancelled +import bloop.util.AnalysisUtils +import sbt.internal.inc.PlainVirtualFileConverter +import java.nio.file.Path +import xsbti.VirtualFile /** * Defines a high-level compiler after [[sbt.internal.inc.MixedAnalyzingCompiler]], with the @@ -44,7 +48,8 @@ final class BloopHighLevelCompiler( tracer: BraveTracer ) { private[this] final val setup = config.currentSetup - private[this] final val classpath = config.classpath.map(_.getAbsoluteFile) + private[this] final val classpath: Seq[VirtualFile] = config.classpath + private[this] final val classpathNio: Seq[Path] = classpath.map(PlainVirtualFileConverter.converter.toPath) private[this] val JavaCompleted: Promise[Unit] = Promise.successful(()) @@ -59,7 +64,7 @@ final class BloopHighLevelCompiler( * @return */ def compile( - sourcesToCompile: Set[File], + sourcesToCompile: Set[VirtualFile], changes: DependencyChanges, callback: AnalysisCallback, classfileManager: ClassFileManager, @@ -74,21 +79,22 @@ final class BloopHighLevelCompiler( val outputDirs = { setup.output match { - case single: SingleOutput => List(single.getOutputDirectory) - case mult: MultipleOutput => mult.getOutputGroups.iterator.map(_.getOutputDirectory).toList + case single: SingleOutput => List(single.getOutputDirectoryAsPath()) + case mult: MultipleOutput => mult.getOutputGroups.iterator.map(_.getOutputDirectoryAsPath()).toList } } outputDirs.foreach { d => - if (!d.getPath.endsWith(".jar") && !d.exists()) - sbt.io.IO.createDirectory(d) + val dFile = d.toFile() + if (!d.endsWith(".jar") && !dFile.exists()) + sbt.io.IO.createDirectory(dFile) } val includedSources = config.sources.filter(sourcesToCompile) - val (javaSources, scalaSources) = includedSources.partition(_.getName.endsWith(".java")) + val (javaSources, scalaSources) = includedSources.partition(_.name().endsWith(".java")) val existsCompilation = javaSources.size + scalaSources.size > 0 if (existsCompilation) { - reporter.reportStartIncrementalCycle(includedSources, outputDirs) + reporter.reportStartIncrementalCycle(includedSources, outputDirs.map(_.toFile())) } val completeJava = JavaCompleted @@ -113,7 +119,7 @@ final class BloopHighLevelCompiler( def compilerArgs: CompilerArguments = { import sbt.internal.inc.CompileFailed - if (scalac.scalaInstance.compilerJar() == null) { + if (scalac.scalaInstance.compilerJars().isEmpty) { throw new CompileFailed(new Array(0), s"Expected Scala compiler jar in Scala instance containing ${scalac.scalaInstance.allJars().mkString(", ")}", new Array(0)) } @@ -125,13 +131,24 @@ final class BloopHighLevelCompiler( } def compileSources( - sources: Seq[File], + sources: Seq[VirtualFile], scalacOptions: Array[String], callback: AnalysisCallback ): Unit = { try { - val args = compilerArgs.apply(Nil, classpath, None, scalacOptions).toArray - scalac.compile(sources.toArray, changes, args, setup.output, callback, config.reporter, config.cache, logger, config.progress.toOptional) + val args = compilerArgs.makeArguments(Nil, classpathNio, scalacOptions) + scalac.compile( + sources.toArray, + classpath.toArray, + PlainVirtualFileConverter.converter, + changes, + args.toArray, + setup.output, + callback, + config.reporter, + config.progress.toOptional, + logger + ) } catch { case NonFatal(t) => // If scala compilation happens, complete the java promise so that it doesn't block @@ -147,7 +164,7 @@ final class BloopHighLevelCompiler( def compileSequentially: Task[Unit] = Task { val scalacOptions = setup.options.scalacOptions - val args = compilerArgs.apply(Nil, classpath, None, scalacOptions).toArray + val args = compilerArgs.makeArguments(Nil, classpathNio, scalacOptions) timed("scalac") { compileSources(sources, scalacOptions, callback) } @@ -165,7 +182,7 @@ final class BloopHighLevelCompiler( ) val javaOptions = setup.options.javacOptions.toArray[String] try { - javac.compile(javaSources, javaOptions, setup.output, callback, incToolOptions, config.reporter, logger, config.progress) + javac.compile(javaSources, Nil, PlainVirtualFileConverter.converter, javaOptions, setup.output, None, callback, incToolOptions, config.reporter, logger, config.progress) completeJava.trySuccess(()) () } catch { 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 121139a322..99baa325ed 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 @@ -15,12 +15,23 @@ import xsbti.AnalysisCallback import xsbti.api.AnalyzedClass import xsbti.compile.analysis.{ReadStamps, Stamp} import xsbti.compile._ +import xsbti.VirtualFile +import xsbti.VirtualFileRef +import bloop.util.AnalysisUtils +import sbt.internal.inc.PlainVirtualFileConverter +import java.nio.file.Path +import xsbti.PathBasedFile +import sbt.internal.inc.MappedFileConverter object BloopIncremental { type CompileFunction = - (Set[File], DependencyChanges, AnalysisCallback, ClassFileManager) => Task[Unit] + (Set[VirtualFile], DependencyChanges, AnalysisCallback, ClassFileManager) => Task[Unit] + // can translate jrt paths, needed for JDK + + private val converter = PlainVirtualFileConverter.converter + def compile( - sources: Iterable[File], + sources: Iterable[VirtualFile], uniqueInputs: UniqueCompileInputs, lookup: Lookup, compile: CompileFunction, @@ -33,7 +44,7 @@ object BloopIncremental { tracer: BraveTracer, isHydraEnabled: Boolean ): Task[(Boolean, Analysis)] = { - def getExternalAPI(lookup: Lookup): (File, String) => Option[AnalyzedClass] = { (_: File, binaryClassName: String) => + def getExternalAPI(lookup: Lookup): (Path, String) => Option[AnalyzedClass] = { (_: Path, binaryClassName: String) => lookup.lookupAnalysis(binaryClassName) flatMap { case (analysis: Analysis) => val sourceClassName = @@ -47,7 +58,7 @@ object BloopIncremental { val previous = previous0 match { case a: Analysis => a } val previousRelations = previous.relations val internalBinaryToSourceClassName = (binaryClassName: String) => previousRelations.productClassName.reverse(binaryClassName).headOption - val internalSourceToClassNamesMap: File => Set[String] = (f: File) => previousRelations.classNames(f) + val internalSourceToClassNamesMap: VirtualFile => Set[String] = (f: VirtualFile) => previousRelations.classNames(f) val builder: () => IBloopAnalysisCallback = { if (!isHydraEnabled) () => new BloopAnalysisCallback(internalBinaryToSourceClassName, internalSourceToClassNamesMap, externalAPI, current, output, options, manager) @@ -59,7 +70,7 @@ object BloopIncremental { } def compileIncremental( - sources: Iterable[File], + sources: Iterable[VirtualFile], uniqueInputs: UniqueCompileInputs, lookup: Lookup, previous: Analysis, @@ -77,9 +88,15 @@ object BloopIncremental { )(implicit equivS: Equiv[Stamp]): Task[(Boolean, Analysis)] = { val setOfSources = sources.toSet val incremental = new BloopNameHashing(log, reporter, uniqueInputs, options, profiler.profileRun, tracer) - val initialChanges = incremental.detectInitialChanges(setOfSources, previous, current, lookup, output) + val initialChanges = incremental.detectInitialChanges(setOfSources, previous, current, lookup, converter, output) + def isJrt(path: Path) = path.getFileSystem.provider().getScheme == "jrt" val binaryChanges = new DependencyChanges { - val modifiedBinaries = initialChanges.binaryDeps.toArray + val modifiedLibraries = initialChanges.libraryDeps.toArray + + val modifiedBinaries: Array[File] = modifiedLibraries.map(converter.toPath(_)).collect { + case path if !isJrt(path) => + path.toFile() + } val modifiedClasses = initialChanges.external.allModified.toArray def isEmpty = modifiedBinaries.isEmpty && modifiedClasses.isEmpty } @@ -99,14 +116,14 @@ object BloopIncremental { import sbt.internal.inc.{ClassFileManager => ClassFileManagerImpl} val analysisTask = { - val doCompile = (srcs: Set[File], changes: DependencyChanges) => { + val doCompile = (srcs: Set[VirtualFile], changes: DependencyChanges) => { for { callback <- Task.now(callbackBuilder()) _ <- compile(srcs, changes, callback, manager) } yield callback.get } - incremental.entrypoint(initialInvClasses, initialInvSources, setOfSources, binaryChanges, lookup, previous, doCompile, manager, 1) + incremental.entrypoint(initialInvClasses, initialInvSources.collect { case v: VirtualFile => v }, setOfSources, binaryChanges, lookup, previous, doCompile, manager, 1) } analysisTask.materialize.map { diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopLookup.scala b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopLookup.scala index c4f9e24b56..ed17e57b85 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopLookup.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopLookup.scala @@ -45,9 +45,9 @@ object BloopLookup { // bloop server sessions, it's just an optimization to avoid checking for isDir BloopStamps.isDirectoryHash(fh) || // If directory exists, filter it out - fh.file.isDirectory() || + fh.file.toFile().isDirectory() || // If directory is empty classes dir, filter it out - CompileOutPaths.hasEmptyClassesDir(AbsolutePath(fh.file.toPath)) + CompileOutPaths.hasEmptyClassesDir(AbsolutePath(fh.file)) } } } 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 2a8b684250..8d865f9302 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 @@ -11,6 +11,10 @@ import sbt.util.Logger import sbt.internal.inc._ import xsbti.compile.{ClassFileManager, DependencyChanges, IncOptions} import xsbti.compile.Output +import xsbti.VirtualFile +import xsbti.VirtualFile +import xsbti.VirtualFileRef +import xsbti.FileConverter /** * Defines Bloop's version of `IncrementalNameHashing` that extends Zinc's original @@ -59,12 +63,12 @@ private final class BloopNameHashing( */ def entrypoint( invalidatedClasses: Set[String], - initialChangedSources: Set[File], - allSources: Set[File], + initialChangedSources: Set[VirtualFile], + allSources: Set[VirtualFile], binaryChanges: DependencyChanges, lookup: ExternalLookup, previous: Analysis, - compileTask: (Set[File], DependencyChanges) => Task[Analysis], + compileTask: (Set[VirtualFile], DependencyChanges) => Task[Analysis], manager: ClassFileManager, cycleNum: Int ): Task[Analysis] = { @@ -77,7 +81,12 @@ private final class BloopNameHashing( // Computes which source files are mapped to the invalidated classes and recompile them val invalidatedSources = - mapInvalidationsToSources(classesToRecompile, initialChangedSources, allSources, previous) + mapInvalidationsToSources( + classesToRecompile, + initialChangedSources.map(v => v: VirtualFileRef), + allSources.map(v => v: VirtualFileRef), + previous + ).collect { case f: VirtualFile => f } recompileClasses(invalidatedSources, binaryChanges, previous, compileTask, manager).flatMap { current => @@ -138,26 +147,28 @@ private final class BloopNameHashing( import xsbti.compile.analysis.{ReadStamps, Stamp => XStamp} override def detectInitialChanges( - sources: Set[File], + sources: Set[VirtualFile], previousAnalysis: Analysis, stamps: ReadStamps, lookup: Lookup, + converter: FileConverter, output: Output )(implicit equivS: Equiv[XStamp]): InitialChanges = { tracer.traceVerbose("detecting initial changes") { tracer => // Copy pasting from IncrementalCommon to optimize/remove IO work - import IncrementalCommon.{isBinaryModified, findExternalAnalyzedClass} + // import IncrementalCommon.{isLibraryModified, findExternalAnalyzedClass} val previous = previousAnalysis.stamps val previousRelations = previousAnalysis.relations - val hashesMap = uniqueInputs.sources.map(kv => kv.source.toFile -> kv.hash).toMap + val hashesMap = uniqueInputs.sources.map(kv => kv.source -> kv.hash).toMap val sourceChanges = tracer.traceVerbose("source changes") { _ => lookup.changedSources(previousAnalysis).getOrElse { val previousSources = previous.allSources.toSet - new UnderlyingChanges[File] { - private val inBoth = previousSources & sources + new UnderlyingChanges[VirtualFileRef] { + private val sourceRefs = sources.map(f => f: VirtualFileRef) + private val inBoth = previousSources & sourceRefs val removed = previousSources -- inBoth - val added = sources -- inBoth + val added = sourceRefs -- inBoth val (changed, unmodified) = inBoth.partition { f => import sbt.internal.inc.Hash // We compute hashes via xxHash in Bloop, so we adapt them to the zinc hex format @@ -165,26 +176,34 @@ private final class BloopNameHashing( .get(f) .map(bloopHash => BloopStamps.fromBloopHashToZincHash(bloopHash)) .getOrElse(BloopStamps.forHash(f)) - !equivS.equiv(previous.source(f), newStamp) + !equivS.equiv(previous.sources(f), newStamp) } } } } // Unnecessary to compute removed products because we can ensure read-only classes dir is untouched - val removedProducts = Set.empty[File] - val changedBinaries: Set[File] = tracer.traceVerbose("changed binaries") { _ => + val removedProducts = Set.empty[VirtualFileRef] + val changedBinaries: Set[VirtualFileRef] = tracer.traceVerbose("changed binaries") { _ => lookup.changedBinaries(previousAnalysis).getOrElse { - val detectChange = - isBinaryModified(false, lookup, previous, stamps, previousRelations, log) - previous.allBinaries.filter(detectChange).toSet + val detectChange = IncrementalCommon.isLibraryModified( + false, + lookup, + previous, + stamps, + previousRelations, + PlainVirtualFileConverter.converter, + log + ) + previous.allLibraries.filter(detectChange).toSet } } val externalApiChanges: APIChanges = tracer.traceVerbose("external api changes") { _ => val incrementalExternalChanges = { val previousAPIs = previousAnalysis.apis - val externalFinder = findExternalAnalyzedClass(lookup) _ + val externalFinder = + lookup.lookupAnalyzedClass(_: String, None).getOrElse(APIs.emptyAnalyzedClass) detectAPIChanges( previousAPIs.allExternals, previousAPIs.externalAPI, @@ -205,21 +224,28 @@ private final class BloopNameHashing( } def recompileClasses( - sources: Set[File], + sources: Set[VirtualFile], binaryChanges: DependencyChanges, previous: Analysis, - compileTask: (Set[File], DependencyChanges) => Task[Analysis], + compileTask: (Set[VirtualFile], DependencyChanges) => Task[Analysis], classfileManager: ClassFileManager ): Task[Analysis] = { val pruned = - IncrementalCommon.pruneClassFilesOfInvalidations(sources, previous, classfileManager) + IncrementalCommon.pruneClassFilesOfInvalidations( + sources, + previous, + classfileManager, + PlainVirtualFileConverter.converter + ) debug("********* Pruned: \n" + pruned.relations + "\n*********") - compileTask(sources, binaryChanges).map { fresh => + compileTask(sources: Set[VirtualFile], binaryChanges).map { fresh => debug("********* Fresh: \n" + fresh.relations + "\n*********") /* This is required for both scala compilation and forked java compilation, despite * being redundant for the most common Java compilation (using the local compiler). */ - classfileManager.generated(fresh.relations.allProducts.toArray) + classfileManager.generated(fresh.relations.allProducts.collect { + case v: VirtualFile => v + }.toArray) val merged = pruned ++ fresh debug("********* Merged: \n" + merged.relations + "\n*********") diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopStamps.scala b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopStamps.scala index e21c299a68..58e5a3ce9a 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopStamps.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/internal/BloopStamps.scala @@ -10,14 +10,19 @@ import sbt.internal.inc.Stamps import sbt.internal.inc.Stamper import sbt.internal.inc.EmptyStamp import xsbti.compile.analysis.{ReadStamps, Stamp} +import xsbti.VirtualFileRef +import bloop.util.AnalysisUtils +import java.nio.file.Path +import sbt.internal.inc.PlainVirtualFileConverter object BloopStamps { + private val converter = PlainVirtualFileConverter.converter def initial: ReadStamps = { Stamps.initial( - Stamper.forLastModified, + Stamper.forLastModifiedInRootPaths(converter), // The hash is for the sources BloopStamps.forHash, - Stamper.forLastModified + Stamper.forHashInRootPaths(converter) ) } @@ -25,14 +30,14 @@ object BloopStamps { private final val directoryHash = scala.util.Random.nextInt() final val cancelledHash = scala.util.Random.nextInt() - def emptyHash(file: File): FileHash = FileHash.of(file, emptyHash) - def cancelledHash(file: File): FileHash = FileHash.of(file, cancelledHash) + def emptyHash(path: Path): FileHash = FileHash.of(path, emptyHash) + def cancelledHash(path: Path): FileHash = FileHash.of(path, cancelledHash) - def directoryHash(file: File): FileHash = FileHash.of(file, directoryHash) + def directoryHash(path: Path): FileHash = FileHash.of(path, directoryHash) def isDirectoryHash(fh: FileHash): Boolean = fh.hash == directoryHash - def forHash(file: File): Hash = { - fromBloopHashToZincHash(ByteHasher.hashFileContents(file)) + def forHash(file: VirtualFileRef): Hash = { + fromBloopHashToZincHash(ByteHasher.hashFileContents(converter.toPath(file).toFile())) } def emptyStampFor(file: File): Stamp = EmptyStamp diff --git a/backend/src/main/scala/sbt/internal/inc/bloop/internal/ConcurrentAnalysisCallback.scala b/backend/src/main/scala/sbt/internal/inc/bloop/internal/ConcurrentAnalysisCallback.scala index 1dd0e9a181..2d52fe6c25 100644 --- a/backend/src/main/scala/sbt/internal/inc/bloop/internal/ConcurrentAnalysisCallback.scala +++ b/backend/src/main/scala/sbt/internal/inc/bloop/internal/ConcurrentAnalysisCallback.scala @@ -30,6 +30,14 @@ import sbt.internal.inc.UsedName import sbt.internal.inc.Analysis import sbt.internal.inc.Compilation import sbt.internal.inc.SourceInfos +import xsbti.VirtualFile +import sbt.internal.inc.UsedNames +import sbt.internal.inc.PlainVirtualFile +import java.nio.file.Path +import xsbti.VirtualFileRef +import xsbti.T2 +import bloop.util.AnalysisUtils +import sbt.internal.inc.PlainVirtualFileConverter /** * This class provides a thread-safe implementation of `xsbti.AnalysisCallback` which is required to compile with the @@ -43,14 +51,15 @@ import sbt.internal.inc.SourceInfos */ final class ConcurrentAnalysisCallback( internalBinaryToSourceClassName: String => Option[String], - internalSourceToClassNamesMap: File => Set[String], - externalAPI: (File, String) => Option[AnalyzedClass], + internalSourceToClassNamesMap: VirtualFile => Set[String], + externalAPI: (Path, String) => Option[AnalyzedClass], stampReader: ReadStamps, output: Output, options: IncOptions, manager: ClassFileManager ) extends IBloopAnalysisCallback { - private[this] val compilation: Compilation = Compilation(output) + + private[this] val compilation: Compilation = Compilation(System.currentTimeMillis(), output) override def toString = (List("Class APIs", "Object APIs", "Binary deps", "Products", "Source deps") zip @@ -69,48 +78,56 @@ final class ConcurrentAnalysisCallback( private type ConcurrentSet[A] = ConcurrentHashMap.KeySetView[A, java.lang.Boolean] - private[this] val srcs = ConcurrentHashMap.newKeySet[File]() + // TODO change to Path + private[this] val srcs = ConcurrentHashMap.newKeySet[Path]() private[this] val classApis = new TrieMap[String, ApiInfo] private[this] val objectApis = new TrieMap[String, ApiInfo] private[this] val classPublicNameHashes = new TrieMap[String, Array[NameHash]] private[this] val objectPublicNameHashes = new TrieMap[String, Array[NameHash]] private[this] val usedNames = new TrieMap[String, ConcurrentSet[UsedName]] - private[this] val unreportedProblems = new TrieMap[File, ConcurrentLinkedQueue[Problem]] - private[this] val reportedProblems = new TrieMap[File, ConcurrentLinkedQueue[Problem]] - private[this] val mainClasses = new TrieMap[File, ConcurrentLinkedQueue[String]] - private[this] val binaryDeps = new TrieMap[File, ConcurrentSet[File]] + private[this] val unreportedProblems = new TrieMap[Path, ConcurrentLinkedQueue[Problem]] + private[this] val reportedProblems = new TrieMap[Path, ConcurrentLinkedQueue[Problem]] + private[this] val mainClasses = new TrieMap[Path, ConcurrentLinkedQueue[String]] + private[this] val binaryDeps = new TrieMap[Path, ConcurrentSet[Path]] // source file to set of generated (class file, binary class name); only non local classes are stored here - private[this] val nonLocalClasses = new TrieMap[File, ConcurrentSet[(File, String)]] - private[this] val localClasses = new TrieMap[File, ConcurrentSet[File]] + private[this] val nonLocalClasses = new TrieMap[Path, ConcurrentSet[(Path, String)]] + private[this] val localClasses = new TrieMap[Path, ConcurrentSet[Path]] // mapping between src class name and binary (flat) class name for classes generated from src file - private[this] val classNames = new TrieMap[File, ConcurrentSet[(String, String)]] + private[this] val classNames = new TrieMap[Path, ConcurrentSet[(String, String)]] // generated class file to its source class name - private[this] val classToSource = new TrieMap[File, String] + private[this] val classToSource = new TrieMap[Path, String] // internal source dependencies private[this] val intSrcDeps = new TrieMap[String, ConcurrentSet[InternalDependency]] // external source dependencies private[this] val extSrcDeps = new TrieMap[String, ConcurrentSet[ExternalDependency]] - private[this] val binaryClassName = new TrieMap[File, String] + private[this] val binaryClassName = new TrieMap[Path, String] // source files containing a macro def. private[this] val macroClasses = ConcurrentHashMap.newKeySet[String]() + private[this] val converter = PlainVirtualFileConverter.converter + private def add[A, B](map: TrieMap[A, ConcurrentSet[B]], a: A, b: B): Unit = { map.getOrElseUpdate(a, ConcurrentHashMap.newKeySet[B]()).add(b) () } - def startSource(source: File): Unit = { + def startSource(source: VirtualFile): Unit = { + val sourcePath = converter.toPath(source) if (options.strictMode()) { assert( !srcs.contains(source), s"The startSource can be called only once per source file: $source" ) } - srcs.add(source) + srcs.add(sourcePath) () } + def startSource(source: File): Unit = { + startSource(PlainVirtualFile(source.toPath())) + } + def problem( category: String, pos: Position, @@ -121,7 +138,7 @@ final class ConcurrentAnalysisCallback( for (source <- InterfaceUtil.jo2o(pos.sourceFile)) { val map = if (reported) reportedProblems else unreportedProblems map - .getOrElseUpdate(source, new ConcurrentLinkedQueue) + .getOrElseUpdate(source.toPath(), new ConcurrentLinkedQueue) .add(InterfaceUtil.problem(category, pos, msg, severity, None)) } } @@ -132,13 +149,13 @@ final class ConcurrentAnalysisCallback( } private[this] def externalBinaryDependency( - binary: File, + binary: Path, className: String, - source: File, + source: VirtualFileRef, context: DependencyContext ): Unit = { binaryClassName.put(binary, className) - add(binaryDeps, source, binary) + add(binaryDeps, converter.toPath(source), binary) } private[this] def externalSourceDependency( @@ -153,10 +170,10 @@ final class ConcurrentAnalysisCallback( } def binaryDependency( - classFile: File, + classFile: Path, onBinaryClassName: String, fromClassName: String, - fromSourceFile: File, + fromSourceFile: VirtualFileRef, context: DependencyContext ) = { internalBinaryToSourceClassName(onBinaryClassName) match { @@ -175,11 +192,27 @@ final class ConcurrentAnalysisCallback( } } - private[this] def externalDependency( + def binaryDependency( classFile: File, + onBinaryClassName: String, + fromClassName: String, + fromSourceFile: File, + context: DependencyContext + ) = { + binaryDependency( + classFile.toPath(), + onBinaryClassName, + fromClassName, + PlainVirtualFile(fromSourceFile.toPath()), + context + ) + } + + private[this] def externalDependency( + classFile: Path, onBinaryName: String, sourceClassName: String, - sourceFile: File, + sourceFile: VirtualFileRef, context: DependencyContext ): Unit = { externalAPI(classFile, onBinaryName) match { @@ -194,28 +227,46 @@ final class ConcurrentAnalysisCallback( } def generatedNonLocalClass( - source: File, - classFile: File, + source: VirtualFileRef, + classFile: Path, binaryClassName: String, srcClassName: String ): Unit = { + val sourcePath = converter.toPath(source) //println(s"Generated non local class ${source}, ${classFile}, ${binaryClassName}, ${srcClassName}") - add(nonLocalClasses, source, (classFile, binaryClassName)) - add(classNames, source, (srcClassName, binaryClassName)) + add(nonLocalClasses, sourcePath, (classFile, binaryClassName)) + add(classNames, sourcePath, (srcClassName, binaryClassName)) classToSource.put(classFile, srcClassName) () } - def generatedLocalClass(source: File, classFile: File): Unit = { - //println(s"Generated local class ${source}, ${classFile}") - add(localClasses, source, classFile) + def generatedNonLocalClass( + source: File, + classFile: File, + binaryClassName: String, + srcClassName: String + ): Unit = { + generatedNonLocalClass( + PlainVirtualFile(source.toPath()), + classFile.toPath(), + binaryClassName, + srcClassName + ) + } + + override def generatedLocalClass(source: VirtualFileRef, classFile: Path): Unit = { + add(localClasses, converter.toPath(source), classFile) () } - def api(sourceFile: File, classApi: ClassLike): Unit = { + def generatedLocalClass(source: File, classFile: File): Unit = { + generatedLocalClass(PlainVirtualFile(source.toPath()), classFile.toPath()) + } + + override def api(sourceFile: VirtualFileRef, classApi: ClassLike): Unit = { import xsbt.api.{APIUtil, HashAPI} val className = classApi.name - if (APIUtil.isScalaSourceName(sourceFile.getName) && APIUtil.hasMacro(classApi)) + if (APIUtil.isScalaSourceName(sourceFile.name()) && APIUtil.hasMacro(classApi)) macroClasses.add(className) val shouldMinimize = !Incremental.apiDebug(options) val savedClassApi = if (shouldMinimize) APIUtil.minimize(classApi) else classApi @@ -236,11 +287,21 @@ final class ConcurrentAnalysisCallback( } } - def mainClass(sourceFile: File, className: String): Unit = { - mainClasses.getOrElseUpdate(sourceFile, new ConcurrentLinkedQueue).add(className) + override def api(sourceFile: File, classApi: ClassLike): Unit = { + api(converter.toVirtualFile(sourceFile.toPath()), classApi) + } + + override def mainClass(sourceFile: VirtualFileRef, className: String): Unit = { + mainClasses + .getOrElseUpdate(converter.toPath(sourceFile), new ConcurrentLinkedQueue) + .add(className) () } + def mainClass(sourceFile: File, className: String): Unit = { + mainClass(PlainVirtualFile(sourceFile.toPath()), className) + } + def usedName(className: String, name: String, useScopes: ju.EnumSet[UseScope]) = add(usedNames, className, UsedName(name, useScopes)) @@ -250,15 +311,21 @@ final class ConcurrentAnalysisCallback( addUsedNames(addCompilation(addProductsAndDeps(Analysis.empty))) } + // TODO make sure this is correct + override def isPickleJava(): Boolean = false + + // TODO make sure this is correct + override def getPickleJarPair(): ju.Optional[T2[Path, Path]] = ju.Optional.empty() + def getOrNil[A, B](m: collection.Map[A, Seq[B]], a: A): Seq[B] = m.get(a).toList.flatten def addCompilation(base: Analysis): Analysis = base.copy(compilations = base.compilations.add(compilation)) def addUsedNames(base: Analysis): Analysis = (base /: usedNames) { case (a, (className, names)) => import scala.collection.JavaConverters._ - names.asScala.foldLeft(a) { - case (a, name) => a.copy(relations = a.relations.addUsedName(className, name)) - } + a.copy(relations = + a.relations.addUsedNames(UsedNames.fromMultiMap(Map(className -> names.asScala))) + ) } private def companionsWithHash(className: String): (Companions, HashAPI.Hash, HashAPI.Hash) = { @@ -307,7 +374,8 @@ final class ConcurrentAnalysisCallback( import scala.collection.JavaConverters._ srcs.asScala.foldLeft(base) { case (a, src) => - val stamp = stampReader.source(src) + val sourceV = PlainVirtualFile(src) + val stamp = stampReader.source(sourceV) val classesInSrc = classNames .getOrElse(src, ConcurrentHashMap.newKeySet[(String, String)]()) @@ -319,13 +387,14 @@ final class ConcurrentAnalysisCallback( getOrNil(unreportedProblems.mapValues { _.asScala.toSeq }, src), getOrNil(mainClasses.mapValues { _.asScala.toSeq }, src) ) - val binaries = binaryDeps.getOrElse(src, ConcurrentHashMap.newKeySet[File]).asScala + val binaries = binaryDeps.getOrElse(src, ConcurrentHashMap.newKeySet[Path]).asScala val localProds = localClasses - .getOrElse(src, ConcurrentHashMap.newKeySet[File]()) + .getOrElse(src, ConcurrentHashMap.newKeySet[Path]()) .asScala .map { classFile => - val classFileStamp = stampReader.product(classFile) - Analysis.LocalProduct(classFile, classFileStamp) + val classFileV = PlainVirtualFile(classFile) + val classFileStamp = stampReader.product(classFileV) + Analysis.LocalProduct(classFileV, classFileStamp) } val binaryToSrcClassName = (classNames @@ -336,13 +405,14 @@ final class ConcurrentAnalysisCallback( }) .toMap val nonLocalProds = nonLocalClasses - .getOrElse(src, ConcurrentHashMap.newKeySet[(File, String)]()) + .getOrElse(src, ConcurrentHashMap.newKeySet[(Path, String)]()) .asScala .map { case (classFile, binaryClassName) => + val classFileV = PlainVirtualFile(classFile) val srcClassName = binaryToSrcClassName(binaryClassName) - val classFileStamp = stampReader.product(classFile) - Analysis.NonLocalProduct(srcClassName, binaryClassName, classFile, classFileStamp) + val classFileStamp = stampReader.product(classFileV) + Analysis.NonLocalProduct(srcClassName, binaryClassName, classFileV, classFileStamp) } val internalDeps = classesInSrc.flatMap(cls => @@ -351,10 +421,13 @@ final class ConcurrentAnalysisCallback( val externalDeps = classesInSrc.flatMap(cls => extSrcDeps.getOrElse(cls, ConcurrentHashMap.newKeySet[ExternalDependency]()).asScala ) - val binDeps = binaries.map(d => (d, binaryClassName(d), stampReader binary d)) + val binDeps = binaries.map { d => + val virtual = PlainVirtualFile(d) + (virtual, binaryClassName(d), stampReader.library(virtual)) + } a.addSource( - src, + sourceV, analyzedApis, stamp, info, @@ -369,7 +442,7 @@ final class ConcurrentAnalysisCallback( override def apiPhaseCompleted(): Unit = { // See [[BloopAnalysisCallback.apiPhaseCompleted]] - manager.generated(classToSource.keysIterator.toArray) + manager.generated(classToSource.keysIterator.map(PlainVirtualFile(_): VirtualFile).toArray) } override def dependencyPhaseCompleted(): Unit = () override def classesInOutputJar(): java.util.Set[String] = ju.Collections.emptySet() diff --git a/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala b/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala index dbdd5f2e3f..a339bc0583 100644 --- a/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala +++ b/frontend/src/main/scala/bloop/engine/caches/ResultsCache.scala @@ -217,9 +217,8 @@ object ResultsCache { val r = PreviousResult.of(Optional.of(res.getAnalysis), Optional.of(res.getMiniSetup)) res.getAnalysis.readCompilations.getAllCompilations.lastOption match { case Some(lastCompilation) => - lastCompilation.getOutput.getSingleOutput.toOption match { - case Some(classesDirFile) => - val classesDir = classesDirFile.toPath + lastCompilation.getOutput.getSingleOutputAsPath.toOption match { + case Some(classesDir) => val originPath = p.origin.path.syntax val originHash = p.origin.hash val inputs = UniqueCompileInputs.emptyFor(originPath) diff --git a/frontend/src/main/scala/bloop/engine/tasks/Tasks.scala b/frontend/src/main/scala/bloop/engine/tasks/Tasks.scala index 55710b703f..ac96bbcdf5 100644 --- a/frontend/src/main/scala/bloop/engine/tasks/Tasks.scala +++ b/frontend/src/main/scala/bloop/engine/tasks/Tasks.scala @@ -17,6 +17,9 @@ import sbt.internal.inc.{Analysis, AnalyzingCompiler, ConcreteAnalysisContents, import sbt.internal.inc.classpath.ClasspathUtilities import sbt.testing._ import xsbti.compile.{ClasspathOptionsUtil, CompileAnalysis, MiniSetup, PreviousResult} +import sbt.internal.inc.PlainVirtualFileConverter +import sbt.internal.inc.PlainVirtualFile +import sbt.internal.inc.classpath.ClasspathUtil object Tasks { private[bloop] val TestFailedStatus: Set[Status] = @@ -62,13 +65,21 @@ object Tasks { DebugFilter.All ) val javacBin = project.runtimeJdkConfig.flatMap(_.javacBin) - val loader = ClasspathUtilities.makeLoader(entries, instance) + val pathEntries = entries.map(e => e.toPath()) + val loader = ClasspathUtil.makeLoader(pathEntries, instance) val compiler = state.compilerCache.get(instance, javacBin).scalac.asInstanceOf[AnalyzingCompiler] val opts = ClasspathOptionsUtil.repl val options = project.scalacOptions :+ "-Xnojline" // We should by all means add better error handling here! - compiler.console(entries, options, "", "", state.logger)(Some(loader)) + compiler.console( + pathEntries.map(e => PlainVirtualFile(e)), + PlainVirtualFileConverter.converter, + options, + "", + "", + state.logger + )(Some(loader)) case None => logger.error(s"Missing Scala configuration on project '${project.name}'") } diff --git a/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileBundle.scala b/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileBundle.scala index 05cf512eab..d6b853e938 100644 --- a/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileBundle.scala +++ b/frontend/src/main/scala/bloop/engine/tasks/compilation/CompileBundle.scala @@ -25,6 +25,7 @@ import xsbti.compile.PreviousResult import scala.concurrent.ExecutionContext import bloop.CompileOutPaths import bloop.cli.CommonOptions +import sbt.internal.inc.PlainVirtualFileConverter sealed trait CompileBundle @@ -187,7 +188,7 @@ object CompileBundle { val sourceHashesTask = tracer.traceTaskVerbose("discovering and hashing sources") { _ => bloop.io.SourceHasher .findAndHashSourcesInProject(project, 20, cancelCompilation, ioScheduler) - .map(res => res.map(_.sortBy(_.source.syntax))) + .map(res => res.map(_.sortBy(_.source.id()))) .executeOn(ioScheduler) } @@ -203,7 +204,7 @@ object CompileBundle { val javaSources = new ListBuffer[AbsolutePath]() val scalaSources = new ListBuffer[AbsolutePath]() sourceHashes.foreach { hashed => - val source = hashed.source + val source = AbsolutePath(PlainVirtualFileConverter.converter.toPath(hashed.source)) val sourceName = source.underlying.getFileName().toString if (sourceName.endsWith(".scala")) { scalaSources += source diff --git a/frontend/src/main/scala/bloop/io/SourceHasher.scala b/frontend/src/main/scala/bloop/io/SourceHasher.scala index ac95d4897c..b98b430a9f 100644 --- a/frontend/src/main/scala/bloop/io/SourceHasher.scala +++ b/frontend/src/main/scala/bloop/io/SourceHasher.scala @@ -30,6 +30,7 @@ import monix.execution.Scheduler import monix.reactive.internal.operators.MapAsyncParallelObservable import monix.execution.Cancelable import monix.execution.cancelables.CompositeCancelable +import sbt.internal.inc.PlainVirtualFile object SourceHasher { private final val sourceMatcher = @@ -141,7 +142,7 @@ object SourceHasher { val hashSourcesInParallel = observable.mapAsync(parallelUnits) { (source: Path) => Task.eval { val hash = ByteHasher.hashFileContents(source.toFile) - HashedSource(AbsolutePath(source), hash) + HashedSource(PlainVirtualFile(source), hash) } } diff --git a/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala b/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala index bf5a51739c..a887a0889e 100644 --- a/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala +++ b/frontend/src/main/scala/bloop/reporter/BspProjectReporter.scala @@ -17,6 +17,7 @@ import scala.util.Try import scala.concurrent.Promise import bloop.CompileOutPaths import monix.execution.atomic.AtomicInt +import xsbti.VirtualFile final class BspProjectReporter( val project: Project, @@ -163,7 +164,11 @@ final class BspProjectReporter( ) } - override def reportStartIncrementalCycle(sources: Seq[File], outputDirs: Seq[File]): Unit = { + override def reportStartIncrementalCycle( + sources: Seq[VirtualFile], + outputDirs: Seq[File] + ): Unit = { + val plainFiles = sources.map(converter.toPath(_).toFile()) cycleCount.incrementAndGet() statusForNextEndCycle match { @@ -174,11 +179,11 @@ final class BspProjectReporter( case None => () } - val msg = Reporter.compilationMsgFor(project.name, sources) + val msg = Reporter.compilationMsgFor(project.name, plainFiles) logger.publishCompilationStart( CompilationEvent.StartCompilation(project.name, project.bspUri, msg, taskId) ) - compilingFiles ++ sources + compilingFiles ++ plainFiles } private def clearProblemsAtPhase( diff --git a/frontend/src/main/scala/bloop/reporter/LogReporter.scala b/frontend/src/main/scala/bloop/reporter/LogReporter.scala index 2b5be857da..d838ac256c 100644 --- a/frontend/src/main/scala/bloop/reporter/LogReporter.scala +++ b/frontend/src/main/scala/bloop/reporter/LogReporter.scala @@ -14,6 +14,8 @@ import sbt.util.InterfaceUtil import scala.collection.mutable import scala.collection.concurrent.TrieMap import bloop.logging.CompilationEvent +import xsbti.VirtualFile +import bloop.util.AnalysisUtils final class LogReporter( val project: Project, @@ -64,11 +66,15 @@ final class LogReporter( () } - override def reportStartIncrementalCycle(sources: Seq[File], outputDirs: Seq[File]): Unit = { + override def reportStartIncrementalCycle( + sources: Seq[VirtualFile], + 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 - logger.info(Reporter.compilationMsgFor(project.name, sources)) + val plainFiles = sources.map(converter.toPath(_).toFile()) + compilingFiles ++ plainFiles + logger.info(Reporter.compilationMsgFor(project.name, plainFiles)) } override def reportEndIncrementalCycle(durationMs: Long, result: scala.util.Try[Unit]): Unit = { diff --git a/frontend/src/test/scala/bloop/BaseCompileSpec.scala b/frontend/src/test/scala/bloop/BaseCompileSpec.scala index 03b220243f..1c3e2fa8a4 100644 --- a/frontend/src/test/scala/bloop/BaseCompileSpec.scala +++ b/frontend/src/test/scala/bloop/BaseCompileSpec.scala @@ -815,10 +815,14 @@ abstract class BaseCompileSpec extends bloop.testing.BaseSuite { | cannot find symbol | symbol: class Bar | location: class Foo + | L3: Bar + | ^^^ |[E1] ${targetFoo}:3 | cannot find symbol | symbol: class Bar | location: class Foo + | L3: Bar + | ^^^ |${targetFoo}: L3 [E1], L3 [E2] |""".stripMargin } else { @@ -1065,29 +1069,17 @@ abstract class BaseCompileSpec extends bloop.testing.BaseSuite { s"""[E1] ${targetB}:1 | cannot find symbol | symbol: class A - |${targetB}: L1 [E1]""".stripMargin - } - - val cannotFindSymbolError2: String = { - s"""[E1] ${targetB}:1 - | error: cannot find symbol + | L1: A + | ^ |${targetB}: L1 [E1]""".stripMargin } assertDiagnosticsResult(compiledState.getLastResultFor(`A`), 1) import bloop.testing.DiffAssertions - try { - assertNoDiff( - logger.renderErrors(exceptContaining = "Failed to compile"), - cannotFindSymbolError - ) - } catch { - case _: DiffAssertions.TestFailedException => - assertNoDiff( - logger.renderErrors(exceptContaining = "Failed to compile"), - cannotFindSymbolError2 - ) - } + assertNoDiff( + logger.renderErrors(exceptContaining = "Failed to compile"), + cannotFindSymbolError + ) } } diff --git a/frontend/src/test/scala/bloop/io/SourcesGlobsSpec.scala b/frontend/src/test/scala/bloop/io/SourcesGlobsSpec.scala index df14b4a147..44a1364bd0 100644 --- a/frontend/src/test/scala/bloop/io/SourcesGlobsSpec.scala +++ b/frontend/src/test/scala/bloop/io/SourcesGlobsSpec.scala @@ -12,6 +12,7 @@ import java.util.concurrent.TimeUnit import bloop.cli.ExitStatus import bloop.config.Config import bloop.data.SourcesGlobs +import sbt.internal.inc.PlainVirtualFileConverter object SourcesGlobsSpec extends bloop.testing.BaseSuite { @@ -58,13 +59,14 @@ object SourcesGlobsSpec extends bloop.testing.BaseSuite { val Right(result) = TestUtil.await(10, TimeUnit.SECONDS)(hashedSources) import scala.collection.JavaConverters._ val obtainedFilenames = result - .map( - _.source + .map { file => + val path = AbsolutePath(PlainVirtualFileConverter.converter.toPath(file.source)) + path .toRelative(globDirectory) .toUri(isDirectory = false) .toString() .stripPrefix("globs/src/") - ) + } .sorted .mkString("\n") assertNoDiff(obtainedFilenames, expectedFilenames) diff --git a/integrations/maven-bloop/src/test/scala/bloop/integrations/maven/MavenConfigGenerationSuite.scala b/integrations/maven-bloop/src/test/scala/bloop/integrations/maven/MavenConfigGenerationSuite.scala index 6a6db149fb..de9cae650c 100644 --- a/integrations/maven-bloop/src/test/scala/bloop/integrations/maven/MavenConfigGenerationSuite.scala +++ b/integrations/maven-bloop/src/test/scala/bloop/integrations/maven/MavenConfigGenerationSuite.scala @@ -226,6 +226,7 @@ class MavenConfigGenerationSuite extends BaseConfigSuite { } checking(configFile, projectName, subProjects) tempDir.toFile().delete() + () } catch { case NonFatal(e) => println("Maven output:\n" + result) @@ -248,7 +249,7 @@ class MavenConfigGenerationSuite extends BaseConfigSuite { private def exec(cmd: Seq[String], cwd: File): Try[String] = { Try { val lastError = new StringBuilder - val swallowStderr = ProcessLogger(_ => (), err => lastError.append(err)) + val swallowStderr = ProcessLogger(_ => (), err => { lastError.append(err); () }) val processBuilder = new ProcessBuilder() val out = new StringBuilder() processBuilder.directory(cwd) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 605945d81c..4aa1505f3d 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -13,7 +13,7 @@ object Dependencies { val nailgunCommit = "a2520c1e" // Keep in sync in BloopComponentCompiler - val zincVersion = "1.3.5" + val zincVersion = "1.6.0" val bspVersion = "2.0.0-M13" val javaDebugVersion = "0.21.0+1-7f1080f1"