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:
Ethan Atkins 2019-05-20 18:20:09 -07:00
parent 20f6d22439
commit 03bf539ce9
9 changed files with 273 additions and 42 deletions

View File

@ -9,7 +9,8 @@ package sbt
import java.io.File 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.server.ServerHandler
import sbt.internal.util.AttributeKey import sbt.internal.util.AttributeKey
import sbt.librarymanagement.ModuleID import sbt.librarymanagement.ModuleID
@ -82,11 +83,16 @@ object BasicKeys {
"True if commands are currently being entered from an interactive environment.", "True if commands are currently being entered from an interactive environment.",
10 10
) )
private[sbt] val classLoaderCache = AttributeKey[ClassLoaderCache]( private[sbt] val classLoaderCache = AttributeKey[IncClassLoaderCache](
"class-loader-cache", "class-loader-cache",
"Caches class loaders based on the classpath entries and last modified times.", "Caches class loaders based on the classpath entries and last modified times.",
10 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]]]( private[sbt] val OnFailureStack = AttributeKey[List[Option[Exec]]](
"on-failure-stack", "on-failure-stack",
"Stack that remembers on-failure handlers.", "Stack that remembers on-failure handlers.",

View File

@ -9,6 +9,9 @@ package sbt
import java.io.File import java.io.File
import java.util.concurrent.Callable import java.util.concurrent.Callable
import sbt.internal.classpath.ClassLoaderCache
import sbt.internal.inc.classpath.{ ClassLoaderCache => IncClassLoaderCache }
import sbt.util.Logger import sbt.util.Logger
import sbt.internal.util.{ import sbt.internal.util.{
AttributeKey, AttributeKey,
@ -19,7 +22,6 @@ import sbt.internal.util.{
GlobalLogging GlobalLogging
} }
import sbt.internal.util.complete.{ HistoryCommands, Parser } import sbt.internal.util.complete.{ HistoryCommands, Parser }
import sbt.internal.inc.classpath.ClassLoaderCache
/** /**
* Data structure representing all command execution information. * Data structure representing all command execution information.
@ -193,7 +195,7 @@ trait StateOps extends Any {
def setInteractive(flag: Boolean): State def setInteractive(flag: Boolean): State
/** Get the class loader cache for the application.*/ /** 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.*/ /** Create and register a class loader cache. This should be called once at the application entry-point.*/
def initializeClassLoaderCache: State 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. * 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 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. * @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 interactive = getBoolean(s, BasicKeys.interactive, false)
def setInteractive(i: Boolean) = s.put(BasicKeys.interactive, i) def setInteractive(i: Boolean) = s.put(BasicKeys.interactive, i)
def classLoaderCache: ClassLoaderCache = def classLoaderCache: IncClassLoaderCache =
s get BasicKeys.classLoaderCache getOrElse newClassLoaderCache s get BasicKeys.classLoaderCache getOrElse (throw new IllegalStateException(
def initializeClassLoaderCache = s.put(BasicKeys.classLoaderCache, newClassLoaderCache) "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 = private[this] def newClassLoaderCache =
new ClassLoaderCache(s.configuration.provider.scalaProvider.launcher.topLoader) new ClassLoaderCache(s.configuration.provider.scalaProvider)
} }
import ExceptionCategory._ import ExceptionCategory._

View File

@ -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
)
}
}

View File

