mirror of https://github.com/sbt/sbt.git
Add new ClassLoaderCache implementation
This commit adds a new ClassLoaderCache that builds on the ClassLoaderCache that is present in zinc (and can be used to build an instance of the zinc ClassLoaderCache to preserve compatibility). It differs from the zinc classloader cache that it does not use direct SoftReferences to classloaders. Instead, we create a wrapper loader that can't load any classes and just delegates to its parent. This allows us to add a thread that reaps the soft reference to the wrapper loader. Crucially, we add a custom SoftReference class that has a strong reference to the underlying classloader. This allows us to call close on the strong reference. The one issue with this approach is that we can't rescue the jvm from crashing with an OOM: metaspace because the jvm doesn't give us a chance to close and dereference the underlying classloaders before it crashes. It WILL collect classloaders under normal memory pressure, just not metaspace pressure. To fix this, I check if the MaxMetaspaceSize is set via an MxBean and, if it is, we fill the cache with regular soft references. We are going to change the bash script to not set -XX:MaxMetaspaceSize by default so most builds should probably end up correctly closing the classloaders after this change. But we should break existing builds that set MaxMetaspaceSize but don't crash. As part of this commit, I audited all of the places where we were instantiating ClassLoaderCache instances and instead pass in the state's ClassLoaderCache instance. This reduces the total number of classloaders created.
This commit is contained in:
parent
20f6d22439
commit
03bf539ce9
|
|
@ -9,7 +9,8 @@ package sbt
|
|||
|
||||
import java.io.File
|
||||
|
||||
import sbt.internal.inc.classpath.ClassLoaderCache
|
||||
import sbt.internal.inc.classpath.{ ClassLoaderCache => IncClassLoaderCache }
|
||||
import sbt.internal.classpath.ClassLoaderCache
|
||||
import sbt.internal.server.ServerHandler
|
||||
import sbt.internal.util.AttributeKey
|
||||
import sbt.librarymanagement.ModuleID
|
||||
|
|
@ -82,11 +83,16 @@ object BasicKeys {
|
|||
"True if commands are currently being entered from an interactive environment.",
|
||||
10
|
||||
)
|
||||
private[sbt] val classLoaderCache = AttributeKey[ClassLoaderCache](
|
||||
private[sbt] val classLoaderCache = AttributeKey[IncClassLoaderCache](
|
||||
"class-loader-cache",
|
||||
"Caches class loaders based on the classpath entries and last modified times.",
|
||||
10
|
||||
)
|
||||
private[sbt] val extendedClassLoaderCache = AttributeKey[ClassLoaderCache](
|
||||
"extended-class-loader-cache",
|
||||
"Caches class loaders based on the classpath entries and last modified times.",
|
||||
10
|
||||
)
|
||||
private[sbt] val OnFailureStack = AttributeKey[List[Option[Exec]]](
|
||||
"on-failure-stack",
|
||||
"Stack that remembers on-failure handlers.",
|
||||
|
|
|
|||
|
|
@ -9,6 +9,9 @@ package sbt
|
|||
|
||||
import java.io.File
|
||||
import java.util.concurrent.Callable
|
||||
|
||||
import sbt.internal.classpath.ClassLoaderCache
|
||||
import sbt.internal.inc.classpath.{ ClassLoaderCache => IncClassLoaderCache }
|
||||
import sbt.util.Logger
|
||||
import sbt.internal.util.{
|
||||
AttributeKey,
|
||||
|
|
@ -19,7 +22,6 @@ import sbt.internal.util.{
|
|||
GlobalLogging
|
||||
}
|
||||
import sbt.internal.util.complete.{ HistoryCommands, Parser }
|
||||
import sbt.internal.inc.classpath.ClassLoaderCache
|
||||
|
||||
/**
|
||||
* Data structure representing all command execution information.
|
||||
|
|
@ -193,7 +195,7 @@ trait StateOps extends Any {
|
|||
def setInteractive(flag: Boolean): State
|
||||
|
||||
/** Get the class loader cache for the application.*/
|
||||
def classLoaderCache: ClassLoaderCache
|
||||
def classLoaderCache: IncClassLoaderCache
|
||||
|
||||
/** Create and register a class loader cache. This should be called once at the application entry-point.*/
|
||||
def initializeClassLoaderCache: State
|
||||
|
|
@ -221,6 +223,7 @@ object State {
|
|||
|
||||
/**
|
||||
* Provides a list of recently executed commands. The commands are stored as processed instead of as entered by the user.
|
||||
*
|
||||
* @param executed the list of the most recently executed commands, with the most recent command first.
|
||||
* @param maxSize the maximum number of commands to keep, or 0 to keep an unlimited number.
|
||||
*/
|
||||
|
|
@ -334,11 +337,18 @@ object State {
|
|||
def interactive = getBoolean(s, BasicKeys.interactive, false)
|
||||
def setInteractive(i: Boolean) = s.put(BasicKeys.interactive, i)
|
||||
|
||||
def classLoaderCache: ClassLoaderCache =
|
||||
s get BasicKeys.classLoaderCache getOrElse newClassLoaderCache
|
||||
def initializeClassLoaderCache = s.put(BasicKeys.classLoaderCache, newClassLoaderCache)
|
||||
def classLoaderCache: IncClassLoaderCache =
|
||||
s get BasicKeys.classLoaderCache getOrElse (throw new IllegalStateException(
|
||||
"Tried to get classloader cache for uninitialized state."
|
||||
))
|
||||
def initializeClassLoaderCache: State = {
|
||||
s.get(BasicKeys.extendedClassLoaderCache).foreach(_.close())
|
||||
val cache = newClassLoaderCache
|
||||
s.put(BasicKeys.extendedClassLoaderCache, cache)
|
||||
.put(BasicKeys.classLoaderCache, new IncClassLoaderCache(cache))
|
||||
}
|
||||
private[this] def newClassLoaderCache =
|
||||
new ClassLoaderCache(s.configuration.provider.scalaProvider.launcher.topLoader)
|
||||
new ClassLoaderCache(s.configuration.provider.scalaProvider)
|
||||
}
|
||||
|
||||
import ExceptionCategory._
|
||||
|
|
|
|||
|
|
@ -0,0 +1,212 @@
|
|||
package sbt.internal.classpath
|
||||
|
||||
import java.io.File
|
||||
import java.lang.management.ManagementFactory
|
||||
import java.lang.ref.{ Reference, ReferenceQueue, SoftReference }
|
||||
import java.net.URLClassLoader
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import sbt.internal.inc.classpath.{
|
||||
AbstractClassLoaderCache,
|
||||
ClassLoaderCache => IncClassLoaderCache
|
||||
}
|
||||
import sbt.internal.inc.{ AnalyzingCompiler, ZincUtil }
|
||||
import sbt.io.IO
|
||||
import xsbti.ScalaProvider
|
||||
import xsbti.compile.{ ClasspathOptions, ScalaInstance }
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
private object ClassLoaderCache {
|
||||
private def threadID = new AtomicInteger(0)
|
||||
}
|
||||
private[sbt] class ClassLoaderCache(
|
||||
override val commonParent: ClassLoader,
|
||||
private val miniProvider: Option[(File, ClassLoader)]
|
||||
) extends AbstractClassLoaderCache {
|
||||
def this(commonParent: ClassLoader) = this(commonParent, None)
|
||||
def this(scalaProvider: ScalaProvider) =
|
||||
this(scalaProvider.launcher.topLoader, {
|
||||
scalaProvider.jars.find(_.getName == "scala-library.jar").flatMap { lib =>
|
||||
val clazz = scalaProvider.getClass
|
||||
try {
|
||||
val loader = clazz.getDeclaredMethod("libraryLoaderOnly").invoke(scalaProvider)
|
||||
Some(lib -> loader.asInstanceOf[ClassLoader])
|
||||
} catch { case NonFatal(_) => None }
|
||||
}
|
||||
})
|
||||
private val scalaProviderKey = miniProvider.map {
|
||||
case (f, cl) =>
|
||||
new Key((f -> IO.getModifiedTimeOrZero(f)) :: Nil, commonParent) {
|
||||
override def toClassLoader: ClassLoader = cl
|
||||
}
|
||||
}
|
||||
private class Key(val fileStamps: Seq[(File, Long)], val parent: ClassLoader) {
|
||||
def this(files: List[File]) =
|
||||
this(files.map(f => f -> IO.getModifiedTimeOrZero(f)), commonParent)
|
||||
lazy val files: Seq[File] = fileStamps.map(_._1)
|
||||
lazy val maxStamp: Long = fileStamps.maxBy(_._2)._2
|
||||
class CachedClassLoader
|
||||
extends URLClassLoader(fileStamps.map(_._1.toURI.toURL).toArray, parent) {
|
||||
override def toString: String =
|
||||
s"CachedClassloader {\n parent: $parent\n urls:\n" + getURLs.mkString(" ", "\n", "\n}")
|
||||
}
|
||||
def toClassLoader: ClassLoader = new CachedClassLoader
|
||||
override def equals(o: Any): Boolean = o match {
|
||||
case that: Key => this.fileStamps == that.fileStamps && this.parent == that.parent
|
||||
}
|
||||
override def hashCode(): Int = (fileStamps.hashCode * 31) ^ parent.hashCode
|
||||
override def toString: String = s"Key(${fileStamps mkString ","}, $parent)"
|
||||
}
|
||||
private[this] val delegate =
|
||||
new java.util.concurrent.ConcurrentHashMap[Key, Reference[ClassLoader]]()
|
||||
private[this] val referenceQueue = new ReferenceQueue[ClassLoader]
|
||||
|
||||
private[this] def closeExpiredLoaders(): Unit = {
|
||||
val toClose = lock.synchronized(delegate.asScala.groupBy(_._1.files.toSet).flatMap {
|
||||
case (_, pairs) if pairs.size > 1 =>
|
||||
val max = pairs.maxBy(_._1.maxStamp)._1
|
||||
pairs.filterNot(_._1 == max).flatMap {
|
||||
case (k, v) =>
|
||||
delegate.remove(k)
|
||||
Option(v.get)
|
||||
}
|
||||
case _ => Nil
|
||||
})
|
||||
toClose.foreach(close)
|
||||
}
|
||||
private[this] class CleanupThread(private[this] val id: Int)
|
||||
extends Thread(s"classloader-cache-cleanup-$id") {
|
||||
setDaemon(true)
|
||||
start()
|
||||
@tailrec
|
||||
override final def run(): Unit = {
|
||||
val stop = try {
|
||||
referenceQueue.remove(1000) match {
|
||||
case ClassLoaderReference(key, classLoader) =>
|
||||
close(classLoader)
|
||||
delegate.remove(key)
|
||||
case _ =>
|
||||
}
|
||||
closeExpiredLoaders()
|
||||
false
|
||||
} catch {
|
||||
case _: InterruptedException => true
|
||||
}
|
||||
if (!stop) run()
|
||||
}
|
||||
}
|
||||
|
||||
private[this] val metaspaceIsLimited =
|
||||
ManagementFactory.getMemoryPoolMXBeans.asScala
|
||||
.exists(b => (b.getName == "Metaspace") && (b.getUsage.getMax > 0))
|
||||
private[this] val mkReference: (Key, ClassLoader) => Reference[ClassLoader] =
|
||||
if (metaspaceIsLimited)(_, cl) => new SoftReference(cl, referenceQueue)
|
||||
else ClassLoaderReference.apply
|
||||
private[this] val cleanupThread = new CleanupThread(ClassLoaderCache.threadID.getAndIncrement())
|
||||
private[this] val lock = new Object
|
||||
private class WrappedLoader(parent: ClassLoader) extends URLClassLoader(Array.empty, parent) {
|
||||
override def toString: String = s"WrappedLoader($parent)"
|
||||
}
|
||||
private def close(classLoader: ClassLoader): Unit = classLoader match {
|
||||
case a: AutoCloseable => a.close()
|
||||
case _ =>
|
||||
}
|
||||
private case class ClassLoaderReference(key: Key, classLoader: ClassLoader)
|
||||
extends SoftReference[ClassLoader](
|
||||
new WrappedLoader(classLoader),
|
||||
referenceQueue
|
||||
)
|
||||
def apply(
|
||||
files: List[(File, Long)],
|
||||
parent: ClassLoader,
|
||||
mkLoader: () => ClassLoader
|
||||
): ClassLoader = {
|
||||
val key = new Key(files, parent)
|
||||
get(key, mkLoader)
|
||||
}
|
||||
override def apply(files: List[File]): ClassLoader = {
|
||||
val key = new Key(files)
|
||||
get(key, () => key.toClassLoader)
|
||||
}
|
||||
override def cachedCustomClassloader(
|
||||
files: List[File],
|
||||
mkLoader: () => ClassLoader
|
||||
): ClassLoader = {
|
||||
val key = new Key(files)
|
||||
get(key, mkLoader)
|
||||
}
|
||||
private[this] def get(key: Key, f: () => ClassLoader): ClassLoader = {
|
||||
scalaProviderKey match {
|
||||
case Some(k) if k == key => k.toClassLoader
|
||||
case _ =>
|
||||
def addLoader(): ClassLoader = {
|
||||
val ref = mkReference(key, f())
|
||||
val loader = ref.get
|
||||
delegate.put(key, ref)
|
||||
closeExpiredLoaders()
|
||||
loader
|
||||
}
|
||||
lock.synchronized {
|
||||
delegate.get(key) match {
|
||||
case null => addLoader()
|
||||
case ref =>
|
||||
ref.get match {
|
||||
case null => addLoader()
|
||||
case l => l
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
private def clear(lock: Object): Unit = {
|
||||
delegate.forEach {
|
||||
case (_, ClassLoaderReference(_, classLoader)) => close(classLoader)
|
||||
case (_, r: Reference[ClassLoader]) =>
|
||||
r.get match {
|
||||
case null =>
|
||||
case classLoader => close(classLoader)
|
||||
}
|
||||
case (_, _) =>
|
||||
}
|
||||
delegate.clear()
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears any ClassLoader instances from the internal cache and closes them. Calling this
|
||||
* method will not stop the cleanup thread. Call [[close]] to fully clean up this cache.
|
||||
*/
|
||||
def clear(): Unit = lock.synchronized(clear(lock))
|
||||
|
||||
/**
|
||||
* Completely shuts down this cache. It stops the background thread for cleaning up classloaders
|
||||
*
|
||||
* Clears any ClassLoader instances from the internal cache and closes them. It also
|
||||
* method will not stop the cleanup thread. Call [[close]] to fully clean up this cache.
|
||||
*/
|
||||
override def close(): Unit = lock.synchronized {
|
||||
cleanupThread.interrupt()
|
||||
cleanupThread.join()
|
||||
clear(lock)
|
||||
}
|
||||
}
|
||||
|
||||
private[sbt] object AlternativeZincUtil {
|
||||
def scalaCompiler(
|
||||
scalaInstance: ScalaInstance,
|
||||
compilerBridgeJar: File,
|
||||
classpathOptions: ClasspathOptions,
|
||||
classLoaderCache: Option[IncClassLoaderCache]
|
||||
): AnalyzingCompiler = {
|
||||
val bridgeProvider = ZincUtil.constantBridgeProvider(scalaInstance, compilerBridgeJar)
|
||||
new AnalyzingCompiler(
|
||||
scalaInstance,
|
||||
bridgeProvider,
|
||||
classpathOptions,
|
||||
_ => (),
|
||||
classLoaderCache
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
@ -11,50 +11,38 @@ import java.io.File
|
|||
import java.nio.file.Files
|
||||
|
||||
import org.scalatest.{ FlatSpec, Matchers }
|
||||
import sbt.internal.classpath.ClassLoaderCache
|
||||
import sbt.io.IO
|
||||
|
||||
object ClassLoaderCacheTest {
|
||||
private val initLoader = this.getClass.getClassLoader
|
||||
implicit class CacheOps(val c: ClassLoaderCache) {
|
||||
def get(classpath: Seq[File]): ClassLoader =
|
||||
c.get((classpath, initLoader, Map.empty, new File("/dev/null")))
|
||||
def get(classpath: Seq[File]): ClassLoader = c(classpath.toList)
|
||||
}
|
||||
}
|
||||
class ClassLoaderCacheTest extends FlatSpec with Matchers {
|
||||
import ClassLoaderCacheTest._
|
||||
def withCache[R](size: Int)(f: CacheOps => R): R = {
|
||||
val cache = ClassLoaderCache(size)
|
||||
try f(new CacheOps(cache))
|
||||
private def withCache[R](f: ClassLoaderCache => R): R = {
|
||||
val cache = new ClassLoaderCache(ClassLoader.getSystemClassLoader)
|
||||
try f(cache)
|
||||
finally cache.close()
|
||||
}
|
||||
"ClassLoaderCache.get" should "make a new loader when full" in withCache(0) { cache =>
|
||||
"ClassLoaderCache" should "make a new loader when full" in withCache { cache =>
|
||||
val classPath = Seq.empty[File]
|
||||
val firstLoader = cache.get(classPath)
|
||||
cache.clear()
|
||||
val secondLoader = cache.get(classPath)
|
||||
assert(firstLoader != secondLoader)
|
||||
}
|
||||
it should "not make a new loader when it already exists" in withCache(1) { cache =>
|
||||
it should "not make a new loader when it already exists" in withCache { cache =>
|
||||
val classPath = Seq.empty[File]
|
||||
val firstLoader = cache.get(classPath)
|
||||
val secondLoader = cache.get(classPath)
|
||||
assert(firstLoader == secondLoader)
|
||||
}
|
||||
it should "evict loaders" in withCache(2) { cache =>
|
||||
val firstClassPath = Seq.empty[File]
|
||||
val secondClassPath = new File("foo") :: Nil
|
||||
val thirdClassPath = new File("foo") :: new File("bar") :: Nil
|
||||
val firstLoader = cache.get(firstClassPath)
|
||||
val secondLoader = cache.get(secondClassPath)
|
||||
val thirdLoader = cache.get(thirdClassPath)
|
||||
assert(cache.get(thirdClassPath) == thirdLoader)
|
||||
assert(cache.get(secondClassPath) == secondLoader)
|
||||
assert(cache.get(firstClassPath) != firstLoader)
|
||||
assert(cache.get(thirdClassPath) != thirdLoader)
|
||||
}
|
||||
"Snapshots" should "be invalidated" in IO.withTemporaryDirectory { dir =>
|
||||
val snapshotJar = Files.createFile(dir.toPath.resolve("foo-SNAPSHOT.jar")).toFile
|
||||
val regularJar = Files.createFile(dir.toPath.resolve("regular.jar")).toFile
|
||||
withCache(1) { cache =>
|
||||
withCache { cache =>
|
||||
val jarClassPath = snapshotJar :: regularJar :: Nil
|
||||
val initLoader = cache.get(jarClassPath)
|
||||
IO.setModifiedTimeOrFalse(snapshotJar, System.currentTimeMillis + 5000L)
|
||||
|
|
@ -28,6 +28,7 @@ import sbt.Project.{
|
|||
import sbt.Scope.{ GlobalScope, ThisScope, fillTaskAxis }
|
||||
import sbt.internal.CommandStrings.ExportStream
|
||||
import sbt.internal._
|
||||
import sbt.internal.classpath.AlternativeZincUtil
|
||||
import sbt.internal.inc.JavaInterfaceUtil._
|
||||
import sbt.internal.inc.classpath.ClasspathFilter
|
||||
import sbt.internal.inc.{ ZincLmUtil, ZincUtil }
|
||||
|
|
@ -550,10 +551,11 @@ object Defaults extends BuildCommon {
|
|||
val scalac =
|
||||
scalaCompilerBridgeBinaryJar.value match {
|
||||
case Some(jar) =>
|
||||
ZincUtil.scalaCompiler(
|
||||
AlternativeZincUtil.scalaCompiler(
|
||||
scalaInstance = scalaInstance.value,
|
||||
classpathOptions = classpathOptions.value,
|
||||
compilerBridgeJar = jar
|
||||
compilerBridgeJar = jar,
|
||||
classLoaderCache = st.get(BasicKeys.classLoaderCache)
|
||||
)
|
||||
case _ =>
|
||||
ZincLmUtil.scalaCompiler(
|
||||
|
|
@ -565,6 +567,7 @@ object Defaults extends BuildCommon {
|
|||
dependencyResolution = dr,
|
||||
compilerBridgeSource = scalaCompilerBridgeSource.value,
|
||||
scalaJarsTarget = zincDir,
|
||||
classLoaderCache = st.get(BasicKeys.classLoaderCache),
|
||||
log = streams.value.log
|
||||
)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -864,7 +864,9 @@ object BuiltinCommands {
|
|||
|
||||
def clearCaches: Command = {
|
||||
val help = Help.more(ClearCaches, ClearCachesDetailed)
|
||||
Command.command(ClearCaches, help)(registerGlobalCaches _ andThen registerCompilerCache)
|
||||
Command.command(ClearCaches, help)(
|
||||
registerGlobalCaches _ andThen registerCompilerCache andThen (_.initializeClassLoaderCache)
|
||||
)
|
||||
}
|
||||
|
||||
def shell: Command = Command.command(Shell, Help.more(Shell, ShellDetailed)) { s0 =>
|
||||
|
|
|
|||
|
|
@ -8,9 +8,10 @@
|
|||
package sbt
|
||||
package internal
|
||||
|
||||
import sbt.util.Logger
|
||||
import sbt.internal.classpath.AlternativeZincUtil
|
||||
import sbt.internal.inc.{ ScalaInstance, ZincLmUtil }
|
||||
import sbt.internal.util.JLine
|
||||
import sbt.internal.inc.{ ScalaInstance, ZincLmUtil, ZincUtil }
|
||||
import sbt.util.Logger
|
||||
import xsbti.compile.ClasspathOptionsUtil
|
||||
|
||||
object ConsoleProject {
|
||||
|
|
@ -35,10 +36,11 @@ object ConsoleProject {
|
|||
val launcher = app.provider.scalaProvider.launcher
|
||||
val compiler = scalaCompilerBridgeBinaryJar match {
|
||||
case Some(jar) =>
|
||||
ZincUtil.scalaCompiler(
|
||||
AlternativeZincUtil.scalaCompiler(
|
||||
scalaInstance = scalaInstance,
|
||||
classpathOptions = ClasspathOptionsUtil.repl,
|
||||
compilerBridgeJar = jar
|
||||
compilerBridgeJar = jar,
|
||||
classLoaderCache = state1.get(BasicKeys.classLoaderCache)
|
||||
)
|
||||
case None =>
|
||||
ZincLmUtil.scalaCompiler(
|
||||
|
|
@ -50,6 +52,7 @@ object ConsoleProject {
|
|||
dependencyResolution = dependencyResolution,
|
||||
compilerBridgeSource = extracted.get(Keys.scalaCompilerBridgeSource),
|
||||
scalaJarsTarget = zincDir,
|
||||
classLoaderCache = state1.get(BasicKeys.classLoaderCache),
|
||||
log = log
|
||||
)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -87,6 +87,7 @@ private[sbt] object Load {
|
|||
dependencyResolution = dependencyResolution,
|
||||
compilerBridgeSource = ZincLmUtil.getDefaultBridgeModule(scalaProvider.version),
|
||||
scalaJarsTarget = zincDir,
|
||||
state.get(BasicKeys.classLoaderCache),
|
||||
log = log
|
||||
)
|
||||
val compilers = ZincUtil.compilers(
|
||||
|
|
@ -374,8 +375,9 @@ private[sbt] object Load {
|
|||
val projectSettings = build.defined flatMap {
|
||||
case (id, project) =>
|
||||
val ref = ProjectRef(uri, id)
|
||||
val defineConfig: Seq[Setting[_]] = for (c <- project.configurations)
|
||||
yield ((configuration in (ref, ConfigKey(c.name))) :== c)
|
||||
val defineConfig: Seq[Setting[_]] =
|
||||
for (c <- project.configurations)
|
||||
yield ((configuration in (ref, ConfigKey(c.name))) :== c)
|
||||
val builtin: Seq[Setting[_]] =
|
||||
(thisProject :== project) +: (thisProjectRef :== ref) +: defineConfig
|
||||
val settings = builtin ++ project.settings ++ injectSettings.project
|
||||
|
|
|
|||
|
|
@ -8,10 +8,9 @@
|
|||
package sbt.internal.inc
|
||||
|
||||
import java.io.File
|
||||
import java.net.URLClassLoader
|
||||
|
||||
import sbt.librarymanagement.{ DependencyResolution, ModuleID }
|
||||
import sbt.internal.inc.classpath.ClassLoaderCache
|
||||
import sbt.librarymanagement.{ DependencyResolution, ModuleID }
|
||||
import xsbti._
|
||||
import xsbti.compile._
|
||||
|
||||
|
|
@ -32,6 +31,7 @@ object ZincLmUtil {
|
|||
dependencyResolution: DependencyResolution,
|
||||
compilerBridgeSource: ModuleID,
|
||||
scalaJarsTarget: File,
|
||||
classLoaderCache: Option[ClassLoaderCache],
|
||||
log: Logger
|
||||
): AnalyzingCompiler = {
|
||||
val compilerBridgeProvider = ZincComponentCompiler.interfaceProvider(
|
||||
|
|
@ -40,8 +40,13 @@ object ZincLmUtil {
|
|||
dependencyResolution,
|
||||
scalaJarsTarget,
|
||||
)
|
||||
val loader = Some(new ClassLoaderCache(new URLClassLoader(new Array(0))))
|
||||
new AnalyzingCompiler(scalaInstance, compilerBridgeProvider, classpathOptions, _ => (), loader)
|
||||
new AnalyzingCompiler(
|
||||
scalaInstance,
|
||||
compilerBridgeProvider,
|
||||
classpathOptions,
|
||||
_ => (),
|
||||
classLoaderCache
|
||||
)
|
||||
}
|
||||
|
||||
def getDefaultBridgeModule(scalaVersion: String): ModuleID =
|
||||
|
|
|
|||
Loading…
Reference in New Issue