From 92c67ebea564f6612c422094b2ffeb447b8454b2 Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Wed, 17 Dec 2025 09:10:54 +0100 Subject: [PATCH 01/10] #809 Add support for reading compressed EBCDIC files. --- .../cobol/reader/stream/SimpleStream.scala | 2 ++ .../spark/cobol/SparkCobolProcessor.scala | 8 ++--- .../spark/cobol/source/CobolRelation.scala | 31 ++++++++-------- .../spark/cobol/source/DefaultSource.scala | 24 +++++++++++-- .../cobol/source/index/IndexBuilder.scala | 11 ++++-- .../cobol/source/scanners/CobolScanners.scala | 9 +++-- .../streaming/BufferedFSDataInputStream.scala | 35 +++++++++++++++---- .../cobol/source/streaming/FileStreamer.scala | 20 +++++++---- .../cobol/source/types/FileWithOrder.scala | 2 +- .../cobol/source/CobolRelationSpec.scala | 21 ++++++++--- .../cobol/source/index/IndexBuilderSpec.scala | 26 +++++++------- .../integration/Test5MultisegmentSpec.scala | 8 ++--- .../Test12MultiRootSparseIndex.scala | 5 ++- .../source/streaming/FileStreamerSpec.scala | 16 ++++----- 14 files changed, 138 insertions(+), 80 deletions(-) diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala index 561116121..50aae1959 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala @@ -29,6 +29,8 @@ trait SimpleStream { def isEndOfStream: Boolean = offset >= size + def isCompressed: Boolean = false + @throws(classOf[Exception]) def copyStream(): SimpleStream diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala index 8f0763584..9d70cfa68 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala @@ -229,8 +229,8 @@ object SparkCobolProcessor { val numOfBytesMsg = if (numOfBytes > 0) s"${numOfBytes / Constants.megabyte} MB" else "until the end" log.info(s"Going to process offsets ${indexEntry.offsetFrom}...${indexEntry.offsetTo} ($numOfBytesMsg) of $fileName") - val dataStream = new FileStreamer(filePathName, fileSystem, indexEntry.offsetFrom, numOfBytes) - val headerStream = new FileStreamer(filePathName, fileSystem) + val dataStream = new FileStreamer(filePathName, sconf.value, indexEntry.offsetFrom, numOfBytes) + val headerStream = new FileStreamer(filePathName, sconf.value) CobolProcessorBase.getRecordExtractor(readerParameters, copybookContents, dataStream, Some(headerStream)) }) @@ -240,7 +240,7 @@ object SparkCobolProcessor { val hadoopConfig = sconf.value log.info(s"Going to process data from $inputFile") val inputFs = new Path(inputFile).getFileSystem(hadoopConfig) - val ifs = new FileStreamer(inputFile, inputFs) + val ifs = new FileStreamer(inputFile, sconf.value) CobolProcessorBase.getRecordExtractor(readerParameters, copybookContents, ifs, None) } @@ -266,7 +266,7 @@ object SparkCobolProcessor { Future { val hadoopConfig = sconf.value val inputFs = new Path(inputFIle).getFileSystem(hadoopConfig) - val ifs = new FileStreamer(inputFIle, inputFs) + val ifs = new FileStreamer(inputFIle, sconf.value) val outputFile = new Path(outputPath, fileName) val outputFs = outputFile.getFileSystem(hadoopConfig) val ofs = new BufferedOutputStream(outputFs.create(outputFile, true)) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala index ff7e639f5..55339c23b 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala @@ -16,22 +16,22 @@ package za.co.absa.cobrix.spark.cobol.source -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.FileInputFormat +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types._ import org.apache.spark.sql.{Row, SQLContext} -import za.co.absa.cobrix.spark.cobol.reader.{FixedLenReader, FixedLenTextReader, Reader, VarLenReader} import za.co.absa.cobrix.cobol.reader.index.entry.SparseIndexEntry +import za.co.absa.cobrix.spark.cobol.reader.{FixedLenReader, FixedLenTextReader, Reader, VarLenReader} import za.co.absa.cobrix.spark.cobol.source.index.IndexBuilder import za.co.absa.cobrix.spark.cobol.source.parameters.LocalityParameters import za.co.absa.cobrix.spark.cobol.source.scanners.CobolScanners import za.co.absa.cobrix.spark.cobol.source.types.FileWithOrder import za.co.absa.cobrix.spark.cobol.utils.FileUtils +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import scala.util.control.NonFatal @@ -63,6 +63,7 @@ class SerializableConfiguration(@transient var value: Configuration) extends Ser * Its constructor is expected to change after the hierarchy of [[za.co.absa.cobrix.spark.cobol.reader.Reader]] is put in place. */ class CobolRelation(sourceDirs: Seq[String], + filesList: Array[FileWithOrder], cobolReader: Reader, localityParams: LocalityParameters, debugIgnoreFileSize: Boolean) @@ -71,8 +72,6 @@ class CobolRelation(sourceDirs: Seq[String], with Serializable with TableScan { - private val filesList = CobolRelation.getListFilesWithOrder(sourceDirs, sqlContext, isRecursiveRetrieval) - private lazy val indexes: RDD[SparseIndexEntry] = IndexBuilder.buildIndex(filesList, cobolReader, sqlContext, cobolReader.getReaderProperties.isIndexCachingAllowed)(localityParams) override def schema: StructType = { @@ -94,15 +93,7 @@ class CobolRelation(sourceDirs: Seq[String], } } - /** - * Checks if the recursive file retrieval flag is set - */ - private def isRecursiveRetrieval: Boolean = { - val hadoopConf = sqlContext.sparkContext.hadoopConfiguration - hadoopConf.getBoolean(FileInputFormat.INPUT_DIR_RECURSIVE, false) - } - - private[source] def parseRecords(reader: FixedLenReader, records: RDD[Array[Byte]]) = { + private[source] def parseRecords(reader: FixedLenReader, records: RDD[Array[Byte]]): RDD[Row] = { records.flatMap(record => { val it = reader.getRowIterator(record) for (parsedRecord <- it) yield { @@ -125,8 +116,16 @@ object CobolRelation { .getFiles(sourceDir, sqlContext.sparkContext.hadoopConfiguration, isRecursiveRetrieval) }).toArray + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val factory = new CompressionCodecFactory(hadoopConf) + allFiles .zipWithIndex - .map(file => FileWithOrder(file._1, file._2)) + .map { case (fileName, order) => + val codec = factory.getCodec(new Path(fileName)) + val isCompressed = codec != null + + FileWithOrder(fileName, order, isCompressed) + } } } \ No newline at end of file diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala index 63619cc9b..a052ed656 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala @@ -18,6 +18,7 @@ package za.co.absa.cobrix.spark.cobol.source import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, NullWritable} +import org.apache.hadoop.mapred.FileInputFormat import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession} @@ -59,8 +60,17 @@ class DefaultSource val cobolParameters = CobolParametersParser.parse(new Parameters(parameters)) CobolParametersValidator.checkSanity(cobolParameters) + val filesList = CobolRelation.getListFilesWithOrder(cobolParameters.sourcePaths, sqlContext, isRecursiveRetrieval(sqlContext)) + + val hasCompressedFiles = filesList.exists(_.isCompressed) + + if (hasCompressedFiles) { + logger.info(s"Compressed files found. Binary parallelism and indexes won't be used for them.") + } + new CobolRelation(cobolParameters.sourcePaths, - buildEitherReader(sqlContext.sparkSession, cobolParameters), + filesList, + buildEitherReader(sqlContext.sparkSession, cobolParameters, hasCompressedFiles), LocalityParameters.extract(cobolParameters), cobolParameters.debugIgnoreFileSize)(sqlContext) } @@ -126,6 +136,14 @@ class DefaultSource //TODO fix with the correct implementation once the correct Reader hierarchy is put in place. override def buildReader(spark: SparkSession, parameters: Map[String, String]): FixedLenReader = null + + /** + * Checks if the recursive file retrieval flag is set + */ + private def isRecursiveRetrieval(sqlContext: SQLContext): Boolean = { + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + hadoopConf.getBoolean(FileInputFormat.INPUT_DIR_RECURSIVE, false) + } } object DefaultSource { @@ -136,10 +154,10 @@ object DefaultSource { * * This method will probably be removed once the correct hierarchy for [[FixedLenReader]] is put in place. */ - def buildEitherReader(spark: SparkSession, cobolParameters: CobolParameters): Reader = { + def buildEitherReader(spark: SparkSession, cobolParameters: CobolParameters, hasCompressedFiles: Boolean): Reader = { val reader = if (cobolParameters.isText && cobolParameters.variableLengthParams.isEmpty) { createTextReader(cobolParameters, spark) - } else if (cobolParameters.variableLengthParams.isEmpty) { + } else if (cobolParameters.variableLengthParams.isEmpty && !hasCompressedFiles) { createFixedLengthReader(cobolParameters, spark) } else { diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala index ef334d6e5..8f9391f17 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala @@ -205,7 +205,12 @@ private[cobol] object IndexBuilder extends Logging { val (inputStream, headerStream, maximumBytes) = getStreams(filePath, startOffset, endOffset, config) val index = try { - reader.generateIndex(inputStream, headerStream, fileOrder, reader.isRdwBigEndian) + if (inputStream.isCompressed) { + val element = SparseIndexEntry(0, -1, fileOrder, 0L) + ArrayBuffer[SparseIndexEntry](element) + } else { + reader.generateIndex(inputStream, headerStream, fileOrder, reader.isRdwBigEndian) + } } finally { inputStream.close() headerStream.close() @@ -238,8 +243,8 @@ private[cobol] object IndexBuilder extends Logging { bytesToRead } - val inputStream = new FileStreamer(filePath, fileSystem, startOffset, maximumBytes) - val headerStream = new FileStreamer(filePath, fileSystem) + val inputStream = new FileStreamer(filePath, config, startOffset, maximumBytes) + val headerStream = new FileStreamer(filePath, config) (inputStream, headerStream, maximumBytes) } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala index 9d7456a9d..2e75fa92c 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala @@ -43,14 +43,13 @@ private[source] object CobolScanners extends Logging { indexes.flatMap(indexEntry => { val filePathName = filesMap(indexEntry.fileId) val path = new Path(filePathName) - val fileSystem = path.getFileSystem(sconf.value) val fileName = path.getName val numOfBytes = if (indexEntry.offsetTo > 0L) indexEntry.offsetTo - indexEntry.offsetFrom else 0L val numOfBytesMsg = if (numOfBytes > 0) s"${numOfBytes / Constants.megabyte} MB" else "until the end" logger.info(s"Going to process offsets ${indexEntry.offsetFrom}...${indexEntry.offsetTo} ($numOfBytesMsg) of $fileName") - val dataStream = new FileStreamer(filePathName, fileSystem, indexEntry.offsetFrom, numOfBytes) - val headerStream = new FileStreamer(filePathName, fileSystem) + val dataStream = new FileStreamer(filePathName, sconf.value, indexEntry.offsetFrom, numOfBytes) + val headerStream = new FileStreamer(filePathName, sconf.value) reader.getRowIterator(dataStream, headerStream, indexEntry.offsetFrom, indexEntry.fileId, indexEntry.recordIndex) }) } @@ -75,8 +74,8 @@ private[source] object CobolScanners extends Logging { } else { fileSystem.getFileStatus(path).getLen - reader.getReaderProperties.fileEndOffset - startFileOffset } - val dataStream = new FileStreamer(filePath, fileSystem, startFileOffset, maximumFileBytes) - val headerStream = new FileStreamer(filePath, fileSystem, startFileOffset) + val dataStream = new FileStreamer(filePath, sconf.value, startFileOffset, maximumFileBytes) + val headerStream = new FileStreamer(filePath, sconf.value, startFileOffset) reader.getRowIterator(dataStream, headerStream, startFileOffset, fileOrder, 0L) }) }) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala index b6d3cf653..c5d453d89 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala @@ -16,21 +16,21 @@ package za.co.absa.cobrix.spark.cobol.source.streaming -import org.apache.hadoop.fs.{FSDataInputStream, FileSystem, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FSDataInputStream, Path} +import org.apache.hadoop.io.compress.CompressionCodecFactory -import java.io.IOException +import java.io.{IOException, InputStream} -class BufferedFSDataInputStream(filePath: Path, fileSystem: FileSystem, startOffset: Long, bufferSizeInMegabytes: Int, maximumBytes: Long ) { +class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, startOffset: Long, bufferSizeInMegabytes: Int, maximumBytes: Long ) { val bytesInMegabyte: Int = 1048576 + private var isCompressedStream = false if (bufferSizeInMegabytes <=0 || bufferSizeInMegabytes > 1000) { throw new IllegalArgumentException(s"Invalid buffer size $bufferSizeInMegabytes MB.") } - var in: FSDataInputStream = fileSystem.open(filePath) - if (startOffset > 0) { - in.seek(startOffset) - } + private var in: InputStream = openStream() private val bufferSizeInBytes = bufferSizeInMegabytes * bytesInMegabyte private var isStreamClosed = in == null @@ -51,6 +51,8 @@ class BufferedFSDataInputStream(filePath: Path, fileSystem: FileSystem, startOff def isClosed: Boolean = isStreamClosed && bufferPos >= bufferConitainBytes + def isCompressed: Boolean = isCompressedStream + def readFully(b: Array[Byte], off: Int, len: Int): Int = { if (isClosed) { @@ -115,4 +117,23 @@ class BufferedFSDataInputStream(filePath: Path, fileSystem: FileSystem, startOff } } + private def openStream(): InputStream = { + val fileSystem = filePath.getFileSystem(hadoopConfig) + val fsIn: FSDataInputStream = fileSystem.open(filePath) + + if (startOffset > 0) { + fsIn.seek(startOffset) + } + + val factory = new CompressionCodecFactory(hadoopConfig) + val codec = factory.getCodec(filePath) + + if (codec != null) { + isCompressedStream = true + codec.createInputStream(fsIn) + } else { + // No compression detected + fsIn + } + } } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala index dc61db2fb..faed0dc54 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala @@ -16,11 +16,11 @@ package za.co.absa.cobrix.spark.cobol.source.streaming -import org.apache.hadoop.fs.{FSDataInputStream, FileSystem, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{ContentSummary, Path} import org.apache.log4j.Logger -import za.co.absa.cobrix.cobol.reader.stream.SimpleStream -import org.apache.hadoop.fs.ContentSummary import za.co.absa.cobrix.cobol.reader.common.Constants +import za.co.absa.cobrix.cobol.reader.stream.SimpleStream import java.io.IOException @@ -33,10 +33,10 @@ import java.io.IOException * file be consumed. * * @param filePath String containing the fully qualified path to the file. - * @param fileSystem Underlying Hadoop file system. + * @param hadoopConfig Hadoop configuration. * @note This class is not thread-safe and should only be accessed from a single thread */ -class FileStreamer(filePath: String, fileSystem: FileSystem, startOffset: Long = 0L, maximumBytes: Long = 0L) extends SimpleStream { +class FileStreamer(filePath: String, hadoopConfig: Configuration, startOffset: Long = 0L, maximumBytes: Long = 0L) extends SimpleStream { private val logger = Logger.getLogger(FileStreamer.this.getClass) @@ -59,6 +59,11 @@ class FileStreamer(filePath: String, fileSystem: FileSystem, startOffset: Long = override def offset: Long = byteIndex + override def isCompressed: Boolean = { + ensureOpened() + bufferedStream.isCompressed + } + /** * Retrieves a given number of bytes from the file stream. * @@ -123,18 +128,19 @@ class FileStreamer(filePath: String, fileSystem: FileSystem, startOffset: Long = } override def copyStream(): SimpleStream = { - new FileStreamer(filePath, fileSystem, startOffset, maximumBytes) + new FileStreamer(filePath, hadoopConfig, startOffset, maximumBytes) } @throws[IOException] private def ensureOpened(): Unit = { if (!wasOpened) { - bufferedStream = new BufferedFSDataInputStream(new Path(filePath), fileSystem, startOffset, Constants.defaultStreamBufferInMB, maximumBytes) + bufferedStream = new BufferedFSDataInputStream(new Path(filePath), hadoopConfig, startOffset, Constants.defaultStreamBufferInMB, maximumBytes) wasOpened = true } } private def getHadoopFileSize(hadoopPath: Path): Long = { + val fileSystem = hadoopPath.getFileSystem(hadoopConfig) val cSummary: ContentSummary = fileSystem.getContentSummary(hadoopPath) cSummary.getLength } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala index 8efb0db5b..bb9b59897 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala @@ -19,4 +19,4 @@ package za.co.absa.cobrix.spark.cobol.source.types /** * Represents a file attached to an order. */ -private[source] case class FileWithOrder(filePath: String, order: Int) +private[source] case class FileWithOrder(filePath: String, order: Int, isCompressed: Boolean) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala index 84aa01788..e233488bf 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala @@ -16,8 +16,6 @@ package za.co.absa.cobrix.spark.cobol.source -import java.io.File - import org.apache.commons.io.FileUtils import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -31,6 +29,8 @@ import za.co.absa.cobrix.spark.cobol.source.base.impl.{DummyCobolSchema, DummyFi import za.co.absa.cobrix.spark.cobol.source.parameters.LocalityParameters import za.co.absa.cobrix.spark.cobol.source.utils.SourceTestUtils.{createFileInRandomDirectory, sampleCopybook} +import java.io.File + class CobolRelationSpec extends SparkCobolTestBase with Serializable { private val logger = LoggerFactory.getLogger(this.getClass) @@ -61,7 +61,12 @@ class CobolRelationSpec extends SparkCobolTestBase with Serializable { it should "return an RDD[Row] if data are correct" in { val testReader: FixedLenReader = new DummyFixedLenReader(sparkSchema, cobolSchema, testData)(() => ()) - val relation = new CobolRelation(Seq(copybookFile.getParentFile.getAbsolutePath), + + val sourcePaths = Seq(copybookFile.getParentFile.getAbsolutePath) + val filesList = CobolRelation.getListFilesWithOrder(sourcePaths, sqlContext, isRecursiveRetrieval = true) + + val relation = new CobolRelation(sourcePaths, + filesList, testReader, localityParams = localityParams, debugIgnoreFileSize = false)(sqlContext) @@ -85,7 +90,10 @@ class CobolRelationSpec extends SparkCobolTestBase with Serializable { it should "manage exceptions from Reader" in { val exceptionMessage = "exception expected message" val testReader: FixedLenReader = new DummyFixedLenReader(sparkSchema, cobolSchema, testData)(() => throw new Exception(exceptionMessage)) - val relation = new CobolRelation(Seq(copybookFile.getParentFile.getAbsolutePath), + val sourcePaths = Seq(copybookFile.getParentFile.getAbsolutePath) + val filesList = CobolRelation.getListFilesWithOrder(sourcePaths, sqlContext, isRecursiveRetrieval = true) + val relation = new CobolRelation(sourcePaths, + filesList, testReader, localityParams = localityParams, debugIgnoreFileSize = false)(sqlContext) @@ -100,7 +108,10 @@ class CobolRelationSpec extends SparkCobolTestBase with Serializable { val absentField = "absentField" val modifiedSparkSchema = sparkSchema.add(StructField(absentField, StringType, false)) val testReader: FixedLenReader = new DummyFixedLenReader(modifiedSparkSchema, cobolSchema, testData)(() => ()) - val relation = new CobolRelation(Seq(copybookFile.getParentFile.getAbsolutePath), + val sourcePaths = Seq(copybookFile.getParentFile.getAbsolutePath) + val filesList = CobolRelation.getListFilesWithOrder(sourcePaths, sqlContext, isRecursiveRetrieval = true) + val relation = new CobolRelation(sourcePaths, + filesList, testReader, localityParams = localityParams, debugIgnoreFileSize = false)(sqlContext) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala index 4affdfefd..6d4884223 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala @@ -53,7 +53,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -75,7 +75,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -97,7 +97,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val reader = mock(classOf[Reader]) @@ -117,7 +117,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -137,7 +137,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -159,7 +159,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -178,7 +178,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -198,10 +198,10 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest "buildIndexForFullFiles()" should { "generate a spare index placeholder for each file" in { val files = Array( - FileWithOrder("dummy_file1", 0), - FileWithOrder("dummy_file2", 1), - FileWithOrder("dummy_file3", 2), - FileWithOrder("dummy_file4", 3) + FileWithOrder("dummy_file1", 0, isCompressed = false), + FileWithOrder("dummy_file2", 1, isCompressed = false), + FileWithOrder("dummy_file3", 2, isCompressed = false), + FileWithOrder("dummy_file4", 3, isCompressed = false) ) val sparseIndexRDD = IndexBuilder.buildIndexForFullFiles(files, spark.sqlContext) @@ -239,8 +239,8 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest Files.createFile(Paths.get(tempDir, "text1.txt")) Files.createFile(Paths.get(tempDir, "text2.txt")) - val files = Array(FileWithOrder(Paths.get(tempDir, "text1.txt").toString, 0), - FileWithOrder(Paths.get(tempDir, "text2.txt").toString, 1)) + val files = Array(FileWithOrder(Paths.get(tempDir, "text1.txt").toString, 0, isCompressed = false), + FileWithOrder(Paths.get(tempDir, "text2.txt").toString, 1, isCompressed = false)) val filesWIthPreferredLocations = IndexBuilder.toRDDWithLocality(files, spark.sparkContext.hadoopConfiguration, spark.sqlContext) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala index 9ff23e376..56d8ec0c5 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala @@ -16,11 +16,7 @@ package za.co.absa.cobrix.spark.cobol.source.integration -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Paths} - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.scalatest.funsuite.AnyFunSuite import za.co.absa.cobrix.cobol.parser.CopybookParser import za.co.absa.cobrix.cobol.parser.ast.Primitive @@ -31,6 +27,8 @@ import za.co.absa.cobrix.spark.cobol.source.base.SparkTestBase import za.co.absa.cobrix.spark.cobol.source.streaming.FileStreamer import za.co.absa.cobrix.spark.cobol.utils.{FileUtils, SparkUtils} +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ //noinspection NameBooleanParameters @@ -210,7 +208,7 @@ class Test5MultisegmentSpec extends AnyFunSuite with SparkTestBase { val segmentIdField = copybook.getFieldByName("SEGMENT_ID").asInstanceOf[Primitive] val segmentIdRootValue = "C" - val stream = new FileStreamer("../data/test5_data/COMP.DETAILS.SEP30.DATA.dat", FileSystem.get(new Configuration())) + val stream = new FileStreamer("../data/test5_data/COMP.DETAILS.SEP30.DATA.dat", new Configuration()) val recordHeaderParser = RecordHeaderParserFactory.createRecordHeaderParser(Constants.RhRdwLittleEndian, 0, 0, 0, 0) val indexes = IndexGenerator.sparseIndexGenerator(0, stream, 0L, diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala index a087502bf..abdb0c096 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala @@ -17,7 +17,6 @@ package za.co.absa.cobrix.spark.cobol.source.regression import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.scalatest.wordspec.AnyWordSpec import org.slf4j.{Logger, LoggerFactory} import za.co.absa.cobrix.cobol.parser.CopybookParser @@ -62,7 +61,7 @@ class Test12MultiRootSparseIndex extends AnyWordSpec with SparkTestBase with Bin val segmentIdField = copybook.getFieldByName("S").asInstanceOf[Primitive] val segmentIdRootValues = "0,1" - val stream = new FileStreamer(tmpFileName, FileSystem.get(new Configuration())) + val stream = new FileStreamer(tmpFileName, new Configuration()) val recordHeaderParser = RecordHeaderParserFactory.createRecordHeaderParser(Constants.RhRdwFixedLength, 3, 0, 0, 0) val indexes = IndexGenerator.sparseIndexGenerator(0, stream, 0L, @@ -78,7 +77,7 @@ class Test12MultiRootSparseIndex extends AnyWordSpec with SparkTestBase with Bin val segmentIdField = copybook.getFieldByName("S").asInstanceOf[Primitive] val segmentIdRootValues = "0,1" - val stream = new FileStreamer(tmpFileName, FileSystem.get(new Configuration())) + val stream = new FileStreamer(tmpFileName, new Configuration()) val recordHeaderParser = RecordHeaderParserFactory.createRecordHeaderParser(Constants.RhRdwFixedLength, 3, 0, 0, 0) val indexes = IndexGenerator.sparseIndexGenerator(0, stream, 0L, diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala index 0e476db64..c05281a3d 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala @@ -16,14 +16,14 @@ package za.co.absa.cobrix.spark.cobol.source.streaming -import java.io.{File, FileNotFoundException} import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.scalatest.BeforeAndAfter import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import java.io.{File, FileNotFoundException} + class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { private val TEMP_DIR = new File("tmp-"+System.currentTimeMillis()) @@ -44,14 +44,14 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { it should "throw if file does not exist" in { assertThrows[FileNotFoundException] { - val stream = new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, FileSystem.get(new Configuration())) + val stream = new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, new Configuration()) stream.size } } it should "not throw if the stream is never used, even if the file does not exist" in { noException should be thrownBy { - new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, FileSystem.get(new Configuration())) + new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, new Configuration()) } } it should "return array of same length than expected number of bytes if enough data" in { @@ -59,7 +59,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { val iterations = 10 val tmpFile = createTempFile(iterations * batchLength) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) for (i <- 0 until iterations) { assert(streamer.next(batchLength).length == batchLength) } @@ -71,7 +71,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { val extraBytes = 1 val tmpFile = createTempFile(iterations * batchLength + extraBytes) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) streamer.next(iterations * batchLength ) // consumes all but the extra bytes assert(streamer.next(batchLength).length == extraBytes) } @@ -79,7 +79,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { it should "return empty array if end of file was reached" in { val totalBytes = 10 val tmpFile = createTempFile(totalBytes) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) streamer.next(totalBytes) // consumes all bytes assert(streamer.next(totalBytes).length == 0) } @@ -87,7 +87,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { it should "return empty array if file is empty" in { val totalBytes = 0 val tmpFile = createTempFile(totalBytes) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) assert(streamer.next(totalBytes).length == totalBytes) } From 14ead4a2184fa26701648b46d370f8b6527db7a7 Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Thu, 18 Dec 2025 08:59:26 +0100 Subject: [PATCH 02/10] #809 Add test suites for reading compressed files. --- data/test40_copybook.cob | 260 +++++ data/test40_data/example.dat | Bin 0 -> 149300 bytes data/test40_data/example.dat.bz2 | Bin 0 -> 58013 bytes data/test40_data/example.dat.gz | Bin 0 -> 48581 bytes data/test40_expected/test40.txt | 20 + data/test40_expected/test40_layout.txt | 393 +++++++ data/test40_expected/test40_schema.json | 985 ++++++++++++++++++ .../streaming/BufferedFSDataInputStream.scala | 32 +- .../cobol/source/streaming/FileStreamer.scala | 20 +- .../Test40CompressesFilesSpec.scala | 118 +++ 10 files changed, 1808 insertions(+), 20 deletions(-) create mode 100644 data/test40_copybook.cob create mode 100644 data/test40_data/example.dat create mode 100644 data/test40_data/example.dat.bz2 create mode 100644 data/test40_data/example.dat.gz create mode 100644 data/test40_expected/test40.txt create mode 100644 data/test40_expected/test40_layout.txt create mode 100644 data/test40_expected/test40_schema.json create mode 100644 spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala diff --git a/data/test40_copybook.cob b/data/test40_copybook.cob new file mode 100644 index 000000000..669fb2f45 --- /dev/null +++ b/data/test40_copybook.cob @@ -0,0 +1,260 @@ + **************************************************************************** + * * + * Copyright 2018 ABSA Group Limited * + * * + * Licensed under the Apache License, Version 2.0 (the "License"); * + * you may not use this file except in compliance with the License. * + * You may obtain a copy of the License at * + * * + * http://www.apache.org/licenses/LICENSE-2.0 * + * * + * Unless required by applicable law or agreed to in writing, software * + * distributed under the License is distributed on an "AS IS" BASIS, * + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * + * See the License for the specific language governing permissions and * + * limitations under the License. * + * * + **************************************************************************** + + 01 RECORD. + +*********************************************************************** +******* EDGE CASES +*********************************************************************** + 10 ID PIC 9(7) BINARY. + +********** String + 10 STRING-VAL PIC X(10). + +********** Integral unsigned numbers formatted as strings + 10 NUM-STR-INT01 PIC 9(1). + 10 NUM-STR-INT02 PIC 9(2). + 10 NUM-STR-INT03 PIC 9(3). + 10 NUM-STR-INT04 PIC 9(4). + 10 NUM-STR-INT05 PIC 9(5). + 10 NUM-STR-INT06 PIC 9(8). + 10 NUM-STR-INT07 PIC 9(9). + 10 NUM-STR-INT08 PIC 9(10). + 10 NUM-STR-INT09 PIC 9(11). + 10 NUM-STR-INT10 PIC 9(17). + 10 NUM-STR-INT11 PIC 9(18). + 10 NUM-STR-INT12 PIC 9(19). + 10 NUM-STR-INT13 PIC 9(20). + 10 NUM-STR-INT14 PIC 9(37). + +********** Integral signed numbers formatted as strings + 10 NUM-STR-SINT02 PIC S9(2). + 10 NUM-STR-SINT03 PIC S9(3). + 10 NUM-STR-SINT04 PIC S9(4). + 10 NUM-STR-SINT05 PIC S9(5). + 10 NUM-STR-SINT06 PIC S9(8). + 10 NUM-STR-SINT07 PIC S9(9). + 10 NUM-STR-SINT08 PIC S9(10). + 10 NUM-STR-SINT09 PIC S9(11). + 10 NUM-STR-SINT10 PIC S9(17). + 10 NUM-STR-SINT11 PIC S9(18). + 10 NUM-STR-SINT12 PIC S9(19). + 10 NUM-STR-SINT13 PIC S9(20). + 10 NUM-STR-SINT14 PIC S9(37). + +********** Decimal numbers formatted as strings + 10 NUM-STR-DEC01 PIC 99V9. + 10 NUM-STR-DEC02 PIC 99V99. + 10 NUM-STR-DEC03 PIC 9(3)V99. + 10 NUM-STR-DEC04 PIC 9(4)V9(4). + 10 NUM-STR-DEC05 PIC 9(5)V9(4). + 10 NUM-STR-DEC06 PIC 9(5)V9(5). + 10 NUM-STR-DEC07 PIC 9(15)V99. + 10 NUM-STR-DEC08 PIC 9(16)V99. + 10 NUM-STR-DEC09 PIC 9(17)V99. + 10 NUM-STR-DEC10 PIC 9(18)V9(10). + 10 NUM-STR-SDEC01 PIC S99V9. + 10 NUM-STR-SDEC02 PIC S99V99. + 10 NUM-STR-SDEC03 PIC S9(3)V99. + 10 NUM-STR-SDEC04 PIC S9(4)V9(4). + 10 NUM-STR-SDEC05 PIC S9(5)V9(4). + 10 NUM-STR-SDEC06 PIC S9(5)V9(5). + 10 NUM-STR-SDEC07 PIC S9(15)V99. + 10 NUM-STR-SDEC08 PIC S9(16)V99. + 10 NUM-STR-SDEC09 PIC S9(17)V99. + 10 NUM-STR-SDEC10 PIC S9(18)V9(10). +********** These types are currently not supported, added for the future + 10 NUM-STR-EDEC03 PIC S9(3).99. + 10 NUM-STR-EDEC04 PIC S9(4).9(4). + 10 NUM-STR-EDEC05 PIC S9(5).9(4). + 10 NUM-STR-EDEC06 PIC S9(5).9(5). + +********** Binary formatted integral numbers + 10 NUM-BIN-INT01 PIC 9(1) COMP. + 10 NUM-BIN-INT02 PIC 9(2) COMP. + 10 NUM-BIN-INT03 PIC 9(3) COMP-0. + 10 NUM-BIN-INT04 PIC 9(4) COMP-4. + 10 NUM-BIN-INT05 PIC 9(5) COMP-5. + 10 NUM-BIN-INT06 PIC 9(8) BINARY. + 10 NUM-BIN-INT07 PIC 9(9) BINARY. + 10 NUM-BIN-INT08 PIC 9(10) BINARY. + 10 NUM-BIN-INT09 PIC 9(11) BINARY. + 10 NUM-BIN-INT10 PIC 9(17) BINARY. + 10 NUM-BIN-INT11 PIC 9(18) BINARY. + 10 NUM-BIN-INT12 PIC 9(19) BINARY. + 10 NUM-BIN-INT13 PIC 9(20) BINARY. + 10 NUM-BIN-INT14 PIC 9(37) BINARY. + 10 NUM-SBIN-SINT01 PIC S9(1) COMP. + 10 NUM-SBIN-SINT02 PIC S9(2) COMP. + 10 NUM-SBIN-SINT03 PIC S9(3) COMP. + 10 NUM-SBIN-SINT04 PIC S9(4) COMP. + 10 NUM-SBIN-SINT05 PIC S9(5) COMP. + 10 NUM-SBIN-SINT06 PIC S9(8) BINARY. + 10 NUM-SBIN-SINT07 PIC S9(9) BINARY. + 10 NUM-SBIN-SINT08 PIC S9(10) BINARY. + 10 NUM-SBIN-SINT09 PIC S9(11) BINARY. + 10 NUM-SBIN-SINT10 PIC S9(17) BINARY. + 10 NUM-SBIN-SINT11 PIC S9(18) BINARY. + 10 NUM-SBIN-SINT12 PIC S9(19) BINARY. + 10 NUM-SBIN-SINT13 PIC S9(20) BINARY. + 10 NUM-SBIN-SINT14 PIC S9(37) BINARY. + +********** Binary formatted decimal numbers + 10 NUM-BIN-DEC01 PIC 99V9 COMP. + 10 NUM-BIN-DEC02 PIC 99V99 COMP. + 10 NUM-BIN-DEC03 PIC 9(3)V99 COMP. + 10 NUM-BIN-DEC04 PIC 9(4)V9(4) COMP. + 10 NUM-BIN-DEC05 PIC 9(5)V9(4) COMP. + 10 NUM-BIN-DEC06 PIC 9(5)V9(5) COMP. + 10 NUM-BIN-DEC07 PIC 9(15)V99 COMP. + 10 NUM-BIN-DEC08 PIC 9(16)V99 COMP. + 10 NUM-BIN-DEC09 PIC 9(17)V99 COMP. + 10 NUM-BIN-DEC10 PIC 9(18)V9(10) COMP. + 10 NUM-SBIN-DEC01 PIC S99V9 COMP. + 10 NUM-SBIN-DEC02 PIC S99V99 COMP. + 10 NUM-SBIN-DEC03 PIC S9(3)V99 COMP. + 10 NUM-SBIN-DEC04 PIC S9(4)V9(4) COMP. + 10 NUM-SBIN-DEC05 PIC S9(5)V9(4) COMP. + 10 NUM-SBIN-DEC06 PIC S9(5)V9(5) COMP. + 10 NUM-SBIN-DEC07 PIC S9(15)V99 COMP. + 10 NUM-SBIN-DEC08 PIC S9(16)V99 COMP. + 10 NUM-SBIN-DEC09 PIC S9(17)V99 COMP. + 10 NUM-SBIN-DEC10 PIC S9(18)V9(10) COMP. + +********** BCD formatted integral numbers + 10 NUM-BCD-INT01 PIC 9(1) COMP-3. + 10 NUM-BCD-INT02 PIC 9(2) COMP-3. + 10 NUM-BCD-INT03 PIC 9(3) COMP-3. + 10 NUM-BCD-INT04 PIC 9(4) COMP-3. + 10 NUM-BCD-INT05 PIC 9(5) COMP-3. + 10 NUM-BCD-INT06 PIC 9(8) COMP-3. + 10 NUM-BCD-INT07 PIC 9(9) COMP-3. + 10 NUM-BCD-INT08 PIC 9(10) COMP-3. + 10 NUM-BCD-INT09 PIC 9(11) COMP-3. + 10 NUM-BCD-INT10 PIC 9(17) COMP-3. + 10 NUM-BCD-INT11 PIC 9(18) COMP-3. + 10 NUM-BCD-INT12 PIC 9(19) COMP-3. + 10 NUM-BCD-INT13 PIC 9(20) COMP-3. + 10 NUM-BCD-INT14 PIC 9(37) COMP-3. + + 10 NUM-BCD-SINT01 PIC S9(1) COMP-3. + 10 NUM-BCD-SINT02 PIC S9(2) COMP-3. + 10 NUM-BCD-SINT03 PIC S9(3) COMP-3. + 10 NUM-BCD-SINT04 PIC S9(4) COMP-3. + 10 NUM-BCD-SINT05 PIC S9(5) COMP-3. + 10 NUM-BCD-SINT06 PIC S9(8) COMP-3. + 10 NUM-BCD-SINT07 PIC S9(9) COMP-3. + 10 NUM-BCD-SINT08 PIC S9(10) COMP-3. + 10 NUM-BCD-SINT09 PIC S9(11) COMP-3. + 10 NUM-BCD-SINT10 PIC S9(17) COMP-3. + 10 NUM-BCD-SINT11 PIC S9(18) COMP-3. + 10 NUM-BCD-SINT12 PIC S9(19) COMP-3. + 10 NUM-BCD-SINT13 PIC S9(20) COMP-3. + 10 NUM-BCD-SINT14 PIC S9(37) COMP-3. + +********** BCD formatted decimal numbers + 10 NUM-BCD-DEC01 PIC 99V9 COMP-3. + 10 NUM-BCD-DEC02 PIC 99V99 COMP-3. + 10 NUM-BCD-DEC03 PIC 9(3)V99 COMP-3. + 10 NUM-BCD-DEC04 PIC 9(4)V9(4) COMP-3. + 10 NUM-BCD-DEC05 PIC 9(5)V9(4) COMP-3. + 10 NUM-BCD-DEC06 PIC 9(5)V9(5) COMP-3. + 10 NUM-BCD-DEC07 PIC 9(15)V99 COMP-3. + 10 NUM-BCD-DEC08 PIC 9(16)V99 COMP-3. + 10 NUM-BCD-DEC09 PIC 9(17)V99 COMP-3. + 10 NUM-BCD-DEC10 PIC 9(18)V9(10) COMP-3. + 10 NUM-BCD-SDEC01 PIC S99V9 COMP-3. + 10 NUM-BCD-SDEC02 PIC S99V99 COMP-3. + 10 NUM-BCD-SDEC03 PIC S9(3)V99 COMP-3. + 10 NUM-BCD-SDEC04 PIC S9(4)V9(4) COMP-3. + 10 NUM-BCD-SDEC05 PIC S9(5)V9(4) COMP-3. + 10 NUM-BCD-SDEC06 PIC S9(5)V9(5) COMP-3. + 10 NUM-BCD-SDEC07 PIC S9(15)V99 COMP-3. + 10 NUM-BCD-SDEC08 PIC S9(16)V99 COMP-3. + 10 NUM-BCD-SDEC09 PIC S9(17)V99 COMP-3. + 10 NUM-BCD-SDEC10 PIC S9(18)V9(10) COMP-3. + +********** sign trailing numbers + 10 NUM-SL-STR-INT01 PIC S9(9) SIGN IS + LEADING SEPARATE. + 10 NUM-SL-STR-DEC01 PIC 99V99 SIGN IS + LEADING SEPARATE CHARACTER. + 10 NUM-ST-STR-INT01 PIC S9(9) SIGN IS + TRAILING SEPARATE. + 10 NUM-ST-STR-DEC01 PIC 99V99 SIGN + TRAILING SEPARATE. + 10 NUM-SLI-STR-DEC01 PIC SV9(7) SIGN LEADING. + 10 NUM-STI-STR-DEC01 PIC SV9(7) SIGN TRAILING. + 10 NUM-SLI-DEBUG PIC X(7). + 10 NUM-STI-DEBUG PIC X(7). + +*********************************************************************** +******* FLOATING POINT TYPES +*********************************************************************** + + 10 FLOAT-01 COMP-1. + 10 DOUBLE-01 COMP-2. + +*********************************************************************** +******* COMMON TYPES +*********************************************************************** + 10 COMMON-8-BIN PIC 9(8) BINARY. + 10 COMMON-S3-BIN PIC S9(3) BINARY. + 10 COMMON-S94COMP PIC S9(04) COMP. + 10 COMMON-S8-BIN PIC S9(8) BINARY. + 10 COMMON-DDC97-BIN PIC S9V9(7) BINARY. + 10 COMMON-97COMP3 PIC 9(07) COMP-3. + 10 COMMON-915COMP3 PIC 9(15) COMP-3. + 10 COMMON-S95COMP3 PIC S9(5) COMP-3. + 10 COMMON-S999DCCOMP3 PIC S9(09)V99 COMP-3. + 10 COMMON-S913COMP3 PIC S9(13) COMP-3. + 10 COMMON-S913DCCOMP3 PIC S9(13)V99 COMP-3. + 10 COMMON-S911DCC2 PIC S9(11)V99 COMP-3. + 10 COMMON-S910DCC3 PIC S9(10)V999 COMP-3. + 10 COMMON-S03DDC PIC SV9(5) COMP-3. + 10 COMMON-U03DDC PIC V9(5) COMP-3. + + 10 COMMON-UPC5DDC PIC PPP9(5) COMP-3. + 10 COMMON-SPC5DDC PIC SPP99999 COMP-3. + 10 COMMON-UPI5DDC PIC 9(5)PPP COMP-3. + 10 COMMON-SPI5DDC PIC S99999PPP COMP-3. + + 10 COMMON-UPC5DISP PIC SPPP9(5). + 10 COMMON-UPI5DISP PIC S9(5)PPP. + + 10 COMMON-UPC1BIN PIC SPPP9 COMP. + 10 COMMON-UPI1BIN PIC S9PPPV COMP. + 10 COMMON-UPC3BIN PIC SPPP9(3) COMP. + 10 COMMON-UPI3BIN PIC S9(3)PPP COMP. + 10 COMMON-UPC5BIN PIC SPPP9(5) COMP. + 10 COMMON-UPI5BIN PIC S9(5)PPP COMP. + 10 COMMON-UPC10BIN PIC SPPP9(10) COMP. + 10 COMMON-UPI10BIN PIC S9(10)PPP COMP. + +*********************************************************************** +******* EXOTIC AND COMPILER SPECIFIC +*********************************************************************** + 10 EX-NUM-INT01 PIC +9(8). + 10 EX-NUM-INT02 PIC 9(8)+. + 10 EX-NUM-INT03 PIC -9(8). + 10 EX-NUM-INT04 PIC Z(8)-. + 10 EX-NUM-DEC01 PIC +9(6)V99. + 10 EX-NUM-DEC02 PIC Z(6)VZZ-. + 10 EX-NUM-DEC03 PIC 9(6).99-. + + diff --git a/data/test40_data/example.dat b/data/test40_data/example.dat new file mode 100644 index 0000000000000000000000000000000000000000..ad268428357b99cca2005bab8342d1576c3c2271 GIT binary patch literal 149300 zcmcFs2S60Z*WbJ2C<*}uED^+B5(%gvY9NRW%^{$IC4q2u{VTx^7J{g#7ziph5(tT! zo|s~)DT)n^CPA8FOi^QEOT1oGxbMB0oxMBm4kUch+kJ0-^XBbu-t3I)?(8hbaiVLv zJ9BqrbMQy4R;U$P_@Pih3U)@diblcs~ow56`$i9Dyj z&u8>B!Jns}_;@M^!}ZF)<)*^dYQ9^){M`K$YQKQW3a! z*y{REH*#T*f7vTdU$JBSEWghk@^r+{?dM_FFD{JfTD$O?uR}VHXy|&KdbID(Ls$Cm z^t&A?@eEFpNIiq&BPCKvVNh`VutAWP6!y+Sb!r;f&zm_fjesjx_E}lLp!rz>wiz6N zHQ3ExoRQJ7qg=XUNnC`)BRE+i^$3oSK*K;Zfi(EeB-W{{Ose1~pF#681#B}oXzY7J zlcbxc>#{j}Nmi5Oz(+H*&_||R=*U!K!vRUjt7wwiR9UA^{S$Y&?9apfYeM%wkLM?z z14M!j6^`U94Gj=MR6bfPXx2$cfmJy9Q6&NoO2(@y=m`?hK+!<#33L;B0qG4w9Vkz= zO8zhiHI{)j^d|9%-9(E!@DVG?gT||*7T=XuAX-c zE*c%(7BL*PF)}FF@$}4-!C*@f3u0MXo*juvvsbgm$H@kG*ub$MwKy{Lz_yjQMeK%y zwGOt0j5dZ0P8b?`8Jsz0O<&HraIq4vt_L{Lpe0TpxQ~D0*y<~A&WWn_o;we+{U`$$ zEb8#ttBPNWRBf(x3Z4J9WG%PoS5=1&+j?Ad8u8ik%U51cS|$DSTD!WpxtWJo*G6^j zKXc=M(g+T3IQNYAw{c^NS-8)Gs?7o*`?G+9fE z5H)=fE0ZVq$z#wKcL7^hEP#S<_a;dAfAC}3&LBetb<8oa`$$6=lTQz}(j7|`_wolav^Y2h;b zjM{h{EKi{INxn!qE6M=9_7sD(I$}yHz&k!|&f_^-So4T&DT7XP)TW>vifDv(#FiCc z(tZqE88{ZC7DtBG^l0| zQY3|A#%B!+N!UZQ6=|qWfehG>>6=r4*(+vG%9uto0NNBMV6#1$-KG)kdT;K#Y4(mp z&u~fcuy9FX%&?FoG+c`&L|d_lSVIQ&ohSIA8R$Dkz-GHMJF{>1CP_C>SJZym(yS)Q zfsbZrp^vq(LPsN-bt?8a9bqVX#yS<<`Qyx}zvEx}?WMIjcz$9tqQngl1zp;OYG%4j4BkSakjm^HylWl$QRkP=C}RDBZ_!rbVpn=zf4+GT!5H=8{nY>#~8FUGPDx^IvlKZYJj@jnB<<;Y5 zTzJmKZQ(z^kta?bzG9zzXZkux^qttHixS4pD_rmS`=!#ArU#~)hM0f!CR5@Y^>bfL z{!_H%HMs54@ZgcfLqTdUZkrion&Vm4)o+uFQC0F{pnrIz>G+QueNca=D#rY zzVn+T-8|imOE>jRX_6fHXoeR0NR|toh-lU+c{T0#6s@yPl|T0E(`AatP7l`K8;$2D zo&zXxgD8-efWSio&^YnYO64PDodgsBKk}oBKOU40TF?`mM4yU2#hySnp%;+eAk=~K zG#cd(gHU4`SVM1;R%gFW?x=H|PN&sr74S3QhPnYpQv#qgGQg-YYl4+gqt&W42BU=) z511V=69pWJ`W8wlboR^?N+hE&D7EnK4GimQm99pot$@-O(F5qj4FZH}9aD_ef~H|s z3U)BF6(b}#U?xUIt=WN44?AW$8;X@s5X+MC%n6eg(Geg^{(0m^cUR_W z&MmmQzmLcv9Y-{78KSxtwH>_MKdH`?=A1Ie(>cCij%$i@e9m~!Ij#xYN>Ou*Lv?2g z*(ZkNQgF=VaYe*tCqUcMh#_LmW>D_yD$h;`**q#J)j4^NhjV=X9M@zt^gvf7q{sx~ zHJSOQAo?LT#CNLMw;2?C2R2E%dAg$bO|Ah=k^>*j&_W;na-kCu%{nEorNab8>#S4d zFGOUP86w@Dnt$mqo}YLQpu`OVQ_2UY0HS9Ztq2~^r;<;C0IMYOqe?O!l#E0*&=Z`+ zTg6+kC(upk1*A6!b)YJ{))oJ8w#zQ_qyl%RwS z+VH$in^gljJ}U+lp#VB%);yxHHB#u$*qZ5$jVYoLu|t=v0F(A(*vh~$1}%YWI%QfQF6z%anmQsJ~vg@EAz1xMQKk zoZ-5(&&F;Nw=Y$Lv-^+e%FP(IQqs0Kd$8o(Q^j4E`@6eH$BcaYg}+8j?04<+NaOs^ z3;!Fh4tYA_*I~cQ8rRoWbO@e3xNoZpXSsMT_H^M=7E7~SxT*5E#nP2dDX5L{L^ZyE z?AHwzlk73BlHk5H1E9rb%%zmsEnYQg$gCB!m&xapu61!=?C!!PEtalDk1=SHM0~pw z>pWJ5WE0;s1AUt@mttmT_Fdd0>E`L~>Nz(orAcz&!%V?0E%cETD|9rfv2IOLvE?*L zZH`)}qPJB45OuHfkZ%TDnuzBoHX};hpu#N5AlF((E1VAoPl->004u8>9+ZqJ_;krd z{Ehe<>x*Ts>Y>YUQeYROx2hodPh5A z%L*__a_E5sh(*N)c-X)(1}%;Zt^L0a2dnydU@oJLAro3d2GOLRQPW@GMBF1>q4WgD z4f~M0GNs=2hdDR#q?;za-yMWx@smGI;zH76UVBn>TivHQ-{?S72 zoJunB&lmm(7Wpk4I`Hf}y@$Q{W!nnFqC&VDSD8}pg_}&LCYrjwBhe3fxg%hvx9&Op z2I4`%iQt2#vs(_OoM?!sS5Fmv`|&?XrrX(b+s#+U?J^DacyO?Dc9)8VYdgz-44HE2 z)beOekV|2zL!OTKIi1}8_CZ7TIynX!Oqd$LeXFChC3yQWbvV_4#5u!RhORz_lxJ&4T;62L+0LVITy&K$EX z%U?9XCsP!i>a!B|^EQ>B3DNds66<_chJxq^YXwv#_Qn zIy{aIMRHKS-VP}u2P{jik0JBb_<-h8jFsKMu^_cLGOY}Ukv}koN=6$)2GOCd z0jHOSHV10T3vhihPF&&FpA&U{#VN-1iN_2Re}L0Z^ueJ)aG#y?I&IRVh`0dBOFJ_D zXd@cZbNiQH-sNVy`_K9~Y}ho(ja9w>xYI9Um2^?s3olE?*38iMeDl3Uhc1|mrZ4M* zuU#-T{5j|TA$?_y%fZ1gm^Vy{`2p^;o9>2jZC`IN{r+G{>SjM}!K5SAZTo__rEVQ|5-lnV)s87-KPh2F8fHhW z=p8+UW&kuVNWhkY1F%N0|ImyPOA~VXw+qO030&&o(ye5vn?D*(Kog=xWfJRTRwh;O zLo?7#pnxp}2aSFEH%YpAx`mrJLdBaR2R@pig+2o1LYE<$bxMALc4>;%S*ObLez;h6 z@8`kY_RepE=O>;6C~<=dvw%Xh{Vk)#g2qRRPlE7Y>gGp&RPo1ylKya>T&Y)xE3hY! z&D;w}ZxHH0c^WnHhe4>Z46LCyxt++_w!5Zr+88njD{T$9W>?eZK+RAI&-{Ccl007JxGe+4KH_wr)13~& zImi7}q5BbpYuEk<+Hq@sed^kglHZ+b4=%sH_OYW}@XU&By1Mt)iLPDl{my}Phos`t z%-vqICLFw9o9Pm^^s)x#ssGw=KsD23O8=?;c75*cu)ik3U~UQyavKCff2H=>yQYoZ z%lt;n>~_EZ#1|jkP&2~xQ%O*oTc3n=4Sqc@O?fse=6=0={erv2pGzus^DUxw>G9p7Z zP2B+LIT=HPP+gxw_EST>(+Jpm*~B5943c{a7$?}trRzoyT`)>AA}1(9G9e^iGA=b_ zQ!*MxU^z&G??B=rnK?=o{CF@(?jc~DU}yFn*d*!Z=}vPRwW+j8a^RyGTIi!hE_5`a zS*PSxeD@{dqIIg=b>Qi;oA;6C~<=*6!4J#>I?i) ztCeac+#1oT$VPU04NZWgIjEqSN=-Dqg{p&RnR;8m>;PUOA|@~g#O#1U3cHxWsDnH8 z@HkRw)ESL%8|J)PSqqkxXkEn@AqnJzIg}WfF^U(=Dm}gppcfqI1d7#-OX*db3OkrW z-H2ix7!f-pzzQ&FKZdOh9M7P|k!fW(Sk*D=GTIn0PgX;NDHEClHQjkQl>=>uQLb(*zq)S7>kN!;Vt#QQEJrWT7n z|Mk(N1yja#63_7HaQw@ncV?AW{5JTJ@mVmaUuig@e$!;yxT?W_#j(Hp6upFl`lxKv ziy#d9z3+1T!p1Ey-f`s9A^?wH*$zWeq4yWjcZM*XX*o8i%O zrxXtj>UK`~dA38Ij`+0+yL2w%-7mJqzV+Jw9MW+_LwmaS;A4s7(mtM+vO7`|FfdKh zA!y*tNJ)pD%Rz>Xgmlm4eF9LOkwx|^$E9Zxu;c8_>BMFRKuhBUtd}RVTRLLs?!nHY z&I`7eMo9bzrb;>l4xAZ*h9JX6LK=Mg6YES?CR6Z3Y>4j(0@lle*_nNpG)cO7x{};k z!D&sB10T)MLLaGep`%fa4FV)3ucAq6Q*E6(rT1J@_E*8tN4{M36rP`W4iE`CR5*^W zG&Dd2QTb?1;Bo6DpkNhFepHFTgOc%zzI+re3df#6H=!4h-XPS0^3>qVM|?e)w1dzr z8CXMaYHObTV#Ggc_`w_&xEv=ttyXK6h#~;PE7Oz~DooljYV|gN*#Z1(KulnYgxP@^ zvV$q$aC*2)01rvwnq3Ju*XxYNDkyysJyh~;YM497|QW~nH`NN;^D9?DK9u-yad29hOG=7W6KHv4Z44QlA2gUU2n&rD zYQ|d5jhiL8(x;LWZ5$%Kv!ctB;e{RH^b^M?e|HCD$DZ%37$%zGeIrA)skg`XD_ozLQZ%%8Vkv6V(@@TQdLMgLbFG=jTS-xbTVa0D&_FayMw1d!B$ZhEvoe{2A7Vp%PY|$# z5@u)i?cXHn=IL%roH{9`Npj$$8CvKgSuXT1qFJZpb+i{!w8lDBeyq#bW8(FH>t}TM z2G37C2T62Cku14cacerYPffzgGZ9^@(gS+?*!lPhL?hPDF7-n1aHK% zT%79xZ(7;{me$k+Zof1x1v5tMVJd@OYt$$d3Kh&&hXn(4>RPp~4lJwCx{@za!HP0K zp%Ne!+%#!)FdG0C9cW~th1)MSkm7$n+}o{aMjLnmwW8m2*R~!`h`Gl zL#VpTtE1*fyLHL(dE#{yH#n-j?o8$6kW$Hl$RUM6k_CejJckta9zirQQM;ue&~0k^ z$`s5NOViWm&E`JpJewp3KANG0K0M??4<(v)N?t{~G)1eeQ{`Tx zx0c=BIO(Gu@bVFypLhS}Yj6Gz+jAKz>vS#DkLlfHxu~q9vjw z*c0d`^a9cwggQ{38ko|^G$!pJG)o57(3?EW4@!;r2QH#DaJ7va?sPCif|q86=0lbm zUN>dAaAO$mHsD1#Ee3fMfNRpm2wcY6Kw5j40-nXfGg27VVVMAh2CnnZ7>y^uvK*~J zCvJ|jqJp9(`N&?aH-kJQEKh1CP&{RW=uX>0hLd(MZDWdMU_|WjAglnB7U2>g7AG6v zVFSk)v^X-ff&O(kSk=*ij5dZ0LQdl?gV>ujbve0GzuM<#xHj1%{adl&N!c8n^&*Z} zswaZ%*ne4eplHUTr#`E5x$5!b@MVv#j=jWfePmpgYq?LJboNWtx~Zpr$aGnA7B;K1gh`E{Xh=S?*QhTvc3jgK&{|mYwNTp{skP^A3IgrM6tN)M z5nEP(N&7KuW#AZt7DtBG)wZS&84gx;Y!4Z23>k!+Hi8UdZ`L$boS5_BesuNZxcHYv z!!N1aWAZcbq+4CT=@1B~AO0F8<;KU>I?qX|7ysgu6sbLOo%7pw_T{Xi7xbd8&uU!n zH)!7y<hZE(36cPQXM)~07Vn;J}aOoQ}Flj*~`4JUg)qCT{A3tp+`h*wPjaeq*! zZ|<8u|G2;L<<}(jJ#Ozm+Ga!-)3xM~IqT+TyER-n&{ppxYBRlI>E80kT?`#QiuwK7 zb*tX^%OOuk{G8vdS@xo+?YNchU;XZojw2ez#K-cMH)y`dRgdi!B=+oW1;@c&g?t6Nz%>J?bvJ8n6xIzfe$kU zyR^_ps$A%3M6*uGE9t01(HiU2snBFB z!CEPTdsYyhr)uEp->5RGbUKAr35$prE8t~Mu&hGs8omfgAk}KYN2Q>tE|^v7*lY#= z2oZ`G8=Ghhwnnf-fvuS>MLZnZ5nEP(NsEdL5F=p&JZ#`tkXjs>R)&MMP8~3p(Z+z; zFA5qZna~`lY4A?sGm@=6=W(K$ZKZSTzZfFD`52sYV!tQ5yb03x(Kw$CqLJLkkDZ(o z?!MFc_Isbf>6v+YOvq@{dz`JN;Ga=D97+%{g-0tQ0rr(8VC9O5#up2|{&b7TK@t(T#$AXZCO^ zV9+=-wviMZ33lT~`3&ydzmI(4_^c8)k8P=L&Jo*YC8I}>Qza9K??B=rlR2Uw`p#$2 zI5W196two8*(B-a=?)w6DvA84l8gr><5dm0Qr{}siamjB z=3YR0gHQ*`Q=x`?>XbMgQ=mK9L8yrg@}M_)o!PaSpZx+?@J3j^P@{nPr2{mH*{g-#5q;mhDSq!R|Vc1DGhLT`{V1gr*^w^SmmP=)so*O>3{8{b^IB^t*e%8+BbqhYb6l;g8)vM~qAPCcIsE&3kLEoO8s} zA-}Gh;v$|J{jH|MtFN4KM8_eGE2};_R`&JCwl`DC}sx?QrW}c zz6;z{SHnzf9n9*1n>&U&gRufEYtZ^4U!;l^y}%Sbm@$fI&e)QJk+M8}dUSI+FeSCn z+ky0zjXlAWM!3~)O%c5lJ3J~Yz@$aR1&AHe2Cy(@tn5JogBC}Idi~eoUN4abWv<^0gMulj&EcerG8mTBYs z&Pk(tK7IH80I&L&-;w{FV2nBUc%MPf)ooPi*5$1pddLw^hy46bef!9i)9t_U*q8ae zBRURgTpKo0`A*p1qg2W3hZMM$=6bqrOUacMxNe&^A~ILHQIdsPSShNjJjq@<$~lFA z+Y*B3WHD$eHAE~uNC210XLlUxG-Kwd^AG~edNo9PDC}<^GQm6ktV54g>|a@1Lfdj z-0po}Tucka^Apbjl(<2KS>QqDllt?~n!pFcItgfgt~8JQsFIHdCF9ja=n1Zp)sofN z6Ub)n1*A6!b)Y=)x}0fD+CgX*f2H33c=o=1Ipn4~EU^fG;Rlz*N?7(5cY0VRfR_{n zKm+d>(JWX!#ZrY=RSgCkKxK>AgDl*`taS9GQo%n^3WWjI^MzGklp4Hiqdcxumx03* z==3P>`3yS(L6NiOB!ght5wl7ipKc5oj@ck8JCMrk9_sCgA|4Li5tqy_lNOZ}Am+;k zuoop;BXf{i9GO;zgS8IEQ$`y@2EnJT0jt0mX}nO=oP<}SJ-7|h9vn9~fjhmSyGu;~ zKGNZ0Cl6}}LgG1M_`BQ_AFlYR&r^ShDlfg<@rC8Lxc1>&oo*?%_u)QIo;@&d&U@>) zdY^_%#q*tJN{`+N_;b?t|B+WmJRR|Kdg@-uKjOFYOvbH`I;`Zw6b=wYdIyLG#(DdA ziUy8YS?cZMvLFt%ohhh}@g#eZPeuU&JO)%Xn*v5cwEGJ$$zSL=ZrU>44swi>vr5P-uKGBI&Ma3 zmu{CTlCmTfVV>jidKR0cPoEs^GF&!i@0(A4pEk{u1N{KM}g^EiT(3Pe=SZ9R9c=t*CHfUUIwb4(T|eadS+pXxrRH2bb;p zCk+?$igc00Ee?VGsKup=L&Ds$P}}2)>f!>j&n?_;sw!E%|ThzD_z&m2(@{bVvIJ=yG=6two8+$8Dd z>F((pH7TG;a^RyGTIj=HE_4~9S*PR|Xb+=kg>|Ytch;q{dm&k09h}pE=O>;6C~<=* z(D3ILL0}^+qs4;3OS1s0aPp%{1Rj)(S5;)0fFt4~*b~TR?ggYbnCrj~2(V0mFf`C% z#12ADWD27kdXvwIoa|j2i||RQ8vnqPNsSISCE3A>j7*XoSmqVx7z&U9W-Wp#0kv=o z*$7JrGzMS|!bT<*K4FDlQc3~e;IK7=F$@NCSbG;HsKeV(u)>x4xEel?q1K#2>oUGb zIV*}el+9^DGn_Q5^kByt1FEAoh(hoQGb$NH8&9yxiw#BOfbNJ(=9fwPF>GbvSddyA znO25_RUKn3qm3bhQ-MZK2ImBA4%9TKVRmkJQLIOMj$8kks9~4HTUiNL?HpIWG!$N# z=3KsC>083Byy7)EvEU`A1K;nDn)*r}_e9z)dHy2rEKx;D-;_heAFECF*7?7FR3Ja)J1%Ax&d#Dz=32ggW477ZTj zhlXph9HhZ_9&wSx91RfsxHG8RS^;wr?99Gnnk3yk-KkCs;jeo|zwb~Zscj3)He18-AD&0_l z_u5GUBrNJp0az1D&@4b$h77)Afi6ffjh+W=O*Hl(oozvU(g^o=w6M?>{Br@X=)eME zC$-vgusn*^pp#>jv7#rKqCp#;h8g4;v0`uog{Np^qSe@eG^fl?S>0?Y;*p6R5?}?G zw5YfMF%mX_g|TorTO%`r7DtA9{nz1Ot#bxQ$!KH9;Dn*AA%io9#t=14IVa`9#DTuB z%x+zgWD*z9Jw!AdPCrqMtGf)OyCHtHiaYt&J=Zp816;oLJQ2U}?r?5GY{`4Q!lsRr zNZa@O@PGZ6);a~(>CXY2DWMLJbFD4iv#EtG0u(mBGU{Z~sz~@AH2tP?Gs4ywqptsDU0K5v?$OM*W zv4!}9!f{}x3-P2)xYVb5e{}Nq@ynb>r)KH7ppX1>bNZMghyNR|j(9rc z*Evl-X4!|K4?ddqMnJ0y<2OpPTs8+}EP;K;#c3H!(w!qwn^lVHumZBr8NIobfO`|h ztqNk$4r+*4PjCR%V7FtSQ|D<*I*W#8lW&x+b=mBfu>|%V7cWASaN?$zSm&`a#e(kv z4B9~riEnVw*tdI=q?@Pf6}1cM4741Q10T)MLLYwcHBY{;Q_U&`-zX)S)aIylD!S;q zOHuWcyKwdImE!q{&4>~=sIVtrX=s48MCGFu!{gRTK*7rDhX*C&RXN^%k$6kIu_utt z+zUu=5bD6{g5M~mwPMl^LbGHF9S-Up&FiVd%2{d!%>B?9aZ{3=#;9dd0N}<9=0g@- zMO&&$g9`4?#Uw3u~yS zcM+W*HLv}EeJ*hNai4vcFdl?U=d`DP;}%W6w0KCnwoXru-7?ef-8Am8__E10UEfX- zz125=?Bqd(--u;TeXyy_`SHQOd=Py1iv_d(8?O#|I^yScY0##rofaV=dt&Io^&0|620#N#1*~(L0PY^r zapTHa3G2sZg-hInV@ zW@JRCHA${{TIgd@tk6G*W}S*XPKPFnp0Q4i{b$Jis6W5>K6ju~HlCl@j6{MC6)xo~ z9R$%MDjzKtp#e0XPl9Ot!-JCkKySYUi-NHy&`szCq&En4VC>;$3neZLwssI|A~R}n zQ15iSFn7}?>|XHl5zJ}S;SV0xcjwO@ui;^~lIz_sPW4!t|8`^ZdHKSy*N z()iu)#Zxhf#lL>L-s57aTfyEmw@syc7kIjD+U6CxcY#O}gxa7uRD03=s0Hl<2snDk z_U@h}10XLb2!@C`X9?iEv3)m=7$={U=vm^Hzc19%vYWCV`~L1Y%A1;K*O_ z;~}_#g3NcO01Cc6nk3yk-Q_bgMte3%u6bJM!$U4~G@@CjnygIEm*co&zXxg9@`K2SD^Jqm{}B!=Fk%2?DI#$&V@?cu+DlRFHYqD z3rKGe>OgstSB?0ws0oB-@lzT*n{U66DUC2C(Etk0OvD|&9>hxlz$tq)3od*uRISPY zpFXky;F@2srY7WG3h@mN7z9sdZOmY~7x=6XOwNY6m2j^BW;x=NMwmipfN6~e)JYMN zK&lKNllz!AG`BjK4Pby-c-AHcBTPSP3@J|8!E7kPTu3O`q8PR^a4bkIjtq_czYYhh zIz~@MV-Jf1XlZM}dU1Snpr$S(lMlB_=Wtx8ma8lea(%MA4zDC8CH}Yuge2@4uUc-P z_cLu%mp&o7Rkm^I*cCOL_u~ZvRvvwM5_ja#o^SSVav3ga)5c5lMZ7E^>2Kd@S>1d5 zM_wKAbi~hT%wwJWF%|}!YPe;qM?4?+2LpiGIS!*yoow6kCg!< z-ghyBa@_@N+aduJd=F@nbn|rO<0KiSO_Bp2&Co(0#W4Ag@3lm;PQjE$J`=@B>(tn( zwS%LocP)EWw{t9>pQIU4!Uh#i;VTU?oTz-X2Jmbv0jvB#;jS;KOcaZ&hZMtS|#>$22~=jR}0Z&c;Zs5{?7)rj-TSjVYpcEKABW zCrsLpVJid27_>MttqcdNdIi)@Mq>{z8tG`1;O-vZ9H?n7ze2@vEPkDP_L(2nTwXm^vOUkaw)oT$uIGmtqh|&j&k)VN^vQcQ z5&J$C-3|B4nzsAt+tLf~U04^UfVE%#YA|WFCete~HKgxbrg(qaYPdo-&HZ}$w;=9G zCKaTZ4!kjFW6zw!51wCU>_2N*l<9K(=C4bB58u-8O795?j~z>nYw$XLZQWZ7KJ&TP zFQH%7X)lL79r1IGb9#Q;o3Xp5=EdG{NXHQk{YUq=r%nI(e{V1QxKofgrFWKiYC!KL zapI{y&LBt3Aev7Ws?m{TAKIQv!)%{r6T3MJ(5xT<8$=35gWas|lbx52*bp*sa$2A` zxp$^`s(*Ee`xFV7~)fsbZr zp$`wa(9wuyosw75eoxU_>(r^!e*~0W>$iRW!U$J9Kk*zO5_AYm8Xp`_h@NG%STHQbJ) z9^NP|gh8E5X~d5p!`izFSf3YnI-HxwBvbIFI?aM>cC!leWZ>DE4Pa{m{~vE$3L!ff ze8*U!Qp2nocucE@??l7Oe`>g`u2!5uYqA51Fu#%VflQtl%pPJCEmU~pA1K01JC;WQ zjg5&$i3eeIq!m*t%$W;MdX^wlZ*xL5m~P%5bo%V?1TFF<_NOzBOcU z=FoVd23P7hrSWO^zd0^jCNg@y;C=YzyKwq(C!hAfQC@%2sv$8Em25l9coxV*yHQhK*oVn`ffGPLYU&kzros{jg1uc%v7+Jj83b-Hy_nHc+W|xfG=dgty;C}B6_If-JECcYM27cjtED% zU{)!k+QxtmJvJg;VGAkjn88BtjVNM4QWjk@CrnyIPJmdPYyj$}`M@y-Esjhp!@;VK z;g`|IkU`7S){p^j+6bs=aT%jz$$?&rInkm8(t=D$*vnUY!eA~Q;9fHer0&yQ7AFSifk?agT}{uR7xCkYCqc-`(gYZ#QD< z^`JA3=s2Wt<>#o5zm@)T8JL2aZwji3S!BOr z>Yy|NPEB0aCx}5aJq4_r85GZ(5-;)@vRMT8)FTQ5-3H|s_A5a{e=L_wiX;;knaokD z;3tqlGd%>Xn;8^*do)S9dAfa8%;5r>BnLj4p@lyD zmt!LN-1oY+eIL(HJO@zX22qIQBLT96sC=|&=vGXPp!s|fL`h-4V)CO(2_BS;S9I-{ z1Cj&S6Ub)n1*A6!b)Y=eaHY;PChZ_J3#K&6p*Q)Pm)^zY(cycN@FRliFrS5BD){Oq zQ{XL5L;-@2-I-OGqXLUE+5k#h#I_VXVYOoh7x}O@Ieh&N?$*Onstv2RE0_tAxSa)HUOgU=aG?kMe&tt=jPkcW~$B zdq)myoX=P6F`dnvux0e1-hb4quDeWnN4xs3`o3d2H+0JF7?J$eZ)Zn*^^QZHj`+C` z`z_eF`}L>0{?&bDI50es-nPue%~8Sz!f|jL5JJS(1#f^#x%YGWw;V zwkZzP*+FFAXI8g30*;(J$SHt9Q9%OM-VAmhJbuThev^yZXGKUd!}28=5n&rL(J&v& zrII2Oh>JkxXoBF!pFvT90@mIP3cfR&B;7n+?~tNikxh~VAI;E09}#k)BNNR!C9k3b z0YxjVQ{~=2RF>TdZW!*GvjxvjJO@zX22o&v2bpOZtyDfjeDL@r2(Vg9epJcCgOZ`) z0=ZIOEn1B|fo?)CAiY7T1LdhQz$Y>I5sXPY2+fj#HT0&g=B0OGatTZ?fgjmasHouT zm`Mx848iIz)CxY20c+zb$P5)yOaSoZp+<GDn z7=so^rj_AfRmZT%Xk*CWgrSW9v(uGkO<4o4TnrM`d*6n)=Yl0|Iz)7x{VQIfi-Hs% zUIW?ox4FMwM@_U7>`uXM*N^o9!0cz=+ZZbjX@nx3wmzdi5q zJ%&4S-w#(Ex@P+7$)gu9l{weehbL~ycya6Kga`d^?bwlCkb8PW$D3V7hpuzT(-A+f z)lN$<>wf%X<>G)R9MW+_<8IFV{GM(DC*&6VUYR2Gj4zeCmBtqZN!@lV4u~)6D+)lZ zT?(qcab!QnN1Q^ygoz`1M>1$~fq)Gy5Wp?j+vj!aJ0mnPC|T+eUm|rYi7yI7Lw_{! zCqb>9l5U>vmd+iP6f{W=d^AG~edNo9jz%==l)Q!x zL=>&CPL)RoA1YIs+BAd>y^H53o&zXxg9>}{l?E9|R6bfHioRofJZL_j1OZk9$d4+4 zcu+E4RpQG>qIyw1_5`{Ky@2!vp$?R%09+_xB;mYa0rBjp_L+4(k6qG&=<3Bae%2xp5bRzSYh-KiPH3FAI0w3mm~E zEPZqE&nHi0OJY{P;JoXr4<2zUR6E!B4E|^jjkOP=Gk8PXG7GYU}M5r$XDNS|ukmozQCd+gDv2Z4uKpK4K6Kj7~CRy+u$e>B22=zTd00rMA zO_FY&ZuH*V5phkD10T)MLLU?4LPsN-bxK}KM+b^tv`&?$9`Y&Ey*FgkD6g}4e&RWR z5;v$Y4S(kj&>}QWe6(25GHFoEC}@7c}ELr-uP%f&GG{tcgp!A5p#<_#=7tHrq;8a-SY!w#;WnZ!&2SU|XVI|Jc+ zKk!u3!ioWQOzc4Ppm01bemRJGf;sB&feeEI=1=J0bteU^EK>uY)`8L&(8GD&O$BoY znLHZ-(kf~NkaujhVn?cKY@)^E*i?)h6hqP!#Y!lMWl4GFgh{QSn8V6Qn6p^@Z4ry| z%t$K)sl}0@E^KT1kl|nl##qZ}W5^&nH1skE6ZMRm7Qa#2Q#8}#XO7!iAoh>C<=d+e z7xdz)Qjg$*Ualj@uWiGvp_978%|rZN z|F4eiJ6jnaICS}l$SqI9?U!E~URCurnN}>RzxrK=TR$(t1-(oIc8tIUz4YIHzCmZo zzPB~ZBdPCw?=y4z

c%{dn@1`s^@M=LYY?@BQT-q*K=?)Q*iv4hj6Qvi!v(N7Fxd z$kP$Owk!L;`PWMc1?3C(c{!xxh=wM&c6z5J?=2p>Gg%QSNePXUBxHp~2S^f@r{E0f=gw$Qom47zR(8GPdo=u;szB?;VTWz;7?ROS~PSsIbNCtSox73 zRs8XwWV||0ZokYF&BUHSH=!4h-XPS0^3>utO8HtbX$PTMu+EYkdQ#- z8sTfxFxdoN^MQ>hF!TTl36MrXq4@}Du>{S9*AtcSb|c(nZv^OV5g42s8yRdp!Hpyp z%+G`a!}=yL?GG-AwW{MP^-(B&5SdGj3;4A zV@K{!H<-xvOdC@zK6aN^n;ftyrl0i4W?ft~4uepHw z?z5Kt_{J1bSm) z*>_Ttq?@NZD0|S3xF*Sgk7j70kC<4YqY=$I6?=>y2#7doof`Y8&!MO*1F|>Vyx#%O zPi#h%xIq+X_y=}QgXmdC%a4!HA}aYL2(YsHAr&%S9ffmN%>4;}p(oHy=mn%V2z6lW zV;=^g#xku6-ze>2zNZdPB8~Vl9Qb%NY_N(J?BErBCiQ}1hVc5CTH$J5F#OW;SWEyc z%JC)^K3OTSw($g*S_|{u4X`p7%$-y!RXUZi%&0sGmM75qC|~3hD_YJJy#VQT#MF`z zmM1k6D4NR&d@LLYtZh7L?NB@h-thqBAnZs?TEs$t*dc8I3uA$Tt&urMEshNJ`me*m zS_d7#dg@3`263j5lffxK!-|^z6wH2kR#MpW7$=&uQ@Z@@gCPa_-{ABUcP~Bo4M^Yk z@Tw%yn6RDQy6%f~|Mc-QQG=#ra#?PlU4Bt7zu}~t@n}icSLmB_5`PuS*Xn; z>PpCPTDfve|L$M};Ot2|e6tudHBG?Gc2j4E&YUKl;oYuJP_die_9QponC%JfXt)T= zttCZ>x)L&|Z!qG0uVv8GMFM8FGy5LUB=kigL==}{W*oXUxZ0jncxF$D0^zkvn3P-bAj`5vo)h=2T~ukHB-ZTQPvdEJK7Oj zp#Wne0G2UqW#AZt7DtBG^WHT!e$o~3X>b1g)97hqbny-=`7nh8 zL}?ugMDt5KX0XzU)&*g02&TJ6F*WUk61$n^-Z#g?{Eh73^)6B7eK-HfF?;d zPuF!&|Mavb$$^h%XrYfqu|oeMnsq9+jP^o`p0Q4i-C(#MRrk1Y@z?J>hUX_XBTC$$ z!YuG0N#Hv1(V~gM9@a@f8~^a2q-Vfo4g7r@{z6Zno6rkLZ!p(^A2i_oQ8s)yMyLsV zP7(+8Ue(#_;r0u>d&DV~FcBPoWYeJnYc|0bDyS3ySW1;a@J^yxg)4nEEU4HRurHaGNF^)E0F|zaK{|DfS)~U% z-xyHZn83tp8zZf@&JG6O?XaeZWYCV-vI0!nk6|kV#~8FYGOY{;t2%~CMjHc`+11h3 zkinTln*%jn6`aUzxY?2$9M`u4xA((mq`EqIQJUkPJdiaHgg9Z*y|!G>yss}USS1t9 z%9^sJ>o;o7+8Iuxz7PMz|bzNGAj%E&|KK8(Zj zlbi!6VS@^%@s);VNF^#C8WxNQn$IUefYl=Mqe?0sl=O#MGN{kyW|N+Py}{fINN*78 zKzZs^@`pjFu?(!iU2pR}b$k^czPF}<*Cb&hic$}EfiVHnASgz&W&y&&xH|Zvg+(3( z&?$9bl@7a6*t;8W~#qe;p3i zI#^>FZ48;v8ZtOzXuMD}oPozfJw%BflQ=FXRP6dpH=oHltn__eVuFiQ}byx7c_a6Nm}3)C#sA3OBc3#e^>FgQ|FZ@BYdYt8v4I>ykPTx z_w}BLcBWeC_t@WDXO^zWS{N4 zAdP@~z2@$YW6p3IXcd{fPToM!FCy831q=e{ybk%s3dHX};hAPO}6+0JPYJgKlDvgPvMC=K4 z6M6yZ4MH6ldw8RiAD)=BgU~GgO1;n31G_e6!~X?14yS>u;HegU$_Ec6;obr~pkz`2 z;ESPrmO*E!!jwp@9l-2B2Y=86)B z0zF*h-JECc7@#!NGRUB_Q0eh)0G;3fD#Y`F_ z-C&^S+yj3eHJtmZ_Va&s)z-R-z&$1ZfZC^%GQg3{vXatlQv_jw2d(J3spK;mtoh;-<~YOpyjnij?;BoD^Ij?diHJU{Y}I zM51*{Lv<6~&y`O}!|YCT=St@in@s@C2okXNg58WhyGCqY;biWYn~SR@Rti6jcC>> zc@6E-6s@sNmB%Eg%hYRkNFRGkf#)Zl11NEW@T8Cs&6R`9w~Q7GMl#LklOVtOgsFah^Iq7BOiDp;@q?mmK`{HGiX2cu$Rp$yM21%EUx$NL9cv7;U$BnwIW4|5 zU~vEgZ4T7*_;?5(l)@Tyb6%A$*R&t9eaqKy`iZ-LI2G>#`o`BSXeSyo;^xBTJ@>hP zn%i|y?(9S^%j4NVWAg1uPP&|ev){fqaJcie!S$!Q9f`UaCf+mdsR!Es#;YTq4*B`- z{4?f3`+u^&=-|JnRfYKnr77DJ;X!HqvV!f2E<@o#Y5cOyo~SN{4A@WF*(->E(*7ey z1~8~QDY6+cuQUO49=dLJ&mz~t>`4AWY0~yYcu*R@EDufKK`HolC)SWbeGd?P`!T3H zDMEcO5-0hqqh7_kQl$1@is?~&qlx<+k)=_)EctQK%uYbXbY$I&S&c&rob z2xUxBnlMgkaOsz`@-q5~LPK zhBnZ@4hO#FiwDN=%V=Z30M56D3{C;s2&gH`;QC~^Sk^(wao3kf){Q*cQ~U}(ofkd# zs5e}*a2_8xeYcFO_+Y|*w=FkZ`dz%Xz4W7hdFpIWr-c#e zUE}?~JJ9aTV1=~LjsM20L!OTKwZFIj!0P zGT4WO#Gr{EDdI=0lUSKN!Os8&tsfv@J=O}K;5)BL(#_Ld>7DKs*CaXc(F`s05fdwP zBBEKRVo%a8N6{13spvUNrbcP6ydM1J)VJ~c#AZZ^8&sIIM7v2o&;Ydle6;-dU|1&s zUEAn~2PNYNe%MhemWg38uj@!F4vQ zZ~|YD!%ataa7SB^f&$+lf_%(^+X)sbe$g6T2zdk`9+@>otQ`CwZPx)1#qss`?l=L3 zQ0xf=tY`!gEJOn$c5?#ulE28=HA+xHkO(4JQ4tjtH6lrj>BVS@CMFRZmRJxFNKDfd zqeYM5rD^So25?|0o#XlS%kR7ACZX z7eu9MZ820)iPc;e>ndS&USoo~f=kt9W>E)Uv1_#8q7Fs$SckNQ18SvJRofb>Y$;-O z=#IGLewlv`*oMGd#45B!Y^^L9Ja%N77!G!J@LVY9$xy&~re{L|mj^u^D5)!8Y;HS2 z<(wc0)7MCr^#1&bMe=TV(QSRU_ijk!uLsL?!bH8_`@fACDf?gk<`dd;#X^T2e(6~` z6~U6|^BW(;?q5r|+@Et6lQmRk=312vhIgHo57uO{UjUYqxwmN|k#b7s)Wi z7kxV)lKU~2YFe1yb*>oyxZxIq``f*{!s-(q!munC8TjucUj4e*JJ%u?Q z7lv;N{BHilC)CEic6r+4=UBY-*b{kKfAp%5KV_GWJsQ?~F zX+%Ni_VLn)UDEK*?UkJZQ3}pMv0E%Tj$7O(nt*HlWJ7%z6p_QRcF`PMvvj^+%IdL8 zC&mUyBRhLbBl0`9k3+>UR0$v^*+e^r*+gm=7)7Q|Q_X5%z%xw^F8Zgl=yM5S4sER_Z zQ9^He#`1?!Tp0s4m{t#VdVo(vNe1TY<$}>dvJmxp13XXL0QmDJcmoGU4OM}$aJEpr z7);NI5BqRehviSp{q7XRfR27 zT^|OkVOUc{C1OjsiS6XhxL_H>CI*f%c@~-*65X$H-dUFGBpXpANk6&S-}h_8yqg&c2Pb{<;NYAg`uWfF{d03 zZtxg7Y5nezi~Ht;$-INYWG&NzLg1JX9D^!p#3Y|+hcTN-?#GougR(j1l+8izJG(*B zjnmC&HEe!*gXF+RBec**nv(Y~Dv90CKyu0wnx#6|taE2|Prgz3M{Lde3y(|j`owF1 zq##3s1I0#1L-dF!Mk`LlBWWsTA%XHxvQZ%nDd zgYQS{+(zm_sEPu#p*Qs~-_+qxX_#OIH{6Zn0H1yHlnXF$5z4VhV=jfkiSXyR(QL*5 z;#Ab8DllP+*#U#p^fny9FJ2FiS z2W$FOV6LDiLjl33@x}=Q=;=VoPy);Cj*x!)lu3}ZH8~7BUi{24We{9`lJ~+2VUBgd z{m_Ek5kl=-*EKsXhB&LY%vpV<%1@Y;l4I!VGh?Id<~IdzrS-ouS3W!Ky%RfMK6dfX z2)y%)V}(J_-`N_Urqea`D$ zm+VdLJGMz*GOxg~O-j4iCG)neO-E^R zIEo1bNSHdKKPjHvcUQ|jKBNGkZP7##u~vZ`96V}9JE70);3d&%j^0b+9NVNVnU{}> zF{qM8Ou~pZ0nj%oCceG7n@F?o01k5Bkqwe=oNnh`9;@6MB-c1C^x>-Horp-*Ib}KR zg%qu@&J}Gk=3 zmmFvq4e&3B>WoTv4m68U`=n?BKY7qP2};;^#~~IK8{km`#~3_zWat_D*WqAI$LJ~O$xuM>X)s}Cbt63qC>e@i zbgrAQzgcfV7zi)=dY3pSH5&%koG^Xgz+;fe4nLVNPsd(z%_Z{KhCORUhw zF{J(J>Ptt38zcQBA0Kt@BMCZE=H=hnFecIejg-~CtN$af_ITRkC-;4@ut@n#My~SU zCA*D$6k{JrZ0j7!l=Rl?10_?I<`uMF-#sNAr96PTrzA&{5V4U%r0?x6L{+Z8lO4tz913w^+HcC6PD$vPK>zwy8YO3@EJC3)cw!b3(6ZXFQgTnBA0O=(;0IFZ31sKJfb<65 z2F4yfJ`!8S(s~eD1jC2%oqB-flRCW2fM18J$gV~~4MWdaiiHHwm?3x|HG?qQlnQRb zbpe$vV(E%n!4xm`sKHmHKvt!NUtiTQ{KcSF8%`NCCqNVTs*|G0S!N1_B1H~q^hF$m z)g+Lj4PYp&t71@@U5Qo(m4?*^U)WGjfZ0$)C3HtzGQTXfY&fIJd~qEHJZj)rkRCfS zO$-NX`Z-{(peI8C7Yq$OEMljpF+@po7M@Q=Nx$=2BS^fTahUqoxE`yY3xUf|a>RFx zKcw!aAsxPv$X=*z|4GI8<{h&CmbfQ`3v;434Q~F%suY>-sl_{9jB_|HPcs<53rh4^ za_4*D#D)X+{~NFNc-rOHcGs?(C*2nrnzx^Mx=F_Aj^PPzj=kIxgh0n$(q{1qg74gL zlx7jJUpfN)mi29yPQZEN$FBEb&=RVMSj#{T2Cr$IF>;D;*3!UK$It{9$6hW8f2`~l(kiDxa^Ry8TIgeQ6z@btvd%>n z(Jn{Pv(~v0TXs4{JP4LczkOm7UY~dxF~uDkOvAsdFGQQRUyN3$7z{B|Vipq6vf8jx za{o9#N z-~5n6l^#&YKuSu80K*(b2&QH?i!l2Z4Dqu8bhZdSdsCnAktnf(`MT=Ebeifa7|4k8 z{%X}#@cN~?vf2on@hWOx6HO|ZDV9*bC0x>!n?+jL^}2w*%m&dF+d?WUg z6b#Xp*fS?AB_%d7Y+~RTgU60c6T<;Nw~(`g;aAX;fj4z}HsDr{=L${-O2!hwNth~I z-RWsT65CV$zl++wI}O|5nv=Hu{=E+%ZS~KNPtHri=OpIth#KdTqR&2c(XGFbdg`^T zga7P)&f%vwqLnMgs>d`tm9Ws*YI$a1$dHX_ue3%erYpG8b-2|7u!udvxs5xT#4wa6D~C;;O+1 z^f^uiSp`me+_JU?I_;4*%gNg6J0J4WW?A7V=BAV5(hT=_0)}++Xco($IX)a48^}S= z@vAbPT|V1)Np!YTepbHI9@ng`0jQXbDv$@?VMIHP*~D=_F$|jH&9Sio9OS+u8zkL0 z-7e2MwajUd9QbI27W&9m@{UF%>zuNJ-5u~Iaosvs^iiJEN#W}sV&Y1=;`NEw0H(M@ zgX6_U`#|)FC`OA!F?VT(H!T;lkbssy*{BeJRLK3~8uSDw*=ia5l^NTU?B-rTdIN6* zW3PM^geoiaWRZ+Om`^=(?VbbBvEbbU{L%@V3U_*dIUh*|QY?g*D@=zh!QO zf(=k@i_{0H>rsP2?<)L#v&vwAA6MaZzuHi0&|U@2V$>!F;8D&@p-`lVIwILJvq&j2 zGr&d86?Iiq)%8Ht*KF0QZ78A=>JeRLz|#6-*u=o`AU$qmnivk&bd0rvo(u&)8w$8k z=;=U7dljBfo|pdW^`{`|@RY-X@vr!M5A}p=PI9dC+n+${@!U&!u99XduV?3f7Snv# z!YBK*^?6sA`&nQ!r$0j9caV8s51r~X{R{cF1y8zu`rf5e>*7vN80UW*-l>1~(Cb!P zQ|*txJ=pdBoRXVg{)6w-hkrkLG^GEW-Zo!TYkcBXQQoKnx-WM8cJcPllT3R?T{Rlo zotp5V=4SU(ZyH`3e*dLizp0;}5FDDa{kiu(C~0Spr(J$+@3ejSrSl8C-aQ=uu01+- zY20pod+oMbE2?%`r03)p3w_qQn~&(@-Unn79({7-mC?6VXpL zgW6@AeFt!m`wnQ3bmMfpdrZ!CYmglHXoME}fOqPw+Y`w;rz~fMcxx!N&J}IVK2!K$ z`}3Pdt>1yyCtd@X;tmat78?y`1Of!Hn7I{+tRr)LT z1hSiZ0qG6A4U9d!Qx}P|v>t>O!Q^>L=uN#&$<~D9sX0nhNF*R=R#coZuBCg zwG`Db3n#(94dIJj_?}#d})SHb`{TMu*!`pNi&1i(z>M(Jn&QNZE-%1Ut67&Ev z;XyK%fCla-yao%A*3Zn$gt!AU#d;u3eHbi{XHBsTYUY)jg@$6-#K18Ij~$sNhJ!U7 z;|a^zVOte&pixrr^MjJ16fWgRne6Eff+S$9!yd<=fcqn0P?sRNc=pOmcvIJ%esYY& z(`E9;%zJ^&KUuMGf%oJ}VWG>l;HSF0l;+TLRbQQ3(wN`mm-avZ$^N*KxYBb{*YCrx zz)k%~&BI@QGSy!D=)v6TE7#;l-SDQ~r++W}vh<(k&;53&_HWJG%hKG7b&54RcHMm2 z&$L4maOv0Ywyl0poPMEsRn|vO-4CAesba=<|Lr|O9;!Q^2(ri1F28nvefX+Z#LBrh z(3#Y`9JlX3(XpsW6p*ICtPQ42YPvf*%THIlcWH`V~S}cWikSU<%MP!&F z17RGw5$^G3Ee6;k`2K?U1_z=Q%)wX>rUH+!vZxAJz{f`#_+{^uQBwq(r&0TiXmXaB zo@9cW6~WBCsBTr64PZE5SH)mFXA4!6N--pswChqtL$oFK%n3{Dk6{x7#~3_zWN7UF zbvR(qEIB)c&^iS@848FFJsS!L6FnU$sn5W-pE0s`I>R?~F9peWEqOcS=f(KZo3!WY zIru|q>yr135t2yPCvUWJEOV8vcG-O{>VHCJ`=`?kC2qUqtseBc`{BS>es*+CQHFXS zTzq(Mq<3WDvG@NQul9J_<@faS2^WL3K2Pn;a-7&CV_bMT-8m#WH!;>ZWZ2~R+{B4V zfhf(4MX^gdIWC<#0+axp++{~{0fSOY;>7omzjF_}iRQ<)7YiN5zSC}lIpc7ubuzBe~Wx^cP@I}*G4G)NA7G(rn~ct!D! zMkMQ8)OmU%plFeGZp7#9&JUI(bqct2`5<1Ocp5Rq9U9C64}rCc7R@Z3Dh5T&LQ;3b zO36(LdxEp<9oakB6Uc7v1*A9dHZb<^MX7iPV`)7IEfS~R4KB>xwUf-eD`;T`DV)>` zJ~x8hKo0PFfu&}F7r}rOUTV-);Y=vh7R;cG0>mRMNNj9eM{22~KzD?Uz|FBzSrE0+jZoqnH;^Q&Djr+U+AIsl>%J=E$G>31v_x7moEJhSt7A z8zkL0-Ce7v4$5he9QbI27W&v6#XAv^taDMPX)mYfN$cE*?%~%Wsye$*o#Jo@uTMOU znBopmfHMN&ALAJh(IcW5EiVyYL{l*fNot)9DtPoO*R1*A9dHZb;4 zkAhHT7=4DLUwT_k6M%Ev!Uoe6z#l2(fFmm~11ZdaD+;^_2kvoD1*=$Ss`NGhJ|$BJ zFd~tt!0HONP2&2_U5p-3K+hzD6f3x2Rs_Rwcqy&j8Lf!qAjt7H1()yY$yUj5+7pEoUpY17&b9*EJ%+XnI?vVHT^2IPC-uw+|+W!qc%6)8&(WCN6E39#P7vsXL;mw0q)7EKBOnjOxu zA!e}8&bgD_mCJffjE#}UOiYtc_MW%|j&0XKx;!GAn4}RG1VG<$+)pI;9mcUCW{~@i zX^?c|bVp5^v&5}Ia*fkMAFfK?(THT7QQlnyalqDE4X5C$!iRrYeOjl}R_m+v<$6OYXkJHckcmkJGbJeTQOO{^k(aXi z(cZcy=E#Dgt&%}okDAtoBCYPRV%WsMu^>HmWat^QZD~F2v{iy1&c%vLAf5_(GT@zh zH4OmV-mB^9KuKQ-%k2)4zW(G3g2Zu?gZyQeZv9TL#GCr|PVeg=b^CO7;a=h1$GhM2 zzkSDf+CBY8h3$I?F|WnG{=>!xhh=X^9-sDBTj% zYD~vv4pB9e4-YvoWqS|#{lE43P#UNX&Hc{Ogx=WSoH zOUE9KJH3O2JsE-9kB_`PC%#$wyy#|$@$-hKH%pwiJZIkUa0a#Su68#03G+HzzwjB?e`R|u+9~&KJ#m#vEzU% z(>o{P^@-O2rno~CSl|O8dX~`&6C)HrQ!xt(Xr+>k3Tap==~v}s;px|AHZ{LYvi1Dj>}^(>Z5jsMpz0L_?y7oVWpN8wYHoVp&AnZ;R9$ zI2NSGj!YB7!J3ZIQ_zzE@ARta*?_@SY8rWzU^zQSVWjk`R~JFjKE*+?dfYRb4$t9D z{e$$;IQgji+3E+;5*XU}_J<4h!P1SFcU|>;8Ws|8^7yqwL=dd4G3M}}qc^GL3yUuI znEB0&Et3|kc(Uz{=>Nv6J)U;?wH>+qyzr@mv*zP3Pd3Ro-7#>fo1<^+QhB5Af2oV3 z@3f`zR8;gw6(ad9BHF3UW;FLh3()s8j?LN3^vu4eHAuQ~y6e`h=@{Q2Iq=a4E%dP{ zigy_*v2I6lQAIRMb*g zlDZpKO8P?){!l8tD!qz5f$Zj9Kzf6@4I+COr_S18AE7Gn(i305^fSMH!SU7AD%i-5 zvI-^>=P5-2fQ1mmBKVNqLWC(`;cjdLRM{eCpYU@ovBF{9W~XKhhrR1n@QEgjY=lQc zm9fHDT@0GIt3W0cSxQO>XH~h`8&SoE09)ety17iO{i+R8#eFg>eN}a(ts2s^rij&{ z9?@k6ETw4O7&bLtim!EWGS)W2k+f+<&+a#H`YyP{v4ko(h|J>%_?@1ek4bPmoxk7ktihS&Q z8!9e0i%fBxad_r{l!8`ahc;*ehkLl145s}xJt{vi)vo&De%BK1nKdKU!xQ?$&!4#% z4(XqM?^P|UJ+L5e;*}RRt7rQB`9k}w<)-AaaPKvy>2Kfva6+iv=VXVxk2 zo{Ua*xq9p$jXj=r`T2Yjn6E!yQQ-0J-H!I?*roBa;_6K2etyb-Cbs&qz&U5Pk8_64 zZjT)2jCON#c6+Rw5BYX;GtyCP3k7g&xxp`guOmbt8l9*&5=2XDJtX)~V+O?j% zW3f-ZbM|g;=M3-N9@%J|jVh1_-)Te}3aIZm?%S0?N@7BN=W~$z_HK}L<8=4T-!mhp zL2}@u5nAXYTgf{bk*ssdN_xAX=ymH{(KiPsoRkiK=S^p?A$Wb_HGnDZ(BOEn(SZ;> zB8t(XnU)&@Xt|h$1h&m&qe3=TN_s{G`JuE-R)#%+?B-rTdIN6*VOn|aXh)adUwiw$ zraWjuhIGuv>e|yMf9SVxjps_A9xFeJJ!_ArU4Bou{;^0hD5NB4%~PZ7(XmV8mpdJv zt^MYl|K%0GevYGGe4K;aa@`A@gPgO{bKQ66rK6M_j$%R}Ir{cfh7&MYHdjV;5&#N` z=U7ra(+io~tV_F59^+CXa`K(Sa=o2{Tyx#?Q85iw(uheI(GFlXVcZYVA->}{mK4YI z%)YZ5B;7b&<;<~b3K}E_J{qBgKJt~k6H$rX1V~O38arlF216EqK+4 z*C$>BnBoo%E)W|Hfz2nP7%dt)-+p3H#4IGBwU2C6$j3^_{i>4O)DOrGU{4^sxfhV$ zz}rB18Vzt$7u&(odJtL!Q*tPwH$7XLwQ~(w_(*^S<8&I_^ti+Kf_Q*mCt1n`qC<;B zRn)>4CHU@}sWZUBk*ESQ_Sz`HFqe8T6@15|(%@ zC z-$OjwIiG2}a{GVd)hkanMT*YwNCu?bBwj+KXx5103jAKx!f-h1Yl*zx_k z$YW8O7>?q2D1hVq!CBD+?6;=djz9(t&f!=$sttOBSGy;+oV6j?b4c_wdFc31dGAHz z`}w2dG_;#WOhSnZD4@Pab3gtJ8oZff-Ke&e@6ipCZk%qAV?@932FZbsMrfgr&?w%k zh-96MIwST*lAN&4jTp0ZRfOi37Mu2}Q}O!5(}*eV5Cu0e5;+h(B8t%(E#jMLDrO<^ zud^W)a+^6xekg4%ZH+yF?z|U}-e7KnNEhZ)7bTX~gU}*Hl@4FO^gq4!^$i(s;;=Hf zb5_H5|FGc>LkQu3i#@QE3oyO8SY#yXM9T>8fi-$9wM8=q5T{lu8cPS%N@LVoIOUiY z2I0V(6MC49pbFOhfIkmPs^PnTjiCayuZkvR%#@&{s!9gIlcQOr6wO+UA~@v@66>~b zAT~^6EVWfL!U7Q16p10YBf8vx`R9Ob2+T#SLR-Yv3IfL%Ja%N77!FqPb>OL>Cj(bL zqGtn^%h1t4qf}i9A4iXu4fFj%kc=4WP&+$4=-ywocvIKJM&J*nEi;bh?vZqVv*WwH zpIO!7)wCB5n%(exT0pSpTovOhWX_`$~QKP+;TVZ``1!?B=k5j**#_=OcU@y z&6ht7H`N|J{y_3d#%1@GQg}kIb$Rt`{Gs%(l_M6-tGzz;hl$^otH4oXuBP|Qjv$D|>% zDVR6A-^g$Vtu7#ni1m-=;Esr8>nAPWI%aV}wo};ZG^dd4)yrK`F$Pt#iAg@uPGdF{ zL_c8+TAgq99mzrNJHJ8Fjnf_4cFMB&2FZbsMrfgrI3@2gM6%8)D`+pIXo+>M=)mhS zg{Hn*pJ@Yf@%qGT08`wd!7R%05IxdZX#$4H zhG9=2ySW#T-e7Knc+-Gs0@&@tK0;L#8Zt}cfHP~}KCl706)utiiNy|%FWfo;3XR{sWQ*3s?8m40PS<$Xb5tY!E*mEbWZ5*g4 zPE3b$Fe`Hk^+5RMiUjGgBSXFZ>u|7&D}j`Po(vf2VxaL>@Y8{kx)hcMXfF|(I|;(< z$&&Sk;cn4ye~lO29db8pSx z3dwzko2_@7-uOy-=3(<`HSsf_`ts(yU8WmT%UVyWXz_jRE1BcUW-m@!U+XdS!3@uD z=ls3w&i=E0ZW6mZ?eTLOUHQO4=3CT$|NGH)>DZ%TIMUVOLCsIk{yt;;$AQw=F7eVy zIbG(bODC;g59#vdfhc80qu7}Moiileq6s+C-E&PKDFA4&563#BbFl5)J=@wZ=-+2e zdVn;hOPq94c9;2Ss0iuu+_f0W}Q84{RY6ng^Qc`qQn!Q2M1EwD%iYn6S3szCo!g5&4R zUzEa+kua4DY%oi@Rs)k@;{jGt7E|~}p8`f+WQ3<^4#J}G>MAv{M>8m+0DOMQ>=%nX&o&Y7(qq?FKZpa1O5SWWtm~0VSD>Y1YCI;!TBSY)?*Wq9l zp8=iXEi+=3Vu3JGMoee7`_Y~|2 za_K-R%)9&Zbh`6iKzajj17i;_Tk70K>OrUqtbBy8UqXvAGGUg+T$ra3reA^C2Vuhm zex<`hHQY_bRL_MfvqV)v3qB8mA?5JUXi-K1n67|2;wmZ&SE7{)eqyy%gT*nb)i9Gv zHSmC0&I|^h<_ap)^zwgRt<|}Ab*IDLhs{=-iqhE9i;MUS)Hq*J9d*^dhmJdo`>rWC0%UZ?fHbGu=3IOwVPE-Os1D+ zJS_4%{AbZ-oY2eE;bI0(=w%qIaZ#C8Yze;Ix5sn$OR^t&wmcMR`gX{so1dJ%S@ZDz zfu!%<4*gK^FxX+_t8=9((OG?T<)3aIX_u!xel9IO(?ss5-RG!Wx!o=udo(oKQ7(rU zN@i6TWV{qEOAiT@Elm&cE08Ts$qWqf+ubo9rLJx$Dxm<5OP%ME;;xRqyB7E`Xd6{T ztPRx$y=}@qnQfQv?!GlSK1`Mt5+GZe7UGwW9s|%Wj`(&ZE}($=CdI_JH-olOMdF)k zTltP_kaXj8Wg|Nz=QKzTd^AD}ePkoh$MvxK?=gY|iyOk9@p7 z@fyGscZdQx6A<`#h@NG%STN#fq{J*FptX-|RLI9lNf*I2z+{p{Nh0sI9SuM#R_^d6c9ujZ}=TvPfr3$+SBkyXB*)+CxakNeN!^M&mh+io5Ko?g0SJ+ z1vsC&d~(fGUkhWiF6@8d;5V|s8FvCVrmPTrDoeV2KmY41l7X4wFK=&~*iQ0=+~wBl z*{^2lQx9*t67BRKd9}yW9zVwymzD0<2;M=hWB;++$VV{_lm>PzkamylxYJGAeN0|_ z$DPhe(J0N1MsX+shR&18KncL%i+lA4T_^x(rw_-*26E7|o%^gYW48_Ub@P|{ciacR z=XTubf{Ke!Wi&AvO|%JsjzCHD4Z5OlFOH4%=OFj((je)^>2}Cy*4M2;a*fkMA1+b6 zR}slN7j=?$Uy2r4=Y}7NR7L#KKD*1#g!XuS;%USbcW7|B*l0J19udW8(ab!Vn3@xi ztTwEa+^lfiS_#b)#-FYt{y@9uZv5$Hbger@3zYM&2047w&?j^uHDMt7$5^j9B z!xABQzy<1Ps^bDILL@>eb1B?*tE;PwT8lCfutnfURPI1jg3pWU!QkHrc#%;J@9JR< zKKLnB2Lt8|S3$D`wXcaLmCUq^3A$n~sdy>=%YYP1&1E8I8x<{#hqt9;cF!9N<9O;& zL?yJP_M#K+j0=`AY+~RTgU60c6T`upj&+Wb2|GF!zf+R=>Pg`w*v`t>VpGh+13)QLYeJfp)BbOZ5;H4ARz0+**tg#*T z$_MWqH0zfhW1JmFpPwKNe%`y!#+R2LQO6~~kj8?Ve&4k>)vl?&A2O%)+3xB~@Pz*G zs{^SLNdKHavqDn)*03i+m(I#mZ#cYVTJo(Hrj$v>oP8huIQRZnbH3U4bm>Qi>LdN$ zdwA*7n-`a@T|e4&d$v8EcKP|dpj_BxhtHi##nCW(bnMdj`C!40?-u^ty6Yz+P8B$( z=eapA4$qSXIxh~+Ey$B~?;nfOP9GG<5@77M4bIU7Y~OCx4i1gV;n?n2q78bZ*7on* zH=|onpVhJX&S`nB&WpqHWC5s{k18?5#G7an0DVVtKO7pB&9U7vw!UK;B;7dO@!N+? z@oA7;0+0&1UtJ?Z8ZXE$U{4^sxfhV$z}vvs!vMxQw~=}fs-n;s zmC&2Q&EwRyuxxY{ZW`S24K}=ahF>Sel%fE@gjEc(5@8ewOtEcIMgf>ImpanG`yHYJ zPj0r5u0E_9hV1LAV3-9w$is&_FjeDKZ53=)C8%8?ny|E*pvYNcmMleDf3ZzXp&lLEXSqc>xDIl45W)t@vQaS`Rl4(2^x7)fiamkuycdw(z}rB1YT@G} zCb6_0gcd148+y~ATW`O<3kNjf=oeTJ1P}186+R6m2bD&}Q!W_I63op4s;EV#m4f?= z4FD_A)lq@BLiIrPVen6c2HvG>^i^VhLq)o=fni zhJhkQuFg51tE-}_HoIaCtAgsYby`2y`M7HvN&8u&YGzEy{86$lqs2QT z5_C;Sr zkks99v(wV>wvkZn!GCDPiuopebgMMr9#VM&N?Bj1wZ>u&o?{C-u5Y2EEyR0~FB^ZHYZ| z!qWO<*u=oGAU$?univjtcCf_?dNOc5QW_Grnde~`A=wbeC8ICRZ#``>6NV-KcC4}WrJ z=e;4t39_#@kGVF!f6kRx7FXvF-?$2%)UVXK{gq=fh3$Iilr#0_#68LIgkD=dx%MEW z+6D3IC8p?ria5{xXN+wxBo6+f_j{%<{=U8OmEjI=Ka5nge$oAS&+M9rgpUH=KfAcs z>>S6L$CjY)4_CYa-9^1@{C?KH6+&=9=7YYE%i|1GpF(lxu%sFF+hcwGu z8z1SAKPt^3FKtw^D=LPeT^KR(CNAg^9OZLApey>0<5&_gwDuj>AnC^GKHGI-tM~@V zHBJkC#3^}4Ba(GaSxUP!MXy`uiUz;ELaNxr_qT8#n@O z5#hcqpekEK_>%%A%!0`o)$m1?5hh=O@s$>!beRE5NeQipLo6yb0Ci!-s@}jc29F(?CWeDG z{TeV=(31gw{^{sRfEO`Fvt+yu1GxrBBiqUa;rD*B)x#B?e7}YtO<|435zF3()b;3v z-CqmEO4W>K^y{4h&c|K+;I&c0*z-S*(w-arqb%s=hqrP{g!didTPS*5GM#>A@5k@2 zsU8&qQ}3RAc;xm2ld1iMn$%;T-uNlP8E@*Z9^E(rQuT5-O zia%S%Z%FOT>dG?fic~;nz+(M4OkKo9g`vzUnH!-2UT{*~ohc!sLak|2k+3NxuB-c1C^bw%sorp-*Ib|8WO;EJV zI#)Ee-*<&NZSo7}&#%Ml6R!bGafb%Sij9Uel89ooyhS{QreYQn&>BrPDnw$XBe5sYo%aIL8+aQiPn{7a+7<6_EUgEjMGDY{-ZaEK`UPHG!&?-5kwRXkkOSOD zS*qs(s=+XLHGKLGHe7^RP+-uF4PbVlhM_E|qJgiJ>R7>K>Gfd-G7J%KoAA_L1&d{< zPQst^pm_$h&x$6em?;C`i(Cfj)rDq})?ZW?FmMOZAd%t;8zoH@sZ?w?YY9wXiDMGY z6p;hk5_{%^rS-?KiGgDb9y>Bk3e6GHU4QKuLcRzP4}?GURT8 z5WG(~^GTlLF;`eHLl7bdY#IQG%wg2xVM0%*?mMSmj*)yjwxBH2?V#ZCpx403t@bSy zj$FAJKlid%ieTL6nfp#pk5gG|B0mZk*zZ5`YLBNqehz`B7UUfm|IE2*-Y?s2GU;_tan(tVoe&PAvgiYi3v=TEe~m`y78Lv)B67moD;2X%cfYLIl} zbSEp9%5xed2R<61g+4Y%@s36$>s(YJz3Wi)q;+oi^9ds(N}^uByzb%yygtck#1wXD zFb%)^Oh^}5MvG?LXVMV5i&;qOZdfVl4`<<;l?WN|KlTK&^IkxD18)OkAN42*RTjso z4?Vp$W9OPZ_^_^nr*b21Fv!FR6PVxu=F?&+EO18wST}=9mLhy;39rB{$|zu~0*~`% z2cnV+7J+8&;Z-$yQmLxolOtGe8rHIauJkuQS4CJcZii02B+E>1iftIiyut zh}1DNQ^D`Av?d!Bjja{XIlfXI^4M}Qg7f`kRCfSO$-M+ zI~Y9$JsApqHeePQ6+Im&X$#@`f&whIK|j<;-k=8#Uu+Pn_eoMg^`n-d{*)s8M7 zFO>deT=n{umQG_MzG*hC%{Rh$|7}U(yKg1S!oT`(|BC9HZ#rZS`+CybC)Nd@kk3#I z3H{~2@oJZ+J$@}?a`b;~`!@HFm90B9$v9SC08i$7!l&kj%lGu|;xjcjcx*IEt%w*w zkL@}SBE_vdGKb1I6y!q`5%X|E!wAp|a__Wm$rAUSV}lE($@fj&C*Kn~H8&I$y-;N| zF(G0EJ%SA>CcZfo38pE3ym?+dsDKa4W48eZPOm~tC{JAx@#sR}ISUDpbhtE&%#Uu)q(Utgt#_c|~f zTdPrBQ>iOK6B48Ps%TQqOs_LRZs|)CuBlz&0Q7pDms#4XXh7Y^e6idJm zZHYZ|!cvRKafnrE1JoChdXzYW$Bqnb>%R^Ms~E$gpeF++6jIT%0hgD7o&=Qi*TrXQ z7kD;VE}ZFl#_`DNqj1f^YSdlek5EA-^-1h31pLu?iD6+2$wkLS3GeoNN$~jR`Oq~7 zuSN<-qJEF*`End@!2t>N>YJT^)X^)?S|H-|tWuIF3$BYsC?9#DE<3UN-oaswLceLAobhh}U z9_YCp9=mYHBjRy16|<0lmJ8Xa;EI)!`v=UU&Y#qmlipzN1*A9dHc*}_n5B_PEUgEjMGDXc zcf(6FGS}wf2c`5$T@TAq!1JUYAIIUK(Hd9^&z}^4hh{Fp11SSxZbBFVPwdeQzU`-g ztqQ!auWO}&l{4zWs^NhW=$R9?hPp-^OVhD$JE zBLhW7Xir+kDcY#$HKYRTc4%!4fp%SrC&4>bKq_H1vD8isn-)06;ISh^Pmpb^9x)uO z>F@`PUO`U=jC9e`v!Q@XfSwMNjAzI*^@O&g1>v?)w&e*~r_G!F;F^=nN=w-Usq4+( zzgZ%b+)8`tu=fV1;WvLDr}G;xj5~e96qcJcSoYkZ*Q)28nzq*=_07%7sb0z!IL^E zr08ku+=%(c$r1XH^4*`l`6*tXcp5Rq9U2U01j64pI1r*oL@_ihLjJVejDnWchLw{0 z75>dCmrjsQz@9*N-U~=?;B8>+@wuF~ily}+v`C>f;FJ0Y^ADx)x*Ol1!-hNjaSaEU z!i1%y5CNQsg%-hT%@!hz_%OhNKISqCz|WjqTSQ1D)D_&W(F%UEt49q!l!waTdm)^C z0N%vFv+F4}jG4pHa^UoYXi~&Xp-`-fl+zk`DX=A#*#I!sh}nrM20d0nn$N6&Vttqm zMWO_@#GW}}Y5g&zAu#K(nAjo>2I;XQ)5LJFrk@9%3VJdW5ONw`m@m#iPX|hxlQ4{{ zt?-94Oz8E(7>V-yZ>~i*adt)_>+%mfAd$~(_0|<(T&MC6_I3`G^_{!uoP5Cz!MBYN zezjl6J(58;q$@jmPkkWyYGn6(&9rqD4_3IZe0tY{|H!L7p7!`TE?>ScWP7X9YwMos zZ?}<;VjL(9@8ly53hdV>^WsSXKuKT- ziipkd;h;xIa5v`#As-`H+NCLa#yU6rlUI*K{L=sX!@GY@!s`=H zBc`}RgJZ=;!wGO9q8Kd}H0vznK+9^wO3D3-eo^{|@CWt;y7OK@dIN6*V-Euu#d{G; z>p^IdLPu`uBTwaKty_oP3!c(W-)4)i6-UDlnVy4WT<4Nj1e<~ zdz#4ELR&-(SP5x9SAsQ4>%n0CHfxF{phWc0o*A&zzy%Absexk*9y>C$;(r|u)^v2B zpeI8C7YvOOKA)Q<11`54EU9k!u^^njCzA}91&$={b*@+GDQ2Xa=l+*Dkdst40<REoPI2Q zbM!Y~JRCaZlc<%Zw6L1K&Yd63{OWc6r+4=l=ZQ)vCWaiI`k~KyvQ9EK;1D z(r@`rS^yACs)dS(C7Qtz^Lx!+E0<>^!RQy)F_Cfy?=cZEsF;SX(ufHWlk=HxQcV2N z0`v`r%(ofjzQYusm>>%ZC!k-i-Yp9l)IXYIJGfr|5sAYxhYZ`DwLNyU z!@g0Q9d@UVTE7n!N2A>$V&YF+Kmqj~%KhwPQ2)^!+rjnBzNa-vx^cS6L#K2OZ;%}L zXoME}2#w+$jY!tHsMF$Iha@Mgb0Y?q?~PFJxOMLCN6~nF;%USbcZhRI*9udVT zxQKWtO~ouEpk=io6>^(7PsXXMC2H&mbmzT*^akDr#y;v%5UQ-G(&Fov=ZiCTX02U~ z_jUNGRioC!rp4Vz4mdF)W*~*dheQE}p)kn;gmEf*y%t?ibyzHv0G8S!LMuR@TnRoU zw=sl?#qo6re7ph!)!~nQcvjb3)4=3^YJDkcSBfSGkUA@vP+iO=l^GN(0~d8DT4*!U zqyo}xGesLu@J*!+#d7eD6_83;O)Rx&B!`%P8{km`$Aa|Ok)dt<*WqAIF9GHXdNN@0 zE(1Lq3OH{xUMN);!`yVflHo2V1R*_3q7LrWa+&TnTyw(7S6=ae#Bpz{Ck_hFG$Glmn44mjVz?ixRvr-^X&4p$FKS3V|Om}-BUK~ryJkc zrDKnVE@610z>f;5b>=$l%%es2GQ?vWbZ|@tVzi2XH^M z0DVVttQFTY`%Y_+bmMe0{F3K}H%JbAG(rn~geiF^B9e7ZSwj0gMa!*oXH6|P7TzA8 zF!Q-nGw}MvYk;I6LtxUy;J_Klw~SVrh?8h$tP``4fYv^;Q6V2ICHE^jOXG0KaO??m z=e>aR2HpnB)1Zer8O0kGOY1>s5lrZ%gx;huKdGzWdv1*pHZ|@rf)EZc2%DuG&=DOh z(!sK!q85noiUd9ovH>i$sfy9;lUc!hG`5C@Dj1p$Yb)t+CK|mF<_*`VPN_8KLGvVP zpAk)pm?;AcI@BTTb;i?Xky1QY7pS(-7F%KDK6%58h7&f1K)WtQAo#d3Y+~RTgU60c z6T`upj$u*Ilc9hsfJP3U&kgi+prktwzxcZfo8+({M!z{iY3d(Nqkm}um!B|RxR4Bq zY@p-qcZHyop#hptH%R_D{?5gahP10gT>2sjG5#_3joSW=UAQ_)5}^ctY1Cm+2K2!!o4J+E%!yCh>((GS@X-h@^x+c4dnl2t zb5Vu#o9`JrSILqqG*UPjVVDg&i8q0vZp|vy2uC#v&RNF$)Q3S#4M; z=?`bf=$B0b3=e>wKz7~>NN?b6VC-R>x_C2VX*~!nf|bYcNqzLKtQ}d~$v34&oTU*D z@YD-;P;!8!L0M`R;ER1Ihawd|@p4h4g^9gk0>(N3O!rt9(bWUhV+cbe^{~P7xvC2O z1b|5bG)9=E5vBpenHG&&n68m{hG_z64O~!NXOK!yR7JLgB22(ohY(DkRtHhT41;wc zovoqTmLkj_`M5D`V&GVi9y>Cy<|~{r+g3edIM~_27Axq|N zmUJB~NLKEUUr(4ZK$&q7EAFSaiSp=7KW?D{(@v zherqe;{xfQ!I7gAYu{*b@xhgo4(j>K4A;GPbTut+^~#4ae&>_#AG+M?%7N@3_gBAm z{NCi%hrb=%to4Htm%d*sv%}LaKc9myRsa5N-wwUJ;~v)7p<|cE&o69qY~TCBk5_JP zm*Fgp1q3L0LLLiGjj+yyU)mOfegxw=ak629j>V8jVj*6B#vl90rj2D{RA*5GlpYbvN_0ohc!sLak}f~ z&utstAUW{S2rcvx2J;z;y_QJUIhdu96jJn>b*|`}=ewVj4(yh5AmuH*KJgmB6nAJa zi!ub(m9$@smY56`gJMQq^LDt9jS8+L7UzBQK$v9X96BahYT~7`M9o$GE11Se!(k2GM{njkP6d%SabV1ee zcTEH=>=**Gbk$YD2X$KvCbEH9cVQ7uBfQfvz|clbrN&qRnq{bcT{J0WrVP;Piy5R> zm77IcKe(ti z9y>Bk3~wmwRxxZ%S#i#iL}ep9zU0# ze|qKEGm%mK_ut%UmySIe8Xw=;!^;kTe0GNS!f;tZ{~X!w*#2wdWxFRWi|)U6-UK2I z_d#(+Iyolw-dcdg!`sg4?;OvdU2YsxQf<)NC6O*&9g)&C#2sG00iDS?%SB@#Eww3R+21z$g*WIsMAD;%vfsaOLp$~5*?;k|6&M8Z1 zzo%%Kb*|{CLnjLVKJ2;s=prp%pLh*miaSIB&Ip7*sdoWHkBDNl!bCiureYQn&{{+` zD#T%>q(59Ii)3t(z%NO!9HKk#1*A9dHc*~AExc?IZ&)m?2cbnUBdQX5Q4K&M8yF@gpWTp&&Nxc~a&-P{!*aAgxL0bk5Awab)QXf=ZYGa5??A4)o6_ikq z=rRMA+KXY60>>FVZe*Gm4%T#xo`RkXd_Je4hi3*OJsl`%Dg{R&O7_Il+XP9_2M+JL z9O(JR${BF^NvHT{`0B#r%X2);X5PsFY%_@%i|wR)Wj73FQWi%=CuL2?g5bg zUG(*9U)KKl&?{%i=W}(PZh7xM*j;PNo%@rYKH+1bcMs()j)J76k==E&fdUYb4GSosyT%XziBf zn7ur}F*9a)moPNWM-?LF60v_8v&rVZvl+C(jBSYIAouOsAnC^G4szc)CZ|Di;KNLz zURvlQTgf{bk*ssda@y}HT5O#w+I{_%!iQZiKMYN1kJl$&1DN6t4fYWm9S_kXq8Kek zC61PhSx7)Dlx$Q8!%9hiD1n~fD0@Qo1oi~7n|lH24ZIDMrwZnNVG>L0L1>Wzw4pae zTTT;zD^KI%)3_VRp~|RYDOr{p7Jz^u*c?)uOF@JsC|OleH}fNi*AaUJ17(@H%n?>%Idc$6A3X#IBUB| zv68#ukkQ5yei&7kA}XORv1d+Ls^)@)BvxrIVjOIddIQH8Ja%MgJ^wl!tm$|@6!c^$ z;LOt~!5T;^>KP^VH5ka%N9x)NmN>sOS=Q03x6i~U;p!74{l_~^Uk<`^F{D?*Wdg-$*GeE3~BuK zq2}&#lPUO}njdam`uW}81-z-Beg8o%q?(h3<#$cP24;@A@%fGFPnNXpR?u^s>2sGC z|JW5${My5=sR1p1?AdEnO^e+F?hP7tsY~~%?XwIw?%CyOkDq%@c;cGRoN~`i_&UNa z9eXtH$xduqG0LzgqyQkl0-}gm3fJ@7rrgnC*y2_l*7(H81IES3yTpx~l!l7lXcs_C0*DI&pl=GI zpET|}-|U;~nSFaVNV;*lZJY>8L=HsHGFmJ`*|c2DLIPVV*{G0)m6F>G4(^ppU8Szr6X?!+ z0qG6A4V0%5UcWGjrS%}RNCDc=o5q+I$$&c=PS6F@>5`osbg(2I0+@0Een}M}7-eV{ z;jeiuER|Om(Agri0`v*5pNW+YJ{`3&gz=1;DtOE@>ftYan2kUSwHCvxTF@*-?Q5b* z1v6zpwYri)YNLfnD=RmbiM5y6AUaY3X||bSeV7eJR3h%MIovO68wdU$1IL2&*pX>s zI9SuKL!%V*WWYQxMtU}2zPM_7I#5zq!e>IQg%YR1f-r8EWbnS>E_zyr3t}InE=_F(sqrdrcoQ-I-Xlj(yOYrmTHg=WrggW+aw%DAuH1j&Pzm$K)WKDl{!z-M2z zeCYl|FGZ)B$4ochK5+BijPLu^ez$Ax4^yvgZC5-0vZ}w~_Pi|TtpkTU%4->x6>gixOMa1WJ86_((mXJG2UvdJLYL)1g(D-9*YP zK(RYL=FJ+Cj>VaKlKOU7AF&JV{#4IEM9a{O5jS2x+ zDd`W_peIO$68Il_0^NBpAicrd29dq;Q4rcFSR+^oy=m;}wXbi;cylL+8cnsP8aEa0 zMsk38Z&?cSMgfWw;nH#JL?pX$AL&;7o0&i&qV-+hel-FL4D*g&{%_%lMSU<9SLA*g98VeKQFXaHB8 zu7R;OD*g#Vf7!Ly5>{oC?<7&E&r3Ii-f*6=I(!K z+O+Sa?GL6q^=RcDApLWDxyy~ug*Q(Xdk?HtcZO%`SDvP5E*Xu};%fJP8h`Iv!8yFC zxBckV-jL{Y-FH4QE@*wr>EY%NALU;uoAkDJmGP$zS<0K6HVv))J#?||^q=>R*0zf4 zTG?TwvAEZp*NdcuSMBn&$ItnLmYZkpUaWa><5pk0bnMZnn)lb#oS1ai-=^agTh(q@lHc<68NMjR1GGNlWnXgayu4!+0b7o1*(ZuYgNaEzVqt+C%w5*4 zMc<8%3+8zR$kzC+k*)IdTV8<1iKx<@xB&=!cV{*c+;;?n1``wNJAi}Sw`aYi8>Z_g zUGAo;mmK(LfEN0QQ}a$lB+H!oJnhmHEw{{-xbIj}^x(l8-|ZgsHeR224Pc5pR5)9# zG&Dc~5yfb&5pl~bprDmcHYyZgq2!)Ihcu>1V1?dIunrjRycdw(!0SMH>TyV;crRk9 z4TPRjf;PAtX&Ta~)z-kX2iP>YqpYdXn^O4Nn*uORE|<(emPVU0PYc!X<- zK;>~OL}#s}t1PuvgV(rqC}PQ|M|7D1OU=T?Ax6Rqc-Fu%2G1RtMuvk`y%?A)X=5lM zhYWF5C%edzpg`C$3mA*&V)d*T1qJrrKPI&V&!f5xj_ zp7!{){92JVebngS&m)dqX_T=_ks4E|NOp}`o2p25T$UQMc9CNsN^=2P z1#8!(o!|Ms6JDR#jF{pM6{g|u6Avi~9E%pC;!J}gW+4GBiwz4UJ)?v?ehHC=U{9bs z?**hc@H#N|Fw+Yq&Tq*!5UL^$X&hx5{Q@r|YbtSrXDINs4>{;{dX~a0Fn|I8cUdlR zEiEo&p@#wGX6GniW+$q^OIVWwQ3-B$HZc4E6^1^;mq{?C4X!X)bFQWmw#xIUO%A}Q zoS9x=f);f|vP=tj)AG4G1N0_4RttEk!5whcHq>woy(LBTj<&>}Ibo?;B{{^3v;xeO zIR$Ga6NBfDOe4d=s$K@9l(aFFyfxrAgBVtnwCCXU&@kyakN$$B{VVdw-dV3TlP$%Y z`WIQ=zd_pkY;x?M65->K?c2lfJMg0P+uGM}pD;fC{nF!oC+C$_JXs09(4UT&d)))lhp%VsX#Lcv z`uz9%Ia{=bAAWl5wby9eTBQ2xwEvB&C#}~U8s1iYDEZOa^Q$^7`fx(CWjF2~{a|!^ zdpzy(bB}r9(>p1<2gZ5~TyKw#T^jdxSC4JCGNx$9?V=f}j=pnz9lZnRbPRO#?wyf6 zr=#CA6-w#8D2{{^aO^!zmQBEp{g(Nu7?e&-(h(bF0{hB(j+L+62#cOBb?iQ;yQ6o& zoQ?r#yaw&!h@0-j1)QM1*Kpr)3`!>^)VB%bzSq=Cx?#G}y<4lJ>Lmw08lZ(fBGkNp z5Xmy9zC^n;MK4=}T7}mqUIUoo4pB%FBjF0sBcd3sL=ktPshEWX z&ij*%3ISLsxu=j9rP6cKbJ!EeZt4Z3H}E=8o;p~glu0bLfzVU1o`D*A(`eH(bpx#I z44>WW;hQ4ZQN~f&I+k)kZ-U?*ou~z~ZNa=v8knH4EudM(4)M zu6D^4B+HL0ZfTmnaa}zFE@{u!_Kc-rOXb-#n>>A0FMPr5uB z(J13Wr|iwCPMNCB^Rt~Y$4yDvJU?R!k){zb!xe!H#ass!o}Q+T>5-7ipqN6AC1-PR z^ZLA2!M&oVjttCq%G#Xblo`8ueikauKoufQBVvX#vx((?QWzAI&#~kz4szfA^^$Iw z?&`3d!Cv)}10N00LLVNHyrU7xG8cJ*c3+Afx6BPae0%gDX~OF-Yv#X&*C#e3rnp0e zY4{a$p#f<9#b~hzS!MwREsG5cCHtRe|HEb|05q5Hbk>)Hl31(6yKMOB1Ybp#H_+<~aU zW&?xIIbq~GOy^~&uEGg_;l8D>DF)4Bs9h?WK=V*{Trijh=S>QY8P3S9?6| z^79xJKk3D|Uwc*fx6f;oadwl`%%~>wRGFdKP3Dd3U6>iVazG(Uc|^?fMIf)k?6pY* zT<6%%ft~Y=Ka9QbH}7W#;dAij{_FXAx11R*Hy=krK0zfR@FEg_15(3fF8C**mg# zuqTk+)C)*&;B{biiF_7>Dl4lDcvByHba`IhYBHslPN#wYVT1M2bnsph4~9zk@k zSt$yrMot-EswR^NBUNxVk-7jpKH=I_1wKZxhRmHZHTbF(=3a+q#x?L+D&F?1bZ2y$ zi(q{TJ&*%XENAX6Gr<6Atk5!(SWO%=Gd<9!C0(ei0-w-WL)B(ZMMEuqcxy@VJQ$)a zv1d+LYCDFM1lM7#o;7fc!E;BZk>Oxf$FL}AV<>rRDB;SX&4H5PjNl;nODDL01s_@{ zWCb_6b$a!iJh=QM)Azjg2-0R9_I^_;Tv``A(Ao2~ChvZoH6yv>SHkGNFQ@o_pCFYD zcZWPD^SuHAkw0zNl-{3O0Z`PB=`|H5B7Eb_zRZ@kCWoR&gkh9 zyzhRecLz?Y4YbSC9zVCOuXugc_3+;w`>Ueu(y>S5!Q_j-hL;DnxUF>lcanpvI?2Hy zRXsJ;!C~RLZ1vRSZiOhVc13Y*6gh@SGRW!G@*Zucsu(msiYOvBnregI{Gd&JMycm) z>YEvu=-{GGbZ}Uzp1Kqj*PvYiF>xU-BA6p`n)r@m(EJFKZ>nwKJD^_D4byd+8`{sc zUUJ~00b1z8Ma?@Akt}oSi}apA(Q}r$l0?OWBHflR5AC|S53f(W1~A1PqTnk=A{(Md zL@`<{7;ETxF$)Q3rI3vZOR-RT(}JGhAe|tcfIWfkycdw(!0SMHR>I>KCb858LQg3{ z8+udBvF$s=MKWL&X&pS}z?n#3*OLQ$Ey>aTFrp7h-o#I@Hr`v#(CUz7j)V)pb3daTP~WgRDYiPxX2~=Gf>BISzVy| zvK3+=1z6TYYeNla*P)0U&>eBf{Ib+`3>zCb#^8A)LtE^B4hO6HIjEhIHii-+Mq5J( ztxKB&rK*c?DF;j6ZwJfSJ@uCT`pxbxB|f=$Q(sx~V=<(zoli~QE!_3a$y<0~f|KL; zy-&Xwb3k}2Y}t*<)8pM`88@07N@=(Fq5QWe%A&EEmH#bz_%HX-18U%=eyDcAolQog z}^x8Pi*!KF&DXU)HuG8;|mW=pJY25jBY6q7dyY4?p z?obo7@=kh-$1lqE5BzkIOP_(wN8Q_V+{rFad;D4}R=v`3LiW{zmkU0!OUE9KKOQZ- zcx~IPPN7}mg*$IlwZh*89z7j=1^{AHX3mFuq z;@AcgIDT5MXzxktN#1Z%k4tcIh>lB8tU<*!Xcs_C5{U~J<|u;uDPT}o9LF}8K<+!B zUeXQI-MF%|)VE%8;G+Rr=%c%ucQhhd=F}JIjew%(EORCAd4F4^_g``?=yEDvpLh*m ziaSKXSBwOtByj!2Xa$IPcbbY>NI)x#Y*bi-g_538PHyV&OW((yKzH5?NN?bEpghS< zUA%+UgV0mrO?}**)wwIu^Kf8ICH|)+8ze9;9S+qneq2lqTp*tx;VJxLh@FMcJz=o9 z&P}Xr-}8vho;ML!zxL72OmwcPE%&DB@BD zs*{Sa!c1DMbmC2{kjUq7T`Qns3)9r4NXS5m#GW}}sabS5#7Nix>JW#btd%sJ!E;B3 z#{Pc}2P+sIC~0FTA&9g!lyG@a&nOv+U~Fz1;bxPQf-p{y#1#GDl9+>ElnNO`KgT&> z6a!D5s1l-`mW^IklP>F^l0`V}=_7cBeEqUi{qH6cznz}L-H*e&(|tEjzci%By0qle zH%@&2OWHr=)gDiK{2Yg_Q0I9b$#?1X*A2Usd=_J0sjs(6+S%88WVW<(x4}`~Bhv;K zq7>$ZVvdR&qel;+U~9$T>7WZI09xlt6cOv-%0XeB;+65khHUN>Aa(YLe7w=)O1@9=s_H%!;5!^jPV^^yZ04bVa#ut)~$GDNb> zMINCiDOzlq8}e`AX7I12+V5uh1mN|F&4?-PP+`&%5M@ZSNd3iVv7nJ?=0uxN(6ZRD zP;$R2CF9g@3OBJQ(4F@J(i?ak7<*U}L#!1`Z6Nd%^fi3^GTtOHt2&*Hbc*X{1MG=*# z9@sMjmYT(aL##+Ez)YD($y&)2r00$d_4+@DgH;_JC~0FTA(Dd*CwWILK3tN-lf4IwG+Ps#B5@VMOL9JGO zzc{qu1?jR=F(Z$E>pfGJJp7Y`-&}fW&;2id`dHcGZ{ri=xu?DVsxcas+iL@74*YF# zBzgSuprko|S*i^@UT84BT|GHte)G2sn?Cp}Xx)xf5{hHt1RUr03MHr0r;h40TE(DHFa$-!M!0gY z`SgWTT20A~8RHruPaNVYpSX00dju-3L6taSl1Q}S1ocf$6F+eb3I#*v+l7PNw@baG z8>Z{9X~D#(ddY#0256y=2sQ5?q9pcVl2ez_E=_eVSmsLlOxRUa(K}mkc6kx6PrL>& z#T_clq8tU$BlQ=f6(9zKg7d{JBmjhyjS3N1DCucsWIpv%68Jh}KRklLU66>qfTX+* zl&2ORzc7iVHV}GB3EJTAEz^rq7y?+OhtFeRQ3}{mhBtmVj*zBWF7Q)u$2WnnnkY`x zVg=Yh%pFiGd|uAV(Tkn{tEhqD6crknVFf0T#QDLhN~`ooK(iRNK_=7=Gt*;C(6UqT z;D@MUMSv|S%%sIivPR4ua5^R2Ju|dH@dPNL9?@k6EH#S-hnO!bz)YD_uvRh!>A54* z$Z)XM!B{J4W5DAV4Q&mWFhE1YijuaJe0=m?Gq{6qm?j+^SC)|E?>oZ82 z+&y_^w~%?OHg}H4yNbx`caO~X=q>bZwZ~<1ZhHr5%b&l<)E<(DNPqq0|01W)|5x+r z3C+4GTjc&TUM2Q;+T-W^_SGL_qNm>+^7pcoMj7MBN4~NZQGT7WWh;iwa`o$!DX36V zWTQ9^PQY={tPK=Q>*_Q{#h@|7M1k12R1RjkXH0Aolnhf*`OC8Wd}J%a{W@i#M`u)t zCBCzWHk_coDTsbz88n8NP~RyW?5BAp~^~)27jr)ee08T*ty_8c=t(Wjm4Q`VI(0r zz+IcAuzWC}092bvRF#16$qjr*1LOIamI#;}R5L_pt%P6N)HQ^kc6 z^YRK5Bz_C!#p^rzetsezt~trI6W7ZjZCQBhvt)@!k8TM+run?k?q;rG>MP$1Nz&6L z|8d+lT0Yjp^VE#+hzJMyM*k~cb{XQg@x`+5zg#xupYdvsr(J&S%8&UBy!*vBYnRA2 zG|D*Du`s#NF|RN=&&x3{CoM2JuWd*aN`0bG91JJmxM5bGYyyVG$c7OegjObTte=YM ztxQwH7x$?(ium9ANHTmcqPR6o4h`Op=)hLqXuqYz0`@5fuvu z)T*M&R1UZo)!-2_EPVv$;O`JD1qy$3s*hD`VbhjkF^9#nSgI-Fg&*ZoRk2B=j!)DD z3?@5XwwVJ?hnIsDut%{3ykh~R5LOaPEn;DcX5#|Kg7n;xX=FH9>tI-vv@u{lbpwqO z46YKJ1Eq?i@Q=>!l9Cob3&NMq(r=bcdGR$_E#A~!zv%iqq)qcreW(+*J@i_)Z+b_E zQ88~G{QE_f5EwjdOhiPB@1%=9bQ`0P?t4R)SpKh-7c#OsznfXzZ~Tl?|BP3=Jnivo zc5q~5lX>fAd*n%FW-^38rQSaYAduAde~}khmZ2%05cBqlneAco~khy z;XwdABQa^gsCNXcRcc^aCDAv`{z{eLji?O_{<)wtz^EKJSpf;$OV4U6%0crIYM&QP zE-+IrXvulPzQ#o44-cpBF@Nx7{+hp$PcVGJRO zPUz+AcDs3=@a^!qDXw>>Irh4@qP$RW7lzih_E^wQxF%7ND9!||=cbonmTRG`Q z(yg<8>hCj$z$}fYY9IaiqtVz`_2k~-b~jrUYjO0;Pp9|cgkDt-JRVLp4n2^yQqyj; z{*?RDu4?0bmabtp_#gtJ|aSG27YZPslj>rr^#W+-n zBPIZWZ+ZlqMDB+}Q@e9)j*GSLhu5x>%&9MkCrNU_ zGFLKuTToHeDEZykO**_j@fyGscc?IFiI_#R3n6;u(P9xQpg|F{kOYj#^d}n?0+0&1 zU!5n@1UN_?uqV)+_X5%zOmz_H!kmnv#8MjwJ*CuY)zF(JoK4?AMyU$|JX*kqE){T> zg&h`Ug9A)s&Qglvyylf0f>{JonagIS1m6uH-rHJ;VX1pZsfFrsKR!TZ; znGMW};u%mvJ)+ADSW3~lF>GYuSdgAOGK~xet2)M0NgG26!KcB5fs8uZ94J+sgYW3x zmTl+=>%B|~RxF(LpMIZBO@zx&>b<+^U`SiPwRXxrN$7>nw~VuAJ0Bdi`1+%nvxL;4 zgHG-#jZTp3OSe8faK59?;ZDr@^W*PIwomx7{QI0W)v)r>&L{6)>u!9Scj(co>_I2S zY|e$7dF}NW?GKRtu`?+z>gmB(uXqX{W>v(~um+aG(6?!R~IW5@oJCBuEL zH8r%`{jSfhwol`>MRXlJbB?n;o_6_pZlAbs!wez!OwemC_UPE9aVzel$aiv$Gkgz~ zRTVmU<)k{fd*#dybaHo_law>Ne-|&5CiI@ zmvqB)ht12JpIR?D@X-J*^s!XUI~tKJbLz{q-&6FQWv=9_4%d!IS_h5ndV4fppLh*m ziaS)8hJUjf8X&+tS}a0h@Wd=6pyf|CDg?V?G9zehmP z3eZ_2x(ju!;2#Pm_wcTnR2FF0rHG`^me?~VETt%^xCs({Spm-)IL6?)Bh$!mu&SQ} z<}gbm@>lZKAj{d2Dxp+?@BDj6?z_Dx2#4IIf6Q&?+3mf@aQR6(u7`PH1mT5AXSci| zeE#3-jyY3P98T!AE5B&^ix6^q``GG}GxkY${T7pu{bb;uvM;*a(4_q3{GS1_E6Zp7 zyC;l(IrG&0!S6<6(EZxhS$A(;TCTuf>et?F`Z}c5`L8D&H;(M%t)IC$&Ct&&ZvP0a zxA6y$xKCf`of}m<{>9x}X32_LKN)ekgX_wcRl&!1q;$=WI&GJyJ$|k~ulO_iT&-{V zZN)CTbnMa4^jz9%Z+GdL#U78|ijrp!E0nMH8kPgccD*3Yec2187Evg+N+QRJomwXm za7fGGu^gJB;#dMPYz2B#Ml4dVjqf;p+=9Xg`I=z`^3|Tha^Tpm7o@o_dlHif;x&=^ zPUL<#G$oE>3B=IScR{_R8>Z`?;pE_3FFEkh04?;?#*b|rHI85TeN3RSZY>D z4l!R=z_SL9F?jCCG%_4ov0$vRk~RiRvs+DD1743BXuMF;9*0Z$HOYY%urkJ;Cer-& zIqlmy?ZunA{E_+^q)mI4lx`8We%NOJ*8y7{+_G-FzBH}1;D2=4FPnNazaUlZXtM9_ zue)Pp$`Hes@4cNYiJw&T#+}`F{u!@!dD`RGtnHZwnSAM)zPYmG9Fk_U#Q%KK-B zEQ*r%PfIHdSu`w6h0=y36vz6KW2d&`KncM3;PlK?1}zNa*jyC{`%dW+J#wxjXsvI! z+&?5m-ajj3Q8+5*qe?6>nL)IDm<=e2zEc>q(4S*-V>!ru`_xOiVY*}E6BNGnk^>(N z&_W+Rk-QU8iS=QUi!7#Ds#9#48~R$4OTn6`T`os|c?YjgY(`9RhYClDm4?8E6H$zo zhlq^6c!csdiY+T?tgXfJ*Bg4U}egZ73R-nOD z^5#IP8o#3plCEwyNf2s(lifI)-Q~v}Ti}|LteiF(UKa_j-fs7n33u-QFl646SDX|t zzjZk4^bX;z-Vq<9FY}lv%aAzE?Js%pocwla?*o@+Zd)zg)A@Kr%ry9-^lM5JHRzs?}{=j{iv0a;x?>g~cO z$xroJrB8o*{jo>?lf%2^OjuAm`pSSWcW<`K(;mOpLhQX^w;oh}o$zUzT{`w?{9)YT z{*9-bOW2mNDM=1dlM5Zf3Mc0!IfQNO2x1yd}>`mitPwL;I%Z`m2kv~LCk;%b4zF6PD|GNl6~Hi%{!&%VWpG%APC+I-&ae^~ps69} z32RvXxDLhhb%82t1k2%dWOR5r@KqAStSFX)A=(mq=7gm+TriIz^HsM%lQJ*P2E-XW zcVwtn>&jarHp9URJ`1Fjv@w(ravCLAZdyZQh*C`%jLz*WS<(z1wXAOAAddT>I`VA>-`OHwUkJK{42;H2U-C*+QRB51sFFtkZ01vu*DMuKa4o zJJKuVSNp!P#F*p0WXYZDTkgOj8CRdCSHPo|caPQnpxbhP;=+ z^G;}Qm!~~`&VzI}g%J}rtolmsWS5RT8dX0!y=44i+dB)xAJ+uRQu`OmlB4>sPn9JP zo$uOz{laJ>IRdmWdR`JaP9E1X5KlV}30>YXkU_nvB4YiC;XKgmJu57EL2&%YAtBiT zvZehCWXTcz*Dpn57qlaiBS7FgkvR(Bz5^K4n<^6D#L&|BntDk$Ojo{X(`2uD$$^gs zXrT{Sc$)QEB3b6t=jlm`Ua-uSw0iS!(SuNJ+UU;`@cP7S08`u{3Mw%YUJyMZiqVP_ zaZj3xSxA=lU!OuYDlEl9Nq@LR#;GrnEWw^Ycisy~Z{T&HJhd8F5<{#NOKl+ZloGU| zH%&6#)M1Gs7#5Bj?l9jiKIhipC}^5;L1U5(W}*iETEH@*OdTJF6TliVcOWWhNZF)3 zZU`e8feK9AS_woNBI83ffIh4)TXOM9@a&+HN0f~p^y%K*T%2? zhrHV3X^)>g?$4fU-S0hZv2E0=b}RWT#$J*@HvfV}uYvP5{a#s|a z7n0-J$(zaP`iikB2m>(18lZ(fd?I=OAd+P+vWWID ziXOJi4H@-X@8GhIOBWxM&d2MMG$W?4LxpMh*G`582q*OyqcuYehGiB|(6ZRDP;%ob zhHF+LWWxX06Ufec0qG5yj#Hm({-PA-2Csw-RvLn**yNx$@D#?U z015#7Xyy|96P$s1n34;ARa*nr2>iyYYXy_*+Q1BkstWjb3ru0FgAbcv&KY=OYd8X$ zC8&K=G+`+`^kzO#M_q zBr*>b=7&Ab& z|FKJ-S-}Su70^Hl=}LjI5L|YS0^lpB4D?TF_WVhG}x76L4_Q2%Ig`nIaA%IB|KBl5A%_) zO9|`hkBa`N5=%^ch;|mU3Fm&o88q0LV?p^GNw$~pYj%$wBCpIIdxI=|mK%r$w;6{qknjr=Qq6*Ek xTFgQMS{54?N_qx-ux_IAVsSSjlf`JN={|~mr?oa># literal 0 HcmV?d00001 diff --git a/data/test40_data/example.dat.bz2 b/data/test40_data/example.dat.bz2 new file mode 100644 index 0000000000000000000000000000000000000000..0390c0263b09acc40a6db6b2c39618bc5a87b13f GIT binary patch literal 58013 zcmaf3gHt8m+s`&{*3GtSvu)e9ZR=)pYqM?J+O*ki++6SX_ZPgMnS+^U=FD^O_=xIS z^RkNQP^sxaO*}(_fg6ASzxZ=oTx5HAm^h<}>I{bBzS!7i)2Lpa8m7F=mdnnjr*3J| z=8&CbQwvs6C7;+j<{$~epJYu2td;%#Ezwv?Aqi7St6<@Pu&k+x+$I?*_Jdu4lXOGI zrkY*Pf@?^m)S<+Vv6XHB0Fp^SN@(K(8=K;wR6_y3^!43h`s+8ZI$~HMs$!>$0L_6= zOt;F2RU~>NNVJRA0V^6hvrr2z31$xe--_+}Mb`;nmkTJpH1HveoU$0UU@$GM5nvGZ z>H`QB!u2+cd?YQ++bcJ%V`$QbmxSP1Z7U=Rt`;DQ#HY@2qBxt2^j?*E+KVHgYM z-F(M9y|lLoob+!_Juh8;)c*iuQw_gB1)~A~0R~2Vqf7`!swzqsT?R%#vnnf1ik3_c z28IWMiA{}`r4>cTfR~P-Me}thA_WHvmL&y?Ri0irD~k?}1uOZX0u;sIb|4}z0|N^M zllei=Q9MzJ5eoc|7r4-Jgb;UkLNIW3Ffbap@JD4fbx;U2T{0F1S_+hsGZ+}SY(N0K z3aMl;7+Aa{)MQGy^}@npv^56@*dWIO*kVe>k7cUJV6X)aWH4mNA7C6*U|^vZNtZOqmEoDo}x0 z%P&CvR@pCG6;KEtqM-x_hc60W0OycQlh1-O`w>!+W4>~)Bw3+m%O(FaP36qgM&Kvt zak27=r9f>;5dH!dgh!lm&f#klnqBEDKa$O1E~q4Kjt9ULvjw9Qhms-3myb|d=1_m) zN+wPzB910dd(7cPg-?Z400RSi`mcc@gMs1yZ~PD7MPT47%V1#t?dd;Ifn|Y#p$9Mg zZw4u949@Z&K@AKHYBKmgqP#pr6!U7l`4C#M8 zCYUwUGC0RTJbWn_RXo(PWCb7xDMDE0Nd9N4CwBvbmtHjA!xi}Dn^zb` zxEE63-wo02u&D1qv3%R%C?&%)=ck^bL}$KqeA0H&BhkfhPh{Uqm&Va%Jo~Y%0o^N+ zh}8SsPs^H;Ro2AV&k=^x3H(>B7vad9~5Ea zV3QNIh}`UYzQ;C-)9LMdZfx>c8lZYf3`Q+odbu=qgrrJfxv<9zXW^9DUAo4nq>Iis zYr{ooNy(SC2^tgyZ#DFJ3cq*mM9`JdmEb!}NE*evfZutr>`jBmsU zRD8gM$B_Osr2FRk@hI`!zdVDRtMJcLEC+u-v{<`wEhw^3?Ii*=TW#1hN}`Ed0rqOc zRP$kCcsiFO?`=z-3dQUzUG{#`=W##Kz8U`n-VL}x5&ODCXBxY-G`0r_6loY=q9b|= zni7tqfu}X?(2w`s7D=SGvcv4yoLO!?3Q*Fw#arH!wGdyjPHb3+ABb$y@Yz1Gx1L_z zpmQHK$4>4VIG8^OLT*3!^(!`ALl#set3lUak?v95s+unQtA_lXRz_B~th+oYbI|)& zCN5Lq4M1L<^qb_1>UjC}f*JRtR<=eqUSp@O!ts<&gF{h8Jz26%94!u56czXPSyo(t zKq2Np2~I2SGuKB6x9?*VP)IQzspdXzi?67iKG>C%pR~3SOB#kab0JS(BoKX3X zO4|D%2(5ZQq#nn3I*SvL1W5fAp2{Cik3lZVhAo@RA&174+&RNK8DrRdUp$7zoWs)C z%>2rK%+3>Gzf1wom$zgh2?Y$O+sT=(ZDDHF;VR&R{W?TfG$u0x<3uP{m)^$?CyF0#+? zlH|S^*m;>`LD^xd0=Wb};H|FwhN@eRkVsuRng%LG`2PmjlEv&hL!&=WQf!Xc$BO&$ zlZ+J)I0UkVd-waXIJq^$_;~?Xz_}As7?TJvnN2JZ3#1g4bk8N%V$H-}AF(x&dEMvu z!jdUS#@YmRH3!S!L}7)w8ndy?Z?dbh63)#VI(2I5JS&nEzhDbUqv6n;MkvXISi;Gn zfTEElUinnhOA660^*U|Y&OXXFo!`M<=jbr?s8pY80)yb#5b%1fv-5%}aS);!8_dS5 zyq)O*yd%k`VMk~(q`9EvFt(#Dj3sgSO8Z{z{)cm{8TdZdsY!_84bO%4oVb0K;QKisF}S3k*w zN^=na{Y4cy)1_NCwV6F&wjuVRCz>?dXZUiPiH=ZbMf4E5ue?|(d~)ug1}xX;*37Bg z$IMaEfVgS-DU|cccBx-7U2MxP9b^5sVF6kZfl>biPy1Ti?T>CWd#8!J*%7gaX@+^z zhcz2}%F!~zbNIz8F9CgF(Qeh~I3(^L7oxnW8j`QK4YKtjTY{quso^E6K==M8U#(|n z$H#r5SJSg0Dyl6-V0n8--V#Pb;k!TZY-5A1AT2yWH7ygAUFeLmX}C~Zr>CJbq0c+j zt+dP;Lk8k}Jck#T0s5A;Q;$8%l2yuq8vJWrv@Qd^EWaH?bctwb5+sB3KcA?U&8_}0 z)Q3nX3{n(}lqj~8@N`ntcvO`>m#YhNE4&KuU9ZC*T_e^%X~>Y7Xhe|YwAuXylohuq|RFArNTrIwK%LU`nw6B*Evf}2ic*%kk>PfAsG78>ndRy^YjI{Y;GbC79kPD96WW+qnc8zV7CC0LgvCBi0Uv*LYgrw9 zQvG$-$PExj77UAn8#Rnw61&(>bF`$Q6X!u4V0D{K;MHk&W0S=hPgUE)R({eVuqUKC zE|AIdkSepM^UlQH676my1(mvcDWpOA^o9j89LUJv+C#*Ng3mz;10-b|qlHDgL>`Ak zVvu)mpMS^>V^~F3UUlFp@-`O*j{AvLQ3|6g!lMh9p2R|=Sp9;lOA|%^Qh^t*%KWOU zdL}^F-Ysadj?b@gU|wtMk{FVsNoi5xge3#oL_w?;i$RBg>TC1UE%Xpx?u-yN2^>`A z;s0?$!leMM2gJ678GbCS6mcTZpzwc*sG%1>)abl7GLBISBQ{v(#(@nm%S})9SHdB<$iE~p? zQUvu9r<>_BL5y?qGox?i z>d-FZAq+;dk||-zCW5HQ#D`K|-i=}-EqhyRF42Xs*{{NrneJ%tP!Ll|*PnBSAFk&sS? z+fr52qc_{QpixoOK!L!Q4ylum4SNsTI-Du^^Os-E-+_4avtdaqKp9rxnuW#NQUBPX zsJZpNo8c($>?#dT0r(>ddb$&DHCkjA%@5hb2ROQc?V^vYfym^)++dK+KNVX({VOYz zkw5YK+3@$vHlgdM?F7Pz(-NMkObGaT|643cwJ@NQ5dHK;vHSK}P@<;cxnJ;d;Y&OP ztDI>eX=Mn+>uEQboZc{mJj`P!Vh3BD_hd7g{r5c>%?D3*((08177PQf0)ryNF^-dI zc){cYk5!Tw3(~B>O(C(iJCe+CWbU4+O+YxAp%6v%=~{CUxiENHO0MtlljJjnvY;5Y z7(SAOB~d%METI1%VbkfASdH)P-^S9mf6rgtk%v8mb7#&1Mw#hK2;GDDJ~M7hww~AW zZ-+B|T%!YUzue3mlGb7ZJm9dEO_nsK8*wKz#rBXw~lZBq?hdD{_3X;ky-zu%KJ zmjAIQLCM9-%2Kkzg~uR8i(>%eplOG&sj&_c{zGP;GsPo3T0+Q!ykN9n)V4b!t#rlH ztB?q%+ZXHI*tRa8Qetije!$pv*XBeyk7#riaT*mvJN4wB`w_3yETcO49k~K6)lb%r z#`nQ!QuA2`67j`n->K%cLzyITOebRB`id>8H&7ls@P+#vDnOKZfoi^hn6?WatAb2Y zI(e?9jaF7I0={i2LsAkwTskH1a^F}7MijtM3o>cCBaYmkumb{^+qw-$gN+GMT_RlO zhsM_qxAF?j8T&orDaNW=dHLm?3T&Y~q*^!b1}GP-r_leYyX)G9nL5akHfS5|NuDqK zPRQY|3o^O9br_9YGpi_N3hU6fqTLt6j%A5=rFhB`%#%%yZ1WI4IXhi0&~<<|xFl3! zEz~+}w!2}2c1vRXt$c9Xsn1Ne+O*^=Rh?p3^jEjP+^=ooXkRU}dpeThgoR|EWwBZ# z^1wUvJ=sJLnUz4lLrJN@dfuw4WV8!6o85YNyzk4YtOmBmHkOa-0ZP5_Dv5#PHu`j< zWhN=8t{13e|)M4-{}YzXVp=0%b$zZP5DBs zZ8PWGl+&b}PS5`Ow5Jf04YUM;n)C)vl99^>+mw|Tmjo9P5@OMOZ4}Ow4oY&E4L$YA z3Mgg*T0oi6w!nWX(#i?9B4g+%n8P>ujyxwju}G|ql-{5YE9a3$W}<9U?aUW)rg(w* z;j=UO2su>F(BoOJGQz!mP;=a+tuFiY($w^v*bje0XH|(hS^pvHa8XuDr;%KuZX9OU;)VniZv z6g7h(ls-v0=`+Ltf~Cj^+DI|Jk_qXwUo4hwPd{W74%^k=icV$ir8f|#ZRE+(ibW&P zCak6Zr@sjC2Uk$)UWZr!Qz@yq8H6AhU2B`*-9r3ZK?RDlwNL~0^Db?;h)57#31teGrPK^rmPMKLj%gFiwSWE18KW^II7b3kaMco za{iW`gTKi9wW#|(&DLyP8v65J{^2A?n8=Wuf0&Zb9-{{bUd?WU7Re5y`FH!#DEZrL zOn;g%=X7(-)vuN1US5+7hHkeWy4571YfWKPRT0i@zQaA2G`;Ltl8v2$fPPQx$Zz`S ztj$}6g{*E%$Quznq#|QII!QS)mb4YrjdkODNE#?Jmnuc?Ma+mvQ&sf%n2LA*j~;MX z2qs#A`qAG0g!b?3M2ONfr*>JMO`d?t^+YhHt4mYOS{)G)k^iz(nqu@J8<`C}Rs+kT zmI07vVMA(?=EyPOoDnqx*cDAPPPyyXtjD3{*he`0=A7} z2!vdtya4$O;V1m~6i6Fp30x)~J`CRipOvR?h3iiniD#Vq1=qRWxX~#c7?Bm$D|Sn> ztkaASW=QtSEF`yS!?eH4_C}qc?kOE;g-%Zu3EQ2xC(_y8pBBDv9sihH1D<|edgPky z+2jwD;2(}FjXPf-el*Of{{s*rl^0rVa3PnNVaOaC)lS6lwEe((JXZiLDvHugHRG5j zsyE`$sR3mGgUdB!coYDgEaTKvS(C(^dvN0)jL6{2BcIu(0*g>rr~L;L*6kP27r?96 zZy0)lsx%b0h~qy@0*>4z_WKdmhw-OjkFPUudqZEpou&R9KPV;_QVscrwI7AeK~jSv z3Flc|g?L;r)6|syM!0(DiBB4|>T-2*8yMn*(;7Z^9=N6Jf_7*cOb zVpqE)_R0J`e9mPTt@pOx3#|3FSl;GPn8R=*6yLh*YC~t}HEUf*Xf8T3GolS{?lVk! z>Z&bcTIw&~3mR{B7@6j?C%U7Y5PNr3yX!8~r(~EeyCFkBwVGAH5mv;d+dz}MY_m!z zbIFG&7(%Y#m`$vTt1=D>$?!-kxREuf=4Hr(f98Vw6Cvs0qbi|{qiDFUUwouxx^Y?3 zALmp4xf8+lj}xo{1gKG<*3~BoES1&9gd|j2``YxTMu!9oV3ua)xTDd3_6m3Ou(7|1 z>te-l4+H;lPum^*xA5Ni6!(j~V1GsB(U-_yY_wF+s<*V@t=Q{IMSLkg1q|DN5Sw!f zm84HuKxwb+N(h-n3p1u%g%c^M^-4Ir#lV1<6;Flsjv>pUf_EowVbWpu`D6e{+tKz5f^>-12xL4^*P5~yrn7@ci!=A%0=loT|uLI2Zs0@y7bA7Q)KP2!hEVBx zgKH+&Aoo8T2>{HWB`ZqC*EZG~w*Rv;z#L+p7g#r&rJyg9-x}X>A6_`Y zhh2N~s1g%H3?BW3q%33q0Y$H)UJ^@lwp-0~g_vfkO(YblI>ItzhdvE8X|+o5GF}T=rLO}NPyrpo22T{R#4rSL`NG!FY}cb7md;2 zTG}xvpbQ?6Rh4n=a6m5BZ7(9w&v2k>DJ<)N@`cKV$}_fR+)h0(USJ@Z7LC$F2%aP?C#W_fDS`&aIXS9kWwy*p3@n&MgX3?dgtRq6}yIGaQW!W-k zQr7N*XWRL&whOQp~^Bs-@=cliCkhX+D37k<>zWN_j|}RQ=4JD{U1ND_0ph#ziYB zJUzYEKa880W>!&h zHQl6?e<*bDxWg?Y=jy018sR9#`WP!Ty0Vca2Br~zMv*g8#OgS2t*QdBFbx7CA*dy> z9%GrUU4@bE6BpuN567?&YJSq%0X#LLoUfWSb=K3FYW4jE1)Kg|dv- zQks38WLoHgwCGVfw*n-M5^Tl*%EA1EMcVm3Uiw`!A#hUO-Pu}4|8qsikiy)}0HZ7J zU)>&2w>F4{%_K>yej`?OluV6YuCo!<7OT~&shR4VYnmF>Iy$zdWI=i5**X@RUZgwN z)q&-=M}m?pGH9~Ek4f(bQn z>uoZ*aGs;|$@&bh08g%hf<^IUyAk*6(+z(;YvRa)erX4o5l6S$uRU~Mlgd-(A--B` znX?|K>t2`akhGW}*UTO$&qiad-M&^HUbfe8pZ*$Xm*CRCa%Bd%M{}tEOeA{Xgega> zFOgvzVHiCUOV0N$fY( zVaOz#6M}EU&a}k~N(E_y^Yei=Q2(>sWvU-qX#3@SF6WhaWT-Hb3i{%FBN49Mbk9-htsohH@)usS@-VQ

9xT?QBHg`Fyga(zUXq%klR!QkH-;MFE^8zCZuwU^xB| z8pdd4Jpip^j;}>G*ueQFnF{$HjliWg;XWiB5yRdC7If-}Qzemo4td6ZBK<@sYl4JC$Uvp^qFrm=G3%yTpkT7mKSA(TLfsZPuH1lIrvdEHBr@Z z@Gc9)>-Ss;S$Yb-YJ+gd0OFKS7(>jaX^^nQOYWbJ2dCnXiO_pY@hJ<0=SEZW4NQ_7 zo}A`De7}DO+wY3H=s1&R=yUX<1hMX#-O{TlyJ(l`R9N$*O*m@xR{c@#g5&soj(F%y z0-qCkG5{`bF+(llkUJt zo5{x2(CO#LG@2=;P)eXGzyGLh=8EW3W=d=23F* z*Z%gzU%M2PIvcOYfuF-=LnD`QkZEirRxLKsXCN6TZ;L6O%)1`f_}2i5SxaUuaWs_y z5+K>)%+ylOl>s{bN-IU@ z1a;N*(z8mJ4Kl+6V9v9G8YI-(3uWQ2y~fZFlwM$opIuatt8SX)vv~SlyM=IWGE=)v zkmR0GqsDi<-JPjCU@l|IQ1_n)9A^C`gEs6E42((@$e@s3JnHsROJp&kr)vHsw};(1 z3mb{U3^lNBqMuY@{-(S-?2F)%pnM{|sGg(JtS#K5l011|s>5N$8J@tqz<)bShR?qPZa} z7wuVlOI6DFMVwCRzbzm4wQ9y|xE29j7bc44e@Yl}cua=;WcXJGi0SJZ@Gy1bqH*4I z=!>`d+l9_9{bMCOciq*C-CPmi3v^Tasi%Zm7lLA2f+su|R z3@%90e1Zyiiz_eG`E@y7D*#JJYuy7Ee?InDhXVtLqw7{nHL9V&1&76HI57pdy z5ZUnBwkP$u3MEV3#D{)CetzteBU*yR^S%@EKFOnN4f~2SqIL5lBaW(!{OeKs_}hI; zFPqVMtd$CC2qSx0bYi{y(Z*RgoV2@r4*y(6){<}W9 z^m2byrKxzNGhE_oNYu70Biuqm@C#}K3MqPo zTRd3;Zryb|r1x*Msb~CNET{@FYRKMPnq!Wd~WZ(3U|EHIKaINw}i04vNvK5HHyW*Jf#)wqY9^1 ze}!5oFS>)HBY`(#Otx2j=1nz4mvz0YQ4g#Dt@kJuVBWtRT9e|0B1;ni)z@fJ@v?rL zNVeL(wj+(EZCEhX=9npMp^jYc)QEfh7;SBD8{h#wH=H=VzLSBWO3L< z>G%yBwV1*VCrUZ|k_2AEnwg8TOx)4Cs)dwjoQR52rCOP0m9Xt4GJr=lf~$@C{_;RG>DX~1tLLlZIjs1qobb4A_}-KKf%WTuJ9s?vI*q9ADi#JK!(RX`YJ1Ka$6g8_(ivpm1i4}z&@EQoQLCsM+;sP4r`owE_ z774V!Q^bO$!=3QaNtX@%h;`Ppo6KgutOTLeK%Y@5r5v7PW0i^&%SeX3{}%JWd0_SW z&_YkewUacnAS;O*L`lP37W=y|AEb=0Ar5iYBDDr&$=PU9$tW=ufa~U=3d&vWR-W8e zieL0iORJpWX$sW_L{?b|D}Q_N`E6KFTlV2{RH&_2Pdg*;cYf_w*SW8#_8E2&Fye8> z*+7aJM9}JNnm>ajT^0bukFWk|&jc6H?y}Il^ORPw@VvY#?Zx;r@r5CIDE2JSFd^Rk zXc}@zh-uPgLY5I|Y0FrwCc#(q=9v9u*HRERP<+n&_#j~s?O2PdkjDInj!1&hLBBU@uPb4237)uXjB9+65&cV-z)t&%; z$WLz>PV5l>!jetlPM7P9zf(b!tOdTch-Crp`6|Ofc^iovNgSU0n+cC9@miQ{6od)Q zPN-icwrL7nHD-V+e4IT<{uuR!dim$M?-PPEoZNP$1)a$y=yqB!Z+VH|4@(rc&J|(hFHh5i@Latzwe602GI0Q#5mR1z^TyaMz`?SZn}N+TXV5 zXyZ)pL-bCdpHlIJ6kIJ&-4cVeqSxx?L>^p=)Nb*)|IWjTgU7bb4Q}}8kcx?2b9lLz zEOE40AU2fB1+U4DUX2nzjJ*^LxiUQ<9goVjxM2kD`@gy)sYvYiTec4zZrBNs-qeLt zKii3q-*0(I9CF-$qnJR|y_UGy8cEGkjF@rk*T?;gPzcO}!zHFvul!3XUD3Y=2FGs1 z4XBmjtuKim9TBMLMlhJ0>vu9|9hL0n##+R&?v|366`pZv9L48|wKX7=vnXTV(AvhS^^6Pk8_N7fHsij=PYWKjw@c_n ztEWidFnJTC2By!ZrziyH%CIk$a^4-@IdsH_z?n(7!f?0(V(%~{GsIlIU{e7H@i5jK z83%dh+)@okgvD*Q+b+J86RpAqNEf9QCZilfcP&!R`(7>?Cg}rmXS=ModntrG`zYx( zu|zJ;u-+#p`N?h*KL&^f--wP5QF#5zHu6|j+@q}^fDxFPJ6>ZOm#yV5zYm&XS5G4JOW6x~f#8L?k77J*`z(WrlA*GUWuje`s zX^z4>P{Y%`W$&Ux{n~QxZXYyo7)&0x%+6#{#0rx~Q74`^vu4(iQi3wc3@i5uFp^2b zv1}vmc}D-LTgCwWpRPzB8bub`zK+2y#nXgJ6N#I{xd~ndvTn#L7wvk@Iy9B-Ac9}E zW{ih)0~^4?HUiIIuO3t(#XtAYo#SGiPB*;^LuDaebGN z#Rmn79T-Ff3GlDh-K?^3Xdx$jBfESP6tM|vG*ewlsd0j*#;_O5v}ZE-%9aN> zga`M-6rW&l96rGHV^Hj5Dpu8`a*Yrq?%_AslK#2-Q$Ul+UTp6*8k91J;R0>mjgcR#G+F(@O38Lt({vgVL{ylLPBKb` zqdza&#;x>r#J9|In6Rw80Sj|nAm>O8tl{-e*e6ilOye?AYvjL?`B;CNB}{`lR$jqy zXSR>;v@V{(uo#gn6IU6df)NQ%VZ1OMNxa(fwKXjAAx^zPurv6TuRprP4HW*p*qL}8 zsP$dtslY6|eLzv|pq$$!S0d(5@#{>$v`HBfAGWXn9bwKkmZ(;xR_4dL3u!DJ&*f5r z%_wDlw;Z0}sU(37DYO|KEN*W_+Fw&7(-`+F^)7Q$xt=sE?i-l%{Sx~&h8JAjM}X{=om zZ2}eQk13h1a*p^~n#!e1Rq5jA4f@_MUZU8YLXnpGgp|qF#$g5wsRG>1K3Z<8qW*m) zyb)aF!Q2zLvv_i3U4^cn#wB)Ibb4IV-~KA`wh4<0!@euIt-9YC~fIH&zaeR3fvuAxZe zN|xMJcV_ojgKxZN3*JX-`$hD+4#oke&5B>Mpi_P|MVpaG$WSMhn-umgq$maAW^#Z3 z!Hh&E9Cb1xAQ|KY2qIMcKAf}{>*pOrHIHc=&T>FpE%6Qf-yp`+Z*#WuQuv`=Bs|0{ z%dC_oTa4;(ar#@`w*nb_M9^AzKCCS9O*!`)@H-)poW$bKR5q--Jr0>}Ocn=7WvGO> zyM-k3VQ>A-3AI^QQUvWUJ{?xOW5~dlmomY(!ciMlQrO~Op@^KCrQw7oE+4yHO0#hs z4p00C?%Ytfs4i?G`*fWmd{gr5-8zh%=-(M}vj#oCJnu`|x1;0X)+1-acGX*Y)oaIh zD@(=+-@`qacsJN&OX99<%Xi~WasGLgiE(T!?|M4rZJ<{?>fY$(2;mBs1p=<$841R8 z#|eIY(B;@An5jk`%G?#56{X4mKdKzYcRZ_D-4%eJy)Q{4=^yDbK8rykCJ_Bj51>TUAxt#xgOG6JZ($Sp0%UGEM8&c+%tFkaS$?kWvNc1OTu#Q#G-BVgfmnC&QH{d%saXC zWkT7Vjh4LK=;$h-j8(@Kar?@jR_IG@?+W$ft>|M&!Om{(3fKZiN6G21UD8gmF2ESZ zKv3EY{#rF(7^n3D#kZ=9ydT;$g}$FK1##GdW@k&)`-Q@SmeJHC1%GG6>os1Wjma1K znZk1k#XUzCO$0^;WRhp%4ovhWRw;R^H?D4vZC3su?ERxRSuv5lIK8L}Fa=mZ?xYjQ z=CrhR7r5@Zai4$ix~$(M=WzGuXC~WF-FlgJL*pm0AJ!sO(uVZ=7cBm|Sb*r<+!Mf) zU|;(jC;UZA1sy5Uep?FS5>~fWd_DP7AzGE34r#T-mVsX8a^=4-xsHg-kF7_{a=-1b z#Dx;$ZB-IvTFagcVu)ZREt56SNsgUM;*hf?XjBNbAf@E*v6JJ<8aJ!~P1<)L`{8%-_e9FlTKL0n>7UXfV>scca84r&*>$DdmYT>^a-X4ImGK znMI87ZYnNtQ@>GKN1WkGzFlwkdcl=PH|(5aqSq#VB~bKC#fwD;`Um@w##4`$@;wUW zEq)YfH%dFCrC-b^$pFp9^%;c({O@XS`Ftlf@f6acV$;=R@$3V>Vd8lfqbaj|M1C^u zbQTJ=0k8N6@!rlWM4_b)3V0-)@vTSKeRpJexqcIs*@pJAQ>Zl#cl)E_fw~0lvW*Fm zvkC0H%DH$~PJ$)`^(uP%NF5W!Rcb`Z_?LUKM;kHlq-1;1JDseSf#{rFXmvY0UBp1B zM@a31*gyP}Woh+Yh?Q2F4k_j_3oEaOhuv2n>|1_0YEQD1AApl)%Srtp|bzIq-xKw_eRQ)G{ebFwg$eLI$(Kt4AeL=dRn#Fn43pwH& zMQtRR2TbX1g6pD;QJV^~VWtPhQYu9ZT)t2%K11ZY!HpH%aIq>Hl_(IBA(TSuj04Bh zaHH=UFmpMZyLHut`)}ILKXgJQObm6o6bRp#loRQxJ$ma)WWs21W$|QWKq^*?D(YR? zd5JoCtruL*?ts@zn47^%0QD=Zyy_n_c=Yp%4HSq*WgFS|WIxlX39*Jf)yBSg?=i*hREE9PfD$5xaB-5l; za1^Q3swg5v;`hIO-QvVOB0-!r8|cZ?;rsXsnBjKe?yKyLhV#ao`Oz;@bXniU(zYRUGeg72`TzkgK=`lIQ&1qdRjjL1pYC+qe)U@mkTv@6r zz)-~|HV<*}V|#tGK}*a zc+QCsW2{hx`IMb{M36`xuC%V}$cY)+i#Xs{)h8hBsG+0dQk$={rl5N|j_AlT{8Hh) zDn7V#(8XuNrk=02O~(bAIJcC)to>=!_VcH^R$t1wMRLCLR^KL!(ln71{h-KELP9L) zJ+n0p`{@qHYYsB4$mM#7N>3269UHRj!Ep0Fu#cigk#*EDLz-^+vwzBDKu^rmSRzMw z@9}p@DjT^l|w-wAzVFF#hV&oV$q<`vsYT!#J7|{^djfX9pOhOT->qB4;}Zgv(Ku8}xEaIB%rzH`R3uL)~QyT~tn?Dw+(| zOBm?$w?X&JmB-uzYfy;m?V>345E^~ig5I!B>d9RhZ#r9zb^pLg5PK9VIHhMXihI2o?Hg zvI5hR_{wphINHra_7R?@LN3#oNILj3m=yIERoYYhy*QUwOhTz93!G=G@AA`o4y zg-`A4N-q^JAtf%JsxbX8LYB^c551w5xYR&AW;L|nx ziDA(U%Q=k<^68f=+H#1Q@2031oiIS9eA_x~5N)ZlWLPP?_~OS@JuBolAi^WDl_^Wz zK9Ta5JdnaaJ0~goC7{Zm7(_GE>=VCBoZa;kFZLv@v?5XSqPohlV#|3}Z?>+4*i82m zuEHtX!KpSRdQ0a-eIpPuU}M;hB2$Kzvg}!x+cFG4O=NslxPO@!+dk_ScJr9$--%lH zp91nnjE~eaTZ9KGNw?O}W zkZx?+mqxOzp@%fe>Fr9)IZQ0ivJTaXyD(5-sT+xb^pZD8FUaWzMnrpaff7H8{==`rPis9UK&fyKVBi&yZA-;EA>_i|2V-#4KH8 zsC!61uh-F%iW{qWBNV1x=UJ}Dws;e&3$=#@?X*jdJ~Bl!q7oULCqsJbSS-wJJrRi~ z!U<+o%;}uC#~vC7=5UkmbJD_veT^2Zq6_-!i|s*schA zamWX^DCDE^Dp@9E=zV@xdPPEsBI!qI8$lgx^xe9|b*gEf+(9qy^Jwz?hOO9e97BRr z{aG*c(8`>v9cuGmq|u+P8NHosobY0n&d)d5$6!f7ZXZG;Ds%yZ9xsX>I(#4tuG>0$ z=zKX!`;%Y{3pS_A{BjCbQ;3t)R8<<_b!#GEZQI6@FhDA^4K!{<&14x4oBgqOs5Ly* zU)Ve2Pus+Eh{%!ZSrH_rCi`!_a**#-cIDV0@FVw^o_-&Q=8v&cvHx-yR~nVrF)Qhd z*c|+eSuEtxQw5Lmq^Jjs^%iVhJdVX0kJZfV_g-}vC0Bu6C*SSm@0(R3h;A6_Q;Rrw zO@h}GaU9seh!Jn}x-VkLE3PQXbR^T;(@5AeTjTSvvC%lCQGvVq(ohq6NzlBq`zTRA zR)xJr|2*=c$RP7dB$uV8T94U==#j_lc`g&>VSINBN9EwwQ7JF1QPkN6F>O@`tYq>YcoP?yy@;3u7zk@GHh&u^hcNg9c&3=YRvlu_6xiN^dUIq8VzF{wc35psetseC~7DR>P;fWR6h z628Z(VX>sx7BpUoIASr_Y-BsmYVL3+>Lg2k#2M=u$o`sIW^v#Dm1D^D0Ier=Kkd<4 zHlsF~#c1)TfKopBNJtuYSzCLSmB|tT{I)Z-LfMO1k~9JU7se%@Hgpiaq=Gs0aE7C@ z&W)s)sx4kh#1PcFS;GX#^)96u$D~CYLq(XPztc%3XIag-VwupC9lXWHqdtWH;R!zp zSHYAbU>8J26rATDg97E0Bi)AG!jqkpaj>vQfAEhriQ`a$@6qPh+w;2^h0{LTuLvFKfgOZnafVHr~0} za}t-io8J`4XHOO%92San{@zewUoo=@09OV;aB72F?%2U1hZtj?u13aPfbVdIqD$M_;+Bx3&u~HTav$ zf<$ zacDr##=rL6IwDn8#jVEi;clsz@;bZ1K4VwZbhPJ5PPp-o;u931?6Bl=4S#>1sJm~n zl=vOyG53D}Pe8E0SyTM8lYYz#p&7&*O%rc(O*~r`zK3iiNRwrjCHrPPd=3mfb`+tQ zu8{ss_~EZr)4iFYWu(JydUvD?r)WKoJ()%uemp7uS(5ak`BzaFQWwwR^y>x2$cMWb zY?8}lyc<3M=rLU9PdIP-IMANmRif^ud^vFXo$~dM4 zNR9CrvQK^DpSAlT`kRaq?v$pajVz?z>cI=}WIp|E9NR%Enc=~{TIDv%q^NkXWeH>vfe#ZuQ_iU+avWL zz%A2soQ*4rj4#Cvh=dtKa!!vIS>nUg6GF8s^ZlvCj55PNaLK$}+EC+)7$As0$>|{+ za77;}l_CVntiUu#LLvyz)4YyFDd}lxWWR#lQ(Y$NNn%@U79YY*5&6{D8`=L_>;{ zL(C}c4xy9}D8(So7Y9td{-vTyQ!(k{GQaYsm}*?TpZdDwxLG=}>6rpR3beHSVu=U}8lt!YR#9Mg zr7_bhBC!9CXn^mMoT8vsbJBKAs1BIqAJ6aI;K=LRgdoqA$*f6D|90d@)b~sTL^#fs zlB7ajZ(`w@=O}i*rA5m$h)9=~;u?1ntga12(`THgf&Tw(^9bm-2kA>Cg7Y3|a@`Mh z#2nklU~vah>?w+CfN0PRsTF3>gbI>`8&fnwusE&pq9glv>BhiW1c^VU;@laB zQsKq&iux@D6x-UQ9l{WUT1>jJWU5e#B@7>(ni02i(GgI`Rxt2|A@@2T1@GyZ+?!{3 z`LC{P$}LrVx>Ka6=35hob0-XlR&&p7O%SwPx<0n@-d;Q87aaQVg2CQ-hZhd5#0H6o>jpo4p&4zlo0h z*kHLlK>J|Xydr(s<*{l`BT^*t;5u;_PV}?{c0SVs5^|4yy2L}G5=kf|wLLO5cork0 z>=>u1z2b2mt(vvUHK}bR0%fCK+b9?wKZHsH*Xk&|jA}fT_2a^5oa%udgp}BWx>}4H ziwUNZZ^XIwuutxCVbe53XfD0W0KY5?D>YfEj{%oQPbqVWdPj9*`vMaMjRiwctQEM<37=s3j%WEKz=5#Jj3gPl3EzAG zz-fpR9N4XXRQzqj@i(4&A6RA(kBIsaqv2j)_ZQ+{#qM^%f4BBW)&FDL{%8G1H~u{R z-3ov1H{43w-+iSBh$!PSqr|ex(4s@DP8g!nmXfhGD@KNzH59bADAiJ=1?65zmQaPg z<`tS6#5A>UXIoZXt0Y~T@S@aGloh17>{y74y_IhDNwuk3TGf`NZc5o`Rfb4rC<{#`&=^l_XpWzH)3PMDLBZ|3u7w3%; zeZXzWmNPah+uw_^FjmdT)LxwW<;}6^*)h(nd{)hit%TKs4~>QGQ6e{cl4Vx{hbpLj?f)59$orZdsF@>AQ_BV zBw?(gUVq_ps!C-&)Dgsa5I>7?L~$N9ZjGRza1L?3GDC$cOWm-x3iNh20~}k8wU}Sr z1G{G$2eR)JcdO%Tc@8O^T@@SobtSrDBV>PqJm8XzK8g8w&Rvv<<)%*xjzMoD<7W&B z+ZVD>y@?eh#uB`4cn&@58%R$CQPD*u5+d;K0o|I_N1 z!iU+O?36vRC-=WPVND+fIz`!Xxn7s07sVH)?L}+iiKxCNvQ6WTI>T(Kg}Cb~Y1O|s zt1|w~|0w+F4LBP7vEnsEH-eNpbQEr3aJY6s&B@Gjb8||I1gace7Yf|P!z|0CEhWK9 zb>48w(v-QhKC(F_mx;nXgDKw4xx%sF>o9Efd!%z6!@5g_a;>q={ z5g(Gs>c_q^%~WJ}iW_WR5oEkq(){*CxhHh7>g5Q(qbHU{^c;CH;{6FZ8voemfzIcP z)=7gg>OqSCDbBd9ykAQ~IE%NZ22VGm`-nMq_k1u#lc%i|K?jzkZXUkT&xkc>_#u%E6N~0>buA4}8 zKY6U;oXXq~h6)a?J^it6ZjUZVxh(IcDT>8B{#g3UJ`;3A`5(YKe0$sy&da|oFdr0| zeTO(wI{uSy`dwro+7Sv8<0@OOPClNij-EYPpz%0kq!|Cu| zl{3k_;Y)o%zBqOufD*TL{wBe(Y83TI{n*JeYld8` z`gC|5_h`Zm0flgmNRQ}7z&N;W!BkYn7OIt#W9wAqVk5+2+^gU!jtRd!#+Mi!OO!#w z5vNHqP9_kDAQ&@{ZA{!|jx%JSk6nRl*#KT?=gnDFHAQ=?j=D89lWj#%@qOBBM~y75 zA_rRu`>rq4oeR-OI)5gR9L=!1Bb8~I6^f-Ui3UZLc*5Ls)u@t7z}t{sjXLHc9J)%9h`>oVHmR@G zlx^oI(6BCX`Pasbje#QE``eqVL2NRaO%p_tNoJ*(lQUJQ$^93`OGT$7I5p^1*rA8Fy9_ith zJUJW#_9qi{_KBx|HtT}m&0ZnKdw9meqKRPTk&{JJLq}0~T<*v- zk{UaK`~}{s&)^89rNN#6iSxM-TY*?>*%s0l__^2W;!f9(91vcQPcXFA>!&qmi00!pp!+Z3XnlnBhyuGW(|pHj$np z!^E6xp~MZiVPDOHy&Qla;YNo^vM-{!>$LLm%eqreH`|uRS;Ge7XdafLH;*2)#b8~c z9ysBrE$z&;(Z3E^+nk}zEF`msD59H_049(I=ucs=yp7p1fcb+Mjgf8|kUbxbBM4U$VV(P{X!E(%`*1C4ARUuuk?Bmjr?5*;(QhhF$+`CC-U;5ZHPh&5hYZjFt*UyI=YqD zRzVF|wj&_(?oOrI6|C7!Ikn@~$h+z#CyGX7UfXhNM3pW*Nv~#eRe`Cwy=H|51ke#A z+gIYLMJiQOpLSP`Rz2dcldhHOs=Ly}?0IY?d=6^Oh;l6*(n&im#@^a>OSYA2=B?3H zb(KrBsI80}YjTK>SrF=nZFjr(H7qT!RZ68+n$Y82Na#3`MMI}NB?-FqZW zG7yT$u|$33LwZa4pS^*Ij&@7i`_Dl9n*ucG?jy7=uH=m~lTIuvNhX~PTtqzsd5Df^ zKJuvOH^OStiwiT24NDiZ5cmU=vH;N={9)j>G9*d6LJ`EKLD1WswNYJJ!tjk}==z=7 zew2MlXq|jJ<U2D50x_E?e9($J1G_Yn)2K~X((14 zhs4^dRGiagJnX(`A#5k{FB%Nx(_QijpC6`iI6c8>fkAi!#@$Z zJtfbj=<(H$Nd2f11b=Nqvh0d9$CqO?VekcWVW5pj9Z6xKH4%d_2fA;@L&@#D-GO*V zj>|0Fb1nhmv>Wt9cxYTRiYX~|90?gGtvJbvkVjiHE>afEOsR_`;8M-2R;5X?t18@W z1H)=p{he725+|u4eN!H{(>?+%5-<&GpCe6kXwW2*;{QwKCws1GH~KG?sDaFALJ@IF zG*~YJ_Z5Re5-8uPJk{|JsCx4oja8tyh9=Su=0|E#bpIKci@tp?qo-@{o!L5E%&NEO z>{+dyR*%M0i)fZSCp~Rcgm}sZAo8RRz&>vb)yS<8x`#vP98&sNvm-2Dpv%hZ58VOTMXNL0eR9wsSsc*kq>qX>=HQ}H&rp?70EJ?^Joy06` zJU>8)B^@Hk;RCu#PXJIrI5uS&IAsw@pE3y46hbxF>p>hM0D{l3cW32W&5W=)kpg<8AELnI-vSfnm4~o=0t~AZuLqPkMnvF{g`l$nLK-E3 z|Au&HEU_FrjGh>~@&(RtSnUi^YHSL#Bufp2(8m!ah}@sUelQkmvL+P@(Yv=A;!tvQ zI|L3Grycl^2HUVdHkG7fBMRy|Mm3_W+KNvx+=4~mU^dTAS$LRYjVK>T6Lo^`;No7Z z&_#);9wM;sZZt-Sk$9Arzd54lB`1%Ir8iGbc|{1ZK`LL7&n_bbrs|Rmt<+;j!@J$S zSBRa!-aQ8)W~&4=7bDD9OlhVF2)LLSq1N7APl@cA%L0l zMqihs79p!jq7j8|p&P)a=^8c@=V*=`3t^``AI9P&u$I7JAjrXZ!!$aYG33S>;mDOUCOn53 zPDbwHKtG$W+<|$2GNuW9e~r&P?-5uWNK!5o>}N$}mZMF!HG2o}r@cFHl0-fWMa7--7`+tB}CbKObH+q=KLWqHl z*^}EUW4**(YYPo-OoVexjuf=nl4WE2vxt)zT#|-X``#G>vV_UwzWzD2_IRbi@)Cv0 z$b>@vjxSV%!}&)uUcAN|jv!>r=3g$D85MtZiq)!0#Lu5p{a0D$r6};@!%bR^C-m!h zYC?sX;{EEL_-~{=>GdA$b7*bMe(95V3uxOc%3S&(%vBG}r>NPSLvBWSBUZYygnVp^ z+G^H>UQ$?WGfh7+%C=Rg&oqc@Hj%n_YQIFa{jrZ@n+m2mk4dpCG3hbt*z$`Bs##HJ zlde2l#U`3%YjsO$#u{;^mdcij){52>SXflgIY)}UHm$Q)@H_QATD$6f`oBFK{faiH zEmyxNrHVYa?(a)l{_OS^#St1;fWwihBJ(ujw=YEVPqD}|A6@#|Yx@7!U%Y9~TI8yg zD8Hvnw6IF2MhVHIm|mv4+~5`-;8;?+D;H%8DWzg1gcxxq45lq%CvC*{$P^;5_vR$* zF;*X!^*z_Q_f_>ROFfldo};uXt=iRzicxP%Owy}bt)3bzw;64o_($&^2Lq`&;E#ws zhA^mGxYd-4SqFm2QzDH5j)U?^jY7qdspwWLZF;4T30~zyIf2HKsf%hlv;Ogc@y~LFx?VR%W59&GA>0fCD zAD{3b?T-bk`ZOcBQ$Nm<-f!c%)gDsbe%c&qUsnVq@BOGdIPd#7BD(bZNg~)_m;|FnV+54v&9>F^+n z72yM&IqcRhk|e(Mk#HYU>zNlnYFvksct^Y*$NjsoI@1ns%Isi+282Ul(J<}VZ^p4+ zxgFzTS+<3|qPY7f6;;}i?X89(H6n?m@fZX{z!G|5OYt;0tNRG0a^;Y9H|J|dDNB4GVOGlij$n*;rv>cxc|@5<)_>EX0;41bfTK_Y2MbF zjW^%j*;TF}%Dbh%0hQJsp7j3a8}Jb;mZ5bpZbul08D#C$v|7bcH8OD#*Nv5}QOs=x zYDtA|O_7sj2DN)b)|IMMwRPJBL^c#d{Omp2sD)WU&NfipUy;@h^(Jmk+bYYT!mDSH z|0~EVngM%WEw?njp8dx%+sHq^H5!uZ^IO*`cT{= zA?YDf_=J=cs{=@_TDt!lEISq5n}~(>B0q|8VI>)4$|R>?!r?Bf>LGU)Qcg|S?H}8# zt(X`_O?+_4Kd+&FXxKLpC`5yT5kz^3_aUOY=&V9XZ~?omz6KEwCm!SXR*2%rg$6`f zq_q(c$iL$ZQVb`ZH2gL_8kNLEO(c@&oGUo*D&UqwmI{>z*uK$fioqgP-I64DoSU7i+^%&^L~SJz@iC%K40z(tKzjKiNbS)w z+!ef2N@>BuI{(sKAUd&u`!vayF9UF7lCsdCyDp81pHRftm?&2SHF#jAN1B=SF0q-q zljfF}o-u6yR}H%^&moyQP?0$jk-WB-xX+Q)Hs!h>=@O*XB&5{cbIYNY=OXXBJJpYB(9CKxg*{mF+ zvUso4zQjXZN)@FV?4oJ;188X9TbB)54s|riH?xgqO;6(%`nJh%K4Jr@|%I6$AcXqe6*IneYkr4b&ct$E5;Z% z^R`n*UVb*u!`OUH{%Az$l`N#$V>1T;|JOWKzBo7slebz6nU$3iXia&$7C)b1^S)9Q z?5Tn$M@tGXqdHQ7;c6ZAAfk_4@-%EpP)nO3wuDL^)6_BoqX%irn=<~6uwPqFS}%r% zPeGq22Vl_QhV{td2O`Kf0|A;s*Kq6be8bKG2;PjN$v``o^IYI@PN|&5`uLCIVF_I+ z??tS0HA3)}72}4lUU#$$0>){EAc-x8$RQoOyk^m>mPn2yajzKexN)4*nz419lokug z)4X-gS*0w6fNLWLV^C#FVf_+YPB}ArF_;@Ut+|TS(40(haw{U^pg5F*%MxuH{DN}_ zu38dkp+(ggU*o2ggHVUI=Zmo! z3Vax!w9#(wa4FhGjEqD_qWX(iR9bHZ6^Z_#D<8`Df*Y&MVp5n zdx7oeba}(S#9L2ZJvAf=4;Nj2#tpgVGrD&62@?i%^P;g@t-{N`GGvz>j9`yG^t_O| zp_!Smj5H%-!2@wKq+zK)0c%{zia+u7iDo*13)by}^?SOwGvW0wBGT??N1|YpK6TN^ zU=b!_3_*Z2MWAVpAqc|~SVpoqxsRn}NhV#*5bk14lIt`7IsKp)WT$5_wHQ zeJ#X+Yz2nZS1i|yG%FL{?dP=ym`yU89eRqDs!jc0j`|+gw5rXBe-;>v-omB5OSyKn zqx?Uq8dfd6iaj~DYAJ2$)cwAHgg#ybKRF-22hTrqxAL!;do!ffbQMu(>}tX%?iR|) z^7J6jh$c=za=8YPpSlx~2&WcBZL%_EOEVUAhi8}4G;tBz#6xRHO(?!kFQaLQm`LZj zJgK?9RCVjp4V4i8)8l=_I07Xm>pJMC=JVmp<)6z2_doH$Lu6ggSI0F+q!lV>Hv-88 zt3x_21%a6s=gk7NB|?M`+{EYD;$16tau~^R3uVMO_=sKz>cf>9sEUgTLbYj>NkttL zL`A*xRY;{Nuo12o-gat4o3rt_{=au;o4$L+&G{J>vZUQKv7^9VaXCS!7C|Qi7Xxl( zA!^L|K5!6(7YITaroN()lbl51uu3MKH9?zk9kiv0LqBMS`lQmhA=*sJ5d`iHWF>zL zN|;h?ax5n8XzY9MM*Gq?>dWZW9-+;gNRD-|p)H)&-oyeo|kywfxy zaO0CDexmwrFS%;8EGqbC)tNrFKFx4a&|J_H?s@a)?`1qaBUOK{JSR9BhXx`wSckHljjew0N4mpM+0%am2j%&5iTNg@E-LxZ@LpuI$ z-Qp^$*3n8*jdzB289>{Y@ITQ}`$*Pmi}}-tK`RUS9#D>Pli%S7qpj^EZs+qqj~su? zh>lLpJ07?9Y@IacInH;B=<1#gqasH_k>0&Q{TPqK52Wk15zjn#R_ci37h*r$qLFo4 z_-;U`jGGdorG-k=QKzX=wiV$aaz`ZfZxt$uJ*12-7YpGvBa?4WnWIHXIf*8ph`+g3 zd6MWyPVT-Kh+h(jd(1%ltdjUqNhQ|rIQaIG^7k%HuQo&HZp9B#v32j|_I+!Y9{sJ0 z9QFP(;SkFG!tq9keVduS$yA(kI2(H!;gVh}EuH}z&RGRRXqu>#WpJo{a!V%MM>P1q zTS&B8MLh~9KS;KI*Iq`I9I6;J7nq30k8IQu3KX?TOC<^vr8@-u$JGh-hR5-TB!WIP z5JIZ4YASf`-3=iA$tPlI&yVg`^A!;qRs9{<@sR&2{`Vw)CZF?7Gdig8Q%tbV(Edpy zPvaPjg0&J!UKQmLc1f?vE29|dO!LDt58eE|g=I&@qusnrJSi=M^4{5Okc zRUBd=z{En)`B*DVt1rP)c@)nQ@xD)S|3t=n2wsb#~jOv4C_ zjlOc|xvG}jLAI?7{*?^+RjZP7&Wo-*>FAnHjn1te(A=PVjtjVK6gf^Clto6SkgQ@DZG0ssHIgl*ku9z!!)p3)MHtnptC4?ZI5P}&B#M_-KuQcnkPEa?(FhaSgoYQcw4G{GhM>_W} z-MZ}IJX|IIoGn>uEo@CD@7z4PtA@Vh-AoZXtl;);Z(4^q4iVWBgC>m#LE6+rA(9Yf zAW3bOFw91z$x_ky@y5_UFhLr#l#``IIpudYBV4wz%dtb8MfERmj*@RAty7b8b*fbD z-N576c1e$YkH7QX^e$l^VZyf8+Up+nt*daoN6eh@Zb9R$)^IBY?jkqM1h`i1$snyH zx^_83lv_LCqLNEp)e(R;b^nZDj7kp02@)*DmrT+sS5vTYXI&8*!Lu$DbgVS_pwBvE zW?&$52`gmNN^HY>7o?mVm*YmihNLg;(Y!gtAM}$2x~8554#+?2plaJ|fO<|X6)9;n zr_4B;^SESqs+vD`;5`tIY#h0MNGB}4I2jd zT7k|mh#_tvgsoP&f_etzJChImvntLSnsX3`?U-xVjF7M zl?zyY$#Q_Ml8#1n^lnOWn28$A*%DL=#t_Mq#4&j=Td_h5#%glA(;#TbGZxHK7~_Ia z6JD;YK;S)wO3*Q*WI7X0WgQsvm`$-wkeWn4^?%dfCGaXQX5khzHE9s!I-D{dq-+u7 zm)!P{-SJ2t#Ey6PJ^zM@P>x=KhRXXXAt)I0>I6|1Xq!XkgkOq?DN2%*9wDJcC0Lik z5X5QGDblw_vx0Y9LOD1>N48F!!<*$ZgxzZ#N*FM~p&bh#GlV1Cqe$lbbBnmzct^0W z6Jz6JZ*8rFc@s)(T5Y9tZA?0^lWJCI7iI0Cdn}VA*&0$Gz+81~{(HV4i3~8qc5pZ& z$#W)zjR&yf3WuaXT^;KKM=dCadq%1BpC5+vRBj$oxV-aA3zcmod21bAke%Ej6lF9Y zh`Y1V`)n(thVZ-^PlV1=5lfKX9=a;f@V=$$>FWijM6eqNwji*g}mUx=h~^mU@k@t_hU zyE`J?7KXIf7z5Dp_9QX^axb4P1-!J0F-AaPKISB~7Nbr;7NY=39NO+kiqs2=imR3k zxHsE}(7o!1GYo$nC|h4l0W41h8LKg7G)VR{ngNKsgHVvkmFrQ+7o(t<0=nD#KCi`f z*>L|LLr@cRFAYV;hAgbOmRys~gmCc1jQGN>8CIenkmF?)GR0!1FV&(%Z755yL@yc% zy%9n<7Rp-bZ~tOF$~w|urYsCqifu__KTs&x5OAC^JUwgjt4G+IW3tksDi*|vh#4YJ z8d9%^X28E>+wHc~Mp)-(ETP1w3UchZ#2Zf7;qGta@|x0|3yESX?8M0oYYR2kdUrws zfZ+0uPAaMG{)}=zbK-dfd5{&gVP{cSxxmZCi$2A4->k%yqwTG{f+fy_ar?0W0syre zt~K(}3WQ$S_e#i`jGGkGx^M85p_63aPToe~A2Z!-Ex6xut08m*L7qho*SJu-7jPI$ zvZSZ-8tAjI%+6?y{O?3P@kNWS|1BIT^3SvWn@;n`?)+5k6O>9*oOUqz3b{CwNX@yV z-Y4&S1aMnz=EqTCF`IMC$s7AjJ9l21sLNeqiRRG5w$;~&b8WCWYconufNiK`oT@Y} ztmD*I@}`~}cVvuvZgWu(!MWxALB2{Q{kZjnei!8EToN5m;B88H64(b0-xlw6kvDq@ zNzoela*qZ%EV$-kBI>Z3No844#IWs>A{MQdIEd3n8sfJ6{4F9IyE{uo);gQqF+Uyo z8uM;r-`eKiqup3_T+6L$+_}`Iw@2rbcs5%LV|`g^;x*=2j(s$hQAHG^W}zQN;doJf zl3zm_jkPUE-Pfk>$&JMXi7gmD93_4Z(jq&ON|R1)-QJv%?}ed{W5!7b=19+@S}=?v zF444+T1h10T>;Gu%Njn>BGxrCP?OMOSG;O8CE9yl()&-EjOLGrVBOD~;5rr44U3;^ zO(yk9{fiKmyt@_^qSuW7}gSIi<3+8DvAsF69}8l-?}XIG6djh4(JO zuP@aR#$6i=?=8l4M&wCEl7Bw0(tOXS-Ol>$vcIuSf9rU%ymfzSdM7#6og)MYLg26= z^d3Nf%A-6mdvZwX2Y%y7-y01Si8QY;w7268hwN-lLk?=5g};3B#!Os40^_w9f`TrR z^HRZ)VhNH!r3RenmQCI*_ALodqT4OJyH zQP3^f+jGHpJzFBQBF@qz5|)v^kV;a4DMEARv)(_y+4kg|1JWb-NX>My5cj(Sd89)A z`ZJ@hL^H!UMu8~cD@wVlUQx@pG>z}mBlmd)NQa&n`r?!3bK?<&mx$s8q(m%<3B8HL z)Bl^Y4YAlJWS$gzu0Larl>Fm+M+v!&ZpTt`OUrIOU9@*s_=ez4OrBG|iE`iwNy=J< zLB0^&i4=Ic#EIHCyWioI3y~tx=Dnb|A7lQ1^Rn@`WaZg@|F^=3j5B>@*DvDlg^1_} z#Qj8n)T4nxpmAxpsU^^Q&83AMn}1rU=-R!Eaht3;6@~d)X+C}NvqVF*ly*`WTbvUB z%{cgo&7vBFL-D({KDMcRz z>8XcX;5tWFr+GGC>DaK_y4jh{uDmOefBN2C=dOrTW>Tlgx(eZ^{7wN7zmHIi%y?nB8z z=DNA8xKe4gyx`-}iVdN7=xY_~=gDuosEmUBpQz`W{Y zJyGk%OzCQ%s;JiZl-z6 zUqMHsEyq~g;Sc7-)li$X7()%qIPXEJBhTw0dXPTH=sJh7otE^eQl5Zq6%-^nbvHu$ zVLE3^tCZlp-H&3unX)z6GQK2pEas5amjld8LY32WSccCjoihVNL5ejlVikusj%Y!| zY&y8|t;jH?SVjMolxIhyaL`?eRu{Mmjm~Y#S3ZaP0cI;gJZzf133vSwAh8uhxc2kx zyONB>)krS@?n8)VM(i)8e;?`!q*x+EI7J_5Pa7FrWkG%?%maP4_-8-TKqLi(!SW(7 z@lNnA?lMaIC(!Sd!iL<5ILDlOEXFi)E|!9%uZ4(X8JmkR^e`K9)K|`FfG*38jxP3g zhB6#A=E7^G$9#zTbc+U;k7*pt zh{iRPMnw6UPCJpAB8y=TScVfEIgC++-H$DdCQG2OiZBbii*L@6s#&^Xnr5l0Gb_6A zlZbK*&V7MwtD{oK>Rd6lB}J8o17cD=JzOSylr+SFh%yGq-wy#vowG1Ub(+Q+qElXG z2<0r{JW-xdRT|xE?a3+S6}{JUB{E8#%{HMr$TW3bfl^0 z9S>oGU2XpaZ{|lrSc?iNnZ!+|t4lktO-sVtCTruoHOtrN=W7`&KQNIs1kz7df2KRW+gP}W ztdkiny|oR2%kDt~^8{uH9=5}xZclH`^gUI1>tPg(x;jZO-%7?>Skb2@0T+(jNR2v5 zqV;=seC*Y(_lp>ZFyo>jx37u8%SmlIw#F6RRm2C}GN34283jV4nThiT&!lu>M=EdO zFilv5=Ep77&Zo5L0s z4k98v4;ScdZ{JtPUwK_qSMAuEOzS2IrjR%`-r)GgZRok5RuOMh(AT0|8`j7zwt?Zg zOZUl$T{5$660UH6Jpn|G5QAJGQiNjpQA_|Y7bLMMdv;?TORp))O1aTGx7w(%^7ewe z^u9&4nwR#tSwDLhX-4~EZ|=(m9T3P7l{Z%0qqAiIJKrhqnd9g0((sRyYKVo|o3MyrM3qD(94ogTgYTjpSmaJKkjC|#L1Bh$4riaK zThW?pLS+3HQ+Qd35DX2Oc9kiZDNuwGi2`jXPxPVy+zrH-cVoQjRDM}4W0@ONM#O#> z^3-@OIUi9B<*JGl6gfHavPXm2ad|I=lca9M10(CM0w%sE=4iwS;`(bc7>mtC}O5EaeP!B~7Ud+7&0_$nl{GH@AwY z20v?frm@+zJLqijdf^*nMvgfCN#rsGT#R_pW)V>jNioesY@-37hgf@VANzlh%<|95lQ<~6;|$LKHOx?zE72A%5l(ifLJonjbR8== z2W0M^V%9h0g}cH*aJ}j1UAgnYZ~cTvLWYfS#XA%&B5wf$VtQHZ^WxxoVGQqBtooQr z+8QNSj7w$93K%>Rw7BfEY+c`U`~w<2$cXfmO{AK5eItN%P66h){)vl-hAz}-1LmqL zqGEGr`HmG3=7#4IPE7Iiyi+pr%EdxNmXF-gsSLZ2aX$qsmb(e$J2E6W+Y7MV7eoH7 z{7ycV9z=5!WYIZGvByM14wOQV2?fw!k~VQtr4&&`6moGKvbf$G0@FVc879vWAGt)| z?sMrEh3CsUmi%Rr|2H%@vWdwn-Q0T(lv|D66Nrb&AzEK*Nx6bEKQE?oi0F~KygM;P zO4DH|qhE|vVoqw27KB-lw4p)^Q5+(x)-WTkR#KGPV+~0*Yj8)jlHH|w93*E382B{k zhY0-k?4s1K7|`ol{6tw8C$XY+)vgl4yP))z7=yYU^YkdzILWqlOgI!nT6S_#Nv6#? zA+P_Q-Xb}wleLrR?8?ebDzgQ!21_!fJRw0Aa?JQ%d zL)L%_VXH1%i8BgNR`O;X!ZEd0|7N~Qn%TC<+2pUuOo4xOW6|Uwf*Ei=2zatIh7qDC zgxwuW(fA|6A&aA=3A5VVx1UtHwa=NS= z#3q`LXx+o8L1;GgaINL}Q4EOqg@uKc7K$jLV@!@6I>Y2J#aEeFaLyjzg^poqZ7jTM z_Enmt_z7%$l@Cu0X&*rw@g;s6TFVsYc4yC{wnRB+JlD`vlFCV|yKuX! zUfI#-xpz9El1oPW2SkE$OQj>qFZI z;%Kj$R3QYdSl#H$Vh*FmNsZts#7pYTZ?*_D)pF6@v$P+7A$t?ouk_t>?g__-N@;H%T;BTpF@^_$ z@F%+HL%XXccUE_eKuIy0k`uMK>3WFH`AAn*5GXF$P=_sc^5>j3hCrajC?_@wy-n(T_?BAm%koLepl2^08%C^h>uoU@dUScR%)1G+4~(dYu;-YVE~10Wk>5 z?+t;!H06k3vlz3Ie@OFz7KpT3$h0N2_zc4nanf^o;Y$_p6{b^t*Y%LjC7Ab@RO7Dysh13Xz;F zris{{7|zMjEnTxN+#Ji9W*QZ{GTU2Yi(MMGAm^V~G`cpmqm*z)P*9E=34xq_Ekjt$ zYRq7;$&3e*?TzeN#$&GQKUox9bDrD^s%-P5vHfg5@oizX>XL<3M)$pqc`S-Ikc#bOtF+8_smixXt7Xd(&x^&1^HqpVr$sM)x8s9T%ASMCM z)MZ;OHVOkOY6-I%`dW;r{gW!s1_kq13erB&-&&1j35+Y6!r;O&&HeUO1#-%LBgPUK zdbbvGpJ3sghRyk*;E&~CxVns$g0lSvX)_l$p&laaMzRJR#}6&VfOG8TJu^bKzhQ$I zf2H?Q9}=L+#&C;9R@PTT4tAy^Q~1Xy3zQxXbix^J#?1bPM8&AR3nCjZyh#uhxP9U_hV0SnV~1 z_2V8x1=L$?9_&b!418)+UyKz8lo`X|wGNNL&pM+C&G1#nTn~lL?o<3Q=nm~(XVs7F z?1`IMXT)rV-Q3V?os8V6cOs35#(fVtu4yXY@wc&=$n+0^wCj3oq2)mBs+|8zZQ0 z2R?tG#2az=EkWS;%avzv^o1ye3UfaKQoxtu)8U{#fWP6CA6bO4b;PA1TQS9Re$Qe$ z8B>YAD_ugX5R2%~LmAaR2E^%J(9E+!PMyOKlfbt@noJdx?-iW(?*LxP(CK@wYlt;4 zN{nHTE)}s~cGNBM8HoCiNa|9VwNhJgl-E>d@FTH1TC^by2w8YEB8oeEpse)+)7Ysk zpudIZ&L|o%V_{+bvG8$*q#WRURc65OvS&xJCJ=E1NL`@^-G^M@MLdci_Cp+dX9BMe zI(~bDN&l8#xwe^KbVJc4HH9lR!Bt>b6-8Lo5dzd6Y%yl5a8F<26hn28nKZ$?ZQL6R z1cTOXWyw{-I}oWHTgF1rktL}LzU!b2U932CqGrPxPmar|EqF^#EKefH)A9C zdwMXrW6VhmAvD5X2nBomzwqqIckcvd^5H+HQys)`5#m-A!mX$bF@FrXq&nfln-11r z^)Th9j3$Trk`#>I5TUT!mp{x`nEyrQdCa@faejB*4AB$dF+>4?N@e|US03~@xYe!g z@&lE&z_Rwn88dzLupJKhZ8nF%69evbI@mDB4#$x9dxY`SP}&dvSOQrHd#-&>hJU6P zBWP}!p~|#ykKL+tuY~}xK2Sk=5e_dH_h-|q&OPTi8e|W6(p@s1hoI>mnqQw3YuoI| zj);%^at5MElzk|L&@ch9H{5c=DrAl5{tt&dcOu%o=D!PPu^iaiYH_fT6`U%BZktXA+^=aOs}H$x>lvro2AWBT znvlC)dLNi?`a|ThIJs4gBAQiUcv<7HDq4Z4jqT$Qg{991HPMOagvZtgv7SxwI3Z5Cr3G_@t*a2g zz&mg{V%;gsW*KZEE!gA6-#N2BVJ~2}8C?G=&>A`Pw6~79z-xPuA|^?`F>mgWaUBd* z34QFf81Ys6>oiY{2C5j=L#OXaN|d>%qLqx6#DP|atT}K9D8ldi+;)dBXJ>;mK`b~$ zvV)QG`5_evlMYjB4rDmFvFi+j78s%Eu8@od)18=iGiUr)=F;}3tWi*M-bIIV?6m@U z(IV2UL*XQ&Gh0xICxW?X3-qy}D2PWG7$Q*kOME?#svhlm>pLkAO$UR<={_3EZP55D zt#otZqCAUS$~hjfVa*o*7ktn#9|A*+aC8pt#^d@HgDVAGFP$6a5eU+oYHel1LNp%B zFGKiB9I+DytgY|N4;;UONAod(Rk|PfMH5Z9(ab7ud~;!`sVcjzk8;cJ9M<@H*8CBe z8itmJj7bj~T+f$kUnXlv@OUmqLT*}f&VMgkvRL|J;>;JbZJ!vOyZFblcs>K>FYAtE zT`9@ix}*G$5<)ZP2#9_m)8`(EE3P6&D6G5{@CF>_Ak2M!jE97N%##4djIrgb65#8; zDa@5z=!*))z;zd^X$|mKf)E__X7nv#gE3ILBc^%G@s1fy67tO`4I{7g$G2ju*hbGlL%g4vh1{rg5?4UY;5deNucEOH}3)UjZJgf&Y>5rG64v+P6N*5h=e z(b$h`<<0a<`Bu`8HK3CfY)^?{3ZcX6xauCn5Hthlyv>tGM_u19CEUqvfq9e% zv%pu2ugHhSCpB$SoQ@0c+->O2{=-m!OAFxc> z(k@1>(Z3SX8nWmrG9OM8*oy$@Do* zk0M7LD8zJ26dop<*9*YWVsM?P-0=Nq(WKfu<_^MKAJKw_A2UXe zNoh!7Mhlx-nVIMFjnBsNl)SuiEW0z8R&-=o;HI1g!LdYzK9s_bUktue#}dz&aUANs zlahL*?wFD=|KaaAOl4q(Cr>bQE(1-jL^bAwEk_E=DNzgQsVdp>C!nziaI>XX+mx?^ zGvK!89VqheszMLTJ;0*8zjt&LQ82_wU_+RZL@F#114#Aqvn?UPp}|FoezBYUr#!ak zFX!k!A0$Z0*Sf>x$q*1E*dM!3(^g-el*!|&d^%k+D0b|O>p_pl|6}-evJ|dE=sUml z{8mko8%&3R$kEC0upO1(dCEwWq7flL^@1$AiI-0ATO|0cjdkJwo7(1!Tj5#jytm_e zV|`F$N&*RaJ>MCHQ6sF*y$E%3*1KS2!=4><)$2)P9(QOgMD%6IHhn#}q}c z5gt+UCC)jCv`1IQLKsQz4{{1_sGj61l2Ig*85HDek6OO*Rc=`npvAGHnSsRlkf_3t zHm;QmO_HTU`iHeRdoWzB^sNhUtF(MW3|~u6Y-46+Wlr$%mgpSOrgbHFH{y8NoWRD? z`8zfYXZWbJlRuIUQU7M8HzpRi#^n3kTaoyU zv^q$~thG(j|5qO1(Y?V_QyLlqmQhMMu9g_AmP;mzO-h#(l3^|?RI*&tI3}&6EvKmx zJieCtvvNIt;#9g$_n9Nv3Z$#=6ZGN_6h>r$D+(Z&=49j22D;ofLx|=L*0SL(!jJm@ zwyp{hq=w)&7Oti+igdq@u8yK&MC8KyeFt()1wng7>40!m>2Lx15-|MK5Q^cXn@ZM) z&k(~$%uWqtlwf8wFq)926CJ{b%jOJ=c}y5Co`ESPbQsd8$N9m7Qzp8>e{l4`r?Njh z`}6LO8&!8^SMzB;<9-?StVUN53|tik1w_dO3Ci$99atu_8r zZfj5b!+6}pt?sT{`7}*PL<4=@z|9Un>xuFwLGupA5GSwj8W}y3ih2}rm^jp8!lh!q ze`QqW86`zaC7uRM(bQB>q5E4~MIuY0p3t|qwu>I!sZ(F-kD!kR83+T8J{}l#LXrLj zxwk)z1iWBOf?U5c9|tcd)P8!S$khb^a3MK2e1}&~!#r zgti^^XkLZ2hus)A3!;ahQ%;{+T?!U~_T}+J+Ad{4B0$1S$(BHJS>q<|4?0AG* zgKSYbDs?4gTfe3DYHi#A&tm>Y2#WmW8!x^c?Q60vFxO~Bg@`&PgtbVWbQjJi2tRzv z{Sk~Hh6d^V#)^>#_fEM?>z|Mg%drJyXMUadSWOgs~cgUxDv% zYEUTx!Y~`U>QX5XYx<6NlK;YNep8~5rdXjP;YaJ^j1>WS9dZz~RL*8YA~Xz56#nZK z2N@$ls`mDsFj&a4P%woGN{5LOFq7=`DD!mPVjRtq4u}cw; zwu^Scn`y`Fa;4rf3B|I_;w{u$B^Lkql34ZR3S8Gqmz5NylR-vck(C(GRFa>hei0L3 zwTMduJ1C*DdyLTfR$^2yZDEwaXxM5SewYHSZEd{QVX9LUS$QS;L>?GaaK)H z=BZh)FgwiQ=o*A{=6GAoAnFY(gSrk_f*@osrW0;%M@piQ%osgh88+=9uoMZqwX6L! zX3EfCyWr7=K81kDn4(8UJP3x=jd24(j%YKy<%g92upp@Y8?EWr3}h*gB&*E;>%|ya z#SX?l_Jg%pJ;+3@>YO#{p;5&(a7sVJubn3(A)-y`8I5^4Vf``=z|I`=uergUu^1#j z8N+NYkUl|slWKE4(nz&A{1F=?UOYT_w1cLY;PG}Pg9%i}dcSrXOJ$-%=&2bK{~9Ih z-gn1imP+fa?ZMRkwl-J_lDT6whQ!U3W4WcDBUT&Q-F@-g8v_Z9rPE?ht25J~)yoTy z@-(&htF9UO^AblpS zr+0Uc7{bao6b@u@p6Hv>s$Q^11Q=Wl9#1MqKa)mnhC;i|E95ZS)$SYN<$1nQev5XT zlWJ8kFcs5#d>PF`z%W4Nof< z{pZQ~KNOG24?RQCA+hFzE|e;IGJ`@?amhwXF)(^FdIJ@?75wnJSbSI_5L>1&V!>ar zxLiodQGX!80g-4DU++(p{LidQ-%T3(V=l+#pTfbx+9fGF0sB;IeYJ!mR)64OiWkiq zf#{1E_x^S?5=P&PX~9%4)nUI(wI)RU{=XcZk42qmTGwDndr-1NL-Lu|Xv&~qXs1IK z4=M$hB%pfY!>kN3CS<_klw&5jp?avK<(I%@%qXPjsgiD&SsfOI14KV0_Lf=k`{v)gaqBF!{g;m~HKjQIuw@Pggnh3pi6X-b z>=$~EQ1cF-Zr592N$VzqfNU*4BP2?n2hMY)jI^Bu#Q|u9F&<~&N}5YTR5FzBCW*&9 zj?ZJq#ikCM1-joHxROc4qG`g>IJ8o16>&p?ib?Czmn@6=|LEzWSf-8+F;p%o{EGWE zwGx`?bR2_hDH{XOQmJAg(4eTH_~)$gd{g3|{+;~)T>6*K;K)H?Y)C33HZD#tT>*$M z=?E;LkZ?cSA90_jaPZ2x(Br>HzDH^a9ZxVR^d1qpv>4@&B(V~CCXZ>e>4PnJN9qde z(A(QLI4y?7*0bi7H>zQQh6XXUM96Ya6Q^f=EAR?YiD7|d^ni#9En0e5hq)*mY!>&) zf(*GQs7KNmB_Sz;@iok-qM0-e1^LQc)WJ$QlzHB|jukX0D7iS-C{UtFl@z5kT@z-} zC~Puoph2MmsIhv&N&mOgejf~Prh}_sRI|ViV-2Kzr50zgpJAA^hO{|@tqHFg3B$-% z5Q#}=zvl~8l-_eDC8)2=*nh5T0c-qAyzC7z{~xKt9Jf|L#E^CME^l~m)kM?FNpUolL|inp%&TS;TH3fxB&qLCWlc*W-T}raIP+0aKKz&# zpEHW#65?E8fru9vC?)Y#i?t-N09%Co4JUfxCq3+ml`EPgV}O+PQouN+H)wbdA5oN5hrPi{Eb z4aF=*M5CVZ_-TU#7Bn(IW9~|k39GUVNYMJ&NG_Y`!~&cXQi`IP6vAe?NJ%@N&n=74 zyBEWIQ~N)*|D^U_xc7(hP1&ds@8y-|(uIQ2`!K%aiAG`B3$&^Xyl`gwDjCcyIaD3N z;pVehVTKf?EB+sf*1K)4ItDtcuuFwnM3bm^Bq~-^AYfpH=ZvBLSh5bAlqFn*l2h8M zSY;HsLZiT{U}l5%+HAituwu%lmNhhmVM9TU3Z|t_8EOU-A(k#!g5?aKyEy$9Y5gNh z-#f0;tsadOll+tJ?h6KOnpR4emAY8XuJdj%aSfqc9*SGcX&yFT405v(U4sovw@fZ& z!%x5AP2MO520b3C>ZUpyDk%zti5(slPEl2`!EVFGf$Tv;3<^aCN4T;+lAgq727%O3 z#h>)l`52MT91MzGsYk{AFBPQvTZWoxN?fSi|3d8JZ|DA0gt*0$kCaT8z2J3Al^T&1 zBy~|qA;=h25GD>lXgUXI{39F;tchB^sij6%1BS!NKOB?Fd4jsInjpymSps&#BK%LF zXY}d9fVAOi+5Kr6t(eh_!<)Cz(Qk0{L{CD@8_vwDj`@iDrWUpr#X1IJjL!AumbKdC zUWcN^q5A}+0`#Ob$}17c{O|8QnRW4QQkk9kF?nv^6-te6VeHeCwV_26p*d3S=|s6D z=EH-GmngusPkos$lOmR3(yA$w-dhL#=j9(O%kd9>esF(jH{;KBUE5xb#g)Dok$IWI z2h_TSsPa@686*|3?H@D#Ip-Aq8Y9YlnzsfV+xaJUbnHEkwpwV#2*xBnfku$Rm%3Ui zR-@&gH#817661_n$VDh?lgUBd~jOcj$GA)Jhl8r-sp zjP(ir+Zb-cdcUEcJM=J@BS=4((jpgR&hzE5)xm{quxlVbEtiZ}&-rPRoE{HEx%sBsW2_l7BxTTO^2_qfi7a*B)*O0y@OfAX~ zr#c|kOqLxld~!KahSMXz?+|2%#4(*%OcCTik|uQY6=H*=Rt|?22?|N0)EZ%ei%??Z zgXcIpaOfc&-&TF6-Sv$bVg%U+M2vBu@I{)_`du0USjkG(2QwDrItiQ+uC zo4rv>S(2p@k`86KxA^=8R-(uwZHp!Wi!?HhWf=q6=8@xReDO1>_rP8KWyCfXIL&m7lkZgMcm0*<|gNz@8{|uvkovKYt$Xx?)6@#XC zsyI)&%eB@1?sub#t0?x1cmA9L}!r?dqY5?<A$ z9oc|vVyrz;t9BKvRDsAiI#W75mrHx|tw&|G)TO?>?N_JqsvHTlYX8g6Pw22+&xyn} z&CPuz_8qBT`IwR~Y`Qw>)HkRrG6Rvd9kIz!l4~>#0n!M+;J|^YYe<9qkTW7PMYA|d z5k!&K9m@rq)LW1(^hA})zGk}ZdDh)}2GFbx0yszI@|cWxiy%tF5y~vtI2R`p-f|8Q zGoIsacPm6X5mvqSfq;@WAnU~}r_CO7u64!xdwHu59<+)uaVDG_Y0gbtXE=Dh{)aKt zKK5GFi>T);(Pl7Xt{5i>V7y<92^%1*Qc~XLG9ObORF2y{l-HI&lO9mqPnC$V+RQQW zCAeP0$SymhwG`ciMl@t>68fV!ZAD7WK%yQg7r1$t=l+;Y!!Hp z%RD(LbOuowfxf@YIAFd2xH;}`0v!+8XdQ3RX5T5@KIFHLJT`w1(Bq~wN-DXO;=4N4 zs27O~*~M%x$G9-}#$Jj5pdCb}Jj5p%Lo?!_-n8m;?wQSo6aY@L6 zLK+BQ{)_chlNe717I1AdMu4rDK;J+W1KF@jE4s@#t4 zpHJN$bu(_EVzf~^83yI}3M#yE=kOCKm zK=VbI7B(EA(+JtoMHE>CW)-8-FQmVj!pg%gQmN|~UaFu6tGgNe8t=o|p9Cuj+OWQy z!@th^ozT!+5++08$^WZu0=Hx{7DO=PrrOZc_tMDAmMB4lzYoNV!xhs)f94?L8~}(uV(d9CPkoPz)|Y-2#7)g6V1hPt zOlEM@zL+sWE4!0~My&S&n9gL(;Nj`69U#V<5Q|YNE0`4^a0>xL*B2Fzc{`(XvZG&_ zWLMceRx>VXkd1eFwA2mvWD@rh*5-pG?|8`#ucoA^2s(~?E{pUssxQN*hSy-{g|$X3 zis4uuLmx;h?H8K~v%#J^g>hLB$qpP#**|R|qWuZ09%XnknR5gx7*ZSlBbPw1mL=H& zvA!6t1L_dsUSrOWg_lrs#*GN5&5DSpMsSZ|yZ3S%1J3oII3Xqe^u`m{7A-DCWHq7j zxuCW%cH9)ZkT6C^2xc4n#@urbsa8Lyjw~q_6Kp5bbp#b3VbZu-X@&rYJx-9~YC^Ad zhT&Poylp=iI)Qg6A28m&NP0uKbsL#?ol@dtw%MFuwkD8`>-@Xj;XX6=e}b6L0#A#v zS0m`;2u9LyCj-tw$1z9Xbb_?*<%wiT zB#K3txM2~ID5ZsmNGV}PDvGIRNlw(PBbO(`4q=f?t=d3dQ( znai=g^vXiZZ-!L3qb;cmn3Sms5lU#n8$+tMuHMy_Aw5}V9j+~g3ye(6xs@=8>4uQ0 z9$8JZwlKk#XPuj2Tcz7~5r!DC7KqjxHnyf?6Hjf1+i8}nNiG|yQ;!X^Nixi(DpH|H zrKc28DrmsSA|JODgCSCcS0-Ndb16cFN;9k7%2`*evArs8R9yJ!#bpk1UF{t&xQwml z7bzi07?MS;)HvVqahdrl+jQC-cfknL|sAluBC9omXDxB*O*BEg{VOb0}P` zFuWa9($7khxm2Pp$krTbvn)qc+O(w#L{h1^W6U(d(i@E#Wz`7AnI@*4jqhA(G^EQa zeR|=9gQKY-dgkJ%vANcx!G3I**e>++`P0} zqPcA`iQ08hs$!8mdWR1b+Jd>$i)%OQ7`cWF7WT0!HlpGF9Z3y3d2%EFBAY}}<3(Jk zj~_iKw3$Mf(%oY+sJ2E)DVupdsa<7-%v$6WB{;a+8cmpB<0SqHk{pcccAZq0O(_#B z&lI|a8&cTs+YZjM((dTZ%b1p zjELx;OF6tiTG&S>{Io&ui|Bj;dtQg$ZO+atX_utDnwKg|t+Ayt%U7tW zSsZyRIx!OVwUtppQK+MM2x7xjf8Kuk?ho;`t*dM3v=ov^B%vmnT9k_}vn*9*Zxu#| zS#2(xGh^48Y}kMCcrHBC_;&#H5QIaA1#pI8!(j;+N*hvJoO*(?)}FStQ9_0ZI?0zP z$!A7SYH4Lq!d{DIM|Cbp=i&xQTvkVeEXCa!A`*ru=<=8={7|;U!J<&jplX>0Ln;in zVct|xZZr({7u;4{tD}}hQiP5yV+nY~Ek!R0QqoXn9C$2)rh?kmO4>(ggjPr-x?4N? zl#N+bMm4Ro{;ceq6e|V8*wpr375w)wN41w9QOu-}kV{D?wv!ls!$^Z+T@`(|R+{oF z$Uuh%6$C~MA4;E^nWfStq^R*H#HZmY_pj%Z7;LmXD9J4gi&n~GTsb`2`u1Q(nB8|N)VfrYLaxS?pIE;VR}lhK5CzP z>+U1*Xp3p*Rrp3M)VFwX!hqbUP)!CHU_)_Oa1gL-ZUU4IwqNqP90=}Xl|*HeBb+OPGvoCs*peglq>LS%M=AcV81)w#!Uc=O`aG1kM1oK8XmS#VV4 zaky0L;=i!e;759PAAseONKS8Yk%V2>4~HoM>-35=m(!SWiWhcd)06ua{rC{C|xKKOE&EEi zm!@#T_Nj^x#ptl0{7IF91{hMOb*74wfk!W3mqkh;h`NFKVQwqk;+R6WnA+#HHip}t zXtYJ?Ddw1oAEu|%5*j54#I+@M+*F9#nlhmH++n#?tB_QeqCmkT0*IFckWm8Nqz0Zp z2Z4W!;5H)fsOnJJB!tkGp(46?TLbB1V-^<|qe#M)a)9Co0!@)zahcD16{nY@KvqXY zsdIM?w5I5T^S(S`wvfW9$qOTls49wr0)QuIrg|0Nb6%W`ksR_mj&6!G!2=?f)UBSy zqsmlX3cgOpJCzVaBJkoAo}v~q)8{uImy(Zy>@#p zQOA>NWt1fvWeY;(M_lF1sg0poS``>b>4v4$NdA+@d8*RXQjr@tTzgNik2F+@BBRDxR9jf7Q5QSP>4ZmC;uR4z?FATFth{k9&XAdw9KZ4ilFBfn3eiVl+Aeqe$E^83*T&W0qmDU5Rv_?H_qj)OiRUGm~ zMnx#z7bsAdzXL9n77C9w7ZqfwT2QKT#g!_HfVMJ*2}`DaKkyIPnb}QMEap>0xvDCf zxaqA?k`q)?)+&scWP+tEjNQ9+g12donps(%O*uQ+g)VKZ`%cCvsHo$m0Y``aY5Wt| zde33}LWQhw8MODB!6v5gZqH>ui>^Id9`5j}NokY$Zohh`pTOIUg2D5@IKLXB$=}29 zDqSSjXewZn=Wc26p5ym?#w>JSl~1S5N;I@b+=t^+YI{hkT1(=kML$}J7e=XyM8HqR zrckMf{tmJ*%zOp-A40#b_-6tj)NVta)KSrhXB?uR4X|xjb(*i{za}+4!5@dS@uBI7 zG=0ch5)PmL8R zfps9+C``bq)=BYLs~m)MCoQbRmTpCZYJG2rD~JU%1C<6n2o;77;hsm1Zkr$Zv7j+&Cx zJiSK-b7Ruxs;gqA{0&6VQ<%aP6%Z6rU#L-0(6C5fBLWO;zWagarZ>Z-nK^qW2gcheV zro}~RtV=Sf_QbVFX$~RbavnC>O~DmqB9BONkJBppz5 zWwrVjQwZm$%%Gu}Usr+ZG(N)_NFgvgK*iv%arzmEIB-?X=x8y@?jXtdkjv?b3`)w^ z5qYSsp-h%js{GtG`a>)vIo;{DoMZjgI`dUbO-{Kf3Fo)ixh2-(hryWC0_*ihk#l;6yalA8l>lk3)xYbH!g*n`~VYk-AaSCA` zp?=j{nVpL^^?T92HuG9x>J&8OwZx03mTjEDO>tAsyEM&{5h9r_;vONe-o(>ovhfa> zmo}o*kh3Up;+yg+EtRe$TY7sswwcB;Y;lte&NYcyl{})Rko3Ws4Lo6J(v>LbPN^Br z)iqv~Bx*T0;Ym2wQ505}sSYWWEzL&bWz3Zkad@(gsf7uOVp9Q4p@@;DBs%V2;;#L_ zK6T>0UZ&Qj%|={OwTw+6@rG1nrWD%?MK?(GQTM4K#+NKRP0y!?%txOOh9lKTJ$Qy9 z-K3tl{QZ1FFn?nmOg;NWaHE*mgVwXZtB?3S| zv(I-86#jeStfC*8!#nc3ap)A^1@x;n=S+HVJm8)^gX7Hyv|~}Dz^EQ@uzOeKkn zXi?IL_=!vyLb57Qs*`q9w5A0jMyUw?WA0%bn^MY}5;}XQS|0iJy@o2_{ddCF# zf`TLLV05tw1scSex!dd=PA~C>A=P%rnCc?}N3h8PN|JOqBMH+iu(gtrN^GX)nrx{g zR4D~O?G!`^e!n=T!mC*uLJ~{Za#IZj$}>nah`2$^X6EmJEC%v2!2rOxF=6a#K6)3qk#^$U`tr3g%p)jKR> zsnoHl&qe3K^d{^tO@}&`7(h`2qyn7EpvX)?4A&dD<%NUw>nBFojW<0FcQ3L+gK=4im0a^Ir*eGLe|a|7V9M4<_(Myc1t;_uK? zfvk%(u?>?`i6cvlASwul(|%dQ1{Jd8IM7eIPr_;`<`xij66GdNmM&m!^-*F)NmNv* z%R|fNCGsGBA@rMET}djH4=<5_xTKHf-ox=IT_{Y@@+RV&>SdOB=t?nUxssxS3UU2o z6*-jRKqXFs0~kIUr6~Q5IrLlRrOl~H3W`x9AZ{p#5?-W*PKbcmI<7^PEO>6&M;u|WqNimXK?p9X*rup}<_AQD1W(q<#+g*qbpLcZ_R|#LidZgTDwr(KXh4H> z@mdneLdZF8;wZfryV!R-YEvaml7uD-HAy@A4&CDs%-HX97SqtKrkb2DU6D+HN`jz; zQjf@GtC&P}jZ7$!OQL$)*>m<WN-Ag(Omxj5&zW7!@J4ew4|G(iDizm|PJ)gV3W{+U6k-l^O`=J%D8nPA>2%D(=2Ep2%GqmT zVN6krxs)@Nu%M`Zl9CxTpvs({{I|weC#rPX%8;h%6Wqvc4xvvg>Qg^&lgfU=qw#Od zHAEBA=D&66nCD`bDsGe^H7M00r$%$EfwwtgA#}th#%gmn*bxULfKfUY2(nd11VQlV zkw}-Rx>Jl9*_xFu(VCSyxR1-?j4*~AwV|!P%XZ!sN}azshkVZ(oBUq%n@H=tUMmi! zRVU?8k0PcHA?oIL^@%YN^wg3t%_44MoNkn$rrx#MTzo@Cw1*C1*o>iWA<_h#dxHoa zTd%k1QS40zHc2L6eTnGmDlok{xu~k8lBWT;=%EINyo?r?Ba(BXLS&(97u@jA4Sl`D zol|cb=Gexq?#pgHU`B=%u%O}sfrGhcIEy+dRNyWRu}~1{y86w_5v8m=6%{0O(3YXg za_w`lu|ku{wHphgh-MkWm8KUaDQObXA9RP{qiUr5n{I?t@+w0yBlc9aOcTk(JUu4v zDnw#h;$?PGC$G)L(aTTLQV4>IP*AzlFo+zvulN`si*nd~%M?lwn-pr2cJS+D>KI-0 z6qIg8b-5^xDjq>ZK$3)~G&$kB1wjWd$2+e*$`RQ+57aq?k`o0QNi8JvexT|rMO)eq zS8>KpBAqs3Nw;F8)T!fa#_v?A#vhaMzBF;Elf=&pLVn>)drvWx?yPwi-1m<09)FbR zKE;Je9WF~ylAH^)6{`|LiynSVa79rB6hP(}6msx6BPTM1ifp^39xV}MN`97?#{Tmn zGcy~aAfu8eG7l$Npm=-~Zzqg>$?d|;`el%a6}&2vMd z9I4Tch$=*UQqN;oucuXOX;qoF=j&at)cUT>E`4@A`};ROQ>tAc^LDiyXgu`ll%iBL z`Ys=?q=(WUpZX)uw@)7}A!9JU4)=k6zrtLM&mrtNgpw0-jU=9wobdi3?B^0k+k;Tm zHaZ;jWWYIL#z9?BL5dJ6Dykx&f{-9U=rG7jkcxs8J14?apErlRzfa^SMg$KPI*3t@ zDFh^v(rX6!e*pYwdIu^?Slra)LLAU(Q2~&k!kY>qUzFmS)M5~kz((>F(Te(7v8RnP z_!2zW-(YBvkIuZGFL8Cv0c|YU0gz@I#s>#CH*W%YJI&&%RAyCQDwegHrAm+Gnl&2N z|16`bqKYXrwI;Z@@HuM`qg4JDpYs>N{Ilqo9KMZp$ucXU@X0wMZy&cM>taU(CfC49 ze4xIt^CH84jlZc8Vv1!7U98D2aV(aqREX_70P1&3rD2GoKxCA1Mk=-TzcfLaNzoK_ zOJO9Gc2fqBK_#GcV@rRYdV?8+ty3=fW@k*BB*kPuyZ_0JY+B;o8!bR=RWVUYn^vw| zDw;*vAfil z+Hc!iQ@E)sZnvKTw@MN z!x6R`>yekjj~d&U3qq%oQw*?DksN`vGKM(phfemSsWygIv&>IDftLC;RT-cmb5ou2 zpcn+u#LI*G?T__#i@!#*S#8cpR;{*zjcVb99Th+V; zW5QQsW?Ih?%%y>M2%`W@oWu@AJA0!5Bq9Qis3*bjFW`90_yzEIxt8QEh7QE32?VuC zJ{HTs<)a=gPAs?O1sK9Pq6%FacOelF*e)qTX$Z+=Egll32WFCkCrwOD>57##RK()l zZ+c)qo8wZb*!Za<#XK=}Qe%|t+0F4!b)P1EwnarocMZN zeLk9N2Mx0Cd5V(=Q*0`fs#KJa{ld$XQw{q)BfCS^Ta8jDi<7QDuHg;IBEEg5)WhL( z6vy9xG<`M4H&PVCOd*JkNlRHDOj1ge`XNSJhZKb*#@Ix2_NnXMp})y&Nd<;;Lir#B zG1R|d+BeEZgPWq5i8}M$Zs`LRwGE67!N@R*WNpYQn~KF$L{xKo5HNlWg|ldA#7OmV zab}ufhN((WS1H*fEeV^F5E77DMio5GQ4}bl1763fk1+xS4r9`E5ek*^FWUMAyxx|@ zV9;wF5~HUAtzu2?aLeGdRNqBi_+Zpg0gc25WDvOV24@(gBUDxT~5pXsD5{c;s=m zMRiDWTT71Ar9@Skw8GH5VYtRv3`O=M+!(`3O+b^ z&H~s}uNuNfs%A&D<7_E9r=J+#BI6CS?kBXZE(g8a$A(M7(BkBWimlUFH`ZFAG^J`> zl_C6si8u0mP>+f|)8l80C^#7)a_E8R$COi1x|j7_N2tSUNvc4w$45GuO*UbUt_xbc z6mrRtjB(;1ZUMcEr#cY>3d$;Oo%q5kYL9kG=xLic#HLibf|)KlBq)T#&9=5Aky07T zpB4J|IZxyix{^)3&kqNEe)GyVo0&Es-4t|U8cXbHv-v~Gz~al*v>flSrnNc?%&zuP zgy1>~F@;K`9T;GS9Yhlq6eb@HA#tCFj5^oqXA=)Chv;ph2to};D543I$J)O3<_{S{ zj!@YonPb1fPWZ*mjk2}F;9$aXAc5FK4a{BCphtEN(s~1ccU(@YuV-TXp84D`PkYt! zA9ChWgeHX=r-Ol`aVf(lFlDVr=v9gsICEfdpl=}lu2DM$L=Y%#4E`NmhQzA;y!$b% zK7>7s1f^Mg7FfTmAvTe;jrM;VS=r~N!1=9AAaP{s%0hl(LS}k-rkG1eN?1hr!z)mf zGBVR6YbQpl`PS8C6j4$pQXpZ9kqaWVEimGaIRz9>sY)c8k$;?5f45g+R;g9Jzhk@p zx^J&`S>QjD4Y#*FpX+~k`cwOdKAqmKzUr&@4}VlzrDS$V6(tHI_eXKks!Uae`8%Jt zJSK)zxl+ZBi!f z4}+Vek#u4vToQm(K%jug*Sen`N##aQR*3Rao;-~+5Th|y|V@O#TQEW^^j&Y7hoR1oxDo&*?kvccA zBBuhuelP-Y<59lPuy9Nhz@Vt-7-3RKDJpghG1PYP;7P<&9L$hV;2>x!Tb%}4h1?t- zhbT*;B+J#Q7)KY>3lebqD`z{4aO^WXO7dWJxW{GHIm!PM1 z%CwvkF=TS&Zf+20b|eA{Zm6CiL{pGXX$q1)TS#HFDykzlG~e*G+&!9ki+E||bH<*y zcW7x1MmB}d6o}IZR}l7L3*G+*Z{hq^{twKb9F0jhBpXW_vRsoloR>E}p|sVmHkcIo zhwA=DXhlB$aSUHdTz)hu`4H60&A7u1Tx;xk$qu{69z3|qWyzzhtxBEc=N_`stNRX1 zQ%)G}<>gY^os3&^w9DIKq?0ith;QMAor-xC@e0c>D{RH8r>b2fQf<}PQIs&{B_UCV zRKlU_O)y4A)a5Fu_SYnj)>O7Alj0p*RK+r-)RJYluS}`0Rf#<+2a{el<4Z_wblS5Q zm37yRN~|k5so9zF$Kigu%ilROfx*=sJMV$fO4^ly&mXOH5eG?RQFKjI#?WGr%w|HG zq)sWby?l;7FYRNjZMIwWQ%Z3UtBC6@rixs0GWO@n7sgeGe|>#rYCg-5*r#eqH2yd3 ze;M}YEb$w2>tEV^8QkyjI2Ae;I+CR(q#P{mQIk|(2XXcms$Cr6;$K6^REG2HJ<5)f zLQr;~#LD^4RCxL-x3lhd+@FW&(jTMxhmba?sSMhbm)Gkt5U; zX)Z9IarFI04~*!JF5jT3Ka*c8#cfis^XcmiZl;v|ie;-*rdwJ=Cj3&#f~Q)k3_U#3 zBTl&0ad?ozo+#YY@wDfYPAHu$TAWDBeNynH1tKtvOHw4W7*mTXSwfN*Es9xGhpH4r z1|&eu;jS3Wq?9OKVL3I;222qiw)*wQvj#EuF|#u-lznyc)hTjyLTzDkS+pna7FB5`gsY40f!-Y-f0u< zriw{chLIUzQJ>RgPo=RK%|i%T0m`XVP!PaqF@WEZax2T4{XY`qM^%SW%%Y6&QCFyq zdJmwwb#r8`O$?%rDu{FvPHKn>D~xF}AY~JPsXawX6kgn|PdUy|{crrYR9ot}99c=Vbb*sYw>*l%h=vSdNs@pFc{wN8ry? z_tw^6*`yF_i54@UCmGr%Yy+MjCbZ?Y7xz-jy_7<*iFv8bgdBlgOv+ z{?gr26hK%!9p(7mGsDJs6gVW1o08N#enr@08x>`aX&eqlr3}!IBB4Zjae;*BhgyQF zA|?l*rVds&D#=j;hO6|-S9sWil1W*h@T7_zU2qi?)RNLDO76CW%1el)%+U%oZ|ulr zn&YLgyuL;C2P^o)fO>h-L;CHE`_IU={v%|XWF=HfL1bLG(nU@tUA3?!wve|ORHY(l z)X_yzh8%EV?vBXt<9t(3T1E6>lTLR_RZ5adN}`D;+`q1&)GDdfzxCO-j>{LVVBMmg z`}$|(4b#8W%-uSe_6(i=jLNB2A<_=|(I`I1gQ}IeH*{f$6ws*&Sw}XUtw&fmTG`5$y<@FD{7>Z=?DM`e zN|i-cHyKR0*u$f!%#>H$k)& zG#{&vCflyjQc}*IIIeY^pM9b|b@-21Usg>TRc@7TDlo+frwe6_+UQJ2c2f>Kbc#dg z?Xk9!-W2VA_WyN__U~?f4tsYEZgK3D#k%NVTh2`SI+!aed5E;Dykdsc&{`^pxbg@@wNeZMK=2c!f_X5#q~jq&8LM;uv-+GRoX>nGbZr zgm{Kl>o)NUJhoI99@6za(wRuxs+`nF(-zY**`l4RiT5Z;f;xFIwq*>VOx0VjZJPC2 zoaD^c6*gz*#5%!BwCWb;cqK?umf9O4qkVa%SdFl_k!whJg)6T0bC|S699v8oL&bL` zwGwW*b}ETCqHyn3Rp0zVgOtD0{%WE2Vxg>G4GwxVZB_uNuyD!F%SdP4H zx#+VI)tX^sTTdAOfqvIl%ovJ_@r}FVh5cu}S0PeJx>FI>R7@#&l!bfQPdI!-Qgq6Y z_17E?3lu1FqafrA0uzw}PTouR z*iJFACg~!?jCI&ksN_`c&|FemQLvkfP;+A|#2Ew~sM z-zwvoGQd#|v6Ro`%)ZOX`3uVf@~hPuRdB~xZpvAO`i4cKDrr#er|Hx4yvd0(Qc0JJ zQ$-}SZRRT8G~P_jA^jJuFsiA^bLrg6a~hWQ509pfIKCW10})IVWO+^@rLjU?K9H!j zBza1fhw-@UPZ*}yVodREd(0$b;v1gQMO2b@JVWg;^J|&8ycJKBCT2*om_p**btZbY z14g4qlWk%0mw&`!O*{Jj;p>mN;bvkzLXjKAc()3i#;P`@w6_XjNK|JJld5|^Q=gL8T#bwhkVH^{l80oQRrMaX9-`shlb)pxK_E8?l#x+PGN959C?XK#4^ruJgOgGsKTMI7l+`f_ zauVle%qz3hH5saHX%`_%ntMvMu~HJ)sFUR?W$ev0q_UL7NHp;39+wnQoMn#Yq>3q* zcBwNMTD??AwKRq!)hMY_l@HyiA!!(rlh=wnZ$5h~CSM$CRdu4Ac`+e0(w5UmW-W-g zX9IHza#DnplWkG0E=eDirOs^Hu(z&8R=1L;w!c=zFrhegsg#atFk#_ac+IL>PY%P1 zohm|>#$0W^sZ7PBk_r`)CloG-F0Us4|Bv8XkoEq$Q}R<5-Y!d0l)Mj&p~LbGq!UDf zN-VPw$!Rf}W=3UNWf0QBC6t#`IL&Wsl~UKSa)Z?zkxG@RCCjt0_oZ)my8?7*nwd#>jyOKdLkmViBr^F(^k~0!0 zY*IFIW-k8NDWa})vMrJp#-+?>3v^=cl_aU^s&U-aTQK3mOm8^gMpLk}qhkB{kxPv&{TbCT_m|DT8$$?6(fH{qp$)e#++GyXacoFtYw{}GVIjvBpvzP? zsUzP0x1Lw4wk~F-JOxn>ITp{D_ql3~G=( zLEw|X-XTu+9n@^&Q;9wER8met1F6Y5oN{<*zEwQDtWKntN)t3KMUlrPw`D4Db#coi z5Hp(E80J+6$+>ZvRRt7daC32t3a1@X8=tjKl)74k(?!VUT~;Q&GKH}ukGXTS!W6oE zC`?88Vmv&#O*&w#ts#*3wxw}j8<}$7-F2jr&Ke~VQt@)KjSbu68u8-sFWFG=?K}z! zkX|Hvo;Fk9kqd2KLxN;5l4NBlI3UP@z~Vv}VGl9*=D!PkW;Vk#iSetA>Xhz~w+dwu z_*zV=BC|cBMMmVOm~Ew`I#W%|ZrekMb;LWFn=3}@PB|=0q4dTaRkb%wyX4;Ilt|gi zX;CKSE3Pe@nx(yS4JoQb&X*J;ilWp`s!w$1zFu;^J#CJ0b5mcdcI*SOTodL<8wZQE zd}?^KiV$!~D>N^seXMI8^1|+`;N6XljZ&btHvo|kw<9pJ7z^@FYGkQB#ffs9PpelR zylQCiwpocEvZj#H>gLlck0?i(Wm10OQX?(>DMd<;WGxYLb=ArHDo9%eQ`?iNOEdY=h2yt+ibe+%$b~->&W7cxw!jN#40ltHoKXkPNCzQXRIq8FKA)w z3&$8jsSZ6{X*EM%=GM567GWl)bL-)b>$+-6 zyf%eNBin4la*FXtc^d3ir)OHflhznr-Z;mr%hgTFU1`de_F?0F!mF#E^wY+tJ@w6eN zGN&#=oNGh1i8d*976c#^sm=L_8OoNZ=pg8vx7sTUF_beA`7+mRno%d&i&HYAak_QH zE}mRVqNS?L(Iv!p=U#Zfri+5m%LsVEdP1L6)Q3Iz`6209q&Ge(jVY&yeLd1c4KLL` zk5>Aj7Wjsh`*)>7e@s_vzgPCP)c%5{OGtJy-G@Y~cUNR`O>`v&NHoYYprVB-BBGTb zsHx$Z51TW1J^&l!>H2b(J&tivoy&K$P{KRs0JYgp;Z0jO9%d z36@Vud&hZCRlr{UIkXuJBpEn~YHETZu}fTM0OhGt;KK|MRDlpZR|2&gRrKWJ>gQ60 z(oH6W9C#d=Jy|AgLvxC7T&EI)Fd+%aMBEhs98P0?7-5MDC*pjcnWpKrM4BmI7mCUf|>rBVw+;Pc~OdYEQd9UnlbylS;RJu@^ zqEM@L6g!(r_i;;g%IR*R=@)esDU|yE#z72*+VWN7aPK{vrokxoDBJai{5}-@AKIUZ z-t!&K?|Zx7c}fEJ{Q<4#wA3C7kVPJmBavd2D{svr5lU6$il4*$Gxj9Lt9>+iGUSd) z0|g2qW|5mbjzz7gJq5jFQ^om5vHr35XXyS*e8=w3=lv(tepA6~j)MsPpQ57b52PkY z(#)kcDnzL?iIIW(NlOC>nI?ux5`xpj7$F_1tNK3pdr*D;|frN^?WqN5b6mMwu9$?!QTy$Kx%of5GH@2B z)c5ZD&&0}&&%O5{z8f+c3AH+{u|+U_^p#MDOt~{fQYNY@X_V~+EAeLQi}7;t6v>^` z!LA`sO`)BckivFC)jKD<8i^miRAN&LL22Pa-tB$1*>u^NsM%zXZHFahn9F|RWtF## zq1KgZLZw!!t)?j#v?&ZsHAxy&k}EE|x_r#bDs9Zh-Y#x#V&-Y$r>#d6k6v7PEu#|g z?i*7pi;IrB+bz;1i>J9m)eTr}?HSfevtsryIzZ4ZdQgNeW-ySq4_I9B%UL^d;Y`vd zPjadiN}9IN!WuI<%X(7j6O&Ton!{RHrwp*dLWKQFiB%8asNne972(4Z>X=kW`f4lX z##_!LyxQna8EfkbV|iXYW9hd&_3r0RRXBIbsZMTrDtkkPE~5&F=e6V$3~MfO<8y2w zh^i#^l!dT|S*ewnRQ60a+^w~=hp$*jdfFp>R_BZc9cNP8!wgHMAwn{S(Z{Pu%L?kS zzOCmNEeb_2;+>*ar?nL2HYbQAQOJV;S=1V=hiJ8Kg#D(z5B{OUzOg1I56yq@_GevmWFboTmz1B6p?6 z`l_Q=?jlFrrXy1;YLPE4tqYycXPMfuU3cSkj;V(Fr24L@M&55OS#y)K&No4dNaVXQ zA%dE5MIprsRZ3cwB0N(OQ58E)s#MyXX>5rcQ#4{?Nf!kD`lWt{(=D|AXZec%GuPgH z`8t##AcVnYas~K{@Q3KkFJ4pZB@*t0!ER7g6?9>C7__`$7DH6rXok^QsA4^ugz;Qv_4EJQhZWFQf`qh zZ4Zj|PCm!d(sA)l9(A=ODM}9+TXC4Bwf^*y`xL4kqUz4lqEb3MGJ{P{6%)VFJom zy7gni{Dy<_9wT%y0tEx{Fh<$SdJ*VD@!owxExqHaF!{g7KYQ!L$=*Ik>67U7`Mmyz zmL&7$rDj#$eB~%G#50sd9I<671Cp@6pLwO}1Q0m|nzpTAsRFe~A(#G-#Qs;)9?T~h zx1WPSBo0X<3j@f+uEixN;-!U2tQi9KemI;Et+@~f82^gV-Z3St)nAIHq1m);L$IiE zagTnf144Nwig$^6sHcFkD8m-1D8!7ZcpP{yMM6a?B*AtpNmcMF+^Sh5Ys0|h1(8;h zhDl2jNkmf@!ln`iD8k}X3Sgk=G$`QCNXZJ5cvYH~lISX4!qBo(xF6blg&ma&J# zB$7I$m`fBWmAooZ)fA)@rNK%h6)GuAA8FQpK20l4ZZftBS4~Z8p<%-kpSNFkUQd@U zmPn+Gn{|d+dZ>{}J&SKDLs3)ON<*a=5T-lGpmYOGCOreklF_*VR-Mv85er71+AN)|Z^P(cpjWbXG@?E6n5 z#FA)|X*4`ed0Pb!A>-k>lZm&5T-iJ3#Yz!Vkzo#Jn$<(eQG^gsMiA4}GKJkFjJo;e zTl(Y0y3OA7{?O}EntQ$++fzGK%*5Uq?MZ4>RHZ2q>2NlMILfEld8{gVDJN9Kd{RdJ zL#!#s@A-$jxrIfdlJ{FgU^Inwzq8;42#rA_p-6(4gY5q_8l83L+fNIFi*^+g~ce4%#}Ai6a)x7eo?{7 z2nu<8YxuqM%jKICDw1%T6NOXb9WUoIeND+J)@*8yRB1St$0#r*LPkLl@2SZ3@Y@Zw zGe2UPQx7)d8E%m`DNsV4WWpF_nw3$?jd4+^KstG%@C{;Exu&0U%9j!@HpG}*RW5|O z!zPfzID(9Y$VM3qp+xdzz-!%KCxGygp(g}`X=BEU7(83>EF6(S$;4PNRk#nuQ39d@ zoQ?=nO-3#8Y&~&YsfEQp9K9)JA}uDON9R<2UACr_khqUl_3P$W+@jgj*^E?Mi*V(* z^NUp4%2|BU@luq}B`MZ&ZlXtL8E$gtaKPO7(3z7|iKvZcrK!`3aj41`tVmggO{ytL zF|;(>bg2@M^$JNGdhU?N6(&@qY~t7`>oMsC(uJi$#RvcHk4X?>(njl zPU(^^RgrIgNur4|RGKa025nE!^V!;MSBH+R%}1G>p39%J^RC(Fic*B7H0h}_8D2b> z-FBUwlahxdk>X7udj>ey%u}AE9ET|86QQBSD5NSP7vqDWVZ!7=Nm114`d>=OjG5C> zg-9|*M|yoRC80{Bi)wb4Dm2mAL=}fy|1MZjfSF?$vr7M^M^z8Vq#G=o%+c z18!90qbUHQAl&+fNKv|;)zK(HMJb;CnKs(S6HAz*qVp(JKztB%HpLyB=Y8l;;U=NCN-oKSBh)L3boKeaW9+=UNr?EU z9b&Sk7i3jmHEn0Fr-|LX)LA zOvxi}>wNgz%9}#EQ!La-w2b-tnvMR`jy79NDnx#DN8WjS3+Li#%4$$lgApf_%Dz&E zn4TPPF2_FAlSJrL*nY#YH}54;E_V0VZC6wL=d>29>~r}n=*@zOY*r-ek1 z%EX7*_w3W`lvzPxYJ%q9a`V2Osv|P)r78T$cjqkX`&Jdd9rky}>)wS)?_Ot2pY67= z;%dlq43qXzx9QAChfS>5cn9^XR_TpXEVzSy-j%68~&%-#4 zo&d8G!~u*P2NUXTZ3Ygf-fm4zfYxvQ4@ZLJRF@)M(PtmBMna(&avwy5q)2PF3G67> z-A+T0>t7sCy78psJF$d}mR#(OS%MIY(BhBI`f#goRbJ?Am$v5@P)f$$PlrqCZKA4e ztLIhb9p04a4Pv7jYC_dzrlb{`qsw`dJyQ;H#U^$eMnbGsxQTkynd*is%|+# zvLM9~qSRcLYsYg0LkFARTRULXGj(7#Qth5<{*ue*Rg6`OpHDb?3Ck#8_Jse`xAFDLl;cLl>(n`-M(#sJ z?`|v-gqEQv;60}lu@DOm#=db;Vo2hVy%8dVU`^W<7SrM=bc@zMS}h7$;PO(N zCUKY%B2yHm+0NAUi&22o7pEA`1#qBapu#Ha1~-CI6_92$nPb?0pBY7^q|48@F$Ivi zi;rY31E{P<{U!HQfhr${36-@Cy5fN2j3N(j*+Ct&At!%d;!CcLO2O(E9v*6eU_ufj z+PfJ>D>YmB33XxFrSWdCa?3^AU9iwV6@eWrIc33an2pC~R^g0TC`v}nY^z>!(X0IO zFO>e?QwQG)G#_+#i7~>c-jye1?GqmbuV&@J4*R zE6xrBVv2fSgjt+qoL*EkOYBBA!!2K62dV8I2v!~xpGcJlbeB^$x5ulP>8?G77YQbH z1%~jJF`6q$pjCJOC~~*cZsK}L82ej6^)3Eu=?r4Ld|`k?u;eT3Sgct1ZmUT{?2{NN z&tou-fd+dh+wP0Rg5yvr4w5UN?Py{>M+jVH3sIwk;uf)-TS8lv7c$g2?U*K&@{UQ* z6&`^$lY}YOKM-P3N?U&lhLM*lI1nnE4jJq}GL6TCTrml{i93b3@ZX!oU!x#-1xv)m zpyVBbn@JKZB?QZf!HFnD&>c|V3k$5IZK*_i%1%;si9|j`hfOF{k{w4OUcasH`Txsy zxp+rC{dvc)nV#kPpR4RhWNwaz=g_S)laGJz$3MzkDVFm0u*(n#%YR$1z8`nxz8%NL zt%yKoM2LSMk4f@5Wifp>0{~BqqJ<$=6kw7^`$@SZj_vnY$DXP7x-{AOeH|U@h(pwG zZds03%?y#=Glis^EhgN(kmZ0The5=BJ_ew`5Ttp09gaEDF6Jo6AL$5Z&AAeF%EAzU z2h@lVae5ezBos6DqnXU+Z73`+`koPhNznALAZ_jKj0M3!Mp)1TIVP}1tw<&+OsOQP zBBP=jEn-SqYzzV)P|vE%}CQv&vVohP_HhYo}M?=IQUT4 zRzP>NHl|#BWj^Fa8TK-0ZXZzw+rytaot(^}v;5G^U};n%SuyttqRlz(%Nqj^YS8CM zF>lm#1ara3Q5iCni6#U*xMH#yCJgKHMUb?Q#BktYqpx!0sH2E|#E7uq45`ZnNuw3T zy{}p;;hbko1xucSZ8SAeZLoSsB1#G3Eb)<$X zN}%qLevwBtB;)LS6=O7F1QE`epwyI}J%ZP8+$MhG#&*KGOAA$Zb?5CLOn?=4o?bhb z$nX9r^7W2)wp;hhY7osyOsmaS1~w#y>26Y`33Z2@7`6ZuiD z@`;6o3(3S`>V6iA>FLUXrE9}0+kR2Dm(wY()XA4dw~e8h>`JUsyf8Y&84tCGwwI5P z-xo05*ra#uKc2@-!<%@Y+EwLYRm^2_Q+6QdH&sXD&C(m{XcpPRjwB*f zdux38?c%0~z{-`NwfxF|+v+o4I~b>?Oh|+1>?Z8ks5Qzv8$I6nQq6j1f^UiJ!Shkl zLWL063x?m@s>Q{h9xm!0W_<2#%y~07HSt0jJ$84Yv@+zx;Lbxb4b~ZDF>}PntQQ#T zgt*a{>NLM=28uMFn5)pzdA5@hO#k(RRZoYi&$Cx}B@ z27=gdDQ9sXuyk5f^7XqS9Q&l=q)bL@)AXpiHNHZgoWGI;lr9Jg?pH-$#&k_$74a_l zY(d%>nr!{_4Ec0g$doeiQZqs7T{-hz)xh)X&gdX1DPc9^cf(gDVzh_uh|;vLi_;dO znaW?=>c@p=$wzQf1Vlzq(&!1Ca@q^YpccU%3^v!KT!@^-Jm?OWn}#kHLAj;wNwj-S zoje${yVR`{AVc34ad(N6>w93`9Mn2EM$S#1a4a{ym~4JJUO{%@NF|QZo*#e3X>~YW z_2N5)o%tc>+jqAD8S~Vtoy1@q7il<5)sGZD>RCSILzHD=iIoCjtQ%#eAnF0VQbD3F z?(4JW5Ed1x9;AuB_6z#v8lX_cgA@eVb!N8(CN+JS@#1ZmCT-#@=V=Daw&Eg0fG=em zFf+-L=af%d?=aOB^8hNf*u+$r>tjz@VxtyD66cpoCEEbChf*rYBCHkgOiAf5-qbqX z7henY%!*3D?TD?1j3=wv?S&pbd?f<2}6SvE`vgX6A3u!%C8lUP7gVP=kIv0=&h zn$-UPDVO*E%Jt(d&QO1uTj(3>7DI+W^+k^kBQVarl<(=^~IJP$jqsh)- z@_`re2%&41Spz}l=%~W%a@g>J#QcfVDe{Svt1YsLBppyK0cAnjbEF(#wP!+u zepO3TIraSk+B`rG&D-g8ujmnr;Nz-vtO7nKA`AtsiJ>{T&sJSv)zq`Q*-vWD27Va0IAf|sf6h~(Cf_&Z5|34~kd*xB7w=Z7_o8$t`^ z_yyV})r+(&|5HiJ`2&ZYN-6uvDDRC&@PGwQ0S3ndgASwO!8wbEog+`_eM~zyjnfvm zYVP&QX$(S6J+nZQv^mEMI%^x>; z@H9eWj{>MwD_t}O-alAYS`RrgPz1f zrtj3GsSL@RnC8$#o)w7nx%;)9dd>ZH=TMN8(%>Q@)F!r-fke^XzozZF=_yNug!p-pX>C%==Ml%{R7LP|e90`2$*VEwXvdhPPeX(SvrAh@l`K8qs_YYsyhp0_( z&U;FSmicW~)mZ1)?O*NsNQ<`BxxG_+$v4;++zdQP$s!<{+aGB*Ju;(tu2qqgAM;A^ zj9ML>@q&0SQ`6E(8gmk-_nF-;;zM%x*lBzKUy z?C9aGInT~mw>H@PMIWI|#nOlFGXs$y6#}C`l`V_w-CzkEIH7d`!(1gz9M-*$a>?}0 z?3A`Co&P*g7|&36O-T(|*^|o!*#$XCPqCvA>vbFEOhU^vs4Xc#x>Y?XxA^?qJVwkjd2QQr&86`YnYz) z^w%E1P<;h|mu^;S;{jO?3-qTLfSFaLP_xC!=wuzPKLT31g=3yyOnrei=NmE>@jzE4#m{Bz%6# zy0r$VQX?FoNuC`zFCN%EsehsD>kNc$zq z{I!LVYNOtedk8J+jG8)`-6?SHVSdp;_Od)Ga=WN?@^;WarLe4!n4Qj@&pC5WC4K$2 z_vxnWs*bHwn#t&Rh468!&03SC@1olzwvSNzP%S%X2sE}$-)@nHzq;PF#JSfbM8AFP z`P4p=K|B5lv_9rgKD2TluVF+qw&t!@0<)sT{lPNp-f2H@xsWs+dPPJt1hRI314?{}&U!s)pEYNs$b6+#!*b5Nj>BbJD$_cs&yjD-ZKMiz{c??8yZ&$x z&hAy2=Q=+7x2OJONTkFFWs-w3kc0rhW7t?FhLcKoT~3qY%MPha+l}vqR&)0wYnwK? zPGz+d?u(cnWEYpY#J7;UQAjmEg)j$9{76ryb;jV9n1$fJ~;jNIhL5r*k2H3~%p-QcSApUdzR`?nC#wR;l%M4!z}_$znNy1D4+oL?~O%L|*gr z=Ia=S^prY}tO0Wa>!5x|>ova`3AQg($HPb8!QgMMYA}W1r!}}V5SD7WZJU%@zDqrHbp89!MV9wBMJr-LbZ@&;F*?wbzR}WlfgmdD`2xd+?EhrLE*;hKG=IoH0 zDEaFpKwvfsq_IBT9#4)SXEf-n4z%k=c90A4SZRseNSOOq$4K=kihU#wNCJh4_b2l2 zgJ8^-qmKUif+9}#Mi^!|A%Z)g3f7VR-HEfb7#;q+H_{r}U5&q_PtFJoL!r6OZ^lK>=z9sLeZtm60ot70A z>Yg}+JwwCYT`Tc5Za+N|FwXSlxum(XWu8o~%2*0W#}myTBzWqPi|KWIxJOv`vppoE zh8>FmHLRVrUD)5AZWlkZkQb*6$0r;gkHDS{uMdV?EaO1pKc!1z!hI21{$MNhdb_-h zm0kUqlDt*}2ow#y=J&&%zkp)%;t!mw552OJzR!e{9O=)NqWl16)sUCAD|dv8juI-IQ{NK zU0MtgfBq|1LA81CQ2@@@4Bz;M;jc*rC1U2s0BT2b3J$@ z>*uQck8dfhuRQ0#XC=+=qcsQl@D)B>`Kgq0fmzA7tSD38X%@>YEN(~1J&xophOVmF zIv3mv-L7S#U6-m9RPu*DKh8{c&Nz9}m^8U0E6#A$hy>qRzf{z4br2+T+TQn26!&z> zK6xtB_9kw=z0BG5Q_SQJ-aA-xdB}N*SrPsQr=rvdd-ryebdHN_uix<*ztPGgC*uTpuAhP%TfOgi zwYlZT-hbLt7;%3ZPB?%(%vi9d^wrt@Arh*E2F5T<{&FodWxhd2@suBSHmWROOolwt z#AQCrj4nKc3X~+?6SOzZfAY;o5HwGwM;9F=nFRTKF)DesLdELVN?7L^W5NBs(NBY8 zX@-RsvWvaFiX($sM(<>7R7jN_T71bdLzG94`r}lQxS4Dz{zn&7QJ9h*GEHHW=Y@~I zd$(EMZ(aJOy1{?Hzk}R;2<1#>jIZu|7LRbu{XWEifyDp$T9%YKCHCcdIvPXA+Q3x; z+T8!6$LI2{g5y#O!p5EZ#TA~ybXlEh!Q_?o#Yf-z&j$>phc~>`bOCN~G8x4yuaRv* zo+2CIuNABCUB*0Ohqih2Q^iM&j%bJFEUn`XKd|ySv=uh$EW0K-W!N<{YrS>1dYP0p z!wifzresn`EE5L031tXLQasfyB(g9(+@9vwS8v|j z`u@cr;YwshDyFXjN&K5wX0-{m_j<$~p=IhDI*@bPVs)&sIO_sG85 z?#7YTxOPQi2o_y@&|!AHGwj+iFrDgaQGKc2?C&XkzBOC8*uQbkw|ssKof57sW!9L2 zoghx8E-f)jwk10HIGZBcDcL}$Hn8b|J7prJGvcA~F|~Qn=;k=Nra$ks0drOp7vFNF z#{B(Bi$*Af)j9>Hl`^!{;Fq1Elvfr!Og)Cw6|=psaf7en(Kz0dgeAuPSd8&)H)j%o zVKinUUju2=PMG9hPDSHjO0Ast&=0}0t@=jat50HMjE;}y<7Cbl92~>r(}zC;91JBZ z)s6+9{noi-A-T+N9TU#aa63ud>DaPHxt~7xECunsADP%rUPw(s`U4d3Q&|u`b`t=( zH3+>)5ANLhJ)YqYz}n|kGC5N|-G(E=)(9szVhND(8~`$&e4vF%)mZa~z>>+f>NG_G zwjYr+WVWSj=<1TtQN5(7;Zpq|n^M9iAQ3q2-tyss1rQlN`b6-Bs)05E z#8wG_*eXU{WlnY!Ac+&-A74j<*h_T~*qQO4rvO139nr}po)jglHWq#F^}(UFx=-(z zza8H)V!$um3@!X4eWN^K1JP7)HR*NnNg^G=>?5~fM;cw@LT8G0C1eZ+Qv9MJ;Zz@1 z%!)+$?EPl@cMLvFB^_EVmdVfO{4a=wOd^?5LZUJE!FT3e3H8MfgVJ!mD*Nfm&=7S3 zkjl{PrEtB7+*KqX>e^t&_r$F;J%yH(U>t)t2%)*@YIE2c7{P6}mCuzO0c($H!%oUh zb#2Pf5jI}uX_;xTKJX_(!_?1Lp6shCW#C`ZZp%H&T~o(p*6&L0d(n)k_)P*vt(YbTP$e~oM5Z<*5BXIT!`!f1qi$%A` zvPV&e3#LZ}6n3^lt*XH{9k3GFGIaS?DGyA~ru;|=&yR^u+#?NFa(J)r_8)_!=zm3m zWnX*uXlf$)N>5Zx!Qy+dS7pR%*1~xG1@EKoMRCtWUFU0UueC%f<`wwqXaTf`v(jNi=lg>ia~D&xZ6(q&qVw zzOmhVheJeZE_`~e(KGvm&G?1f_prHkw-;zWd^t$U{6Xj1&7a>FC@nli@?m@`BKW>A zO_+~unSSU%e>>t(`}z5>94OMnd#JUn2`P`vB^G(yPXL=$5oLp!qek3gFl-Qaf4Bx8 zcZ3}o8yBsg{pK1LAD4yiwHgap8kg4B>x1pjfg)*^D`NxT#>UN<7hk$p7q0}MG2@Ib zWdrRE77&6B5+9I*VZZ$(`Yey(02l{!>mySz;IsPHqIQ(^Hg!*;7AZI@KUcmG&VN=u zA;8L~?VpJh~q@{Q-R|E9H-pG2x9MtX)CNiDuQ4? zqq&9t^lx6*s?&8Uv2?vg<@f-9mB8&Pg61lPX3kut07C{Wuc{f9tyCTMqoqJUNR(9P z*GNTu7&P%VrF4lR)PXI*c(DTcyaL7FFM^Gz+SV9p4qraIpZaBgk~O2BonN3xVk&(bX8+P{f73#d?et<% zep{uU{@iFdv7~9I(R`>x?=pO)@`StkeHK~r#Toi?gdo`^W!zqAs4&&KO0hy@>(#UU zi0tHo8^e<0Lus&k=|f$3I~K#WaNe?NrJcWn#8hiwF^4Y|zmBm~?guv+8`tLhsqMBp zZ3b;*R`Q;_AZY$a<`soQbm|HY7foHm29D)^UR_J6s_42A^-+dRZ^3~D{DcVz;}(-! zIk?wNwm;O^5r?MrO+c}ej#t^N4k@1(xjbEI5}Up#Z$G;jWTa*C+rkPES@iPwguu4( zs>O+X&%R@+Q~0UfjqIg7z;pj(ci|&{*j*xk-6`*dQa5ic(~}=_EG2TLLH^P1j0yqT zT_!-g8{1#J^tJel?8~oo%z+zWo6i4D?*wgczAs&QHH`0ZNMut}EpFzeC#zCsx)cML zetnJ~TgjNt@Hjn^Bz0)Vr=+}>3rSkpg^5Mco1q$MA-bED%`n9^z3 zxvV>jkdqS^vb@R7p+PmP|Ce_61M~4OEg-J{HhBd!*NL*(_Vq�Of6ruZ6-C626&_)yqQd)F1z*gWb#;N;e= z`(Z%CjS$!=Q1I8EEezuxkCZ+u_Cb%Fraba*>G&Prj)XL?&n%Qa4^BUD1n;pHSoRLf zz0`ivjK1c%KC=U=0Y6&qG{1~(nee63t8HuOVmQBQKGMV z0~sFWZ-!?fWl{T+;oX82MuYxR_D^^IDEmN$S9mFkfmKUL_X(ftZf!Fj@sG04tpQZ_ zw-1zQbi8`@5HG@22<9nC2uu4v*~;_xa6$i{_i!mpywhj z)rwrfndeo-$n`{tlL^)Zfl|i#6N#C_M8a>pWn;+aPE9YYmx8ULKT@Brr3rp^% zb2Dq^Rbkn)RJ`+|_ZigfmoJoRlV6|jYdk#aq(mbDo`gqt>bB?e-6gyzMOF0XvP=t` zbJZp(j{%%f?7QR|Q{BxV6w4OZBhwk@OV*cEE2Yz+vrO{FnI!TU%YjAay+)>Fkt?nk zoulfsiV=*&4XPIGNCU<0j=l-uT(_5a`jGjzObZonI;O2HZJ{_8tWPrXL2(vYLqOD< zAx9U@zh5uzZI8h$;vJW3z7v9n#Fn(}S#^G{c#a&5pui_t)hXx>d)K4F}W{6Am*GJC?6W85V zzxdgJT$g>bwOtcP&DJP>g;>ImAfO#@oEb#hNS)X!QMPHmX8w%@f$iuoF@RNN-%Ub3 zlu(xcJs&y9avl;pmAPP*La!1KGNjC4?WDsM(-oOPn9!IY#1Zx6>*VshN1Py4{C2yb zKLm)$L-0zbJEW6ua^tH9a`ji4`y2((;KT<4AH(`Mz@&Zjm>jM8pJwd)7t4di)Dt>f zMWTQywaT#WVf^lFbQ^=zmdIbn;VSj@Xp%|?)zUTcbRnZ@QD9gcbCBw z*qdNtcYpQmy_&~C_Kd0YBl~6Q%Iw8!E=OahcZJanI%Qt6Wwxdbam*Abgl`^BQ+%PIT(Zk{Hldy-(?5hF{w=?11z)4ZMfXq0uR-Xi zkQdL(EA0dn@=E;yLSEgov2yPt!MAE?_hhCkc}QN{UB-3-)-ZbQNO=%|r9SY%i{bzg zKRkBXSH+OXS*jK36j=s!^7LQ&J}9os5nb@^H*M5gRCIIHAsHVRy0q#jZt_MaksMtx zDdA5b@;IHHaQR~7n6PV2Z{MrOQyV8C(?uCG9?D4CsTfehX({u?j!b^ z90zvZ14#H}y9xSl=}r^zFX;}0&-q|p*UD`?w|$Uh+~knI&R`8XQx>Ah*Aipy%%iZ9 zTv2xAG9X~zOlci^`sVrUlfHP{YI7&b*y|y z)98D7vsE3Qsn0J`?oS5+eu!?$tnDLPqS~w)poD00t4!Jj>?bPslFMg1)-8T3ym+W= z>TW?VGlcZV6!IF2G(P+mKkz!7ub!d?`YYf?Rp{xj<{lu!O@Y?z6TRC4W@;h}2TQ@t z+n<^~cVy3e1M&5t@5W1a2edP!s+Hi3v)9vQHKbXSTQS5aw>kV!^_+2PxJwuqosA6^ zoG~R6t^q70wq~Qd!HV9Z8g!3HuIg;jRqd*G(bcsdK+`9K^qGBgLd#7Kd6A`?KuPED zHj<}6Xe@bvkC)pj#pYM1CILIuZaVE#r)3s^>Syy8^!nFnUG{67PRwn6{%3aIbAYxa zwOCZ|>TezZMt3YBz?B7zZuQH!uV#m4R=(*uT7Q|cRKg?fUnr$z`Ql;{1CJGATPYyL zvaDx?jVaKmceMi%v!^5W;;hSNqxGQx8#)~Lt4MK}5xrF@Dy;S%QK~NL+pn9b0a@sd zF+wNA(qfRnj-{?PH#`dXjuQ_D*vk-l^%!v?ROkOq@WTFz!L$+imOHY$!RSBS6xDdW zcKI-5xP~+U+7h`sQd`^rOc|`}0_|Z}9xXq|5#u-6Mh3SY-tnqOnIw zijN7&H8Na}eH-E&Gj#b}Y>P)vfD{a6CQ}YU%hNMAjmKlFQZ#Z(9Qx~4*i*PI#~zm< z>upB{STx6>eKQoe;^4W_Bz(aBY#sW8gwf6J*j@t{m%23$@)|Fp%zm0^KsvIU9`rGRKt&&FNsBO|Vl)fY;ywtbf` zNbj?D1e)`G@)aqEcfO)vxG=e5K&uIaW?dk91#^56m?Ng(3A3~q%Fi-YnP(pb3}DA& z(=+FTim3x^Q;dI$?yg!;EB2zSf801LSe+V;6@5?R;jKZEBtNIhoK~|)T}d-kcPJU0 zn)gyseC~#G8@`oYh703mYz(=th#DVRNZhc7wyhq5?zTM)W;o)USZOSz952((gs;z1 zmyP{ze;cR8%@2ExC&>55Z=YG0vQ~{PY`woWjI!&0W_2Lq=;qQt0vnMt=7LGXK@YqZ z7{JUR zI(f0-kf*JCnCl{chLxsVjym9eQ{WR7Ij!Q|msC;kG2KL7| z)%r?q$oeWzgRUMs+_^*br*o22!RoRKmE$dn!Y%9P2rWF!eq7BWma zM71Zwu$fXsO*BE^Q3Jj0Hf|MaMFfBn4)9FFiKHko@>dm1-k)BkD+Z1_L@9%e5dZYH zbA==sd+g{m3|-a_f~2|z7s;SD(E<5Fz5lGwLH}Ey$7%!SkwHCdowMWIcrtqMMpnE| zZYuZrM&{aU?YH4yc)RyKoCR8OOJn)8H(P4EI`mOcZej-ND)@KI852=rFK^|~PQZ@g z4_Y;729hyAJx(N&oQ}plI!uGcC?{GSp9c|FZmW_sF5@f3+fI2TATb+~BB#H(y;Prx zAD|9_r(r074D|s<-&+F}P_R(%k>x`fp$BkA_I4bvluzJT#dy65QGv~wtaJLY2IKV~ zcd_BM9_7Rc&WE!M&KW~#a?U2Kd6<6(lp0#S-u(8kLV(aW&n6b?hyh@&@K>S&b^I$x z=E?kV#(WAOe?*Q-fDi@!My*!84Oi0xJLsp#F?KP@b^Y3fQ(SIhSY`5JK9$uG=w(7+ zK8_||JME(e$Pq_*N<${X9A#1CEPb!?1&q4{;%rqaJrH8@3}%o%M15bm@>|NfLsbr@csxA;VgXg^4oPs@KyPidL?!7z_A!i-DA#vog1NZ}*GRHtI z${>amvdiID>MsV6`unLxMav1nHI?uuT%B~zeDip@mGsS3qvo|eWy}Cjl#+Lm z#h7$mN|w9+KkJkqtHY~GU=<;UIP$MDrQyg$M3&p+Zojq5mZ0*(1Mo;RSb+4Tb??XeXJ>LfXFrV4=`**vJ-)Aqjc1%xro|D)rd*pl8 zwoHBQRR{965%+75-#W5=JiTJBZI(VIXus&HH4u1-N~~RIIGP~y&`ef4phe2Bt3Q-) z6}Hn4<%n%bik>`~+M(HQ)>d>Z(T+#s<(jnX?P$G<{dk9bBFzLap5N~H;ZdZ%Fn^k@ zLO;U>?wYp~XC3FO{*5c87PThSW&LkaEvqY6drdV8lUan}-D<~M|0E55b z4bYFJlYsj1=LsPCS!gD(cGUc0tnB|-&+!1p8vUG?QP!Ii*R+{`5P;nO&uUR+ZuCa3 zR{aa^qPQitUt`gj2oAfWl5+aPq@|5T2aa!o{abmM+)Gok%uJjPFS$i zu6r*BMIJ7hQ08D>Gcve+07W#M)i<}~g&G>Fx%(s~4YzWHWN9+^QQEpn_R1?H1P}@U zekBVS_#^1-rEJ1LkSR1;$k)Aj+i;jB*+7vRw&u z?d4Ngv)E^K7`1$*k%pWVE)yR^N;2Cc0GIcfj@XSmJ);xk0@tk<>jdH+!lM(BpX{;k zO`uq-7<~!W0PZ&$f_!Uk*tHd`B5)JP!oZ{Mc3ufb8CmW7)VIRC01E-%G$b9{AxPS)d$sIt`yluD z^g4;~q&vh~T#v!lS`7;fKRFG);g+8^hx!D%Gv}AOfQu zy*tN1drJ6!qkYCuDp?jILt=iRCc?F>?UoqIZ6ZgTr_FEba}mR*$JySmzF-Sd0bYZ@ zHxv)OED$Q~%~ zH@jYUj{XX5KwSJu_n`vPecAV!Z!n#hPy6;-83^5Xpnw!84;nEG=*691*cM{sw_W*# z|4e9~b302veV$(UFSbTya7f@wB5&oSzLUN5TseM&;w2q`J5RNmXrsu`;Tw`Y%G5@V zGJx|n4meSL7dx$e5Z=~lkVZ&sNhPz*&^t^L<}Y`7 z$p&Dnu;gZP(|%1-N9nYMWwGY5EpoHJ77b1;a_SQ*>8-WZf{&ZV_wD9q8GW0Yw*!ST zMbEkpOYN)|b$B`fnBivK5RK&ucL_VUTbJT1Ro%z*<@wImgK;vM_r5`ocYSj1C;9M% zbvhhCN&mqC?BV6Z=}EPMSAi>h5t#L2G5N=xAgZ#2u<|x8qd~n9W8kRkzZ(E}bu*#70X0fmYCjsI(El zAx!bGjwQBF9&R72c#!O*-A&b)v9_naSIEVpK>S(c|Fjfh4Y1G^Iajgv>(8h)R*mXm= z=`^HW*11J;Og~wOpobZ>22+)nKd`Y@|HX;_T3gz#0^rRgOW{0w>$;GFSPx?DHW4}Z zj!#gPz73z(j$U;HB!3~=uQL;|&K*9?ckXawTWfl1=p3lE;HmfwMb)A`O5*mWVQ+k; zs3$Q6olu8$@?pNnXJ7d$xh6TI&(y9+OmZW~y3J{$jtNV;_%-sKH3e>aI}fu~&cDWG<$sIMWEp94~Bag@6A$zlGG(UKP;z zQ3aG>SO^8g7t~b{{~)MQ^0W1Nh>Y)0mtcdA=&3`)4QbZXN+bg8&%(D8krV zXBBd^_RzAg1CD^m1~fh2o_5cuy1bSGgwBj12r797D+o?!6!ER_yKs4g)Uh{_eVrB5 z@ntxU?KuTwKpdZaHDxxpKaV!U!aT)n!+{fK0AnClboU?m_OHB#gjE1+k@CY1#D}%E zy|EFG-inpAYS;m-+aaORn@5AL6J-CB9s+j{p(RuJ2TvnC9p9cIC^zv1dxZ4}JftGCw}$nt8-8MW1N5~@$A=Gi0r6W|$Ejzqn*sGqVPUuAa|qS)LDWbd~E-10!V-}+DWoBI6cLN4o- zRz8o}i9&2g>71gcG!$T2UhP?7YeL22juGq0O6lzf>6G%uA;Sb{`Eb#w%PPdt`NDAF znxsm7g`4f-6l&Xvx`%CofL|X(Z*4e8;^&x8zk4% zE_F|PI0P69%nl0WLQ4{+H+J!KXx`$wuX-#d2wue(b#aKwI4O);zvWZ9YEirBEHx-r z+DKVRO0hB$>p#7!w8RFacT%7Bie+PJwQo&FtLo^0_mXG&0_rcB=GM=dg+@c`L!a;F zwqwQ~ZH;UO>SDm$>z7K*jmZp!Y-(BNkq={E1;4YB54OrpcN)gV!weMg8)`s~qdQ(1 zNq;lE=$mS2fzu`H;2Aey)1robMJ@CEV6^KV`nfj@W?=cx1wu0Ir#+e8djnsv0 z5{yO{bFRN9!yQ0^{5=^?5!|epDW1M=^GLLH5k2A}LIR@T;ISUI0NQ}*Df_7>2iyHa zZV)WC6%z;*f}TXaDw#61k9D|a0oZz1gmXXp-fR^=5}h+V&tH|hE_FM@{rpO?MzkjTqaZ^<5t}>LvDR(_HaNWlD%^*_o8lJ z$~L9KS2pxK9^W&4Y%Ol)*g^Y3X<2J&O8vG`Iquf$=N-=?>-9wsp^uSD5toILVh9t? zc#v-w_;j)w&)&Nz#;<5cX#KEQw-8)5?K9a6_YjCM-+eK$;`PpqTcUIo((dHi$-eq- zn=uiBLSf|u<5zy}t8j}seomrHeI?@w?bOmlspgoxMs0yO&0dRWy5DSY~lrh(fy zx%5KI!vEREfgsTY*v3V#k1~wD@JnoSTz~Ds9|oQ6__g${^orN2S8?Mtr6ZnO1CDq& z0coyaZFjYPb-4e=esS(Y#`UUmk#cr6--(o~634)_k$WNHyb+VSp*HWWiV^NXZ$rib za0kx-O2l_k*CleX?0$k9A8LOo7G;FATRY#Y0M#x0op5<%9D$%j4Kr zA1^-YAM2FxxEC#r#GhUk4B$;LpbZyy=(9iG&w(koc7#FmJdg@k(zk57MpON8z8M;H znd$z)^CVi-jUCLSR>Xf`b7j#NJmq8PP7c{n2`(~Q!IvRw)%JF<#|r&z4L{G+?S9xhWz%yUVsH2mT=vODIAm2_NK|Dv!cUMm zyp|A1_am63b7B@k`|#(0)shiQF5~~*yKO+p^QYUV3OJwzuIP|{7vFBQ6dZWnmXQA( z(2Bm0pS?@7sD91nShPHf;ktpc2(^=%s^N+VI07#X!fo24YwvQI*o zHFp)*@8j86CT1l@n^?TC4qax_@*e<-S8i41X`#~HX+0W11S+0Q`b7qng6xBGd!S=% zBuhq|E**dCl4GW!e$6`at@}%vXQj&O5`&(5b$C&HGqf)f7lE#u-s4p(EWA9-x2YQk ze#O(&xV~6~ep$f3-S$Xib~nz^#RI;qfpRms7HRN1fF}c8wD9b_a#Hp=(Zyf94N`gd zA*1--_ZYsmdY^}l(p~4r964x}GO>*U<^0HN9$MUQ1KUR@G->HRXV??>beh|hwkQq( zZYLj(os>f!h>P!`PT*^OL&`_VaD3j_C@|uq1prG`6vGqOcQpUAR645_boIZ+Z1|s2 zop1vjbhxJvqYf>%(gtd^c_#eCXt!=i7dnVZb zWhX)L4Bq-*)VFwTfWVUA~}f=S@~bkjZR#CG(OxNP!+|)KRcAXoGRR?WhzVs&_F%@YN;=1?d$w4 z7Yjk}S+?}ZA$cm-9}Fv%F)jzJjEd*^dzy5*d!L3!abpmNKp~A&SzH_qI}897yglpJ2*S(J(}BGqL*psSF!4qCU*3aVpeL%o209B>AX`9L=nT2yPY{dgY5 z_|;RHvQa5uFRlMMbIuo3*(W)Wug%?Q9U=G@SuaQGjc00I71`Tm8wy${Y}RKhWtGkF z!5^?~q0q1pP^qEbPAil? zgm)F&fZWT!4_fta|GH<@CBAtR^Cjg~=dOEaxaIO%f8tH6$>X*-=)HaS>8uK007`tU zW$Ch1g+2tlJ#rl}kvTzL%;1%Jh1aaj$v&ORv}I;MJK<7J(?wHPV&({W=(B5TwR&+( z;e|R>-jwvnBu&R#4?{LnG*^^{%cai%yGLVW)5>ta0q@g~*YkI78p|~k0XMk|d0jHw zJ;TeqF&Q%0Z$AruP86-=*0mp+8d+A3VINu=tGac|2Lj=j<;JwfQR{*3$L^D6Ccq6$ zP>lSzekSo#N4A0BHg~1@k-3_8F+R!&@gPP00t%C4I2$!c0@Ijri(_gj3z(br(TrZ6 zC2Kb^9ODm1z3vQupF;MBdY*FgLvvN{V0;v4{3U?iC2~h{Ldo3?d!d_W85P5m`Wpxd zq`2bn2*+qmN&j#^{I312TbT3~(t+{SeQ)Xx=4nOb#QrpGJayIvYh!2OG# ziXk|6J_GpwIgW$CO#3p{oP)%_LysZ8Y6K}D3zjNJh=%F6@n+Zx$ZYpLX&_chm--eiXUiijmR@qRi^#V9U#|8L)LFQGmY&!U%tt9p| zNIv7E%%I(#Wh)h=4NW95j+eSGTzIU(ikpsJr)SdG_6ZX?_3}i91a$GX#gVHG zB>zb_D-_S7CI7ljS?m2{%}M~QS?w7@$S-d0u&(u{T|JRLXQ!iW9Wa1HbnbhO0BFDu z6Brh^8C>~J|CGHVoFxn(XuxDp3)b!eowWgnDW!nJ6yuUfQ9e#5M!G9>yVQDB@|Ue*^L2Hy5ot6SURzCv3`Xgos!GS>^P-4!P%a{9F=_d%*!Y@r+7#?8u%?RVt zVjQ^LO2=gi3`~y#e1zV@_&PH{(g6xLOm*SkNr%zo|E)7^WQ}ut_{$XjiEBwj8+ge} z?v8}Zom(z$l$_A0(-sanvPABtw`(h2h=`aI`kLFz+L-SL-#&`AduxrzRvf%EMPDwUAwlZA6+R8Jk^(bOb^Z>pAZ!#-S()BN}7sj zQ>l#L^5ovUbcZNt4>h<=Z*8FM6RIFC+0gV6RoH2s#FuaHSR7iGFVH-3p0Ayz-nd@3 zXKr=t{?w@-9>#j(#nFV%2^Q?Jfp_Qw-@J}B=j{q(qou-7hwM_ZTJiOGMo1EQ+RxK2 zcteBgdU0PN4<%2!LwX4Cf~gd(mRRMalO1Cpcs<1gadXxUPu(&3!#~S zDPYXEU<&v*oka4;RkXtXPdd09WWSkms^HEdC62YzPrQ1^_ z<&*9Cf_S;qqAAwp>NB4pB2>m0!Iuz;4s-G{gB$VPBW0E!xK`NOZ_|`auZB!LB^q{H z%N9(+?^wY7DkB@-B}aAUVB=?k8#X+^eSRWjeA@6rEx$no?s+neh2*CYQqcD%{UnK2pNCo5qJ(nMHIdj8E)Tn0Mh0gE?5gZ0)B|W5AzF`s0a&r~{bj zbHbigu#EfVL9kyS=K=`6U(L@j3D|S~G=%|4a-S^cL60$0JoicBGmzdVo{LO7x)$G} zOx-Vnvhg3f0lcB@Wd^->w;+lmGIcb0CSR^Hkzx;}eh=)T3Pe2>6H%s}5i)=_d@Fs7 z;{n7IP=wZkfP3uYzb8v^zQ3U%fXrN6JMvE8*^v>R;Y@i8@^0u-^s}9M-t)x-pYK=G zyBi1sV-rMV1s{1za1{)$G)|Yf+h*|=`XAP_X?Uz=g((eYU3NNNlri52@V6wnb5Gs; zhz;Y|?*2_2q$-g~T6Bww5H}*gyrp1GNUVR~G>87(2y*hq*^hSmySDG!a){)#DGuzt z4L0g7A*}`Nw@Ni5LmwT25HPi@PEi)@4I;B7)l%`g`#(hW&8<}#@jqq*gpju(I|=_} zh>z<;AE`E}iHx6($NRGXVW0Fyp!WZ<2aLbigRg5l7*IX?9^2N|fVTg4Hoz~BACM04 zZEOqaw{-CN4_UIQH%V{g|HbzK(G4An*#zR*lG{2~cO>YZfP?~f&szs(za(u`l~Enl zJ-s68sehdR&nJ3>(i95*%~Sko(i9|&;e7C^sM2IHijac_n{!`bax?i&>4hU>3LChr z@rdP^-#SD-W$SwfiL&zmXebK*H%w>?{g0Y|mWcJvzX0^I&BeIDUW4cS%UTOvD&e{0 zF+0QoIS;odG4bow^hReirod*}(-PZzRgwhS+%wpvxBcB$88l6`M5b?J95UH`)=u_|<{qg8+2tB+x%1QP3 zZN&jgxY`chm!{LXmP& zQ#8|IW~&vVA?z2ki1TWc0EjUwfEW+CZNEN5>idu7Q|FZA)r#E$xsT@Z6VP*$QAz4_ zm**gP_LvZgSgNLWO30Ut$eB;O2uV>ymK2Cs^+lTr?iU~Gz}8<3FdhV0f4T6cAI0Al z03P({1y6+UV(?~o1A1(H{^$8zCH;#BiCVod9M{`<_jxj+{rc{kMCu9tE32RCvOJ#$ z6-!SkOwf#9_-diBV3{BGK`wuUr5ph+;uiH*?KNlMz#FJ$zasKR-7_o2gSod%$7Rnk zH9RU^uS>kK#O&2SpJsG6FBn!m_qZM27OomptV|*AZP1#)CBTiuPfNHTV}11gGU&Y7 zP??$FeV)C4X9Na>N^2cduS?g}SLtxM$v&;@zG^DLbiDq2=L<8O(x5>LVMIgyX?^`k zd1|$d2|0wVq8U?JW?JqPR#{)i;{NNgWiy#6jXVy>0aGlRyDJ31DH>uSB&|;k8m;vOs*{D%Xm2SRAlBL>dYE z{YH_dtOd5+xHz@{wm{W`zjKqD9b0(AlPzS*gxag$(ZCRr^R zRiNB$n2^n?bz=L$s1~uByr`I?wpSL7ujDHf#K&O5FVErp{h1*H@%jn$6wF|rp+5^) zgq@Uri7qc4CZMb1{LMIll<+k+9NxEpfGjMrS&fWcmW%#@8Fho$<2^cHz_}TiciCSW zaEYAKkBi1vjd~YU+DU+{@{rsg;}kC3lDYBY`sZ{w-TPw&QHH&S4z>eb=GE z0$f{0{PBs<-j_;iNd%>!aj#AHG$KIw^i~$feDQfGKRkbr!R);j)IIk|W`8$c15}CY3`G z1AIwvaGhWrlA0Pi5^Gk3{V$u1+Y-)d%oReIh3swjAz1y}8Fh7pvJC4U#BKag&C{b1 zf05zgyBdnY>J@w**ZG4B@^8h#nh(9(I14WwHV`*l?}O(Y2PRMKEj0M$Pf+XTMFtN0 zE<=_384`4tT|kW*M6PT!N{;Kkg3b6zIoxbX`ro)+Tv>ZT1XTNFhI*wh3Upu7N45;P z@0WDrzYy;V;*sxrcjrd`5$ooRKqoOO#%Xt`#0rT;QXrLjpfrlxrLL*u@=(xpFT=RN_5W|XKQe(XYe{Qmi#8?Kxsu1=|M zEs#q?X{|4~#&YcXq)^_&Fcn}Bw=Wne`_}~l_R=jLZg!D7FOkpSU^vWj9^)53J`X{O zsGtQ`Ek(22Kc~wAjb<2;WzC!JIKKi#LB)9HhEe1^9F;tQ!3mH5l#xZxoU~11Q^7}r zF1LY_E3c3hq5h3CEkMkkNGWcJvCmp?P;odV?s<;rWKFMl7scL}wx59BprwTYoFxR; z+x(_Mv|i}v9~WWt5AOKRyD4+BW^0crldDLdbjGaf=8A5o!7B}`%pwH*OJctZ6SW6A zY|JyPxVxwoG=ld9Ff%;qog5qyPG2xt*6T4$&+vE@GOwJ>K6hpbJNS#aNbx*aftHPCKn86B9twN!70pw}R_#r9pCi_5) z-6O595}sH)YhJty$!K9M}|iRyJY6Kog#&85reQg24!#FmDdwY~ANqeHGl z$gebf$BrkKEQI(yR`tsPP#-X1d#vBX1G!Iku7dSnfx9#WzDp#4U1YPg{PSn=(|p+q zRep7=yy1E$>=LxSHc{pg)9*whYLt+2c`@&sOrXgp{o9_sVfLZWGhuKgE`x1I;w|_# z@Ik@^p~Be0%b|icBOJ!O1((^m(ib593dHSqXB9|SCM9B#`aZ<160iPo z&YCe?_Wa{|yvIql>YNLic(u8wU4y;K?M}kH+KAXKB4Wzlj_T0^MWmR5k zy6Z|k2B`y1aw712A9R}UB{f%oys;I zxT)r^0$L5R9||~kI$dP5)#pZUbb4kR*6k9;HY^%BUbEhv-F-VCc8-hGRqSw6Io>*u zJwf@{$6DA|LNh>JNn_Q=xv8txR58%axmnYIRnYg2-d514k*%Ks(bJtcsW2VOC>Xi} z<5Fu#e>kLEL@k|*#YcYzLmy)bgH;dz6sptAih_^mX%|Oy005hW$91R=?)9%D?t$lI zKgrU)=n*gt{8YH|rIZ{IAnAHzrsq2}h7(H3&1Ik_dV46QXVy>8x=#=N)lVoha3u5e z>oaUcDmA}a9E7s)Sj=goRvaL|&xB<1n9=bSJrMf?1VF(70Z`uRyhdK~-{`szuc+R# zxG6Oo+hx)!b+?a_dI}w&*`SXAAur=5x-Ab9)fl61K zEv~KU{XZ|Tlh2Lc49_#e#Xv9ycC(?Y6}inijRouL1YuQ$$U7$80;G6{A!aK z_q(#IO<_}$eUBSDSAVV>Nyiss4=z_j&MJxPKG5i$>&c2A`fg6HTNh$X;MV8HbFQOl zEG%kxN_W-AX3A0rT$8I8rt6!8m_Z zXbZFZZLmM41vD)YwvEX916BtZoI_mSg8>G71>8PbI%DqJo_i;kpWrkJo z5>*g%8vA7EMVsLZW8Q>Y;aoX=-iA;h&b&KIN*IYH_86%Ak3s?YRK$?*H?3qs>38f% zhwz4V9eeLN98}0eh&Ck@U(@UM19h}+LME{Asd^b(p zYoWZcO*~Yzkbski)dkk89!xS_QytS{8@&Bz^WP-5H`G);8neyw@5`MEuN9deaD-U{ zLsB`R?xRi?@Z}zE3-)q%cVpzUDWBK8J=nY}Yprl%Jizg|btO-)5w$DyV?7zcOuZ?( zjria+xpPR~sJIm-rrW>8lwZ6MDK&2cH{EdFU?ft&#%7>@J7>x}>P7GqcGkf)jH(ow z#A(tDR(zYJbZQm4e&84m@~!9mU|dZYlp$g~GiR9RG9&{?5-H?FoPCwOl(3nWa*UL@ zd(J@aoounZ75o!Sy(^#B2hyaZ=qJy#5}t55JW1pz1vg=56(S!o?7>3)rf^E&7+5a- zYQdt9a+WuzI7ny~+Q}b&RKovh!8q+G(#chP$8<&ou@U&gm0+ihh|%d$jaLKxocl`dYHA_IWyxW&-9A z5s58t}At6dC3NdD;;2(Yv!s5?wX3=mAQA zK;$2NSGBiuaE9EU4CU+PtqZ@TH>{RHYXoz4;|Ep2hr*V#!?6=h#s1f$t~oHv-t6$z zGc#EOo{8G29twv%uj?Pga`Uh|({Jbdb@4qI_u4E$h~0c17jJO}@sRA1Xlf3J0mvp~ zn0iF4xJ{FQ>ca8peO5n8;OC{lY{8H2+BGmTAKlAZR%}pX-dumGpoHy9dtJUz=Nj_D zbBX^X;>#;?HfY@|ZLHZTkAa}*0BFOYL5lK|L5Y$RxNZsmC*W6|j|ds??FWc7bNa-Q ziVfnmtPecqws{?+_&V7=exfDPrV;s1;G0UhMHH-v6}s*^S8M}CX8rvlIMHzN&#O+s zdB%^e5GaGG!xZ#RLkP%EUfZM-UD&8c`p=_o7dn4N)6FHyL-HxCSKr1lCl;Y z<5In)b6|UAuiJ9?T+Hz;3eN+Vf!y)=y*;GKxy!~qOM$_weg-z98QC_=3NG9GT3It0 zt2hNMtFL3Msal%hW7SJr?ad8p!Wdo(gb*jnXw!oZQ;S|rG)dZikgtrs6L)l==?+=k zd2_0z$#wWQy`*~3^2oQ=Xl|fbRDMD0L;mKulQ2DlMqDSs$mC89+|=y-Be*skv^HY< z2!Jkjp)49xHUaU;&tid36@wkR9t2;2h=OFT+R(R+jLXg%G{m3&^J-JJLk0pNe=Vz3 zHwcJW=rwB1M^XHYzF|ACK`*iiw)s5=G&Y}$2+Mgzhv(>4;0aKlAcbXuV3_oB7NMZk zi#vQpEk)qP1H4tYL@@{ri5XGDC|GKs@a-}q{G)ZE`ADJM>|q5V*HoU*Bm4-XBrQj_ zW!jv|1LSvQjANy?6LDd>Ex|M;MUPKx5SXQJv7ryJlq(Lm`x{Vy=}jo&;{5vMFba04Hj!$G5{E9GOyB!evcEB?rmDGSi)bqo$+#?*@EA z`kx1(AV6zB_h~T-ZsMKdEz6#;cwHS{*3nWk0Y%RIQSWBNN?Y0B3D2qXWf$#o{P9S{ zeY$^FRl4L|Y4H{Af!U@5@#B@PzB(bghO9_ zvb{|4^LlGYGoOrVn*4)yl%43~+^)P;A5^R|nyhzSS$v#yU*s3#)3~SpHD!;E(qOnf z-)FztEabm!)(%h+K-i&_j6DzJ!|(|*={?%mBnR)zt`PgIy@4LrqIbHYUJgFA8Bt(pL+R8EeJ_Bnj8R>CgO-J8 z!vv_@K&ak7DmO!Q|U4meLNRZUG5YQXI~Q274NE#n|>n9KvmBp-29%J zlWQv?E`Rn&o?lgEVmuEta|LlsI|`Yv2byuVPTZKSlVuT`I^xC!0PkxO>sxrw9be(W zAlPT(t!~vkV4ge znEtubD?neR__bEk!q&1C*iXQtDMM!jd>=b-6C4aQ!7>j=aI>0)G153DCgpOJ)^yE4 z&3^UCDx0ZdUgc9d9J)F3K z*&6#5v783W6`jtzB>(gs9xPkZ=CsYK2E%*NS+QNWtDkq~Z8N=>aNM&qjw9?$QwD1_ z3UU#FF3_%(O6zZDrg-yzrkK4NJ1wa+?`+>_HCbKb*x2>NX>D}BawZ&mczer#6jM2Z z6GBsSW)fo;-N{pYJ~*2vkq2kDkgINeenl1-=^HwhG?UK#N>#o=?XsGQy8Hgz$3y>d zAUg7q3NI}!nY{Aq^y=$opju%$KBue=X!UuZ-MTX^QI~A+Q%?|XunkdHZLDCs8mOIc z`4mkGF){|Z=dy=~@7E*|`4biI)%!Rvh8Z1r1N?qq(}p$tR)eApIEt|vtdMYPT7^GC>6LC{xP!I;@fQs>EQ}8DOJt2BA7p$ymj~$(hY&|%MYzRqfG5%+wF#D{UT?F{!H zB*z03=4$v?oZRGF+aZwcRv3jV1o*;dp%~Y0Q$Nqwyz!mL@Q?ND)j_j%_Sl(o?q`X- z=N|QL=QRb|yUR@bw{F{&y_t>Tu5Q`AA46qkc!GcENM@_J(VbmYe11DuY_L0kxe*UW zOb$Y!Bu#FP@+SJtJzNnS8XYHt?fdL0D{)L?DSGUCct|0rC|>qC5|m#gk;4morL&}w zYJ2ByjU^V;x~RI7cDK$(Cyv-dQ3}^(Uxl+1wj=*6BzH053m-&S6$>irCns4z!4*v_7eDh?t*rw|uMay569_7d zmZja%l*J&LKcM-Y?-=U&5?L(|AUkoxGZ91U5<@V=EkzBPo@r>tE`EQ}@bKE>zB*12 zwZ4e~;=^lcf1cwIZ5=-PP|U3EpSCw;zm@>amVkIsAs;0{i-(v%w>?zMYw(7A0zt;1 z+mXreejSgubirkxhgQOqAKDMu*+`yUSiJ{V^s9CSpPggOF#WT~${q7fGwOWl+*5>) zh4UYkJED|4r8{;BbmkBWG-6$&jOb=VfZYWt!O+>v=rpFMNxny4v?c*O7#`Wn<-j|Ow~rY{UFF}q5`=#^33 zcTk=3&ly%*jC#5MxbD5xdSAIoAV#p;c?LpO2{q*lcfnDry~J6Ffi4GVH~RfGJgx;% zx3i#gH~et=lR>u`)exkr>5F(1x7OlYxA8P+e6m?54t@FC9I++IaDZLj>!ewjPb@UQ zS+6x12q;gKtgk*y6uGH#|D+kkJ;hCCpssdztQm7FU&W6f?Y^#{W1Ro-p$f8cceT*i zWFE!9#jLGqBOt*x(fNR&^mRHYA9T1|CP$DkVyw6jNSm%s%vO&9AK+%|{|I9om2WTQ zR+6R8veoTagqUmjc8$}K>E@>wE`(g>1Qmi%ZJJaS;SI80?4f$48y_m4w$*sTN67@0 zbZcYv!A+MMZjF1LuwWR@KBmtXfXghZ8imRd zpbpPlGy4?$Ui=-UT%2s~u+bZoU=;3G5v>!9qax^MQHI2hESr4T@k1tny3-IaNPg(; zk?$W+*XNVuS9${id=CF;yR1>qu>V)jKZ~7iyyP;2dCi_wCd@p;u)B*yzXdxRVxGb7 zUUrjy*7jgfF1bWGzl2A9t()a?Z`!u#py_^kZ*?UtX}8@;m)g}WzJ7aCIM$uL5>mUn zQ)Xk69>=a}@-U79TcPN(d{csz1m4)Zy2gZ|vF;#;IoVIlaY0M**^zb5GN;U5Dbo}SC%fYwE8!58KKA+Q#hjglOHwNq!#1=#`?&Kq`&vi&V31{trSQ2vCZl6)oZS|E=Z7|2=1#M zQ0->55x=IG$g^jCjk=X@HDTMdvP@5dJQn4}G0#|5n0lfa@f69#`;`z2kX-w>0XMm8 z7IDkFrxRU*GL24})*BilJ8vk<-9FcC0=x@rZ^$YHj@g4?%(){HQPUr!?whR z-hO$5^|PZHSdG4{T(4VQEq2=&A#op`u#f)i4OYlUk&))t&u`8^+f^}jtR(F|FK#Fu z!$)R24jlLC>bdshb$`O<`$SSLIIUNn+*b|?1c(%+0@5)sep0g<INlA-UaL1-@NX_&{1(}#I=U%1`lJCdOyy)A(3 z9gU&;45Cd~KAboDGMxA5##_(hYi%5wSqO6VC@Cs6nShF9>+j~0L;tiSZ%msP%hC)EvguzBG8`SHF^+aF z_YTR4XQX!ZL)r!;SUwbw#hlcGY*V>mIHZDZXq%Ihwk_J%g!K@S{SdI5>xXac<#~7Z z2T_VIu#QA;S*8{Dk}IjZNej<|qYKen2k+#M664JLOPYD37ZltpFzCjPNFy2 zK1tYBjMYf>G4X(%_W%MhFVs|1Qzh$GE~n^%3mX(|7gT#C5PNZpjV>8YiyklU#jPkt z!i1s`@vQqK`10xMMx}49s*@4ZiUr- zjnRPjJzIt%ULynnQp{KjSPxes-U@+cCZL1D=qWOKn0W?U!uKs)+PGcEI9Fa(1Q?$$ zNYuN~NlzKqht!U#fILE#_@1d6v0uEi*nc{)?)QI{0kW3tHz*#{c`tOXYLn1cvN>FT zLb`FR%%_5yA~UX*Xz1?WQFqmSJRWy#}=pO1>v0yXzhg9D7*>D-J5LO%E z0UivNO@DPAEahQstpS0iK6_3L1_(n-+_!9BJVP9{%si2x)UVkJQyLeC#HEDkiYC7D z?RWX}3hy>EKz-no2Cs$E)`4Y^dX{B9V(U`!`{sHYkE7!L>cqZ-JF!RHGxIb|h$=|9 zY|u~iv-?3R*Xes#4^}%XtTcdSMBS2Q2V-9P#jSh6zUPm)=P?Rk8{^6K7=4q5^AB}^ z@BS3s$bPCM75UIycG1rqawL;>SqagDMVIkS4NrssPkk-XQIis(0jNp;>1P@AKHZCS zB^300Qyt0T`yYfnoYT=%;7+ z|F{Hg;eu6loNJcmDTCfd=i8%IXFqEz)QKQ}_)EUxhR!d(rfj{mqCh}k6~`MN3-*B3G;duDu;n6_@3)&z|Q{e>HuB}b~!mDT#ogLf6?9Akrb z?(Y%^ z;575@j#|fYH0K)|^e0kY`n68|nx^v(&(fUmLle3zAs(mcs<-Vdt0RhL+ntE=SSLmWTVJb4O-xK{zkul3;0aZJxd2XGcLShLJd zUEbx$x}Ams=u?^aw%=#4tb#TWLIf940{CXb9gxAVF7U%@gLvKRA&*()n0Sa9>%)(I zm|rc>ho!kAFcND=TKNCK2FB?Nj!4lOS`SB~SdXWjc z&@r%S&D;@K&>3uDT-Q+x^e`KLHxf0u8d5~3XLa)DzK!wPidX9LuXplnsIXx1ch&IT4klCLT#<3 zx^fc16?h4a5>dR-mne9N*GCz?ws7amu^mLt!mrNh*=d`qmcrGY;;?0&wR$omF#Etj`27^mQ_EZT-u^b7Zrr4Iz&NkvyyAJg^8u`UKtMfsXSq3yjr@08v z8y&(4)*uogZL5BbjjOlG&}V~o@D(<3`&xp}#5eT4bV1e6zJ@(n#cOJ-)=iNYN6p^| z(X-{S180@={=8Q9`3BUEu*%W|3#{u|rc6^`8N%dxhiEand3h2JxAq9(KW{GWq z@lOmnc*}I{H^)7|G0Do#aO)J>PzsOh+s&n*_+2^( zorz>xL=OdM#?!0Npt4YCY%#uqJmWr#YmdfDW+CV6i87FUAd1Y%h|O4*?J0w(E8gT2 zjT-_;##}=xz-ZOdV;7xVJL-)q2awEy{|3WjW&Qz%XU^N4lNL>JlKOu?N`lFknz^`w z%pxc`+vtpzH$0O#_jYOfz%g?|M8|Ax*@!J^)RB#9;dN0^aWjxcm;JEqL_-yu`Oq_! zJbrS=c9goCd}`e8;Bj*KfSk_;jML*&D@7;)nu>6k)ObcR-;pMUNBnxxoe)C-hS9~z z5mA$E9GW^-hhA7H?d~VD3)f3Wy}io#PZ!7M0P9vD9lS$qaIbw`atkaJftE})L!JC}V;i4H$vp)NMaVOO$;p5- z!Oh0aYN!~MPBh(%@sDZF^<`ELma3%n%4?lCiM z*j%9q{SGj)60r87L|*|EW57HkQ?t@H?WdHWZ%kms!z5QWEvze=aT`5Yv{1SyaWw&| z2Q^{45-bF0_bm_|R2)mG4v5b6Os<{xPWqh!O+$^F4lL6x`Hm~DGe(Cn$Uxw?*Za$a}jc4OOnCC~}{=+8ktQF>_G4*T&MJFb3CuS3_ zzqfj$!O?D2#Y%zmz^~wHERb|#mc2wqoylbZaa;6GF>xglw-B^wly5$8d?N-MvDk6w zdsz8ao^IhK*<1#~iA9h-n@W*&eGL7|ay3!~W-9lrgHGkx)VOr%psD0S@PGx(mxs}t zbSj)qfsM(6AK&%;z%3Ws+5N?ImcH8tjh)jPWSLE0x8wPwQ;X-t{nhqLwk9n?8D$Dt zDLZLG?5dHy*~)a2lgM~JWHa{0({*L%1qXX+28iT|kOMlMuetyP2H&>0eaZ}HaHB#n zgN|vUz(hB;t-M|L1{%-=#6Ce=4gq7Kufd#!&e`l!v;E1(eBqzpN$OXA(U|_|%f^&# z0FCKg0X*kB!NTZoHC!46epm{p+8#8*|Eb>AKcck3)w_NBP&!?k`nXgW$DDOh!3jq_ zSBw=UPTLN=SwV=eNAiM_5e;VnP?! zTP=&$9^T2rkWgtV?w6bu1=Zgl&7Al@k8T`@)I_+7W;wqnL$WdP4=q@Bj!3sKqHA=$ zJqi7ow{c~hSfQjk$YVBca-(S~QggUA=feBZ3d3zD(Je4ngWa&@$E)e#cDkg3GBHI3 zDNauJr7y)5sp7bo3Q~7Pg8wV-1CbLl5&RTDHTvU*G-fF2>kh5ZViXp=W2+o&? z2XU;!G%=+;w&L%=t+*I<;QZb%A(>y*A7rH<9jFh6oO#L|M;jo=F zGWspP?A=s8d(M0(97E)Eb76cH_qP!zBcGRl*mb`3wmpe6UaE&b zwe5EED7RHv9~hFV&z89TrPD1vrkJtXJ^dD2EL7cSEb|?@voka9Eo0BxdtAfSpQrC) ziFcrhO}`<<<#P_r8;@xrLqMw{9mDB;brcQ?4rn3d8Sb!EIo-yqw6?&di{%^WdJ#*^ zu(*jt+m*2CQ#Tg}EN)r}HQqg&MK;@y1K(Yi57qIhzx94Gf6I6p@R>|F6YfNBXTM@M z=O)$`v$Ur11DI%@%vM~g_qHy4K6-jcvC82 zzD1+VRy4V3_ejFb&SVPjkR|bm~71lYTc)1n5Z%+0h4H?(Yeu-0bfd)MseI; zk<`hirGX?~ImT{*pWNQKbb-e&1?XYcUh4|W*w6w&w)+2ma{tL|~wj$lkb6y@*;nTZtmw?6y`16OmMs4O^qmW!By^!PXv{aji`FX?dU;tZZ`*jte zm6HWwqt>i+=vqw!Tuak-a37)bf@PQ8sg32vX*?%<@L5w_88K8c)WLjt_f{9g!tx9E zrDHyK&xhj=9#(dcu`kg+_Yty?b`YFs85Ja^%%t?IPl9;{8?r*%v&~0bs#W5TR`(I? zV8QQq>UVPvvl%Ai$K?L#^&9N?L9X6I4+gL6>>T}vhv0sJ*MRoU%7RrKg~6W2Ul;TW!+lH^@{s)kB-?CWI2vV#<-}0J<*J$$1i#&Y48$CJ@L@D1xFfxf>`^-(%nUvMBapp zP0>tSn+N?hLd(^H(yc>E4W4Kc<}Etgb${~*zPZJg#;ct(C+pbbhf_N5&Yhl$ZyPp+ z=N%e*QCYd$VNo?!wcRx_sl_ttVmp>M2PXM9b8K3P*+w|U1O2M9_R2TKC$ai78fR?9 z2v>Cvxq0diGvy%S6m9|zt6L>#$O!~;HFX`uY&JJVjm*x40a};4gJ@gW{vHOh^cqsF z{xALaPIr>-sD-t%7uIaz>YdKCFKckuYdbunv7pzdamyEK&Fs-2CJv^#n!`QUU9^JI zp_3JlK6sZU-JBuXZ8+AVSqnMW;(mMrC@=#AiAuUS5(9v+X~aiqZqqvACb;7&f}k!+@Zk5nnytp;1_*|Fmbwk~^}+TG;jp_^JZpyWG^S=I^TE?409Z1?Dc` ze?W9pmh z#3U_a4NRB7*AKZdy_uB30tK?o$!`vGl(uzFVZjnSF*_o3aU+s-@`=7%_E$n-aVhovt?xtO)SFWdML4w~TVnpIR3+1UYy`8?-oK zh++572VJbsXjPdu$$b_5w9Q;`p$rd_!gr&hHjVcnm9`mf(}{Z1=b9LvgOSN+$LEGR zI`l}uk8alJPR?pPDg>_MTanm)sW`Ohek%lIl(voDr^f~h+m4&IctyD*TxmAkko=Jd z`21#v+ut8CVqX)}r6Uyg09kqsakW^aFmu4dh5rfbfl+pS(?f-3!Hsdo4 zO)GBZBYsT%&fX`vw(ZvlBl7QhswDf55r(ANo9d`4nTEHHJ(alNb|h_?n=>|~tL{n+ zf9y!<%>4kP#LB9*v(xYLkhCDUrf0z__HbA1*?bCX&ZoC`v<^3f7i;}V2OA;O5rVrG zX7lUicbVxp_rGDNN(-7(j@jBu=ZnBd8rbtCD8%!{kh532%UyUoY`jd9KA89~6zh=U z8eSYqh`vW7vU2>HWN!^0wdq+BjgMt_euAf6)od9t)Ec%J44TJUhYZ%__&n zv4@!V|4ORpwqF?wFl{oe088q?0CL8|w^Pe)0_OjS?Zq*7xUr)M~cDmLyHT z$w|Yz9eg*j15{L9t)9iBGFXhSI^b@Vf+boOLDRYbN*k8>3?!OWUHEcBHbr4kJsZwg zi?2jmD;Eqvl`t}Yx1C7%CBMu1cYosf>3;x}z1woH%*AyQAuZ2}wLz~&DV?8#A1>YS zMC4LT$APNwGt`Ya*#q=$?1Bo5&<30ej+nSiY*v%2tF5w{g$7~w6rqgRqaum}Wpsq# zBowZ@LvMs^r)h(|e$Wh=w(6ezMoc9fVqB}>qu8If}b%|5z z=)7i-qRfIBv&}B8fqZnKx54k!{GkzG|Es@?A8ZlJK<_^yQ=@IN_G5d(#YCwdN2`wr zrV#*F*$miK_8Ji2g*z&YH%GGz_sg`xr{icm&%vw&%t~NYvwbEaL1SM&XQ?5kTV}|< zoRVB3Yy#9?lnxt^>p?UIHo=g_G;8)$K{JuQM17eU&?0YB!x!9u%b%`j$|UPkCtFnW z=qk&kU1dader_AJJw*v*#g{NDl-g>7i2U01M0K1VZ|(Qb`-+cp%zvJTA5!YmrZ~J& zAS-6bj#>}XI|%WWaEJ#;D_J^cBF5ND*UuierJa5Ssiz_D#qb?c;+VIbhv62EYG&zf zSG{avuNt~)?F`a)7TD@#|=jcwMSdubmV zBy4NIgo_lh&Q*PVIYR#jH?|$kgwFt^SG^j3H#i7;;jzQOA{!FmU(yra@^9D}j5!Qa zDHsbgI_Nh!sgp+bWQdwX@{fGQyDoG8B@GO+G2Z`iE-=9JInFjIiZGpguOS+THpQqNT%r*;u~ zwEV{Pfq1a^pk?nCdIt;_77LI22v3lrjrRX=xDWzxntrW$DNVrM{I`N9dyc(AZ`lPI9k0tp?tLqSk7@+CrBu)&lXGE zjlVrfsj{TDRL-%qLuT#x42Wl~h3BWlgIK_XPnz?-8OXup@kD?)jm@=wc|xE5^@REY zo=~r&%`cS`4H15U%SsY7YA|5rVLSJcNs#6F0LX7-?Hvn->Cv6gzXgN70Cha|Cu`zc zoc{ubJ>QJp-$nlrfWXumCH2XA$C0Vr4$PQ<%Ag+>i|zF07qRxukqFgZjX*#R4=!!N03`8udTKH;7B` zb~h~F_y?H!Pe(PH^Sd9wL|PzQ)4%`%PJ@#}NFB3dam%S#7q$oYi~BS6qTT?FK*c(m zlRL6tav1ii_oLgxUb+Xh4;EI|tv>cAhLR4e&mV*f*0K5-R08f3j94<*UE3FQHNTTj zdZA9{$Gx4(o>(FuSWV-TDQ*=r539(;>RZz`;t8bdl(N}UfKM2$o%{$b8kl@?l3Y6eYAr10&@17To3wRq4A#qY={?( z4YhB+Jqv@5cZ&BLE*2e@!`3GABNgI^mv1?B=FhEKg~}4ART3Mce%B*}MoNh43457h zmKpJbJJvs!O6B<(1ZQSRz;VIzgBh^a9StYKnM=*wWm!lu%jDxeJF3m3C+TC5BHei!?;IcA%MLqDax>DHjteh2h#*pLq zeuoFu1z(tsb&QoEk^ppm6zAU$sWesa;v;(^3j4lGKXJaLtDhOal<}#>HSbIe#)+$* zqP5e}N5{{KJ1lbd6@J-F*S@}Q%Iil?oKFg6skW|Z@|-W~w5JQ}rCd?i>KM1K3|U+j z*pu&eM9yXwxZnm^FGk>ZxchA_W~cPQayfj%dFh9<5pPf3SzWb$uyC?U32Ew-z+rDf z?E>j!ks`SLW4CtaG45VO_Kh=c?+U`{lU_E&z6Nb00?xNw(4NC98zG0fI1>BuhQMR&zDDCX z^Y>p2?UO$Y?WDjjhW2QMidd{>L})ch(jDo4iW_-tJb19$9e8ZN{2gJ0q>Kxe#(tZ~ z{>m#NN*4#--)Tnm=wV-?h@XlpRPX`8Og>3lKrquLO|)lj#)~#=u*g{lHLMK{3TaKI zcNHZXszH0h8DiazJy8XCjLrEa!P3#sGu&t}=aW?Ue_|U3`iQ?8=gfIK z#0^|4PJx$X+$*q}^Si1x=Mr&tQb_C3a>;W^Z>wluPCYlSxZtlmyf9KYEWO^@scJRQXA=>eJ>;1g*-q_&oItZ6wbyW31O+Xjk!`hmv zWZIzZGN$ULVc~o)n1shSVa!-E_0IA_@`EFrJpI+`9q}XEb8;AzqVBMNLPa&FiL{C? zHL$cMEHRJX*{k`3#O%1pJ}{hl2C^^ub-5I-!*?vtqI|NMc7`Dt;UE|;`mQC5ffu6q z>l<%i7ttjK*4Kn3=h5Lp(iT4#Oznf!R}?M42I3Ch!M%ZX$w#wkvTrEX!9Q5PN3&@G z7<>XSLdT;rF%qf4npF=}V!stcV4^jW2V)vKU`zwT&M|zL{)toj*IBSAJ6rpnng(b`K%mYku8fARNI`7RnOHl`eaN>QC|B0v0vBaU^4Y zEni-igmXh-EJs#JF2#OPJsr+ihYy@AlTw(*Wlc1Qn+}!5-kJysot*h;&aooAR0h`9HUwt37W+r|4s-j( zE*0}B{uGC1<-K1fwb6$-9dt)4UsK(CJtZy@jm31GR=OSxB%o1C8#5ZLc<&K9QaSfw zv`zp+<*cz>r+Ti9W1>58J5}-a@Vv_Wq5Q|%Gmm;jhD%SE`}lAh^ZjX;*65>b2c}q6 zx8#%Y0R@`DxqGuiTStezTMyjp`;yIiIWPe{;Xzw5;JnsndS|<>V++_zFL!oQ+S{c- z?(KM1SH$5G1BwKRr#XfEkmf%z5vb?cN2_>BZNk}9yw639P5-a1uZ)Xw>)u8X6iEqb z5G19$K|nxBY3Y{klHnsw_c6x(kf#mn403aJn)ZT!s zFuM&ODmoSfWQ8y#J@|m=cuxyb5A)yWUqqJ^$;3r{SV&6VVs9RxJCpeq%~X#?fDqp{ zq-ZjRK?9{zha(2{XZw^$jW(71!g^Wew=r%ToT)}yl*PB%Tq}el-|nZZk6KSrWD3|T zLiyD52EwSJ-LY+7Z^cpQY5#pu>XP__ZN)(_m#R_gPchoUt-;SLZHOfin*1Lc9gxv7 zjL$xjA{Y*=djIKS9a1e~-UWw7;HPBMrZFAuK~NkK@sq<)vGoaBE+=XyWubUYmICY}>c-F;}S8d46^HhG7l`O5~67T)@NY@>ug2 zCq+XXD^ei)dgkUwW8{TW{X7p{d+lmTR%f5tdXnJs^pPrOK!gTZP{H9e>Fz=4$3x1k zDt#QtKU6kbw;WSZc-(1MVniH+I^~p+f9OBYi$A*M1^ub*%t!@ zQNT?1ww+DX#nOqR(iJ)_%uZU0?dIQ4`DA~n^+dp4PPrr}YFuR&yw#Ps?(3M)@_QjG z`)45wi#5;Eeffuh=XG<|<-G~R8fNK?{Y7gEpmuEm)UIp<_bV3dTME1}7yn20Zy~>4 zvsWk>k5zRBh*{w!@11ERg}?)T1al7!Fcbh2O`H5|@VZ&HUiHI<2`yoZPilL`6aH*0 z8RLmrZwH~FmELg$5ua-(?$ML~*bz+rk1Ve0HA!ZNBZ3eeUr&-8*X+%i7m3b@#p>yt zu!FL!8)y3SH>cp`!w-DR0tEJ(p|zrRk5Bg=JLScQ6x#$?QsW*KJI#pC`0=>e!(!7)g0hgfxVM6r%K|iBBLv;ejWk_% zRndy}gm-Q_m7xtH18;df`+0~_C;5@({-O)=5riiYWox_DA+3zhNM|?Wyi0-+!H&FM z5XAG|ugRxv-$M?$f&y7Tw3Ug=?kUx2{^*;*FHBa&m#>_P?be#WU`Qeg1MnYVu&9*O z%a!0Z=x`4xj?tX%ar5lSG;69+$`k z!Vohnkzdj9QAMfkY(KyZ>cBD>k)~5>Wi@7(IGZqu(;zd8Ehv!Iga=F4M7dB_;6sT* zUuW-#LQg-&Aab*7M!z5bU(nlO>~GZ#F>_e?s13OgA#eZZ#-r>_h*Fj={%!sD9R#;r zOO@-d*wN=*n(Gyote(e1a{jwVQxZ~DIJfPRo0HlQ)T*jpODU^Kn4KksaMV))z=N`i z&seR{Iv=ETX{R1iB~++MGJUarSx5|~{~Azk+!(_c={q-F9$1vU9L*M{c!qaU9c*=ZMaa-7;7E6n|L=bcqCkL4m8ks z?5`n`K-aw}3O-FIYPW@pOsq-fXcJpR-;bL@a2)3@Yt3(e zKDgud;e!va8|4RNLakHcjbtKJ@z4f)5>5smGjd;lAr~`PDxdEB@D8%l?9JiZu{|)& z=R9&jUFvx5Rn3IH-C4A{{M>r4jVLA)*I4Cw$Fucj#JV!cB9;i5)#dCNXR8ws9#-i` zYZ$H%zLMn7e*Qp8iUWF$0|Mta98QjR*SMz!iSGM*j3(m`Mg#Tjs^8Bit37R`7$k|0 zNc-W!_?J}fj_rwE*h&-@P}bE;Vo~wbONHV>l%_IS9p%5n9mU0&Vc89VD4DCpUPSj&uoMU*H#)|e6q}A=#~N`v zYmfVD=n9wUMbNr{B!N6z)ufkG)wBRddj|V1I^ZvkcGbf)v{!!wb1f0?+iO|r5+K={ ziwiiXi3g!f2s?c!+;)ILX&yNKN(kAgecocDr1Jxho7~ji@tzTFa32nze&8xy$zo$N zKdZB+(1m#l=h?VOYFgDi-$92Ln2N|55J_>1l}i!dI@c_OD599*ow#|6Hn%%+pOe+{ z%bly+Zk^Dn6C^Ymo~juXiLY3SSFCobi<7>?75ESiAmO3R@u|MeS9LyMp@zdrdStqa z-X7VqS1 zirA~W5L48n0E)4|D*+9?#{foc{ z?MfNSnz?+v&6i)HrfTg0-Z?ZDvTpuy1)o0G_W7kA0A_w$|8uMAb^T@cQ%ceDu>>Wv zSUNuSr>qnSpNCGm15^06R)JIY1^B5f*#S3ZCd0qLBH~)(;1Rk zZ^On#d|X_y~W9e2{RIDx|%wO(#tlYS|T`V~2AW<|cWt)$#ftf+Xp0 z?Q!u4pm|^=?mz#sv5eP7D=8=`!29s`7lL)5p*aV>ki_t!zgR2QooCm_xa4wB-K29# z^MCZmP!(H_;jPMSTob#E#csDSI#OmP8Fq-MjY(NmDE7mU=Uh==GQYrMD88c&n!tv) z(H(vk8bt79xuD2XUEO_VM$`=cwYNag^IC_3Ub_NNe*<;mZM_oM&W8+OK@9G@m zs>kY5U%fs?ViLWmDDRMoW&xJ3r;Y-U?AN540>a-4I!>EwiVGfg7m|7akEt6W-_^dt z6fQxj(gjf%^?DQ$Eu~tB#p&GzK(*fpMugEU+M#wo2ns(xh~W}C zJ)_86@PJ&naQpJRMKdIR0bc~Cb(i-&<3%>tR9xQbRA^5;_3o3>hlmfG*X?Ox^+vo#>qfWX8+hV|CSR zQSNvHKJBq1SLaaA^hep(jS3$lG8Ks#6)L!5I@K$AwJW$RUeg>oatS*nz-;@z=U!VT znj410K{7E8htZddz(t-C^Z>23USrx`D>$QOVi7ZnRB*u|?D&yV7ze?HM11Kq&`INx z!pvB8)Umz1vx;*p^r5=9e~M}q^2;lDTp~1w$u6h)q+WNfP@?RkRxzLR%Fi;_ygLJJ zHNZ~YN>gI5)eu)XRJn3Af72^-WLS(F53s55jcd~HEMHng&cfG%?+THU#m_8$9xEa- zJjpnm5J@=BPcBlRNR18g{eg?Wr$xM@C~t1)BEB1pq5U`Zr3XZ zT{1@^5fSXGF`chC=F=sHJD*7)tnR=MRZos5V{kID`~VWelB4x0!)VGH0JI$hcmV@% zWIaHzh?Upmze$2V1%zEUU4}G5bqA2Eo9lgDC9OSP!3oe;q_`#?7cF&{ROcrZ$YL%XmTpoq{wFM}W5F`C|wtC`|EPYQI`BEg# zX0j!b7Bidkln;CXpKj(0M)^eJn8DDnb0)!IN+&%%4$@vWaV5S`a}Kur zx*@vigRDXak(cWk?1@<#gO1a&RRMR2Kz(6^cDN#U#4*D~W2hlQ-0~Mp5!nB=hMwEB z!S3Vo!PdNq5s`RMCm++5bi-a(8PY#i$j(AGVre(_L5ahtGzwGKNsn)l_-H=Ge7MoI z)J?!CWzlW<^v%S&>*T^?yn0^R?Th*3L#OXIq{@0H&}xtgY^18{s6+5RCT|2rMe6cfpM6XVJk9SFR7QBxMIwpzwr z-WjlrpagNc+0kBoWnyIHK+<;OAli_KoK7r#Vn!+bl2wqAZJ-mBQt#+%v`v2>(kjk; zAJSUYS@NX@OU+~s>73JsaqrZ$J(9NW3X=9V-Fi!l$uxHuyw~d>;*6|)d+EB`x8SAn~%)!oMlsD>ULFT+g;_>0@g+UU9aN3&tL$A z^Q#+ie=9PE?PvQ2-X%{!8<57???)>lq*uZK zwW_$c`7d z6q>9+tq#I*7m$R)&)zU8skSJX-hz#LZgx-0Fw~QkwpWNMpd|WFGZ)1nvDkoL*#T2* zt>2mwShH_&q}rot&D~-o5=WMR`DA{9tr)L{9FkCOPnJU1i z<7)?+ioPn&`rA)du<1q}1HosRDA4S#41bhtfZ#v}4+!H#ak8fJQz9~N3d~x>cw3fs6M2($m6vS3E2FhnE>Y4}TKkpSRGS-QLaan>N`m7vs;;Irh zD6aFKjcP1C0FuG17PzU7ZPm*byi&wXm|A}i?4pA%ij zz?>HY^Y`RpgV?-1)}3c=gyq`ufKA}NyMSfQh>+~}4Yz#amJTxXDMAr;y+Lz-n*aZ*5CRrK<_yempo#X==x>jVbn7|UTDa2esivmbbzD$9(M|8N^YHB1jmhd zGCvmC3or9iGPLjQ?I;uoc~X-_!@$WVye_m$_A?N#aD<42zKrP7z)X^6QA)~ccelooY~RoQ+M|9muh_AkJJR)rQ@Li( za^sqZWcYjzeOq$-#|h3LgYgecKMPros<};-0xxa7Tia{4x{QRa>dl4FAESdS7MQh! z@3id(a2ZAf#&4Q#mYO5*x0_CeEw#`vh^gBx;IKzD$Fk`NmBG4z{(X%c+Y;X_eaR2lH>8JVJhn)(%L}u1WG=z@kC@wZ~ zI8mb^eR+V2`z&(Aj+d$W1@viALwRO|T=x!q;{N&CIXMI%dGgC;6^BYe+g`__m+iFq z&eGJig!9}Yq1Q8xD4Bdo)hAH1u6TL9vHn+b4aQl7f^{!9hb>^8&58fp_8=c$C3{3S_EZJVJT934|<)2v2)&8CPX`^1-f}ScPCd$@XtM1!9aVM)Etg5c{&YzIkUAU^q zDvDFuj*kaweJ&!&{KX#vRuJ4}-h6YF6n_ZMyQ~&>atJc46f@Dr)^sc5zpYQ^e)ME9 zE5K5m)&uq=a!?`kVaY(f!)W*!x%RUwB~YMrGsE@@2}4+Dvi-)8%MPw?p7hGNBi95#*`h6e+9;gt~2=w-KUvr%M9Mp6~LdPa3A4)Bdny z?5ePHWPHDFwYqhrYueoNwskHe0fI8$kYnumszMOc`-R=sM9#>N8fgaxJZ@MIiY87y zzv;GfCkAh&4>GS&rB9p(6BB$uRR(F+J9|WaO1Q&pon$V4r-w1K8F$IHnxAVa%kLu> z@&1fKaU1eLB!$*Er>~V#>BXZy{F~=bpt%)afXgAv6T=$j3b{~Jl6fch3$=EITm*?$ z-0@n-NPTL(^__$5-&-t7z2C*`!>(o1YBe|xMDwm;z=~-GpUXwo1R{}b8|;IaD1`Ol zc`&yWyzFy5iYqvWaA6b~yMt%dHQ3OjpgbSN<3jC7j^YBMRM12CVnEg-#(JkY{{gJ8 z-|wSr{KGn{nY`4Xq{X^K|MDW%8>wI$5uOiPXLE{68)D#)Vw zr@dDA!Q+qLSBc2hnxm@^3cK^BJBh`pRw||4JGi;Gin*?3FVm?yEzMN-cbR86EVgM^ zS8vBn;nF~-gRX9kA45^x%-qaW2c5j)a2jKrMS0EY0$t9%>H1&SR#_Rn{wZC+RmTvIqAkiw4Q8DqM{OOSwPDMNr`V zWX8Le3D=vnw039KZt6#)U*54sqla-#C~JDwu#Q?U0K0Y~DSGLbajozacZAcLA>b1b zm}1`v={wHUL<*n_S5?Pvp2t&bnO}US_6`GT!~ zNpGg=cOKrEygcuuewvm`R)=6CJNC+f<|x3Xq-Wuya<%>U=a#?5Mfa*&xnY1N{<)HY z;%rd@xsY8VuolljJR@JPLQV!zE3OD=6M5hAtOMi#*G&UaU@?*d7NcSQ0U5nX$+SuC1R*BvoJp$r?{WD+Hc>9Y(DuH-MVQbkNa)s*R1G~ z;s6@ld}x#TsP^pGXHs)ZrrD&=owNqN@tRmc$vukesN2h`@sPL8uE!N}*751aZ#Spk z=_GPzzUvwBw#~)9GcS;^&mNBAZvtAc>%6 z#J86&IKRagqDHcB)MQda z)f5u;@`AlQNK}a9fSTgZ$R!_Pi5WFix_tI*V-sg@;vLPTSGcJ$Rh)@2Z6~m~;8kNJ zy{6HH!KWl+g=0>Fp`6U8U`bN@HWt*}i4-UN)%;zLx)8Z}GUa@fvWqd?it%vyk3@_~ zW+?77nGy{I+EVRz+#xYgy&#wq)CB*B!#7cOr}8(YNaARjW^3_u zK8-JJkE~t3CpjM-}Poj@PY z7m2no(^SYZ6O6TuduJy3Uk;m27hCwctAHACQv4Ve=RDt_mQdGU&wVVPFnl!>Nn*rK zt37u<=Q+N{wY{n`_zfI&{-uI`JC3L1V!YK!t45}VNB_0Z)yzqLO3Wakpw1oe6g)+h zahfjKD0xNJn@K3iax+Bia0Bk|Cf$O}9Vw>i$k8x5$!NZSu7e}>?3IE6O+UUq`ct`$ z3^}~$#_#DliFQ7WIf%%08)v0rK`maeb9kqCA^(9Pdxn;_o~`X(lf?j*KPXcW?Vc_r z<2?Bwlh6lo8oM+o8r&`}u$WAkdI%UPt&nu7ud%kl@=w4)KAe@Ns^wDSPsj*#?8`0R z(DPT|L5-7>rGhu{k`jZyO^2n<6u&VCqF9UQ2#cWN^Y4WAKY8}>|BdwX5pn*`aRN)i zaw?_!Xqp1$j4ch0YTkXf1mZu!k8gF&mEKj5B49XkGHIUsQAyl`sQMG`ZF~ELZbD$O z!6{E&x92QpH%X}#=!#=;K70mtHJpHCUf}G_Z1MHB66hCEk>m98NqX#~bV@?YK-nDY z!q4uHaL3g}gdZhsQB8{XB6c*MoB~B@<4by;ccrRy2}mjB}Ga+-_@( z3FSe{vfsaJ@L_0I=8t|7xIB$L4NBT~t@LdDQBK|Y33(d6ma~Ei&9ju@DyJtshPxm( zhBC2&Dkg@*J$xyff0HcQh}LUoL$N{|Yjl#to3vRy;B0iLN-9j+wha4X2n zZ0Ll0}m5}rjW^51DmrY za@4*?o3!oao=S961aGkf(FFpwZ=~Z#Q2KYSQ6V!$v4O!$qZJ0$7pw9B8{j`@0ni`) zbrwXq(5pE{5%sPZ+e!ok_+7$@;HWfz$+VL+S@Q`Bu!Z3EpS!t(TfDz51Qzqx?nn5o zsi(QwzZ#uJ3pz}f3iqtCZnA#HX^y-4&Urj7RLIxevVZm%txQd?qOC35mE%LgVT_79 z>YzvO*XA8MPbyL~pW0EW2WG;J{pT{QBCb^;R-rHKT@x~|i$xy|^{Q_<>#FXsVXXy~ zfp2hsncGOrs8!MSQ;x?iZ({W(LpIo02@zJB(N);mh(&Qm^;?)DX>-da?JilWW1wDL z8tDR<3C{oIZ-z$Z(U1V|2TKVGnosCicl`T2Ts3rBqry;Dr_3Nw3b@)35XVaO;L-EB zQnhZLxvG?~J79yhIo0rKHH0LUJRgv+(k&ciH}3*iG5H=$KUu4Gtm!X&nkmK7Zon+H z8NeV5zzPIa_&Kn3R{F3~ugwug5?k5ulODPizbjOdD3fA{mRM%Dm4*-c<};jNFJeqv z8^-WI2UJ1d*-jQJ z`R0Fy4jJO4alyq469y?T*xHtMKzxM^ctK_jb>m7+5wq zNbOFw(&h=%e(KCXN_v_Vk{N(^3rw0~l2Q?a$s$Br@op!IiLAq4n_(?Y5;>imK1^AA z?(8V>UbO~XVm-;=Y$m8FKN$89|Ecx>R(vZ|i~LK5Ndztub^901;lxt7>dbxj`qt`p zU9GwKb=_8X-7k#D!J;B{@9c^BDG3k#^zmK!otR^03!#-n|8h$u_Me1nzWZ=vn@H`q zz|GoG#mWDC8{mAqp9%^DucJW2$cI{hgll+s!R!*= bufferConitainBytes + def isClosed: Boolean = isStreamClosed && bufferPos >= bufferContainBytes def isCompressed: Boolean = isCompressedStream @@ -57,40 +57,40 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta { if (isClosed) { -1 - } else if (bufferPos + len < bufferConitainBytes) { + } else if (bufferPos + len < bufferContainBytes) { System.arraycopy(buffer, bufferPos, b, off, len) bufferPos += len len } else { var offsetLeft = off var lengthLeft = len - if (bufferPos < bufferConitainBytes) { - val bytesLeft = bufferConitainBytes - bufferPos + if (bufferPos < bufferContainBytes) { + val bytesLeft = bufferContainBytes - bufferPos System.arraycopy(buffer, bufferPos, b, off, bytesLeft) - lengthLeft -= bufferConitainBytes - bufferPos + lengthLeft -= bufferContainBytes - bufferPos offsetLeft += bytesLeft } bufferPos = 0 - bufferConitainBytes = if ( (maximumBytes>0 && bytesRead >= maximumBytes) || isStreamClosed) { + bufferContainBytes = if ( (maximumBytes>0 && bytesRead >= maximumBytes) || isStreamClosed) { close() 0 } else { - val toRead = if (maximumBytes >0) Math.min(bufferSizeInBytes, maximumBytes - bytesRead) else bufferSizeInBytes - readFullyHelper(buffer, 0, bufferSizeInBytes) + val toRead = if (maximumBytes > 0) Math.min(bufferSizeInBytes, maximumBytes - bytesRead) else bufferSizeInBytes + readFullyHelper(buffer, 0, toRead.toInt) } - bytesRead += bufferConitainBytes - if (bufferConitainBytes > 0) { - if (bufferPos + lengthLeft < bufferConitainBytes) { + bytesRead += bufferContainBytes + if (bufferContainBytes > 0) { + if (bufferPos + lengthLeft < bufferContainBytes) { System.arraycopy(buffer, bufferPos, b, offsetLeft, lengthLeft) bufferPos += lengthLeft offsetLeft += lengthLeft lengthLeft = 0 } else { - if (bufferConitainBytes > 0) { + if (bufferContainBytes > 0) { System.arraycopy(buffer, bufferPos, b, offsetLeft, lengthLeft) - bufferPos += bufferConitainBytes - offsetLeft += bufferConitainBytes - lengthLeft -= bufferConitainBytes + bufferPos += bufferContainBytes + offsetLeft += bufferContainBytes + lengthLeft -= bufferContainBytes } } } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala index faed0dc54..e486b51f1 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala @@ -18,6 +18,7 @@ package za.co.absa.cobrix.spark.cobol.source.streaming import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{ContentSummary, Path} +import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.log4j.Logger import za.co.absa.cobrix.cobol.reader.common.Constants import za.co.absa.cobrix.cobol.reader.stream.SimpleStream @@ -40,6 +41,7 @@ class FileStreamer(filePath: String, hadoopConfig: Configuration, startOffset: L private val logger = Logger.getLogger(FileStreamer.this.getClass) + private val hadoopPath = new Path(filePath) private var byteIndex = startOffset // This ensures that the file is never opened if the stream is never used. This serves two purposes: @@ -49,7 +51,14 @@ class FileStreamer(filePath: String, hadoopConfig: Configuration, startOffset: L private var wasOpened = false private var bufferedStream: BufferedFSDataInputStream = _ - private lazy val fileSize = getHadoopFileSize(new Path(filePath)) + private lazy val isCompressedStream = { + val factory = new CompressionCodecFactory(hadoopConfig) + val codec = factory.getCodec(hadoopPath) + + codec != null + } + + private lazy val fileSize = getHadoopFileSize(hadoopPath) override def inputFileName: String = filePath @@ -59,9 +68,12 @@ class FileStreamer(filePath: String, hadoopConfig: Configuration, startOffset: L override def offset: Long = byteIndex - override def isCompressed: Boolean = { - ensureOpened() - bufferedStream.isCompressed + override def isCompressed: Boolean = isCompressedStream + + override def isEndOfStream: Boolean = if (isCompressed) { + wasOpened && (bufferedStream == null || bufferedStream.isClosed) + } else { + offset >= size } /** diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala new file mode 100644 index 000000000..df6d08005 --- /dev/null +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -0,0 +1,118 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.cobrix.spark.cobol.source.integration + +import org.scalatest.Assertion +import org.scalatest.funsuite.AnyFunSuite +import org.slf4j.{Logger, LoggerFactory} +import za.co.absa.cobrix.cobol.parser.CopybookParser +import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy +import za.co.absa.cobrix.spark.cobol.source.base.{SimpleComparisonBase, SparkTestBase} +import za.co.absa.cobrix.spark.cobol.source.fixtures.BinaryFileFixture +import za.co.absa.cobrix.spark.cobol.utils.{FileUtils, SparkUtils} + +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} +import scala.collection.JavaConverters._ + +class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with BinaryFileFixture with SimpleComparisonBase { + private implicit val logger: Logger = LoggerFactory.getLogger(this.getClass) + + private val exampleName = "Test40 (compressed files)" + + private val inputCopybookPath = "file://../data/test40_copybook.cob" + private val inputCopybookFSPath = "../data/test40_copybook.cob" + private val expectedSchemaPath = "../data/test40_expected/test40_schema.json" + private val expectedLayoutPath = "../data/test40_expected/test40_layout.txt" + private val actualSchemaPath = "../data/test40_expected/test40_schema_actual.json" + private val actualLayoutPath = "../data/test40_expected/test40_layout_actual.txt" + private val expectedResultsPath = "../data/test40_expected/test40.txt" + private val actualResultsPath = "../data/test40_expected/test40_actual.txt" + + def testCompressedFile(inputDataPath: String): Assertion = { + // Comparing layout + val copybookContents = Files.readAllLines(Paths.get(inputCopybookFSPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") + val cobolSchema = CopybookParser.parseTree(copybookContents, debugFieldsPolicy = DebugFieldsPolicy.HexValue) + val actualLayout = cobolSchema.generateRecordLayoutPositions() + val expectedLayout = Files.readAllLines(Paths.get(expectedLayoutPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") + + if (actualLayout != expectedLayout) { + FileUtils.writeStringToFile(actualLayout, actualLayoutPath) + assert(false, s"The actual layout doesn't match what is expected for $exampleName example. Please compare contents of $expectedLayoutPath to " + + s"$actualLayoutPath for details.") + } + + val df = spark + .read + .format("cobol") + .option("copybook", inputCopybookPath) + .option("schema_retention_policy", "collapse_root") + .option("floating_point_format", "IEEE754") + .option("strict_sign_overpunching", "true") + .option("pedantic", "true") + .load(inputDataPath) + + val expectedSchema = Files.readAllLines(Paths.get(expectedSchemaPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") + val actualSchema = SparkUtils.prettyJSON(df.schema.json) + + if (actualSchema != expectedSchema) { + FileUtils.writeStringToFile(actualSchema, actualSchemaPath) + assert(false, s"The actual schema doesn't match what is expected for $exampleName example. Please compare contents of $expectedSchemaPath to " + + s"$actualSchemaPath for details.") + } + + // Fill nulls with zeros so by looking at json you can tell a field is missing. Otherwise json won't contain null fields. + val actualDf = df.orderBy("ID").na.fill(0).toJSON.take(20) + FileUtils.writeStringsToFile(actualDf, actualResultsPath) + val actual = Files.readAllLines(Paths.get(actualResultsPath), StandardCharsets.ISO_8859_1).asScala.toArray + + // toList is used to convert the Java list to Scala list. If it is skipped the resulting type will be Array[AnyRef] instead of Array[String] + val expected = Files.readAllLines(Paths.get(expectedResultsPath), StandardCharsets.ISO_8859_1).asScala.toArray + + if (!actual.sameElements(expected)) { + assert(false, s"The actual data doesn't match what is expected for $exampleName example. Please compare contents of $expectedResultsPath to " + + s"$actualResultsPath for details.") + } + Files.delete(Paths.get(actualResultsPath)) + + succeed + } + + test("Test gzip") { + testCompressedFile("../data/test40_data/example.dat.gz") + } + + test("Test bzip2") { + testCompressedFile("../data/test40_data/example.dat.bz2") + } + + test("read mixed compressed files") { + val inputDataPath = "../data/test40_data" + + val df = spark + .read + .format("cobol") + .option("copybook", inputCopybookPath) + .option("schema_retention_policy", "collapse_root") + .option("floating_point_format", "IEEE754") + .option("strict_sign_overpunching", "true") + .option("pedantic", "true") + .load(inputDataPath) + + assert(df.count == 300) + } +} From 54b2abd375dd212f3e3232e6ee27e102a1626690 Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 09:01:11 +0100 Subject: [PATCH 03/10] #809 Add test suites for reading compressed ASCII files. --- README.md | 3 ++ .../Test40CompressesFilesSpec.scala | 54 ++++++++++++++++--- 2 files changed, 51 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 2c6ddeb07..fea8a6ab0 100644 --- a/README.md +++ b/README.md @@ -39,6 +39,9 @@ Among the motivations for this project, it is possible to highlight: - The COBOL copybooks parser doesn't have a Spark dependency and can be reused for integrating into other data processing engines. +- Supports reading files compressed in Hadoop-compatible way (gzip, bzip2, etc), but with limited parallelism (only per-file parallelism). + Uncompressed files are preferred for performance. + ## Videos We have presented Cobrix at DataWorks Summit 2019 and Spark Summit 2019 conferences. The screencasts are available here: diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala index df6d08005..1331eb156 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -18,7 +18,6 @@ package za.co.absa.cobrix.spark.cobol.source.integration import org.scalatest.Assertion import org.scalatest.funsuite.AnyFunSuite -import org.slf4j.{Logger, LoggerFactory} import za.co.absa.cobrix.cobol.parser.CopybookParser import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy import za.co.absa.cobrix.spark.cobol.source.base.{SimpleComparisonBase, SparkTestBase} @@ -30,8 +29,6 @@ import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with BinaryFileFixture with SimpleComparisonBase { - private implicit val logger: Logger = LoggerFactory.getLogger(this.getClass) - private val exampleName = "Test40 (compressed files)" private val inputCopybookPath = "file://../data/test40_copybook.cob" @@ -92,15 +89,41 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina succeed } - test("Test gzip") { + def testAsciiFile(options: Map[String, String]): Assertion = { + val inputDataPath = "../data/test40_data_ascii/ascii.txt.gz" + + val df = spark + .read + .format("cobol") + .option("copybook_contents", + """ + | 01 RECORD. + | 05 DATA PIC X(5). + |""".stripMargin) + .option("record_format", "D") + .option("pedantic", "true") + .options(options) + .load(inputDataPath) + + assert(df.count == 3) + + val actual = df.orderBy("data") + .collect() + .map(a => a.getString(0)) + .mkString(",") + + assert(actual == "12345,67890,A1234") + } + + test("Test compressed EBCDIC gzip file") { testCompressedFile("../data/test40_data/example.dat.gz") } - test("Test bzip2") { + test("Test compressed EBCDIC bzip2 file") { testCompressedFile("../data/test40_data/example.dat.bz2") } - test("read mixed compressed files") { + test("read mixed compressed EBCDIC files") { val inputDataPath = "../data/test40_data" val df = spark @@ -115,4 +138,23 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina assert(df.count == 300) } + + test("read a compressed ASCII file 1") { + testAsciiFile(Map( + "record_format" -> "D" + )) + } + + test("read a compressed ASCII file 2") { + testAsciiFile(Map( + "record_format" -> "D", + "ascii_charset" -> "ISO-8859-1" + )) + } + + test("read a compressed ASCII file 3") { + testAsciiFile(Map( + "record_format" -> "D2" + )) + } } From 0b3aa3e64c02e4a376316c0e85abed16e098d1e7 Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 09:13:15 +0100 Subject: [PATCH 04/10] #809 Add a test suite for reading mixed compressed and uncompressed ASCII files. --- data/test40_data_ascii/ascii.txt | 3 + data/test40_data_ascii/ascii.txt.gz | Bin 0 -> 44 bytes .../Test40CompressesFilesSpec.scala | 55 ++++++++++++++++-- 3 files changed, 54 insertions(+), 4 deletions(-) create mode 100644 data/test40_data_ascii/ascii.txt create mode 100644 data/test40_data_ascii/ascii.txt.gz diff --git a/data/test40_data_ascii/ascii.txt b/data/test40_data_ascii/ascii.txt new file mode 100644 index 000000000..d7b80d288 --- /dev/null +++ b/data/test40_data_ascii/ascii.txt @@ -0,0 +1,3 @@ +12345 +67890 +A1234 \ No newline at end of file diff --git a/data/test40_data_ascii/ascii.txt.gz b/data/test40_data_ascii/ascii.txt.gz new file mode 100644 index 0000000000000000000000000000000000000000..1cf90242b3f12af065f1dd8faee455333f1e4dac GIT binary patch literal 44 zcmb2|=HSS`>YT~IT#{N*qE}K;!eDG-WM=r($awn(lLtj6EJX~#d#CaWGB7Xz06%pN Ar2qf` literal 0 HcmV?d00001 diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala index 1331eb156..5c71dfeb2 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -89,7 +89,7 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina succeed } - def testAsciiFile(options: Map[String, String]): Assertion = { + def testCompressedAsciiFile(options: Map[String, String]): Assertion = { val inputDataPath = "../data/test40_data_ascii/ascii.txt.gz" val df = spark @@ -115,6 +115,32 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina assert(actual == "12345,67890,A1234") } + def testMixedAsciiFiles(options: Map[String, String]): Assertion = { + val inputDataPath = "../data/test40_data_ascii/" + + val df = spark + .read + .format("cobol") + .option("copybook_contents", + """ + | 01 RECORD. + | 05 DATA PIC X(5). + |""".stripMargin) + .option("record_format", "D") + .option("pedantic", "true") + .options(options) + .load(inputDataPath) + + assert(df.count == 6) + + val actual = df.orderBy("data") + .collect() + .map(a => a.getString(0)) + .mkString(",") + + assert(actual == "12345,12345,67890,67890,A1234,A1234") + } + test("Test compressed EBCDIC gzip file") { testCompressedFile("../data/test40_data/example.dat.gz") } @@ -140,21 +166,42 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina } test("read a compressed ASCII file 1") { - testAsciiFile(Map( + testCompressedAsciiFile(Map( "record_format" -> "D" )) } test("read a compressed ASCII file 2") { - testAsciiFile(Map( + testCompressedAsciiFile(Map( "record_format" -> "D", "ascii_charset" -> "ISO-8859-1" )) } test("read a compressed ASCII file 3") { - testAsciiFile(Map( + testCompressedAsciiFile(Map( + "record_format" -> "D2" + )) + } + + test("read a mixed ASCII files 1") { + testMixedAsciiFiles(Map( + "record_format" -> "D" + )) + } + + test("read a mixed ASCII files 2") { + testMixedAsciiFiles(Map( + "record_format" -> "D", + "ascii_charset" -> "ISO-8859-1" + )) + } + + test("read a mixed ASCII files 3") { + testMixedAsciiFiles(Map( "record_format" -> "D2" )) } } + + From d3dd49cb87764ab0099e476f7101459b1b41b55d Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 09:39:19 +0100 Subject: [PATCH 05/10] #809 Fix file end offset setting for compressed files. --- .../cobol/source/scanners/CobolScanners.scala | 20 ++++++++++++++++- .../streaming/BufferedFSDataInputStream.scala | 22 ++++++++++++++++--- 2 files changed, 38 insertions(+), 4 deletions(-) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala index 2e75fa92c..a60f1f2ed 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala @@ -18,6 +18,7 @@ package za.co.absa.cobrix.spark.cobol.source.scanners import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.spark.rdd.RDD @@ -72,7 +73,17 @@ private[source] object CobolScanners extends Logging { val maximumFileBytes = if (reader.getReaderProperties.fileEndOffset == 0) { 0 } else { - fileSystem.getFileStatus(path).getLen - reader.getReaderProperties.fileEndOffset - startFileOffset + if (isCompressed(path, sconf.value)) { + // ToDo determine if the uncompressed file size can be effectively fetched + if (reader.getReaderProperties.fileEndOffset > 0) { + logger.warn(s"File end offset for $path is ignored because the file is compressed.") + } + 0L + } else { + val fileSize = fileSystem.getFileStatus(path).getLen + + fileSize - reader.getReaderProperties.fileEndOffset - startFileOffset + } } val dataStream = new FileStreamer(filePath, sconf.value, startFileOffset, maximumFileBytes) val headerStream = new FileStreamer(filePath, sconf.value, startFileOffset) @@ -81,6 +92,13 @@ private[source] object CobolScanners extends Logging { }) } + private[source] def isCompressed(file: Path, hadoopConfig: Configuration): Boolean = { + val factory = new CompressionCodecFactory(hadoopConfig) + val codec = factory.getCodec(file) + + codec != null + } + private[source] def buildScanForFixedLength(reader: FixedLenReader, sourceDirs: Seq[String], recordParser: (FixedLenReader, RDD[Array[Byte]]) => RDD[Row], debugIgnoreFileSize: Boolean, diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala index 53a89cee5..0eaff3064 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala @@ -86,8 +86,10 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta offsetLeft += lengthLeft lengthLeft = 0 } else { - if (bufferContainBytes > 0) { - System.arraycopy(buffer, bufferPos, b, offsetLeft, lengthLeft) + if (bufferContainBytes > 0 && lengthLeft > 0) { + val available = bufferContainBytes - bufferPos + val bytesToCopy = Math.min(lengthLeft, available) + System.arraycopy(buffer, bufferPos, b, offsetLeft, bytesToCopy) bufferPos += bufferContainBytes offsetLeft += bufferContainBytes lengthLeft -= bufferContainBytes @@ -128,12 +130,26 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta val factory = new CompressionCodecFactory(hadoopConfig) val codec = factory.getCodec(filePath) - if (codec != null) { + val baseStream = if (codec != null) { isCompressedStream = true codec.createInputStream(fsIn) } else { // No compression detected fsIn } + + if (startOffset > 0) { + if (codec == null) { + baseStream.seek(startOffset) + } else { + var toSkip = startOffset + while (toSkip > 0) { + val skipped = baseStream.skip(toSkip) + if (skipped <= 0) return baseStream + toSkip -= skipped + } + } + } + baseStream } } From 178a5105e8cc61f5c96a618cfb34ab01439b878b Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 10:06:01 +0100 Subject: [PATCH 06/10] #809 Add support for indexes in compressed files. --- README.md | 4 +- .../cobol/reader/VarLenNestedReader.scala | 19 ++-- .../parameters/CobolParametersParser.scala | 18 ++-- .../reader/parameters/ReaderParameters.scala | 4 +- .../parameters/VariableLengthParameters.scala | 90 ++++++++++--------- .../cobol/source/index/IndexBuilder.scala | 7 +- .../streaming/BufferedFSDataInputStream.scala | 4 - .../Test37RecordLengthMappingSpec.scala | 6 +- .../Test40CompressesFilesSpec.scala | 25 +++++- 9 files changed, 103 insertions(+), 74 deletions(-) diff --git a/README.md b/README.md index fea8a6ab0..ef64b2cf0 100644 --- a/README.md +++ b/README.md @@ -39,7 +39,7 @@ Among the motivations for this project, it is possible to highlight: - The COBOL copybooks parser doesn't have a Spark dependency and can be reused for integrating into other data processing engines. -- Supports reading files compressed in Hadoop-compatible way (gzip, bzip2, etc), but with limited parallelism (only per-file parallelism). +- Supports reading files compressed in Hadoop-compatible way (gzip, bzip2, etc), but with limited parallelism. Uncompressed files are preferred for performance. ## Videos @@ -1605,7 +1605,7 @@ The output looks like this: | .option("redefine-segment-id-map:0", "REDEFINED_FIELD1 => SegmentId1,SegmentId2,...") | Specifies a mapping between redefined field names and segment id values. Each option specifies a mapping for a single segment. The numeric value for each mapping option must be incremented so the option keys are unique. | | .option("segment-children:0", "COMPANY => EMPLOYEE,DEPARTMENT") | Specifies a mapping between segment redefined fields and their children. Each option specifies a mapping for a single parent field. The numeric value for each mapping option must be incremented so the option keys are unique. If such mapping is specified hierarchical record structure will be automatically reconstructed. This require `redefine-segment-id-map` to be set. | | .option("enable_indexes", "true") | Turns on indexing of multisegment variable length files (on by default). | -| .option("enable_index_cache", "false") | When true, calculated indexes are cached in memory for later use. This improves performance of processing when same files are processed more than once. | +| .option("enable_index_cache", "true") | When true (default), calculated indexes are cached in memory for later use. This improves performance of processing when same files are processed more than once. | | .option("input_split_records", 50000) | Specifies how many records will be allocated to each split/partition. It will be processed by Spark tasks. (The default is not set and the split will happen according to size, see the next option) | | .option("input_split_size_mb", 100) | Specify how many megabytes to allocate to each partition/split. (The default is 100 MB) | diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala index caf568364..636aa806c 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala @@ -144,7 +144,7 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], fileNumber: Int, isRdwBigEndian: Boolean): ArrayBuffer[SparseIndexEntry] = { val inputSplitSizeRecords: Option[Int] = readerProperties.inputSplitRecords - val inputSplitSizeMB: Option[Int] = getSplitSizeMB + val inputSplitSizeMB: Option[Int] = getSplitSizeMB(dataStream.isCompressed) if (inputSplitSizeRecords.isDefined) { if (inputSplitSizeRecords.get < 1 || inputSplitSizeRecords.get > 1000000000) { @@ -153,7 +153,7 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], logger.info(s"Input split size = ${inputSplitSizeRecords.get} records") } else { if (inputSplitSizeMB.nonEmpty) { - if (inputSplitSizeMB.get < 1 || inputSplitSizeMB.get > 2000) { + if (inputSplitSizeMB.get < 1 || inputSplitSizeMB.get > 200000) { throw new IllegalArgumentException(s"Invalid input split size of ${inputSplitSizeMB.get} MB.") } logger.info(s"Input split size = ${inputSplitSizeMB.get} MB") @@ -214,11 +214,18 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], } } - private def getSplitSizeMB: Option[Int] = { - if (readerProperties.inputSplitSizeMB.isDefined) { - readerProperties.inputSplitSizeMB + private def getSplitSizeMB(isCompressed: Boolean): Option[Int] = { + if (isCompressed) { + readerProperties.inputSplitSizeCompressedMB match { + case Some(size) => readerProperties.inputSplitSizeCompressedMB + case None => Some(1024) + } } else { - readerProperties.hdfsDefaultBlockSize + if (readerProperties.inputSplitSizeMB.isDefined) { + readerProperties.inputSplitSizeMB + } else { + readerProperties.hdfsDefaultBlockSize + } } } diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala index 3a4a9ad6f..f40a5814b 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala @@ -118,13 +118,14 @@ object CobolParametersParser extends Logging { val PARAM_SEGMENT_REDEFINE_PREFIX_ALT = "redefine-segment-id-map" // Indexed multisegment file processing - val PARAM_ENABLE_INDEXES = "enable_indexes" - val PARAM_ENABLE_INDEX_CACHE = "enable_index_cache" - val PARAM_INPUT_SPLIT_RECORDS = "input_split_records" - val PARAM_INPUT_SPLIT_SIZE_MB = "input_split_size_mb" - val PARAM_SEGMENT_ID_PREFIX = "segment_id_prefix" - val PARAM_OPTIMIZE_ALLOCATION = "optimize_allocation" - val PARAM_IMPROVE_LOCALITY = "improve_locality" + val PARAM_ENABLE_INDEXES = "enable_indexes" + val PARAM_ENABLE_INDEX_CACHE = "enable_index_cache" + val PARAM_INPUT_SPLIT_RECORDS = "input_split_records" + val PARAM_INPUT_SPLIT_SIZE_MB = "input_split_size_mb" + val PARAM_INPUT_SPLIT_SIZE_COMPRESSED_MB = "input_split_size_compressed_mb" + val PARAM_SEGMENT_ID_PREFIX = "segment_id_prefix" + val PARAM_OPTIMIZE_ALLOCATION = "optimize_allocation" + val PARAM_IMPROVE_LOCALITY = "improve_locality" // Parameters for debugging val PARAM_DEBUG_LAYOUT_POSITIONS = "debug_layout_positions" @@ -385,6 +386,7 @@ object CobolParametersParser extends Logging { isIndexCachingAllowed = false, inputSplitRecords = None, inputSplitSizeMB = None, + inputSplitSizeCompressedMB = None, improveLocality = false, optimizeAllocation = false, inputFileNameColumn = "", @@ -421,6 +423,7 @@ object CobolParametersParser extends Logging { isIndexCachingAllowed = varLenParams.isIndexCachingAllowed, inputSplitRecords = varLenParams.inputSplitRecords, inputSplitSizeMB = varLenParams.inputSplitSizeMB, + inputSplitSizeCompressedMB = varLenParams.inputSplitSizeCompressedMB, hdfsDefaultBlockSize = defaultBlockSize, startOffset = parameters.recordStartOffset, endOffset = parameters.recordEndOffset, @@ -508,6 +511,7 @@ object CobolParametersParser extends Logging { params.getOrElse(PARAM_ENABLE_INDEX_CACHE, "false").toBoolean, params.get(PARAM_INPUT_SPLIT_RECORDS).map(v => v.toInt), params.get(PARAM_INPUT_SPLIT_SIZE_MB).map(v => v.toInt), + params.get(PARAM_INPUT_SPLIT_SIZE_COMPRESSED_MB).map(v => v.toInt), params.getOrElse(PARAM_IMPROVE_LOCALITY, "true").toBoolean, params.getOrElse(PARAM_OPTIMIZE_ALLOCATION, "false").toBoolean, params.getOrElse(PARAM_INPUT_FILE_COLUMN, ""), diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala index ac679e1a9..a8099ae28 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala @@ -20,7 +20,7 @@ import za.co.absa.cobrix.cobol.parser.decoders.FloatingPointFormat import za.co.absa.cobrix.cobol.parser.decoders.FloatingPointFormat.FloatingPointFormat import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy.DebugFieldsPolicy import za.co.absa.cobrix.cobol.parser.policies.StringTrimmingPolicy.StringTrimmingPolicy -import za.co.absa.cobrix.cobol.parser.policies.{CommentPolicy, DebugFieldsPolicy, FillerNamingPolicy, MetadataPolicy, StringTrimmingPolicy} +import za.co.absa.cobrix.cobol.parser.policies._ import za.co.absa.cobrix.cobol.parser.recordformats.RecordFormat import za.co.absa.cobrix.cobol.parser.recordformats.RecordFormat.FixedLength import za.co.absa.cobrix.cobol.reader.policies.SchemaRetentionPolicy @@ -50,6 +50,7 @@ import za.co.absa.cobrix.cobol.reader.policies.SchemaRetentionPolicy.SchemaReten * @param isIndexGenerationNeeded Is indexing input file before processing is requested * @param inputSplitRecords The number of records to include in each partition. Notice mainframe records may have variable size, inputSplitMB is the recommended option * @param inputSplitSizeMB A partition size to target. In certain circumstances this size may not be exactly that, but the library will do the best effort to target that size + * @param inputSplitSizeCompressedMB A partition size to target for compressed files. * @param hdfsDefaultBlockSize Default HDFS block size for the HDFS filesystem used. This value is used as the default split size if inputSplitSizeMB is not specified * @param startOffset An offset to the start of the record in each binary data block. * @param endOffset An offset from the end of the record to the end of the binary data block. @@ -102,6 +103,7 @@ case class ReaderParameters( isIndexCachingAllowed: Boolean = false, inputSplitRecords: Option[Int] = None, inputSplitSizeMB: Option[Int] = None, + inputSplitSizeCompressedMB: Option[Int] = None, hdfsDefaultBlockSize: Option[Int] = None, startOffset: Int = 0, endOffset: Int = 0, diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala index 153dd8fa2..a89d68e86 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala @@ -17,51 +17,53 @@ package za.co.absa.cobrix.cobol.reader.parameters /** - * This class holds the parameters currently used for parsing variable-length records. + * This class is used to hold the parameters currently used for parsing variable-length records. * - * @param isRecordSequence Does input files have 4 byte record length headers - * @param bdw Block descriptor word (if specified), for FB and VB record formats - * @param isRdwBigEndian Is RDW big endian? It may depend on flavor of mainframe and/or mainframe to PC transfer method - * @param isRdwPartRecLength Does RDW count itself as part of record length itself - * @param rdwAdjustment Controls a mismatch between RDW and record length - * @param recordHeaderParser An optional custom record header parser for non-standard RDWs - * @param recordExtractor An optional custom raw record parser class non-standard record types - * @param rhpAdditionalInfo An optional additional option string passed to a custom record header parser - * @param reAdditionalInfo An optional additional option string passed to a custom record extractor - * @param recordLengthField A field that stores record length - * @param recordLengthMap A mapping between field value and record size. - * @param fileStartOffset A number of bytes to skip at the beginning of each file - * @param fileEndOffset A number of bytes to skip at the end of each file - * @param generateRecordId Generate a sequential record number for each record to be able to retain the order of the original data - * @param isUsingIndex Is indexing input file before processing is requested - * @param isIndexCachingAllowed Is caching of generated index allowed - * @param inputSplitSizeMB A partition size to target. In certain circumstances this size may not be exactly that, but the library will do the best effort to target that size - * @param inputSplitRecords The number of records to include in each partition. Notice mainframe records may have variable size, inputSplitMB is the recommended option - * @param improveLocality Tries to improve locality by extracting preferred locations for variable-length records - * @param optimizeAllocation Optimizes cluster usage in case of optimization for locality in the presence of new nodes (nodes that do not contain any blocks of the files being processed) - * @param inputFileNameColumn A column name to add to the dataframe. The column will contain input file name for each record similar to 'input_file_name()' function + * @param isRecordSequence Do input files have 4 byte record length headers + * @param bdw Block descriptor word (if specified), for FB and VB record formats + * @param isRdwBigEndian Is RDW big endian? It may depend on flavor of mainframe and/or mainframe to PC transfer method + * @param isRdwPartRecLength Does RDW count itself as part of record length itself + * @param rdwAdjustment Controls a mismatch between RDW and record length + * @param recordHeaderParser An optional custom record header parser for non-standard RDWs + * @param recordExtractor An optional custom raw record parser class for non-standard record types + * @param rhpAdditionalInfo An optional additional option string passed to a custom record header parser + * @param reAdditionalInfo An optional additional option string passed to a custom record extractor + * @param recordLengthField A field that stores record length + * @param recordLengthMap A mapping between field value and record size. + * @param fileStartOffset A number of bytes to skip at the beginning of each file + * @param fileEndOffset A number of bytes to skip at the end of each file + * @param generateRecordId Generate a sequential record number for each record to be able to retain the order of the original data + * @param isUsingIndex Is indexing input file before processing is requested + * @param isIndexCachingAllowed Is caching of generated index allowed + * @param inputSplitSizeMB A partition size to target. In certain circumstances this size may not be exactly that, but the library will do the best effort to target that size + * @param inputSplitSizeCompressedMB A partition size to target for compressed files. + * @param inputSplitRecords The number of records to include in each partition. Notice mainframe records may have variable size, inputSplitMB is the recommended option + * @param improveLocality Tries to improve locality by extracting preferred locations for variable-length records + * @param optimizeAllocation Optimizes cluster usage in case of optimization for locality in the presence of new nodes (nodes that do not contain any blocks of the files being processed) + * @param inputFileNameColumn A column name to add to the dataframe. The column will contain input file name for each record similar to 'input_file_name()' function */ case class VariableLengthParameters( - isRecordSequence: Boolean, // [deprecated by recordFormat] - bdw: Option[Bdw], - isRdwBigEndian: Boolean, - isRdwPartRecLength: Boolean, - rdwAdjustment: Int, - recordHeaderParser: Option[String], - recordExtractor: Option[String], - rhpAdditionalInfo: Option[String], - reAdditionalInfo: String, - recordLengthField: String, - recordLengthMap: Map[String, Int], - fileStartOffset: Int, - fileEndOffset: Int, - generateRecordId: Boolean, - isUsingIndex: Boolean, - isIndexCachingAllowed: Boolean, - inputSplitRecords: Option[Int], - inputSplitSizeMB: Option[Int], - improveLocality: Boolean, - optimizeAllocation: Boolean, - inputFileNameColumn: String, - occursMappings: Map[String, Map[String, Int]] + isRecordSequence: Boolean, // [deprecated by recordFormat] + bdw: Option[Bdw], + isRdwBigEndian: Boolean, + isRdwPartRecLength: Boolean, + rdwAdjustment: Int, + recordHeaderParser: Option[String], + recordExtractor: Option[String], + rhpAdditionalInfo: Option[String], + reAdditionalInfo: String, + recordLengthField: String, + recordLengthMap: Map[String, Int], + fileStartOffset: Int, + fileEndOffset: Int, + generateRecordId: Boolean, + isUsingIndex: Boolean, + isIndexCachingAllowed: Boolean, + inputSplitRecords: Option[Int], + inputSplitSizeMB: Option[Int], + inputSplitSizeCompressedMB: Option[Int], + improveLocality: Boolean, + optimizeAllocation: Boolean, + inputFileNameColumn: String, + occursMappings: Map[String, Map[String, Int]] ) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala index 8f9391f17..5b51c25af 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala @@ -205,12 +205,7 @@ private[cobol] object IndexBuilder extends Logging { val (inputStream, headerStream, maximumBytes) = getStreams(filePath, startOffset, endOffset, config) val index = try { - if (inputStream.isCompressed) { - val element = SparseIndexEntry(0, -1, fileOrder, 0L) - ArrayBuffer[SparseIndexEntry](element) - } else { - reader.generateIndex(inputStream, headerStream, fileOrder, reader.isRdwBigEndian) - } + reader.generateIndex(inputStream, headerStream, fileOrder, reader.isRdwBigEndian) } finally { inputStream.close() headerStream.close() diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala index 0eaff3064..17077f1e0 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala @@ -123,10 +123,6 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta val fileSystem = filePath.getFileSystem(hadoopConfig) val fsIn: FSDataInputStream = fileSystem.open(filePath) - if (startOffset > 0) { - fsIn.seek(startOffset) - } - val factory = new CompressionCodecFactory(hadoopConfig) val codec = factory.getCodec(filePath) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala index 7c8c77fe4..af627b41b 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala @@ -16,7 +16,6 @@ package za.co.absa.cobrix.spark.cobol.source.integration -import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.scalatest.wordspec.AnyWordSpec import za.co.absa.cobrix.cobol.reader.parameters.CobolParametersParser.{PARAM_ENABLE_INDEXES, PARAM_ENABLE_INDEX_CACHE} @@ -59,6 +58,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "F") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"A":4,"B":7,"C":8}""") .load(tempFile) @@ -79,6 +79,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "V") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"A":4,"B":7,"C":8}""") .load(tempFile) @@ -99,6 +100,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "F") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"1":4,"2":7,"3":8}""") .load(tempFile) @@ -119,6 +121,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "F") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"1":4,"2":7,"3":8}""") .option("strict_integral_precision", "true") @@ -163,6 +166,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("file_start_offset", 1) .option("file_end_offset", 2) .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"A":4,"B":7,"C":8}""") .load(tempFile) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala index 5c71dfeb2..c078ac5e4 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -40,7 +40,7 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina private val expectedResultsPath = "../data/test40_expected/test40.txt" private val actualResultsPath = "../data/test40_expected/test40_actual.txt" - def testCompressedFile(inputDataPath: String): Assertion = { + def testCompressedFile(inputDataPath: String, useIndexes: Boolean = false): Assertion = { // Comparing layout val copybookContents = Files.readAllLines(Paths.get(inputCopybookFSPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") val cobolSchema = CopybookParser.parseTree(copybookContents, debugFieldsPolicy = DebugFieldsPolicy.HexValue) @@ -53,6 +53,15 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina s"$actualLayoutPath for details.") } + val options = if (useIndexes) { + Map( + "input_split_records" -> "1", + "generate_record_id" -> "true" + ) + } else { + Map.empty[String, String] + } + val df = spark .read .format("cobol") @@ -60,8 +69,10 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina .option("schema_retention_policy", "collapse_root") .option("floating_point_format", "IEEE754") .option("strict_sign_overpunching", "true") + .options(options) .option("pedantic", "true") .load(inputDataPath) + .drop("File_Id", "Record_Id", "Record_Byte_Length") val expectedSchema = Files.readAllLines(Paths.get(expectedSchemaPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") val actualSchema = SparkUtils.prettyJSON(df.schema.json) @@ -141,11 +152,19 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina assert(actual == "12345,12345,67890,67890,A1234,A1234") } - test("Test compressed EBCDIC gzip file") { + test("Test compressed EBCDIC gzip file without indexes") { + testCompressedFile("../data/test40_data/example.dat.gz") + } + + test("Test compressed EBCDIC bzip2 file without indexes") { + testCompressedFile("../data/test40_data/example.dat.bz2") + } + + test("Test compressed EBCDIC gzip file with indexes") { testCompressedFile("../data/test40_data/example.dat.gz") } - test("Test compressed EBCDIC bzip2 file") { + test("Test compressed EBCDIC bzip2 file with indexes") { testCompressedFile("../data/test40_data/example.dat.bz2") } From 7cfb193037e88447880d5a53d2a2214789fe719c Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 10:54:09 +0100 Subject: [PATCH 07/10] #809 Add support for file end offset for compressed files. --- .../spark/cobol/source/DefaultSource.scala | 2 +- .../cobol/source/index/IndexBuilder.scala | 10 ++++- .../cobol/source/scanners/CobolScanners.scala | 22 +++------- .../cobrix/spark/cobol/utils/FileUtils.scala | 44 +++++++++++++++++-- .../Test40CompressesFilesSpec.scala | 17 +++++++ 5 files changed, 71 insertions(+), 24 deletions(-) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala index a052ed656..7e5266d01 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala @@ -65,7 +65,7 @@ class DefaultSource val hasCompressedFiles = filesList.exists(_.isCompressed) if (hasCompressedFiles) { - logger.info(s"Compressed files found. Binary parallelism and indexes won't be used for them.") + logger.info(s"Compressed files found. Binary parallelism and indexes will be adjusted accordingly.") } new CobolRelation(cobolParameters.sourcePaths, diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala index 5b51c25af..4d01f8c0c 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala @@ -32,7 +32,7 @@ import za.co.absa.cobrix.spark.cobol.source.SerializableConfiguration import za.co.absa.cobrix.spark.cobol.source.parameters.LocalityParameters import za.co.absa.cobrix.spark.cobol.source.streaming.FileStreamer import za.co.absa.cobrix.spark.cobol.source.types.FileWithOrder -import za.co.absa.cobrix.spark.cobol.utils.{HDFSUtils, SparkUtils} +import za.co.absa.cobrix.spark.cobol.utils.{FileUtils, HDFSUtils, SparkUtils} import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer @@ -231,7 +231,13 @@ private[cobol] object IndexBuilder extends Logging { val maximumBytes = if (fileEndOffset == 0) { 0 } else { - val bytesToRead = fileSystem.getContentSummary(path).getLength - fileEndOffset - startOffset + val fileSize = if (FileUtils.isCompressed(path, config)) { + FileUtils.getCompressedFileSize(path,config) + } else { + fileSystem.getFileStatus(path).getLen + } + + val bytesToRead = fileSize - fileEndOffset - startOffset if (bytesToRead < 0) 0 else diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala index a60f1f2ed..8a5893344 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala @@ -18,7 +18,6 @@ package za.co.absa.cobrix.spark.cobol.source.scanners import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.spark.rdd.RDD @@ -73,17 +72,13 @@ private[source] object CobolScanners extends Logging { val maximumFileBytes = if (reader.getReaderProperties.fileEndOffset == 0) { 0 } else { - if (isCompressed(path, sconf.value)) { - // ToDo determine if the uncompressed file size can be effectively fetched - if (reader.getReaderProperties.fileEndOffset > 0) { - logger.warn(s"File end offset for $path is ignored because the file is compressed.") - } - 0L + val fileSize = if (FileUtils.isCompressed(path, sconf.value)) { + FileUtils.getCompressedFileSize(path, sconf.value) } else { - val fileSize = fileSystem.getFileStatus(path).getLen - - fileSize - reader.getReaderProperties.fileEndOffset - startFileOffset + fileSystem.getFileStatus(path).getLen } + + fileSize - reader.getReaderProperties.fileEndOffset - startFileOffset } val dataStream = new FileStreamer(filePath, sconf.value, startFileOffset, maximumFileBytes) val headerStream = new FileStreamer(filePath, sconf.value, startFileOffset) @@ -92,13 +87,6 @@ private[source] object CobolScanners extends Logging { }) } - private[source] def isCompressed(file: Path, hadoopConfig: Configuration): Boolean = { - val factory = new CompressionCodecFactory(hadoopConfig) - val codec = factory.getCodec(file) - - codec != null - } - private[source] def buildScanForFixedLength(reader: FixedLenReader, sourceDirs: Seq[String], recordParser: (FixedLenReader, RDD[Array[Byte]]) => RDD[Row], debugIgnoreFileSize: Boolean, diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala index 35ec188fc..92ef8c78e 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala @@ -16,13 +16,14 @@ package za.co.absa.cobrix.spark.cobol.utils -import java.io.{FileOutputStream, OutputStreamWriter, PrintWriter} -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Paths} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ +import org.apache.hadoop.io.compress.CompressionCodecFactory import za.co.absa.cobrix.cobol.internal.Logging +import java.io.{FileOutputStream, IOException, OutputStreamWriter, PrintWriter} +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ /** @@ -33,7 +34,6 @@ import scala.collection.JavaConverters._ * Applies the same filter as Hadoop's FileInputFormat, which excludes files starting with '.' or '_'. */ object FileUtils extends Logging { - val THRESHOLD_DIR_LENGTH_FOR_SINGLE_FILE_CHECK = 50 private val hiddenFileFilter = new PathFilter() { @@ -216,6 +216,42 @@ object FileUtils extends Logging { allNonDivisibleFiles.map(status => (status.getPath.toString, status.getLen)) } + def isCompressed(file: Path, hadoopConfig: Configuration): Boolean = { + val factory = new CompressionCodecFactory(hadoopConfig) + val codec = factory.getCodec(file) + + codec != null + } + + def getCompressedFileSize(file: Path, hadoopConfig: Configuration): Long = { + logger.warn(s"Using full scan to determine file size of $file..") + val factory = new CompressionCodecFactory(hadoopConfig) + val codec = factory.getCodec(file) + val fileSystem = file.getFileSystem(hadoopConfig) + val fsIn: FSDataInputStream = fileSystem.open(file) + val ifs = codec.createInputStream(fsIn) + + val size = try { + val SKIP_BUFFER_SIZE = 1024*1024*50 + var totalBytesSkipped = 0L + var skippedLast = 1L + while (skippedLast > 0) { + skippedLast = ifs.skip(SKIP_BUFFER_SIZE) + if (skippedLast > 0) + totalBytesSkipped += skippedLast + } + totalBytesSkipped + } catch { + case e: IOException => + throw new IOException(s"Unable to determine compressed file size for $file", e) + } finally { + ifs.close() + fsIn.close() + } + logger.info(s"The size of the uncompressed file $file is $size bytes.") + size + } + private def isNonDivisible(fileStatus: FileStatus, divisor: Long) = fileStatus.getLen % divisor != 0 /** diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala index c078ac5e4..7cc34c6b2 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -184,6 +184,23 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina assert(df.count == 300) } + test("read mixed compressed EBCDIC files and file_end_offset") { + val inputDataPath = "../data/test40_data" + + val df = spark + .read + .format("cobol") + .option("copybook", inputCopybookPath) + .option("schema_retention_policy", "collapse_root") + .option("floating_point_format", "IEEE754") + .option("strict_sign_overpunching", "true") + .option("file_end_offset", 1493) + .option("pedantic", "true") + .load(inputDataPath) + + assert(df.count == 297) + } + test("read a compressed ASCII file 1") { testCompressedAsciiFile(Map( "record_format" -> "D" From 27665e764ece75c967bcf28fcc37793e74447f7a Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 11:26:51 +0100 Subject: [PATCH 08/10] #809 Fix PR suggestions (Thanks @coderabbitai). --- .../co/absa/cobrix/cobol/reader/VarLenNestedReader.scala | 5 +++-- .../source/streaming/BufferedFSDataInputStream.scala | 8 ++++---- .../source/integration/Test40CompressesFilesSpec.scala | 6 +++--- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala index 636aa806c..81f68fc5a 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala @@ -153,8 +153,9 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], logger.info(s"Input split size = ${inputSplitSizeRecords.get} records") } else { if (inputSplitSizeMB.nonEmpty) { - if (inputSplitSizeMB.get < 1 || inputSplitSizeMB.get > 200000) { - throw new IllegalArgumentException(s"Invalid input split size of ${inputSplitSizeMB.get} MB.") + val maxSplitSizeMB = if (dataStream.isCompressed) 200000 else 2000 + if (inputSplitSizeMB.get < 1 || inputSplitSizeMB.get > maxSplitSizeMB) { + throw new IllegalArgumentException(s"Invalid input split size of ${inputSplitSizeMB.get} MB (max allowed: $maxSplitSizeMB MB).") } logger.info(s"Input split size = ${inputSplitSizeMB.get} MB") } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala index 17077f1e0..910c12d9e 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala @@ -90,9 +90,9 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta val available = bufferContainBytes - bufferPos val bytesToCopy = Math.min(lengthLeft, available) System.arraycopy(buffer, bufferPos, b, offsetLeft, bytesToCopy) - bufferPos += bufferContainBytes - offsetLeft += bufferContainBytes - lengthLeft -= bufferContainBytes + bufferPos += bytesToCopy + offsetLeft += bytesToCopy + lengthLeft -= bytesToCopy } } } @@ -136,7 +136,7 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta if (startOffset > 0) { if (codec == null) { - baseStream.seek(startOffset) + fsIn.seek(startOffset) } else { var toSkip = startOffset while (toSkip > 0) { diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala index 7cc34c6b2..d17c7cfe4 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -161,11 +161,11 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina } test("Test compressed EBCDIC gzip file with indexes") { - testCompressedFile("../data/test40_data/example.dat.gz") + testCompressedFile("../data/test40_data/example.dat.gz", useIndexes = true) } - test("Test compressed EBCDIC bzip2 file with indexes") { - testCompressedFile("../data/test40_data/example.dat.bz2") + test("Test compressed EBCDIC bzip2 file with indexes") { + testCompressedFile("../data/test40_data/example.dat.bz2", useIndexes = true) } test("read mixed compressed EBCDIC files") { From 23aabb0ddf960857bb6975eb773622dbafad31c7 Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 11:37:28 +0100 Subject: [PATCH 09/10] #809 Fix more PR suggestions, especially a potential integer overflow (Thanks @coderabbitai). --- .../za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala | 5 +---- .../cobol/source/streaming/BufferedFSDataInputStream.scala | 4 ++-- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala index 81f68fc5a..34cd81021 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala @@ -217,10 +217,7 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], private def getSplitSizeMB(isCompressed: Boolean): Option[Int] = { if (isCompressed) { - readerProperties.inputSplitSizeCompressedMB match { - case Some(size) => readerProperties.inputSplitSizeCompressedMB - case None => Some(1024) - } + readerProperties.inputSplitSizeCompressedMB.orElse(Some(1024)) } else { if (readerProperties.inputSplitSizeMB.isDefined) { readerProperties.inputSplitSizeMB diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala index 910c12d9e..33ed2de4a 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala @@ -38,7 +38,7 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta private val buffer = new Array[Byte](bufferSizeInBytes) private var bufferPos = 0 private var bufferContainBytes = 0 - private var bytesRead = 0 + private var bytesRead = 0L @throws[IOException] def close(): Unit = { @@ -67,7 +67,7 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta if (bufferPos < bufferContainBytes) { val bytesLeft = bufferContainBytes - bufferPos System.arraycopy(buffer, bufferPos, b, off, bytesLeft) - lengthLeft -= bufferContainBytes - bufferPos + lengthLeft -= bytesLeft offsetLeft += bytesLeft } bufferPos = 0 From 8000d43152a389791ad1129904fd05959e34213e Mon Sep 17 00:00:00 2001 From: Ruslan Iushchenko Date: Fri, 19 Dec 2025 13:36:04 +0100 Subject: [PATCH 10/10] #809 Make index cache 'true' by default, remove some code duplication. --- .../cobrix/cobol/reader/VarLenNestedReader.scala | 10 ++++------ .../reader/parameters/CobolParametersParser.scala | 4 ++-- .../cobol/reader/parameters/ReaderParameters.scala | 4 ++-- .../cobrix/spark/cobol/source/DefaultSource.scala | 6 +++--- .../streaming/BufferedFSDataInputStream.scala | 6 ++---- .../spark/cobol/source/streaming/FileStreamer.scala | 9 ++------- .../absa/cobrix/spark/cobol/utils/FileUtils.scala | 13 +++++++------ .../absa/cobrix/spark/cobol/utils/SparkUtils.scala | 6 +++--- .../integration/Test40CompressesFilesSpec.scala | 4 ++++ 9 files changed, 29 insertions(+), 33 deletions(-) diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala index 34cd81021..5ac664de3 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala @@ -42,6 +42,8 @@ import scala.reflect.ClassTag class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], readerProperties: ReaderParameters, handler: RecordHandler[T]) extends VarLenReader with Logging with Serializable { + private val DEFAULT_INDEX_SIZE_COMPRESSED_FILES_MB = 1024 + private val DEFAULT_FS_INDEX_SIZE_MULTIPLIER = 4 protected val cobolSchema: CobolSchema = loadCopyBook(copybookContents) @@ -217,13 +219,9 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], private def getSplitSizeMB(isCompressed: Boolean): Option[Int] = { if (isCompressed) { - readerProperties.inputSplitSizeCompressedMB.orElse(Some(1024)) + readerProperties.inputSplitSizeCompressedMB.orElse(Some(DEFAULT_INDEX_SIZE_COMPRESSED_FILES_MB)) } else { - if (readerProperties.inputSplitSizeMB.isDefined) { - readerProperties.inputSplitSizeMB - } else { - readerProperties.hdfsDefaultBlockSize - } + readerProperties.inputSplitSizeMB.orElse(readerProperties.fsDefaultBlockSize).map(_ * DEFAULT_FS_INDEX_SIZE_MULTIPLIER) } } diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala index f40a5814b..e563d6650 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala @@ -424,7 +424,7 @@ object CobolParametersParser extends Logging { inputSplitRecords = varLenParams.inputSplitRecords, inputSplitSizeMB = varLenParams.inputSplitSizeMB, inputSplitSizeCompressedMB = varLenParams.inputSplitSizeCompressedMB, - hdfsDefaultBlockSize = defaultBlockSize, + fsDefaultBlockSize = defaultBlockSize, startOffset = parameters.recordStartOffset, endOffset = parameters.recordEndOffset, fileStartOffset = varLenParams.fileStartOffset, @@ -508,7 +508,7 @@ object CobolParametersParser extends Logging { fileEndOffset, isRecordIdGenerationEnabled, params.getOrElse(PARAM_ENABLE_INDEXES, "true").toBoolean, - params.getOrElse(PARAM_ENABLE_INDEX_CACHE, "false").toBoolean, + params.getOrElse(PARAM_ENABLE_INDEX_CACHE, "true").toBoolean, params.get(PARAM_INPUT_SPLIT_RECORDS).map(v => v.toInt), params.get(PARAM_INPUT_SPLIT_SIZE_MB).map(v => v.toInt), params.get(PARAM_INPUT_SPLIT_SIZE_COMPRESSED_MB).map(v => v.toInt), diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala index a8099ae28..01148faec 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala @@ -51,7 +51,7 @@ import za.co.absa.cobrix.cobol.reader.policies.SchemaRetentionPolicy.SchemaReten * @param inputSplitRecords The number of records to include in each partition. Notice mainframe records may have variable size, inputSplitMB is the recommended option * @param inputSplitSizeMB A partition size to target. In certain circumstances this size may not be exactly that, but the library will do the best effort to target that size * @param inputSplitSizeCompressedMB A partition size to target for compressed files. - * @param hdfsDefaultBlockSize Default HDFS block size for the HDFS filesystem used. This value is used as the default split size if inputSplitSizeMB is not specified + * @param fsDefaultBlockSize Default HDFS block size for the HDFS filesystem used. This value is used as the default split size if inputSplitSizeMB is not specified * @param startOffset An offset to the start of the record in each binary data block. * @param endOffset An offset from the end of the record to the end of the binary data block. * @param fileStartOffset A number of bytes to skip at the beginning of each file @@ -104,7 +104,7 @@ case class ReaderParameters( inputSplitRecords: Option[Int] = None, inputSplitSizeMB: Option[Int] = None, inputSplitSizeCompressedMB: Option[Int] = None, - hdfsDefaultBlockSize: Option[Int] = None, + fsDefaultBlockSize: Option[Int] = None, startOffset: Int = 0, endOffset: Int = 0, fileStartOffset: Int = 0, diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala index 7e5266d01..cea0b29da 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala @@ -174,7 +174,7 @@ object DefaultSource { */ def createTextReader(parameters: CobolParameters, spark: SparkSession): FixedLenReader = { val copybookContent = CopybookContentLoader.load(parameters, spark.sparkContext.hadoopConfiguration) - val defaultHdfsBlockSize = SparkUtils.getDefaultHdfsBlockSize(spark, parameters.sourcePaths.headOption) + val defaultHdfsBlockSize = SparkUtils.getDefaultFsBlockSize(spark, parameters.sourcePaths.headOption) new FixedLenTextReader(copybookContent, getReaderProperties(parameters, defaultHdfsBlockSize) ) } @@ -185,7 +185,7 @@ object DefaultSource { def createFixedLengthReader(parameters: CobolParameters, spark: SparkSession): FixedLenReader = { val copybookContent = CopybookContentLoader.load(parameters, spark.sparkContext.hadoopConfiguration) - val defaultHdfsBlockSize = SparkUtils.getDefaultHdfsBlockSize(spark, parameters.sourcePaths.headOption) + val defaultHdfsBlockSize = SparkUtils.getDefaultFsBlockSize(spark, parameters.sourcePaths.headOption) new FixedLenNestedReader(copybookContent, getReaderProperties(parameters, defaultHdfsBlockSize) ) } @@ -199,7 +199,7 @@ object DefaultSource { val copybookContent = CopybookContentLoader.load(parameters, spark.sparkContext.hadoopConfiguration) - val defaultHdfsBlockSize = SparkUtils.getDefaultHdfsBlockSize(spark, parameters.sourcePaths.headOption) + val defaultHdfsBlockSize = SparkUtils.getDefaultFsBlockSize(spark, parameters.sourcePaths.headOption) new VarLenNestedReader( copybookContent, getReaderProperties(parameters, defaultHdfsBlockSize) ) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala index 33ed2de4a..7aa39cbb7 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala @@ -18,7 +18,7 @@ package za.co.absa.cobrix.spark.cobol.source.streaming import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FSDataInputStream, Path} -import org.apache.hadoop.io.compress.CompressionCodecFactory +import za.co.absa.cobrix.spark.cobol.utils.FileUtils import java.io.{IOException, InputStream} @@ -121,11 +121,9 @@ class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, sta private def openStream(): InputStream = { val fileSystem = filePath.getFileSystem(hadoopConfig) + val codec = FileUtils.getCompressionCodec(filePath, hadoopConfig) val fsIn: FSDataInputStream = fileSystem.open(filePath) - val factory = new CompressionCodecFactory(hadoopConfig) - val codec = factory.getCodec(filePath) - val baseStream = if (codec != null) { isCompressedStream = true codec.createInputStream(fsIn) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala index e486b51f1..acf786d4b 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala @@ -18,10 +18,10 @@ package za.co.absa.cobrix.spark.cobol.source.streaming import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{ContentSummary, Path} -import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.log4j.Logger import za.co.absa.cobrix.cobol.reader.common.Constants import za.co.absa.cobrix.cobol.reader.stream.SimpleStream +import za.co.absa.cobrix.spark.cobol.utils.FileUtils import java.io.IOException @@ -51,12 +51,7 @@ class FileStreamer(filePath: String, hadoopConfig: Configuration, startOffset: L private var wasOpened = false private var bufferedStream: BufferedFSDataInputStream = _ - private lazy val isCompressedStream = { - val factory = new CompressionCodecFactory(hadoopConfig) - val codec = factory.getCodec(hadoopPath) - - codec != null - } + private lazy val isCompressedStream = FileUtils.isCompressed(hadoopPath, hadoopConfig) private lazy val fileSize = getHadoopFileSize(hadoopPath) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala index 92ef8c78e..4499ff0ea 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala @@ -18,7 +18,7 @@ package za.co.absa.cobrix.spark.cobol.utils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.io.compress.CompressionCodecFactory +import org.apache.hadoop.io.compress.{CompressionCodec, CompressionCodecFactory} import za.co.absa.cobrix.cobol.internal.Logging import java.io.{FileOutputStream, IOException, OutputStreamWriter, PrintWriter} @@ -217,16 +217,17 @@ object FileUtils extends Logging { } def isCompressed(file: Path, hadoopConfig: Configuration): Boolean = { - val factory = new CompressionCodecFactory(hadoopConfig) - val codec = factory.getCodec(file) + getCompressionCodec(file, hadoopConfig) != null + } - codec != null + def getCompressionCodec(file: Path, hadoopConfig: Configuration): CompressionCodec = { + val factory = new CompressionCodecFactory(hadoopConfig) + factory.getCodec(file) } def getCompressedFileSize(file: Path, hadoopConfig: Configuration): Long = { logger.warn(s"Using full scan to determine file size of $file..") - val factory = new CompressionCodecFactory(hadoopConfig) - val codec = factory.getCodec(file) + val codec = getCompressionCodec(file, hadoopConfig) val fileSystem = file.getFileSystem(hadoopConfig) val fsIn: FSDataInputStream = fileSystem.open(file) val ifs = codec.createInputStream(fsIn) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala index 80fb49869..72c040228 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala @@ -19,12 +19,12 @@ package za.co.absa.cobrix.spark.cobol.utils import com.fasterxml.jackson.databind.ObjectMapper import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkContext -import org.apache.spark.sql.functions.{array, col, expr, max, struct} -import za.co.absa.cobrix.spark.cobol.utils.impl.HofsWrapper.transform +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, SparkSession} import za.co.absa.cobrix.cobol.internal.Logging import za.co.absa.cobrix.spark.cobol.parameters.MetadataFields.MAX_ELEMENTS +import za.co.absa.cobrix.spark.cobol.utils.impl.HofsWrapper.transform import scala.annotation.tailrec import scala.collection.mutable @@ -489,7 +489,7 @@ object SparkUtils extends Logging { }.getOrElse(None) } - def getDefaultHdfsBlockSize(spark: SparkSession, pathOpt: Option[String]): Option[Int] = { + def getDefaultFsBlockSize(spark: SparkSession, pathOpt: Option[String]): Option[Int] = { val conf = spark.sparkContext.hadoopConfiguration val fileSystem =pathOpt match { diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala index d17c7cfe4..f5fb54c34 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -56,6 +56,7 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina val options = if (useIndexes) { Map( "input_split_records" -> "1", + "enable_index_cache" -> "false", "generate_record_id" -> "true" ) } else { @@ -178,6 +179,8 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina .option("schema_retention_policy", "collapse_root") .option("floating_point_format", "IEEE754") .option("strict_sign_overpunching", "true") + .option("generate_record_id", "true") + .option("enable_index_cache", "false") .option("pedantic", "true") .load(inputDataPath) @@ -195,6 +198,7 @@ class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with Bina .option("floating_point_format", "IEEE754") .option("strict_sign_overpunching", "true") .option("file_end_offset", 1493) + .option("enable_index_cache", "false") .option("pedantic", "true") .load(inputDataPath)