From e6bf85a30b07ebb2f9cdfc92b69007d64800b012 Mon Sep 17 00:00:00 2001 From: Benjy Date: Sat, 16 Nov 2013 13:59:02 -0800 Subject: [PATCH] Replace binary Analysis format with a text-based one. Reads/writes are a little faster with the text format, and it's far more useful. E.g., it allows external manipulation and inspection of the analysis. We don't gzip the output. It does greatly shrink the files, however it makes reads and writes 1.5x-2x slower, and we're optimizing for speed over compactness. --- .../inc/src/main/scala/sbt/inc/Stamp.scala | 26 +- .../sbt/compiler/AggressiveCompile.scala | 2 - .../sbt/compiler/IncrementalCompiler.scala | 13 +- .../main/scala/sbt/inc/AnalysisFormats.scala | 2 +- .../main/scala/sbt/inc/FileBasedStore.scala | 16 +- .../src/main/scala/sbt/inc/InternPool.scala | 82 ----- .../sbt/inc/InternedAnalysisFormats.scala | 98 ----- .../scala/sbt/inc/TextAnalysisFormat.scala | 341 ++++++++++++++++++ 8 files changed, 375 insertions(+), 205 deletions(-) delete mode 100644 compile/persist/src/main/scala/sbt/inc/InternPool.scala delete mode 100644 compile/persist/src/main/scala/sbt/inc/InternedAnalysisFormats.scala create mode 100644 compile/persist/src/main/scala/sbt/inc/TextAnalysisFormat.scala diff --git a/compile/inc/src/main/scala/sbt/inc/Stamp.scala b/compile/inc/src/main/scala/sbt/inc/Stamp.scala index 4d8194538..98f216967 100644 --- a/compile/inc/src/main/scala/sbt/inc/Stamp.scala +++ b/compile/inc/src/main/scala/sbt/inc/Stamp.scala @@ -6,6 +6,7 @@ package inc import java.io.{File, IOException} import Stamp.getStamp +import scala.util.matching.Regex trait ReadStamps { @@ -47,19 +48,18 @@ sealed trait Stamp case o: Stamp => Stamp.equivStamp.equiv(this, o) case _ => false } + + override def toString: String = Stamp.toString(this) } final class Hash(val value: Array[Byte]) extends Stamp { override def hashCode: Int = java.util.Arrays.hashCode(value) - override def toString: String = "hash(" + Hash.toHex(value) + ")" } final class LastModified(val value: Long) extends Stamp { override def hashCode: Int = (value ^ (value >>> 32)).toInt - override def toString: String = "lastModified(" + value + ")" } final class Exists(val value: Boolean) extends Stamp { override def hashCode: Int = if(value) 0 else 1 - override def toString: String = if(value) "exists" else "absent" } object Stamp @@ -72,6 +72,26 @@ object Stamp case _ => false } } + + // NOTE: toString/fromString used for serialization, not just for debug prints. + + def toString(s: Stamp): String = s match { + case e: Exists => if(e.value) "exists" else "absent" + case h: Hash => "hash(" + Hash.toHex(h.value) + ")" + case lm: LastModified => "lastModified(" + lm.value + ")" + } + + private val hashPattern = """hash\((\w+)\)""".r + private val lastModifiedPattern = """lastModified\((\d+)\)""".r + + def fromString(s: String): Stamp = s match { + case "exists" => new Exists(true) + case "absent" => new Exists(false) + case hashPattern(value) => new Hash(Hash.fromHex(value)) + case lastModifiedPattern(value) => new LastModified(java.lang.Long.parseLong(value)) + case _ => throw new IllegalArgumentException("Unrecognized Stamp string representation: " + s) + } + def show(s: Stamp): String = s match { case h: Hash => "hash(" + Hash.toHex(h.value) + ")" case e: Exists => if(e.value) "exists" else "does not exist" diff --git a/compile/integration/src/main/scala/sbt/compiler/AggressiveCompile.scala b/compile/integration/src/main/scala/sbt/compiler/AggressiveCompile.scala index b9f62b912..5fc1bafba 100644 --- a/compile/integration/src/main/scala/sbt/compiler/AggressiveCompile.scala +++ b/compile/integration/src/main/scala/sbt/compiler/AggressiveCompile.scala @@ -180,8 +180,6 @@ class AggressiveCompile(cacheFile: File) private[this] def explicitBootClasspath(options: Seq[String]): Seq[File] = options.dropWhile(_ != CompilerArguments.BootClasspathOption).drop(1).take(1).headOption.toList.flatMap(IO.parseClasspath) - val formats = new sbt.inc.InternedAnalysisFormats() - import formats._ val store = AggressiveCompile.staticCache(cacheFile, AnalysisStore.sync(AnalysisStore.cached(FileBasedStore(cacheFile)))) } object AggressiveCompile diff --git a/compile/integration/src/main/scala/sbt/compiler/IncrementalCompiler.scala b/compile/integration/src/main/scala/sbt/compiler/IncrementalCompiler.scala index fddb04f09..8600f6a70 100644 --- a/compile/integration/src/main/scala/sbt/compiler/IncrementalCompiler.scala +++ b/compile/integration/src/main/scala/sbt/compiler/IncrementalCompiler.scala @@ -1,11 +1,11 @@ package sbt.compiler - import sbt.CompileSetup - import sbt.inc.{Analysis, IncOptions} + import java.io.File + import sbt.{CompileSetup, IO, Using} + import sbt.inc.{Analysis, IncOptions, TextAnalysisFormat} import xsbti.{Logger, Maybe} import xsbti.compile._ - import java.io.File object IC extends IncrementalCompiler[Analysis, AnalyzingCompiler] { @@ -40,10 +40,5 @@ object IC extends IncrementalCompiler[Analysis, AnalyzingCompiler] try { readCacheUncaught(file)._1 } catch { case _: Exception => Analysis.Empty } def readCacheUncaught(file: File): (Analysis, CompileSetup) = - { - import sbinary.DefaultProtocol.{immutableMapFormat, immutableSetFormat, StringFormat, tuple2Format} - val formats = new sbt.inc.InternedAnalysisFormats() - import formats._ - sbt.IO.gzipFileIn(file)( in => sbinary.Operations.read[(Analysis, CompileSetup)](in) ) - } + Using.fileReader(IO.utf8)(file) { reader => TextAnalysisFormat.read(reader) } } diff --git a/compile/persist/src/main/scala/sbt/inc/AnalysisFormats.scala b/compile/persist/src/main/scala/sbt/inc/AnalysisFormats.scala index dc452565a..1416d83f2 100644 --- a/compile/persist/src/main/scala/sbt/inc/AnalysisFormats.scala +++ b/compile/persist/src/main/scala/sbt/inc/AnalysisFormats.scala @@ -15,7 +15,7 @@ package inc import Logger.{m2o, position, problem} import Relations.{Source => RSource} -@deprecated("Replaced by InternedAnalysisFormats. OK to remove in 0.14.", since="0.13.1") +@deprecated("Replaced by TextAnalysisFormat. OK to remove in 0.14.", since="0.13.1") object AnalysisFormats { type RFF = Relation[File, File] diff --git a/compile/persist/src/main/scala/sbt/inc/FileBasedStore.scala b/compile/persist/src/main/scala/sbt/inc/FileBasedStore.scala index 419c172fb..49c7be0bc 100644 --- a/compile/persist/src/main/scala/sbt/inc/FileBasedStore.scala +++ b/compile/persist/src/main/scala/sbt/inc/FileBasedStore.scala @@ -4,22 +4,18 @@ package sbt package inc - import java.io.{File, IOException} - import sbinary._ - import Operations.{read, write} - import DefaultProtocol._ + import java.io.File object FileBasedStore { - def apply(file: File)(implicit analysisF: Format[Analysis], setupF: Format[CompileSetup]): AnalysisStore = new AnalysisStore { - def set(analysis: Analysis, setup: CompileSetup): Unit = - IO.gzipFileOut(file) { out => - write[(Analysis, CompileSetup)](out, (analysis, setup) ) - } + def apply(file: File): AnalysisStore = new AnalysisStore { + def set(analysis: Analysis, setup: CompileSetup) { + Using.fileWriter(IO.utf8)(file) { writer => TextAnalysisFormat.write(writer, analysis, setup) } + } def get(): Option[(Analysis, CompileSetup)] = try { Some(getUncaught()) } catch { case _: Exception => None } def getUncaught(): (Analysis, CompileSetup) = - IO.gzipFileIn(file)( in => read[(Analysis, CompileSetup)](in) ) + Using.fileReader(IO.utf8)(file) { reader => TextAnalysisFormat.read(reader) } } } \ No newline at end of file diff --git a/compile/persist/src/main/scala/sbt/inc/InternPool.scala b/compile/persist/src/main/scala/sbt/inc/InternPool.scala deleted file mode 100644 index 01161af50..000000000 --- a/compile/persist/src/main/scala/sbt/inc/InternPool.scala +++ /dev/null @@ -1,82 +0,0 @@ -package sbt -package inc - - -import sbinary._ -import DefaultProtocol._ -import xsbti.api.Lazy -import xsbti.SafeLazy - - -/** - * A simple intern pool for sharing references to a unique instance of some immutable type. - * The primary purpose of this class is to improve performance of analysis serialization/deserialization. - * - * Pooling occurs on serialization: all equal objects are serialized as a 32-bit integer index of a single representative. - * This allows us to benefit from pooling without changing all the object allocation sites. - * - * On deserialization that single representative is used where previously there might have been multiple equal instances. - * Thus, round-tripping an analysis may yield a smaller in-memory object graph. - * - * Note that the set of pooled items must be known before serialization begins. We can't build it up on-the-fly as - * we serialize: We must serialize the entire pool before serializing anything that uses it, as we'll need to read the - * entire pool in first when deserializing. The InternPool is immutable to enforce this. - */ -class InternPool[T <: AnyRef](itemsArray: Array[T]) extends Serializable { - def toIdx(item: T): Int = itemToIdx.get(item) match { - case None => sys.error("No such item in intern pool: %s".format(item.toString)) - case Some(x) => x - } - - def toItem(idx: Int): T = if (idx >= 0 && idx < items.length) items(idx) else sys.error("No such index in intern pool: %d".format(idx)) - - def allItems: Array[T] = items - - private[this] val items = itemsArray - private[this] val itemToIdx = Map.empty[T, Int] ++ itemsArray.zipWithIndex -} - - -/** - * Serialization formats that use an InternPool. - * - * fullFormat is the format to use for T when serializing the pool itself. - */ -class InternPoolFormats[T <: AnyRef](fullFormat: Format[T])(implicit mf: Manifest[T]) { - var pool: Option[InternPool[T]] = None - - /** - * Create the intern pool immediately before writing it/after reading it, so it can be used for subsequent writes/reads. - */ - def initPool(items: Array[T]): InternPool[T] = { - pool = Some(new InternPool[T](items)) - pool.get - } - - /** Format for writing a T as a pool reference. */ - implicit def itemFormat: Format[T] = wrap[T, Int](toIdx, toItem) - - /** Format for writing a T as a lazily-resolved pool reference. */ - implicit def lazyItemFormat: Format[Lazy[T]] = wrap[Lazy[T], Int](x => toIdx(x.get), idx => SafeLazy.apply[T](toItem(idx))) - - /** Format for writing an array of T as a lazily-resolved pool reference. */ - implicit def lazyItemsFormat: Format[Lazy[Array[T]]] = wrap[Lazy[Array[T]], Array[Int]]( - x => x.get map toIdx, - idxes => SafeLazy.apply[Array[T]](idxes map toItem) - ) - - /** Format for writing the pool itself. */ - implicit def poolFormat: Format[InternPool[T]] = wrap[InternPool[T], Array[T]](_.allItems, initPool)(arrayFormat(fullFormat, mf)) - - private[this] def toIdx(item: T): Int = try { - pool.get.toIdx(item) - } catch { - case e: NoSuchElementException => throw new RuntimeException("Intern pool not available for " + mf.runtimeClass.getName) - } - - private[this] def toItem(idx: Int): T = try { - pool.get.toItem(idx) - } catch { - case e: NoSuchElementException => throw new RuntimeException("Intern pool not available for " + mf.runtimeClass.getName) - } -} diff --git a/compile/persist/src/main/scala/sbt/inc/InternedAnalysisFormats.scala b/compile/persist/src/main/scala/sbt/inc/InternedAnalysisFormats.scala deleted file mode 100644 index ae65c7b7e..000000000 --- a/compile/persist/src/main/scala/sbt/inc/InternedAnalysisFormats.scala +++ /dev/null @@ -1,98 +0,0 @@ -/* sbt -- Simple Build Tool - * Copyright 2010 Mark Harrah - */ -package sbt -package inc - -import java.io.File -import sbinary._ -import DefaultProtocol.{FileFormat => _, _} -import xsbti.api._ -import xsbti.compile.{Output => APIOutput, SingleOutput, MultipleOutput} -import MultipleOutput.OutputGroup - - -/** - * sbinary serialization format for Analysis objects. - * - * Uses an intern pool for entities that are repeated many times in relations: File and Compilation instances. - * - * Has state, so cannot be an object. To use, wild-card import the implicits from an instance of this class. - */ -final class InternedAnalysisFormats -{ - // Formats that reads/write internable objects directly (i.e., not via the intern pool). - // Note: Not implicit. We only want to use the full format explicitly, in specific places (such as when - // serializing the pool itself). - val fullFileFormat: Format[File] = AnalysisFormats.fileFormat - - // Format that reads/writes files via the intern pool. - val filePool = new InternPoolFormats[File](fullFileFormat) - implicit val fileFormat = filePool.itemFormat - - // A convenient wrapper class for pool serialization. We can add more pools here in the future if needed. - final class Pools(var fileP: InternPool[File]) - - def initPools(analysis: Analysis) = new Pools( - filePool.initPool(allFiles(analysis)) - ) - - val poolsFormat: Format[Pools] = wrap[Pools, InternPool[File]](_.fileP, new Pools(_))(filePool.poolFormat) - - // Get all files mentioned in an Analysis. - // We take the union of all possible file sets, even though in theory some of these should be contained in others. - // This is safer in cases where the Analysis has discrepancies, for whatever reason. - def allFiles(analysis: Analysis): Array[File] = { - def all(rel: Relation[File, File]) = rel._1s ++ rel._2s - (analysis.stamps.allInternalSources ++ analysis.stamps.allBinaries ++ analysis.stamps.allProducts ++ - analysis.stamps.classNames.keySet ++ analysis.apis.allInternalSources ++ - all(analysis.relations.srcProd) ++ all(analysis.relations.binaryDep) ++ all(analysis.relations.internalSrcDep) ++ - analysis.relations.externalDep._1s ++ analysis.relations.classes._1s ++ - analysis.infos.allInfos.keySet).toArray - } - - // Format that tacks a serialized version of the File and Compilation intern pools in front of the serialized Analysis. - def tupleToAnalysis(pools: Pools, stamps: Stamps, apis: APIs, relations: Relations, infos: SourceInfos, compilations: Compilations) = { - Analysis.Empty.copy(stamps, apis, relations, infos, compilations) // Ignore the pools. Their work is done. - } - def analysisToTuple(a: Analysis) = (initPools(a), a.stamps, a.apis, a.relations, a.infos, a.compilations) - implicit def analysisFormat(implicit stampsF: Format[Stamps], apisF: Format[APIs], relationsF: Format[Relations], - infosF: Format[SourceInfos], compilationsF: Format[Compilations]): Format[Analysis] = - asProduct6(tupleToAnalysis)(analysisToTuple)(poolsFormat, stampsF, apisF, relationsF, infosF, compilationsF) - - // Formats in which we use the fullFileFormat. We don't want to use the pool when serializing CompileSetup. - implicit val outputGroupFormat: Format[OutputGroup] = - asProduct2((a: File,b: File) => new OutputGroup{ def sourceDirectory = a; def outputDirectory = b })({ out => (out.sourceDirectory, out.outputDirectory) })(fullFileFormat, fullFileFormat) - - val multipleOutputFormat: Format[MultipleOutput] = - wrap[MultipleOutput, Array[OutputGroup]](_.outputGroups, { groups => new MultipleOutput { def outputGroups = groups } }) - - val singleOutputFormat: Format[SingleOutput] = - wrap[SingleOutput, File](_.outputDirectory,{ out => new SingleOutput{def outputDirectory = out } })(fullFileFormat) - - implicit val outputFormat: Format[APIOutput] = asUnion(singleOutputFormat, multipleOutputFormat) - - // We use these implementations unchanged. - // TODO: Refactor out so we can import them with a wildcard? - implicit def relationFormat[A,B](implicit af: Format[Map[A, Set[B]]], bf: Format[Map[B, Set[A]]]) = AnalysisFormats.relationFormat(af, bf) - implicit def seqFormat[T](implicit optionFormat: Format[T]): Format[Seq[T]] = AnalysisFormats.seqFormat(optionFormat) - implicit val integerFormat = AnalysisFormats.integerFormat - implicit val severityFormat = AnalysisFormats.severityFormat - implicit val orderFormat = AnalysisFormats.orderFormat - implicit val existsFormat = AnalysisFormats.existsFormat - implicit val infoFormat = AnalysisFormats.infoFormat - implicit val infosFormat = AnalysisFormats.infosFormat - implicit val problemFormat = AnalysisFormats.problemFormat - implicit val positionFormat = AnalysisFormats.positionFormat - implicit val sourceFormat = AnalysisFormats.sourceFormat - implicit val apisFormat = AnalysisFormats.apisFormat - implicit val relationsSourceFormat = AnalysisFormats.relationsSourceFormat - implicit val relationsFormat = AnalysisFormats.relationsFormat - implicit val optsFormat = AnalysisFormats.optsFormat - implicit val hashStampFormat = AnalysisFormats.hashStampFormat - implicit val lastModFormat = AnalysisFormats.lastModFormat - implicit val stampFormat = AnalysisFormats.stampFormat - implicit val stampsFormat = AnalysisFormats.stampsFormat - implicit val setupFormat = AnalysisFormats.setupFormat - implicit val compilationsFormat = AnalysisFormats.compilationsFormat -} diff --git a/compile/persist/src/main/scala/sbt/inc/TextAnalysisFormat.scala b/compile/persist/src/main/scala/sbt/inc/TextAnalysisFormat.scala new file mode 100644 index 000000000..d19cb9f77 --- /dev/null +++ b/compile/persist/src/main/scala/sbt/inc/TextAnalysisFormat.scala @@ -0,0 +1,341 @@ +package sbt +package inc + +import java.io._ +import sbt.{CompileSetup, Relation} +import xsbti.api.{Compilation, Source} +import javax.xml.bind.DatatypeConverter + + +// Very simple timer for timing repeated code sections. +// TODO: Temporary. Remove once we've milked all available performance gains. +private[inc] object FormatTimer { + private val timers = scala.collection.mutable.Map[String, Long]() + private val printTimings = "true" == System.getProperty("sbt.analysis.debug.timing") + + def aggregate[T](key: String)(f: => T) = { + val start = System.nanoTime() + val ret = f + val elapsed = System.nanoTime() - start + timers.update(key, timers.getOrElseUpdate(key, 0) + elapsed) + ret + } + + def time[T](key: String)(f: => T) = { + val ret = aggregate(key)(f) + close(key) + ret + } + + def close(key: String) { + if (printTimings) { + println("[%s] %dms".format(key, timers.getOrElse(key, 0L) / 1000000)) + } + timers.remove(key) + } +} + + +class ReadException(s: String) extends Exception(s) { + def this(expected: String, found: String) = this("Expected: %s. Found: %s.".format(expected, found)) +} + +class EOFException extends ReadException("Unexpected EOF.") + + +// A text-based serialization format for Analysis objects. +// This code has been tuned for high performance, and therefore has non-idiomatic areas. +// Please refrain from making changes that significantly degrade read/write performance on large analysis files. +object TextAnalysisFormat { + // Some types are not required for external inspection/manipulation of the analysis file, + // and are complex to serialize as text. So we serialize them as base64-encoded sbinary-serialized blobs. + // TODO: This is a big performance hit. Figure out a more efficient way to serialize API objects? + import sbinary.DefaultProtocol.{immutableMapFormat, immutableSetFormat, StringFormat, tuple2Format} + import AnalysisFormats._ + implicit val compilationF = xsbt.api.CompilationFormat + + def write(out: Writer, analysis: Analysis, setup: CompileSetup) { + VersionF.write(out) + // We start with relations because that's the part of greatest interest to external readers, + // who can abort reading early once they're read them. + FormatTimer.time("write relations") { RelationsF.write(out, analysis.relations) } + FormatTimer.time("write stamps") { StampsF.write(out, analysis.stamps) } + FormatTimer.time("write apis") { APIsF.write(out, analysis.apis) } + FormatTimer.time("write sourceinfos") { SourceInfosF.write(out, analysis.infos) } + FormatTimer.time("write compilations") { CompilationsF.write(out, analysis.compilations) } + FormatTimer.time("write setup") { CompileSetupF.write(out, setup) } + out.flush() + } + + def read(in: BufferedReader): (Analysis, CompileSetup) = { + VersionF.read(in) + val relations = FormatTimer.time("read relations") { RelationsF.read(in) } + val stamps = FormatTimer.time("read stamps") { StampsF.read(in) } + val apis = FormatTimer.time("read apis") { APIsF.read(in) } + val infos = FormatTimer.time("read sourceinfos") { SourceInfosF.read(in) } + val compilations = FormatTimer.time("read compilations") { CompilationsF.read(in) } + val setup = FormatTimer.time("read setup") { CompileSetupF.read(in) } + + (Analysis.Empty.copy(stamps, apis, relations, infos, compilations), setup) + } + + private[this] object VersionF { + val currentVersion = "1" + + def write(out: Writer) { + out.write("format version: %s\n".format(currentVersion)) + } + + private val versionPattern = """format version: (\w+)""".r + def read(in: BufferedReader) { + in.readLine() match { + case versionPattern(version) => validateVersion(version) + case s: String => throw new ReadException("\"format version: \"", s) + case null => throw new EOFException + } + } + + def validateVersion(version: String) { + // TODO: Support backwards compatibility? + if (version != currentVersion) { + throw new ReadException("File uses format version %s, but we are compatible with version %s only.".format(version, currentVersion)) + } + } + } + + private[this] object RelationsF { + object Headers { + val srcProd = "products" + val binaryDep = "binary dependencies" + val internalSrcDep = "source dependencies" + val externalDep = "external dependencies" + val internalSrcDepPI = "public inherited source dependencies" + val externalDepPI = "public inherited external dependencies" + val classes = "class names" + } + + def write(out: Writer, relations: Relations) { + def writeRelation[T](header: String, rel: Relation[File, T])(implicit ord: Ordering[T]) { + writeHeader(out, header) + writeSize(out, rel.size) + // We sort for ease of debugging and for more efficient reconstruction when reading. + // Note that we don't share code with writeMap. Each is implemented more efficiently + // than the shared code would be, and the difference is measurable on large analyses. + rel.forwardMap.toSeq.sortBy(_._1).foreach { case (k, vs) => + val kStr = k.toString + vs.toSeq.sorted foreach { v => + out.write(kStr); out.write(" -> "); out.write(v.toString); out.write("\n") + } + } + } + + writeRelation(Headers.srcProd, relations.srcProd) + writeRelation(Headers.binaryDep, relations.binaryDep) + writeRelation(Headers.internalSrcDep, relations.internalSrcDep) + writeRelation(Headers.externalDep, relations.externalDep) + writeRelation(Headers.internalSrcDepPI, relations.publicInherited.internal) + writeRelation(Headers.externalDepPI, relations.publicInherited.external) + writeRelation(Headers.classes, relations.classes) + } + + def read(in: BufferedReader): Relations = { + def readRelation[T](expectedHeader: String, s2t: String => T): Relation[File, T] = { + val items = readPairs(in)(expectedHeader, new File(_), s2t).toIterator + // Reconstruct the forward map. This is more efficient than Relation.empty ++ items. + var forward: List[(File, Set[T])] = Nil + var currentItem: (File, T) = null + var currentFile: File = null + var currentVals: List[T] = Nil + def closeEntry() { + if (currentFile != null) forward = (currentFile, currentVals.toSet) :: forward + currentFile = currentItem._1 + currentVals = currentItem._2 :: Nil + } + while (items.hasNext) { + currentItem = items.next() + if (currentItem._1 == currentFile) currentVals = currentItem._2 :: currentVals else closeEntry() + } + if (currentItem != null) closeEntry() + Relation.reconstruct(forward.toMap) + } + + def readFileRelation(expectedHeader: String) = readRelation(expectedHeader, { new File(_) }) + def readStringRelation(expectedHeader: String) = readRelation(expectedHeader, identity[String]) + + val srcProd = readFileRelation(Headers.srcProd) + val binaryDep = readFileRelation(Headers.binaryDep) + val internalSrcDep = readFileRelation(Headers.internalSrcDep) + val externalDep = readStringRelation(Headers.externalDep) + val internalSrcDepPI = readFileRelation(Headers.internalSrcDepPI) + val externalDepPI = readStringRelation(Headers.externalDepPI) + val classes = readStringRelation(Headers.classes) + + Relations.make(srcProd, binaryDep, Relations.makeSource(internalSrcDep, externalDep), + Relations.makeSource(internalSrcDepPI, externalDepPI), classes) + } + } + + private[this] object StampsF { + object Headers { + val products = "product stamps" + val sources = "source stamps" + val binaries = "binary stamps" + val classNames = "class names" + } + + def write(out: Writer, stamps: Stamps) { + def doWriteMap[V](header: String, m: Map[File, V]) = writeMap(out)(header, m, { v: V => v.toString }) + + doWriteMap(Headers.products, stamps.products) + doWriteMap(Headers.sources, stamps.sources) + doWriteMap(Headers.binaries, stamps.binaries) + doWriteMap(Headers.classNames, stamps.classNames) + } + + def read(in: BufferedReader): Stamps = { + def doReadMap[V](expectedHeader: String, s2v: String => V) = readMap(in)(expectedHeader, new File(_), s2v) + val products = doReadMap(Headers.products, Stamp.fromString) + val sources = doReadMap(Headers.sources, Stamp.fromString) + val binaries = doReadMap(Headers.binaries, Stamp.fromString) + val classNames = doReadMap(Headers.classNames, identity[String]) + + Stamps(products, sources, binaries, classNames) + } + } + + private[this] object APIsF { + object Headers { + val internal = "internal apis" + val external = "external apis" + } + + val stringToSource = ObjectStringifier.stringToObj[Source] _ + val sourceToString = ObjectStringifier.objToString[Source] _ + + def write(out: Writer, apis: APIs) { + writeMap(out)(Headers.internal, apis.internal, sourceToString, inlineVals=false) + writeMap(out)(Headers.external, apis.external, sourceToString, inlineVals=false) + FormatTimer.close("bytes -> base64") + FormatTimer.close("byte copy") + FormatTimer.close("sbinary write") + } + + def read(in: BufferedReader): APIs = { + val internal = readMap(in)(Headers.internal, new File(_), stringToSource) + val external = readMap(in)(Headers.external, identity[String], stringToSource) + FormatTimer.close("base64 -> bytes") + FormatTimer.close("sbinary read") + APIs(internal, external) + } + } + + private[this] object SourceInfosF { + object Headers { + val infos = "source infos" + } + + val stringToSourceInfo = ObjectStringifier.stringToObj[SourceInfo] _ + val sourceInfoToString = ObjectStringifier.objToString[SourceInfo] _ + + def write(out: Writer, infos: SourceInfos) { writeMap(out)(Headers.infos, infos.allInfos, sourceInfoToString, inlineVals=false) } + def read(in: BufferedReader): SourceInfos = SourceInfos.make(readMap(in)(Headers.infos, new File(_), stringToSourceInfo)) + } + + private[this] object CompilationsF { + object Headers { + val compilations = "compilations" + } + + val stringToCompilation = ObjectStringifier.stringToObj[Compilation] _ + val compilationToString = ObjectStringifier.objToString[Compilation] _ + + def write(out: Writer, compilations: Compilations) { + def toMapEntry(x: (Compilation, Int)): (String, Compilation) = "%03d".format(x._2) -> x._1 + writeMap(out)(Headers.compilations, compilations.allCompilations.zipWithIndex.map(toMapEntry).toMap, compilationToString, inlineVals=false) + } + def read(in: BufferedReader): Compilations = + Compilations.make(readMap(in)(Headers.compilations, identity[String], stringToCompilation).values.toSeq) + } + + private[this] object CompileSetupF { + object Headers { + val setup = "compile setup" + } + + val stringToSetup = ObjectStringifier.stringToObj[CompileSetup] _ + val setupToString = ObjectStringifier.objToString[CompileSetup] _ + + def write(out: Writer, setup: CompileSetup) { writeMap(out)(Headers.setup, Map("1" -> setup), setupToString, inlineVals=false)} + def read(in: BufferedReader): CompileSetup = readMap(in)(Headers.setup, identity[String], stringToSetup).head._2 + } + + private[this] object ObjectStringifier { + def objToString[T](o: T)(implicit fmt: sbinary.Format[T]) = { + val baos = new ByteArrayOutputStream() + val out = new sbinary.JavaOutput(baos) + FormatTimer.aggregate("sbinary write") { try { fmt.writes(out, o) } finally { baos.close() } } + val bytes = FormatTimer.aggregate("byte copy") { baos.toByteArray } + FormatTimer.aggregate("bytes -> base64") { DatatypeConverter.printBase64Binary(bytes) } + } + + def stringToObj[T](s: String)(implicit fmt: sbinary.Format[T]) = { + val bytes = FormatTimer.aggregate("base64 -> bytes") { DatatypeConverter.parseBase64Binary(s) } + val in = new sbinary.JavaInput(new ByteArrayInputStream(bytes)) + FormatTimer.aggregate("sbinary read") { fmt.reads(in) } + } + } + + // Various helper functions. + + private[this] def writeHeader(out: Writer, header: String) { + out.write(header + ":\n") + } + + private[this] def expectHeader(in: BufferedReader, expectedHeader: String) { + val header = in.readLine() + if (header != expectedHeader + ":") throw new ReadException(expectedHeader, if (header == null) "EOF" else header) + } + + private[this] def writeSize(out: Writer, n: Int) { + out.write("%d items\n".format(n)) + } + + private val itemsPattern = """(\d+) items""".r + private[this] def readSize(in: BufferedReader): Int = { + in.readLine() match { + case itemsPattern(nStr) => Integer.parseInt(nStr) + case s: String => throw new ReadException("\" items\"", s) + case null => throw new EOFException + } + } + + private[this] def writeMap[K, V](out: Writer)(header: String, m: Map[K, V], v2s: V => String, inlineVals: Boolean=true)(implicit ord: Ordering[K]) { + writeHeader(out, header) + writeSize(out, m.size) + m.keys.toSeq.sorted foreach { k => + out.write(k.toString) + out.write(" -> ") + if (!inlineVals) out.write("\n") // Put large vals on their own line, to save string munging on read. + out.write(v2s(m(k))) + out.write("\n") + } + } + + private[this] def readPairs[K, V](in: BufferedReader)(expectedHeader: String, s2k: String => K, s2v: String => V): Traversable[(K, V)] = { + def toPair(s: String): (K, V) = { + if (s == null) throw new EOFException + val p = s.indexOf(" -> ") + val k = s2k(s.substring(0, p)) + // Pair is either "a -> b" or "a -> \nb". This saves us a lot of substring munging when b is a large blob. + val v = s2v(if (p == s.length - 4) in.readLine() else s.substring(p + 4)) + (k, v) + } + expectHeader(in, expectedHeader) + val n = readSize(in) + for (i <- 0 until n) yield toPair(in.readLine()) + } + + private[this] def readMap[K, V](in: BufferedReader)(expectedHeader: String, s2k: String => K, s2v: String => V): Map[K, V] = { + readPairs(in)(expectedHeader, s2k, s2v).toMap + } +}