forked from bigdatagenomics/adam
-
Notifications
You must be signed in to change notification settings - Fork 1
/
VariantContextRDD.scala
233 lines (206 loc) · 7.7 KB
/
VariantContextRDD.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
/**
* Licensed to Big Data Genomics (BDG) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The BDG licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.bdgenomics.adam.rdd.variant
import htsjdk.samtools.ValidationStringency
import htsjdk.variant.vcf.{ VCFHeader, VCFHeaderLine }
import java.io.OutputStream
import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.fs.Path
import org.apache.spark.rdd.RDD
import org.bdgenomics.adam.converters.{
DefaultHeaderLines,
VariantContextConverter
}
import org.bdgenomics.adam.models.{
ReferencePosition,
ReferenceRegion,
ReferenceRegionSerializer,
SequenceDictionary,
VariantContext,
VariantContextSerializer
}
import org.bdgenomics.adam.rdd.{
FileMerger,
MultisampleGenomicRDD,
VCFHeaderUtils
}
import org.bdgenomics.adam.rich.RichVariant
import org.bdgenomics.adam.serialization.AvroSerializer
import org.bdgenomics.formats.avro.Sample
import org.bdgenomics.utils.cli.SaveArgs
import org.bdgenomics.utils.misc.Logging
import org.bdgenomics.utils.interval.array.{
IntervalArray,
IntervalArraySerializer
}
import org.seqdoop.hadoop_bam._
import scala.collection.JavaConversions._
import scala.reflect.ClassTag
private[adam] case class VariantContextArray(
array: Array[(ReferenceRegion, VariantContext)],
maxIntervalWidth: Long) extends IntervalArray[ReferenceRegion, VariantContext] {
protected def replace(arr: Array[(ReferenceRegion, VariantContext)],
maxWidth: Long): IntervalArray[ReferenceRegion, VariantContext] = {
VariantContextArray(arr, maxWidth)
}
}
private[adam] class VariantContextArraySerializer extends IntervalArraySerializer[ReferenceRegion, VariantContext, VariantContextArray] {
protected val kSerializer = new ReferenceRegionSerializer
protected val tSerializer = new VariantContextSerializer
protected def builder(arr: Array[(ReferenceRegion, VariantContext)],
maxIntervalWidth: Long): VariantContextArray = {
VariantContextArray(arr, maxIntervalWidth)
}
}
/**
* An RDD containing VariantContexts attached to a reference and samples.
*
* @param rdd The underlying RDD of VariantContexts.
* @param sequences The genome sequence these variants were called against.
* @param samples The genotyped samples in this RDD of VariantContexts.
* @param headerLines The VCF header lines that cover all INFO/FORMAT fields
* needed to represent this RDD of VariantContexts.
*/
case class VariantContextRDD(rdd: RDD[VariantContext],
sequences: SequenceDictionary,
@transient samples: Seq[Sample],
@transient headerLines: Seq[VCFHeaderLine] = DefaultHeaderLines.allHeaderLines) extends MultisampleGenomicRDD[VariantContext, VariantContextRDD]
with Logging {
protected def buildTree(rdd: RDD[(ReferenceRegion, VariantContext)])(
implicit tTag: ClassTag[VariantContext]): IntervalArray[ReferenceRegion, VariantContext] = {
IntervalArray(rdd, VariantContextArray.apply(_, _))
}
/**
* @return Returns a GenotypeRDD containing the Genotypes in this RDD.
*/
def toGenotypeRDD: GenotypeRDD = {
GenotypeRDD(rdd.flatMap(_.genotypes),
sequences,
samples,
headerLines)
}
/**
* @return Returns the Variants in this RDD.
*/
def toVariantRDD: VariantRDD = {
VariantRDD(rdd.map(_.variant.variant),
sequences,
headerLines)
}
/**
* Converts an RDD of ADAM VariantContexts to HTSJDK VariantContexts
* and saves to disk as VCF.
*
* @param filePath The filepath to save to.
* @param sortOnSave Whether to sort before saving.
*/
def saveAsVcf(args: SaveArgs,
sortOnSave: Boolean) {
saveAsVcf(args.outputPath, sortOnSave)
}
/**
* Converts an RDD of ADAM VariantContexts to HTSJDK VariantContexts
* and saves to disk as VCF.
*
* @param filePath The filepath to save to.
* @param asSingleFile If true, saves the output as a single file by merging
* the sharded output after completing the write to HDFS. If false, the
* output of this call will be written as shards, where each shard has a
* valid VCF header. Default is false.
* @param stringency The validation stringency to use when writing the VCF.
*/
def saveAsVcf(filePath: String,
asSingleFile: Boolean = false,
stringency: ValidationStringency = ValidationStringency.LENIENT) {
val vcfFormat = VCFFormat.inferFromFilePath(filePath)
assert(vcfFormat == VCFFormat.VCF, "BCF not yet supported") // TODO: Add BCF support
log.info(s"Writing $vcfFormat file to $filePath")
// map samples to sample ids
val sampleIds = samples.map(_.getSampleId)
// convert the variants to htsjdk VCs
val converter = new VariantContextConverter(headerLines, stringency)
val writableVCs: RDD[(LongWritable, VariantContextWritable)] = rdd.flatMap(vc => {
converter.convert(vc)
.map(htsjdkVc => {
val vcw = new VariantContextWritable
vcw.set(htsjdkVc)
(new LongWritable(vc.position.pos), vcw)
})
})
// make header
val header = new VCFHeader(
headerLines.toSet,
samples.map(_.getSampleId))
header.setSequenceDictionary(sequences.toSAMSequenceDictionary)
// write header
val headPath = new Path("%s_head".format(filePath))
// configure things for saving to disk
val conf = rdd.context.hadoopConfiguration
val fs = headPath.getFileSystem(conf)
// write vcf header
VCFHeaderUtils.write(header,
headPath,
fs)
// set path to header file and the vcf format
conf.set("org.bdgenomics.adam.rdd.variant.vcf_header_path", headPath.toString)
conf.set(VCFOutputFormat.OUTPUT_VCF_FORMAT_PROPERTY, vcfFormat.toString)
if (asSingleFile) {
// write shards to disk
val tailPath = "%s_tail".format(filePath)
writableVCs.saveAsNewAPIHadoopFile(
tailPath,
classOf[LongWritable],
classOf[VariantContextWritable],
classOf[ADAMHeaderlessVCFOutputFormat[LongWritable]],
conf
)
// merge shards
FileMerger.mergeFiles(rdd.context.hadoopConfiguration,
fs,
new Path(filePath),
new Path(tailPath),
Some(headPath))
} else {
// write shards
writableVCs.saveAsNewAPIHadoopFile(
filePath,
classOf[LongWritable],
classOf[VariantContextWritable],
classOf[ADAMVCFOutputFormat[LongWritable]],
conf
)
// remove header file
fs.delete(headPath, true)
}
}
/**
* @param newRdd The RDD of VariantContexts to replace the underlying RDD.
* @return Returns a new VariantContextRDD where the underlying RDD has
* been replaced.
*/
protected def replaceRdd(newRdd: RDD[VariantContext]): VariantContextRDD = {
copy(rdd = newRdd)
}
/**
* @param elem The variant context to get a reference region for.
* @return Returns a seq containing the position key from the variant context.
*/
protected def getReferenceRegions(elem: VariantContext): Seq[ReferenceRegion] = {
Seq(ReferenceRegion(elem.position))
}
}