/
CoverageRDD.scala
426 lines (375 loc) · 14.2 KB
/
CoverageRDD.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
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
/**
* 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.feature
import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.serializers.FieldSerializer
import org.apache.parquet.hadoop.metadata.CompressionCodecName
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{ Dataset, SQLContext }
import org.bdgenomics.adam.models.{
Coverage,
ReferenceRegion,
ReferenceRegionSerializer,
SequenceDictionary
}
import org.bdgenomics.adam.rdd.ADAMContext._
import org.bdgenomics.adam.rdd.GenomicDataset
import org.bdgenomics.utils.interval.array.{
IntervalArray,
IntervalArraySerializer
}
import scala.annotation.tailrec
import scala.reflect.ClassTag
import scala.reflect.runtime.universe._
private[adam] case class CoverageArray(
array: Array[(ReferenceRegion, Coverage)],
maxIntervalWidth: Long) extends IntervalArray[ReferenceRegion, Coverage] {
def duplicate(): IntervalArray[ReferenceRegion, Coverage] = {
copy()
}
protected def replace(arr: Array[(ReferenceRegion, Coverage)],
maxWidth: Long): IntervalArray[ReferenceRegion, Coverage] = {
CoverageArray(arr, maxWidth)
}
}
private[adam] class CoverageArraySerializer(kryo: Kryo) extends IntervalArraySerializer[ReferenceRegion, Coverage, CoverageArray] {
protected val kSerializer = new ReferenceRegionSerializer
protected val tSerializer = new FieldSerializer[Coverage](kryo, classOf[Coverage])
protected def builder(arr: Array[(ReferenceRegion, Coverage)],
maxIntervalWidth: Long): CoverageArray = {
CoverageArray(arr, maxIntervalWidth)
}
}
case class ParquetUnboundCoverageRDD private[rdd] (
@transient private val sc: SparkContext,
private val parquetFilename: String,
sequences: SequenceDictionary) extends CoverageRDD {
lazy val rdd: RDD[Coverage] = {
sc.loadParquetCoverage(parquetFilename,
forceRdd = true).rdd
}
protected lazy val optPartitionMap = sc.extractPartitionMap(parquetFilename)
lazy val dataset = {
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._
sqlContext.read.parquet(parquetFilename)
.select("contigName", "start", "end", "score")
.withColumnRenamed("score", "count")
.as[Coverage]
}
def toFeatures(): FeatureRDD = {
ParquetUnboundFeatureRDD(sc, parquetFilename, sequences)
}
def replaceSequences(
newSequences: SequenceDictionary): CoverageRDD = {
copy(sequences = newSequences)
}
}
/**
* A Dataset containing Coverage data.
*
* @param dataset A SQL Dataset containing data describing how many reads cover
* a genomic locus/region.
* @param sequences A dictionary describing the reference genome.
*/
case class DatasetBoundCoverageRDD private[rdd] (
dataset: Dataset[Coverage],
sequences: SequenceDictionary) extends CoverageRDD {
protected lazy val optPartitionMap = None
lazy val rdd: RDD[Coverage] = {
dataset.rdd
}
def toFeatures(): FeatureRDD = {
import dataset.sqlContext.implicits._
DatasetBoundFeatureRDD(dataset.map(_.toSqlFeature), sequences)
}
def replaceSequences(
newSequences: SequenceDictionary): CoverageRDD = {
copy(sequences = newSequences)
}
}
/**
* An RDD containing Coverage data.
*
* @param rdd An RDD containing data describing how many reads cover a genomic
* locus/region.
* @param sequences A dictionary describing the reference genome.
*/
case class RDDBoundCoverageRDD private[rdd] (
rdd: RDD[Coverage],
sequences: SequenceDictionary,
optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends CoverageRDD {
lazy val dataset: Dataset[Coverage] = {
val sqlContext = SQLContext.getOrCreate(rdd.context)
import sqlContext.implicits._
sqlContext.createDataset(rdd)
}
def toFeatures(): FeatureRDD = {
val featureRdd = rdd.map(_.toFeature)
new RDDBoundFeatureRDD(featureRdd, sequences, optPartitionMap = optPartitionMap)
}
def replaceSequences(
newSequences: SequenceDictionary): CoverageRDD = {
copy(sequences = newSequences)
}
}
abstract class CoverageRDD extends GenomicDataset[Coverage, Coverage, CoverageRDD] {
protected val productFn = (c: Coverage) => c
protected val unproductFn = (c: Coverage) => c
@transient val uTag: TypeTag[Coverage] = typeTag[Coverage]
protected def buildTree(rdd: RDD[(ReferenceRegion, Coverage)])(
implicit tTag: ClassTag[Coverage]): IntervalArray[ReferenceRegion, Coverage] = {
IntervalArray(rdd, CoverageArray.apply(_, _))
}
def union(rdds: CoverageRDD*): CoverageRDD = {
val iterableRdds = rdds.toSeq
val mergedSequences = iterableRdds.map(_.sequences).fold(sequences)(_ ++ _)
if (iterableRdds.forall(rdd => rdd match {
case DatasetBoundCoverageRDD(_, _) => true
case _ => false
})) {
DatasetBoundCoverageRDD(iterableRdds.map(_.dataset)
.fold(dataset)(_.union(_)), mergedSequences)
} else {
RDDBoundCoverageRDD(rdd.context.union(rdd, iterableRdds.map(_.rdd): _*),
mergedSequences,
None)
}
}
def saveAsParquet(filePath: String,
blockSize: Int = 128 * 1024 * 1024,
pageSize: Int = 1 * 1024 * 1024,
compressCodec: CompressionCodecName = CompressionCodecName.GZIP,
disableDictionaryEncoding: Boolean = false) {
toFeatures().saveAsParquet(filePath,
blockSize,
pageSize,
compressCodec,
disableDictionaryEncoding)
}
def transformDataset(
tFn: Dataset[Coverage] => Dataset[Coverage]): CoverageRDD = {
DatasetBoundCoverageRDD(tFn(dataset), sequences)
}
/**
* Saves coverage as feature file.
*
* @see FeatureRDD.save
*
* Supported file formats include bed, narrowPeak and parquet. Coverage is saved
* as a feature where coverage is stored in score attribute.
* val chrom = feature.getContigName
* val start = feature.getStart
* val end = feature.getEnd
* val name = Features.nameOf(feature)
* val coverage = feature.getScore
*
* @param filePath The location to write the output.
* @param asSingleFile If false, writes file to disk as shards with
* one shard per partition. If true, we save the file to disk as a single
* file by merging the shards.
* @param disableFastConcat If asSingleFile is true, disables the use of the
* fast file concatenation engine.
*/
def save(filePath: java.lang.String,
asSingleFile: java.lang.Boolean,
disableFastConcat: java.lang.Boolean) = {
toFeatures.save(filePath,
asSingleFile,
disableFastConcat)
}
/**
* Merges adjacent ReferenceRegions with the same coverage value.
* This reduces the loss of coverage information while reducing the number of records in the RDD.
* For example, adjacent records Coverage("chr1", 1, 10, 3.0) and Coverage("chr1", 10, 20, 3.0)
* would be merged into one record Coverage("chr1", 1, 20, 3.0).
*
* @note Data must be sorted before collapse is called.
*
* @return merged tuples of adjacent ReferenceRegions and coverage.
*/
def collapse(): CoverageRDD = {
val newRDD: RDD[Coverage] = rdd
.mapPartitions(iter => {
if (iter.hasNext) {
val first = iter.next
collapse(iter, first, List.empty)
} else iter
})
transform(rdd => newRDD)
}
/**
* Tail recursion for merging adjacent ReferenceRegions with the same value.
*
* @param iter partition iterator of ReferenceRegion and coverage values.
* @param lastCoverage the last coverage from a sorted Iterator that has been considered to merge.
* @param condensed Condensed iterator of iter with adjacent regions with the same value merged.
* @return merged tuples of adjacent ReferenceRegions and coverage.
*/
@tailrec private def collapse(iter: Iterator[Coverage],
lastCoverage: Coverage,
condensed: List[Coverage]): Iterator[Coverage] = {
if (!iter.hasNext) {
// if lastCoverage has not yet been added, add to condensed
val nextCondensed =
if (condensed.map(r => ReferenceRegion(r)).filter(_.overlaps(ReferenceRegion(lastCoverage))).isEmpty) {
lastCoverage :: condensed
} else {
condensed
}
nextCondensed.toIterator
} else {
val cov = iter.next
val rr = ReferenceRegion(cov)
val lastRegion = ReferenceRegion(lastCoverage)
val (nextCoverage, nextCondensed) =
if (rr.isAdjacent(lastRegion) && lastCoverage.count == cov.count) {
(Coverage(rr.merge(lastRegion), lastCoverage.count), condensed)
} else {
(cov, lastCoverage :: condensed)
}
collapse(iter, nextCoverage, nextCondensed)
}
}
/**
* Converts CoverageRDD to FeatureRDD.
*
* @return Returns a FeatureRDD from CoverageRDD.
*/
def toFeatures(): FeatureRDD
/**
* Gets coverage overlapping specified ReferenceRegion.
*
* For large ReferenceRegions, base pairs per bin (bpPerBin) can be specified
* to bin together ReferenceRegions of equal size. The coverage of each bin is
* coverage of the first base pair in that bin. Java friendly variant.
*
* @param bpPerBin base pairs per bin, number of bases to combine to one bin.
* @return RDD of Coverage Records.
*/
def coverage(bpPerBin: java.lang.Integer): CoverageRDD = {
val bp: Int = bpPerBin
coverage(bpPerBin = bp)
}
/**
* Gets coverage overlapping specified ReferenceRegion.
* For large ReferenceRegions, base pairs per bin (bpPerBin) can be specified
* to bin together ReferenceRegions of equal size. The coverage of each bin is
* coverage of the first base pair in that bin.
*
* @param bpPerBin base pairs per bin, number of bases to combine to one bin.
* @return RDD of Coverage Records.
*/
def coverage(bpPerBin: Int = 1): CoverageRDD = {
val flattened = flatten()
if (bpPerBin == 1) {
flattened // no binning, return raw results
} else {
// subtract region.start to shift mod to start of ReferenceRegion
val newRDD = flattened.rdd.filter(r => r.start % bpPerBin == 0)
flattened.transform(rdd => newRDD)
}
}
/**
* Gets coverage overlapping specified ReferenceRegion.
*
* For large ReferenceRegions, base pairs per bin (bpPerBin) can be specified
* to bin together ReferenceRegions of equal size. The coverage of each bin is
* the mean coverage over all base pairs in that bin. Java friendly variant.
*
* @param bpPerBin base pairs per bin, number of bases to combine to one bin.
* @return RDD of Coverage Records.
*/
def aggregatedCoverage(bpPerBin: java.lang.Integer): CoverageRDD = {
val bp: Int = bpPerBin
aggregatedCoverage(bpPerBin = bp)
}
/**
* Gets coverage overlapping specified ReferenceRegion.
*
* For large ReferenceRegions, base pairs per bin (bpPerBin) can be specified
* to bin together ReferenceRegions of equal size. The coverage of each bin is
* the mean coverage over all base pairs in that bin.
*
* @param bpPerBin base pairs per bin, number of bases to combine to one bin.
* @return RDD of Coverage Records.
*/
def aggregatedCoverage(bpPerBin: Int = 1): CoverageRDD = {
val flattened = flatten()
def reduceFn(a: (Double, Int), b: (Double, Int)): (Double, Int) = {
(a._1 + b._1, a._2 + b._2)
}
if (bpPerBin == 1) {
flattened // no binning, return raw results
} else {
val newRDD = flattened.rdd
.keyBy(r => {
// key coverage by binning start site mod bpPerbin
// subtract region.start to shift mod to start of ReferenceRegion
val start = r.start - (r.start % bpPerBin)
ReferenceRegion(r.contigName, start, start + bpPerBin)
}).mapValues(r => (r.count, 1))
.reduceByKey(reduceFn)
.map(r => {
// compute average coverage in bin
Coverage(r._1.referenceName, r._1.start, r._1.end, r._2._1 / r._2._2)
})
flattened.transform(rdd => newRDD)
}
}
/**
* Gets sequence of ReferenceRegions from Coverage element.
* Since coverage maps directly to a single genomic region, this method will always
* return a Seq of exactly one ReferenceRegion.
*
* @param elem The Coverage to get an underlying region for.
* @return Sequence of ReferenceRegions extracted from Coverage.
*/
protected def getReferenceRegions(elem: Coverage): Seq[ReferenceRegion] = {
Seq(ReferenceRegion(elem.contigName, elem.start, elem.end))
}
/**
* @param newRdd The RDD to replace the underlying RDD with.
* @return Returns a new CoverageRDD with the underlying RDD replaced.
*/
protected def replaceRdd(newRdd: RDD[Coverage],
newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): CoverageRDD = {
RDDBoundCoverageRDD(newRdd, sequences, newPartitionMap)
}
/**
* Gets flattened RDD of coverage, with coverage mapped to a ReferenceRegion at each base pair.
*
* @return CoverageRDD of flattened Coverage records.
*/
def flatten(): CoverageRDD = {
transform(rdd => flatMapCoverage(rdd))
}
/**
* Flat maps coverage into ReferenceRegion and counts for each base pair.
*
* @param rdd RDD of Coverage.
* @return RDD of flattened Coverage.
*/
private def flatMapCoverage(rdd: RDD[Coverage]): RDD[Coverage] = {
rdd.flatMap(r => {
val positions = r.start until r.end
positions.map(n => Coverage(r.contigName, n, n + 1, r.count))
})
}
}