Skip to content

Commit

Permalink
Merge pull request #233 from hammerlab/refpos-reconcile
Browse files Browse the repository at this point in the history
Build up reference information during cigar processing
  • Loading branch information
massie committed May 6, 2014
2 parents 78bc6c1 + c39dd25 commit aca27a4
Show file tree
Hide file tree
Showing 8 changed files with 237 additions and 81 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,9 @@ package org.bdgenomics.adam.models

import com.esotericsoftware.kryo.{ Kryo, Serializer }
import com.esotericsoftware.kryo.io.{ Input, Output }
import org.bdgenomics.adam.avro.{ ADAMRecord, ADAMNucleotideContigFragment, Base }
import org.bdgenomics.adam.avro.{ ADAMRecord, ADAMNucleotideContigFragment }
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rich.RichADAMRecord
import org.bdgenomics.adam.rich.RichADAMRecord._
import scala.math.{ min, max }

object ReferenceRegion {
Expand Down
Original file line number Diff line number Diff line change
@@ -1,12 +1,9 @@
package org.bdgenomics.adam.models

import org.bdgenomics.adam.avro.{ ADAMContig, ADAMVariant, ADAMRecord }
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rich.{ RichADAMVariant, DecadentRead, ReferenceLocation }
import org.bdgenomics.adam.rich.RichADAMVariant
import org.bdgenomics.adam.rich.DecadentRead._
import org.apache.spark.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
import scala.collection.immutable._
import scala.collection.mutable
import java.io.File
Expand All @@ -18,15 +15,15 @@ class SnpTable(private val table: Map[String, Set[Long]]) extends Serializable w
* Is there a known SNP at the reference location of this Residue?
*/
def isMasked(residue: Residue): Boolean =
contains(residue.referenceLocation)
contains(residue.referencePosition)

/**
* Is there a known SNP at the given reference location?
*/
def contains(location: ReferenceLocation): Boolean = {
val bucket = table.get(location.contig)
if (bucket.isEmpty) unknownContigWarning(location.contig)
bucket.map(_.contains(location.offset)).getOrElse(false)
def contains(location: ReferencePosition): Boolean = {
val bucket = table.get(location.referenceName)
if (bucket.isEmpty) unknownContigWarning(location.referenceName)
bucket.map(_.contains(location.pos)).getOrElse(false)
}

private val unknownContigs = new mutable.HashSet[String]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,10 @@ import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rich.RichADAMRecord._
import org.bdgenomics.adam.util.MdTag
import org.bdgenomics.adam.util.QualityScore
import org.bdgenomics.adam.util.Util
import org.apache.spark.rdd.RDD
import org.apache.spark.Logging
import scala.Some
import org.bdgenomics.adam.models.ReferencePosition

object DecadentRead {
type Residue = DecadentRead#Residue
Expand Down Expand Up @@ -104,13 +105,15 @@ class DecadentRead(val record: RichADAMRecord) extends Logging {
def isInsertion: Boolean =
assumingAligned(record.isMismatchAtReadOffset(offset).isEmpty)

def referenceLocationOption: Option[ReferenceLocation] = assumingAligned {
record.readOffsetToReferencePosition(offset).
map(refOffset => new ReferenceLocation(record.getContig.getContigName.toString, refOffset))
}
def referencePositionOption: Option[ReferencePosition] =
assumingAligned(
record.readOffsetToReferencePosition(offset))

def referenceSequenceContext: Option[ReferenceSequenceContext] =
assumingAligned(record.readOffsetToReferenceSequenceContext(offset))

def referenceLocation: ReferenceLocation =
referenceLocationOption.getOrElse(
def referencePosition: ReferencePosition =
referencePositionOption.getOrElse(
throw new IllegalArgumentException("Residue has no reference location (may be an insertion)"))
}

Expand Down Expand Up @@ -162,17 +165,3 @@ class DecadentRead(val record: RichADAMRecord) extends Logging {
func
}
}

// TODO: merge with models.ReferencePosition
class ReferenceLocation(val contig: String, val offset: Long) {
override def toString = "%s@%s".format(contig, offset)

override def equals(other: Any): Boolean = other match {
case that: ReferenceLocation =>
this.contig == that.contig && this.offset == that.offset

case _ => false
}

override def hashCode = Util.hashCombine(0x922927F8, contig.hashCode, offset.hashCode)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Copyright (c) 2014. Mount Sinai School of Medicine
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.bdgenomics.adam.rich

import org.bdgenomics.adam.models.ReferencePosition
import net.sf.samtools.CigarElement

/**
* Represents information on the reference relative to a particular residue
*/

private[rich] case class ReferenceSequenceContext(pos: Option[ReferencePosition], referenceBase: Option[Char], cigarElement: CigarElement, cigarElementOffset: Int)
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.bdgenomics.adam.util._
import scala.Some
import scala.concurrent.JavaConversions._
import scala.collection.immutable.NumericRange
import org.bdgenomics.adam.models.Attribute
import org.bdgenomics.adam.models.{ ReferenceRegion, ReferencePosition, Attribute }

object RichADAMRecord {
val CIGAR_CODEC: TextCigarCodec = TextCigarCodec.getSingleton
Expand All @@ -40,6 +40,8 @@ class IlluminaOptics(val tile: Long, val x: Long, val y: Long) {}

class RichADAMRecord(val record: ADAMRecord) {

lazy val readRegion = ReferenceRegion(this)

// Returns the quality scores as a list of bytes
lazy val qualityScores: Array[Int] = record.getQual.toString.toCharArray.map(q => (q - 33))

Expand Down Expand Up @@ -119,18 +121,13 @@ class RichADAMRecord(val record: ADAMRecord) {
}

// Does this read overlap with the given reference position?
// FIXME: doesn't check contig! should use ReferenceLocation, not Long
def overlapsReferencePosition(pos: Long): Option[Boolean] = {
if (record.getReadMapped) {
Some(record.getStart <= pos && pos < end.get)
} else {
None
}
def overlapsReferencePosition(pos: ReferencePosition): Option[Boolean] = {
readRegion.map(_.contains(pos))
}

// Does this read mismatch the reference at the given reference position?
def isMismatchAtReferencePosition(pos: Long): Option[Boolean] = {
if (!overlapsReferencePosition(pos).get) {
def isMismatchAtReferencePosition(pos: ReferencePosition): Option[Boolean] = {
if (mdTag.isEmpty || !overlapsReferencePosition(pos).get) {
None
} else {
mdTag.map(!_.isMatch(pos))
Expand All @@ -143,43 +140,85 @@ class RichADAMRecord(val record: ADAMRecord) {
if (referencePositions.isEmpty) {
None
} else {
readOffsetToReferencePosition(offset).flatMap(isMismatchAtReferencePosition)
readOffsetToReferencePosition(offset).flatMap(pos => isMismatchAtReferencePosition(pos))
}
}

lazy val referencePositions: Seq[Option[Long]] = {
def getReferenceContext(readOffset: Int, referencePosition: Long, cigarElem: CigarElement, elemOffset: Int): ReferenceSequenceContext = {
val position = if (ReferencePosition.mappedPositionCheck(record)) {
Some(new ReferencePosition(record.getContig.getContigName.toString, referencePosition))
} else {
None
}

def getReferenceBase(cigarElement: CigarElement, refPos: Long, readPos: Int): Option[Char] = {
mdTag.flatMap(tag => {
cigarElement.getOperator match {
case CigarOperator.M => {
if (!tag.isMatch(refPos)) {
tag.mismatchedBase(refPos)
} else {
Some(record.getSequence()(readPos))
}
}
case CigarOperator.D => {
// if a delete, get from the delete pool
tag.deletedBase(refPos)
}
case _ => None
}
})
}

val referenceBase = getReferenceBase(cigarElem, referencePosition, readOffset)
ReferenceSequenceContext(position, referenceBase, cigarElem, elemOffset)
}

lazy val referencePositions: Seq[Option[ReferencePosition]] = referenceContexts.map(ref => ref.flatMap(_.pos))

lazy val referenceContexts: Seq[Option[ReferenceSequenceContext]] = {
if (record.getReadMapped) {
val resultTuple = samtoolsCigar.getCigarElements.foldLeft((unclippedStart.get, List[Option[Long]]()))((runningPos, elem) => {
val resultTuple = samtoolsCigar.getCigarElements.foldLeft((unclippedStart.get, List[Option[ReferenceSequenceContext]]()))((runningPos, elem) => {
// runningPos is a tuple, the first element holds the starting position of the next CigarOperator
// and the second element is the list of positions up to this point
val op = elem.getOperator
val currentRefPos = runningPos._1
val resultAccum = runningPos._2
val advanceReference = op.consumesReferenceBases || op == CigarOperator.S
val newRefPos = currentRefPos + (if (advanceReference) elem.getLength else 0)
val resultParts: Seq[Option[Long]] =
val resultParts: Seq[Option[ReferenceSequenceContext]] =
if (op.consumesReadBases) {
val range = NumericRange(currentRefPos, currentRefPos + elem.getLength, 1L)
range.map(pos => if (advanceReference) Some(pos) else None)
range.zipWithIndex.map(kv =>
if (advanceReference)
Some(getReferenceContext(resultAccum.size + kv._2, kv._1, elem, kv._2))
else None)
} else {
Seq.empty
}
(newRefPos, resultAccum ++ resultParts)
})

val endRefPos = resultTuple._1
val results = resultTuple._2
results.toIndexedSeq
} else {
qualityScores.map(t => None)
}
}

def readOffsetToReferencePosition(offset: Int): Option[Long] = {
def readOffsetToReferencePosition(offset: Int): Option[ReferencePosition] = {
if (record.getReadMapped) {
referencePositions(offset)
} else {
None
}
}

def readOffsetToReferenceSequenceContext(offset: Int): Option[ReferenceSequenceContext] = {
if (record.getReadMapped) {
referenceContexts(offset)
} else {
None
}
}

}
6 changes: 6 additions & 0 deletions adam-core/src/main/scala/org/bdgenomics/adam/util/MdTag.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ import scala.collection.immutable.NumericRange
import scala.util.matching.Regex
import net.sf.samtools.{ Cigar, CigarOperator, CigarElement }
import org.bdgenomics.adam.avro.ADAMRecord
import org.bdgenomics.adam.models.ReferencePosition

//import org.bdgenomics.adam.util.ImplicitJavaConversions._
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rich.RichADAMRecord
Expand Down Expand Up @@ -248,6 +250,10 @@ class MdTag(
matches.exists(_.contains(pos))
}

def isMatch(pos: ReferencePosition): Boolean = {
matches.exists(_.contains(pos.pos))
}

/**
* Returns the mismatched base at a position.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* Copyright (c) 2014. Mount Sinai School of Medicine
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.bdgenomics.adam.rich

import org.scalatest.FunSuite
import org.bdgenomics.adam.avro.{ ADAMContig, ADAMRecord }
import org.bdgenomics.adam.models.ReferencePosition

class DecadentReadSuite extends FunSuite {

test("reference position of decadent read") {
val contig = ADAMContig.newBuilder
.setContigName("chr1")
.build

val hardClippedRead = RichADAMRecord(ADAMRecord
.newBuilder()
.setReadMapped(true)
.setStart(1000)
.setContig(contig)
.setMismatchingPositions("10")
.setSequence("AACCTTGGC")
.setQual("FFFFFFFFF")
.setCigar("9M1H").build())

val record = DecadentRead(hardClippedRead)
assert(record.residues.size === 9)

val residueSeq = record.residues
assert(residueSeq.head.referencePosition === ReferencePosition("chr1", 1000))
}

test("reference position of decadent read with insertions") {
val contig = ADAMContig.newBuilder
.setContigName("chr1")
.build

val hardClippedRead = RichADAMRecord(ADAMRecord
.newBuilder()
.setReadMapped(true)
.setStart(1000)
.setContig(contig)
.setMismatchingPositions("1TT10")
.setSequence("ATTGGGGGGGGGG")
.setQual("FFFFFFFFFFFFF")
.setCigar("1M2I10M").build())

val record = DecadentRead(hardClippedRead)
assert(record.residues.size === 13)

val residueSeq = record.residues
assert(residueSeq.head.referencePosition === ReferencePosition("chr1", 1000))
}

}
Loading

0 comments on commit aca27a4

Please sign in to comment.