Skip to content

Commit

Permalink
Adding fragment InFormatter for Bowtie tab6 format
Browse files Browse the repository at this point in the history
  • Loading branch information
heuermh authored and fnothaft committed Apr 25, 2017
1 parent 5b6a109 commit dbe5c97
Show file tree
Hide file tree
Showing 6 changed files with 263 additions and 35 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,31 +30,26 @@ import scala.collection.JavaConversions._
class AlignmentRecordConverter extends Serializable {

/**
* Converts a single record to FASTQ. FASTQ format is:
*
* {{{
* @readName
* sequence
* +<optional readname>
* ASCII quality scores
* }}}
* Prepare a single record for conversion to FASTQ and similar formats by
* splitting into a tuple of (name, sequence, qualityScores).
*
* If the base qualities are unknown (qual is null or equals "*"), the quality
* scores will be a repeated string of 'B's that is equal to the read length.
*
* @param adamRecord Read to convert to FASTQ.
* @param adamRecord Read to prepare for conversion to FASTQ and similar formats.
* @param maybeAddSuffix If true, check if a "/%d" suffix is attached to the
* read. If there is no suffix, a slash and the number of the read in the
* sequenced fragment is appended to the readname. Default is false.
* @param outputOriginalBaseQualities If true and the original base quality
* field is set (SAM "OQ" tag), outputs the original qualities. Else,
* output the qual field. Defaults to false.
* @return Returns this read in string form.
* @return Returns tuple of (name, sequence, qualityScores).
*/
def convertToFastq(
private def prepareFastq(
adamRecord: AlignmentRecord,
maybeAddSuffix: Boolean = false,
outputOriginalBaseQualities: Boolean = false): String = {
maybeAddSuffix: Boolean,
outputOriginalBaseQualities: Boolean): (String, String, String) = {

val readNameSuffix =
if (maybeAddSuffix &&
!AlignmentRecordConverter.readNameHasPairedSuffix(adamRecord) &&
Expand Down Expand Up @@ -84,9 +79,8 @@ class AlignmentRecordConverter extends Serializable {
else
adamRecord.getQual

"@%s%s\n%s\n+\n%s".format(
adamRecord.getReadName,
readNameSuffix,
(
adamRecord.getReadName + readNameSuffix,
if (adamRecord.getReadNegativeStrand)
Alphabet.dna.reverseComplement(adamRecord.getSequence)
else
Expand All @@ -98,6 +92,71 @@ class AlignmentRecordConverter extends Serializable {
)
}

/**
* Converts a single record to FASTQ format.
*
* FASTQ format is:
* {{{
* @readName
* sequence
* +<optional readname>
* ASCII quality scores
* }}}
*
* If the base qualities are unknown (qual is null or equals "*"), the quality
* scores will be a repeated string of 'B's that is equal to the read length.
*
* @param adamRecord Read to convert to FASTQ.
* @param maybeAddSuffix If true, check if a "/%d" suffix is attached to the
* read. If there is no suffix, a slash and the number of the read in the
* sequenced fragment is appended to the readname. Default is false.
* @param outputOriginalBaseQualities If true and the original base quality
* field is set (SAM "OQ" tag), outputs the original qualities. Else,
* output the qual field. Defaults to false.
* @return Returns this read in string form.
*/
def convertToFastq(
adamRecord: AlignmentRecord,
maybeAddSuffix: Boolean = false,
outputOriginalBaseQualities: Boolean = false): String = {

val (name, sequence, qualityScores) =
prepareFastq(adamRecord, maybeAddSuffix, outputOriginalBaseQualities)

"@%s\n%s\n+\n%s".format(name, sequence, qualityScores)
}

/**
* Converts a single record to Bowtie tab6 format.
*
* In Bowtie tab6 format, each alignment record or pair is on a single line.
* An unpaired alignment record line is [name]\t[seq]\t[qual]\n.
* For paired-end alignment records, the second end can have a different name
* from the first: [name1]\t[seq1]\t[qual1]\t[name2]\t[seq2]\t[qual2]\n.
*
* If the base qualities are unknown (qual is null or equals "*"), the quality
* scores will be a repeated string of 'B's that is equal to the read length.
*
* @param adamRecord Read to convert to FASTQ.
* @param maybeAddSuffix If true, check if a "/%d" suffix is attached to the
* read. If there is no suffix, a slash and the number of the read in the
* sequenced fragment is appended to the readname. Default is false.
* @param outputOriginalBaseQualities If true and the original base quality
* field is set (SAM "OQ" tag), outputs the original qualities. Else,
* output the qual field. Defaults to false.
* @return Returns this read in string form.
*/
def convertToTab6(
adamRecord: AlignmentRecord,
maybeAddSuffix: Boolean = false,
outputOriginalBaseQualities: Boolean = false): String = {

val (name, sequence, qualityScores) =
prepareFastq(adamRecord, maybeAddSuffix, outputOriginalBaseQualities)

"%s\t%s\t%s".format(name, sequence, qualityScores)
}

/**
* Converts a single ADAM record into a SAM record.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,20 @@ private[adam] class FragmentArraySerializer extends IntervalArraySerializer[Refe
*/
object FragmentRDD {

/**
* Hadoop configuration path to check for a boolean value indicating whether
* the current or original read qualities should be written. True indicates
* to write the original qualities. The default is false.
*/
val WRITE_ORIGINAL_QUALITIES = "org.bdgenomics.adam.rdd.fragment.FragmentRDD.writeOriginalQualities"

/**
* Hadoop configuration path to check for a boolean value indicating whether
* to write the "/1" "/2" suffixes to the read name that indicate whether a
* read is first or second in a pair. Default is false (no suffixes).
*/
val WRITE_SUFFIXES = "org.bdgenomics.adam.rdd.fragment.FragmentRDD.writeSuffixes"

/**
* Creates a FragmentRDD where no record groups or sequence info are attached.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,20 +30,6 @@ import org.bdgenomics.utils.misc.Logging
*/
object InterleavedFASTQInFormatter extends InFormatterCompanion[Fragment, FragmentRDD, InterleavedFASTQInFormatter] {

/**
* Hadoop configuration path to check for a boolean value indicating whether
* the current or original read qualities should be written. True indicates
* to write the original qualities. The default is false.
*/
val WRITE_ORIGINAL_QUAL = "org.bdgenomics.adam.rdd.fragment.InterleavedFASTQInFormatter.writeOriginalQual"

/**
* Hadoop configuration path to check for a boolean value indicating whether
* to write the "/1" "/2" suffixes to the read name that indicate whether a
* read is first or second in a pair. Default is false (no suffixes).
*/
val WRITE_SUFFIXES = "org.bdgenomics.adam.rdd.fragment.InterleavedFASTQInFormatter.writeSuffixes"

/**
* Builds an InterleavedFASTQInFormatter to write Interleaved FASTQ.
*
Expand All @@ -60,8 +46,8 @@ class InterleavedFASTQInFormatter private (

protected val companion = InterleavedFASTQInFormatter
private val converter = new AlignmentRecordConverter
private val writeSuffixes = conf.getBoolean(InterleavedFASTQInFormatter.WRITE_SUFFIXES, false)
private val writeOQ = conf.getBoolean(InterleavedFASTQInFormatter.WRITE_ORIGINAL_QUAL, false)
private val writeSuffixes = conf.getBoolean(FragmentRDD.WRITE_SUFFIXES, false)
private val writeOriginalQualities = conf.getBoolean(FragmentRDD.WRITE_ORIGINAL_QUALITIES, false)

/**
* Writes alignment records to an output stream in interleaved FASTQ format.
Expand All @@ -88,10 +74,10 @@ class InterleavedFASTQInFormatter private (
// convert both reads to fastq
val fastq1 = converter.convertToFastq(read1,
maybeAddSuffix = writeSuffixes,
outputOriginalBaseQualities = writeOQ) + "\n"
outputOriginalBaseQualities = writeOriginalQualities) + "\n"
val fastq2 = converter.convertToFastq(read2,
maybeAddSuffix = writeSuffixes,
outputOriginalBaseQualities = writeOQ) + "\n"
outputOriginalBaseQualities = writeOriginalQualities) + "\n"

// write both to the output stream
os.write(fastq1.getBytes)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you 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.rdd.fragment

import java.io.OutputStream
import org.apache.hadoop.conf.Configuration
import org.bdgenomics.adam.converters.AlignmentRecordConverter
import org.bdgenomics.adam.rdd.{ InFormatter, InFormatterCompanion }
import org.bdgenomics.formats.avro.Fragment
import org.bdgenomics.utils.misc.Logging

/**
* InFormatter companion that creates an InFormatter that writes Bowtie tab6 format.
*/
object Tab6InFormatter extends InFormatterCompanion[Fragment, FragmentRDD, Tab6InFormatter] {

/**
* Builds an Tab6InFormatter to write Bowtie tab6 format.
*
* @param gRdd GenomicRDD of Fragments. Used to get HadoopConfiguration.
* @return Returns a new Tab6InFormatter.
*/
def apply(gRdd: FragmentRDD): Tab6InFormatter = {
new Tab6InFormatter(gRdd.rdd.context.hadoopConfiguration)
}
}

class Tab6InFormatter private (
conf: Configuration) extends InFormatter[Fragment, FragmentRDD, Tab6InFormatter] with Logging {

protected val companion = Tab6InFormatter
private val newLine = "\n".getBytes
private val converter = new AlignmentRecordConverter
private val writeSuffixes = conf.getBoolean(FragmentRDD.WRITE_SUFFIXES, false)
private val writeOriginalQualities = conf.getBoolean(FragmentRDD.WRITE_ORIGINAL_QUALITIES, false)

/**
* Writes alignment records to an output stream in Bowtie tab6 format.
*
* In Bowtie tab6 format, each alignment record or pair is on a single line.
* An unpaired alignment record line is [name]\t[seq]\t[qual]\n.
* For paired-end alignment records, the second end can have a different name
* from the first: [name1]\t[seq1]\t[qual1]\t[name2]\t[seq2]\t[qual2]\n.
*
* @param os An OutputStream connected to a process we are piping to.
* @param iter An iterator of records to write.
*/
def write(os: OutputStream, iter: Iterator[Fragment]) {
iter.map(frag => {
val reads = converter.convertFragment(frag).toSeq

if (reads.size < 2) {
reads
} else {
if (reads.size > 2) {
log.warn("More than two reads for %s. Taking first 2.".format(frag))
}
reads.take(2)
}
}).foreach(reads => {

// write unpaired read or first of paired-end reads
val first = converter.convertToTab6(reads(0),
maybeAddSuffix = writeSuffixes,
outputOriginalBaseQualities = writeOriginalQualities)

os.write(first.getBytes)

// write second of paired-end reads, if present
if (reads.size > 1) {
val second = "\t" + converter.convertToTab6(reads(1),
maybeAddSuffix = writeSuffixes,
outputOriginalBaseQualities = writeOriginalQualities)

os.write(second.getBytes)
}

// end line
os.write(newLine)
})
}
}
51 changes: 51 additions & 0 deletions adam-core/src/test/resources/tab6_to_usam.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
#!/usr/bin/env python

from __future__ import print_function
import sys

# read lines from stdin
lines = sys.stdin.readlines()

# print sam header
print("@HD\tVN:1.5\tSO:unsorted")

# loop and print sam lines
for line in lines:
fields = line.split()
firstReadName = fields[0]
firstSequence = fields[1]
firstQualities = fields[2]
secondReadName = fields[3]
secondSequence = fields[4]
secondQualities = fields[5]

# flags:
# 1 = paired (we assume that in this script)
# 4 = unmapped
# 8 = mate unmapped
# 64 = first of pair
# 128 = second of pair
firstFlags = 64 | 8 | 4 | 1
secondFlags = 128 | 8 | 4 | 1

# sam is the following tab-delimited columns:
#
# 1. read name
# 2. flags
# 3. ref (* = unaligned)
# 4. pos (0 = unaligned)
# 5. map qual (0 if unmapped)
# 6. cigar (* = unavailable)
# 7. mate ref (* = unaligned)
# 8. mate pos (0 = unaligned)
# 9. tlen (0 = unknown)
# 10. sequence
# 11. qualities
print("%s\t%d\t*\t0\t0\t*\t*\t0\t0\t%s\t%s" % (firstReadName,
firstFlags,
firstSequence,
firstQualities))
print("%s\t%d\t*\t0\t0\t*\t*\t0\t0\t%s\t%s" % (secondReadName,
secondFlags,
secondSequence,
secondQualities))
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ class FragmentRDDSuite extends ADAMFunSuite {

sparkTest("don't lose any reads when piping interleaved fastq to sam") {
// write suffixes at end of reads
sc.hadoopConfiguration.setBoolean(InterleavedFASTQInFormatter.WRITE_SUFFIXES, true)
sc.hadoopConfiguration.setBoolean(FragmentRDD.WRITE_SUFFIXES, true)

val fragmentsPath = testFile("interleaved_fastq_sample1.ifq")
val ardd = sc.loadFragments(fragmentsPath)
Expand All @@ -44,6 +44,27 @@ class FragmentRDDSuite extends ADAMFunSuite {
assert(2 * records === newRecords)
}

sparkTest("don't lose any reads when piping tab6 to sam") {
// write suffixes at end of reads
sc.hadoopConfiguration.setBoolean(FragmentRDD.WRITE_SUFFIXES, true)

val fragmentsPath = testFile("interleaved_fastq_sample1.ifq")
val ardd = sc.loadFragments(fragmentsPath)
val records = ardd.rdd.count
assert(records === 3)

implicit val tFormatter = Tab6InFormatter
implicit val uFormatter = new AnySAMOutFormatter

// this script converts tab6 to unaligned sam
val scriptPath = testFile("tab6_to_usam.py")

val pipedRdd: AlignmentRecordRDD = ardd.pipe("python $0",
files = Seq(scriptPath))
val newRecords = pipedRdd.rdd.count
assert(2 * records === newRecords)
}

sparkTest("use broadcast join to pull down fragments mapped to targets") {
val fragmentsPath = testFile("small.1.sam")
val targetsPath = testFile("small.1.bed")
Expand Down

0 comments on commit dbe5c97

Please sign in to comment.