mirror of https://github.com/sbt/sbt.git
Merge pull request #988 from benjyw/text_analysis_file2
Replace binary Analysis format with a text-based one.
This commit is contained in:
commit
c16d3e53cc
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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) }
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -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) }
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
}
|
||||
|
|
@ -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: <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("\"<n> 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
|
||||
}
|
||||
}
|
||||
Loading…
Reference in New Issue