From f539802871bf733f98a2a92277973ee669298abd Mon Sep 17 00:00:00 2001 From: Matt Bovel Date: Fri, 19 Dec 2025 13:33:04 +0000 Subject: [PATCH 1/6] Introduce `CacheStore` to store classfile, file and source caches --- .../ZipAndJarFileLookupFactory.scala | 33 +--- .../dotty/tools/dotc/core/CacheStores.scala | 49 ++++++ .../src/dotty/tools/dotc/core/Caches.scala | 149 ++++++++++++++++++ .../src/dotty/tools/dotc/core/Contexts.scala | 21 ++- 4 files changed, 216 insertions(+), 36 deletions(-) create mode 100644 compiler/src/dotty/tools/dotc/core/CacheStores.scala create mode 100644 compiler/src/dotty/tools/dotc/core/Caches.scala diff --git a/compiler/src/dotty/tools/dotc/classpath/ZipAndJarFileLookupFactory.scala b/compiler/src/dotty/tools/dotc/classpath/ZipAndJarFileLookupFactory.scala index d5473e6b26c3..41c9f0b17d8f 100644 --- a/compiler/src/dotty/tools/dotc/classpath/ZipAndJarFileLookupFactory.scala +++ b/compiler/src/dotty/tools/dotc/classpath/ZipAndJarFileLookupFactory.scala @@ -22,17 +22,12 @@ import FileUtils.* * when there are a lot of projects having a lot of common dependencies. */ sealed trait ZipAndJarFileLookupFactory { - private val cache = new FileBasedCache[ClassPath] - def create(zipFile: AbstractFile)(using Context): ClassPath = val release = Option(ctx.settings.javaOutputVersion.value).filter(_.nonEmpty) if (ctx.settings.YdisableFlatCpCaching.value || zipFile.file == null) createForZipFile(zipFile, release) - else createUsingCache(zipFile, release) + else ctx.cacheStore.classPaths(zipFile, createForZipFile(zipFile, release)) protected def createForZipFile(zipFile: AbstractFile, release: Option[String]): ClassPath - - private def createUsingCache(zipFile: AbstractFile, release: Option[String]): ClassPath = - cache.getOrCreate(zipFile.file.toPath, () => createForZipFile(zipFile, release)) } /** @@ -172,29 +167,3 @@ object ZipAndJarSourcePathFactory extends ZipAndJarFileLookupFactory { override protected def createForZipFile(zipFile: AbstractFile, release: Option[String]): ClassPath = ZipArchiveSourcePath(zipFile.file) } - -final class FileBasedCache[T] { - private case class Stamp(lastModified: FileTime, fileKey: Object) - private val cache = collection.mutable.Map.empty[java.nio.file.Path, (Stamp, T)] - - def getOrCreate(path: java.nio.file.Path, create: () => T): T = cache.synchronized { - val attrs = Files.readAttributes(path, classOf[BasicFileAttributes]) - val lastModified = attrs.lastModifiedTime() - // only null on some platforms, but that's okay, we just use the last modified timestamp as our stamp - val fileKey = attrs.fileKey() - val stamp = Stamp(lastModified, fileKey) - cache.get(path) match { - case Some((cachedStamp, cached)) if cachedStamp == stamp => cached - case _ => - val value = create() - cache.put(path, (stamp, value)) - value - } - } - - def clear(): Unit = cache.synchronized { - // TODO support closing - // cache.valuesIterator.foreach(_.close()) - cache.clear() - } -} diff --git a/compiler/src/dotty/tools/dotc/core/CacheStores.scala b/compiler/src/dotty/tools/dotc/core/CacheStores.scala new file mode 100644 index 000000000000..c84c89bb95f8 --- /dev/null +++ b/compiler/src/dotty/tools/dotc/core/CacheStores.scala @@ -0,0 +1,49 @@ +package dotty.tools.dotc.core + +import dotty.tools.dotc.core.Caches.{Cache, FileBasedCache, NoopCache} +import dotty.tools.dotc.core.Names.TermName +import dotty.tools.dotc.util.SourceFile +import dotty.tools.io.{AbstractFile, ClassPath} + +object CacheStores: + + /** A store of caches used by the compiler. + * + * These caches can be shared across different runs. + * + * Set on a [[Context]] via `setCacheStore` and retrieved via `cacheStore`. + */ + trait CacheStore: + def classPaths: Cache[AbstractFile, ClassPath] + def files: Cache[TermName, AbstractFile] + def sources: Cache[AbstractFile, SourceFile] + + override def toString: String = + s"""CacheStore( + | classPaths = $classPaths, + | files = $files, + | sources = $sources + |)""".stripMargin + + /** Default, per-run cache store implementation. */ + object DefaultCacheStore extends CacheStore: + + /** A unique global cache for classpaths, shared across all runs. + * + * This instance is thread-safe. + */ + val classPaths = FileBasedCache() + + /** By default, we do not cache files across runs. + * + * Regardless, files are always cached within a single run via + * `ContextBase.files`. See also `Context.getFile`. + */ + val files = NoopCache() + + /** By default, we do not cache source files across runs. + * + * Regardless, source files are always cached within a single run via + * `ContextBase.sources`. See also `Context.getSource`. + */ + val sources = NoopCache() diff --git a/compiler/src/dotty/tools/dotc/core/Caches.scala b/compiler/src/dotty/tools/dotc/core/Caches.scala new file mode 100644 index 000000000000..8033874f693c --- /dev/null +++ b/compiler/src/dotty/tools/dotc/core/Caches.scala @@ -0,0 +1,149 @@ +package dotty.tools.dotc.core + +import java.nio.file.Files +import java.nio.file.Path +import java.nio.file.attribute.{BasicFileAttributes, FileTime} +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.LongAdder + +import scala.collection.mutable.Map + +import dotty.tools.io.AbstractFile + +object Caches: + + /** Cache for values of type `V`, associated with keys of type `K`. */ + trait Cache[K, V]: + def apply(key: K, value: => V): V + def stats(): CacheStats + override def toString: String = + s"${this.getClass.getSimpleName}(stats() = ${stats()})" + + /** Statistics about a cache */ + final class CacheStats(total: Long, misses: Long, uncached: Long): + val hits: Long = total - misses - uncached + + override def toString: String = + s"(total = $total, hits = $hits, misses = $misses, uncached = $uncached)" + + /** A no-op cache implementation that does not cache anything. */ + final class NoopCache[K, V] extends Cache[K, V]: + private var total = 0L + + def apply(key: K, value: => V): V = + total += 1 + value + + def stats(): CacheStats = + CacheStats(total, misses = 0, uncached = total) + + /** A thread-unsafe cache implementation based on a mutable [[Map]]. + * + * Entries are not evicted. + * + * @param getStamp + * Function to obtain a stamp for a given key. If the function returns + * `None`, no caching is performed for that key. If the function returns + * `Some(stamp)`, the stamp is used to validate cached entries: cache + * values are only reused if the stamp matches the cached stamp. + */ + final class MapCache[K, S, V](getStamp: K => Option[S]) extends Cache[K, V]: + private val map = Map.empty[K, (S, V)] + private var total = 0L + private var misses = 0L + private var uncached = 0L + + def apply(key: K, value: => V): V = + total += 1 + getStamp(key) match + case None => + uncached += 1 + value + case Some(stamp) => + map.get(key) match + case Some((cachedStamp, cachedValue)) if cachedStamp == stamp => + cachedValue + case _ => + misses += 1 + val v = value + map.put(key, (stamp, v)) + v + + def stats(): CacheStats = + CacheStats(total, misses, uncached) + + /** A thread-safe cache implementation based on a Java [[ConcurrentHashMap]]. + * + * Entries are not evicted. + */ + final class SynchronizedMapCache[K, S, V](getStamp: K => Option[S]) extends Cache[K, V]: + private val map = ConcurrentHashMap[K, (S, V)]() + private val total = LongAdder() + private val misses = LongAdder() + private val uncached = LongAdder() + + def apply(key: K, value: => V): V = + total.increment() + getStamp(key) match + case None => + uncached.increment() + value + case Some(stamp) => + map.compute( + key, + (_, cached) => + if cached != null && cached._1 == stamp then + cached + else + misses.increment() + (stamp, value) + )._2 + + def stats(): CacheStats = + CacheStats(total.longValue(), misses.longValue(), uncached.longValue()) + + /** A cache where keys are [[AbstractFile]]s. + * + * The cache uses file modification time and file key (inode) as stamp to + * invalidate cached entries when the underlying file has changed. + * + * For files with an underlying source (e.g. files inside a zip/jar), the + * stamp is obtained from the underlying source file. + * + * If the [[AbstractFile]] does not correspond to a physical file on disk, no + * caching is performed. + * + * See https://github.com/scala/bug/issues/10295 for discussion about the + * invalidation strategy. + */ + final class FileBasedCache[V]() extends Cache[AbstractFile, V]: + private case class FileStamp(lastModified: FileTime, fileKey: Object) + + private def getFileStamp(abstractFile: AbstractFile): Option[FileStamp] = + abstractFile.underlyingSource match + case Some(underlyingSource) if underlyingSource ne abstractFile => + getFileStamp(underlyingSource) + case _ => + val javaFile = abstractFile.file + if javaFile == null then + None + else + val attrs = Files.readAttributes(javaFile.toPath, classOf[BasicFileAttributes]) + val lastModified = attrs.lastModifiedTime() + // This can be `null` on some platforms, but that's okay, we just use + // the last modified timestamp as our stamp in that case. + val fileKey = attrs.fileKey() + Some(FileStamp(lastModified, fileKey)) + + private val underlying = SynchronizedMapCache[AbstractFile, FileStamp, V](getFileStamp) + + def apply(key: AbstractFile, value: => V): V = + underlying(key, value) + + def stats(): CacheStats = + underlying.stats() + + override def toString: String = + s"FileBasedCache(${underlying.toString})" + + diff --git a/compiler/src/dotty/tools/dotc/core/Contexts.scala b/compiler/src/dotty/tools/dotc/core/Contexts.scala index 9896cfaa6a1d..71a6a626103f 100644 --- a/compiler/src/dotty/tools/dotc/core/Contexts.scala +++ b/compiler/src/dotty/tools/dotc/core/Contexts.scala @@ -3,6 +3,7 @@ package dotc package core import interfaces.CompilerCallback +import CacheStores.{CacheStore, DefaultCacheStore} import Decorators.* import Periods.* import Names.* @@ -57,8 +58,9 @@ object Contexts { private val (importInfoLoc, store9) = store8.newLocation[ImportInfo | Null]() private val (typeAssignerLoc, store10) = store9.newLocation[TypeAssigner](TypeAssigner) private val (progressCallbackLoc, store11) = store10.newLocation[ProgressCallback | Null]() + private val (cacheStoreLoc, store12) = store11.newLocation[CacheStore](DefaultCacheStore) - private val initialStore = store11 + private val initialStore = store12 /** The current context */ inline def ctx(using ctx: Context): Context = ctx @@ -189,6 +191,8 @@ object Contexts { val local = progressCallback if local != null then op(local) + def cacheStore: CacheStore = store(cacheStoreLoc) + /** The current plain printer */ def printerFn: Context => Printer = store(printerFnLoc) @@ -256,7 +260,9 @@ object Contexts { /** Sourcefile corresponding to given abstract file, memoized */ def getSource(file: AbstractFile, codec: => Codec = Codec(settings.encoding.value)) = { util.Stats.record("Context.getSource") - base.sources.getOrElseUpdate(file, SourceFile(file, codec)) + // `base.sources` is run-local (it is reset at the beginning of each run), + // while `cacheStore.sources` can cache files across runs. + base.sources.getOrElseUpdate(file, cacheStore.sources(file, SourceFile(file, codec))) } /** SourceFile with given path name, memoized */ @@ -273,7 +279,9 @@ object Contexts { file case None => try - val file = new PlainFile(Path(name.toString)) + // `base.files` is run-local (it is reset at the beginning of each run), + // while `cacheStore.files` can cache files across runs. + val file = cacheStore.files(name, new PlainFile(Path(name.toString))) base.files(name) = file file catch @@ -712,6 +720,7 @@ object Contexts { def setCompilerCallback(callback: CompilerCallback): this.type = updateStore(compilerCallbackLoc, callback) def setIncCallback(callback: IncrementalCallback): this.type = updateStore(incCallbackLoc, callback) def setProgressCallback(callback: ProgressCallback): this.type = updateStore(progressCallbackLoc, callback) + def setCacheStore(cacheStore: CacheStore): this.type = updateStore(cacheStoreLoc, cacheStore) def setPrinterFn(printer: Context => Printer): this.type = updateStore(printerFnLoc, printer) def setSettings(settingsState: SettingsState): this.type = updateStore(settingsStateLoc, settingsState) def setRun(run: Run | Null): this.type = updateStore(runLoc, run) @@ -990,7 +999,11 @@ object Contexts { private var _nextSymId: Int = 0 def nextSymId: Int = { _nextSymId += 1; _nextSymId } - /** Sources and Files that were loaded */ + /** Sources and Files that were loaded. + * + * Those are intra-run caches. See also [[CacheStore.sources]] and + * [[CCacheStore.files]] for inter-run caching of source files and files. + */ val sources: util.HashMap[AbstractFile, SourceFile] = util.HashMap[AbstractFile, SourceFile]() val files: util.HashMap[TermName, AbstractFile] = util.HashMap() From 8823120e7a666d51ea72011b65c272634b0fd16e Mon Sep 17 00:00:00 2001 From: Matt Bovel Date: Fri, 19 Dec 2025 16:10:59 +0000 Subject: [PATCH 2/6] Cache standard library sources during compilation tests --- .../src/dotty/tools/dotc/core/Caches.scala | 36 +++++++++++++++++-- .../test/dotty/tools/TestCacheStore.scala | 22 ++++++++++++ .../dotty/tools/dotc/CompilationTests.scala | 2 ++ .../dotty/tools/vulpix/ParallelTesting.scala | 18 +++++++--- 4 files changed, 70 insertions(+), 8 deletions(-) create mode 100644 compiler/test/dotty/tools/TestCacheStore.scala diff --git a/compiler/src/dotty/tools/dotc/core/Caches.scala b/compiler/src/dotty/tools/dotc/core/Caches.scala index 8033874f693c..6ca3569f2eb7 100644 --- a/compiler/src/dotty/tools/dotc/core/Caches.scala +++ b/compiler/src/dotty/tools/dotc/core/Caches.scala @@ -20,7 +20,7 @@ object Caches: s"${this.getClass.getSimpleName}(stats() = ${stats()})" /** Statistics about a cache */ - final class CacheStats(total: Long, misses: Long, uncached: Long): + final case class CacheStats(total: Long, misses: Long, uncached: Long): val hits: Long = total - misses - uncached override def toString: String = @@ -37,6 +37,9 @@ object Caches: def stats(): CacheStats = CacheStats(total, misses = 0, uncached = total) + /** Default value for stamp function that indicates no stamping. */ + private def noStamp[K](key: K): Option[Unit] = Some(()) + /** A thread-unsafe cache implementation based on a mutable [[Map]]. * * Entries are not evicted. @@ -47,7 +50,7 @@ object Caches: * `Some(stamp)`, the stamp is used to validate cached entries: cache * values are only reused if the stamp matches the cached stamp. */ - final class MapCache[K, S, V](getStamp: K => Option[S]) extends Cache[K, V]: + final class MapCache[K, S, V](getStamp: K => Option[S] = noStamp) extends Cache[K, V]: private val map = Map.empty[K, (S, V)] private var total = 0L private var misses = 0L @@ -76,7 +79,7 @@ object Caches: * * Entries are not evicted. */ - final class SynchronizedMapCache[K, S, V](getStamp: K => Option[S]) extends Cache[K, V]: + final class SynchronizedMapCache[K, S, V](getStamp: K => Option[S] = noStamp) extends Cache[K, V]: private val map = ConcurrentHashMap[K, (S, V)]() private val total = LongAdder() private val misses = LongAdder() @@ -146,4 +149,31 @@ object Caches: override def toString: String = s"FileBasedCache(${underlying.toString})" + /** Filtering cache wrapper that only caches values whose key satisfies a + * given predicate. + * + * @param underlying + * Underlying cache + * @param shouldCache + * Should the value associated with the given key should be cached? + */ + final class FilteringCache[K, V](underlying: Cache[K, V], shouldCache: K => Boolean) extends Cache[K, V]: + private val uncached = LongAdder() + def apply(key: K, value: => V): V = + if shouldCache(key) then + underlying(key, value) + else + uncached.increment() + value + + def stats(): CacheStats = + val baseStats = underlying.stats() + CacheStats( + total = baseStats.total + uncached.longValue(), + misses = baseStats.misses, + uncached = baseStats.uncached + uncached.longValue() + ) + + override def toString: String = + s"FilteringCache(${underlying.toString}, uncached = ${uncached.longValue()})" diff --git a/compiler/test/dotty/tools/TestCacheStore.scala b/compiler/test/dotty/tools/TestCacheStore.scala new file mode 100644 index 000000000000..c0033b4e36a8 --- /dev/null +++ b/compiler/test/dotty/tools/TestCacheStore.scala @@ -0,0 +1,22 @@ +package dotty.tools + +import dotty.tools.dotc.core.Caches.{FilteringCache, SynchronizedMapCache} +import dotty.tools.dotc.core.CacheStores.{CacheStore, DefaultCacheStore} +import dotty.tools.io.AbstractFile + +object TestCacheStore extends CacheStore: + /** Use the default global classpath cache. */ + val classPaths = DefaultCacheStore.classPaths + + private val stdLibDir = "library/src" + + /** Cache files across runs, without invalidation. */ + val files = FilteringCache(SynchronizedMapCache(), _.startsWith((stdLibDir))) + + /** Cache source files across runs, without invalidation. + * + * We use a [[SynchronizedMapCache]] and not a [[FileBasedCache]] here + * because we assume that source files in `library/src` do not change during + * a test run. + */ + val sources = FilteringCache(SynchronizedMapCache(), _.canonicalPath.startsWith(stdLibDir)) diff --git a/compiler/test/dotty/tools/dotc/CompilationTests.scala b/compiler/test/dotty/tools/dotc/CompilationTests.scala index 74ff53b9ebf3..85a2ccb4951a 100644 --- a/compiler/test/dotty/tools/dotc/CompilationTests.scala +++ b/compiler/test/dotty/tools/dotc/CompilationTests.scala @@ -19,6 +19,7 @@ import TestSources.sources import reporting.TestReporter import vulpix._ import dotty.tools.dotc.config.ScalaSettings +import dotty.tools.dotc.core.CacheStores.DefaultCacheStore class CompilationTests { import ParallelTesting._ @@ -406,6 +407,7 @@ object CompilationTests extends ParallelTesting { implicit val summaryReport: SummaryReporting = new SummaryReport @AfterClass def tearDown(): Unit = { + println(s"Cache statistics: $TestCacheStore") super.cleanup() summaryReport.echoSummary() } diff --git a/compiler/test/dotty/tools/vulpix/ParallelTesting.scala b/compiler/test/dotty/tools/vulpix/ParallelTesting.scala index 3a2f261a8e16..159c3820e619 100644 --- a/compiler/test/dotty/tools/vulpix/ParallelTesting.scala +++ b/compiler/test/dotty/tools/vulpix/ParallelTesting.scala @@ -63,6 +63,14 @@ trait ParallelTesting extends RunnerOrchestration: protected def testPlatform: TestPlatform = TestPlatform.JVM + private def setupTestContext(initCtx: FreshContext): FreshContext = + initCtx.setCacheStore(TestCacheStore) + initCtx + + private class TestDriver extends Driver: + override protected def initCtx = + setupTestContext(super.initCtx.fresh) + /** A test source whose files or directory of files is to be compiled * in a specific way defined by the `Test` */ @@ -533,8 +541,8 @@ trait ParallelTesting extends RunnerOrchestration: val reporter = mkReporter val driver = - if (times == 1) new Driver - else new Driver { + if (times == 1) TestDriver() + else new TestDriver() { private def ntimes(n: Int)(op: Int => Reporter): Reporter = (1 to n).foldLeft(emptyReporter) ((_, i) => op(i)) @@ -686,7 +694,7 @@ trait ParallelTesting extends RunnerOrchestration: val classes = flattenFiles(targetDir).filter(isBestEffortTastyFile).map(_.toString) val flags = flags0 `and` "-from-tasty" `and` "-Ywith-best-effort-tasty" val reporter = mkReporter - val driver = new Driver + val driver = TestDriver() driver.process(flags.all ++ classes, reporter = reporter) @@ -698,7 +706,7 @@ trait ParallelTesting extends RunnerOrchestration: .and("-Ywith-best-effort-tasty") .and("-d", targetDir.getPath) val reporter = mkReporter - val driver = new Driver + val driver = TestDriver() val args = Array("-classpath", flags.defaultClassPath + JFile.pathSeparator + bestEffortDir.toString) ++ flags.options @@ -716,7 +724,7 @@ trait ParallelTesting extends RunnerOrchestration: val reporter = mkReporter - val driver = new Driver + val driver = TestDriver() driver.process(flags.all ++ classes, reporter = reporter) From f17e93bda11e8a384aa559d74a88a4f37eeb5f86 Mon Sep 17 00:00:00 2001 From: Matt Bovel Date: Fri, 19 Dec 2025 16:57:07 +0000 Subject: [PATCH 3/6] Use `ownersIterator` instead of `outersIterator` in `checkOwner` message --- compiler/src/dotty/tools/dotc/transform/TreeChecker.scala | 5 +++-- tests/neg-macros/wrong-owner.check | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/compiler/src/dotty/tools/dotc/transform/TreeChecker.scala b/compiler/src/dotty/tools/dotc/transform/TreeChecker.scala index 0b0be661d2f7..62ba30404622 100644 --- a/compiler/src/dotty/tools/dotc/transform/TreeChecker.scala +++ b/compiler/src/dotty/tools/dotc/transform/TreeChecker.scala @@ -572,7 +572,8 @@ object TreeChecker { ctxOwner.isWeakOwner && ownerMatches(symOwner, ctxOwner.owner) assert(ownerMatches(tree.symbol.owner, ctx.owner), i"bad owner; ${tree.symbol} has owner ${tree.symbol.owner}, expected was ${ctx.owner}\n" + - i"owner chain = ${tree.symbol.ownersIterator.toList}%, %, ctxOwners = ${ctx.outersIterator.map(_.owner).toList}%, %") + i"owner chain = ${tree.symbol.ownersIterator.toList}%, %\n" + + i"ctx owners = ${ctx.owner.ownersIterator.toList}%, %") } private def checkParents(tree: untpd.TypeDef)(using Context): Unit = { @@ -888,4 +889,4 @@ object TreeChecker { case _ => Nil } -} \ No newline at end of file +} diff --git a/tests/neg-macros/wrong-owner.check b/tests/neg-macros/wrong-owner.check index 00ddecdab36a..09a622602c60 100644 --- a/tests/neg-macros/wrong-owner.check +++ b/tests/neg-macros/wrong-owner.check @@ -16,6 +16,7 @@ | |Error: |assertion failed: bad owner; method toString has owner class String, expected was class Foo - |owner chain = method toString, class String, package java.lang, package java, package , ctxOwners = class Foo, class Foo, package , package , package , package , package , package , package , package , package , package , , , , , + |owner chain = method toString, class String, package java.lang, package java, package + |ctx owners = class Foo, package , package | |stacktrace available when compiling with `-Ydebug` From 254b87c55f531d6c719096430db6da01b73b5beb Mon Sep 17 00:00:00 2001 From: Matt Bovel Date: Fri, 19 Dec 2025 18:01:09 +0000 Subject: [PATCH 4/6] Add cache for class bytes --- .../dotty/tools/dotc/core/CacheStores.scala | 8 +++ .../src/dotty/tools/dotc/core/Caches.scala | 60 +++++++++++++++---- .../dotc/core/classfile/ClassfileParser.scala | 13 +++- .../test/dotty/tools/TestCacheStore.scala | 11 ++++ 4 files changed, 78 insertions(+), 14 deletions(-) diff --git a/compiler/src/dotty/tools/dotc/core/CacheStores.scala b/compiler/src/dotty/tools/dotc/core/CacheStores.scala index c84c89bb95f8..7fdc54bcc47e 100644 --- a/compiler/src/dotty/tools/dotc/core/CacheStores.scala +++ b/compiler/src/dotty/tools/dotc/core/CacheStores.scala @@ -17,12 +17,14 @@ object CacheStores: def classPaths: Cache[AbstractFile, ClassPath] def files: Cache[TermName, AbstractFile] def sources: Cache[AbstractFile, SourceFile] + def classBytes: Cache[AbstractFile, Array[Byte]] override def toString: String = s"""CacheStore( | classPaths = $classPaths, | files = $files, | sources = $sources + | classBytes = $classBytes |)""".stripMargin /** Default, per-run cache store implementation. */ @@ -47,3 +49,9 @@ object CacheStores: * `ContextBase.sources`. See also `Context.getSource`. */ val sources = NoopCache() + + /** By default, we do not cache class bytes across runs. */ + val classBytes = NoopCache() + + /** By default, we do not cache tasty loaders. */ + val tastyLoaders = NoopCache() diff --git a/compiler/src/dotty/tools/dotc/core/Caches.scala b/compiler/src/dotty/tools/dotc/core/Caches.scala index 6ca3569f2eb7..d208bc4d0cef 100644 --- a/compiler/src/dotty/tools/dotc/core/Caches.scala +++ b/compiler/src/dotty/tools/dotc/core/Caches.scala @@ -14,8 +14,23 @@ object Caches: /** Cache for values of type `V`, associated with keys of type `K`. */ trait Cache[K, V]: + + /** Get the value associated with `key` from the cache, or compute it using + * the by-name parameter `value`. + * + * The value is cached iff `mightContain(key) == true`. + */ def apply(key: K, value: => V): V + + /** Check whether the cache might contain a value for `key`. + * + * `true` means that the cache will cache the value for `key` if requested + * via [[apply]], not that it has already cached it. + */ + def mightContain(key: K): Boolean + def stats(): CacheStats + override def toString: String = s"${this.getClass.getSimpleName}(stats() = ${stats()})" @@ -34,6 +49,9 @@ object Caches: total += 1 value + def mightContain(key: K): Boolean = + false + def stats(): CacheStats = CacheStats(total, misses = 0, uncached = total) @@ -72,6 +90,9 @@ object Caches: map.put(key, (stamp, v)) v + def mightContain(key: K): Boolean = + getStamp(key).isDefined + def stats(): CacheStats = CacheStats(total, misses, uncached) @@ -102,6 +123,9 @@ object Caches: (stamp, value) )._2 + def mightContain(key: K): Boolean = + getStamp(key).isDefined + def stats(): CacheStats = CacheStats(total.longValue(), misses.longValue(), uncached.longValue()) @@ -122,27 +146,36 @@ object Caches: final class FileBasedCache[V]() extends Cache[AbstractFile, V]: private case class FileStamp(lastModified: FileTime, fileKey: Object) - private def getFileStamp(abstractFile: AbstractFile): Option[FileStamp] = + private def getPath(abstractFile: AbstractFile): Option[Path] = abstractFile.underlyingSource match case Some(underlyingSource) if underlyingSource ne abstractFile => - getFileStamp(underlyingSource) + getPath(underlyingSource) case _ => - val javaFile = abstractFile.file - if javaFile == null then - None - else - val attrs = Files.readAttributes(javaFile.toPath, classOf[BasicFileAttributes]) - val lastModified = attrs.lastModifiedTime() - // This can be `null` on some platforms, but that's okay, we just use - // the last modified timestamp as our stamp in that case. - val fileKey = attrs.fileKey() - Some(FileStamp(lastModified, fileKey)) + val javaPath = abstractFile.jpath + if javaPath != null then Some(javaPath) else None + + private def getFileStamp(abstractFile: AbstractFile): Option[FileStamp] = + getPath(abstractFile) match + case Some(path) => + val attrs = Files.readAttributes(path, classOf[BasicFileAttributes]) + val lastModified = attrs.lastModifiedTime() + // This can be `null` on some platforms, but that's okay, we just use + // the last modified timestamp as our stamp in that case. + val fileKey = attrs.fileKey() + Some(FileStamp(lastModified, fileKey)) + case None => + None private val underlying = SynchronizedMapCache[AbstractFile, FileStamp, V](getFileStamp) def apply(key: AbstractFile, value: => V): V = underlying(key, value) + def mightContain(key: AbstractFile): Boolean = + // We just check that a path exists here to avoi IO. `getFileStamp` will + // return `None` iff `getPath` returns `None`. + getPath(key).isDefined + def stats(): CacheStats = underlying.stats() @@ -175,5 +208,8 @@ object Caches: uncached = baseStats.uncached + uncached.longValue() ) + def mightContain(key: K): Boolean = + shouldCache(key) && underlying.mightContain(key) + override def toString: String = s"FilteringCache(${underlying.toString}, uncached = ${uncached.longValue()})" diff --git a/compiler/src/dotty/tools/dotc/core/classfile/ClassfileParser.scala b/compiler/src/dotty/tools/dotc/core/classfile/ClassfileParser.scala index ea8a74d18192..a0a44e3eb54d 100644 --- a/compiler/src/dotty/tools/dotc/core/classfile/ClassfileParser.scala +++ b/compiler/src/dotty/tools/dotc/core/classfile/ClassfileParser.scala @@ -294,8 +294,17 @@ class ClassfileParser( private def mismatchError(className: SimpleName) = throw new IOException(s"class file '${classfile.canonicalPath}' has location not matching its contents: contains class $className") - def run()(using Context): Option[Embedded] = try ctx.base.reusableDataReader.withInstance { reader => - implicit val reader2 = reader.reset(classfile) + def run()(using Context): Option[Embedded] = + if ctx.cacheStore.classBytes.mightContain(classfile) then + val bytes = ctx.cacheStore.classBytes(classfile, classfile.toByteArray) + given DataReader = AbstractFileReader(bytes) + runWithReader() + else + ctx.base.reusableDataReader.withInstance: reader => + given DataReader = reader.reset(classfile) + runWithReader() + + private def runWithReader()(using Context, DataReader): Option[Embedded] = try { report.debuglog("[class] >> " + classRoot.fullName) classfileVersion = parseHeader(classfile) this.pool = new ConstantPool diff --git a/compiler/test/dotty/tools/TestCacheStore.scala b/compiler/test/dotty/tools/TestCacheStore.scala index c0033b4e36a8..3d26dcfa9899 100644 --- a/compiler/test/dotty/tools/TestCacheStore.scala +++ b/compiler/test/dotty/tools/TestCacheStore.scala @@ -8,6 +8,7 @@ object TestCacheStore extends CacheStore: /** Use the default global classpath cache. */ val classPaths = DefaultCacheStore.classPaths + /** Standard library sources directory */ private val stdLibDir = "library/src" /** Cache files across runs, without invalidation. */ @@ -20,3 +21,13 @@ object TestCacheStore extends CacheStore: * a test run. */ val sources = FilteringCache(SynchronizedMapCache(), _.canonicalPath.startsWith(stdLibDir)) + + /** Test output directory */ + private val outDir = "out" + + /** Cache class bytes across runs, except for classes in the `out` directory. + * + * Classes in the `out` directory are generated during tests, so we do not + * want to cache them. + */ + val classBytes = FilteringCache(SynchronizedMapCache(), !_.canonicalPath.startsWith(outDir)) From 34b98b5831370b6e291926ad0f09b48676dca083 Mon Sep 17 00:00:00 2001 From: Matt Bovel Date: Fri, 19 Dec 2025 18:37:41 +0000 Subject: [PATCH 5/6] Add cache for TASTy bytes --- compiler/src/dotty/tools/dotc/config/JavaPlatform.scala | 2 +- compiler/src/dotty/tools/dotc/core/CacheStores.scala | 8 +++++--- compiler/src/dotty/tools/dotc/core/SymbolLoaders.scala | 8 +++++--- compiler/test/dotty/tools/TestCacheStore.scala | 7 +++++-- 4 files changed, 16 insertions(+), 9 deletions(-) diff --git a/compiler/src/dotty/tools/dotc/config/JavaPlatform.scala b/compiler/src/dotty/tools/dotc/config/JavaPlatform.scala index afa701283eb6..8048bafeecc6 100644 --- a/compiler/src/dotty/tools/dotc/config/JavaPlatform.scala +++ b/compiler/src/dotty/tools/dotc/config/JavaPlatform.scala @@ -78,5 +78,5 @@ class JavaPlatform extends Platform { new ClassfileLoader(bin) def newTastyLoader(bin: AbstractFile)(using Context): SymbolLoader = - new TastyLoader(bin) + new TastyLoader(bin, ctx.cacheStore.tastyBytes) } diff --git a/compiler/src/dotty/tools/dotc/core/CacheStores.scala b/compiler/src/dotty/tools/dotc/core/CacheStores.scala index 7fdc54bcc47e..17fdc5125d9a 100644 --- a/compiler/src/dotty/tools/dotc/core/CacheStores.scala +++ b/compiler/src/dotty/tools/dotc/core/CacheStores.scala @@ -18,6 +18,7 @@ object CacheStores: def files: Cache[TermName, AbstractFile] def sources: Cache[AbstractFile, SourceFile] def classBytes: Cache[AbstractFile, Array[Byte]] + def tastyBytes: Cache[AbstractFile, Array[Byte]] override def toString: String = s"""CacheStore( @@ -25,6 +26,7 @@ object CacheStores: | files = $files, | sources = $sources | classBytes = $classBytes + | tastyBytes = $tastyBytes |)""".stripMargin /** Default, per-run cache store implementation. */ @@ -50,8 +52,8 @@ object CacheStores: */ val sources = NoopCache() - /** By default, we do not cache class bytes across runs. */ + /** By default, we do not cache class bytes. */ val classBytes = NoopCache() - /** By default, we do not cache tasty loaders. */ - val tastyLoaders = NoopCache() + /** By default, we do not cache tasty bytes. */ + val tastyBytes = NoopCache() diff --git a/compiler/src/dotty/tools/dotc/core/SymbolLoaders.scala b/compiler/src/dotty/tools/dotc/core/SymbolLoaders.scala index 96f0d0cb9d20..560b4313a4e6 100644 --- a/compiler/src/dotty/tools/dotc/core/SymbolLoaders.scala +++ b/compiler/src/dotty/tools/dotc/core/SymbolLoaders.scala @@ -17,6 +17,7 @@ import NameOps.* import StdNames.* import classfile.{ClassfileParser, ClassfileTastyUUIDParser} import Decorators.* +import Caches.Cache import util.Stats import reporting.trace @@ -471,10 +472,11 @@ class ClassfileLoader(val classfile: AbstractFile) extends SymbolLoader { classfileParser.run() } -class TastyLoader(val tastyFile: AbstractFile) extends SymbolLoader { - val isBestEffortTasty = tastyFile.hasBetastyExtension +class TastyLoader(val tastyFile: AbstractFile, tastyBytesCache: Cache[AbstractFile, Array[Byte]]) extends SymbolLoader { + private val isBestEffortTasty = tastyFile.hasBetastyExtension - lazy val tastyBytes = tastyFile.toByteArray + private lazy val tastyBytes: Array[Byte] = + tastyBytesCache(tastyFile, tastyFile.toByteArray) private lazy val unpickler: tasty.DottyUnpickler = handleUnpicklingExceptions: diff --git a/compiler/test/dotty/tools/TestCacheStore.scala b/compiler/test/dotty/tools/TestCacheStore.scala index 3d26dcfa9899..8174e5333fdc 100644 --- a/compiler/test/dotty/tools/TestCacheStore.scala +++ b/compiler/test/dotty/tools/TestCacheStore.scala @@ -27,7 +27,10 @@ object TestCacheStore extends CacheStore: /** Cache class bytes across runs, except for classes in the `out` directory. * - * Classes in the `out` directory are generated during tests, so we do not - * want to cache them. + * Files in the `out` directory are generated during tests, so we do not want + * to cache them. */ val classBytes = FilteringCache(SynchronizedMapCache(), !_.canonicalPath.startsWith(outDir)) + + /** Cache tasty bytes across runs, except for those in the `out` directory. */ + val tastyBytes = FilteringCache(SynchronizedMapCache(), !_.canonicalPath.startsWith(outDir)) From 8d322fa8779678da061f914f3fd6cbb08ea565c2 Mon Sep 17 00:00:00 2001 From: Matt Bovel Date: Fri, 19 Dec 2025 18:37:57 +0000 Subject: [PATCH 6/6] Add size to cache stats --- compiler/src/dotty/tools/dotc/core/Caches.scala | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/compiler/src/dotty/tools/dotc/core/Caches.scala b/compiler/src/dotty/tools/dotc/core/Caches.scala index d208bc4d0cef..953272176b64 100644 --- a/compiler/src/dotty/tools/dotc/core/Caches.scala +++ b/compiler/src/dotty/tools/dotc/core/Caches.scala @@ -35,11 +35,11 @@ object Caches: s"${this.getClass.getSimpleName}(stats() = ${stats()})" /** Statistics about a cache */ - final case class CacheStats(total: Long, misses: Long, uncached: Long): + final case class CacheStats(total: Long, misses: Long, size: Long, uncached: Long): val hits: Long = total - misses - uncached override def toString: String = - s"(total = $total, hits = $hits, misses = $misses, uncached = $uncached)" + s"(total = $total, hits = $hits, misses = $misses, size = $size, uncached = $uncached)" /** A no-op cache implementation that does not cache anything. */ final class NoopCache[K, V] extends Cache[K, V]: @@ -53,7 +53,7 @@ object Caches: false def stats(): CacheStats = - CacheStats(total, misses = 0, uncached = total) + CacheStats(total, misses = 0, size = 0, uncached = total) /** Default value for stamp function that indicates no stamping. */ private def noStamp[K](key: K): Option[Unit] = Some(()) @@ -94,11 +94,17 @@ object Caches: getStamp(key).isDefined def stats(): CacheStats = - CacheStats(total, misses, uncached) + CacheStats(total, misses, map.size, uncached) /** A thread-safe cache implementation based on a Java [[ConcurrentHashMap]]. * * Entries are not evicted. + * + * @param getStamp + * Function to obtain a stamp for a given key. If the function returns + * `None`, no caching is performed for that key. If the function returns + * `Some(stamp)`, the stamp is used to validate cached entries: cache + * values are only reused if the stamp matches the cached stamp. */ final class SynchronizedMapCache[K, S, V](getStamp: K => Option[S] = noStamp) extends Cache[K, V]: private val map = ConcurrentHashMap[K, (S, V)]() @@ -127,7 +133,7 @@ object Caches: getStamp(key).isDefined def stats(): CacheStats = - CacheStats(total.longValue(), misses.longValue(), uncached.longValue()) + CacheStats(total.longValue(), misses.longValue(), map.size(), uncached.longValue()) /** A cache where keys are [[AbstractFile]]s. * @@ -205,6 +211,7 @@ object Caches: CacheStats( total = baseStats.total + uncached.longValue(), misses = baseStats.misses, + size = baseStats.size, uncached = baseStats.uncached + uncached.longValue() )