From c0e07d86da07ebf60605e0d5af84054d6699c090 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Tue, 6 Jun 2017 12:06:37 -0700 Subject: [PATCH 01/13] Creating set theory package --- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 19 +- .../adam/rdd/settheory/SetTheory.scala | 194 ++++++++++++++++++ .../{ => settheory}/ShuffleRegionJoin.scala | 7 +- .../rdd/InnerShuffleRegionJoinSuite.scala | 1 + .../rdd/LeftOuterShuffleRegionJoinSuite.scala | 1 + .../RightOuterShuffleRegionJoinSuite.scala | 1 + 6 files changed, 200 insertions(+), 23 deletions(-) create mode 100644 adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala rename adam-core/src/main/scala/org/bdgenomics/adam/rdd/{ => settheory}/ShuffleRegionJoin.scala (98%) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index f7212b7ca9..dbd52353f6 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -34,6 +34,7 @@ import org.bdgenomics.adam.models.{ ReferenceRegion, SequenceDictionary } +import org.bdgenomics.adam.rdd.settheory._ import org.bdgenomics.formats.avro.{ Contig, RecordGroup => RecordGroupMetadata, @@ -66,7 +67,6 @@ private[rdd] object GenomicRDD { * Replaces file references in a command. * * @see pipe - * * @param cmd Command to split and replace references in. * @param files List of paths to files. * @return Returns a split up command string, with file paths subbed in. @@ -203,7 +203,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * if LENIENT, log a warning, otherwise does nothing. * * @throws IllegalArgumentException If stringency is STRICT. - * * @param message The error or warning message. * @param stringency The validation stringency. */ @@ -226,7 +225,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @param partitions The number of partitions for the new RDD. * @param stringency The level of ValidationStringency to enforce. * @return Returns a new RDD containing sorted data. - * * @note Uses ValidationStringency to handle unaligned or where objects align * to multiple positions. * @see sortLexicographically @@ -278,7 +276,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @param storageLevel The level at which to persist the resulting RDD. * @param stringency The level of ValidationStringency to enforce. * @return Returns a new RDD containing sorted data. - * * @note Uses ValidationStringency to handle data that is unaligned or where objects * align to multiple positions. * @see sort @@ -343,7 +340,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * in the environment for the newly created process. Default is empty. * @param flankSize Number of bases to flank each command invocation by. * @return Returns a new GenomicRDD of type Y. - * * @tparam X The type of the record created by the piped command. * @tparam Y A GenomicRDD containing X's. * @tparam V The InFormatter to use for formatting the data being piped to the @@ -545,7 +541,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @param genomicRdd The right RDD in the join. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space. - * * @see broadcastRegionJoinAgainst */ def broadcastRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(T, X), Z]]( @@ -575,11 +570,9 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * * @note This function differs from other region joins as it treats the calling RDD * as the right side of the join, and not the left. - * * @param broadcastTree The data on the left side of the join. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space. - * * @see broadcastRegionJoin */ def broadcastRegionJoinAgainst[X, Z <: GenomicRDD[(X, T), Z]]( @@ -610,7 +603,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space, and all keys from the * right RDD that did not overlap a key in the left RDD. - * * @see rightOuterBroadcastRegionJoin */ def rightOuterBroadcastRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Option[T], X), Z]]( @@ -645,11 +637,9 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * * @note This function differs from other region joins as it treats the calling RDD * as the right side of the join, and not the left. - * * @param broadcastTree The data on the left side of the join. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space. - * * @see rightOuterBroadcastRegionJoin */ def rightOuterBroadcastRegionJoinAgainst[X, Z <: GenomicRDD[(Option[X], T), Z]]( @@ -679,7 +669,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @param genomicRdd The right RDD in the join. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space. - * * @see broadcastRegionJoinAgainstAndGroupByRight */ def broadcastRegionJoinAndGroupByRight[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Iterable[T], X), Z]](genomicRdd: GenomicRDD[X, Y])( @@ -708,11 +697,9 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * * @note This function differs from other region joins as it treats the calling RDD * as the right side of the join, and not the left. - * * @param broadcastTree The data on the left side of the join. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space. - * * @see broadcastRegionJoinAndGroupByRight */ def broadcastRegionJoinAgainstAndGroupByRight[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Iterable[X], T), Z]]( @@ -743,7 +730,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space, and all keys from the * right RDD that did not overlap a key in the left RDD. - * * @see rightOuterBroadcastRegionJoinAgainstAndGroupByRight */ def rightOuterBroadcastRegionJoinAndGroupByRight[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Iterable[T], X), Z]](genomicRdd: GenomicRDD[X, Y])( @@ -777,11 +763,9 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * * @note This function differs from other region joins as it treats the calling RDD * as the right side of the join, and not the left. - * * @param broadcastTree The data on the left side of the join. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space. - * * @see rightOuterBroadcastRegionJoinAndGroupByRight */ def rightOuterBroadcastRegionJoinAgainstAndGroupByRight[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Iterable[X], T), Z]]( @@ -1067,7 +1051,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * of the records of this.rdd. It requires a pass through the co-located * RDD to get the correct partition(s) for each record. It will assign a * record to multiple partitions if necessary. - * * @param rddToCoPartitionWith The rdd to copartition to. * @return The newly repartitioned rdd. */ diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala new file mode 100644 index 0000000000..15769bdd37 --- /dev/null +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala @@ -0,0 +1,194 @@ +package org.bdgenomics.adam.rdd.settheory + +import org.apache.spark.rdd.RDD +import org.bdgenomics.adam.models.ReferenceRegion +import scala.collection.mutable.ListBuffer +import scala.reflect.ClassTag + +sealed abstract class SetTheory extends Serializable { + + protected val partitionMap: Array[Option[(ReferenceRegion, ReferenceRegion)]] + protected val threshold: Long + + /** + * The primitive operation for intersection, computes the primitive of the + * two regions. + * + * @param firstRegion The first region for the primitive. + * @param secondRegion The second region for the primitive. + * @param distanceThreshold The threshold for the primitive. + * @return The computed primitive for the two regions. + */ + protected def primitive(firstRegion: ReferenceRegion, + secondRegion: ReferenceRegion, + distanceThreshold: Long = 0L): ReferenceRegion + + /** + * The condition that should be met in order for the primitive to be + * computed. + * + * @param firstRegion The region to test against. + * @param secondRegion The region to test. + * @param distanceThreshold The threshold for the primitive. + * @return True if the threshold requirement is met. + * False if the threshold requirement is not met. + */ + protected def condition(firstRegion: ReferenceRegion, + secondRegion: ReferenceRegion, + distanceThreshold: Long = 0L): Boolean + + /** + * Gets the partition bounds from a ReferenceRegion keyed Iterator + * + * @param iter The data on a given partition. ReferenceRegion keyed + * @return The bounds of the ReferenceRegions on that partition, in an Iterator + */ + protected def getRegionBoundsFromPartition[X](iter: Iterator[(ReferenceRegion, X)]): Iterator[Option[(ReferenceRegion, ReferenceRegion)]] = { + if (iter.isEmpty) { + // This means that there is no data on the partition, so we have no bounds + Iterator(None) + } else { + val firstRegion = iter.next + val lastRegion = + if (iter.hasNext) { + // we have to make sure we get the full bounds of this partition, this + // includes any extremely long regions. we include the firstRegion for + // the case that the first region is extremely long + (iter ++ Iterator(firstRegion)).maxBy(f => (f._1.referenceName, f._1.end, f._1.start)) + // only one record on this partition, so this is the extent of the bounds + } else { + firstRegion + } + Iterator(Some((firstRegion._1, lastRegion._1))) + } + } +} + +/** + * The parent class for all inter-collection set theory operations. + * + * @tparam T The left side row data. + * @tparam U The right side row data. + * @tparam RT The return type for the left side row data. + * @tparam RU The return type for the right side row data. + */ +abstract class SetTheoryBetweenCollections[T: ClassTag, U: ClassTag, RT, RU] extends SetTheory { + + protected val leftRdd: RDD[(ReferenceRegion, T)] + protected val rightRdd: RDD[(ReferenceRegion, U)] + + // Holds candidates from the right side. + private val cache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty[(ReferenceRegion, U)] + + /** + * Processes all hits from the cache and creates an iterator for the current + * left based on the primitive operation. + * + * @param current The current left row, keyed by the ReferenceRegion. + * @param cache The cache of potential hits. + * @return An iterator containing all processed hits. + */ + protected def processHits(current: (ReferenceRegion, T), + cache: ListBuffer[(ReferenceRegion, U)]): Iterator[(ReferenceRegion, (RT, RU))] + + /** + * The condition by which a candidate is removed from the cache. + * + * @see pruneCache + * @param cachedRegion The current region in the cache. + * @param to The region that is compared against. + * @return True for regions that should be removed. + * False for all regions that should remain in the cache. + */ + protected def pruneCacheCondition(cachedRegion: ReferenceRegion, + to: ReferenceRegion): Boolean + + /** + * The condition by which a candidate region is added to the cache. + * + * @see advanceCache + * @param candidateRegion The current candidate region. + * @param until The region to compare against. + * @return True for all regions to be added to the cache. + * False for regions that should not be added to the cache. + */ + protected def advanceCacheCondition(candidateRegion: ReferenceRegion, + until: ReferenceRegion): Boolean + + /** + * Handles the situation where the left or right iterator is empty. + * + * @param left The left iterator. + * @param right The right iterator. + * @return The formatted resulting RDD. + */ + protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] + + /** + * Prunes the cache based on the condition set in pruneCacheCondition. + * + * @see pruneCacheCondition + * @param to The region to prune against. + */ + private def pruneCache(to: ReferenceRegion) = { + + cache.trimStart({ + val index = cache.indexWhere(f => !pruneCacheCondition(f._1, to)) + if (index <= 0) { + 0 + } else { + index + } + }) + } + + /** + * Advances the cache based on the condition set in advanceCacheCondition + * + * @see advanceCacheCondition + * @param right The right buffered iterator to pull from. + * @param until The region to compare against. + */ + private def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], + until: ReferenceRegion) = { + while (right.hasNext && advanceCacheCondition(right.head._1, until)) { + cache += right.next + } + } + + /** + * Computes the set theory primitive for the two RDDs. + * + * @return An RDD resulting from the primitive operation. + */ + def compute(): RDD[(RT, RU)] = { + leftRdd.zipPartitions(rightRdd)(makeIterator) + } + + /** + * Computes the set theory primitive for the two Iterators on each partition. + * + * @see processHits + * @param leftIter The iterator for the left side of the primitive. + * @param rightIter The iterator for the right side of the primitive. + * @return The resulting Iterator based on the primitive operation. + */ + protected def makeIterator(leftIter: Iterator[(ReferenceRegion, T)], + rightIter: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] = { + + if(leftIter.isEmpty || rightIter.isEmpty) { + emptyFn(leftIter, rightIter) + } else { + val leftBuffered = leftIter.buffered + val rightBuffered = rightIter.buffered + + leftBuffered.flatMap(f => { + val (currentRegion, _) = f + advanceCache(rightBuffered, currentRegion) + pruneCache(currentRegion) + processHits(f, cache) + }) + } + } +} diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala similarity index 98% rename from adam-core/src/main/scala/org/bdgenomics/adam/rdd/ShuffleRegionJoin.scala rename to adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index 9d9e434323..33209cfe7f 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd +package org.bdgenomics.adam.rdd.settheory import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion @@ -33,7 +33,7 @@ import scala.reflect.ClassTag * join. */ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] - extends RegionJoin[T, U, RT, RU] { + extends SetTheoryBetweenCollections[T, U, RT, RU] { protected def advanceCache(cache: SetTheoryCache[U, RT, RU], right: BufferedIterator[(ReferenceRegion, U)], @@ -44,8 +44,6 @@ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] currentLeft: T): Iterable[(RT, RU)] protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] - protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] protected val leftRdd: RDD[(ReferenceRegion, T)] protected val rightRdd: RDD[(ReferenceRegion, U)] @@ -374,7 +372,6 @@ sealed trait VictimlessSortedIntervalPartitionJoin[T, U, RT, RU] * and reassigned the pointers every time. We fixed this by using trimStart() * and ++=() to improve performance. Overall, we see roughly 25% improvement * in runtime by doing things this way. - * * @param cache The cache for this partition. * @param to The next region in the left iterator. */ diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala index 80a013cc15..c6df529d92 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala @@ -18,6 +18,7 @@ package org.bdgenomics.adam.rdd import org.bdgenomics.adam.models.{ ReferenceRegion, SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.rdd.settheory.InnerShuffleRegionJoin import org.bdgenomics.adam.util.ADAMFunSuite import org.bdgenomics.formats.avro.{ AlignmentRecord, Contig } diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala index e881bd6fec..fa192e2add 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala @@ -23,6 +23,7 @@ import org.bdgenomics.adam.models.{ SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.rdd.settheory.LeftOuterShuffleRegionJoin import org.bdgenomics.formats.avro.AlignmentRecord class LeftOuterShuffleRegionJoinSuite(partitionMap: Seq[Option[(ReferenceRegion, ReferenceRegion)]]) diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala index e3ba2fd649..4c148aa405 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala @@ -23,6 +23,7 @@ import org.bdgenomics.adam.models.{ SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.rdd.settheory.RightOuterShuffleRegionJoin import org.bdgenomics.formats.avro.AlignmentRecord class RightOuterShuffleRegionJoinSuite(partitionMap: Seq[Option[(ReferenceRegion, ReferenceRegion)]]) From 966be935007fca4291a00ca89dd74e46b5fad3e5 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Fri, 9 Jun 2017 09:58:07 -0700 Subject: [PATCH 02/13] Added Closest implementation --- .../adam/models/ReferenceRegion.scala | 7 + .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 87 ++- .../adam/rdd/settheory/Closest.scala | 190 ++++++ .../adam/rdd/settheory/SetTheory.scala | 374 ++++++++---- .../rdd/settheory/ShuffleRegionJoin.scala | 563 ++++++------------ .../resources/intersect_with_overlap_00.bed | 21 + .../resources/intersect_with_overlap_01.bed | 5 + .../adam/rdd/SortedGenomicRDDSuite.scala | 6 +- .../adam/rdd/settheory/ClosestSuite.scala | 47 ++ 9 files changed, 768 insertions(+), 532 deletions(-) create mode 100644 adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala create mode 100644 adam-core/src/test/resources/intersect_with_overlap_00.bed create mode 100644 adam-core/src/test/resources/intersect_with_overlap_01.bed create mode 100644 adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala 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 cbbfc6b26e..61de43c044 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 @@ -84,6 +84,13 @@ object ReferenceRegion { implicit def orderingForPositions = RegionOrdering implicit def orderingForOptionalPositions = OptionalRegionOrdering + /** + * Creates an empty ReferenceRegion. + * + * @return An empty ReferenceRegion. + */ + private[adam] val empty: ReferenceRegion = ReferenceRegion("", 0L, 0L) + /** * Creates a reference region that starts at the beginning of a contig. * diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index dbd52353f6..00d8cbc9f0 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -476,7 +476,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { protected def getReferenceRegions(elem: T): Seq[ReferenceRegion] - protected def flattenRddByRegions(): RDD[(ReferenceRegion, T)] = { + private[rdd] def flattenRddByRegions(): RDD[(ReferenceRegion, T)] = { rdd.flatMap(elem => { getReferenceRegions(elem).map(r => (r, elem)) }) @@ -792,19 +792,20 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { */ private def prepareForShuffleRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(T, X), Z]]( genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int] = None)( + optPartitions: Option[Int], + threshold: Long)( implicit tTag: ClassTag[T], xTag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { val partitions = optPartitions.getOrElse(this.rdd.partitions.length) val (leftRdd, rightRdd) = (isSorted, genomicRdd.isSorted) match { - case (true, _) => (this, genomicRdd.copartitionByReferenceRegion(this)) - case (false, true) => (copartitionByReferenceRegion(genomicRdd), genomicRdd) + case (true, _) => (this, genomicRdd.copartitionByReferenceRegion(this, threshold)) + case (false, true) => (copartitionByReferenceRegion(genomicRdd, threshold), genomicRdd) case (false, false) => { val repartitionedRdd = sortLexicographically(storePartitionMap = true, partitions = partitions) - (repartitionedRdd, genomicRdd.copartitionByReferenceRegion(repartitionedRdd)) + (repartitionedRdd, genomicRdd.copartitionByReferenceRegion(repartitionedRdd, threshold)) } } (leftRdd.flattenRddByRegions(), rightRdd.flattenRddByRegions()) @@ -820,24 +821,30 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * overlap a value from the other RDD are dropped. * * @param genomicRdd The right RDD in the join. + * @param optPartitions Optionally sets the number of output partitions. If + * None, the number of partitions on the resulting RDD does not change. + * @param threshold The threshold for the join. In the case of 0, the join + * requires overlap to join. Otherwise records must be within the threshold + * distance to be joined. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space. */ def shuffleRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(T, X), Z]]( genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int] = None)( + optPartitions: Option[Int] = None, + threshold: Long = 0L)( implicit tTag: ClassTag[T], xTag: ClassTag[X], txTag: ClassTag[(T, X)]): GenomicRDD[(T, X), Z] = InnerShuffleJoin.time { val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions) + prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, X)]( - InnerShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin) + InnerShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) .compute(), combinedSequences, kv => { @@ -857,25 +864,31 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * RDD, it will be paired with a `None` in the product of the join. * * @param genomicRdd The right RDD in the join. + * @param optPartitions Optionally sets the number of output partitions. If + * None, the number of partitions on the resulting RDD does not change. + * @param threshold The threshold for the join. In the case of 0, the join + * requires overlap to join. Otherwise records must be within the threshold + * distance to be joined. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space, and all keys from the * right RDD that did not overlap a key in the left RDD. */ def rightOuterShuffleRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Option[T], X), Z]]( genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int] = None)( + optPartitions: Option[Int] = None, + threshold: Long = 0L)( implicit tTag: ClassTag[T], xTag: ClassTag[X], otxTag: ClassTag[(Option[T], X)]): GenomicRDD[(Option[T], X), Z] = RightOuterShuffleJoin.time { val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions) + prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], X)]( - RightOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin) + RightOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) .compute(), combinedSequences, kv => { @@ -896,25 +909,31 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * RDD, it will be paired with a `None` in the product of the join. * * @param genomicRdd The right RDD in the join. + * @param optPartitions Optionally sets the number of output partitions. If + * None, the number of partitions on the resulting RDD does not change. + * @param threshold The threshold for the join. In the case of 0, the join + * requires overlap to join. Otherwise records must be within the threshold + * distance to be joined. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space, and all keys from the * left RDD that did not overlap a key in the right RDD. */ def leftOuterShuffleRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(T, Option[X]), Z]]( genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int] = None)( + optPartitions: Option[Int] = None, + threshold: Long = 0L)( implicit tTag: ClassTag[T], xTag: ClassTag[X], toxTag: ClassTag[(T, Option[X])]): GenomicRDD[(T, Option[X]), Z] = LeftOuterShuffleJoin.time { val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions) + prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, Option[X])]( - LeftOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin) + LeftOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) .compute(), combinedSequences, kv => { @@ -934,25 +953,31 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * a `None` in the product of the join. * * @param genomicRdd The right RDD in the join. + * @param optPartitions Optionally sets the number of output partitions. If + * None, the number of partitions on the resulting RDD does not change. + * @param threshold The threshold for the join. In the case of 0, the join + * requires overlap to join. Otherwise records must be within the threshold + * distance to be joined. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space, and values that did not * overlap will be paired with a `None`. */ def fullOuterShuffleRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Option[T], Option[X]), Z]]( genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int] = None)( + optPartitions: Option[Int] = None, + threshold: Long = 0L)( implicit tTag: ClassTag[T], xTag: ClassTag[X], otoxTag: ClassTag[(Option[T], Option[X])]): GenomicRDD[(Option[T], Option[X]), Z] = FullOuterShuffleJoin.time { val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions) + prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], Option[X])]( - FullOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin) + FullOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) .compute(), combinedSequences, kv => { @@ -973,25 +998,31 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * we group all values by the left item in the RDD. * * @param genomicRdd The right RDD in the join. + * @param optPartitions Optionally sets the number of output partitions. If + * None, the number of partitions on the resulting RDD does not change. + * @param threshold The threshold for the join. In the case of 0, the join + * requires overlap to join. Otherwise records must be within the threshold + * distance to be joined. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space, grouped together by * the value they overlapped in the left RDD.. */ def shuffleRegionJoinAndGroupByLeft[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(T, Iterable[X]), Z]]( genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int] = None)( + optPartitions: Option[Int] = None, + threshold: Long = 0L)( implicit tTag: ClassTag[T], xTag: ClassTag[X], tixTag: ClassTag[(T, Iterable[X])]): GenomicRDD[(T, Iterable[X]), Z] = ShuffleJoinAndGroupByLeft.time { val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions) + prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, Iterable[X])]( - InnerShuffleRegionJoinAndGroupByLeft[T, X](leftRddToJoin, rightRddToJoin) + InnerShuffleRegionJoinAndGroupByLeft[T, X](leftRddToJoin, rightRddToJoin, None, threshold) .compute(), combinedSequences, kv => { @@ -1015,6 +1046,9 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @param genomicRdd The right RDD in the join. * @param optPartitions Optionally sets the number of output partitions. If * None, the number of partitions on the resulting RDD does not change. + * @param threshold The threshold for the join. In the case of 0, the join + * requires overlap to join. Otherwise records must be within the threshold + * distance to be joined. * @return Returns a new genomic RDD containing all pairs of keys that * overlapped in the genomic coordinate space, grouped together by * the value they overlapped in the left RDD, and all values from the @@ -1022,19 +1056,20 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { */ def rightOuterShuffleRegionJoinAndGroupByLeft[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(Option[T], Iterable[X]), Z]]( genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int] = None)( + optPartitions: Option[Int] = None, + threshold: Long = 0L)( implicit tTag: ClassTag[T], xTag: ClassTag[X], otixTag: ClassTag[(Option[T], Iterable[X])]): GenomicRDD[(Option[T], Iterable[X]), Z] = RightOuterShuffleJoinAndGroupByLeft.time { val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions) + prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], Iterable[X])]( - RightOuterShuffleRegionJoinAndGroupByLeft[T, X](leftRddToJoin, rightRddToJoin) + RightOuterShuffleRegionJoinAndGroupByLeft[T, X](leftRddToJoin, rightRddToJoin, None, threshold) .compute(), combinedSequences, kv => { @@ -1052,10 +1087,12 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * RDD to get the correct partition(s) for each record. It will assign a * record to multiple partitions if necessary. * @param rddToCoPartitionWith The rdd to copartition to. + * @param threshold The threshold for the join. * @return The newly repartitioned rdd. */ private[rdd] def copartitionByReferenceRegion[X, Y <: GenomicRDD[X, Y]]( - rddToCoPartitionWith: GenomicRDD[X, Y])(implicit tTag: ClassTag[T], xTag: ClassTag[X]): U = { + rddToCoPartitionWith: GenomicRDD[X, Y], + threshold: Long)(implicit tTag: ClassTag[T], xTag: ClassTag[X]): U = { // if the other RDD is not sorted, we can't guarantee proper copartition assert(rddToCoPartitionWith.isSorted, @@ -1108,7 +1145,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { referenceRegionKeyedGenomicRDD.mapPartitions(iter => { iter.flatMap(f => { - val intervals = partitionMapIntervals.get(f._1, requireOverlap = false) + val intervals = partitionMapIntervals.get(f._1.pad(threshold), requireOverlap = false) intervals.map(g => ((f._1, g._2), f._2)) }) }, preservesPartitioning = true) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala new file mode 100644 index 0000000000..2973f18d42 --- /dev/null +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala @@ -0,0 +1,190 @@ +package org.bdgenomics.adam.rdd.settheory + +import org.apache.spark.rdd.RDD +import org.bdgenomics.adam.models.ReferenceRegion +import org.bdgenomics.adam.rdd.ManualRegionPartitioner +import org.bdgenomics.utils.interval.array.IntervalArray +import scala.reflect.ClassTag + +/** + * A trait describing closest implementations that are based on sort-merge. + * + * @tparam T The type of the left records. + * @tparam U The type of the right records. + * @tparam RT The resulting type of the left after the join. + * @tparam RU The resulting type of the right after the join. + */ +sealed abstract class Closest[T: ClassTag, U: ClassTag, RT, RU] + extends SetTheoryBetweenCollections[T, U, RT, RU] + with SetTheoryPrimitive { + + var currentClosest: ReferenceRegion = ReferenceRegion.empty + + override protected def condition(firstRegion: ReferenceRegion, + secondRegion: ReferenceRegion, + threshold: Long = 0L): Boolean = { + firstRegion.unstrandedDistance(currentClosest) + .exists(_ == firstRegion.unstrandedDistance(secondRegion).getOrElse(Long.MaxValue)) + } + + override protected def pruneCacheCondition(cachedRegion: ReferenceRegion, + to: ReferenceRegion): Boolean = { + if (cachedRegion.referenceName != to.referenceName) { + true + } else { + to.unstrandedDistance(cachedRegion).get > to.unstrandedDistance(currentClosest).getOrElse(Long.MaxValue) + } + } + + override protected def advanceCacheCondition(candidateRegion: ReferenceRegion, + until: ReferenceRegion): Boolean = { + + if (candidateRegion.referenceName != until.referenceName) { + false + } else if (until.referenceName != currentClosest.referenceName || + until.unstrandedDistance(candidateRegion).get <= + until.unstrandedDistance(currentClosest).getOrElse(Long.MaxValue)) { + + currentClosest = candidateRegion + true + } else { + false + } + } + + override protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) = { + + val (partitionMap: Array[Option[(ReferenceRegion, ReferenceRegion)]], + preparedLeftRdd: RDD[(ReferenceRegion, T)]) = { + if (optPartitionMap.isDefined) { + (optPartitionMap.get, leftRdd) + } else { + val sortedLeft = leftRdd.sortByKey() + val newPartitionMap = + sortedLeft.mapPartitions(getRegionBoundsFromPartition) + .collect + (newPartitionMap, sortedLeft) + } + } + + val adjustedPartitionMapWithIndex = partitionMap + // the zipWithIndex gives us the destination partition ID + .zipWithIndex + .filter(_._1.nonEmpty) + .map(f => (f._1.get, f._2)) + .map(g => { + // in the case where we span multiple referenceNames + if (g._1._1.referenceName != g._1._2.referenceName) { + // create a ReferenceRegion that goes to the end of the chromosome + (ReferenceRegion( + g._1._1.referenceName, + g._1._1.start, + g._1._1.end), + g._2) + } else { + // otherwise we just have the ReferenceRegion span from partition + // start to end + (ReferenceRegion( + g._1._1.referenceName, + g._1._1.start, + g._1._2.end), + g._2) + } + }) + + val partitionMapIntervals = IntervalArray( + adjustedPartitionMapWithIndex, + adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, + sorted = true) + + val assignedRightRdd = { + val firstPass = rightRdd.mapPartitions(iter => { + iter.flatMap(f => { + val rangeOfHits = partitionMapIntervals.get(f._1, requireOverlap = false) + rangeOfHits.map(g => ((f._1, g._2), f._2)) + }) + }, preservesPartitioning = true) + + val partitionsWithoutData = + partitionMap.indices.filterNot(firstPass.map(_._1._2).distinct().collect.contains) + + val partitionsToSend = partitionsWithoutData.foldLeft(List.empty[List[Int]])((b, a) => { + if (b.isEmpty) { + List(List(a)) + } else if (a == b.last.last + 1) { + b.dropRight(1).:+(b.last.:+(a)) + } else { + b.:+(List(a)) + } + }).flatMap(f => List((f.head - 1, f.length), (f.last + 1, -1 * f.length))) + + firstPass.flatMap(f => { + val index = partitionsToSend.indexWhere(_._1 == f._1._2) + if (index < 0) { + List(f) + } else { + if (partitionsToSend(index)._2 < 0) { + (partitionsToSend(index)._2 to 0) + .map(g => ((f._1._1, f._1._2 + g), f._2)) + } else { + (0 to partitionsToSend(index)._2) + .map(g => ((f._1._1, f._1._2 + g), f._2)) ++ { + if (index == partitionsToSend.lastIndexWhere(_._1 == f._1._2)) { + List() + } else { + val endIndex = partitionsToSend.lastIndexWhere(_._1 == f._1._2) + (partitionsToSend(endIndex)._2 to -1) + .map(g => ((f._1._1, f._1._2 + g), f._2)) + } + } + } + } + }) + } + + val preparedRightRdd = + assignedRightRdd + .repartitionAndSortWithinPartitions( + ManualRegionPartitioner(partitionMap.length)) + // return to an RDD[(ReferenceRegion, T)], removing the partition ID + .map(f => (f._1._1, f._2)) + + (preparedLeftRdd, preparedRightRdd) + } +} + +/** + * Perform a sort-merge closest operation. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param optPartitionMap An optional partition map defining the left RDD + * partition bounds. + * @param threshold The maximum distance allowed for the closest. + * @tparam T The type of the left records. + * @tparam U The type of the right records. + */ +case class ShuffleClosestRegion[T: ClassTag, U: ClassTag]( + protected val leftRdd: RDD[(ReferenceRegion, T)], + protected val rightRdd: RDD[(ReferenceRegion, U)], + protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]], + protected val threshold: Long = Long.MaxValue) + extends Closest[T, U, T, Iterable[U]] + with VictimlessSetTheoryBetweenCollections[T, U, T, Iterable[U]] { + + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Iterable[U])] = { + + // if the left iterator is not empty, we have failed to correctly + // partition the data. the right iterator is only allowed to be empty + // when the left iterator is empty, but we don't care if there's data + // on the right side if there's no data on the left. + require(left.isEmpty) + Iterator.empty + } + + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, Iterable[U])] = { + Iterable((currentLeft._2, iter.map(_._2))) + } +} diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala index 15769bdd37..215c21b310 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala @@ -3,47 +3,38 @@ package org.bdgenomics.adam.rdd.settheory import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion import scala.collection.mutable.ListBuffer -import scala.reflect.ClassTag -sealed abstract class SetTheory extends Serializable { +private[settheory] sealed abstract class SetTheory extends Serializable { - protected val partitionMap: Array[Option[(ReferenceRegion, ReferenceRegion)]] protected val threshold: Long + protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] /** - * The primitive operation for intersection, computes the primitive of the - * two regions. - * - * @param firstRegion The first region for the primitive. - * @param secondRegion The second region for the primitive. - * @param distanceThreshold The threshold for the primitive. - * @return The computed primitive for the two regions. - */ - protected def primitive(firstRegion: ReferenceRegion, - secondRegion: ReferenceRegion, - distanceThreshold: Long = 0L): ReferenceRegion - - /** - * The condition that should be met in order for the primitive to be - * computed. - * - * @param firstRegion The region to test against. - * @param secondRegion The region to test. - * @param distanceThreshold The threshold for the primitive. - * @return True if the threshold requirement is met. - * False if the threshold requirement is not met. - */ + * The condition that should be met in order for the primitive to be + * computed. + * + * @param firstRegion The region to test against. + * @param secondRegion The region to test. + * @param distanceThreshold The threshold for the primitive. + * @return True if the threshold requirement is met. + * False if the threshold requirement is not met. + */ protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, distanceThreshold: Long = 0L): Boolean +} + +private[settheory] trait SetTheoryPrimitive extends SetTheory { /** - * Gets the partition bounds from a ReferenceRegion keyed Iterator - * - * @param iter The data on a given partition. ReferenceRegion keyed - * @return The bounds of the ReferenceRegions on that partition, in an Iterator - */ - protected def getRegionBoundsFromPartition[X](iter: Iterator[(ReferenceRegion, X)]): Iterator[Option[(ReferenceRegion, ReferenceRegion)]] = { + * Gets the partition bounds from a ReferenceRegion keyed Iterator + * + * @param iter The data on a given partition. ReferenceRegion keyed + * @return The bounds of the ReferenceRegions on that partition, in an Iterator + */ + protected def getRegionBoundsFromPartition[X]( + iter: Iterator[(ReferenceRegion, X)]): Iterator[Option[(ReferenceRegion, ReferenceRegion)]] = { + if (iter.isEmpty) { // This means that there is no data on the partition, so we have no bounds Iterator(None) @@ -65,53 +56,49 @@ sealed abstract class SetTheory extends Serializable { } /** - * The parent class for all inter-collection set theory operations. - * - * @tparam T The left side row data. - * @tparam U The right side row data. - * @tparam RT The return type for the left side row data. - * @tparam RU The return type for the right side row data. - */ -abstract class SetTheoryBetweenCollections[T: ClassTag, U: ClassTag, RT, RU] extends SetTheory { + * The parent class for all inter-collection set theory operations. + * + * @tparam T The left side row data. + * @tparam U The right side row data. + * @tparam RT The return type for the left side row data. + * @tparam RU The return type for the right side row data. + */ +private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] extends SetTheory { protected val leftRdd: RDD[(ReferenceRegion, T)] protected val rightRdd: RDD[(ReferenceRegion, U)] - // Holds candidates from the right side. - private val cache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty[(ReferenceRegion, U)] - /** - * Processes all hits from the cache and creates an iterator for the current - * left based on the primitive operation. - * - * @param current The current left row, keyed by the ReferenceRegion. - * @param cache The cache of potential hits. - * @return An iterator containing all processed hits. - */ - protected def processHits(current: (ReferenceRegion, T), - cache: ListBuffer[(ReferenceRegion, U)]): Iterator[(ReferenceRegion, (RT, RU))] + * Post process and format the hits for a given left record. + * + * @param currentLeft The current left record. + * @param iter The iterable of hits. + * @return The post processed hits. + */ + protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(RT, RU)] /** - * The condition by which a candidate is removed from the cache. - * - * @see pruneCache - * @param cachedRegion The current region in the cache. - * @param to The region that is compared against. - * @return True for regions that should be removed. - * False for all regions that should remain in the cache. - */ + * The condition by which a candidate is removed from the cache. + * + * @see pruneCache + * @param cachedRegion The current region in the cache. + * @param to The region that is compared against. + * @return True for regions that should be removed. + * False for all regions that should remain in the cache. + */ protected def pruneCacheCondition(cachedRegion: ReferenceRegion, to: ReferenceRegion): Boolean /** - * The condition by which a candidate region is added to the cache. - * - * @see advanceCache - * @param candidateRegion The current candidate region. - * @param until The region to compare against. - * @return True for all regions to be added to the cache. - * False for regions that should not be added to the cache. - */ + * The condition by which a candidate region is added to the cache. + * + * @see advanceCache + * @param candidateRegion The current candidate region. + * @param until The region to compare against. + * @return True for all regions to be added to the cache. + * False for regions that should not be added to the cache. + */ protected def advanceCacheCondition(candidateRegion: ReferenceRegion, until: ReferenceRegion): Boolean @@ -126,58 +113,91 @@ abstract class SetTheoryBetweenCollections[T: ClassTag, U: ClassTag, RT, RU] ext right: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] /** - * Prunes the cache based on the condition set in pruneCacheCondition. - * - * @see pruneCacheCondition - * @param to The region to prune against. - */ - private def pruneCache(to: ReferenceRegion) = { - - cache.trimStart({ - val index = cache.indexWhere(f => !pruneCacheCondition(f._1, to)) - if (index <= 0) { - 0 - } else { - index - } - }) - } + * Prunes the cache based on the condition set in pruneCacheCondition. + * + * @see pruneCacheCondition + * + * @param to The region to prune against. + * @param cache The cache for this partition. + */ + protected def pruneCache(to: ReferenceRegion, + cache: SetTheoryCache[U, RT, RU]) /** - * Advances the cache based on the condition set in advanceCacheCondition - * - * @see advanceCacheCondition - * @param right The right buffered iterator to pull from. - * @param until The region to compare against. - */ - private def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], - until: ReferenceRegion) = { - while (right.hasNext && advanceCacheCondition(right.head._1, until)) { - cache += right.next - } - } + * Advances the cache based on the condition set in advanceCacheCondition + * + * @see advanceCacheCondition + * + * @param right The right buffered iterator to pull from. + * @param until The region to compare against. + * @param cache The cache for this partition. + */ + protected def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], + until: ReferenceRegion, + cache: SetTheoryCache[U, RT, RU]) + /** + * Computes all victims for the partition. + * + * @param cache The cache for this partition. + * @param right The right iterator. + * @return The finalized hits for this partition. + */ + protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], + right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] /** - * Computes the set theory primitive for the two RDDs. - * - * @return An RDD resulting from the primitive operation. - */ + * Prepares and partitions the left and right. Makes no assumptions about the + * underlying data. This is particularly important to avoid computation and + * shuffle until the user calls compute(). + * + * @return The prepared and partitioned left and right RDD. + */ + protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) + + /** + * Computes the set theory primitive for the two RDDs. + * + * @return An RDD resulting from the primitive operation. + */ def compute(): RDD[(RT, RU)] = { - leftRdd.zipPartitions(rightRdd)(makeIterator) + val (preparedLeft, preparedRight) = prepare() + preparedLeft.zipPartitions(preparedRight)(makeIterator) + } + + /** + * Processes all hits from the cache and creates an iterator for the current + * left based on the primitive operation. + * + * @param cache The cache of potential hits. + * @param currentLeft The current left record. + * @return An iterator containing all processed hits. + */ + protected def processHits(currentLeft: (ReferenceRegion, T), + cache: SetTheoryCache[U, RT, RU]): Iterable[(RT, RU)] = { + + val (currentLeftRegion, _) = currentLeft + // post processing formats the hits for each individual type of join + postProcessHits(currentLeft, + cache.cache.filter(y => { + // everything that overlaps the left region is a hit + condition(currentLeftRegion, y._1, threshold) + })) } /** - * Computes the set theory primitive for the two Iterators on each partition. - * - * @see processHits - * @param leftIter The iterator for the left side of the primitive. - * @param rightIter The iterator for the right side of the primitive. - * @return The resulting Iterator based on the primitive operation. - */ + * Computes the set theory primitive for the two Iterators on each partition. + * + * @see processHits + * @param leftIter The iterator for the left side of the primitive. + * @param rightIter The iterator for the right side of the primitive. + * @return The resulting Iterator based on the primitive operation. + */ protected def makeIterator(leftIter: Iterator[(ReferenceRegion, T)], rightIter: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] = { - if(leftIter.isEmpty || rightIter.isEmpty) { + val cache = new SetTheoryCache[U, RT, RU] + + if (leftIter.isEmpty || rightIter.isEmpty) { emptyFn(leftIter, rightIter) } else { val leftBuffered = leftIter.buffered @@ -185,10 +205,142 @@ abstract class SetTheoryBetweenCollections[T: ClassTag, U: ClassTag, RT, RU] ext leftBuffered.flatMap(f => { val (currentRegion, _) = f - advanceCache(rightBuffered, currentRegion) - pruneCache(currentRegion) + advanceCache(rightBuffered, currentRegion, cache) + pruneCache(currentRegion, cache) processHits(f, cache) - }) + }) ++ finalizeHits(cache, rightBuffered) } } } + +/** + * Perform a set theory primitive with victims. + * + * @tparam T The left side row data. + * @tparam U The right side row data. + * @tparam RT The return type for the left side row data. + * @tparam RU The return type for the right side row data. + */ +private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U, RT, RU] + extends SetTheoryBetweenCollections[T, U, RT, RU] { + + /** + * Post processes the pruned records to format them appropriately. + * + * @param pruned The pruned record. + * @return The formatted, post processed record. + */ + protected def postProcessPruned(pruned: U): (RT, RU) + + override protected def pruneCache(to: ReferenceRegion, + cache: SetTheoryCache[U, RT, RU]) = { + + val toThreshold = to.pad(threshold) + // remove everything from cache that will never again be joined + cache.cache.trimStart({ + val trimLocation = + cache.cache + .indexWhere(f => !pruneCacheCondition(f._1, toThreshold)) + + if (trimLocation < 0) { + 0 + } else { + trimLocation + } + }) + + // add the values from the victimCache that are candidates to be joined + // the the current left + val cacheAddition = + cache.victimCache + .takeWhile(f => !pruneCacheCondition(f._1, toThreshold)) + + cache.cache ++= cacheAddition + // remove the values from the victimCache that were just added to cache + cache.victimCache.trimStart(cacheAddition.size) + + // add to pruned any values that do not have any matches to a left + // and perform post processing to format the new pruned values + val prunedAddition = + cache + .victimCache + .takeWhile(f => f._1.compareTo(toThreshold) <= 0) + + cache.pruned ++= prunedAddition + .map(u => postProcessPruned(u._2)) + // remove the values from victimCache that were added to pruned + cache.victimCache.trimStart(prunedAddition.size) + } + + override protected def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], + until: ReferenceRegion, + cache: SetTheoryCache[U, RT, RU]) = { + + while (right.hasNext && + advanceCacheCondition(right.head._1, until.pad(threshold))) { + + val x = right.next() + cache.victimCache += ((x._1, x._2)) + } + } + + override protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], + right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] = { + cache.pruned ++ + right.map(f => postProcessPruned(f._2)) + } +} + +/** + * Perform a set theory primitive without victims. + * + * @tparam T The left side row data. + * @tparam U The right side row data. + * @tparam RT The return type for the left side row data. + * @tparam RU The return type for the right side row data. + */ +private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U, RT, RU] + extends SetTheoryBetweenCollections[T, U, RT, RU] { + + override protected def pruneCache(to: ReferenceRegion, + cache: SetTheoryCache[U, RT, RU]) = { + cache.cache.trimStart({ + val index = cache.cache.indexWhere(f => !pruneCacheCondition(f._1, to)) + if (index <= 0) { + 0 + } else { + index + } + }) + } + + override protected def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], + until: ReferenceRegion, + cache: SetTheoryCache[U, RT, RU]) = { + while (right.hasNext && advanceCacheCondition(right.head._1, until)) { + cache.cache += right.next + } + } + + override protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], + right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] = { + // Victimless Set Theory drops the remaining records + Iterable.empty + } +} + +/** + * Contains all the caching data for a set theory operation. + * + * @tparam U The right side record type. + * @tparam RT The left side result type. + * @tparam RU The right side result type. + */ +private[settheory] class SetTheoryCache[U, RT, RU] { + // caches potential hits + val cache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty + // caches potential pruned and joined values + val victimCache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty + // the pruned values that do not contain any hits from the left + val pruned: ListBuffer[(RT, RU)] = ListBuffer.empty +} diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index 33209cfe7f..ae8ae22e3b 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -19,481 +19,258 @@ package org.bdgenomics.adam.rdd.settheory import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion -import scala.collection.mutable.ListBuffer import scala.reflect.ClassTag /** * A trait describing join implementations that are based on a sort-merge join. * - * @tparam T The type of the left RDD. - * @tparam U The type of the right RDD. - * @tparam RT The type of data yielded by the left RDD at the output of the - * join. This may not match T if the join is an outer join, etc. - * @tparam RU The type of data yielded by the right RDD at the output of the - * join. + * @tparam T The type of the left records. + * @tparam U The type of the right records. + * @tparam RT The resulting type of the left after the join. + * @tparam RU The resulting type of the right after the join. */ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] - extends SetTheoryBetweenCollections[T, U, RT, RU] { - - protected def advanceCache(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)], - until: ReferenceRegion) - protected def pruneCache(cache: SetTheoryCache[U, RT, RU], - to: ReferenceRegion) - protected def postProcessHits(iter: Iterable[U], - currentLeft: T): Iterable[(RT, RU)] - protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] - - protected val leftRdd: RDD[(ReferenceRegion, T)] - protected val rightRdd: RDD[(ReferenceRegion, U)] - - /** - * Performs a region join between two RDDs (shuffle join). All data should be pre-shuffled and - * copartitioned. - * - * @return An RDD of joins (x, y), where x is from leftRDD, y is from rightRDD, and the region - * corresponding to x overlaps the region corresponding to y. - */ - def compute(): RDD[(RT, RU)] = { - leftRdd.zipPartitions(rightRdd)(makeIterator) - } - - def partitionAndJoin(left: RDD[(ReferenceRegion, T)], right: RDD[(ReferenceRegion, U)]): RDD[(RT, RU)] = { - left.zipPartitions(right)(makeIterator) - } + extends SetTheoryBetweenCollections[T, U, RT, RU] with SetTheoryPrimitive { - protected def makeIterator(leftIter: Iterator[(ReferenceRegion, T)], - rightIter: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] = { + override protected def condition(firstRegion: ReferenceRegion, + secondRegion: ReferenceRegion, + distanceThreshold: Long = 0L): Boolean = { - if (leftIter.isEmpty || rightIter.isEmpty) { - emptyFn(leftIter, rightIter) - } else { - val bufferedLeft = leftIter.buffered - val bufferedRight = rightIter.buffered + firstRegion.isNearby(secondRegion, + distanceThreshold, + requireStranded = false) + } - val cache = new SetTheoryCache[U, RT, RU] + override protected def pruneCacheCondition(cachedRegion: ReferenceRegion, + to: ReferenceRegion): Boolean = { - bufferedLeft.flatMap(f => { - val currentLeftRegion = f._1 + cachedRegion.compareTo(to) < 0 && !cachedRegion.covers(to) + } - advanceCache(cache, bufferedRight, currentLeftRegion) - pruneCache(cache, currentLeftRegion) + override protected def advanceCacheCondition(candidateRegion: ReferenceRegion, + until: ReferenceRegion): Boolean = { - processHits(cache, f._2, f._1) - }) ++ finalizeHits(cache, bufferedRight) - } + candidateRegion.compareTo(until) < 0 || candidateRegion.covers(until) } - /** - * Process hits for a given object in left. - * - * @param cache The cache containing potential hits. - * @param currentLeft The current object from the left - * @param currentLeftRegion The ReferenceRegion of currentLeft. - * @return An iterator containing all hits, formatted by postProcessHits. - */ - protected def processHits(cache: SetTheoryCache[U, RT, RU], - currentLeft: T, - currentLeftRegion: ReferenceRegion): Iterable[(RT, RU)] = { - // post processing formats the hits for each individual type of join - postProcessHits(cache.cache - .filter(y => { - // everything that overlaps the left region is a hit - y._1.overlaps(currentLeftRegion) - }) - .map(y => y._2), currentLeft) + override protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) = { + (leftRdd, rightRdd) } } -case class InnerShuffleRegionJoin[T: ClassTag, U: ClassTag](leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)]) - extends VictimlessSortedIntervalPartitionJoin[T, U, T, U] { - - /** - * Handles the case where either the left or the right iterator were empty. - * In the case of inner join, we return an empty iterator. - * - * @param left The left iterator. - * @param right The right iterator. - * @return An empty iterator. - */ - protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(T, U)] = { +/** + * Perform an Inner Shuffle Region Join. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param optPartitionMap An optional partition map defining the left RDD + * partition bounds. + * @param threshold The threshold for the join. + * @tparam T The type of the left records. + * @tparam U The type of the right records. + */ +case class InnerShuffleRegionJoin[T: ClassTag, U: ClassTag]( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, U)], + optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, + threshold: Long = 0L) + extends ShuffleRegionJoin[T, U, T, U] + with VictimlessSetTheoryBetweenCollections[T, U, T, U] { + + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(T, U)] = { Iterator.empty } - /** - * Computes post processing required to complete the join and properly format - * hits. - * - * @param iter The iterable of hits. - * @param currentLeft The current value from the left iterator. - * @return The post processed iterable. - */ - protected def postProcessHits(iter: Iterable[U], - currentLeft: T): Iterable[(T, U)] = { - // no post-processing required - iter.map(f => (currentLeft, f)) + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, U)] = { + iter.map(f => (currentLeft._2, f._2)) } } -case class InnerShuffleRegionJoinAndGroupByLeft[T: ClassTag, U: ClassTag](leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)]) - extends VictimlessSortedIntervalPartitionJoin[T, U, T, Iterable[U]] { - - /** - * Handles the case where either the left or the right iterator were empty. - * In the case of inner join, we return an empty iterator. - * - * @param left The left iterator. - * @param right The right iterator. - * @return An empty iterator. - */ - protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Iterable[U])] = { +/** + * Perform an Inner Shuffle Region Join and Group By left records. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param optPartitionMap An optional partition map defining the left RDD + * partition bounds. + * @param threshold The threshold for the join. + * @tparam T The type of the left records. + * @tparam U THe type of the right records. + */ +case class InnerShuffleRegionJoinAndGroupByLeft[T: ClassTag, U: ClassTag]( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, U)], + optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, + threshold: Long = 0L) + extends ShuffleRegionJoin[T, U, T, Iterable[U]] + with VictimlessSetTheoryBetweenCollections[T, U, T, Iterable[U]] { + + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Iterable[U])] = { Iterator.empty } - /** - * Computes post processing required to complete the join and properly format - * hits. - * - * @param iter The iterator containing all hits. - * @param currentLeft The current left value. - * @return The post processed iterator. - */ - protected def postProcessHits(iter: Iterable[U], - currentLeft: T): Iterable[(T, Iterable[U])] = { + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, Iterable[U])] = { + if (iter.nonEmpty) { // group all hits for currentLeft into an iterable - Iterable((currentLeft, iter.toIterable)) + Iterable((currentLeft._2, iter.map(_._2))) } else { Iterable.empty } } } -case class LeftOuterShuffleRegionJoin[T: ClassTag, U: ClassTag](leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)]) - extends VictimlessSortedIntervalPartitionJoin[T, U, T, Option[U]] { - - /** - * Handles the case where the left or the right iterator were empty. - * - * @param left The left iterator. - * @param right The right iterator. - * @return The iterator containing properly formatted tuples. - */ - protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Option[U])] = { +/** + * Perform a Left Outer Shuffle Region Join. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param optPartitionMap An optional partition map defining the left RDD + * partition bounds. + * @param threshold The threshold for the join. + * @tparam T The type of the left records. + * @tparam U The type of the right records. + */ +case class LeftOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, U)], + optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, + threshold: Long = 0L) + extends ShuffleRegionJoin[T, U, T, Option[U]] + with VictimlessSetTheoryBetweenCollections[T, U, T, Option[U]] { + + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Option[U])] = { left.map(t => (t._2, None)) } - /** - * Computes post processing required to complete the join and properly format - * hits. - * - * @param iter The iterator of hits. - * @param currentLeft The current left value. - * @return the post processed iterator. - */ - protected def postProcessHits(iter: Iterable[U], - currentLeft: T): Iterable[(T, Option[U])] = { + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, Option[U])] = { if (iter.nonEmpty) { // left has some hits - iter.map(f => (currentLeft, Some(f))) + iter.map(f => (currentLeft._2, Some(f._2))) } else { // left has no hits - Iterable((currentLeft, None)) + Iterable((currentLeft._2, None)) } } } -case class RightOuterShuffleRegionJoin[T: ClassTag, U: ClassTag](leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)]) - extends SortedIntervalPartitionJoinWithVictims[T, U, Option[T], U] { - - /** - * Handles the case where the left or the right iterator were empty. - * - * @param left The left iterator. - * @param right The right iterator. - * @return The iterator containing properly formatted tuples. - */ - protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], U)] = { +/** + * Perform a Right Outer Shuffle Region Join. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param optPartitionMap An optional partition map defining the left RDD + * partition bounds. + * @param threshold The threshold for the join. + * @tparam T The type of the left records. + * @tparam U The type of the right records. + */ +case class RightOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, U)], + optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, + threshold: Long = 0L) + extends ShuffleRegionJoin[T, U, Option[T], U] + with SetTheoryBetweenCollectionsWithVictims[T, U, Option[T], U] { + + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], U)] = { right.map(u => (None, u._2)) } - /** - * Computes post processing required to complete the join and properly format - * hits. - * - * @param iter The iterator of hits. - * @param currentLeft The current left value. - * @return the post processed iterator. - */ - protected def postProcessHits(iter: Iterable[U], - currentLeft: T): Iterable[(Option[T], U)] = { + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(Option[T], U)] = { if (iter.nonEmpty) { // group all hits for currentLeft into an iterable - iter.map(f => (Some(currentLeft), f)) + iter.map(f => (Some(currentLeft._2), f._2)) } else { Iterable.empty } } - /** - * Properly formats right values that did not join with a left. - * - * @param pruned A record on the right. - * @return The formatted tuple containing the pruned value. - */ - protected def postProcessPruned(pruned: U): (Option[T], U) = { + override protected def postProcessPruned(pruned: U): (Option[T], U) = { (None, pruned) } } -case class FullOuterShuffleRegionJoin[T: ClassTag, U: ClassTag](leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)]) - extends SortedIntervalPartitionJoinWithVictims[T, U, Option[T], Option[U]] { - - /** - * Handles the case where the left or the right iterator were empty. - * - * @param left The left iterator. - * @param right The right iterator. - * @return The iterator containing properly formatted tuples. - */ - protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], Option[U])] = { +/** + * Perform a Full Outer Shuffle Region Join. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param optPartitionMap An optional partition map defining the left RDD + * partition bounds. + * @param threshold The threshold for the join. + * @tparam T The type of the left records. + * @tparam U The type of the right records. + */ +case class FullOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, U)], + optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, + threshold: Long = 0L) + extends ShuffleRegionJoin[T, U, Option[T], Option[U]] + with SetTheoryBetweenCollectionsWithVictims[T, U, Option[T], Option[U]] { + + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], Option[U])] = { left.map(t => (Some(t._2), None)) ++ right.map(u => (None, Some(u._2))) } - /** - * Computes post processing required to complete the join and properly format - * hits. - * - * @param iter The iterator of hits. - * @param currentLeft The current left value for the join. - * @return the post processed iterator. - */ - protected def postProcessHits(iter: Iterable[U], - currentLeft: T): Iterable[(Option[T], Option[U])] = { + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(Option[T], Option[U])] = { if (iter.nonEmpty) { // formatting these as options for the full outer join - iter.map(u => (Some(currentLeft), Some(u))) + iter.map(u => (Some(currentLeft._2), Some(u._2))) } else { // no hits for the currentLeft - Iterable((Some(currentLeft), None)) + Iterable((Some(currentLeft._2), None)) } } - /** - * Properly formats right values that did not join with a left. - * - * @param pruned The right value with no join. - * @return The formatted tuple containing the right value. - */ - protected def postProcessPruned(pruned: U): (Option[T], Option[U]) = { + override protected def postProcessPruned(pruned: U): (Option[T], Option[U]) = { (None, Some(pruned)) } } -case class RightOuterShuffleRegionJoinAndGroupByLeft[T: ClassTag, U: ClassTag](leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)]) - extends SortedIntervalPartitionJoinWithVictims[T, U, Option[T], Iterable[U]] { +/** + * Perform a Right Outer Shuffle Region Join and group by left values. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param optPartitionMap An optional partition map defining the left RDD + * partition bounds. + * @param threshold The threshold for the join. + * @tparam T The type of the left records. + * @tparam U The type of the right records. + */ +case class RightOuterShuffleRegionJoinAndGroupByLeft[T: ClassTag, U: ClassTag]( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, U)], + optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, + threshold: Long = 0L) + extends ShuffleRegionJoin[T, U, Option[T], Iterable[U]] + with SetTheoryBetweenCollectionsWithVictims[T, U, Option[T], Iterable[U]] { - /** - * Handles the case where one of the iterators contains no data. - * - * @param left The left iterator. - * @param right The right iterator. - * @return The iterator containing properly formatted tuples. - */ - protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], Iterable[U])] = { + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], Iterable[U])] = { left.map(v => (Some(v._2), Iterable.empty)) ++ right.map(v => (None, Iterable(v._2))) } - /** - * Computes post processing required to complete the join and properly format - * hits. - * - * @param iter The iterator of all hits - * @param currentLeft The current left value. - * @return The post processed iterator. - */ - protected def postProcessHits(iter: Iterable[U], - currentLeft: T): Iterable[(Option[T], Iterable[U])] = { - Iterable((Some(currentLeft), iter.toIterable)) + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, U)]): Iterable[(Option[T], Iterable[U])] = { + Iterable((Some(currentLeft._2), iter.map(_._2))) } - /** - * Properly formats right values that did not join with a region on the left. - * - * @param pruned The right value that did not join. - * @return A tuple with the postProcessed right value. - */ - protected def postProcessPruned(pruned: U): (Option[T], Iterable[U]) = { + override protected def postProcessPruned(pruned: U): (Option[T], Iterable[U]) = { (None, Iterable(pruned)) } } - -sealed trait VictimlessSortedIntervalPartitionJoin[T, U, RT, RU] - extends ShuffleRegionJoin[T, U, RT, RU] { - - /** - * Adds elements from right to cache based on the next region encountered. - * - * @param cache The cache for this partition. - * @param right The right iterator. - * @param until The next region to join with. - */ - protected def advanceCache(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)], - until: ReferenceRegion) = { - while (right.hasNext && - (right.head._1.compareTo(until) <= 0 || right.head._1.covers(until))) { - - val x = right.next() - cache.cache += ((x._1, x._2)) - } - } - - /** - * Removes elements from cache in place that do not meet the condition for - * the next region. - * - * @note At one point these were all variables and we built new collections - * and reassigned the pointers every time. We fixed this by using trimStart() - * and ++=() to improve performance. Overall, we see roughly 25% improvement - * in runtime by doing things this way. - * @param cache The cache for this partition. - * @param to The next region in the left iterator. - */ - protected def pruneCache(cache: SetTheoryCache[U, RT, RU], - to: ReferenceRegion) { - cache.cache.trimStart({ - val trimLocation = - cache - .cache - .indexWhere(f => !(f._1.compareTo(to) < 0 && !f._1.covers(to))) - - if (trimLocation < 0) { - 0 - } else { - trimLocation - } - }) - } - - /** - * Computes all victims for the partition. NOTE: These are victimless joins - * so we have no victims. - * - * @param cache The cache for this partition. - * @param right The right iterator. - * @return An empty iterator. - */ - protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] = { - Iterable.empty - } -} - -sealed trait SortedIntervalPartitionJoinWithVictims[T, U, RT, RU] - extends ShuffleRegionJoin[T, U, RT, RU] { - - protected def postProcessPruned(pruned: U): (RT, RU) - - /** - * Adds elements from right to victimCache based on the next region - * encountered. - * - * @param cache The cache for this partition. - * @param right The right iterator. - * @param until The next value on the left to perform the join. - */ - protected def advanceCache(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)], - until: ReferenceRegion) = { - while (right.hasNext && - (right.head._1.compareTo(until) <= 0 || right.head._1.covers(until))) { - - val x = right.next() - cache.victimCache += ((x._1, x._2)) - } - } - - /** - * Removes elements from cache in place that do not meet the condition for - * join. Also adds the elements that are not hits to the list of pruned. - * - * @param cache The cache for this partition. - * @param to The next region in the left iterator. - */ - protected def pruneCache(cache: SetTheoryCache[U, RT, RU], - to: ReferenceRegion) { - // remove everything from cache that will never again be joined - cache.cache.trimStart({ - val trimLocation = - cache.cache - .indexWhere(f => !(f._1.compareTo(to) < 0 && !f._1.covers(to))) - - if (trimLocation < 0) { - 0 - } else { - trimLocation - } - }) - - // add the values from the victimCache that are candidates to be joined - // the the current left - val cacheAddition = - cache - .victimCache - .takeWhile(f => f._1.compareTo(to) > 0 || f._1.covers(to)) - - cache.cache ++= cacheAddition - // remove the values from the victimCache that were just added to cache - cache.victimCache.trimStart(cacheAddition.size) - - // add to pruned any values that do not have any matches to a left - // and perform post processing to format the new pruned values - val prunedAddition = - cache - .victimCache - .takeWhile(f => f._1.compareTo(to) <= 0) - cache.pruned ++= prunedAddition - .map(u => postProcessPruned(u._2)) - // remove the values from victimCache that were added to pruned - cache.victimCache.trimStart(prunedAddition.size) - } - - /** - * Computes all victims for the partition. If there are any remaining values - * in the right iterator, those are considered victims. - * - * @param cache The cache for this partition. - * @param right The right iterator containing unmatched regions. - * @return An iterable containing all pruned hits. - */ - override protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] = { - cache.pruned ++ - right.map(f => postProcessPruned(f._2)) - } -} - -private class SetTheoryCache[U, RT, RU] { - // caches potential hits - val cache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty - // caches potential pruned and joined values - val victimCache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty - // the pruned values that do not contain any hits from the left - val pruned: ListBuffer[(RT, RU)] = ListBuffer.empty -} diff --git a/adam-core/src/test/resources/intersect_with_overlap_00.bed b/adam-core/src/test/resources/intersect_with_overlap_00.bed new file mode 100644 index 0000000000..225f0a8f7d --- /dev/null +++ b/adam-core/src/test/resources/intersect_with_overlap_00.bed @@ -0,0 +1,21 @@ +chr1 28735 29810 CpG:_116 +chr1 135124 135563 CpG:_30 +chr1 135453 139441 CpG:_99 +chr1 327790 328229 CpG:_29 +chr1 437151 438164 CpG:_84 +chr1 449273 450544 CpG:_99 +chr1 533219 534114 CpG:_94 +chr1 544738 546649 CpG:_171 +chr1 713984 714547 CpG:_60 +chr1 762416 763445 CpG:_115 +chr1 788863 789211 CpG:_28 +chr1 801975 802338 CpG:_24 +chr1 805198 805628 CpG:_50 +chr1 839694 840619 CpG:_83 +chr1 844299 845883 CpG:_153 +chr1 854765 854973 CpG:_16 +chr1 858970 861632 CpG:_257 +chr1 869332 871872 CpG:_178 +chr1 875730 878363 CpG:_246 +chr1 886356 886602 CpG:_18 +chr1 894313 902654 CpG:_615 \ No newline at end of file diff --git a/adam-core/src/test/resources/intersect_with_overlap_01.bed b/adam-core/src/test/resources/intersect_with_overlap_01.bed new file mode 100644 index 0000000000..245afe4913 --- /dev/null +++ b/adam-core/src/test/resources/intersect_with_overlap_01.bed @@ -0,0 +1,5 @@ +chr1 135000 135444 CpG:_116 +chr1 135124 135563 CpG:_39 +chr1 135333 135777 CpG:_29 +chr1 886356 886602 CpG:_18 +chr1 894313 902654 CpG:_615 \ No newline at end of file diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala index 1be9b80dbb..c1a2511bd8 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala @@ -101,8 +101,8 @@ class SortedGenomicRDDSuite extends SparkFunSuite { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) val z = x.sortLexicographically(storePartitionMap = true, partitions = 16) val y = x.sortLexicographically(storePartitionMap = true, partitions = 32) - val a = x.copartitionByReferenceRegion(y) - val b = z.copartitionByReferenceRegion(y) + val a = x.copartitionByReferenceRegion(y, 0L) + val b = z.copartitionByReferenceRegion(y, 0L) assert(isSorted(a.optPartitionMap.get)) assert(isSorted(b.optPartitionMap.get)) @@ -242,7 +242,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { iter.map(f => (idx, f)) }).collect val features = FeatureRDD(sc.parallelize(featureRddBuilder), sd) - val x = features.copartitionByReferenceRegion(genotypes) + val x = features.copartitionByReferenceRegion(genotypes, 0L) val z = x.rdd.mapPartitionsWithIndex((idx, iter) => { if (idx == 0 && iter.size != 6) { Iterator(true) diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala new file mode 100644 index 0000000000..836574e5fc --- /dev/null +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala @@ -0,0 +1,47 @@ +package org.bdgenomics.adam.rdd.settheory + +import org.bdgenomics.adam.models.ReferenceRegion +import org.bdgenomics.adam.util.ADAMFunSuite +import org.bdgenomics.adam.rdd.ADAMContext._ +import scala.collection.mutable.ListBuffer + +class ClosestSuite extends ADAMFunSuite { + sparkTest("testing closest") { + val leftFile = sc.loadBed(resourceUrl("intersect_with_overlap_00.bed").getFile) + .sortLexicographically(storePartitionMap = true) + val rightFile = sc.loadBed(resourceUrl("intersect_with_overlap_01.bed").getFile) + + val copartitionedRight = rightFile.copartitionByReferenceRegion(leftFile, 0L) + val x = ShuffleClosestRegion(leftFile.flattenRddByRegions(), + copartitionedRight.flattenRddByRegions(), leftFile.optPartitionMap) + .compute() + val result = x.map(f => + (ReferenceRegion(f._1.getContigName, f._1.getStart, f._1.getEnd), + f._2.map(g => ReferenceRegion(g.getContigName, g.getStart, g.getEnd)))) + .collect + + val correctOutput = Array((ReferenceRegion("chr1", 28735, 29810), ListBuffer(ReferenceRegion("chr1", 135000, 135444))), + (ReferenceRegion("chr1", 135124, 135563), ListBuffer(ReferenceRegion("chr1", 135000, 135444), ReferenceRegion("chr1", 135124, 135563), ReferenceRegion("chr1", 135333, 135777))), + (ReferenceRegion("chr1", 135453, 139441), ListBuffer(ReferenceRegion("chr1", 135124, 135563), ReferenceRegion("chr1", 135333, 135777))), + (ReferenceRegion("chr1", 327790, 328229), ListBuffer(ReferenceRegion("chr1", 135333, 135777))), + (ReferenceRegion("chr1", 437151, 438164), ListBuffer(ReferenceRegion("chr1", 135333, 135777))), + (ReferenceRegion("chr1", 449273, 450544), ListBuffer(ReferenceRegion("chr1", 135333, 135777))), + (ReferenceRegion("chr1", 533219, 534114), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 544738, 546649), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 713984, 714547), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 762416, 763445), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 788863, 789211), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 801975, 802338), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 805198, 805628), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 839694, 840619), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 844299, 845883), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 854765, 854973), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 858970, 861632), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 869332, 871872), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 875730, 878363), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 886356, 886602), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), + (ReferenceRegion("chr1", 894313, 902654), ListBuffer(ReferenceRegion("chr1", 894313, 902654)))) + + assert(result === correctOutput) + } +} From d9fff6885e0a5e6162bd81f1f8817260f122c457 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Fri, 9 Jun 2017 10:45:30 -0700 Subject: [PATCH 03/13] Moving copartitioning into ShuffleRegionJoin --- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 198 ++++++------------ .../rdd/settheory/ShuffleRegionJoin.scala | 61 +++++- .../adam/rdd/SortedGenomicRDDSuite.scala | 15 +- .../adam/rdd/settheory/ClosestSuite.scala | 3 +- 4 files changed, 128 insertions(+), 149 deletions(-) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index 00d8cbc9f0..ca7916f825 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -781,36 +781,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { .asInstanceOf[GenomicRDD[(Iterable[X], T), Z]] } - /** - * Prepares two RDDs to be joined with any shuffleRegionJoin. This includes copartition - * and sort of the rightRdd if necessary. - * - * @param genomicRdd The RDD to join to. - * @param optPartitions Optionally sets the number of output partitions. If - * None, the number of partitions on the resulting RDD does not change. - * @return a case class containing all the prepared data for ShuffleRegionJoins - */ - private def prepareForShuffleRegionJoin[X, Y <: GenomicRDD[X, Y], Z <: GenomicRDD[(T, X), Z]]( - genomicRdd: GenomicRDD[X, Y], - optPartitions: Option[Int], - threshold: Long)( - implicit tTag: ClassTag[T], xTag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { - - val partitions = optPartitions.getOrElse(this.rdd.partitions.length) - - val (leftRdd, rightRdd) = (isSorted, genomicRdd.isSorted) match { - case (true, _) => (this, genomicRdd.copartitionByReferenceRegion(this, threshold)) - case (false, true) => (copartitionByReferenceRegion(genomicRdd, threshold), genomicRdd) - case (false, false) => { - val repartitionedRdd = - sortLexicographically(storePartitionMap = true, partitions = partitions) - - (repartitionedRdd, genomicRdd.copartitionByReferenceRegion(repartitionedRdd, threshold)) - } - } - (leftRdd.flattenRddByRegions(), rightRdd.flattenRddByRegions()) - } - /** * Performs a sort-merge inner join between this RDD and another RDD. * @@ -837,15 +807,22 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], txTag: ClassTag[(T, X)]): GenomicRDD[(T, X), Z] = InnerShuffleJoin.time { - val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) + val preparedLeft = + if(!isSorted) { + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + } else { + this + } // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, X)]( - InnerShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) - .compute(), + InnerShuffleRegionJoin[T, X]( + preparedLeft.flattenRddByRegions(), + genomicRdd.flattenRddByRegions(), + preparedLeft.optPartitionMap, + threshold).compute(), combinedSequences, kv => { getReferenceRegions(kv._1) ++ genomicRdd.getReferenceRegions(kv._2) @@ -881,15 +858,22 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], otxTag: ClassTag[(Option[T], X)]): GenomicRDD[(Option[T], X), Z] = RightOuterShuffleJoin.time { - val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) + val preparedLeft = + if(!isSorted) { + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + } else { + this + } // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], X)]( - RightOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) - .compute(), + RightOuterShuffleRegionJoin[T, X]( + preparedLeft.flattenRddByRegions(), + genomicRdd.flattenRddByRegions(), + preparedLeft.optPartitionMap, + threshold).compute(), combinedSequences, kv => { Seq(kv._1.map(v => getReferenceRegions(v))).flatten.flatten ++ @@ -926,15 +910,22 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], toxTag: ClassTag[(T, Option[X])]): GenomicRDD[(T, Option[X]), Z] = LeftOuterShuffleJoin.time { - val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) + val preparedLeft = + if(!isSorted) { + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + } else { + this + } // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, Option[X])]( - LeftOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) - .compute(), + LeftOuterShuffleRegionJoin[T, X]( + preparedLeft.flattenRddByRegions(), + genomicRdd.flattenRddByRegions(), + preparedLeft.optPartitionMap, + threshold).compute(), combinedSequences, kv => { Seq(kv._2.map(v => genomicRdd.getReferenceRegions(v))).flatten.flatten ++ @@ -970,15 +961,22 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], otoxTag: ClassTag[(Option[T], Option[X])]): GenomicRDD[(Option[T], Option[X]), Z] = FullOuterShuffleJoin.time { - val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) + val preparedLeft = + if(!isSorted) { + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + } else { + this + } // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], Option[X])]( - FullOuterShuffleRegionJoin[T, X](leftRddToJoin, rightRddToJoin, None, threshold) - .compute(), + FullOuterShuffleRegionJoin[T, X]( + preparedLeft.flattenRddByRegions(), + genomicRdd.flattenRddByRegions(), + preparedLeft.optPartitionMap, + threshold).compute(), combinedSequences, kv => { Seq(kv._2.map(v => genomicRdd.getReferenceRegions(v)), @@ -1015,15 +1013,22 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], tixTag: ClassTag[(T, Iterable[X])]): GenomicRDD[(T, Iterable[X]), Z] = ShuffleJoinAndGroupByLeft.time { - val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) + val preparedLeft = + if(!isSorted) { + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + } else { + this + } // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, Iterable[X])]( - InnerShuffleRegionJoinAndGroupByLeft[T, X](leftRddToJoin, rightRddToJoin, None, threshold) - .compute(), + InnerShuffleRegionJoinAndGroupByLeft[T, X]( + preparedLeft.flattenRddByRegions(), + genomicRdd.flattenRddByRegions(), + preparedLeft.optPartitionMap, + threshold).compute(), combinedSequences, kv => { (kv._2.flatMap(v => genomicRdd.getReferenceRegions(v)) ++ @@ -1062,15 +1067,22 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], otixTag: ClassTag[(Option[T], Iterable[X])]): GenomicRDD[(Option[T], Iterable[X]), Z] = RightOuterShuffleJoinAndGroupByLeft.time { - val (leftRddToJoin, rightRddToJoin) = - prepareForShuffleRegionJoin(genomicRdd, optPartitions, threshold) + val preparedLeft = + if(!isSorted) { + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + } else { + this + } // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], Iterable[X])]( - RightOuterShuffleRegionJoinAndGroupByLeft[T, X](leftRddToJoin, rightRddToJoin, None, threshold) - .compute(), + RightOuterShuffleRegionJoinAndGroupByLeft[T, X]( + preparedLeft.flattenRddByRegions(), + genomicRdd.flattenRddByRegions(), + preparedLeft.optPartitionMap, + threshold).compute(), combinedSequences, kv => { (kv._2.flatMap(v => genomicRdd.getReferenceRegions(v)) ++ @@ -1078,84 +1090,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { }).asInstanceOf[GenomicRDD[(Option[T], Iterable[X]), Z]] } - /** - * Copartitions two RDDs according to their ReferenceRegions. - * - * @note This is best used under the condition that (repeatedly) - * repartitioning is more expensive than calculating the proper location - * of the records of this.rdd. It requires a pass through the co-located - * RDD to get the correct partition(s) for each record. It will assign a - * record to multiple partitions if necessary. - * @param rddToCoPartitionWith The rdd to copartition to. - * @param threshold The threshold for the join. - * @return The newly repartitioned rdd. - */ - private[rdd] def copartitionByReferenceRegion[X, Y <: GenomicRDD[X, Y]]( - rddToCoPartitionWith: GenomicRDD[X, Y], - threshold: Long)(implicit tTag: ClassTag[T], xTag: ClassTag[X]): U = { - - // if the other RDD is not sorted, we can't guarantee proper copartition - assert(rddToCoPartitionWith.isSorted, - "Cannot copartition with an unsorted rdd!") - - val destinationPartitionMap = rddToCoPartitionWith.optPartitionMap.get - - // number of partitions we will have after repartition - val numPartitions = destinationPartitionMap.length - - // here we create a partition map with a single ReferenceRegion that spans - // the entire range, however we have to handle the case where the partition - // spans multiple referenceNames because of load balancing. - val adjustedPartitionMapWithIndex = - - // the zipWithIndex gives us the destination partition ID - destinationPartitionMap.flatten.zipWithIndex.map(g => { - val (firstRegion, secondRegion, index) = (g._1._1, g._1._2, g._2) - - // in the case where we span multiple referenceNames using - // IntervalArray.get with requireOverlap set to false will assign all - // the remaining regions to this partition, in addition to all the - // regions up to the start of the next partition. - if (firstRegion.referenceName != secondRegion.referenceName) { - - // the first region is enough to represent the partition for - // IntervalArray.get. - (firstRegion, index) - } else { - // otherwise we just have the ReferenceRegion span from partition - // lower bound to upper bound. - // We cannot use the firstRegion bounds here because we may end up - // dropping data if it doesn't map anywhere. - (ReferenceRegion( - firstRegion.referenceName, - firstRegion.start, - secondRegion.end), - index) - } - }) - - // convert to an IntervalArray for fast range query - val partitionMapIntervals = IntervalArray( - adjustedPartitionMapWithIndex, - adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, - sorted = true) - - val finalPartitionedRDD = { - val referenceRegionKeyedGenomicRDD = flattenRddByRegions() - - referenceRegionKeyedGenomicRDD.mapPartitions(iter => { - iter.flatMap(f => { - val intervals = partitionMapIntervals.get(f._1.pad(threshold), requireOverlap = false) - intervals.map(g => ((f._1, g._2), f._2)) - }) - }, preservesPartitioning = true) - .repartitionAndSortWithinPartitions( - ManualRegionPartitioner(numPartitions)) - } - - replaceRdd(finalPartitionedRDD.values, rddToCoPartitionWith.optPartitionMap) - } - /** * Gets the partition bounds from a ReferenceRegion keyed Iterator. * diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index ae8ae22e3b..98164cfae0 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -19,6 +19,8 @@ package org.bdgenomics.adam.rdd.settheory import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion +import org.bdgenomics.adam.rdd.ManualRegionPartitioner +import org.bdgenomics.utils.interval.array.IntervalArray import scala.reflect.ClassTag /** @@ -54,7 +56,64 @@ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] } override protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) = { - (leftRdd, rightRdd) + //(leftRdd, rightRdd) + + val (preparedLeft, destinationPartitionMap) = { + if(optPartitionMap.isDefined) { + (leftRdd, optPartitionMap.get.map(_.get)) + } else { + val sortedLeft = leftRdd.sortByKey() + val partitionMap = + sortedLeft.mapPartitions(getRegionBoundsFromPartition).collect + (sortedLeft, partitionMap.map(_.get)) + } + } + + val adjustedPartitionMapWithIndex = + // the zipWithIndex gives us the destination partition ID + destinationPartitionMap.zipWithIndex.map(g => { + val (firstRegion, secondRegion, index) = (g._1._1, g._1._2, g._2) + // in the case where we span multiple referenceNames using + // IntervalArray.get with requireOverlap set to false will assign all + // the remaining regions to this partition, in addition to all the + // regions up to the start of the next partition. + if (firstRegion.referenceName != secondRegion.referenceName) { + + // the first region is enough to represent the partition for + // IntervalArray.get. + (firstRegion, index) + } else { + // otherwise we just have the ReferenceRegion span from partition + // lower bound to upper bound. + // We cannot use the firstRegion bounds here because we may end up + // dropping data if it doesn't map anywhere. + (ReferenceRegion( + firstRegion.referenceName, + firstRegion.start, + secondRegion.end), + index) + } + }) + + // convert to an IntervalArray for fast range query + val partitionMapIntervals = IntervalArray( + adjustedPartitionMapWithIndex, + adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, + sorted = true) + + val preparedRight = { + rightRdd.mapPartitions(iter => { + iter.flatMap(f => { + val intervals = partitionMapIntervals.get(f._1.pad(threshold), requireOverlap = false) + intervals.map(g => ((f._1, g._2), f._2)) + }) + }, preservesPartitioning = true) + .repartitionAndSortWithinPartitions( + ManualRegionPartitioner(destinationPartitionMap.length)) + .map(f => (f._1._1, f._2)) + } + + (preparedLeft, preparedRight) } } diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala index c1a2511bd8..a95188b073 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala @@ -97,19 +97,6 @@ class SortedGenomicRDDSuite extends SparkFunSuite { assert(partitionTupleCounts.sum == partitionTupleCounts2.sum) } - sparkTest("testing copartition maintains or adds sort") { - val x = sc.loadBam(resourceUrl("reads12.sam").getFile) - val z = x.sortLexicographically(storePartitionMap = true, partitions = 16) - val y = x.sortLexicographically(storePartitionMap = true, partitions = 32) - val a = x.copartitionByReferenceRegion(y, 0L) - val b = z.copartitionByReferenceRegion(y, 0L) - - assert(isSorted(a.optPartitionMap.get)) - assert(isSorted(b.optPartitionMap.get)) - - val starts = z.rdd.map(f => f.getStart) - } - sparkTest("testing that we don't drop any data on the right side even though it doesn't map to a partition on the left") { // testing the left side with an extremely large region that is // not the last record on a partition @@ -242,7 +229,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { iter.map(f => (idx, f)) }).collect val features = FeatureRDD(sc.parallelize(featureRddBuilder), sd) - val x = features.copartitionByReferenceRegion(genotypes, 0L) + val x = features val z = x.rdd.mapPartitionsWithIndex((idx, iter) => { if (idx == 0 && iter.size != 6) { Iterator(true) diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala index 836574e5fc..22e48f7a2f 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala @@ -11,9 +11,8 @@ class ClosestSuite extends ADAMFunSuite { .sortLexicographically(storePartitionMap = true) val rightFile = sc.loadBed(resourceUrl("intersect_with_overlap_01.bed").getFile) - val copartitionedRight = rightFile.copartitionByReferenceRegion(leftFile, 0L) val x = ShuffleClosestRegion(leftFile.flattenRddByRegions(), - copartitionedRight.flattenRddByRegions(), leftFile.optPartitionMap) + rightFile.flattenRddByRegions(), leftFile.optPartitionMap) .compute() val result = x.map(f => (ReferenceRegion(f._1.getContigName, f._1.getStart, f._1.getEnd), From af9b6d8af00bce15eab242cdd7e1fa0e69963e56 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Fri, 9 Jun 2017 10:53:27 -0700 Subject: [PATCH 04/13] Fixing an issue where data was sorted twice --- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 18 ++++++++++++------ .../adam/rdd/settheory/ShuffleRegionJoin.scala | 3 +-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index ca7916f825..7d7281301a 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -809,7 +809,8 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val preparedLeft = if(!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), + storePartitionMap = true) } else { this } @@ -860,7 +861,8 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val preparedLeft = if(!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), + storePartitionMap = true) } else { this } @@ -912,7 +914,8 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val preparedLeft = if(!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), + storePartitionMap = true) } else { this } @@ -963,7 +966,8 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val preparedLeft = if(!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), + storePartitionMap = true) } else { this } @@ -1015,7 +1019,8 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val preparedLeft = if(!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), + storePartitionMap = true) } else { this } @@ -1069,7 +1074,8 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val preparedLeft = if(!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length)) + sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), + storePartitionMap = true) } else { this } diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index 98164cfae0..cf9893b1e0 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -56,8 +56,7 @@ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] } override protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) = { - //(leftRdd, rightRdd) - + val (preparedLeft, destinationPartitionMap) = { if(optPartitionMap.isDefined) { (leftRdd, optPartitionMap.get.map(_.get)) From 348d1504d09194c3fa7dd76fc44db2b49797a8f7 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Fri, 9 Jun 2017 13:09:02 -0700 Subject: [PATCH 05/13] Fixing some formatting issues --- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 12 +- .../rdd/settheory/ShuffleRegionJoin.scala | 12 +- .../adam/rdd/SortedGenomicRDDSuite.scala | 150 ------------------ 3 files changed, 12 insertions(+), 162 deletions(-) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index 7d7281301a..c47a54dd99 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -808,7 +808,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { txTag: ClassTag[(T, X)]): GenomicRDD[(T, X), Z] = InnerShuffleJoin.time { val preparedLeft = - if(!isSorted) { + if (!isSorted) { sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), storePartitionMap = true) } else { @@ -860,7 +860,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { otxTag: ClassTag[(Option[T], X)]): GenomicRDD[(Option[T], X), Z] = RightOuterShuffleJoin.time { val preparedLeft = - if(!isSorted) { + if (!isSorted) { sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), storePartitionMap = true) } else { @@ -913,7 +913,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { toxTag: ClassTag[(T, Option[X])]): GenomicRDD[(T, Option[X]), Z] = LeftOuterShuffleJoin.time { val preparedLeft = - if(!isSorted) { + if (!isSorted) { sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), storePartitionMap = true) } else { @@ -965,7 +965,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { otoxTag: ClassTag[(Option[T], Option[X])]): GenomicRDD[(Option[T], Option[X]), Z] = FullOuterShuffleJoin.time { val preparedLeft = - if(!isSorted) { + if (!isSorted) { sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), storePartitionMap = true) } else { @@ -1018,7 +1018,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { tixTag: ClassTag[(T, Iterable[X])]): GenomicRDD[(T, Iterable[X]), Z] = ShuffleJoinAndGroupByLeft.time { val preparedLeft = - if(!isSorted) { + if (!isSorted) { sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), storePartitionMap = true) } else { @@ -1073,7 +1073,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { otixTag: ClassTag[(Option[T], Iterable[X])]): GenomicRDD[(Option[T], Iterable[X]), Z] = RightOuterShuffleJoinAndGroupByLeft.time { val preparedLeft = - if(!isSorted) { + if (!isSorted) { sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), storePartitionMap = true) } else { diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index cf9893b1e0..967cea0032 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -56,21 +56,21 @@ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] } override protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) = { - + val (preparedLeft, destinationPartitionMap) = { - if(optPartitionMap.isDefined) { - (leftRdd, optPartitionMap.get.map(_.get)) + if (optPartitionMap.isDefined) { + (leftRdd, optPartitionMap.get) } else { val sortedLeft = leftRdd.sortByKey() val partitionMap = sortedLeft.mapPartitions(getRegionBoundsFromPartition).collect - (sortedLeft, partitionMap.map(_.get)) + (sortedLeft, partitionMap) } } val adjustedPartitionMapWithIndex = - // the zipWithIndex gives us the destination partition ID - destinationPartitionMap.zipWithIndex.map(g => { + // the zipWithIndex gives us the destination partition ID + destinationPartitionMap.flatten.zipWithIndex.map(g => { val (firstRegion, secondRegion, index) = (g._1._1, g._1._2, g._2) // in the case where we span multiple referenceNames using // IntervalArray.get with requireOverlap set to false will assign all diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala index a95188b073..aec65789a8 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala @@ -97,156 +97,6 @@ class SortedGenomicRDDSuite extends SparkFunSuite { assert(partitionTupleCounts.sum == partitionTupleCounts2.sum) } - sparkTest("testing that we don't drop any data on the right side even though it doesn't map to a partition on the left") { - // testing the left side with an extremely large region that is - // not the last record on a partition - // this test also tests the case that our - val genotypeRddBuilder = new ListBuffer[Genotype]() - - genotypeRddBuilder += { - Genotype.newBuilder() - .setContigName("chr1") - .setStart(2L) - .setEnd(100L) - .setVariant( - Variant.newBuilder() - .setStart(2L) - .setEnd(100L) - .setAlternateAllele("A") - .setReferenceAllele("T") - .build() - ) - .setSampleId("1") - .build() - } - - genotypeRddBuilder += { - Genotype.newBuilder() - .setContigName("chr1") - .setStart(3L) - .setEnd(5L) - .setVariant( - Variant.newBuilder() - .setStart(3L) - .setEnd(5L) - .setAlternateAllele("A") - .setReferenceAllele("T") - .build() - ) - .setSampleId("2") - .build() - } - - genotypeRddBuilder += { - Genotype.newBuilder() - .setContigName("chr1") - .setStart(6L) - .setEnd(7L) - .setVariant( - Variant.newBuilder() - .setStart(6L) - .setEnd(7L) - .setAlternateAllele("A") - .setReferenceAllele("T") - .build() - ) - .setSampleId("3") - .build() - } - - genotypeRddBuilder += { - Genotype.newBuilder() - .setContigName("chr1") - .setStart(8L) - .setEnd(12L) - .setVariant( - Variant.newBuilder() - .setStart(8L) - .setEnd(12L) - .setAlternateAllele("A") - .setReferenceAllele("T") - .build() - ) - .setSampleId("3") - .build() - } - - val featureRddBuilder = new ListBuffer[Feature]() - - featureRddBuilder += { - Feature.newBuilder() - .setContigName("chr1") - .setStart(61L) - .setEnd(62L) - .build() - } - - featureRddBuilder += { - Feature.newBuilder() - .setContigName("chr1") - .setStart(11L) - .setEnd(15L) - .build() - } - - featureRddBuilder += { - Feature.newBuilder() - .setContigName("chr1") - .setStart(3L) - .setEnd(6L) - .build() - } - - featureRddBuilder += { - Feature.newBuilder() - .setContigName("chr1") - .setStart(6L) - .setEnd(8L) - .build() - } - - featureRddBuilder += { - Feature.newBuilder() - .setContigName("chr1") - .setStart(50L) - .setEnd(52L) - .build() - } - - featureRddBuilder += { - Feature.newBuilder() - .setContigName("chr1") - .setStart(1L) - .setEnd(2L) - .build() - } - - val genotypes = - GenotypeRDD(sc.parallelize(genotypeRddBuilder), - sd, Seq(), DefaultHeaderLines.allHeaderLines) - .sortLexicographically(storePartitionMap = true, partitions = 2) - genotypes.rdd.mapPartitionsWithIndex((idx, iter) => { - iter.map(f => (idx, f)) - }).collect - val features = FeatureRDD(sc.parallelize(featureRddBuilder), sd) - val x = features - val z = x.rdd.mapPartitionsWithIndex((idx, iter) => { - if (idx == 0 && iter.size != 6) { - Iterator(true) - } else if (idx == 1 && iter.size != 2) { - Iterator(true) - } else { - Iterator() - } - }) - - x.rdd.mapPartitionsWithIndex((idx, iter) => { - iter.map(f => (idx, f)) - }).collect - assert(z.collect.length == 0) - - } - sparkTest("testing that sorted shuffleRegionJoin matches unsorted") { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) // sort and make into 16 partitions From a5481118a657180d4ad324eb30346dbd50f2aadb Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Fri, 9 Jun 2017 14:23:31 -0700 Subject: [PATCH 06/13] Formatting source files --- .../bdgenomics/adam/rdd/settheory/Closest.scala | 17 +++++++++++++++++ .../adam/rdd/settheory/SetTheory.scala | 17 +++++++++++++++++ .../adam/rdd/settheory/ClosestSuite.scala | 17 +++++++++++++++++ 3 files changed, 51 insertions(+) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala index 2973f18d42..f5c5e7e371 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala @@ -1,3 +1,20 @@ +/** + * 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.settheory import org.apache.spark.rdd.RDD diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala index 215c21b310..c7afdc65bc 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala @@ -1,3 +1,20 @@ +/** + * 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.settheory import org.apache.spark.rdd.RDD diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala index 22e48f7a2f..301ad33295 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala @@ -1,3 +1,20 @@ +/** + * 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.settheory import org.bdgenomics.adam.models.ReferenceRegion From a4d196e975042abecf9d82562107b97609ee677b Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Mon, 12 Jun 2017 11:53:47 -0700 Subject: [PATCH 07/13] Set theory package now accepts GenomicRDDs as input --- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 92 ++-------- .../adam/rdd/settheory/Closest.scala | 48 +++-- .../adam/rdd/settheory/SetTheory.scala | 87 +++++----- .../rdd/settheory/ShuffleRegionJoin.scala | 164 +++++++++--------- .../rdd/InnerShuffleRegionJoinSuite.scala | 25 ++- .../rdd/LeftOuterShuffleRegionJoinSuite.scala | 7 +- .../adam/rdd/OuterRegionJoinSuite.scala | 44 +++-- .../RightOuterShuffleRegionJoinSuite.scala | 7 +- .../rdd/RightOuterTreeRegionJoinSuite.scala | 14 +- .../adam/rdd/settheory/ClosestSuite.scala | 8 +- 10 files changed, 231 insertions(+), 265 deletions(-) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index c47a54dd99..305e416385 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -154,7 +154,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { // The (ReferenceRegion, ReferenceRegion) tuple contains the bounds of the // partition, such that the lowest start is first and the highest end is // second. - protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] + private[rdd] val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] assert(optPartitionMap.isEmpty || optPartitionMap.exists(_.length == rdd.partitions.length), @@ -807,23 +807,12 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], txTag: ClassTag[(T, X)]): GenomicRDD[(T, X), Z] = InnerShuffleJoin.time { - val preparedLeft = - if (!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), - storePartitionMap = true) - } else { - this - } - // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, X)]( - InnerShuffleRegionJoin[T, X]( - preparedLeft.flattenRddByRegions(), - genomicRdd.flattenRddByRegions(), - preparedLeft.optPartitionMap, - threshold).compute(), + InnerShuffleRegionJoin[T, U, X, Y](this, genomicRdd, threshold, optPartitions) + .compute(), combinedSequences, kv => { getReferenceRegions(kv._1) ++ genomicRdd.getReferenceRegions(kv._2) @@ -859,23 +848,12 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], otxTag: ClassTag[(Option[T], X)]): GenomicRDD[(Option[T], X), Z] = RightOuterShuffleJoin.time { - val preparedLeft = - if (!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), - storePartitionMap = true) - } else { - this - } - // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], X)]( - RightOuterShuffleRegionJoin[T, X]( - preparedLeft.flattenRddByRegions(), - genomicRdd.flattenRddByRegions(), - preparedLeft.optPartitionMap, - threshold).compute(), + RightOuterShuffleRegionJoin[T, U, X, Y](this, genomicRdd, threshold, optPartitions) + .compute(), combinedSequences, kv => { Seq(kv._1.map(v => getReferenceRegions(v))).flatten.flatten ++ @@ -912,23 +890,12 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], toxTag: ClassTag[(T, Option[X])]): GenomicRDD[(T, Option[X]), Z] = LeftOuterShuffleJoin.time { - val preparedLeft = - if (!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), - storePartitionMap = true) - } else { - this - } - // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, Option[X])]( - LeftOuterShuffleRegionJoin[T, X]( - preparedLeft.flattenRddByRegions(), - genomicRdd.flattenRddByRegions(), - preparedLeft.optPartitionMap, - threshold).compute(), + LeftOuterShuffleRegionJoin[T, U, X, Y](this, genomicRdd, threshold, optPartitions) + .compute(), combinedSequences, kv => { Seq(kv._2.map(v => genomicRdd.getReferenceRegions(v))).flatten.flatten ++ @@ -964,23 +931,12 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], otoxTag: ClassTag[(Option[T], Option[X])]): GenomicRDD[(Option[T], Option[X]), Z] = FullOuterShuffleJoin.time { - val preparedLeft = - if (!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), - storePartitionMap = true) - } else { - this - } - // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], Option[X])]( - FullOuterShuffleRegionJoin[T, X]( - preparedLeft.flattenRddByRegions(), - genomicRdd.flattenRddByRegions(), - preparedLeft.optPartitionMap, - threshold).compute(), + FullOuterShuffleRegionJoin[T, U, X, Y](this, genomicRdd, threshold, optPartitions) + .compute(), combinedSequences, kv => { Seq(kv._2.map(v => genomicRdd.getReferenceRegions(v)), @@ -1017,23 +973,12 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], tixTag: ClassTag[(T, Iterable[X])]): GenomicRDD[(T, Iterable[X]), Z] = ShuffleJoinAndGroupByLeft.time { - val preparedLeft = - if (!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), - storePartitionMap = true) - } else { - this - } - // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, Iterable[X])]( - InnerShuffleRegionJoinAndGroupByLeft[T, X]( - preparedLeft.flattenRddByRegions(), - genomicRdd.flattenRddByRegions(), - preparedLeft.optPartitionMap, - threshold).compute(), + InnerShuffleRegionJoinAndGroupByLeft[T, U, X, Y](this, genomicRdd, threshold, optPartitions) + .compute(), combinedSequences, kv => { (kv._2.flatMap(v => genomicRdd.getReferenceRegions(v)) ++ @@ -1072,23 +1017,12 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { xTag: ClassTag[X], otixTag: ClassTag[(Option[T], Iterable[X])]): GenomicRDD[(Option[T], Iterable[X]), Z] = RightOuterShuffleJoinAndGroupByLeft.time { - val preparedLeft = - if (!isSorted) { - sortLexicographically(optPartitions.getOrElse(rdd.partitions.length), - storePartitionMap = true) - } else { - this - } - // what sequences do we wind up with at the end? val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(Option[T], Iterable[X])]( - RightOuterShuffleRegionJoinAndGroupByLeft[T, X]( - preparedLeft.flattenRddByRegions(), - genomicRdd.flattenRddByRegions(), - preparedLeft.optPartitionMap, - threshold).compute(), + RightOuterShuffleRegionJoinAndGroupByLeft[T, U, X, Y](this, genomicRdd, threshold, optPartitions) + .compute(), combinedSequences, kv => { (kv._2.flatMap(v => genomicRdd.getReferenceRegions(v)) ++ diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala index f5c5e7e371..53943b2cb7 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala @@ -19,7 +19,7 @@ package org.bdgenomics.adam.rdd.settheory import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion -import org.bdgenomics.adam.rdd.ManualRegionPartitioner +import org.bdgenomics.adam.rdd.{ GenomicRDD, ManualRegionPartitioner } import org.bdgenomics.utils.interval.array.IntervalArray import scala.reflect.ClassTag @@ -28,11 +28,11 @@ import scala.reflect.ClassTag * * @tparam T The type of the left records. * @tparam U The type of the right records. - * @tparam RT The resulting type of the left after the join. - * @tparam RU The resulting type of the right after the join. + * @tparam RT The resulting type of the left after the operation. + * @tparam RX The resulting type of the right after the operation. */ -sealed abstract class Closest[T: ClassTag, U: ClassTag, RT, RU] - extends SetTheoryBetweenCollections[T, U, RT, RU] +sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] with SetTheoryPrimitive { var currentClosest: ReferenceRegion = ReferenceRegion.empty @@ -49,7 +49,8 @@ sealed abstract class Closest[T: ClassTag, U: ClassTag, RT, RU] if (cachedRegion.referenceName != to.referenceName) { true } else { - to.unstrandedDistance(cachedRegion).get > to.unstrandedDistance(currentClosest).getOrElse(Long.MaxValue) + to.unstrandedDistance(cachedRegion).get > + to.unstrandedDistance(currentClosest).getOrElse(Long.MaxValue) } } @@ -69,18 +70,14 @@ sealed abstract class Closest[T: ClassTag, U: ClassTag, RT, RU] } } - override protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) = { + override protected def prepare()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { - val (partitionMap: Array[Option[(ReferenceRegion, ReferenceRegion)]], - preparedLeftRdd: RDD[(ReferenceRegion, T)]) = { - if (optPartitionMap.isDefined) { - (optPartitionMap.get, leftRdd) + val (preparedLeftRdd, partitionMap) = { + if (leftRdd.optPartitionMap.isDefined) { + (leftRdd.flattenRddByRegions, leftRdd.optPartitionMap.get) } else { - val sortedLeft = leftRdd.sortByKey() - val newPartitionMap = - sortedLeft.mapPartitions(getRegionBoundsFromPartition) - .collect - (newPartitionMap, sortedLeft) + val sortedLeft = leftRdd.sortLexicographically(storePartitionMap = true) + (sortedLeft.flattenRddByRegions, sortedLeft.optPartitionMap.get) } } @@ -115,7 +112,7 @@ sealed abstract class Closest[T: ClassTag, U: ClassTag, RT, RU] sorted = true) val assignedRightRdd = { - val firstPass = rightRdd.mapPartitions(iter => { + val firstPass = rightRdd.flattenRddByRegions.mapPartitions(iter => { iter.flatMap(f => { val rangeOfHits = partitionMapIntervals.get(f._1, requireOverlap = false) rangeOfHits.map(g => ((f._1, g._2), f._2)) @@ -181,16 +178,17 @@ sealed abstract class Closest[T: ClassTag, U: ClassTag, RT, RU] * @tparam T The type of the left records. * @tparam U The type of the right records. */ -case class ShuffleClosestRegion[T: ClassTag, U: ClassTag]( - protected val leftRdd: RDD[(ReferenceRegion, T)], - protected val rightRdd: RDD[(ReferenceRegion, U)], +case class ShuffleClosestRegion[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( + protected val leftRdd: GenomicRDD[T, U], + protected val rightRdd: GenomicRDD[X, Y], protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]], - protected val threshold: Long = Long.MaxValue) - extends Closest[T, U, T, Iterable[U]] - with VictimlessSetTheoryBetweenCollections[T, U, T, Iterable[U]] { + protected val threshold: Long = Long.MaxValue, + protected val optPartitions: Option[Int] = None) + extends Closest[T, U, X, Y, T, Iterable[X]] + with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Iterable[U])] = { + right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Iterable[X])] = { // if the left iterator is not empty, we have failed to correctly // partition the data. the right iterator is only allowed to be empty @@ -201,7 +199,7 @@ case class ShuffleClosestRegion[T: ClassTag, U: ClassTag]( } override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, Iterable[U])] = { + iter: Iterable[(ReferenceRegion, X)]): Iterable[(T, Iterable[X])] = { Iterable((currentLeft._2, iter.map(_._2))) } } diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala index c7afdc65bc..1ac0a2a10d 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala @@ -19,12 +19,13 @@ package org.bdgenomics.adam.rdd.settheory import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion +import org.bdgenomics.adam.rdd.GenomicRDD import scala.collection.mutable.ListBuffer +import scala.reflect.ClassTag -private[settheory] sealed abstract class SetTheory extends Serializable { +private[settheory] sealed trait SetTheory extends Serializable { protected val threshold: Long - protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] /** * The condition that should be met in order for the primitive to be @@ -80,11 +81,12 @@ private[settheory] trait SetTheoryPrimitive extends SetTheory { * @tparam RT The return type for the left side row data. * @tparam RU The return type for the right side row data. */ -private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] extends SetTheory { - - protected val leftRdd: RDD[(ReferenceRegion, T)] - protected val rightRdd: RDD[(ReferenceRegion, U)] +private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RU] + extends SetTheory { + protected val leftRdd: GenomicRDD[T, U] + protected val rightRdd: GenomicRDD[X, Y] + protected val optPartitions: Option[Int] /** * Post process and format the hits for a given left record. * @@ -93,7 +95,7 @@ private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] exte * @return The post processed hits. */ protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(RT, RU)] + iter: Iterable[(ReferenceRegion, X)]): Iterable[(RT, RU)] /** * The condition by which a candidate is removed from the cache. @@ -127,31 +129,30 @@ private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] exte * @return The formatted resulting RDD. */ protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] + right: Iterator[(ReferenceRegion, X)]): Iterator[(RT, RU)] /** * Prunes the cache based on the condition set in pruneCacheCondition. * * @see pruneCacheCondition - * * @param to The region to prune against. * @param cache The cache for this partition. */ protected def pruneCache(to: ReferenceRegion, - cache: SetTheoryCache[U, RT, RU]) + cache: SetTheoryCache[X, RT, RU]) /** * Advances the cache based on the condition set in advanceCacheCondition * * @see advanceCacheCondition - * * @param right The right buffered iterator to pull from. * @param until The region to compare against. * @param cache The cache for this partition. */ - protected def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], + protected def advanceCache(right: BufferedIterator[(ReferenceRegion, X)], until: ReferenceRegion, - cache: SetTheoryCache[U, RT, RU]) + cache: SetTheoryCache[X, RT, RU]) + /** * Computes all victims for the partition. * @@ -159,8 +160,8 @@ private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] exte * @param right The right iterator. * @return The finalized hits for this partition. */ - protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] + protected def finalizeHits(cache: SetTheoryCache[X, RT, RU], + right: BufferedIterator[(ReferenceRegion, X)]): Iterable[(RT, RU)] /** * Prepares and partitions the left and right. Makes no assumptions about the @@ -169,14 +170,14 @@ private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] exte * * @return The prepared and partitioned left and right RDD. */ - protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) + protected def prepare()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) /** * Computes the set theory primitive for the two RDDs. * * @return An RDD resulting from the primitive operation. */ - def compute(): RDD[(RT, RU)] = { + def compute()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): RDD[(RT, RU)] = { val (preparedLeft, preparedRight) = prepare() preparedLeft.zipPartitions(preparedRight)(makeIterator) } @@ -190,7 +191,7 @@ private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] exte * @return An iterator containing all processed hits. */ protected def processHits(currentLeft: (ReferenceRegion, T), - cache: SetTheoryCache[U, RT, RU]): Iterable[(RT, RU)] = { + cache: SetTheoryCache[X, RT, RU]): Iterable[(RT, RU)] = { val (currentLeftRegion, _) = currentLeft // post processing formats the hits for each individual type of join @@ -210,9 +211,9 @@ private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] exte * @return The resulting Iterator based on the primitive operation. */ protected def makeIterator(leftIter: Iterator[(ReferenceRegion, T)], - rightIter: Iterator[(ReferenceRegion, U)]): Iterator[(RT, RU)] = { + rightIter: Iterator[(ReferenceRegion, X)]): Iterator[(RT, RU)] = { - val cache = new SetTheoryCache[U, RT, RU] + val cache = new SetTheoryCache[X, RT, RU] if (leftIter.isEmpty || rightIter.isEmpty) { emptyFn(leftIter, rightIter) @@ -236,10 +237,10 @@ private[settheory] abstract class SetTheoryBetweenCollections[T, U, RT, RU] exte * @tparam T The left side row data. * @tparam U The right side row data. * @tparam RT The return type for the left side row data. - * @tparam RU The return type for the right side row data. + * @tparam RX The return type for the right side row data. */ -private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U, RT, RU] - extends SetTheoryBetweenCollections[T, U, RT, RU] { +private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { /** * Post processes the pruned records to format them appropriately. @@ -247,10 +248,10 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U, RT, RU] * @param pruned The pruned record. * @return The formatted, post processed record. */ - protected def postProcessPruned(pruned: U): (RT, RU) + protected def postProcessPruned(pruned: X): (RT, RX) override protected def pruneCache(to: ReferenceRegion, - cache: SetTheoryCache[U, RT, RU]) = { + cache: SetTheoryCache[X, RT, RX]) = { val toThreshold = to.pad(threshold) // remove everything from cache that will never again be joined @@ -289,9 +290,9 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U, RT, RU] cache.victimCache.trimStart(prunedAddition.size) } - override protected def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], + override protected def advanceCache(right: BufferedIterator[(ReferenceRegion, X)], until: ReferenceRegion, - cache: SetTheoryCache[U, RT, RU]) = { + cache: SetTheoryCache[X, RT, RX]) = { while (right.hasNext && advanceCacheCondition(right.head._1, until.pad(threshold))) { @@ -301,8 +302,8 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U, RT, RU] } } - override protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] = { + override protected def finalizeHits(cache: SetTheoryCache[X, RT, RX], + right: BufferedIterator[(ReferenceRegion, X)]): Iterable[(RT, RX)] = { cache.pruned ++ right.map(f => postProcessPruned(f._2)) } @@ -314,13 +315,13 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U, RT, RU] * @tparam T The left side row data. * @tparam U The right side row data. * @tparam RT The return type for the left side row data. - * @tparam RU The return type for the right side row data. + * @tparam RX The return type for the right side row data. */ -private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U, RT, RU] - extends SetTheoryBetweenCollections[T, U, RT, RU] { +private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { override protected def pruneCache(to: ReferenceRegion, - cache: SetTheoryCache[U, RT, RU]) = { + cache: SetTheoryCache[X, RT, RX]) = { cache.cache.trimStart({ val index = cache.cache.indexWhere(f => !pruneCacheCondition(f._1, to)) if (index <= 0) { @@ -331,16 +332,16 @@ private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U, RT, RU] }) } - override protected def advanceCache(right: BufferedIterator[(ReferenceRegion, U)], + override protected def advanceCache(right: BufferedIterator[(ReferenceRegion, X)], until: ReferenceRegion, - cache: SetTheoryCache[U, RT, RU]) = { + cache: SetTheoryCache[X, RT, RX]) = { while (right.hasNext && advanceCacheCondition(right.head._1, until)) { cache.cache += right.next } } - override protected def finalizeHits(cache: SetTheoryCache[U, RT, RU], - right: BufferedIterator[(ReferenceRegion, U)]): Iterable[(RT, RU)] = { + override protected def finalizeHits(cache: SetTheoryCache[X, RT, RX], + right: BufferedIterator[(ReferenceRegion, X)]): Iterable[(RT, RX)] = { // Victimless Set Theory drops the remaining records Iterable.empty } @@ -349,15 +350,15 @@ private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U, RT, RU] /** * Contains all the caching data for a set theory operation. * - * @tparam U The right side record type. + * @tparam X The right side record type. * @tparam RT The left side result type. - * @tparam RU The right side result type. + * @tparam RX The right side result type. */ -private[settheory] class SetTheoryCache[U, RT, RU] { +private[settheory] class SetTheoryCache[X, RT, RX] { // caches potential hits - val cache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty + val cache: ListBuffer[(ReferenceRegion, X)] = ListBuffer.empty // caches potential pruned and joined values - val victimCache: ListBuffer[(ReferenceRegion, U)] = ListBuffer.empty + val victimCache: ListBuffer[(ReferenceRegion, X)] = ListBuffer.empty // the pruned values that do not contain any hits from the left - val pruned: ListBuffer[(RT, RU)] = ListBuffer.empty + val pruned: ListBuffer[(RT, RX)] = ListBuffer.empty } diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index 967cea0032..e3088633b2 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -19,7 +19,7 @@ package org.bdgenomics.adam.rdd.settheory import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion -import org.bdgenomics.adam.rdd.ManualRegionPartitioner +import org.bdgenomics.adam.rdd.{ GenomicRDD, ManualRegionPartitioner } import org.bdgenomics.utils.interval.array.IntervalArray import scala.reflect.ClassTag @@ -29,10 +29,10 @@ import scala.reflect.ClassTag * @tparam T The type of the left records. * @tparam U The type of the right records. * @tparam RT The resulting type of the left after the join. - * @tparam RU The resulting type of the right after the join. + * @tparam RX The resulting type of the right after the join. */ -sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] - extends SetTheoryBetweenCollections[T, U, RT, RU] with SetTheoryPrimitive { +sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] with SetTheoryPrimitive { override protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, @@ -55,16 +55,21 @@ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] candidateRegion.compareTo(until) < 0 || candidateRegion.covers(until) } - override protected def prepare(): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, U)]) = { + override protected def prepare()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { + + val numPartitions = optPartitions.getOrElse(leftRdd.rdd.partitions.length) val (preparedLeft, destinationPartitionMap) = { - if (optPartitionMap.isDefined) { - (leftRdd, optPartitionMap.get) + if (leftRdd.optPartitionMap.isDefined && + leftRdd.rdd.partitions.length == numPartitions) { + (leftRdd.flattenRddByRegions(), leftRdd.optPartitionMap.get) } else { - val sortedLeft = leftRdd.sortByKey() + val sortedLeft = + leftRdd.sortLexicographically(numPartitions, storePartitionMap = true) + val partitionMap = - sortedLeft.mapPartitions(getRegionBoundsFromPartition).collect - (sortedLeft, partitionMap) + sortedLeft.optPartitionMap.get + (sortedLeft.flattenRddByRegions(), partitionMap) } } @@ -101,12 +106,13 @@ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] sorted = true) val preparedRight = { - rightRdd.mapPartitions(iter => { - iter.flatMap(f => { - val intervals = partitionMapIntervals.get(f._1.pad(threshold), requireOverlap = false) - intervals.map(g => ((f._1, g._2), f._2)) - }) - }, preservesPartitioning = true) + rightRdd.flattenRddByRegions() + .mapPartitions(iter => { + iter.flatMap(f => { + val intervals = partitionMapIntervals.get(f._1.pad(threshold), requireOverlap = false) + intervals.map(g => ((f._1, g._2), f._2)) + }) + }, preservesPartitioning = true) .repartitionAndSortWithinPartitions( ManualRegionPartitioner(destinationPartitionMap.length)) .map(f => (f._1._1, f._2)) @@ -121,27 +127,25 @@ sealed abstract class ShuffleRegionJoin[T: ClassTag, U: ClassTag, RT, RU] * * @param leftRdd The left RDD. * @param rightRdd The right RDD. - * @param optPartitionMap An optional partition map defining the left RDD - * partition bounds. * @param threshold The threshold for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ -case class InnerShuffleRegionJoin[T: ClassTag, U: ClassTag]( - leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, - threshold: Long = 0L) - extends ShuffleRegionJoin[T, U, T, U] - with VictimlessSetTheoryBetweenCollections[T, U, T, U] { +case class InnerShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( + protected val leftRdd: GenomicRDD[T, U], + protected val rightRdd: GenomicRDD[X, Y], + protected val threshold: Long = 0L, + protected val optPartitions: Option[Int] = None) + extends ShuffleRegionJoin[T, U, X, Y, T, X] + with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, X] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(T, U)] = { + right: Iterator[(ReferenceRegion, X)]): Iterator[(T, X)] = { Iterator.empty } override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, U)] = { + iter: Iterable[(ReferenceRegion, X)]): Iterable[(T, X)] = { iter.map(f => (currentLeft._2, f._2)) } } @@ -151,27 +155,25 @@ case class InnerShuffleRegionJoin[T: ClassTag, U: ClassTag]( * * @param leftRdd The left RDD. * @param rightRdd The right RDD. - * @param optPartitionMap An optional partition map defining the left RDD - * partition bounds. * @param threshold The threshold for the join. * @tparam T The type of the left records. * @tparam U THe type of the right records. */ -case class InnerShuffleRegionJoinAndGroupByLeft[T: ClassTag, U: ClassTag]( - leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, - threshold: Long = 0L) - extends ShuffleRegionJoin[T, U, T, Iterable[U]] - with VictimlessSetTheoryBetweenCollections[T, U, T, Iterable[U]] { +case class InnerShuffleRegionJoinAndGroupByLeft[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( + protected val leftRdd: GenomicRDD[T, U], + protected val rightRdd: GenomicRDD[X, Y], + protected val threshold: Long = 0L, + protected val optPartitions: Option[Int] = None) + extends ShuffleRegionJoin[T, U, X, Y, T, Iterable[X]] + with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Iterable[U])] = { + right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Iterable[X])] = { Iterator.empty } override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, Iterable[U])] = { + iter: Iterable[(ReferenceRegion, X)]): Iterable[(T, Iterable[X])] = { if (iter.nonEmpty) { // group all hits for currentLeft into an iterable @@ -187,27 +189,25 @@ case class InnerShuffleRegionJoinAndGroupByLeft[T: ClassTag, U: ClassTag]( * * @param leftRdd The left RDD. * @param rightRdd The right RDD. - * @param optPartitionMap An optional partition map defining the left RDD - * partition bounds. * @param threshold The threshold for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ -case class LeftOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( - leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, - threshold: Long = 0L) - extends ShuffleRegionJoin[T, U, T, Option[U]] - with VictimlessSetTheoryBetweenCollections[T, U, T, Option[U]] { +case class LeftOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( + protected val leftRdd: GenomicRDD[T, U], + protected val rightRdd: GenomicRDD[X, Y], + protected val threshold: Long = 0L, + protected val optPartitions: Option[Int] = None) + extends ShuffleRegionJoin[T, U, X, Y, T, Option[X]] + with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, Option[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(T, Option[U])] = { + right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Option[X])] = { left.map(t => (t._2, None)) } override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(T, Option[U])] = { + iter: Iterable[(ReferenceRegion, X)]): Iterable[(T, Option[X])] = { if (iter.nonEmpty) { // left has some hits iter.map(f => (currentLeft._2, Some(f._2))) @@ -223,27 +223,25 @@ case class LeftOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( * * @param leftRdd The left RDD. * @param rightRdd The right RDD. - * @param optPartitionMap An optional partition map defining the left RDD - * partition bounds. * @param threshold The threshold for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ -case class RightOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( - leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, - threshold: Long = 0L) - extends ShuffleRegionJoin[T, U, Option[T], U] - with SetTheoryBetweenCollectionsWithVictims[T, U, Option[T], U] { +case class RightOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( + protected val leftRdd: GenomicRDD[T, U], + protected val rightRdd: GenomicRDD[X, Y], + protected val threshold: Long = 0L, + protected val optPartitions: Option[Int] = None) + extends ShuffleRegionJoin[T, U, X, Y, Option[T], X] + with SetTheoryBetweenCollectionsWithVictims[T, U, X, Y, Option[T], X] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], U)] = { + right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], X)] = { right.map(u => (None, u._2)) } override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(Option[T], U)] = { + iter: Iterable[(ReferenceRegion, X)]): Iterable[(Option[T], X)] = { if (iter.nonEmpty) { // group all hits for currentLeft into an iterable iter.map(f => (Some(currentLeft._2), f._2)) @@ -252,7 +250,7 @@ case class RightOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( } } - override protected def postProcessPruned(pruned: U): (Option[T], U) = { + override protected def postProcessPruned(pruned: X): (Option[T], X) = { (None, pruned) } } @@ -262,27 +260,25 @@ case class RightOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( * * @param leftRdd The left RDD. * @param rightRdd The right RDD. - * @param optPartitionMap An optional partition map defining the left RDD - * partition bounds. * @param threshold The threshold for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ -case class FullOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( - leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, - threshold: Long = 0L) - extends ShuffleRegionJoin[T, U, Option[T], Option[U]] - with SetTheoryBetweenCollectionsWithVictims[T, U, Option[T], Option[U]] { +case class FullOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( + protected val leftRdd: GenomicRDD[T, U], + protected val rightRdd: GenomicRDD[X, Y], + protected val threshold: Long = 0L, + protected val optPartitions: Option[Int] = None) + extends ShuffleRegionJoin[T, U, X, Y, Option[T], Option[X]] + with SetTheoryBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Option[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], Option[U])] = { + right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], Option[X])] = { left.map(t => (Some(t._2), None)) ++ right.map(u => (None, Some(u._2))) } override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(Option[T], Option[U])] = { + iter: Iterable[(ReferenceRegion, X)]): Iterable[(Option[T], Option[X])] = { if (iter.nonEmpty) { // formatting these as options for the full outer join iter.map(u => (Some(currentLeft._2), Some(u._2))) @@ -292,7 +288,7 @@ case class FullOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( } } - override protected def postProcessPruned(pruned: U): (Option[T], Option[U]) = { + override protected def postProcessPruned(pruned: X): (Option[T], Option[X]) = { (None, Some(pruned)) } } @@ -302,33 +298,31 @@ case class FullOuterShuffleRegionJoin[T: ClassTag, U: ClassTag]( * * @param leftRdd The left RDD. * @param rightRdd The right RDD. - * @param optPartitionMap An optional partition map defining the left RDD - * partition bounds. * @param threshold The threshold for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ -case class RightOuterShuffleRegionJoinAndGroupByLeft[T: ClassTag, U: ClassTag]( - leftRdd: RDD[(ReferenceRegion, T)], - rightRdd: RDD[(ReferenceRegion, U)], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None, - threshold: Long = 0L) - extends ShuffleRegionJoin[T, U, Option[T], Iterable[U]] - with SetTheoryBetweenCollectionsWithVictims[T, U, Option[T], Iterable[U]] { +case class RightOuterShuffleRegionJoinAndGroupByLeft[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( + protected val leftRdd: GenomicRDD[T, U], + protected val rightRdd: GenomicRDD[X, Y], + protected val threshold: Long = 0L, + protected val optPartitions: Option[Int] = None) + extends ShuffleRegionJoin[T, U, X, Y, Option[T], Iterable[X]] + with SetTheoryBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, U)]): Iterator[(Option[T], Iterable[U])] = { + right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], Iterable[X])] = { left.map(v => (Some(v._2), Iterable.empty)) ++ right.map(v => (None, Iterable(v._2))) } override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, U)]): Iterable[(Option[T], Iterable[U])] = { + iter: Iterable[(ReferenceRegion, X)]): Iterable[(Option[T], Iterable[X])] = { Iterable((Some(currentLeft._2), iter.map(_._2))) } - override protected def postProcessPruned(pruned: U): (Option[T], Iterable[U]) = { + override protected def postProcessPruned(pruned: X): (Option[T], Iterable[X]) = { (None, Iterable(pruned)) } } diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala index c6df529d92..921acce36b 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala @@ -17,7 +17,8 @@ */ package org.bdgenomics.adam.rdd -import org.bdgenomics.adam.models.{ ReferenceRegion, SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.models.{ RecordGroupDictionary, ReferenceRegion, SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD import org.bdgenomics.adam.rdd.settheory.InnerShuffleRegionJoin import org.bdgenomics.adam.util.ADAMFunSuite import org.bdgenomics.formats.avro.{ AlignmentRecord, Contig } @@ -51,11 +52,14 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { val record2 = AlignmentRecord.newBuilder(built).setStart(3L).setEnd(4L).build() val baseRecord = AlignmentRecord.newBuilder(built).setCigar("4M").setEnd(5L).build() - val baseRdd = sc.parallelize(Seq(baseRecord), 1).keyBy(ReferenceRegion.unstranded(_)) - val recordsRdd = sc.parallelize(Seq(record1, record2), 1).keyBy(ReferenceRegion.unstranded(_)) + val baseRdd = sc.parallelize(Seq(baseRecord), 1) + val recordsRdd = sc.parallelize(Seq(record1, record2), 1) + + val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) + val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) assert( - InnerShuffleRegionJoin[AlignmentRecord, AlignmentRecord](baseRdd, recordsRdd) + InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) .compute() .aggregate(true)( InnerShuffleRegionJoinSuite.merge, @@ -64,7 +68,7 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { ) assert( - InnerShuffleRegionJoin[AlignmentRecord, AlignmentRecord](baseRdd, recordsRdd) + InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) .compute() .count() === 2 ) @@ -104,11 +108,14 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { val baseRecord1 = AlignmentRecord.newBuilder(builtRef1).setCigar("4M").setEnd(5L).build() val baseRecord2 = AlignmentRecord.newBuilder(builtRef2).setCigar("4M").setEnd(5L).build() - val baseRdd = sc.parallelize(Seq(baseRecord1, baseRecord2), 1).keyBy(ReferenceRegion.unstranded(_)) - val recordsRdd = sc.parallelize(Seq(record1, record2, record3), 1).keyBy(ReferenceRegion.unstranded(_)) + val baseRdd = sc.parallelize(Seq(baseRecord1, baseRecord2), 1) + val recordsRdd = sc.parallelize(Seq(record1, record2, record3), 1) + + val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) + val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) assert( - InnerShuffleRegionJoin[AlignmentRecord, AlignmentRecord](baseRdd, recordsRdd) + InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) .compute() .aggregate(true)( InnerShuffleRegionJoinSuite.merge, @@ -117,7 +124,7 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { ) assert({ - InnerShuffleRegionJoin[AlignmentRecord, AlignmentRecord](baseRdd, recordsRdd) + InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) .compute() .count() === 3 } diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala index fa192e2add..9e16ac4585 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala @@ -23,6 +23,7 @@ import org.bdgenomics.adam.models.{ SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD import org.bdgenomics.adam.rdd.settheory.LeftOuterShuffleRegionJoin import org.bdgenomics.formats.avro.AlignmentRecord @@ -38,9 +39,9 @@ class LeftOuterShuffleRegionJoinSuite(partitionMap: Seq[Option[(ReferenceRegion, SequenceRecord("chr2", 15, url = "tes=t://chrom2")) } - def runJoin(leftRdd: RDD[(ReferenceRegion, AlignmentRecord)], - rightRdd: RDD[(ReferenceRegion, AlignmentRecord)]): RDD[(Option[AlignmentRecord], AlignmentRecord)] = { - LeftOuterShuffleRegionJoin[AlignmentRecord, AlignmentRecord](rightRdd, leftRdd) + def runJoin(leftRdd: AlignmentRecordRDD, + rightRdd: AlignmentRecordRDD): RDD[(Option[AlignmentRecord], AlignmentRecord)] = { + LeftOuterShuffleRegionJoin(rightRdd, leftRdd) .compute().map(_.swap) } } diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala index 6791452253..e4cdb99a1e 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala @@ -17,16 +17,16 @@ */ package org.bdgenomics.adam.rdd -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.bdgenomics.adam.models.ReferenceRegion +import org.bdgenomics.adam.models.{ RecordGroupDictionary, SequenceDictionary, ReferenceRegion } +import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD import org.bdgenomics.adam.util.ADAMFunSuite import org.bdgenomics.formats.avro.{ AlignmentRecord, Contig } trait OuterRegionJoinSuite extends ADAMFunSuite { - def runJoin(leftRdd: RDD[(ReferenceRegion, AlignmentRecord)], - rightRdd: RDD[(ReferenceRegion, AlignmentRecord)]): RDD[(Option[AlignmentRecord], AlignmentRecord)] + def runJoin(leftRdd: AlignmentRecordRDD, + rightRdd: AlignmentRecordRDD): RDD[(Option[AlignmentRecord], AlignmentRecord)] sparkTest("Ensure same reference regions get passed together") { val contig = Contig.newBuilder @@ -53,10 +53,22 @@ trait OuterRegionJoinSuite extends ADAMFunSuite { .setEnd(11L) .build() - val rdd1 = sc.parallelize(Seq(record1, record3)).keyBy(ReferenceRegion.unstranded(_)) - val rdd2 = sc.parallelize(Seq(record2, record4)).keyBy(ReferenceRegion.unstranded(_)) + val baseRdd = sc.parallelize(Seq(record1, record3)) + val recordsRdd = sc.parallelize(Seq(record2, record4)) - val jrdd = runJoin(rdd1, rdd2).cache() + val baseGenomicRdd = + AlignmentRecordRDD(baseRdd, + SequenceDictionary.empty, + RecordGroupDictionary.empty, + None) + + val recordsGenomicRdd = + AlignmentRecordRDD(recordsRdd, + SequenceDictionary.empty, + RecordGroupDictionary.empty, + None) + + val jrdd = runJoin(baseGenomicRdd, recordsGenomicRdd).cache() assert(jrdd.count === 2) assert(jrdd.filter(_._1.isDefined).count === 1) @@ -86,10 +98,22 @@ trait OuterRegionJoinSuite extends ADAMFunSuite { val baseRecord = AlignmentRecord.newBuilder(built).setCigar("4M").setEnd(5L).build() val record3 = AlignmentRecord.newBuilder(built).setStart(6L).setEnd(7L).build() - val baseRdd = sc.parallelize(Seq(baseRecord)).keyBy(ReferenceRegion.unstranded(_)) - val recordsRdd = sc.parallelize(Seq(record1, record2, record3)).keyBy(ReferenceRegion.unstranded(_)) + val baseRdd = sc.parallelize(Seq(baseRecord)) + val recordsRdd = sc.parallelize(Seq(record1, record2, record3)) + + val baseGenomicRdd = + AlignmentRecordRDD(baseRdd, + SequenceDictionary.empty, + RecordGroupDictionary.empty, + None) + + val recordsGenomicRdd = + AlignmentRecordRDD(recordsRdd, + SequenceDictionary.empty, + RecordGroupDictionary.empty, + None) - val jrdd = runJoin(baseRdd, recordsRdd).cache + val jrdd = runJoin(baseGenomicRdd, recordsGenomicRdd).cache() assert(jrdd.count() === 3) assert(jrdd.filter(_._1.isDefined).count == 2) diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala index 4c148aa405..1495c8a764 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala @@ -23,6 +23,7 @@ import org.bdgenomics.adam.models.{ SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD import org.bdgenomics.adam.rdd.settheory.RightOuterShuffleRegionJoin import org.bdgenomics.formats.avro.AlignmentRecord @@ -38,8 +39,8 @@ class RightOuterShuffleRegionJoinSuite(partitionMap: Seq[Option[(ReferenceRegion SequenceRecord("chr2", 15, url = "tes=t://chrom2")) } - def runJoin(leftRdd: RDD[(ReferenceRegion, AlignmentRecord)], - rightRdd: RDD[(ReferenceRegion, AlignmentRecord)]): RDD[(Option[AlignmentRecord], AlignmentRecord)] = { - RightOuterShuffleRegionJoin[AlignmentRecord, AlignmentRecord](leftRdd, rightRdd).compute() + def runJoin(leftRdd: AlignmentRecordRDD, + rightRdd: AlignmentRecordRDD): RDD[(Option[AlignmentRecord], AlignmentRecord)] = { + RightOuterShuffleRegionJoin(leftRdd, rightRdd).compute() } } diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterTreeRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterTreeRegionJoinSuite.scala index f01a25c82b..13550d3cab 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterTreeRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterTreeRegionJoinSuite.scala @@ -19,17 +19,21 @@ package org.bdgenomics.adam.rdd import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion -import org.bdgenomics.adam.rdd.read.AlignmentRecordArray +import org.bdgenomics.adam.rdd.read.{ AlignmentRecordRDD, AlignmentRecordArray } import org.bdgenomics.formats.avro.AlignmentRecord import org.bdgenomics.utils.interval.array.IntervalArray class RightOuterTreeRegionJoinSuite extends OuterRegionJoinSuite { - def runJoin(leftRdd: RDD[(ReferenceRegion, AlignmentRecord)], - rightRdd: RDD[(ReferenceRegion, AlignmentRecord)]): RDD[(Option[AlignmentRecord], AlignmentRecord)] = { + def runJoin(leftRdd: AlignmentRecordRDD, + rightRdd: AlignmentRecordRDD): RDD[(Option[AlignmentRecord], AlignmentRecord)] = { + + val keyedLeft = leftRdd.flattenRddByRegions() + val keyedRight = rightRdd.flattenRddByRegions() + RightOuterTreeRegionJoin[AlignmentRecord, AlignmentRecord]().broadcastAndJoin( - IntervalArray[ReferenceRegion, AlignmentRecord](leftRdd, + IntervalArray[ReferenceRegion, AlignmentRecord](keyedLeft, AlignmentRecordArray.apply(_, _)), - rightRdd) + keyedRight) } } diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala index 301ad33295..4078a39d0f 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala @@ -28,9 +28,11 @@ class ClosestSuite extends ADAMFunSuite { .sortLexicographically(storePartitionMap = true) val rightFile = sc.loadBed(resourceUrl("intersect_with_overlap_01.bed").getFile) - val x = ShuffleClosestRegion(leftFile.flattenRddByRegions(), - rightFile.flattenRddByRegions(), leftFile.optPartitionMap) - .compute() + val x = + ShuffleClosestRegion(leftFile, + rightFile, + leftFile.optPartitionMap).compute() + val result = x.map(f => (ReferenceRegion(f._1.getContigName, f._1.getStart, f._1.getEnd), f._2.map(g => ReferenceRegion(g.getContigName, g.getStart, g.getEnd)))) From 564bc7d24498fb396a8066d1b820df77bcf2b93b Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Tue, 13 Jun 2017 13:37:10 -0700 Subject: [PATCH 08/13] More complete docs, adding edge test case --- .../adam/models/ReferenceRegion.scala | 7 - .../adam/rdd/settheory/Closest.scala | 164 +++++++++++------- .../adam/rdd/settheory/SetTheory.scala | 88 ++++------ .../rdd/settheory/ShuffleRegionJoin.scala | 19 +- .../resources/intersect_with_overlap_01.bed | 1 + .../adam/rdd/settheory/ClosestSuite.scala | 6 +- 6 files changed, 154 insertions(+), 131 deletions(-) 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 61de43c044..cbbfc6b26e 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 @@ -84,13 +84,6 @@ object ReferenceRegion { implicit def orderingForPositions = RegionOrdering implicit def orderingForOptionalPositions = OptionalRegionOrdering - /** - * Creates an empty ReferenceRegion. - * - * @return An empty ReferenceRegion. - */ - private[adam] val empty: ReferenceRegion = ReferenceRegion("", 0L, 0L) - /** * Creates a reference region that starts at the beginning of a contig. * diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala index 53943b2cb7..91600dd7f9 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala @@ -32,52 +32,80 @@ import scala.reflect.ClassTag * @tparam RX The resulting type of the right after the operation. */ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] - with SetTheoryPrimitive { - - var currentClosest: ReferenceRegion = ReferenceRegion.empty + extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { override protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX], threshold: Long = 0L): Boolean = { - firstRegion.unstrandedDistance(currentClosest) + + // we must maintain this invariant throughout the computation + cache.closest.isDefined && + // we want to identify all the regions that share the same distance as our + // current closest. + firstRegion.unstrandedDistance(cache.closest.get) .exists(_ == firstRegion.unstrandedDistance(secondRegion).getOrElse(Long.MaxValue)) } override protected def pruneCacheCondition(cachedRegion: ReferenceRegion, - to: ReferenceRegion): Boolean = { - if (cachedRegion.referenceName != to.referenceName) { - true - } else { + to: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX]): Boolean = { + + // we must maintain this invariant throughout the computation + cache.closest.isDefined && + // we want to prune in the case that the unstranded distance between the + // current query region is greater than our current closest + cachedRegion.referenceName == to.referenceName && to.unstrandedDistance(cachedRegion).get > - to.unstrandedDistance(currentClosest).getOrElse(Long.MaxValue) - } + to.unstrandedDistance(cache.closest.get).getOrElse(Long.MaxValue) } override protected def advanceCacheCondition(candidateRegion: ReferenceRegion, - until: ReferenceRegion): Boolean = { + until: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX]): Boolean = { + + // if our current closest isn't on the same reference name, we don't + // consider it the closest, thus we have no current closest + if (cache.closest.isDefined && + cache.closest.get.referenceName != candidateRegion.referenceName) { + + cache.closest = None + } + + // if the reference names don't match, we don't consider them the closest, + // unless we have no current closest + if (candidateRegion.referenceName != until.referenceName && + cache.closest.isDefined) { - if (candidateRegion.referenceName != until.referenceName) { false - } else if (until.referenceName != currentClosest.referenceName || + // current closest must be set if there is no current closest. This + // prevents us from dropping results when we don't have any records of that + // reference name in the dataset. otherwise, we set current closest if it + // is closer than our current + } else if (cache.closest.isEmpty || + until.referenceName != cache.closest.get.referenceName || until.unstrandedDistance(candidateRegion).get <= - until.unstrandedDistance(currentClosest).getOrElse(Long.MaxValue)) { - - currentClosest = candidateRegion + until.unstrandedDistance(cache.closest.get).getOrElse(Long.MaxValue)) { + // this object can be short lived, but the overhead should be low for + // options + cache.closest = Some(candidateRegion) true } else { + // we reach this on the region immediately after we have passed the + // closest region false } } - override protected def prepare()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { + override protected def prepare()( + implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { val (preparedLeftRdd, partitionMap) = { if (leftRdd.optPartitionMap.isDefined) { - (leftRdd.flattenRddByRegions, leftRdd.optPartitionMap.get) + (leftRdd.flattenRddByRegions(), leftRdd.optPartitionMap.get) } else { val sortedLeft = leftRdd.sortLexicographically(storePartitionMap = true) - (sortedLeft.flattenRddByRegions, sortedLeft.optPartitionMap.get) + (sortedLeft.flattenRddByRegions(), sortedLeft.optPartitionMap.get) } } @@ -85,43 +113,51 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] // the zipWithIndex gives us the destination partition ID .zipWithIndex .filter(_._1.nonEmpty) - .map(f => (f._1.get, f._2)) - .map(g => { + .map(f => (f._1.get, f._2)).map(g => { + // first region for the bound + val rr = g._1._1 + // second region for the bound + val secondrr = g._1._2 // in the case where we span multiple referenceNames - if (g._1._1.referenceName != g._1._2.referenceName) { + if (rr.referenceName != g._1._2.referenceName) { // create a ReferenceRegion that goes to the end of the chromosome - (ReferenceRegion( - g._1._1.referenceName, - g._1._1.start, - g._1._1.end), - g._2) + (ReferenceRegion(rr.referenceName, rr.start, rr.end), g._2) } else { // otherwise we just have the ReferenceRegion span from partition // start to end - (ReferenceRegion( - g._1._1.referenceName, - g._1._1.start, - g._1._2.end), - g._2) + (ReferenceRegion(rr.referenceName, rr.start, secondrr.end), g._2) } }) + // we use an interval array to quickly look up the destination partitions val partitionMapIntervals = IntervalArray( adjustedPartitionMapWithIndex, adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, sorted = true) - val assignedRightRdd = { - val firstPass = rightRdd.flattenRddByRegions.mapPartitions(iter => { - iter.flatMap(f => { - val rangeOfHits = partitionMapIntervals.get(f._1, requireOverlap = false) - rangeOfHits.map(g => ((f._1, g._2), f._2)) - }) - }, preservesPartitioning = true) - + val assignedRightRdd: RDD[((ReferenceRegion, Int), X)] = { + // copartitioning for the closest is tricky, and requires that we handle + // unique edge cases, described below. + // the first pass gives us the initial destination partitions. + val firstPass = rightRdd.flattenRddByRegions() + .mapPartitions(iter => { + iter.flatMap(f => { + val rangeOfHits = partitionMapIntervals.get(f._1, requireOverlap = false) + rangeOfHits.map(g => ((f._1, g._2), f._2)) + }) + }, preservesPartitioning = true) + + // we have to find the partitions that don't have right data going there + // so we can send the flanking partitions' data there val partitionsWithoutData = partitionMap.indices.filterNot(firstPass.map(_._1._2).distinct().collect.contains) + // this gives us a list of partitions that are sending copies of their + // data and the number of nodes to send to. a negative number of nodes + // indicates that the data needs to be sent to lower numbered nodes, a + // positive number indicates that the data needs to be sent to higher + // numbered nodes. the way this is written, it will handle an arbitrary + // run of empty partitions. val partitionsToSend = partitionsWithoutData.foldLeft(List.empty[List[Int]])((b, a) => { if (b.isEmpty) { List(List(a)) @@ -130,29 +166,35 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] } else { b.:+(List(a)) } + // we end up getting all the data from both flanking nodes. we use the + // length here so we know how many destinations we have resulting from + // runs of empty partitions. }).flatMap(f => List((f.head - 1, f.length), (f.last + 1, -1 * f.length))) firstPass.flatMap(f => { - val index = partitionsToSend.indexWhere(_._1 == f._1._2) - if (index < 0) { - List(f) - } else { - if (partitionsToSend(index)._2 < 0) { - (partitionsToSend(index)._2 to 0) - .map(g => ((f._1._1, f._1._2 + g), f._2)) + // extract the destinations for this data record + val destinations = partitionsToSend.filter(g => g._1 == f._1._2) + // we use an inclusive range to specify all destinations + val duplicatedRecords = { + if (destinations.length == 1) { + // the data is only going to lower numbered nodes + if (destinations.head._2 < 0) { + destinations.head._2 to 0 + // the data is only going to higher numbered nodes + } else { + 0 to destinations.head._2 + } + // the data is going to higher and lower numbered nodes + } else if (destinations.length == 2) { + destinations.last._2 to destinations.head._2 + // the data is only going to its original destination } else { - (0 to partitionsToSend(index)._2) - .map(g => ((f._1._1, f._1._2 + g), f._2)) ++ { - if (index == partitionsToSend.lastIndexWhere(_._1 == f._1._2)) { - List() - } else { - val endIndex = partitionsToSend.lastIndexWhere(_._1 == f._1._2) - (partitionsToSend(endIndex)._2 to -1) - .map(g => ((f._1._1, f._1._2 + g), f._2)) - } - } + 0 to 0 } - } + // add the destination + }.map(g => ((f._1._1, f._1._2 + g), f._2)) + + duplicatedRecords }) } @@ -172,16 +214,14 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] * * @param leftRdd The left RDD. * @param rightRdd The right RDD. - * @param optPartitionMap An optional partition map defining the left RDD - * partition bounds. * @param threshold The maximum distance allowed for the closest. + * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ case class ShuffleClosestRegion[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( protected val leftRdd: GenomicRDD[T, U], protected val rightRdd: GenomicRDD[X, Y], - protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]], protected val threshold: Long = Long.MaxValue, protected val optPartitions: Option[Int] = None) extends Closest[T, U, X, Y, T, Iterable[X]] diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala index 1ac0a2a10d..c11fd7f190 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala @@ -23,7 +23,8 @@ import org.bdgenomics.adam.rdd.GenomicRDD import scala.collection.mutable.ListBuffer import scala.reflect.ClassTag -private[settheory] sealed trait SetTheory extends Serializable { +private[settheory] trait SetTheory[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends Serializable { protected val threshold: Long @@ -39,50 +40,20 @@ private[settheory] sealed trait SetTheory extends Serializable { */ protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX], distanceThreshold: Long = 0L): Boolean } -private[settheory] trait SetTheoryPrimitive extends SetTheory { - - /** - * Gets the partition bounds from a ReferenceRegion keyed Iterator - * - * @param iter The data on a given partition. ReferenceRegion keyed - * @return The bounds of the ReferenceRegions on that partition, in an Iterator - */ - protected def getRegionBoundsFromPartition[X]( - iter: Iterator[(ReferenceRegion, X)]): Iterator[Option[(ReferenceRegion, ReferenceRegion)]] = { - - if (iter.isEmpty) { - // This means that there is no data on the partition, so we have no bounds - Iterator(None) - } else { - val firstRegion = iter.next - val lastRegion = - if (iter.hasNext) { - // we have to make sure we get the full bounds of this partition, this - // includes any extremely long regions. we include the firstRegion for - // the case that the first region is extremely long - (iter ++ Iterator(firstRegion)).maxBy(f => (f._1.referenceName, f._1.end, f._1.start)) - // only one record on this partition, so this is the extent of the bounds - } else { - firstRegion - } - Iterator(Some((firstRegion._1, lastRegion._1))) - } - } -} - /** * The parent class for all inter-collection set theory operations. * * @tparam T The left side row data. * @tparam U The right side row data. * @tparam RT The return type for the left side row data. - * @tparam RU The return type for the right side row data. + * @tparam RX The return type for the right side row data. */ -private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RU] - extends SetTheory { +private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetTheory[T, U, X, Y, RT, RX] { protected val leftRdd: GenomicRDD[T, U] protected val rightRdd: GenomicRDD[X, Y] @@ -95,7 +66,7 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * @return The post processed hits. */ protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, X)]): Iterable[(RT, RU)] + iter: Iterable[(ReferenceRegion, X)]): Iterable[(RT, RX)] /** * The condition by which a candidate is removed from the cache. @@ -107,7 +78,8 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * False for all regions that should remain in the cache. */ protected def pruneCacheCondition(cachedRegion: ReferenceRegion, - to: ReferenceRegion): Boolean + to: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX]): Boolean /** * The condition by which a candidate region is added to the cache. @@ -119,7 +91,8 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * False for regions that should not be added to the cache. */ protected def advanceCacheCondition(candidateRegion: ReferenceRegion, - until: ReferenceRegion): Boolean + until: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX]): Boolean /** * Handles the situation where the left or right iterator is empty. @@ -129,7 +102,7 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * @return The formatted resulting RDD. */ protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, X)]): Iterator[(RT, RU)] + right: Iterator[(ReferenceRegion, X)]): Iterator[(RT, RX)] /** * Prunes the cache based on the condition set in pruneCacheCondition. @@ -139,7 +112,7 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * @param cache The cache for this partition. */ protected def pruneCache(to: ReferenceRegion, - cache: SetTheoryCache[X, RT, RU]) + cache: SetTheoryCache[X, RT, RX]) /** * Advances the cache based on the condition set in advanceCacheCondition @@ -151,7 +124,7 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] */ protected def advanceCache(right: BufferedIterator[(ReferenceRegion, X)], until: ReferenceRegion, - cache: SetTheoryCache[X, RT, RU]) + cache: SetTheoryCache[X, RT, RX]) /** * Computes all victims for the partition. @@ -160,8 +133,8 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * @param right The right iterator. * @return The finalized hits for this partition. */ - protected def finalizeHits(cache: SetTheoryCache[X, RT, RU], - right: BufferedIterator[(ReferenceRegion, X)]): Iterable[(RT, RU)] + protected def finalizeHits(cache: SetTheoryCache[X, RT, RX], + right: BufferedIterator[(ReferenceRegion, X)]): Iterable[(RT, RX)] /** * Prepares and partitions the left and right. Makes no assumptions about the @@ -177,7 +150,7 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * * @return An RDD resulting from the primitive operation. */ - def compute()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): RDD[(RT, RU)] = { + def compute()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): RDD[(RT, RX)] = { val (preparedLeft, preparedRight) = prepare() preparedLeft.zipPartitions(preparedRight)(makeIterator) } @@ -191,14 +164,14 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * @return An iterator containing all processed hits. */ protected def processHits(currentLeft: (ReferenceRegion, T), - cache: SetTheoryCache[X, RT, RU]): Iterable[(RT, RU)] = { + cache: SetTheoryCache[X, RT, RX]): Iterable[(RT, RX)] = { val (currentLeftRegion, _) = currentLeft // post processing formats the hits for each individual type of join postProcessHits(currentLeft, cache.cache.filter(y => { // everything that overlaps the left region is a hit - condition(currentLeftRegion, y._1, threshold) + condition(currentLeftRegion, y._1, cache, threshold) })) } @@ -211,9 +184,9 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * @return The resulting Iterator based on the primitive operation. */ protected def makeIterator(leftIter: Iterator[(ReferenceRegion, T)], - rightIter: Iterator[(ReferenceRegion, X)]): Iterator[(RT, RU)] = { + rightIter: Iterator[(ReferenceRegion, X)]): Iterator[(RT, RX)] = { - val cache = new SetTheoryCache[X, RT, RU] + val cache = new SetTheoryCache[X, RT, RX] if (leftIter.isEmpty || rightIter.isEmpty) { emptyFn(leftIter, rightIter) @@ -258,7 +231,7 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U <: GenomicR cache.cache.trimStart({ val trimLocation = cache.cache - .indexWhere(f => !pruneCacheCondition(f._1, toThreshold)) + .indexWhere(f => !pruneCacheCondition(f._1, toThreshold, cache)) if (trimLocation < 0) { 0 @@ -271,7 +244,7 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U <: GenomicR // the the current left val cacheAddition = cache.victimCache - .takeWhile(f => !pruneCacheCondition(f._1, toThreshold)) + .takeWhile(f => !pruneCacheCondition(f._1, toThreshold, cache)) cache.cache ++= cacheAddition // remove the values from the victimCache that were just added to cache @@ -280,8 +253,7 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U <: GenomicR // add to pruned any values that do not have any matches to a left // and perform post processing to format the new pruned values val prunedAddition = - cache - .victimCache + cache.victimCache .takeWhile(f => f._1.compareTo(toThreshold) <= 0) cache.pruned ++= prunedAddition @@ -295,7 +267,7 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U <: GenomicR cache: SetTheoryCache[X, RT, RX]) = { while (right.hasNext && - advanceCacheCondition(right.head._1, until.pad(threshold))) { + advanceCacheCondition(right.head._1, until.pad(threshold), cache)) { val x = right.next() cache.victimCache += ((x._1, x._2)) @@ -323,7 +295,7 @@ private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U <: GenomicRD override protected def pruneCache(to: ReferenceRegion, cache: SetTheoryCache[X, RT, RX]) = { cache.cache.trimStart({ - val index = cache.cache.indexWhere(f => !pruneCacheCondition(f._1, to)) + val index = cache.cache.indexWhere(f => !pruneCacheCondition(f._1, to, cache)) if (index <= 0) { 0 } else { @@ -335,7 +307,7 @@ private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U <: GenomicRD override protected def advanceCache(right: BufferedIterator[(ReferenceRegion, X)], until: ReferenceRegion, cache: SetTheoryCache[X, RT, RX]) = { - while (right.hasNext && advanceCacheCondition(right.head._1, until)) { + while (right.hasNext && advanceCacheCondition(right.head._1, until, cache)) { cache.cache += right.next } } @@ -355,10 +327,16 @@ private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U <: GenomicRD * @tparam RX The right side result type. */ private[settheory] class SetTheoryCache[X, RT, RX] { + // caches potential hits val cache: ListBuffer[(ReferenceRegion, X)] = ListBuffer.empty + // caches potential pruned and joined values val victimCache: ListBuffer[(ReferenceRegion, X)] = ListBuffer.empty + // the pruned values that do not contain any hits from the left val pruned: ListBuffer[(RT, RX)] = ListBuffer.empty + + // the closest values for the hits. + var closest: Option[ReferenceRegion] = None } diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index e3088633b2..e270d476e5 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -32,10 +32,11 @@ import scala.reflect.ClassTag * @tparam RX The resulting type of the right after the join. */ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] with SetTheoryPrimitive { + extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { override protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX], distanceThreshold: Long = 0L): Boolean = { firstRegion.isNearby(secondRegion, @@ -44,21 +45,25 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y } override protected def pruneCacheCondition(cachedRegion: ReferenceRegion, - to: ReferenceRegion): Boolean = { + to: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX]): Boolean = { cachedRegion.compareTo(to) < 0 && !cachedRegion.covers(to) } override protected def advanceCacheCondition(candidateRegion: ReferenceRegion, - until: ReferenceRegion): Boolean = { + until: ReferenceRegion, + cache: SetTheoryCache[X, RT, RX]): Boolean = { candidateRegion.compareTo(until) < 0 || candidateRegion.covers(until) } override protected def prepare()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { + // default to current partition number if user did not specify val numPartitions = optPartitions.getOrElse(leftRdd.rdd.partitions.length) + // we don't know if the left is sorted unless it has a partition map val (preparedLeft, destinationPartitionMap) = { if (leftRdd.optPartitionMap.isDefined && leftRdd.rdd.partitions.length == numPartitions) { @@ -109,7 +114,9 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y rightRdd.flattenRddByRegions() .mapPartitions(iter => { iter.flatMap(f => { + // we pad by the threshold here to ensure that our invariant is met val intervals = partitionMapIntervals.get(f._1.pad(threshold), requireOverlap = false) + // for each index identified in intervals, create a record intervals.map(g => ((f._1, g._2), f._2)) }) }, preservesPartitioning = true) @@ -128,6 +135,7 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y * @param leftRdd The left RDD. * @param rightRdd The right RDD. * @param threshold The threshold for the join. + * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ @@ -156,6 +164,7 @@ case class InnerShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X * @param leftRdd The left RDD. * @param rightRdd The right RDD. * @param threshold The threshold for the join. + * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. * @tparam U THe type of the right records. */ @@ -190,6 +199,7 @@ case class InnerShuffleRegionJoinAndGroupByLeft[T, U <: GenomicRDD[T, U], X, Y < * @param leftRdd The left RDD. * @param rightRdd The right RDD. * @param threshold The threshold for the join. + * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ @@ -224,6 +234,7 @@ case class LeftOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicR * @param leftRdd The left RDD. * @param rightRdd The right RDD. * @param threshold The threshold for the join. + * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ @@ -261,6 +272,7 @@ case class RightOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: Genomic * @param leftRdd The left RDD. * @param rightRdd The right RDD. * @param threshold The threshold for the join. + * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ @@ -299,6 +311,7 @@ case class FullOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicR * @param leftRdd The left RDD. * @param rightRdd The right RDD. * @param threshold The threshold for the join. + * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. * @tparam U The type of the right records. */ diff --git a/adam-core/src/test/resources/intersect_with_overlap_01.bed b/adam-core/src/test/resources/intersect_with_overlap_01.bed index 245afe4913..7c9d84469b 100644 --- a/adam-core/src/test/resources/intersect_with_overlap_01.bed +++ b/adam-core/src/test/resources/intersect_with_overlap_01.bed @@ -1,3 +1,4 @@ +chr0 1 10 CpG:_222 chr1 135000 135444 CpG:_116 chr1 135124 135563 CpG:_39 chr1 135333 135777 CpG:_29 diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala index 4078a39d0f..b60f1c43ba 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala @@ -28,10 +28,7 @@ class ClosestSuite extends ADAMFunSuite { .sortLexicographically(storePartitionMap = true) val rightFile = sc.loadBed(resourceUrl("intersect_with_overlap_01.bed").getFile) - val x = - ShuffleClosestRegion(leftFile, - rightFile, - leftFile.optPartitionMap).compute() + val x = ShuffleClosestRegion(leftFile, rightFile).compute() val result = x.map(f => (ReferenceRegion(f._1.getContigName, f._1.getStart, f._1.getEnd), @@ -60,6 +57,7 @@ class ClosestSuite extends ADAMFunSuite { (ReferenceRegion("chr1", 886356, 886602), ListBuffer(ReferenceRegion("chr1", 886356, 886602))), (ReferenceRegion("chr1", 894313, 902654), ListBuffer(ReferenceRegion("chr1", 894313, 902654)))) + println(result.zip(correctOutput).filter(f => f._1 !== f._2).mkString("\n")) assert(result === correctOutput) } } From a271b45fd4ffd0bf964ae5da2dbf946b160c65c7 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Tue, 13 Jun 2017 14:58:46 -0700 Subject: [PATCH 09/13] Created PartitionMap class to interact with partitionmap outside GenomicRDD --- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 54 ++++++++++++++++++- .../adam/rdd/settheory/Closest.scala | 35 +++--------- .../rdd/settheory/ShuffleRegionJoin.scala | 40 ++------------ 3 files changed, 63 insertions(+), 66 deletions(-) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index 305e416385..dd9f02d0c6 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -154,7 +154,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { // The (ReferenceRegion, ReferenceRegion) tuple contains the bounds of the // partition, such that the lowest start is first and the highest end is // second. - private[rdd] val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] + protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] assert(optPartitionMap.isEmpty || optPartitionMap.exists(_.length == rdd.partitions.length), @@ -162,6 +162,8 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val isSorted: Boolean = optPartitionMap.isDefined + private[rdd] val partitionMap = PartitionMap(optPartitionMap) + /** * Repartitions all data in rdd and distributes it as evenly as possible * into the number of partitions provided. @@ -1190,7 +1192,7 @@ abstract class AvroGenomicRDD[T <% IndexedRecord: Manifest, U <: AvroGenomicRDD[ protected def savePartitionMap(filePath: String): Unit = { if (isSorted) { // converting using json4s - val jsonString = "partitionMap" -> optPartitionMap.get.toSeq.map(f => + val jsonString = "partitionMap" -> partitionMap.get.toSeq.map(f => if (f.isEmpty) { ("ReferenceRegion1" -> "None") ~ ("ReferenceRegion2" -> "None") } else { @@ -1307,6 +1309,54 @@ abstract class AvroGenomicRDD[T <% IndexedRecord: Manifest, U <: AvroGenomicRDD[ } } +private[rdd] case class PartitionMap(private val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) { + + lazy val get: Array[Option[(ReferenceRegion, ReferenceRegion)]] = { + optPartitionMap.get + } + + def isEmpty: Boolean = { + optPartitionMap.isEmpty + } + + def isDefined: Boolean = { + optPartitionMap.isDefined + } + + def exists(f: (Array[Option[(ReferenceRegion, ReferenceRegion)]]) => Boolean): Boolean = { + optPartitionMap.exists(f) + } + + def toIntervalArray(): IntervalArray[ReferenceRegion, Int] = { + + val adjustedPartitionMapWithIndex = + // the zipWithIndex gives us the destination partition ID + get.zipWithIndex + .filter(_._1.nonEmpty) + .map(f => (f._1.get, f._2)).map(g => { + // first region for the bound + val rr = g._1._1 + // second region for the bound + val secondrr = g._1._2 + // in the case where we span multiple referenceNames + if (rr.referenceName != g._1._2.referenceName) { + // create a ReferenceRegion that goes to the end of the chromosome + (ReferenceRegion(rr.referenceName, rr.start, rr.end), g._2) + } else { + // otherwise we just have the ReferenceRegion span from partition + // start to end + (ReferenceRegion(rr.referenceName, rr.start, secondrr.end), g._2) + } + }) + + // we use an interval array to quickly look up the destination partitions + IntervalArray( + adjustedPartitionMapWithIndex, + adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, + sorted = true) + } +} + /** * A trait for genomic data that is not aligned to a reference (e.g., raw reads). */ diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala index 91600dd7f9..5564dadea0 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala @@ -101,39 +101,16 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { val (preparedLeftRdd, partitionMap) = { - if (leftRdd.optPartitionMap.isDefined) { - (leftRdd.flattenRddByRegions(), leftRdd.optPartitionMap.get) + if (leftRdd.partitionMap.isDefined) { + (leftRdd.flattenRddByRegions(), leftRdd.partitionMap) } else { val sortedLeft = leftRdd.sortLexicographically(storePartitionMap = true) - (sortedLeft.flattenRddByRegions(), sortedLeft.optPartitionMap.get) + (sortedLeft.flattenRddByRegions(), sortedLeft.partitionMap) } } - val adjustedPartitionMapWithIndex = partitionMap - // the zipWithIndex gives us the destination partition ID - .zipWithIndex - .filter(_._1.nonEmpty) - .map(f => (f._1.get, f._2)).map(g => { - // first region for the bound - val rr = g._1._1 - // second region for the bound - val secondrr = g._1._2 - // in the case where we span multiple referenceNames - if (rr.referenceName != g._1._2.referenceName) { - // create a ReferenceRegion that goes to the end of the chromosome - (ReferenceRegion(rr.referenceName, rr.start, rr.end), g._2) - } else { - // otherwise we just have the ReferenceRegion span from partition - // start to end - (ReferenceRegion(rr.referenceName, rr.start, secondrr.end), g._2) - } - }) - // we use an interval array to quickly look up the destination partitions - val partitionMapIntervals = IntervalArray( - adjustedPartitionMapWithIndex, - adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, - sorted = true) + val partitionMapIntervals = partitionMap.toIntervalArray() val assignedRightRdd: RDD[((ReferenceRegion, Int), X)] = { // copartitioning for the closest is tricky, and requires that we handle @@ -150,7 +127,7 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] // we have to find the partitions that don't have right data going there // so we can send the flanking partitions' data there val partitionsWithoutData = - partitionMap.indices.filterNot(firstPass.map(_._1._2).distinct().collect.contains) + partitionMap.get.indices.filterNot(firstPass.map(_._1._2).distinct().collect.contains) // this gives us a list of partitions that are sending copies of their // data and the number of nodes to send to. a negative number of nodes @@ -201,7 +178,7 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] val preparedRightRdd = assignedRightRdd .repartitionAndSortWithinPartitions( - ManualRegionPartitioner(partitionMap.length)) + ManualRegionPartitioner(partitionMap.get.length)) // return to an RDD[(ReferenceRegion, T)], removing the partition ID .map(f => (f._1._1, f._2)) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala index e270d476e5..c7c2c05691 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala @@ -65,50 +65,20 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y // we don't know if the left is sorted unless it has a partition map val (preparedLeft, destinationPartitionMap) = { - if (leftRdd.optPartitionMap.isDefined && + if (leftRdd.partitionMap.isDefined && leftRdd.rdd.partitions.length == numPartitions) { - (leftRdd.flattenRddByRegions(), leftRdd.optPartitionMap.get) + (leftRdd.flattenRddByRegions(), leftRdd.partitionMap) } else { val sortedLeft = leftRdd.sortLexicographically(numPartitions, storePartitionMap = true) - val partitionMap = - sortedLeft.optPartitionMap.get + val partitionMap = sortedLeft.partitionMap (sortedLeft.flattenRddByRegions(), partitionMap) } } - val adjustedPartitionMapWithIndex = - // the zipWithIndex gives us the destination partition ID - destinationPartitionMap.flatten.zipWithIndex.map(g => { - val (firstRegion, secondRegion, index) = (g._1._1, g._1._2, g._2) - // in the case where we span multiple referenceNames using - // IntervalArray.get with requireOverlap set to false will assign all - // the remaining regions to this partition, in addition to all the - // regions up to the start of the next partition. - if (firstRegion.referenceName != secondRegion.referenceName) { - - // the first region is enough to represent the partition for - // IntervalArray.get. - (firstRegion, index) - } else { - // otherwise we just have the ReferenceRegion span from partition - // lower bound to upper bound. - // We cannot use the firstRegion bounds here because we may end up - // dropping data if it doesn't map anywhere. - (ReferenceRegion( - firstRegion.referenceName, - firstRegion.start, - secondRegion.end), - index) - } - }) - // convert to an IntervalArray for fast range query - val partitionMapIntervals = IntervalArray( - adjustedPartitionMapWithIndex, - adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, - sorted = true) + val partitionMapIntervals = destinationPartitionMap.toIntervalArray() val preparedRight = { rightRdd.flattenRddByRegions() @@ -121,7 +91,7 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y }) }, preservesPartitioning = true) .repartitionAndSortWithinPartitions( - ManualRegionPartitioner(destinationPartitionMap.length)) + ManualRegionPartitioner(destinationPartitionMap.get.length)) .map(f => (f._1._1, f._2)) } From 1fc2e743e3511c2ed1ee136cad88d4bdedd8e0d7 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Tue, 13 Jun 2017 15:06:43 -0700 Subject: [PATCH 10/13] Package rename settheory -> sets, class rename SetTheory -> SetOperation --- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 2 +- .../adam/rdd/{settheory => sets}/Closest.scala | 6 +++--- .../SetOperation.scala} | 18 +++++++++--------- .../ShuffleRegionJoin.scala | 16 ++++++++-------- .../rdd/{settheory => sets}/ClosestSuite.scala | 2 +- .../InnerShuffleRegionJoinSuite.scala | 7 +++---- .../LeftOuterShuffleRegionJoinSuite.scala | 10 +++------- .../RightOuterShuffleRegionJoinSuite.scala | 10 +++------- 8 files changed, 31 insertions(+), 40 deletions(-) rename adam-core/src/main/scala/org/bdgenomics/adam/rdd/{settheory => sets}/Closest.scala (98%) rename adam-core/src/main/scala/org/bdgenomics/adam/rdd/{settheory/SetTheory.scala => sets/SetOperation.scala} (94%) rename adam-core/src/main/scala/org/bdgenomics/adam/rdd/{settheory => sets}/ShuffleRegionJoin.scala (95%) rename adam-core/src/test/scala/org/bdgenomics/adam/rdd/{settheory => sets}/ClosestSuite.scala (98%) rename adam-core/src/test/scala/org/bdgenomics/adam/rdd/{ => sets}/InnerShuffleRegionJoinSuite.scala (94%) rename adam-core/src/test/scala/org/bdgenomics/adam/rdd/{ => sets}/LeftOuterShuffleRegionJoinSuite.scala (88%) rename adam-core/src/test/scala/org/bdgenomics/adam/rdd/{ => sets}/RightOuterShuffleRegionJoinSuite.scala (88%) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index dd9f02d0c6..1abc5b9c8f 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -34,7 +34,7 @@ import org.bdgenomics.adam.models.{ ReferenceRegion, SequenceDictionary } -import org.bdgenomics.adam.rdd.settheory._ +import org.bdgenomics.adam.rdd.sets._ import org.bdgenomics.formats.avro.{ Contig, RecordGroup => RecordGroupMetadata, diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/Closest.scala similarity index 98% rename from adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala rename to adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/Closest.scala index 5564dadea0..e4032df0a9 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/Closest.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/Closest.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd.settheory +package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion @@ -32,7 +32,7 @@ import scala.reflect.ClassTag * @tparam RX The resulting type of the right after the operation. */ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { + extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { override protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, @@ -202,7 +202,7 @@ case class ShuffleClosestRegion[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, protected val threshold: Long = Long.MaxValue, protected val optPartitions: Option[Int] = None) extends Closest[T, U, X, Y, T, Iterable[X]] - with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, Iterable[X]] { + with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Iterable[X])] = { diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/SetOperation.scala similarity index 94% rename from adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala rename to adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/SetOperation.scala index c11fd7f190..f626306739 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/SetTheory.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/SetOperation.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd.settheory +package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion @@ -23,7 +23,7 @@ import org.bdgenomics.adam.rdd.GenomicRDD import scala.collection.mutable.ListBuffer import scala.reflect.ClassTag -private[settheory] trait SetTheory[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] +private[sets] trait SetOperation[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] extends Serializable { protected val threshold: Long @@ -52,8 +52,8 @@ private[settheory] trait SetTheory[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[ * @tparam RT The return type for the left side row data. * @tparam RX The return type for the right side row data. */ -private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetTheory[T, U, X, Y, RT, RX] { +private[rdd] abstract class SetOperationBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetOperation[T, U, X, Y, RT, RX] { protected val leftRdd: GenomicRDD[T, U] protected val rightRdd: GenomicRDD[X, Y] @@ -212,8 +212,8 @@ private[rdd] abstract class SetTheoryBetweenCollections[T, U <: GenomicRDD[T, U] * @tparam RT The return type for the left side row data. * @tparam RX The return type for the right side row data. */ -private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { +private[sets] trait SetOperationBetweenCollectionsWithVictims[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { /** * Post processes the pruned records to format them appropriately. @@ -289,8 +289,8 @@ private[settheory] trait SetTheoryBetweenCollectionsWithVictims[T, U <: GenomicR * @tparam RT The return type for the left side row data. * @tparam RX The return type for the right side row data. */ -private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { +private[sets] trait VictimlessSetOperationBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { override protected def pruneCache(to: ReferenceRegion, cache: SetTheoryCache[X, RT, RX]) = { @@ -326,7 +326,7 @@ private[settheory] trait VictimlessSetTheoryBetweenCollections[T, U <: GenomicRD * @tparam RT The left side result type. * @tparam RX The right side result type. */ -private[settheory] class SetTheoryCache[X, RT, RX] { +private[sets] class SetTheoryCache[X, RT, RX] { // caches potential hits val cache: ListBuffer[(ReferenceRegion, X)] = ListBuffer.empty diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala similarity index 95% rename from adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala rename to adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala index c7c2c05691..0cadd8392d 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/settheory/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd.settheory +package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion @@ -32,7 +32,7 @@ import scala.reflect.ClassTag * @tparam RX The resulting type of the right after the join. */ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetTheoryBetweenCollections[T, U, X, Y, RT, RX] { + extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { override protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, @@ -115,7 +115,7 @@ case class InnerShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) extends ShuffleRegionJoin[T, U, X, Y, T, X] - with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, X] { + with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, X] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(T, X)] = { @@ -144,7 +144,7 @@ case class InnerShuffleRegionJoinAndGroupByLeft[T, U <: GenomicRDD[T, U], X, Y < protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) extends ShuffleRegionJoin[T, U, X, Y, T, Iterable[X]] - with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, Iterable[X]] { + with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Iterable[X])] = { @@ -179,7 +179,7 @@ case class LeftOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicR protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) extends ShuffleRegionJoin[T, U, X, Y, T, Option[X]] - with VictimlessSetTheoryBetweenCollections[T, U, X, Y, T, Option[X]] { + with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, Option[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Option[X])] = { @@ -214,7 +214,7 @@ case class RightOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: Genomic protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) extends ShuffleRegionJoin[T, U, X, Y, Option[T], X] - with SetTheoryBetweenCollectionsWithVictims[T, U, X, Y, Option[T], X] { + with SetOperationBetweenCollectionsWithVictims[T, U, X, Y, Option[T], X] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], X)] = { @@ -252,7 +252,7 @@ case class FullOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicR protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) extends ShuffleRegionJoin[T, U, X, Y, Option[T], Option[X]] - with SetTheoryBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Option[X]] { + with SetOperationBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Option[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], Option[X])] = { @@ -291,7 +291,7 @@ case class RightOuterShuffleRegionJoinAndGroupByLeft[T, U <: GenomicRDD[T, U], X protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) extends ShuffleRegionJoin[T, U, X, Y, Option[T], Iterable[X]] - with SetTheoryBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Iterable[X]] { + with SetOperationBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], Iterable[X])] = { diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/ClosestSuite.scala similarity index 98% rename from adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala rename to adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/ClosestSuite.scala index b60f1c43ba..2f83d03724 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/settheory/ClosestSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/ClosestSuite.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd.settheory +package org.bdgenomics.adam.rdd.sets import org.bdgenomics.adam.models.ReferenceRegion import org.bdgenomics.adam.util.ADAMFunSuite diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala similarity index 94% rename from adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala rename to adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala index 921acce36b..8710fc5a4a 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/InnerShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd +package org.bdgenomics.adam.rdd.sets -import org.bdgenomics.adam.models.{ RecordGroupDictionary, ReferenceRegion, SequenceDictionary, SequenceRecord } +import org.bdgenomics.adam.models.{RecordGroupDictionary, ReferenceRegion, SequenceDictionary, SequenceRecord} import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD -import org.bdgenomics.adam.rdd.settheory.InnerShuffleRegionJoin import org.bdgenomics.adam.util.ADAMFunSuite -import org.bdgenomics.formats.avro.{ AlignmentRecord, Contig } +import org.bdgenomics.formats.avro.{AlignmentRecord, Contig} class InnerShuffleRegionJoinSuite extends ADAMFunSuite { val partitionSize = 3 diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala similarity index 88% rename from adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala rename to adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala index 9e16ac4585..15511a0ae6 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/LeftOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala @@ -15,16 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd +package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD -import org.bdgenomics.adam.models.{ - ReferenceRegion, - SequenceDictionary, - SequenceRecord -} +import org.bdgenomics.adam.models.{ReferenceRegion, SequenceDictionary, SequenceRecord} +import org.bdgenomics.adam.rdd.OuterRegionJoinSuite import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD -import org.bdgenomics.adam.rdd.settheory.LeftOuterShuffleRegionJoin import org.bdgenomics.formats.avro.AlignmentRecord class LeftOuterShuffleRegionJoinSuite(partitionMap: Seq[Option[(ReferenceRegion, ReferenceRegion)]]) diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala similarity index 88% rename from adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala rename to adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala index 1495c8a764..5845980787 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/RightOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala @@ -15,16 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.bdgenomics.adam.rdd +package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD -import org.bdgenomics.adam.models.{ - ReferenceRegion, - SequenceDictionary, - SequenceRecord -} +import org.bdgenomics.adam.models.{ReferenceRegion, SequenceDictionary, SequenceRecord} +import org.bdgenomics.adam.rdd.OuterRegionJoinSuite import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD -import org.bdgenomics.adam.rdd.settheory.RightOuterShuffleRegionJoin import org.bdgenomics.formats.avro.AlignmentRecord class RightOuterShuffleRegionJoinSuite(partitionMap: Seq[Option[(ReferenceRegion, ReferenceRegion)]]) From 2a0324116561517a79cb9d49738a65aba6043d4a Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Tue, 13 Jun 2017 15:22:27 -0700 Subject: [PATCH 11/13] Moving join tests into sets/tests --- .../adam/rdd/sets/InnerShuffleRegionJoinSuite.scala | 4 ++-- .../adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala | 2 +- .../adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala index 8710fc5a4a..1df308cebf 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala @@ -17,10 +17,10 @@ */ package org.bdgenomics.adam.rdd.sets -import org.bdgenomics.adam.models.{RecordGroupDictionary, ReferenceRegion, SequenceDictionary, SequenceRecord} +import org.bdgenomics.adam.models.{ RecordGroupDictionary, ReferenceRegion, SequenceDictionary, SequenceRecord } import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD import org.bdgenomics.adam.util.ADAMFunSuite -import org.bdgenomics.formats.avro.{AlignmentRecord, Contig} +import org.bdgenomics.formats.avro.{ AlignmentRecord, Contig } class InnerShuffleRegionJoinSuite extends ADAMFunSuite { val partitionSize = 3 diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala index 15511a0ae6..57d6c93653 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/LeftOuterShuffleRegionJoinSuite.scala @@ -18,7 +18,7 @@ package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD -import org.bdgenomics.adam.models.{ReferenceRegion, SequenceDictionary, SequenceRecord} +import org.bdgenomics.adam.models.{ ReferenceRegion, SequenceDictionary, SequenceRecord } import org.bdgenomics.adam.rdd.OuterRegionJoinSuite import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD import org.bdgenomics.formats.avro.AlignmentRecord diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala index 5845980787..dd091649d2 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/RightOuterShuffleRegionJoinSuite.scala @@ -18,7 +18,7 @@ package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD -import org.bdgenomics.adam.models.{ReferenceRegion, SequenceDictionary, SequenceRecord} +import org.bdgenomics.adam.models.{ ReferenceRegion, SequenceDictionary, SequenceRecord } import org.bdgenomics.adam.rdd.OuterRegionJoinSuite import org.bdgenomics.adam.rdd.read.AlignmentRecordRDD import org.bdgenomics.formats.avro.AlignmentRecord From 39140924eb92eb6ea62dc8f69f7017431ee7efb9 Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Wed, 21 Jun 2017 14:33:42 -0700 Subject: [PATCH 12/13] Changing the way PartitionMap works, allowing RDDs for inner join --- .../org/bdgenomics/adam/rdd/ADAMContext.scala | 12 +- .../org/bdgenomics/adam/rdd/GenomicRDD.scala | 149 +++--------------- .../bdgenomics/adam/rdd/PartitionMap.scala | 123 +++++++++++++++ .../contig/NucleotideContigFragmentRDD.scala | 8 +- .../adam/rdd/feature/CoverageRDD.scala | 10 +- .../adam/rdd/feature/FeatureRDD.scala | 8 +- .../adam/rdd/fragment/FragmentRDD.scala | 10 +- .../adam/rdd/read/AlignmentRecordRDD.scala | 26 +-- .../bdgenomics/adam/rdd/sets/Closest.scala | 23 ++- .../adam/rdd/sets/SetOperation.scala | 33 ++-- .../adam/rdd/sets/ShuffleRegionJoin.scala | 128 ++++++++++----- .../adam/rdd/variant/GenotypeRDD.scala | 14 +- .../adam/rdd/variant/VariantContextRDD.scala | 14 +- .../adam/rdd/variant/VariantRDD.scala | 14 +- .../adam/rdd/OuterRegionJoinSuite.scala | 12 +- .../adam/rdd/SortedGenomicRDDSuite.scala | 22 +-- .../adam/rdd/sets/ClosestSuite.scala | 2 +- .../sets/InnerShuffleRegionJoinSuite.scala | 34 +++- 18 files changed, 372 insertions(+), 270 deletions(-) create mode 100644 adam-core/src/main/scala/org/bdgenomics/adam/rdd/PartitionMap.scala 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 7eff588dc8..7d30f3195b 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 @@ -917,7 +917,7 @@ class ADAMContext(@transient val sc: SparkContext) extends Serializable with Log // convert avro to sequence dictionary val rgd = loadAvroRecordGroupDictionary(pathName) - AlignmentRecordRDD(rdd, sd, rgd, optPartitionMap = extractPartitionMap(pathName)) + AlignmentRecordRDD(rdd, sd, rgd, isSorted = extractPartitionMap(pathName).nonEmpty) } /** @@ -1216,7 +1216,7 @@ class ADAMContext(@transient val sc: SparkContext) extends Serializable with Log val samples = loadAvroSamples(pathName) GenotypeRDD(rdd, sd, samples, headers, - optPartitionMap = extractPartitionMap(pathName)) + isSorted = extractPartitionMap(pathName).nonEmpty) } /** @@ -1242,7 +1242,7 @@ class ADAMContext(@transient val sc: SparkContext) extends Serializable with Log val headers = loadHeaderLines(pathName) VariantRDD(rdd, sd, headers, - optPartitionMap = extractPartitionMap(pathName)) + isSorted = extractPartitionMap(pathName).nonEmpty) } /** @@ -1527,7 +1527,7 @@ class ADAMContext(@transient val sc: SparkContext) extends Serializable with Log val sd = loadAvroSequenceDictionary(pathName) val rdd = loadParquet[Feature](pathName, optPredicate, optProjection) - FeatureRDD(rdd, sd, optPartitionMap = extractPartitionMap(pathName)) + FeatureRDD(rdd, sd, isSorted = extractPartitionMap(pathName).nonEmpty) } /** @@ -1549,7 +1549,7 @@ class ADAMContext(@transient val sc: SparkContext) extends Serializable with Log val sd = loadAvroSequenceDictionary(pathName) val rdd = loadParquet[NucleotideContigFragment](pathName, optPredicate, optProjection) - NucleotideContigFragmentRDD(rdd, sd, optPartitionMap = extractPartitionMap(pathName)) + NucleotideContigFragmentRDD(rdd, sd, isSorted = extractPartitionMap(pathName).nonEmpty) } /** @@ -1577,7 +1577,7 @@ class ADAMContext(@transient val sc: SparkContext) extends Serializable with Log // load fragment data from parquet val rdd = loadParquet[Fragment](pathName, optPredicate, optProjection) - FragmentRDD(rdd, sd, rgd, optPartitionMap = extractPartitionMap(pathName)) + FragmentRDD(rdd, sd, rgd, isSorted = extractPartitionMap(pathName).nonEmpty) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala index 1abc5b9c8f..005f846685 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/GenomicRDD.scala @@ -145,24 +145,13 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { replaceRdd(tFn(rdd)) } - // The partition map is structured as follows: - // The outer option is for whether or not there is a partition map. - // - This is None in the case that we don't know the bounds on each - // partition. - // The Array is the length of the number of partitions. - // The inner option is in case there is no data on a partition. - // The (ReferenceRegion, ReferenceRegion) tuple contains the bounds of the - // partition, such that the lowest start is first and the highest end is - // second. - protected val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] - - assert(optPartitionMap.isEmpty || - optPartitionMap.exists(_.length == rdd.partitions.length), - "Partition map length differs from number of partitions.") + val isSorted: Boolean - val isSorted: Boolean = optPartitionMap.isDefined + private[rdd] lazy val partitionMap = PartitionMap(this) - private[rdd] val partitionMap = PartitionMap(optPartitionMap) + assert(partitionMap.isEmpty || + partitionMap.exists(_.length == rdd.partitions.length), + "Partition map length differs from number of partitions.") /** * Repartitions all data in rdd and distributes it as evenly as possible @@ -192,12 +181,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { .repartitionAndSortWithinPartitions( ManualRegionPartitioner(finalPartitionNumber.toInt)) - val newPartitionMap = finalPartitionedRDD.mapPartitions(iter => - getRegionBoundsFromPartition( - iter.map(f => (f._1._1, f._2))), - preservesPartitioning = true).collect - - replaceRdd(finalPartitionedRDD.values, Some(newPartitionMap)) + replaceRdd(finalPartitionedRDD.values, isSorted = true) } /** @@ -273,8 +257,6 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * lexicographically. * * @param partitions The number of partitions for the new RDD. - * @param storePartitionMap A Boolean flag to determine whether to store the - * partition bounds from the resulting RDD. * @param storageLevel The level at which to persist the resulting RDD. * @param stringency The level of ValidationStringency to enforce. * @return Returns a new RDD containing sorted data. @@ -283,42 +265,27 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { * @see sort */ def sortLexicographically(partitions: Int = rdd.partitions.length, - storePartitionMap: Boolean = false, storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, stringency: ValidationStringency = ValidationStringency.STRICT)( implicit tTag: ClassTag[T]): U = { - val partitionedRdd = rdd.flatMap(elem => { + replaceRdd(rdd.sortBy(elem => { val coveredRegions = getReferenceRegions(elem) // We don't use ValidationStringency here because multimapped elements // break downstream methods. require(coveredRegions.size == 1, - "Cannot sort RDD containing a multimapped element. %s covers %s.".format( - elem, coveredRegions.mkString(","))) + "Cannot sort RDD containing a multimapped element. %s covers %s." + .format(elem, coveredRegions.mkString(","))) if (coveredRegions.isEmpty) { - throwWarnOrNone[(ReferenceRegion, T)]( + throwWarnOrNone[(ReferenceRegion)]( "Cannot sort RDD containing an unmapped element %s.".format(elem), stringency) } else { - Some(coveredRegions.head, elem) - } - }).sortByKey(ascending = true, numPartitions = partitions) - - partitionedRdd.persist(storageLevel) - - storePartitionMap match { - case true => { - val newPartitionMap = partitionedRdd.mapPartitions(iter => - getRegionBoundsFromPartition(iter), preservesPartitioning = true).collect - - replaceRdd(partitionedRdd.values, Some(newPartitionMap)) - } - case false => { - replaceRdd(partitionedRdd.values) + Some(coveredRegions.head) } - } + }, ascending = true, numPartitions = partitions), isSorted = true) } /** @@ -474,7 +441,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { protected def replaceRdd( newRdd: RDD[T], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): U + isSorted: Boolean = false): U protected def getReferenceRegions(elem: T): Seq[ReferenceRegion] @@ -500,7 +467,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { // do any of these overlap with our query region? regions.exists(_.overlaps(query)) - }), optPartitionMap) + }), isSorted) } /** @@ -519,7 +486,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { querys.map(query => { regions.exists(_.overlaps(query)) }).fold(false)((a, b) => a || b) - }), optPartitionMap) + }), isSorted) } protected def buildTree( @@ -813,7 +780,7 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { val combinedSequences = sequences ++ genomicRdd.sequences GenericGenomicRDD[(T, X)]( - InnerShuffleRegionJoin[T, U, X, Y](this, genomicRdd, threshold, optPartitions) + InnerShuffleRegionJoinOnGenomicRDD[T, U, X, Y](this, genomicRdd, threshold, optPartitions) .compute(), combinedSequences, kv => { @@ -1031,41 +998,13 @@ trait GenomicRDD[T, U <: GenomicRDD[T, U]] extends Logging { kv._1.toSeq.flatMap(v => getReferenceRegions(v))).toSeq }).asInstanceOf[GenomicRDD[(Option[T], Iterable[X]), Z]] } - - /** - * Gets the partition bounds from a ReferenceRegion keyed Iterator. - * - * @param iter The data on a given partition. ReferenceRegion keyed. - * @return The bounds of the ReferenceRegions on that partition, in an Iterator. - */ - private def getRegionBoundsFromPartition( - iter: Iterator[(ReferenceRegion, T)]): Iterator[Option[(ReferenceRegion, ReferenceRegion)]] = { - - if (iter.isEmpty) { - // This means that there is no data on the partition, so we have no bounds - Iterator(None) - } else { - val firstRegion = iter.next - val lastRegion = - if (iter.hasNext) { - // we have to make sure we get the full bounds of this partition, this - // includes any extremely long regions. we include the firstRegion for - // the case that the first region is extremely long - (iter ++ Iterator(firstRegion)).maxBy(f => (f._1.referenceName, f._1.end, f._1.start)) - } else { - // only one record on this partition, so this is the extent of the bounds - firstRegion - } - Iterator(Some((firstRegion._1, lastRegion._1))) - } - } } -private case class GenericGenomicRDD[T]( +case class GenericGenomicRDD[T]( rdd: RDD[T], sequences: SequenceDictionary, regionFn: T => Seq[ReferenceRegion], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None)( + isSorted: Boolean = false)( implicit tTag: ClassTag[T]) extends GenomicRDD[T, GenericGenomicRDD[T]] { def union(rdds: GenericGenomicRDD[T]*): GenericGenomicRDD[T] = { @@ -1087,9 +1026,9 @@ private case class GenericGenomicRDD[T]( protected def replaceRdd( newRdd: RDD[T], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): GenericGenomicRDD[T] = { + isSorted: Boolean = false): GenericGenomicRDD[T] = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + copy(rdd = newRdd, isSorted = isSorted) } } @@ -1309,54 +1248,6 @@ abstract class AvroGenomicRDD[T <% IndexedRecord: Manifest, U <: AvroGenomicRDD[ } } -private[rdd] case class PartitionMap(private val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) { - - lazy val get: Array[Option[(ReferenceRegion, ReferenceRegion)]] = { - optPartitionMap.get - } - - def isEmpty: Boolean = { - optPartitionMap.isEmpty - } - - def isDefined: Boolean = { - optPartitionMap.isDefined - } - - def exists(f: (Array[Option[(ReferenceRegion, ReferenceRegion)]]) => Boolean): Boolean = { - optPartitionMap.exists(f) - } - - def toIntervalArray(): IntervalArray[ReferenceRegion, Int] = { - - val adjustedPartitionMapWithIndex = - // the zipWithIndex gives us the destination partition ID - get.zipWithIndex - .filter(_._1.nonEmpty) - .map(f => (f._1.get, f._2)).map(g => { - // first region for the bound - val rr = g._1._1 - // second region for the bound - val secondrr = g._1._2 - // in the case where we span multiple referenceNames - if (rr.referenceName != g._1._2.referenceName) { - // create a ReferenceRegion that goes to the end of the chromosome - (ReferenceRegion(rr.referenceName, rr.start, rr.end), g._2) - } else { - // otherwise we just have the ReferenceRegion span from partition - // start to end - (ReferenceRegion(rr.referenceName, rr.start, secondrr.end), g._2) - } - }) - - // we use an interval array to quickly look up the destination partitions - IntervalArray( - adjustedPartitionMapWithIndex, - adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, - sorted = true) - } -} - /** * A trait for genomic data that is not aligned to a reference (e.g., raw reads). */ diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/PartitionMap.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/PartitionMap.scala new file mode 100644 index 0000000000..003bcffb34 --- /dev/null +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/PartitionMap.scala @@ -0,0 +1,123 @@ +package org.bdgenomics.adam.rdd + +import org.apache.spark.rdd.RDD +import org.bdgenomics.adam.models.ReferenceRegion +import org.bdgenomics.utils.interval.array.IntervalArray + +private[rdd] object PartitionMap { + + def apply[T, U <: GenomicRDD[T, U]](genomicRdd: GenomicRDD[T, U]): PartitionMap = { + if (!genomicRdd.isSorted) { + PartitionMap(None) + } else { + PartitionMap(Some(genomicRdd.flattenRddByRegions().mapPartitions(iter => { + getRegionBoundsFromPartition(iter) + }).collect)) + } + } + + def apply[T](rdd: RDD[(ReferenceRegion, T)]): PartitionMap = { + PartitionMap(Some(rdd.mapPartitions(iter => { + getRegionBoundsFromPartition(iter) + }).collect)) + } + + /** + * Gets the partition bounds from a ReferenceRegion keyed Iterator. + * + * @param iter The data on a given partition. ReferenceRegion keyed. + * @return The bounds of the ReferenceRegions on that partition, in an Iterator. + */ + private def getRegionBoundsFromPartition[T]( + iter: Iterator[(ReferenceRegion, T)]): Iterator[Option[(ReferenceRegion, ReferenceRegion)]] = { + + if (iter.isEmpty) { + // This means that there is no data on the partition, so we have no bounds + Iterator(None) + } else { + val firstRegion = iter.next + val lastRegion = + if (iter.hasNext) { + // we have to make sure we get the full bounds of this partition, this + // includes any extremely long regions. we include the firstRegion for + // the case that the first region is extremely long + (iter ++ Iterator(firstRegion)).maxBy(f => (f._1.referenceName, f._1.end, f._1.start)) + } else { + // only one record on this partition, so this is the extent of the bounds + firstRegion + } + Iterator(Some((firstRegion._1, lastRegion._1))) + } + } +} + +/** + * The partition map is structured as follows: + * The outer option is for whether or not there is a partition map. + * - This is None in the case that we don't know the bounds on each + * partition. + * The Array is the length of the number of partitions. + * The inner option is in case there is no data on a partition. + * The (ReferenceRegion, ReferenceRegion) tuple contains the bounds of the + * partition, such that the lowest start is first and the highest end is + * second. + * + * @param optPartitionMap An optional PartitionMap, format described above. + */ +private[rdd] case class PartitionMap( + private val optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) { + + lazy val get: Array[Option[(ReferenceRegion, ReferenceRegion)]] = { + optPartitionMap.get + } + + def isEmpty: Boolean = { + optPartitionMap.isEmpty + } + + def isDefined: Boolean = { + optPartitionMap.isDefined + } + + def exists(f: (Array[Option[(ReferenceRegion, ReferenceRegion)]]) => Boolean): Boolean = { + optPartitionMap.exists(f) + } + + def length: Int = { + get.length + } + + def indices: Range = { + get.indices + } + + def toIntervalArray(): IntervalArray[ReferenceRegion, Int] = { + + val adjustedPartitionMapWithIndex = + // the zipWithIndex gives us the destination partition ID + get.zipWithIndex + .filter(_._1.nonEmpty) + .map(f => (f._1.get, f._2)).map(g => { + // first region for the bound + val rr = g._1._1 + // second region for the bound + val secondrr = g._1._2 + // in the case where we span multiple referenceNames + if (rr.referenceName != g._1._2.referenceName) { + // create a ReferenceRegion that goes to the end of the chromosome + (ReferenceRegion(rr.referenceName, rr.start, rr.end), g._2) + } else { + // otherwise we just have the ReferenceRegion span from partition + // start to end + (ReferenceRegion(rr.referenceName, rr.start, secondrr.end), g._2) + } + }) + + // we use an interval array to quickly look up the destination partitions + IntervalArray( + adjustedPartitionMapWithIndex, + adjustedPartitionMapWithIndex.maxBy(_._1.width)._1.width, + sorted = true) + } +} + diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/contig/NucleotideContigFragmentRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/contig/NucleotideContigFragmentRDD.scala index 59e1e380f9..34e625407e 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/contig/NucleotideContigFragmentRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/contig/NucleotideContigFragmentRDD.scala @@ -98,7 +98,7 @@ object NucleotideContigFragmentRDD extends Serializable { def apply(rdd: RDD[NucleotideContigFragment], sequences: SequenceDictionary): NucleotideContigFragmentRDD = { - NucleotideContigFragmentRDD(rdd, sequences, None) + NucleotideContigFragmentRDD(rdd, sequences, isSorted = false) } } @@ -114,7 +114,7 @@ object NucleotideContigFragmentRDD extends Serializable { case class NucleotideContigFragmentRDD( rdd: RDD[NucleotideContigFragment], sequences: SequenceDictionary, - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends AvroGenomicRDD[NucleotideContigFragment, NucleotideContigFragmentRDD] with ReferenceFile { + isSorted: Boolean) extends AvroGenomicRDD[NucleotideContigFragment, NucleotideContigFragmentRDD] with ReferenceFile { protected def buildTree(rdd: RDD[(ReferenceRegion, NucleotideContigFragment)])( implicit tTag: ClassTag[NucleotideContigFragment]): IntervalArray[ReferenceRegion, NucleotideContigFragment] = { @@ -145,8 +145,8 @@ case class NucleotideContigFragmentRDD( * has been replaced. */ protected def replaceRdd(newRdd: RDD[NucleotideContigFragment], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): NucleotideContigFragmentRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): NucleotideContigFragmentRDD = { + copy(rdd = newRdd, isSorted = isSorted) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/CoverageRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/CoverageRDD.scala index 86f96e5374..51d2fdbfa8 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/CoverageRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/CoverageRDD.scala @@ -69,7 +69,7 @@ object CoverageRDD { * @return A new Coverage RDD. */ def apply(rdd: RDD[Coverage], sd: SequenceDictionary): CoverageRDD = { - CoverageRDD(rdd, sd, None) + CoverageRDD(rdd, sd, isSorted = false) } } @@ -82,7 +82,7 @@ object CoverageRDD { */ case class CoverageRDD(rdd: RDD[Coverage], sequences: SequenceDictionary, - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends GenomicRDD[Coverage, CoverageRDD] { + isSorted: Boolean) extends GenomicRDD[Coverage, CoverageRDD] { protected def buildTree(rdd: RDD[(ReferenceRegion, Coverage)])( implicit tTag: ClassTag[Coverage]): IntervalArray[ReferenceRegion, Coverage] = { @@ -186,7 +186,7 @@ case class CoverageRDD(rdd: RDD[Coverage], */ def toFeatureRDD: FeatureRDD = { val featureRdd = rdd.map(_.toFeature) - FeatureRDD(featureRdd, sequences, optPartitionMap) + FeatureRDD(featureRdd, sequences, isSorted) } /** @@ -262,8 +262,8 @@ case class CoverageRDD(rdd: RDD[Coverage], * @return Returns a new CoverageRDD with the underlying RDD replaced. */ protected def replaceRdd(newRdd: RDD[Coverage], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): CoverageRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): CoverageRDD = { + copy(rdd = newRdd, isSorted = isSorted) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/FeatureRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/FeatureRDD.scala index 374f28ad40..0f16cb6066 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/FeatureRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/feature/FeatureRDD.scala @@ -144,7 +144,7 @@ object FeatureRDD { * @return Returns a new FeatureRDD. */ def apply(rdd: RDD[Feature], sd: SequenceDictionary): FeatureRDD = { - FeatureRDD(rdd, sd, None) + FeatureRDD(rdd, sd, isSorted = false) } /** * @param feature Feature to convert to GTF format. @@ -255,7 +255,7 @@ object FeatureRDD { */ case class FeatureRDD(rdd: RDD[Feature], sequences: SequenceDictionary, - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends AvroGenomicRDD[Feature, FeatureRDD] with Logging { + isSorted: Boolean) extends AvroGenomicRDD[Feature, FeatureRDD] with Logging { protected def buildTree(rdd: RDD[(ReferenceRegion, Feature)])( implicit tTag: ClassTag[Feature]): IntervalArray[ReferenceRegion, Feature] = { @@ -331,8 +331,8 @@ case class FeatureRDD(rdd: RDD[Feature], * @return Returns a new FeatureRDD with the underlying RDD replaced. */ protected def replaceRdd(newRdd: RDD[Feature], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): FeatureRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): FeatureRDD = { + copy(rdd = newRdd, isSorted = isSorted) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/fragment/FragmentRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/fragment/FragmentRDD.scala index a50e3103f3..8a1d988b0e 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/fragment/FragmentRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/fragment/FragmentRDD.scala @@ -109,7 +109,7 @@ object FragmentRDD { sequences: SequenceDictionary, recordGroupDictionary: RecordGroupDictionary): FragmentRDD = { - FragmentRDD(rdd, sequences, recordGroupDictionary, None) + FragmentRDD(rdd, sequences, recordGroupDictionary, false) } } @@ -123,7 +123,7 @@ object FragmentRDD { case class FragmentRDD(rdd: RDD[Fragment], sequences: SequenceDictionary, recordGroups: RecordGroupDictionary, - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends AvroReadGroupGenomicRDD[Fragment, FragmentRDD] { + isSorted: Boolean) extends AvroReadGroupGenomicRDD[Fragment, FragmentRDD] { protected def buildTree(rdd: RDD[(ReferenceRegion, Fragment)])( implicit tTag: ClassTag[Fragment]): IntervalArray[ReferenceRegion, Fragment] = { @@ -138,8 +138,8 @@ case class FragmentRDD(rdd: RDD[Fragment], * swapped out. */ protected def replaceRdd(newRdd: RDD[Fragment], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): FragmentRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): FragmentRDD = { + copy(rdd = newRdd, isSorted = isSorted) } def union(rdds: FragmentRDD*): FragmentRDD = { @@ -161,7 +161,7 @@ case class FragmentRDD(rdd: RDD[Fragment], val newRdd = rdd.flatMap(converter.convertFragment) // are we aligned? - AlignmentRecordRDD(newRdd, sequences, recordGroups) + AlignmentRecordRDD(newRdd, sequences, recordGroups, isSorted) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/read/AlignmentRecordRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/read/AlignmentRecordRDD.scala index 601e3b780f..57fa095e01 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/read/AlignmentRecordRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/read/AlignmentRecordRDD.scala @@ -98,7 +98,7 @@ object AlignmentRecordRDD extends Serializable { AlignmentRecordRDD(rdd, SequenceDictionary.empty, RecordGroupDictionary.empty, - None) + isSorted = false) } /** @@ -140,7 +140,7 @@ object AlignmentRecordRDD extends Serializable { def apply(rdd: RDD[AlignmentRecord], sequences: SequenceDictionary, recordGroupDictionary: RecordGroupDictionary): AlignmentRecordRDD = { - AlignmentRecordRDD(rdd, sequences, recordGroupDictionary, None) + AlignmentRecordRDD(rdd, sequences, recordGroupDictionary, isSorted = false) } } @@ -148,7 +148,7 @@ case class AlignmentRecordRDD( rdd: RDD[AlignmentRecord], sequences: SequenceDictionary, recordGroups: RecordGroupDictionary, - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends AvroReadGroupGenomicRDD[AlignmentRecord, AlignmentRecordRDD] { + isSorted: Boolean) extends AvroReadGroupGenomicRDD[AlignmentRecord, AlignmentRecordRDD] { /** * Replaces the underlying RDD and SequenceDictionary and emits a new object. @@ -159,16 +159,16 @@ case class AlignmentRecordRDD( */ protected def replaceRddAndSequences(newRdd: RDD[AlignmentRecord], newSequences: SequenceDictionary, - partitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): AlignmentRecordRDD = { + isSorted: Boolean = false): AlignmentRecordRDD = { AlignmentRecordRDD(newRdd, newSequences, recordGroups, - partitionMap) + isSorted) } protected def replaceRdd(newRdd: RDD[AlignmentRecord], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): AlignmentRecordRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): AlignmentRecordRDD = { + copy(rdd = newRdd, isSorted = isSorted) } protected def buildTree(rdd: RDD[(ReferenceRegion, AlignmentRecord)])( @@ -192,7 +192,8 @@ case class AlignmentRecordRDD( def toFragments: FragmentRDD = { FragmentRDD(groupReadsByFragment().map(_.toFragment), sequences, - recordGroups) + recordGroups, + isSorted) } /** @@ -208,7 +209,7 @@ case class AlignmentRecordRDD( * Convert this set of reads into fragments. * * Assumes that reads are sorted by readname. - * * + * * * @return Returns a FragmentRDD where all reads have been grouped together by * the original sequence fragment they come from. @@ -216,7 +217,8 @@ case class AlignmentRecordRDD( private[rdd] def querynameSortedToFragments: FragmentRDD = { FragmentRDD(locallyGroupReadsByFragment().map(_.toFragment), sequences, - recordGroups) + recordGroups, + isSorted) } /** @@ -243,7 +245,7 @@ case class AlignmentRecordRDD( }).reduceByKey(_ + _) .map(r => Coverage(r._1, r._2.toDouble)) - CoverageRDD(covCounts, sequences) + CoverageRDD(covCounts, sequences, isSorted) } /** @@ -633,7 +635,7 @@ case class AlignmentRecordRDD( } else { ReferencePosition(s"~~~${r.getReadName}", 0) } - }), sequences.stripIndices.sorted) + }), sequences.stripIndices.sorted, isSorted = true) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/Closest.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/Closest.scala index e4032df0a9..5903fc8c9b 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/Closest.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/Closest.scala @@ -32,7 +32,10 @@ import scala.reflect.ClassTag * @tparam RX The resulting type of the right after the operation. */ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { + extends SetOperationBetweenCollections[T, X, RT, RX] { + + protected val leftRdd: GenomicRDD[T, U] + protected val rightRdd: GenomicRDD[X, Y] override protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, @@ -100,17 +103,21 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] override protected def prepare()( implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { - val (preparedLeftRdd, partitionMap) = { - if (leftRdd.partitionMap.isDefined) { + val numPartitions = optPartitions.getOrElse(leftRdd.rdd.partitions.length) + + val (preparedLeftRdd, destinationPartitionMap) = { + if (leftRdd.partitionMap.isDefined && + numPartitions != leftRdd.rdd.partitions.length) { + (leftRdd.flattenRddByRegions(), leftRdd.partitionMap) } else { - val sortedLeft = leftRdd.sortLexicographically(storePartitionMap = true) + val sortedLeft = leftRdd.sortLexicographically(numPartitions) (sortedLeft.flattenRddByRegions(), sortedLeft.partitionMap) } } // we use an interval array to quickly look up the destination partitions - val partitionMapIntervals = partitionMap.toIntervalArray() + val partitionMapIntervals = destinationPartitionMap.toIntervalArray() val assignedRightRdd: RDD[((ReferenceRegion, Int), X)] = { // copartitioning for the closest is tricky, and requires that we handle @@ -127,7 +134,7 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] // we have to find the partitions that don't have right data going there // so we can send the flanking partitions' data there val partitionsWithoutData = - partitionMap.get.indices.filterNot(firstPass.map(_._1._2).distinct().collect.contains) + destinationPartitionMap.indices.filterNot(firstPass.map(_._1._2).distinct().collect.contains) // this gives us a list of partitions that are sending copies of their // data and the number of nodes to send to. a negative number of nodes @@ -178,7 +185,7 @@ sealed trait Closest[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] val preparedRightRdd = assignedRightRdd .repartitionAndSortWithinPartitions( - ManualRegionPartitioner(partitionMap.get.length)) + ManualRegionPartitioner(destinationPartitionMap.length)) // return to an RDD[(ReferenceRegion, T)], removing the partition ID .map(f => (f._1._1, f._2)) @@ -202,7 +209,7 @@ case class ShuffleClosestRegion[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, protected val threshold: Long = Long.MaxValue, protected val optPartitions: Option[Int] = None) extends Closest[T, U, X, Y, T, Iterable[X]] - with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, Iterable[X]] { + with VictimlessSetOperationBetweenCollections[T, X, T, Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Iterable[X])] = { diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/SetOperation.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/SetOperation.scala index f626306739..53bcf35464 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/SetOperation.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/SetOperation.scala @@ -23,8 +23,7 @@ import org.bdgenomics.adam.rdd.GenomicRDD import scala.collection.mutable.ListBuffer import scala.reflect.ClassTag -private[sets] trait SetOperation[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends Serializable { +private[sets] trait SetOperation[X, RT, RX] extends Serializable { protected val threshold: Long @@ -44,6 +43,21 @@ private[sets] trait SetOperation[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, distanceThreshold: Long = 0L): Boolean } +private[sets] trait SetOperationBetweenCollectionsOnGenomicRDD[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends SetOperationBetweenCollections[T, X, RT, RX] { + + protected val leftRdd: GenomicRDD[T, U] + protected val rightRdd: GenomicRDD[X, Y] + +} + +private[sets] trait SetOperationBetweenCollectionsOnReferenceRegionKeyedRDD[T, X, RT, RX] + extends SetOperationBetweenCollections[T, X, RT, RX] { + + protected val leftRdd: RDD[(ReferenceRegion, T)] + protected val rightRdd: RDD[(ReferenceRegion, X)] +} + /** * The parent class for all inter-collection set theory operations. * @@ -52,12 +66,11 @@ private[sets] trait SetOperation[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, * @tparam RT The return type for the left side row data. * @tparam RX The return type for the right side row data. */ -private[rdd] abstract class SetOperationBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetOperation[T, U, X, Y, RT, RX] { +private[rdd] abstract class SetOperationBetweenCollections[T, X, RT, RX] + extends SetOperation[X, RT, RX] { - protected val leftRdd: GenomicRDD[T, U] - protected val rightRdd: GenomicRDD[X, Y] protected val optPartitions: Option[Int] + /** * Post process and format the hits for a given left record. * @@ -212,8 +225,8 @@ private[rdd] abstract class SetOperationBetweenCollections[T, U <: GenomicRDD[T, * @tparam RT The return type for the left side row data. * @tparam RX The return type for the right side row data. */ -private[sets] trait SetOperationBetweenCollectionsWithVictims[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { +private[sets] trait SetOperationBetweenCollectionsWithVictims[T, X, RT, RX] + extends SetOperationBetweenCollections[T, X, RT, RX] { /** * Post processes the pruned records to format them appropriately. @@ -289,8 +302,8 @@ private[sets] trait SetOperationBetweenCollectionsWithVictims[T, U <: GenomicRDD * @tparam RT The return type for the left side row data. * @tparam RX The return type for the right side row data. */ -private[sets] trait VictimlessSetOperationBetweenCollections[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { +private[sets] trait VictimlessSetOperationBetweenCollections[T, X, RT, RX] + extends SetOperationBetweenCollections[T, X, RT, RX] { override protected def pruneCache(to: ReferenceRegion, cache: SetTheoryCache[X, RT, RX]) = { diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala index 0cadd8392d..be0a4ba5de 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala @@ -19,20 +19,19 @@ package org.bdgenomics.adam.rdd.sets import org.apache.spark.rdd.RDD import org.bdgenomics.adam.models.ReferenceRegion -import org.bdgenomics.adam.rdd.{ GenomicRDD, ManualRegionPartitioner } -import org.bdgenomics.utils.interval.array.IntervalArray +import org.bdgenomics.adam.rdd.{ PartitionMap, GenomicRDD, ManualRegionPartitioner } import scala.reflect.ClassTag /** * A trait describing join implementations that are based on a sort-merge join. * * @tparam T The type of the left records. - * @tparam U The type of the right records. + * @tparam X The type of the right records. * @tparam RT The resulting type of the left after the join. * @tparam RX The resulting type of the right after the join. */ -sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] - extends SetOperationBetweenCollections[T, U, X, Y, RT, RX] { +sealed trait ShuffleRegionJoin[T, X, RT, RX] + extends SetOperationBetweenCollections[T, X, RT, RX] { override protected def condition(firstRegion: ReferenceRegion, secondRegion: ReferenceRegion, @@ -58,22 +57,21 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y candidateRegion.compareTo(until) < 0 || candidateRegion.covers(until) } - override protected def prepare()(implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { + protected def prepareRdds( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, X)], + partitionMap: PartitionMap)(implicit tTag: ClassTag[T], xtag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { - // default to current partition number if user did not specify - val numPartitions = optPartitions.getOrElse(leftRdd.rdd.partitions.length) + val numPartitions = optPartitions.getOrElse(leftRdd.partitions.length) - // we don't know if the left is sorted unless it has a partition map val (preparedLeft, destinationPartitionMap) = { - if (leftRdd.partitionMap.isDefined && - leftRdd.rdd.partitions.length == numPartitions) { - (leftRdd.flattenRddByRegions(), leftRdd.partitionMap) - } else { - val sortedLeft = - leftRdd.sortLexicographically(numPartitions, storePartitionMap = true) + if (!partitionMap.isEmpty && + numPartitions != leftRdd.partitions.length) { - val partitionMap = sortedLeft.partitionMap - (sortedLeft.flattenRddByRegions(), partitionMap) + (leftRdd, partitionMap) + } else { + val sortedLeft = leftRdd.sortByKey(ascending = true, numPartitions = numPartitions) + (sortedLeft, PartitionMap(sortedLeft)) } } @@ -81,7 +79,7 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y val partitionMapIntervals = destinationPartitionMap.toIntervalArray() val preparedRight = { - rightRdd.flattenRddByRegions() + rightRdd .mapPartitions(iter => { iter.flatMap(f => { // we pad by the threshold here to ensure that our invariant is met @@ -91,7 +89,7 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y }) }, preservesPartitioning = true) .repartitionAndSortWithinPartitions( - ManualRegionPartitioner(destinationPartitionMap.get.length)) + ManualRegionPartitioner(destinationPartitionMap.length)) .map(f => (f._1._1, f._2)) } @@ -99,6 +97,61 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y } } +private[rdd] sealed trait ShuffleRegionJoinOnGenomicRDD[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] + extends ShuffleRegionJoin[T, X, RT, RX] { + + protected val leftRdd: GenomicRDD[T, U] + protected val rightRdd: GenomicRDD[X, Y] + + override protected def prepare()(implicit tTag: ClassTag[T], xTag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { + prepareRdds( + leftRdd.flattenRddByRegions(), + rightRdd.flattenRddByRegions(), + leftRdd.partitionMap) + } +} + +private[rdd] sealed trait ShuffleRegionJoinOnReferenceRegionKeyedRDD[T, X, RT, RX] + extends ShuffleRegionJoin[T, X, RT, RX] { + + protected val leftRdd: RDD[(ReferenceRegion, T)] + protected val rightRdd: RDD[(ReferenceRegion, X)] + + override protected def prepare()(implicit tTag: ClassTag[T], xTag: ClassTag[X]): (RDD[(ReferenceRegion, T)], RDD[(ReferenceRegion, X)]) = { + prepareRdds(leftRdd, rightRdd, PartitionMap(None)) + } +} + +sealed trait InnerShuffleRegionJoin[T, X] extends VictimlessSetOperationBetweenCollections[T, X, T, X] { + + override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], + right: Iterator[(ReferenceRegion, X)]): Iterator[(T, X)] = { + Iterator.empty + } + + override protected def postProcessHits(currentLeft: (ReferenceRegion, T), + iter: Iterable[(ReferenceRegion, X)]): Iterable[(T, X)] = { + iter.map(f => (currentLeft._2, f._2)) + } +} + +object InnerShuffleRegionJoin { + def apply[T, X]( + leftRdd: RDD[(ReferenceRegion, T)], + rightRdd: RDD[(ReferenceRegion, X)]): InnerShuffleRegionJoin[T, X] = { + + InnerShuffleRegionJoinOnReferenceRegionKeyedRDD(leftRdd, rightRdd) + } +} + +private[rdd] case class InnerShuffleRegionJoinOnReferenceRegionKeyedRDD[T, X]( + protected val leftRdd: RDD[(ReferenceRegion, T)], + protected val rightRdd: RDD[(ReferenceRegion, X)], + protected val threshold: Long = 0L, + protected val optPartitions: Option[Int] = None) + extends InnerShuffleRegionJoin[T, X] + with ShuffleRegionJoinOnReferenceRegionKeyedRDD[T, X, T, X] + /** * Perform an Inner Shuffle Region Join. * @@ -109,24 +162,13 @@ sealed trait ShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y * @tparam T The type of the left records. * @tparam U The type of the right records. */ -case class InnerShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( +private[rdd] case class InnerShuffleRegionJoinOnGenomicRDD[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( protected val leftRdd: GenomicRDD[T, U], protected val rightRdd: GenomicRDD[X, Y], protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) - extends ShuffleRegionJoin[T, U, X, Y, T, X] - with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, X] { - - override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], - right: Iterator[(ReferenceRegion, X)]): Iterator[(T, X)] = { - Iterator.empty - } - - override protected def postProcessHits(currentLeft: (ReferenceRegion, T), - iter: Iterable[(ReferenceRegion, X)]): Iterable[(T, X)] = { - iter.map(f => (currentLeft._2, f._2)) - } -} + extends ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, T, X] + with InnerShuffleRegionJoin[T, X] /** * Perform an Inner Shuffle Region Join and Group By left records. @@ -143,8 +185,8 @@ case class InnerShuffleRegionJoinAndGroupByLeft[T, U <: GenomicRDD[T, U], X, Y < protected val rightRdd: GenomicRDD[X, Y], protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) - extends ShuffleRegionJoin[T, U, X, Y, T, Iterable[X]] - with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, Iterable[X]] { + extends ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, T, Iterable[X]] + with VictimlessSetOperationBetweenCollections[T, X, T, Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Iterable[X])] = { @@ -178,8 +220,8 @@ case class LeftOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicR protected val rightRdd: GenomicRDD[X, Y], protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) - extends ShuffleRegionJoin[T, U, X, Y, T, Option[X]] - with VictimlessSetOperationBetweenCollections[T, U, X, Y, T, Option[X]] { + extends ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, T, Option[X]] + with VictimlessSetOperationBetweenCollections[T, X, T, Option[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(T, Option[X])] = { @@ -213,8 +255,8 @@ case class RightOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: Genomic protected val rightRdd: GenomicRDD[X, Y], protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) - extends ShuffleRegionJoin[T, U, X, Y, Option[T], X] - with SetOperationBetweenCollectionsWithVictims[T, U, X, Y, Option[T], X] { + extends ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, Option[T], X] + with SetOperationBetweenCollectionsWithVictims[T, X, Option[T], X] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], X)] = { @@ -251,8 +293,8 @@ case class FullOuterShuffleRegionJoin[T, U <: GenomicRDD[T, U], X, Y <: GenomicR protected val rightRdd: GenomicRDD[X, Y], protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) - extends ShuffleRegionJoin[T, U, X, Y, Option[T], Option[X]] - with SetOperationBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Option[X]] { + extends ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, Option[T], Option[X]] + with SetOperationBetweenCollectionsWithVictims[T, X, Option[T], Option[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], Option[X])] = { @@ -290,8 +332,8 @@ case class RightOuterShuffleRegionJoinAndGroupByLeft[T, U <: GenomicRDD[T, U], X protected val rightRdd: GenomicRDD[X, Y], protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) - extends ShuffleRegionJoin[T, U, X, Y, Option[T], Iterable[X]] - with SetOperationBetweenCollectionsWithVictims[T, U, X, Y, Option[T], Iterable[X]] { + extends ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, Option[T], Iterable[X]] + with SetOperationBetweenCollectionsWithVictims[T, X, Option[T], Iterable[X]] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], right: Iterator[(ReferenceRegion, X)]): Iterator[(Option[T], Iterable[X])] = { diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/GenotypeRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/GenotypeRDD.scala index b8587b1d18..6a7f296971 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/GenotypeRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/GenotypeRDD.scala @@ -77,7 +77,7 @@ object GenotypeRDD extends Serializable { sequences: SequenceDictionary, samples: Seq[Sample], headerLines: Seq[VCFHeaderLine]): GenotypeRDD = { - GenotypeRDD(rdd, sequences, samples, headerLines, None) + GenotypeRDD(rdd, sequences, samples, headerLines, isSorted = false) } } @@ -95,7 +95,7 @@ case class GenotypeRDD(rdd: RDD[Genotype], sequences: SequenceDictionary, @transient samples: Seq[Sample], @transient headerLines: Seq[VCFHeaderLine], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends MultisampleAvroGenomicRDD[Genotype, GenotypeRDD] { + isSorted: Boolean) extends MultisampleAvroGenomicRDD[Genotype, GenotypeRDD] { def union(rdds: GenotypeRDD*): GenotypeRDD = { val iterableRdds = rdds.toSeq @@ -124,7 +124,11 @@ case class GenotypeRDD(rdd: RDD[Genotype], } } - VariantContextRDD(vcRdd, sequences, samples, headerLines) + VariantContextRDD(vcRdd, + sequences, + samples, + headerLines, + isSorted = isSorted) } /** @@ -132,8 +136,8 @@ case class GenotypeRDD(rdd: RDD[Genotype], * @return Returns a new GenotypeRDD with the underlying RDD replaced. */ protected def replaceRdd(newRdd: RDD[Genotype], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): GenotypeRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): GenotypeRDD = { + copy(rdd = newRdd, isSorted = isSorted) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantContextRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantContextRDD.scala index bb9e558ab9..e4dd576344 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantContextRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantContextRDD.scala @@ -89,7 +89,7 @@ object VariantContextRDD extends Serializable { sequences: SequenceDictionary, samples: Seq[Sample], headerLines: Seq[VCFHeaderLine]): VariantContextRDD = { - VariantContextRDD(rdd, sequences, samples, headerLines, None) + VariantContextRDD(rdd, sequences, samples, headerLines, isSorted = false) } def apply(rdd: RDD[VariantContext], @@ -112,7 +112,7 @@ case class VariantContextRDD(rdd: RDD[VariantContext], sequences: SequenceDictionary, @transient samples: Seq[Sample], @transient headerLines: Seq[VCFHeaderLine], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends MultisampleGenomicRDD[VariantContext, VariantContextRDD] + isSorted: Boolean) extends MultisampleGenomicRDD[VariantContext, VariantContextRDD] with Logging { protected def buildTree(rdd: RDD[(ReferenceRegion, VariantContext)])( @@ -136,7 +136,8 @@ case class VariantContextRDD(rdd: RDD[VariantContext], GenotypeRDD(rdd.flatMap(_.genotypes), sequences, samples, - headerLines) + headerLines, + isSorted = isSorted) } /** @@ -145,7 +146,8 @@ case class VariantContextRDD(rdd: RDD[VariantContext], def toVariantRDD: VariantRDD = { VariantRDD(rdd.map(_.variant.variant), sequences, - headerLines) + headerLines, + isSorted = isSorted) } /** @@ -270,8 +272,8 @@ case class VariantContextRDD(rdd: RDD[VariantContext], * been replaced. */ protected def replaceRdd(newRdd: RDD[VariantContext], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): VariantContextRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): VariantContextRDD = { + copy(rdd = newRdd, isSorted = isSorted) } /** diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantRDD.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantRDD.scala index 81490d3c4b..b42dd80192 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantRDD.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/variant/VariantRDD.scala @@ -79,7 +79,7 @@ object VariantRDD extends Serializable { def apply(rdd: RDD[Variant], sequences: SequenceDictionary, headerLines: Seq[VCFHeaderLine]): VariantRDD = { - VariantRDD(rdd, sequences, headerLines, None) + VariantRDD(rdd, sequences, headerLines, isSorted = false) } } @@ -94,7 +94,7 @@ object VariantRDD extends Serializable { case class VariantRDD(rdd: RDD[Variant], sequences: SequenceDictionary, @transient headerLines: Seq[VCFHeaderLine], - optPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]]) extends AvroGenomicRDD[Variant, VariantRDD] { + isSorted: Boolean) extends AvroGenomicRDD[Variant, VariantRDD] { protected def buildTree(rdd: RDD[(ReferenceRegion, Variant)])( implicit tTag: ClassTag[Variant]): IntervalArray[ReferenceRegion, Variant] = { @@ -127,7 +127,11 @@ case class VariantRDD(rdd: RDD[Variant], * @return Returns this VariantRDD as a VariantContextRDD. */ def toVariantContextRDD: VariantContextRDD = { - VariantContextRDD(rdd.map(VariantContext(_)), sequences, Seq.empty[Sample], headerLines) + VariantContextRDD( + rdd.map(VariantContext(_)), + sequences, Seq.empty[Sample], + headerLines, + isSorted = isSorted) } /** @@ -135,8 +139,8 @@ case class VariantRDD(rdd: RDD[Variant], * @return Returns a new VariantRDD with the underlying RDD replaced. */ protected def replaceRdd(newRdd: RDD[Variant], - newPartitionMap: Option[Array[Option[(ReferenceRegion, ReferenceRegion)]]] = None): VariantRDD = { - copy(rdd = newRdd, optPartitionMap = newPartitionMap) + isSorted: Boolean = false): VariantRDD = { + copy(rdd = newRdd, isSorted = isSorted) } /** diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala index e4cdb99a1e..edbf1d5aac 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/OuterRegionJoinSuite.scala @@ -59,14 +59,12 @@ trait OuterRegionJoinSuite extends ADAMFunSuite { val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, - RecordGroupDictionary.empty, - None) + RecordGroupDictionary.empty) val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, - RecordGroupDictionary.empty, - None) + RecordGroupDictionary.empty) val jrdd = runJoin(baseGenomicRdd, recordsGenomicRdd).cache() @@ -104,14 +102,12 @@ trait OuterRegionJoinSuite extends ADAMFunSuite { val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, - RecordGroupDictionary.empty, - None) + RecordGroupDictionary.empty) val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, - RecordGroupDictionary.empty, - None) + RecordGroupDictionary.empty) val jrdd = runJoin(baseGenomicRdd, recordsGenomicRdd).cache() diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala index aec65789a8..1cdf91618c 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala @@ -70,12 +70,12 @@ class SortedGenomicRDDSuite extends SparkFunSuite { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) // sort and make into 16 partitions - val y = x.sortLexicographically(storePartitionMap = true, partitions = 16) - assert(isSorted(y.optPartitionMap.get)) + val y = x.sortLexicographically(partitions = 16) + assert(isSorted(y.partitionMap.get)) // sort and make into 32 partitions - val z = x.sortLexicographically(storePartitionMap = true, partitions = 32) - assert(isSorted(z.optPartitionMap.get)) + val z = x.sortLexicographically(partitions = 32) + assert(isSorted(z.partitionMap.get)) val arrayRepresentationOfZ = z.rdd.collect //verify sort worked on actual values @@ -101,7 +101,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) // sort and make into 16 partitions val z = - x.sortLexicographically(storePartitionMap = true, partitions = 1600) + x.sortLexicographically(partitions = 1600) // perform join using 1600 partitions // 1600 is much more than the amount of data in the GenomicRDD @@ -119,7 +119,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { sparkTest("testing that sorted fullOuterShuffleRegionJoin matches unsorted") { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) - val z = x.sortLexicographically(storePartitionMap = true, partitions = 16) + val z = x.sortLexicographically(partitions = 16) val d = x.fullOuterShuffleRegionJoin(z, Some(1)) val e = z.fullOuterShuffleRegionJoin(x, Some(1)) @@ -130,7 +130,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { sparkTest("testing that sorted rightOuterShuffleRegionJoin matches unsorted") { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) - val z = x.sortLexicographically(storePartitionMap = true, partitions = 1) + val z = x.sortLexicographically(partitions = 1) val f = z.rightOuterShuffleRegionJoin(x, Some(1)).rdd.collect val g = x.rightOuterShuffleRegionJoin(x).rdd.collect @@ -141,7 +141,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { sparkTest("testing that sorted leftOuterShuffleRegionJoin matches unsorted") { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) - val z = x.sortLexicographically(storePartitionMap = true, partitions = 1) + val z = x.sortLexicographically(partitions = 1) val h = z.leftOuterShuffleRegionJoin(x, Some(1)).rdd val i = z.leftOuterShuffleRegionJoin(x).rdd @@ -152,7 +152,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { sparkTest("testing that we can persist the sorted knowledge") { val x = sc.loadBam(resourceUrl("reads12.sam").getFile) - val z = x.sortLexicographically(storePartitionMap = true, partitions = 4) + val z = x.sortLexicographically(partitions = 4) val fileLocation = tmpLocation() val saveArgs = new JavaSaveArgs(fileLocation, asSingleFile = false) z.save(saveArgs, isSorted = true) @@ -168,7 +168,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { val fitsWithinPartitionMap = if (ReferenceRegion(next.getContigName, next.getStart, - next.getEnd).compareTo(t.optPartitionMap.get.apply(idx).get._1) >= 0) { + next.getEnd).compareTo(t.partitionMap.get.apply(idx).get._1) >= 0) { true } else { false @@ -176,7 +176,7 @@ class SortedGenomicRDDSuite extends SparkFunSuite { tempList += fitsWithinPartitionMap && ( if (ReferenceRegion(next.getContigName, next.getStart, - next.getEnd).compareTo(t.optPartitionMap.get.apply(idx).get._2) <= 0 && + next.getEnd).compareTo(t.partitionMap.get.apply(idx).get._2) <= 0 && fitsWithinPartitionMap) { true } else { diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/ClosestSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/ClosestSuite.scala index 2f83d03724..77ed6a03b8 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/ClosestSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/ClosestSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ListBuffer class ClosestSuite extends ADAMFunSuite { sparkTest("testing closest") { val leftFile = sc.loadBed(resourceUrl("intersect_with_overlap_00.bed").getFile) - .sortLexicographically(storePartitionMap = true) + .sortLexicographically() val rightFile = sc.loadBed(resourceUrl("intersect_with_overlap_01.bed").getFile) val x = ShuffleClosestRegion(leftFile, rightFile).compute() diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala index 1df308cebf..09501665d6 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/sets/InnerShuffleRegionJoinSuite.scala @@ -54,11 +54,29 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { val baseRdd = sc.parallelize(Seq(baseRecord), 1) val recordsRdd = sc.parallelize(Seq(record1, record2), 1) - val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) - val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) + assert( + InnerShuffleRegionJoin( + baseRdd.keyBy(ReferenceRegion.unstranded), + recordsRdd.keyBy(ReferenceRegion.unstranded) + ).compute() + .aggregate(true)( + InnerShuffleRegionJoinSuite.merge, + InnerShuffleRegionJoinSuite.and) + ) + + assert( + InnerShuffleRegionJoin( + baseRdd.keyBy(ReferenceRegion.unstranded), + recordsRdd.keyBy(ReferenceRegion.unstranded) + ).compute() + .count() === 2 + ) + + val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, RecordGroupDictionary.empty) + val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, RecordGroupDictionary.empty) assert( - InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) + InnerShuffleRegionJoinOnGenomicRDD(baseGenomicRdd, recordsGenomicRdd) .compute() .aggregate(true)( InnerShuffleRegionJoinSuite.merge, @@ -67,7 +85,7 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { ) assert( - InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) + InnerShuffleRegionJoinOnGenomicRDD(baseGenomicRdd, recordsGenomicRdd) .compute() .count() === 2 ) @@ -110,11 +128,11 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { val baseRdd = sc.parallelize(Seq(baseRecord1, baseRecord2), 1) val recordsRdd = sc.parallelize(Seq(record1, record2, record3), 1) - val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) - val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, RecordGroupDictionary.empty, None) + val baseGenomicRdd = AlignmentRecordRDD(baseRdd, SequenceDictionary.empty, RecordGroupDictionary.empty) + val recordsGenomicRdd = AlignmentRecordRDD(recordsRdd, SequenceDictionary.empty, RecordGroupDictionary.empty) assert( - InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) + InnerShuffleRegionJoinOnGenomicRDD(baseGenomicRdd, recordsGenomicRdd) .compute() .aggregate(true)( InnerShuffleRegionJoinSuite.merge, @@ -123,7 +141,7 @@ class InnerShuffleRegionJoinSuite extends ADAMFunSuite { ) assert({ - InnerShuffleRegionJoin(baseGenomicRdd, recordsGenomicRdd) + InnerShuffleRegionJoinOnGenomicRDD(baseGenomicRdd, recordsGenomicRdd) .compute() .count() === 3 } From 1f73378771ed8343b11201842c22f28ce5e7d9fc Mon Sep 17 00:00:00 2001 From: devin-petersohn Date: Wed, 21 Jun 2017 16:02:28 -0700 Subject: [PATCH 13/13] Adding documentation and formatting --- .../adam/rdd/sets/ShuffleRegionJoin.scala | 61 ++++++++++++++++++- .../adam/rdd/SortedGenomicRDDSuite.scala | 4 -- 2 files changed, 58 insertions(+), 7 deletions(-) diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala index be0a4ba5de..644f56141e 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/sets/ShuffleRegionJoin.scala @@ -57,6 +57,14 @@ sealed trait ShuffleRegionJoin[T, X, RT, RX] candidateRegion.compareTo(until) < 0 || candidateRegion.covers(until) } + /** + * Prepares the two RDDs for the join. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param partitionMap The partition map for the left side, if exists. + * @return A tuple with the prepared left and right RDDs. + */ protected def prepareRdds( leftRdd: RDD[(ReferenceRegion, T)], rightRdd: RDD[(ReferenceRegion, X)], @@ -97,6 +105,16 @@ sealed trait ShuffleRegionJoin[T, X, RT, RX] } } +/** + * Perform a ShuffleRegionJoin on a GenomicRDD. + * + * @tparam T The type of the left records. + * @tparam U The type of the left, a subclass of GenomicRDD. + * @tparam X The type of the right records. + * @tparam Y The type of the right, a subclass of GenomicRDD. + * @tparam RT The resulting type of the left after the join. + * @tparam RX The resulting type of the right after the join. + */ private[rdd] sealed trait ShuffleRegionJoinOnGenomicRDD[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y], RT, RX] extends ShuffleRegionJoin[T, X, RT, RX] { @@ -111,6 +129,14 @@ private[rdd] sealed trait ShuffleRegionJoinOnGenomicRDD[T, U <: GenomicRDD[T, U] } } +/** + * Perform a ShuffleRegionJoin on ReferenceRegion keyed RDDs. + * + * @tparam T The type of the left records. + * @tparam X The type of the right records. + * @tparam RT The resulting type of the left after the join. + * @tparam RX The resulting type of the right after the join. + */ private[rdd] sealed trait ShuffleRegionJoinOnReferenceRegionKeyedRDD[T, X, RT, RX] extends ShuffleRegionJoin[T, X, RT, RX] { @@ -122,6 +148,12 @@ private[rdd] sealed trait ShuffleRegionJoinOnReferenceRegionKeyedRDD[T, X, RT, R } } +/** + * Perform an Inner ShuffleRegionJoin. + * + * @tparam T The left side row data. + * @tparam X The right side row data. + */ sealed trait InnerShuffleRegionJoin[T, X] extends VictimlessSetOperationBetweenCollections[T, X, T, X] { override protected def emptyFn(left: Iterator[(ReferenceRegion, T)], @@ -135,7 +167,20 @@ sealed trait InnerShuffleRegionJoin[T, X] extends VictimlessSetOperationBetweenC } } +/** + * Perform an Inner ShuffleRegionJoin. This is publicly accessible to be + * compatible with legacy code. + */ object InnerShuffleRegionJoin { + + /** + * Create a new InnerShuffleRegionJoin object by passing in ReferenceRegion + * keyed RDDs. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @return An InnerShuffleRegionJoin object. + */ def apply[T, X]( leftRdd: RDD[(ReferenceRegion, T)], rightRdd: RDD[(ReferenceRegion, X)]): InnerShuffleRegionJoin[T, X] = { @@ -144,6 +189,16 @@ object InnerShuffleRegionJoin { } } +/** + * Perform an Inner ShuffleRegionJoin on ReferenceRegion keyed RDDs. + * + * @param leftRdd The left RDD. + * @param rightRdd The right RDD. + * @param threshold The threshold for the join. + * @param optPartitions Optionally sets the number of partitions for the join. + * @tparam T The left side row data. + * @tparam X The right side row data. + */ private[rdd] case class InnerShuffleRegionJoinOnReferenceRegionKeyedRDD[T, X]( protected val leftRdd: RDD[(ReferenceRegion, T)], protected val rightRdd: RDD[(ReferenceRegion, X)], @@ -160,15 +215,15 @@ private[rdd] case class InnerShuffleRegionJoinOnReferenceRegionKeyedRDD[T, X]( * @param threshold The threshold for the join. * @param optPartitions Optionally sets the number of partitions for the join. * @tparam T The type of the left records. - * @tparam U The type of the right records. + * @tparam X The type of the right records. */ private[rdd] case class InnerShuffleRegionJoinOnGenomicRDD[T, U <: GenomicRDD[T, U], X, Y <: GenomicRDD[X, Y]]( protected val leftRdd: GenomicRDD[T, U], protected val rightRdd: GenomicRDD[X, Y], protected val threshold: Long = 0L, protected val optPartitions: Option[Int] = None) - extends ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, T, X] - with InnerShuffleRegionJoin[T, X] + extends InnerShuffleRegionJoin[T, X] + with ShuffleRegionJoinOnGenomicRDD[T, U, X, Y, T, X] /** * Perform an Inner Shuffle Region Join and Group By left records. diff --git a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala index 1cdf91618c..f9e6cb3d07 100644 --- a/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala +++ b/adam-core/src/test/scala/org/bdgenomics/adam/rdd/SortedGenomicRDDSuite.scala @@ -17,12 +17,8 @@ */ package org.bdgenomics.adam.rdd -import org.bdgenomics.adam.converters.DefaultHeaderLines import org.bdgenomics.adam.models.{ SequenceRecord, SequenceDictionary, ReferenceRegion } import org.bdgenomics.adam.rdd.ADAMContext._ -import org.bdgenomics.adam.rdd.feature.FeatureRDD -import org.bdgenomics.adam.rdd.variant.GenotypeRDD -import org.bdgenomics.formats.avro._ import org.bdgenomics.utils.misc.SparkFunSuite import scala.collection.mutable.ListBuffer