Skip to content

Commit

Permalink
Merge a371f9f into 78bb9e5
Browse files Browse the repository at this point in the history
  • Loading branch information
heuermh committed Dec 11, 2018
2 parents 78bb9e5 + a371f9f commit e17890d
Show file tree
Hide file tree
Showing 71 changed files with 681 additions and 642 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,8 @@ class FlagStat(protected val args: FlagStatArgs) extends BDGSparkCommand[FlagSta
AlignmentRecordField.readMapped,
AlignmentRecordField.mateMapped,
AlignmentRecordField.readPaired,
AlignmentRecordField.contigName,
AlignmentRecordField.mateContigName,
AlignmentRecordField.referenceName,
AlignmentRecordField.mateReferenceName,
AlignmentRecordField.primaryAlignment,
AlignmentRecordField.duplicateRead,
AlignmentRecordField.readMapped,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,6 @@ class TransformFeaturesSuite extends ADAMFunSuite {
val converted = sc.loadFeatures(outputPath).rdd.collect

assert(converted.size === 10)
assert(converted.find(_.getContigName != "chr1").isEmpty)
assert(converted.find(_.getReferenceName != "chr1").isEmpty)
}
}
4 changes: 2 additions & 2 deletions adam-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -102,14 +102,14 @@
<arguments>
<argument>org.bdgenomics.adam.sql</argument>
<argument>org.bdgenomics.formats.avro.AlignmentRecord</argument>
<argument>org.bdgenomics.formats.avro.Contig</argument>
<argument>org.bdgenomics.formats.avro.Dbxref</argument>
<argument>org.bdgenomics.formats.avro.Feature</argument>
<argument>org.bdgenomics.formats.avro.Fragment</argument>
<argument>org.bdgenomics.formats.avro.Genotype</argument>
<argument>org.bdgenomics.formats.avro.NucleotideContigFragment</argument>
<argument>org.bdgenomics.formats.avro.OntologyTerm</argument>
<argument>org.bdgenomics.formats.avro.ProcessingStep</argument>
<argument>org.bdgenomics.formats.avro.Reference</argument>
<argument>org.bdgenomics.formats.avro.TranscriptEffect</argument>
<argument>org.bdgenomics.formats.avro.Variant</argument>
<argument>org.bdgenomics.formats.avro.VariantAnnotation</argument>
Expand All @@ -130,7 +130,6 @@
<arguments>
<argument>org.bdgenomics.adam.projections</argument>
<argument>org.bdgenomics.formats.avro.AlignmentRecord</argument>
<argument>org.bdgenomics.formats.avro.Contig</argument>
<argument>org.bdgenomics.formats.avro.Dbxref</argument>
<argument>org.bdgenomics.formats.avro.Feature</argument>
<argument>org.bdgenomics.formats.avro.Fragment</argument>
Expand All @@ -139,6 +138,7 @@
<argument>org.bdgenomics.formats.avro.OntologyTerm</argument>
<argument>org.bdgenomics.formats.avro.Read</argument>
<argument>org.bdgenomics.formats.avro.RecordGroup</argument>
<argument>org.bdgenomics.formats.avro.Reference</argument>
<argument>org.bdgenomics.formats.avro.Sample</argument>
<argument>org.bdgenomics.formats.avro.Sequence</argument>
<argument>org.bdgenomics.formats.avro.Slice</argument>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ private[adam] class ConsensusGeneratorFromKnowns(rdd: RDD[Variant],
// get region
val start = reads.map(_.record.getStart).min
val end = reads.map(_.getEnd).max
val refId = reads.head.record.getContigName
val refId = reads.head.record.getReferenceName

val region = ReferenceRegion(refId, start, end + 1)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ private[adam] class ConsensusGeneratorFromReads extends ConsensusGenerator {
Consensus.generateAlternateConsensus(
r.getSequence,
ReferencePosition(
r.getContigName,
r.getReferenceName,
r.getStart
),
r.samtoolsCigar
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -257,7 +257,7 @@ class AlignmentRecordConverter extends Serializable {
})

// set the reference name, and alignment position, for mate
Option(adamRecord.getMateContigName)
Option(adamRecord.getMateReferenceName)
.foreach(builder.setMateReferenceName)
Option(adamRecord.getMateAlignmentStart)
.foreach(s => builder.setMateAlignmentStart(s.toInt + 1))
Expand Down Expand Up @@ -293,8 +293,8 @@ class AlignmentRecordConverter extends Serializable {
// only set alignment flags if read is aligned
if (m) {
// if we are aligned, we must have a reference
require(adamRecord.getContigName != null, "Cannot have null contig if aligned.")
builder.setReferenceName(adamRecord.getContigName)
require(adamRecord.getReferenceName != null, "Cannot have null reference if aligned.")
builder.setReferenceName(adamRecord.getReferenceName)

// set the cigar, if provided
Option(adamRecord.getCigar).foreach(builder.setCigarString)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.bdgenomics.adam.converters

import org.apache.spark.rdd.RDD
import org.bdgenomics.formats.avro.{ Contig, NucleotideContigFragment }
import org.bdgenomics.formats.avro.{ NucleotideContigFragment, Reference }
import scala.collection.mutable

/**
Expand All @@ -45,25 +45,25 @@ private[adam] object FastaConverter {
seqId: Int = 0,
descriptionLine: Option[String] = None) {
/**
* The contig name and description that was parsed out of this description line.
* The reference name and description that was parsed out of this description line.
*/
val (contigName, contigDescription) = parseDescriptionLine(descriptionLine, fileIndex)
val (referenceName, referenceDescription) = parseDescriptionLine(descriptionLine, fileIndex)

/**
* Parses the text of a given line.
*
* Assumes that the line contains the contig name followed by an optional
* description of the contig, with the two separated by a space.
* Assumes that the line contains the reference name followed by an optional
* description of the reference, with the two separated by a space.
*
* @throws IllegalArgumentException if there is no name in the line and the
* line is not the only record in a file (i.e., the file contains multiple
* contigs).
* reference sequences).
*
* @param descriptionLine The optional string describing the contig. If this
* @param descriptionLine The optional string describing the reference. If this
* is not set and this isn't the only line in the file, we throw.
* @param id The index of this contig in the file.
* @return Returns a tuple containing (the optional contig name, and the
* optional contig description).
* @param id The index of this reference in the file.
* @return Returns a tuple containing (the optional reference name, and the
* optional reference description).
*/
private def parseDescriptionLine(descriptionLine: Option[String],
id: Long): (Option[String], Option[String]) = {
Expand All @@ -80,10 +80,10 @@ private[adam] object FastaConverter {
if (split._1.contains('|')) {
(None, Some(dL.stripPrefix(">").trim))
} else {
val contigName: String = split._1.stripPrefix(">").trim
val contigDescription: String = split._2.trim
val referenceName: String = split._1.stripPrefix(">").trim
val referenceDescription: String = split._2.trim

(Some(contigName), Some(contigDescription))
(Some(referenceName), Some(referenceDescription))
}
} else {
(Some(dL.stripPrefix(">").trim), None)
Expand Down Expand Up @@ -121,33 +121,33 @@ private[adam] object FastaConverter {
.filter((kv: (Long, String)) => isFasta(kv._2))

val descriptionLines: Map[Long, FastaDescriptionLine] = getDescriptionLines(filtered)
val indexToContigDescription = rdd.context.broadcast(descriptionLines)
val indexToReferenceDescription = rdd.context.broadcast(descriptionLines)

val sequenceLines = filtered.filter(kv => !isDescriptionLine(kv._2))

val keyedSequences =
if (indexToContigDescription.value.isEmpty) {
if (indexToReferenceDescription.value.isEmpty) {
sequenceLines.keyBy(kv => -1L)
} else {
sequenceLines.keyBy(row => findContigIndex(row._1, indexToContigDescription.value.keys.toList))
sequenceLines.keyBy(row => findReferenceIndex(row._1, indexToReferenceDescription.value.keys.toList))
}

val groupedContigs = keyedSequences.groupByKey()
val groupedReferences = keyedSequences.groupByKey()

val converter = new FastaConverter(maximumLength)

groupedContigs.flatMap {
groupedReferences.flatMap {
case (id, lines) =>

val descriptionLine = indexToContigDescription.value.getOrElse(id, FastaDescriptionLine())
val descriptionLine = indexToReferenceDescription.value.getOrElse(id, FastaDescriptionLine())
assert(lines.nonEmpty, s"Sequence ${descriptionLine.seqId} has no sequence data.")

val sequence: Seq[String] = lines.toSeq.sortBy(_._1).map(kv => cleanSequence(kv._2))
converter.convert(
descriptionLine.contigName,
descriptionLine.referenceName,
descriptionLine.seqId,
sequence,
descriptionLine.contigDescription
descriptionLine.referenceDescription
)
}
}
Expand Down Expand Up @@ -194,18 +194,18 @@ private[adam] object FastaConverter {
}

/**
* Finds the index of a contig.
* Finds the index of a reference sequence.
*
* The index of a contig is the highest index below the index of our row.
* The index of a reference is the highest index below the index of our row.
* Here, we define the index as the row number of the description line that
* describes this contig.
* describes this reference sequence.
*
* @param rowIdx The row number of the contig row to check.
* @param rowIdx The row number of the reference sequence row to check.
* @param indices A list containing the row numbers of all description lines.
* @return Returns the row index of the description line that describes this
* sequence line.
*/
private[converters] def findContigIndex(rowIdx: Long, indices: List[Long]): Long = {
private[converters] def findReferenceIndex(rowIdx: Long, indices: List[Long]): Long = {
val idx = indices.filter(_ <= rowIdx)
idx.max
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,26 +114,26 @@ private[adam] object FragmentConverter extends Serializable {
}

/**
* Converts a contig assembly into one or more reads.
* Converts a reference assembly into one or more reads.
*
* Takes in a reduced key value pair containing merged FragmentCollectors.
* The strings that are in this FragmentCollector are used to create reads.
* The Contig key is used to populate the metadata for the contig.
* The reference key is used to populate the metadata for the reference.
*
* @param kv (Contig metadata, FragmentCollector) key value pair.
* @param kv (Reference metadata, FragmentCollector) key value pair.
* @return Returns one alignment record per sequence in the collector.
*/
private[converters] def convertFragment(kv: (String, FragmentCollector)): Seq[AlignmentRecord] = {
// extract kv pair
val (contigName, fragment) = kv
val (referenceName, fragment) = kv

// extract the fragment string and region
fragment.fragments.map(p => {
val (fragmentRegion, fragmentString) = p

// build record
AlignmentRecord.newBuilder()
.setContigName(contigName)
.setReferenceName(referenceName)
.setStart(fragmentRegion.start)
.setEnd(fragmentRegion.end)
.setSequence(fragmentString)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,11 +101,11 @@ private[adam] class SAMRecordConverter extends Serializable with Logging {
// This prevents looking up a -1 in the sequence dictionary
val readReference: Int = samRecord.getReferenceIndex
if (readReference != SAMRecord.NO_ALIGNMENT_REFERENCE_INDEX) {
builder.setContigName(samRecord.getReferenceName)
builder.setReferenceName(samRecord.getReferenceName)

// set read alignment flag
val start: Int = samRecord.getAlignmentStart
assert(start != 0, "Start cannot equal 0 if contig is set.")
assert(start != 0, "Start cannot equal 0 if reference is set.")
builder.setStart(start - 1L)

// set OP and OC flags, if applicable
Expand Down Expand Up @@ -146,7 +146,7 @@ private[adam] class SAMRecordConverter extends Serializable with Logging {
val mateReference: Int = samRecord.getMateReferenceIndex

if (mateReference != SAMRecord.NO_ALIGNMENT_REFERENCE_INDEX) {
builder.setMateContigName(samRecord.getMateReferenceName)
builder.setMateReferenceName(samRecord.getMateReferenceName)

val mateStart = samRecord.getMateAlignmentStart
if (mateStart > 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1679,7 +1679,7 @@ class VariantContextConverter(

// create the builder
val variantBuilder = Variant.newBuilder
.setContigName(vc.getChr)
.setReferenceName(vc.getChr)
.setStart(vc.getStart - 1)
.setEnd(vc.getEnd)
.setReferenceAllele(vc.getReference.getBaseString)
Expand Down Expand Up @@ -1791,7 +1791,7 @@ class VariantContextConverter(
// create the builder
val builder = Genotype.newBuilder()
.setVariant(variant)
.setContigName(variant.getContigName)
.setReferenceName(variant.getReferenceName)
.setStart(variant.getStart)
.setEnd(variant.getEnd)
.setSampleId(g.getSampleName)
Expand Down Expand Up @@ -2182,7 +2182,7 @@ class VariantContextConverter(
val hasNonRefAlleles = vc.genotypes
.exists(_.getNonReferenceLikelihoods.length != 0)
val builder = new VariantContextBuilder()
.chr(v.getContigName)
.chr(v.getReferenceName)
.start(v.getStart + 1)
.stop(v.getEnd)
.alleles(VariantContextConverter.convertAlleles(v, hasNonRefAlleles))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,14 +45,14 @@ private[adam] object Coverage {
* @return Coverage spanning the specified feature
*/
def apply(feature: Feature): Coverage = {
require(feature.getContigName != null && feature.getContigName.length > 0,
"Features must have Contig name to convert to Coverage")
require(feature.getReferenceName != null && feature.getReferenceName.length > 0,
"Features must have reference name to convert to Coverage")
require(feature.getStart != null && feature.getEnd != null,
"Features must have valid position data to convert to Coverage")
require(feature.getScore != null,
"Features must have valid score to convert to Coverage")

Coverage(feature.getContigName,
Coverage(feature.getReferenceName,
feature.getStart,
feature.getEnd,
feature.getScore,
Expand All @@ -73,18 +73,18 @@ private[adam] object Coverage {
/**
* Coverage record for CoverageDataset.
*
* Contains Region indexed by contig name, start and end, as well as the average
* Contains Region indexed by reference name, start and end, as well as the average
* coverage at each base pair in that region.
*
* @param contigName The chromosome that this coverage was observed on.
* @param referenceName The chromosome that this coverage was observed on.
* @param start The start coordinate of the region where this coverage value was
* observed.
* @param end The end coordinate of the region where this coverage value was
* observed.
* @param optSampleId Option of sampleId for this Coverage record
* @param count The average coverage across this region.
*/
case class Coverage(contigName: String, start: Long, end: Long, count: Double, optSampleId: Option[String] = None) {
case class Coverage(referenceName: String, start: Long, end: Long, count: Double, optSampleId: Option[String] = None) {

/**
* Converts Coverage to Feature, setting Coverage count in the score attribute.
Expand All @@ -93,7 +93,7 @@ case class Coverage(contigName: String, start: Long, end: Long, count: Double, o
*/
def toFeature: Feature = {
val featureBuilder = Feature.newBuilder()
.setContigName(contigName)
.setReferenceName(referenceName)
.setStart(start)
.setEnd(end)
.setScore(count)
Expand All @@ -114,7 +114,7 @@ case class Coverage(contigName: String, start: Long, end: Long, count: Double, o
name = None,
source = None,
featureType = None,
contigName = Some(contigName),
referenceName = Some(referenceName),
start = Some(start),
end = Some(end),
strand = None,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.bdgenomics.formats.avro.Variant
import org.bdgenomics.utils.misc.Logging

private[adam] class IndelTable(private val table: Map[String, Iterable[Consensus]]) extends Serializable with Logging {
log.info("Indel table has %s contigs and %s entries".format(
log.info("Indel table has %s reference sequences and %s entries".format(
table.size,
table.values.map(_.size).sum
))
Expand Down Expand Up @@ -58,7 +58,7 @@ private[adam] object IndelTable {
def apply(variants: RDD[Variant]): IndelTable = {
val consensus: Map[String, Iterable[Consensus]] = variants.filter(v => v.getReferenceAllele.length != v.getAlternateAllele.length)
.map(v => {
val referenceName = v.getContigName
val referenceName = v.getReferenceName
val consensus = if (v.getReferenceAllele.length > v.getAlternateAllele.length) {
// deletion
val deletionLength = v.getReferenceAllele.length - v.getAlternateAllele.length
Expand Down
Loading

0 comments on commit e17890d

Please sign in to comment.