@ -11,50 +11,38 @@ import java.io.File
import java.nio.file.Files import java.nio.file.Files
import org.scalatest.{ FlatSpec, Matchers } import org.scalatest.{ FlatSpec, Matchers }
import sbt.internal.classpath.ClassLoaderCache
import sbt.io.IO import sbt.io.IO
object ClassLoaderCacheTest { object ClassLoaderCacheTest {
private val initLoader = this.getClass.getClassLoader
implicit class CacheOps(val c: ClassLoaderCache) { implicit class CacheOps(val c: ClassLoaderCache) {
def get(classpath: Seq[File]): ClassLoader = def get(classpath: Seq[File]): ClassLoader = c(classpath.toList)
c.get((classpath, initLoader, Map.empty, new File("/dev/null")))
} }
} }
class ClassLoaderCacheTest extends FlatSpec with Matchers { class ClassLoaderCacheTest extends FlatSpec with Matchers {
import ClassLoaderCacheTest._ import ClassLoaderCacheTest._
def withCache[R](size: Int)(f: CacheOps => R): R = { private def withCache[R](f: ClassLoaderCache => R): R = {
val cache = ClassLoaderCache(size) val cache = new ClassLoaderCache(ClassLoader.getSystemClassLoader)
try f(new CacheOps(cache)) try f(cache)
finally cache.close() 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 classPath = Seq.empty[File]
val firstLoader = cache.get(classPath) val firstLoader = cache.get(classPath)
cache.clear()
val secondLoader = cache.get(classPath) val secondLoader = cache.get(classPath)
assert(firstLoader != secondLoader) 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 classPath = Seq.empty[File]
val firstLoader = cache.get(classPath) val firstLoader = cache.get(classPath)
val secondLoader = cache.get(classPath) val secondLoader = cache.get(classPath)
assert(firstLoader == secondLoader) 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 => "Snapshots" should "be invalidated" in IO.withTemporaryDirectory { dir =>
val snapshotJar = Files.createFile(dir.toPath.resolve("foo-SNAPSHOT.jar")).toFile val snapshotJar = Files.createFile(dir.toPath.resolve("foo-SNAPSHOT.jar")).toFile
val regularJar = Files.createFile(dir.toPath.resolve("regular.jar")).toFile val regularJar = Files.createFile(dir.toPath.resolve("regular.jar")).toFile
withCache(1) { cache => withCache { cache =>
val jarClassPath = snapshotJar :: regularJar :: Nil val jarClassPath = snapshotJar :: regularJar :: Nil
val initLoader = cache.get(jarClassPath) val initLoader = cache.get(jarClassPath)
IO.setModifiedTimeOrFalse(snapshotJar, System.currentTimeMillis + 5000L) IO.setModifiedTimeOrFalse(snapshotJar, System.currentTimeMillis + 5000L)

View File

@ -28,6 +28,7 @@ import sbt.Project.{
import sbt.Scope.{ GlobalScope, ThisScope, fillTaskAxis } import sbt.Scope.{ GlobalScope, ThisScope, fillTaskAxis }
import sbt.internal.CommandStrings.ExportStream import sbt.internal.CommandStrings.ExportStream
import sbt.internal._ import sbt.internal._
import sbt.internal.classpath.AlternativeZincUtil
import sbt.internal.inc.JavaInterfaceUtil._ import sbt.internal.inc.JavaInterfaceUtil._
import sbt.internal.inc.classpath.ClasspathFilter import sbt.internal.inc.classpath.ClasspathFilter
import sbt.internal.inc.{ ZincLmUtil, ZincUtil } import sbt.internal.inc.{ ZincLmUtil, ZincUtil }
@ -550,10 +551,11 @@ object Defaults extends BuildCommon {
val scalac = val scalac =
scalaCompilerBridgeBinaryJar.value match { scalaCompilerBridgeBinaryJar.value match {
case Some(jar) => case Some(jar) =>
ZincUtil.scalaCompiler( AlternativeZincUtil.scalaCompiler(
scalaInstance = scalaInstance.value, scalaInstance = scalaInstance.value,
classpathOptions = classpathOptions.value, classpathOptions = classpathOptions.value,
compilerBridgeJar = jar compilerBridgeJar = jar,
classLoaderCache = st.get(BasicKeys.classLoaderCache)
) )
case _ => case _ =>
ZincLmUtil.scalaCompiler( ZincLmUtil.scalaCompiler(
@ -565,6 +567,7 @@ object Defaults extends BuildCommon {
dependencyResolution = dr, dependencyResolution = dr,
compilerBridgeSource = scalaCompilerBridgeSource.value, compilerBridgeSource = scalaCompilerBridgeSource.value,
scalaJarsTarget = zincDir, scalaJarsTarget = zincDir,
classLoaderCache = st.get(BasicKeys.classLoaderCache),
log = streams.value.log log = streams.value.log
) )
} }

View File

@ -864,7 +864,9 @@ object BuiltinCommands {
def clearCaches: Command = { def clearCaches: Command = {
val help = Help.more(ClearCaches, ClearCachesDetailed) 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 => def shell: Command = Command.command(Shell, Help.more(Shell, ShellDetailed)) { s0 =>

View File

@ -8,9 +8,10 @@
package sbt package sbt
package internal 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.util.JLine
import sbt.internal.inc.{ ScalaInstance, ZincLmUtil, ZincUtil } import sbt.util.Logger
import xsbti.compile.ClasspathOptionsUtil import xsbti.compile.ClasspathOptionsUtil
object ConsoleProject { object ConsoleProject {
@ -35,10 +36,11 @@ object ConsoleProject {
val launcher = app.provider.scalaProvider.launcher val launcher = app.provider.scalaProvider.launcher
val compiler = scalaCompilerBridgeBinaryJar match { val compiler = scalaCompilerBridgeBinaryJar match {
case Some(jar) => case Some(jar) =>
ZincUtil.scalaCompiler( AlternativeZincUtil.scalaCompiler(
scalaInstance = scalaInstance, scalaInstance = scalaInstance,
classpathOptions = ClasspathOptionsUtil.repl, classpathOptions = ClasspathOptionsUtil.repl,
compilerBridgeJar = jar compilerBridgeJar = jar,
classLoaderCache = state1.get(BasicKeys.classLoaderCache)
) )
case None => case None =>
ZincLmUtil.scalaCompiler( ZincLmUtil.scalaCompiler(
@ -50,6 +52,7 @@ object ConsoleProject {
dependencyResolution = dependencyResolution, dependencyResolution = dependencyResolution,
compilerBridgeSource = extracted.get(Keys.scalaCompilerBridgeSource), compilerBridgeSource = extracted.get(Keys.scalaCompilerBridgeSource),
scalaJarsTarget = zincDir, scalaJarsTarget = zincDir,
classLoaderCache = state1.get(BasicKeys.classLoaderCache),
log = log log = log
) )
} }

View File

@ -87,6 +87,7 @@ private[sbt] object Load {
dependencyResolution = dependencyResolution, dependencyResolution = dependencyResolution,
compilerBridgeSource = ZincLmUtil.getDefaultBridgeModule(scalaProvider.version), compilerBridgeSource = ZincLmUtil.getDefaultBridgeModule(scalaProvider.version),
scalaJarsTarget = zincDir, scalaJarsTarget = zincDir,
state.get(BasicKeys.classLoaderCache),
log = log log = log
) )
val compilers = ZincUtil.compilers( val compilers = ZincUtil.compilers(
@ -374,7 +375,8 @@ private[sbt] object Load {
val projectSettings = build.defined flatMap { val projectSettings = build.defined flatMap {
case (id, project) => case (id, project) =>
val ref = ProjectRef(uri, id) val ref = ProjectRef(uri, id)
val defineConfig: Seq[Setting[_]] = for (c <- project.configurations) val defineConfig: Seq[Setting[_]] =
for (c <- project.configurations)
yield ((configuration in (ref, ConfigKey(c.name))) :== c) yield ((configuration in (ref, ConfigKey(c.name))) :== c)
val builtin: Seq[Setting[_]] = val builtin: Seq[Setting[_]] =
(thisProject :== project) +: (thisProjectRef :== ref) +: defineConfig (thisProject :== project) +: (thisProjectRef :== ref) +: defineConfig

View File

@ -8,10 +8,9 @@
package sbt.internal.inc package sbt.internal.inc
import java.io.File import java.io.File
import java.net.URLClassLoader
import sbt.librarymanagement.{ DependencyResolution, ModuleID }
import sbt.internal.inc.classpath.ClassLoaderCache import sbt.internal.inc.classpath.ClassLoaderCache
import sbt.librarymanagement.{ DependencyResolution, ModuleID }
import xsbti._ import xsbti._
import xsbti.compile._ import xsbti.compile._
@ -32,6 +31,7 @@ object ZincLmUtil {
dependencyResolution: DependencyResolution, dependencyResolution: DependencyResolution,
compilerBridgeSource: ModuleID, compilerBridgeSource: ModuleID,
scalaJarsTarget: File, scalaJarsTarget: File,
classLoaderCache: Option[ClassLoaderCache],
log: Logger log: Logger
): AnalyzingCompiler = { ): AnalyzingCompiler = {
val compilerBridgeProvider = ZincComponentCompiler.interfaceProvider( val compilerBridgeProvider = ZincComponentCompiler.interfaceProvider(
@ -40,8 +40,13 @@ object ZincLmUtil {
dependencyResolution, dependencyResolution,
scalaJarsTarget, scalaJarsTarget,
) )
val loader = Some(new ClassLoaderCache(new URLClassLoader(new Array(0)))) new AnalyzingCompiler(
new AnalyzingCompiler(scalaInstance, compilerBridgeProvider, classpathOptions, _ => (), loader) scalaInstance,
compilerBridgeProvider,
classpathOptions,
_ => (),
classLoaderCache
)
} }
def getDefaultBridgeModule(scalaVersion: String): ModuleID = def getDefaultBridgeModule(scalaVersion: String): ModuleID =