Skip to content

Commit

Permalink
[ADAM-601] Consolidate ReferencePosition so that it extends Reference…
Browse files Browse the repository at this point in the history
…Region. Consolidates ordered classes down into unordered classes.
  • Loading branch information
fnothaft committed Mar 31, 2015
1 parent ad9e510 commit 9a318ba
Show file tree
Hide file tree
Showing 16 changed files with 187 additions and 556 deletions.
2 changes: 2 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -10,3 +10,5 @@ build
*.log
.*.swp
.DS_Store

*#*
Original file line number Diff line number Diff line change
Expand Up @@ -17,194 +17,44 @@
*/
package org.bdgenomics.adam.models

import org.bdgenomics.formats.avro._
import org.bdgenomics.adam.rdd.ADAMContext._
import com.esotericsoftware.kryo.{ Kryo, Serializer }
import com.esotericsoftware.kryo.io.{ Input, Output }
import scala.annotation.tailrec

object ReferencePositionWithOrientation {

def apply(record: AlignmentRecord): Option[ReferencePositionWithOrientation] = {
if (record.getReadMapped) {
Some(new ReferencePositionWithOrientation(ReferencePosition(record).get, record.getReadNegativeStrand))
} else {
None
}
}

def fivePrime(record: AlignmentRecord): Option[ReferencePositionWithOrientation] = {
if (record.getReadMapped) {
Some(new ReferencePositionWithOrientation(ReferencePosition.fivePrime(record).get, record.getReadNegativeStrand))
} else {
None
}
}
/**
* Given an index into a sequence (e.g., a transcript), and a sequence
* of regions that map this sequence to a reference genome, returns a
* reference position.
*
* @param idx Index into a sequence.
* @param mapping Sequence of reference regions that maps the sequence to a
* reference genome.
* @return Returns a lifted over reference position.
*
* @throws AssertionError Throws an assertion error if the strandedness does
* not match between the reference position and the mapping regions.
* @throws IllegalArgumentException Throws an exception if the index given
* is out of the range of the mappings given.
*/
def liftOverToReference(idx: Long,
mapping: Seq[ReferenceRegionWithOrientation]): ReferencePositionWithOrientation = {
val negativeStrand = mapping.head.negativeStrand

@tailrec def liftOverHelper(regions: Iterator[ReferenceRegionWithOrientation],
idx: Long): ReferencePositionWithOrientation = {
if (!regions.hasNext) {
throw new IllegalArgumentException("Liftover is out of range")
} else {
val reg = regions.next()
assert(reg.negativeStrand == negativeStrand,
"Strand is inconsistent across set of regions.")

// is our position in this region?
if (reg.width > idx) {
// get contig
val chr = reg.referenceName

// get our pos
val pos = if (negativeStrand) {
reg.end - idx - 1 // need -1 because of open end coordinate
} else {
reg.start + idx
}

// return
ReferencePositionWithOrientation(ReferencePosition(chr, pos),
negativeStrand)
} else {
// recurse
liftOverHelper(regions, idx - reg.width)
}
}
}
import org.bdgenomics.formats.avro._

// call out to helper
if (negativeStrand) {
liftOverHelper(mapping.reverseIterator, idx)
} else {
liftOverHelper(mapping.toIterator, idx)
}
}
object PositionOrdering extends ReferenceOrdering[ReferencePosition] {
}

case class ReferencePositionWithOrientation(refPos: ReferencePosition,
negativeStrand: Boolean)
extends Ordered[ReferencePositionWithOrientation] {

/**
* Given a sequence of regions that map from a reference genome to a sequence,
* returns an index into a sequence.
*
* @param mapping Sequence of reference regions that maps the sequence to a
* reference genome.
* @return Returns an index into a sequence.
*
* @throws AssertionError Throws an assertion error if the strandedness does
* not match between the reference position and the mapping regions.
* @throws IllegalArgumentException Throws an exception if the index given
* is out of the range of the mappings given.
*/
def liftOverFromReference(mapping: Seq[ReferenceRegionWithOrientation]): Long = {
val pos = refPos.pos

@tailrec def liftOverHelper(regions: Iterator[ReferenceRegionWithOrientation],
idx: Long = 0L): Long = {
if (!regions.hasNext) {
throw new IllegalArgumentException("Position was not contained in mapping.")
} else {
// get region
val reg = regions.next()
assert(reg.negativeStrand == negativeStrand,
"Strand is inconsistent across set of regions.")

// are we in this region?
if (reg.contains(this)) {
// how far are we from the end of the region
val into = if (negativeStrand) {
reg.end - pos
} else {
pos - reg.start
}

// add to index
idx + into
} else {
liftOverHelper(regions, idx + reg.width)
}
}
}

// call out to helper
liftOverHelper(mapping.toIterator)
}

override def compare(that: ReferencePositionWithOrientation): Int = {
val posCompare = refPos.compare(that.refPos)
if (posCompare != 0) {
posCompare
} else {
negativeStrand.compare(that.negativeStrand)
}
}
object OptionalPositionOrdering extends OptionalReferenceOrdering[ReferencePosition] {
val baseOrdering = PositionOrdering
}

object ReferencePosition {
object ReferencePosition extends Serializable {

implicit def orderingForPositions = PositionOrdering
implicit def orderingForOptionalPositions = OptionalPositionOrdering

/**
* The UNMAPPED value is a convenience value, which can be used to indicate a position
* which is not located anywhere along the reference sequences (see, e.g. its use in
* GenomicPositionPartitioner).
*/
val UNMAPPED = new ReferencePosition("", -1)

/**
* Checks to see if a read is mapped with a valid position.
*
* @param record Read to check for mapping.
* @return True if read is mapped and has a valid position, else false.
*/
def mappedPositionCheck(record: AlignmentRecord): Boolean = {
val contig = Option(record.getContig)
val start = Option(record.getStart)
record.getReadMapped && (contig.isDefined && Option(contig.get.getContigName).isDefined) && start.isDefined
}
val UNMAPPED = new ReferencePosition("", 0)

/**
* Generates a reference position from a read. This function generates the
* position from the start mapping position of the read.
*
* @param record Read from which to generate a reference position.
* @return Reference position wrapped inside of an option. If the read is
* mapped, the option will be stuffed, else it will be empty (None).
* @return Reference position of the start position.
*
* @see fivePrime
*/
def apply(record: AlignmentRecord): Option[ReferencePosition] = {
if (mappedPositionCheck(record)) {
Some(new ReferencePosition(record.getContig.getContigName.toString, record.getStart))
} else {
None
}
def apply(record: AlignmentRecord): ReferencePosition = {
new ReferencePosition(record.getContig.getContigName.toString, record.getStart)
}

/**
* Generates a reference position from a called variant.
*
* @note An invariant of variants is that they have a defined position,
* therefore we do not wrap them in an option.
*
* @param variant Called variant from which to generate a
* reference position.
* @return The reference position of this variant.
Expand All @@ -216,9 +66,6 @@ object ReferencePosition {
/**
* Generates a reference position from a genotype.
*
* @note An invariant of genotypes is that they have a defined position,
* therefore we do not wrap them in an option.
*
* @param genotype Genotype from which to generate a reference position.
* @return The reference position of this genotype.
*/
Expand All @@ -227,63 +74,33 @@ object ReferencePosition {
new ReferencePosition(variant.getContig.getContigName, variant.getStart)
}

/**
* Generates a reference position from the five prime end of a read. This
* will be different from the start mapping position of a read if this
* read is a reverse strand read.
*
* @param record Read from which to generate a reference position.
* @return Reference position wrapped inside of an option. If the read is
* mapped, the option will be stuffed, else it will be empty (None).
*
* @see apply(record: Read)
*/
def fivePrime(record: AlignmentRecord): Option[ReferencePosition] = {
if (mappedPositionCheck(record)) {
Some(new ReferencePosition(record.getContig.getContigName, record.fivePrimePosition.get))
} else {
None
}
def apply(referenceName: String, pos: Long): ReferencePosition = {
new ReferencePosition(referenceName, pos)
}
}

case class ReferencePosition(referenceName: String, pos: Long) extends Ordered[ReferencePosition] {

override def compare(that: ReferencePosition): Int = {
// Note: important to compare by reference first for coordinate ordering
val refCompare = referenceName.compare(that.referenceName)
if (refCompare != 0) {
refCompare
} else {
pos.compare(that.pos)
}
def apply(referenceName: String, pos: Long, orientation: Strand): ReferencePosition = {
new ReferencePosition(referenceName, pos, orientation)
}
}

class ReferencePositionWithOrientationSerializer extends Serializer[ReferencePositionWithOrientation] {
val referencePositionSerializer = new ReferencePositionSerializer()

def write(kryo: Kryo, output: Output, obj: ReferencePositionWithOrientation) = {
output.writeBoolean(obj.negativeStrand)
referencePositionSerializer.write(kryo, output, obj.refPos)
}

def read(kryo: Kryo, input: Input, klazz: Class[ReferencePositionWithOrientation]): ReferencePositionWithOrientation = {
val negativeStrand = input.readBoolean()
val referencePosition = referencePositionSerializer.read(kryo, input, classOf[ReferencePosition])
new ReferencePositionWithOrientation(referencePosition, negativeStrand)
}
class ReferencePosition(override val referenceName: String,
val pos: Long,
override val orientation: Strand = Strand.Independent) extends ReferenceRegion(referenceName, pos, pos + 1, orientation) {
}

class ReferencePositionSerializer extends Serializer[ReferencePosition] {
private val enumValues = Strand.values()

def write(kryo: Kryo, output: Output, obj: ReferencePosition) = {
output.writeString(obj.referenceName)
output.writeLong(obj.pos)
output.writeInt(obj.orientation.ordinal)
}

def read(kryo: Kryo, input: Input, klazz: Class[ReferencePosition]): ReferencePosition = {
val refName = input.readString()
val pos = input.readLong()
new ReferencePosition(refName, pos)
val orientation = input.readInt()
new ReferencePosition(refName, pos, enumValues(orientation))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,18 +19,25 @@ package org.bdgenomics.adam.models

import com.esotericsoftware.kryo.{ Kryo, Serializer }
import com.esotericsoftware.kryo.io.{ Input, Output }
import org.apache.spark.Logging
import Ordering.Option
import org.apache.spark.Logging
import org.bdgenomics.adam.instrumentation.Timers.CreateReferencePositionPair
import org.bdgenomics.adam.models.ReferenceRegion._
import org.bdgenomics.adam.rich.RichAlignmentRecord
import org.bdgenomics.formats.avro.AlignmentRecord

object ReferencePositionPair extends Logging {
private def posForRead(read: AlignmentRecord): Option[ReferencePosition] = {
RichAlignmentRecord(read).fivePrimeReferencePosition
}

def apply(singleReadBucket: SingleReadBucket): ReferencePositionPair = CreateReferencePositionPair.time {
singleReadBucket.primaryMapped.toSeq.lift(0) match {
case None =>
// No mapped reads
new ReferencePositionPair(None, None)
case Some(read1) =>
val read1pos = ReferencePositionWithOrientation.fivePrime(read1)
val read1pos = posForRead(read1)
if (read1.getReadPaired && read1.getMateMapped) {
singleReadBucket.primaryMapped.toSeq.lift(1) match {
case None =>
Expand All @@ -39,8 +46,8 @@ object ReferencePositionPair extends Logging {
new ReferencePositionPair(read1pos, None)
case Some(read2) =>
// Both reads are mapped
val read2pos = ReferencePositionWithOrientation.fivePrime(read2)
if (read1pos < read2pos) {
val read2pos = posForRead(read2)
if (read1pos.get.disorient.compareTo(read2pos.get.disorient) < 0) {
new ReferencePositionPair(read1pos, read2pos)
} else {
new ReferencePositionPair(read2pos, read1pos)
Expand All @@ -52,9 +59,9 @@ object ReferencePositionPair extends Logging {
// Mate is not mapped...
new ReferencePositionPair(read1pos, None)
case Some(read2) =>
val read2pos = ReferencePositionWithOrientation.fivePrime(read2)
val read2pos = posForRead(read2)
log.warn("%s claimed to not have mate but mate found".format(read1.getReadName))
if (read1pos < read2pos) {
if (read1pos.get.disorient.compareTo(read2pos.get.disorient) < 0) {
new ReferencePositionPair(read1pos, read2pos)
} else {
new ReferencePositionPair(read2pos, read1pos)
Expand All @@ -65,13 +72,13 @@ object ReferencePositionPair extends Logging {
}
}

case class ReferencePositionPair(read1refPos: Option[ReferencePositionWithOrientation],
read2refPos: Option[ReferencePositionWithOrientation])
case class ReferencePositionPair(read1refPos: Option[ReferencePosition],
read2refPos: Option[ReferencePosition])

class ReferencePositionPairSerializer extends Serializer[ReferencePositionPair] {
val rps = new ReferencePositionWithOrientationSerializer()
val rps = new ReferencePositionSerializer()

def writeOptionalReferencePos(kryo: Kryo, output: Output, optRefPos: Option[ReferencePositionWithOrientation]) = {
def writeOptionalReferencePos(kryo: Kryo, output: Output, optRefPos: Option[ReferencePosition]) = {
optRefPos match {
case None =>
output.writeBoolean(false)
Expand All @@ -81,10 +88,10 @@ class ReferencePositionPairSerializer extends Serializer[ReferencePositionPair]
}
}

def readOptionalReferencePos(kryo: Kryo, input: Input): Option[ReferencePositionWithOrientation] = {
def readOptionalReferencePos(kryo: Kryo, input: Input): Option[ReferencePosition] = {
val exists = input.readBoolean()
if (exists) {
Some(rps.read(kryo, input, classOf[ReferencePositionWithOrientation]))
Some(rps.read(kryo, input, classOf[ReferencePosition]))
} else {
None
}
Expand Down
Loading

0 comments on commit 9a318ba

Please sign in to comment.