From 16a8ed83f187abb7124233dea52ee7bc5676df2a Mon Sep 17 00:00:00 2001 From: Neal Sidhwaney Date: Wed, 26 Feb 2014 18:47:34 -0500 Subject: [PATCH 1/3] Add back CLI code from our variant branch, unit tests, as well as adam2vcf code --- .../cs/amplab/adam/cli/Adam2Vcf.scala | 15 ++-- .../rdd/variation/ADAMVariationContext.scala | 73 +++++++++++++++++-- .../ADAMVariantContextRDDFunctionsSuite.scala | 2 - .../variation/ADAMVariationContextSuite.scala | 49 +++++++++++-- 4 files changed, 117 insertions(+), 22 deletions(-) diff --git a/adam-cli/src/main/scala/edu/berkeley/cs/amplab/adam/cli/Adam2Vcf.scala b/adam-cli/src/main/scala/edu/berkeley/cs/amplab/adam/cli/Adam2Vcf.scala index 714d04f6d0..abf8f2125a 100644 --- a/adam-cli/src/main/scala/edu/berkeley/cs/amplab/adam/cli/Adam2Vcf.scala +++ b/adam-cli/src/main/scala/edu/berkeley/cs/amplab/adam/cli/Adam2Vcf.scala @@ -16,18 +16,13 @@ package edu.berkeley.cs.amplab.adam.cli -import edu.berkeley.cs.amplab.adam.converters.VariantContextConverter -import edu.berkeley.cs.amplab.adam.models.ADAMVariantContext +import edu.berkeley.cs.amplab.adam.avro.ADAMGenotype import edu.berkeley.cs.amplab.adam.rdd.AdamContext._ import edu.berkeley.cs.amplab.adam.rdd.variation.ADAMVariationContext._ -import edu.berkeley.cs.amplab.adam.util.{AdamVCFOutputFormat, VcfHeaderUtils} -import fi.tkk.ics.hadoop.bam.VariantContextWritable -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.mapreduce.Job -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.SparkContext._ +import org.kohsuke.args4j.Argument import org.apache.spark.rdd.RDD -import org.kohsuke.args4j.{Argument, Option => Args4jOption} +import org.apache.spark.{Logging, SparkContext} +import org.apache.hadoop.mapreduce.Job object Adam2Vcf extends AdamCommandCompanion { @@ -50,5 +45,7 @@ class Adam2Vcf(val args: Adam2VcfArgs) extends AdamSparkCommand[Adam2VcfArgs] wi val companion = Adam2Vcf def run(sc: SparkContext, job: Job) { + val adamGTs: RDD[ADAMGenotype] = sc.adamLoad(args.adamFile) + sc.adamVCFSave(args.outputPath, adamGTs.toADAMVariantContext) } } diff --git a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala index 63609353c8..32c7953447 100644 --- a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala +++ b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala @@ -16,23 +16,56 @@ package edu.berkeley.cs.amplab.adam.rdd.variation -import org.apache.spark.rdd.RDD -import edu.berkeley.cs.amplab.adam.models.ADAMVariantContext -import org.apache.spark.{Logging, SparkContext} -import org.apache.hadoop.mapreduce.Job +import edu.berkeley.cs.amplab.adam.avro.ADAMGenotype import edu.berkeley.cs.amplab.adam.converters.VariantContextConverter -import fi.tkk.ics.hadoop.bam.{VariantContextWritable, VCFInputFormat} +import edu.berkeley.cs.amplab.adam.models.{ADAMVariantContext, SequenceDictionary} +import edu.berkeley.cs.amplab.adam.rdd.variation.ADAMVariationContext._ +import fi.tkk.ics.hadoop.bam._ import org.apache.hadoop.io.LongWritable +import org.apache.hadoop.mapreduce.Job +import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.broadinstitute.variant.vcf.{VCFHeaderLine, VCFHeader} import parquet.hadoop.util.ContextUtil +import scala.collection.JavaConversions._ + + +private object ADAMVCFOutputFormat { + private var header : Option[VCFHeader] = None + def getHeader : VCFHeader = header match { + case Some(h) => h + case None => setHeader(Seq()) + } + + def setHeader(samples: Seq[String]) : VCFHeader = { + header = Some(new VCFHeader( + (VariantContextConverter.infoHeaderLines ++ VariantContextConverter.formatHeaderLines).toSet : Set[VCFHeaderLine], + samples + )) + header.get + } +} + +/** + * Wrapper for Hadoop-BAM to work around requirement for no-args constructor. Depends on + * ADAMVCFOutputFormat object to maintain global state (such as samples) + * + * @tparam K + */ +private class ADAMVCFOutputFormat[K] extends KeyIgnoringVCFOutputFormat[K](VCFFormat.VCF) { + setHeader(ADAMVCFOutputFormat.getHeader) +} object ADAMVariationContext { implicit def sparkContextToADAMVariationContext(sc: SparkContext): ADAMVariationContext = new ADAMVariationContext(sc) - implicit def rddToADAMVariantContextRDD(rdd: RDD[ADAMVariantContext]) = new ADAMVariantContextRDDFunctions(rdd) + implicit def rddToADAMGenotypeRDD(rdd: RDD[ADAMGenotype]) = new ADAMGenotypeRDDFunctions(rdd) } class ADAMVariationContext(sc: SparkContext) extends Serializable with Logging { + initLogging() /** * This method will create a new RDD of VariantContext objects @@ -51,6 +84,34 @@ class ADAMVariationContext(sc: SparkContext) extends Serializable with Logging { log.info("Converted %d records".format(records.count())) records.flatMap(p => vcc.convert(p._2.get)) } + + def adamVCFSave(filePath: String, variants: RDD[ADAMVariantContext], dict: Option[SequenceDictionary] = None) = { + val vcfFormat = VCFFormat.inferFromFilePath(filePath) + assert(vcfFormat == VCFFormat.VCF, "BCF not yet supported") // TODO: Add BCF support + + log.info("Writing %s file to %s".format(vcfFormat, filePath)) + + // Initialize global header object required by Hadoop VCF Writer + ADAMVCFOutputFormat.setHeader(variants.adamGetCallsetSamples) + + // TODO: Sort variants according to sequence dictionary (if supplied) + val converter = new VariantContextConverter(dict) + val gatkVCs: RDD[VariantContextWritable] = variants.map(v => { + val vcw = new VariantContextWritable + vcw.set(converter.convert(v)) + vcw + }) + val withKey = gatkVCs.keyBy(v => new LongWritable(v.get.getStart)) + + val conf = sc.hadoopConfiguration + conf.set(VCFOutputFormat.OUTPUT_VCF_FORMAT_PROPERTY, vcfFormat.toString) + withKey.saveAsNewAPIHadoopFile(filePath, + classOf[LongWritable], classOf[VariantContextWritable], classOf[ADAMVCFOutputFormat[LongWritable]], + conf + ) + + log.info("Write %d records".format(gatkVCs.count)) + } } diff --git a/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariantContextRDDFunctionsSuite.scala b/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariantContextRDDFunctionsSuite.scala index beba3fee6a..44484a0f6a 100644 --- a/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariantContextRDDFunctionsSuite.scala +++ b/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariantContextRDDFunctionsSuite.scala @@ -47,8 +47,6 @@ class ADAMVariantContextRDDFunctionsSuite extends SparkFunSuite { // TODO: implicit conversion to ADAMVariantContextRDD val annotated = vc.joinDatabaseVariantAnnotation(vda) - println("nealsid: " + annotated.count()) - println("nealsid: " + annotated.first.databases.isDefined) assert( annotated.map(_.databases.isDefined).reduce { (a,b) => a && b } ) } diff --git a/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContextSuite.scala b/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContextSuite.scala index 714d27094d..3ec1da4897 100644 --- a/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContextSuite.scala +++ b/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContextSuite.scala @@ -19,16 +19,55 @@ package edu.berkeley.cs.amplab.adam.rdd.variation import org.apache.spark.rdd.RDD import edu.berkeley.cs.amplab.adam.util.SparkFunSuite import edu.berkeley.cs.amplab.adam.models.ADAMVariantContext +import edu.berkeley.cs.amplab.adam.rdd.variation.ADAMVariationContext._ +import com.google.common.io.Files +import java.io.File +import edu.berkeley.cs.amplab.adam.avro.{ADAMGenotypeAllele, ADAMGenotype, ADAMContig, ADAMVariant} +import scala.collection.JavaConversions._ + class ADAMVariationContextSuite extends SparkFunSuite { + val tempDir = Files.createTempDir() + + def variants: RDD[ADAMVariantContext] = { + val v0 = ADAMVariant.newBuilder + .setContig(ADAMContig.newBuilder.setContigId(1).setContigName("chr11").build) + .setPosition(17409572) + .setReferenceAllele("T") + .setVariantAllele("C") + .build + + val g0 = ADAMGenotype.newBuilder().setVariant(v0) + .setSampleId("NA12878") + .setAlleles(List(ADAMGenotypeAllele.Ref, ADAMGenotypeAllele.Alt)) + .build + + sc.parallelize(List( + ADAMVariantContext(v0, Seq(g0)) + )) + } + + sparkTest("can read a small .vcf file") { val path = ClassLoader.getSystemClassLoader.getResource("small.vcf").getFile - // TODO: Why doesn't implict work here? - val VCs: RDD[ADAMVariantContext] = ADAMVariationContext.sparkContextToADAMVariationContext(sc).adamVCFLoad(path) - assert(VCs.count === 5) - val VC = VCs.first - assert(VC.genotypes.length === 3) + val vcs: RDD[ADAMVariantContext] = sc.adamVCFLoad(path) + assert(vcs.count === 5) + + val vc = vcs.first + assert(vc.genotypes.length === 3) + + val gt = vc.genotypes.head + assert(gt.getVariantCallingAnnotations != null) + assert(gt.getVariantCallingAnnotations.getReadDepth === 69) + // Recall we are testing parsing, so we assert that our value is + // the same as should have been parsed + assert(gt.getVariantCallingAnnotations.getClippingRankSum === java.lang.Float.valueOf("0.138")) } + sparkTest("can write, then read in .vcf file") { + val path = new File(tempDir, "test.vcf") + sc.adamVCFSave(path.getAbsolutePath, variants) + assert(path.exists) + } } From 363c25acb002f2b10f36a02835ca08b5303f085c Mon Sep 17 00:00:00 2001 From: Neal Sidhwaney Date: Wed, 26 Feb 2014 19:16:57 -0500 Subject: [PATCH 2/3] Add concordance class and rich AdamGenotype class, and Genotype RDD functions class --- .../cs/amplab/adam/rich/GenotypeType.java | 21 +++++ .../amplab/adam/models/ConcordanceTable.scala | 83 +++++++++++++++++++ .../variation/ADAMVariationRDDFunctions.scala | 30 ++++++- .../amplab/adam/rich/RichADAMGenotype.scala | 37 +++++++++ .../ADAMGenotypeRDDFunctionsSuite.scala | 56 +++++++++++++ 5 files changed, 225 insertions(+), 2 deletions(-) create mode 100644 adam-core/src/main/java/edu/berkeley/cs/amplab/adam/rich/GenotypeType.java create mode 100644 adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/models/ConcordanceTable.scala create mode 100644 adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rich/RichADAMGenotype.scala create mode 100644 adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMGenotypeRDDFunctionsSuite.scala diff --git a/adam-core/src/main/java/edu/berkeley/cs/amplab/adam/rich/GenotypeType.java b/adam-core/src/main/java/edu/berkeley/cs/amplab/adam/rich/GenotypeType.java new file mode 100644 index 0000000000..dbba01109d --- /dev/null +++ b/adam-core/src/main/java/edu/berkeley/cs/amplab/adam/rich/GenotypeType.java @@ -0,0 +1,21 @@ +/* + * Copyright (c) 2014. Mount Sinai School of Medicine + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package edu.berkeley.cs.amplab.adam.rich; + +public enum GenotypeType { + HOM_REF, HET, HOM_ALT, NO_CALL +} diff --git a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/models/ConcordanceTable.scala b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/models/ConcordanceTable.scala new file mode 100644 index 0000000000..e464d3a377 --- /dev/null +++ b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/models/ConcordanceTable.scala @@ -0,0 +1,83 @@ +/* + * Copyright (c) 2014. Mount Sinai School of Medicine + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package edu.berkeley.cs.amplab.adam.models + +import edu.berkeley.cs.amplab.adam.rich.GenotypeType + +object ConcordanceTable { + def apply() = new ConcordanceTable() + + def create(v: (GenotypeType, GenotypeType)) = new ConcordanceTable(v) + + /** + * + * @param l Modified + * @param r + * @return + */ + + def addComparison(l: ConcordanceTable, r: (GenotypeType, GenotypeType)) = { + l.add(r) + l + } + + /** + * + * @param l Modified + * @param r + * @return + */ + def mergeTable(l: ConcordanceTable, r: ConcordanceTable) = { + l.add(r) + l + } + + // Relevant sub-groups of concordance table entries + val CALLED = Seq(GenotypeType.HOM_REF, GenotypeType.HET, GenotypeType.HOM_ALT) + val VARIANT = Seq(GenotypeType.HET, GenotypeType.HOM_ALT) + val ALL = GenotypeType.values.toSeq + + implicit def typesToIndices(t : Seq[GenotypeType]) : Seq[Int] = t.map(_.ordinal) +} + + +class ConcordanceTable { + import ConcordanceTable._ + + private val table_ = Array.fill[Long](GenotypeType.values.length, GenotypeType.values.length)(0L) + + def this(p : (GenotypeType, GenotypeType)) = { + this() + add(p) + } + + def add(p : (GenotypeType, GenotypeType)) = table_(p._1.ordinal)(p._2.ordinal) += 1L + def add(that : ConcordanceTable) = { + for (r <- ALL : Seq[Int]; c <- ALL : Seq[Int]) + table_(r)(c) += that.table_(r)(c) + } + + def get(test: GenotypeType, truth: GenotypeType) : Long = table_(test.ordinal)(truth.ordinal) + + + def total(diagonal : Seq[Int]) = ( for (i <- diagonal) yield table_(i)(i) ).sum + def total(test : Seq[Int] = ALL, truth : Seq[Int] = ALL) = + ( for (r <- test; c <- truth) yield table_(r)(c) ).sum + + def concordance : Double = total(CALLED).toDouble / total(CALLED, CALLED).toDouble + def nonReferenceSensitivity : Double = total(VARIANT, VARIANT).toDouble / total(ALL, VARIANT).toDouble +} diff --git a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationRDDFunctions.scala b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationRDDFunctions.scala index 64ad1f8432..955ef2a825 100644 --- a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationRDDFunctions.scala +++ b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationRDDFunctions.scala @@ -17,8 +17,12 @@ package edu.berkeley.cs.amplab.adam.rdd.variation import edu.berkeley.cs.amplab.adam.avro.{ADAMGenotype, ADAMDatabaseVariantAnnotation} -import edu.berkeley.cs.amplab.adam.models.{ADAMVariantContext, SequenceDictionary, SequenceRecord} -import edu.berkeley.cs.amplab.adam.rich.RichADAMVariant +import edu.berkeley.cs.amplab.adam.models.{ADAMVariantContext, + ConcordanceTable, + SequenceDictionary, + SequenceRecord} +import edu.berkeley.cs.amplab.adam.rich.{GenotypeType, RichADAMVariant} +import edu.berkeley.cs.amplab.adam.rich.RichADAMGenotype._ import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ @@ -60,4 +64,26 @@ class ADAMGenotypeRDDFunctions(rdd: RDD[ADAMGenotype]) extends Serializable with .groupByKey .map { case (v:RichADAMVariant, g) => new ADAMVariantContext(v, g, None) } } + + def concordanceWith(truth: RDD[ADAMGenotype]) : RDD[(String, ConcordanceTable)] = { + val keyedTest = rdd.keyBy(g => (g.getVariant, g.getSampleId.toString) : (RichADAMVariant, String)) + val keyedTruth = truth.keyBy(g => (g.getVariant, g.getSampleId.toString) : (RichADAMVariant, String)) + + val inTest = keyedTest.leftOuterJoin(keyedTruth) + val justInTruth = keyedTruth.subtractByKey(inTest) + + // Compute RDD[sample -> ConcordanceTable] across variants/samples + val bySample = inTest.map({ + case ((_, sample), (l, Some(r))) => sample -> (l.getType, r.getType) + case ((_, sample), (l, None)) => sample -> (l.getType, GenotypeType.NO_CALL) + }).union(justInTruth.map({ // add in "truth-only" entries + case ((_, sample), r) => sample -> (GenotypeType.NO_CALL, r.getType) + })).combineByKey( + ConcordanceTable.create, + ConcordanceTable.addComparison, + ConcordanceTable.mergeTable + ) + + bySample + } } diff --git a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rich/RichADAMGenotype.scala b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rich/RichADAMGenotype.scala new file mode 100644 index 0000000000..33ec9b51db --- /dev/null +++ b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rich/RichADAMGenotype.scala @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2014. Mount Sinai School of Medicine + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package edu.berkeley.cs.amplab.adam.rich + +import edu.berkeley.cs.amplab.adam.avro.{ADAMGenotypeAllele, ADAMGenotype} +import scala.collection.JavaConversions._ + +object RichADAMGenotype { + implicit def genotypeToRichGenotype(g: ADAMGenotype) = new RichADAMGenotype(g) + implicit def richGenotypeToGenotype(g: RichADAMGenotype) = g.genotype +} + +class RichADAMGenotype(val genotype: ADAMGenotype) { + def getType: GenotypeType = { + genotype.getAlleles.toList.distinct match { + case List(ADAMGenotypeAllele.Ref) => GenotypeType.HOM_REF + case List(ADAMGenotypeAllele.Alt) => GenotypeType.HOM_ALT + case List(ADAMGenotypeAllele.Ref, ADAMGenotypeAllele.Alt) | + List(ADAMGenotypeAllele.Alt, ADAMGenotypeAllele.Ref) => GenotypeType.HET + case _ => GenotypeType.NO_CALL + } + } +} diff --git a/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMGenotypeRDDFunctionsSuite.scala b/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMGenotypeRDDFunctionsSuite.scala new file mode 100644 index 0000000000..37ae5b36b8 --- /dev/null +++ b/adam-core/src/test/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMGenotypeRDDFunctionsSuite.scala @@ -0,0 +1,56 @@ +/* + * Copyright (c) 2014. Mount Sinai School of Medicine + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package edu.berkeley.cs.amplab.adam.rdd.variation + +import edu.berkeley.cs.amplab.adam.util.SparkFunSuite +import edu.berkeley.cs.amplab.adam.avro._ +import edu.berkeley.cs.amplab.adam.models.ConcordanceTable +import edu.berkeley.cs.amplab.adam.rdd.variation.ADAMVariationContext._ +import scala.collection.JavaConversions._ +import org.apache.spark.SparkContext._ +import edu.berkeley.cs.amplab.adam.rich.GenotypeType + + +class ADAMGenotypeRDDFunctionsSuite extends SparkFunSuite { + def v0 = ADAMVariant.newBuilder + .setContig(ADAMContig.newBuilder.setContigName("11").build) + .setPosition(17409572) + .setReferenceAllele("T") + .setVariantAllele("C") + .build + + sparkTest("concordance of identical and non-identical genotypes") { + val gb = ADAMGenotype.newBuilder().setVariant(v0) + .setSampleId("NA12878") + .setAlleles(List(ADAMGenotypeAllele.Ref, ADAMGenotypeAllele.Alt)) + + val g0 = gb.build + val g1 = gb.build + val tables0 = sc.parallelize(Seq(g0)).concordanceWith(sc.parallelize(Seq(g1))).collectAsMap + assert(tables0.size === 1) + val table0 = tables0.getOrElse("NA12878", ConcordanceTable()) + assert(table0.total() === 1) + assert(table0.get(GenotypeType.HET, GenotypeType.HET) === 1) + + val g2 = gb.setAlleles(List(ADAMGenotypeAllele.Ref, ADAMGenotypeAllele.Ref)).build + val table1 = sc.parallelize(Seq(g0)) + .concordanceWith(sc.parallelize(Seq(g2))).collectAsMap() + .getOrElse("NA12878", ConcordanceTable()) + assert(table1.total() === 1) + assert(table1.get(GenotypeType.HET, GenotypeType.HOM_REF) === 1) + } +} From 0d3687fda2f3e9e97e214ff84df56347b3d3c3b7 Mon Sep 17 00:00:00 2001 From: Neal Sidhwaney Date: Thu, 27 Feb 2014 17:21:12 -0500 Subject: [PATCH 3/3] Code review feedback --- .../cs/amplab/adam/rdd/variation/ADAMVariationContext.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala index 32c7953447..9e682e8d4c 100644 --- a/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala +++ b/adam-core/src/main/scala/edu/berkeley/cs/amplab/adam/rdd/variation/ADAMVariationContext.scala @@ -65,7 +65,6 @@ object ADAMVariationContext { } class ADAMVariationContext(sc: SparkContext) extends Serializable with Logging { - initLogging() /** * This method will create a new RDD of VariantContext objects