diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 6a81d8f100..d3af480be3 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -74,10 +74,10 @@ jobs: if: ${{ github.event_name == 'pull_request' && matrix.jobtype == 4 }} shell: bash run: | - sbt -v -Dfile.encoding=UTF-8 "-Dbenchmark.pattern=.*Scalac.*" "runBenchmarks" + sbt -v -Dfile.encoding=UTF-8 "-Dbenchmark.pattern=.*Scalac.*" "zincBenchmarks/jmh:clean" "runBenchmarks" - name: Benchmark (Shapeless) against Develop Branch (5) if: ${{ github.event_name == 'pull_request' && matrix.jobtype == 5 }} shell: bash run: | - sbt -v -Dfile.encoding=UTF-8 "-Dbenchmark.pattern=.*Shapeless.*" "runBenchmarks" + sbt -v -Dfile.encoding=UTF-8 "-Dbenchmark.pattern=.*Shapeless.*" "zincBenchmarks/jmh:clean" "runBenchmarks" diff --git a/internal/zinc-benchmarks/src/test/scala/xsbt/AnalysisFormatBenchmark.scala b/internal/zinc-benchmarks/src/test/scala/xsbt/AnalysisFormatBenchmark.scala new file mode 100644 index 0000000000..353a52eb3d --- /dev/null +++ b/internal/zinc-benchmarks/src/test/scala/xsbt/AnalysisFormatBenchmark.scala @@ -0,0 +1,176 @@ +package xsbt + +import java.io.File +import java.util.concurrent.TimeUnit +import scala.collection.mutable + +import org.openjdk.jmh.annotations._ +import org.openjdk.jmh.infra.Blackhole +import sbt.internal.inc.consistent._ +import sbt.internal.inc.{ Analysis, FileAnalysisStore } +import sbt.io.IO +import xsbti.compile.analysis.ReadWriteMappers +import xsbti.compile.{ AnalysisContents, AnalysisStore } + +@BenchmarkMode(Array(Mode.AverageTime)) +@Fork(1) +@Threads(1) +@Warmup(iterations = 5) +@Measurement(iterations = 5) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@State(Scope.Benchmark) +class AnalysisFormatBenchmark { + + var temp: File = _ + val sets = IndexedSeq("compiler", "reflect", "library") + var cached: Map[String, AnalysisContents] = _ + + @Setup + def setup(): Unit = { + this.temp = IO.createTemporaryDirectory + sets.foreach { s => + val f = new File("../../../test-data", s"${s}.zip") + assert(f.exists()) + val f2 = new File(temp, f.getName) + IO.copyFile(f, f2) + assert(f2.exists()) + } + this.cached = readAll("", FileAnalysisStore.binary(_)) + writeAll("-ref-text", FileAnalysisStore.text(_), cached) + // writeAll("-ref-ctext", ConsistentFileAnalysisStore.text(_, ReadWriteMappers.getEmptyMappers), cached) + writeAll( + "-ref-cbin", + ConsistentFileAnalysisStore.binary(_, ReadWriteMappers.getEmptyMappers), + cached + ) + writeAll( + "-ref-cbin-nosort", + ConsistentFileAnalysisStore.binary(_, ReadWriteMappers.getEmptyMappers, sort = false), + cached + ) + println("Sizes:") + temp.listFiles().foreach { p => println(s"$p: ${p.length()}") } + val cbinTotal = temp.listFiles().filter(_.getName.endsWith("-cbin.zip")).map(_.length()).sum + println(s"cbin total = $cbinTotal, ${cbinTotal / 1024}k") + val cbinNoSortTotal = + temp.listFiles().filter(_.getName.endsWith("-cbin-nosort.zip")).map(_.length()).sum + println(s"cbin-nosort total = $cbinNoSortTotal, ${cbinNoSortTotal / 1024}k") + } + + @TearDown + def tearDown(): Unit = { + if (temp != null) IO.delete(temp) + } + + @Benchmark + def readBinary(bh: Blackhole): Unit = bh.consume(readAll("", FileAnalysisStore.binary(_))) + + @Benchmark + def readText(bh: Blackhole): Unit = bh.consume(readAll("-ref-text", FileAnalysisStore.text(_))) + + @Benchmark + def readConsistentBinary(bh: Blackhole): Unit = + bh.consume( + readAll("-ref-cbin", ConsistentFileAnalysisStore.binary(_, ReadWriteMappers.getEmptyMappers)) + ) + + @Benchmark + def writeBinary(bh: Blackhole): Unit = + bh.consume(writeAll("-test-bin", FileAnalysisStore.binary(_), cached)) + + @Benchmark + def writeText(bh: Blackhole): Unit = + bh.consume(writeAll("-test-text", FileAnalysisStore.text(_), cached)) + + @Benchmark + def writeConsistentBinary(bh: Blackhole): Unit = + bh.consume( + writeAll( + "-test-cbin", + ConsistentFileAnalysisStore.binary(_, ReadWriteMappers.getEmptyMappers), + cached + ) + ) + + @Benchmark + def writeConsistentBinaryNoSort(bh: Blackhole): Unit = + bh.consume( + writeAll( + "-test-cbin-nosort", + ConsistentFileAnalysisStore.binary(_, ReadWriteMappers.getEmptyMappers, sort = false), + cached + ) + ) + + @Benchmark + def writeNull(bh: Blackhole): Unit = { + cached.foreach { + case (s, a) => + val ser = new NullSerializer + val af = new ConsistentAnalysisFormat(ReadWriteMappers.getEmptyMappers, sort = true) + af.write(ser, a.getAnalysis, a.getMiniSetup) + bh.consume(ser.count) + } + } + + @Benchmark + def writeNullNoSort(bh: Blackhole): Unit = { + cached.foreach { + case (s, a) => + val ser = new NullSerializer + val af = new ConsistentAnalysisFormat(ReadWriteMappers.getEmptyMappers, sort = false) + af.write(ser, a.getAnalysis, a.getMiniSetup) + bh.consume(ser.count) + } + } + + def readAll(suffix: String, store: File => AnalysisStore): Map[String, AnalysisContents] = + sets.iterator.map(s => (s, read(s, suffix, store))).toMap + + def writeAll( + suffix: String, + store: File => AnalysisStore, + map: Map[String, AnalysisContents] + ): Unit = + map.foreach { case (s, a) => write(s, suffix, store, a) } + + def read(set: String, suffix: String, store: File => AnalysisStore): AnalysisContents = { + val api = store((new File(temp, s"${set}${suffix}.zip"))).unsafeGet() + assert(api.getAnalysis.asInstanceOf[Analysis].apis.internal.head._2.api() != null) + api + } + + def write( + set: String, + suffix: String, + store: File => AnalysisStore, + analysis: AnalysisContents + ): Unit = { + assert(analysis.getMiniSetup.storeApis()) + val f = new File(temp, s"${set}${suffix}.zip") + IO.delete(f) + store(f).set(analysis) + assert(f.exists()) + } +} + +class NullSerializer extends Serializer { + private[this] val strings = mutable.HashMap.empty[String, String] + private[this] var _count = 0 + def count: Int = _count + def startBlock(name: String): Unit = _count += 1 + def startArray(name: String, length: Int): Unit = _count += 1 + def endBlock(): Unit = _count += 1 + def endArray(): Unit = _count += 1 + def string(s: String): Unit = { + if (!strings.contains(s)) { + strings.put(s, s) + _count += 1 + } + } + def bool(b: Boolean): Unit = _count += 1 + def int(i: Int): Unit = _count += 1 + def byte(b: Byte): Unit = _count += 1 + def long(l: Long): Unit = _count += 1 + def end(): Unit = _count += 1 +} diff --git a/internal/zinc-persist/src/main/scala-2.12/sbt/internal/inc/consistent/Compat.scala b/internal/zinc-persist/src/main/scala-2.12/sbt/internal/inc/consistent/Compat.scala new file mode 100644 index 0000000000..22a8fbf8ed --- /dev/null +++ b/internal/zinc-persist/src/main/scala-2.12/sbt/internal/inc/consistent/Compat.scala @@ -0,0 +1,55 @@ +package sbt.internal.inc.consistent + +import java.util.Arrays +import scala.collection.{ MapLike, SetLike, SortedMap, SortedMapLike } +import scala.collection.generic.{ + CanBuildFrom, + GenericTraversableTemplate, + MapFactory, + SeqFactory, + SetFactory, + SortedMapFactory +} + +// some simple compatibility shims for 2.12 so we don't need to depend on collection-compat +object Compat { + type Factory[-A, +C] = CanBuildFrom[Nothing, A, C] + + implicit def sortedMapFactoryToCBF[CC[A, B] <: SortedMap[A, B] with SortedMapLike[ + A, + B, + CC[A, B] + ], K: Ordering, V](f: SortedMapFactory[CC]): Factory[(K, V), CC[K, V]] = + new f.SortedMapCanBuildFrom + + implicit def mapFactoryToCBF[CC[A, B] <: Map[A, B] with MapLike[A, B, CC[A, B]], K, V]( + f: MapFactory[CC] + ): Factory[(K, V), CC[K, V]] = + new f.MapCanBuildFrom + + implicit def seqFactoryToCBF[CC[X] <: Seq[X] with GenericTraversableTemplate[X, CC], E]( + f: SeqFactory[CC] + ): Factory[E, CC[E]] = + new f.GenericCanBuildFrom + + implicit def setFactoryToCBF[CC[X] <: Set[X] with SetLike[X, CC[X]], E](f: SetFactory[CC]) + : Factory[E, CC[E]] = + f.setCanBuildFrom + + implicit class FactoryOps[-A, +C](private val factory: Factory[A, C]) { + def newBuilder: scala.collection.mutable.Builder[A, C] = factory() + } + + type IterableOnce[+E] = TraversableOnce[E] + + implicit class IterableOnceOps[+E](private val it: IterableOnce[E]) { + def iterator: Iterator[E] = it match { + case it: Iterator[?] => it.asInstanceOf[Iterator[E]] + case it => it.asInstanceOf[Iterable[E]].iterator + } + } + + implicit class ArrayOps[A <: AnyRef](private val a: Array[A]) { + def sortInPlaceBy[B](f: A => B)(implicit ord: Ordering[B]): Unit = Arrays.sort(a, ord on f) + } +} diff --git a/internal/zinc-persist/src/main/scala-2.13/sbt/internal/inc/consistent/Compat.scala b/internal/zinc-persist/src/main/scala-2.13/sbt/internal/inc/consistent/Compat.scala new file mode 100644 index 0000000000..8fa9219345 --- /dev/null +++ b/internal/zinc-persist/src/main/scala-2.13/sbt/internal/inc/consistent/Compat.scala @@ -0,0 +1,5 @@ +package sbt.internal.inc.consistent + +object Compat { + type Factory[-A, +C] = scala.collection.Factory[A, C] +} diff --git a/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ConsistentAnalysisFormat.scala b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ConsistentAnalysisFormat.scala new file mode 100644 index 0000000000..b6bb03a01d --- /dev/null +++ b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ConsistentAnalysisFormat.scala @@ -0,0 +1,798 @@ +package sbt.internal.inc.consistent + +import java.nio.file.Paths +import java.util.{ Arrays, Comparator, EnumSet } +import sbt.internal.inc.{ UsedName, Stamp => StampImpl, _ } +import sbt.internal.util.Relation +import sbt.util.InterfaceUtil +import xsbti.{ Problem, Severity, UseScope, VirtualFileRef } +import xsbti.api._ +import xsbti.compile._ +import xsbti.compile.analysis.{ ReadWriteMappers, SourceInfo, Stamp } + +import scala.collection.immutable.TreeMap +import sbt.internal.inc.binary.converters.InternalApiProxy +import Compat._ + +/** A new implementation of zinc's incremental state serialization. + * - Full structural serialization (like the existing protobuf format), no shortcuts with sbinary + * or Java serialization (like the existing text format). + * - A single implementation that supports an efficient binary format for production use and a + * text format for development and debugging. + * - Consistent output files: If two compiler runs result in the same internal representation of + * incremental state (after applying WriteMappers), they produce identical zinc files. + * - Smaller output files than the existing binary format. + * - Faster serialization and deserialization than the existing binary format. + * - Smaller implementation than either of the existing formats. + */ +class ConsistentAnalysisFormat(val mappers: ReadWriteMappers, sort: Boolean) { + import ConsistentAnalysisFormat._ + + private[this] final val VERSION = 1100028 + private[this] final val readMapper = mappers.getReadMapper + private[this] final val writeMapper = mappers.getWriteMapper + + def write(out: Serializer, analysis: CompileAnalysis, setup: MiniSetup): Unit = { + val analysis0 = analysis match { case analysis: Analysis => analysis } + out.int(VERSION) + writeMiniSetup(out, setup) + writeRelations(out, analysis0.relations) + writeStamps(out, analysis0.stamps) + writeAPIs(out, analysis0.apis, setup.storeApis()) + writeSourceInfos(out, analysis0.infos) + out.int(VERSION) + out.end() + } + + def read(in: Deserializer): (CompileAnalysis, MiniSetup) = { + readVersion(in) + val setup = readMiniSetup(in) + val relations = readRelations(in) + val stamps = readStamps(in) + val apis = readAPIs(in, setup.storeApis()) + val infos = readSourceInfos(in) + readVersion(in) + in.end() + (Analysis.Empty.copy(stamps, apis, relations, infos, Compilations.of(Nil)), setup) + } + + @inline + private[this] final def writeMaybeSortedStringMap[V]( + out: Serializer, + name: String, + map: scala.collection.Iterable[(String, V)], + perEntry: Int = 1 + )(f: V => Unit): Unit = + if (sort) out.writeSortedStringMap(name, map, perEntry)(f) + else out.writeColl(name, map, perEntry + 1) { kv => out.string(kv._1); f(kv._2) } + + private[this] def readVersion(in: Deserializer): Unit = { + val ver = in.int() + if (ver != VERSION) throw new Exception(s"Unsupported format version $ver") + } + + private[this] def writeStamp2(out: Serializer, stamp: Stamp): Unit = stamp match { + case hash: FarmHash => out.byte(0); out.long(hash.hashValue) + case hash: Hash => out.byte(1); out.string(hash.hexHash) + case hash: LastModified => out.byte(2); out.long(hash.value) + case _ => out.byte(3); out.string(stamp.toString) + } + + private[this] def readStamp2(in: Deserializer): Stamp = in.byte() match { + case 0 => FarmHash.fromLong(in.long()) + case 1 => Hash.unsafeFromString(in.string()) + case 2 => new LastModified(in.long()) + case 3 => StampImpl.fromString(in.string()) + } + + private[this] def writeStamps(out: Serializer, stamps: Stamps): Unit = { + writeMaybeSortedStringMap( + out, + "stamps.products", + stamps.products.view.map { case (f, stamp) => + (writeMapper.mapProductFile(f).id, writeMapper.mapProductStamp(f, stamp)) + }, + 2 + )(writeStamp2(out, _)) + writeMaybeSortedStringMap( + out, + "stamps.sources", + stamps.sources.view.map { case (f, stamp) => + (writeMapper.mapSourceFile(f).id, writeMapper.mapSourceStamp(f, stamp)) + }, + 2 + )(writeStamp2(out, _)) + writeMaybeSortedStringMap( + out, + "stamps.libraries", + stamps.libraries.view.map { case (f, stamp) => + (writeMapper.mapBinaryFile(f).id, writeMapper.mapBinaryStamp(f, stamp)) + }, + 2 + )(writeStamp2(out, _)) + } + + private[this] def readStamps(in: Deserializer): Stamps = { + import VirtualFileUtil._ + val products = + in.readColl[(VirtualFileRef, Stamp), TreeMap[VirtualFileRef, Stamp]](TreeMap, 3) { + val f = readMapper.mapProductFile(VirtualFileRef.of(in.string())) + (f, readMapper.mapProductStamp(f, readStamp2(in))) + } + val sources = in.readColl[(VirtualFileRef, Stamp), TreeMap[VirtualFileRef, Stamp]](TreeMap, 3) { + val f = readMapper.mapSourceFile(VirtualFileRef.of(in.string())) + (f, readMapper.mapSourceStamp(f, readStamp2(in))) + } + val libraries = + in.readColl[(VirtualFileRef, Stamp), TreeMap[VirtualFileRef, Stamp]](TreeMap, 3) { + val f = readMapper.mapBinaryFile(VirtualFileRef.of(in.string())) + (f, readMapper.mapBinaryStamp(f, readStamp2(in))) + } + Stamps(products, sources, libraries) + } + + private[this] def writeAnalyzedClass( + out: Serializer, + ac: AnalyzedClass, + storeApis: Boolean + ): Unit = { + out.writeBlock("analyzedClass") { + out.string(ac.name()) + out.long(ac.compilationTimestamp()) + out.int(ac.apiHash()) + out.bool(ac.hasMacro) + out.string(ac.provenance()) + val nh0 = ac.nameHashes() + val nh = if (nh0.length > 1 && sort) { + val nh = nh0.clone() + Arrays.sort(nh, nameHashComparator) + nh + } else nh0 + out.writeArray("nameHashes.name", nh) { h => out.string(h.name()) } + out.writeArray("nameHashes.scope", nh) { h => out.byte(h.scope().ordinal().toByte) } + out.writeArray("nameHashes.hash", nh) { h => out.int(h.hash()) } + if (storeApis) { + val comp = ac.api() + writeClassLike(out, comp.classApi()) + writeClassLike(out, comp.objectApi()) + } + } + } + + private[this] def readAnalyzedClass(in: Deserializer, storeApis: Boolean): AnalyzedClass = { + in.readBlock { + val name = in.string() + val ts = in.long() + val ah = in.int() + val hm = in.bool() + val p = in.string() + val nhNames = in.readStringArray() + val nhScopes = in.readArray[UseScope]() { UseScope.values()(in.byte().toInt) } + val nhHashes = in.readArray[Int]() { in.int() } + val nameHashes = new Array[NameHash](nhNames.length) + var i = 0 + while (i < nameHashes.length) { + nameHashes(i) = NameHash.of(nhNames(i), nhScopes(i), nhHashes(i)) + i += 1 + } + val comp = + if (storeApis) Companions.of(readClassLike(in), readClassLike(in)) + else APIs.emptyCompanions + AnalyzedClass.of(ts, name, SafeLazyProxy.strict(comp), ah, nameHashes, hm, ah, p) + } + } + + private[this] def writeAPIs(out: Serializer, apis: APIs, storeApis: Boolean): Unit = { + def write(n: String, m: Map[String, AnalyzedClass]): Unit = + writeMaybeSortedStringMap( + out, + n, + m.mapValues(_.withCompilationTimestamp(DefaultCompilationTimestamp)) + ) { ac => + writeAnalyzedClass(out, ac, storeApis) + } + write("internal", apis.internal) + write("external", apis.external) + } + + private[this] def readAPIs(in: Deserializer, storeApis: Boolean): APIs = { + def read() = in.readColl[(String, AnalyzedClass), Map[String, AnalyzedClass]](Map, 2) { + (in.string(), readAnalyzedClass(in, storeApis)) + } + APIs(read(), read()) + } + + private[this] def writeSourceInfos(out: Serializer, infos: SourceInfos): Unit = { + def writeProblem(p: Problem): Unit = out.writeBlock("problem") { + out.string(p.category()) + out.byte(p.severity().ordinal().toByte) + out.string(p.message()) + out.writeOptionalString(p.rendered()) + val pos = p.position() + out.int(pos.line.orElse(-1)) + out.int(pos.offset.orElse(-1)) + out.int(pos.pointer.orElse(-1)) + out.string(pos.lineContent) + out.string(pos.pointerSpace.orElse(null)) + out.string(pos.sourcePath.orElse(null)) + out.writeOptionalString(pos.sourceFile.map[String](_.toPath.toString)) + out.int(pos.startOffset.orElse(-1)) + out.int(pos.endOffset.orElse(-1)) + out.int(pos.startLine.orElse(-1)) + out.int(pos.startColumn.orElse(-1)) + out.int(pos.endLine.orElse(-1)) + out.int(pos.endColumn.orElse(-1)) + } + val mapped = infos.allInfos.view.map { case (file, info) => + (writeMapper.mapSourceFile(file).id, info) + } + writeMaybeSortedStringMap(out, "sourceInfos", mapped, 3) { info => + out.writeStringArray("mainClasses", info.getMainClasses) + out.writeArray("reportedProblems", info.getReportedProblems)(writeProblem) + out.writeArray("unreportedProblems", info.getUnreportedProblems)(writeProblem) + } + } + + private[this] def readSourceInfos(in: Deserializer): SourceInfos = { + def readProblem(): Problem = in.readBlock { + val category = in.string() + val severity = Severity.values.apply(in.byte().toInt) + val message = in.string() + val rendered = Option(in.string()) + def io(): Option[Integer] = in.int() match { case -1 => None; case i => Some(i) } + val line, offset, pointer = io() + val lineContent = in.string() + val pointerSpace, sourcePath = Option(in.string()) + val sourceFile = Option(in.string()).map(s => Paths.get(s).toFile) + val startOffset, endOffset, startLine, startColumn, endLine, endColumn = io() + val position = InterfaceUtil.position( + line, + lineContent, + offset, + pointer, + pointerSpace, + sourcePath, + sourceFile, + startOffset, + endOffset, + startLine, + startColumn, + endLine, + endColumn + ) + InterfaceUtil.problem(category, position, message, severity, rendered, None, Nil, Nil) + } + SourceInfos.of(in.readColl[(VirtualFileRef, SourceInfo), Map[VirtualFileRef, SourceInfo]]( + Map, + 4 + ) { + val file = readMapper.mapSourceFile(VirtualFileRef.of(in.string())) + val mainClasses = in.readStringSeq() + val reportedProblems = in.readArray()(readProblem()) + val unreportedProblems = in.readArray()(readProblem()) + val info = SourceInfos.makeInfo(reportedProblems, unreportedProblems, mainClasses) + (file, info) + }) + } + + private[this] def writeMiniSetup(out: Serializer, setup0: MiniSetup): Unit = { + val setup = writeMapper.mapMiniSetup(setup0) + out.writeBlock("MiniSetup") { + out.writeArray("classpathHash", setup.options.classpathHash, 2) { fh => + out.string(writeMapper.mapClasspathEntry(fh.file).toString) + out.int(fh.hash()) + } + out.writeArray("scalacOptions", setup.options.scalacOptions) { s => + out.string(writeMapper.mapScalacOption(s)) + } + out.writeArray("javacOptions", setup.options.javacOptions) { s => + out.string(writeMapper.mapJavacOption(s)) + } + out.string(setup.compilerVersion) + out.byte(setup.order.ordinal().toByte) + out.bool(setup.storeApis()) + out.writeArray("extra", setup.extra, 2) { t => out.string(t.get1); out.string(t.get2) } + } + } + + private[this] def readMiniSetup(in: Deserializer): MiniSetup = { + in.readBlock { + val classpathHash = in.readArray(2) { + FileHash.of(readMapper.mapClasspathEntry(Paths.get(in.string())), in.int()) + } + val scalacOptions = in.readArray() { readMapper.mapScalacOption(in.string()) } + val javacOptions = in.readArray() { readMapper.mapJavacOption(in.string()) } + val compilerVersion = in.string() + val compileOrder = CompileOrder.values()(in.byte().toInt) + val skipApiStoring = in.bool() + val extra = in.readArray(2) { InterfaceUtil.t2(in.string() -> in.string()) } + readMapper.mapMiniSetup(MiniSetup.of( + CompileOutput(Analysis.dummyOutputPath), + MiniOptions.of(classpathHash, scalacOptions, javacOptions), + compilerVersion, + compileOrder, + skipApiStoring, + extra + )) + } + } + + private[this] def writeRelations(out: Serializer, rs: Relations): Unit = { + writeMaybeSortedStringMap(out, "usedNames", rs.names.toMultiMap)(writeUsedNameSet(out, _)) + def mapProduct(f: VirtualFileRef) = writeMapper.mapProductFile(f).id + def mapSource(f: VirtualFileRef) = writeMapper.mapSourceFile(f).id + def mapBinary(f: VirtualFileRef) = writeMapper.mapBinaryFile(f).id + def wr[A, B](name: String, rel: Relation[A, B], kf: A => String, vf: B => String): Unit = + writeMaybeSortedStringMap( + out, + name, + rel.forwardMap.view.map { case (k, vs) => kf(k) -> vs } + ) { vs => + val a = vs.iterator.map(vf).toArray + if (sort) Arrays.sort(a, implicitly[Ordering[String]]) + out.writeColl("item", a)(out.string) + } + def wrS(name: String, rel: Relation[String, String]): Unit = + wr(name, rel, identity[String], identity[String]) + wr("srcProd", rs.srcProd, mapSource, mapProduct) + wr("libraryDep", rs.libraryDep, mapSource, mapBinary) + wr("libraryClassName", rs.libraryClassName, mapBinary, identity[String]) + wr("classes", rs.classes, mapSource, identity[String]) + wrS("memberRef.internal", rs.memberRef.internal) + wrS("memberRef.external", rs.memberRef.external) + wrS("inheritance.internal", rs.inheritance.internal) + wrS("inheritance.external", rs.inheritance.external) + wrS("localInheritance.internal", rs.localInheritance.internal) + wrS("localInheritance.external", rs.localInheritance.external) + wrS("productClassNames", rs.productClassName) + } + + private[this] def readRelations(in: Deserializer): Relations = { + val un = + UsedNames.fromMultiMap(in.readColl[(String, Set[UsedName]), Map[String, Set[UsedName]]]( + Map, + 2 + ) { + (in.string(), readUsedNameSet(in)) + }) + def mapProduct(s: String) = readMapper.mapProductFile(VirtualFileRef.of(s)) + def mapSource(s: String) = readMapper.mapSourceFile(VirtualFileRef.of(s)) + def mapBinary(s: String) = readMapper.mapBinaryFile(VirtualFileRef.of(s)) + def rd[A, B](kf: String => A, vf: String => B): Relation[A, B] = + Relation.reconstruct(in.readColl[(A, Set[B]), Map[A, Set[B]]](Map, 2) { + (kf(in.string()), in.readColl[B, Set[B]](Set) { vf(in.string()) }) + }) + def rdS() = rd(identity[String], identity[String]) + val p = rd(mapSource, mapProduct) + val bin = rd(mapSource, mapBinary) + val lcn = rd(mapBinary, identity[String]) + val cn = rd(mapSource, identity[String]) + val mri, mre, ii, ie, lii, lie, bcn = rdS() + def deps( + m: Relation[String, String], + i: Relation[String, String], + l: Relation[String, String] + ) = + Map( + DependencyContext.DependencyByMemberRef -> m, + DependencyContext.DependencyByInheritance -> i, + DependencyContext.LocalDependencyByInheritance -> l + ) + Relations.make( + p, + bin, + lcn, + InternalDependencies(deps(mri, ii, lii)), + ExternalDependencies(deps(mre, ie, lie)), + cn, + un, + bcn + ) + } + + private[this] def writeUsedNameSet(out: Serializer, uns: scala.collection.Set[UsedName]): Unit = { + out.writeBlock("UsedName") { + val groups0 = uns.iterator.map { un => + val sc = un.scopes + var i = 0 + if (sc.contains(UseScope.Default)) i += 1 + if (sc.contains(UseScope.Implicit)) i += 2 + if (sc.contains(UseScope.PatMatTarget)) i += 4 + (un.name, i.toByte) + }.toArray.groupBy(_._2) + val groups = if (sort) groups0.toVector.sortBy(_._1) else groups0 + out.writeColl("groups", groups, 2) { case (g, gNames) => + out.byte(g) + val names = gNames.map(_._1) + if (sort) Arrays.sort(names, implicitly[Ordering[String]]) + out.writeStringColl("names", names) + } + } + } + + private[this] def readUsedNameSet(in: Deserializer): Set[UsedName] = { + in.readBlock { + val data = in.readColl[Vector[UsedName], Vector[Vector[UsedName]]](Vector, 2) { + val i = in.byte().toInt + val names = in.readStringSeq() + names.iterator.map { n => UsedName(n, useScopes(i)) }.toVector + } + data.flatten.toSet + } + } + + private[this] def writeClassLike(out: Serializer, cl: ClassLike): Unit = + out.writeBlock("ClassLike") { + out.string(cl.name()) + writeAccess(out, cl.access()) + out.byte(cl.modifiers().raw()) + out.writeArray("annotations", cl.annotations())(writeAnnotation(out, _)) + writeDefinitionType(out, cl.definitionType()) + writeType(out, cl.selfType()) + writeStructure(out, cl.structure()) + out.writeStringArray("savedAnnotations", cl.savedAnnotations()) + out.writeArray("childrenOfSealedClass", cl.childrenOfSealedClass())(writeType(out, _)) + out.bool(cl.topLevel()) + out.writeArray("typeParameters", cl.typeParameters())(writeTypeParameter(out, _)) + } + + private[this] def readClassLike(in: Deserializer): ClassLike = in.readBlock { + val name = in.string() + val access = readAccess(in) + val modifiers = InternalApiProxy.Modifiers(in.byte().toInt) + val annotations = in.readArray[Annotation]()(readAnnotation(in)) + val definitionType = readDefinitionType(in) + val selfType = SafeLazyProxy.strict(readType(in)) + val structure = SafeLazyProxy.strict(readStructure(in)) + val savedAnnotations = in.readStringArray() + val childrenOfSealedClass = in.readArray[Type]()(readType(in)) + val topLevel = in.bool() + val typeParameters = in.readArray[TypeParameter]()(readTypeParameter(in)) + ClassLike.of( + name, + access, + modifiers, + annotations, + definitionType, + selfType, + structure, + savedAnnotations, + childrenOfSealedClass, + topLevel, + typeParameters + ) + } + + private[this] def writeAccess(out: Serializer, access: Access): Unit = out.writeBlock("Access") { + def writeQualifier(q: Qualifier): Unit = q match { + case q: IdQualifier => out.byte(0); out.string(q.value()) + case _: ThisQualifier => out.byte(1) + case _: Unqualified => out.byte(2) + } + access match { + case _: Public => out.byte(0) + case access: Protected => out.byte(1); writeQualifier(access.qualifier()) + case access: Private => out.byte(2); writeQualifier(access.qualifier()) + } + } + + private[this] def readAccess(in: Deserializer): Access = in.readBlock { + def readQualifier(): Qualifier = in.byte() match { + case 0 => IdQualifier.of(in.string()) + case 1 => ThisQualifierSingleton + case 2 => UnqualifiedSingleton + } + in.byte() match { + case 0 => PublicSingleton + case 1 => Protected.of(readQualifier()) + case 2 => Private.of(readQualifier()) + } + } + + private[this] def writeAnnotation(out: Serializer, a: Annotation): Unit = + out.writeBlock("Annotation") { + writeType(out, a.base()) + out.writeArray("arguments", a.arguments(), 2) { a => + out.string(a.name()); out.string(a.value()) + } + } + + private[this] def readAnnotation(in: Deserializer): Annotation = in.readBlock { + val base = readType(in) + val args = in.readArray(2)(AnnotationArgument.of(in.string(), in.string())) + Annotation.of(base, args) + } + + private[this] def writeDefinitionType(out: Serializer, dt: DefinitionType): Unit = + out.byte(dt.ordinal().toByte) + + private[this] def readDefinitionType(in: Deserializer): DefinitionType = + DefinitionType.values()(in.byte().toInt) + + private[this] def writeTypeParameter(out: Serializer, tp: TypeParameter): Unit = + out.writeBlock("TypeParameter") { + out.string(tp.id()) + out.writeArray("annotations", tp.annotations())(writeAnnotation(out, _)) + out.writeArray("typeParameters", tp.typeParameters())(writeTypeParameter(out, _)) + out.byte(tp.variance().ordinal().toByte) + writeType(out, tp.lowerBound()) + writeType(out, tp.upperBound()) + } + + private[this] def readTypeParameter(in: Deserializer): TypeParameter = in.readBlock { + TypeParameter.of( + in.string(), + in.readArray[Annotation]()(readAnnotation(in)), + in.readArray[TypeParameter]()(readTypeParameter(in)), + Variance.values()(in.byte().toInt), + readType(in), + readType(in) + ) + } + + private[this] def writeType(out: Serializer, tpe: Type): Unit = out.writeBlock("Type") { + tpe match { + case tpe: ParameterRef => + out.byte(0) + out.string(tpe.id()) + case tpe: Parameterized => + out.byte(1) + writeType(out, tpe.baseType()) + out.writeArray("typeArguments", tpe.typeArguments())(writeType(out, _)) + case tpe: Structure => + out.byte(2) + writeStructure(out, tpe) + case tpe: Polymorphic => + out.byte(3) + writeType(out, tpe.baseType()) + out.writeArray("parameters", tpe.parameters())(writeTypeParameter(out, _)) + case tpe: Constant => + out.byte(4) + writeType(out, tpe.baseType()) + out.string(tpe.value()) + case tpe: Existential => + out.byte(5) + writeType(out, tpe.baseType()) + out.writeArray("clause", tpe.clause())(writeTypeParameter(out, _)) + case tpe: Singleton => + out.byte(6) + writePath(out, tpe.path()) + case tpe: Projection => + out.byte(7) + writeType(out, tpe.prefix()) + out.string(tpe.id()) + case tpe: Annotated => + out.byte(8) + writeType(out, tpe.baseType()) + out.writeArray("annotations", tpe.annotations())(writeAnnotation(out, _)) + case _: EmptyType => out.byte(9) + } + } + + private[this] def readType(in: Deserializer): Type = in.readBlock { + in.byte() match { + case 0 => ParameterRef.of(in.string()) + case 1 => Parameterized.of(readType(in), in.readArray[Type]()(readType(in))) + case 2 => readStructure(in) + case 3 => Polymorphic.of(readType(in), in.readArray[TypeParameter]()(readTypeParameter(in))) + case 4 => Constant.of(readType(in), in.string()) + case 5 => Existential.of(readType(in), in.readArray[TypeParameter]()(readTypeParameter(in))) + case 6 => Singleton.of(readPath(in)) + case 7 => Projection.of(readType(in), in.string()) + case 8 => Annotated.of(readType(in), in.readArray[Annotation]()(readAnnotation(in))) + case 9 => EmptyTypeSingleton + } + } + + private[this] def writeStructure(out: Serializer, tpe: Structure): Unit = + out.writeBlock("Structure") { + out.writeArray("parents", tpe.parents())(writeType(out, _)) + out.writeArray("declared", tpe.declared())(writeClassDefinition(out, _)) + out.writeArray("inherited", tpe.inherited())(writeClassDefinition(out, _)) + } + + private[this] def readStructure(in: Deserializer): Structure = in.readBlock { + val parents = in.readArray[Type]()(readType(in)) + val declared, inherited = in.readArray[ClassDefinition]()(readClassDefinition(in)) + Structure.of( + SafeLazyProxy.strict(parents), + SafeLazyProxy.strict(declared), + SafeLazyProxy.strict(inherited) + ) + } + + private[this] def writeClassDefinition(out: Serializer, cd: ClassDefinition): Unit = + out.writeBlock("ClassDefinition") { + out.string(cd.name()) + writeAccess(out, cd.access()) + out.byte(cd.modifiers().raw()) + out.writeArray("annotations", cd.annotations())(writeAnnotation(out, _)) + cd match { + case cd: ClassLikeDef => + out.byte(0) + out.writeArray("typeParameters", cd.typeParameters())(writeTypeParameter(out, _)) + writeDefinitionType(out, cd.definitionType()) + case cd: Val => + out.byte(1) + writeType(out, cd.tpe) + case cd: Var => + out.byte(2) + writeType(out, cd.tpe) + case cd: Def => + out.byte(3) + out.writeArray("typeParameters", cd.typeParameters())(writeTypeParameter(out, _)) + out.writeArray("valueParameters", cd.valueParameters())(writeParameterList(out, _)) + writeType(out, cd.returnType()) + case cd: TypeAlias => + out.byte(4) + out.writeArray("typeParameters", cd.typeParameters())(writeTypeParameter(out, _)) + writeType(out, cd.tpe()) + case cd: TypeDeclaration => + out.byte(5) + out.writeArray("typeParameters", cd.typeParameters())(writeTypeParameter(out, _)) + writeType(out, cd.lowerBound()) + writeType(out, cd.upperBound()) + } + } + + private[this] def readClassDefinition(in: Deserializer): ClassDefinition = in.readBlock { + val name = in.string() + val access = readAccess(in) + val modifiers = InternalApiProxy.Modifiers(in.byte().toInt) + val annotations = in.readArray[Annotation]()(readAnnotation(in)) + in.byte() match { + case 0 => ClassLikeDef.of( + name, + access, + modifiers, + annotations, + in.readArray[TypeParameter]()(readTypeParameter(in)), + readDefinitionType(in) + ) + case 1 => Val.of(name, access, modifiers, annotations, readType(in)) + case 2 => Var.of(name, access, modifiers, annotations, readType(in)) + case 3 => Def.of( + name, + access, + modifiers, + annotations, + in.readArray[TypeParameter]()(readTypeParameter(in)), + in.readArray[ParameterList]()(readParameterList(in)), + readType(in) + ) + case 4 => TypeAlias.of( + name, + access, + modifiers, + annotations, + in.readArray[TypeParameter]()(readTypeParameter(in)), + readType(in) + ) + case 5 => TypeDeclaration.of( + name, + access, + modifiers, + annotations, + in.readArray[TypeParameter]()(readTypeParameter(in)), + readType(in), + readType(in) + ) + } + } + + private[this] def writeParameterList(out: Serializer, pl: ParameterList): Unit = + out.writeBlock("ParameterList") { + out.writeArray("parameters", pl.parameters(), 4) { mp => + out.string(mp.name()) + writeType(out, mp.tpe()) + out.bool(mp.hasDefault) + out.byte(mp.modifier().ordinal().toByte) + } + out.bool(pl.isImplicit) + } + + private[this] def readParameterList(in: Deserializer): ParameterList = in.readBlock { + ParameterList.of( + in.readArray[MethodParameter](4) { + MethodParameter.of( + in.string(), + readType(in), + in.bool(), + ParameterModifier.values()(in.byte().toInt) + ) + }, + in.bool() + ) + } + + private[this] def isSimplePath(comps: Array[PathComponent]): Boolean = { + if (comps.isEmpty || !comps.last.isInstanceOf[This]) false + else { + var i = 0 + while (i < comps.length - 1) { + if (!comps(i).isInstanceOf[Id]) return false + i += 1 + } + true + } + } + + private[this] def writePath(out: Serializer, path: Path): Unit = out.writeBlock("Path") { + out.dedup(path)(_.components().length) { + val comps = path.components() + val simple = isSimplePath(comps) + if (simple) { + out.byte(0) + var i = 0 + while (i < comps.length - 1) { + out.string(comps(i).asInstanceOf[Id].id) + i += 1 + } + } else { + var i = 0 + while (i < comps.length) { + comps(i) match { + case c: Id => out.byte(1); out.string(c.id) + case c: Super => out.byte(2); writePath(out, c.qualifier) + case _: This => out.byte(3); out.writeBlock("This") {} + } + i += 1 + } + } + } + } + + private[this] def readPath(in: Deserializer): Path = { + in.dedup[Path] { len => + val comps = new Array[PathComponent](len) + val kind = in.byte() + if (kind == 0) { // simple path + var i = 0 + while (i < len - 1) { + comps(i) = Id.of(in.string()) + i += 1 + } + comps(i) = ThisSingleton + } else { + var i = 0 + while (i < len) { + val k = if (i == 0) kind else in.byte() // we already read the first kind + comps(i) = k match { + case 1 => Id.of(in.string()) + case 2 => Super.of(readPath(in)) + case 3 => in.readBlock {}; ThisSingleton + } + i += 1 + } + } + Path.of(comps) + } + } +} + +object ConsistentAnalysisFormat { + private final val EmptyTypeSingleton = EmptyType.of() + private final val ThisSingleton = This.of() + private final val ThisQualifierSingleton = ThisQualifier.of() + private final val UnqualifiedSingleton = Unqualified.of() + private final val PublicSingleton = Public.of() + private final val DefaultCompilationTimestamp: Long = 1262304042000L // 2010-01-01T00:00:42Z + + private final val useScopes: Array[EnumSet[UseScope]] = + Array.tabulate(8) { i => + val e = EnumSet.noneOf(classOf[UseScope]) + if ((i & 1) != 0) e.add(UseScope.Default) + if ((i & 2) != 0) e.add(UseScope.Implicit) + if ((i & 4) != 0) e.add(UseScope.PatMatTarget) + e + } + + private final val nameHashComparator: Comparator[NameHash] = new Comparator[NameHash] { + def compare(o1: NameHash, o2: NameHash): Int = { + o1.name().compareTo(o2.name()) match { + case 0 => o1.scope().ordinal() - o2.scope().ordinal() + case i => i + } + } + } +} diff --git a/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ConsistentFileAnalysisStore.scala b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ConsistentFileAnalysisStore.scala new file mode 100644 index 0000000000..01d5af4742 --- /dev/null +++ b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ConsistentFileAnalysisStore.scala @@ -0,0 +1,91 @@ +// Based on zinc's FileAnalysisStore: +package sbt.internal.inc.consistent + +/* + * Zinc - The incremental compiler for Scala. + * Copyright Lightbend, Inc. and Mark Harrah + * + * Licensed under Apache License 2.0 + * (http://www.apache.org/licenses/LICENSE-2.0). + * + * See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. + */ + +import java.io.{ File, FileInputStream, FileOutputStream } +import java.util.Optional +import sbt.io.{ IO, Using } +import xsbti.compile.{ AnalysisContents, AnalysisStore => XAnalysisStore } + +import scala.util.control.Exception.allCatch +import xsbti.compile.analysis.ReadWriteMappers + +import scala.concurrent.ExecutionContext + +object ConsistentFileAnalysisStore { + def text( + file: File, + mappers: ReadWriteMappers, + sort: Boolean = true, + ec: ExecutionContext = ExecutionContext.global, + parallelism: Int = Runtime.getRuntime.availableProcessors() + ): XAnalysisStore = + new AStore( + file, + new ConsistentAnalysisFormat(mappers, sort), + SerializerFactory.text, + ec, + parallelism + ) + + def binary( + file: File, + mappers: ReadWriteMappers, + sort: Boolean = true, + ec: ExecutionContext = ExecutionContext.global, + parallelism: Int = Runtime.getRuntime.availableProcessors() + ): XAnalysisStore = + new AStore( + file, + new ConsistentAnalysisFormat(mappers, sort), + SerializerFactory.binary, + ec, + parallelism + ) + + private final class AStore[S <: Serializer, D <: Deserializer]( + file: File, + format: ConsistentAnalysisFormat, + sf: SerializerFactory[S, D], + ec: ExecutionContext = ExecutionContext.global, + parallelism: Int = Runtime.getRuntime.availableProcessors() + ) extends XAnalysisStore { + + def set(analysisContents: AnalysisContents): Unit = { + val analysis = analysisContents.getAnalysis + val setup = analysisContents.getMiniSetup + val tmpAnalysisFile = File.createTempFile(file.getName, ".tmp") + if (!file.getParentFile.exists()) file.getParentFile.mkdirs() + val fout = new FileOutputStream(tmpAnalysisFile) + try { + val gout = new ParallelGzipOutputStream(fout, ec, parallelism) + val ser = sf.serializerFor(gout) + format.write(ser, analysis, setup) + gout.close() + } finally fout.close + IO.move(tmpAnalysisFile, file) + } + + def get(): Optional[AnalysisContents] = { + import sbt.internal.inc.JavaInterfaceUtil.EnrichOption + allCatch.opt(unsafeGet()).toOptional + } + + def unsafeGet(): AnalysisContents = + Using.gzipInputStream(new FileInputStream(file)) { in => + val deser = sf.deserializerFor(in) + val (analysis, setup) = format.read(deser) + AnalysisContents.create(analysis, setup) + } + } +} diff --git a/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ParallelGzipOutputStream.scala b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ParallelGzipOutputStream.scala new file mode 100644 index 0000000000..bdce23d100 --- /dev/null +++ b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/ParallelGzipOutputStream.scala @@ -0,0 +1,123 @@ +package sbt.internal.inc.consistent + +import java.io.{ ByteArrayOutputStream, FilterOutputStream, OutputStream } +import java.util.zip.{ CRC32, Deflater, DeflaterOutputStream } + +import scala.annotation.tailrec +import scala.concurrent.duration.Duration +import scala.concurrent.{ Await, ExecutionContext, Future } +import scala.collection.mutable + +/** + * Parallel gzip compression. Algorithm based on https://github.com/shevek/parallelgzip + * with additional optimization and simplification. This is essentially a block-buffered + * stream but instead of writing a full block to the underlying output, it is passed to a + * thread pool for compression and the Futures of compressed blocks are collected when + * flushing. + */ +object ParallelGzipOutputStream { + private val blockSize = 64 * 1024 + private val compression = Deflater.DEFAULT_COMPRESSION + + private class BufOut(size: Int) extends ByteArrayOutputStream(size) { + def writeTo(buf: Array[Byte]): Unit = System.arraycopy(this.buf, 0, buf, 0, count) + } + + private class Worker { + private[this] val defl = new Deflater(compression, true) + private[this] val buf = new BufOut(blockSize + (blockSize >> 3)) + private[this] val out = new DeflaterOutputStream(buf, defl, true) + def compress(b: Block): Unit = { + defl.reset() + buf.reset() + out.write(b.data, 0, b.length) + out.flush() + b.length = buf.size + if (b.length > b.data.length) b.data = new Array[Byte](b.length) + buf.writeTo(b.data) + } + } + + private val localWorker = new ThreadLocal[Worker] { + override def initialValue = new Worker + } + + private class Block { + var data = new Array[Byte](blockSize + (blockSize >> 3)) + var length = 0 + } + + private val header = Array[Byte](0x1f.toByte, 0x8b.toByte, Deflater.DEFLATED, 0, 0, 0, 0, 0, 0, 0) +} + +final class ParallelGzipOutputStream(out: OutputStream, ec: ExecutionContext, parallelism: Int) + extends FilterOutputStream(out) { + import ParallelGzipOutputStream._ + + private final val crc = new CRC32 + private final val queueLimit = parallelism * 3 + // preferred on 2.13: new mutable.ArrayDeque[Future[Block]](queueLimit) + private final val pending = mutable.Queue.empty[Future[Block]] + private var current: Block = new Block + private var free: Block = _ + private var total = 0L + + out.write(header) + + override def write(b: Int): Unit = write(Array[Byte]((b & 0xff).toByte)) + override def write(b: Array[Byte]): Unit = write(b, 0, b.length) + + @tailrec override def write(b: Array[Byte], off: Int, len: Int): Unit = { + val copy = math.min(len, blockSize - current.length) + crc.update(b, off, copy) + total += copy + System.arraycopy(b, off, current.data, current.length, copy) + current.length += copy + if (copy < len) { + submit() + write(b, off + copy, len - copy) + } + } + + private[this] def submit(): Unit = { + flushUntil(queueLimit - 1) + val finalBlock = current + pending += Future { localWorker.get.compress(finalBlock); finalBlock }(ec) + if (free != null) { + current = free + free = null + } else current = new Block() + } + + private def flushUntil(remaining: Int): Unit = + while (pending.length > remaining || pending.headOption.exists(_.isCompleted)) { + val b = Await.result(pending.dequeue(), Duration.Inf) + out.write(b.data, 0, b.length) + b.length = 0 + free = b + } + + override def flush(): Unit = { + if (current.length > 0) submit() + flushUntil(0) + super.flush() + } + + override def close(): Unit = { + flush() + val buf = new Array[Byte](10) + def int(i: Int, off: Int): Unit = { + buf(off) = ((i & 0xff).toByte) + buf(off + 1) = (((i >>> 8) & 0xff).toByte) + buf(off + 2) = (((i >>> 16) & 0xff).toByte) + buf(off + 3) = (((i >>> 24) & 0xff).toByte) + } + buf(0) = 3 + int(crc.getValue.toInt, 2) + int((total & 0xffffffffL).toInt, 6) + out.write(buf) + out.close() + total = Integer.MIN_VALUE + free = null + } +} diff --git a/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/Serializer.scala b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/Serializer.scala new file mode 100644 index 0000000000..8f111a9b6f --- /dev/null +++ b/internal/zinc-persist/src/main/scala/sbt/internal/inc/consistent/Serializer.scala @@ -0,0 +1,516 @@ +package sbt.internal.inc.consistent + +import java.io.{ + BufferedReader, + BufferedWriter, + EOFException, + InputStream, + InputStreamReader, + OutputStream, + OutputStreamWriter, + Writer +} +import java.nio.charset.StandardCharsets +import java.util.Optional + +import scala.annotation.tailrec +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable +import scala.reflect.ClassTag +import Compat._ + +/** Structural serialization for text and binary formats. */ +abstract class Serializer { + private final val dedupMap: mutable.Map[AnyRef, Int] = mutable.Map.empty + + def startBlock(name: String): Unit + def startArray(name: String, length: Int): Unit // use length = -1 for null + def endBlock(): Unit + def endArray(): Unit + def string(s: String): Unit + def bool(b: Boolean): Unit + def int(i: Int): Unit + def byte(b: Byte): Unit + def long(l: Long): Unit + def end(): Unit + + @inline final def dedup[T >: Null <: AnyRef](o: T)(id: T => Int)(writeBody: => Unit): Unit = { + if (o == null) int(-1) + else { + val nextId = -2 - dedupMap.size + val idx = dedupMap.getOrElseUpdate(o, nextId) + if (idx == nextId) { + int(id(o)) + writeBody + } else int(idx) + } + } + @inline final def writeArray[T]( + name: String, + a: Array[T], + perEntry: Int = 1 + )(f: T => Unit): Unit = { + if (a == null) startArray(name, -1) + else { + startArray(name, a.length * perEntry) + a.foreach(f) + } + endArray() + } + @inline final def writeStringArray(name: String, a: Array[String]): Unit = + writeArray(name, a)(string) + + @inline final def writeColl[T]( + name: String, + a: Iterable[T], + perEntry: Int = 1 + )(f: T => Unit): Unit = { + if (a == null) startArray(name, -1) + else { + startArray(name, a.size * perEntry) + a.iterator.foreach(f) + } + endArray() + } + + @inline final def writeStringColl(name: String, a: Iterable[String]): Unit = + writeColl(name, a)(string) + + @inline final def writeBlock(name: String)(f: => Unit): Unit = { + startBlock(name) + f + endBlock() + } + @inline final def writeSortedStringMap[V]( + name: String, + map: scala.collection.Iterable[(String, V)], + perEntry: Int = 1 + )(f: V => Unit): Unit = { + if (map == null) { + startArray(name, -1) + endArray() + } else { + val a = map.toArray + a.sortInPlaceBy(_._1) + writeArray(name, a, perEntry + 1) { kv => + string(kv._1) + f(kv._2) + } + } + } + @inline final def writeOptionalString(o: Optional[String]): Unit = string(o.orElse(null)) +} + +/** Derialization for text and binary formats produced by Serializer. */ +abstract class Deserializer { + private final val dedupBuffer: ArrayBuffer[AnyRef] = ArrayBuffer.empty + + def startBlock(): Unit + def startArray(): Int + def endBlock(): Unit + def endArray(): Unit + def string(): String + def bool(): Boolean + def int(): Int + def byte(): Byte + def long(): Long + def end(): Unit + + @inline final def dedup[T >: Null <: AnyRef](readBody: Int => T): T = int() match { + case -1 => null + case id if id >= 0 => + val o = readBody(id) + dedupBuffer += o + o + case idx => + dedupBuffer(-2 - idx).asInstanceOf[T] + } + + @inline final def readArray[T: ClassTag](perEntry: Int = 1)(f: => T): Array[T] = { + startArray() match { + case -1 => + endArray() + null + case rawLen => + val len = rawLen / perEntry + val a = new Array[T](len) + var i = 0 + while (i < a.length) { a(i) = f; i += 1 } + endArray() + a + } + } + @inline final def readStringArray(): Array[String] = readArray[String]()(string()) + @inline final def readColl[T, C >: Null]( + factory: Factory[T, C], + perEntry: Int = 1 + )(f: => T): C = { + startArray() match { + case -1 => + endArray() + null + case rawLen => + val b = factory.newBuilder + val len = rawLen / perEntry + b.sizeHint(len) + var i = 0 + while (i < len) { b += f; i += 1 } + endArray() + b.result() + } + } + @inline final def readBlock[T](f: => T): T = { + startBlock() + val r = f + endBlock() + r + } + @inline final def readStringSeq(): Seq[String] = + readColl[String, Vector[String]](Vector)(string()) + @inline final def readOptionalString(): Optional[String] = string() match { + case null => Optional.empty[String] + case s => Optional.of(s) + } +} + +class TextSerializer(out: Writer) extends Serializer { + private[this] final class Block(val array: Boolean, val expected: Int, var actual: Int) + private[this] var indent = 0 + private[this] var stack: List[Block] = Nil + private[this] def printIndent(): Unit = (0 until indent * 2).foreach(_ => out.write(' ')) + private[this] def count(): Unit = + if (stack.nonEmpty) stack.head.actual += 1 + private[this] def println(s: String): Unit = { + out.write(s) + out.write('\n') + } + def startBlock(name: String): Unit = { + count() + printIndent() + println(name + " {") + stack = new Block(false, 0, 0) :: stack + indent += 1 + } + def startArray(name: String, length: Int): Unit = { + count() + printIndent() + println(name + " [") + stack = new Block(true, length max 0, 0) :: stack + indent += 1 + printIndent() + println(length.toString) + } + def endBlock(): Unit = { + assert(stack.nonEmpty) + val b = stack.head + stack = stack.tail + assert(!b.array) + indent -= 1 + printIndent() + println("}") + } + def endArray(): Unit = { + assert(stack.nonEmpty) + val b = stack.head + stack = stack.tail + assert(b.array) + assert(b.expected == b.actual, s"Expected to write ${b.expected} values, wrote ${b.actual}") + indent -= 1 + printIndent() + println("]") + } + def string(s: String): Unit = { + count() + printIndent() + if (s == null) out.write("\\0\n") + else { + s.foreach { + case '\n' => out.write("\\n") + case '\r' => out.write("\\r") + case '\\' => out.write("\\\\") + case c => out.write(c.toInt) + } + out.write('\n') + } + } + def bool(b: Boolean): Unit = long(if (b) 1 else 0) + def int(i: Int): Unit = long(i.toLong) + def byte(b: Byte): Unit = long(b.toLong) + def long(l: Long): Unit = { + count() + printIndent() + println(l.toString) + } + def end(): Unit = { + out.flush() + assert(stack.isEmpty && indent == 0) + } +} + +class TextDeserializer(in: BufferedReader) extends Deserializer { + private[this] final class Block(val array: Boolean, val expected: Int, var actual: Int) + private[this] var indent = 0 + private[this] var stack: List[Block] = Nil + private[this] def raw(): String = in.readLine().drop(indent * 2) + private[this] def count(): Unit = + if (stack.nonEmpty) stack.head.actual += 1 + def startBlock(): Unit = { + count() + val r = raw() + if (!r.endsWith(" {")) new IllegalStateException("Expected block header") + indent += 1 + stack = new Block(false, 0, 0) :: stack + } + def startArray(): Int = { + count() + val r = raw() + assert(r.endsWith(" ["), "Expected array header") + indent += 1 + val length = raw().toInt + stack = new Block(true, length max 0, 0) :: stack + length + } + def endBlock(): Unit = { + assert(stack.nonEmpty) + val b = stack.head + stack = stack.tail + assert(!b.array) + indent -= 1 + val r = raw() + assert(r == "}") + } + def endArray(): Unit = { + assert(stack.nonEmpty) + val b = stack.head + stack = stack.tail + assert(b.array) + assert(b.expected == b.actual, s"Expected to read ${b.expected} values, read ${b.actual}") + indent -= 1 + val r = raw() + assert(r == "]") + } + def string(): String = { + count() + val s = raw() + var i = 0 + val b = new StringBuilder + while (i < s.length) { + s.charAt(i) match { + case '\\' => + i += 1 + s.charAt(i) match { + case '0' => return null + case 'n' => b.append('\n') + case 'r' => b.append('\r') + case '\\' => b.append('\\') + } + case c => b.append(c) + } + i += 1 + } + b.result() + } + def bool(): Boolean = long() == 1L + def int(): Int = long().toInt + def byte(): Byte = long().toByte + def long(): Long = { + count() + raw().toLong + } + def end(): Unit = assert(stack.isEmpty && indent == 0) +} + +class BinarySerializer(_out: OutputStream) extends Serializer { + private[this] val stringsMap: mutable.Map[String, Int] = mutable.Map.empty + private[this] val buffer: Array[Byte] = new Array(65536) + private[this] var pos: Int = 0 + // Ensure that at least `count` bytes can be written to the buffer starting at `pos` + @inline private[this] def ensure(count: Int): Unit = + if (pos + count > buffer.length) flush() + // Flush unconditionally, ensuring `pos` = 0 + @inline private[this] def flush(): Unit = { + if (pos > 0) _out.write(buffer, 0, pos) + pos = 0 + } + @inline private[this] def unsafeWriteByte(b: Byte): Unit = { + buffer(pos) = b + pos += 1 + } + def startBlock(name: String): Unit = () + def endBlock(): Unit = () + def startArray(name: String, length: Int): Unit = int(length) + def endArray(): Unit = () + def string(s: String): Unit = { + if (s == null) int(-1) + else if (s.isEmpty) int(0) + else { + val nextString = -2 - stringsMap.size + val idx = stringsMap.getOrElseUpdate(s, nextString) + if (idx == nextString) { + val bytes = s.getBytes(StandardCharsets.UTF_8) + val len = bytes.length + int(len) + if (len <= buffer.length) { + ensure(len) + System.arraycopy(bytes, 0, buffer, pos, len) + pos += len + } else { + flush() + _out.write(bytes) + } + } else int(idx) + } + } + def bool(b: Boolean): Unit = byte(if (b) 1 else 0) + def int(i: Int): Unit = { + ensure(4) + unsafeWriteByte(((i >>> 24) & 0xff).toByte) + unsafeWriteByte(((i >>> 16) & 0xff).toByte) + unsafeWriteByte(((i >>> 8) & 0xff).toByte) + unsafeWriteByte((i & 0xff).toByte) + } + def byte(b: Byte): Unit = { ensure(1); unsafeWriteByte(b) } + def long(l: Long): Unit = { + ensure(8) + unsafeWriteByte(((l >>> 56).toInt & 0xff).toByte) + unsafeWriteByte(((l >>> 48).toInt & 0xff).toByte) + unsafeWriteByte(((l >>> 40).toInt & 0xff).toByte) + unsafeWriteByte(((l >>> 32).toInt & 0xff).toByte) + unsafeWriteByte(((l >>> 24).toInt & 0xff).toByte) + unsafeWriteByte(((l >>> 16).toInt & 0xff).toByte) + unsafeWriteByte(((l >>> 8).toInt & 0xff).toByte) + unsafeWriteByte((l.toInt & 0xff).toByte) + } + def end(): Unit = { flush(); _out.flush() } +} + +class BinaryDeserializer(_in: InputStream) extends Deserializer { + private[this] val strings: ArrayBuffer[String] = ArrayBuffer.empty + private[this] val buffer: Array[Byte] = new Array(8192) + private[this] var pos: Int = buffer.length + private[this] var bufLen: Int = buffer.length + @inline @tailrec private[this] def readAllUnderlying( + a: Array[Byte], + off: Int, + len: Int, + accum: Int = 0 + ): Int = { + val read = _in.read(a, off, len) + if (read == -1 && accum == 0) -1 + else if (read == -1) accum + else if (read == len) accum + read + else readAllUnderlying(a, off + read, len - read, accum + read) + } + // Ensure that there are at least `count` bytes to read in the buffer starting at `pos` + @inline private[this] def ensure(count: Int): Unit = { + if (pos + count > bufLen) { + if (pos + count > buffer.length || pos >= buffer.length / 2) moveToLeft() + while ({ + val read = _in.read(buffer, bufLen, buffer.length - bufLen) + if (read <= 0) throw new EOFException() + bufLen += read + bufLen - pos < count + }) {} + } + } + // Move the data in the buffer so that `pos` = 0 + @inline private[this] def moveToLeft(): Unit = { + val rem = bufLen - pos + if (rem > 0 && pos > 0) System.arraycopy(buffer, pos, buffer, 0, rem) + pos = 0 + bufLen = rem + } + @inline private[this] def unsafeReadByte(): Byte = { + val b = buffer(pos) + pos += 1 + b + } + @inline private[this] def readInto(a: Array[Byte]): Int = { + var off = 0 + var len = a.length + if (pos < bufLen) { + val toCopy = len min (bufLen - pos) + System.arraycopy(buffer, pos, a, 0, toCopy) + len -= toCopy + off += toCopy + pos += toCopy + } + if (len > 0) { + if (len >= buffer.length) off += readAllUnderlying(a, off, len) + else { + bufLen = readAllUnderlying(buffer, 0, buffer.length) + val toRead = len min bufLen + System.arraycopy(buffer, 0, a, off, toRead) + pos = toRead + off += toRead + } + } + off + } + + def startBlock(): Unit = () + def endBlock(): Unit = () + def startArray(): Int = int() + def endArray(): Unit = () + def string(): String = int() match { + case -1 => null + case 0 => "" + case len if len > 0 => + val s = if (len <= buffer.length) { + ensure(len) + val s = new String(buffer, pos, len, StandardCharsets.UTF_8) + pos += len + s + } else { + val a = new Array[Byte](len) + val read = readInto(a) + assert(read == len) + new String(a, StandardCharsets.UTF_8) + } + strings += s + s + case idx => + strings(-2 - idx) + } + def bool(): Boolean = byte() != 0 + def int(): Int = { + ensure(4) + val i1, i2, i3, i4 = unsafeReadByte() & 0xff + (i1 << 24) | (i2 << 16) | (i3 << 8) | i4 + } + def byte(): Byte = { + ensure(1) + unsafeReadByte() + } + def long(): Long = { + ensure(8) + val i1, i2, i3, i4, i5, i6, i7, i8 = unsafeReadByte() & 0xffL + (i1 << 56) | (i2 << 48) | (i3 << 40) | (i4 << 32) | (i5 << 24) | (i6 << 16) | (i7 << 8) | i8 + } + def end(): Unit = () +} + +trait SerializerFactory[S <: Serializer, D <: Deserializer] { + def serializerFor(out: OutputStream): S + def deserializerFor(in: InputStream): D +} + +object SerializerFactory { + + /** Simple human-readable text format, not self-describing, not optimized for performance. Has + * checks for structural correctness in serializer and deserializer. */ + val text: SerializerFactory[TextSerializer, TextDeserializer] = + new SerializerFactory[TextSerializer, TextDeserializer] { + def serializerFor(out: OutputStream): TextSerializer = + new TextSerializer(new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) + def deserializerFor(in: InputStream): TextDeserializer = + new TextDeserializer(new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) + } + + /** Optimized binary format with string deduplication across multiple outputs. */ + val binary: SerializerFactory[BinarySerializer, BinaryDeserializer] = + new SerializerFactory[BinarySerializer, BinaryDeserializer] { + def serializerFor(out: OutputStream): BinarySerializer = new BinarySerializer(out) + def deserializerFor(in: InputStream): BinaryDeserializer = new BinaryDeserializer(in) + } +} diff --git a/internal/zinc-persist/src/test/scala/sbt/inc/consistent/ConsistentAnalysisFormatIntegrationSuite.scala b/internal/zinc-persist/src/test/scala/sbt/inc/consistent/ConsistentAnalysisFormatIntegrationSuite.scala new file mode 100644 index 0000000000..1e26809df9 --- /dev/null +++ b/internal/zinc-persist/src/test/scala/sbt/inc/consistent/ConsistentAnalysisFormatIntegrationSuite.scala @@ -0,0 +1,64 @@ +package sbt.inc.consistent + +import java.io.File +import java.util.Arrays +import org.scalatest.funsuite.AnyFunSuite +import sbt.internal.inc.consistent.ConsistentFileAnalysisStore +import sbt.internal.inc.{ Analysis, FileAnalysisStore } +import sbt.io.IO +import xsbti.compile.{ AnalysisContents, AnalysisStore } +import xsbti.compile.analysis.ReadWriteMappers + +class ConsistentAnalysisFormatIntegrationSuite extends AnyFunSuite { + val data = + Seq("compiler.zip", "library.zip", "reflect.zip").map(f => new File("../../../test-data", f)) + + test("Consistent output") { + for (d <- data) { + assert(d.exists()) + val api = read(FileAnalysisStore.binary(d)) + val f1 = write("cbin1.zip", api) + val f2 = write("cbin2.zip", api) + assert(Arrays.equals(IO.readBytes(f1), IO.readBytes(f2)), s"same output for $d") + } + } + + test("Roundtrip") { + for (d <- data) { + assert(d.exists()) + val api = read(FileAnalysisStore.binary(d)) + val f1 = write("cbin1.zip", api) + val api2 = read(ConsistentFileAnalysisStore.binary(f1, ReadWriteMappers.getEmptyMappers)) + val f2 = write("cbin2.zip", api2) + assert(Arrays.equals(IO.readBytes(f1), IO.readBytes(f2)), s"same output for $d") + } + } + + test("Unsorted roundtrip") { + for (d <- data) { + assert(d.exists()) + val api = read(FileAnalysisStore.binary(d)) + val f1 = write("cbin1.zip", api) + val api2 = read(ConsistentFileAnalysisStore.binary(f1, ReadWriteMappers.getEmptyMappers)) + val f2 = write("cbin2.zip", api2, sort = false) + val api3 = read(ConsistentFileAnalysisStore.binary(f2, ReadWriteMappers.getEmptyMappers)) + val f3 = write("cbin3.zip", api3) + assert(Arrays.equals(IO.readBytes(f1), IO.readBytes(f3)), s"same output for $d") + } + } + + def read(store: AnalysisStore): AnalysisContents = { + val api = store.unsafeGet() + // Force loading of companion file and check that the companion data is present: + assert(api.getAnalysis.asInstanceOf[Analysis].apis.internal.head._2.api() != null) + assert(api.getMiniSetup.storeApis()) + api + } + + def write(name: String, api: AnalysisContents, sort: Boolean = true): File = { + val out = new File(IO.temporaryDirectory, name) + if (out.exists()) IO.delete(out) + ConsistentFileAnalysisStore.binary(out, ReadWriteMappers.getEmptyMappers, sort).set(api) + out + } +} diff --git a/internal/zinc-persist/src/test/scala/sbt/inc/consistent/ConsistentAnalysisFormatSuite.scala b/internal/zinc-persist/src/test/scala/sbt/inc/consistent/ConsistentAnalysisFormatSuite.scala new file mode 100644 index 0000000000..a92fe2c6bb --- /dev/null +++ b/internal/zinc-persist/src/test/scala/sbt/inc/consistent/ConsistentAnalysisFormatSuite.scala @@ -0,0 +1,112 @@ +package sbt.inc.consistent + +import java.io.{ + BufferedInputStream, + BufferedReader, + ByteArrayInputStream, + ByteArrayOutputStream, + StringReader, + StringWriter +} +import java.util.zip.GZIPInputStream +import java.util.Arrays +import scala.util.Random +import org.scalatest.funsuite.AnyFunSuite +import sbt.internal.inc.consistent._ +import sbt.io.IO +import Compat._ + +import scala.concurrent.ExecutionContext + +class ConsistentAnalysisFormatSuite extends AnyFunSuite { + + def writeTo(out: Serializer): Unit = { + out.int(0) + out.int(Int.MinValue) + out.int(Int.MaxValue) + out.long(0) + out.long(Long.MinValue) + out.long(Long.MaxValue) + out.byte(0) + out.byte(Byte.MinValue) + out.byte(Byte.MaxValue) + out.bool(false) + out.bool(true) + out.string(null) + out.string("abc") + out.string("ab\r\n\\c") + out.writeBlock("block") { + out.int(42) + out.int(43) + } + out.writeColl("c1", null)(out.int) + out.writeColl("c2", Nil)(out.int) + out.writeColl("c3", Seq(1, 2, 3))(out.int) + out.writeColl("c4", Seq(1, 2, 3), 2) { i => out.int(i); out.int(i * 2) } + out.end() + } + + def readFrom(in: Deserializer): Unit = { + val i1, i2, i3 = in.int() + assert(i1 == 0) + assert(i2 == Int.MinValue) + assert(i3 == Int.MaxValue) + val l1, l2, l3 = in.long() + assert(l1 == 0) + assert(l2 == Long.MinValue) + assert(l3 == Long.MaxValue) + val b1, b2, b3 = in.byte() + assert(b1 == 0) + assert(b2 == Byte.MinValue) + assert(b3 == Byte.MaxValue) + val p1, p2 = in.bool() + assert(p1 == false) + assert(p2 == true) + val s1, s2, s3 = in.string() + assert(s1 == null) + assert(s2 == "abc") + assert(s3 == "ab\r\n\\c") + val (i4, i5) = in.readBlock((in.int(), in.int())) + assert(i4 == 42) + assert(i5 == 43) + val c1, c2, c3 = in.readColl[Int, Seq[Int]](Seq)(in.int()) + val c4 = in.readColl[(Int, Int), Seq[(Int, Int)]](Seq, 2)((in.int(), in.int())) + assert(c1 == null) + assert(c2.isEmpty) + assert(c3 == Seq(1, 2, 3)) + assert(c4 == Seq((1, 2), (2, 4), (3, 6))) + () + } + + test("TextSerializer") { + val out = new StringWriter() + writeTo(new TextSerializer(out)) + readFrom(new TextDeserializer(new BufferedReader(new StringReader(out.toString)))) + } + + test("BinarySerializer") { + val out = new ByteArrayOutputStream() + writeTo(SerializerFactory.binary.serializerFor(out)) + readFrom(SerializerFactory.binary.deserializerFor(new ByteArrayInputStream(out.toByteArray))) + } + + test("ParallelGzip") { + val bs = 64 * 1024 + val rnd = new Random(0L) + for { + threads <- Seq(1, 8) + size <- Seq(0, bs - 1, bs, bs + 1, bs * 8 - 1, bs * 8, bs * 8 + 1) + } { + val a = new Array[Byte](size) + rnd.nextBytes(a) + val bout = new ByteArrayOutputStream() + val gout = new ParallelGzipOutputStream(bout, ExecutionContext.global, parallelism = threads) + gout.write(a) + gout.close() + val gin = + new BufferedInputStream(new GZIPInputStream(new ByteArrayInputStream(bout.toByteArray))) + val a2 = IO.readBytes(gin) + assert(Arrays.equals(a, a2), s"threads = $threads, size = $size") + } + } +} diff --git a/test-data/compiler.zip b/test-data/compiler.zip new file mode 100644 index 0000000000..618df99c73 Binary files /dev/null and b/test-data/compiler.zip differ diff --git a/test-data/library.zip b/test-data/library.zip new file mode 100644 index 0000000000..d8c11bb8d9 Binary files /dev/null and b/test-data/library.zip differ diff --git a/test-data/reflect.zip b/test-data/reflect.zip new file mode 100644 index 0000000000..c028058e10 Binary files /dev/null and b/test-data/reflect.zip differ