From 09f01b32679f4054b12318a4f89c9e648a897fdc Mon Sep 17 00:00:00 2001 From: Uri Laserson Date: Tue, 6 Oct 2015 00:29:07 -0700 Subject: [PATCH] [ADAM-848] TwoBitFile now support nBlocks and maskBlocks Fixes #848. --- .../adam/models/NonoverlappingRegions.scala | 211 ++++++++++++++++++ .../adam/rdd/BroadcastRegionJoin.scala | 204 +---------------- .../org/bdgenomics/adam/util/TwoBitFile.scala | 88 +++++--- .../resources/human_g1k_v37_chr1_59kb.2bit | Bin 0 -> 15031 bytes .../adam/rdd/BroadcastRegionJoinSuite.scala | 2 +- .../bdgenomics/adam/util/TwoBitSuite.scala | 15 ++ 6 files changed, 292 insertions(+), 228 deletions(-) create mode 100644 adam-core/src/main/scala/org/bdgenomics/adam/models/NonoverlappingRegions.scala create mode 100644 adam-core/src/test/resources/human_g1k_v37_chr1_59kb.2bit diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/models/NonoverlappingRegions.scala b/adam-core/src/main/scala/org/bdgenomics/adam/models/NonoverlappingRegions.scala new file mode 100644 index 0000000000..ef0c06ad2e --- /dev/null +++ b/adam-core/src/main/scala/org/bdgenomics/adam/models/NonoverlappingRegions.scala @@ -0,0 +1,211 @@ +/** + * 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.models + +/** + * The evaluation of a regionJoin takes place with respect to a complete partition on the total space + * of the genome. NonoverlappingRegions is a class to compute the value of that partition, and to allow + * us to assign one or more elements of that partition to a new ReferenceRegion (see the 'regionsFor' method). + * + * NonoverlappingRegions takes, as input, and 'input-set' of regions. These are arbitrary ReferenceRegions, + * which may be overlapping, identical, disjoint, etc. The input-set of regions _must_ all be located on + * the same reference chromosome (i.e. must all have the same refName); the generalization to reference + * regions from multiple chromosomes is in MultiContigNonoverlappingRegions, below. + * + * NonoverlappingRegions produces, internally, a 'nonoverlapping-set' of regions. This is basically + * the set of _distinct unions_ of the input-set regions. + * + * @param regions The input-set of regions. + */ +class NonoverlappingRegions(regions: Iterable[ReferenceRegion]) extends Serializable { + + assert(regions != null, "regions parameter cannot be null") + + // The regions Seq needs to be of non-zero size, since otherwise we have to add special-case + // checks to all the methods below to make sure that 'endpoints' isn't empty. Also, it shouldn't + // make any sense to have a set of non-overlapping regions for ... no regions. + // Also, without this check, we can't tell which chromosome this NonoverlappingRegions object is for. + assert(regions.size > 0, "regions list must be non-empty") + assert(regions.head != null, "regions must have at least one non-null entry") + + val referenceName: String = regions.head.referenceName + + // invariant: all the values in the 'regions' list have the same referenceId + assert(regions.forall(_.referenceName == referenceName)) + + // We represent the distinct unions, the 'nonoverlapping-set' of regions, as a set of endpoints, + // so that we can do reasonably-fast binary searching on them to determine the slice of nonoverlapping-set + // regions that are overlapped by a new, query region (see findOverlappingRegions, below). + val endpoints: Array[Long] = + mergeRegions(regions.toSeq.sortBy(r => r.start)).flatMap(r => Seq(r.start, r.end)).distinct.sorted.toArray + + private def updateListWithRegion(list: List[ReferenceRegion], newRegion: ReferenceRegion): List[ReferenceRegion] = { + list match { + case head :: tail => + + // using overlaps || isAdjacent is an important feature! it means that + // we can use the "alternating" optimization, described below, which reduces + // the number of regions returned as keys (by a factor of 2) and therefore + // the number of partitions that need to be examined during a regionJoin. + if (head.overlaps(newRegion) || head.isAdjacent(newRegion)) { + head.hull(newRegion) :: tail + } else { + newRegion :: list + } + case _ => List(newRegion) + } + } + + def mergeRegions(regs: Seq[(ReferenceRegion)]): List[ReferenceRegion] = + regs.aggregate(List[ReferenceRegion]())( + (lst: List[ReferenceRegion], p: (ReferenceRegion)) => updateListWithRegion(lst, p), + (a, b) => a ++ b) + + def binaryPointSearch(pos: Long, lessThan: Boolean): Int = { + var i = 0 + var j = endpoints.size - 1 + + while (j - i > 1) { + val ij2 = (i + j) / 2 + val mid = endpoints(ij2) + if (mid < pos) { + i = ij2 + } else { + j = ij2 + } + } + + if (lessThan) i else j + } + + def findOverlappingRegions(query: ReferenceRegion): Seq[ReferenceRegion] = { + + assert(query != null, "query region was null") + assert(endpoints != null, "endpoints field was null") + + if (query.end <= endpoints.head || query.start >= endpoints.last) { + Seq() + } else { + val firsti = binaryPointSearch(query.start, lessThan = true) + val lasti = binaryPointSearch(query.end, lessThan = false) + + // Slice is an inclusive start, exclusive end operation + val firstRegionIsHit = firsti % 2 == 0 + + val startSlice = endpoints.slice(firsti, lasti) + val endSlice = endpoints.slice(firsti + 1, lasti + 1) + + /* + * The use of NonoverlappingRegions.alternating is an important optimization -- + * basically, because we used "overlaps || isAdjacent" as the predicate for + * when to join two regions in the input-set into a single nonoverlapping-region, above, + * then we know that the set of nonoverlapping-regions defined by the points in 'endpoints' + * are "alternating." In other words, we know that each nonoverlapping-region + * defined by an overlapping set of input-regions (in the constructor) is followed by + * a nonoverlapping-region which had _no_ input-set regions within it, and vice-versa. + * + * And _this_ is important because it means that, here, we only need to return + * _half_ the regions we otherwise would have -- because this is being used for a + * regionJoin, we don't need to return the 'empty' regions since we know a priori + * that these don't correspond to any region the input-set, and therefore + * will never result in any pairs in the ultimate join result. + */ + NonoverlappingRegions.alternating(startSlice.zip(endSlice).map { + case (start, end) => + ReferenceRegion(referenceName, start, end) + }.toSeq, firstRegionIsHit) + } + } + + /** + * Given a "regionable" value (corresponds to a ReferencRegion through an implicit ReferenceMapping), + * return the set of nonoverlapping-regions to be used as partitions for the input value in a + * region-join. Basically, return the set of any non-empty nonoverlapping-regions that overlap the + * region corresponding to this input. + * + * @param regionable The input, which corresponds to a region + * @tparam U The type of the input + * @return An Iterable[ReferenceRegion], where each element of the Iterable is a nonoverlapping-region + * defined by 1 or more input-set regions. + */ + def regionsFor[U](regionable: (ReferenceRegion, U)): Iterable[ReferenceRegion] = + findOverlappingRegions(regionable._1) + + /** + * A quick filter, to find out if we even need to examine a particular input value for keying by + * nonoverlapping-regions. Basically, reject the input value if its corresponding region is + * completely outside the hull of all the input-set regions. + * + * @param regionable The input value + * @tparam U + * @return a boolean -- the input value should only participate in the regionJoin if the return value + * here is 'true'. + */ + def hasRegionsFor[U](regionable: (ReferenceRegion, U)): Boolean = { + !(regionable._1.end <= endpoints.head || regionable._1.start >= endpoints.last) + } + + override def toString: String = + "%s:%d-%d (%s)".format(referenceName, endpoints.head, endpoints.last, endpoints.mkString(",")) +} + +object NonoverlappingRegions { + def apply[T](values: Seq[(ReferenceRegion, T)]) = + new NonoverlappingRegions(values.map(_._1)) + + def alternating[T](seq: Seq[T], includeFirst: Boolean): Seq[T] = { + val inds = if (includeFirst) { 0 until seq.size } else { 1 until seq.size + 1 } + seq.zip(inds).filter(p => p._2 % 2 == 0).map(_._1) + } +} + +/** + * Creates a multi-reference-region collection of NonoverlappingRegions -- see + * the scaladocs to NonoverlappingRegions. + * + * @param regions A Seq of ReferencRegions, pre-partitioned by their + * referenceNames. So, for a given pair (x, regs) in + * this Seq, all regions R in regs must satisfy + * R.referenceName == x. Furthermore, all the x's must + * be valid reference names with respect to the sequence + * dictionary. + */ +class MultiContigNonoverlappingRegions(regions: Seq[(String, Iterable[ReferenceRegion])]) extends Serializable { + assert(regions != null, + "Regions was set to null") + + val regionMap: Map[String, NonoverlappingRegions] = + Map(regions.map(r => (r._1, new NonoverlappingRegions(r._2))): _*) + + def regionsFor[U](regionable: (ReferenceRegion, U)): Iterable[ReferenceRegion] = + regionMap.get(regionable._1.referenceName).fold(Iterable[ReferenceRegion]())(_.regionsFor(regionable)) + + def filter[U](value: (ReferenceRegion, U)): Boolean = + regionMap.get(value._1.referenceName).fold(false)(_.hasRegionsFor(value)) +} + +object MultiContigNonoverlappingRegions { + def apply[T](values: Seq[(ReferenceRegion, T)]): MultiContigNonoverlappingRegions = { + new MultiContigNonoverlappingRegions( + values.map(kv => (kv._1.referenceName, kv._1)) + .groupBy(t => t._1) + .map(t => (t._1, t._2.map(k => k._2))) + .toSeq) + } +} + diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/BroadcastRegionJoin.scala b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/BroadcastRegionJoin.scala index 0929b741e5..5daa9fe487 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/rdd/BroadcastRegionJoin.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/rdd/BroadcastRegionJoin.scala @@ -17,11 +17,10 @@ */ package org.bdgenomics.adam.rdd -import org.bdgenomics.adam.models.{ SequenceDictionary, ReferenceRegion } +import org.bdgenomics.adam.models.{ MultiContigNonoverlappingRegions, ReferenceRegion } import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ import scala.Predef._ -import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import scala.reflect.ClassTag /** @@ -149,202 +148,3 @@ object BroadcastRegionJoin extends RegionJoin { }).map(p => (p._1._2, p._2._2)) } } - -/** - * The evaluation of a regionJoin takes place with respect to a complete partition on the total space - * of the genome. NonoverlappingRegions is a class to compute the value of that partition, and to allow - * us to assign one or more elements of that partition to a new ReferenceRegion (see the 'regionsFor' method). - * - * NonoverlappingRegions takes, as input, and 'input-set' of regions. These are arbitrary ReferenceRegions, - * which may be overlapping, identical, disjoint, etc. The input-set of regions _must_ all be located on - * the same reference chromosome (i.e. must all have the same refName); the generalization to reference - * regions from multiple chromosomes is in MultiContigNonoverlappingRegions, below. - * - * NonoverlappingRegions produces, internally, a 'nonoverlapping-set' of regions. This is basically - * the set of _distinct unions_ of the input-set regions. - * - * @param regions The input-set of regions. - */ -private[rdd] class NonoverlappingRegions(regions: Iterable[ReferenceRegion]) extends Serializable { - - assert(regions != null, "regions parameter cannot be null") - - // The regions Seq needs to be of non-zero size, since otherwise we have to add special-case - // checks to all the methods below to make sure that 'endpoints' isn't empty. Also, it shouldn't - // make any sense to have a set of non-overlapping regions for ... no regions. - // Also, without this check, we can't tell which chromosome this NonoverlappingRegions object is for. - assert(regions.size > 0, "regions list must be non-empty") - assert(regions.head != null, "regions must have at least one non-null entry") - - val referenceName: String = regions.head.referenceName - - // invariant: all the values in the 'regions' list have the same referenceId - assert(regions.forall(_.referenceName == referenceName)) - - // We represent the distinct unions, the 'nonoverlapping-set' of regions, as a set of endpoints, - // so that we can do reasonably-fast binary searching on them to determine the slice of nonoverlapping-set - // regions that are overlapped by a new, query region (see findOverlappingRegions, below). - val endpoints: Array[Long] = - mergeRegions(regions.toSeq.sortBy(r => r.start)).flatMap(r => Seq(r.start, r.end)).distinct.sorted.toArray - - private def updateListWithRegion(list: List[ReferenceRegion], newRegion: ReferenceRegion): List[ReferenceRegion] = { - list match { - case head :: tail => - - // using overlaps || isAdjacent is an important feature! it means that - // we can use the "alternating" optimization, described below, which reduces - // the number of regions returned as keys (by a factor of 2) and therefore - // the number of partitions that need to be examined during a regionJoin. - if (head.overlaps(newRegion) || head.isAdjacent(newRegion)) { - head.hull(newRegion) :: tail - } else { - newRegion :: list - } - case _ => List(newRegion) - } - } - - def mergeRegions(regs: Seq[(ReferenceRegion)]): List[ReferenceRegion] = - regs.aggregate(List[ReferenceRegion]())( - (lst: List[ReferenceRegion], p: (ReferenceRegion)) => updateListWithRegion(lst, p), - (a, b) => a ++ b) - - def binaryPointSearch(pos: Long, lessThan: Boolean): Int = { - var i = 0 - var j = endpoints.size - 1 - - while (j - i > 1) { - val ij2 = (i + j) / 2 - val mid = endpoints(ij2) - if (mid < pos) { - i = ij2 - } else { - j = ij2 - } - } - - if (lessThan) i else j - } - - def findOverlappingRegions(query: ReferenceRegion): Seq[ReferenceRegion] = { - - assert(query != null, "query region was null") - assert(endpoints != null, "endpoints field was null") - - if (query.end <= endpoints.head || query.start >= endpoints.last) { - Seq() - } else { - val firsti = binaryPointSearch(query.start, lessThan = true) - val lasti = binaryPointSearch(query.end, lessThan = false) - - // Slice is an inclusive start, exclusive end operation - val firstRegionIsHit = firsti % 2 == 0 - - val startSlice = endpoints.slice(firsti, lasti) - val endSlice = endpoints.slice(firsti + 1, lasti + 1) - - /* - * The use of NonoverlappingRegions.alternating is an important optimization -- - * basically, because we used "overlaps || isAdjacent" as the predicate for - * when to join two regions in the input-set into a single nonoverlapping-region, above, - * then we know that the set of nonoverlapping-regions defined by the points in 'endpoints' - * are "alternating." In other words, we know that each nonoverlapping-region - * defined by an overlapping set of input-regions (in the constructor) is followed by - * a nonoverlapping-region which had _no_ input-set regions within it, and vice-versa. - * - * And _this_ is important because it means that, here, we only need to return - * _half_ the regions we otherwise would have -- because this is being used for a - * regionJoin, we don't need to return the 'empty' regions since we know a priori - * that these don't correspond to any region the input-set, and therefore - * will never result in any pairs in the ultimate join result. - */ - NonoverlappingRegions.alternating(startSlice.zip(endSlice).map { - case (start, end) => - ReferenceRegion(referenceName, start, end) - }.toSeq, firstRegionIsHit) - } - } - - /** - * Given a "regionable" value (corresponds to a ReferencRegion through an implicit ReferenceMapping), - * return the set of nonoverlapping-regions to be used as partitions for the input value in a - * region-join. Basically, return the set of any non-empty nonoverlapping-regions that overlap the - * region corresponding to this input. - * - * @param regionable The input, which corresponds to a region - * @param mapping The implicit mapping to turn the input into a region - * @tparam U The type of the input - * @return An Iterable[ReferenceRegion], where each element of the Iterable is a nonoverlapping-region - * defined by 1 or more input-set regions. - */ - def regionsFor[U](regionable: (ReferenceRegion, U)): Iterable[ReferenceRegion] = - findOverlappingRegions(regionable._1) - - /** - * A quick filter, to find out if we even need to examine a particular input value for keying by - * nonoverlapping-regions. Basically, reject the input value if its corresponding region is - * completely outside the hull of all the input-set regions. - * - * @param regionable The input value - * @param mapping an implicity mapping of the input value to a ReferenceRegion - * @tparam U - * @return a boolean -- the input value should only participate in the regionJoin if the return value - * here is 'true'. - */ - def hasRegionsFor[U](regionable: (ReferenceRegion, U)): Boolean = { - !(regionable._1.end <= endpoints.head || regionable._1.start >= endpoints.last) - } - - override def toString: String = - "%s:%d-%d (%s)".format(referenceName, endpoints.head, endpoints.last, endpoints.mkString(",")) -} - -private[rdd] object NonoverlappingRegions { - - def apply[T](values: Seq[(ReferenceRegion, T)]) = - new NonoverlappingRegions(values.map(_._1)) - - def alternating[T](seq: Seq[T], includeFirst: Boolean): Seq[T] = { - val inds = if (includeFirst) { 0 until seq.size } else { 1 until seq.size + 1 } - seq.zip(inds).filter(p => p._2 % 2 == 0).map(_._1) - } - -} - -/** - * Creates a multi-reference-region collection of NonoverlappingRegions -- see - * the scaladocs to NonoverlappingRegions. - * - * @param regions A Seq of ReferencRegions, pre-partitioned by their - * referenceNames. So, for a given pair (x, regs) in - * this Seq, all regions R in regs must satisfy - * R.referenceName == x. Furthermore, all the x's must - * be valid reference names with respect to the sequence - * dictionary. - */ -private[rdd] class MultiContigNonoverlappingRegions( - regions: Seq[(String, Iterable[ReferenceRegion])]) extends Serializable { - - assert(regions != null, - "Regions was set to null") - - val regionMap: Map[String, NonoverlappingRegions] = - Map(regions.map(r => (r._1, new NonoverlappingRegions(r._2))): _*) - - def regionsFor[U](regionable: (ReferenceRegion, U)): Iterable[ReferenceRegion] = - regionMap.get(regionable._1.referenceName).fold(Iterable[ReferenceRegion]())(_.regionsFor(regionable)) - - def filter[U](value: (ReferenceRegion, U)): Boolean = - regionMap.get(value._1.referenceName).fold(false)(_.hasRegionsFor(value)) -} - -private[rdd] object MultiContigNonoverlappingRegions { - def apply[T](values: Seq[(ReferenceRegion, T)]): MultiContigNonoverlappingRegions = { - new MultiContigNonoverlappingRegions( - values.map(kv => (kv._1.referenceName, kv._1)) - .groupBy(t => t._1) - .map(t => (t._1, t._2.map(k => k._2))) - .toSeq) - } -} - diff --git a/adam-core/src/main/scala/org/bdgenomics/adam/util/TwoBitFile.scala b/adam-core/src/main/scala/org/bdgenomics/adam/util/TwoBitFile.scala index 5e98bd7d3d..3e253eaebf 100644 --- a/adam-core/src/main/scala/org/bdgenomics/adam/util/TwoBitFile.scala +++ b/adam-core/src/main/scala/org/bdgenomics/adam/util/TwoBitFile.scala @@ -22,7 +22,7 @@ import java.nio.{ ByteOrder, ByteBuffer } import com.esotericsoftware.kryo.io.{ Output, Input } import com.esotericsoftware.kryo.{ Kryo, Serializer } import org.bdgenomics.utils.io.{ ByteArrayByteAccess, ByteAccess } -import org.bdgenomics.adam.models.ReferenceRegion +import org.bdgenomics.adam.models.{ NonoverlappingRegions, ReferencePosition, ReferenceRegion } object TwoBitFile { val MAGIC_NUMBER: Int = 0x1A412743 @@ -96,9 +96,10 @@ class TwoBitFile(byteAccess: ByteAccess) extends ReferenceFile { * Extract reference sequence from the .2bit data. * * @param region The desired ReferenceRegion to extract. + * @param mask Whether to apply masks (with lowercase letters) * @return The reference sequence at the desired locus. */ - def extract(region: ReferenceRegion): String = { + def extract(region: ReferenceRegion, mask: Boolean): String = { val record = seqRecords.getOrElse( region.referenceName, @@ -111,26 +112,63 @@ class TwoBitFile(byteAccess: ByteAccess) extends ReferenceFile { assert(region.end <= contigLength.toLong) val offset = record.dnaOffset val sb = StringBuilder.newBuilder + + // define predicate for N blocks + val isNBlock = if (record.nBlocks.isEmpty || !record.nBlocks.get.hasRegionsFor(region -> None)) { + // our region has no overlap with an N block, so the predicate is trivial + pos: Long => false + } else { + // our region does have some kind of overlap with N blocks, so we need to check each position + pos: Long => record.nBlocks.get.findOverlappingRegions(ReferencePosition(region.referenceName, pos)).nonEmpty + } + + // define predicate for mask blocks + val isMaskBlock = if (record.maskBlocks.isEmpty || !record.maskBlocks.get.hasRegionsFor(region -> None)) { + // our region has no overlap with a mask block, so the predicate is trivial + pos: Long => false + } else { + // our region does have some kind of overlap with mask blocks, so we need to check each position + pos: Long => record.maskBlocks.get.findOverlappingRegions(ReferencePosition(region.referenceName, pos)).nonEmpty + } + + // iterate over every position in the query region (0 until region.width.toInt).foreach(i => { - // TODO: this redundantly reads the byte at a given offset - // pull out the byte that contains the current base - val byte = bytes.get(offset + (region.start.toInt + i) / TwoBitFile.BASES_PER_BYTE) - // which slot in the byte does our base occupy? - // 1: 11000000; 2: 00110000; 3: 00001100; 4: 00000011 - val slot = (region.start + i) % TwoBitFile.BASES_PER_BYTE + 1 - val shift = TwoBitFile.BYTE_SIZE - 2 * slot - // move the 2-bit base to the least significant position - // and zero out the more significant bits - val nt = (byte >> shift) & TwoBitFile.MASK match { - case 0 => 'T' - case 1 => 'C' - case 2 => 'A' - case 3 => 'G' + // check whether we're in an N block + val nt = if (isNBlock(region.start + i)) { + 'N' + } else { + // TODO: this redundantly reads the byte at a given offset + // pull out the byte that contains the current base + val byte = bytes.get(offset + (region.start.toInt + i) / TwoBitFile.BASES_PER_BYTE) + // which slot in the byte does our base occupy? + // 1: 11000000; 2: 00110000; 3: 00001100; 4: 00000011 + val slot = (region.start + i) % TwoBitFile.BASES_PER_BYTE + 1 + val shift = TwoBitFile.BYTE_SIZE - 2 * slot + // move the 2-bit base to the least significant position + // and zero out the more significant bits + (byte >> shift) & TwoBitFile.MASK match { + case 0 => 'T' + case 1 => 'C' + case 2 => 'A' + case 3 => 'G' + } } - sb += nt + // if nt is masked then make it lower case + val maskedNt = if (mask && isMaskBlock(region.start + i)) nt.toLower else nt + sb += maskedNt }) sb.toString() } + + /** + * Extract reference sequence from the .2bit data. Do not apply masks. + * + * @param region The desired ReferenceRegion to extract. + * @return The reference sequence at the desired locus. + */ + def extract(region: ReferenceRegion): String = { + extract(region, false) + } } class TwoBitFileSerializer extends Serializer[TwoBitFile] { @@ -147,28 +185,28 @@ class TwoBitFileSerializer extends Serializer[TwoBitFile] { } } -object TwoBitRecord { +private[util] object TwoBitRecord { def apply(twoBitBytes: ByteBuffer, name: String, seqRecordStart: Int): TwoBitRecord = { val dnaSize = twoBitBytes.getInt(seqRecordStart) val nBlockCount = twoBitBytes.getInt(seqRecordStart + TwoBitFile.DNA_SIZE_SIZE) val nBlockArraysOffset = seqRecordStart + TwoBitFile.DNA_SIZE_SIZE + TwoBitFile.BLOCK_COUNT_SIZE - val nBlocks = (0 until nBlockCount).map(i => { + val nBlocks = if (nBlockCount == 0) None else Some(NonoverlappingRegions((0 until nBlockCount).map(i => { // reading into an array of ints val nBlockStart = twoBitBytes.getInt(nBlockArraysOffset + i * TwoBitFile.INT_SIZE) val nBlockSize = twoBitBytes.getInt(nBlockArraysOffset + (nBlockCount * TwoBitFile.INT_SIZE) + i * TwoBitFile.INT_SIZE) - ReferenceRegion(name, nBlockStart, nBlockStart + nBlockSize) - }) + ReferenceRegion(name, nBlockStart, nBlockStart + nBlockSize) -> None + }))) val maskBlockCount = twoBitBytes.getInt(nBlockArraysOffset + (nBlockCount * TwoBitFile.PER_BLOCK_SIZE)) val maskBlockArraysOffset = nBlockArraysOffset + (nBlockCount * TwoBitFile.PER_BLOCK_SIZE) + TwoBitFile.BLOCK_COUNT_SIZE - val maskBlocks = (0 until maskBlockCount).map(i => { + val maskBlocks = if (maskBlockCount == 0) None else Some(NonoverlappingRegions((0 until maskBlockCount).map(i => { // reading into an array of ints val maskBlockStart = twoBitBytes.getInt(maskBlockArraysOffset + i * TwoBitFile.INT_SIZE) val maskBlockSize = twoBitBytes.getInt(maskBlockArraysOffset + (maskBlockCount * TwoBitFile.INT_SIZE) + i * TwoBitFile.INT_SIZE) - ReferenceRegion(name, maskBlockStart, maskBlockStart + maskBlockSize) - }) + ReferenceRegion(name, maskBlockStart, maskBlockStart + maskBlockSize) -> None + }))) val dnaOffset = maskBlockArraysOffset + (maskBlockCount * TwoBitFile.PER_BLOCK_SIZE) + TwoBitFile.SEQ_RECORD_RESERVED_SIZE TwoBitRecord(dnaSize, nBlocks, maskBlocks, dnaOffset) } } -case class TwoBitRecord(dnaSize: Int, nBlocks: Seq[ReferenceRegion], maskBlocks: Seq[ReferenceRegion], dnaOffset: Int) +private[util] case class TwoBitRecord(dnaSize: Int, nBlocks: Option[NonoverlappingRegions], maskBlocks: Option[NonoverlappingRegions], dnaOffset: Int) diff --git a/adam-core/src/test/resources/human_g1k_v37_chr1_59kb.2bit b/adam-core/src/test/resources/human_g1k_v37_chr1_59kb.2bit new file mode 100644 index 0000000000000000000000000000000000000000..f744bcf69c75f75a5e1dc0571e26d5b4413264df GIT binary patch literal 15031 zcmeIX`B&Q4);|skTthUXpmk_(o#Hi^IDyt^98lrWL_`FsO4~RfbpqNaP43Y42FGiR zV;rj30Z}QwQPen8A)<*xO`@Vfjmqs)6BDbTV&V`Lhi~pPe4q9C;qwQ4*6M5Rb=Gs{LaAd z4E)Z(?+pAu%D}m(%Kw`*UZ(!H{P&oO|JA%T{Y%++`G029DpUVUqALG$+W+X^K1ToV z3;uh-KR+aQ?OeZp6Ir2^hEH9GtyO(ehhgvk$a;tR=W&ypNUT6WWPMU{)Tg77NwjFEhyeg4f=<z8Yv-cS=h|H1Fq8yh!WDl+Q+)^hV7-$WZ3 zbsgQ|R8CI)fF0-j5{-Rx=cqxo9fEaH__1avr0XXP)lLZ3VYEZ>>BmIB&i-nzL$Z<8 z*7@D}nF^(fv|oO81~HX-m-Kgo{-#P@B6i@FwF@3?zlS=6Hv%(uS$2H$fJsvnM; zd#v|kk7xJQ(fDL81pJ`%sdXge3>gBsUpQ*yj_6|JJ{krm9|dl_s(p0^t3V`pZ6QnP zsn{J1eJed;-e1uM#cb$erey&ovxXKpKk6Vd=^Xqlk3QG0Ueu|VI%EkACYFt^bm#Mv z;}&@1@aNZ_&b()?B2|VFKgZ6#-<9@sdxB_P^;D_VZnN%@0(VFoDuu*aHT#}tZDYos z-xb9(WM^s%8M*a%T=s)@cm2jLE4@ELTH^9Uw{ij#K)V{5ek14!ocRg{mIgSZi z$?pB_Y3Uv_1k+)bG05JMZ9J~^4e0pi^{6A$n@|6VcSq^^{!m1XtwcP7hA&x=x`@~h z6jA8usG;q+9X}butX~|~(k&dCJ4~Qjnsu^g`Zzl@X&+J{rFi*lk5mocUYOpVrjvfI ztv&MB9BbswS|cLGBrIZE+bTUNDmT`t!W+`2FNEBW)u?vzcH{apJ-g3Ok*X#fKb)ta z_P=x6@ke%%{9lP<12>8B%5X)qC2ATRNR^S3E5FQ(Hz|)aB+|iE8doy(VqfPxWBN__ z3{*d4M!E6Tz(@dPNVP`^6Hhd9WSBL2hY_c;uXgVI%-}{-|H_Awdd09`!nGb8eRql) zeqf$6WRCjhj&^q2qGR|cdnQ9%e(Ke+X~;0oz)-=R<_n&rJFw_0CVXd!S<)rFRiW%m#r$5-%%+J`fjhy~RzX~dqP!-(W5Vwp7qt&qZ*W|GTRbF4}0keOId*O$T*PG5U@<(J$^3mP*XRfS<$*uQLP5(le4qZ$NlJUc7LT#M^Qf@7t@2! z$a0sh#$$u06d{a-o8s8sfC(9FBaf=AqlPM!9_U0$rt5 z%E_1B%=F_}g)^=Psw0-4M0FCtZmHaQ!BJ{;+P?(T`pAFv>ZI>{k!*yT!eNy-^U3#6 z7RnowJ4Sz*p$Gq3Qg%V+$4b^6rZUU|I;s2qPj!j|&Vo+8J=NDwI+!eXPw&DWAGcRT zTYayi{t}?QT4CIa!JJTbBXC|-l%H-Ugpc=>QE(wwqLYa-Qf0)GS!_fGVUYQ0SGRW4 za2Dr}?LSqntLdKTRQrg=u#fc(T%sY+>;b;Kn@{5%P;u4W zRf&yHNs~>Q$d`|-3gT7P0_GofFxJ0g*WKH(k`c`+G#h^=VwLYwD8B?x*Bz;& zIk+@?!aiB>JfXeXEv(kQ>sNnOHM0A9?;xShdY(jkT%^PV`LMYmtH5V zR7vF6JU(e_rFAE@>kLV@j$GIw%@QkyVnc?L$+1K?I?)87Ea9EZrLiDncVsjxFX@&3 zsct&`#Qb2{0TC7^`;Vj(>jh=a$LF@FQrj51HS;wp`mXsfY70~Q#Qx8-stVQ47oT&Ewt_?U zD1ImZ3{lM*%v$UQF^?F0qaUL2obV7gYlTWiUj`C9oUZcR~5NtpLZI>WM&Tx>bsy(E?vP}5}~ zk{@HB^lGYtg!i7&(mIeudVST&u@;liLWJLqg`Fu&ujX^eqOnKJYBE0KI^}{{7+x~d z6-?vIGGfPNEW*6y%};;o!d&{LzgVN{`w!A)lDwZIKU6F!`Uf`pWWT&Pf%&SBPqc-{ zmvl#7FGx?0TLKD0#Dgm z>MP+Wb!WSG2|rCX-%qM7AJr&g`D_UwU9U31WoTQDW0P59QwbmvN!N$VQKApYES?g< ze|ZJUQIYR^8joN6KJu%3b1lgDFNbT7I&=pSFHZV-eb{iyxEW8b*?v|gHd?LfOs(%C z0bu`CA8uKmBTrc0y-NsXbWS5JTafO&50j>VeQ}z9Zp}BdWzSUqghVV3+ig z&XPlaoix!E$XieIiS3H(XATNW$}?HPpK71^$;{2~X9_6~)a6I4ntsje&P|JFE9q0c ze7#nBzYDRhszVS`%y^Cc+}7DLXq_*m7aK_QFFAr$8<8~ z-Q0_g1uV$nFe!l%g2x$B-?_Lqt!;$Hl|hCRDAXyyJdb5|?z@P*QVnXz*Xg69sIs!$ zR0M-D;~87&+23WRNLK9gUj7ZkC}XlhY}zXfFNjDyhW$SbP{Inhcc>&Di@tnpc3xu!T@4$RRyt zCk7gPoSPGJWBWX99kzAk$y`TeT%PS18#y2CM}zxAj$M-EakE={YikOe|AAt^c6kng zQog)MaW70ZI{W>42?crJHTyD6lNOPrQ7<(1QRE5S3z21i+n#b*+~n&AO}+|PL)QLt zHV$%j(ZoEYmm;0{ZZqjh%DK9__o6V+cskj&rb885U~G$MHJ{h|&zJabu)QWItMoPP zI;X5JZkr;TBT=cpVCXDkDUFe`s$#^R<%p(HGbRMHT$1WIDb?Na-t1la@_}4(9mxHO zNS3uO#H!2^jMLRj?kC43pdE8avvN5=5mV={ED1rs^g`jHw#?q2=XR$(?@oK`DGWo0 zYz0Tp$FEVW^0;ZWm{)tSae+%|#a$WaFz&}kdbyo}>*d1*nP=slRR(8yI9~DUH+`r3 z2RCP0Psua@M@Pz8PoGOQzt>1^FJ+7kQy!3(As3t8-|gCJ?>LbMawJ=uw806sOiL8| z=s4T-p*KAG!7)d*x9uOhu}f+1Yp*V)kzSoG)FzD`qmJb-TL|Dbto^KnQ!!S#TuFMV z7YE4Fk~7Q73SH2QHI1eJOl$cE_gLj|j#4Fy%z6(`D2aUsOOH`bMxGuvCv(EvlCTKm z%6({RSgaM59ax2S=aRoA;wTjxakJZI#b1(Z&&J>>!levBMRnuNS?|#8Gr4|9OIP|3 zVFofz7$D~q)c5?R^28=JPdq3p6Qyj9|Db0tIPL8y%Cd5a97%P{mfXoRH8MCo1i^bs zF<;OXveCiFTm>orc8^(&r%}y%Iq6Zpp`X{mk3I*k)VKh-IH_%7T^7qcsM0;@I6OF> zn0mjAz6MOj4Q>-~UF6u>gW09cX!+H}=++}FO*=M7TAYz{mS#-ywoZKgx@nY%o*{ZK z&g(U?KjC_g^0#U7dpBPB`THFGkof8%k+Y^uAaL4CJTb3ySAVkZs{B6o>+hngII3%7 zzC7lP1?`Sk;v|b$d;9qDF?-I)w~zBv;?h1c!V|l;qE#RDf4Cp865em+uY+yeTZl(i zMx~D%o%QUi1SdugWmw~5gP#kf6_>oUfMolC*Z){_a-o02D8h=EcB!U~1k3~VUd)>i zv(j@^ZJqD*RD|%wQOA>skbAe+a=VC@^e5}5Pk37Ky=ldvZu#J#sf@&5=;&t;aq|Q8 zvdTiJTd`ydxw^zzQ|-i|%wwp%M6)aNj*6Y& z!BpPk$9sEgilo+Q`J$!q0Dzr{Tx<3mipY5u_!nqAzR91$kacOS)k=Ykz$N#FoEYDOOY-x=;JSFeV`jxS+EPT1uc#p1W z^^&kMIcRBZun-t~eEd!NSHT?-hdhhOljb`baeYB;Uxp!CND!f)==5J;R4Zzf3k5qY zxYSj(o5W(Qm#Q((hgN(GXPrGnMOA#OJ{Lez2-q6f7-VN%yDX@(3CJ&Bgp2YSe2Z`h zsPwdmX@3!(4*`P^C<5W16mN4$eLFH7bkzMECG`(^>6`$-EfT-A<=W5SpmxZ|foSiS zg*|n8hSWe-?`u#Hw${G zg)`4u=%st%pZh>tKyKXZ`~kNCiv_2JoC|T)`lruVl{;GoArb?J3?5a8T#z8|@_vAK zoFlwU|M2|SD`DVgrRQ(uMe_%SESU`BwtII+LqWU(b!8u~e=uvoAke2JylpB+)BM}Z ziXc5jZSA@9z7o!!zVY7yK+mfAb4cFtfW5Tx0xH8iJ@KKxAYV2n?Y8YAm@ufup`EHN&r6p*KQ(sqp;~3gLY8QX3kFs9)G-P7x=gF z_q%ZT-p}yN=qb(+{_hXc$x7Gg#;4tlhh|iH`AYPaempR>ooY2FKNBgAHMndi2n7PKV;~x&fjG1 z{3@ge|I!Efiu2>}P4dSdnd>w3cRsxS&a2DUpZ*&)_5*74`mcC-^v{*LFNWQZzI^Y9G=D(&g7L}JC)b!? zn-vWrKKUo-V`}9%7)$IWLtV(92M#AFBcrXR`s0-E#F4kjH(BJFlMkX#IF)qTb6q zOUKXjm6Mj%zYp$Cjadb_3~)$WC9UKB^&OAB-^>fgR>I(B1gN&z9LU*#tvJUUaBZ;Dq}Zqs++{`x6`!I1P9$}j3^`eI`QzX@ zN%d^_zmbSPVTL2Cv}jGw9=@#5V|itukZfKbD;M990L2c$ zpoioPY|ObM-OWt?6Wowc0@%dlJzEPL|o3%w6>6Q|iM%T!w$;Xw$owJdv z#-(x7^oEIJw1>A|^Yx6e*PwxmPCN<_4<(l+`=;cqZ96}-UQQKn)SDIYwvv-5uUoqO zYWrJi`mFbeHt~UDFG<KK_##$uWlYdvL?a8Bdgcf4B@-vbqa;gjBw{w4Fq8ikjDd zto&|cAL~n#>G!Zm3pNvrF(Et=fWA$)*s`n%Z>NWqXd~X}o|Uc}Z~SpvK^2w&C zUIL@0!q9EhiaLko?Sv02`0yU6Yo{upj#R-S@uV!TSD6-L`?W1dN+fot^T_um>$32FVse)r2We+iRI8x9IyXiM9i3sGfVrjv$G zGLQ#sya`HOg(KgXa{*WN3}G<`z0lSol1#pLP)G^6Tzk8=sH;~h7|6kD!A_%RBs&47 z8dI8f3S&6G&?BiZ!|0$gIw(;{d;cN^wzg(jg?i~3@!;jD5(GI!-i2h*SMibmQ2VgB z8eG-#%ihilLj6@om*-=CyrhZ3^7>eB zw-k8&NQ0dZxFQDq_Ji%SF@uEU#N5t~g^%+{?Yq5Y6y5$ezAG27tSq+aB3tXfO_Tjy=o?ko2|O;)!01RpNEJ$$gM> zGx+ifzVfHiDa#zhPB*EFS%!0Q)V5z)4Be3QYZVLqEEa~9qufY77rCX~RA7%algQu3 zmQ_7X@V<1{w#rKyj%t%PXTFfs(yeoY!zF!f%5$l9LP2YgFy&x?OSpVo9hY0|03SJ~ zZIWX`H;D;%at?9K?2o2`wE+^JCxt@hn81eBNcucG=rML zTz#-h8Pogs#GDkQ@9sMl3bicU5LQn|O-Av?PIo$8gNdqd)$83lCE;!zi?U(@srugj z>iRzY)&WWBOzZpf@s6p&(7L#{`-_Mz-Lpw8ea-4+mv8EA-%@u|wCF7WUa& z-1l7tMY)ihs$@SC`ozY*If%LYCZN+gIRUxDE~b%sgIFfv$+S#jG?EppmHI1Nr)Q9^ z6YAq_F=af~lSF(cUbX%K6$SOHeZ`#lTjghC$I4xfigPE|SUA-$H=xaivwj-y+>WQ& zooMM@{nNAQ#;v4-lRlPgjG_$iL-ubIP6D3PSs%#_9_2Oiuf+Di+x*`;HEsHhdIg?^ zZg9ElU#PaAxMFS2{U}QKgjqam@6()sqt)cnYUfeuWV6a<O!dAx^aV_qPBl#oVPLKTk!j%m*z`={-J zV-U!YbJp^_jh;wuROnOe6zFef{6N6B$!d0j1p#hOkHcK}aDGc6HrZex)cwTDtk%9d3zSHX={+_gF zUeP{A6i>=Wf%n6ea~(s;@2G66Vgs0+Cal_241$X{ETg+IW0VZ0rtA-AYz+5hQcsL5 zAfn>~<0&>ZVvsu9+Ju&*mSENJG(X6C;jZgg!^3vagu$FRueD^YyLg1D zMH)#Kd4~^^*cJGy#?w2&2a_3VXGup1#+**gWAWoQTT^jn$>?@4?;m%3qrYj2l`ZeK z4ji|*S;lsd*v3fLeVLa?ttq$er8N-UCDQHZO%8ru!fVgK6Kd(+OZKjYrcrfCmi=?g zs)L%sPah@8$NE72L>7&2Un0;o5w8q~yQy>1h$7Fzb>kLUOl)n$jEiq$hEiEWDM>yhd)d~9 ziNjwcN|nbo0q=5EM1TDNzI#n~w$8Gaa&acLj!7owP^T@S7iv5s@F7oGpOUCVH=HJx zqm19CO&gh%2jWq(E*!Bif8Yg!>WQ*0eFzs39?d~XVUci=S7m8TbZ%Ty3^x*igF;aM zx}rYA+zc6^P3ycSl5)Pb^r|4@=?m(9ZBdGNDSpvIFRWwP>Zlr_^h=Pu7!{dbG;AAY zI5&w~F!2b7S1!ts{&+}Occv?t7CAA{2@1^=1H;zz9uxWGZDDQ`;yB;gU;Fx6dQEEg zm8Nzk!u$h&ogOxH;(l2|RwhiC-T{~M)|?J07LLJCPH{Xg2ZeL2xNip~7aGv9$SbDz zOXGTD7OVUy)=DSr0yyOqf5akm^LeY?f{iJd;9cKpb*f=@R8Yzd9Gb1Du4!#aE9R^_ z0D+kX{za$k&$6X+CQ(c{B_G=U^dp%!>xEyk9+%5R0D?Irnkb@?fc zy!iC+j5D`+9`(G2>M~)EaG`4Quic-C4--=h{`i=0i|kog*-Xnns46IY?pbqAiWUKR zHhCt3B{nd34tP|Jun_d-J#P}Z3N$MZZ`24{WAcSv%qP`}x6yf)<&S`RaUkmQD`|mU zE0hr&3du}-_l_9VB~NilD-Z!SaoOtUnzTTq9{XGYBL8s{6`8 z%d6&zGCEA`7y@}vtyKCzD!j!I-mXg%0S>ML{;9MC4+d}tUF`#C&J-*sqFq{SEgsy? zQa1|>oeRl#f?D`?&z$BAymD}WnRpbmtVRPOQ&CZIBTo{((kV^}?(SI?l3MZ@6*g9N z4!q9W-f0U-x7-VwYI>SN5Bv*cOtrH)Q!hTn8!Y6xOM2^%#XJ*OK{58A$qi^1R)IuyK7vwx$#~WR6k%V6?&hQcYEiFfBXpJz@J)GX z((kQYCV=M^!Sy;3ci|4f{F~!5w(NY~Bc{L|cqI5Q@I@jUbltADcBPj~(D4$}j+oCA zfOuzmc|K84TyMK-J;AVba%kN4$WTWWOK&+2Se zJ71}h_KLETau_Z-K%Ec4rF-MHeaR`;2S}Ap?yC-{b`yxz04(eAVzu=#E&uzZGvRze zVYYoRVB}P5XDE7*uXV~Fb2@iM4+i5UGVS0(%W|7oQGNl)oev1OaM6Yk=v)S|;Xsz7 zhR=@|qlNVWz$K8VK(A2*7j)Z6Lid4mQCtClPV`yHY_bH|ibapO==^RYn=x=ffm43c zQWr)pTqZaW#VIc&PWIM$KBpdJO)e=c-7KFA9PMp^glfm+f}Z>aw>&^?r-Jj*TsZJK z$oKw3LQs2+TUx%wyaj{+!@xZ_504lHnYiw9^+Bn|r&Uw-+O5&f;@GOR|g2hnZdwOq}DrDNQ~RcCn5VoQO&C zOOdr+d>65 z@ak>@;v--Oc$9Cq9o84MHZ*e1Ar+B*p`giKxO_{>D;|4sXg<+ka0*y-k6BC8i^ca` zk=BCNBMtNkweY#^nBgFs^X#(MZtw%?CC9%lJHlXkjx}6sx0UC7W5I*kVswWgxIH&C z&gT&TjQIpe>Lj-fzRyq2ju(gEg28C1?HOi#i{-`Wnxy*v1Q_qq-fxIDGb_S_oaPpb z*OuV$DD7%K)az~0hH`-B#XSl^=7UVmsI&54BtE-jCQ)1Y?_6?}n@Z*JU89fm@TrKfEQ_QGuuBaxAXiQa`waO?*g zan<*$uYCaJqUzO(ljcRpu*GdD_9Vs(X&=LV-okyJb3SGID4j1pi`?ihDSrofN4I7WhXBcosy&pE8~HVT}kyyZ<&>(=T`ovu-hI0rUVY_SsV3fR*m_- z`3V+PyKjG5xY$Z0eEz=g1Gc(f7~~j!={Q_uV-E7s`@bD%H*{=qLDbCE}k!_Lv z?2Q~NOP>*4O|>bKYZ{+3puTsOVbcNclGw6ZZ>OdrS{?8~xBa_1!H`c{j~o@+bkFJf_lb{fTpnC;rW7%1_*Vi= z^!649;xmFwKvJPAz~>4I_)dO+&uc#e$rw1)4K^kxgqsS=_bXd7Jw&tS4=fBaqsjmn zEXgJwWGdu$d#oTIh>R0gD-XHg0xw!Ycszr@QQ{3x+Y>a~H>J#`RfmT(Y+?Qj&iyhO z04^gZ)>+48c1t4_AOU={0`SZ-Z{7@%6?u=| zI+d@NF=|yHvpZF=Y#ZoNRcaV7`~HxASpLU7rt+uWcgSz^yL9oH>@Jzw`mm@+R11HTRSgq-^i$P%haKsS*0o=(+)L@_UuTFz)UJ^H} zAgiDt$RK$R8VA!uAS}Qp`WR3+E*~~;#I!Nv)p1L#cnqs+7r3L46a={x5y`%Vp2V3( zM2wWg`MW$Sfc$EU*0)*Q2e%-OmlECvwoM$;Ev#h4mIRo<4DZ)E*}Nd=@uTmRN0H*< zZ6V&IA(ODSh(Oq7Vz&hZoL-dY_#9+mIPw9Szi~-PIF~dh<2Vvi!dvh$XD>2~E7J}@oDA0!)qVz+PDW-d2 z)o*Kr>eZo`gWdqZ-{_vHYRN)SKb}wTNH2yBB};02+N-y#=4>XM8}5LPpAU4mdKOBv zv)W~&feoHjY)y|7%nld7A?uOc0^*ch)mqaTn=}V5rM?>MTm2;4$pJQ*RxRM$THNin zTUrqTmhtQCn6NVRwRc3ezd_2s@e72Ii{4`g(gE zY!q+<>%-Ga(Jqe5>s(6^syZYK0u--irnNWbHXE1qWUBv?e4BtDE5rHB%mgCG^=l4(bKs2(W6qlS?getEj0 zFkW9HwHt~0pu(&aK;6;%!Wu0F_GJpCKu>TDZG+t52t@<5|J8+TlxJ{g{19hd>L(4*_7X{!;#(lwO1 zVLs{#qauq8dh|j#W5ow&b`;_WMqFoD^}0OpXSZZi%RIE*LM{4^k?^9^sLx*}SzySY zV0j#i;>%K8AlxL-6q4cuK~fSb(oF&;>KOI>NTW{YIB4CO*&L3KhJrR=5Qa?*?_3#5 zYQSaj@SP@ZU1?n)Wt6(I80KlMX5mE4>iZr@gW#Zl2l*E|aA5#ke=jw)Gzg*$H{B4$LoFby5_PeFWh$(X^=NGMpEfGBo{f24 zq03qXV-R*$PU5&peE(>yYp_ZElz@ZM_whb!NiJ!sK|EkLu!|vFFeu+Je*h{V`v4K< zfQZL+@|YdTl*`etfKn@Ig>|%$(WI|qZtj|E(ec=Vh;wqyvv`|ix4bM){3yJK?PkE` z;zTgmrCDd6*Cy`C5&2w8GYi59bd}F!NKVB(sXvB*9J=bySEQj(W@b^~Jk|Z%+an4Z z>SFVosKrjOf#ECm%?R|8$AD?6#gutg78=Jj4^hjPkU1)^JjvGsTV(U}43r>rvxP;v4PqT+>RUVSc6#BF!Msn(f*4F; zB)x!xstwegynL5}M*-3N5B0J}=pZ!R$nb zoCD#j^Cm)}j*}3*z?cEmMS&waV^m$%448%xwqQ=9DWlneEfMz`ozY8Y3(e!Oh%*-P z1{4tF^kQHT6iTUt=t1JRfpIW|8!i$BDnp>-L-&znT#a}3kF@hJ8{_Q1xEbh)cI#no o1u{OkUGIfpGwHnXA{5~azzrBoyd*7|o6()W0(