diff --git a/adam-apis/src/main/scala/org/bdgenomics/adam/apis/java/JavaADAMContext.scala b/adam-apis/src/main/scala/org/bdgenomics/adam/apis/java/JavaADAMContext.scala
index 2521f3ac6c..e6f0749466 100644
--- a/adam-apis/src/main/scala/org/bdgenomics/adam/apis/java/JavaADAMContext.scala
+++ b/adam-apis/src/main/scala/org/bdgenomics/adam/apis/java/JavaADAMContext.scala
@@ -27,9 +27,9 @@ import org.bdgenomics.adam.rdd.feature.FeatureRDD
import org.bdgenomics.adam.rdd.fragment.FragmentRDD
import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD
import org.bdgenomics.adam.rdd.variation.{
- DatabaseVariantAnnotationRDD,
GenotypeRDD,
- VariantRDD
+ VariantRDD,
+ VariantAnnotationRDD
}
import org.bdgenomics.formats.avro._
import scala.collection.JavaConversions._
@@ -93,9 +93,9 @@ class JavaADAMContext private (val ac: ADAMContext) extends Serializable {
* Loads in variant annotations.
*
* @param filePath The path to load the file from.
- * @return Returns a DatabaseVariantAnnotationRDD.
+ * @return Returns a VariantAnnotationRDD.
*/
- def loadVariantAnnotations(filePath: java.lang.String): DatabaseVariantAnnotationRDD = {
+ def loadVariantAnnotations(filePath: java.lang.String): VariantAnnotationRDD = {
ac.loadVariantAnnotations(filePath)
}
diff --git a/adam-apis/src/test/java/org/bdgenomics/adam/apis/java/JavaADAMAnnotationConduit.java b/adam-apis/src/test/java/org/bdgenomics/adam/apis/java/JavaADAMAnnotationConduit.java
index 24e07059ae..0f79a26cd5 100644
--- a/adam-apis/src/test/java/org/bdgenomics/adam/apis/java/JavaADAMAnnotationConduit.java
+++ b/adam-apis/src/test/java/org/bdgenomics/adam/apis/java/JavaADAMAnnotationConduit.java
@@ -25,20 +25,20 @@
import org.bdgenomics.adam.models.RecordGroupDictionary;
import org.bdgenomics.adam.models.SequenceDictionary;
import org.bdgenomics.adam.rdd.ADAMContext;
-import org.bdgenomics.adam.rdd.variation.DatabaseVariantAnnotationRDD;
+import org.bdgenomics.adam.rdd.variation.VariantAnnotationRDD;
/**
* A simple test class for the JavaADAMRDD/Context. Writes an RDD of annotations to
* disk and reads it back.
*/
class JavaADAMAnnotationConduit {
- public static DatabaseVariantAnnotationRDD conduit(DatabaseVariantAnnotationRDD recordRdd,
- ADAMContext ac) throws IOException {
+ public static VariantAnnotationRDD conduit(VariantAnnotationRDD annotationRdd,
+ ADAMContext ac) throws IOException {
// make temp directory and save file
Path tempDir = Files.createTempDirectory("javaAC");
String fileName = tempDir.toString() + "/testRdd.annotation.adam";
- recordRdd.save(fileName);
+ annotationRdd.save(fileName);
// create a new adam context and load the file
JavaADAMContext jac = new JavaADAMContext(ac);
diff --git a/adam-cli/src/main/scala/org/bdgenomics/adam/cli/AlleleCount.scala b/adam-cli/src/main/scala/org/bdgenomics/adam/cli/AlleleCount.scala
index 0e347869d7..39206d6a80 100644
--- a/adam-cli/src/main/scala/org/bdgenomics/adam/cli/AlleleCount.scala
+++ b/adam-cli/src/main/scala/org/bdgenomics/adam/cli/AlleleCount.scala
@@ -49,8 +49,8 @@ class AlleleCountArgs extends Args4jBase with ParquetArgs {
object AlleleCountHelper extends Serializable {
def chooseAllele(x: (String, java.lang.Long, String, String, GenotypeAllele)) =
x match {
- case (chr, position, refAllele, varAllele, GenotypeAllele.Ref) => Some(chr, position, refAllele)
- case (chr, position, refAllele, varAllele, GenotypeAllele.Alt) => Some(chr, position, varAllele)
+ case (chr, position, refAllele, varAllele, GenotypeAllele.REF) => Some(chr, position, refAllele)
+ case (chr, position, refAllele, varAllele, GenotypeAllele.ALT) => Some(chr, position, varAllele)
case _ => None
}
diff --git a/adam-cli/src/main/scala/org/bdgenomics/adam/cli/VcfAnnotation2ADAM.scala b/adam-cli/src/main/scala/org/bdgenomics/adam/cli/VcfAnnotation2ADAM.scala
index c933571fe4..00178a6bff 100644
--- a/adam-cli/src/main/scala/org/bdgenomics/adam/cli/VcfAnnotation2ADAM.scala
+++ b/adam-cli/src/main/scala/org/bdgenomics/adam/cli/VcfAnnotation2ADAM.scala
@@ -22,7 +22,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.bdgenomics.adam.models.VariantContext
import org.bdgenomics.adam.rdd.ADAMContext._
-import org.bdgenomics.adam.rdd.variation.DatabaseVariantAnnotationRDD
+import org.bdgenomics.adam.rdd.variation.VariantAnnotationRDD
import org.bdgenomics.adam.rich.RichVariant
import org.bdgenomics.formats.avro._
import org.bdgenomics.utils.cli._
@@ -60,7 +60,7 @@ class VcfAnnotation2ADAM(val args: VcfAnnotation2ADAMArgs) extends BDGSparkComma
val keyedAnnotations = existingAnnotations.rdd.keyBy(anno => new RichVariant(anno.getVariant))
val joinedAnnotations = keyedAnnotations.join(annotations.rdd.keyBy(anno => new RichVariant(anno.getVariant)))
val mergedAnnotations = joinedAnnotations.map(kv => VariantContext.mergeAnnotations(kv._2._1, kv._2._2))
- DatabaseVariantAnnotationRDD(mergedAnnotations, existingAnnotations.sequences).saveAsParquet(args)
+ VariantAnnotationRDD(mergedAnnotations, existingAnnotations.sequences).saveAsParquet(args)
} else {
annotations.saveAsParquet(args)
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantAnnotations.scala b/adam-core/src/main/scala/org/bdgenomics/adam/converters/TranscriptEffectConverter.scala
similarity index 52%
rename from adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantAnnotations.scala
rename to adam-core/src/main/scala/org/bdgenomics/adam/converters/TranscriptEffectConverter.scala
index 75ee9ab244..5e6eba7bd0 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantAnnotations.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/converters/TranscriptEffectConverter.scala
@@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.bdgenomics.adam.converters
import htsjdk.samtools.ValidationStringency
@@ -31,7 +30,14 @@ import org.bdgenomics.formats.avro.{
import org.bdgenomics.utils.misc.Logging
import scala.collection.JavaConverters._
-object VariantAnnotations extends Serializable with Logging {
+/**
+ * Convert between htsjdk VCF INFO reserved key "ANN" values and TranscriptEffects.
+ */
+private[adam] object TranscriptEffectConverter extends Serializable with Logging {
+
+ /**
+ * Look up variant annotation messages by name and message code.
+ */
private val MESSAGES: Map[String, VariantAnnotationMessage] = Map(
// name -> enum
ERROR_CHROMOSOME_NOT_FOUND.name() -> ERROR_CHROMOSOME_NOT_FOUND,
@@ -57,15 +63,33 @@ object VariantAnnotations extends Serializable with Logging {
"I3" -> INFO_NON_REFERENCE_ANNOTATION
)
+ /**
+ * Split effects by &
character.
+ *
+ * @param s effects to split
+ * @return effects split by &
character
+ */
private def parseEffects(s: String): List[String] = {
s.split("&").toList
}
+ /**
+ * Split variant effect messages by &
character.
+ *
+ * @param s variant effect messages to split
+ * @return variant effect messages split by &
character
+ */
private def parseMessages(s: String): List[VariantAnnotationMessage] = {
// todo: haven't seen a delimiter here, assuming it is also '&'
s.split("&").map(MESSAGES.get(_)).toList.flatten
}
+ /**
+ * Split a single or fractional value into optional numerator and denominator values.
+ *
+ * @param s single or fractional value to split
+ * @return single or fractional value split into optional numerator and denominator values
+ */
private def parseFraction(s: String): (Option[Integer], Option[Integer]) = {
if ("".equals(s)) {
return (None, None)
@@ -78,10 +102,23 @@ object VariantAnnotations extends Serializable with Logging {
}
}
- def setIfNotEmpty(s: String, setFn: String => Unit) {
+ /**
+ * Set a value via a function if the value is not empty.
+ *
+ * @param s value to set
+ * @param setFn function to call if the value is not empty
+ */
+ private def setIfNotEmpty(s: String, setFn: String => Unit) {
Option(s).filter(_.nonEmpty).foreach(setFn)
}
+ /**
+ * Parse zero or one transcript effects from the specified string value.
+ *
+ * @param s value to parse
+ * @param stringency validation stringency
+ * @return zero or one transcript effects parsed from the specified string value
+ */
private[converters] def parseTranscriptEffect(
s: String,
stringency: ValidationStringency): Seq[TranscriptEffect] = {
@@ -110,7 +147,8 @@ object VariantAnnotations extends Serializable with Logging {
val te = TranscriptEffect.newBuilder()
setIfNotEmpty(alternateAllele, te.setAlternateAllele(_))
- if (!effects.isEmpty) te.setEffects(effects.asJava)
+ if (effects.nonEmpty) te.setEffects(effects.asJava)
+ // note: annotationImpact is output by SnpEff but is not part of the VCF ANN specification
setIfNotEmpty(geneName, te.setGeneName(_))
setIfNotEmpty(geneId, te.setGeneId(_))
setIfNotEmpty(featureType, te.setFeatureType(_))
@@ -132,6 +170,13 @@ object VariantAnnotations extends Serializable with Logging {
Seq(te.build())
}
+ /**
+ * Parse the VCF INFO reserved key "ANN" value into zero or more TranscriptEffects.
+ *
+ * @param s string to parse
+ * @param stringency validation stringency
+ * @return the VCF INFO reserved key "ANN" value parsed into zero or more TranscriptEffects
+ */
private[converters] def parseAnn(
s: String,
stringency: ValidationStringency): List[TranscriptEffect] = {
@@ -139,19 +184,101 @@ object VariantAnnotations extends Serializable with Logging {
s.split(",").map(parseTranscriptEffect(_, stringency)).flatten.toList
}
- def createVariantAnnotation(
+ /**
+ * Convert the htsjdk VCF INFO reserved key "ANN" value into zero or more TranscriptEffects,
+ * matching on alternate allele.
+ *
+ * @param variant variant
+ * @param vc htsjdk variant context
+ * @param stringency validation stringency, defaults to strict
+ * @return the htsjdk VCF INFO reserved key "ANN" value converted into zero or more
+ * TranscriptEffects, matching on alternate allele, and wrapped in an option
+ */
+ def convertToTranscriptEffects(
variant: Variant,
vc: VariantContext,
- stringency: ValidationStringency = ValidationStringency.STRICT): VariantAnnotation = {
+ stringency: ValidationStringency = ValidationStringency.STRICT): Option[List[TranscriptEffect]] = {
+
+ def parseAndFilter(attr: String): Option[List[TranscriptEffect]] = {
+ if (attr == VCFConstants.MISSING_VALUE_v4) {
+ None
+ } else {
+ val filtered = parseAnn(attr, stringency)
+ .filter(_.getAlternateAllele == variant.getAlternateAllele)
+ if (filtered.isEmpty) {
+ None
+ } else {
+ Some(filtered)
+ }
+ }
+ }
- val va = VariantAnnotation.newBuilder()
- .setVariant(variant)
+ val attrOpt = Option(vc.getAttributeAsString("ANN", null))
+ try {
+ attrOpt.flatMap(attr => parseAndFilter(attr))
+ } catch {
+ case t: Throwable => {
+ if (stringency == ValidationStringency.STRICT) {
+ throw t
+ } else if (stringency == ValidationStringency.LENIENT) {
+ log.warn("Could not convert VCF INFO reserved key ANN value to TranscriptEffect, caught %s.".format(t))
+ }
+ None
+ }
+ }
+ }
+
+ /**
+ * Convert the specified transcript effects into a string suitable for a VCF INFO reserved
+ * key "ANN" value.
+ *
+ * @param effects zero or more transcript effects
+ * @return the specified transcript effects converted into a string suitable for a VCF INFO
+ * reserved key "ANN" value
+ */
+ def convertToVcfInfoAnnValue(effects: Seq[TranscriptEffect]): String = {
+ def toFraction(numerator: java.lang.Integer, denominator: java.lang.Integer): String = {
+ val numOpt = Option(numerator)
+ val denomOpt = Option(denominator)
+
+ (numOpt, denomOpt) match {
+ case (None, None) => {
+ ""
+ }
+ case (Some(n), None) => {
+ "%d".format(n)
+ }
+ case (None, Some(d)) => {
+ log.warn("Incorrect fractional value ?/%d, missing numerator".format(d))
+ ""
+ }
+ case (Some(n), Some(d)) => {
+ "%d/%d".format(n, d)
+ }
+ }
+ }
- val attr = vc.getAttributeAsString("ANN", null)
- if (attr != null && attr != VCFConstants.MISSING_VALUE_v4) {
- va.setTranscriptEffects(parseAnn(attr, stringency).asJava)
+ def toAnn(te: TranscriptEffect): String = {
+ Seq(
+ Option(te.getAlternateAllele).getOrElse(""), // 0
+ te.getEffects.asScala.mkString("&"), // 1
+ "", // annotationImpact
+ Option(te.getGeneName).getOrElse(""), // 3
+ Option(te.getGeneId).getOrElse(""),
+ Option(te.getFeatureType).getOrElse(""),
+ Option(te.getFeatureId).getOrElse(""),
+ Option(te.getBiotype).getOrElse(""),
+ toFraction(te.getRank, te.getTotal), // 8
+ Option(te.getTranscriptHgvs).getOrElse(""), // 9
+ Option(te.getProteinHgvs).getOrElse(""), // 10
+ toFraction(te.getCdnaPosition, te.getCdnaLength), // 11
+ toFraction(te.getCdsPosition, te.getCdsLength), // 12
+ toFraction(te.getProteinPosition, te.getProteinLength), // 13
+ Option(te.getDistance).getOrElse(""),
+ te.getMessages.asScala.mkString("&")
+ ).mkString("|")
}
- va.build()
+ effects.map(toAnn).mkString(",")
}
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantAnnotationConverter.scala b/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantAnnotationConverter.scala
index b291fd88c2..0766de6ca2 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantAnnotationConverter.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantAnnotationConverter.scala
@@ -22,8 +22,8 @@ import htsjdk.variant.vcf._
import org.apache.avro.Schema
import org.apache.avro.specific.SpecificRecord
import org.bdgenomics.formats.avro.{
- DatabaseVariantAnnotation,
Genotype,
+ VariantAnnotation,
VariantCallingAnnotations
}
import scala.collection.JavaConversions._
@@ -206,13 +206,13 @@ private[converters] object VariantAnnotationConverter extends Serializable {
* Mapping betweem VCF info field names and fields IDs in the
* VariantCallingAnnotations schema.
*/
- lazy val VCF2VariantCallingAnnotations: Map[String, (Int, Object => Object)] =
+ lazy val vcfToVariantCallingAnnotations: Map[String, (Int, Object => Object)] =
createFieldMap(INFO_KEYS, VariantCallingAnnotations.getClassSchema)
/**
* Mapping between VCF format field names and field IDs in the Genotype schema.
*/
- lazy val VCF2GenotypeAnnotations: Map[String, (Int, Object => Object)] =
+ lazy val vcfToGenotypeAnnotations: Map[String, (Int, Object => Object)] =
createFieldMap(FORMAT_KEYS, Genotype.getClassSchema)
/**
@@ -223,11 +223,10 @@ private[converters] object VariantAnnotationConverter extends Serializable {
DBNSFP_KEYS) // ::: COSMIC_KEYS
/**
- * Mapping between VCF info field names and DatabaseVariantAnnotation schema
- * field IDs for all database specific fields.
+ * Mapping between VCF info field names and VariantAnnotation schema field IDs.
*/
- lazy val VCF2DatabaseAnnotations: Map[String, (Int, Object => Object)] = createFieldMap(EXTERNAL_DATABASE_KEYS,
- DatabaseVariantAnnotation.getClassSchema)
+ lazy val vcfToVariantAnnotations: Map[String, (Int, Object => Object)] = createFieldMap(EXTERNAL_DATABASE_KEYS,
+ VariantAnnotation.getClassSchema)
/**
* Creates a mapping between a Seq of attribute keys, and the field ID for
@@ -277,15 +276,15 @@ private[converters] object VariantAnnotationConverter extends Serializable {
}
/**
- * Remaps fields from an htsjdk variant context into a site annotation.
+ * Remaps fields from an htsjdk variant context into a variant annotation.
*
* @param vc htsjdk variant context for a site.
- * @param annotation Pre-populated site annotation in Avro.
- * @return Annotation with additional info fields filled in.
+ * @param annotation Pre-populated variant annotation in Avro.
+ * @return Variant annotation with additional info fields filled in.
*/
def convert(vc: VariantContext,
- annotation: DatabaseVariantAnnotation): DatabaseVariantAnnotation = {
- fillRecord(VCF2DatabaseAnnotations, vc, annotation)
+ annotation: VariantAnnotation): VariantAnnotation = {
+ fillRecord(vcfToVariantAnnotations, vc, annotation)
}
/**
@@ -298,7 +297,7 @@ private[converters] object VariantAnnotationConverter extends Serializable {
*/
def convert(vc: VariantContext,
call: VariantCallingAnnotations): VariantCallingAnnotations = {
- fillRecord(VCF2VariantCallingAnnotations, vc, call)
+ fillRecord(vcfToVariantCallingAnnotations, vc, call)
}
/**
@@ -312,7 +311,7 @@ private[converters] object VariantAnnotationConverter extends Serializable {
*/
def convert(g: htsjdk.variant.variantcontext.Genotype,
genotype: Genotype): Genotype = {
- for ((v, a) <- VariantAnnotationConverter.VCF2GenotypeAnnotations) {
+ for ((v, a) <- VariantAnnotationConverter.vcfToGenotypeAnnotations) {
// Add extended attributes if present
val attr = g.getExtendedAttribute(v)
if (attr != null && attr != VCFConstants.MISSING_VALUE_v4) {
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantContextConverter.scala b/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantContextConverter.scala
index ee46edcd82..5bc4549f5e 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantContextConverter.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/converters/VariantContextConverter.scala
@@ -17,6 +17,8 @@
*/
package org.bdgenomics.adam.converters
+import com.google.common.base.Splitter
+import com.google.common.collect.ImmutableList
import htsjdk.variant.variantcontext.{
Allele,
GenotypesContext,
@@ -24,6 +26,7 @@ import htsjdk.variant.variantcontext.{
VariantContext => HtsjdkVariantContext,
VariantContextBuilder
}
+import htsjdk.variant.vcf.VCFConstants
import java.util.Collections
import org.bdgenomics.utils.misc.Logging
import org.bdgenomics.adam.models.{
@@ -72,10 +75,10 @@ private[adam] object VariantContextConverter {
* @return The Avro representation for this allele.
*/
private def convertAllele(vc: HtsjdkVariantContext, allele: Allele): GenotypeAllele = {
- if (allele.isNoCall) GenotypeAllele.NoCall
- else if (allele.isReference) GenotypeAllele.Ref
- else if (allele == NON_REF_ALLELE || !vc.hasAlternateAllele(allele)) GenotypeAllele.OtherAlt
- else GenotypeAllele.Alt
+ if (allele.isNoCall) GenotypeAllele.NO_CALL
+ else if (allele.isReference) GenotypeAllele.REF
+ else if (allele == NON_REF_ALLELE || !vc.hasAlternateAllele(allele)) GenotypeAllele.OTHER_ALT
+ else GenotypeAllele.ALT
}
/**
@@ -123,9 +126,9 @@ private[adam] object VariantContextConverter {
var alleles = g.getAlleles
if (alleles == null) return Collections.emptyList[Allele]
else g.getAlleles.map {
- case GenotypeAllele.NoCall => Allele.NO_CALL
- case GenotypeAllele.Ref | GenotypeAllele.OtherAlt => Allele.create(g.getVariant.getReferenceAllele, true)
- case GenotypeAllele.Alt => Allele.create(g.getVariant.getAlternateAllele)
+ case GenotypeAllele.NO_CALL => Allele.NO_CALL
+ case GenotypeAllele.REF | GenotypeAllele.OTHER_ALT => Allele.create(g.getVariant.getReferenceAllele, true)
+ case GenotypeAllele.ALT => Allele.create(g.getVariant.getAlternateAllele)
}
}
}
@@ -294,9 +297,9 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
* Extracts a variant annotation from a htsjdk VariantContext.
*
* @param vc htsjdk variant context to extract annotations from.
- * @return The database annotations in Avro format.
+ * @return The variant annotations in Avro format.
*/
- def convertToAnnotation(vc: HtsjdkVariantContext): DatabaseVariantAnnotation = {
+ def convertToVariantAnnotation(vc: HtsjdkVariantContext): VariantAnnotation = {
val variant = vc.getAlternateAlleles.toList match {
case List(NON_REF_ALLELE) => {
createADAMVariant(vc, None /* No alternate allele */ )
@@ -324,7 +327,7 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
}
}
- extractVariantDatabaseAnnotation(variant, vc)
+ extractVariantAnnotation(variant, vc)
}
/**
@@ -340,6 +343,36 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
contigToRefSeq.getOrElse(vc.getChr, vc.getChr)
}
+ /**
+ * Split the htsjdk variant context ID field into an array of names.
+ *
+ * @param vc htsjdk variant context
+ * @return Returns an Option wrapping an array of names split from the htsjdk
+ * variant context ID field
+ */
+ private def splitIds(vc: HtsjdkVariantContext): Option[java.util.List[String]] = {
+ if (vc.hasID()) {
+ Some(ImmutableList.copyOf(vc.getID().split(VCFConstants.ID_FIELD_SEPARATOR)))
+ } else {
+ None
+ }
+ }
+
+ /**
+ * Join the array of variant names into a string for the htsjdk variant context ID field.
+ *
+ * @param variant variant
+ * @return Returns an Option wrapping a string for the htsjdk variant context ID field joined
+ * from the array of variant names
+ */
+ private def joinNames(variant: Variant): Option[String] = {
+ if (variant.getNames != null && variant.getNames.length > 0) {
+ Some(variant.getNames.mkString(VCFConstants.ID_FIELD_SEPARATOR))
+ } else {
+ None
+ }
+ }
+
/**
* Builds an avro Variant for a site with a defined alt allele.
*
@@ -349,30 +382,34 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
* @return Returns an Avro description of the genotyped site.
*/
private def createADAMVariant(vc: HtsjdkVariantContext, alt: Option[String]): Variant = {
- // VCF CHROM, POS, REF and ALT
+ // VCF CHROM, POS, ID, REF, FORMAT, and ALT
val builder = Variant.newBuilder
.setContigName(createContig(vc))
.setStart(vc.getStart - 1 /* ADAM is 0-indexed */ )
.setEnd(vc.getEnd /* ADAM is 0-indexed, so the 1-indexed inclusive end becomes exclusive */ )
.setReferenceAllele(vc.getReference.getBaseString)
- if (vc.hasLog10PError) {
- builder.setVariantErrorProbability(vc.getPhredScaledQual.intValue())
- }
alt.foreach(builder.setAlternateAllele(_))
+ splitIds(vc).foreach(builder.setNames(_))
+ builder.setFiltersApplied(vc.filtersWereApplied)
+ if (vc.filtersWereApplied) {
+ builder.setFiltersPassed(!vc.isFiltered)
+ }
+ if (vc.isFiltered) {
+ builder.setFiltersFailed(new java.util.ArrayList(vc.getFilters));
+ }
builder.build
}
/**
- * Populates a site annotation from an htsjdk variant context.
+ * Populates a variant annotation from an htsjdk variant context.
*
* @param variant Avro variant representation for the site.
* @param vc htsjdk representation of the VCF line.
- * @return Returns the Avro representation of the annotations at this site
- * that indicate membership in an annotation database.
+ * @return Returns the Avro representation of the variant annotations at this site.
*/
- private def extractVariantDatabaseAnnotation(variant: Variant,
- vc: HtsjdkVariantContext): DatabaseVariantAnnotation = {
- val annotation = DatabaseVariantAnnotation.newBuilder()
+ private def extractVariantAnnotation(variant: Variant,
+ vc: HtsjdkVariantContext): VariantAnnotation = {
+ val annotation = VariantAnnotation.newBuilder()
.setVariant(variant)
.build
@@ -417,10 +454,20 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
.setContigName(contigName)
.setStart(start)
.setEnd(end)
- .setVariantCallingAnnotations(annotations)
.setSampleId(g.getSampleName)
.setAlleles(g.getAlleles.map(VariantContextConverter.convertAllele(vc, _)))
- .setIsPhased(g.isPhased)
+ .setPhased(g.isPhased)
+
+ // copy variant calling annotations to update filter attributes
+ // (because the htsjdk Genotype is not available when build is called upstream)
+ val copy = VariantCallingAnnotations.newBuilder(annotations)
+ // htsjdk does not provide a field filtersWereApplied for genotype as it does in VariantContext
+ copy.setFiltersApplied(true)
+ copy.setFiltersPassed(!g.isFiltered)
+ if (g.isFiltered) {
+ copy.setFiltersFailed(Splitter.on(";").splitToList(g.getFilters))
+ }
+ genotype.setVariantCallingAnnotations(copy.build())
if (g.hasGQ) genotype.setGenotypeQuality(g.getGQ)
if (g.hasDP) genotype.setReadDepth(g.getDP)
@@ -517,13 +564,6 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
private def extractVariantCallingAnnotations(vc: HtsjdkVariantContext): VariantCallingAnnotations = {
val call: VariantCallingAnnotations.Builder = VariantCallingAnnotations.newBuilder
- // VCF QUAL, FILTER and INFO fields
- if (vc.filtersWereApplied && vc.isFiltered) {
- call.setVariantIsPassing(false).setVariantFilters(new java.util.ArrayList(vc.getFilters))
- } else if (vc.filtersWereApplied) {
- call.setVariantIsPassing(true)
- }
-
VariantAnnotationConverter.convert(vc, call.build())
}
@@ -562,7 +602,10 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
.stop(variant.getStart + variant.getReferenceAllele.length)
.alleles(VariantContextConverter.convertAlleles(variant))
- vc.databases.flatMap(d => Option(d.getDbSnpId)).foreach(d => vcb.id("rs" + d))
+ joinNames(variant) match {
+ case None => vcb.noID()
+ case Some(s) => vcb.id(s)
+ }
// TODO: Extract provenance INFO fields
try {
@@ -571,7 +614,7 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
g.getSampleId, VariantContextConverter.convertAlleles(g)
)
- Option(g.getIsPhased).foreach(gb.phased(_))
+ Option(g.getPhased).foreach(gb.phased(_))
Option(g.getGenotypeQuality).foreach(gb.GQ(_))
Option(g.getReadDepth).foreach(gb.DP(_))
@@ -589,8 +632,8 @@ private[adam] class VariantContextConverter(dict: Option[SequenceDictionary] = N
if (g.getVariantCallingAnnotations != null) {
val callAnnotations = g.getVariantCallingAnnotations()
- if (callAnnotations.getVariantFilters != null) {
- gb.filters(callAnnotations.getVariantFilters)
+ if (callAnnotations.getFiltersPassed() != null && !callAnnotations.getFiltersPassed()) {
+ gb.filters(callAnnotations.getFiltersFailed())
}
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/models/ReferenceRegion.scala b/adam-core/src/main/scala/org/bdgenomics/adam/models/ReferenceRegion.scala
index 740d7e033c..00cb0b8285 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/models/ReferenceRegion.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/models/ReferenceRegion.scala
@@ -145,10 +145,22 @@ object ReferenceRegion {
ReferenceRegion(variant.getContigName, variant.getStart, variant.getEnd)
}
- def apply(annotation: DatabaseVariantAnnotation): ReferenceRegion = {
+ /**
+ * Builds a reference region from a variant annotation.
+ *
+ * @param annotation VariantAnnotation to extract region from.
+ * @return The site where the variant for the specified variant annotation covers.
+ */
+ def apply(annotation: VariantAnnotation): ReferenceRegion = {
ReferenceRegion(annotation.getVariant)
}
+ /**
+ * Builds a reference region from an alignment record.
+ *
+ * @param record AlignmentRecord to extract region from.
+ * @return The site where the alignment record aligns.
+ */
def apply(record: AlignmentRecord): ReferenceRegion = {
require(record.getReadMapped,
"Cannot build reference region for unmapped read %s.".format(record))
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/models/VariantContext.scala b/adam-core/src/main/scala/org/bdgenomics/adam/models/VariantContext.scala
index 49d914148b..b45a91c3a5 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/models/VariantContext.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/models/VariantContext.scala
@@ -17,12 +17,12 @@
*/
package org.bdgenomics.adam.models
-import org.bdgenomics.formats.avro.{ Genotype, DatabaseVariantAnnotation, Variant }
+import org.bdgenomics.formats.avro.{ Genotype, Variant, VariantAnnotation }
import org.bdgenomics.adam.rich.RichVariant
/**
* Note: VariantContext inherits its name from the Picard VariantContext, and is not related to the SparkContext object.
- * If you're looking for the latter, see [[org.bdgenomics.adam.rdd.variation.VariationContext]]
+ * If you're looking for the latter, see [[org.bdgenomics.adam.rdd.ADAMContext]].
*/
object VariantContext {
@@ -37,10 +37,10 @@ object VariantContext {
* @return A new VariantContext, where an annotation has been added/merged.
*/
def apply(v: VariantContext,
- optAnn: Option[DatabaseVariantAnnotation]): VariantContext = {
+ optAnn: Option[VariantAnnotation]): VariantContext = {
// if the join yielded one or fewer annotation, pick what we've got. else, merge.
- val ann = (v.databases, optAnn) match {
+ val ann = (v.annotations, optAnn) match {
case (None, a) => a
case (a, None) => a
case (Some(a), Some(b)) => Some(mergeAnnotations(a, b))
@@ -66,11 +66,11 @@ object VariantContext {
* from the left record.
* @return Returns the union of these two annotations.
*/
- def mergeAnnotations(leftRecord: DatabaseVariantAnnotation,
- rightRecord: DatabaseVariantAnnotation): DatabaseVariantAnnotation = {
- val mergedAnnotation = DatabaseVariantAnnotation.newBuilder(leftRecord)
+ def mergeAnnotations(leftRecord: VariantAnnotation,
+ rightRecord: VariantAnnotation): VariantAnnotation = {
+ val mergedAnnotation = VariantAnnotation.newBuilder(leftRecord)
.build()
- val numFields = DatabaseVariantAnnotation.getClassSchema.getFields.size
+ val numFields = VariantAnnotation.getClassSchema.getFields.size
def insertField(fieldIdx: Int) =
{
@@ -91,7 +91,7 @@ object VariantContext {
* optional domain annotation at site))
* @return VariantContext corresponding to the data above.
*/
- def apply(kv: (ReferencePosition, Variant, Iterable[Genotype], Option[DatabaseVariantAnnotation])): VariantContext = {
+ def apply(kv: (ReferencePosition, Variant, Iterable[Genotype], Option[VariantAnnotation])): VariantContext = {
new VariantContext(kv._1, kv._2, kv._3, kv._4)
}
@@ -111,10 +111,10 @@ object VariantContext {
*
* @param v Variant which is used to construct the ReferencePosition
* @param genotypes Seq[Genotype]
- * @param annotation Option[DatabaseVariantAnnotation]
+ * @param annotation Option[VariantAnnotation]
* @return VariantContext corresponding to the Variant
*/
- def apply(v: Variant, genotypes: Iterable[Genotype], annotation: Option[DatabaseVariantAnnotation] = None): VariantContext = {
+ def apply(v: Variant, genotypes: Iterable[Genotype], annotation: Option[VariantAnnotation] = None): VariantContext = {
apply((ReferencePosition(v), v, genotypes, annotation))
}
@@ -143,6 +143,5 @@ class VariantContext(
val position: ReferencePosition,
val variant: RichVariant,
val genotypes: Iterable[Genotype],
- val databases: Option[DatabaseVariantAnnotation] = None) {
+ val annotations: Option[VariantAnnotation] = None) {
}
-
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/AlignmentRecordField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/AlignmentRecordField.scala
index 7e6d869221..cf1c3ed3e3 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/projections/AlignmentRecordField.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/AlignmentRecordField.scala
@@ -24,5 +24,5 @@ import org.bdgenomics.formats.avro.AlignmentRecord
*/
object AlignmentRecordField extends FieldEnumeration(AlignmentRecord.SCHEMA$) {
- val contigName, start, end, mapq, readName, sequence, mateAlignmentStart, cigar, qual, recordGroupId, recordGroupName, readPaired, properPair, readMapped, mateMapped, readNegativeStrand, mateNegativeStrand, readInFragment, primaryAlignment, failedVendorQualityChecks, duplicateRead, mismatchingPositions, attributes, recordGroupSequencingCenter, recordGroupDescription, recordGroupRunDateEpoch, recordGroupFlowOrder, recordGroupKeySequence, recordGroupLibrary, recordGroupPredictedMedianInsertSize, recordGroupPlatform, recordGroupPlatformUnit, recordGroupSample, mateContigName, origQual, supplementaryAlignment, secondaryAlignment = SchemaValue
+ val readInFragment, contigName, start, oldPosition, end, mapq, readName, sequence, qual, cigar, oldCigar, basesTrimmedFromStart, basesTrimmedFromEnd, readPaired, properPair, readMapped, mateMapped, failedVendorQualityChecks, duplicateRead, readNegativeStrand, mateNegativeStrand, primaryAlignment, secondaryAlignment, supplementaryAlignment, mismatchingPositions, origQual, attributes, recordGroupName, recordGroupSample, mateAlignmentStart, mateAlignmentEnd, mateContigName, inferredInsertSize = SchemaValue
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/DatabaseVariantAnnotationField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/ContigField.scala
similarity index 52%
rename from adam-core/src/main/scala/org/bdgenomics/adam/projections/DatabaseVariantAnnotationField.scala
rename to adam-core/src/main/scala/org/bdgenomics/adam/projections/ContigField.scala
index 077c69d864..52bf0bd783 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/projections/DatabaseVariantAnnotationField.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/ContigField.scala
@@ -17,18 +17,12 @@
*/
package org.bdgenomics.adam.projections
-import org.bdgenomics.formats.avro.DatabaseVariantAnnotation
+import org.bdgenomics.formats.avro.Contig
/**
- * Enumeration of DatabaseVariantAnnotation field names for predicates and projections.
+ * Enumeration of Contig field names for predicates and projections.
*/
-object DatabaseVariantAnnotationField extends FieldEnumeration(DatabaseVariantAnnotation.SCHEMA$) {
+object ContigField extends FieldEnumeration(Contig.SCHEMA$) {
- val variant, dbsnpId, //domain information
- hgvs, geneSymbol, ensemblGeneId, ensemblTranscriptIds, //clinical fields
- omimId, cosmicId, clinvarId, clinicalSignificance, //conservation
- gerpNr, gerpRs, phylop, ancestralAllele, //population statistics
- thousandGenomesAlleleCount, thousandGenomesAlleleFrequency, //effect
- referenceAminoAcid, alternateAminoAcid, //predicted effects
- siftScore, siftScoreConverted, siftPred, mutationTasterScore, mutationTasterScoreConverted, mutationTasterPred = SchemaValue
+ val contigName, contigLength, contigMD5, referenceURL, assembly, species, referenceIndex = SchemaValue
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/FeatureField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/FeatureField.scala
index a421f7d0b1..297a2e51e7 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/projections/FeatureField.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/FeatureField.scala
@@ -24,5 +24,5 @@ import org.bdgenomics.formats.avro.Feature
*/
object FeatureField extends FieldEnumeration(Feature.SCHEMA$) {
- val featureId, name, source, featureType, contigName, start, end, strand, phase, frame, score, geneId, transcriptId, exonId, aliases, parentIds, target, gap, derivesFrom, notes, dbxrefs, ontologyTerms, isCircular, attributes = SchemaValue
+ val featureId, name, source, featureType, contigName, start, end, strand, phase, frame, score, geneId, transcriptId, exonId, aliases, parentIds, target, gap, derivesFrom, notes, dbxrefs, ontologyTerms, circular, attributes = SchemaValue
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/FragmentField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/FragmentField.scala
new file mode 100644
index 0000000000..4f56171ec0
--- /dev/null
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/FragmentField.scala
@@ -0,0 +1,28 @@
+/**
+ * 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.projections
+
+import org.bdgenomics.formats.avro.Fragment
+
+/**
+ * Enumeration of Fragment field names for predicates and projections.
+ */
+object FragmentField extends FieldEnumeration(Fragment.SCHEMA$) {
+
+ val readName, instrument, runId, fragmentSize, alignments = SchemaValue
+}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/GenotypeField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/GenotypeField.scala
index f1bf083c5c..ee58581998 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/projections/GenotypeField.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/GenotypeField.scala
@@ -24,5 +24,5 @@ import org.bdgenomics.formats.avro.Genotype
*/
object GenotypeField extends FieldEnumeration(Genotype.SCHEMA$) {
- val variant, contigName, start, end, variantCallingAnnotations, sampleId, sampleDescription, processingDescription, alleles, referenceReadDepth, alternateReadDepth, readDepth, genotypeQuality, genotypeLikelihoods, splitFromMultiAllelic, isPhased, phaseSetId, phaseQuality = SchemaValue
+ val variant, contigName, start, end, variantCallingAnnotations, sampleId, sampleDescription, processingDescription, alleles, expectedAlleleDosage, referenceReadDepth, alternateReadDepth, readDepth, minReadDepth, genotypeQuality, genotypeLikelihoods, nonReferenceLikelihoods, strandBiasComponents, splitFromMultiAllelic, phased, phaseSetId, phaseQuality = SchemaValue
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/NucleotideContigFragmentField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/NucleotideContigFragmentField.scala
index 1b8bdcbc41..a9674c557f 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/projections/NucleotideContigFragmentField.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/NucleotideContigFragmentField.scala
@@ -24,5 +24,5 @@ import org.bdgenomics.formats.avro.NucleotideContigFragment
*/
object NucleotideContigFragmentField extends FieldEnumeration(NucleotideContigFragment.SCHEMA$) {
- val contig, description, fragmentSequence, fragmentNumber, fragmentStartPosition, fragmentEndPosition, numberOfFragmentsInContig, url = SchemaValue
+ val contig, description, fragmentSequence, fragmentNumber, fragmentStartPosition, fragmentEndPosition, fragmentLength, numberOfFragmentsInContig = SchemaValue
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/RecordGroupMetadataField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/RecordGroupMetadataField.scala
new file mode 100644
index 0000000000..a45a230d2c
--- /dev/null
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/RecordGroupMetadataField.scala
@@ -0,0 +1,28 @@
+/**
+ * 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.projections
+
+import org.bdgenomics.formats.avro.RecordGroupMetadata
+
+/**
+ * Enumeration of RecordGroupMetadata field names for predicates and projections.
+ */
+object RecordGroupMetadataField extends FieldEnumeration(RecordGroupMetadata.SCHEMA$) {
+
+ val name, sample, sequencingCenter, description, runDateEpoch, flowOrder, keySequence, library, predictedMedianInsertSize, platform, platformUnit = SchemaValue
+}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/SampleField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/SampleField.scala
new file mode 100644
index 0000000000..beda5ccaa9
--- /dev/null
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/SampleField.scala
@@ -0,0 +1,28 @@
+/**
+ * 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.projections
+
+import org.bdgenomics.formats.avro.Sample
+
+/**
+ * Enumeration of Sample field names for predicates and projections.
+ */
+object SampleField extends FieldEnumeration(Sample.SCHEMA$) {
+
+ val sampleId, name, attributes = SchemaValue
+}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/TranscriptEffectField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/TranscriptEffectField.scala
new file mode 100644
index 0000000000..646aa8661b
--- /dev/null
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/TranscriptEffectField.scala
@@ -0,0 +1,28 @@
+/**
+ * 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.projections
+
+import org.bdgenomics.formats.avro.TranscriptEffect
+
+/**
+ * Enumeration of TranscriptEffect field names for predicates and projections.
+ */
+object TranscriptEffectField extends FieldEnumeration(TranscriptEffect.SCHEMA$) {
+
+ val alternateAllele, effects, geneName, geneId, featureType, featureId, biotype, rank, total, genomicHgvs, transcriptHgvs, proteinHgvs, cdnaPosition, cdnaLength, cdsPosition, cdsLength, proteinPosition, proteinLength, distance, messages = SchemaValue
+}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantAnnotationField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantAnnotationField.scala
new file mode 100644
index 0000000000..5279c91d83
--- /dev/null
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantAnnotationField.scala
@@ -0,0 +1,28 @@
+/**
+ * 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.projections
+
+import org.bdgenomics.formats.avro.VariantAnnotation
+
+/**
+ * Enumeration of VariantAnnotation field names for predicates and projections.
+ */
+object VariantAnnotationField extends FieldEnumeration(VariantAnnotation.SCHEMA$) {
+
+ val variant, ancestralAllele, alleleCount, readDepth, forwardReadDepth, reverseReadDepth, alleleFrequency, cigar, dbSnp, hapMap2, hapMap3, validated, thousandGenomes, transcriptEffects, attributes = SchemaValue
+}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantCallingAnnotationsField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantCallingAnnotationsField.scala
index 5d9cfd7a98..5728a43aa7 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantCallingAnnotationsField.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantCallingAnnotationsField.scala
@@ -24,5 +24,5 @@ import org.bdgenomics.formats.avro.VariantCallingAnnotations
*/
object VariantCallingAnnotationsField extends FieldEnumeration(VariantCallingAnnotations.SCHEMA$) {
- val readDepth, downsampled, baseQRankSum, clippingRankSum, haplotypeScore, inbreedingCoefficient, alleleCountMLE, alleleFrequencyMLE, rmsMapQ, mapq0Reads, mqRankSum, usedForNegativeTrainingSet, usedForPositiveTrainingSet, variantQualityByDepth, readPositionRankSum, vqslod, culprit, variantCallErrorProbability, variantIsPassing, variantFilters = SchemaValue
+ val variantFilters, downsampled, baseQRankSum, fisherStrandBiasPValue, rmsMapQ, mapq0Reads, mqRankSum, readPositionRankSum, genotypePriors, vqslod, culprit, attributes = SchemaValue
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantField.scala b/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantField.scala
index f3464162e9..d1f2bd63dc 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantField.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/projections/VariantField.scala
@@ -24,5 +24,5 @@ import org.bdgenomics.formats.avro.Variant
*/
object VariantField extends FieldEnumeration(Variant.SCHEMA$) {
- val contig, start, end, referenceAllele, variantAllele = SchemaValue
+ val contigName, start, end, names, referenceAllele, alternateAllele, filtersApplied, filtersPassed, filtersFailed, somatic = SchemaValue
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/ADAMContext.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/ADAMContext.scala
index 61806fe6f5..f797b71fc5 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/ADAMContext.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/ADAMContext.scala
@@ -267,7 +267,7 @@ class ADAMContext private (@transient val sc: SparkContext) extends Serializable
* @tparam T The type of records to return
* @return An RDD with records of the specified type
*/
- private[rdd] def loadParquet[T](
+ def loadParquet[T](
filePath: String,
predicate: Option[FilterPredicate] = None,
projection: Option[Schema] = None)(implicit ev1: T => SpecificRecord, ev2: Manifest[T]): RDD[T] = {
@@ -1172,37 +1172,33 @@ class ADAMContext private (@transient val sc: SparkContext) extends Serializable
/**
* Loads variant annotations stored in VCF format.
*
- * @see loadVcf
- * @see loadVariantAnnotations
- *
* @param filePath The path to the VCF file(s) to load annotations from.
- * @return Returns DatabaseVariantAnnotationRDD.
+ * @return Returns VariantAnnotationRDD.
*/
- def loadVcfAnnotations(filePath: String): DatabaseVariantAnnotationRDD = {
- loadVcf(filePath).toDatabaseVariantAnnotationRDD
+ def loadVcfAnnotations(
+ filePath: String): VariantAnnotationRDD = {
+ loadVcf(filePath).toVariantAnnotationRDD
}
/**
- * Loads DatabaseVariantAnnotations stored in Parquet, with metadata.
- *
- * @see loadVariantAnnotations
+ * Loads VariantAnnotations stored in Parquet, with metadata.
*
* @param filePath The path to load files from.
* @param predicate An optional predicate to push down into the file.
* @param projection An optional projection to use for reading.
- * @return Returns DatabaseVariantAnnotationRDD.
+ * @return Returns VariantAnnotationRDD.
*/
def loadParquetVariantAnnotations(
filePath: String,
predicate: Option[FilterPredicate] = None,
- projection: Option[Schema] = None): DatabaseVariantAnnotationRDD = {
+ projection: Option[Schema] = None): VariantAnnotationRDD = {
val sd = loadAvroSequences(filePath)
- val rdd = loadParquet[DatabaseVariantAnnotation](filePath, predicate, projection)
- DatabaseVariantAnnotationRDD(rdd, sd)
+ val rdd = loadParquet[VariantAnnotation](filePath, predicate, projection)
+ VariantAnnotationRDD(rdd, sd)
}
/**
- * Loads DatabaseVariantAnnotations into an RDD, and automatically detects
+ * Loads VariantAnnotations into an RDD, and automatically detects
* the underlying storage format.
*
* Can load variant annotations from either Parquet or VCF.
@@ -1212,16 +1208,16 @@ class ADAMContext private (@transient val sc: SparkContext) extends Serializable
*
* @param filePath The path to load files from.
* @param projection An optional projection to use for reading.
- * @return Returns DatabaseVariantAnnotationRDD.
+ * @return Returns VariantAnnotationRDD.
*/
def loadVariantAnnotations(
filePath: String,
- projection: Option[Schema] = None): DatabaseVariantAnnotationRDD = {
+ projection: Option[Schema] = None): VariantAnnotationRDD = {
if (filePath.endsWith(".vcf")) {
log.info(s"Loading $filePath as VCF, and converting to variant annotations. Projection is ignored.")
loadVcfAnnotations(filePath)
} else {
- log.info(s"Loading $filePath as Parquet containing DatabaseVariantAnnotations.")
+ log.info(s"Loading $filePath as Parquet containing VariantAnnotations.")
loadParquetVariantAnnotations(filePath, None, projection)
}
}
@@ -1339,7 +1335,7 @@ class ADAMContext private (@transient val sc: SparkContext) extends Serializable
/**
* Auto-detects the file type and loads a GenotypeRDD.
*
- * If the file has a .vcf/.vcf.gz extension, loads as VCF. Else, falls back to
+ * If the file has a .vcf/.vcf.gz/.vcf.bgzf/.vcf.bgz extension, loads as VCF. Else, falls back to
* Parquet.
*
* @param filePath The path to load.
@@ -1364,7 +1360,7 @@ class ADAMContext private (@transient val sc: SparkContext) extends Serializable
/**
* Auto-detects the file type and loads a VariantRDD.
*
- * If the file has a .vcf/.vcf.gz extension, loads as VCF. Else, falls back to
+ * If the file has a .vcf/.vcf.gz/.vcf.bgzf/.vcf.bgz extension, loads as VCF. Else, falls back to
* Parquet.
*
* @param filePath The path to load.
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/Features.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/Features.scala
index 696df238a9..f44770828f 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/Features.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/Features.scala
@@ -146,7 +146,7 @@ private[feature] object Features {
case "Target" => f.setTarget(entry._2)
case "Gap" => f.setGap(entry._2)
case "Derives_from" => f.setDerivesFrom(entry._2)
- case "Is_circular" => f.setIsCircular(entry._2.toBoolean)
+ case "Is_circular" => f.setCircular(entry._2.toBoolean)
case "Alias" => aliases += entry._2
case "Note" => notes += entry._2
case "Parent" => parentIds += entry._2
@@ -193,7 +193,7 @@ private[feature] object Features {
Option(feature.getTarget).foreach(attrs += Tuple2("Target", _))
Option(feature.getGap).foreach(attrs += Tuple2("Gap", _))
Option(feature.getDerivesFrom).foreach(attrs += Tuple2("Derives_from", _))
- Option(feature.getIsCircular).foreach(addBooleanTuple)
+ Option(feature.getCircular).foreach(addBooleanTuple)
Option(feature.getGeneId).foreach(attrs += Tuple2("gene_id", _))
Option(feature.getTranscriptId).foreach(attrs += Tuple2("transcript_id", _))
Option(feature.getExonId).foreach(attrs += Tuple2("exon_id", _))
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/DatabaseVariantAnnotationRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/VariantAnnotationRDD.scala
similarity index 67%
rename from adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/DatabaseVariantAnnotationRDD.scala
rename to adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/VariantAnnotationRDD.scala
index a06916f759..8ef8702beb 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/DatabaseVariantAnnotationRDD.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/VariantAnnotationRDD.scala
@@ -20,7 +20,7 @@ package org.bdgenomics.adam.rdd.variation
import org.apache.spark.rdd.RDD
import org.bdgenomics.adam.models.{ ReferenceRegion, SequenceDictionary }
import org.bdgenomics.adam.rdd.{ AvroGenomicRDD, JavaSaveArgs }
-import org.bdgenomics.formats.avro.DatabaseVariantAnnotation
+import org.bdgenomics.formats.avro.VariantAnnotation
/**
* An RDD containing variant annotations against a given reference genome.
@@ -28,8 +28,8 @@ import org.bdgenomics.formats.avro.DatabaseVariantAnnotation
* @param rdd Variant annotations.
* @param sequences A dictionary describing the reference genome.
*/
-case class DatabaseVariantAnnotationRDD(rdd: RDD[DatabaseVariantAnnotation],
- sequences: SequenceDictionary) extends AvroGenomicRDD[DatabaseVariantAnnotation, DatabaseVariantAnnotationRDD] {
+case class VariantAnnotationRDD(rdd: RDD[VariantAnnotation],
+ sequences: SequenceDictionary) extends AvroGenomicRDD[VariantAnnotation, VariantAnnotationRDD] {
/**
* Java-friendly method for saving to Parquet.
@@ -42,17 +42,18 @@ case class DatabaseVariantAnnotationRDD(rdd: RDD[DatabaseVariantAnnotation],
/**
* @param newRdd An RDD for replacing the underlying RDD.
- * @return A new DatabaseVariantAnnotationRDD with the underlying RDD replaced.
+ * @return A new VariantAnnotationRDD with the underlying RDD replaced.
*/
- protected def replaceRdd(newRdd: RDD[DatabaseVariantAnnotation]): DatabaseVariantAnnotationRDD = {
+ protected def replaceRdd(newRdd: RDD[VariantAnnotation]): VariantAnnotationRDD = {
copy(rdd = newRdd)
}
/**
- * @param elem Database variant annotation to get a region for.
- * @return Returns the singular region covered by the variant.
+ * @param elem Variant annotation to extract a region from.
+ * @return Returns the singular region covered by the variant for the
+ * specified variant annotation.
*/
- protected def getReferenceRegions(elem: DatabaseVariantAnnotation): Seq[ReferenceRegion] = {
- Seq(ReferenceRegion(elem))
+ protected def getReferenceRegions(annotation: VariantAnnotation): Seq[ReferenceRegion] = {
+ Seq(ReferenceRegion(annotation))
}
}
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDD.scala
index 09adec93f6..5e43404285 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDD.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDD.scala
@@ -61,7 +61,7 @@ case class VariantContextRDD(rdd: RDD[VariantContext],
* @param ann Annotation RDD to join against.
* @return Returns a VariantContextRDD where annotations have been filled in.
*/
- def joinDatabaseVariantAnnotation(ann: DatabaseVariantAnnotationRDD): VariantContextRDD = {
+ def joinVariantAnnotations(ann: VariantAnnotationRDD): VariantContextRDD = {
replaceRdd(rdd.keyBy(_.variant)
.leftOuterJoin(ann.rdd.keyBy(_.getVariant))
.values
@@ -69,12 +69,11 @@ case class VariantContextRDD(rdd: RDD[VariantContext],
}
/**
- * @return Returns a DatabaseVariantAnnotationRDD containing the variant
+ * @return Returns a VariantAnnotationRDD containing the variant
* annotations attached to this VariantContextRDD.
*/
- def toDatabaseVariantAnnotationRDD: DatabaseVariantAnnotationRDD = {
- DatabaseVariantAnnotationRDD(rdd.flatMap(_.databases),
- sequences)
+ def toVariantAnnotationRDD: VariantAnnotationRDD = {
+ VariantAnnotationRDD(rdd.flatMap(_.annotations), sequences)
}
/**
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rich/RichGenotype.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rich/RichGenotype.scala
index ec20b6a324..fca64e30f3 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/rich/RichGenotype.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/rich/RichGenotype.scala
@@ -40,26 +40,26 @@ class RichGenotype(val genotype: Genotype) {
distinctListOfAlleles match {
// If all alleles are the reference allele, the genotype is Homozygous Reference (HOM_REF)
- case List(GenotypeAllele.Ref) => GenotypeType.HOM_REF
+ case List(GenotypeAllele.REF) => GenotypeType.HOM_REF
// If all alleles are the primary alternative allele, the genotype is Homozygous Alternative (HOM_ALT)
- case List(GenotypeAllele.Alt) => GenotypeType.HOM_ALT
+ case List(GenotypeAllele.ALT) => GenotypeType.HOM_ALT
// If all alleles are not called, the genotype is not called (NO_CALL)
- case List(GenotypeAllele.NoCall) => GenotypeType.NO_CALL
+ case List(GenotypeAllele.NO_CALL) => GenotypeType.NO_CALL
- // If all alleles are OtherAlt.
- // If genotype.getAlleles returns a single OtherAlt, the genotype is Homozygous Alternative (HOM_ALT)
- // If genotype.getAlleles returns a multiple OtherAlt, the genotype is
- // A) The OtherAlt alleles are the same OtherAlt alleles: Homozygous Alternative (HOM_ALT)
- // B) The OtherAlt allales are different OtherAlt alleles: Heterozygous
+ // If all alleles are OTHER_ALT.
+ // If genotype.getAlleles returns a single OTHER_ALT, the genotype is Homozygous Alternative (HOM_ALT)
+ // If genotype.getAlleles returns a multiple OTHER_ALT, the genotype is
+ // A) The OTHER_ALT alleles are the same OTHER_ALT alleles: Homozygous Alternative (HOM_ALT)
+ // B) The OTHER_ALT alleles are different OTHER_ALT alleles: Heterozygous
// For now return NO_CALL as the genotypes, as was done in the previous getType function
// See also issue https://github.com/bigdatagenomics/adam/issues/897
- case List(GenotypeAllele.OtherAlt) => GenotypeType.NO_CALL
+ case List(GenotypeAllele.OTHER_ALT) => GenotypeType.NO_CALL
// only the four above alleles are possible
// https://github.com/bigdatagenomics/bdg-formats/blob/master/src/main/resources/avro/bdg.avdl#L464
- case _ => throw new IllegalStateException("Found single distinct allele other than the four possible alleles: Ref, Alt, NoCall and OtherAlt")
+ case _ => throw new IllegalStateException("Found single distinct allele other than the four possible alleles: REF, ALT, NO_CALL and OTHER_ALT")
}
} // In the case that there are multiple distinct alleles
// This should be applicable to any genome ploidy.
@@ -69,9 +69,9 @@ class RichGenotype(val genotype: Genotype) {
// IN HTS-JDK this would be GenotypeType.MIXED , this type is not available in BDG / ADAM
// https://github.com/bigdatagenomics/bdg-formats/blob/master/src/main/resources/avro/bdg.avdl#L483
// https://github.com/samtools/htsjdk/blob/master/src/java/htsjdk/variant/variantcontext/Genotype.java#L218
- if (distinctListOfAlleles contains GenotypeAllele.NoCall) {
+ if (distinctListOfAlleles contains GenotypeAllele.NO_CALL) {
GenotypeType.NO_CALL
- } // Otherwise the distinct alleles are a combination of 2 or 3 alleles from the list (GenotypeAllele.Ref, GenotypeAllele.Alt, GenotypeAllele.OtherAlt)
+ } // Otherwise the distinct alleles are a combination of 2 or 3 alleles from the list (GenotypeAllele.REF, GenotypeAllele.ALT, GenotypeAllele.OTHER_ALT)
// Therefore the genotype is Heterozygous HET
else {
GenotypeType.HET
diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/serialization/ADAMKryoRegistrator.scala b/adam-core/src/main/scala/org/bdgenomics/adam/serialization/ADAMKryoRegistrator.scala
index bbb848a06e..70581a8182 100644
--- a/adam-core/src/main/scala/org/bdgenomics/adam/serialization/ADAMKryoRegistrator.scala
+++ b/adam-core/src/main/scala/org/bdgenomics/adam/serialization/ADAMKryoRegistrator.scala
@@ -183,8 +183,6 @@ class ADAMKryoRegistrator extends KryoRegistrator {
new AvroSerializer[org.bdgenomics.formats.avro.AlignmentRecord])
kryo.register(classOf[org.bdgenomics.formats.avro.Contig],
new AvroSerializer[org.bdgenomics.formats.avro.Contig])
- kryo.register(classOf[org.bdgenomics.formats.avro.DatabaseVariantAnnotation],
- new AvroSerializer[org.bdgenomics.formats.avro.DatabaseVariantAnnotation])
kryo.register(classOf[org.bdgenomics.formats.avro.Dbxref],
new AvroSerializer[org.bdgenomics.formats.avro.Dbxref])
kryo.register(classOf[org.bdgenomics.formats.avro.Feature],
@@ -204,8 +202,6 @@ class ADAMKryoRegistrator extends KryoRegistrator {
kryo.register(classOf[org.bdgenomics.formats.avro.Sample],
new AvroSerializer[org.bdgenomics.formats.avro.Sample])
kryo.register(classOf[org.bdgenomics.formats.avro.Strand])
- kryo.register(classOf[org.bdgenomics.formats.avro.StructuralVariant],
- new AvroSerializer[org.bdgenomics.formats.avro.StructuralVariant])
kryo.register(classOf[org.bdgenomics.formats.avro.TranscriptEffect],
new AvroSerializer[org.bdgenomics.formats.avro.TranscriptEffect])
kryo.register(classOf[org.bdgenomics.formats.avro.Variant],
@@ -225,16 +221,21 @@ class ADAMKryoRegistrator extends KryoRegistrator {
kryo.register(classOf[scala.Array[htsjdk.variant.vcf.VCFHeader]])
kryo.register(classOf[scala.Array[java.lang.Object]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.AlignmentRecord]])
- kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.DatabaseVariantAnnotation]])
+ kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Contig]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Dbxref]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Feature]])
+ kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Fragment]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Genotype]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.GenotypeAllele]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.OntologyTerm]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.NucleotideContigFragment]])
+ kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.RecordGroupMetadata]])
+ kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Sample]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.TranscriptEffect]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.Variant]])
+ kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.VariantAnnotation]])
kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.VariantAnnotationMessage]])
+ kryo.register(classOf[scala.Array[org.bdgenomics.formats.avro.VariantCallingAnnotations]])
kryo.register(classOf[scala.Array[org.bdgenomics.adam.models.Coverage]])
kryo.register(classOf[scala.Array[org.bdgenomics.adam.models.ReferencePosition]])
kryo.register(classOf[scala.Array[org.bdgenomics.adam.models.ReferenceRegion]])
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/converters/TranscriptEffectConverterSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/converters/TranscriptEffectConverterSuite.scala
new file mode 100644
index 0000000000..3a28c41d67
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/converters/TranscriptEffectConverterSuite.scala
@@ -0,0 +1,314 @@
+/**
+ * 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.converters
+
+import com.google.common.collect.ImmutableList
+import htsjdk.samtools.ValidationStringency
+import htsjdk.variant.variantcontext.VariantContext
+import htsjdk.variant.vcf.VCFConstants
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.{
+ TranscriptEffect,
+ Variant
+}
+import org.mockito.Mockito
+import org.mockito.Mockito.when
+import scala.collection.JavaConverters._
+
+class TranscriptEffectConverterSuite extends ADAMFunSuite {
+ final val EMPTY = ""
+ final val INVALID = "T|upstream_gene_variant||TAS1R3|ENSG00000169962|transcript|ENST00000339381.5|protein_coding|1/2|c.-485C>T|||4|1/42|453"
+ final val INVALID_NUMBER = "T|upstream_gene_variant||TAS1R3|ENSG00000169962|transcript|ENST00000339381.5|protein_coding|1/2|c.-485C>T|||4|1/42|not a number|"
+ final val INVALID_FRACTION = "T|upstream_gene_variant||TAS1R3|ENSG00000169962|transcript|ENST00000339381.5|protein_coding|not a number/2|c.-485C>T|||4|1/42|453|"
+ final val VALID = "T|upstream_gene_variant||TAS1R3|ENSG00000169962|transcript|ENST00000339381.5|protein_coding|1/2|c.-485C>T|||4|1/42|453|"
+
+ var variant: Variant = null
+ var variantContext: VariantContext = null
+
+ before {
+ variant = Variant.newBuilder().build()
+ variantContext = Mockito.mock(classOf[VariantContext])
+ }
+
+ test("parse empty transcript effect") {
+ TranscriptEffectConverter.parseTranscriptEffect(EMPTY, ValidationStringency.SILENT).isEmpty
+ }
+
+ test("parse empty transcript effect strict validation stringency") {
+ intercept[IllegalArgumentException] {
+ TranscriptEffectConverter.parseTranscriptEffect(EMPTY, ValidationStringency.STRICT)
+ }
+ }
+
+ test("parse invalid transcript effect") {
+ TranscriptEffectConverter.parseTranscriptEffect(INVALID, ValidationStringency.SILENT).isEmpty
+ }
+
+ test("parse invalid transcript effect strict validation stringency") {
+ intercept[IllegalArgumentException] {
+ TranscriptEffectConverter.parseTranscriptEffect(INVALID, ValidationStringency.STRICT)
+ }
+ }
+
+ test("parse transcript effect") {
+ val te = TranscriptEffectConverter.parseTranscriptEffect(VALID, ValidationStringency.STRICT).head
+
+ assert(te.getAlternateAllele == "T")
+ assert(te.getEffects.contains("upstream_gene_variant"))
+ assert(te.getGeneName == "TAS1R3")
+ assert(te.getGeneId == "ENSG00000169962")
+ assert(te.getFeatureType == "transcript")
+ assert(te.getFeatureId == "ENST00000339381.5")
+ assert(te.getBiotype == "protein_coding")
+ assert(te.getRank == 1)
+ assert(te.getTotal == 2)
+ assert(te.getTranscriptHgvs == "c.-485C>T")
+ assert(te.getProteinHgvs == null)
+ assert(te.getCdnaPosition == null)
+ assert(te.getCdnaLength == null)
+ assert(te.getCdsPosition == 4)
+ assert(te.getCdsLength == null)
+ assert(te.getProteinPosition == 1)
+ assert(te.getProteinLength == 42)
+ assert(te.getDistance == 453)
+ assert(te.getMessages.isEmpty)
+ }
+
+ test("parse empty VCF ANN attribute") {
+ TranscriptEffectConverter.parseAnn(EMPTY, ValidationStringency.SILENT).isEmpty
+ }
+
+ test("parse empty VCF ANN attribute strict validation stringency") {
+ intercept[IllegalArgumentException] {
+ TranscriptEffectConverter.parseAnn(EMPTY, ValidationStringency.STRICT)
+ }
+ }
+
+ test("parse invalid VCF ANN attribute") {
+ TranscriptEffectConverter.parseAnn(INVALID, ValidationStringency.SILENT).isEmpty
+ }
+
+ test("parse invalid VCF ANN attribute strict validation stringency") {
+ intercept[IllegalArgumentException] {
+ TranscriptEffectConverter.parseAnn(INVALID, ValidationStringency.STRICT)
+ }
+ }
+
+ test("parse VCF ANN attribute with one transcript effect") {
+ val ann = TranscriptEffectConverter.parseAnn(VALID, ValidationStringency.STRICT)
+ assert(ann.length == 1)
+
+ val te = ann.head
+ assert(te.getAlternateAllele == "T")
+ assert(te.getEffects.contains("upstream_gene_variant"))
+ assert(te.getGeneName == "TAS1R3")
+ assert(te.getGeneId == "ENSG00000169962")
+ assert(te.getFeatureType == "transcript")
+ assert(te.getFeatureId == "ENST00000339381.5")
+ assert(te.getBiotype == "protein_coding")
+ assert(te.getRank == 1)
+ assert(te.getTotal == 2)
+ assert(te.getTranscriptHgvs == "c.-485C>T")
+ assert(te.getProteinHgvs == null)
+ assert(te.getCdnaPosition == null)
+ assert(te.getCdnaLength == null)
+ assert(te.getCdsPosition == 4)
+ assert(te.getCdsLength == null)
+ assert(te.getProteinPosition == 1)
+ assert(te.getProteinLength == 42)
+ assert(te.getDistance == 453)
+ assert(te.getMessages.isEmpty)
+ }
+
+ test("convert to transcript effect from null VCF ANN attribute in variant context") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(null)
+
+ val transcriptEffects = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext)
+ assert(!transcriptEffects.isDefined)
+ }
+
+ test("convert to transcript effect from missing value VCF ANN attribute in variant context") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(VCFConstants.MISSING_VALUE_v4)
+
+ val transcriptEffects = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext)
+ assert(!transcriptEffects.isDefined)
+ }
+
+ test("convert to transcript effect from empty VCF ANN attribute in variant context") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(EMPTY)
+
+ val transcriptEffects = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.SILENT)
+ assert(!transcriptEffects.isDefined)
+ }
+
+ test("convert to transcript effect from empty VCF ANN attribute in variant context strict validation stringency") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(EMPTY)
+
+ intercept[IllegalArgumentException] {
+ TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.STRICT)
+ }
+ }
+
+ test("convert to transcript effect from invalid VCF ANN attribute in variant context") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID)
+
+ val transcriptEffects = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.SILENT)
+ assert(!transcriptEffects.isDefined)
+ }
+
+ test("convert to transcript effect from invalid VCF ANN attribute in variant context strict validation stringency") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID)
+
+ intercept[IllegalArgumentException] {
+ TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.STRICT)
+ }
+ }
+
+ test("convert to transcript effect from VCF ANN attribute with invalid number in variant context lenient validation stringency") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID_NUMBER)
+
+ val transcriptEffects = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.LENIENT)
+ assert(!transcriptEffects.isDefined)
+ }
+
+ test("convert to transcript effect from VCF ANN attribute with invalid fraction in variant context lenient validation stringency") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID_FRACTION)
+
+ val transcriptEffects = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.LENIENT)
+ assert(!transcriptEffects.isDefined)
+ }
+
+ test("convert to transcript effect from VCF ANN attribute with invalid number in variant context strict validation stringency") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID_NUMBER)
+
+ intercept[NumberFormatException] {
+ TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.STRICT)
+ }
+ }
+
+ test("convert to transcript effect from VCF ANN attribute with invalid fraction in variant context strict validation stringency") {
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID_FRACTION)
+
+ intercept[NumberFormatException] {
+ TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext, ValidationStringency.STRICT)
+ }
+ }
+
+ test("convert to transcript effect from VCF ANN attribute in variant context different alt allele") {
+ variant = Variant.newBuilder()
+ .setAlternateAllele("A")
+ .build()
+
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(VALID)
+
+ val transcriptEffects = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext)
+ assert(!transcriptEffects.isDefined)
+ }
+
+ test("convert to transcript effect from VCF ANN attribute in variant context same alt allele") {
+ variant = Variant.newBuilder()
+ .setAlternateAllele("T")
+ .build()
+
+ when(variantContext.getAttributeAsString("ANN", null)).thenReturn(VALID)
+
+ val transcriptEffectsOpt = TranscriptEffectConverter.convertToTranscriptEffects(variant, variantContext)
+ assert(transcriptEffectsOpt.isDefined)
+
+ transcriptEffectsOpt.foreach(transcriptEffects => {
+ val te = transcriptEffects(0)
+ assert(te.getAlternateAllele == "T")
+ assert(te.getEffects.contains("upstream_gene_variant"))
+ assert(te.getGeneName == "TAS1R3")
+ assert(te.getGeneId == "ENSG00000169962")
+ assert(te.getFeatureType == "transcript")
+ assert(te.getFeatureId == "ENST00000339381.5")
+ assert(te.getBiotype == "protein_coding")
+ assert(te.getRank == 1)
+ assert(te.getTotal == 2)
+ assert(te.getTranscriptHgvs == "c.-485C>T")
+ assert(te.getProteinHgvs == null)
+ assert(te.getCdnaPosition == null)
+ assert(te.getCdnaLength == null)
+ assert(te.getCdsPosition == 4)
+ assert(te.getCdsLength == null)
+ assert(te.getProteinPosition == 1)
+ assert(te.getProteinLength == 42)
+ assert(te.getDistance == 453)
+ assert(te.getMessages.isEmpty)
+ })
+ }
+
+ test("convert transcript effect to VCF ANN attribute value") {
+ val te = TranscriptEffect.newBuilder()
+ .setAlternateAllele("T")
+ .setEffects(ImmutableList.of("upstream_gene_variant"))
+ .setGeneName("TAS1R3")
+ .setGeneId("ENSG00000169962")
+ .setFeatureType("transcript")
+ .setFeatureId("ENST00000339381.5")
+ .setBiotype("protein_coding")
+ .setTranscriptHgvs("c.-485C>T")
+ .setRank(1)
+ .setTotal(2)
+ .setCdsPosition(4)
+ .setProteinPosition(1)
+ .setProteinLength(42)
+ .setDistance(453)
+ .build()
+
+ assert(VALID === TranscriptEffectConverter.convertToVcfInfoAnnValue(Seq(te)))
+ }
+
+ test("convert transcript effect with null fields to VCF ANN attribute value") {
+ val te = TranscriptEffect.newBuilder()
+ .setAlternateAllele("T")
+ .setEffects(ImmutableList.of("upstream_gene_variant"))
+ .setGeneName("TAS1R3")
+ .setGeneId("ENSG00000169962")
+ .setFeatureType("transcript")
+ .setFeatureId("ENST00000339381.5")
+ .setBiotype("protein_coding")
+ .setTranscriptHgvs("c.-485C>T")
+ .setRank(1)
+ .setTotal(2)
+ .setCdnaPosition(null)
+ .setCdnaLength(null)
+ .setCdsPosition(4)
+ .setCdsLength(null)
+ .setProteinPosition(1)
+ .setProteinLength(42)
+ .setDistance(453)
+ .build()
+
+ assert(VALID === TranscriptEffectConverter.convertToVcfInfoAnnValue(Seq(te)))
+ }
+
+ test("convert transcript effect with incorrect fractional value to VCF ANN attribute value") {
+ val te = TranscriptEffect.newBuilder()
+ .setAlternateAllele("T")
+ .setEffects(ImmutableList.of("upstream_gene_variant"))
+ .setRank(null)
+ .setTotal(2)
+ .build()
+
+ // should log warning "Incorrect fractional value ?/2, missing numerator" and set to empty string
+ // when ValidationStringency is made available for --> VCF, test STRICT throws exception
+ assert(!TranscriptEffectConverter.convertToVcfInfoAnnValue(Seq(te)).contains("2"))
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/converters/VariantAnnotationsSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/converters/VariantAnnotationsSuite.scala
deleted file mode 100644
index 3061e953ac..0000000000
--- a/adam-core/src/test/scala/org/bdgenomics/adam/converters/VariantAnnotationsSuite.scala
+++ /dev/null
@@ -1,225 +0,0 @@
-/**
- * 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.converters
-
-import htsjdk.samtools.ValidationStringency
-import htsjdk.variant.vcf.VCFConstants
-import htsjdk.variant.variantcontext.VariantContext
-import org.bdgenomics.adam.util.ADAMFunSuite
-import org.bdgenomics.formats.avro.Variant
-import org.mockito.Mockito
-import org.mockito.Mockito.when
-
-class VariantAnnotationsSuite extends ADAMFunSuite {
- final val EMPTY = ""
- final val INVALID = "T|upstream_gene_variant|MODIFIER|TAS1R3|ENSG00000169962|transcript|ENST00000339381.5|protein_coding||c.-485C>T|||||453"
- final val VALID = "T|upstream_gene_variant|MODIFIER|TAS1R3|ENSG00000169962|transcript|ENST00000339381.5|protein_coding||c.-485C>T|||||453|"
-
- var variant: Variant = null
- var variantContext: VariantContext = null
-
- before {
- variant = Variant.newBuilder().build()
- variantContext = Mockito.mock(classOf[VariantContext])
- }
-
- test("parse empty transcript effect") {
- VariantAnnotations.parseTranscriptEffect(EMPTY, ValidationStringency.SILENT).isEmpty
- }
-
- test("parse empty transcript effect strict validation stringency") {
- intercept[IllegalArgumentException] {
- VariantAnnotations.parseTranscriptEffect(EMPTY, ValidationStringency.STRICT)
- }
- }
-
- test("parse invalid transcript effect") {
- VariantAnnotations.parseTranscriptEffect(INVALID, ValidationStringency.SILENT).isEmpty
- }
-
- test("parse invalid transcript effect strict validation stringency") {
- intercept[IllegalArgumentException] {
- VariantAnnotations.parseTranscriptEffect(INVALID, ValidationStringency.STRICT)
- }
- }
-
- test("parse transcript effect") {
- val te = VariantAnnotations.parseTranscriptEffect(VALID, ValidationStringency.STRICT).head
-
- assert(te.getAlternateAllele == "T")
- assert(te.getEffects.contains("upstream_gene_variant"))
- assert(te.getGeneName == "TAS1R3")
- assert(te.getGeneId == "ENSG00000169962")
- assert(te.getFeatureType == "transcript")
- assert(te.getFeatureId == "ENST00000339381.5")
- assert(te.getBiotype == "protein_coding")
- assert(te.getRank == null)
- assert(te.getTotal == null)
- assert(te.getTranscriptHgvs == "c.-485C>T")
- assert(te.getProteinHgvs == null)
- assert(te.getCdnaPosition == null)
- assert(te.getCdnaLength == null)
- assert(te.getCdsPosition == null)
- assert(te.getCdsLength == null)
- assert(te.getProteinPosition == null)
- assert(te.getProteinLength == null)
- assert(te.getDistance == 453)
- assert(te.getMessages.isEmpty)
- }
-
- test("parse empty VCF ANN attribute") {
- VariantAnnotations.parseAnn(EMPTY, ValidationStringency.SILENT).isEmpty
- }
-
- test("parse empty VCF ANN attribute strict validation stringency") {
- intercept[IllegalArgumentException] {
- VariantAnnotations.parseAnn(EMPTY, ValidationStringency.STRICT)
- }
- }
-
- test("parse invalid VCF ANN attribute") {
- VariantAnnotations.parseAnn(INVALID, ValidationStringency.SILENT).isEmpty
- }
-
- test("parse invalid VCF ANN attribute strict validation stringency") {
- intercept[IllegalArgumentException] {
- VariantAnnotations.parseAnn(INVALID, ValidationStringency.STRICT)
- }
- }
-
- test("parse VCF ANN attribute with one transcript effect") {
- val ann = VariantAnnotations.parseAnn(VALID, ValidationStringency.STRICT)
- assert(ann.length == 1)
-
- val te = ann.head
- assert(te.getAlternateAllele == "T")
- assert(te.getEffects.contains("upstream_gene_variant"))
- assert(te.getGeneName == "TAS1R3")
- assert(te.getGeneId == "ENSG00000169962")
- assert(te.getFeatureType == "transcript")
- assert(te.getFeatureId == "ENST00000339381.5")
- assert(te.getBiotype == "protein_coding")
- assert(te.getRank == null)
- assert(te.getTotal == null)
- assert(te.getTranscriptHgvs == "c.-485C>T")
- assert(te.getProteinHgvs == null)
- assert(te.getCdnaPosition == null)
- assert(te.getCdnaLength == null)
- assert(te.getCdsPosition == null)
- assert(te.getCdsLength == null)
- assert(te.getProteinPosition == null)
- assert(te.getProteinLength == null)
- assert(te.getDistance == 453)
- assert(te.getMessages.isEmpty)
- }
-
- test("create variant annotation from null VCF ANN attribute in variant context") {
- when(variantContext.getAttributeAsString("ANN", null)).thenReturn(null)
-
- val ann = VariantAnnotations.createVariantAnnotation(variant, variantContext)
- assert(ann.getVariant() == variant)
- assert(ann.getTranscriptEffects().isEmpty)
- }
-
- test("create variant annotation from missing value VCF ANN attribute in variant context") {
- when(variantContext.getAttributeAsString("ANN", null)).thenReturn(VCFConstants.MISSING_VALUE_v4)
-
- val ann = VariantAnnotations.createVariantAnnotation(variant, variantContext)
- assert(ann.getVariant() == variant)
- assert(ann.getTranscriptEffects().isEmpty)
- }
-
- test("create variant annotation from empty VCF ANN attribute in variant context") {
- when(variantContext.getAttributeAsString("ANN", null)).thenReturn(EMPTY)
-
- val ann = VariantAnnotations.createVariantAnnotation(variant, variantContext, ValidationStringency.SILENT)
- assert(ann.getVariant() == variant)
- assert(ann.getTranscriptEffects().isEmpty)
- }
-
- test("create variant annotation from empty VCF ANN attribute in variant context strict validation stringency") {
- when(variantContext.getAttributeAsString("ANN", null)).thenReturn(EMPTY)
-
- intercept[IllegalArgumentException] {
- VariantAnnotations.createVariantAnnotation(variant, variantContext, ValidationStringency.STRICT)
- }
- }
-
- test("create variant annotation from invalid VCF ANN attribute in variant context") {
- when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID)
-
- val ann = VariantAnnotations.createVariantAnnotation(variant, variantContext, ValidationStringency.SILENT)
- assert(ann.getVariant() == variant)
- assert(ann.getTranscriptEffects().isEmpty)
- }
-
- test("create variant annotation from invalid VCF ANN attribute in variant context strict validation stringency") {
- when(variantContext.getAttributeAsString("ANN", null)).thenReturn(INVALID)
-
- intercept[IllegalArgumentException] {
- VariantAnnotations.createVariantAnnotation(variant, variantContext, ValidationStringency.STRICT)
- }
- }
-
- test("create variant annotation from VCF ANN attribute in variant context") {
- when(variantContext.getAttributeAsString("ANN", null)).thenReturn(VALID)
-
- val ann = VariantAnnotations.createVariantAnnotation(variant, variantContext)
- assert(ann.getVariant() == variant)
- assert(ann.getTranscriptEffects().size == 1)
-
- val te = ann.getTranscriptEffects().get(0)
- assert(te.getAlternateAllele == "T")
- assert(te.getEffects.contains("upstream_gene_variant"))
- assert(te.getGeneName == "TAS1R3")
- assert(te.getGeneId == "ENSG00000169962")
- assert(te.getFeatureType == "transcript")
- assert(te.getFeatureId == "ENST00000339381.5")
- assert(te.getBiotype == "protein_coding")
- assert(te.getRank == null)
- assert(te.getTotal == null)
- assert(te.getTranscriptHgvs == "c.-485C>T")
- assert(te.getProteinHgvs == null)
- assert(te.getCdnaPosition == null)
- assert(te.getCdnaLength == null)
- assert(te.getCdsPosition == null)
- assert(te.getCdsLength == null)
- assert(te.getProteinPosition == null)
- assert(te.getProteinLength == null)
- assert(te.getDistance == 453)
- assert(te.getMessages.isEmpty)
- }
-
- test("create java.util.List[Int] from SB tag String value") {
- val sb_tagData = "2,3,4,5"
- val sb_converter = VariantAnnotationConverter.FORMAT_KEYS
- .filter(x => x.adamKey == "strandBiasComponents").head.attrConverter
-
- val sb_parsed = sb_converter(sb_tagData).asInstanceOf[java.util.List[Int]]
- val sb_component1 = sb_parsed.get(0)
- val sb_component2 = sb_parsed.get(1)
- val sb_component3 = sb_parsed.get(2)
- val sb_component4 = sb_parsed.get(3)
-
- assert(sb_component1 == 2 &&
- sb_component2 == 3 &&
- sb_component3 == 4 &&
- sb_component4 == 5)
-
- }
-}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/converters/VariantContextConverterSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/converters/VariantContextConverterSuite.scala
index cf68a88b75..9621d5a8eb 100644
--- a/adam-core/src/test/scala/org/bdgenomics/adam/converters/VariantContextConverterSuite.scala
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/converters/VariantContextConverterSuite.scala
@@ -17,6 +17,7 @@
*/
package org.bdgenomics.adam.converters
+import com.google.common.collect.ImmutableList
import htsjdk.samtools.SAMFileReader
import htsjdk.variant.utils.SAMSequenceDictionaryExtractor
import htsjdk.variant.variantcontext.{
@@ -41,25 +42,25 @@ class VariantContextConverterSuite extends ADAMFunSuite {
SequenceDictionary(SAMSequenceDictionaryExtractor.extractDictionary(new File(path)))
}
- def gatkSNVBuilder: VariantContextBuilder = new VariantContextBuilder()
+ def htsjdkSNVBuilder: VariantContextBuilder = new VariantContextBuilder()
.alleles(List(Allele.create("A", true), Allele.create("T")))
.start(1L)
.stop(1L)
.chr("1")
- def gatkMultiAllelicSNVBuilder: VariantContextBuilder = new VariantContextBuilder()
+ def htsjdkMultiAllelicSNVBuilder: VariantContextBuilder = new VariantContextBuilder()
.alleles(List(Allele.create("A", true), Allele.create("T"), Allele.create("G")))
.start(1L)
.stop(1L)
.chr("1")
- def gatkRefSNV: VariantContextBuilder = new VariantContextBuilder()
+ def htsjdkRefSNV: VariantContextBuilder = new VariantContextBuilder()
.alleles(List(Allele.create("A", true), Allele.create("", false)))
.start(1L)
.stop(1L)
.chr("1")
- def gatkCNVBuilder: VariantContextBuilder = new VariantContextBuilder()
+ def htsjdkCNVBuilder: VariantContextBuilder = new VariantContextBuilder()
.alleles(List(Allele.create("A", true), Allele.create("", false)))
.start(10L)
.stop(20L)
@@ -71,10 +72,10 @@ class VariantContextConverterSuite extends ADAMFunSuite {
.setReferenceAllele("A")
.setAlternateAllele("T")
- test("Convert GATK site-only SNV to ADAM") {
+ test("Convert htsjdk site-only SNV to ADAM") {
val converter = new VariantContextConverter
- val adamVCs = converter.convert(gatkSNVBuilder.make)
+ val adamVCs = converter.convert(htsjdkSNVBuilder.make)
assert(adamVCs.length === 1)
val adamVC = adamVCs.head
@@ -87,10 +88,10 @@ class VariantContextConverterSuite extends ADAMFunSuite {
assert(variant.getStart === 0L)
}
- test("Convert GATK site-only SNV to ADAM with contig conversion") {
+ test("Convert htsjdk site-only SNV to ADAM with contig conversion") {
val converter = new VariantContextConverter(Some(dictionary))
- val adamVCs = converter.convert(gatkSNVBuilder.make)
+ val adamVCs = converter.convert(htsjdkSNVBuilder.make)
assert(adamVCs.length === 1)
val adamVC = adamVCs.head
@@ -98,10 +99,10 @@ class VariantContextConverterSuite extends ADAMFunSuite {
assert(variant.getContigName === "NC_000001.10")
}
- test("Convert GATK site-only CNV to ADAM") {
+ test("Convert htsjdk site-only CNV to ADAM") {
val converter = new VariantContextConverter
- val adamVCs = converter.convert(gatkCNVBuilder.make)
+ val adamVCs = converter.convert(htsjdkCNVBuilder.make)
assert(adamVCs.length === 1)
val adamVC = adamVCs.head
@@ -116,8 +117,8 @@ class VariantContextConverterSuite extends ADAMFunSuite {
assert(variant.getEnd === 20L)
}
- test("Convert GATK SNV w/ genotypes w/ phase information to ADAM") {
- val vcb = gatkSNVBuilder
+ test("Convert htsjdk SNV w/ genotypes w/ phase information to ADAM") {
+ val vcb = htsjdkSNVBuilder
val genotypeAttributes = Map[String, Object]("PQ" -> new Integer(50), "PS" -> new Integer(1))
val vc = vcb.genotypes(GenotypeBuilder.create("NA12878", vcb.getAlleles(), genotypeAttributes)).make()
@@ -130,69 +131,110 @@ class VariantContextConverterSuite extends ADAMFunSuite {
val adamGTs = adamVCs.flatMap(_.genotypes)
assert(adamGTs.length === 1)
val adamGT = adamGTs.head
- assert(adamGT.getAlleles.sameElements(List(GenotypeAllele.Ref, GenotypeAllele.Alt)))
+ assert(adamGT.getAlleles.sameElements(List(GenotypeAllele.REF, GenotypeAllele.ALT)))
assert(adamGT.getPhaseSetId === 1)
assert(adamGT.getPhaseQuality === 50)
}
- test("Convert GATK SNV with different filters to ADAM") {
- val vcb = gatkSNVBuilder
+ test("Convert htsjdk SNV with different variant filters to ADAM") {
+ val vcb = htsjdkSNVBuilder
vcb.genotypes(GenotypeBuilder.create("NA12878", vcb.getAlleles))
val converter = new VariantContextConverter
{ // No filters
val adamVCs = converter.convert(vcb.make)
- val adamGT = adamVCs.flatMap(_.genotypes).head
- assert(adamGT.getVariantCallingAnnotations.getVariantIsPassing === null)
+ val adamVariant = adamVCs.map(_.variant).head
+ assert(adamVariant.getFiltersApplied === false)
+ assert(adamVariant.getFiltersPassed === null)
+ assert(adamVariant.getFiltersFailed.isEmpty)
}
{ // PASSing
vcb.unfiltered.passFilters
val adamVCs = converter.convert(vcb.make)
- val adamGT = adamVCs.flatMap(_.genotypes).head
- assert(adamGT.getVariantCallingAnnotations.getVariantIsPassing)
+ val adamVariant = adamVCs.map(_.variant).head
+ assert(adamVariant.getFiltersApplied === true)
+ assert(adamVariant.getFiltersPassed === true)
+ assert(adamVariant.getFiltersFailed.isEmpty)
}
{ // not PASSing
vcb.unfiltered.filter("LowMQ")
val adamVCs = converter.convert(vcb.make)
+ val adamVariant = adamVCs.map(_.variant).head
+ assert(adamVariant.getFiltersApplied === true)
+ assert(adamVariant.getFiltersPassed === false)
+ assert(adamVariant.getFiltersFailed.sameElements(List("LowMQ")))
+ }
+ }
+
+ test("Convert htsjdk SNV with different genotype filters to ADAM") {
+ val vcb = htsjdkSNVBuilder
+ val gb = new GenotypeBuilder("NA12878", vcb.getAlleles)
+
+ val converter = new VariantContextConverter
+
+ { // No filters
+ gb.unfiltered
+ vcb.genotypes(gb.make)
+ val adamVCs = converter.convert(vcb.make)
+ val adamGT = adamVCs.flatMap(_.genotypes).head
+ // htsjdk does not distinguish between filters not applied and filters passed in Genotype
+ assert(adamGT.getVariantCallingAnnotations.getFiltersApplied === true)
+ assert(adamGT.getVariantCallingAnnotations.getFiltersPassed === true)
+ assert(adamGT.getVariantCallingAnnotations.getFiltersFailed.isEmpty)
+ }
+ { // PASSing
+ gb.filter("PASS")
+ vcb.genotypes(gb.make)
+ val adamVCs = converter.convert(vcb.make)
val adamGT = adamVCs.flatMap(_.genotypes).head
- assert(adamGT.getVariantCallingAnnotations.getVariantIsPassing === false)
- assert(adamGT.getVariantCallingAnnotations.getVariantFilters.sameElements(List("LowMQ")))
+ assert(adamGT.getVariantCallingAnnotations.getFiltersApplied === true)
+ assert(adamGT.getVariantCallingAnnotations.getFiltersPassed === true)
+ assert(adamGT.getVariantCallingAnnotations.getFiltersFailed.isEmpty)
+ }
+ { // not PASSing
+ gb.filter("LowMQ")
+ vcb.genotypes(gb.make)
+ val adamVCs = converter.convert(vcb.make)
+ val adamGT = adamVCs.flatMap(_.genotypes).head
+ assert(adamGT.getVariantCallingAnnotations.getFiltersApplied === true)
+ assert(adamGT.getVariantCallingAnnotations.getFiltersPassed === false)
+ assert(adamGT.getVariantCallingAnnotations.getFiltersFailed.sameElements(List("LowMQ")))
}
}
- test("Convert ADAM site-only SNV to GATK") {
+ test("Convert ADAM site-only SNV to htsjdk") {
val vc = ADAMVariantContext(adamSNVBuilder().build)
val converter = new VariantContextConverter
- val gatkVC = converter.convert(vc)
- assert(gatkVC.getContig === "1")
- assert(gatkVC.getStart === 1)
- assert(gatkVC.getEnd === 1)
- assert(gatkVC.getReference === Allele.create("A", true))
- assert(gatkVC.getAlternateAlleles.sameElements(List(Allele.create("T"))))
- assert(!gatkVC.hasLog10PError)
- assert(!gatkVC.hasID)
- assert(!gatkVC.filtersWereApplied)
+ val htsjdkVC = converter.convert(vc)
+ assert(htsjdkVC.getContig === "1")
+ assert(htsjdkVC.getStart === 1)
+ assert(htsjdkVC.getEnd === 1)
+ assert(htsjdkVC.getReference === Allele.create("A", true))
+ assert(htsjdkVC.getAlternateAlleles.sameElements(List(Allele.create("T"))))
+ assert(!htsjdkVC.hasLog10PError)
+ assert(!htsjdkVC.hasID)
+ assert(!htsjdkVC.filtersWereApplied)
}
- test("Convert ADAM site-only SNV to GATK with contig conversion") {
+ test("Convert ADAM site-only SNV to htsjdk with contig conversion") {
val vc = ADAMVariantContext(adamSNVBuilder("NC_000001.10").build)
val converter = new VariantContextConverter(dict = Some(dictionary))
- val gatkVC = converter.convert(vc)
- assert(gatkVC.getContig === "1")
+ val htsjdkVC = converter.convert(vc)
+ assert(htsjdkVC.getContig === "1")
}
- test("Convert ADAM SNV w/ genotypes to GATK") {
+ test("Convert ADAM SNV w/ genotypes to htsjdk") {
val variant = adamSNVBuilder().build
val genotype = Genotype.newBuilder
.setVariant(variant)
.setSampleId("NA12878")
- .setAlleles(List(GenotypeAllele.Ref, GenotypeAllele.Alt))
.setStrandBiasComponents(List(0, 2, 4, 6).map(i => i: java.lang.Integer))
+ .setAlleles(List(GenotypeAllele.REF, GenotypeAllele.ALT))
.setVariantCallingAnnotations(VariantCallingAnnotations.newBuilder()
.setFisherStrandBiasPValue(3.0f)
.setRmsMapQ(0.0f)
@@ -202,16 +244,16 @@ class VariantContextConverterSuite extends ADAMFunSuite {
val converter = new VariantContextConverter
- val gatkVC = converter.convert(ADAMVariantContext(variant, Seq(genotype)))
- assert(gatkVC.getNSamples === 1)
- assert(gatkVC.hasGenotype("NA12878"))
- val gatkGT = gatkVC.getGenotype("NA12878")
- assert(gatkGT.getType === GenotypeType.HET)
- assert(gatkVC.hasAttribute("FS"))
- assert(gatkVC.hasAttribute("MQ"))
- assert(gatkVC.hasAttribute("MQ0"))
- assert(gatkGT.hasAnyAttribute("SB"))
- val sbComponents = gatkGT.getAnyAttribute("SB")
+ val htsjdkVC = converter.convert(ADAMVariantContext(variant, Seq(genotype)))
+ assert(htsjdkVC.getNSamples === 1)
+ assert(htsjdkVC.hasGenotype("NA12878"))
+ val htsjdkGT = htsjdkVC.getGenotype("NA12878")
+ assert(htsjdkGT.getType === GenotypeType.HET)
+ assert(htsjdkVC.hasAttribute("FS"))
+ assert(htsjdkVC.hasAttribute("MQ"))
+ assert(htsjdkVC.hasAttribute("MQ0"))
+ assert(htsjdkGT.hasAnyAttribute("SB"))
+ val sbComponents = htsjdkGT.getAnyAttribute("SB")
.asInstanceOf[java.util.List[java.lang.Integer]]
assert(sbComponents.get(0) === 0)
assert(sbComponents.get(1) === 2)
@@ -219,8 +261,8 @@ class VariantContextConverterSuite extends ADAMFunSuite {
assert(sbComponents.get(3) === 6)
}
- test("Convert GATK multi-allelic sites-only SNVs to ADAM") {
- val vc = gatkMultiAllelicSNVBuilder.make
+ test("Convert htsjdk multi-allelic sites-only SNVs to ADAM") {
+ val vc = htsjdkMultiAllelicSNVBuilder.make
val converter = new VariantContextConverter
val adamVCs = converter.convert(vc)
@@ -233,11 +275,11 @@ class VariantContextConverterSuite extends ADAMFunSuite {
}
}
- test("Convert GATK multi-allelic SNVs to ADAM") {
+ test("Convert htsjdk multi-allelic SNVs to ADAM") {
val gb = new GenotypeBuilder("NA12878", List(Allele.create("T"), Allele.create("G")))
gb.AD(Array(4, 2, 3)).PL(Array(59, 0, 181, 1, 66, 102))
- val vcb = gatkMultiAllelicSNVBuilder
+ val vcb = htsjdkMultiAllelicSNVBuilder
vcb.genotypes(gb.make)
val converter = new VariantContextConverter
@@ -254,14 +296,14 @@ class VariantContextConverterSuite extends ADAMFunSuite {
val adamGT1 = adamVCs(0).genotypes.head
val adamGT2 = adamVCs(1).genotypes.head
- assert(adamGT1.getAlleles.sameElements(List(GenotypeAllele.Alt, GenotypeAllele.OtherAlt)))
+ assert(adamGT1.getAlleles.sameElements(List(GenotypeAllele.ALT, GenotypeAllele.OTHER_ALT)))
assert(adamGT1.getAlternateReadDepth === 2)
assert(adamGT1.getGenotypeLikelihoods
.map(f => f: scala.Float)
.map(PhredUtils.logProbabilityToPhred)
.sameElements(List(59, 0, 256)))
- assert(adamGT2.getAlleles.sameElements(List(GenotypeAllele.OtherAlt, GenotypeAllele.Alt)))
+ assert(adamGT2.getAlleles.sameElements(List(GenotypeAllele.OTHER_ALT, GenotypeAllele.ALT)))
assert(adamGT2.getAlternateReadDepth === 3)
assert(adamGT2.getGenotypeLikelihoods
.map(f => f: scala.Float)
@@ -273,7 +315,7 @@ class VariantContextConverterSuite extends ADAMFunSuite {
val gb = new GenotypeBuilder("NA12878", List(Allele.create("A", true), Allele.create("A", true)))
gb.PL(Array(0, 1, 2)).DP(44).attribute("MIN_DP", 38)
- val vcb = gatkRefSNV
+ val vcb = htsjdkRefSNV
vcb.genotypes(gb.make)
val converter = new VariantContextConverter
@@ -285,7 +327,7 @@ class VariantContextConverterSuite extends ADAMFunSuite {
assert(adamGTs.length === 1)
val adamGT = adamGTs.head
assert(adamGT.getVariant.getAlternateAllele === null)
- assert(adamGT.getAlleles.sameElements(List(GenotypeAllele.Ref, GenotypeAllele.Ref)))
+ assert(adamGT.getAlleles.sameElements(List(GenotypeAllele.REF, GenotypeAllele.REF)))
assert(adamGT.getMinReadDepth === 38)
assert(adamGT.getGenotypeLikelihoods.isEmpty)
assert(adamGT.getNonReferenceLikelihoods
@@ -293,4 +335,82 @@ class VariantContextConverterSuite extends ADAMFunSuite {
.map(PhredUtils.logProbabilityToPhred)
.sameElements(List(0, 1, 2)))
}
+
+ test("Convert htsjdk variant context with no IDs to ADAM") {
+ val vcb = htsjdkSNVBuilder
+ vcb.noID()
+
+ val converter = new VariantContextConverter
+
+ val adamVCs = converter.convert(vcb.make)
+ assert(adamVCs.length == 1)
+
+ val variant = adamVCs.head.variant
+ assert(variant.getNames.isEmpty)
+ }
+
+ test("Convert htsjdk variant context with one ID to ADAM") {
+ val vcb = htsjdkSNVBuilder
+ vcb.id("rs3131972")
+
+ val converter = new VariantContextConverter
+
+ val adamVCs = converter.convert(vcb.make)
+ assert(adamVCs.length == 1)
+
+ val variant = adamVCs.head.variant
+ assert(variant.getNames.length === 1)
+ assert(variant.getNames.get(0) === "rs3131972")
+ }
+
+ test("Convert htsjdk variant context with multiple IDs to ADAM") {
+ val vcb = htsjdkSNVBuilder
+ vcb.id("rs3131972;rs201888535")
+
+ val converter = new VariantContextConverter
+
+ val adamVCs = converter.convert(vcb.make)
+ assert(adamVCs.length == 1)
+
+ val variant = adamVCs.head.variant
+ assert(variant.getNames.length === 2)
+ assert(variant.getNames.get(0) === "rs3131972")
+ assert(variant.getNames.get(1) === "rs201888535")
+ }
+
+ test("Convert ADAM variant context with no names to htsjdk") {
+ val variant = adamSNVBuilder()
+ .build
+
+ assert(variant.getNames.isEmpty)
+
+ val converter = new VariantContextConverter
+
+ val htsjdkVC = converter.convert(ADAMVariantContext(variant))
+ assert(!htsjdkVC.hasID)
+ }
+
+ test("Convert ADAM variant context with one name to htsjdk") {
+ val variant = adamSNVBuilder()
+ .setNames(ImmutableList.of("rs3131972"))
+ .build
+
+ val converter = new VariantContextConverter
+
+ val htsjdkVC = converter.convert(ADAMVariantContext(variant))
+ assert(htsjdkVC.hasID)
+ assert(htsjdkVC.getID === "rs3131972")
+ }
+
+ test("Convert ADAM variant context with multiple names to htsjdk") {
+ val variant = adamSNVBuilder()
+ .setNames(ImmutableList.of("rs3131972", "rs201888535"))
+ .build
+
+ val converter = new VariantContextConverter
+
+ val htsjdkVC = converter.convert(ADAMVariantContext(variant))
+ assert(htsjdkVC.hasID)
+ assert(htsjdkVC.getID === "rs3131972;rs201888535")
+ }
}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/AlignmentRecordFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/AlignmentRecordFieldSuite.scala
new file mode 100644
index 0000000000..58ea4a281c
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/AlignmentRecordFieldSuite.scala
@@ -0,0 +1,78 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.AlignmentRecordField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.AlignmentRecord
+
+class AlignmentRecordFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet alignment records") {
+ val path = tmpFile("alignmentRecords.parquet")
+ val rdd = sc.parallelize(Seq(AlignmentRecord.newBuilder()
+ .setContigName("6")
+ .setStart(29941260L)
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ readInFragment,
+ contigName,
+ start,
+ oldPosition,
+ end,
+ mapq,
+ readName,
+ sequence,
+ qual,
+ cigar,
+ oldCigar,
+ basesTrimmedFromStart,
+ basesTrimmedFromEnd,
+ readPaired,
+ properPair,
+ readMapped,
+ mateMapped,
+ failedVendorQualityChecks,
+ duplicateRead,
+ readNegativeStrand,
+ mateNegativeStrand,
+ primaryAlignment,
+ secondaryAlignment,
+ supplementaryAlignment,
+ mismatchingPositions,
+ origQual,
+ attributes,
+ recordGroupName,
+ recordGroupSample,
+ mateAlignmentStart,
+ mateAlignmentEnd,
+ mateContigName,
+ inferredInsertSize
+ )
+
+ val alignmentRecords: RDD[AlignmentRecord] = sc.loadParquet(path, projection = Some(projection))
+ assert(alignmentRecords.count() === 1)
+ assert(alignmentRecords.first.getContigName === "6")
+ assert(alignmentRecords.first.getStart === 29941260L)
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/ContigFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/ContigFieldSuite.scala
new file mode 100644
index 0000000000..0c8e6e25d4
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/ContigFieldSuite.scala
@@ -0,0 +1,62 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.ContigField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.Contig
+
+class ContigFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet contigs") {
+ val path = tmpFile("contigs.parquet")
+ val rdd = sc.parallelize(Seq(Contig.newBuilder()
+ .setContigName("6")
+ .setContigLength(170805979)
+ .setContigMD5("013a29a149b249bb119d27368bb6bf52")
+ .setReferenceURL("http://www.ebi.ac.uk/ena/data/view/GCA_000001405.22")
+ .setAssembly("GRCh38")
+ .setSpecies("Homo sapiens")
+ .setReferenceIndex(0)
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ contigName,
+ contigLength,
+ contigMD5,
+ referenceURL,
+ assembly,
+ species,
+ referenceIndex
+ )
+
+ val contigs: RDD[Contig] = sc.loadParquet(path, projection = Some(projection))
+ assert(contigs.count() === 1)
+ assert(contigs.first.getContigName === "6")
+ assert(contigs.first.getContigLength === 170805979)
+ assert(contigs.first.getContigMD5 === "013a29a149b249bb119d27368bb6bf52")
+ assert(contigs.first.getReferenceURL === "http://www.ebi.ac.uk/ena/data/view/GCA_000001405.22")
+ assert(contigs.first.getAssembly === "GRCh38")
+ assert(contigs.first.getSpecies === "Homo sapiens")
+ assert(contigs.first.getReferenceIndex === 0)
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/DbxrefFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/DbxrefFieldSuite.scala
new file mode 100644
index 0000000000..3256f27b3f
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/DbxrefFieldSuite.scala
@@ -0,0 +1,47 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.DbxrefField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.Dbxref
+
+class DbxrefFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet dbxrefs") {
+ val path = tmpFile("dbxrefs.parquet")
+ val rdd = sc.parallelize(Seq(Dbxref.newBuilder()
+ .setDb("EMBL")
+ .setAccession("AA816246")
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ db,
+ accession
+ )
+
+ val dbxrefs: RDD[Dbxref] = sc.loadParquet(path, projection = Some(projection))
+ assert(dbxrefs.count() === 1)
+ assert(dbxrefs.first.getDb === "EMBL")
+ assert(dbxrefs.first.getAccession === "AA816246")
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/FeatureFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/FeatureFieldSuite.scala
new file mode 100644
index 0000000000..7c33f19f74
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/FeatureFieldSuite.scala
@@ -0,0 +1,123 @@
+/**
+ * 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.projections
+
+import com.google.common.collect.ImmutableList
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.FeatureField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.{
+ Dbxref,
+ Feature,
+ OntologyTerm,
+ Strand
+}
+
+class FeatureFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet features") {
+ val path = tmpFile("features.parquet")
+ val rdd = sc.parallelize(Seq(Feature.newBuilder()
+ .setFeatureId("ENSG00000206503")
+ .setName("HLA-A")
+ .setSource("Ensembl")
+ .setFeatureType("gene")
+ .setContigName("6")
+ .setStart(29941260L)
+ .setEnd(29945884L)
+ .setStrand(Strand.FORWARD)
+ .setPhase(1)
+ .setFrame(2)
+ .setScore(1.0f)
+ .setGeneId("ENSG00000206503")
+ .setTranscriptId("ENST00000396634.5")
+ .setExonId("ENSE00001677386")
+ .setAliases(ImmutableList.of("alias"))
+ .setParentIds(ImmutableList.of("parent_id"))
+ .setTarget("target")
+ .setGap("gap")
+ .setDerivesFrom("derives_from")
+ .setNotes(ImmutableList.of("note"))
+ .setDbxrefs(ImmutableList.of(Dbxref.newBuilder()
+ .setDb("EMBL")
+ .setAccession("AA816246")
+ .build()))
+ .setOntologyTerms(ImmutableList.of(OntologyTerm.newBuilder()
+ .setDb("GO")
+ .setAccession("0046703")
+ .build()))
+ .setCircular(true)
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ featureId,
+ name,
+ source,
+ featureType,
+ contigName,
+ start,
+ end,
+ strand,
+ phase,
+ frame,
+ score,
+ geneId,
+ transcriptId,
+ exonId,
+ aliases,
+ parentIds,
+ target,
+ gap,
+ derivesFrom,
+ notes,
+ dbxrefs,
+ ontologyTerms,
+ circular,
+ attributes
+ )
+
+ val features: RDD[Feature] = sc.loadParquet(path, projection = Some(projection))
+ assert(features.count() === 1)
+ assert(features.first.getFeatureId === "ENSG00000206503")
+ assert(features.first.getName === "HLA-A")
+ assert(features.first.getSource === "Ensembl")
+ assert(features.first.getFeatureType === "gene")
+ assert(features.first.getContigName === "6")
+ assert(features.first.getStart === 29941260L)
+ assert(features.first.getEnd === 29945884L)
+ assert(features.first.getStrand === Strand.FORWARD)
+ assert(features.first.getPhase === 1)
+ assert(features.first.getFrame === 2)
+ assert(features.first.getScore === 1.0f)
+ assert(features.first.getGeneId === "ENSG00000206503")
+ assert(features.first.getTranscriptId === "ENST00000396634.5")
+ assert(features.first.getExonId === "ENSE00001677386")
+ assert(features.first.getAliases.get(0) === "alias")
+ assert(features.first.getParentIds.get(0) === "parent_id")
+ assert(features.first.getTarget === "target")
+ assert(features.first.getGap === "gap")
+ assert(features.first.getDerivesFrom === "derives_from")
+ assert(features.first.getNotes.get(0) === "note")
+ assert(features.first.getDbxrefs.get(0).getAccession === "AA816246")
+ assert(features.first.getOntologyTerms.get(0).getAccession === "0046703")
+ assert(features.first.getCircular === true)
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/FragmentFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/FragmentFieldSuite.scala
new file mode 100644
index 0000000000..2d074dd717
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/FragmentFieldSuite.scala
@@ -0,0 +1,60 @@
+/**
+ * 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.projections
+
+import com.google.common.collect.ImmutableList
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.FragmentField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.{ AlignmentRecord, Fragment }
+
+class FragmentFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet fragments") {
+ val path = tmpFile("fragments.parquet")
+ val rdd = sc.parallelize(Seq(Fragment.newBuilder()
+ .setReadName("read_name")
+ .setInstrument("instrument")
+ .setRunId("run_id")
+ .setFragmentSize(42)
+ .setAlignments(ImmutableList.of(AlignmentRecord.newBuilder()
+ .setContigName("6")
+ .setStart(29941260L)
+ .build()))
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ readName,
+ instrument,
+ runId,
+ fragmentSize,
+ alignments
+ )
+
+ val fragments: RDD[Fragment] = sc.loadParquet(path, projection = Some(projection))
+ assert(fragments.count() === 1)
+ assert(fragments.first.getReadName === "read_name")
+ assert(fragments.first.getInstrument === "instrument")
+ assert(fragments.first.getRunId === "run_id")
+ assert(fragments.first.getFragmentSize === 42)
+ assert(fragments.first.getAlignments.get(0).getContigName === "6")
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/GenotypeFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/GenotypeFieldSuite.scala
new file mode 100644
index 0000000000..676d575444
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/GenotypeFieldSuite.scala
@@ -0,0 +1,67 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.GenotypeField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.Genotype
+
+class GenotypeFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet genotypes") {
+ val path = tmpFile("genotypes.parquet")
+ val rdd = sc.parallelize(Seq(Genotype.newBuilder()
+ .setContigName("6")
+ .setStart(29941260L)
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ variant,
+ contigName,
+ start,
+ end,
+ variantCallingAnnotations,
+ sampleId,
+ sampleDescription,
+ processingDescription,
+ alleles,
+ expectedAlleleDosage,
+ referenceReadDepth,
+ alternateReadDepth,
+ readDepth,
+ minReadDepth,
+ genotypeQuality,
+ genotypeLikelihoods,
+ nonReferenceLikelihoods,
+ strandBiasComponents,
+ splitFromMultiAllelic,
+ phased,
+ phaseSetId,
+ phaseQuality
+ )
+
+ val genotypes: RDD[Genotype] = sc.loadParquet(path, projection = Some(projection))
+ assert(genotypes.count() === 1)
+ assert(genotypes.first.getContigName === "6")
+ assert(genotypes.first.getStart === 29941260L)
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/NucleotideContigFragmentFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/NucleotideContigFragmentFieldSuite.scala
new file mode 100644
index 0000000000..4011433bea
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/NucleotideContigFragmentFieldSuite.scala
@@ -0,0 +1,67 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.NucleotideContigFragmentField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.{ Contig, NucleotideContigFragment }
+
+class NucleotideContigFragmentFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet nucleotide contig fragments") {
+ val path = tmpFile("nucleotideContigFragments.parquet")
+ val rdd = sc.parallelize(Seq(NucleotideContigFragment.newBuilder()
+ .setContig(Contig.newBuilder()
+ .setContigName("6")
+ .build())
+ .setDescription("Chromosome 6")
+ .setFragmentSequence("ACTG")
+ .setFragmentNumber(1)
+ .setFragmentStartPosition(0)
+ .setFragmentEndPosition(4)
+ .setFragmentLength(4)
+ .setNumberOfFragmentsInContig(4)
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ contig,
+ description,
+ fragmentSequence,
+ fragmentNumber,
+ fragmentStartPosition,
+ fragmentEndPosition,
+ fragmentLength,
+ numberOfFragmentsInContig
+ )
+
+ val nucleotideContigFragments: RDD[NucleotideContigFragment] = sc.loadParquet(path, projection = Some(projection))
+ assert(nucleotideContigFragments.count() === 1)
+ assert(nucleotideContigFragments.first.getContig.getContigName === "6")
+ assert(nucleotideContigFragments.first.getDescription === "Chromosome 6")
+ assert(nucleotideContigFragments.first.getFragmentSequence === "ACTG")
+ assert(nucleotideContigFragments.first.getFragmentNumber === 1)
+ assert(nucleotideContigFragments.first.getFragmentStartPosition === 0)
+ assert(nucleotideContigFragments.first.getFragmentEndPosition === 4)
+ assert(nucleotideContigFragments.first.getFragmentLength === 4)
+ assert(nucleotideContigFragments.first.getNumberOfFragmentsInContig === 4)
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/OntologyTermFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/OntologyTermFieldSuite.scala
new file mode 100644
index 0000000000..109dbbc543
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/OntologyTermFieldSuite.scala
@@ -0,0 +1,47 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.OntologyTermField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.OntologyTerm
+
+class OntologyTermFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet ontology terms") {
+ val path = tmpFile("ontologyTerms.parquet")
+ val rdd = sc.parallelize(Seq(OntologyTerm.newBuilder()
+ .setDb("GO")
+ .setAccession("0046703")
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ db,
+ accession
+ )
+
+ val ontologyTerms: RDD[OntologyTerm] = sc.loadParquet(path, projection = Some(projection))
+ assert(ontologyTerms.count() === 1)
+ assert(ontologyTerms.first.getDb === "GO")
+ assert(ontologyTerms.first.getAccession === "0046703")
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/RecordGroupMetadataFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/RecordGroupMetadataFieldSuite.scala
new file mode 100644
index 0000000000..2515102101
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/RecordGroupMetadataFieldSuite.scala
@@ -0,0 +1,72 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.RecordGroupMetadataField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.RecordGroupMetadata
+
+class RecordGroupMetadataFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet record group metadata") {
+ val path = tmpFile("recordGroupMetadata.parquet")
+ val rdd = sc.parallelize(Seq(RecordGroupMetadata.newBuilder()
+ .setName("name")
+ .setSequencingCenter("sequencing_center")
+ .setDescription("description")
+ .setRunDateEpoch(42L)
+ .setFlowOrder("flow_order")
+ .setKeySequence("key_sequence")
+ .setLibrary("library")
+ .setPredictedMedianInsertSize(99)
+ .setPlatform("platform")
+ .setPlatformUnit("platform_unit")
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ name,
+ sample,
+ sequencingCenter,
+ description,
+ runDateEpoch,
+ flowOrder,
+ keySequence,
+ library,
+ predictedMedianInsertSize,
+ platform,
+ platformUnit
+ )
+
+ val recordGroupMetadata: RDD[RecordGroupMetadata] = sc.loadParquet(path, projection = Some(projection))
+ assert(recordGroupMetadata.count() === 1)
+ assert(recordGroupMetadata.first.getName === "name")
+ assert(recordGroupMetadata.first.getSequencingCenter === "sequencing_center")
+ assert(recordGroupMetadata.first.getDescription === "description")
+ assert(recordGroupMetadata.first.getRunDateEpoch === 42L)
+ assert(recordGroupMetadata.first.getFlowOrder === "flow_order")
+ assert(recordGroupMetadata.first.getKeySequence === "key_sequence")
+ assert(recordGroupMetadata.first.getLibrary === "library")
+ assert(recordGroupMetadata.first.getPredictedMedianInsertSize === 99)
+ assert(recordGroupMetadata.first.getPlatform === "platform")
+ assert(recordGroupMetadata.first.getPlatformUnit === "platform_unit")
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/SampleFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/SampleFieldSuite.scala
new file mode 100644
index 0000000000..c361f9168c
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/SampleFieldSuite.scala
@@ -0,0 +1,48 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.SampleField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.Sample
+
+class SampleFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet samples") {
+ val path = tmpFile("samples.parquet")
+ val rdd = sc.parallelize(Seq(Sample.newBuilder()
+ .setSampleId("sample_id")
+ .setName("name")
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ sampleId,
+ name,
+ attributes
+ )
+
+ val samples: RDD[Sample] = sc.loadParquet(path, projection = Some(projection))
+ assert(samples.count() === 1)
+ assert(samples.first.getSampleId === "sample_id")
+ assert(samples.first.getName === "name")
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/TranscriptEffectFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/TranscriptEffectFieldSuite.scala
new file mode 100644
index 0000000000..58c7265152
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/TranscriptEffectFieldSuite.scala
@@ -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 org.bdgenomics.adam.projections
+
+import com.google.common.collect.ImmutableList
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.TranscriptEffectField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.{ TranscriptEffect, VariantAnnotationMessage }
+
+class TranscriptEffectFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet transcript effects") {
+ val path = tmpFile("transcriptEffects.parquet")
+ val rdd = sc.parallelize(Seq(TranscriptEffect.newBuilder()
+ .setAlternateAllele("A")
+ .setEffects(ImmutableList.of("SO:0002012"))
+ .setGeneName("HLA-A")
+ .setGeneId("ENSG00000206503")
+ .setFeatureType("transcript")
+ .setFeatureId("ENST00000396634.5")
+ .setBiotype("Protein_coding")
+ .setRank(1)
+ .setTotal(1)
+ .setGenomicHgvs("gA>T")
+ .setTranscriptHgvs("cA>U")
+ .setProteinHgvs("pG>A")
+ .setCdnaPosition(1)
+ .setCdnaLength(100)
+ .setCdsPosition(2)
+ .setCdsLength(200)
+ .setProteinPosition(3)
+ .setProteinLength(300)
+ .setDistance(42)
+ .setMessages(ImmutableList.of(VariantAnnotationMessage.WARNING_TRANSCRIPT_INCOMPLETE))
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ alternateAllele,
+ effects,
+ geneName,
+ geneId,
+ featureType,
+ featureId,
+ biotype,
+ rank,
+ total,
+ genomicHgvs,
+ transcriptHgvs,
+ proteinHgvs,
+ cdnaPosition,
+ cdnaLength,
+ cdsPosition,
+ cdsLength,
+ proteinPosition,
+ proteinLength,
+ distance,
+ messages
+ )
+
+ val transcriptEffects: RDD[TranscriptEffect] = sc.loadParquet(path, projection = Some(projection))
+ assert(transcriptEffects.count() === 1)
+ assert(transcriptEffects.first.getAlternateAllele === "A")
+ assert(transcriptEffects.first.getEffects.get(0) === "SO:0002012")
+ assert(transcriptEffects.first.getGeneName === "HLA-A")
+ assert(transcriptEffects.first.getGeneId === "ENSG00000206503")
+ assert(transcriptEffects.first.getFeatureType === "transcript")
+ assert(transcriptEffects.first.getFeatureId === "ENST00000396634.5")
+ assert(transcriptEffects.first.getBiotype === "Protein_coding")
+ assert(transcriptEffects.first.getRank === 1)
+ assert(transcriptEffects.first.getTotal === 1)
+ assert(transcriptEffects.first.getGenomicHgvs === "gA>T")
+ assert(transcriptEffects.first.getTranscriptHgvs === "cA>U")
+ assert(transcriptEffects.first.getProteinHgvs === "pG>A")
+ assert(transcriptEffects.first.getCdnaPosition === 1)
+ assert(transcriptEffects.first.getCdnaLength === 100)
+ assert(transcriptEffects.first.getCdsPosition === 2)
+ assert(transcriptEffects.first.getCdsLength === 200)
+ assert(transcriptEffects.first.getProteinPosition === 3)
+ assert(transcriptEffects.first.getProteinLength === 300)
+ assert(transcriptEffects.first.getDistance === 42)
+ assert(transcriptEffects.first.getMessages.contains(VariantAnnotationMessage.WARNING_TRANSCRIPT_INCOMPLETE))
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantAnnotationFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantAnnotationFieldSuite.scala
new file mode 100644
index 0000000000..c7b55c033b
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantAnnotationFieldSuite.scala
@@ -0,0 +1,92 @@
+/**
+ * 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.projections
+
+import com.google.common.collect.ImmutableList
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.VariantAnnotationField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.{ TranscriptEffect, Variant, VariantAnnotation }
+
+class VariantAnnotationFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet variant annotations") {
+ val path = tmpFile("variantAnnotations.parquet")
+ val rdd = sc.parallelize(Seq(VariantAnnotation.newBuilder()
+ .setVariant(Variant.newBuilder()
+ .setContigName("6")
+ .setStart(29941260L)
+ .build())
+ .setAncestralAllele("T")
+ .setAlleleCount(42)
+ .setReadDepth(10)
+ .setForwardReadDepth(4)
+ .setReverseReadDepth(13)
+ .setAlleleFrequency(20.0f)
+ .setCigar("M")
+ .setDbSnp(true)
+ .setHapMap2(true)
+ .setHapMap3(true)
+ .setValidated(true)
+ .setThousandGenomes(true)
+ .setTranscriptEffects(ImmutableList.of(TranscriptEffect.newBuilder()
+ .setEffects(ImmutableList.of("SO:0002012"))
+ .setFeatureType("transcript")
+ .setFeatureId("ENST00000396634.5")
+ .build()))
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ variant,
+ ancestralAllele,
+ alleleCount,
+ readDepth,
+ forwardReadDepth,
+ reverseReadDepth,
+ alleleFrequency,
+ cigar,
+ dbSnp,
+ hapMap2,
+ hapMap3,
+ validated,
+ thousandGenomes,
+ transcriptEffects,
+ attributes
+ )
+
+ val variantAnnotations: RDD[VariantAnnotation] = sc.loadParquet(path, projection = Some(projection))
+ assert(variantAnnotations.count() === 1)
+ assert(variantAnnotations.first.getVariant.getContigName === "6")
+ assert(variantAnnotations.first.getAncestralAllele === "T")
+ assert(variantAnnotations.first.getAlleleCount === 42)
+ assert(variantAnnotations.first.getReadDepth === 10)
+ assert(variantAnnotations.first.getForwardReadDepth === 4)
+ assert(variantAnnotations.first.getReverseReadDepth === 13)
+ assert(variantAnnotations.first.getAlleleFrequency === 20.0f)
+ assert(variantAnnotations.first.getCigar === "M")
+ assert(variantAnnotations.first.getDbSnp === true)
+ assert(variantAnnotations.first.getHapMap2 === true)
+ assert(variantAnnotations.first.getHapMap3 === true)
+ assert(variantAnnotations.first.getValidated === true)
+ assert(variantAnnotations.first.getThousandGenomes === true)
+ assert(variantAnnotations.first.getTranscriptEffects.get(0).getFeatureId === "ENST00000396634.5")
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantCallingAnnotationsFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantCallingAnnotationsFieldSuite.scala
new file mode 100644
index 0000000000..7781fbda81
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantCallingAnnotationsFieldSuite.scala
@@ -0,0 +1,57 @@
+/**
+ * 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.projections
+
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.VariantCallingAnnotationsField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.VariantCallingAnnotations
+
+class VariantCallingAnnotationsFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet variant calling annotations") {
+ val path = tmpFile("variantCallingAnnotations.parquet")
+ val rdd = sc.parallelize(Seq(VariantCallingAnnotations.newBuilder()
+ .setDownsampled(true)
+ .setMapq0Reads(42)
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ variantFilters,
+ downsampled,
+ baseQRankSum,
+ fisherStrandBiasPValue,
+ rmsMapQ,
+ mapq0Reads,
+ mqRankSum,
+ readPositionRankSum,
+ genotypePriors,
+ vqslod,
+ culprit,
+ attributes
+ )
+
+ val variantCallingAnnotations: RDD[VariantCallingAnnotations] = sc.loadParquet(path, projection = Some(projection))
+ assert(variantCallingAnnotations.count() === 1)
+ assert(variantCallingAnnotations.first.getDownsampled === true)
+ assert(variantCallingAnnotations.first.getMapq0Reads === 42)
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantFieldSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantFieldSuite.scala
new file mode 100644
index 0000000000..b304c9e5eb
--- /dev/null
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/projections/VariantFieldSuite.scala
@@ -0,0 +1,72 @@
+/**
+ * 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.projections
+
+import com.google.common.collect.ImmutableList
+import org.apache.spark.rdd.RDD
+import org.bdgenomics.adam.projections.VariantField._
+import org.bdgenomics.adam.rdd.ADAMContext._
+import org.bdgenomics.adam.rdd.TestSaveArgs
+import org.bdgenomics.adam.util.ADAMFunSuite
+import org.bdgenomics.formats.avro.Variant
+
+class VariantFieldSuite extends ADAMFunSuite {
+
+ sparkTest("Use projection when reading parquet variants") {
+ val path = tmpFile("variants.parquet")
+ val rdd = sc.parallelize(Seq(Variant.newBuilder()
+ .setContigName("6")
+ .setStart(29941260L)
+ .setEnd(29941261L)
+ .setNames(ImmutableList.of("rs3948572"))
+ .setReferenceAllele("A")
+ .setAlternateAllele("T")
+ .setFiltersApplied(true)
+ .setFiltersPassed(false)
+ .setFiltersFailed(ImmutableList.of("filter"))
+ .setSomatic(false)
+ .build()))
+ rdd.saveAsParquet(TestSaveArgs(path))
+
+ val projection = Projection(
+ contigName,
+ start,
+ end,
+ names,
+ referenceAllele,
+ alternateAllele,
+ filtersApplied,
+ filtersPassed,
+ filtersFailed,
+ somatic
+ )
+
+ val variants: RDD[Variant] = sc.loadParquet(path, projection = Some(projection))
+ assert(variants.count() === 1)
+ assert(variants.first.getContigName === "6")
+ assert(variants.first.getStart === 29941260L)
+ assert(variants.first.getEnd === 29941261L)
+ assert(variants.first.getNames.get(0) === "rs3948572")
+ assert(variants.first.getReferenceAllele === "A")
+ assert(variants.first.getAlternateAllele === "T")
+ assert(variants.first.getFiltersApplied === true)
+ assert(variants.first.getFiltersPassed === false)
+ assert(variants.first.getFiltersFailed.get(0) === "filter")
+ assert(variants.first.getSomatic === false)
+ }
+}
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/feature/FeatureRDDSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/feature/FeatureRDDSuite.scala
index e4b1f4cfc3..0b995cbbdb 100644
--- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/feature/FeatureRDDSuite.scala
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/feature/FeatureRDDSuite.scala
@@ -44,7 +44,7 @@ class FeatureRDDSuite extends ADAMFunSuite with TypeCheckedTripleEquals {
a.getTarget === b.getTarget &&
a.getGap === b.getGap &&
a.getDerivesFrom === b.getDerivesFrom &&
- a.getIsCircular === b.getIsCircular &&
+ a.getCircular === b.getCircular &&
a.getAliases === b.getAliases &&
a.getNotes === b.getNotes &&
a.getParentIds === b.getParentIds &&
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDDSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDDSuite.scala
index 2628dcbec3..4a70e154f9 100644
--- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDDSuite.scala
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/variation/VariantContextRDDSuite.scala
@@ -17,6 +17,7 @@
*/
package org.bdgenomics.adam.rdd.variation
+import com.google.common.collect.ImmutableList
import com.google.common.io.Files
import java.io.File
import org.bdgenomics.adam.models.{
@@ -43,11 +44,12 @@ class VariantContextRDDSuite extends ADAMFunSuite {
.setStart(17409572)
.setReferenceAllele("T")
.setAlternateAllele("C")
+ .setNames(ImmutableList.of("rs3131972", "rs201888535"))
.build
val g0 = Genotype.newBuilder().setVariant(v0)
.setSampleId("NA12878")
- .setAlleles(List(GenotypeAllele.Ref, GenotypeAllele.Alt))
+ .setAlleles(List(GenotypeAllele.REF, GenotypeAllele.ALT))
.build
VariantContextRDD(sc.parallelize(List(
@@ -61,8 +63,19 @@ class VariantContextRDDSuite extends ADAMFunSuite {
val path = new File(tempDir, "test.vcf")
variants.saveAsVcf(TestSaveArgs(path.getAbsolutePath), false)
assert(path.exists)
+
val vcRdd = sc.loadVcf("%s/test.vcf/part-r-00000".format(tempDir))
assert(vcRdd.rdd.count === 1)
+
+ val variant = vcRdd.rdd.first.variant
+ assert(variant.getContigName === "chr11")
+ assert(variant.getStart === 17409572)
+ assert(variant.getReferenceAllele === "T")
+ assert(variant.getAlternateAllele === "C")
+ assert(variant.getNames.length === 2)
+ assert(variant.getNames.get(0) === "rs3131972")
+ assert(variant.getNames.get(1) === "rs201888535")
+
assert(vcRdd.sequences.records.size === 1)
assert(vcRdd.sequences.records(0).name === "chr11")
}
@@ -77,7 +90,7 @@ class VariantContextRDDSuite extends ADAMFunSuite {
assert(vcRdd.sequences.records(0).name === "chr11")
}
- sparkTest("joins SNV database annotation") {
+ sparkTest("joins SNV variant annotation") {
val v0 = Variant.newBuilder
.setContigName("11")
.setStart(17409572)
@@ -90,16 +103,16 @@ class VariantContextRDDSuite extends ADAMFunSuite {
val vc = VariantContextRDD(sc.parallelize(List(
VariantContext(v0))), sd, Seq.empty)
- val a0 = DatabaseVariantAnnotation.newBuilder
+ val a0 = VariantAnnotation.newBuilder
.setVariant(v0)
- .setDbSnpId(5219)
+ .setDbSnp(true)
.build
- val vda = DatabaseVariantAnnotationRDD(sc.parallelize(List(
+ val vda = VariantAnnotationRDD(sc.parallelize(List(
a0)), sd)
- val annotated = vc.joinDatabaseVariantAnnotation(vda).rdd
- assert(annotated.map(_.databases.isDefined).reduce { (a, b) => a && b })
+ val annotated = vc.joinVariantAnnotations(vda).rdd
+ assert(annotated.map(_.annotations.isDefined).reduce { (a, b) => a && b })
}
sparkTest("don't lose any variants when piping as VCF") {
diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rich/RichGenotypeSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rich/RichGenotypeSuite.scala
index da2f6eb5d0..fb1a344f5c 100644
--- a/adam-core/src/test/scala/org/bdgenomics/adam/rich/RichGenotypeSuite.scala
+++ b/adam-core/src/test/scala/org/bdgenomics/adam/rich/RichGenotypeSuite.scala
@@ -32,7 +32,7 @@ class RichGenotypeSuite extends FunSuite {
test("different ploidy") {
val gb = Genotype.newBuilder.setVariant(v0).setSampleId("NA12878")
for (ploidy <- 0 until 3) {
- val g = gb.setAlleles(List.fill(ploidy)(GenotypeAllele.Ref)).build
+ val g = gb.setAlleles(List.fill(ploidy)(GenotypeAllele.REF)).build
assert(g.ploidy === ploidy)
}
}
@@ -40,23 +40,22 @@ class RichGenotypeSuite extends FunSuite {
test("all types for diploid genotype") {
val gb = Genotype.newBuilder.setVariant(v0).setSampleId("NA12878")
- val hom_ref = gb.setAlleles(List(GenotypeAllele.Ref, GenotypeAllele.Ref)).build
+ val hom_ref = gb.setAlleles(List(GenotypeAllele.REF, GenotypeAllele.REF)).build
assert(hom_ref.getType === GenotypeType.HOM_REF)
- val het1 = gb.setAlleles(List(GenotypeAllele.Ref, GenotypeAllele.Alt)).build
+ val het1 = gb.setAlleles(List(GenotypeAllele.REF, GenotypeAllele.ALT)).build
assert(het1.getType === GenotypeType.HET)
- val het2 = gb.setAlleles(List(GenotypeAllele.Alt, GenotypeAllele.Ref)).build
+ val het2 = gb.setAlleles(List(GenotypeAllele.ALT, GenotypeAllele.REF)).build
assert(het2.getType === GenotypeType.HET)
- val hom_alt = gb.setAlleles(List(GenotypeAllele.Alt, GenotypeAllele.Alt)).build
+ val hom_alt = gb.setAlleles(List(GenotypeAllele.ALT, GenotypeAllele.ALT)).build
assert(hom_alt.getType === GenotypeType.HOM_ALT)
for (a <- GenotypeAllele.values) {
- val no_call1 = gb.setAlleles(List(GenotypeAllele.NoCall, a)).build
+ val no_call1 = gb.setAlleles(List(GenotypeAllele.NO_CALL, a)).build
assert(no_call1.getType === GenotypeType.NO_CALL)
- val no_call2 = gb.setAlleles(List(a, GenotypeAllele.NoCall)).build
+ val no_call2 = gb.setAlleles(List(a, GenotypeAllele.NO_CALL)).build
assert(no_call2.getType === GenotypeType.NO_CALL)
}
}
-
}
diff --git a/pom.xml b/pom.xml
index 0e05fbdf8a..c2375411e4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,7 +27,7 @@
2.7.3
7.7.0
1.7.21
- 0.9.0
+ 0.10.0
0.2.9
2.5.0
1.1.1