Skip to content

Commit

Permalink
Merge branch 'master' into guice
Browse files Browse the repository at this point in the history
  • Loading branch information
heuermh committed Aug 24, 2015
2 parents 7d3bcd1 + 633fdf9 commit 2ba41e5
Show file tree
Hide file tree
Showing 24 changed files with 658 additions and 48 deletions.
24 changes: 24 additions & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
@@ -1,5 +1,29 @@
# ADAM #

### Version 0.17.1 ###
* ISSUE [784](https://github.com/bigdatagenomics/adam/pull/784): [ADAM-783] Write @SQ header lines in sorted order.
* ISSUE [792](https://github.com/bigdatagenomics/adam/pull/792): [ADAM-791] Add repartition parameter to Fasta2ADAM.
* ISSUE [781](https://github.com/bigdatagenomics/adam/pull/781): [ADAM-777] Add validation stringency flag for BQSR.
* ISSUE [757](https://github.com/bigdatagenomics/adam/pull/757): We should print a warning message if the user has ADAM_OPTS set.
* ISSUE [770](https://github.com/bigdatagenomics/adam/pull/770): [ADAM-769] Fix serialization issue in known indel consensus model.
* ISSUE [763](https://github.com/bigdatagenomics/adam/pull/763): Clean up README links, other nits
* ISSUE [749](https://github.com/bigdatagenomics/adam/pull/749): Remove adam-cli jar from classpath during adam-submit
* ISSUE [754](https://github.com/bigdatagenomics/adam/pull/754): Bump ADAM to Spark 1.4
* ISSUE [753](https://github.com/bigdatagenomics/adam/pull/753): Bump Spark to 1.4
* ISSUE [748](https://github.com/bigdatagenomics/adam/pull/748): Fix for mdtag issues with insertions
* ISSUE [746](https://github.com/bigdatagenomics/adam/pull/746): Upgrade to Parquet 1.8.1.
* ISSUE [744](https://github.com/bigdatagenomics/adam/pull/744): [ADAM-743] exclude conflicting jackson dependencies
* ISSUE [737](https://github.com/bigdatagenomics/adam/pull/737): Reverse complement negative strand reads in fastq output
* ISSUE [731](https://github.com/bigdatagenomics/adam/pull/731): Fixed bug preventing use of TLEN attribute
* ISSUE [730](https://github.com/bigdatagenomics/adam/pull/730): [ADAM-729] Stuff TLEN into attributes.
* ISSUE [728](https://github.com/bigdatagenomics/adam/pull/728): [ADAM-709] Remove FeatureHierarchy and FeatureHierarchySuite
* ISSUE [719](https://github.com/bigdatagenomics/adam/pull/719): [ADAM-718] Use filesystem path to get underlying file system.
* ISSUE [712](https://github.com/bigdatagenomics/adam/pull/712): unify header-setting between BAM/SAM and VCF
* ISSUE [696](https://github.com/bigdatagenomics/adam/pull/696): include SequenceRecords from second-in-pair reads
* ISSUE [698](https://github.com/bigdatagenomics/adam/pull/698): class-ify ShuffleRegionJoin, force setting seqdict
* ISSUE [706](https://github.com/bigdatagenomics/adam/pull/706): restore clause guarding pruneCache check
* ISSUE [705](https://github.com/bigdatagenomics/adam/pull/705): GeneFeatureRDDFunctions → FeatureRDDFunctions

### Version 0.17.0 ###
* ISSUE [691](https://github.com/bigdatagenomics/adam/pull/691): fix BAM/SAM header setting when writing on cluster
* ISSUE [688](https://github.com/bigdatagenomics/adam/pull/688): make adamLoad public
Expand Down
2 changes: 1 addition & 1 deletion adam-apis/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@
<parent>
<groupId>org.bdgenomics.adam</groupId>
<artifactId>adam-parent_2.10</artifactId>
<version>0.17.1-SNAPSHOT</version>
<version>0.17.2-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

Expand Down
2 changes: 1 addition & 1 deletion adam-cli/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@
<parent>
<groupId>org.bdgenomics.adam</groupId>
<artifactId>adam-parent_2.10</artifactId>
<version>0.17.1-SNAPSHOT</version>
<version>0.17.2-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@ class Fasta2ADAMArgs extends Args4jBase with ParquetSaveArgs {
var reads: String = ""
@Args4jOption(required = false, name = "-fragment_length", usage = "Sets maximum fragment length. Default value is 10,000. Values greater than 1e9 should be avoided.")
var fragmentLength: Long = 10000L
@Args4jOption(required = false, name = "-repartition", usage = "Sets the number of output partitions to write, if desired.")
var partitions: Int = -1
}

class Fasta2ADAM(protected val args: Fasta2ADAMArgs) extends BDGSparkCommand[Fasta2ADAMArgs] with Logging {
Expand All @@ -57,7 +59,13 @@ class Fasta2ADAM(protected val args: Fasta2ADAMArgs) extends BDGSparkCommand[Fas
}

log.info("Writing records to disk.")
adamFasta.adamParquetSave(args)
val finalFasta = if (args.partitions > 0) {
adamFasta.repartition(args.partitions)
} else {
adamFasta
}

finalFasta.adamParquetSave(args)
}
}

14 changes: 12 additions & 2 deletions adam-cli/src/main/scala/org/bdgenomics/adam/cli/Transform.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.bdgenomics.adam.cli

import htsjdk.samtools.ValidationStringency
import org.apache.spark.rdd.RDD
import org.apache.spark.{ Logging, SparkContext }
import org.bdgenomics.adam.algorithms.consensus._
Expand Down Expand Up @@ -49,6 +50,8 @@ class TransformArgs extends Args4jBase with ADAMSaveAnyArgs with ParquetArgs {
var markDuplicates: Boolean = false
@Args4jOption(required = false, name = "-recalibrate_base_qualities", usage = "Recalibrate the base quality scores (ILLUMINA only)")
var recalibrateBaseQualities: Boolean = false
@Args4jOption(required = false, name = "-strict_bqsr", usage = "Run BQSR with strict validation.")
var strictBQSR: Boolean = false
@Args4jOption(required = false, name = "-dump_observations", usage = "Local path to dump BQSR observations to. Outputs CSV format.")
var observationsPath: String = null
@Args4jOption(required = false, name = "-known_snps", usage = "Sites-only VCF giving location of known SNPs")
Expand Down Expand Up @@ -118,7 +121,14 @@ class Transform(protected val args: TransformArgs) extends BDGSparkCommand[Trans
if (args.recalibrateBaseQualities) {
log.info("Recalibrating base qualities")
val knownSnps: SnpTable = createKnownSnpsTable(sc)
adamRecords = adamRecords.adamBQSR(sc.broadcast(knownSnps), Option(args.observationsPath))
val stringency = if (args.strictBQSR) {
ValidationStringency.STRICT
} else {
ValidationStringency.LENIENT
}
adamRecords = adamRecords.adamBQSR(sc.broadcast(knownSnps),
Option(args.observationsPath),
stringency)
}

if (args.coalesce != -1) {
Expand Down Expand Up @@ -148,7 +158,7 @@ class Transform(protected val args: TransformArgs) extends BDGSparkCommand[Trans
} else {
sc.loadAlignments(args.inputPath)
}
}).adamSave(args)
}).adamSave(args, args.sortReads)
}

private def createKnownSnpsTable(sc: SparkContext): SnpTable = CreateKnownSnpsTable.time {
Expand Down
2 changes: 1 addition & 1 deletion adam-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@
<parent>
<groupId>org.bdgenomics.adam</groupId>
<artifactId>adam-parent_2.10</artifactId>
<version>0.17.1-SNAPSHOT</version>
<version>0.17.2-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,161 @@
/**
* 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.io

import hbparquet.hadoop.util.ContextUtil
import htsjdk.samtools.BAMRecordCodec
import htsjdk.samtools.ValidationStringency
import htsjdk.samtools.SAMRecord
import htsjdk.samtools.util.BlockCompressedInputStream
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.fs.FSDataInputStream
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.mapreduce.InputSplit
import org.apache.hadoop.mapreduce.RecordReader
import org.apache.hadoop.mapreduce.TaskAttemptContext
import org.bdgenomics.adam.models.ReferenceRegion
import org.seqdoop.hadoop_bam.util.SAMHeaderReader
import org.seqdoop.hadoop_bam.util.WrapSeekable
import org.seqdoop.hadoop_bam.SAMRecordWritable
import org.seqdoop.hadoop_bam.BAMRecordReader
import org.seqdoop.hadoop_bam.FileVirtualSplit
import org.seqdoop.hadoop_bam.BAMInputFormat
import scala.annotation.tailrec

object BAMFilteredRecordReader {
private var optViewRegion: Option[ReferenceRegion] = None

def setRegion(viewRegion: ReferenceRegion) {
optViewRegion = Some(viewRegion)
}
}

/**
* Scala implementation of BAMRecordReader, but with
* nextKeyValue() that filters by ReferenceRegion
*/
class BAMFilteredRecordReader extends BAMRecordReader {

val key: LongWritable = new LongWritable()
val record: SAMRecordWritable = new SAMRecordWritable

var stringency: ValidationStringency = _

var bci: BlockCompressedInputStream = _
var codec: BAMRecordCodec = _
var fileStart: Long = _
var virtualEnd: Long = _
var isInitialized: Boolean = false

override def initialize(spl: InputSplit, ctx: TaskAttemptContext) {
// Check to ensure this method is only be called once (see Hadoop API)
if (isInitialized) {
close()
}
isInitialized = true

val conf: Configuration = ContextUtil.getConfiguration(ctx)

val split: FileVirtualSplit = spl.asInstanceOf[FileVirtualSplit]
val file: Path = split.getPath()
val fs: FileSystem = file.getFileSystem(conf)

this.stringency = SAMHeaderReader.getValidationStringency(conf)

val in: FSDataInputStream = fs.open(file)

// Sets codec to translate between in-memory and disk representation of record
codec = new BAMRecordCodec(SAMHeaderReader.readSAMHeaderFrom(in, conf))

in.seek(0)

bci = new BlockCompressedInputStream(
new WrapSeekable[FSDataInputStream](
in, fs.getFileStatus(file).getLen(), file))

// Gets BGZF virtual offset for the split
val virtualStart = split.getStartVirtualOffset()

fileStart = virtualStart >>> 16
virtualEnd = split.getEndVirtualOffset()

// Starts looking from the BGZF virtual offset
bci.seek(virtualStart)
// Reads records from this input stream
codec.setInputStream(bci)
}

override def close() = {
bci.close()
}

override def getCurrentKey(): LongWritable = {
key
}

override def getCurrentValue(): SAMRecordWritable = {
record
}

/**
* This method gets the nextKeyValue for our RecordReader, but filters by only
* returning records within a specified ReferenceRegion.
* This function is tail recursive to avoid stack overflow when predicate data
* can be sparse.
*/
@tailrec final override def nextKeyValue(): Boolean = {
if (bci.getFilePointer() >= virtualEnd) {
false
} else {
val r: SAMRecord = codec.decode()

// Since we're reading from a BAMRecordCodec directly we have to set the
// validation stringency ourselves.
if (this.stringency != null) {
r.setValidationStringency(this.stringency)
}

// This if/else block pushes the predicate down onto a BGZIP block that
// the index has said contains data in our specified region.
if (r == null) {
false
} else {
val start = r.getStart
val end = r.getEnd
val refReg = BAMFilteredRecordReader.optViewRegion.get
val regStart = refReg.start
val regEnd = refReg.end

if ((r.getContig() == refReg.referenceName) &&
(((start >= regStart) && (end <= regEnd))
|| ((start <= regStart) && (end >= regStart) && (end <= regEnd))
|| ((end >= regEnd) && (start >= regStart) && (start <= regEnd)))) {
key.set(BAMRecordReader.getKey(r))
record.set(r)
true
} else {
nextKeyValue()
}
}
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
/**
* 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 htsjdk.samtools

import htsjdk.samtools.util.BlockCompressedFilePointerUtil
import java.io.File
import java.io.FileNotFoundException
import org.apache.hadoop.fs.{ FileSystem, Path }
import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.mapreduce.JobContext
import org.apache.hadoop.mapreduce.RecordReader
import org.apache.hadoop.mapreduce.InputSplit
import org.apache.hadoop.mapreduce.TaskAttemptContext
import org.apache.hadoop.mapreduce.lib.input.FileSplit
import org.bdgenomics.adam.models.ReferenceRegion
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.io.BAMFilteredRecordReader
import org.seqdoop.hadoop_bam.BAMInputFormat
import org.seqdoop.hadoop_bam.FileVirtualSplit
import org.seqdoop.hadoop_bam.SAMRecordWritable
import scala.collection.mutable

object IndexedBamInputFormat {

private var optFilePath: Option[Path] = None
private var optIndexFilePath: Option[Path] = None
private var optViewRegion: Option[ReferenceRegion] = None
private var optDict: Option[SAMSequenceDictionary] = None

def setVars(filePath: Path, indexFilePath: Path, viewRegion: ReferenceRegion, dict: SAMSequenceDictionary) {
optFilePath = Some(filePath)
optIndexFilePath = Some(indexFilePath)
optViewRegion = Some(viewRegion)
optDict = Some(dict)
}

}

class IndexedBamInputFormat extends BAMInputFormat {

override def createRecordReader(split: InputSplit, ctx: TaskAttemptContext): RecordReader[LongWritable, SAMRecordWritable] = {
val rr: RecordReader[LongWritable, SAMRecordWritable] = new BAMFilteredRecordReader()
assert(IndexedBamInputFormat.optViewRegion.isDefined)
BAMFilteredRecordReader.setRegion(IndexedBamInputFormat.optViewRegion.get)
rr.initialize(split, ctx)
rr
}

override def getSplits(job: JobContext): java.util.List[InputSplit] = {
assert(IndexedBamInputFormat.optIndexFilePath.isDefined &&
IndexedBamInputFormat.optFilePath.isDefined &&
IndexedBamInputFormat.optViewRegion.isDefined &&
IndexedBamInputFormat.optDict.isDefined)

val indexFilePath = IndexedBamInputFormat.optIndexFilePath.get

val idxFile: File = new File(indexFilePath.toString)
if (!idxFile.exists()) {
throw new java.io.FileNotFoundException("Bam index file not provided")
} else {
// Use index to get the chunks for a specific region, then use them to create InputSplits
val filePath = IndexedBamInputFormat.optFilePath.get
val viewRegion = IndexedBamInputFormat.optViewRegion.get
val refName = viewRegion.referenceName
val dict = IndexedBamInputFormat.optDict.get
val start = viewRegion.start.toInt
val end = viewRegion.end.toInt
val dbbfi: DiskBasedBAMFileIndex = new DiskBasedBAMFileIndex(idxFile, dict)
val referenceIndex: Int = dict.getSequenceIndex(refName)
// Get the chunks in the region we want (chunks give start and end file pointers into a BAM file)
var regions: List[Chunk] = dbbfi.getSpanOverlapping(referenceIndex, start, end).getChunks

var splits = new mutable.ListBuffer[FileVirtualSplit]()
for (chunk <- regions) {
// Create InputSplits from chunks in a given region
val start: Long = chunk.getChunkStart()
val end: Long = chunk.getChunkEnd()
val locs = Array[String]()
val newSplit = new FileVirtualSplit(filePath, start, end, locs)
splits += newSplit
}
splits.toList
}
}

}
Loading

0 comments on commit 2ba41e5

Please sign in to comment.