Skip to content

Commit

Permalink
Add expression evaluation
Browse files Browse the repository at this point in the history
Upgrade scala-debug-adapter to 2.0.2
For expression evaluation, the scala-debug-adapter now needs:
- a set of all the class path entries to be able to map the source files to the class files
- the jvm runtime
- the expression compiler class loader containing the expression-compiler jar and the compiler jars

Add a simple test on expression evaluation
  • Loading branch information
adpi2 committed Sep 17, 2021
1 parent cc6a357 commit 0c26af4
Show file tree
Hide file tree
Showing 9 changed files with 314 additions and 112 deletions.
4 changes: 2 additions & 2 deletions frontend/src/main/scala/bloop/bsp/BloopBspServices.scala
Original file line number Diff line number Diff line change
Expand Up @@ -594,7 +594,7 @@ final class BloopBspServices(
filters => BloopDebuggeeRunner.forTestSuite(projects, filters, state, ioScheduler)
)
case bsp.DebugSessionParamsDataKind.ScalaAttachRemote =>
Right(BloopDebuggeeRunner.forAttachRemote(state, ioScheduler))
Right(BloopDebuggeeRunner.forAttachRemote(state, ioScheduler, projects))
case dataKind => Left(JsonRpcResponse.invalidRequest(s"Unsupported data kind: $dataKind"))
}
}
Expand Down Expand Up @@ -1102,7 +1102,7 @@ final class BloopBspServices(
val sourceJars = project.resolution.toList.flatMap { res =>
res.modules.flatMap { m =>
m.artifacts.iterator
.filter(a => a.classifier.toList.contains("sources"))
.filter(a => a.classifier.contains("sources"))
.map(a => bsp.Uri(AbsolutePath(a.path).toBspUri))
.toList
}
Expand Down
211 changes: 183 additions & 28 deletions frontend/src/main/scala/bloop/dap/BloopDebuggeeRunner.scala
Original file line number Diff line number Diff line change
@@ -1,21 +1,23 @@
package bloop.dap

import bloop.ScalaInstance
import bloop.cli.ExitStatus
import bloop.data.{JdkConfig, Platform, Project}
import bloop.engine.State
import bloop.engine.{Build, State}
import bloop.engine.tasks.{RunMode, Tasks}
import bloop.logging.Logger
import bloop.testing.{LoggingEventHandler, TestInternals}
import ch.epfl.scala.bsp.ScalaMainClass
import ch.epfl.scala.debugadapter.{CancelableFuture, DebuggeeRunner, DebuggeeListener}
import ch.epfl.scala.debugadapter._
import monix.eval.Task
import monix.execution.Scheduler
import xsbti.compile.analysis.SourceInfo

import java.nio.file.Path
import java.net.URLClassLoader
import scala.collection.mutable
import scala.annotation.tailrec

abstract class BloopDebuggeeRunner(initialState: State, ioScheduler: Scheduler)
extends DebuggeeRunner {
private lazy val allAnalysis = initialState.results.allAnalysis

override def run(listener: DebuggeeListener): CancelableFuture[Unit] = {
val debugSessionLogger = new DebuggeeLogger(listener, initialState.logger)
Expand All @@ -28,31 +30,18 @@ abstract class BloopDebuggeeRunner(initialState: State, ioScheduler: Scheduler)
}

protected def start(state: State): Task[ExitStatus]

override def classFilesMappedTo(
origin: Path,
lines: Array[Int],
columns: Array[Int]
): List[Path] = {
def isInfoEmpty(info: SourceInfo) = info == sbt.internal.inc.SourceInfos.emptyInfo

val originFile = origin.toFile
val foundClassFiles = allAnalysis.collectFirst {
case analysis if !isInfoEmpty(analysis.infos.get(originFile)) =>
analysis.relations.products(originFile).iterator.map(_.toPath).toList
}

foundClassFiles.toList.flatten
}
}

private final class MainClassDebugAdapter(
project: Project,
mainClass: ScalaMainClass,
val classPathEntries: Seq[ClassPathEntry],
val evaluationClassLoader: Option[ClassLoader],
env: JdkConfig,
initialState: State,
ioScheduler: Scheduler
) extends BloopDebuggeeRunner(initialState, ioScheduler) {
val javaRuntime = JavaRuntime(env.javaHome.underlying)
def name: String = s"${getClass.getSimpleName}(${project.name}, ${mainClass.`class`})"
def start(state: State): Task[ExitStatus] = {
val workingDir = state.commonOptions.workingPath
Expand All @@ -78,6 +67,9 @@ private final class MainClassDebugAdapter(
private final class TestSuiteDebugAdapter(
projects: Seq[Project],
filters: List[String],
val classPathEntries: Seq[ClassPathEntry],
val javaRuntime: Option[JavaRuntime],
val evaluationClassLoader: Option[ClassLoader],
initialState: State,
ioScheduler: Scheduler
) extends BloopDebuggeeRunner(initialState, ioScheduler) {
Expand All @@ -103,8 +95,13 @@ private final class TestSuiteDebugAdapter(
}
}

private final class AttachRemoteDebugAdapter(initialState: State, ioScheduler: Scheduler)
extends BloopDebuggeeRunner(initialState, ioScheduler) {
private final class AttachRemoteDebugAdapter(
val classPathEntries: Seq[ClassPathEntry],
val javaRuntime: Option[JavaRuntime],
val evaluationClassLoader: Option[ClassLoader],
initialState: State,
ioScheduler: Scheduler
) extends BloopDebuggeeRunner(initialState, ioScheduler) {
override def name: String = s"${getClass.getSimpleName}(${initialState.build.origin})"
override def start(state: State): Task[ExitStatus] = Task(ExitStatus.Ok)
}
Expand All @@ -121,11 +118,22 @@ object BloopDebuggeeRunner {
case Seq(project) =>
project.platform match {
case jvm: Platform.Jvm =>
Right(new MainClassDebugAdapter(project, mainClass, jvm.config, state, ioScheduler))
val classPathEntries = getClassPathEntries(state.build, project)
val evaluationClassLoader = getEvaluationClassLoader(project, state, ioScheduler)
Right(
new MainClassDebugAdapter(
project,
mainClass,
classPathEntries,
evaluationClassLoader,
jvm.config,
state,
ioScheduler
)
)
case platform =>
Left(s"Unsupported platform: ${platform.getClass.getSimpleName}")
}

case projects => Left(s"Multiple projects specified for main class [$mainClass]: $projects")
}
}
Expand All @@ -138,10 +146,157 @@ object BloopDebuggeeRunner {
): Either[String, DebuggeeRunner] = {
projects match {
case Seq() => Left(s"No projects specified for the test suites: [${filters.sorted}]")
case projects => Right(new TestSuiteDebugAdapter(projects, filters, state, ioScheduler))
case Seq(project) if project.platform.isInstanceOf[Platform.Jvm] =>
val Platform.Jvm(config, _, _, _, _, _) = project.platform
val classPathEntries = getClassPathEntries(state.build, project)
val javaRuntime = JavaRuntime(config.javaHome.underlying)
val evaluationClassLoader = getEvaluationClassLoader(project, state, ioScheduler)
Right(
new TestSuiteDebugAdapter(
projects,
filters,
classPathEntries,
javaRuntime,
evaluationClassLoader,
state,
ioScheduler
)
)
case _ =>
Right(
new TestSuiteDebugAdapter(projects, filters, Seq.empty, None, None, state, ioScheduler)
)
}
}

def forAttachRemote(
state: State,
ioScheduler: Scheduler,
projects: Seq[Project]
): DebuggeeRunner = {
projects match {
case Seq(project) if project.platform.isInstanceOf[Platform.Jvm] =>
val Platform.Jvm(config, _, _, _, _, _) = project.platform
val classPathEntries = getClassPathEntries(state.build, project)
val javaRuntime = JavaRuntime(config.javaHome.underlying)
val evaluationClassLoader = getEvaluationClassLoader(project, state, ioScheduler)
new AttachRemoteDebugAdapter(
classPathEntries,
javaRuntime,
evaluationClassLoader,
state,
ioScheduler
)
case _ => new AttachRemoteDebugAdapter(Seq.empty, None, None, state, ioScheduler)
}
}

def forAttachRemote(state: State, ioScheduler: Scheduler): DebuggeeRunner =
new AttachRemoteDebugAdapter(state, ioScheduler)
private def getClassPathEntries(build: Build, project: Project): Seq[ClassPathEntry] = {
val allProjects = getAllDepsRecursively(build, project)
getLibraries(allProjects) ++ getClassDirectories(allProjects)
}

private def getEvaluationClassLoader(
project: Project,
state: State,
ioScheduler: Scheduler
): Option[ClassLoader] = {
project.scalaInstance
.flatMap(getEvaluationClassLoader(_, state.logger, ioScheduler))
}

private def getEvaluationClassLoader(
scalaInstance: ScalaInstance,
logger: Logger,
ioScheduler: Scheduler
): Option[ClassLoader] = {
import ch.epfl.scala.debugadapter.BuildInfo._
import coursier._
val scalaVersion = scalaInstance.version
val module = s"${expressionCompilerName}_$scalaVersion"
val expressionCompilerDep = Dependency(
Module(
Organization(expressionCompilerOrganization),
ModuleName(module)
),
expressionCompilerVersion
)
val resolution = Fetch()
.addDependencies(expressionCompilerDep)
.either()(ioScheduler)

resolution match {
case Left(error) =>
logger.warn(
s"Failed fetching $expressionCompilerOrganization:$module:$expressionCompilerVersion"
)
logger.warn(error.getMessage)
logger.warn(s"Expression evaluation will not work.")
None
case Right(files) =>
val expressionCompilerJar = files
.find(_.getName.startsWith(expressionCompilerName))
.map(_.toURI.toURL)
val evaluationClassLoader =
new URLClassLoader(expressionCompilerJar.toArray, scalaInstance.loader)
Some(evaluationClassLoader)
}
}

private def getAllDepsRecursively(build: Build, project: Project): Seq[Project] = {
@tailrec def tailApply(projects: Seq[Project], acc: Set[Project]): Seq[Project] = {
if (projects.isEmpty) acc.toSeq
else {
val dependencies = projects
.flatMap(_.dependencies)
.flatMap(build.getProjectFor)
.distinct
.filterNot(acc.contains)
tailApply(dependencies, acc ++ dependencies)
}
}
tailApply(Seq(project), Set(project))
}

private def getLibraries(allProjects: Seq[Project]): Seq[ClassPathEntry] = {
allProjects
.flatMap(_.resolution)
.flatMap(_.modules)
.distinct
.flatMap { module =>
for {
classJar <- module.artifacts.find(_.classifier.isEmpty)
sourceJar <- module.artifacts.find(_.classifier.contains("sources"))
} yield {
val sourceEntry = SourceJar(sourceJar.path)
ClassPathEntry(classJar.path, Seq(sourceEntry))
}
}
.distinct
}

private def getClassDirectories(allProjects: Seq[Project]): Seq[ClassPathEntry] = {
allProjects.map { project =>
val sourceBuffer = mutable.Buffer.empty[SourceEntry]
for (sourcePath <- project.sources) {
if (sourcePath.isDirectory) {
sourceBuffer += SourceDirectory(sourcePath.underlying)
} else {
sourceBuffer += StandaloneSourceFile(
sourcePath.underlying,
sourcePath.underlying.getFileName.toString
)
}
}
for (glob <- project.sourcesGlobs) {
glob.walkThrough { file =>
sourceBuffer += StandaloneSourceFile(
file.underlying,
file.toRelative(glob.directory).toString
)
}
}
ClassPathEntry(project.out.underlying, sourceBuffer.toSeq)
}
}
}
48 changes: 9 additions & 39 deletions frontend/src/main/scala/bloop/data/Project.scala
Original file line number Diff line number Diff line change
@@ -1,34 +1,20 @@
package bloop.data

import java.net.URI

import bloop.io.AbsolutePath
import bloop.logging.{DebugFilter, Logger}
import bloop.ScalaInstance
import bloop.bsp.ProjectUris
import bloop.config.Config
import bloop.engine.{Build, Dag}
import bloop.engine.caches.SemanticDBCache
import bloop.config.{Config, ConfigCodecs}
import bloop.engine.Dag
import bloop.engine.tasks.toolchains.{JvmToolchain, ScalaJsToolchain, ScalaNativeToolchain}
import bloop.io.ByteHasher
import java.nio.charset.StandardCharsets
import java.nio.file.attribute.BasicFileAttributes
import java.nio.file.PathMatcher
import java.nio.file.Path
import java.nio.file.FileVisitor
import java.nio.file.FileVisitResult
import java.nio.file.Files
import java.nio.file.FileVisitOption
import java.nio.file.SimpleFileVisitor

import scala.util.Try
import scala.collection.mutable
import bloop.io.{AbsolutePath, ByteHasher}
import bloop.logging.{DebugFilter, Logger}
import ch.epfl.scala.{bsp => Bsp}
import monix.eval.Task
import xsbti.compile.{ClasspathOptions, CompileOrder}
import bloop.config.ConfigCodecs

import java.nio.charset.StandardCharsets
import scala.collection.mutable
import scala.util.Try
import scala.util.control.NonFatal
import monix.eval.Task

final case class Project(
name: String,
Expand Down Expand Up @@ -92,23 +78,7 @@ final case class Project(
def allSourceFilesAndDirectories: Task[List[AbsolutePath]] = Task {
val buf = mutable.ListBuffer.empty[AbsolutePath]
buf ++= sources
sourcesGlobs.foreach { glob =>
if (Files.isDirectory(glob.directory.underlying)) {
Files.walkFileTree(
glob.directory.underlying,
java.util.EnumSet.of(FileVisitOption.FOLLOW_LINKS),
glob.walkDepth,
new SimpleFileVisitor[Path] {
override def visitFile(file: Path, attrs: BasicFileAttributes): FileVisitResult = {
if (glob.matches(file)) {
buf += AbsolutePath(file)
}
FileVisitResult.CONTINUE
}
}
)
}
}
for (glob <- sourcesGlobs) glob.walkThrough(buf += _)
buf.result()
}

Expand Down
Loading

0 comments on commit 0c26af4

Please sign in to comment.