From df81c65b97c860abc0e81a6bbfebb3699cb9d70d Mon Sep 17 00:00:00 2001 From: Michael Goddard Date: Wed, 23 May 2018 11:21:58 -0400 Subject: [PATCH 1/5] Initial integration of PXF S3 support (from a separate repo); tests are passing --- pxf/build.gradle | 35 ++- pxf/pxf-s3/.gitignore | 30 +++ pxf/pxf-s3/README.md | 206 ++++++++++++++++++ pxf/pxf-s3/csv_to_parquet_pyspark.py | 54 +++++ .../data/sample_from_pyspark.snappy.parquet | Bin 0 -> 100183 bytes pxf/pxf-s3/images/Greenplum_Logo.png | Bin 0 -> 41936 bytes pxf/pxf-s3/images/Parquet_Logo.png | Bin 0 -> 25469 bytes pxf/pxf-s3/images/Plus_Sign.png | Bin 0 -> 7897 bytes pxf/pxf-s3/images/S3_Logo.png | Bin 0 -> 17178 bytes .../java/org/greenplum/pxf/s3/AvroUtil.java | 104 +++++++++ .../main/java/org/greenplum/pxf/s3/PxfS3.java | 188 ++++++++++++++++ .../org/greenplum/pxf/s3/S3Fragmenter.java | 73 +++++++ .../greenplum/pxf/s3/S3ParquetAccessor.java | 68 ++++++ .../pxf/s3/S3ParquetJsonAccessor.java | 81 +++++++ .../pxf/s3/S3ParquetJsonResolver.java | 26 +++ .../greenplum/pxf/s3/S3ParquetResolver.java | 23 ++ .../pxf/s3/S3ParquetWriteAccessor.java | 83 +++++++ .../pxf/s3/S3ParquetWriteResolver.java | 98 +++++++++ .../org/greenplum/pxf/s3/S3TextAccessor.java | 66 ++++++ .../org/greenplum/pxf/s3/S3TextResolver.java | 59 +++++ .../org/greenplum/pxf/s3/AvroUtilTest.java | 68 ++++++ .../org/greenplum/pxf/s3/InputDataMock.java | 33 +++ .../java/org/greenplum/pxf/s3/PxfS3Mock.java | 89 ++++++++ .../java/org/greenplum/pxf/s3/PxfS3Test.java | 49 +++++ .../greenplum/pxf/s3/S3FragmenterTest.java | 97 +++++++++ pxf/settings.gradle | 1 + 26 files changed, 1530 insertions(+), 1 deletion(-) create mode 100644 pxf/pxf-s3/.gitignore create mode 100644 pxf/pxf-s3/README.md create mode 100644 pxf/pxf-s3/csv_to_parquet_pyspark.py create mode 100644 pxf/pxf-s3/data/sample_from_pyspark.snappy.parquet create mode 100644 pxf/pxf-s3/images/Greenplum_Logo.png create mode 100644 pxf/pxf-s3/images/Parquet_Logo.png create mode 100644 pxf/pxf-s3/images/Plus_Sign.png create mode 100644 pxf/pxf-s3/images/S3_Logo.png create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/AvroUtil.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/PxfS3.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3Fragmenter.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetAccessor.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonAccessor.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonResolver.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetResolver.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteAccessor.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteResolver.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextAccessor.java create mode 100644 pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextResolver.java create mode 100644 pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/AvroUtilTest.java create mode 100644 pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/InputDataMock.java create mode 100644 pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Mock.java create mode 100644 pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Test.java create mode 100644 pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/S3FragmenterTest.java diff --git a/pxf/build.gradle b/pxf/build.gradle index 1ed42bf08e..0b580453d9 100644 --- a/pxf/build.gradle +++ b/pxf/build.gradle @@ -123,7 +123,7 @@ subprojects { subProject -> force 'commons-codec:commons-codec:1.4' force 'commons-collections:commons-collections:3.2.1' force 'commons-logging:commons-logging:1.1.3' - force 'org.apache.avro:avro:1.7.4' + //force 'org.apache.avro:avro:1.7.4' force 'org.apache.zookeeper:zookeeper:3.4.6' force 'org.codehaus.jackson:jackson-mapper-asl:1.9.13' force 'junit:junit:4.11' @@ -510,6 +510,39 @@ project('pxf-jdbc') { } } +project('pxf-s3') { + dependencies { + compile(project(':pxf-api')) + compile(project(':pxf-hdfs')) + compile "com.amazonaws:aws-java-sdk-s3:1.11.313" + compile "org.apache.parquet:parquet-hadoop:$parquetVersion" + compile "org.apache.parquet:parquet-tools:$parquetVersion" + compile "org.apache.hadoop:hadoop-common:2.8.2" + compile "org.apache.hadoop:hadoop-aws:2.8.2" + compile "org.apache.avro:avro:1.8.2" + compile "org.apache.parquet:parquet-avro:1.8.2" + testCompile "io.findify:s3mock_2.12:0.2.5" + } + + ospackage { + packageName = versionedPackageName("${project.name}") + summary = 'HAWQ Extension Framework (PXF), S3 plugin' + description = 'Querying external data stored in S3' + packager = ' ' + packageGroup = 'Development/Libraries' + release = buildNumber() + '.' + project.osFamily + buildHost = ' ' + + requires(versionedPackageName('pxf-hdfs'), project.version, GREATER | EQUAL) + + from(jar.outputs.files) { + into "/usr/lib/pxf-${project.version}" + } + + link("/usr/lib/pxf-${project.version}/${project.name}.jar", "${project.name}-${project.version}.jar") + } +} + def buildNumber() { System.getenv('BUILD_NUMBER') ?: System.getProperty('user.name') } diff --git a/pxf/pxf-s3/.gitignore b/pxf/pxf-s3/.gitignore new file mode 100644 index 0000000000..8a3a4e4b4f --- /dev/null +++ b/pxf/pxf-s3/.gitignore @@ -0,0 +1,30 @@ +# Compiled class file +*.class + +# Log file +*.log + +# BlueJ files +*.ctxt + +# Mobile Tools for Java (J2ME) +.mtj.tmp/ + +# Package Files # +*.jar +*.war +*.ear +*.zip +*.tar.gz +*.rar + +# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml +hs_err_pid* +.gradle +gradle +.classpath +.settings +.project +bin +build +.*.swp diff --git a/pxf/pxf-s3/README.md b/pxf/pxf-s3/README.md new file mode 100644 index 0000000000..88a8afe420 --- /dev/null +++ b/pxf/pxf-s3/README.md @@ -0,0 +1,206 @@ +# PXF extension to Greenplum to provide access to Parquet formatted data stored in S3 + +
+Greenplum +Parquet +S3 +
+ +## TODO +* Add support for Avro data + +## Collect all JARs and install on Greenplum segment server(s) +* The Gradle build will deposit all the transitive dependencies into `./build/libs` +* `tar -C ./pxf-s3-parquet/build/libs -cvf pxf-s3-jars.tar .` +* Copy this up to the Greenplum server(s) and extract into, say, `~/gpadmin/pxf-jars` +* Add this list of JARS to the `$GPHOME/pxf/conf/pxf-public.classpath` file: + ``` + ls -1 ~/pxf-jars/* >> $GPHOME/pxf/conf/pxf-public.classpath + ``` + +## Add new PXF profile +* Edit `$GPHOME/pxf/conf/pxf-profiles.xml`: +``` + + + S3ParquetWrite + A profile for writing Parquet data to S3 + + org.greenplum.pxf.s3.S3Fragmenter + org.greenplum.pxf.s3.S3ParquetWriteAccessor + org.greenplum.pxf.s3.S3ParquetWriteResolver + + + + S3ParquetRead + A profile for reading Parquet data from S3; does not support nested data + + org.greenplum.pxf.s3.S3Fragmenter + org.greenplum.pxf.s3.S3ParquetAccessor + org.greenplum.pxf.s3.S3ParquetResolver + + + + S3ParquetJsonRead + A profile for reading Parquet data from S3, as JSON; useful with nested data + + org.greenplum.pxf.s3.S3Fragmenter + org.greenplum.pxf.s3.S3ParquetJsonAccessor + org.greenplum.pxf.s3.S3ParquetJsonResolver + + + +``` + +## Restart the PXF server on all segment hosts +``` + $GPHOME/pxf/bin/pxf restart +``` + +## Example: reading data from Parquet files in S3 +* Data source: you can upload into S3 the five Parquet files found [here](https://github.com/Teradata/kylo/tree/master/samples/sample-data/parquet) +* DDL: replace the `S3_ACCESS_KEY`, `S3_SECRET_KEY` and `S3_REGION` values +``` +-- Parquet input, JSON output +DROP EXTERNAL TABLE IF EXISTS parquet_json; +CREATE EXTERNAL TABLE parquet_json (js JSON) +LOCATION ('pxf://pxf-s3-devel/the-data/userdata?S3_REGION=us-east-1&S3_ACCESS_KEY=ACFOCBFFDJFHZ69M7GM7&S3_SECRET_KEY=3DFq0kV01aEdNzr5uxkeN7Hr/cZv6erDjM3z4NsB&PROFILE=S3ParquetJsonRead') +FORMAT 'TEXT' (DELIMITER 'OFF'); +``` +* Queries on this data use the Greenplum JSON operators described [here](https://gpdb.docs.pivotal.io/500/admin_guide/query/topics/json-data.html#topic_gn4_x3w_mq) +* One possible query for this data set. Note the use of parenthesis around the `js->>'salary'`, since the cast operator (`::`) binds more tightly than the `->>` operator. +``` +SELECT js->>'country', AVG((js->>'salary')::float)::NUMERIC(9, 2) +FROM parquet_json +GROUP BY 1 +ORDER BY 2 ASC +LIMIT 10; +``` + +## Example: write data into Parquet files in S3 +* DDL: replace the `S3_ACCESS_KEY`, `S3_SECRET_KEY` and `S3_REGION` values +``` +DROP EXTERNAL TABLE IF EXISTS write_parquet; +CREATE WRITABLE EXTERNAL TABLE write_parquet (a INT, b TEXT, c BOOLEAN) +LOCATION ('pxf://pxf-s3-devel/test-write?S3_REGION=us-east-1&S3_ACCESS_KEY=ACFOCBFFDJFHZ69M7GM7&S3_SECRET_KEY=3DFq0kV01aEdNzr5uxkeN7Hr/cZv6erDjM3z4NsB&PROFILE=S3ParquetWrite') +FORMAT 'CUSTOM' (formatter='pxfwritable_export'); +``` +* Insert some rows into this table +``` +INSERT INTO write_parquet +(b, a, c) VALUES ('First value for b', 1001, false), ('Second value for b', 1002, true); +``` +* Build a *readable* Parquet S3 table, to verify what was written (JSON format) +``` +DROP EXTERNAL TABLE IF EXISTS read_what_we_wrote; +CREATE EXTERNAL TABLE read_what_we_wrote (js JSON) +LOCATION ('pxf://pxf-s3-devel/test-write?S3_REGION=us-east-1&S3_ACCESS_KEY=ACFOCBFFDJFHZ69M7GM7&S3_SECRET_KEY=3DFq0kV01aEdNzr5uxkeN7Hr/cZv6erDjM3z4NsB&PROFILE=S3ParquetJsonRead') +FORMAT 'TEXT' (DELIMITER 'OFF'); +``` +* Finally, query that table to verify that values made it +``` +SELECT js->>'b' a, (js->>'a')::INT a, (js->>'c')::BOOLEAN c +FROM read_what_we_wrote +ORDER BY 2 ASC; +``` +* Or, use a table which uses the primitive data types (doesn't support nested data structures) +``` +DROP EXTERNAL TABLE IF EXISTS read_primitives; +CREATE EXTERNAL TABLE read_primitives (LIKE write_parquet) +LOCATION ('pxf://pxf-s3-devel/test-write?S3_REGION=us-east-1&S3_ACCESS_KEY=ACFOCBFFDJFHZ69M7GM7&S3_SECRET_KEY=3DFq0kV01aEdNzr5uxkeN7Hr/cZv6erDjM3z4NsB&PROFILE=S3ParquetRead') +FORMAT 'CUSTOM' (FORMATTER='pxfwritable_import'); +``` +* And read from this table +``` +SELECT * +FROM read_primitives +ORDER BY a ASC; +``` + +## Caveats +* You have to replace a JAR in the standard classpath with one we need due to the S3 libs: +``` +[gpadmin@gpdbsne conf]$ diff -u pxf-private.classpath.ORIG pxf-private.classpath +--- pxf-private.classpath.ORIG 2018-04-17 19:17:48.178225024 +0000 ++++ pxf-private.classpath 2018-04-17 19:18:35.114744844 +0000 +@@ -52,7 +52,7 @@ + /usr/lib/hadoop/client/commons-logging.jar + /usr/lib/hadoop/client/guava.jar + /usr/lib/hadoop/client/htrace-core4.jar +-/usr/lib/hadoop/client/jackson-core.jar ++/home/gpadmin/pxf-jars/jackson-core-2.6.7.jar + /usr/lib/hadoop/client/jackson-mapper-asl.jar + /usr/lib/hadoop/client/jetty-*.jar + /usr/lib/hadoop/client/jersey-core.jar + ``` + +* Out of memory? +If the query fails, and you find `java.lang.OutOfMemoryError: GC overhead limit exceeded` in the catalina.out, +try increasing the heap size for the Java PXF process (it's running Tomcat). + - File: `/usr/local/greenplum-db/pxf/pxf-service/bin/setenv.sh` on each segment host + - Line to change: `JVM_OPTS="-Xmx16g -Xms8g -Xss256K -Dpxf.log.dir=/usr/local/greenplum-db/pxf/logs "` + - The `-Xmx16g -Xms8g` is the *new* value (it was set to a much lower value initially) + +* Inconsistency in handling TIMESTAMP and DATE types, with Parquet / Avro +The `BridgeOutputBuilder` class contains the following check, which makes it impossible to serialize +these two types, using the Avro schema approach typical with Parquet, in the way that they are usually +handled (e.g. `DATE` would be represented at `int`, and `TIMESTAMP` as `long`): +``` + for (int i = 0; i < size; i++) { + OneField current = recFields.get(i); + if (!isTypeInSchema(current.type, schema[i])) { + throw new BadRecordException("For field " + colNames[i] + + " schema requires type " + + DataType.get(schema[i]).toString() + + " but input record has type " + + DataType.get(current.type).toString()); + } + + fillOneGPDBWritableField(current, i); + } +``` +That `isTypeInSchema()` method incorporates the following logic, which basically implies that these two +types can be handled, so long at they are serialized within Parquet as `String` type: +``` + boolean isStringType(DataType type) { + return Arrays.asList(DataType.VARCHAR, DataType.BPCHAR, DataType.TEXT, + DataType.NUMERIC, DataType.TIMESTAMP, DataType.DATE).contains(type); + } +``` +So, if *we* are writing Parquet data from Greenplum, we would write consistent with this approach, but +it remains to be seen how well we will handle Parquet data written by some other system. + +## Notes +* Here's an example of an INSERT into the S3 Parquet external table, reading from an s3 external table +``` +gpadmin=# INSERT INTO osm_parquet_write SELECT id, extract(epoch from date_time)::int date_time, uid, lat, lon, name, key_value from osm_ext; +NOTICE: Found 31 data formatting errors (31 or more input rows). Rejected related input data. +INSERT 0 5327616 +Time: 12495.106 ms +``` +* Example: read of that Parquet data, using the `S3ParquetJsonRead` profile +``` +gpadmin=# select count(*) from osm_parquet_read; + count +--------- + 5327616 +(1 row) + +Time: 184331.683 ms +``` +* Alternatively, read that same data, but from a table created using the `S3ParquetRead` profile +``` +gpadmin=# select count(*) from osm_prim_read; count +--------- + 5327616 +(1 row) + +Time: 6549.945 ms +``` +So far, it seems that the JSON read mode is quite a bit (about 30x) slower, so there's work to be +done to explore what's causing that. + +* A [reference on tuning Parquet](https://www.slideshare.net/RyanBlue3/parquet-performance-tuning-the-missing-guide) +* An [article about moving to Parquet file](https://www.enigma.com/blog/moving-to-parquet-files-as-a-system-of-record) +* [This script](./csv_to_parquet_pyspark.py) was used to verify that this solution is able to read Parquet data files written by Spark + diff --git a/pxf/pxf-s3/csv_to_parquet_pyspark.py b/pxf/pxf-s3/csv_to_parquet_pyspark.py new file mode 100644 index 0000000000..56cb6a59fb --- /dev/null +++ b/pxf/pxf-s3/csv_to_parquet_pyspark.py @@ -0,0 +1,54 @@ +# +# Used to convert the Open Street Map dump data, which I have in .gz files, +# delimited by '<', in S3 +# +# To to this, I first set up a Spark cluster in Amazon EC2 "EMR", then logged +# into the master node via SSH, then started the pyspark shell. +# +# See: +# http://spark.apache.org/docs/latest/sql-programming-guide.html#programmatically-specifying-the-schema +# https://spark.apache.org/docs/1.6.2/api/python/_modules/pyspark/sql/types.html +# https://stackoverflow.com/questions/33129918/pyspark-typeerror-integertype-can-not-accept-object-in-type-type-unicode +# + +import datetime + +# Handle potential junk data for FLOAT type +def get_float(val, defaultVal=0.0): + try: + rv = float(val) + except ValueError: + rv = defaultVal + return rv + +# And, for LONG type +def get_long(val, defaultVal=-1L): + try: + rv = long(val) + except ValueError: + rv = defaultVal + return rv + +lines = sc.textFile('s3://goddard.datasets/osm_exerpt_0*') +lines.count() # Should be 5327647 +cols = lines.map(lambda l: l.split('<')) + +typedCols = cols.map(lambda c: ( + get_long(c[0], -1L), # LONG + datetime.datetime.strptime(c[1], '%Y-%m-%dT%H:%M:%SZ'), # TIMESTAMP + c[2], # STRING + get_float(c[3], 91.0), # FLOAT + get_float(c[4], 181.0), + c[5], + c[6])) +df = spark.createDataFrame(typedCols) + +# Writes 8 files into this bucket. The bucket must exist, but the "from-spark" part will be created +# if it doesn't exist. +df.write.parquet('s3a://pxf-s3-devel/from-spark', mode='overwrite') + +# Downsample this data set, then take 1k rows and save it for future testing. The resulting +# file is here (in this repo): ./data/sample_from_pyspark.snappy.parquet +df = df.sample(False, 0.0002, seed=137).limit(1000) +df.write.parquet('s3a://pxf-s3-devel/sample-from-spark', mode='overwrite') + diff --git a/pxf/pxf-s3/data/sample_from_pyspark.snappy.parquet b/pxf/pxf-s3/data/sample_from_pyspark.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d35d59df943879c9568d6447a409fc8b4d472f7d GIT binary patch literal 100183 zcma&Od0b52A3uI4Gr5!7bgL<2x-F9??akcT36)Bv2qA=`D9ReLuWgil-y=yvi0nl6 z5Rwo=B};_t+xL0jpU3y{`~Cj=`Qz2O=bZaGuk%`-uh%)ZMELbnS`k*u=2&eQ;bB#2 zhPJU4$w)}$sf{lQ{F9KgzFG3PmNiGNbPH&MAC+dj|J&_+qL+XqU!0l`T3x*nwDn^y z=%xI#pk?xF0+K&#nUEyc=2r`7GOo{4bpIIB0NU{8hk&Gx&unYNiy=o^DbY1;!2-~h zbvdA!!)$p=D1tRvfmu)ejQBhS*<;{fj94)V=sK!-E#OJ0sYf-&Jd!lMyBK|%!3kijQGPF6z9IjKiyD)=0%aQk!{hHrEUl6+QSaCVt^L(Ro|YVzRSWv)k`OW9-cZC)a6w^==-b_peZ%i zLB0HLgPt7y2z0@CD30nr+{T=j%HK8-^6p&c3bl4Ecf;uVqb%seh5*p7*8M=sB$Gh5 ze&9qm(xijVIJX5fw}=yZVcIcJ?*J%^?9=HXIKLx2EO@D;bb^MPlmtfn)xJJh z<A53$(*87>e}FC;&b4pb)gBvKc(TJTkT9O(Y+#<2FCE z8zXBBAM z(DNAXv5)G&b0H^gH$_+68%|BPb(qR0{h+-Gq6}!y#)ey8;8qXX#k&b_O|)jEykN5X z=rD9$TeuMP;iF3!Sb1fP4K%Yl7h+J`j0uKE^Efto_&H^1rq-7%R-VW_2${}=JQYY!`L>29eLi=7;?C{J7XbRG76L) zycQCa*|7EqzwJW<(KXaG8MO0u#5dU`h|~Ovpyi;I#&@v#hpXVzv;AF2_NEv3nvcfw z=?;$s5^A!}(*elR^%6I`wbv;yZg06n#*^@ucNCJuc2kl+Z?2H$Pgsoi+GOE9Ua!4S z<=0oo@xs%)oVjX^KjO7&`+knR2r^^hP;`m5F9Y3bb`o@3C^tNCqkBhK|6x4`f(Msw z0PUS;L$MfZ`V~T7yaL4>6D)NCR`hzi82?CBM}N+*9UgB3^<(aU=GAdnb-j#qfxcIpb1rjkupacvVYm$Wia!YS zY(Fj@AF*3OUH6>>9n}5_Uh6O^*cJX8P|ErH74M^jnWPq;N=~ z58`m`%~nJ+b+FRS9q#=t#0y<3j*S3qf2aub)HDcAI@Ck+RO9vb9@y{DUenRl^Y?nt z7r&rL%E~Lm6L06b{ltTA<9ILN`P_02L8X7V@s9=Vy?GLH+RR>fI6P0Oz_|ZO1Tp!< zoik~rStSNe&k`%aMX2Lk?83N3pjG!VK;CBGV7Ty&5ewP7KAnZGpQoX1vTzwEdra8? z72aF*Gz$+WRn}^7|8Bwg#m%~G&|@b{F%a2)fCjUwF3m*O5tABptsgv42eHmyi9**u zM{d)vj&myO;sBLVeJ#E9Sg_H71HwWO17ND?5zgnpvN^@LZ^pl+doodz#>Q^wodlcf%_-n>OL$N^<}1c-Y~= zd1{PkH)xr%6$9D+iM~7s>fm3MAKr9#ipI!pE~!&?ZwP-#|KJdbuIZnbf(p6<$*6uQ zQ9ZCz!3_==n+iEhJ}~1T=$?=80kYqV`=HHK6QbNu_KlLFyoJEn6s+NI`#nf}0O^zV=pL0Ecje|RJnxT{-F zPA7RBlhfapfzCGlg8_fLF2R_UGL#!=HzycfKBwM=U^DcN-{`W?VHP>o?;qk)G!XF@ zwHm~78_1Gsz{Fp`gzTq~raYGrp}YwG>{}Qyszm@hDtL&#Fc!~$mvhrgL89-0jCzaV zE@nmo1;4iko(T@ILjcx%T>{AS6p&rVKt7)ZdD)EFyNuchg83%~;JIKF$cBv|Hn||n z4uRY`4x;;ti5ikG@a0=|!n3P8$nnk~Pr8B-|1jJwK!8<+01`6D#uxx|U@C@`GeL|# zgIxN8kxtLr3S7vxV>x3M3=m@H4hu1sc><&uNKLZA6hOQgB9ce^O~K{H8#I@g(`f4Z zW0vK3JijnAhXOuW;`z2C$TwGzcps3#+?)_@A8n5U->g1ahRW*rBqv}Su^3Daxnayp z4^owbPFb)O*7Mwe=1O;Kj1x;RlnYV#ViZ*L z5oyxc+65Zk$uSr;9lbN(MIt+GC|wXL+d;nTaTZ3%oLUB&6vahU<2zX|IPC0cj_5jn z(G_&W1|_Iyb{Fum;`9~vcO)2}I+~98+guaEP?~i8z87=qv$Reu;?0a6(~ir3IG6AY1yMcRy^OC z1Mp7w3PDw#&oJQO7Zizs{=GT)*?OAOLaGUpBb5~*8^Ckm`-Ebje+DiR(A3rez5^kF z2O3J|vC`}QG#u%yQg7`dCoLAv?3 z7ejGg_+XV5%_I4pt-(IDZa>%`qYH*%VCZer;dmuf<%U<9isM1|4C2zpv|cknyIkB1 zCfn|`9l;fj##}A6Z25!%F;h;LBVW4l$j0e3|mk*k?p%yeG zmD_%HeBW4D?drfm=$c%`t#c}s%N8+PIE4&9_79VUd6P$CwJsY0RMeTZgGRwLKW`x! zQ|%`$kHf>a>H<8(eDIFPLl=*+c+*usoqPW>C-eAMbr{YzB}c=JPme;+)p#!V$DHJH zM#KJV7>yE$#~`yTdJkkIyv;|8&__*zSp?jilk@CQvu62wn! zUWl%=(3_yYzi=M;c~)#PEOO}{SGz3ItU=eQJJZHP%xDuNLGspZcoTL0S(gduQuiK* zu1kMUfjZxLf&n+N(L}sX{M4c=a!*o+*qf9}9jvc&E=9OvdfViHfmHv*iHV=Iaf zk>!ri6UBZ^n~Znc&B(%AT}!yGZ%z-Uz{h$i?NZ=l&z_#a$Pj%qMvku!pNf&kr{_<@ z$j$<;h~W3%33@vR3Zp_TJ5C3)m`>mE@NmSKR2*rQe?>ER5dMn)O#GPQr=EpZThyHN z3q)0zmo-8=8{6_WS=iX4$D9KH{De-(SC9T+;Lwz~C3sOdljGQn zo(*dIjtdUO#`#MzFw~60Lf_vfKpTdafwB+2VR!?XoDG44O4IRh=JMtoEO_`I$8p{| zb{QTjcX7e6TFTWl^FDFjF8(B04vxusJ9M47n*utk2dDcQ+g1$U`yR9c96J@JqDyQC z>yvvMI4G>!@^K|pU>xth3VZK#mJ5x=2{0FF_m6Ys-)?TJ!S@5h`S_fT2vo}O_|h6U z$ITvH*5Z|m+vZ}#Wc3!%gE&2q)W20$>+nkOs@3Rf91A^CXE$lrLqpHz<>A37W$gy+ z!SN&lm&_f;rGVNvlT8@E=+O^d*Y9&K=s#oGW^^UL*p7!&`@FW`6$i>28YuoOC z^!7^vFfuS90rY6X9MJK7xj=kShy75$OrGT;%uUB|yv{pj?1amC%XaL7v&es0=HdN_ z4*rpIhrx^=UMbD!SfYtrv7<85Aodk8<$tiPMIv2?P zA@&EM#$W2*=sK{FbNx00M;3#XvU~zJx6c=J&G^kfgsvCkI5@OmR#RkL;DkN9i>|znCdaUBb%Oz2_R$=xfG#gE;Cfe5in#;-JD2}S;pAavJ;Fe2 zg!DKD_{VhUGL8)dUAti@=-8Nzpr6t>m(Bd|Oj{9q8||4N5Pak$<_}&zk!N%Qr)$$o zLUeU&wgUYo=?MDT>i~u$MlvUPF8m8+uz0kol0(@lHAYipKA`uGVi{`V4X0Ds-4HQn z#dZSiY3Sq2L@pg(qZff53pfs1;&}t~)~Vl^)I3Cd25;DTKgYxTdhuDjv8cTt#?Lll7NJpUYwb#(`)ZEuPEJm#MB)nia97pzH6ie9-1X?2ek#-S;}2+vv!k8+g5X z(j@eibsJUzVTUjyF|GCs6hi$o8g>)Lo&JoIQdhqe^!wKhpj}VENvX5T+uQ>CHYd1X zTe+bRXj9!4&=WZbE%N2!7EJolO@157n`*+jb>TK_feJZmRtdIka=A)|%>*7%s;N`% zplgbVi-%A1sJjU1*B^cFK|058chGm%FTM)W9sRQn57i0%@8h9;d(MOw?wzagF#Nyk zK*^~!;5qy3cF;AM_c1)mdhP=V>GS|Ek`+I9qAU2xff_9KY0GCkG$ivM;`P0wA7DJh zlCx0&A%BGN;O(5+?ulb>qe3wu zqA};27IW6PjKCb5{tGZ#oa*ufh1K0zn9p0m0ZeQi|0!0P^N`ctu4*oqU-IK>EnzBW zZKEb2041B4{0!Q&T(<>Xu?yf~-ipyRn}mC6={*I{$EuAF1H43Ip>P7;qOFurU@ z1gOj(17xYgGYp&7o4ml>prTIbs+!#yH1>HUH>r3cXi&ptOxkiS`Xx`{hd z_iEpYZ*~3@&+67zUKn@t&nJkwVJ!i~hFw>joe5&ubU8t=r(LFm6GYN9w69*0cy}X+ zd9%rqteT5zv5`?M@I&XQH=Y=DKI*1ksI0 zG5po#l@R8JjARD(sh9f^L@lp|xoczSAt8vT=v_F1?F7xN;QeM!d%x|1=1LwFW;EEO zr;H$cI9Vm_`YBup;*>1fp=sl`Us8hTI#otWR<+A=B!~gLVCrsWZ3wT1m&7cdkwbSR zh_+~Ms5CzrOb{u&;4ou-X99W}$?YTZ@XP@4E#W1FJ+0|v)*D-zfo|ts6Wm~uwmc`M z_V|oSXgh@$!&r6#iP=*r35i}Fjzx%NSFJB9!s6a8z_Ifsgv|80W~+o}V56af|Jq>* z9MqVK(UVCS@3vnNQ?u=MFzmhybN^1Q8Vw7K;Z-mehvMRbIJ+pA%#KEohS&lQ`XBpW zIB~1_FcbGK?BA0hjIrA3gIgWZT!CIX{B%?p9FVtyKF)Nr4>GM_9xXVw1lInHRTn30 z`3hFUvGat6jm}-6O}yig@bfZscNj`o@vo)=+YOQ>Fb6ILKReZnRwWDqAZ0O{wysL7IA?9`Nih<;5E$pa9{e5#Se5Eu5`d=cJ2O1#Yz*ec@Dy~#>Or^sbE28| zAMf4*5XiBo?2ZCC)aQi=$~?2=6&%L_l78CU^I->q*ebiq44J(5sTt?TCCu_S%&jZLojGd;GfSSn=mHM8*kF-a5A#rh*yhJ)(p4jR zhZDq3%zb*d>I5{n7f$x3;Y=Us^(y9SbADt(vVCaU-5K8xig$sK#rNs8036h7%+A{v z`@`~wz@f3q>qmQnXoKxq_;#d1IuM%#>*m&0Eg??@!^iW4EpT(#C@z5K%}*S^Y(%a+e){Lz&1JU z&+HysfJktc&1VLhIXA$6J7dD#m_6qO1kuFvA-@)klZlB3kS1xL(G#rj6y{4$91-YE z5Y;@2zHWMRS?A=%%%ho+{ovbW!ZQ~z`7~KrY8+bgRFxP;bGn* zy1-CYkm}tBVk^{ql(&NVI_l3Ea|sokL4Gr*=(v>9EDuu!ehhz?a8ElMqJg~osx#oJPx&k46Yi&6TbI*m{6m=;)E2Kl76aCwM>0e({ zAG&_sJu5#`ifNshQwIMie@F)7`~pDM6E0RCHc64$t<+)jP1RZMKfDc6aMz( zV$zABU0&9~f}uQ$D%=!cmm3E~EIcKh0e^3UFm!p+?wMzJCF8ukmluMlFZP*VIdMuC z2tz+^6YPVZlXtyL>GM$wJQ)G*MlXCk z;2p=|T%Q((PI5w!SAP%HLRf#ilfJJ@7_2z~qD5wAUjSYwWFzb>JdO-|?JzLyxubNkkeO!tTnB}p#!KCv_yyQx@%Y+Lahg73rJ zZ7)ryB26|UIXCHZw*zihgIOUz+Y)m_p=jURza9cjKf~>OsPh$SX&6ZtjF@$(n{>Vj zQ|I~TGS-(vMP9r2t$~?(g6E3IHFu$J9o);V^p8Q_9Kb}(uBG8qcTtDTy*uPJnl}#hEGuRrZaQy+?JZ#=wL{V&%*}^;QBK|G0CKIt^wFEF+gd4~ zWcr<(H9D9eF2TE7FY-pqxEhUOE-Wi8f~VTyeQC+fbf{_o5aUI{id(RdBX(Ep=)4-) z6d#+Jf)2NvNP=)fLJ!a<4@E@EQO2>VuQ|~BTdtHug7F=|S54@$ z>4;=Mu!s_7{6Yn~ju%XqB)^%^Q@Vvh-Wi?_X~JM{r{t}lu!YM|n3V#M?_(X1B<68LQnJ29Dnfi1f=aJ;1FyYumQapX!i4a6^MXcf`vye99ahIR-ws$ zIavpJ?*N4@qQ%89S{=;ud-%k%E~ZILYC%J99aqQ}F#OKicP^~p9v zH}SdMNF?vV@AQFjEUuc;)cj7jB~o7z)0DaYxg`z?nAvTt)gstD3ecN0ZQ43`bOj{v zFm`H&*F6+OO4fa!hP`^AhN-5na|$to zzomnfob%|&GQj9ak45#da;YC0X03UsnD$MUwd)op%W8f zBsq+wPNL}+RyrFf0xL4=vuhtfPf1X?s0-PG<{?bbXZaHVzYsJ@Gqan}?@okMrRCFs zSZy80Y{-a0pct1;d(Sj^2I!KaN_yd3as_koEsCj{6cz+#@z7=c*pURbI|RMQw)UC> zr`jw_p;t*hBnL}f+A))zFaD6=h%LKHzuEOX*i{-OV&aVAf}jgr%~JVK$A+MAXw93| z!(b|5-q$Y_nQ^DN3g&!8N`C>)21xfVE31cM((tJdSGGkB5x4TBq=RV^UCbEeyGv`^ zQIeBwcbQA*v;H9`JDM)0>MvF2V*2S%9lP6@`Y_#MN7!RCYj|5Ib#ifgB=j|hGIL-o zAT#ZJ!6%TbJE}+d_?i3A`9q!Gq2i$=zC8j^*}cm?W?`1lp8WKJz;FL?+G*0R z4l>kI)oY0&KhsF+tFp~|EZA9f)8K2WpoZAzN16Ljh39P7VZox11Ha9kDC2?&bMRx? zlSymMQ|YSl1B_fvx6t~tw}NDz2{gadb%P5^u$rq;e;}(z-}#>)MFS zQ3*w?kR{OPD+8?b;WWKt?4I>{(*io+a!IHTdJMH+t#UJsp}t=0mW@UETlQ|jYCHU{ zem8fbCx%?_?&*n-o%O$E*sQ|&^luLsNV4UN0!7DK#8qS5MeFWRUCQE~60Coqy7Hl@ zoJ3x8%lOsMT^eCRHFk=9h2pU4tot)_Cw!~gy}}decbp{2_^lJPFr#lwxuAK`eF4xy@*M z=q<`W_>K-XYoczPM)#J;ZHW$Ga=|^>5eArl&26p*Zc(vtbV(P}7HXjD@9UUV<6>^^ z133#`85+$cq}N$fE7Zw7oCfPmok;QczYFRD*U+<1F3Rj|8bd2ee|pPJ7t=*&zc1>7 z74!G4|K)_2hfEXSga0<_`|*%kZ{_;ZVNAr(AZU&$w*Kn__e%9v-0Os&qK1N~u6V=$ zPdAv4s(kS&5k{@Q6JGC(YPDFdjYJ8k2X9#qY|c}vYD%w(5+Lx9XF zxlweoWoo*l}Z7%6mN1gkwSW~!pz(7>5l0D<1#Arx?IVJv~qW#b2g zF-Cm*7=YAVAjptvqp&tyu}9JQ50amD2Czjmb~FZ>ZASiD^MmE z9TQUZR@>3U`InL=C0vqK0l%#yYujKUdn~kO&zh%5P6yH4bUA4+R5^^-LU)cXstK3+ zSTRqV?IHk=N1uxsJ7uTQzzz#!qTrp)=KxyC*lkpT!UOf*4fGbCYCj?Lze1CqJKUkO zoitd;3@k0k03ycY&STB$o<|62xtN(e*W?qb;~KopM(x;&)L93ak}{{QMhN6RkEYM3 zta7oE2ADFN%0Arz%F1}>nPbGdwh;L(HdoVBx>b!EL+EDR?^~mgygs2xMojMrnLZ<_ zyf`p9x3VFcsU7Q_fz1DxCy|zu|CKX4S3F&Tq&o-qEkynO_Ym9|2g@R#;XcUa*PtSk zVPUUiDUj^^`%{rXbqUW&{I}$UYX`!*%9LrH>t$mN8D)A}?LMK0gLDPpy8O}0UeZc2 zAF*GVUrgydEmK`{Pt?CxX}j=N*_Hb<-Q8 zb`rP0NV?-t2wJ}~+JHo|38IXD-DxP+j}XL2*7tvp^6?^Sla}>^o?}x-Fur5Rr5Fb; zz56gFi{k>|RC#dJ^xU3X@z!oM)dkCsz)V8OGrK73a2F}ljtPiP9)SI=!uvIICl;X0 zX@OF{wrH*+8rPvr+2cW9#nn} zuEc$#$T!087;wQu@9dSIY9ksab%8p!EHb_b`5wUbJ6X}w9)xvX2gd2ZSZkm?SG<-e zI!=M4&#`{kl}!(^!FqUI&Ey{~8ooS@S3*yu@9}%&=$TrNh&`Ct4DnY; zx)@>okC>>q8qq$KJE0aZLEe*NPzGPY$+xJddm^ecoQs&e)b54QL^+y@TA?Xo4Crz*V-bps7IfaIX_v0f1(xnEGXQbpjp-V)+9Gn>b8}z?wPVTUhr5rPJc|;RMPetJS!5Lo*S`L3Hb)#9^q@yPcKxy`PhFx+cO*!fWVAz zo2MoSLhcw9-TtI6>@gX(%bs()81{&V*0rnykc(ns25;<(U>}Fr>bK0j2$|}tcGz;b zdJ;D0Gl%a9|F{b~Nbh$sfpi>{FMF7KkIx<7aTU9Mv?ai7Dw;t-@6_g8_CV2>cIv7< zcrzm^CW7~T@Sfl1m)6cjir920gu0gDg@~^^CLlU+4N70;ho+0AMI_^2U;7q;;00)} z?`L+v$rNXy-I8D+^M2@Una{EBP;e4%#t&aLn_`ok%6w1nIt5|{g5RUoIdkFoM=^1r z)56akL^9k2M$g8(ahax)8~PGE2LnCN1d0!9^iURIO`tnu9&(L&MnYH8;rIbeo%{$ z#Sx!59@othu5k&sz~hZql)`S#cqce+_Wm$baF0626s?S){$48{E1E6!JI)m3=GYo@ zXB~yO@?E3Eo3LIZX6}A1-+&z=_GogF4Mj-R=1kKNJMK17gI#-lSuhE!HQ~dMnxXhS81?kMm6J5v1WqVn#+pr@ z0n{i3l22|JvJ1^~Rsh1%qwAH(>bcS%R+jKXzC*v-sXghM|tU9$DrVFW=oqO|qZXiu!V3IfF4NgWJL ztVVO@gemPM+?^Li&Gp^;)2|7A)3UqjtO%kL>gHO{5lhU&E6JFFUV~E>)ua*lHOv!(anl%I9V%Js_OkU7P{w3kMFBWj>0#;py@lg zvkCB*1hZ|Yx}Jc++M(B}eyk~s1B9g)4r};MApGiV8{A+S?qiFGQq~WgP--xdJ)M%^ z&-Z{}RlN4suoGM=QD;RJ14VWAU?e#)EC;tzZfKsi`zgapYBa;nEn5mV)}rbEL;n*I zqZ3fnfYWo$aFf7Dd;Ro-5m5j#BL@-7fv*ufD#rm=>J;8P*HUdm~Y<`nEoJhAa9;BF=uJ#zmf2kOs4bLOxqQs`$kn%N^h@4{TJ+O2Ir zcr;AF-PqS1lJ10xve3NDKiD4{UW#UQd4d4FInbpQH`v;D~!aEjvpLzpL;@*-2TTWY9l`M8wT)TbIE?vZ>kgGpLWKH^|1~8P4u@Y9 z{uFmliC_qcMP16X`i8hV4UY(mFaL)qKZAw(z0Ej)&Hq3%Dou3?*n9;{PtI8U$x*7H zm^at1?Sdbi;6>6#3#MuurBNi4zgy%79dM``bYyuKByUeRPv>VX8c25!#Y~H1K`aN( z*aj}A5P)5wh|$`$R&a>dkp8%Ca5zHvEw}uy#0F@JyO&wBb%!@_0o4dyIcG_SK4aky z|LmkK5@x($K^4{%V0p=_zFJHiEz|QS+loes_^2fDHEV0<84%inM@h!cI58f}TX>i- zNt>pvz@|H3f=gO?qdjg~Am7=^@^XN<1LUJDk0|iLgb6!u(!I?Le8-_D9~;!( zH(9}iwywPemFZDroL%iTQ;hR{K+Dbwzz#}xdVb@oM>yXz**S+^g>j#rCDIB>U=%a= zW6&+cPXJFMF51iM(Uq{CC=w?ezFgA@8j&*kgsR!_2wX1F>>u~va;XbR*Oyh!^pLht zbiXdcmpa{4&=xoD&kmM4S-f67fpzFj zp+gx9e7qjwdr&0j?s=VP>%X&;P>ze)M(ZyjgmpG8U2K6G#fcJ|os^p66p^5k&BqOS z3Y{NzJYFN6-+{h<+5DxIRBTC?ENWS;mM%s*%AR}#tH$A15vot742=DaWw^I04G9X9`~GqgBS#km6C>fQEsEFiMP-La6 ziO7e$wftMJ7)^4YGw}uIPTE+6AF=>Le@TgfnA}-IyR+pQLSYXye5-Nqr}ok&5mR!p z-eAXFNh@fRvzAZ0tc>RWuo3-sLKNdB-ORfUoc9?dKlbBKqQ5i~H+#z*m$l(8u5qDg z&@^xDp;`Yi+DmuXs=}V-7Egk-p}Hh zs}bVw!KQFmIRhQ<<)zZZ7-^VJS}7$rB_9$MyC77Z$oF&epKm~&?!*)c2V@!k3sxr8 z{nAs5|BeOBDt=oUy}z-ZdTFOY*x+BR*VGjM#S_PJ8nD$iR9eu9I=|pWlgLHHixj8y ze$l@hA&g-ZPKHDzmJaw^g-Q3u;#(ji-Z|?z!bl2Tzv~*==qru1VAk2#%K)XFaa<`^ zT@}GIIzpI64_t0>AO0!vaYghCoSFmd>EUiYI|Xu?=z!}@s_-?2F@*zFiSyB{=&r1ag%P6?YFifu-It?M{mL_!3UXwBAs8?XP4m{7IdGFq{Qt6Gq`)I_YdN)5N>#hWJ?w8LZ#@c-)oy z^#F>QXgo6$S(D3;cWqBsBi#Y}>BZYdXBu$!o$cJ)9NWZ+fMzz^pFyso9c`Zka41%z zupNOwSOE`C==0o$`<(3+HsA!p5#dCpI9rHXDBv_D9?MJJ)sYZV^p1v^59Q$CL{?UD zSN!ae9(#}>Hoz>j&G%+Fc_E~zw7n|;6YjD?Z941@!t)>mT9?FsuGml0E4r^Y)k-&5 z(`OY2mh{5STkFq9k36Mu=CpY2ms6J9UCNM}H$@DVS*)UqgK`4Ig)ePacoNp-4vg5$ zz6tSj6`#)~CQs+tgfsMpLe~*JAWON|5q=PN?F9FGxn*9*|NUlzzUHrdbJ(Rb@ zaJrB~w#pdCGtg%PY*0DUcM*zOE|{%Pc^rlNA*WKz5mSFrEy<&zCbqAffD+{s_l>W3 zZVe|=E86%Y`x;BD7ncA1!9D+fe{fIX7u^2$wFJM{C83D_j>1SV`291Kgaq+U*!q@_ zK;i!M-eexZcQ{Y*#WCl3q|=V`MwGhld>cBzp^Pt%j42aH;&zl7GbwdtZ5+Z$njyF6?sz%=MJYK)^hS%xV7dO1m^E*~3%EhB9 zlvka0Rj?K6mldkYfQ~oSRgt4^YHH$k-PG2myu7KaTio%MzJAT9TZX2*UAH>FD|vaV zOG|mj+di#Tqi%QoUcc*hx8JQVZ+9mIPL=o@y0}U|Ql3}oPig8ad(eSScLKx%;_d`W z;`8nVF;naB^mIsbx*P1YHtueS%kI3pp&m!;@Agt$a=I6$z8`llT;Gs)ueZ;a`g?tR z1@fwh0IT?_zQOX{RsF&=4ORUk0_FE3BL~Fa9}pG4`~JY_sSWoB#iYrrqvF=aR}W6u zy}Noy($R+Mp%X93A4I3zkAE;MwPE*z;j_OqJQy)w;9L{4*lKi5Y^FTFW@NUev1ZhY zK<9^XYX*#d7{4Jt|KaE@QyU+S*^%b_C?Rj{=tqeKyYnB7EjrrxXk76n=f_DU_eVcY zKHiZ3c>L)vjgKds7j&wfSZ+0@cG6Y(p4!P3npd?`Dg!$`NvRqz=E>BW_&rai)lPl& zWO`j%r>Cj)YsWmD(X@Nd)0yv%zIr;V?vc{XRx_l7;s=KlWj>e)QP*rjeh z-#Vdg0qI;&w~*2{)h(ifT%Io$4@`KTCK+AuJe`@=^gP2M-Q`85)4GHgSuXhnFP3;5 zYkIL%aoOc%wz@juWsbhF;N>!(uT3wP`x?8}uL!VCtX~=IytjT;nD%x3>WCoMhBc7` z6C2h>jo#a^E_&MQhV?P&u8kYw)+IJ>OvvBcxGCw_>&DF!FT1|ll2V=cYHMoa-dEdZ ze|`OG`+Q@!rX7o|$2R3=Iu|zW%+|hX+O;Cc?RDOofn#6q-Y~lGb^exVZ(i@&k?!`U zAaC8+H+u{63*QtL9eeX;U-4zPw?!q@W8dyS-dOnd!0E4V-X1(}?EbE}+mH@=KHX4U@%UV!evtI}rv6pY=UYDC-hIC9 z+t#zSGN65O>z&|E`&;jZ>6%;bMfCLiQWZHU`OE#NG5fz%M^A75@*pO|^J`7q`sA+< z6ZY)?`Y5Ti`Rn6}S3JMfraVag_9XSy{%=obe{25sY<^p>?{$mYkN^HWv(tg^FS2#- zzrS43)9Xk5nnB}#G;A1i;78+@>F6rl|D&&$q=_yneka zc`*K0^YK>)e!V~a?ftJ0=i7S!ZYgg+;rGX@oeuu~RH6Iu`*USa??0_ogC_j>QZwe@ zpRcvkKm7Stm*M^Qd;R(ee}CY&lbhnKj`s|+T4TcfZgL45Yz+8gl?uIHfg*&^=nN{B zXDLyt)Osb$=h>1Pt(IpYR%uv+Uc<8>RT{m(iB_>{wL)M)DYZHSPfBal3PW3-TFmOz zTD_9jk5=mpIz7)3C6;L8VpmyghVER;%QoS&v6|vDRQ<^(vtWrBEw1JQGTz*C~ZN#5$H$;w>?))adoB z@G`B#TN>VE39B(ElxhXfUW^@S)EZtasaG3JODT;?t>alx8jV&Z=u9h>N)0Q_rB!;h zO2rEhD?6N=U=ftOmS=Pics(Rq;kqDy@c< z&!uz$HL!Vx``o)f;#N zDFe&u1hXmDpjFBH&^m)bqvzRBN`+P_m_(}}uR<0}YIO?1P+FyARZ5v9me*m^@sz;; zTfC`P7@({${Q06*UP|h8>JS;F zW?{=?v`Whw41!3pN^gK21&K7P(rdMX-D2!ot!658f zr4hE!DjoEjXvSBmYiQ%_ORwGym^Ruiz zB!tqd&|gd971vdiPNmn#_fv2YR=5L8XbgfWB-|_{iB>AqI-S6u)?z`eybq;S>(r)~ zNo-TrNI_wmzyL5;8^Bya!`9I60sK@c)B=04QlnC-mGT#G4=B@hEsZTIS;2GcMIX`+ z3fHO;8fukZ6*81mE46|s=ta*e<($zC8rN==%AixqFOV7y>$-*ThiL8(t%<(ER}gfOVWpv@XaXtE3#)#{Qj&|1WUg13;bG05@Dl?r&lFN%eQ z&3_R}!EU~eP5X@1wFvyQcF-?0E=Z-+8ih{2mTypdj-=IE7BOTA#lks;!LJeW@+b;n z%?6sqt{3FfO1QjAz)%LIRwFn_V!c2KXDWrxRSAFBsD<7XoKYz^<|~y8>I7+&R-@7e ze#sRPdeavOC#6EZfQ0?}4#;f{H1KuGkh7E$Kot0euU8BG#Y(jvMwSz#POT3)1;c4I z@(YMJtvb9bsnoII?j*u0%K>qt6z0*aLZj5Vx&dch|O_^fvgkzE!z!&*A+{y(G^e(g;GOSOTM z_^cxKhOiT*)G2hk|6#)htv19SsxSzrlHh8Z3H<|$_nY{w%6(0;8h!J4*&^W(3W8|& z-sC;xW97b-N@-9u7eO-ybz%e@Mz6!CYn|UD!8r0pvH3sR%dhCD?M&_Knq zwtiAVwYL|+KEELU1rus?SswuvD!m{OfTduSg3ZWpK)}E^h%&V*a4fn3EUym%Y!n8e z3j$7|R|VdplmIGg7ZVC-$OXR!0U;dBZC^NwuhR0C2-Z@V$%d=|-~$;fCgE?uky(1v z1_k^p&z6I> zhBU0#>F`JQtcqo|o-;`utMk0h*BNpaaDr-;uDgJbI`#fQS_`jN32r=ZL+It^@N=a~ zV{u8y2MpA53=MXBO<*S-s|;L$--J6ZTrP4W1#h_QuLwCIG8Ynh>ux5bUZu9~Nf8E7 zFe&3UtxKFt=tMqbUG`spD@dX` z*LaAnP;26s05*}j)Uo{tr6`&dl+tPz`XuA!Weqv_VMYI@c>S zq7}umQ#2vuhN>96qbf{VemK0W%tvz=T49$GS$+YQ&&2+zn8Q z@?k&za|(U)GQLLbcZ=5*&Zky13;71UEOvLSG?)Tjv1<7kY(tZH0&l3aiT((^?ty*a znXFXY4!J<5OPPMyU5qwZD+(+ruEGr!^Bov}>GTx6+ z#eYhn)mXPNgTf7ZRu?!8MjLM;?33## zU=aukeaIzL5-8;4Pf%s5rS|v}P^CU+Xzq-xT)s+ws0x9>8Ol%8jr1G~73=oC<|;1j zwL&0(()u39U-Z2l?+n}<+9^f>!xxitw_QeJgtaUMGG*-J;gviia_Y~5Q zN3X?)K~3&B-Vh)nYp`Z$E)^>@uJ7>w?qu~rY^Yh~HA@-=C9Af;{||3lL1H zd>t+Z^y`$G1!p0w&b2#C2q1X(+=5VBcM?KP6qQwltgcYU2h&KFdTsm|WFQ@@b}i?; zR%z`mEE_^-;?I&=g=+6^i=h;uboC~Yb(=+;>*<93V0x87=muP7Sp_Q`1`I?q-kh)0 zh`LkpNhFFAYd_>`R@9*#?}9~!Iqwfn7pPA3f#+!*VwjcpMnYCy`-@}*R1Q3c5?PIk ztv89JDfKi#wT-ihzz0e}sJc!E;_39#*Cb!BlWrz~2puitNiuUn8DfTX#p+!LLngf@ z5oXqCtaq71@hW|m1CBOG9D!+wca1`0Ef(_ingvgI*GL`sY!y;Uq&m4bl7nbuJK+W# zbW{rAF75yT)tPoDftiWt;9@8*mAnwdvR>_K;CO0vf#HZW*1#qP1FOnN(-SB&p#6Lr z^@38J7C@o2QsYda#r8s+kq1?JmN%3FJgEhnxP*pa3p_0b{-_Nq*Q12KY?XW&LXg$o zOo4kT5{FZ;qDrU|Z4yaSX?y->aASRq%CK(ns)q(Y$!Po?$91zIFC zq%B3xnAk=_EqGbR5~{>pO2fkWFY^uT{s^ud(z^~9;|PWtO{k&3SP||IO&k@yFH(J)it)Ur!J9oK7wVu6+59IZvYpbI(s++WraiAk@tK5POxv}!|Ee6AUx zkjA2fWl>rx#4I+18upWHd-6IwplVeJMu>J;?lr;_DpRe{orW^?8rM8phhwE)ZVy|l z49c{h2nUVI`hbwo%F^ZicvA@|6F`q*OHfD{{PJ8aaB5SbN)N_q0Vmwu9Yl{T2t~Lf zjba5wjZDQ=KY24ZMmIx<=z5sW!Nt%2f$Z zQyS!ZJNz$MI80lYnDALP_lsaI@L!?8X@g)z7c5Ng!DaP|@S`HalF$lb2)3an@qozG z(lr>R$dqg8A|4s8L}7~ZL=a1;EdE-gm2#HRuz@v%M#Lu(xhyJXr9kZ0IpiXephA`C zg7X;cD$ac+{EdyNmCA4(jFl59(vgW9X_V}GZJG&pK2vDaiQSOGWAE_NA8`NAgl2%15JsL(QM1M@|DLTQTy#j?HQ>%4{e1gXd{9`ovU`(1PF& zjKETyGedw`sKLWqd^Okvt7wZGlm9MfG%PN}cUo8#TKvB+#ErS!reY<=|J9S~|1QB- z3R}Ma-#hTw{~veYHWMosv^AYw$s zEtNeex9=@&Xg^ZPHxHCd;#-8c9O7Gs>p$|vLFIIsSQF+KXf-f;L7?@Jq`pS&hoyc@ z6UF4w=@PeWzn~7Iw=6I=PAoWNY?FNa<7FGS^X3`Ml*+ytw$tlYX4uVY%FWyed|lUdRFZ;PxKt|>=DF0)WEpU)*Of+dYcw{r zaBE&{ueqyfgbC$nU47}4qpk5Sl}G2sRLlRc>DHNgjY@jQxHSZvkLf?apFC#pM6dUl z;fn?)JEK>&)^^7IyK1>MGRL0Spz$U&4Y%q2!?FL8>kZqEn@*m^b8nt*xXNcX^YReJ zJbR19VT;K_Lw@t6!5IFnD^pkbE!KW6@WTbxzQe#GNEEQt5J3N(5&2()y9S8l6NL#x zM^Z%OT`tv%NIyYKR>6CJ|3YsiDwtSUqyAnOK$i+|6-+7Op8tM`(^|}8{(DPQK|WbJ z>K=_ES2rg}B}^#3q7IyJDWIvIDJ-BHHQjTv(dlG$wl#Q0bhhIiiWRmut*ibY#=h>@ zHfurGLPrXrsL;tqzuU#x-kiba$SmSed_| z9oa*z$m1`SUngq4=S)zmWMEA!IX7_60rnk1HM#d$g6qomN$qWHY?TUexb_dJ- z7-gfbdog`J?4GfG)J?K+uRY>r_YEc1${N3o8mNpPIV^rUVLadMbmHV0ie1v@ix0e# z``S59JAAt%Ugfa0^S|W(&k2s=gTKEIRHZ1da`+r#=A-%?W|zzI(csb;eEQ!40Cj`0 zkbQfau!xiMnRIc#(lZ&-#~%=kW%mu9$?V%t%}ur6#^Gm2$j*A5Wq*QdcU0wcmV$9Z z9jXlUH4iEnE8h62a7_QPiDGVF-@oMn#YEOwsz{#sub;L4|JDek+|TAy6w1yP*hoF} zKWV>}t4`RFZdxa7zs;krsBzcPfSn$DZ`ASc+IOR}!0Y~;a;ZmBaX_hG(*23j^XZMa zvVihLm2#o_#(MscmLV1X@Y`I062dn+)mPKWQ+Lvay3RJfCKJJl%7 z3d$Cjof23{@JoRl8@q!Bt#7)#)a0mO-fkAsMxevQhpe+tg{WY!-+5KbgbGib$Za>6 zX=@-5x!{MFr3ogDy$Zk7gV10xJsuwSuuQ0x%c9OdWD`{J4pk5>TT&acz&bcfu@7ril|EQnXe4Yv%r#9zh zAEZHzu0)#_6A9v9Jc(93LxallNEhM#RJdU}*Bw1T1BU#teGc z0y`CEwO;$gou>oG{(}+(A{`W%PG0ubr2~(zadGntD(KAUijA_-K<#O_`Vj$Z__iUo z_8<=({MurpuzqWhsx%+n7(;_D*&8YQNObtGsDpQ%e8lQWgR;T>yQJSz;JS+`bv%{= zR#|b2TvRGp_6lj%vJk;6Syt8TDhXZVsSF-Y16Ex^MLLGgL+PN3u(yn`%20e^$1!5T$CcK|$4tP+Z z+ayJpyOj#cDlDg-pF_{Lkx9fm(I8IO(k;}04AwpK^BNjRjt<<7`ODGm;r!I{Wg^n=lA&V>pOD;xLXLL|(||;yo(+>g_sLq>Y%rvO z=esX7Hd#8zNMxsn*wEpTkwvZKWh$I2zs4FHNP`!0atR^TG#D?pRcR;FVa0!M8?y%u zJdTN;a76OkfqAQL+ewA1MJ1~arbs`fhKh=j9fyb?WxV63KnCkEod_K&D2*Pw@Rmx0 zCndM)n;Oym2akV#Awq{H7X0jIr0?er@!yE$rNV)DO`F&q6p%8Is}q{0!pN`Y;f8J+ zClG%rcfZ$Rov$5VG3NeUhOI-QsHItZcZjNKaYP__?*2- zgVW4xX#z<8!-4hP8Ur-2=40Ym=`_QXqw|RO=+a4MYeHCz>Na^6p_qioy*Vkl#*DZrN)KuQM(B9vmP;{<__8 z>sva=7mJ^78lXe-;+Im^I6AoZyyo>r`jnFtFD0Kuhj$&P&TWpRLro1y@S+sDKfkW~ zg&rLS@5q#HLgR3IV)bEWmIie$2^rpB(fA@MvsLAEI4x4qd=<5uS7lIpFUbauKGxi? zfb=3vvx&Pa0KJO!?GkRJLG{RTgjX;f%I@(Wk@BHI^Oxwll~@|EdYs>*kNjHk%9>-g z2@y_mLWLUK*je5>s@3T+<3FT`PKBPZpkK;$&(Z)PwyJxz&ba=39_o%Tjvg6-Z&Tg7Rew4QVq>U39v?N<) zPT!@$k^9V|L5Tekn>F}8AQG00w9WL+xUTqXZhl|>y zaWw}i@Nm$7$}kn_ckG7aGg8Pe2Ado3ex!qnGh1x|@~g)DH|ei@X%Iv%@5NYX@ccrN zf5LMr1l(h*c053YdzT7~l)S0nH@uZSLyHPIA2T*{BvV1;*f+g`04i9l1wPB$K!bN9 zMuWQ7DbVXG8fUqI44PkN(%Vs-*jS~m(KA5>W($V~+iO&Cw4?QpzNdqA>sZfz939x? zO|y>iks)vIfsNvgL>M;PYnv5B1&~Qu3(+8fZqMF=U=-I32$r_8yKR7RU09h=M*)i` z_T|3WRQSI9sesLh4BcN`8QPaf@RWV)g(#%AnZ`vV-&6|xc$o8@y^037vm2wLksd@Z zRX?-&(R?y zEj(Kxga)}M16f={=pa${R-zcmYx>kKq$fwd-pY-q0#S=Y{@Vlzo^KKHx%!+0_X#~6s|CoP+W$x=Cy{{XdHTk>b7U~r zVf)jH;*VN~&gut*FPsja4Eu1N4jVrcq~0UHT9x!C=|dihlSgxA{>W0G>BRiWy=Zx!`-4aR^NFln6z34HKtKPT<@L8XB4lk zK1ECh&XZv9{_*`HT@-K*JX?65iwvGpQs+B)ZJ;LIn{;Nc6}U?pbbS#=@^{m&I<<=e zX1vR9r;L#wx#FJKj`Z^Mw{!PxP@H|`+Hvzaig&6@0m0|)Q^5D-&sfb`61< z8od18Gj=Qq`G=;A;3E$x!0V`yu?@umIma^{H`2+_ZltKDsAmHu8wiz=79_Bb{H;qs ze$0r+BUK;yF>i}tyHaN={06%$kx@FlsEOD3&`yC5UgJbIq_5@HPCUwfBw)Mmik1E( zLS1BO7Y#j^A}K~)N}>Rl3bmE>CIzU*UzhLIQedW8NBA@Hm)zaF*5wxw&Pe@@~*C;&e``JA;G(#-Y0q^k+!B16xgEushezyh=LJ(iI$&*&-%UEy8-DUHMSjChSHF0g zMgokNVAYmJ1f8glhac`EgFyU}hCw^BKfLn3FnxsAtXd9n-KPP?pU(uBPl214SeF~_ z(18DI<-SVf=iRd0`L>FnxV&rBQ^%4DpWgU-s_dqM+95yxM+k2Y&%9PxL-WahzbtZP zhc%oRbakHTqe78=isO4t=AuT-QEL9^6Kzl%VGZ*~cn3iXH}TkN~( zff2f{?eDaSa5uG@S|Mc(pTY@4VK0eLYZ>5nUB(*BzFptG zmyZP38SY0oCW&yoYQNqHI}xtk>5ZE|Xa#RVJ{e0twE8z*2a#a?aP-^T$iEdk+kVrk zwuZHMLyv-MR&YZ1OlnuRHJlT2vF`awgbJQ8dRLP*tZ=+7b6zFFXr}4%hA|>!b$boJ zN+UtdtN91G@=Ot%J7DW&$@gcWQPS1(MbSp$F7i?@cT|4aSY zb}m*Dkec|7r5_QY5DF8Q?i0a0bg*us9|Q zj|AJ&uGYCABJ57qp6VR4f~6k&Xf0%SFFxy(kD&OR<94kuGl~S>9(-jbudLvue*a9( zpcRBQbZW235dZa;Xnt1DNvq9qkYLAVy~hVMi2#))x_ui+@G_wLi4Ky3wxnRc0Fp~> zPr?Dq9BbIr&;Q&0lr`+RbXR5p$?r>H&74uTHC&!M`f~TWHOwT|{vQ5m4G#MZYvq@% zA$ODYO_vla7`DSDX|7m9c1L%M!yYSmG5x6 z)Pf}r$>5hzfB^|B?&0Sot zCxeu<=*a?v>yekYsvjXkLeuX0^?3^Tln)xbKz<}Gr8Q}RhYER0%NkQCFTG?HQM@UM z3`I}3vzRteAnRe3*T^1}@9h{9%5SHF-h0%u~WW9GaKq+l#8oUSw&pea-bqWote*gN~yV-#2Z@G8?- zY{241Tj=erwjjA%>_ij7t;yCIznE`O!8Y}47 zg$k_;Ykdjm^?UfIrQJhSE>X`2$Nd+-gt!sf)UclYX3VVmer-RoX7XaTmKvwdVpiw%62O}BwCiYgAzNr%bb_9i~a z@7e7QVrg!mg3GB{&F*?C#47egMx@c;kJaa%%OzCUeuu^D4xI*tUK3Ky6BO8(*X6U0 z`kTt(`>6Q~!V#f2rqg{;K2$R?;i^Z2;2=A5nF?h8t@LISgpZE-d42V3M)65Rt@AO$ zd74oP6>G6n2!6wVcjlT6K;MJh&_0y6Oe}wm_&@{u`GcEGj*ubHEhO3s@f}lc>I)fEPEVt~s6>Ls{GqNf##d;}lQdzyKPn8VLwuLbg2&c+(zR@`t zMg^64!}Y-)G8}P083EFNN0k&y89{{aPjBk@W=Dlrej}_!2zOMkSTcW^LEmGvO~1e> zpjW?mZt5cuES#pFj-fnuM|;!VJx@rW75ME~E1I|Gxzk)WdfJx7Q2Igl4r>$zju$Q9OL-8k+@7%IRTv%mv|i4hpiC?fqnan_NyX!CuHxr zJZTLZJ!<7@5dS9Q+w@p+mI{vwOpD`hA{=C1^K%l}_l<)*tMl6^@O_8#UU*9cn$5%v zr#rF}?~XqQOX$$Hn-t`Y{M7n|!A2GlDzwX`e{n?lh`EdZ?PvLj2P_qx&_0Rq>PG6T z6om8qw70Th4m3E=c537`%EL;s4Y2duh;SzG>gSj?3LH>AA$0RT@;{Fg=$&*lZ^pby zkCFd8eR(~$0okR+UxI>(3=+w@w}Y5$;l}x|Hx-C+h0GqgRUPu+?}CJ2h)@3=hgP!=CZ> zAU@*Zw})3%epBFsXF|kwGg~;Z?YQR}!hw4i&VFx3_){*@Q(y7B4WwyHTW!98a05Sm zK_L*~;P@xXX`y} z4Urw4%gwua5#ifGRprL-DBmA!B7V-qP&^pDQq?U+2NqrxkFhx#oZ`Rzi3jPm<5kDW zQ?1C}9`9I@uB3tJku6Ufe%e6S<(h3i$X~O!e*4iqMS%z3r@ovPM0wd^=t%3+;MH8b>BcT9Y?9qoRE6wvJ4$zrPf)--ik>}m7x`bU=FprRl%HO9 zpgWTr#2cKyJ=}k1710sdrVhp@bzGa3Jvj!h6gaa6W%l!Gby@4 zMtn4@1)H?NXA-#7n?KgpMt-5S_9PRsKLPo{@7}0CYHy_nq{c{)P#`H1{gn)Bmu^Q^ z{;&cC*n(q0dGcVuPlrv2w{&RHtKvZMI`~4t!>tA=KgrcRN|C37cMwThZ8eXP{}FlT!Im6B1@0f-ll3S+B1s`cl0bny3-6s`PE(-Gy34Q68u292?+fQ~ zRG0}p*dl@MV-K8L&S|B?sihc~lpbr4nB65Yy@d`jXOB<&UqXEF<%?D8lSu!x%Aeau zAs+m1L7@uqI@UzIT{0OgHM)=AccVa8yYs%ej21e{JCPz%M2RCSSkoR}9l7GPt+t-SIA` zg8PfJ4achy-(+0J{t@M6PfhP$wFktTwrL+4LH1_0KWlaRZ#*x9)*c}md?^0_Gq0%- zKYo(6B%1=~rDAf^6Rkih^Oxr!!YkwM(UC-Che-!Hmd4Ti_BkPDw*m1GtrGLMkzd;@ z78FlCU=0oU@;*&@6!#bt+b@?s*&`5`N4!8vRw;jx)Vyy;6n z^@7XdD4vw${Io~<&=b}>ojpVa?e41i6`(=);`0_0q}S_l?SbdaX>e)3nU2E`Dl~I? zM6qd7;7*nT?fQL`SLpQ@Oe-V%=rX`-gejatqTeGyx6Q(V^G~` zp<=7;CmQHTS1vds{mAWdc-3Ny@}sK>SM*SR@m{iRObgA|#j&aM;vy1!k-V$_&g<>QIb28hqOs%a{*NnUSbpB+&xXeH zkNx1ukhyqBj{heOp1w>_$U^-4%L(y_xJ@V?;;tT(i$?s}%iLea=5!$HC$c;ZLv#LaSX_TkDk(t~$%7yBZBdYG&N>r$5`@N?L@p%zt2~D?M=`i-m#efO%X4(Y##n4<5 z;Ed=c>&g^Ji+WPSvx;!@LrX40#6OL4tM&DI(!j*ir}{cC3BEFm40SdjyzOBrTaEnl zrURK@pNSz}od(}IrO2>j(Vp!c;`g|+<#*4DQlX#oDx(JBUxw87b2k>qz^b&gk``?P z=k6iGrXKM}liW;I2zMGsd^&@+=EGkYnU4I+@<-n@{RsEt*iB;Aw~`_5_YX!G!lD20 zNDqmi(d4qt5zVuBav$ptgm2uheFJ9$8#t)Xw9p1TG0PYV=qsrTx1lv>EDzIRIZ>||39$4dm zfq*O(8VYuKtRa8Kp!HS<4UpmcxE!uv2Nm8Yj+ySXAOr5iqtYnE8_VPumc`#iI4myv z*CT|x{;5aV+rY5OrilOv#1pbT-t_|2F;_kaupO@^LFtSO_mdVnq`(#LJEPXXM6oU3 zc9;r+Tt(ryVH-$xy&Gj$6HD zpXY+=jQ{>@_5ZHBo3e!6O7}2Cb@$6Vwr(2AQ^fvH-CgWhU2S~bI>WPzH&->ahhesn z`FqFuk_G9_{i=qq{P1YgAw`vEeT?&t>t|1tZ@@YOA7wQc+@<}0q?Z{Y1@@sH>E&F zNQsHIPZI_k8=5&~v|;SqGi9D(0ywl$9!Fi$f~P#qdG_zM;Um$D3+&%LTwL*7j+zG7c0F-S$;j8BR1@ zD1K2S0hd!yWoDHR^L)biE^tL2-unud-B`p!xUGZxSdRw0>U#P4;}JzjGW~jc_Xi~q z;*dHZ)-4I+VRZw0u4=&M-jVw9Lj>^E&pX3;kmIkl?0f!F8Y%=vbgD58aAd-#vZ3}g z349MlUP*#_t~ovq&x+CgQO0*h1ekwl?LPfg0hs>&JM@b|_~=WP)ey(HH~P=tl9Gmj zVw*+lR4uR$!;U>SP=wPgk?xFr+JI@AM4xok1F;RaUyqzrg+1htK5seZEus`LqPVvV zsDi_@4?0D=39>Y--x zP(zt?dwx_1({gq97H$y0TFmclE5&hO)#K|2k3H1{gY#F^S2rtwUfByT1z{QJ9eVUZ zZD@rNx`C;=$Xf;a?MG-)bs3q>Kn%!SMyG0$AZb|cSJ1Bz04y$pUw|L0T=kJ`%<$|enIXFHLEn8Nm2qX9JFv&{FfvtX1X5XkF7SHaMtz-I{Vd=Z{ z)k|0hiYte>vW3*)%@)426=a_p`9$4X_a_XBjN3z;&qc;^VTEvV&}EAV%*yhJ@L9%O zha%@{#v?{vN^_;D@mJFTb%VY<49kAV9|v=Z*WnXw;)Qg$yD}BL+&hoUp+4YveY-` z+!9=1Yd!Lmx3I|Q z*n3)W-gAOM_~EQ=*gDKOdXdyD=D)zmT?%3wB`q@Uj75GzKTwHP(%BDb$T4Gg?#_OA zD#C`vDai2Ef5lwpejNAWoigBmTp-_KG~}VHUZEyBg2f% zEJ#e0O&Y*dzrdaMKQw@{LnDGB!G)O?lA{bg6oLQm!igX}xJ{`wtrWku*gD&~Cqs=D z!zMEY?{;xuc67_OlGtAit}w(%Y-YmZ@~u36YUu-H7ipz3O9QIHuf#J{z@=(w^TMs0 zj7pm**P(b$%$+L=>QB65Y{(dyUt4+0Fy*_oW7c+x(W9<-^v1{{YHX@)=cn zmZAWApT61k#X|vp6%W_emZ^e8$_Z(s$Eyqh=6Pk8NMYzOXxM0yFZM5wpezNYZ#&WTy9;MFuHZ)NCunWtHfURQqpEo|Rm{H0@p#ueZ!Dx(-$GDG*NFQ19-%-~)&B45YJ zikhZ85&T4_IKxvhd1%U=|4fV z{PSw?MP)K6pV4bUF7kc1L1E28wWHT6JXrtIjqj0RaqEd*2V`ez4e6!`b-uC>}oRzxB@i zEd%z`Y((vS1o%;-WiWSK6LIsj;y*Z`6RS^Oxi~Gf=#4(Ac_tI1IDlieyFyZH{ z23xythRw-q1HmaF&Jd660MRMBV>qlXbM^R^FZ~w%!L+X_b<>QUJ6c|F2I>JDPt*?y zgxeIcY0q2otBlQkb;h@BbU~~=WB0j6Zfwgp0fG0=gfWx0OXlwJcr5>Aag6#UPE0(f zDATNy7mLX6ro2wj1n!cm0M(1X88U*CTW+^YVbMR&3UkR3unF1kal3m3AnU-n@ZNhL z83YMOrN*rquy|}R)pSV$d$DcPg|Y-e?Cf~A?BkPCnEu~|S=t1wRkvDvRzeL9T4^~p zamm6(2UC@t93dFY^OD;i{g82_{Qjvkp1&9~)@e6VTV(%Tr;0w;rN`CBok0FFfc=B3 zojMflL6s3VJjA>n&5hy~#=<`Y=TeK@X?C#F~ ztNq7ivHX6CSSzY9w*B}UJ~tmR%ti0Tw`wn8Ov&wcdeChlEN1a%r`@Ckw!DO0;X&?T{O zsg26lrbIDzSC7Y2tv@ZE$=2-iN9{>YMpzM2mn|A%s>e~16{q8pSO`PlS&5Jgrb4{D^t+va>2$jWjJ%h`NT;$CuUSc8gz+Dd;Sc4o z_YK~L=m$G6)*f2j@gox0E)C@lS%DQs`QU))@f;ECxk7}@U5vJz5;o zt#b4>BZ*^aygGqQlIXqf!S*G80nBE&E-_Sq16$a7{L{lE95&-BHK}a;iIG!gn4dL* z!+6C?z0tuWj9;g}-uZJ_0K2wrAjPVMA4^ZCo%6rE%y=VMTAI+a$|w-2$qiHd%^3Na zk~A13h=rzlg@3xg0efsy8_v6j7d!j<4QcEJ3zk26_lfL)AU2}6FR-9f0Bbtip*&1p zWz_4@H6^WuFuwq!`NRo!EdQGHfv4i!nE0b0ox`s=uxNeu{-{C$tn%s6Ov?^IjEtk4 zbMEEGBF5PsNau|*6t9;voKsn`w&(t5)3`Tay=?3|o_O(Mo9^2uaBbzn6xhzcd8EmW zU0U8(ui3?g1%LROoY=Yn3#$yzD|F_??l$gy?;GC3uk>#%*04 z*I=H0MrwH9Wl1(JOoxzkaYXeu1MhS3jqxZO_BDK^mPLDo@jKGkE&9z*Mz^+J+!>Z- zMqN(B$7TIBhUQ=3rz(lnm8wnZ6>7m+(^R)bjRKf*#6G#op#^0_R$sP|)M0{E=tlZU zHmo*`EIzUOrv=x8Orv!5Wec@=rRnlUIiN(;%uk#;{^BvL#n%&LPJ+GAEu8FXY#RV3qZMnrhC?x~uPphtky6ZuN*})q=9Wq!+ z?(-FOc3J4SIDSdQgagd9kFH&|l>sJUW)>%uf12H?HQ1~8jUnoE&}qg{5c*6eq@^!# zLs8kQUpwzgLqTPj662OM@bx`^{kuaGriFiv&51vRQ^B-Pl+xB^jADvyoCeWZ6WH>09B|cG#M2M(tyL;+4vLk zwUH38!zmau_-nXtRk8ccI2nQ}69`u>j$SkSV{8b0W-1k`Y9c^hvnW-zd1NFUHnRdykHZrJAJgR?aifamX~IMc&^T3nNLjt>z3%&3TM zeS4H74Vl8`MG5B&;PKsofsR#0FrxgvS6@tkaH8er3^Y!Tqw7*TZVQ2d=Hs3IN2MX} zr0l-UPc*;~-1o06$U)NlulMPfr6AfIqaEF?24#wIwyGx$fK%bq%x)P4kTV}rBah;- zjC`-YS$;z(8Q3Y3c~K3x{sKsCRp7Z^pZGvs1|~w%t7bP#!O5~a&sppzEI!@-wC7QP z0;Dx_$uqfe!ywa65v^+*un8}=%Ig|>(AFP)cO+6D&i;C?OW&dbhls*ze&6+h#cR)? z8q$;OX{)An&J_#dMR`Xz4n4Ru&wAJXlpa|7*Enr#{mr97s+ezHSA_FN zQZI&GA;9kMr|#0zwczs&GlIrpJ!l?VxiX9Nvg5d_$psl9kaeBk&U{c5#-!1Xgnc>? z?-kp{l!k|Ibtupl>Vj;7KrwHKG`t=Ep#9~EJ_Jr4EPXDb2@g3NB3Xl(u%#`J0=v4j zApNwOzt!Zbg}Pex$KYLZ;Gvq}LKf8r5k`z@#Ki@M*Y1*r;f^K7klu=Ggtsc76GKfy zb@YK(`%F~HYdxr!jy@1wy8EukOZ;YtE}cw%@>~}(8UjCs%PcdVwd$PsO)!A&hiG8} zLlu%ne!el}5yY-|ZIAX&5QXe_vb#-wa$tKp_MhN3QiT6fscvKS7e4Htvv@fLPvmGepYV zcaIs;f-VNdRIp%w;rsV^QII+;d)7AZgT>npn0-(%Vv!Mk%HEBr566z(>o^ys3AbGL zPpRtj11+z}{>@_^m_M{({L#@3VCwy~^XFbAh!)w?wzn6QNmF1`-`|o~-b$&R^ zg?11Oi2qx!`J)e^d?D#XHW7%Gy1H?ISq2nls~^(NNr35Dj>Uv(E^G)xM6IAa%(uP@ zxf-Scc$bAea-S4I_?N@X-HVFw7tRL<=)*hjF{5d9WDjbO=_WV=&{zJb8VeZ0TM1Nw zsn!QE(WuRj4hjQzI;Y7IjTHuaSH#Nt>JsAyfuP*urwlLH&!5;LDS(-TAN$$JCJsD4 zk1<>~9@<^6(>@<%0W%Hifm(ij_@!S#;9=2%cg6$q;?lxEm(@v2b`ypN9tytdsJ*=% z*1FVh5@5_Ve1bfx12^r21w8_VG1un(wmX(JzzlMjn2+kfcZy@+EXqUIwTcy1-Wfm| zyW_>!>$33HsC6KTY7wObEZUUX*KeAa^z>t01SiXoWztPUBO zX~KcO4q#FZ&R?*#GCWLx*=&ssDJEIO+-Y$Q!J!S+n4FZGlHcqG`ZzyH3Uv3^NXLjduznd}m_ zEhfrv-3D_R3{wWX`;)AS6N3!D2gZwRi09lJvs}2$i^f^|s9PdZ6q1Vji_$#>u-qi= zTvmz-xN*M|bxmRg#|_m-wk|2c!8YCP*^YXkI~8+w+G+zbh0lHpFA8tvt}5T45|#oYx;f{B4p$_HQF;aghFz>{t1Q z8#@)dQ$hTr0Ceu<=IX4yY$4oTen?b^fMq<}`LZ8Vh3u5$q@%}F!CEI(;7t<&4#-3g zDxB8>_0Vl3GZ_sC4Z5K)X*ggJemZYJ?T{us^|X1&u_OpzzqQJ&}O`l%ULpEs{@27`!joOAN2-!yhf%qTvIQ z@OQbGhoB5xTo4(~eq;bmIfpLic5-2>8}B7N`^^l(lDz&}T)!C`4($0e|9}AJ11m;- z!vrw2TkMaBnjmDnmvNRUl*DpnPe-`+ihJqOo!(Te{Fb^KB=Y1`}qbLriJZx~m z$ppBq?S7ueRtvveEVchqJ4U?V7>|8!2K~Sp8&bRjvS}>OUx1N)d#l5VrYzM;T1Ne6GHgM+x4% zGoLWuX#nPBDT{U*1kA$j)tQ7ADePp|)A85#JTS)VVRH7J0z^*afAnIjV{FyixI0r` z4a%X9 zliOI}@AAf!EP2@fcR`us0OPj0hR!(UAx;x0w!o!o!0wD7{(&-j~r!{;rWB> zXZ-Lg@OHU@;pJ@rJ9I}iMe_vU@ABz65oTbykf+7ggNG)w%jPEg(&pSl)r01v&s zttapk;4~lOhw4|vqaMsn`L$IGLK3s@1nrz>^xi@VyQKj*$Zb00}T3 za5{CeLJ#WZNO+cwg222MJ3pwd0@*fIfom3mXr2RZ2$J-nw=GH!N;p9zqpkDgGgPk# z8#Yl#-`mx2riHz5ox$C?uC#Sr4_-CL=-@L|!ENi4g%7g^kU!VSur$;Lem99wqYe%z zEIeOZ_FV}+A9~|;r%3>gPk1gJ(pH0!dnYPvo7lj8%Mvwhw+;yT^?nOP{ONezseAZl z73e-PHh8ZV>FvSH8c5;568JFRwJ|~XyDU0!Sq?tQjC!c%=)#mz-RGcnMc7_-;o{&Q z1&BXN3S8_^fraYNXC9Pk{JS1$hc-kNr9PTMek&Tgq~c(Xc-qXU0!t4L?2bu$?9)d~ zuwmd@MNzRD>}qPbV?|s-WKm-POz}qOG?HHybDGI* za&%_skgif{|A7B@ow6)DgfR#!-&FJk68jO19DKd@fGJ+Jyy86v^0>> zLjYD(X51Q~0xwq7kF)Glhn8K39^gngO!pV}t7GZBSmT17h23T?NUeWd(VM?$@j+DP z#-jyxZ0F_US~Uq- zofPO%A%@~Z=w-If2t7D%944?_bJJoiT~0sg`7Z{lyML}>7JWH~`~P>m3+xMs4hi** zj=%@(3kZ*mh>^f|9SMsK4Tz5KIvcndudj#K#@mEO#KZVV29vdAHAdIeC2kZ&QyF@4Y1^D}GQ{sa| z6T|+t?-77^2=&Li#7E$*q66XsqGRy1@bJF}&AR!9g$2amxJ9f(BjN&s&>yzJF%j_* zxN~||!2zMZ@xFLsL}<7#E(;wPA1V=`P4^8AMF+xZS%&%rM8~2&2Ja2WnTgo@21bPX zGPc2?p~2yS zY&bFA-M+B_I1w2;G^@VhzB0JMprB~qnAq^HGiXL#@sSa+-Iwp>hQw*%{o-P9jiJPV zxY(H8XtuVY$(BHkk^BQfeDRhcvBCQY9x@@ZcuH4mbRgSxJ&JF1jJB0;PXuN*_-a6DbI2vyi5gfx!D2d!16cHX59vBcVfS%`o<}NbWH_|sc1n(9Q z6N|ef;v5tZ8p?``ZitDA;6t}KMWD%xjtL2lMjZ*~LRaj%TKz-)e0K-^eX}htJR|^@ zM3A&~iinL32=|MN4#fM0;q3!xX+SvY3sU_M(k2P||0C?pnRHcMG$$M4xU8kH z_~+5@HV2T<_Vd!QC>l<&|p0E-Z33E$8N?7o9`c>ilu~ zMzPn>@e)loYP+0+hN7CLwBAeqC!W4S?HwK%85VoOVxKfD^@PVpqGI>R@SeTl;b=H2 zM*aA1WYk}HLAP%x!g_n$;qkGN;gKy z@vJqORu5Uqh3tMkEuW!G@2SNb`v(m@bNcA&9r+WhPx|Kb%UL;2hiVlO5vxy8Vb4+% z{dsYpoSxK;j2g!oRNZoKAfp>{PSrKB9|2%d_l(2#=frW#`>Hu? z)(t79C|Q$+jNg2kjmDDl8Ai4i_wO7{%Gl(Xs)$`OzQl0|HGI`Pk6ONUr>3c9PPY&= zd#scqMwf4^$!!c(WAJ5Gzi)@W3!G#P&==jnoKcJa#pi}NqJ#ARB z=KW=J?-Zg`T$~eorw|)f`q*f9)caKXK~>ZASv5B=c8NG9&%4=!OVf}IRQc`2Z?#8d z*_br)kHa!Gam2J;c3{+sn+mo;ha>gE5cOGHx0YHSa2`|>ilxyvG1g>lUGtz~Bn%zy zwNJ%n(ogXFRU>CEUNs7#5*kd3RDc)f!BCQt_*gKet)^_&2GcZuZ8BOfVO!kXgX$2u#3@6fR$F+=- zUVTJLy9coiezA)_!r|jp?@><#fkzy(09ijFwKCbjE8Cn{KqNDIFQSR1DYG)-aEeDvRSo(F?UE_Sbwk9>L}8n< zbYacv-AKLDvbOr9cb9wUpxA_S+q~O6`;|n>P^@|B^L&?Vq$kyOM+s{xIi*jYclRNU z>amoKm#W=eRyu8Fbel?P$8K>)Eo28xTzr|)HCx^Cj|1Iu20IHJFsq5;+6X%<&&iLD z_Tp-$fHUGEmK?FI3OCd*#d)Lv)3jaS#?_^BN!@!&ewLnT9#WBR)mRE?8!^+%h+Szp zmdXM|)LzKTa4%z}n@Oj`Uy^n*@}%!Rxr!N*yUGzb=JE_>d=EeeIfld0z zPNY=)188Fou3;HD??pV~e_F0>rR4fd!R=uV0fGM2UUHYdg9 z%(E5V?-cw=Q+312EZ>1AV(nvrPiu=gY*SnvAXgPPi&8oc&?a^#E#M!z&K*Vu$B*n8 zy3{kM!7h*~Y=2t*Voh}1FYX?RhDZBd&)d1EIMz*WWP2i`W8r?J*@1mwG1}cfGBUK- z!2ytI1D84XVNd0RZJfuw87lo6i!d7gtGA!)()An}<-CS_1hB!#>Bo+^0c8~}qpp?l zc&(KQ-#{5F77Jia`>1~MY!RcVw%@Y-`aHR;*g1-AP3ZDf&ggtvo|J`MT>m(NXHGW| zEeF!Tf-xl4@QdR9(QzD%jRQ{U`|G-NO`Sse1O6c2vNK|fmvLe_R7H#Y%KL-%{Ye$; z(2ql+M$EUr%nqkF4in%AJq}4C*;wVgz{?OeGo0*Zdr4?r-?yO5EM;tZsR1Wyt5XTgu zuE>)T_i72*&}55=D$7IoLpU{lH?{l>mHyM{N`RBnIb9O!x-A7UE+WtAhH!=b0MMw4 zGApAht5=EJ#nYcg3XP|AJZZCEDWrn8l!!AY$8$cZ!o$(;GQG0!9lIM<0x*K)aS%QS^VL^gU=-6ES^70=j$Jg!>%RTni<2mg$Sg37}tJ)U{gdeaJbGOvCn; zQnCe*pulkJdg>P+W%_mR6I6Vq@bthp>40!9p>*gC7E_%6drBuJEVBPQe6;%53uXDTD&*#>kV5P|h+y0}k? z#ggjTKeOY9b@?tz39BfOu$gLVMtS1|H)^Rl;p;5dtLJp`3zoi5=;WP%?Rev*&CVDA zs~V$=ox^}xrc(Pl6P3N6m$%v8l!lrI{R94Ce~;f6%_1o*iHl9_L0zx+dP(+wsd&77VAX)4NuF_0tol`bUeE>Tor}L-C zWrUvK#{rGfZ!|wsOEHUkSzXV{U+4Shvid$UU>6ZKfzd>%kG_P7$lgBVPIJ$!w)#(=*O{Wht(Uf-sr^z!aSz{^7)mQ9^DhYWC z<&Lddc%+yEk~7yXm5OX_JgRnIM7qq4eMLEw@_vWuRmtT=-UT;ID=UlrB*W4-czOYN z6tPDky>==>;zjr^idIvhTb#`zP`T=LQbW&7^I?Fq^sxs$T>#-~{ysxU6K6zSU7nV0 zNoO~9=kztT<>9+gxKHVtiCr8R7?3|Z3PfT05EQlfBnmIzAa+{ z`+GN{ZlNkA7qW-m^FC7-m4UzHNyNYO*>`Kk@y9U*K}D5|D3Lr20<-{U&&gIzIWFUE;FHshK1XMC+KR}PyWhtqsTDyEA-Kt-4=FQjH(&E z_Di0OKB=Z-dd6GLcjQ({y}vRKraQ~jRWbJao|((Iw${;8YKrxcmsrnLKB1n zKu2=19ghY69aiw1QclgPI@#%@QnSTnYhC`}PBlKFdah05_Cpge1izG zK%ni6qOIJ>^}!?vDRue)?=_P|d8SuiJGlG}Jg4*%tduZTHnK)4NB@Z*$cSSqxxL5> zpola%MSf45h%ma6q+j#ytN~gC6fqDkSHDT#BThXdgSJ}Jd6ZvP#eLYT8M5%Yb2-cH$D1F^dK&{> zX*HqE`O=CHbuQSBXwKiSX0v*FF6Z@N6XoJM)zAwZQ?Fs=$klcyf_hrbkPFKgLtSpi zvSwg^ruPD8RL&r;EPjp6WaYIdoTw0WfUgV4$4lIk;7=5-q54#Dp_Izy3Ixp}x#I@! z4_((d*ShHc7y?|C5n)hA0d2wk#nHhi{SBr&P5cf%r-Mmb6TM&Z0N~vVhDRjpO8AJ9 zn@=oz$)ja_Hj7e5eg^U5Jl6`3y^!wZ$B@7EwGK*JeEY&0JlClkC{7V0(k&Ok^JF|h zkJ^n-#IDaEaIG%qPo&7ToUl~F4T_^0p-NbkO0|~($18^R*>3qS9tk-B$+AcOvC<|? zm++%1;Ed}_qoXO{&=L>HtI z;qfxEV?9|bWx~4feo=onP@JgAfC%6+y748JTyoSzqk}DNEe$wl-}{CK--vCE4INUu z6aU21iH^@w&1X*nEov2G#m?wpb4Np9@p9*I--S4lU!vW^eNi7yPPXk}LBTCPfG|t` zB}nT@u*ezLGvdgU(C8o^2&}9p!#6VhD%n=X0tAu=OX!!~L#hsMP+boQZ&bN)RmU@m zB|KavZ)aswckL~z>ynUs{&^r0jKT#T!Vu#NN7 z?URv^a`jS2Lo_L;6aWeUbe^PLNcM}*Q#tulEW(^n&(=wksGc=7muqUKWvY*Cb1?Xq zpV}wiRU;x!_39RkS$J(d6(;98Ib`xQeMR#i0vlqJ2-eA(H(ld)S!0HD`gSL*ALX}j zqrFf~f|1PVnzBj`7BTr{@~pt3mLZ`M+Yk%0e019R`mKG(vn1p~SxOfLqeySQ`u&li%}9T)~6Dx@8SDC;i#NOZyS%>7P*3$G%pm z|9X_9V#@R%8c?ldfFZ$c_Zg(jP6wUcR%X&r9HsOe{(e%Km&vJODlWVrU}0T{i+Q9r z%dj0~M$_~OXBtroE_Q$$Rq`k80@r7$)3!I5s7`)Y%!uiM!{Ay#`>s0eI~q{dp)5vx zSMzsE_d9V-LuuJ{f-OWCR4*WZ;>)9q8e84$!B2LBo{&kSodeC3r0-+WI=#rlAjjmh zU!uZm-zH;4-nGJ;jH)rg#?i&(XQgZ!vhgM-uK-5M0_9$LvcA&WU(;)$$W=fKoDdGV zkS1O&5&*=jQPt!h^meghN@Dp+4|w5mivwIQYFT=k5r-VzONZDY-6SsoC8N$pLL;Xfg||`b zuXhgv79+e%SzFe*#{wRZR~nt7G%2fxYu^V*#a49dU;-{F+#CXhont|j&?mzCYDP?e zlQ_YrMO*xKOeqV5-O7WnqJ;NH_`BuC!9!Ys;<~j~j~&(mkOb5{`JN z40)GO3FS95&#CjeDA-DDmn~0IlisTU5I*pF=YG=?rpQdIlAU<2z1xzcQ`c_aW14*u+}bd21Y?L6k9!>Zh^Wl zCePDX(Cze7(!uM_f{c-V!zkJCxzrf(a4T<^?#h=@ODe`v=BFQ{H?Q2lB2w>?sQ1+_x;dS_I@|R6iy4Hy|Bqy;^@yNqBL~*^I@b2(m{kD(!zpHP%QEv zgx6r#Q!5{Hxt!z=MLo#5+H9SDCfLUpU@lQIzrl{>b_iAG;iHR{(9XDQie zUv)GC*n&}oC8zVpWzFWrPtU$u(>Dk7W&&NFOI`=agS~?Z{bhl&jw^u6CTZK;4iY)8 zg4@gAhXgE+l7}~NaZ8LzHH#9pSLhZOrNZVIM6Ra0YNBXW#Ly}*yySH;plVZc612^@ zs!SyWwcL${W*QjdY!Mz1gav-#q5xESbl*T+p=yGX)JCZN8PUh z86kd((WFK8I$3m}loy;`*frTAQ$@C$+$iQzaw+5!4t_5>!bZ4H*i?Z!Et|uRh+AIT zz#g2#V8!9)v0QQjss~gt^-?p5ZMx*?<22ymN&AadYqIMFJ2=!OTd?+P$qYj zGclF^@RO0#N6j3%08gL`uzCj?9;ZL8r1Q%uogA$6e%C#i5nGyC80lPurd4GgaLbQyQAItx__hbs4%)P)B7E6F?R9;br+wr!Cp^8$do>%0iNft=Qj8ju zi&$-XGf~$g+L9NEP+ZcVK~FI^Cn6c6c?2wr)(5#oq|q?bm8p~*ckM4G4{>WA@@vOR zj2Tmf8(eZ1S;spwS=?r~JS6tT@3w$wUEbncH7670bN&&G80N6No9=KN1eqGhMHRSSg(zubC zQoXkv&ZSgPqpOcA(_(Z!g?5f&%xm~&RvfZ&2uw@WOv<8lk2E?2CESgZl z2`BkiS-8I90bC!sdNb-t8UE+`R~Z$S-gRdt(SA^6p|b?3+azyn;y??kUe+U{5{A>N z-fLhlRmJ3iD$mSZ5)Gz!O8B}%{x11B&%!>1r@hOMHUHnL4A-yho@mNSXLFW8eo_k} z%E(Dw6$xHcR56U#qB(5QyVy}!@P`g2e}deOEp)x&AQx=p6nT3$;Yk5xZlU&*(icmfGYvgLYaXcH$W<)es&7BzvYmn=XRjQ19Cuku(U2Z zr$HP6qGNA*v4k?oh|^w+r}B_%EBHCHfKK1p2w0~`Pf(`q{H1frt9Hs%Z}KJhJ_Cse zz-h6sVes@FIb(iq^(r$ZFJHk%N22t8@&H$Rj53#RTDt%%%d)QjVhe35;wcgS2!taz zE1ZKW2!$LT5G6h8?kf=Yr>`OFO8H*IxWnWc2Z|I`sr^qUx-fCQf!yX~WO4b+s2=9Q zRjSfWf|p+zC$9BWl=PKx+N_dTJV}hruF)dCUu+kk^-wTYSa_C)ZVwY;p=PKSI?PysV*fbR_L|qI9 zg{R9wUZiJKmz@{#6?`|C993974+zjgy$%y64>_2G;`*JiT)_~+Q;L-NIP;0xQ*MJOF%5r;oLd<7LPpXq1Zsh8}kb zMRJD92^GaFWqbs=V+%>tbC#JuzBEUrE3V;L^jy8e%T0{9oBX_nG0mkvbGsGm@UrBn z^W=}m&{Pz&WZhQ2zu>{_IGBGJ%(n1L9&(UL3wR`^i11bK ze>+E^;4$VFgM6VsoT!*{IS-=34o{LlZb7M?kZ9NUy>3AMhhwR<@VeMOj`SHzqLDkF zFi_nIpQ!{Q)aJ=ITymxGY$elWkaZq6ifI+SBJ#z}9EiG@aKVO^?H;hyllU-2_=lb8 zOOsn1qKWofT30fP@D%G@eN3KA$+j=JjylnMSsb9cR-ACK`-|w)R2Ny=AlQoG)ak8G z@c-y7kcTSWt9R$}%aiG|M|U=Wd&>e%LOqZ}%!<<@)h)LRXKKlxOY5%zU8NIwE2Yza zlkaj$C{L}K<-~H1MH|$$zlc8E17AE~Wxa4t$(lki0S|VI`|Nr~lx^!NjnIxl5xu;- z0Yt!qlj1>rN>xP{y^$UR9r$q{hq}R#kI{#O^R{r$ZFn`N_DZ#BiCXGo)L8&8;6_>E#Xg9fJePQ*?wLjS( zpfv9gq%!E08KBLDFMHf0hMEDJwe-bf*YTz%&na>Vhu(XPe81k4($u6r19tMOlu>&T z(qX)qF%0sx4OE(btEOAc8Tp5i!K9Uk$n`uKC(oS za50tKT13qAoVoll)G@^GW6YewyMD^Ez4?a)S1~9m`qw1jb~a)8^4BRmpFa*bwn9_# z*)#RevJkY9r$i-+mZL17-c(&rxyapa4z0r2u>{UZjCP7reB~4aTHvj!uawups_|?_;TNvzKifILg(NJJ8eDmS&o4v1?#u7otRI*82l0kou(RJZx9J+g8 zh@eA3;{?1Mxy2>;H>~_iROO5*Mh1olRxHon-u?+ZwX2Q@Rc9)~9L(^v(R-aVTelzFqM$aODLR-v3vEKsiYQTq3%Ldf zonvhdp;Xdx9^iT-)08LZiQyXWY-2eGpa6Esq^e$S|Lpi z7UmRAgP&E_vk4^ws)I_0Tgkb0y0k!eVG&jsR?bZyQ@9@8G|48;4fzDLi5cMou`UWG zKP!TA)aGQh4$8t*fhF3PmK{5m_L1w0xk**AmP6?Gf~3n~5c`>*R9sv|j#GBnhz!F$4x`gk8&zW>NNLEcv7z&A3qmvCTK-R3n7kTkRsS97jaI>GSm5cNp-1vQh`*(h@brtI|NJITq=K)%+-PAc3rj}b@bK^LSz#q z3SZhR{J47gB7Ou_MM~PtjF`)IYHs=Ins6L##yC1^26=^L44*4oBhP>|ud*B^ok-*8{wm4JJF=a)vX*=Joqa{cF zj9trypA~*q=DH>5(Ey=eB<8b!`Ro1j6=sHg9U&(Dd?066W-SFwE~rqC&UBryaHuS8isK5Cb!_ituAQ6}|2? zH`2QWbMQVSe-C2S$+YhBRj~k)WM*^NV9@WJ3Q#$6ZV8vPEQ9`$@G;_nA0+hxvv9JW z9m(a$))L5@QA3SPnmIvnc>0t~{&B=HVk;saKaU@slGF2xpJlpb@|y})7b9fU-XPY= zS;(M4zv}h!K`t_vo)hvl{1BRm%E?4SKTLZLfzVW}jCYfJ2<$b!6)FXJ0qNYsQz`Nq z*ypa=MLvc7PtX^3o>f_#8yPv%GI!s8otpY z@l@ox^Sw3Ts+5H53uqhi0ubah|9)&E^Ek#Ra0Esz#=%3Yl^@=K$|JYZC@lO zHwb*u%Jt-*9D^G2Nlt=7!1XpOgb*Td>zcO;$odu!a5)+kLWyJPQ`G$0dIrY50IEx# z;WV(=)$n~r!9~$B8m=CGekUAeYBW7Qb|7a@q=*FN%^z~vNsENq)Gz0dOKwi z2hWIgg49gDT}OUaCDgjOg6wnk68g{WP`kx|*L|bOIrJ?};WKrM=M)-LITd=-!_v24 zq=|&;`>HcNVHuOMYYhztG>@m`9}0V#y{$y|*Z#2LM|R=HVsJi{k0?I9_C=De0>mMI-r!{lEwh9o{6~<7>!uTHS95VF>0F=Mz=0!3(FvqD z!PU&czf*9LhSkr{LRARVgtE((u3-ih65dEz6&p8`7mC0TKT-@st$#~UOUMhQ2VYImSl!yhyE`x=$ze&lp&vL6zs58seZs=osP9KfiKSx{v zPX@9v==3J(`yhcYh&$5>;%_Ay3&oQWU%Mq@uwmJfUZBQZ53eV$l!+r*@NHT2a!9>{ z$>_(nfufa#*Bofa%QH$EB3s4vhdT0to$B?r2`4K#$Odw*6LuD5HR&cSdR>dR)u84E zBM5U>g)6pfd8y1T&RZEE0b+N$9_e7ZRPw#%m9t#(Z zb;Y(_BPK+ysCq`#)hkR$W>duJr<~+Fn|Wm(>NE1(dS))>DiU13Wl@fw5SaNnL8t+I z$XG(LKt2zBU}jGEG&aD@tz5=- z=}GeEViMZ`ciC8SmJ?_|P-cGm|d9x&ew)unWD$-2AjcKDm=ekp>oKjr9zI z&O4lvyUFj)qg3+nW<~`2%-6+b5rRQ6n_tG`Z(PXoH|=oh0JDlr>QU!U(c9dRK&Z%k zTx76?Y7iR1df7NuYzorTWVVeFdlrwATUq&9s!{kF$HS8Jx%w4! zP>_7jzC?Bfft9{aT-0`SSRO93U&qEzA0l3?$w1I*Nf*o|_ z0|QUr540{#7Zg<(ol@-$;JP5wzmG%+{d~2!e(^(CjR*y7@xz&^NhMdYjxK>VBFcZKVp+eD?dHhIZWwz)5Vdw>t`Aj@5oGoJz z)XIqRh+I??f&YWRN0nvo5iWFluw>8`q*d3k_0XmpsNseb*&zSA;H{gz6SZGNF54w= zg~0^+H&mC<;sa8C!VjfmjPMw{Sbb7BM{IVe2l?S(WG>W0aAuLQ&*DMmfh(S?6Npp>5tL$Rn-f=@JIgo9o+bzlWBN zWSo%xYpzfG_?~cNBr1=Sy|t9?c$@yDbGL>011Kp34h4q1ydYg%m_ul3X>KErmsd22 z)Qk{nCVist64}`|ws*K^pih(rF|-vK2-6o)gW+)|WstkJ(-%SG54xoBZXUJwu&TT( zylv;wO45MdZ}}SbprSZ#CJ(gn7^#dXu63p2DC%DYb)CHUU4$%|^j2{2EJNPLVyaO1 zeL1C994W#MzuU&=ADm6evgPapy3tZDS0xp#_@{@yr8%0@WAo5J;>{jhy@DKYQu1<& z2<3?4u1mM$IqLV9un;l3&hI>4>4c6N?K$Cd95beqi^80*%hHI+S4`E&Un4)?NK54F z6&w^FaaZwX@;X8B5%-flrBs3(=tkQ^B_%~HDo^?ZH7Gn1Vl?x*AGnG|cMVX@L0u!? zuj-$KJhMPkK(a6c&H-3TBRjV91wmRmeErg&sU*3X)M|~_hpDU(7Rl3m|g=Rw&P8EfJ!wzWE`iupUYBs2w!oJGErT7T4b@rO&dF zh~#~md`2K6gjDVV_!74iM9`3udfSP`lHb%8@X-}TR4=*O!KTn*eYDvV)zesCEv2q{ zhIO$OJ>ML8zk)<+7zLkIjJCpzEIhpnWg>`(X_3XGDEW>P)K`-1C6ps>s9j?XOsOE&Kg}R2MzG10* z!Gw{=lo0eih0#9KkI6~DuBAaQ`lpNqIUWb`ZH8Jz=!V3h{BevG7;#YQ7&5|e*@Eo& zJU<0eV1c$B=uBr5A+VhbI<^@dKqK_g$g*wVrcO+m=3;tW2{knqB!jq!Li=4cGoctj z=7b5TywLe7MYY>%d1$+Hrfomn&C2NsTdz1c-X;bPj4kBs;&>+r?84>5fZ+AQ+^r99 zW!hiasb!S~L>%~QIs~#HL{EEOde8iK$Uw5e|4UN9Gga#hax^qrlBahFgq(Aew?1bvN- z_ICPZ+Y3(2Nd>zP*vs0Js4bwh%z%ImHMQ(%dFm!~Qt^{G8u93AfY8EJh-%>9VJ(4r zDc&IN9bCwQstzUPOj;?OuPy3~)2u*Kk+OGs6EFxwlhrHq+*|_&pl88ZK#VpA#yq7q zh_m{%GMj%eKA*!>_dHOr9>=&&w$M|cGORngTfhK}Dych5kJZ-XQkbmMFzL!s1o+8(RX- zwxF-6G2qxIX8%2DsD$RfK{>XIg+6>t&B|%-_xu#inV?WwpP@df!FoGk0SF+d2|?qg zWorTRQWJD}tBbP#g*P&xKx6qCK9o&&cG0cPP~3hrcD|6+v%pJpTqxMl?r5#w>TK+2 zY;O(P->9wUjk(b1kmDg+X-j9_e4?Pwg|s83+5;`+&Xx{eM^m8mz%JDD6Pk=&3v~Db zjrhM~cwdkXU{+DnSC6JoFJZA7zsp!z>CvT2ZLf(z-OK>cXC@Z|p6r|m`Al=up0ky| zYWa!79G;;)g;5hTm7~s|rQ0z_-Z_9d;czI}+7xIFw$Myln{Ct1-m%W;z*u-V6oei& z&=hFwXtDFb*2c!b?$1%S1MMPgsHqfdQy@e`rhNZJ8BAO&xrHn^c_kERf`v7;c62ng zLO;l}v&!V+1I?jiE|=}xzFkiGweKxFS}tw4IC4LO3;bK9g!{v zvvU>!3bn0O7*Y?lx3!+XiE+XMTZj9XCl?B{%M-9z!wf}q-*{3Cw08yqzF+`fccApToBWF`!jj5kOzLr>hfl z?6}ibzd-SNY(kAo^-ObH<%Xr3sR>6V+tSutxiJ6%KDbq70;PUVd1jvJ-d`z#B~B~h zdVT^iLAHCjKwEQ*|5}QL8AyL_$j#xc3Ir7#Px(7FapFaHpgq{s5@_}xr__D+Ei9sL z?T9BfA;;me>Pm#lt-Wv&^i`O-EQo(iv_c@_0I5@$da7?Gm8=t{6KZLUq~alIxMv}W zmKX*iLmBW6_^Y{zX$u;-VXDRd;W?P}OJT1D$|8@xbAI7O1)pzABDVvB50C_DA{cdy z$Yf4H38Z26C{&=hemC7cC0&8_H!qGFoAKp4bJZF#!z^`R_yMH z_Ju}Ahisf@_efu8csSUxmZAGRL(+JtCFpOm(KexBZ|^|J2f+Y5q0L?@4iCU`ngi{P z17E2@kdgj<9)z7|5;Ew1u{R00nv-o!ZmB2I7;0>^aWz9D!+jxNhrg|zj@#A`Mf%2~ z2nl;5qeK+sms^+ci*_M164d!T&1y&Jpla5fAhkR>{YxPwU<&8&vw*b>}Bm;V(;Fep^0HB5~hD-`*pd8i;j$qJ3P*@p5Acb zA9O0%P6dl-YS<>W?xdXkVSpydaRL4P(a1>8K<|YUJ<@nM)Y#F{=4%Z20`=SMy9_Ty zca<%&6T>~7k5j?BDaWSLFHx0U$~m3u(WmJ`S~Oh)MY|x1K$cPci7BKGznWXH*%!yS zHVEbi${Wm>7D9Ut`O8qZh^!M52&A#CudrfB&2>tEH;VtqjI$%y)-g?0-DC$wN2&OV zT?G1T5C>ophlFFHD{i5<=8m?&`7cub4CQHT4Ys#mOi@dLl#yLIV@euEa+9Q7kgs!or9-8J-$#`=?o<-xX0Qexu&>kXEDvY0YVC% zQhxrko9q64JO}2fen#!S{#?h6&6os2W=~<1KCQ$P{ws?)$j+_B58EMgN-WGKLCwNV zRKxZIt^hxde3u!NiQVo-mkA|}QYtOx3UR+DY~MzI#r3ITroHjm(Sy_HzHyN4HGKcL zm<@IYc1O;2HnwkX?AX-uVjTz`m^|bem{B=;P37bf`f-M)>+hmErcT)oFN{aJ zJ7a2mqNTkvCr`%W6OEmiLj}|r#YkChcoPL3prIW5ix|={%!LC+l;JH@`%w1~v_xhB zVs-cffiZAP`ZLdH0=uc3Kmy|boOZIEZi15|=VIz?;jK$~02$3(`I+q!egdFOrBm(aVKtz( z*W#g%&NivuN~kH~DzR#99;&u%8;5y2{U9!PBlFV7K5ILWL>TjZbbv-dO=Fr3p!2}s zto`*OOb6ij5VYwY2jWQ=q@WFj0YAgmWP6$e{?YEa)&3`yJkJTY+dp|X!u#X+-$Tz| z{`}&ZN6(zXf0xcYbmj>=cv~an*nH4S4b3M*zkTBN-`ulw_WoOcbK5u1-u39&CD$wV z2DpbD+7oPPK?rVr$6hF){!!%l--oaS?`ODRb9*rOj-79AY-#QA#wy;i(~m5-k`GHt zB{UpMtDL#HtP<)I>g-f}>2gOa*u2jWD%6{#LBweBRPEtj5B0W5*PG>^oBA?1(qLOt zW&NQWFADgB{#JiuBSuXTV&L3CU!WP^R9w&XPszR6`Qys=#UE1JX=7!@ zGr2u(WwP5BcE4D}42LWA<&5;IJA@_y1m4T@m>x2$7~re_0w5;DYZp_}J~k4GgvX+T z17qEokzN?9DCUEW?TtI=bEu&09RDlS)<#nQCVkTNOfe5?R-bTPid!f@547ITSgOJM zx5|dv=c@jzm<9Hz+_?Ng#RURoc7@hmT0(!~Im(}>u6UzZY=Q3wg6)ZQhhBjc)0&EW z&0T0!1EMW{sUepuaJ!JnsEX;R>Tf8V27&V>03o4Kz%TN*m2<(5J1X8S?Wkjs%R)Bq z_L|WDIRK}!ToKt3BUib;cIbn9sn&a`j-L*^=fB2LvFPA{ZG!#{)Ccda>wIrrXN!$xsrh+pSYO5;`lnMp*C8XwDEhsS@_ey)X8;BTch7Y zU7B9!K^_Tq1cQTnQ-IheE;7;3yZ93_T=H~u-PZGpo5AHH)x#+Q{>H$RKeLW%^FKz1 zGyF%#^Q`Tw{QG2P-H*OZg?@}<_Wf5@lz3ar$y~Bgy;r@}{%EvF> z8w}G2_pb9^G(WNquWIrC7vDE*IP|{^zfF{tA*~vu92|f93O+pLx)C=7}@+`+$5Nf-^jN=6?L_p@r&e z%j%+q!C)PLMU&4zzpgnX~WPl4cHMRQx zV%yz*4m+N|v$B1x;~cK7xh+^(bLpit&`Wb8VCe_dLUpx3jqN5hb^d3!I-?&Q4zlPQ z@(q2&sizwvD5H3p~GNvHTw;rJrkWXLw@WL~_9`jm^I z+Q=`;D(gRRG4H#;ChtWF-xVI~tGs2hpFX&O*-~bosbuJX`mbCUtYp~s_NH%CGM?7P z=3qylarDBjI<(%Q%-GwoBt6_BI%6p(}#4-qXmivd-F|C16RXFnR6O$^- zrUur%eLH2l{l@WiED9R`ZQKXr>)sz<$48~{@uA977mo&a(;r~F@XM+6Iqs(R#)jsm zc7Jx=rGIAJ&5ey%v{tf=o|4@e>Vm8$P2Mj;uRpW=pS*8BPAsDBfrdU+$=`)d>_`3i z)pv-;v@j|->7Y8z3hokwP*)#IHC5kJ^8cG}b!`@e*UG6dt=ajinhoOrIlD=?s92a2 zCO;wY%`J`1ZT1@f;c~XE`Qz1f0yhMXYWW&ycyNxxSWQC&(`HI~3N4Z`JJZlb-eRAA zwS<1PehlmrDrK!o=3H?eJ-0r$E!UOgd58nk+Pjmy>nKM_t}~_R@%&n7eZEwAhngS{ zyQ$#(Ole1LE?O8p?kh|=n7;ioBvB(M(R~owAo`&Z-NOqw8+oSUNEJ0(C08|sv~*|T zSg$zFj|%?dT)qQ!JlX5A-NM0TZ*I8_vI~rlisR7l-h<&HVWyP9RNh$ekvBn^QLbhj z{DZTkR4r~Ie`G#*jIMpG^DDq0?SW?EFLT{0pi;>c*pCIYuL|AmkZpsb)S>bCqiCo>(-xbY(3(n z#)Z#sZr!$l+~evxcT?+TiV0CwZ*Sy3I;(5xNR0*!%)gK7dZ>}X2}YMVG1Y_vviw~X z#@PKouCd+R`bmnKwGycO5WSQgA547giiwY1_c?0Ab)SRc&k5Qlp@o)z6PoPH%5UT^`n3ytv%G?c745Uebyy@Xk(g*x++nL5zM?cj` zt{1sT&v*+e=6FSKEen#NtcIOZ)77_lm;S*54icBUxo}#9>5oB&=2y(^*Im!{D5l-* z*{+~Vmjwvi>eRP)^-M%uu9{sKK=FlPhH+HVrhi_|md*=S5dam+x>!2Dj3P1kBION~ zNi5E(D(j{zpqiLWU-y^#G}{SP?!D&)hiS*gs!q24<7Xe-@{P^=8YvIn4{gOttG3UD7H5$bPE{28hp1}AJ|6-y6-0|sJj|#zaH#6M8t4vPGY;%TS_j554tL1?OgB@ip;q!$} z#S!o5=*ZAOAC8Cu%($Nu|B{wAEyqmGHFnSkbBlW=a+lLqE zezr5Nk_ztZ?n<7S=&}jqq+`pIm29wid)4K2RJLksowFO~(sfH2`mMjCKl-j+40TKS zBjNGz_=R3BGnZ58=eR(7pq)HY@29s}4m8Gc7(UbuTBuxbM`a&)oPaA=OU|pJjOjeh zx3o32H~V+2V_I64H;^(`y%RrbZ4K1E?vJbkS%lv<`!4an>Dcn-7Rq1o?j|m{kTa^T zDt2|SpIGxy3tsAyt7=cJKF9YVb;VoP&Ltmgvw3|lkRzVY|Kc1b&~b?M+ZeV*9$5Pv zG@8P;hOKci_{YivuppdpQkZo4NII33alQ@s&fu*g5Al@|m)~Y4E!Rjx9S#EorJV}x z8yV~U7)}0EOWqPiF9;~aN>j1DL}dkM-Og>z9b$W+qeX0QYi@6C4D9{I);cl{c{n8h zb0Lf;pzC`tJ`i&Dap5bXSUA67AqJXv<-KQUQBLo`5glOC&cCqkle4q4e)*%L`FzIz zcQl_rrdJ%OtG`w(2pS-aoKl3FxZom<4%$`r+peU!_LkNbSIPRhCT84{zRff?3ft#>xnHW^+H`E!^1WK!pWR>i#r|q=r|+c zBFEG{G$ro28^<=n&In)6;zRvg&*8wbKU7-djHL@FCO-Sc+7L=;WThaDsf+3Kp>F91 zo{*&)V51CsEzIV7ey$bfz57is6|6W}kM0BOulV(OY_M^`5w7c$dd9+0X<~4shq%im ziE6T6hyZrGu&lL+NCxi-#MQ97S z7XS+&2?q%4Om34)7SP!GUu2yNT+{Xb|38CGzk}_=_H`R$vNdO`+0xf@7L@33MJwr=zf%-y$o}8aPSlqP{fEmAY0C^eh)!<{rDIZJS%mUl2(N3fY#;hQEo8>kW5jINqM0osX}q zM%$AP%Pd#Z-&uG=X)t_(2Bm3 zen`(Zk}3LuqM|I_^d@1fj())kA<(qNYM?$~K^W)ii30v@wD>E0MAsv!{3%~nYISf< zvDY+27>D_xlb0Dv84@>b%VWmj?|)=YZmH95LsNY^G@?-(>U?mG#(7jo!uT6Da@uh3 zT$IxAp(}HFx#>AaY0+Yv3_3C2w0DqIaJioTxSJRM==(aHWa32u2|v1o^qd?ymbGh7 zJF7scilMh^HOrt=tE-XN2Kk8-wj*bui;(v4Eg7U|Q)A`Lte@LI1Xo$`eeFGg$Z%L^*13C+l( zS(a?iZ_rD3J=|%uPs)B{m-T>m8>g%6qEK!XD4nCR0Om1WE`kPT0WyYcXr*vCq;wXi z`$`N%Q8KLYIu2C+7nG^lni(ym^UV9-O+lgNG;5YyP4A`huAVc9<5*{3xj=ZeO zSCj9)YZyTj1yV~O`iwo3Mjzhl7++zQ9YZ^Ogxg>j7VCAJGaTzx4i|Uqgc@xSp1irw zBQcE+mn{EM5{|i>#afd@BEgk%T6BBlm~he~UoTauTGJGsR=i6G*U&!| zrlR4#v%oUoLmXP7>X!&>E2&peX%`jrg~xDFz?LG>&9I@KlXhBci@Y1@Cf%w}&6Tf^qLF%{FPF(co2mbt+ zc&>1K*ntW9cG28pFVWLfVsV*W^dv`5Dt~;XIGk|*2ZhRgAQW<(^d{;}SN*7Df3>N* zM<8Dx@#w}R#I1QTeP*@_x1VOKT1=`>mHAeM{O?l_DIG^ydoyriW13IcgRi3Dw%9um ztqV`l9!I~ULOqSy5oY_UZH`<|&pFR`l;|Auq7UB9;<@*QLgs8^-$2CJAJ>aB8NFL? zNSbp&uRaLMA4b}$XT^&8=-BZGl5|E)s-)%RCVe9uCi>$y>3NZg-c3&x6W6gW64N~8 z5%PpW24XJxvv_crEdDO0Yh);aEtV2TnZxXKl~`w9x~;g1x~!bpnEx~DJ498N0Jh-RA!4xiqm}vkSPbU?k9u`%6_F+M z`w~bC~acJm;Hz&+AI(8orQ9I45WG%jOC>tJiAx%V5j_{pyCfg^$oZy#8#s zSoOCO2@kqb^;u@l==;y;iOj!_(zm#lf8&b6&gxP=*Lq z%_oVCAK8Q!E1jzrjhiyk8HX)9{}X2BjnvQ^5`CFI>U%R4yPWYJdcrDrvtp-Mv!@cw zOrCoBjzh=L2BRK~-I(z#URGZa1t}svN(bfwaD=r`-zbvFa9s?eicb{{3B&!gSE^`e ztZMhzsk1^>YO^;^3o?q`O_lzdsvLHczcLSflsXcmw)|no4hL zkKz6_2@cL`dWDP`7${~1x!&V+ut-=RQ$1WldNg;;m(Jds=h;%XL4M#isf=W-{aQZ) zsu_s%okaDMR$(>UttR`y@0WV225|yFyA2cBSOPm29*GPM=WK;;ujIZKGDekEzw9iI zxp{pV55A4{7TGh%pr*XsurNdC&18a&9R??q@uMfd)|92=+z9a`+{PPm`SpzhAId$P znTt&z+;OltQLz^6*T96 zo+)e`E&5X_MO?Veq!uz3TXW+mV|O{Q<00d^*{sZ7T53150rm5DF|BR&A}yS(5}B}T zHG5=kPA~fX_Xs0%inJ0E$?tcQrOPi=H892a=|NOMA*>o9I#3}D4N$eNWh-7rF#ji| zth%+6v-IC4r&maP>qKPHu}`_f8G`%h8HED4HJBp3uc-B-Jv@q;E-oMg;(6x8i08kU zV0wEEJ)~zWW-Hymz;C{4kuV%(9gCRW$!=AVmcfjP-ssZFrP^<~BN{_}85w(eu}00H zdjso7U%0!MrqIHhYc%+AyzOzrIHm2=s-iS6V+$m7e&eSyn~>u~;U8HI<&fquk=yIw}LN%*sp)Q}}+^Ni%4 zfKg!jRq1k(rIw>cS2bD))inWqU43nxzO^k_Q`08Tv3Tr{I_@J9C=x8nMpuz?j`&8_ zto*;oWvb7Y;J^n@%#~qx2UFJb6~f-|Yq!uD{WUYRu3SlpTafLZL!R!%2-7$^#{TLv zy~;}T3-t;1A+_%xB9mU=_&5cSQJN)};Ci0OTOrrku2MxoS=KlkKHne8?KT26v&j33szrwSyqn;F9fTl6rLCLYc z)$($;L!hp2ZtbkEs-w3kPiKcUlyfyYb$`gv{g5&7RR$R~e3`){6Y(y!1m;b?*KjbW zawv=L&ZD^t*$s<`ZpA@?u_<~#+M@%kB!?fYwBqsp2vX)Su-%ym$d)PRE{U;2TFY#KZUGn zZ#;miT>suer%+X|HRYily z%*pHR8toE^9$O%4Ym$ykzQ#0-jroMB^d1S_mn$z>ZtHI&jv|3#pf5BKHJoT=AQpVO z6?*1Xud{b#slf~w8=MSv+?4)WNYmxbjq!MWkkCM!*||VCl2GmN(|fe&j@(^IAJQ`Y z{i8Ibb$p!0)@g`T3v$5RqK&db<=%@$q>8n+&`)Xbu@v9Mq5{+^=E|Vg0uc^HASQ%O~a zovi&A7W&xGz*sUmXMb^u@t4LdCV&~(r8xjQMJ5dM(%RMX;{4Hc>3fXLZZmAvDc{Ql zq7RE0Mqrb4pq>3FaA21IeKNVUBbK&e332~OBx9=w#^kqD1!5f*p5Z z&0iakbT*l7@{rMTj!Dj2R|^*y8!epw8ll8$b(u}I#9pGTr`Jevju!WC5}F#x+8SjO zNZBJJ@+OZNOW%qG!sNhrpSjHfS)eUYo3_Fzke-4PHyKWNtkz?v8C(h5Q;gUtW2D&i zu+(?2Na`bY*UDWz>fi9^j-4rX__{~smoKY9KfHamhYi*dj>QOmU?|yBI-CA==}V1o**p-zeA|U0#l2Iugh$2Y0u^9>H9|hO-~=+7ncf%H~56K%TGKR*{er; z2l}9Cw{*L6QnEXnf>o`)*7Oy*DN{ff;H?}=^6dMLS7!TiQt~nxyP2a(SL6D2gTxC6 zyL*xJg*gJhkGL}gOmtvuMTS7Vq&nCbXl&9@iQ1~xb3gE%=fJp6{DwM zZmq54%;@Y^N(J&I>)y*)tNAa>onAM87pwgRV1^NB>Bh)gp9vVKo%b1-&d{X%ko;}G zV|Fg4!{*usyh5kk=>L;I`zZBzP~hxyfSP76*Zx6<%{`b_4-3?_4NbgKTRu|pq+D-z zl$Ru5Edg(nuwl4xq29hugS7umJ{m-z4sjRoP@t^)w!8Kpd7p2ky zj~eDo6qNK&2BLSokybHR&@Y^2oJ&k^oe|Jk1#m3%Yo*u8zw&*9=!4$e#Hhj{Wjj+?o}J|rj)s(1dRS%{ zc!l@o*yp%&X64of0dsGy2~?Z+FHXrnDnU8&7fk~rX{>K(2nN;4Lc`89p$#6~bb%mW zS7_JV^j{?YxfmWEif__aHT{23e8a2CxJ#g1x4izDUgBxfCs)S@lB=VkzrA%pqT0VO z;G1RmA;Vr+fL*N_3otJ4cEgR5@ikj)hiYb>#p$0u0l~Ww1zg}V5y&mkfuXUXhDLKY0JKeD99#p3^9c%rv!Z+cxcKc*i5hZZI_v|91k(xjQR0+lOsNowxDSTki`E9AZWf==X)?tfYyIC`T<~U$waKYkCDGHp83hO%9Kno3r+!$5Db z>Go#kf94drxcmc44aGXayr156QbsMl97v4KE|TQ>O1M3QRw#6@7*N0?TZX<|i771} ze?mffuyI8H(=$M?FUK&qcd{4q)z~PZEA_NxzM;ZOeC*8zkxWxDFZaK+f0|H|DRH(A zUGwTndl%Y~R{!4@qM@&Z9M-pNmTs$CSQ#+=kroZuXb@mK${7_S5SIdl@PLKyZD{x3 zEsECPy$|kzZ@QRJB-Sx4^gS_4ZH5@KrnQOQIk)}NBe!CmUonE{t zoW3*aTo|?Mn%Y}y^j5cUkL}F#HDyoMk@wXU*Ad}-GqFfBI&8@g#a1NhM+|X3g ziUz0|XL2x56VzAMS5_|LI!Sy`;MhuvOAz_c+SU}%2WmRGYT}vJazZ`}fq3&jGMF|9 z6zEX*4U8E!iz|8s1sQ@IZtLY z+wFGW*sQEVDmClYmNTBwe)hVM=pFGWTPVdJ1c=QfCFw+8}{ zQApx3O2KFFx(|OJl7k~As0&+w|NO0HA-cjg#X%@YxU%jjbn*odM zwfnU%f+u!1XTsk~+>UfQFkk;A<_ee&_ju?xWtb&={sIxdz}_s|a7XYvq2D_TrLX)& z;;<;~pE6C@3RMA=5nvYrUt_Tv-ek02W+TG~W@l{ba$S`d5dXJ?%w3V5YTWsunu_Hz z-tlInLs;FDhJiVs6|hmBR~f1tBbK9n3b~ep=9QCDzeI#15Spy83D*53g}&8eXqR|0 zP_b^ZsDDel$loJh{<0dXlp&Ca47(NXmvpyY&-z|Y*5;cSe1`wsxO#d=MdR6kI{@K!KqYW<2r&@0KFja;td#%O)Lb@g`C_RsE4Pyp#uStRW0poG| zUzFblUd-xXv&A>cmuWvzL#GRY=ynB3X z1aHF<$XjsXEkSG-uJw=~CBxnD4372nPwSM3&H)WDk$~X%Z;Y_6U8O3auht4jlLk>P z6W0fISz}mbF1C=E7?<*xr$+3hB{4Jw!}0~BS0EGCjSCia?(Hl!m1w^&m#h*wKs~Q744^(8jV@uTVCQS{yheMNQ z_+5Gvp5FpVXtg#MUl<$lG(%`lv#oH6FG@HrC8{%O)>CH2MFv`viQ!38@07FX$9BW| zO8?HqsCo=N!q6Bs*6WAEMl_<)Nv2Es^KTL3iPGyAF+g2vwtCqOX3`-lEL>y1fo+X& z7ju@ID=XwemxZ!}q?cY?%k(DQEl=jU^e&5?1FYSqO6lhA<#_AfoAF$$*=VuC^a20x zE&g1i#dX#I(iw)lQm0H*6cbsg+iEV;y^!i5eaZ^DGvCt;ACEs4=_!!gk~= z>>*`*oriW`IPrWp0}!hzy{yCPr0JY@A!Fn0tkJIaRkwN@F#dQVik{>9?6Ez85|QCh zfd1Z549o_`Lw%TVgd>it7Z@(&h@wa+HNDd=tRnvQ+OHlJtj&segSnV+Ao~jp_r9Vx z(;2SJN$-_d&L{Ew=y^qgIb=jNSWE3q@!2#66pRi$E0=g%lN`TF|p@KI{2w< zg_1o30=UOvl^5MvNkvh1N02*h1FxsbdO~P6=bTa) z2f*1IVNaijetY{rq*}9#f}4Xq~7WvC};+ z_O51y$AKIcSJAFg8-|x>vs=IhKvWi&HbF|GhwRTV4vTeaQLCf|7ucdVyH29VJ{UGa z%#aZ6mq2|q%3fB}GD3SQ$h07&@eXBss_QIzYp{Vnxe-hFz zuuy)7A_~70`s9Lg*aG{)i{K4Qu7eHB;^ZVQ8J^MfYWiFo-&}IjU!=vJVW7b=W=H)< zRPVWjIOo1m-F-^vsXvwgwrQP_!D323944XQt#Jv&t5yoSCLDk|h2ty|TGvtE*3|85YHN~S zPj#8RH=(9gEX(1yd4Ki^Ho&QJ zcTEx3hkd7S!Xl)X%1&O)?pdrEoa1|2M1KjGejxlwpHl(B`~FmQr=?AA4b;+)T)OEc}$!yMnQN$Mg5(`8oa8$a=vxn9*mrcu4Rdd zbNsRtIxb5;mMc7Em=a^?>iQ-_wtr$Bqm7zSH^d zE6Xcs@HI8Jd0-;5dtZ| zxTJ2XjnJoh93Kh_56e8w!9cApaUuKNdKt#Oc8jKsA!GIr6*eQZ)Y6>t#xE5R2gE{t zt^Da#-0cFvO(a&?luAks*R-pSr-ha&j1~tjqkBC~~`x^BK#@XR` zNc6*@Hp0~q4DH|%2GX!1bd*7Z(0dgV=?@#eXvHP>0*mUli|Cq6m5xFq1$6*j7ccf8 z^dJnJDomA$Q0nGONO8}2Umri3tHYbvFdn2|cWSI2k`VM`dy=CU0)#WYsz5&y0)!Is z2N;B0UQH9VAA#i3cJR=a5Arv-=&xcL&6HJQ)L>CPyf~XrV+reR&dT$Jv@V_gAzjE> z=v4~wXS9rx?8I-^Xboy1cwa#}D62ZY5BWe4WY(-fqaVT) z$jZw!|EwovnzaG?b($`rHR#c@1$?WibzYwvE{(;gD z#Y40ve_xdhF00lgRZmyrXz)k}m?dgM*rT8X$oq;Ui{9_3%$kKUtpoyKC4IiPaf7TE zG^TEPQ0t!y3B`C@uc4n_XZ!u;L zdN|@r9i`T39PP*myV9Gx7;vrrND|j?9VbP?N`VCy7VK8w3YgXvF#OnfW0!>2T+{La z|2(T_KA*)cZu#m=W(39 ztc9Q88Nq!VpX3Ufjun{Z3q?8D8S=4;8w=-4Y)-q`ucShYbFEM4?kjrrg4_Z8?h>Fo z$A)uqa%1`^7`c4V5__tq55!NT9>o#e1V1DXw`zlQUJs&TzhJ(}R*S$O2{c`$M_MFk zhlJ=CG}qVw&*t$5lT*bs4P)X+teD;z&NzKv)}_NNHxAFJ4-j3un;5zk!er>{*(d)9763luRbG+MT726-1Ux;0{G3IIR5p z{1Rm1dgw_RNN4dx(XBBF#zFn`>k&+Yc09r$!PEO`&Y(lUYPvTb{C^CPMi(M>TN~2J)}}vH1DuNn@hZ2;&M1Fs+4$T zdOC4(>FM<7%G1k*5ZWAFDc=xyNd&-b$HfF3RJ>=g?suyms4eiU#_kI?$JXkA@}wH6 zehJLfMIS&Y`>LL}MK4*dvoN_)NJ_#GgU9iXI-28G7APm22uzBHsf)k=|w5$p`Cprh(1Z^6-9NUw$Rd9B0AnS6b-Cut%%{yJIeGDQ1O-j4rrTvCq$^U> zQ!>7I2sAwJ!Md0!mEmn{S)zF`%M_9eEteZ~=?wD6=+-UjzV1FGYK_n@V>6k;b(uoV z$~4it1=ymrhQ`sA2qjHaO_$fa^C0m=zayKdU(80%$7uzKJk{5_f>u% zlurw^p|6Viib!l|pNRYbk)LLjxS+rVC0QXh>Mj@l%D*v<7@OefvU?ku#gJmYET+?9 zz(<~2D2h)0O}dy^>BHR`Hlt{^n)GVk@5CUvENHkw48oMQH`_Q6M)4U=2>NigTc%>TgqGSDoAZ%?Et6Q%0U7D;C7_w;!XUibpv{f$Z3!*`51-}EJ@F6rZWqK(}OyB0}~BL=?e-i843s-E~OcbU@6BJE!EWu-fz7I%11gy!bU!24h62p63 z@&H}JDXyn4x3XU>`z?x%3eqHHu&vOm)7c^kM5QK|2|PrDRmKv(v`&L4Y4i4tfAlu&}`$#9pB%zA5x+Kby1?on4ge_+8vKIP&2KEC$?~R%CQ~FWUsA)P%@5?b zRgJ~j9boBn;7v^TMJmrrl^}{WeB^UYcY4?RqF}g=jaCcmqeXpnf{8%AmfzE#nm5y% zwxTyJ5GRb;^-57opY-*j%W|N(iL2axOpSi+hr(TTiN>&5qf{ReURgotH~O4em~7C& zQW>9G#J*_hQqwM=ui1w~z}a@nuQh22Tebkzt9c(Stn+=NqP`2^ z(O;<{7T-f73LF@7sl6gkiOoR z+1LYZB$iFyoproe~L1WJBj9=xv)g)?u`pQmF$^zI7*V|Qaw6&0NsHJ?k%i-CuSEH64f|F^t6o;@_>>z8tdOmdm@p8P5G z%poE=1ot)FuQ!yXafb+<8Y=iLP*1-Ek_~M-?FtKleZ4H5meb%RX{Hxa*hnXRW(X4K z8)sufG&(?|Vg>d{z^V9$Qq9q+VH0G$Q*a=Sjr6l(5@W|R>pNtPT}$*0A}et}x-Y2i zU=jccMv-!W3B`2LPwg~!Ct0bjxPcS7p3h5Z=a&wv!#^X(?EGdsmX*tLWuDH^NP^B9 zFM4?^FkB<3QH)4}U7U{4r)l9xvFfjTsBqh%sh79{ ziKfM3$es8sb;h#t-0h5IrEYnqYF$2KwLZUmm+B>n@3wvk(yKyrp+V9cO_b2jy4Vkf zvloq_t-G$&!rsM33Jos|E4;XmzLCC16$mDJ5UDJzvi{hCt%^>tbV6~l{KXq&&7&wq z-T4#ut*yKcMY@%~cV$>^`XyC}UG{zP&idQ&DxN3>rjzvnnT4 z9)U3mCkpIyXd!|nLdVi15Z@PG^CaVKs}hfip2%%Lltlt3>*zFlXg+(3nEvesp*QWs z1MpPivf~KE7lFSlVx0yt5U3gYd<20nm^H)n{ zXb#!+wC-9ORB+95xF<9oIkci~tD$27y>*_>w}1hdL@J}t%u_w178EN7{8cIYJ`%=^ zhRy;KmHx}yA%4&Fyg+DawA3;EIQ5R-w4)fONu2K5X__NI?Q(b-y-tIA|GPe%5RGsK zam~P9)5g4fs}wuphP9GdQbp*K`GIB2=+{{Xefnykt}qtC72(nh*0CxZrP=@tB*pA& zQp}PjHqzT$Zr!$F%M86FH(;ZsXg0tRHxaFj5 zjP91zUaHDgidU|CIa7^u7y@-W{cF4GF_kKO!4?d`Ilaq?jCj43(_0MH;@nD9(WpQG zJJn1r4-eGuVmwg)T}9ZSchNAE$i5v z<-nL1UWfQEJ$s#lezTm`$Ei$D^qSjgu1Vhx$t(KlA%PvoO_Q35poktYN#WoOG_@fJ zcS10qn08b$Paf1ytYT1#E@ z><5KjBvdjw(7P|Knn$tsh5^s&g8<(h8|@pVE7GJHtl^^E)|x6X=e)ssue>*0EewX) z!~sbFFbGZAx(y#QP|_ZIm;u$0K7OG)unu|b*n&ht@)g-I5)Otu{YiT8CP{yC%oKV= zaIM6IEw$kO++a<0z}s8`!6RK>DYTT(d$o+&VWRJqNN?ity0EJn(v_Mglx(v~K7U^$ z&pG~7D>OSaHYFK5y(~NI@$%yOak_Cx+CqQWzc@MG=doE$7|U5L*aW(4^y9@fpQ4j} zS0?wtJh_trB1lwPY%DdQfQMrWJtvVsZlJ5I*}QMng9W2zfjmpK>>^s7MlWyI987aK z^Z@!|jhY%w$gTAIRt0_tG%0jbp^ex7dYmqk!vO@#?`G~v`QMxAo)q2T5uv`}1-rRkdwYU1_&rE;trR9N*UiU1s8rnafiC_hz~*1=B8Q$PsOQxBZJl6{1`l zU?BJd(!uK&^FK1!ezC+KI)3oTV9Po<+)Pm3S*&pISse83oj15exsbZL&}c1nI7I@K zm``7|^$PeNu%%sP;I2qf;FpEYNz8>iS}gc(_)*Gquchm*Hn>+8(7%egtr8i!YA8u! z684JznF8j2Z(IRG-`^%Yem~pHFmTwv#WA@UqLkndHg&Im%Rj@%+rM>OLE66)I!)A( ztH&Y@^FIEQIyO&>3M?i(Stp!(PWv6}MG}}%IoYKd{Gp_?FAEy)dLa|%1olmk_Q_7yFuxvhjn<(3CFTsS; zZ1dkH5~%On7(1ONfR#;N*nU*^l|sI|v5-_CW?X>+4J-{1wiyf}c13hy0=3bGy)MU4V(|}+R2fKs0^AOmLbn&)P;`w~4jGZbYdc!3Xxa{hdX(YJ5 z*>0#=ZxGDQZiU{e52$6;t9cc}^Z8Eb@49}}DJmPAN}5nY%GvY!ZAi-o3r0T_icJ9K zMNia|D0|^$fCoeM+y=%AJV`8<#lY{wI&BJ|il8Zx&r$olfwrdlpcs&Oi^J9~*U(K@ zNf4Li5hzJ@5vwk`O~Za6XCGD-pGzuD#Q4B_%|GRu*$7<)%sQbC!os3=a;DBB5@cun zQ8S!PjJiKe3H_t=wq6oaJ!i_D7>&Rw0dHKK{o|Cn<}4djPw&?<0MR-`ck1p}sJ6?A z=Y|v)!&z-7U-6~5^0o8f=66@o4HZJP`DIGozU_2u4*P^thEf)DoNMygvN@(3b*@}R zC3?@}QL)^p6+&eFK|#^1%TI(!wQB8w{C^?DZd_w_s)jO&nLBC4BKvBAD8j9r12KMR zoi3ElOoV#reS3^K?9&fYu|Drn!DcezzV2}L50APtOT8dagaBJo`-$2OgY_>9uqUqAA;cpEx~>4}v#PnCZ>uJ} zq8*+cL$JTnR?2p-zGg^oc9m9|ZO+CG*LMVJtoSxZOi;fvTE<|Nq$iX}Ap!MQs%gKR zuO)wez=r)?4S<31Sm^j|$8VPAnydl^2P~XOTx%jd>`jHVXKrPkK(kUsJoLqJ zOTM5GBu=y&-{#QQbZ~bqLMiL36wr?Q-B-^-X5@yAP#w#5OF~1VQzivbU8yAw`j5SC z7io-4Jgk!Dk3V`$uZ@dW05+2$Ag^~5ZG*ydd;|D_!GBbX@5Cz7>|6B_+hBd>@?F`1 z+R7S?Z@Xz`t*jSam2UQKt(xZ{g+@$WJ9klNL$2DzR36_)Q);MF;^=eaZ7x&@?epKA zBhcAxih;t>MJVF?lgN?Lv*+(A#ZnBWA#&%? zRR+SGjTh20OG!*se09#A^Rbrdc{n29?LbH2unVDU#SgPnbSX z!Pspa+ZWA+SkMi*U@P1Ndp8R)3Yx1aM~(ML3_g}FoF_oy&v;K^wYnKP*n0_B64 zT=(?GIJ*a4HHXnAR?+skGQP<$N*`Pz@JQB-C!^vng%eCWD<@t+CY36M&E^6}iOx$C zc1JYN)NjcaTAedzQgE9%lLEPVDK@Z^<@oD+XHtY#b7A1zKc4%z7hkiRyx%C`>f&6E zLYL`GiXILIC!EhVuJPuLcnF)*WweUxhVUs@3WL~51loL>DAkqE*Jg@6#4@uz9k+??=`z4T?9Q1z%ujom zb6-67@yYG!daIek?*yQN$~_E90E>fzP8l`c4f{nBoD)XPQ_RpF24qeL_MDaYA`YC* zuq6NbM;NQqHnCr%IKF|GGWknV$%wC%Y&me~M#vw>2S%mNiR*EgoHLEWO^t{y0YP+k zqY!=AncF3}P2DcRJlryKSp3sr@r+FPpWBl%AzIot2Pzlp0m?eVbXc72LeCi`rIr3G zNkbVM4~x;ScNn9mV=^Ijo-eKtj}{#k%TPR6?7_ogda8b?YBn%23mj;}aI55}y@1r* zq?BQ1)EDvJsDzImR8)1<@%w?F=?QES5*zSBn^X{c(8urC{X01J#%6vHZuo(8(d*X> zN2hitc@g8rodN_hgkpi60%G0hCTQrz=}ikNGsMI;rJjYvd+4VUI0`5Y+LyW)r#z#c zMSE|=Rv42@-6uU0x>>}w?!gq*d3Ter|6YWw=4a0RWIzI2@$vh0-F*{x622Q<5dAzy zke8!C5c=puWcT6}tXkV^-3j9PDnBw`tBm|bXtmPLt7xifbAy>pgP`T0=F_ah4d$U6 zh+$og|3*B4nMOfgqoN}+)f<}J+bAHVGbdAOPU?}LK6d<(sov4u6NKT+vUh`7p1(*u zZFnK|&ql7`ZcmZUnORA3JB*M1)QBJXbjZiut4JfT_)*oXRC7h4F{zuY53azw*E};e z>*+C2SCG{8Q4og-M^Yg=pim7Nw||INV_jFqNDxTf>nB2xO=hOG?6j6?h30*UNc!lE zDiycGs#LU9f{S8JB3Wb{coH_4YswZL{1R5UsY*HR<^U-eIrjPW%cLH{&1^?Fi&)kM zBU|5Y6@^M_hRk}JAQJ+Xs=a-z#8R4>s#)PA^)pAb9Y?fzOBs-IXx1ZvPZvFM8^bnA zkQYlwn{obfGjD2e3%;pgm*la#WOTs_(S53GvIFmE3$w)eGjD0n-EMtLE3}$u@Bs!* zZrT^jGvt?3kWQeAb=sXyJ6#jqT~5I!*!qEnULGNK&F@*RoH8yar+cC_@8}kSr#&|n zw@>h~LH>DzoNbVIeXBk9#d9C8oSVwnVNX%q4ne5hKQ}eo1&$iFQZ|j>u(?ytKm33a zcqcdsH0$K1xvAhNIPAK^WgH&#Suy*pSlAJ|<5@9db0lW&l+in7!rF0z>^7ZD96y{U zw7BTtBZk@I^uq;)1(kgLIZVGP1U-Wmy+5^ffv&w3MSIJQA1Vujhoc$fqoUkG({RMF ztO|>}Cy$#;7_`yQOB9au(LkFJK}T;6mgB+^S{w}xm>}wyf;>xEp?Y^w@j}Q{aEZnr z;E|TTkUv!nCmM^%SItVH(Ly`gl^*&+g)ovdytaUzssjo~+>5oWDn?pWA6KBeN#3X{ zK?K-pSRlqw0+HVyPbD2@A*JeNG>~SGTpWe|@2_qqQhm$nJSg74a3ccGLhQ zG71^rq`ao`^YiJiemc9C{`GqHps}cEhiSMd=i9&qrJD8*x}lWzNPe1~^5BCR%nq=& zp7SwS$213vgedf9-cZM;2a^4XZfX#NFJPlj#eEe5-QglRXV`2O*2d`#ehDAGLqAy| z=)-3M5b%o+T4hKKnRWVE_ZsHw4Pik-6g~7T54ahn?Q--vIQ@wo!L!dyNU1b z0?ku(w4jY{&4+aB;im=pLQ~AKqgmgKttf)pLLvIu?Q}Atnpa1fRF7kvmQcizrPc_y z>vH-pkh^>e@QX%9^lVD)^Iw|=iq#gg4RBBgy<`d9;YS&6)|gn;_chygp)NmShG>tY zLk`~&v#G0^ly109x4I-=*i26?E=Do|5}8BDJ%QxY(q6Q=i~)j)s_*r3BzBEL=!?^> zWq6lnkIK+VAB zE78Nk@4cGVUQ5Q6>BM+oC8^r*IvLRXe4Q4)q45~&xF^>Wf>Rn9KD~O?y>Xcy)mX3C zMvt2@GO~0UE^KujWI#%zBYm&^BNwarUdJN;Wz(cb;6q;gw|F4!W2 zFo@$!!C3{oP7A@Cw*a*)%FFq&-1DE5@?t$&&4^YGu|Jj2nijamYeyuLo~q^E#S$LS zZnz#J%z#R55jr_~w07;i!cv=2sC-i5M882d6gAoEL|)98ZaWnVfsh6eG{-%GQbV)9 zQyJp_Zl4GdGO4TXrQg?@q59U_O7*3j-fEdvC7JA=XkTP#G`5(2J6{+!BHINRca5da zZO8S!)Nr&-bKC90>D71dFy*HTqegkHxnvZ{o#0EOnhhs3J%Q{{VraTX`f@6MC zEO1J|kG=k@F7~za6#Tfd)J|`|N~gM$0kuH&>nf4HW-rqo@26z}G;~WkNVlNiK(o7B z#5vE;hIO;GrlCIIcAqb;uiPr~zGsNsNFIAny3x2R8sOvOZe z9oNeU?QazoNc`Us86Y;1kn~B>YoxQHbP^VW&L(tEBfHBdHC+vp#SFmS^tvkTJ2XQ~ zFR{}X2Ul1nYAA0ZYd2=yhxn$vDbu@H@z`#rrB;DF7VKK+=6( zq!*kYUL`n)nT*})du&iNFZ@_8;vc|uOhWx2!oEmz-9H$Qx0Q~*Ez=itb{2{dwxOfP z#TK$!aBJuiNp!RaX%|}=^jkDnFU$LwUcH3=aG!7>%;kvOrv+MDW7r+B8r5DI5 zcMAAYm42SoEnyVE^?}iocVt7>^FONQt!5u}2zJwLkzymdj1D6#K2AFqIFKqDjP|l` z1^JMY7HI8Pi2607U3i#2i-7d%7PX0fI2%?5odE7}Xqfw%&~Tb*fG~~7ut-m_jShAnrZOA{LwkL15mMxH1t3>9!(sUF)DG_y)SYb#jY zm*^gd_6#mg8uN2Q_KGYEW1b#mZ`>q$iLCbdXRC1sI7y+(36m1(M*1k992VOqGWMYX zoulvgi5IRa4v9USL;%9M)$le`@uy5R657Ftn^uW_&x+bbEL$cPCM-%P1Fc-qmaj$g zSJ(+DF5>X3({v9izlICTj4e=T7*%X<7%F7(J)%z)$Ug&YPOK4U*6)To;uoEJPHF=m z^z_uV$gH|sZ#U_OC!V@C8Phxzv%Dy*>XI(itxn_jIGfBh%L_SbS*~b&>e}Ob_1pxZ zX=yxw0nsl%ti*K3;i7+;Ho1u^upC`)oi@GjUU{a*lty}e`OjoqGND}qE{@XA$)vk< z4V6kZ6`Q`7wI4_GwQpohpbE=`rDn~KHsVoz{2I-!;|5NNGwGk7C-F`Hgxxifd@yus zix!jb`MGXj!mgLHd37BMm*H2hiTgFz!*Gf4G-^>+znCrd9AmZ$iN* zn*wR!Qz&A_VSXA1Z4@zzRLv!e`cM;boSt{M8Y{yS7~u`)dFxEQi3v20cvem#RD!;D zp_0B6(>(kV^xxHUgaFsfgGr)kyad+^y8EgZW2BOP(w~z{`c&?Z6c+ftEKZYJfLhX} z&47n{Cwt2x-KtiI5`>rJ+Xv~x6KqJ$8c{ohxATx?+ z_4RVxpMIOR=_@0{)U`GGW$7k?to6~t$_gUC2Xd0-1C+_ESS;AozeOeVc-ht)B$zBk=)oh%b&5`k z=k2;i0!p2^XOkF+0`$n>q#UOQ27sUJWn2FwEGhA-jKfTzxh8*BraxasFBk9KCD)B+ z5a+QUiKVFIJc+%Gv-e}Kbz$rxf#z_Y#M9B(>iU)mcGQTX8+zF{#xZ;T?m=B61G0pw zYbjYx|MJTRMMgK}mv$L+!QErD>~jV?gZvTM=ATG*$ML%{o;kiR zYcC$l&Z$0qMf5WsX)i8ItL*2wJA{FsiM}b;hd}iqSeTF@6&A3vIR3I)nZ!v1B#}#$ zAxh$Vk_;wCbi40End+ZQbF{1ab(dF?14W(EylY8!y#1HRYW6}TmVr4Pr4ax7MvqZ;Ww8T4?c5PBk3 z-;XA}F%l^$*1m)7lzobXCr=?dFiNj}1K8Zl?!=ha`v{32xJSHM=t-#Ntrp_{P( z?8<+68C^9a!%906-q0g;NC54R&{w;lV_szjNkAib9g?m;y+HpVEx@XQTDTuF(FD8f z5w*UHXM~4UiF#qZk3KxTIFxkT9BypRkiOwgVC3u@5T4)+%|6LMq(5oDTg}r5v7&xi zJvsb zJ&XSv#{=&JwjgG4=9f$VH@b$itjE>&-x)m^XJZ^P{NGp@UQ2zJGYol>4-d)*TtwmV z8IR6y9UdPU<7rzlc+Q6Pm_fEm*%6+;@58jv#JPTB^a0@x%Z#hkJ}vFhcb}!PBamHO z^<@{B5a&Ws&f!CU3q;}Jjs?(8a)o-%fexsp^na29U{2#hhyHJ#3CdCIeRyjNpK1cq zD=%^f+E8W#0P}3ni3P=#xx_s|@>Z3fjlVgo+@C(1kEE1;8R6NmF+ZM8&v@c&un}L2 zIst64PX9Zt1OL*^6Z`+0dE{+FsM6V#liqHWnBWxQbtBUTMsK7iY{jFU%SP~oc&7_& zKajA5a(-y&#WmFfGz|67Oqd`>Ot(-DGS9VU8Ch7+dkcfKiJV$e$A; z1_8#y7KzM-uAft^Fo{#(((lg6fbEYDcH)Qf%SB&Sk0sqT`1SLX1SE-KE`)Xd_&x|) z`0;rSFw3JDHJm+F{{LsszrwR|0p2^wzU3njROh5Za-8{Pa2gv2KE3JR$u#__Iy4S@ z2zHNQDEII`Oumz_E&<{=au&@F{qcW~7V-~~gN=v@*h^y2=En84N?QZYxo*{bJqSd-9M2? zx?`bnuau-gf?_bnmuXSlqOxEr7hiA1WMYbZMvjwngA_D1pz?>0R;gSslX_> zjDUa}j-?{*ii)Go;E0NfyW{RC^WUfFH}lPRUH|oe*FV=-N<&U_l5^hodG6<4RtJ{l zu(+lX);Yu?PX#fr++1U1F!M%U8mRNjW|r5FgoD951xY;D{7!m7Nho!W@3suja9gO` ziwhLNBNbG$@3652^NT*+9x!%H%OZWeW*XEc-LlTywm>+TTi+Aw?+(qyRnmj-qBhLk zqRXT0XX3;tD#CMc zrQuWmE$vD0T+x+qA~^Kq*{>uirOy$2VKi8NMt6*AU4`<&4ANS!qC}3u5Ybzb%Tk1L z!IG@tzbwhhKh?mJjLW}h^=DZOR{p19EsRP)v$Y9R1>sD_?dQ@>~#3`8Tr=FXuu zEE&-elZ495hr~Mq+93KUrV4T0{;ih4@x#jduKTB$VY#SgT@5yTMO704*SLL zA31*xat&A$2isf=G6Yx5bT1W9VPA*}T2Ju>THH!88jDobj8#u)eh(_msQ)?Z#HXjvzKMv$VgCGg>=KoY<|12g#c3A4pDwGVKft@CLqw&-9q z^S!l9PSem}UpLZ4tHw`8oitxw0{8QlAz`-SY7CK_<^#`DVP1zmmOvW8#>W@f*Ltd3 z#hAW1V?HVKdV(CRR=0~KH;KhA7X&5C#aU{hFeS4U`F7enoFy4j}pgz#as5h!*JmGGj(r$RHK|T&YsoDoJaFrp32(~#IWv7ISck} zU!U_&l(_eMqC3!`7`h=Pls#5W@>M9?T ziJQmlDkDN_xzDv~joS+*KWYLbI9_4$?-76g(&ZX{OATLj@QRM-Kxcu|iL5Wkc--zQ zDxT5@sUmW{*o@l}7RK#HL}PJSngnt|ym$sh8psIBjK_ft>$UOgb>{J+6mw*b|F zm#y33sQ#W;(Z?6V!Z4=WYH!0-swEPG3Ks=PZbh*OmO~FxB?6Oq{BfL21RD zHcs{#@{l?A+#a5>z=$kD*p0&k47?5~Z%wcB?F}#$M&w3ifTLHpe4Vw{q=9S$Rs3e6J@vm^&=3U0Cgn^GDxv){# za*Cw=F5G5o$aM^--6k)R!?wB?3FSLK)3jr<=68DQrqu!j$q)6DZXv(F&$>>`FPcjw6s?eQAgO&e-Xk{*u7^=gx1?@|8!@PWx(#IwlzxEl z>*`HMcnHA4)uAlL=*(4djN8bUfjgaxnzt^L!|)@E6RuN!~CsRnfCEa>*7?ncsXr3X_V6_GOT-f`i~ikmhFPn zGP22=>hIuLlIW7krmVGzs*)gE8R`nBze1M;n|b0KPE8B2ZiEx=Dxg)hlAB+XGrplMrhF>_?nI_! zE)Oebl}gov$oC*-SYr~JMeW7dn{XRq(*>x4J7g(?fM|#FfrQ^uPXynY8-3+o6l^Hj ztaC?%D%Mk0=N&)TGk%ar6i^FCg4;bWK1;E@$9)nm&;up3nk0U1UHQ+$RvK-e7XFZD zW%;IIbljK(DqRZkAOa<^CbC@kD;NBUl-gt|m!Fc$BktTXG`ODGNLFnI)CsjIQ8kxY zO)?tyho zVdCelIdw720pV1D9Qd}go3w)y&Rd2Pl7io?xj6wiG9wTgQD$nLOk3Oyz2DOy$D}7q zt1E1I4im-irJgm{5&j+Op>L=^RPT%*_!~@gu+9RPNYZVTgL#Qi4BgAh=T&rd0OqH{ zN`ycnTL^jY6CsclP{(6S9!LZ?mSjrk-j-Q{66U?637>2%62OihNV0R|w@3%u0&uPa z=!Pkw{V5)GUfrPNf~KI}tawnS`Q0a6$xboD%nyBZ3;c56wIS0?_=M~3rRrreDd!Vj zOfr?SCP^vpVUh}QIxM%yXJSo1A|Y##=n?u%l1`UHB0o(kM|>~b0A&gl)+x>|oV+Lx z7ho?F+%H%Gk31{5rPe&x-yf97ac;;~cQ%X2_C)>GBy3r}DRBP~I%Dx% zoQPeRWcoag@7~C}_;r9la zuTo~U^|hj(7(~zjTq!%70WF2d3yjjqD38jBS$IP=EkMtmhogY$C^40*GJ-~+CDADE%qaPmAoH}X&w5dX@X_Vb?N>Q3tMEbPHEzgC zPHb&&%mn(att=p=`G@3e78zpSec@aPZ_;Ouj6OE^*Ye)L zh1(er%VtSc7$^p?SC#!;XzG}(pDUv48mijIwwlq;gJE9tR_04()@2!xTaqkd_Ufqhms@z2 z7BT$ttoby)`;S1{f5^<8OCm4fYOaz(HUke#N-~~n;Yzf7(@^#EVl9AY? zL^#&vm*tJF%%g|K!(hZL0A6EmSx8PR5NgP-K&8 zjbs*A``EDDvv$ln-K@*1zdivse=jl0sHkC^f}CS`N<<`em|7u4_-Gn6L|$g7`B}ZR zbi!iR%R-{ZIF~| zGv5a{c!BM2{&U$no#kggKs9g)#7H0wN&?w5+>WNVoRpJUTZuBe~jKB zd0ImS!R$0_ZBFBTAO7?sw7`xp%Po2hrg{}#(kjzt4UA&veywbk@ zXsDIArk0kcC#Nkyv5xi#>K*k7iR6x=>VM6^>X-Jl^tDN*{?vl|%T-Yiv4Uk^U@AAR z^yRAZjMycOR$ey7?+zgnUQ@*trnm-#m0@=_4)&wp2Zz9jO#va-Bwd(tM7uT-E%P5U zKtjgC8w>;85ULdSh#}@Tt9{9)B|N_S*l30d&usI~Hjv$k@>vM93U{ktOU%i(Iav$j zh0yjx!Nyd5O11FxrV|wLNPVOH?oJXX_JYt-)-qi?V>%p8`S}g{>gk{Wzmc_;!?c|L_nswO&*MKR*TGhfd z46O0_gu|OEjq|aK!9WYUP%l}lqr!!YK%sV%J&EbJn_zkBa3_*A1YUSOp0t!gyc7^((SmwlOMDi zOlJ}EW%mm&9S$M|ODJvZl3U%#7rA<%#~9m`u=!4j>dQZ!4C6mjNji-`tflD_7Vo4i zU5vl7hMDPasAltJR-4|4K@oU)vZ8j^MXtNDaZfmWgPrQNK8 z?*si?X_i%gBgATkrP=A+O5EsyKXX6rzhgvn+ss$D*{Q4Q1AJS*=V=K+Ck`skgs(Q2>zbt+hW zdq=BF>Fz9`WVs@uvq{nSjn9UWmoeM>&qIi z)~x|^7X_J)Sy?V&25Qsrgm+4)Rp`zcjnK=Kp{ zb|T|wDR5?LASwZ zm1r#kAjSPdqi#1yr1}q~;*zp@R^M%Km-WOt;IRhKhMVEfB+BF(_zIxY1*(pKaR-!V zY}Ksc5n-v$B1JavoCcXx@RFtkp<}rvRi!dTwU1&>n=?IF!|!CflH*A<881Vk>ov_z z0PrF+?bDPoli@Y7C4p0`B0GS3k*p9c-j%8yqS=zl%1}P6UMhLj*5t`NCHV^qN1j&f ztt_hwtXAKfD8+-wcF@9cPpiF3L)k~;3h{@{@VJcU1k|_2sL6*pa=zV`ccf{u*p}CF z$stgFDkSTzj=$v81OOGIkjSTgayA-31AD&1upiJ2MC|_F;C@MPo&daP}AKa@#($&p_D{T zv&CEAgOP^5$ij#c#uhiNBb(=l_~Qqk-kM9<0!jXzN!r)sxP<51GrW)k~?e@`sfGi1FdURwxdu3(DX1n;l*hyf`*4&4LgKe=ZD zTIdf#&CI1hjvvetifLf0!ub@l>N@q^65cf`tWX$0_e*Z)NSVl}8PPiwvTA)PB;KD&}I=%KiM2h>|~DZq31UK6l`_;W=9Yib0fj9!x0v+7Wv+qR4x; z3cwH&(@3#N4szT?V%m!{$n~@`eVTFYA;s2H zNmT#VKf-uBk62|iqT8+4^7)P>rwjhjPVq~7$X@118->!lEH&^d<+$vn^FPCrdM_8ru06lj_gR%R| zX%ZFCf9Y~Wktbwv7pBFF8)C+}S$D`N8@^n80bfr0$9&4+B+6bF&Y{6Y)=37oz>Fri zjCsdFbyqjD=S*o_Hf>#?4Im(sl}(zbvWT<8^$>~mUi`G7e8N2Jw;4SE0nF$lNcTR; zr&@JI)_>5lZB$2=GCQx9E^Q0JQ?yekH`a?w0zKX2?Hpx3D=?M*q}Ju52VvgIB)LL$ zezGtveIF4{Q(@*jJbqBhyOEdfeSz9Rj0`F9$i3JBP#sU#r9r?3B2kXNukAuRKiEw+ zD5Mi<>&$df4o0aHY3tnZhB;)j=insZE;AQ-%$SIg!}<;LlNgdQyn8@I?vC%H9?j(RQnKb!33C8eyd0y!hTs?zEP z5bbZDW?W2p^y^sa;DKM$*NaCmjbfUJ{{f-aC+2cKQ#V^vvLcP{WK1<+-4L97QET6(3rnHZF zh(kuE>m@%w_c82^s!wE7#a(MWeEana?xC0@xAI^%V`QnI^B(3NklZnjuSQiJLrn+f z!qNaFcZPvjUo;=EX$iuWz+S-r;TsWB%UM(&YzbLaM2Evcp`q+ZvVLwB$50+=Yx!MW zFUIK~WfhxLF1sz?Ww(#5ung(%qxD)Pn6cqD@ccff*WOuF9bDFmCKG-bkl=@r)!`{l z$*2qlhCxm!#_5w&c^j_PJ};+cs+&~X_sgqG(Rfu>G^NoNOY!E1krjlv((ftq7qR|q zKBH1IoklTC$$KxD2KtzH_4;KlF}(LeS;;w?LADJOfaoA|&P}zG$1vg0k{l8#)KAHw zx~x-jzyK3;6K}2*V;1|3o5!imlwJ&v6Pxo3kUx(b#VV^MAGscfwIlSrKu>EQ-lh@c zV28WCt*|1BHn#+WhKow|9SY2uu?Y?$TL3UvkQ^nMFQ2_RWBgIjK)5kPY?$m#%YcL# zhG}?7BW4A8@#g4nS_O7fkRmC|8 z-5H9IlX+=jV@s^H)709TtABx|T9me_3h1AK_9i0EFwAH z@&+Kj$Xi0aftcMYC7}2K0ttHIKonZq*QPd$93M&H=4x>$)8XzH=xJv1_0$}442BYL zN|Ak0#Q6c%KMg!wihGh{x! zdszrqDppBs*WA)n6FR+m5LtrVA<}2mWlKHS^Ftt`vEDg@*iSHMu%=PuAMBwh)mafh zgA9+I5y*9hqMpG}Z>!gwhYs6r^@J&^g1o0Hhy=S?KSrrTr`O<^xF;8B4}~f59GqTZ z55>1xeTB}qDY)vUmq~|*!hs@kw@6*eEc$c$S5-aMyPSd)YXKjVy?b{j#DT8bp=^@I zN&Q}bZx8uX7Wn2;I{g<_G2*<#txNB}rCbkm(WwW7<#lJ1KNc`(;IPOmpf3PzE;AA zf5vB)>IR<4&(oiE>`9KJ=G7&~vGct1TvkU*Tg#$k8s9iCbsLYURnjd8HNlkmE#f2tnoHBF9l0 z3gqeQ5uL&SajJ!{yzOlL1-`7a3_D0W@;`4Dwo{ zV_01&)~#a0!DT}rQlie}ds>z}nQi?pOK3WvXs%a(VV*!n2&2bl9CLOlY?B{WYa%yKYsiADp= zje0=_982*%d7ecJS85eUFn%CE&t{>Zhwn`^*h*^4;PBNLX@MXGQ$Zm2N)~;V7n};7 z7wU(yL~=(tRYlI%id(_vbFMFCmL}xHI%42wvBjYnLN+e@J$J`ZX433N#Z(*5cg$Y( zrgCbsrv8eRj**#MC~n0-fUGYA()g2`vgxgg&zI9EMGtDJnv3fV&U_-Ab9Q6xcczCj z+I(Q#z=wvN*%uxjAMFVagICptzHKnXTTiAW!}h@C5grnJC-or&p-(;F@7;Nj3)JLfCQs&3r?T+1_2*WVH~AmQ;(PPD zew2}lnarCB*LpJ=BEWOuljZ|uH9UyE8f>^6mQzXNEerIJJ95AXXH0eYYTcGhbt*rb zvG#!CIVtuYi&bkAtMgp~uTxI3&0JstyrADYHr1VNa{!V>p$h#wn`buc&dCCkrd6Fo z*umIfXJ`bjo6H&!vmyUgfjSMg=j$%|?F8A;^s5~&Nev$n3@B-0~|_D-d= z1Q=Lg$Rj2W_!0fr>al)Oo znDp{)_qR?m?pjrhrB>mPV|l58fJUx@>Tts>lLewsz<=7Lg~X4%~!%w-(rpHwVeKOfuOn6QF^w3JI2xUFjJpE^k5Ci?^K4Pty=;~R=$L4qC1!){Y>Fn4K3*of(&5q1+q zB!eETBX_BKOI}XbVE+s}hKI0s*9hBsuFebETkW>C78E!*n8n=G5pF83vLGKUr>)i8 z8gRFo>x%12(#wVUMSYkty4=7kTUuJ|@QG~oxG0Ue1vWhabAab6+sTbnMt?Z>sQ%SF z!vZn*Tf7`TW{^Ks$x+sIMw@+;eAUAbOd?IQEhmWCc&rZHE@UymW@tjOE~4;&bENu{ z2HspMylzX1W*Jg45!e&7zJZ{tP?TIv%X|B=>okzPHfcZ5kT5ykMAiGorRdBt2=6Gj z@(6P=g_@%F61WO+F=^;@XbTW`1xFLxp z%gS1+1&4*=ruym;?YH%KM_|rk8JXD^Z1Q^;Z5CPUTB;OKyH6fH_;=t?W$z1QVR?B; zJ?o!seO{3FE0}^y1Kq&Qh+30_rZm_*;p=X7WTlEUfYkExAtUzm^8NcL->2ljJnFc5 zb-D`nQPw<2vR+b*5=~LG?}~u;*94}6Y~sDRzt%%9#d%K4(e9pu;9Y5&w>5-GF;?$j zT~mT-A?MPWTp4)l;%aFP$mGN1sp3$A)(Fv{bpArCqEfNN{H5KsWsNMU52S{>)^3DY7QT63BWpAG;gC~f~R+4q5 zjOyk671xB1r8cj60gLV-@4$6-6bdbvEx>a))DC6fj%rM;z|9I{^tB9}BsQa|&Y$_+ zZM-7S>I9Psq+$J{dHg{mlWp{GoJ0+pW-cU0WVLS?4vUW6&adw$gA(W8G&BGYrk4p> zKVLlBAVykklxv^HwRwwxWwwEz2v@3#J*9lRmCTor>ClJUZ3ADMOUkQZ)6;hPaL?(j z-J~W@JdAVI(>if~FwiZ5poE7wDe^F#25xOM(2WsGWWEqHEnu6lm+8>{m_-WdN)*-g zf$@jJL1Qp#tb~UXh?>fs*;J4>RbDcF5JdnvG$-&gpKV|iLUA-S5aUe;bIEy8l4l}; z48}1gaso*Eu!ek<~GA;k=3JN=IK9X*D1dDdM#D!3;~iLGv@ zLvJ)XZ=;TVa{6(}t@=AbP-1wsD5$bClnYA8ENsu}Lt_2Xa;$t2X%3<~KzWNmSTNE< zp2%lzo=MHB7$w$}Nz{DbY|@>meNy(8o$rr$KAlgF%BAgt-Ow8%NxSw3%E{b6OOR*! zMT9WiJtui(Wywl`P-(ZQNn>w2%)v6QQW<9j!yqAh4lFSRC=_TWk&y3cRX;|kHud9k zxqbR|xQc`s#S-RAM-j@GaKH%JX~;MmtUss$i-_=_av{N2l>9b)G?L&O_=Ax!8mtDb}TrR%Drge3BUf78EKhYq>b^ zmXhG3n!#Me{{)Q@Qp#XT7wx_4gZdJT$%0s=Fvkv>P_Kmsa}8f8wdpd(MY}l-IzajW zWPq`Vv7X$pIP+uUK_XYsZ6G9ml&#H$;a_dC)a8PDQeQ4sP7z~t-%O=vEgIew2 zf*j0ZEp|{q5I@2ycj?vZ#41Q|2^hPlp$DFYe>toYP*9)Go&jl+QS6zk=fL0_^ zm?C-P$jliO(QZV}L(KXlNOHJLEVaY*fn1)Jo7I{Gr$|CSze(AcwkR0JzC> ziQB&-^W|w*^!lhjdziH}it!t0Wu*qDEik(>@H~6_Z zg_8vw*En7%$wI+ng__9-EFxsu7Um(K4IUkNBaxVTn@CbqI80@F6z!B<(9SGhFd;0r z*)sj44JaulMl<=oezM+z5R-75RT5wLd_3LOVkE|N++rXNcK48b6WBU5dTYOHQTl&*|c=n;GK0a$f|B$I(z zjY_SKTBY79#;R#HxfFkq14sunbKy339eATykJz%F@y8*Xx%@umf1k?-wzwR;wd749 z`HJW-2}c#Cxf0TmWa`W$7BlCQGG9Us6KC?mzJAW+qL+otxQ>(Ga09P;*u*ot=Ssn- z#nzD^qE3=INx*a7lazosn_rq}q7v;TTWC3Q zt1-;feYl~2?FJecYDJ}ps!Fk|ubR8I^;i+@w%V$zMR1;|GEjpTj)LDbG-w#oVw%o6 z;3U<*NV`ag!99i-t^RJUsu+o<2v>7HFQAH*FHBcz=Xh?QRK-5z=2ZA#5j;sCwZp8+ zz3O_%@s61|T&g>Ywt!mzBlA!R>{U_CU{Y z&Yt}yf3t?%=2q5ZtGp2I3GL=9tX)VR^K+s#NxYA~p)ljH#BFr@rxT7?U2i|^+ux`c{{2NeN#=(3X^R*#W*@byDi^R0 zcqr=E$@DkZs6S%VXFCpmLlR~D2u!<0);kz}M@&68L~S)4nj}U!Y0*8)M+_YB;u@^*53r}Rx%?=pP8cd z%o-i>?~Kc$QpgK!%p{Vr0KpD3feeOm%Vj640LBH`V5hn=6Dirir#jFQBP;-PTkxN9 z1iAvK$|?e#*s?j90Sil|zgb}*5>3yR?3)1h;!G-@~N5SOE4N|ndwA* zKo|Tp4KyGKcYq^C1Qz&r4HCOl6@|@gxTnwXPO|=1F>~u;8g6q$x)2fOZkM`KbNDrX zr46u~!)ch`0av4q*}-vfQFkoJiz%O$s*L?yywo76@Hqb>PVI)6WR=bNkWkDKxtLe8 zm^ZVSJ;gALX37Ht+9<*fbZFzR>R{GYN;*@jaDEsl=)!#vq;U2kVhmc z2&N*T=&y}Z|eAAeh#8?`?ipp~#Gjv+Arp#8i6i`3T zKVO(1SN?62KaS)osX<@y4R7=twzR>I8nTdAw*KRwK@7SOl-p%2m>@+U=AgHH3FEG^ z%QAAPisfu|FxPaV04yM)BIZq!PD%4o(z41N1M}vrvSnz5kMMp8Jg&Ui*0_>Pl|WX1 zp(72pjKI4Zk_Gm;Du&Bh}0^ zfqQWoXIz+>ic$iRp*|CS`ze1CzIi2)C%T5>vrkfd`I0A5Q}2B;PHcbo$w|@@c8o7? zSTZJ1we1~?*F@hPo6N3ZPtj?&EIB32*uVFbDCePfPbHY2V#mey*O!b-+#l>6Ck5ZW zJDym?*F7aIk&mvH&1AQ(VP@a+WJywW0k2eESLrKNG%Z|Js$9@BR+_wcgjc3&xzSga zQuV@;r&Bujk3OyLdFaB^Q~ICMozX;J51dI|{`%JIwJSatn>qFBEqhN*y_Uc2diFYL z-Fn^i?6&p#jpkR^Pm2}Am8Wg1^p~f{=0${tU7bgS)9+aRcVWipdaEe&z|M#$>)<^{ zMA`R$aN)V^)lV`TbB?}Jx6ydw-EA99kA3y(#@s(qNjI9urS&&j&J?Y#v_6+JUTJ%& zVA3Yh*;%%6`wmsnCidJs^tW$y9+kMxFaJj3zOdd#^5So4eJStb{rfhnKYj4E&G}zG znRHXZH?)6__xrzIH>dFD&&FTY{5tQwn?(Pl;osZCb*K2mee{-X71_d5b+b(;`Sq37 z7fb4^oafinS9>;WJyTuq-3PDDDH5c=UQ?n7HfU$+95r8jAV?Q<8cJg1nqWbLuWc`a)d zZ_I1kTG})}xIOsB{El6*H|BSaZf{x;I>3E7cuaX!?;{tL3p!7Jx8th*(~4Uc_K&3} zFC08un!G4_w(-`5OV0=YylB~l?YAyoap}OH7q56R{nmw%ug~4ue9ep0yd}vOb#E?N zqiBCKuyTF+AKq-a@#0yPaI-gle%rRHviZRsi`LC=-`)H4TPb%8eV@`fy1}Uq9oV&0 z-F5Kbadr2z4}JJ{_k*VmxAh!7+jLv+i3>Ysh9CX>>8tv>e*At)f9!SH?fv7jxwj9T z(cOA`EOulFN`^QQa z4}EyY^?Tfx4xZ4i`23L{v?E_X>zeBQ_EPQ4mAp4^J+t!1v28Q8KV3R8^_pKVd~~O1 z_?O8`u8lKnJtLmfQnqAOe0B0WtLRhP_O4E_X3V+{SVTZ8t8fRjAJxnoU7Or8Z{|96 z_?f`<+N?GPujsU>^RvUx{Ji5n(Q|JczE8CJo+o>bz9jm4&O>J_ z@{PxCZp_}Req-^*-s3}^j~P##AHM3$i3hKp^1#FIZ@V|naB=qv)1#LTJ!X3B^AB}< zAN%su^E-Dd;!<2v^23$NubK^Xfr_>~w(EUv z;Z@Nqen0{S{)crqMT5niQpDkJh98?Xig@n%eD3+yE6?Bm3Qzg?5gr~aP>HFlM}8@T zyMk$XI)%Sod;O(@a*8S(PMao%;c z{-XalweLTf^zVmo{QDvB7!Hv)srT`<^KCf99~Tt5-nwbplmBsu4?pJrb}kcduk*?w z*6oi|e)cubfJX~dQgjj%mMkoK3eICi#g4B*rg1O=PEZHb-e%x-$iYFc5P^RfB(**N z@k?HPJO1C_FxQn|^284b$}N8s81QJp|L?~8Ki^;FE$5SPrQlLfWpdPKNHB19nkFt( z)8If4hf1Lqikc#Q%S^Bw!1OB`P>oGmq32v$^d3EUqp{YgK1bh<(8IPtz2~YGdd>_1 z4>eoxEgS=f7hSpDgQo>^^uHfb?^&^OWzN5zY5mtTqv(8qGW7QUFP?7u z*V8#(>;LLk+yC|4w!RqmTL0f)w&VZfxz2w*_kaDGUH{wXE?P;c%76Q4jlB^T*DmL0 z!dgs70bw&H6ftb8(_yt@?&GxP1>tEJaM*$^NS<)nTk;(3&Q@D{yT#IOakges@9-(A I##d+gKXWz(7Qxv=B|JP@#i+Z#XQHQ{biinkG%H0< z4~&Zj?gwG|wh#5ddCe6y`hH+@Ri`>@Q1Z3h2SRwr{cvJ~f+7dTW@e~l zWFVRy0#^7&VI(O<9*{_Nn>z{s140mHQCJW}aAsQ&wR2z3P)Xe}QYHo%1HOJ zGi!gB?j7lw1{9!RS)%0&0|UQVV)6e*8#5f~F)AJJ=~)j(lTRD!sqsUF3Lc;hN%<=P z3+rzK`_t|a0V4`y8^!W%NG#ki!@?HY_?Cy2rDpT>^Rpfw3hI>t_GdTd=jUhV(dTD( zRNw>D?wJS*fWpHX9BqSv`cFCF9c8p#002bHe+L90Edv(-fJC%b({k1Nrod)^ z=3rvZuIXGJ~v-0xt zGPAHTv#~M$gJ5*=vUfH1WVCl7|4)(st0Q6VV(M(|=xXg?Px`N}v5AA5s~{QKzmER< z`p2b@Pp+M< zO=ay}&7J>o_FwG(yTSi&#D8fCF#nsb|B=4`tj&K||0#|Tf&lY>7g`7bRgx|q01yGl zN{Fg?LY(=*XXr^b`fprhal%ua!Pk+~q~CzLbA(9$VrEr9U~o5uP?82Qd<_Ix^!Mv=5boA`Kcxt5u<{G;rxbfE3K6y{ zbSM9gyqt?0qX5?XCiq|8C4>nGu-oDL?<)K0R98+gvTAW#2R=;_+ZJ%nKmus5Od(#t z;R!2H3%E`oo1qxDU%N;QEdo=l9_w*z^u!9O!g=dZ8ZG7^elaE}-H$Y@YqyqhQ9$*| zWI1RdCBg8^lB;WrTREq@?W(W$t#Q#>L#moh`ZYo&RiZ3GGsk?v?324-ymMo~23CwJ zN>!qxU7=GH@)^P=Gn?zeXdb9dL8m_+YCHmE^u_L|6h$iV>Ok;Tdhf<5~p z53BUc2F!u2;!YVb1GGr8J^L)rCRLFcm#9WQ>DJ9-(bFcbp0{B5>?z)qwRF{aY}wjX z=Fq27#u!)!xfptuusZxzV}}F)>nPZ>b=?-X^$OT|^*s0{@XFVfoZn$Sa*q7w-)~#* z+KT5PUHag*$1Q5N6-r&?cExD(7sb!{%1pSShZOXhi*8!y2vD-yEzYTC(!*tf^vI~q zvbI|tYDqm;_`3$N*G(?cK-p6d-&2ErvjzB^fQ?=#=mn@I!S7TITH~?p2Z#T7*0=XD z;4Rra4Z2Q#V%D12-QzDA(*-;e0G`K6#)P9!1#DCX55G0l%)9CVrfqXk&^_^_&gFRpv1u%N>v^ih~ zQh`kDA>(;Qwbtr}V1i+gv<(XSNeI~i5RHA z2Lzn+R__oPxo7qK`ktclQhm^ex3|6^J7_OhOy;4sUzSbqmRJI_Y#nT^u1iXTABKKB zek%>S5r)XyB9Q3D$*Dt(tqmPD%jw2oC;^Ddp^OCN&9h?{R1igEc`SU~>I6_GDZe@# zL+x<88Iq>nCkE*4Y*aYZE(eo*_FYo{ihmJQ{FqD`$2IcPGkmDNNCevb5kmPAc=2M5_W4jm%PtX~t(TBT7qNwzgb!T~LmOMC zL8&X6WJetNZ-1p zB?&v#+lgA}h=OWOCtQW(GX%eOgEW1hp;n+;u1*b%iX{Ecw4gTd&dHGLCmRRS%C1O3 z@l73(0ZNKrS7CXjhF1tu7Vj*3>`?j}r$njO7dapr$aFBaNS#08SYWD8*^I8SYeyk< z?DLj?al`5koh5KjiIK}YWgeLmz{pM4qCG;l{=yR$HPBWU__2p9T%ijh0#-dEkFQ5l@EhY z{EI3Rf8y&HmW~C{Sa;$<+u5l~eKr|AtTDwyr1U!a)Y-m=O;0xCH9J_7-z zPiX?!Xytc*rrAtce0hrN$`qWM)Vvf5LLuizfviV->FzGn%yrst^@-tfJ$3C zWrU|pJhyL^y$0Qo(Zn`s>B3AX$Yu4)s6}59Ef#w@0E{A}b2m#ATrl{Mx1rspId5O@ z_DDp(`z!w~NT<~_c5`(keOD?*q3gLHamvk5 z9wt&K?}ni}V}&bWP_y#P2lKF`l&s#XPt(c`PP; z4zaa61g#O%yx_R$O~E*3Hp;6dmEziT(@w?hKu;wG#Alp;6F`v<6vF8zxV)Zv?tYFs zC}^aPBcJ1%(YqjD@&i2d+l+^5_VPkU%wpk0&+{#f3L@s75(u@0+(*;TW)1K2atz{?=Yoos#tLAerHm`Dg0gOpwk{Rg<{FTRSGzbA)c z^n~fN1dt%ZMgj;l+h;6trApwQ5g zw|&sfBtGkd&=w1?qT6V6Ie03>o@e(xTkLp|aa&xbKa-JOX+Z!?Jv8 zzUCg>Qpij)T8N~TmosWn$*W#c*!@aC%hSIcb4h@cv&Vj9?5|F~hSPZE(YvlkviX_3 z{e&!VE7au^oqoa=YJU{mFT1>9eh+$v#vD`#9{fDmh_?RW(649HZcW!@N_685xTVrM zS?i7O^Q{51wMt*)d)4dYRXreR^yuu@+oZ)|xeVXcz3Ma_ z&@^2WDk+C%I+Ok$!`=%Y%@f;dWneD z>I{?R+E#}8oj-n`*pvs$PZzqr94QXOv|rE zH_g7CJ#to$fv4`c`Tidk4?vuy0CPK7r6uxxD+Yr9~(S9m|mu1T$$tLd!g-LbhN0&ZwrsutwZ zJtznNy~ePdTmsPq?btSoXMNayPzqM`HzK_MdQW zooS#6YX=dQ)o3N-l+M}v4>do1_w56!_*Yb(LOTy_q)E_al+|PUcZQ-&n#L?7MViqqoGyk6jVdIxCe2N1B^?=O_Np&P{(cN!~Ta{ zzai1Y<$I53YSj3Asvbw+{W^=|ifGnl9ZGQkDga1*W0}>C!*XZmxZ&seBiw%siti?# zMwaDUUNCygNvEXZYd7hV&o|xeX7gewc?Rs7#4Qni^ln9pGTMa(RYWVk;X96V1Ej8buLaOD%vnL z99G^{b!x~h6hH7;9KUEF+C?TZLSsXSlHrA`K`;?rX{!iL;VS(G&23tzfvt@6KeYHH z)fDa2TFSHb58JYhFr}baQnDqsLMSBYYrhf2bFof1COCEEJN_^XUruT2LuWkC@@z*Y zjWu_4WAQ9tsph`A4r48?Sm8r*$_AnzqQj@j@_ExGjdU2bN-vgN3YK*i zan*-mI{XH&S4)axDB<-85^>-@G`zP0A#uE+>;@pk8YjR5jyz+ibUfbsogNJl+ndOKtWWoG=qiO*)7uP!5GT^O3Ds4Y6kK!F zyI*V*FzmR1SU!oFZ=EN)c|}VVruHt@N)^R^&zA9|bzD!-ibsS|_{P%=di_||b6t?% z-_aO8!FA>~$JTlg5;V^uI!-|+Zy?dEC~=HKC#RvcTE$gW==cp^#c%+#-pzRNZUmg_ zcrLOW5))&Yv*Oq#N)&i17sh<8;u46HxV)=zo#Tgf8EX%}=L@6d;&ew=Fr1Z}K$F>6 zF$~h7N*k^JjOdl*4X9+=%Dxl^cv%;thzvpfp(I$5ddX(BCP7#+MdrFt_-pJbgOpkae- z>?Jyiy5)&vLR>^cFwqYR16qkB z>42%v1NXxEJ@8*Cjp_bgYvjdTr4HR`{5|c1wY)SA;tD&}XKC>>8j2pi#GRBUfC8dB z-+vH^5LO&l!rpvAs}yv>5aA>S5CnQ^XNZ%))#|kR+Q`Fl+_7b-D$-WlBFv__KCHgh z+5>o&bCSQ~16fbocQ8^hqa*?d%3WQHTE>r2mO^7}s#sMmg*7gu+AJP`j=XTOm;!f7 ze+7{2aky@_I2~{W8o0P6EnVi#3F4iaptZP%RS9-xkVTIgj!Z;@=3tTf%B1Wm6)|zW z{$@l0dV~ul_|$Q8Acf|J&{}FT_#Mczxt-3x06xqCex=q-Oza!thJm?2ViwK_P?6RL z3)N_!dtOPaVDEQeltS5V>;%sxyJvO%^@a2^(;sL;%0U0Up%0ObATKJXUJgTUobjUe z=w6XQ7?!^*djqS_wGU7{%dzZ}g4ps${vc~OhEH-&$*uEZkJw3uUMCWv#+pELm+yg~ zDh~3^S7qKFqQLea=ooHoFyRnR(_(AsA{Sj=@~W=FuuN?Po2Y#i^+Y4?hyu0nU18u2 zW2swpagEeb|FN>Kf-p6=K_5BxV})UFcFCkSk({%)>0Z;G*yr~^)08=zTvf$Mdm}vfLX@p!EdSV(x2=D!n@oEiq5k-tg#hY6$08JKqrk}k2`s| z_EBG};V>nt>La;Bd$4Ssz){v9Q-X=W{Wa+p%#P@=-qXP02f6Qu(DRJNXrDfA379(P_5=sdG(nHo8n_4nmF5>>CX$|_}OD76l*mTza=74Mf{a_ zvG|Q#uiK3pS2;tF{zfJhZsMS3vc`y1~^|@S9ALcY|aq8{i zIwf;%PkG3u_1T14apq9i51zTa^jbTs#h`;Mj&+# z1-*Iqy~kEAQo?RQ%}CG|bYqRx2ICB5`j^}*RXWi#uJ59B0a#N4Ryvtm=}^DRTf*zk z$6b;1N~}|UfacsVfsCwm`{xl{%^F_L=n0U^DC%1ro zl%hsXa@hb(UFn@S%rA9E>wIIdb>4JkCH29#3JQiVB%5VjD-%}X2@mF2At}65 zCAJ{}1*=LXe}7?vQozP0B371(OVJ2R=QO$H zG6kcUm316D(EJ`V6WO6WAL@GnxUTCHlGXDYgelCeu$W8mb2W|b_rZ&dz0{&etl2;5 zyTlRX*n1T&Doh^ccs?vc{|lIB8Od6jH{}Q8-xp0ZF5b$H{<1EhsI1_h z8G&g!=&!?&B+(6Im?_XDCDSct@jpAZnbUzN@LXKTUG!4)7A+_DSmG%lUYIX%acnHG z)n|Pj^|z+;Mp?XcaC?Rp;~%f$H*v4p!!Hq+ynf!H1XCGrZXF{?0DHd%K`AS1kU9~~ ziiYI0NxTBRcOm)2e0erZCSB!X^*(5;wv1>&_)7ya8xRrmrZlvu(~YAe`^AQ$P{tOz zV`V+6CdD7hKK4;=(604Y#~`z)Eq06V#U#OF=n>qGQw%oPg0);lOb-tRL;k{sfPNZR zO5MeXNBblrM)4NJ#i+!nWlo%uve%tBt1WBR<0$mtuZYU3OjG+wXk(>NpX30MvmQV6 zdjD-)a};#h8!%~DJcA0&mT%1j$j^xKGT*944mL8p+chGYOmikZLEZ9fF|lwVJkqL) zJmopky`gO8!X@;n`wTPBGh~ae0g!3gZ?Rn@7r)5qy{ojCnGgi+9NqqA2EDGcTG4O` z&6UOE6qsyjCHfI6@xZ@a7&<9xj&ST)i!;3{flfC!o>@JmGf^fvH<9qz2JY?v0Js(L ze#Na;#;x>^C7Ff?NVh#bY+vY-HtYjORIyP$IPgL51m>n%R*Pg;@GfN4%#&EG737(d zA5@1sd)FFZWZTmo_ktjh;Hew^BP>rqpG-E#1&uVv+?*=fpDi|aE-JT#*|ooU*5AAU zNhh}58~hx~>%lG9eMaj1e!m=gg$3)BtkJ`MAFs&E<}f9L#L?2x)O1le#8}H+K^Fe* zU4F@-!HSH>Jeyaz@xn=1!~Zgi{;r9)K%JmI&f~xrEx3hY8bW>HU%q;T<-p0M!t~iq zIut*^n!lKqx@Q-p=dsf{b$7ecjLl8>@Zwn=V{Y`)G zoek8A)yx1x+Xu8>xH3>HE|r_23iON;FUu!_egr{dL6_rLT(i8t~>qqwl0v!amfv z-g@g;0k*v@yUPyid*p#vt*YV;%LLMLvTJ_!^^|la-T5; zLRgLunG!P9SxAJnV1reE!wS(daI_!FKEt7wio3}!o5G=~Xv0sO+w?Qg+UBDU)Dn=C zhJAg37pD;I5>2uqPUMbHBog28J;-Mf3q&6{$nkSw129-b3I_>Ow?Id!fCvgq*^<-qEi-I*5_(CWWIzhNo^&5=Zxl0JqXhz)AC26&aG(D0 z=8E~j|F=!|&&4Yr3C}lO?ShJl^t`wAIBV@Co26SF;XQEYo^7uL$`8y9zER;4m%nuR z`!Nh8(K<@KhZh}A^a`lwkX64WVZ%j^z1qb$w#5*?oQ}2Yaxj?COHYaT9{#PavX{Z# zR8ZjGpjp_@k)LTt2Q7D5#CGQ063L)bkHlaz<^QQ-bE$C|sHtcVBO6-WwV7qppBtMs zgZeaf1O6(yRz>R*Ld$gtZ z?4t4vL;Dw|mLf;L6;&eRO4fV01%QSDT8D>J=Yl46^jRFclYwMJGK3OMgKR<KpYEy_9%`1tZzLd(rvTh#4#)!B-!x?0NM?x^ zt2%Re?cSS^2|`5jM1}*+7j@Gk9ku5!ku6gW_rkag-Ctwa-(hx|7OAGs9MBeaSt3r1 zt##{G8oQX8rZCNz8tW%JU57U+@?o41zPv;&hgcrJ%j`yE<~?tYSH z2J;ctU*{mu9aOj5z{y8xQWVf=Y=;5cHor0zLO{;XmaKNSs*b5X-+W%K4L^2GO)ZavXvg0j4ryMb0(_cF;MF|ew=&FvbBg=ZFrPnMw!&DU z3>7W1Ono`0rSrZZq(9-~UK}Oj>go~s&2GQ+jGBC|3x4uCOglPHRg(AgTJ~KV_&>%-^Fd2c86wT>~*b@K`@i9d=_6-DQYKp zkvm~yKL|;^`H5hXDD|hKZhzoxD~SmHY}CL`8zzOPZ){imd?m?+b|;`n zMus)K$bWbpwbp{dwMTtH9;+i7EtT;N_tupi{h8r>7Y-5GgNrZF<e^())bAa}o~YiCo);8M}JkH7=})_-UfGT4>!sIf3UL#$|{dTbCmVWHHUtHvA^kvOPS z@c8`&;)~U_dKsFfO%&8LWCsj1#xS8}?x8*Vh3n6T83{03tX?emywUABJ*545NUlIe`FH=?qvQdQ|gIpCyg=rhwW_B$Sou7=aqmM5kEr zad_mCRDGj?>t^LDEPR&fCTKIO8;78b2;NlJ-%aXPOAtWys1ogMaj-}0=$TDf^rW74 zX}%k{))+an>*P2}zPM#_t zqD1BEV4350cn+8SvfNdi`ta~eB1FA0!cE;&+jbkRls4$pb?z1y_j_pKi>OxtdW>Zu z8#0<>F`v(f8n}!FJ(J|X``F~zSAkJK(A&EF!DH=bgR-w>n80+gYSP$o!pxLW?>d?y zgda}k3d?-CP8xbFLpN6bH9A?>vhyD`HY+skbuU)pR%kN8%BA_!5x3JuWC+k?f9D>` z3IpVTsM=&fHES60W1X64Z(@~B0u#>5lkvUlt0yHP1F(Kb)y{uBw>xN&)gg!-7tz~^ z`XHY5_l1!$s6@cLmM_9CFk>zFi!gFEw`X>tBD*wU0RJSXMQ#%Z*^i8R=s#dCB{w6$ zpwWcaUKeHfY)uw+p2Jrq-rZz94H&S0C>6a*I2Mqdp49r94XN1-_@f7eS zOBm5(&0at1L1&o}9ApK0=I{MmZ9YeqIB{;a-+$X&+EwgDR}C-G)bOyUgkk*xSkIvO z!!vURdIe@0FgJk)-bf+S@nd9t2io9)r7?ckLRwz*Yth)vZ3la)Eo;=DgGERkGa7-p z(|-nYfD?6E2L|s66~#`%V6ffA3rhI58YkBb)VOTUWh77W;{FBk3%MzC(Y*TpagI6+ zQ{Zr-qLcScHTuTxuS5b-*cDs_6m&Z9jTLU;>>64|#I`wlA%@V@-JQwJKYbRZ%-xC* zPC^a+OFP2u&>#;c~6&kGd99_~ms1(bNGexrEN@6?^V1bs|NkGJNdFZiLJj zNbg&2G5lEsQ8)yIY_63|Hl9?`!Ar90@j)qWrJfM}L=I2-2^+$$vJzj`*=twN`S2v{ zcvX5oK5?}Gx~xI`G%0uQ;I1_h2^b{_iSx0w*juMCXzx$oVLg^Zq*01v85~%H3ksPn zgO$mAgG3RYSD0!4Ic%QKXg?eM+#kDcNs7NSYQ|ZWKC-x@^go1j_brU-2 z0WI`@H(-;ShIj!>1dtciU+r5`va*yD3xm0U>ycFl!o!N%1t85L;u9Xn4GA7?Q-gs| zVW&3=LirC1=S@V5b=gZbwpjA{+vMZr-ok@G1L0C~cJ1A{ok>&Wu(6(B%>xPO24R~j zval-Squjk>jFUbPX!Rs#q%-ALJrSf6;~B0*8nS-ku+9w#7qfh@_TN260%7tv=J}dk zr%$xJ!-zG8vxR#c5cJdDL1Y+-9x6Zmj46!Lsx+8I{|*a|F8K2WjJ(<0avK1M zap*ge2!cU4(Mh9SZLLwXWN92I^ub9bA5yjD#!${9xen*lAOJD4!iyOOuX!7SaB zAhc{P=B{{(54JQbD<6cagwFvQzn{F`eNZ=b8idvUw@-$~J?7ynYR3QpI8>^dhQ*|B zAp|EsNHU6K;1#vCEyvkli?*;p&|g2GKvX$Av5wR9SlPqs)C1>it-|H3ll^`<~0_b*7Ffibf z+R z770D_wq0-jA# z1<|r&Gd5mXi%5L(o*4v>Z)fGLmx<_SA7eYwtDNw<&zaStsjOq7z`G2W#!nS~v@Ux7 zz13GOt7-S*Y#ni<3)69B%Ia<2& zJM{J?jj?lEt@rdU0<|X|XOjSH+e7&|0xYkhsuPg|0d95(mNUuF>swE>mvwG+0N+uy z>qak^mgPhBuOTOHlOM_B6IYw71a#5+eXOL|X^s#t9A{rpO>~#@2MA(R0*OR|&0&5o z)C|3}Eh~T>qYFKftgE{+DN^Ez`Prsf;e>4AE+`Z{3_=lebP&`DZC|JdIi+?f9oxVC z*(;21>Rp1~s!JVC?E)n;m(Cy|yOTGpWwIQ33tnOHU>#nZuOopwQS#Y(bh#o2I&i}O z&@0f0xp{o{VEr^w$9zj+M2qJVxEFQCq-SDI{iK}RjLv~crB_ZSqD1J2CK zgQ;E&J0A$Z;K^5i(fb39^;9fWz2vE%E&C$mnd5;`+Xep+f+6AmPz&Ms(Eqw_l4RRB zy}lxg*hUTBH=@H(zSQ&1{0ce}5Su3ABX$ql(E>dlj;zg=!8lh-JBi_8KruBm5duoj zFhM7`X*2- zFwl6O(wyAq6xicJQKW2k@nXc^b1@HPys%t`7FKTJ9<{)GnJY(6-;Ft>id|+xwt741 zn5pL(|G=h;UANb8WW&C<79(w5S$bzvZY$dsU%X>Ncj>Nsu}j_RhDJ?C5<=To$Sm{Q zWGI8RWB;-VLNIJ1y|lq)lID7MhwCf(}JJ``HQwJhVtvy*Ob4 z86DUAVs#W;aLF5S=y|@su%*q~CON=RIlBuGk=WpoQjsAdVRf3J-@Bh)%X_@L&yfnZF6B5+H&Q#h0~J|R-p#MmaN z9^GPKShsyujp8FGayv*FJu`l}OKh5a?@{F+eEwjj@hAR6ue)|xwQwL*~*5TijMf5d1J(RM( zk2Cr5KmB^nwbyr`WhA>f`+x3877Cl5z!FG9*w^}U59`^sn9~E*P~h${tfT#8)5y=^ zk%O=$DXJbFt4~Yl#hK9|dqRw$!m#N(?H%5rT@3EA_PStr%DA}$2+2_iTKt1U6BUCf zQ(9WN9uLh;&@R2nNc7QCtwK3ACB^0`lld(41m{I&HxB@3Q7oL!WxTiJ8IGHdOJcNn zC&6Uwn84qmXlpngo8gAHxdir%>)8^OLW_JO*PBlg7e_nqi8Am^u!Y6;!_O-ii!_Je zbMcuv#He|f=O`oU6aX1NF82C606eu@^cnQS5A@c~!_=DHKk1(zcZd#nES=V74n6W= zKY64-gYO_*S~-YG5EUucn!2&>wgQX;P=PeC`^A(^ej}FGuT1rxy=D-wKpy8EGqOhD zqoP)mpJE2r@OI`W15~nBv1tKJWK9EWL*oTBS{CKRx5hH!NAh2BizzZb1){TgO@SHq zSN(^5Jrccm^ivPyI91FV6s3&3s1c8mqxsXUk3q{qv_vVk#v*R;t~R!lj?j`7rnU%N zZV#p-y0_I`^)JZ_s%{4NOj}o?@FG~LC+TC|GC4o)j!yhX8a#K$ai}|1VwVXUGMG|P zt7#8)qThS~Y6u*hAPOQHMkUwmuO~)(6%-O}y!dCMHXXu75})*NHl?GaxSP3{xmT59 zdEqu}>7tig*k3lG?zaf8)`&h1@_SN`2SgJgD;QTq|KA;Kx_k*F680yCTyi|`RLf~***aA+cKDsb#>Jae5pu=OukgS<_WepCa`ds0v8zq z8jyS+ThRQkN>@#T$P||d_>%x;1h=|#EfIP%gP#H5PfIZ{L9;znoEwSb3p04#>d29N zhMIP}1gn2#ZWN?|e>G4}7iwMn_1sWQ*8vqROJh@qk_WJJFiqhHb8ot>XzeoJGW^vz z)7p7O_ve2;L!&VgB))l3 z*D}h6s%>gcB}3RiKvbc)GMZ?#mcot`SYA8$J4N%^q%(3KN0gO8CI%FJ6buH~eXmJSOEZJ{ z4KnZQd}!=b+NV#;ug^K&mlUZ*9&k;VMJDAt(AIK1*+8h)%i7_E#XuEi#-P_UT;wI0{h z^}Ob6=?nEjsJV=F|BGF~!g!iybYh-%oCfPP(ro zxQJHaP90trf6uzdpvI(2ak1}Rm0YLaH}cqyfitdNwVD#FUmA-nts6)icW>4RLf(Hhp+DXf^pgS&~| zR{mPUkyZpJGJQ6y7yHh9f2C`Id^wb<|iru=Xa3|i2{nHgeKr4pYNM( zz2`?(Y|fCeh>=hIH}7VCoRj)&-8AVyU6OF04>+cb=vVF4bnk_^E1MMHYKMZA+FDkQ zErpY*;G(1O^t!(%8nNeWSkuL?J@u~U8My?Kgy2v_>wC(tqFA7h5qLiwqOu+J`%bq2 zQHi|T9Hke6j5vRqW*6xBxBl!R8nem)#bXZiOv=6h2F)H2ASO1h_H+$U?d*+VuczGv z+vg=J;@Y)UfJb26m=%al+?MB1Qif{KqzOTWS2=WR1lf_Ih$Z-IK!$1b zahDqtM{W8~nOg-&m+7G&*!#Cb`{x&CJEA8=VBewFIJ`$%zo7t<-X#rgFLCVS0U4^I zVZ@aChBsJfync@T_fGY-eF*J65pTu_Ta9`Sf!JiNg;;Bt)Yxn#_IdCp4OSy*5v z)sGb|SC5dj&Z2YRffSjIcM>%hN|=@dUtdEnhPD@$)ucug-)8D9B3E6eQ@TQmt=@*N zTHsRHsN5GV2NxnA$ZPRFRLUe>>_j7mt^bSupY#~$FFb1augpAEdHI+ehP)aHtl_?B zWNgdv;uPbw`4F@9hY=zWxtp7K@Y8dhDk5isAEZT#P5u6KvfIJ9Ru)wVWWL)E>8i@i%-aHYC znPBzpH@Xppjn>uAvv%!n39ILxcJWmn+Otc z#Ss+mX-_-g(vMrAYA3s?YB2Um13``i>V|~l?NP{jFLoYC1M4Fba_a=jEfQv)U-(mA zDuKel`|0mW_ifsX z`u61d4n?5}!|KimJa?)^_+0CHVl0J5q^MGc1dGzbcP$X$&Sam+>R`=Was1Izcc3^D z`VZ44Qsx_NZgz>K7PZb?E21f1ed4-1>W(9|yq<0D9|-Gs4!P3~l+{t^lJ!K4=mto7 zjwd#BaoaheA|YWdcmCZ6njyj%#U9OU2G_l=X+PE)@v&g%-^v&v0v`4p=vRELSaW|W zQTlHUuZg&JC=eKAIhIr8C_eB^#*O+}nOr)}fQXyTvUAN}r7{7*L;;lB9Tn6?>@2@} z^0TsZS%>_TB|IR5cXD7ZR-tve^e+ocnuvI4A)Hq2#zSaR{PCkaUKuyz6)7n^Grt^V zN?##CQ9SK1igc8?1nJF-GGuWL$Cye7-GKMDQ?e0{m2VI;t6@5)n}W2heSd=?3g8P# zo|O-S8%Bi4?EXlVvc*}bhgIG>E|||2^UZl~8Xuh0bH|o1hSr}fnhBqr5*j~GKbc_X zNnR}^eJ~{4Yz*cEHY2omPV&NDDvI_7pby1{I#45F4GTcI9X(P zp^h;bkGl$lE>s08Tcw<>f}V{195+T-@2lo!7_E1(O_)6CWpS7ZFM;9cOjcDHdNRsF zCz?=X=DhdxI+2(D3H|Plzk2u2(G%s&ZKgQe*Q0?#5C@$_Pv!@3ao--d#6S8aZaTkM zJB55L)+o6%_L@%2-;C^Qgr(O;Xm91U1d~F~jh{VmSeV$grcyzfQWYnN^29a6>49aX z$j`|*0uPbFdcE$d4VYcUdqDsy4%XuH?eq95H20J+miv`fX!u_J+c94dttH#0pv8%L z#v=zXZfue z#|=kn-k$ z<)6uyTz=j|K>k6VF3Wys3UDDsKM{V607^Rp*)(|P`Fjwth1k;fIvSGO>sX1AuiTrC zDPFsFkx9ZyMY)?q>cydm=~tUxE|1^R`|X`WrZZd0#DJ*%H9n7LHB%9O4- zTyy}HXCh%4uedye*g}CrvOJRT4$sU$WmJRw!69PpgyoWT-tVJzoB8(GeiMXXyEA>n znfvc5F<*N3r3ZjxWFwau-1wZC!L%euY8m|K*R(rddA}K6G-&O&w_gs1`~Af8xKSbD zI1U+#Uww0vUo7^3)eO2bHd6+|PV|T@ISqMJ)o-$llQf%bx`fNl9OI4lmf=nvvmUE7 z9~1XY^q{vwNm35&tyb!y@j|Wnox7#}Y@$Xn&O=NhM5kW)F_?Km{Fv~lr3v6i zGTQqCZOWNjuKe4jKjXK@gs#I>KcmZEN$CY~U{8l5*knmP#Se6g=C`T*6QymR;SEYN zTDowyJttUmFXN(X#G6tcp$=*FC;cLJki~EGONPm?r7lg|7$GUO#5EF^KHW$w;#Y;c zL7KeZ{$jj2G4sdsR!976Pq0BhwV1U5;c(@qGNyylR~WzLaw&333d~RMQ8kDoN&`D{ zJ+|#dE%#I*3y3A+Y+X2$)FtmCy)ZrW`{%tjj}B!Lun2hV zX6F3rwOenHT*gx`qe^)F_;}4>IA&Vas%VNO9G_5-5TGmVO&s_p@na^+JwPvYn6NWM z*F8z3TN5mOBEv4LqN;F=x?+?7wW<73g0JSPm+47~4(X|f$_{!6Q>>mjty3w+tpRE5 z{3EGYEgAZ_h|T)W4>TxRrN;Z=;6`>RY0|stoymL%1*X4SbHG|k_i5B3WW4Qo*xV}n zGch>n6xQAMUM-Zbc;t(Jd=T9C2iCU z_#Nm-;?)JU`BdUv-?SdU0g^IgJDdJQmF$I= z<;SzIN=HuN;n*U1botFpK0Q7_dNxbPJA?>LFUV8`C7`J*Erv`K~p9^CDG$6Y>ADYUJ)48domRL8r6C_RNYg1z*XV9is+8e zftX>%^8*zfXR(9!i$CDPQ{9Ujv0Qa2shA5&b2Y9!O0MuS@YXqiaXq4!6s1E?E&o)^ z-@FSU9|w~>kSmP_T??Ob;=nm9xJw1_6}S&u1C38%$7MVdPCrSBcqM?N9G>!TI_cW?Yh}Pbh?4RrUFdg^uz@7WY#L3+fbQro{ zRI)^O_O}-P5oR`nJ2h;cF%tY6IxX8~v5J1O1MFO09C25&U($`EM2GzeR2kYJsh)kw zZ$+N1lKF>3U4g@m$C0C-sC+wdHMxW#OB>qQKGPmS-sF<3lZDtyLN-%6KlE*s#{z zko+;$Gd*rP^ZyS3TtK70I4@g-0p<|ersr*xw@9+KqQ3lS@Waw*-hBY8pbds^Iy;gKHK5M5RYVWqc zA41^r4iFLd@$o!t#D*rKi)7sVGDp)C1~3})y0i1mUtc;$vTGE)1c@USZxW?3mrouI zT^7bO`0+LKGd+38fn$(2c-+V%h>h@brv)f#-Fk|?gZ=sS)8$OFbl2V9X`jHFVd zai0HYz*ut|2G{x6u61c(zS03W@m!t6dl+)31Dxq_0rA(+7q{Z6k3;3U!Zfvf&m8i# z6_N|NuGt^DH`I?{BB;s9E8yjzI1MV%aM79d{iDIFk`}_+;1t6IkozDw_8h|%avVu)BW<|VmB`;h&bC^ZIPu|AhOZ+{VC zHCTJ1!YVhRV+jmS{(H{da^8B@RBj3$VKF~`cHsY$Tkef#J}*S9hM0&(Lm91bTkGmt zgei_+7&AemGSDA4Dv9U}@RFe3UD^TBc@zkRV2W!e0su3N?GEl;Y=)%I6awccpUl&T z#6<^=iet>IDZ4^9orLp8Lr3J?rE*yXEo;wruC7z!F6ZL{b3)y3J03y|%@>t?7*?pc zXxvWt1EM2SM0KgETL1t+07*naRGhxs8ugn2Abv%3*&f+Zkd=-DFN-L`K-LV~ z#Xq3$GbWXKkU71$yj)87;gzSaJS;QJ7bl`lLoq0!2Uw*Bk_Wk-^X9qy{&?y~Ixu3; zYQP%pcVGjB@>X@;*aw}%Iuj6$)SM4NXOp)7i)k(XUP_O%3UiLr`saEMe6Cy2VB{Q; z<%0V{mhI}w0_-=UI1&;q%F*DJO86h|Ej>{T6m&kP5h%JGK2U^L7RB`6L5KW?_zX89 zqERZKL`9FB*^`CD+L*&PHh~sbs6gMK)ouK7^-}(&?MkZZ)nL&#}!U3CS zaC5qMw=``99s3fK?fNy_+k2m6Jx>D~ z2Tn|~AfA{p4`6`&iqKn5e9rm)_1Yj0?<|0~2c76%hzOPc6yVhk7A&LDWNB4s*Zqsq zvWUf`m86M-nmToGm6u{F>9ghgsz>qi>4*axKes9U35EA+MoTRD=Uyv2%|ixLU{H5*egLOFr?EK> z#5d?e>fi#raDqs~o2~f_tPTFDxeSIi!h^c*Lg?;E67S_Z*XqzTcG+vpN4PGwaT(Hn z67)mPz1udeUygtm{laOZQfUs~IlwfQILHu7 z#1s=TMw(BUk^`9fuN8;L@@WO73{TNCdEyH)C>y)}uO>g_&+;f;`=U}%?`F&=&Kqj1 zsEBf51fqeZe6rx> zI@9O3#$0CeMQ96JmMN~2!zAyM_>%hfs&WP8fE;LA5N5~q7QhmF68cfa=p;I^){6Tf zQDrH~|H7-=5N7A=czbf)W2HT~or?mZegHhq0?-K8h9!gat1cC%z#D8T@}Y!Kk#zdA zMeOG3Lc^eC(zNzn0{~{nTRqs>RlKs8i^Up`>!jqwvNK%xH0_MW8l^yLpsswDT#%zp zKF`pdk4kpz{OB>jj}C!5UGD&h8k_oFU-D?k2Ro4>*NgBv()mRaN}T94cdioU;+r56 zI!B3VKYD;bkr17mU?3CYh;ymLsiM`|4gw*SDxV61_nq~>2WjFIv8HPf1ac1B#^p!W zvIBNFIuJWatX1V8%)y&2TDC@Nw1fb>sAKMMe{yunn&DWTARKL)nyA914|Hn z2}lm03EBgWA&?DaAEj(|IcN!+n@zY_InU_WUFl2XKm?mWcc#ZxRJ)L{ zu>B@37mBw@gbHT%@Wat%*uqFmU5CM*U|370bFEFk0z)O8hAb|m%usN z2oQ5Q17lT_xMRYT%sa;%G2wS|leTw#i71d`gNLM99bibyPT>X6-bb;pqQPvC1@+(z zFjXj0%KRDrnF;d_#|19f6h)}^x_+z3Yisri8EItRi~6s=L-hk;zmN%+OwlNj_x^~@ zo|cE*RdbFb95yHW;f|Qv_zx9RXnhU^nEKYP>;fP2J-h04;#}y)%%8fKF-J;6#zvEb zxWgD$J=tY=?W?7(71uxc8Tvs74XcAScU=xnbMiU}UcMTF62L&jMIGG;Jm@&H5IJ#IG=n*^%#dweanxXFCR}b7V^A{t; z?ptN9XxlriKK*@;v^-<-A3@t6MGFBC3YcWYP!Gmmie5T~mS}QzPA}2sBtPRmk~NrJ z);06|5g33!=3|yXA^;L5j+hX?WHEH=4~ZVKi>^zBz40<1QR_1?9WpRc50Ypb*}p-Y z`Hkcsx8j5ZXT{#Mdob1tZzCKfO7|6SE6XXJn-EO{4)|zfC8p@tQNmVEFl*98csxy( z_gJ?$F9&a|9E0;Yr=)vwKf=$;%moPc#zbRW(DhSJ;=^*?(Y}=PlN)w}>mj_AOY?lWfrIJ(`_rX0gg`1;49JmV{s7wk zGV&%T`+5fgfQaU)=9|#tRgFR3Mg#@cfQS`k22f3nfEGN z^xtA09P*EHO^G`^A02^|@Kerb#H&iSNnr!^O4J@zm-ftKX57S=<&HR)Io-vwe>5_NaXWHdjTerULkruma%dv*IU7c*EzyR?j~oj6 zYlQ29voSaVT}ivUe+=wjF7d6{KD2d=#ww-NnYe;TX&8WY{|L!4kHJOfpBt+O*aNa~ zfUPtzW(N*seSxz+Cf5 zyc}PUp-sZ-jP;&Sel;THqmnmuO~;>}2ozR6eDh7eTz4OE#l1xo;~)^0l>1_Y#3P=$ z#f*b*FTu%J%5iBS<_bB*!q6$X)RhpO*Y~H2)&cN}7zl$MgI1yKv)CuO$(shh=)cM< zOUz_x-|99XLGOolv{OS|d|#D zirvqPe$9Ih8R?jgu$ytNN;gh@0j`kEAS~rHVULje(omAkzlxF;$Sw5U)`bUQPk_cm zhfxV4mp&ij{igt4)<1lZ2+w6rouya{rwy*+$-WDz>vEGF@*MyMuBmu0hBTVk(Hk(8 zc6H67n>NHkjI(h+Lmmwm^`sl&WJN_0)g#HfY5Q?{h;FSbyKY0k{~6}6@lVOa z08;nQ`fV-`u8|KC3jqPwN$fI_&HWA_Jlvc=dyasYQVmmWPy%w@3V1nZ|3#=t_d(^j z5Ca!1%soZ4G4(u39^ri`8J9oM=Hlt58@hF8pTWR+ta;+4W6fbFD0nHN(}uAo^xbkb zabRhfY6NLp;f8mN#a%E59Mg~mppn55V@ztulEb(U?U1-TtO1LmZajxh9K4%Ejna}P zicIk2p@W4dd+_LkoDA>i=$%QBdpQW0*8+oua4%ZDdnHW!q_7zphduAlLR|Vh>ll`t z^0bfBx(8lmH_^#F)GwU07|j8pV0)mr`Qb2R`l%zjhe*&g>PVY;lV!?<9!zUD_htPc z$z(KgXEyf(OvQf7Trl%ZX+%4RUEN=m_{_YC1EK@add>tyFliI6>tU)#0^3Eju4G>R zV4Es!wNvLtn-{vSN1Xx-{sG*FagPe*?vjSpAn#^t);G zmI008R9JmW=wL?r>&ASf&MkAkky9RFDY$Gg%Vjr1>XSUD3*R>ibK@8 zt@%c`fo-#`*28{6-oa`;XTzam*UrSYrO3<(DQ}^7FEAI(zEP@53-i+w#gZyUC9(g4 z2CT)M2rOqVfw5~>k(Rm{qgzB}4E)heB^i>7(ubXzaSJvkXMcBRA4QdDVg6sa$BOQ2 z3vg{qsJEATqLH>;(0DopKZ!c;Y@e@uvklk_n;2rvm{+8IE7wDh!T?cyBO@PjbSRin z;JE~`!Z(~-RXEvfbEnPy#C9i z%rlitB91opH$uDbmi4f*=~H@_v|QL5spLTY#5ngr^zV`Tr$^+?)0X>obTk{M>m+br zdWJEBwGX^hY8WAr9Fi9HZ0@%pqO;7@IqD2St}D|+cT`hjP+X?2%f%c9Q5xG?5i~!G zLx{he79k+(CE-+Pe}+D4Zebj;*U?m0o38hNAhE>wyo&Bz1W`SR)XRHyUAMGM+D@fj z1+Bwo(^o^5*%;N{4F9i~2Om`I#k@y-0NaN%F1W9QDHjneU3JPXG{xoYeox9R#Sg34}a?Q*3lN{I5I^?e(7(*UZGzCE>aD`&qo4hlL** zjDGCRTyBABa?dl9u6hf+h8o<kndD5#_+x zRvg}3{oPpB|qzc5lzF_ujDW2R_*6Tut z52hg3Wl+Ik*oWDe$>6%}VxOQ0Ogo2)&A}(bjx}%U+cXYlvvdfVg{YK2EZ@xXbj#D2 z(Yt*2+HpYIo_XFY80`B}AHbd60*pE07Y!);*o(=n9)K~|>Bp&ZQkzM|6fDW9kz9XjPsPef)v8fUO3L3dHc3NxiE@cb8~PkXooQjCjSH57a*?3g#u%B`MM-96plR5zHE zdu4g^@(OUP2avGb)|PX$5tPMA?_XtxqqLu_mRyiQshi;UM1%$-66}!%BO5jzzJB?;~Pbrw%R#pkD8x04|lNn?5g( zSp8XkM10(LUyI4T=FhjGm4$Ki%A5YFw1hH~$0a8YF4CAC^0U{+PT01D$t#V99`q4? z=U#WM$rF?gAhJ@|%>5ocdz!iMN_84gcSw~UYz-;Gic`jDJ-=xFc=Y_GF#K(fd2`-H z{W+PgD}9J3IJZncg6^J&(Wl&_${MLaxzlk9(=qFEa?HPcwF&RjfJ>!_+8`fFB*l`7 z&{hN|)kUOu01I~gr&A6l=8?IY`D3ySDT!7^a&9-J1}x>Wy$~Z$S`n2-rd(lD9o0Dq z2zYvDuJgYL8S1?%7p?x>`_9Ulo@JW>v^%Q!jLIo6E7$c%1MlT>ub8x}bB}SZlO*=C zKd|#T5xsLcihmHF*NQ}Mri{3Z=CX@08*nq{iTOo=H*lfh;Oz*;%6NuuEVB|?=0n@7_VKHw~5{r)e-=;(2@7LDdjcqxNRfL&KK#CXcR1G!(bn2WB}pO^)Hor z28;HWvD&d|*)j#{jB`JR`*!zVkK81>^$qxORfR`dO!=%<4 zPsC?5A{k|JFYXvmtu)D?zyD(Mr(!#d^#uA)bb9YWO{5y@f_IBXPWocgXD!h zcUUKms0}3*QH}d9p3)K5TJ6HJ88JdD?772G)(9U65oLlZ*iepnOa7SOG>l>$AsO2d zl9$rnR-nIDkqW|_2jy{az(vn{7R%H`UJS{#a%vQI{$omIc0&h##MgtubFVzLI*zb? zC;XYY{L}L;HW$pfTnbfR=H7GV!9@X!<{yu?ogq<57=v3oqyD_Ez7>fWx-P;Sb$7oC z+L>*(D11*s?XoXuFX)d5k*lKxlax`RX~ufPUSFwAz)&OTHY!74OP=tCV#7uJn|I6G z_(nVu&w;s-7E@ajtUcUsr@{Kj!+1tUBkw(6qDxN2e$XI^R4fs^LuF73=aPI`t)FM! zE-wu<)50rRrCX=A5D{0~Hwt8F-*fDFtq?xT9K3wS9w2V@G;$lN3OfTZnTj4{Z&}UX zYP%@F(u@#8#cs6ysny_xr%x~uROEm;M@ z>Ry7~R1SxQ`P%|s4iHRh>zuAsZWTUA&ys|Da>6nk-%*nBzu?rmdGK#WRh~!Uz~ZAO7?&9X_`AgdZ2aEe52B81zW>RJ26*3#f#9~D4NN|8b{B? zKaO=|m(z(q_%e>^GTjBb>5!8}6Lxu-pMePNp-dDSWhG|5Y8B`DNd=iSocyZ(;0^%g zfFw?t3aosw-d%W6vD27$ zV8D7Wtmvw9Be8UmDerB7yf>3UM?b1=TMa+hb^tFNG;-j8DXieB0x!2G4J2|z3tsv% zBfNotcO~>)+3_!d_ZS^GNOD8k84a`>ocAKLGJSLzyjCn<)2oI8uRXAbq;YKL8pTNJ z7LG2Y-$1F9L|ws+acTJ|B}~@^n8*>C1d)1}>zI@-JVfJa6gw`Uu~NqqC@wPGQQZ*wMK3dn;OTcx7RmdEnlwfWKsw@ z1OZg?^F``j6&SD+3SKTq$YQ@=&pX~sne!Wb>%L12ED%ftmNgo(km%y!^o{|DT z1RwOi>r?WvMcYpOK9CaPIk2q+Lxhw?Ad0hnqRnK3g|!wV0+$a2TZ@`+Fj}EC5Uz1_ zdMb$*7VM({<@)8k%L7v|d*m&ab4+_VP0E3-juX7i9xwN&kV=Mz2LM~)leWz{Vzq4m1h>x5_Cd8gOOUMH*z zrNO}9V0aLj6=h+tN;%`r1y^%PpsFIhzzdz1;JRzxFl@K^J&3-? zh#$FIVk7JHPHbVYGE+Zm$E;#^&tG65`=&YSgh%8N>eKs=Rp*b`JU#>$lhAs-W+3Jg zB#X)fx7u5jz1Uv^@a^+2=vE3){;u;feUw;|RDJ{4r4lE%5AB&a^ROFro|jWa+>v0) zl|T2v>(p>wo#W+V`1|G~C}H|GfbbZ#G-GFjdXXW8AaNhPxwyQt8_WI*sYA-(CFf<{ z{Xw{4Z#j4_vIUY6oSRy8)P{E9!V zB(z@dO%r0SUgv2sV#F$0z$B#nUNwxj4)?mf-U;Abj71>8k_WFXq~*)&*R6z=c7vgqc0!6XkS(+G>K9MY`+w~!FPB%%iR%qF_G+|MU=9f6Fmxs}& zyA0sa=g=&4v&5EJri8s)>!m&R&^tWp;u6FSzXy!2i&%1ofmiA`72usKg$sCVtj*L* zsq>Qa7PB`Cco{oeql(sRH0rZxfp)~(?s*2q9gU5$6}%!cBi{jhNo~+|^wU9*i9RJC zTf~|Uuyu2Z2XJ|Mq_h&9sfMC@>S%3|oWms;JQ$@qI@G_j_MXab?2QT$kcpmKI>*c7 z5HwJPCK2Oyx1{;TaY`bUzHO;Ba%~ zgoj1F@dicaax;Y;kj6c;p3pogx1r?UPOPPYIVOw2hG{T1ITLpc+BfM@00-?mAeM7-w@mm zE3xV(6`3+Athn;t@Y^X_+t$sxd;`kBQ~(#_=ks@%+>ps~g{z8c7Y5jQURs-TsrhE; zP=Z%S^}67Nv5L+avgp6(4pZ>Pnj=qn7G-3_7u@O4cP&?I@SwT{NaIkULK#E#o#DSg zzQh_ammtLxnVsMC9Jq$jJq$ULqc)oOGSnyAxvB{%j#JvGktPMMPlFJv7}^@(?4Tt^ zFmPPefmNTCBH};bdP&Y5oiuN*3h_#t$ZZ@P!gOWLLy5Y32*LY`woo5cjnFzt1A6e3 zaxm@l8{_8B0HE?lmIT%;OrUORR}{HvOck{rBAk~!P*3Nj0WJ+oGo3TC`MKDk=QuNN z!ke=1P)En$6j5m3_$Mt*lg3Hwp(GoZ0j?2SBuiqAm}9aSlpno6;sI>$ce7XqC>B;^ zZcirtwxTXX=u+Ajq3?)(c$n{3?7$D%GkM{_o>lIv9%D@yQU_mk7ANqr;8H*j0RS|Z zUDW$4fHSo5rMBYZ0DMKfmlJ`HX&nTI0jLP%7u3&MbpBQ6R-!(@v!|tnvAIHv6 zKNmAQaAo9zg-{(>D&sqINC*(2sxMY`orJTRuwpT^856?X=m)dK(SU|I>?ASQ7h#FjjP=}5sGp0pT@$Rr-XmTS>S;y4V6gCx{lAs#&$|0z>4aMg1?1=+%d zM0EJe*}2^H9M09gC3l4{=tmZyY|VQUh}s)UC2UNs-Zk>dTU9ve<;eRr5x>K@yXd-T zOfKm6ypNkrn>Op6UCo*0qm(0#Sn&+5JSl?%jZR_t*E|~W6K#a&Wm^~b2E=W?M($JY zTjA`it3r;P_b5mVv-KJ7K@4M5e6rzoAS<}%VAFCf!|KuT&kkO#v;g;r(mo-LJJ366lNvSH>uFdNPUAVk!(SggeZIEXRVds}ro!)IbMm4bsG4dWO)-k$Qff;6s0wLM+u zet`cC9U%Io2V=uFgv9L0A)*HM*u7xZ3JeYz{OhP}-^ETBE$-YJApahmE=FR7kS;i| z_k@09swfRE!M=1|=)3{WI|$y;7E64RG)`I<`-jk0vPUVYu|E`JuggQkUL3XIrg#7o za(7^tc_z<&O~6E=5A{u!yLx}HQhXV;7FPfq6&6dTg22f%Ed@70Ed^=Hnf(j;1rd{+ zuDO}cB0I`C`sGGV_f%P;oj=21!$+d5vwTs;>E-)=rmP1?>DXE!t>Z;kGL+0?IzTXg zRbus)x}VT{;b|q@-<@JEoXx=Nm7sa{aVzFB0G$Vm^N{vWV!8;8A@goA-3}*>6hq9S zT7}}wvd*ink<|l)Eo<;Qz6HwBTX|c`ZbrbZX5ya%4y}5p<1NJC!+{IsC0*bx zK-cX|T?9U#deu!*=0XgjRz%~8iYp_hi>57oUISwKl)F5DX?|MyOla6 zub93BL|x(!n-^z3P-9cYYS+7ZR*y}mUk2~St_u;>R8Ff4|2^-cX3>qCOJ;Te=$$in z@6gP0h3D`t&-hbCy$1((t1PSWzp25`Mm-G6;*_`*yGuV6oHJu-3J_?1leTB*5B~O z!oUrQ4k6+~x-;18sENBGAaQS`sATn6*L@iz_o4zgLLOl*q6cFiVUzl3RF>uQ)??lnA??1x$*=2nlF~@ltjpj8YK7> z6D-0oIrn5aV4Wv=Z`geAerOTal}0PaU~5mfRu<}7oGH%1Kd2ou>pc&loldK9(vS)a ze)O>gEG?%HuGldneGV*WY$!+36eUEmiqIB&F`3i{u+9g^0Q|p| z+iT~0Mbw2MYthY{;Qp%s$W7p^0ksBUe4>&JOvt#s4XcW}{*J{`6U~vwzb%|sJ*=Xh z96X<-1t^V?mYIs)4npIqE+%<8c#m>m4@{C+W3t!_-c@s4AHdcsJeNMV2y(9TTYRn_ zM+p>#m!t^CIZfkztb8HLYl=i?byY)MRSa3-WzspH0rOhuLv9pcGB`lg3`@-Wq3Nn) zJI>?a-Y>vSk@bpt9~GlpTQK&j0x6oqs3n6GkP+d$b%1%<5KTpN;%JpC>-yC z7<}447HLAY;gs1)j4-_N)Kk%7hSg(J&YwWoALx`!D3>Jps~+q)pD+ur-%<*-&-*o~ z`g(IG<%DBScpm2;fmn|(06RSK-Ib$z?$Kspr*-Oxt~>LwT=@Js_rhSdKX;_5gBbW#iktherKClfbF{M+Y&&o6GbCq^uv#1YXsf^M4 zJo68LNvBryd~D`{i~2UKpy%vjp8bu2z!G`j#`58a5lZMe_wRU-rRtg(QC)g!iPx07 zBlOizNFF=v5KW+C$EnkW-01wwtVC3r6y`;voG!?1DFtLQ7-3UK%VQe9I=Yw0g{i|b zxy4IOF84Hm$`=R%ShwBeRG9~^I`wKU+e<;bt>%t)>WVujxaM+$Ss)Cg@kT2V%fX#R zuW~NIj)1pHU8dnv#A^99__3-jU}=~%P1+`nYl@}~dq?FPQ%E2UVv88lL|Tca;lNG) zVS=x;S&H6&QwmFPd(ACsiS}S-Pp3Z`xEB7RRu3LG{x!VDQ+mHdItY0wto$hyT+`N0 zS@=KYUW%IpLs+6-JJO%3a?`(p=<*j>e|cxQ^{XD^JhTfj9ga&t&WYy!8#h4S0O-Q- z{O!ZzWb4ip!n|VFcJF+M(wq{b$P4iKViTuyJnsff8U0WmBUpm~-l? zy%%wim3*|*OX;PkEVdeep+B?wd1ER#Q4rL=Z%|RFa5QECGSTT1-P#q(cW9 zOm_)j`JXn6ZrV^PVnG^Kt5bYK*HOnV$K|^fPvX?kdw{U|u5#f|nx6ErVOfTptK@aj zoV$UkOJuIkp^vhYh2EQj5$bPns!)J&eQz*890wr#duU+_0J*=e?GnE ztt|yH`*D*hOpb`C$Hm<0CQQ*$ACfF4umDQm-Cao~y2jwYSv%wu}2c=Qj{d-d1aFM48kf2y%~`YrA_% zR&io9IT2kEnNsPWg4o{>KnZu0v$Y>-bPkvHk++>HyZ8|h1V&sEPVI}*x%Mlt)BQY| zdu~FkEn1KbMB?+Wco~FHy=x#X2GjhEmvkO#Tyvgp4QIzj_nYjPKsU@DgqrovGKXWj zi~}LVz7D2=YXQ&f3CWWAqksBhz~ZCuOE~|E_jr`VfW_OQ34vE_qB30(N1u>UMhsaP z=@TbpB772$IL;pC@~Ju>9QXXgYz(B*kS74lCq#7jg!@!DTjhC*(byZI9Sf?>f5~Zz z(Pe9P<;L=n)w6XMy?gutB3iN_Cq__9Sgk*X1rFcPG5`$Aq01T9957y$wJK-3&oX)J zc+ZWm9cKYn8Qm^-KlN1gJKX{CoP!LFW*$J^j$Q7Hb(}S*0?8 zN7U4K@Mj|}Kw5UZJ=Bkx>@9)yshq!Wz#`}N%wOd+v6e84WB_K?7VIhaZ}8L2svcZT zbLBe49f*>TMef3YY#&hBS-`^ibeh?H?b|H}H5F`mP85D3rHWId%U=~S2Z~!#|A*@1TSY#Xg z&;&<8KR^bRpo|cfAyH=7Lh`m=YX1M6y503UUDeC$eoIm}dA(Fu-R0bK?pf|R=dv-_{qpm$V+4c+odh+{@Fd7w#WkCytUPL1Y}_B`UT zXRpWo_b3;R0t*agy}emDM(`xU4zuRkc1bxyCyR203kHev;uWvq@Ug4R+!cS4?{@nR!CG<7tZBw zf{|z|4rnGZQa>$Kbb{!?)s*_INwv+8o1(1V!CE-!Bw{TABZz-xeTQ0|DM7f!_Oxghr z@mbOoX^X|untX|Vi9m{?2Pw`LbUkE7L|+})zMX3HU>O{eW&*Dq^9j?mUBzg-8D7++ ze5C=62(NB`59qUDVlQ9Or1$-F~>#VYzIn`gzUj!penzB*q9YY|Bb2L7+Ys z3Z1JN2rBrZTs`v0%HXL|2PfK>YUF9PGgxC&CVMG7sQw;f;XRN=LJT8R8ytim5(ymV za-0;nL26^v6eXCrGzJ0|XNkj}caY$v&Sw<_O#PQzl%nB0E*<4Olh!IMl_4A}BQlH| zr}uksK}=`5)TmTpONHCi<0O@0=#`4OYp`ERbSqWJL9HW0&O8Wl^3MU^;!;o0m<4fi zL>sodAcCTh%i1leQcncAuDNNt{zO6fu0B9JqjDs8BENF-=>qmKrI}$NRlSz^&s~;wo{L z#q*AV_}uR6zol_Fs%h%Kq$Sc63u&y1g*O~AN;PU(dpi3Je*Tll?%L&wR}4JIC*aK` z4<2oMqHt7%GGLQJ?dfp$>E`}vrO5Q2vN7o09Wq2f{SY7$#v!2Gn z@_jK>F~A_qxkIXfKHS*_eYox8CgZL!H{U!$B!wsa?WvHaN?s<>aUr?)hS31gTr3Po zRv_+W;kq@#ETAqVVSIZTFuc?pcYcqk1J-A{WJB@AenWt`N+Kify5@5}E>|f&`z%ZJ zU(k}$R8`_L$59;t+mUxHXU?9-MQno;7(Ry(l6|1hI|vW5zgXAYU8_eH-_MX*HYCFVeMhSeEae09q#>{;wdyO07P z$>+SqNCo>lbk#9jh~m;%2v}Sd+yxHn&I)NDoC)pNBdVl!yXbMFG_~VhWv*05z(GPX zW`iJ)1~!ywrub(lY?2Bm*^VKfD8G(_e{vWAxY}Lj*OBP`DG2J95LZ?Rb4 z%oohknLNJ4Lm+%ae_+fafuA@H$==N^q95vshhc5I01g~GT9L8Rd|lcurwj9@M<3P6 z`O>5Z@!Y?Qo?lj!2FGV`0gVx^dl~kJ=Sh1c;waslH*vur_gw@3YBw4IXrHGqXY7GJId1QbU-!C4Qu3nUR-u9lXF?~>>dG(=jm zB}ilTTEu$wqU=>XZtvI^d0U9DL)mS1BC8S36i&g^gyobd6zPf^%G<-O<3$#2J%f?E6lXc3=wI?J+Vv_krkyS{j4(+M70I|#9lI_ zhR6aU&LF*wZ0Wcl;??to!?Mj%sB9*YH^|hz$)8Q;OM0b#tHR+$qRDXHRhRD zRCHduR*fp9pNX&53j&-&>vTh$6%~eJSEPqriPk|Y#7z;jKto!Xd9*Yhm#D#B3MUqO zNAWv3p^YHQ=GMbMcasLIM{%d(O86bv$~{bVKRd!Kne!RBYxCwX)bHIs_+nNp@5&Ez` zn}YSYN49o6Z@hWy4|?>eHGJ6mDLPZcW?ER+n^g!C#P(Ep$-uxx(fJh^@fKhnG$4aq zJ}!-ifW^%vGl;84h^<0m$hq=XRUkeCl}I~|`!jixG*yHBqIv=j_EW@YvtC`n_=AWB z!8@Wq(@4XelBOoOC1%IV%}gf}MO>B(kDc-|!VIs|1|pm;*alPXFzzr&man5Y28_yXpI7{3OH#COyL1+ZxGtkd5 zKh1kzh1^~4ax@Z$JxxoQB{J8!58R0i6`ZqLF&e~0mql5>z#%W6{V{2B+O%*34vCM> z|14?eW}IH|TZqW40niLBJXhC4M9<$FH#t}CBG)P<)&{F!TbAQL^YQ{%npgY!Fc$?i zb_MU3&%R?tCVQXRr&kvk8;NA^W$*(Ri2H6t5I978io{vu+zw9*tD+Uvjcn#2vrmua z&034#e=G;K+W5VJ5U@BYI0{@`-B!)zUncdZ!1=ov)^kr{F{MNk%l&w(V~q1II$c{a`Kt zR&V01{H|`HSb)445S{-HA~7uT7=37=1OaFH&Mur6Wxgf#TE~yjXW}2~2muH76J-#@ z(NuC@!P)QyH>zo*DJH5qPvOK~&<+Y|i8NJ1sf~cykUjQLH=ysOJU}*#mufr0J}_$| zA3Y7a@ZSh`nmRQcp-D{EQtTpiYdW(YM8%fFbwM#`LE&yiTrKlT-LvWgvi(9rC|AU& z1tOj>VXJYo(?Fn)n%&wJ=mjFC=$uf|$XQ`doBV63Hh7W)S6cgh9TpMCnd~D1Z!NuG zk<*K>Gw$%bYy24Bf(R~+#4inn0CABL6LFI`s)748a9^Pfb>48iADemCz9Id#v{b_& zYA0Y45fhlRBp^oj^dO^rQaN4q9Z@cvOdJ3tRYOxbE__snZqLwC?3m>sDm^OG!8S{& zq9Q*HS`?)Z1Sl8DZL$Xou2`;EB0l--%}7!=!ED?1C>9MohxJN{n8^evs;|cwu9fPz zI0Su?^Ct-+<&W#l!~HOLF)4e5)tQJx(YdGh*D#JwElre;`oczQj%9 zD7PlV=Par%7(ziK-JliMsQ648A}!TeYA0Yec>3wu?Chma18E=fgAA(Wz^sEhWm>J- z-h@0CUKzoKNj9nj8guf*2SCKDMVBO@h9kEwFx*3R6ZFtJPX-W7_O2r;{mMmyBJkL+ z{SPdVO2oVGMqy&MTyRk6_Qx) zJG;{+he)Gw)83KxW2!uixf-Hnk^UNmxe)rxJMjhAx;uEX?&`c z`|{LDZfTNM$beYCb<1R}HI)7lFgro+$fjWoGwo+UocEkRDLrfcLb))N$+De`i$;hwMKTmsSguMjK&QQwQwzbQI1!^R7c zE|y5WY7h%93{gFef9o#>gw0iLr5d8ql>)OQ;Wo!Z@Iq&%~CC(QOv#L~7N03H5ngz@br*5OKsnDts6eB8KgCf$CfNv6k35Cn}; z*2>Jrb(K{ZjdZ5MntUwU#SdsdtWc-swjB2vjk6dc`AgDgn=3o)x%hhhAV8eUb6?;h zagw-+yf-y8!soDu&-sMWwUBu-oXJ$!NH`KzSb-3b-It|ep3VH17+s<`U})W9#a8bu z&zocx%{p9ewf?7J&spwCF6ZSghe2x-0FEf4$h~15JAIE^T8=|L-m8SzNPLwNF}W=L zkl8MSEYwiechRt0Rg8%8atLzZDlXDNY1os)yU5hQP{vH=UXyNHEyJecNw{t=x5O`X zgn-34;vjKRJqAIDReN2rQ#hXrhR96zY3x$mC%q)i)L}kLBT@BmP_b^#6|m5nJ0%!i zR;rrzg_BsR5UAcV61YCyFxk2KNJp=HQ{l`{VmfsS^L2HIu8~S&!!5$~QGB=oTuBw% zG}(OegKC*35m&OkL^v^}5ViYJ63Z$VFAfJmdgBq3gu2*$0jR5A%J{U+e}EqUJ7 zQ~kWePt*bdb-aN}1>7SJ5*OtN*IKx*&;YhBK_Z@a2Tq>c>~~3Nrk4H%o~aK3(+tkR zvShtGfuzr2#UZOHy`>{JM!e8Hjm84k9c$kDE8SM2sIK6SwEh49AInKZK~x;B^n>DD z0;hhKFHO7|uiy1I%o)!c1uSM0%t%b9i(vY@S6wW1&|+LE5p(>-TLJ8=6?yk;`Z|d6 zL|IgH4BAg=sLu?4X)>b zWX4?NtGo!(kI;R&;BhaxtNSs-tRO7KZkTd0erXs4h+~wbh;trrPjC>pNSv&`0PjEp zgVODpoFe=qeT;2;;ym^-lIZ1i>E+PE-9a2gF-nqg#0&Xi$Nl(r}Sg^kg5k3M^+XVphiA-U_kwN#8rFdNEpu!=Qg2fDJtrC;Kh@w^5>N&hLm8XQ&U)_u+B! zq_yB^7mHynk$@pA!l6gY7rL2>%q;{9Y3)hQvC1~z}Q-eLEw8jtC^0R6|k@YISD zooOu}(U!no^3$f*SuT}R|E5Y1mw|x*af(tTag1q-nvuYPxL2Fg1{ds{;67=9w7_D$ zy5viJUg}H0>=XCSD2iOP=r16nl@LL|=Myw=)hJ5Cs3W3u(s-JOun0s7YAsQMYGS8 zikPE1iW~AK5|fQuN{^V0?V{VTCBR=I;(7Tq3k_S4e|jOpx3qp0o{e|G?ebGnr!|&w z>xhN@pomQZXOxriz6S@9yBSxz=tKM%?&TTMS9RmwEys6}_@cNmU zy&Vxr+dH4a$#f@)YbFh5TnOSaFc7dfv~cEuaG?1Q;Mm^4G2&FT-$8iq7?eXbtP^YV zY4Eyw2=2a03*pby{7X1;)m)|j;Anqai34!Vtr91zK&f3HW}4XM=Do2nUIkz2-Gq2K z0p*I_(>)IHylFEP#Jd3m>2pt0u0Pn)!oTgE^rpeqf^!|tLRx3;T6Lr}%es%kB68PF z9|7^+0S669usf5*{t91mUxLv#HkkG7ICary;lQJ zl0L5AD#Xj)T3V=I&+|1!Az1#Vnm$kmwZ!FRmz(b1P4ELIPbx<+Va@SbJz%?~>2u)f z>NQn~KFk`0h(YYk;*oYIfFL)>ah$9*+R?*uX&#!E5hsfgt*2sx|HlYNyu;jc(-fqT zor7WB590l-@2o-Y8wFchDCy+?@vOF9Aln9T zoVZS$XVFEXR-7-}3qhf>ch!D_g8}d_XBN*s7b1!tzLle6X{2r#(IV2zxNudXhg#M< z*Icxe4iANPBA%4{Y-G@yoAx#-9O~UZVhlJkg=qpgN?)}#bA@&wYPskKOmmR7T+k{Z z+0)B<_!mXIY{yfpbLYLLd9U&2xZ`?6S0^WrOXDG+uCF>*3uk=;!7SH^x?2ytBp#%5 z=L0XV2HtZ^(c(ly#ubHV^=Qz;Va2*g?D6(WaLotBprv*x4y=X(!l&zF62#-0#Rzyf z0hQncL$p)QsKxtiJTI-iY4a5j?|sl&x!;Nk%Wg5Tza(-Ti2=&x(Vz6v?pDjJkdgUH$?gUh2Iu@%rFv@EiAgi7Uhf+Gkv3W+5&Sr)p;^s1;Rmmm>ol zfBBv7%P180YsB-);1Gdql~qvouC;5K{Si5kVbIEEFGm*1H=*PMViL?=_VJJ#81TUT zVIJ*ywOKgx6J{LRDd%d*%D7i;<9R7gH>V~b=Jqz|)*#I!^=Z0PmIFFw%ikKx=IEns z^(I}cLOPB6nAe$l@BbmA$N6wHc~M~#8?C}XR`=`N?< ziPJqjZx|=nA;w$cJaNClB^u%C)i^n$TI4;5(sQulEs^l70^--ul0Xu6fId_6!X`$z(4ybFcU|>Z2=> zechoO^}J9&1dZwy5!4=B*3-5Ekv|WJmx??igc$X7$ZrHV{&->nCnAx!-YmXwq)d>F zj0$-&f7x26>1B1w#F!eDYy-K{;p76~1LcF#UG4WbRH1^#R zFL8vpBE$>a5yGvNczxVuUhho$0YltoA+9S4H+D>@ePB4S9fpl-rTF3)?720Kg8oP% zDZdqD>)7#K#a9Bcr}s{i#POW7mTj+U?~TqLRtLTNhR-nQw1fWh|{kor91F1veHn-9M1az+qf~0xVolb##m{t)Y{Cf&9=4nu=O7R=S6tmXl9So- zi#5_Q_*0H^f&&eX& zAC--LXUeA(aQ1Q20~c7HPGWJo@gwWg!r%}vk$DXieVX$Z&p^fsB>HMfAzzdP|5nB^ z`wXMc7l!UB5vO}^fe?F+=)`tH``jAwg>k(bp!-_gn2sOG91eB%zJTr@j3-BfSX=$9 zAwEH8-UTtgi1ybbBy9~2-+o+L;N&tee}Gr#h|%rbKiRofo%b5yygZ1zUc}4T^4|1c zP~nMY?(!!j`fl7>D~$~(YzK>g$%PAEDvM{I0d56eQtY%GIy^VX2KTI$>&^{t19SMfQo=CNnS4y|dCn#9*xN$dNe-C#>W- z8ql_Ep%Y8x?Nm~4d{1czkRi~+iufe^you{!ZT@^f=k2f9toyxGlX}t%%)F(SNk@V> zT)Jlsy*juA%%NG`ft>AyCwTrYJfpfT()ZWM7O+mk*&}DBh^e2eCim)U@swddKlBzd zU~i$(Ehml`^XDrZk37;f&p+?PgxgS@-LY~8Cam>ESZ-0e4z0~_(vV|N`fF`wdy0>( zEjTHo`&RSeLng{XZgF?V9u3%)!6#r2)dFJKjQ1f@=uSVX2ON|c5o+CJ&%+QaCq6Sp zN_WpoFgcxv)l*M~v+d?@t@=Y@yIvtlqK?ZzLLfMY^oyr!ZA`}1vm%|0?s{J!{7EZ)WSo6H(IRMnNm;}fw*s3XzNUilV zo);claG7gv+83X0H;h1GvF%bsuC1_U{ZnMAt(N$y>LI{63lcBqlyl2D?xriOlztK~ z7XaWOagjLbWzQK(#LE%Zint1%`o6+4?H>ZRXy&0{ylXHS`@lgQ+e?yPx}UB2D=AMs z;P#Jau7Qnt9x~;>3q)`wYN{5Ix}Uv@AFlxd6q%U@hxP>q<1?PuanF^gA_lqCM{;Aw zH0eT5?<)|?#{*Y>HJBFXQji)-1Wb;o5&@vJYR!?$Ap?!CyVT#r3sE?0E1Ks>RV zl9fu_I4 zDm%(Fa1Yx>y@(vkO)wWNe^~Hyhy|Yv4H4RctYEQq%-p5_fqlDk0l+To5p!>tMBKkZ z1mIQz1Gnjs)6#qtMD~aBiSj5?&+p(Kb$>;#{`E6{dPfLwPB=H5BNis0V-Bm9=A&gU zB^bY+#LK;&^RGxaO8N79v1b663N>&1+~dBstq z3w!(1%)I5d$k>B8DHn#u%h2RB8#XZf5c{%ov=-e3V_xaXLrh z283pwAjaX0IQ(#Vx%;)1$9R$=!RS=iOF@xu0L7e3&9eQ37McH~RI-gO#-Z>3Jqzy;B)yXRTde1#eL&JQJ` zg?*uqEl-v!=<2*0mwG{fV(O^Pg%b=2Zyxc`gNlF1uBJ4C0=>dx*W)3Hcj5svsa2&ipcr7uR4VeytBcjVVLQppoz2@cZD%xk15*u|=~E#{xPX>djb9bA*#B72b8^y0U09 zw;EF&A$rVw9mZ2V!#9X?Ll~aJgMbOZAV77C={l^+`}mvo)Pd@y2car|Vwt;Hk_2?W z4I*9zgt{I^uIJbt+yejTX-w7^6S4rLrWc&nH#KQi;#5m(-Tn6Lk6~Gy0{BjX_tQws zFJVn9BOQC{iLYD`czkUVZNSEon=0034kL(bH4SUv9;zom8sBTR1d{7{Mx8yb!z3R7 zFPR3c&N3X-y)Yx9o;q?}>6J=Z>f8J*B)Tc$q`|q=tt!s@uqeZefC*9( zvA*Zbq6}weX4xWy>wN|AoB-Pc&nIGTGv$trnoI61P_pXmUXS(pU(LKLpYMZL zwr|7WefSYDfl5T!ABSHv4UPsttblcT0t_6~S&1BU{HSvw<-*uOw@ywzJ@zu$jrhI} zoa1)v7ycpNV8LolMC5b3a?!}I18`!cb;2MtE{CX`G~)rD)8GPmfMik8=%Gf4%NG7= zNK%ldzyo^Ix8h`u>EeG+Cy~?F;5eM2<)mRd9rZpu30Ox&gwWt^hWP#IiAx~>T?#_x zE5dgUQcXL1cvJMJpy9O~^QB)V%zKhtB zIiJCT2f}qufNHX@swVJ?6FpZ%`aJFBMaQ%>6Tnl7nr|2WI-S#kRtW4oSWfH zz&bnRc6Mdg%pS`jQt#D3z1U)f5;D0x37OO1+xr$?e-6*Fby(dWMk=4z?alZd5(8q8 z>4dNTwESsUB29iwg%4PoFcii`^Ei<;h|LWyR!e8@i7-*7BN~fGj=+}9Nc_%)77(C; zs!@b19aMd{r5T6Q_52wN$4ozy1|_)y?#Qjf?`8NCFkv2?2+gte9f-My3WA7(dtLVp zi)Xn&$jsKjv_L8fB0$3V3GAE?nAY^K%mpjD`aEcLd3&+JhhW)e>hw-|?w zZ#7-1e}?7Ycu?u*(eB4Z^-zh0-Doi3`dpWRE0oH*dVYyxPriv9R!<2Gf;b}=hUZJ) zNf@4E&|Yh56BXAwkQkuZy&Pihzlv^^mNQ>@U%joFL5Q(!{<$Dx{^29IQg3=Y-k-*o zN5GjMF)8oAB*`8w^eL90$-xR-dT4NH^+#6jYEQE-CRh%JN#xSbC&6it#y9*qp91lW zz`Lp%SsE#P7}a&#kB#>!q3asqhQ&jxCYBpL%A`U^H+XQD{ut~@Ic5U<{XK+7l(N}okO-HuQ#6MuIIvnFV0>l3} zMC5j{CX?Gr>Rl%8r-5jIU{5PE)tavEE$HS&`Mk|s9r(7}+Rxws!tCoE|=cfqV*ZJOpF- zG{&$Wh*phHhz*h5068>W;LLtCz%V5}54^}C;lMEF%JgP34}jOt^K+mffg%i@Je3li z4m>pMKUO#C!)dtmrdi`}4bb)R&G-Bp=57x3FeC_o`lZ3&M63ZoEK~rVIDIAjZE2gg$Cq??Chi z`|i&E0Nw{7o~m;Txhy4PCcDQg{_Gez&Vj%$wwcQ(X%Lr4=L)mO+gIgJW#EYkJ88C+ zfdDYzJ4l7P+uF6zcd%c~A_lI+WjK-jc$@`HE=Xjfw@)2|BBZ6L7bRp`X~{{o_}rO9 zZ8@|oGTr!jGic%!(86oD`&CHhuL57U;Qe(V?N+>Z;T)&!Cd~t!-b2pa2@WpQkOShR+rVXb@><$TcPb?av(FXp!51>8#8F)wGd%|rILlA4)kzh%t zb}R!$w6;x-7e=Zk2hn3(NRX^B*O50=k?Zn6_hGodOHfx*ECN+>@9@1xeOr|~&|7N1 z_e_JBbphf^;j)Lz8G2<%0aXul5iyfQ&0^nG@hCdg^#v}(;&=N z5D_^vuE{$Te(+6;xLnA{rHp}MXm-P23eVthbgv!v?2gaa*1^6uMf@Z{KyN7O?3S~# zT@i@`{X-?Vh#TwbpDeXj@S^jTL}%IO?p~-O&Xs1<+bex9uiDcFSJ3!Sx@FfOU_xB= zMs@t)cXh*@f$I}+5LufqR+9k80)Z8SRzNwVFRm3L`|gw8(};s^gwgjo5(|IUvCUbvDV0Rz0LbZZFVf5@BWWtWRP zB)daBeHY4WHkgw!;Il6ck{i>KYsJhZbB;xhtZ#zTeL*ajVyz~3g9TX$W^no$Y}GnN z2z~J-NZftC^C`J2_;}Ihz?>HorT?9G6vX?Y#|P!B60YN_(+uY*2S*t<@f~=+JtRG{ z7WY5)0ckikMtW(2PI~X0aaNb_VP5%piJR2g~0iu8A z9qb2Pv>pcNRJSftgeO62;0Sf_-t@EheuK%SZZ{V#!pVRrarD=q>N7G2L*j)i`2NUw zWe-|geBr)W_z~jfoCyo($3)MJ33(u#i^x)!ddo1{)!(=`hUjg4^u1Y!a%Mgtmeet--c!Td}M z=Z%7BeLPTX0)9Obu9@`c;+75(kAEO!F#yHzEQV*v??SX-ukwyvcz91S5Tv`?12o}g z3d?dX^#DX=xc_zhiM;RMJ9anQI>*2WOz-4l#jA@r zs9|x+MU*oGt}$os3tCIGVrX)pmVjsxB?OAZ`Bw}wcnE9YTGQUP!JOZ*HUBILH_il_ z2S@RTn1G8RQvAKx&}DyLd?9>+GbiEei9o`GHB=5XOh*ekJrul?h@eA=~;DgqE?f6|aT?hW(A269_1 z9n<$PEs4XyJxI)-g&y}IaWbG66W#kfa321{Cw*XN)|-XPTG(*eqDPq{ZmaYpOLPC$ zdl7K>8KjsWo9^r{O~;Bi6)t$?Vo7!6viL}wJWR@bhzYn1fJ*$VX>!&glBd{dfF}qz z?vW4=zkufs1M(^{6R)jXqDPk}2S`qOCisUlL0f_ijKzT;rgi_M7NP0|i59x8`!L=g zgY(5JcrO$>*PITmWG9RkDh#qQpJD>;4_wJH7}8fNA(~`c%pD3kJrdmJFmULp=MMCQ#ROarh*Q61K4F$pakYyt90}$4ARzQV1OE@f-3Nk3KLCBORrEnVN5)L# z=O3-Jw8s{Rg+yZsJNS;vbt{%QgYOnUC=6pk@rjijmH`helq@nCL_&Fplrl)Si`Ztz z`xW%sXQ0Ytn_)~=tuoG0_?eT48ArM4T-W=4I?Ze!bN@PC)YCQ;q zdk~Nwakk#xc;6HI%?Sx9r1(kZ%s(4wK;jmLohNP*n=OIQ_9Lg)e&%g`raSnBpZVtp zN+_)gk+L7QFPdQk9#Y=xSd5TK#Csn1JcGxcG1;&DF=5&I*cxsQT45Q__fcFg-Qv_0_GB!n$l zJ$&g}XwxUqJe}r0gh0ffay#=RPFuJ@i1CeHjD{hJFfcj)ixmWa&MWB4A0Z)tXuTIq zcY2fQxVo_2W9#4o#sK<4?kskwT&L{Q@q3jJ2+UrUPsX=b4}qYl&78^17hQ@&vu3&r zZrTrrStHOsYxcyI(IBEx;J|y~{r$MUJLm$jyiOYks|_U5iuVNm+HnngHOi_Bodl&i z$O-0|ZTLf8YXh;@*$(35I}-zd7`^`mw}qdvM+b_{sEwTVb+g~5EqOQ7LWTNC>;V2) zcqJB%iy_p|?=b;46uf0C#||l&6Z0t-j8=WOsluwD<@T_>cSHV-R^zs`8ZX^qTx8J8 zW!piJAYh2uZcgVn=aK`0ZG(W_#5R*dra33oZF1dRApI`$*lX|ZU{NpCuHEL*-*P)) ze(O@B9@5uh0v>wPQy^+uxE~qmnuF2Fkw@nSBt48+hmnTptw&t*6xu!dt(+#kyA1M! zpi3+c4Z_h8YC@A0U>TcMxx@i~s-t07*qoM6N<$f@dTC{{R30 literal 0 HcmV?d00001 diff --git a/pxf/pxf-s3/images/Parquet_Logo.png b/pxf/pxf-s3/images/Parquet_Logo.png new file mode 100644 index 0000000000000000000000000000000000000000..4c0002b8c9a8b151e20bb8f5b874cfeb6a5c524a GIT binary patch literal 25469 zcmce-WmKF&kS>h7OK=Gm+=FXyC&1wD?iO5v1b265aCawo@L_NX?hXlhC%b$1?%8v` zzaQ^m&O1HTRc%#W{Z#c2Rb?4;6k-%8C@6F}SxGf0C}?KL`wkKUM~t5X$9KvgQ30TZCyBFVW|-l za&y#ja!}0=p;iRO;iYKB9?&TD+PX`ihDD&wqw%20kj%HB8x}sEVN!bHWc(V2(Zt2s zLo-)^T7i+nYDI|5#W2weS{^6*!64R(IK~=D(jT-iBsH+!uLe#G>XJj~;4ftt6f69h zoJBy4=8A|l$?nbG&Q8ys!9~cf&3VMhmsX;g5`-skNTBh-qRc9Vo)1QX2Qa`d+E&m^ z(;_P~z{RGcA~P`7KMPfYfoFwPEQ%PcWQ8Xfk2PsD)^A)n)!)A!ilvY>(qHe72^%`h z@I7Nf2mv9$4k6s(5Ctb1X9vUTW#mhwQI4fOoXIU82YdbI^ZR=fF)Zve4MKPy?)&?D z&(ZsPUv%&T?CzNu1{95#&323(4klzekey|9+@PRPasT~8LuKUM!i8qU*8})xO`M-Q5E!@mpZJgb0oE$0t@ij4Z@^BZX zqWUN3zyJQTo$fZ4|4)*m+kbTn(m}R=zp!zzva|g+Z%9?4f29Iy7H&@V9{<<_9Btf1 zIEDU$_&$9nbui|QB0qNgAG5<~fKg<3%y{fg7 zyAx!rU2V+d9NjHkA;SKv_Wy0*|2M^d@e*SDH(dW?`2KTl{-YE!IU*=RZ2vvcA}E+r zjLA??Vo-9D;u_x2=RJry7J$^Y9zjhnkC18ZS@SIVOlUlO4@v0p$e^gAv28+U=~3jm zOQ+t)=Mk?QqEQaBFH+*PC`HC!;E|&HXsbf0X?bGh$_sr>{^9p*p~=%2kF7o)ezac;AXrol?DEOO+B1lg~DO%F=Vhe-JU zr`hkvN8#un|KzKei4`-jH_zc8JRz{c;~poKo-Z)n5VW)yxR%E+8 z0QzabW>w)PYo&JVA14PZRYMaV9~B9C=;f6{wD}n{kX5;r$+3U3jES+3&&M$Ze+0I{K!7<)? zD(6u|$$lpO>>|GQ#D2T)v9rH~&f&y4uUCc^nR`0c#|4iA-}pV1J~5t*ewIk=1C{i~ zi=tjTc7YGZZVKgRv|;Q7;I>A!!m%wneXF+;(n2<3nEE3r_GY_+`m&{DAzm*`kZ=%V z$Km%vlM_O$SPg_RZ>m6Zl$3u9S2E_Mh3Ys+ER-p{%%bLWD3`4o13Y7ouF~BoVjrrw zDP*oHHgt+p(N4NRWr?0W9ymBXG$i;yO5{nZ6*RBwJ|S~Zg8>$rBfc1x?R(A`*eMZ~ z{!U$n>4eyHzxP#X3TPU@I0>15K-mb?+on#DZil5rJLdDEhe}KMb6PdJnONamq@uh! zNJCVvJnEAsOZgciXK#(UwF>0}DdT<}$Gp*_z8+Ret9Yq?SjGd&dqfNcvGNxbLGy?m z6TCyEufKtdBr1FH&`J>1gME4+2qnNv61Mr=e~o3kiVg*GkQJqdj&2Bw5G#GsVnJqs zQ9~t4D|{oXKTmL8HH5$7)(Wy>NLsB~?yVhAeTQM(-zyVuh)K$FV4CfLnKju&aq4Fe z6v6>Rw|_xn``)X0-IeD<4vzSczX+~&DBuV9;I7U^Y_#D<;4~}>YBW_u+V0gT6}0s(dp69?bBSI{wQ~O2 zq4Pqcv9%PGp;M3=V5MZhrP<2eE3vQuouFn;%Q4W)3*OGv6GNDrtU*uCw+l5px|aBd z#VEQ?DoI)UDNC!?e#m1U^jr~#XQG@-g55=R0?@M|cP$2aUsLp9IW78$q1t-5R9O&I z9fywmHx?Xr{^Fk&?fB9Z#o{PyiIvjuz_F;1fWN%dJy>;K(FlR#b2jm$wU$s&=pK8U z1@wPBJ&viJqMOjrFQ!VS%I4c^qb34d3v0P59_zd)uv^J=GJ!O^W~fWw^M%%N4D}mC zat77%dC%4tYdzCwM>MQN9CIP7w8di+%mNO=(3Is~zu8g%4g~r`}V$%OO!h_$R7R>b3;Z)eNe^vgn zW=e0agsS+o=!(g&@L2{vbX zgK%LxHFzNnrY?EIMiFQaXpc*{!JdeH0jKpA26^+eCZ|Ag-LBsaY0ig(EhZ>Q>2bh3 zg1j_X!%66lVmb`uQ4_%h{4X}s<~`$^_z;Dv`M3A9X6fmml`vaWAB^tag2PWa{I)m( zmCVi2Ps%Q!ev$$Ms}jsp{NhG_ZA-osW8{q3=W4plqx8!~%vU7>^K^UjXazo6+o|E4 zvg+vSf6MFy-@JN@UG4cxZ)d8C`Z_GwQO%+_q9@peW0j zN(d6MZ}?8kJ`nvx(^a!Rm4~z^=~)MXG2#x66{T-L+Jm|Phm70AhqjD7I7-Mtqm__y zIhl{699l!>V)5&_4-(sbKCDJyPPqibDV#Xv8i8u30!*rcM+y*Q+0&X2wSd!bss2lG zw6WJ#w8KuLjNVYF0&1nLH6|r}!0Lm1f*KSzK#}^>c}va{1!QLcyHz}Qv-V9GPD^gTQ}u4(qT2ZbgC-8Zh6*ZOSH*Rx;x)y zC@x2ML{ab+@RNqKFwu^loA(o**$QebaWKZ}FvDsUsl5wW_MaF%GOZSx55v(ZK$Wk# zeEkFekXL?HH7{sD08i8;h_6(Zx|3U|H}Q_0d{sV(;6D2>B%v+|qfc0|$2Rp;`DXJc z4mDrAAWuv+NjIzjST%^Z^q3P?x5Oc9KMw!p*ye*Bo>SHePVJ}(2ns_PZc(l6o@=T; zBw%+ROWDHZzUz8L%~bu>W9I>-q$4{4p)VpjY<^YFdx|2AY4}f6l_Uady= z_3X8Tc5wxhr26Pj&|*4mtu5Zx&9p`T>dl2*1bGW=s+EcKZn>xsm>$Jp$FYQ*HrB;& z5rz(b0NK(gK7VoBnKWtKWjU#g=zC9+J7drO%98VEv5NGk;VASNOJBt$wgRFGCj`cf zS`>)tEV4G}!0VXm%1HCp{Gqlz1GP&{(RGI^xWZwSR@=%6rp8r`fCWB)?yJF{BXV2C zfmD5BrQ+{AC(oM_H73~p`C|S{7P-KlX+;#n2x_I1yUuq{shq!_#O2cfJE8nFDv0A@ z7u}8y68uO#A@Sn}1$98D$&bM3tg>!XJb0oOCIpq1tWh7d=MV&)eEvT7n zeH=GTySJFH7JNh!1>BiuTML9qA;A@|R^ge>I1W z@}36k6!`pO4)CrSB2d>~h**G=i;zGXWum(+oAi?!=Hval%f+hay$Wpi>k&gR#au-% z9G3Iz*2;4%8X{!}k||x4HyTBKzVIr}24zapX2|ucM$DyeVI83H=f2?h*To8qwW0~+ z3_9BKu!_36QHrykx*^;=*9YO?Nm$vOt-yL(A-F50FdofSvPiv}eS*TYIH`F*)c*Sq zVv9cNffM(m%6suFA^(he(xxr@+(0F3{W_wYE9{v z$@8)BvQN~B-9#!k=I+X^G|>AeGMZt5fd#aR65Vr}r7UuC3>;1^PsthMp}M5b#bIMjPq0_aVG z6N3JQw|3C=j6%m?f#L|oS^bxBC2Gp=X^jVt@+##%aA=UI!rHeHRayR~V}4#G5%|_^ z&C+We79DnWx970n|Kzkm&@LjOaG^(Qgppk7kN+)EV~p1|G@5NG3FSG8xpEQS&j z6@!l8)9K}%LJ2S;&E%@KLP{7q)nm;5MvIfVkNkPjhOh_Fr7@(qbW3k|!uwm>hn{?= zlW5r!26|{v|MiG5ZMZrPHRIlxO@X=9#E?K-MaEVz`YATcSmWy`RdJV2*aLk!=*YAH z7by1AC_-{+nayhxx@}O4&AKO+qr7e3%^26|L=;Eqc1pI`P50k(oW9 zYU?I9+27e`fMZ62QwR(HFSYkih2mvV5z^4>fm�D@9Ubo5fzY&%4*1@BBSH|N2Ed z3vuPfGajiE2}NH6&Uk-016ZPy&pW%R6WVh|>e&znuBL;?1{VEMRP1~C|5(39Jq03D zc6L-2MySXD`QXz-f3zdgh|P+gOdskEJwks!gr+EE$z1VFYIyqI{^n%vKB6dQ(eb=o z_9heG96pNGLkD9DW03?rlW0j1n?*uXhbl`(hO5Zp1DcjMDc;oJsMf)&)i+Wv*dd}31iSQMsB%re49I%_$O~z` z8|w#@T~yr)!!A`x94^aS6{xZJWc)eiVCE2!c;XhgwNgtuV!%h!uHZ>_nHECNp3P3o zG9OGyS$+F%cATox8g>|2T`*uMjVF8Hc1}t4J>V)RamRyjLW#2&XDphn&SELnf{*Pt z0|sA^*N#U!R!-Q{;gjf^vTZqz}mA>Bn<%5=QYL{w$1d7kgCUx;w(H4_H5AIhwGx8~~ zpex4d6LE&P>wz^)hPNhFm>v5HDFx!I zdo6iMYCpzp0y1C8WQ9JlKWk~0r+B}G7@Vwr(j6rGW9NW6dIIhA%434h*`VNBz56;` z=<1)ZCTVKwpQoSh4UbXc1TPzx&%qgkh=!~B&4B%7+^oigP9YR0VUZKz*@=Qwp^?@< zfk=AS{@5nJgt1fw{V~0$NNT8)kg()aPnsU(hDUF~J3&6in7yy<)_>SD$8`RK;L)M;=g815Qt*QlG@>`%OT( zYxaWvx#Sabhbsm1Bs|oU*zaom$OShwb#wf*1Y#tLmvr5WfO~G{$?Xu!L-Vz{Q!=Zm zHf*uWS45ssRW{_5sXE4kBGcF0`ER3zxrfp;?6eX9?QnIsD6=vrHy4)KUpw~e_?E`a zO(<)TYUI0rt|h7loebyMs#~uaT3A{AGufN;P7bM6rh-dw+*lyM600XfNi&PE(F&*Hn4}Y3mG<~ zh{AXEUJGc?g>X-Rep{Xg#h=WXI~+`@%K*>*t_!8;%2>~zQmMXYE< zv=$+PEX1nZo4QZcXS#IeA~rde*&_K>b|dN9TqBqCQ@``4V$L4)eBk)hd})cTC$HfN zxTtd@dd|>BlI=v z9GvAIgwonr9Vqx$7Nu;4f77Z;(I?m96iaM*A}OpWJIEq@4n}Z1n~q$N;@hhcejIAX zbuw1aEtxtsp`Xf~9QY;J5`6fZ8=*l<=_4X1bkkl_BM+4ZTL976~GoEq07Me^} zqJFQ?4wZayv`^$u%}g;Ke0H7$CKj~=N;@VrZE_yVgK8A^pH{uto8LhtA9oZ>$lAgP zhUd?6fSFyOmzZnGL#2_Ty)-CF5NKo0bG>)dBW4tq%Iac^v{fcb!DDP&nxls_{+=We zYlFGmFy3SB(cjpfnSeV#`t_W~Qi!^KUNs4?Mav)e9jG&$UpKnaEF>~o%$DUj)F*1M z;alI|nDU27pSt%?dfHI|u@xT2)MdnEq=nMHRT*Eh=;=R<+ou9Bu4l=Y`Nu{hBRc`h zQMyqV8GKYk$+g7zV>q3?pbYV^_C#L$(YBv5I`6r))86Dn7nQl-vR(w$( zF8E9(jACllqERIz38QRUIWI2scr6htCNwoj-X8Sh-n9hc7B>Md&vIhd!p-J2KYtr8 zaAtiV19LDs&{R%bBXw_?&D})HM&tSxS_)VOi*+jfgbIMhiGQZ?31C4-D;2gWzI&*N z&@H;Nq$daT`y?l&c70Vjl)7O!^R9y9r-8^6%zf&v6z-CNjSz;a1V4|w29GX zX*{Na)WkTEbZM>~k|!7eB`%`Ff61Xd&D?NJ{`s9q;e@*+LD+;c8I9()-2z<4zW$E( z88=xE+bY^_b)&jCjV1_>M5(`G4Z9~e0}!sqLn~)q8Sas>=Gi}Cy;g<4UNdU47WX)G zHO49-@Sbo`&&7(Gr#BN3RLK3rWp64v5_eQqtCWLVZA%vIkf{~_`t9zo@zK7yWS1Qr zxFnnL9KZS2Kwqxa%5?b4Jonretw;yHK$ug3t*8^>p2pgy+go#Aeh(Fhc5Wfho2O<96FaMckC7#|OmL<#ol#oPwwliE`Tm2a zR{RV5%QfDaMhmkSLmQjwxDaZkot3@7g~skvpDQzi)!2neTh=5rHb@bDlhd>&$lBLM z(Qhm#ZYQUw`A2L_T*dgdgNT>&+W>6U;0EoGkMOyy5kL%}+eRtq2Hx3G{J{1pJxm;H zx|ig5&@&*knLg5e-^e1rQ$B9I^Pt7CU&cZE&y~q96M0&+CfJ88aiqYz^wb@BLpm3A zjA>fr37@}2wDq6wz#Admh@&6HnOw?Zt$#U*Tz*sD_!m@py!+vFc1OuTwEly5cY2(w zAuLA9+4@6;_qKV7ZjzG$5n&B$_$QH-eQu#{pao2^14E%u2-z9qVqez><93<#IcKA) z?2y;1mA0C8!mLYa2R`h;a~Fr6Pwg zl+U|pw(^VxBkrZc6ut?U5SzNuzzN`n=(*6>c;$5*dGT+A^_ErGA`zb{$Z% zZ0;NLaa!oj1X#az@=ZbhQS@z$0suj!s~B~nl8>Q%G9w-gUveD306Qacli?Zt{rvbA zVAqvCUnij%`%5jSi|=UAg!YB6cvhbsBu$)_*6Kc>3hlHoff}!E7_;MG@Rzau+t%MN zh1(fPr=LWh6BZkrs}DG#_H;C+$9qzM)eHBY#|kH0?rOr7no!? zpk;_`&3JWVv%_&f0|9r})%I7DJ`{R7B?@&`_vmD#^*v`BMXb%2fnD8+%>*`G}$GdvQI~ z+kJhv9Iagz6tjH_1b3lxL$%x0OzD#3ZNa{(Mh-SX9w>e*2L2%9qlxD9+uOE2gYHZ%a zgm~o5XH~AC+`Jck+aD3Pq*XEzE&y=P56$$T z5=V?eh~U74c|SB-FdHU%(n=U(XWOV~4W;#5jxl&RTIkm}jcVjtrdpAXu#`=eHi%>? zN1W0%IZFzTLVlm>*!{s>w1Ug6m|q%Nu4G%S%3(GRQ|7}=5A!Nq>J(e!Sg``b3NT#5 zYKWZ!R=2t-|M>Lend^1Rpas7@Y4;$HkM3nuE9m~(vwZtxY+)=8(v@AUDq&Frc@i8w z{20nMgBrf8z6TCMtI@-1vhc1(Bt?MEw?qR+C)zDl9$|@Q5j)&)D*O@%ihC7*WD`BWV>>?u=Kq)VAJ=$J)kjv zI!Hel?nbXjg3z{uw4+eBCwFFrZOyeioHb9#Rrb$zKx?T)6;z`Oa=iP~$B*fk+sh=d zy$)+}?%eW&k~B@I&0qZ==Rig_P^%ily&s_%&0>>A^Du>^`?C?D2*9bGZU*w&jEpS*}NL-h3TEdYD9 zZ#MoRsXU^&QKD|c)}8~bJd5>`2-PRbcaY1R!E!=G(B*>1hmNxu5^rllI7vG+WIbDSe;8d-C@i+){v63Pw+i-ABPHSX{jsveYpmYAJ^R%K zXZU%_h%|jvk9{vNu>u1oJITgrwUK<1gxH!y<64D+gON0$KkD{@wmk2jO|~L#IlL0* z`Ag?lL1n|RYR5A-YAH(P+8L|4s>27JYN26JJJfLXSZ!uk);+J&lN8s#9eVyG>cGoN zB|_d6*DG#8r!Z+75MsWM!MCb=*7;ib3At6?_BL@o&U_RI01}h;z-#r51d497yN0%LzX!Zc(CU`RdqHgf}69xBBI1B7FA^uF0 zHc5qgYjb3O)C}OYg@2_4I4$k1e9n{!Um7vhPWNhhoAK6mc{K{t`_%+x+zpLbq+d^eog<jo%9v?5g$zx$7N_I^Yau5qzWe&~Gv4iJ;&1yIb6>h}Vtt=`k8ZHjQoh1{x?Sp| zR2i`-%kCdPgp_I#J?N^NW!q{-t5eBfQ^U@M@s}n~l(8|+ESrxPb6T+E)@mHp@=SG> ztl*rf)6m)-odY4V>}5GZP@5ME7DI6AL+V;06f5{ygteEIRBY$HbG1pi6vNE9Lu<>W zT{`E71G7VQ!fG;fORRtT&tYKO>WIFgBJZiAit#oZE4@gH1;b&}uw}g3B2rzc`SVw* z;IV1alS@E+O4dnX*PNHFuViyH6sDC^atSpRgAVeD5p4B^xvOOTNi3d~@BD=Ck+2%o zt0F>`I|;Gt6pp6E*jOmz?M=a3=I4laM2Q!qli2HgTXS0q*I)^WtB-$3f1a;-$)3JP zMQqiIL^E}Dfl4es*V*?Q-ipV2^z<7MsLPBF_Ym4;vl^LueLC52 zn-wW9q4|WwFHL~ghxnXflEjanY;2j7mUq(@Q%A81s_GTbK`G~erkrGBTm>&mk{XLiaR@e1VPiVF;tsMyiR(dI{h z-a|flhV%mg8xPI5$#1L^$H2h-oU>?VeluHpBVj=Pgqzi(-@&K}I10Y^d-Wy9AbWlMn}9|ONhktHGH z6g$m(^M0~l=siAd7R8BKTcD8KgMCic=U{f+7WCTb$O>`~DbCyIX*gU~$z1O396r!a zyUmI2Vf?w<$i~_D&PxAA$K{6k`_(C%6)2KP>2-MFEY%z0r3YZTLI0EQR)175@y-qfG!s}WEgyDm zvXTR0kU@k3o)x7p^^7+D;W_z*XN%=Q(%hFV~jRqk<_yi$@?D$sFnQt|lG|oqX%_wFI z{7^SGe*`@c%tya{+Og`DF9}O$Qd4eJ`XwQ-8X@8b<#Esc(ys`N3_Xr{bpaRTi*_>V z!`BnPD|@WcK$(bQ5^yejgPAWjQ-P|ZvC3W@?u`PTPMJ8R-!G?mtW-ZL)lb+KXsmQI zKv47|Cm7hw)3S2i!N83zoET7qYX4hc=uqY`PynVMXLur}0$m0i$pnoKl6uYA4x<-W z1sKR7gc%hJwBnj>c07}if0InPYBpJEdk|pSIvh9`{WHl}WL?Ypl0dJ*%X3gwh)CRN z;9D_qAYJCS1^Y4v`OtcKPy( zs873|U+=edTElgC`7fK(r|fH;+n+0G%k81_{NAao<|b7FG+^6T5JQHFWhZ6=y=Rq< z-tdphP>ua;(UnnmE%weq_j?w#l?*bKD{P`z&Ic{nzHDCo@S2LHU(&d+dWOeG#+CwX zeC8Svv8?tA^{zF$n9x!~oCFB&B`K0jT!19oYX8qUcO1<0fubiHj^`x0_Bs3>RqVnG zmqSaB@%A`{Rsk>=&lcGsYbxWn!lbGTwLG9zMF%f0MKge_9$bxxKUVOY^A~)%o`Y!0 z@?ZawS+gT*&s(Hte?x0J%!)L_7GQ%PuLaLD56eFE^=J5@lPD9)2>sN(*{S=KX(T5t z4Hh}L5V(|i^(8soPuVkP=a=uJMxWxvMH%TOBrqV42G>wOfX^Fr2f+>a$1sHxpj3I= zmM~k%E?LcT(?EfU`)~U{hvl+-iCy+g&j62H3k9!o1te#F!IEiBMIbJHcwZ?YD-=9nV-kObhTXBQX^u@04V(?D`m97mi8Ag*6mF^V9Mjk`=cix84wZ=e!kk+8^evMe&OjOXO_C zv+l7l{fD_Jk;bRl*B>O8mRN|5eT8zBjPT0~toI75FW8hF(8h{_l#`*HzwOVIFN6My z!X2fMF*`_9eyKcDD=9Xrk5dIG+e{HjzE;JV3@zT$x0y+iHO1Tuke>t6IjFtxAAprm z6&{tkK0TY}J#pZ7L>(%3XkUr|LcvTsVq61UZ5uF7^#v91dlLTMv3lcHNb2@m+xKu0 zGUOTc${rCw9xC`>O7`buAO}$~O@DH&(y2h&vR^JyEdzU_N*`vRupD4Y3D;dy2(~Nk z0tNT_v@y0(W8alLb{+w95qa_TSs*mr(w%PnQ^g;YW_}hePYiMwS)=h^j?wWr_PI+*<8?`FngIE9d@XuCu4~u3)lX zYj@xcAf{ke&+zwSi`B&)Ek`YVk!>@VXU=T`FpgPDus#xvmY(SG1?R*ls04Sn!>zW8 z$DMZe)R!?%Dc?AKHvg}*SS`-QfuhObaG=|%(zL$KAFL5T_742pib@GKt84V* zUug|Gss02lGxP}^Jyby2ix;{|M!WlEi@C)`tDwCq7HMVDyi1uc<3KDue)mI>s18C^ z2wmMzD@Zy<$VoGU_TO#aZj{+h!sobw-{v^)UGki@BUZdSLIF?p&IoADp%{iGap4|R zGOCvV!lfS4*Bo18L-9#Ity<7CpJ!Iv_Kf)nl&-t#bHW@zDQ;lTyIJel=-%NaH{8WA zTib}16r@6CWRU)C?&B|xyV&!U@{|Y*dvJ>ah%|F*v zh=nz)txXOJgzUlT{+DLWiEPTX=pjJWraC)b54r7@){={N_OgAcG5%b)uC<4t;XHSS zW`zH2efQFR#z!zuzt-uWwcvfa?{N3g>nhoE6+6J`UdOej^;Lnx;)#MQ;nC~o8@+nEMdPC&adU>CeKgAmfU;JqVm<*_a~9--MdKFs1hvz7vm)Ko zK6V>uY%rTCmQ(ER{q-%o`ot;6hvw+@p^jHByd?m33w_^vQo$}#qyWX*DQftb9`IK| z>j&E|ngAxUo$u1haxJ)~oR$T$eS$N*bkN@S1+1j zP8?|1W@qHE_O&xx8G(tR(V~>u#Np#3bx%)mNtll}gB@~t5aDei5QBV%%D4phnpmrN z@R_!($KykxvGHp9a>3_wy5y23Dy#E+%^G(T*B`$~Y;OI|q{l79@6{@m?iYeWiMRWt z)YX!XAIoxh0|Ol(v8U6PLSK+3uGv^jdf{)s=oUYUpI});DcGk~z)Ro%(=Ok;o5D;kyzHJ@Xw$W<2s`IqXOU!RyIE9z=29<)sMf4c*)Dw%}R0h5&w{8F;m*uPYY6r}KK91WCq=mS(3a>}cqXD=HaUP+Un5zj ze|IFG=15cgym{&TVy}W<@0hY=)4I4`Wo;d7-Qwk+!?`yGSxpMK>UDXMxh}Td2@76G z>agC>L9v5`t#Sc7cHK=V9U3z4Ok!-J(35JHhix|J#GohhP{($E;!Z`9Ikk3XigSuR zt2lgjSi-M00NOe_G}DP;&@XGF5B7mk=WS@X2I$rLwzv7rEjr6Vr93|?qErTMXwl9_ zao4Y4Wj%?iCl*&+`>c@dCVP0#g_9HE6KNIijnHo25c=p}b5ByazY72;L*b^yZGDE! zin^0e(^Gn=&F75$ev6F|&c5;m;L z8rVTd_eNImIaP;Kyd=$>=Uj^T1UqXmvkM*5%Cs(O%?KGzAD0;p_Z-ld`y&v)DM32b z`F7D~pusKQUNnSu^qZ{ar=sxO2=nCDt$F3tpq0iz^SK8CXa!y;kCEwzW8Hjp4Op9Z zhak`vl%@qmorqQ*)AAX_%>Q$$+7WQ9`CHy=;B@h4Zt`@os|}^qI5ckldQWvtHbLQ> zmN*N+5U*D0bH5gsU)2p7&u&Q93%6v}qH4l~i5)%YUH)XaRrDNU=)x#z$UFqwuU>-? zb$_do_!klvhO82Rj0bKU?D^AZtK7Ywz#Ooe@0~{HKm|eT2E5T0Ul0vtFbsKY|6W!% z5wlYV)~@S{nPDaE!EUtpCF{ok&@bk^CBsr#>Y$ABw(-tnb1UnNzK@^!iT&AHOdza50aJnI>zVAe_clv|H?cAHKX>X?$#&+V79FY=vc9LqqZ$$X2TV;JRK>lVtV(9`!I~)VsTUZ)Fuj~)r_Nk#Jqbp8pm<|KaA;|w}OCMs-g*f6YmB`D+Fk8HWg|WAp zY{%rUtwy>F`l3V_*st(RBw+X`rUtE!_X6=ZVjDIb`VTO8T`@$YZy0T1M9Dl)+NlG! znOynabz73()p*;T710MLE24lX6w+(?d{m);I*MkN3S)=~J+izjV?u7$Z@s}6J0)L~ zfELfMi9c!-vwzrQOgMRb(tntIVzs+*96{hI6@OqnB_NF5E6~`r1X93OmxYFp6Mp)vmaSwZvo;qlZAk}2>195wxE7U%oh^EAE* zV#WeX*L+G3r6~?G`QQ;UF83g*u8{P`@;m2WllX?9vBU>ZjG%y`eo%{o;C}*bqBh zzLMgCJ6})V-i-T5;trAq=hXncI#!&3Z^-y(_78mAgeEuF^KDFW-$D))&m7(XI<{Pu z=3Y&;y>wPI9=XeA>)Tg0XlH^P8yO>8;7+%{;5^6*F?{x&ttY3MZ5+H4Cew4abZv@O zK=$TMyAM(-kfn|}4FTlD!r`+QhM^*%yP?#g?rW=A)cXY0^Ih2P5)V2+FWFuY_RvNe zm!3O%^`CGBf$q>WduaDR=&?e&;A`gtZs6pjsh{t?_g{kD99B!0{;O5GiLB`}Q)m8g zZ3g%32K~8wwKQuH@pfy#^V(>{*Wh^C8q-)I->+MZLTEL<6u*8V36ROD2Y!Sko`YZQ zDU@JPKy1`xLeYV#&#*ibh87Y}ydAGgs&@k_4O0i!mefje@Cq_BuNZ*4D#@?-Q&}WO zJ+19l<6$KIj`%I?Q&WNDu_%{c1p&HwYzy3FxYWoT602HUDEN?!{Vzo zwtH3}XYy=uann4v^tTbb*Qsx$2m#R)vAo=VEgmB0i=Zn|{CneaS_E+tYEAD2Mh6z^|lkKCbM#KpvLCyCiX&ykysWe7k? z7JLP{)%AN}%j-~7IWXlN=ke0^YsVoy`p2AVP21XS;b|t=NXi?xkoLGHL4ZwK?T?>F z0=QwYN#^GCM=k&PHNU`klJ~yxr;R{lRGpMtD=lv0MB< z-JSJY6o0hG0Rd^GyFmn`1!?e$bji})%>vROwKM_}vUJD7(z$d?FP(yvbgXoX+~s-h z{U`44Gtb2N%$fJR&pEG=C-N2mfyMu;DH;gIs3dre_OX>05jJ9miHDRE_~}NS2Qp*% zUW>@#A?ZH`WV>j(CzZ!<1T#8ceJ@y>8EN_B<1>GEZf4l$c8y0f-BTjGkkeD%gr@s+ zsW-kEeV|5F$vT0efxOV$7oT)bTn1bZEo^T+jVPlMbZ{)*#vn`nDn8KhG;F9uZW*C0 zzhyxk89?uaeMZkH#tco_L*8jIMdj$Dvy5Qt*C1u2IUkP&Kpn52A2fG2~S8B2*V+#4f_`Og=>nX|ba zz-m)=M5I}B7v?9c2*kBhx3H{3jaZTa^#`FSC=B7xEC3m(brWW-K=pzhA#lQc9;?qK z!pj=liK_z|PYFn_-|eX935Xi!bjcbLwtO3`5clbJ#MlZ?(Mj=fsCtiHWEb|M?mdjo7fY`7R{*+rBWsW}-Mk!z^=)xs+72&a;d zmu^M_Ab=nUk3#weqw%d19dhOg8Pw4(+Yx2XZS8pE zsx8-K%Znv_C6dg20Q?e%m1q4urUXvz=yt+wk4&iqNA>DQ>KtJ@g&(DKpDR_4+9P|* z9EmbxFL%NNcwg zgGVIcnFf)ZdI`ngo>76+CW2=4F>`QUz5jFDBfTU3MhAAIHU>s!cU4)Gvb-cT3cC_s zVQph@6ul#oHs6E#*L+^|5>MZY*qPDJ^(A-oDG$n??3V@dZu6^?lmc6cq8*O}V(KFi7&r#`U zPltpz%&8zR@(S>FFM(9?umHsqQ^vXteJbzc`yFezB9i?cA?As0GZ06alij6aOo|l3ml_!`vX28cF-rufvbNoW6Wy*GMLRS;;;mK;<|OBhgm5#<;h*v z;5KmIVs)1PCsD!e_W>P2Q&gR{4W3bF`v8~&jbfckI`;crgPCT2GjCOp&@gv#19BC7 zW=Vc|!xgT6m)EWr{xYqSa4V8*MOUXn9jqBbJq`e4hs?(TqrwB(3Gu&ZP>~OwiFX#X z_jfp6?RwO3V|3JX*{3#F5svwikH)qyHcStS{D+96OB(+X-=TsO@iJ-*OZd)w)C0v= z5^c;H;B8s8S7DTs0aC<2C2K?=e;{9QTHSN}YA0)ph>|I#f~0<&M;HHk+wiJw#CJrp|CMIYCu5=M zSjCr9vH3b$9z6J((@S@9`lcbY6iR-;xKkw*TqJ^H_g~lw$aKRc7+zW?~dN2)EFi+3gUO#yAAZ_5)AYOYO7xveBsD!VG{! zwKM%dk2o$MS2laCkcVmq-#>pS|voX;h^s+0_Wz4-JXTVm7vwH2gqAA;V35*N}ay9_4Q2<*9 zH)C@bmzEF|=CGf$A1t`z#9X%XYgcXyLT2;%f_LsuPk+sCQk^6CToiq2${k4%#bU(z zBikDMAimgu2E9t$3YnHOxc{HNxBF7j@mlhPF*doTjj!JVf4%}~9ERi1$qjku@V$#; zFn}lHIMO}np0);Ni8#6~QH=;rGhfNlOsgK6#_Mkh)qR+v?bYhs`(aUleGlod5xZUe zJ65m>k8tX{=uq5slRkc;)Nbc!e4lfDo&D}@+y`LuH^(c9PG9^ROWE`$XS$mLG{ZTy zw5>qDKS(Y~SDNmMvD<c5Ho8#u0mg2VA}_FkuZ&Gu(tG6^)jqGWMvv_2{wAjYv4BStK6RmDZH#T+kNPxk&o*4p`-`-ya`hbY@l~zfn69MMV?Me zX7$}6=k2;#eBE-!ER@FXPfNb&M%d|A<63c9u8v*W@XB1TCs5I##yTT7K;g~1rnc8X zXVy+Fg!eozyAj$#%I zsNkp`)C6Q}@7c&oCxkFF5F1MOMbFMUlp56}`_W95Qv&TOvUZKkie|`d7;4omkBWS+10hDAL~;X=6*9A)H|(2g47`$v@%K5+ zH~1Ytc|RyShQ-43()qwnHKDwzX+OcLL6Mum1b{JZ5fzRJDpjh*?cO$W#_@oiKj2MK z)>;Ir&mZ@-A2%NrNz}WpbVnY$K9r?2&j36G#eHjdvvi>xm7M1v#eslfcj`cAG z)jbhI@qK~3WHhTV22}aeMWa=cs8_1zivQD<>bni9AE4_eQ zd?vpcNLw8A2dwLT`eWhzm5a&JPQJdwt1B9z4t_BBGcT~8Xo_My0i_@#zc7OIAu@cD7@s^*_1*ZVn&xTbagp@xO?2R2or-zzBL3<{hN)AX>%wyAcMIrNOak zwrF_pSC#_ZO5zJd-y- z=l*MplD^UqYha*;IZd3nkyh!ddqsm}T?ryG`t{>qNw&3S_mlG*|L4wO)35XC!LmC- z$Dr<8UdD8KJIE}Iv2F0|mn5G;*`7WBWu=uVhr0MDdW7G!m+2{jCbRUyxqoOdX=tSu{J(Kw~8ai zKBkVLKVi9W=7+a&vt^mM(BSaOG;%_iLP@=cGbMXM$oVhuOT{LvH{fpW1IzE@RjhP< zDQ`h-gt$h}sY7Tigk_c&bg&+-?Zs)%mRM~?`%NKH9K+Tmdi)57^}AA8vw(-J7XFoz zu+Fb`Q$vzS8NLE7WqdbML+7AWJlfKh`b}O z1m^-J%A8`h8=X z`2PXwSDF1PCwOvo`Yz7b8J20k{UBpv`!HR6e@8o$EocIL9V>Elv)H0ZY91lU?arf$ zSGpY4{G0gf+h zCoGOfXd0>Qq??<*=cqdFy=t#>V3HwcLu> z^xxnRf9g^N$%h=joKTS%H%iH$h@^4u|19H7)~)e&%W~nFrZV#$+YL?6DyS-@TyULo zuIyY8@gkN)2A!uDH#ZtyBi{S!JZL+A*FKN9n0ViBZq19$J73)VmW-6PS821?Nr1pm zc?pxr!l<}%dTvW}1*nl5?eLxJtkQ2l`myTU8WhWeq~w*jqB3!+ydSDEqq5B5cAjdL zw*BFhi$qaM$=>ZN6ifWYiXA?yt1g}L{rf2?*f#^QJmhc^S)e z`73Ke-D2|Pp7S7rhTZ13a(!GOS&ZzRZx>9fttifG#y=vzFSBB5lZ5V~hm>hfeXM;^ z6}GNbRQ+7nC0GN_6L;M<8Z9tHJ~5(U4k>&(lL{Wfu`Lk(_;UP=F9yzPZyz>&Wr4;$ z{9f5)9<{f^g#E}JxaE0lymb&8qop$gxmwaFNwO|I`V_59yyWoG{kZP8{y4&?m# zL(YS%w51vC+mVHN>_nCW;5^E?pa&6X|j$YmTAnItF*h67Agu)b*5p=pyUW=t;n+YSu$+F|Fd9dM& zpH=~v>7JVeGX&6Xxf5qycGG*$z=e`Bx-BqF!R;Mq;(~#Q?2a>M^eIQ6-$YRQOx^8{ z-QD3}gGum8V`TM(t`qMLEVr;ZXci{Y*CK7g%FZ;S1EO+{g|&EA$6DT9+CH;M#y=XT z{R&}65O&NYxE@1`Fe5taMCn3e`#j^ss0%TK+1L#@g7$8)q*QMjJnc9SQ_XjT5IodT zPu_VDx8cUVZe@sn3{~Gc$-$yRBO4m(=5 z(p&c4d43K;H$T*bL;bL2dtk|N>hjt5;|fz zNRF$gS_$W`vFK#9(v6a$BhW{y+l}7wy5#CZrGK8vR&5;nODir7Ds(@K2Yy$3>DYmf z@3iBQ#<&hk!Qsemm>BqHgHkoTD%si_C?<7+U1=#iflDheXsO-1O@U4>S9EsGx3(!x zI(tg}m4w7$Nqt7sHKVw~J%-(8ZX{&J*!Guag!U95Gb^6ufJ7CKGE@Fiaf`U6iNql) zYXrXg<{TELt`{%I?D_Z8d`_4&NIn>UV`lL8Le<%qB*C=pB*;v)VP-hcMZoc@Z$X)o zQb5#dKd!>+)zp2mSaVk7UguT&6A!TdOU}i0$tp-6=IZ0f2&f$7AIe}qKTKd11V07Z z2d1|N_>R=DSlzY3|I83ls8c}sU=+?hapbDpiu$hoB*6BEvA8qYQ@NQIGgkE>tn$TI zQoEw5h%y1lXY733=zBLhCqlY*K>VGRGFkDu2`ycNcUx4n;n^{%+*1Inzgy#x@M%f$jx^hVFb`9=G2r(?EKAD0sas% z`Rjcg!;^CX5j(FFMm~$>@#1Gt4 z=7!iX5Cx}=WMvk1MTvO20CM%2io|egO2~EVS_sZa!vp95rR>*do2<*VOpT7Uv6ChNZCX2-W&Pk(g^3^7X{W)(l z&yXI_%{S#FN{w>nkHLDbXfQMuX_p)_>s`TfZfpD@)RTs3yyq{l`6a^l(GHqbYy2-+ zRb>C&Ah`O~$1z?>)OfJKEw#hZbyGRcCSP-Zkt{)apVb$dyV_r;YlCQ=j{mUn<*9P@ zuRxwcQx8vy6p2j*ElWS3noTQe$Wvv`%GG-yq;Yne;jp=G-trX_MZqnHIe#@+)ep;D zn|rknjBH$V20D@m1vV$R**_Pp)Mi4Sk3p0%Cw{E&tW21&$_v8t0-vkD==z{MW>J?|2gr8m=U~j=mLgDO2qaCu`-9 zCU%KlfzbSGE{w?!sfMg;_JLr5bO%Dpo1AdMAXH4R$Z=t7{SUaCoN$b?)Sk~IcijEd zUuc7(@vInzi63S%WLxIehSC#A>Yjv`)#|ek>{AJIIu*3i<+>&8vgmD`Go}X?w=7P} zIFt8K=7t{=-zKMZczLF+)q0Sem17R=C{hnrtptA<+Rl_)L~r|V1BJBf zsiSC()i|*7qW}ftv73}MwqA12#5xg~-|^#{HPSh(phTi2Dvju3tF)c;>(6XQE`FtD z8mwK9sFfZ&_7){To7P?y!WIYg^NzyK{fJi-TCl+Tr(i zTgUSc1hiD&B02H~@_G6zsPA3r@yGq!acS8bZg0_li&8C3vc0+Pe@eTfP_!(kedM;P z>LInIf9OmJrZdr6P>vyZxsrm45q?9zL_4wW^bv2)t}#-`IFatrM|iR+L1T7L7G*VI zW35gS>!`l)4?Rr#iKoRr09i~?JI19s^=ol&Uh9Z3sl@4>u;_Nh6(EQ)gW99{rU*3F=Is zj|5y?VAY~Ewivxu5XQ{LhY(*7@?N?x`=--Id!h4Ci}O%Y?(wI$s80%ICZ4j&&N45l z)q&?8q!#kJ6v6*CTc)E7cw5b_@;=;fmL{6~>_m}mv5;6=Y4u(MoB2Mlb?G7OK{#Ry z1+IsJ$~Goh(2e4XD^{oe;jmnoyq(b=p`L=N({6<&?|b0^oR-o>(L>e8ku7g&PoSaw zW<4^O<;%1WVa`0G0F__p6Ub5jlgRQlen3`WB0AMh-!WHAK&{-8#j$--NLn>`0-6cE z5{o>vUfka>vZ9`RQ18qSfw$ufDJIY@C9RM10nn;UwdrD07@Tti^X$|n#wfFzHnCE~ zEjQ?m416oAemn`V=m(+4r9mDMMSqCpiJJPMle-NyF5N zESX-;5BqL@x>CfA4#_yH|pCTb%8$$f6$?M;Kv3@@r`t4rstS)&VzjDtH~ zIfQ2XTg|N+M`xpA@Szg}?mBq8xI4&k9?&Z3u&WIE8H(3ab`)yUq=NWZ9K%m`zg#0Q zK6;h<;2e0pm|@gSquGy`h{M|q9ESdH%PsJF#>Td6LEoB{$FnzU{_DGKV+{o42yJfh zsiV-gr``kVtY>w2;LrV{vXtP$dp>PWm-}iSpi$1tP@%=hfj3{X+kmt}CCjp`w{4cA zZ=i9%$0VUF5$b?O+sBxswqpej4NXTD#^UvZ^h4$Dc5z4tjOIaaG<%Tl!m~~P@3CjS z_Ck%$5e$ngZOD18dNAjZPopWNTJI=$lwrss>r6$==rR4g_@5`e#a@tJ4O<=}2AQ4x z3qKJq&W0_^!lX=G>}Z`1Lj4X86eRtSOcXDjBY@?vB^!Yt zm^U<=c<+X^C7d?a9`u0Q6R)WIyQ0?&;1B}T+LSi;QPQ#~w4eK?giY>iZ5;WIx5T+S zH5AcX(-y)S55OVS69!a4t1)tiiRxYt2FPP9>FZ47?7T z06GEypSF}>TO94@j)ZdS5q26Sko@Eip?cTW>VI+|=bK|BQ}jky%UlMT#}-1m9{o;{ ztn}xX)&1h*Xy-U77(+5Cgo}PP0{&{t~y9#1pzC*fa`1>n)FY`F2x}R&K z!K*dDR#ZX%hmYRtMV=EcQqnwQOaI><;z`=j#_MV_PZ9L|V+JKT^^ev6nTP%l=0!;r literal 0 HcmV?d00001 diff --git a/pxf/pxf-s3/images/Plus_Sign.png b/pxf/pxf-s3/images/Plus_Sign.png new file mode 100644 index 0000000000000000000000000000000000000000..595271d0e2a39ad569714b5f2bae53962b0bd986 GIT binary patch literal 7897 zcmb_>cQjm4*Y_BMK`^5P(Z-CD(MA`eccVw|UDQDsokU5Lh-gE!=zdJRF8gs4%2 zh$ILSf&@u?H&1(?^{(~(^Q~{znwfL=-RJ8R68_5JmxhH#~jZK%lGXE{=`@IH9W}PEL-FBjbW-s(=vV z#Kil?j?V}B#`^{kgN{Rp!W>)Mbr#tAKr};S*ULU=DdtP;Yn|`E!e$<~ME}JfCBbPG*q-*^r~o%1dFJp_?Gy!>~eErG$Zyiv*~5G z$L_?d*T8DhX8p2CZN<$zE!tZB5*RB(t^Sd|Nl-a0qdTlji6Tbdol!0o_R3+T&$0S- zU*Ad`OgHakUqcifY1|Nl(hs}Jv>=`k&-MG>^mHH)nA+3Sl3=N?hjsD8 zi#WOZIlGC3;{Abj0)ga1vA_`TMsPxe;(dGrv7rjce{f)d@n13qiTDQ#!CL`osc(c( z^9yi8NQp>_h$0oK5D0{Pfa?vcvAX8J#DOaXqz8fEkHug@LPA7BBt-lI+%aOZva%Ra zag4aQFu)-k80JfG3KjMZMEz6AzxAlQ1-b-y`V&0;d=Y>3Iyw6V5fqTfzlQ$v`KO-* z&l~?^$v5y{(*h=l`FjT=CL)UYk8VIz{x214>=x+f6ZBWTnXe~7QC$8X%>SGGKi2*! zrRInC4{!?%1jH1@{w4C?wExX-7~ts!%)Bf+Vwn1+(<+36;7 zXfW^R>hXCwW{4$S8IjNUd2y3p-5(8^&4Qs+HrNI!yk?=QmqZBsH_MWh{goMLNSH;` zjY7yFWG<#F8a$~kK7)33`@a`RU<%`WsLu-D+LLZ9Xo13owM_W^l?_b>QH&pt~SRfZ7f+R^K zF&>8}e*Rp^qs5wIBO?K8L>}S(rZDBCYcYXjAq%3w z{_67az`T_ATL=n;N?(YlVJj7WAmh98v;G;&`)nu?9`n%QRtFQgjOEO#vx5RHj*+7# z3|Y3Uis7+xWe!@W3p=dbxku+!*vW-Sup=f%rJH?8m-k6XNU-^V@2qc$_~E-H29F6q zHr)Ds7Us$T9+N(jN5WfwCTFT0a<4trbKS5pvwxTR>Mm+sUP;4V%RfCvVK-SQu{-KK zI70#wU#$+UwQMZq*ebC}S)!1wFHF;!MG-2dpu0;gufj}g*ls?=I83BJn-aQe_2Hqg z&E=H)EMhyF(#t2dkCw7)-Jg7+{aRY_wabGD#j(EYxxdDt@7Me5^5n_4bSf8gS>4W? z5d)WT1Y#h)wDFGm^1(2X1I**5Bjn~v#OnlRnP_!2esdgtz|mdzy6T7a*rjM60dA5< zujzY#ZX{nyE2l$V8r`>`5<*>q{5fOvVqj@m2r9hvB%~++4@C-l(Z#|AfgoQdY9ev5hdT z2kc4i2MY`&8Pg2cs+?5XM5cf#Q+2x9}ytudJ)a%s-aJ`>7CHuHUo*^}QK@KX*_Ec!6`*tYxkn->I-L$q~Bl{=lgPn|OF z@Z>n>X@euu;2dn{X;z}5GR8@T`U=i$SR`ElDv2X~yOl|{`}75o1_&n=J6=;KhdE=Y z{s+N~8b=VvxhaM}d62doj1?%?JG>iq3%c;;`mbG+8fPeURM|AEmdRVQ;vFwKa9>fX zVeWT9&9O8_vCg(*I-Kf@9V-(99D!zV8Wl0oEZh~k>@xaM(@>04s`7G%xQ2tFzP^>G zMOKb^ipd>5KEAx34Ap+*xCkznT!H%CV;KF+YWW)5v*<*m7({!F1!2WCCUs@^$+^}1 zjBbPnvwsjZ6Gy#H#3&T=%t89H# zhB(HRVRtMs!^kvG=m4HPhu`m`f|s~=<&;XM2v?(c+$K0sD*1s@xp@@*bse9s29}SFK>&9NDV!^qFd*F*H zViA0+R~nC*4pl#Kw%c?G;KmBN7^%B>+x(ERi{c%z{Om1~OT!3q->X5Oz1N0@Bw^*I^+n0mErl&<#N*xh7uLM{8Q+i%yFkx-95}+~CJY&~Gx~(=L{})U6>&8V}})i+4%1 z#&#QOvnonTN~~KdbyB#|HM2HkXPv+~1on0@69Ds6+#~<7oGK*(%(HPTW|A=hRF6&` z=%kk#&F4I<8rW=J2^wmtuyx2ISk`GpqbfMLvaOLI)jIp0CF4A2ERc2_SfM6WP?@0l zcYlpkCRy?pk=%!o0}$I&rb#y5QSoX0IS8WY%(^I(H5O9WMv?324}Prjx-CVr%v6ba(K ziBC+=sFnftOizDLbITJMZ_$(>oT0)oF+Vo>fP6HANVDT3-EswBnA@tHju`bG+wDv& ze@**p$qG3YK2yQt=I4)zdRjN1S>b1Nmz@VKIBTWRIo4ILQTon`8Mm0E^&!=D414X` z$PlT*J@SlaT73_DtZO}6^b2;DkZ$~1#!K%vg)7K?vTns**VOwOZ!*OO90+V!ISoqR zwC`tM>f20qN{Hd~@U@_QJBTV4{&xa&616#k<zi+G%HDlI4g6FZPC==m_OZJ(PyX*dHU^=u9n(7K*HQsY3}sOz2|BBtSqcm z=|t9OODzUPnJPYh{x49C_jq6rB@{5xlPCa50jx58%nM*E|C>~Oi(@;>!rrsz`-EJ7 zu7rE3b*Dn+fIeyD5M|rqO$)!sNO|kLyIl_#hIN=%B>Ug2zM{%8t1;Io`l9(q;*GpS zh_2CEwyG|ki=-;REbIZlQX$!G{rPOKq8hw#vxJvusbfZ<`a^) zqGKUPY5D%=uhZ}SMYsPf+M{Hpq~_ue*8Ah&aGFQ;QGAKmY@n>nF}20@ffrTw%e7c0 zIoQ~eBQqIPmpl9^V#srW8hh0yu_ff|>eV%ASc%2mmii`%CY6iW4ob#v_dZlR$3-<$ zi}$_XnkqYR=8QnBOM>tAvpOpc(?PtZdnYKi5GsaPlg*(*NDoBK)?m|(TH%+9{jP4V z&yZm^sw(2@gHCnnI>F&On*+*VIu0!3O7Jqij*sdgMRwmt3 z%-jtQ7}Pa53jiOA6qW;0nKeF|?@ zL|EAJVYGUDmOl&*Ur>rT@tG~>VXv<ps`ZDc^7DJWWGqC%MM$DNh8R)FMQ*Y?^n$ z2yiWv{0t@*7utsdOC;ZoqmGmN+AR@!ILiNsT=o&y%V!i^k+1=m`R5JXcCQHGR>b3H zzBU&_!~W%$T%KJ)U+T=RUaR;$tth)c*LvyJO6Z4_r@}U^&B1$1QZf~n`bPwh?-QKj zyi7vua8{LyW1Yf+=CvMst3As}#V@sxb>!c%f|DZfDkM1PL4qNj=$`N1LgT=_0!`!i z%%>ilTvwhV`Hpg-M~oyL%p{z5F)=o$&O9HhO)8OUqGP7FA>~CMU2QKeGtaBu&kHU0 zcrv9u1{CgAnhul2iFJcLSQ;z5^~>9*HFoFW?}|w&Xj$w;Gw1~`ZoVZh_h}|GR!M?) zw5_(|iL+t(WS2FK$FxVc^>7ig6m6mO6C|@WeE32$HfB0@x!~Hpk^+roR!X_szYesT zok?f7Y&XXU+Ug&MGuH6bY`1?OTO6%((KX>Cw{B3MvK%xfuD`fTJth6e|5{ygCOyFj zbJ_6cES*C02VgB86)E0OFNhvE}=E2zoZnh zZ)+T^RVamipF{IXuGZZ@I-D6}&;Hx#`PLnSn|s=GT3ZgeG#0`%LPkxxJ(xWXx`!PK zN0VyRks7ZN{y)C1agl`xxZHb&5wkmn1=5q23jXZlcz-9avR&5C0xDmxJSwXA*|L-1 zNxj=DM&4zWN^C$EJ1TwQQ4VPmbS(Qlt@6$_^RNW-Cpm?uLwu9+^7JR6Ncw!Mg;<)O z@uA(3rvcF|uUl`TkBbzetb?DnZUw^VFxQKP4=NHy&VPM>6E|-kAuJ%ndbelNUHI#L z)W@~k7aUTyv9C-$IX}|#fDp+B@d}OTW#3z6ip;0J!D*?qp4Q;tSf`~{-HXc8-^IMT z*`Xa`oD)}OuNkF`xTQ3k2`7L3z!R$h@#{X&v~a$HLnpW6M~ks_`@@gE%T6p6rqER} z;98}zH*+OBZ`y=$e*Wlg9^CZoLtM%J9RU-|`=u=X`Z#y$&T+Y*e3*+Y++Q8UPYoN7 zPfL^1@4)E{(-vQYC!(U~`$aP)-;E}4ZeBBf3EvchkHAPeY6qZKXDl;MKMbdOYc!*H z*I-l5!>ULaxBmc*3`=FXfax=Y*0{-!mU2_F4ns&{UnE3Sb!>d{wBmapqF!Vq7t_zC zfXfk0g`7i7;9}a4pid)C@W418FXo zRM^yu^*snmUS8apT%FjY1pt#ZK%eyrM%PY9#M$Nb`5-4vAmaRPhBgCZNy;=Z;8A2U z$w##ZHkpwj6TUOI=%fD?-Vo+f%3JX!rEpaypA!}nfN~6WeVTA1AxxDBKgs8UbD zNEnS@qpRVRJj#I{(Z(En`*Dka)73F>2%2oX58aH5&L*Ck;%WfAwgngo+G@BDJpz$r zJfkRX(m@uhDeLOwf92e|-*g`UcOig}gws9vk0;eCkVIvAu(dHC8VDIICK4!2--jN) zO^TKPh)cgVngUZy5nw5`zGfc-yvW<$I=e3O@w~QrUS5l%*8|C{>W-Y?c5bhx$;d+I zhnIxW7PXdB8Yzo_8&EK?kBIN|TCI)qE>G;pj~u=(sY!5CFMbF`OdK3om)DB?075t( z6+ubL+x}cTaz9(*b#v|YF=yaii7Vbd>?nux0>MYHBXJMiiK2kum zBpduPTHBR$l6MD~w~LF*gyniJjTX**p~?sb-ia2IlVF<@2wNEIZD&w2|E_Uyv_12d z>t{^wHt%PWY97xohendPqA>Mh4@hUa_A#kA%e76JdyT35G`KjNF{cmr^V=%7)ty(? z8>=mjRe;&L1b*wZY_f(77s*RL3Y{)r%#11L1n_c*Sr@#FZ{2i#=$et$mH{>QJ zCU91uUYW9^E$Z!`p1q7h8et(5MM`V$O(F+h%o^(>E*A>Mx*mF!rg?{ZVXp-oY3ElZ z&~ee8>@8RFvTI{de%-z;_79`>w9)Qx<~WwIn=kt(>*RDh7W7bp(Thu4 ze)LmtRV7p`GZ(pee<((Ciu3a(3YmbIwLYYX$#OpY8hlCao7(PZuTR%@_32GE57f~| zy_zu{#eG$bb@Np$%B5m>YpY7YJoh&M!X-nG=PW+gmAnX!v-ChIqW2H=tVL_}1_BAQ zk_(==jvM1w+~<3LpDa6WPghLknkBCLs9*1!Me3`9oU8-FJ_QUf{@EPOOt&J|P?}tp z;)eb`Pu-Ggb)!p2Im0&79p4lx>@mc7Jr_9lpwQIWHnCB?C*P3yJ2anczTC}A96KEJ zBvNQ^?=877x+rvmGIFp-!mG=wGw=4iT?i}BJz-7J>KQ(6#zPVVc7(}#v8`(cVMuYT zA+_@+2WohzRWS)fl~l+X_jVbPNtaD zW~zNcZq9_Qsa4FI*l|1Vs=b9@?Oxo^Kcq8tV9!?BjJYw7|J!)KwP-(xWG zXkCf^ZhY)r|u#tCnJz$8(Z zmbG$V^QG}rw4CxD8rmalUwc(OexDup|5&@7YR89kR96O@w;mv})}^jckqy3l=JpDk z=v?*%71uS+vbf$PBr${|Q~bbUBg@IfRb)Z7S{wT&^uhVGE=EKB*H&5Tr_pWkT5VSP zXBbF@l!C(3rXg`8jO@t9uZM{{W?d8J_pVcl%79s%iddQ%3{P4JTy#F@$O9rFcP!c# zC1_lFl9m=VFfbr|O~4&P@_I;aOVtpoH&NOuUF`lHVKWiDQC9G7bD>qyjMm*7_iR!& zMWNBGQuyrCU1s}+!Y6apdsZz&dBiGF9@n*Hzvu7H0H9H(F{lJQX-X2~ETQ4tIceqv z#J-`O$XZG!=X>2Fym%D|H}P!xcwuXWdg0>q>uu~Tj4{W&*kEel&-XVlfX|}!uA5NKG4C_PAYuM|ZW$&4D*2`_<$(s>3QSb7k|3ew35=(aKubm=?TiC@ ziH0VK{d*q=0GBuGfs!!uUTcp9{Oc^_5-e3MW13X*2|)|7S9-jkoo}LQ zEL~Z@IDHR*)8^%6TQ>p${`{GFA5RTnAUJ)U&?F0htMP5`)@CFyVA{UgxyTM6?btJH zuhH{jFDrdApl}54OLjlztFMxZ$vK!m35ajY22fUPCsTc`6!+q-qe@E4pRJ9o=^M!g z1iEEDFnywjx&a$wnJm^v(Dx zeoo}Z;QcByQD}v4YW_OKd9(PBdcomV2I7~K6>-}!KY%XLm8a8s^qtxu|K*>rk=XS;YZoEZY{Zas=5 zZ<0M!1_ILto+05i&Bbbf+a)sLoi3pS(H6ChAG*d2P{{Bg;W zF9D~My4r_=L%bL`;qCO(W~-!smTJ<7oOZw7C|#&q>$kI2Pe5m*=oYG+lR!96N}|BY zE6j8C1bu4b2eDp^n{9^1;f{i?K00YS5%`k4f`VqI3R^JHumWFwECi{3=#w?V%k+vSG0?|r?y(Y5lZ zweC(V2ar}A{;s!+MMSGMpyJoAvBYeMk6AQ6TKv)dTq9$eOHNQf_m7GtZ&g+R<(>HW z_|j^@rRy3Ox}cIKYMu%S*L^DS<3LvYiub6N%g3>-y)U?YI1K+NH+OV6H{OyLt$?-G zzEQS=^meKDV1>PozrVq3ARoF$9OVKw3aK2VkZv)OAiwg;RHF`!PjS9yGsGH$KfjlT zLUUXikj{7@oWAW*T&-S8Ri)!o@=AWWgosfj}0gtf=vZoA1MXc{3$= z$BEzDL!Ju$QmB0XdQ5Yw_kI(D8G{n~@W;KVw)o)L5W8hAU$5BlZ`$dz2 zeF}J{YbEQA(2ViqcWKX0mb_b}t*W$ug#D0-Lu=APyd9o@pCRh)7`j0C86!6-?lhmq zQ*CN#YEGLjhQ8amwOOiam*A}f5&VM;5LhnFk2#xz6cKQncqpY@S{v*2pdjg<`{l1? z!LTFxNmXsrq$tXt@^*=9t~yBW$8*%=pA@6neo!7hyQ*NHUAd;;st@#L&Y?R1Zd5nH z%0DMnQ6ZXC8j$^tI(1Me__a3r;PT5PvI|-+#TCx?n-HW9%Q4X4+)T4U8d*a`DVooy z;}kOhK)p29=!4%Y&r*wejLjTNXD(E)T@p}?=ZTlk8S*WEs4<@BzgZ$dm z&RHyI9fve1?b+d7A3S)VT5x&^R?JH?aS`*5Q-@j0j_V5Cw%#|(Ywe$#^}A$DmpzP6 zNLZH9l%K6ug){@nY8{$E4Lsj*2v2^}KE){JjG>-7JpZ$%w;%TRlvavb!gM(`>`y|c_6K22;P4D*2Y_3d^r~3FF{(dspB3A0Cj^j1`NJhek?mtFamv7%JnhTCe#Pt3|Cb57YX>F*j z-oLKVaO?Er!znuYSNz|NG+1U0l)MTWZQg!T6(|2`+7F;>(#RDGo9@l~gkg>l2%EGI zRN8lCq`gsd)-G+m)8$k7!ND+B+XD`{x4kyW#cw2r*4ze1LCLRW{8kJ6hQg%pwI=<) ezD80|Tfb;WYHGR8%#{Cq6N=L?P_I*UjQu~3$|D#6 literal 0 HcmV?d00001 diff --git a/pxf/pxf-s3/images/S3_Logo.png b/pxf/pxf-s3/images/S3_Logo.png new file mode 100644 index 0000000000000000000000000000000000000000..1d4cd5f24b370806ffc219459d97a510ea1b6d74 GIT binary patch literal 17178 zcmb5UbyQrzvmiXULy+K>;O_2$;1=B7b#MqacnBWc3GVJL39f^?4DRl- ze|&S!ovEo?UENh(+V_SlD@vgu6Cwiu05lnC36=NX^Y^PB5&r$VG~@R^008A8PUry(heFM8yO9bG0-79d#L>Q5vgL-)z>}N-ADe%=r4Iz zik{vE{)de|5VlWSe>Y+-%TeH{?A5c^bYX+fD=v3csV;qKycH%s+^_LDPL-dsezt=HozZGVm>Y1*`(1zyHRk z;Sr)bBYYTP^<-^krDaXwz-QHB-)HAd%GXE$;qdR_so|OzS|-r)LW^^&_wb1{<87l&76W#K;|H}`<3=2D0<11b^*6OvVx3@Y%7?>wY_|Puwx3{;B z{kONSh=5xd$gwCofYQTyBhnTN<9$1jfYRD7000X1zY7YGmVpldK%-czX}M}C$n%>z z+A|xQIhvR=d)fovsQ~~%PyTn%-rUug+|%C9!G+&b=<|P3@V~?V&@7+H|BJ-cR_L>q zf-t|tPa&mG(XEO_a6$#1z41fP8^x4YQ70A!R;^E=J?7_k8=xoWt z#>dCU!phFV&d&5s!Q|rQ;A-s2(f?<*{~jgoXb*HYcX4?iCd~GqA^$h-|D{)Qwl;s4@82~4L;ru{{$F}!D@Rwy zcU?PMo60!2nmfNw_Mg%Jdx8Jo6#wBR$nsBJ|CjpydvE>=`@T8C$bu~Yv(duH7+>k* z0RRyu83{2pPpA`LgcK5~#M9Ikk5e<5ybP|NNsN0L!0sd`z|&$scS^{P{XGbQF-G60 zZ&}|+VMV`J*FD(?6`v>wrXR|bA|om)?^hxt(c&6AeQ;*G=2w~2iV*bz8(JKyT#1Y4y&o|LQ6Vc+eNZ$3R~^g>Te-y4 zPU%NSW@FeR=F^!XaaR`d$#nCbO0tR~Qdm2QIyx{lArl_Bk-N`$IhyXK8-jLFg}5+{ zfXXUd>}H}YFBOf~V~xe{6M5{$#wB{fmNg0WGqRV*4EdKr^D+1oWqDBr9`kw z@)w81PfIeIj0G!O>xALl1jZm_{WN)L&0J~aMR12BhQS=nV6xcfF3n*@zP^&V2QYl? z_B0ePimJ(ux#>#rAKybZ?VGD*v-ysg(mzb8Zi2y9O|DH%P{pg@fG=veABSoWtO2H% zvQ&1^)1{ev~7j{#==P@d#^guhu=VfFEwQ`FZG!`rrQ zbqjWr|BI0d}lYXqi{pT?T2Dnm+fPW1T<|m5hHg>joiH|XMK9s6`>fE7sicr{UY`CY|kWgE? zs5At?OzxAgYQe{1d~90(>`m(Vm;%HOeE`a8+{9{{@z`La1u%G!aUN)8sd*sA19)rsr* zI3_*ERN!l0FO;P$AEVu=5GL24bc~^4>z!iD-r-TfYlCL~b9sS^pnZv_a^lB!RBXb% zTg3~+buB`YlN92kUxB4ED!le_uH}YI-s}G1$Nt-|uF!)RV#g~#oV%esB^J+U6vJia z*H#$Kn{k!084Pq?vLSj=u_Gd|>`=%J`mOx=z+N1>;w+j~G&H}`qyxR(o~hGh81(L! zTC00#%A7`(LMJA?E%$q@Qx+<-;Uhq7+j0TGc4Yi-*gMfh2?}ES^<`XX4V$XcVm7P) zfnfg+`#gRI^zOXYNBzyK%y*&(gCOR&ytH@d|1a)TxoyqngNC%tF1}Y45&Gx~>*cr* zDFH&ldcVi{g1j~|GP&jP=HM^hLlXgjROt*M=1H`@hy$f`|5SNnkd@YPLLR{oWG}}B zby1a}OT4>b>wsHwp@Em2Na}kfdbd#fc>n?QX+Q^8Eu;X?OvgGrL46`|JD+ZWNZ|B? zk*%qGcGEGX>yO`Yx|c&BIbn0iN2R&)^#)a&^(DsXVOevqwWX|>rwDt%G5R_R?-795 zL9NHw$*r~GEM(b_Jp73)*?ey-v0sKGzxsO|4hx@{3JR{A&A1+&if9a9{-;NxD{JIb zhkP?0!sBA+Z3dK#-709@XZ1-!ipc{mvK5vmTN|@GGM=GIx2mDak1VpQ3>b1xWM3-< z-7*Hh@<L`vya2ge?v1$g#S(2VSMHwr0F%b%A0j_8uqn1&Tky9NZRC5 zVl{EP#v#hJ{hpVQD)Ny=UC#Nn=9fj$BKKB(Fm>$5VJU(f zUjurxm;R5xs^{Fi7P1zqJz{1YmOs1cHVvQ7&&!RaeN=h76bD0hg9|J+XT~mEYBFo( zu~dtspOzBA*G~xf7tgi@Tl!}gF@0z<+jc4|MV$Pv(K>*-o`=L3mqah}N7a~-pV@1l zx$Z;?Efu5q?kKHy4D^msFI)MY+|^W&`=K(hsms*j8o3X&npF4=zJEz;ma`7afB_hDPtr8H-9)^F11El+%yXdF+3(l*)8DGI z*yPeSWA|O(y+WU&O|ygcDcd(xO3hX(DT7{x)4o$J>xhhd;v_H4p-rI z&#kZW5xW&aAwPn;;nf0Ix~EDMCQPu9{xTJZNK%2&+U*shYSFXFy>4#FWXFVfpI$IsSnb7;mw?P%hz<-Du6K6{0e7o%R`qUr7j5#c{8Gubj6!fPr_dF|A>B;@!zzi=$}&pnb#$`{jJPXdG*+n<}RJ~nLk z5$x^_vY$i7!&-W>$LB2g0Ex{G%No)PQME2N$ct^47~pfxP(s9BW_m6Ws;@41zekVj z7v_K01xZdv!JueiFjkfH5bMP@f(N*Ok%QP@fK0_-5PqnBS<>vJZRmQ+qCC~2GnZ2W z^N&nA2M$WzjqSq%TNM{QhuOvz#`rPON@cvi+fy!q;gv5?Eu+y?@8txzxihMcVF)(? z`=_n^)(heo%aF>2VmhX`bM-rUf!&USdbhx_ps} zYeI;L>u6qfOozCx?vTpb)1sl8LA9GXi}9}|W1NbsB=Y3AJE#$58lK5-Z(Dqh1;z`9 zP9cBf*vq!PP$D#eIfGZBT@Cu-kV{f&CGXxr4JJj%-9u=pTa9>XxX>)hjFvvbhLy6_ z)#Q0pX6mweKMh_;Oz#tfs=uL=y6g*P6L2f<4$P{L3~F89`VK z8doWU*-_9ZTk!SZjmC}_*3Nm#laB8M-wN}u>;i=moBK)sL2B^aZZ82ry)-fSrbXPc+_;ovyC%R!P9!Tw^}1vm*$hGH9g%4R&m9=*Uw4xERn zElf%A?N}Pc!NjbeJDhNgmh+$VOR$=_3$Koo6-Ia@hAgP^e9RX^t9`Vwe>UyWA^=;R zLo3j#opjTr6Y7?Cd4FSb%3cvabI~F@NLPOF=Pf*=z88(gnyO{>Jd_2{qYP9ZT=?f; z1(oi;RTq<5^)xt(lbIeL<0ZMDtTu@`){HiTX3OBC)CVlT^$AVf&HfCjHVNH=JRU@h(tQ`?( z2^l6{1JEE-zz9GQA%WN|Ko?JT5-L>Vi63J9)YsZwT3z|POqjOqC5R`Ej6KQmanhW< zqg;X-9@HPn*E?yJhZ2r8`6%A+qt(~fq6O@R;>Dfaunf$RF=;75M5NRTkOReo6AxKY z-zY%ljL{x2-CKO9tZN?7uK6RtXGI`^O~xy%kG@>`v7Ma;(wGPAHKVj;;CFh;>+N4A z1o5)6Y}5;gtCW?*rk0ewmV1(s?==BM|6^7#5^Z2PO!4aS67>ySW+y!=tm;?llo!?v z_e;uvZls@@ypQQ9>*?8K!x)w}H609!)Bu~vFw7sn+{wUev#_F&tTSrpZnYLm^#rKN z*tvzZB9brs_IZ#>ih*}U#X)=}ht)Lq)YL=|aR*P6+)Jz-5|=8ui$_S5lWXRX!(}FM zjU`(Uf5qMWgoDET>(Zbv>YP5YA5E$OL`Dl7F_|MgxPsiUev{}6C^bMxeOY0KZO zr=Bsxx@g7UwYdj^IX}O8j>SQumer0Hw}hfhaJE-YPdB=D%IQ!&TwXHA=&o;SP`?#D zsp#`Hm7<|+94C|e*$q+oMz|mk;zW{w&ertqx7W$-%Vyc&{r0uQAXr2#S?(8HN=d)E z0E#Q&I}b{}4S?XU6^sG}(1&JNDk@vHZq=Xqa>QZ1Cc2hk51j!>G~69D&G9fvf)wNs)1d?IHpYYV1UM*0q8&6b6v@5>*=WOM=a$!FImY`NJ@a9H)bWG*p@AP06 ze)p|tC4E>vw(1X~yEs~&`|!on&fj{*KX8uH_T^9GB+d!nBfk17cVsWA2ZdKNCpHV; zvlrO3##EbK%ccp@S++Z4utDC}H%g!1bPNmXMJuS{3shk$#aY;mxL~9_Sx}2B9kQ?1 zw@9hJ=0tWs2|qZ!4F6Y%s_AN95&}Memrjw?O1p`bT$oY^(U7hf)iN|Yokiz&uDGt4 z*E?K5z&U@3hEHvG8>b3c?`1Tv5`%;CBKpyI(9eD7fqx~fQ`K*=ZI%kAC)+Om-4nJv zRHYwif%zzeF6hl{z)^<5+WG7cTngF^=9LG%g{?5;yS&X&Qcw9L0d~gTu1BPYLLRt% z`jF9g<4l=V1dw<1WM3c0qnUp6G5&Cf;?Kf5MmCENtDzTVKywT2VKibC&dh;1cyjNy zC1rGUN|lhgjbzO=GP6etN}rJgHjirCJ@WEg{)<@;{DXr~1Dy0!NadWKd=c`f2hNz2 zGw9)mavG<~hcjr;oNe1xaPTC0cLn}*b73s0Z2I#c>*JmVV}o?31dP6i%^D|YzXWjp z2s((|^7|6chAeP-;(6ml50!TVKpZ5CXc;g!MgmU@2rrDU_|*MSWPhJS^#iHdf%Hd_ z^9+KJePY{&=1-V?M8Px#cwMMztmMlCN#cY*)R}a@k?aCfNdW##2i}EpAPy?9D>Lao zTM!dcUcz*BB<7_}CnChrVggi@w8N&#NSK5zpCEmf-8bo2dQgjcIfe@TENY%eF8A@dRa@-7f;2ln{6 zEewa?l_<~!BJN}iolXnfb>ZwF&uSxZK<5Qu@C-aj#|Q-QI#5}mQNrT;!?Sjr1xcR& z+$RE|1QrPV044yJjz7dQ6d&W9Y$~7n{l{%%@)f#)kF)^A?H8bkq|;&9>bLdH@kBY6 zQ7Fqn%w0*wQwUNll-#S^U1J(Q68mnK3tX@R z0fM{lo`JgwFxl-r`Wel`6C+%vyxXjoo)_~`^gB?A#*4nq zhCmFuYQEH)BY6k(iGaj|rtdQCIqkHR3%mnu+#9^H1Mj;t4jFdQ_IL-}?Yo`GW8NcA z#z5j;?diKU94~<&qNFbG{t?44VZQ404!jd(4r2bF#hyw;FR05za%=;L+lHO6c-_^p zAgFwRfqLgHc+(=hKXt#GU4EOYvA`mA^%-f3*xs@wY>rvg!))@zJy!`ytfM-1>e;0J zTgLXl4MQEf`GJvkp-TVw zRbo4YjBj~rrre7XZBH`mUBeWH2x%~ZE*4&?qy0)Ge!E5z{$*BDw>s`UUCC+m}F(aI0O1PJTcu-o#H z+aT%DIMq4tS)IGmUr#AVOtFhTBBL>5$$Leuo9b<7RQ7KRx; zPU)Nc)8T@K0;MUv`ZS121&O~L%}YNsrQ)zxsM1x2kts1r#&}1Ijzqu!cie!o&D}13 zw^aySd!}$;Bd*mf;S_>{8xto{3M;3gWy=jgoxGVu^3?MHy8@@818XuOi2i9rgJM(( zUh6_$6f-}Q1i%22rHf#ND>0!fFHY#Kd2hA~Ww9XrGA5&1GTl`<`8WWkI25QsZt1PU zeo1RyPvzUQviQfP4wbGVN%YfIrm=?Qf`;FUGPtxiPzO>&2sg>1OdE-axrcJ18Va$!P5^o+lvu1{WTSQl+Q4mfxYVyxpaBU#Ji4zPK7S zS5wjrZH412;XxL;Qt`P6oEk9mmZY71bo3ZWDa#U)!R353`Zeq!n%~2PCNCUD(mqR9EM7$$BA`xkCpIP;lG4J=P!;=gn2p_Sm$9Ap z&v~or((gc?nOWJsq$acJ;bgPv+Y@}7zR>BCX<-+s^D3BgM0Sx3*H4s(yF%XPklO+C z{`x3(zhSlF_G$C2d>M9J8yC8KM|zpXWY}UKaw0!O)4*`Bn`pE#CM$po^2s+wZow?UQNsZnh#g9e+`FH?}Dt7O_oRQyH3WJM0&y~r|(E|V{sn|xW_&60dNc|H;v z;QYoi3*=Rdbs8!;+JHJJU!~Ab****$Ug;p~+LeR_!Gn5$Je+}w$2MEdOD&%Nx(L#n zoEGwKlpY!h`!ngZQP)ftu}fS8*{d3f6Cprj2BWG_gopOSd`XIowK#6fz=cdMa+9JVjd z+DN9f+9wh^OEUHKwpT-ioM6wS``sP!)xPke!0R|XQJiDN)`rhe(&2LuRQI@x$#c$j zbjY|@c`F(Ou`;HzD)f(~us}Zj=5r_rCgt%HR6BL1`?8`aYXE1*p=Khj6=^PT_o2Lj z-nb*lf)b`>GKpaad*{lPNjB`PRQwX~yT zn4XT5<6Cwc|5E;K1sdS`%p4{VS5qO;iDPdhn4r#l_t+*?RpT@1oR0vxaR*FnwQ{05 z2&){{{;wAGAxt7mbAM!n%R|q*IM2mJS$bbSJC7cErG!&YLhWM(dNERR4wNdl+}STu z3d{@B=*l9StvpZX;}m^&-E@ulQu(ng*!(Uwsd7o|I+koQ6XqVYz)2iH3)=S(OxC)S@U?9$hlp2Z8nG+2%%4*@8djiVl zyD#?rv<$@36w)Y4s0L(AmfbH5ycC1!Ytp`vimk<#w^G1X?&JW>_Z=LQhrLkD`i~!+ zfhQy2OEA2a*YAlQmIl{WN_ykN@CF;`)OaVGt_l$ReY4atYFSrTWq z3RLujlvL_UskoeS6V@Ju4mz(s(eZlV=3j`B-3JXxR`v$ctng0NxWtdP>`H2XKT&Wt z@<5}W;2x<|AS$ppu~~c?<^W&!+I}xKfd0~AH3yp#(Cf#KRHw{q1K;ZMv#qdRl@FE9 z-Yux_d6MX-W|w%Aq78K3^~C#nzs`E5dLA68v^|^CWg(C&2|-1nf?J96eDYWo_5$VJl#2L*WN$M_i(~B)3;$>M46lwL88^7@q4T1b8X1kyGxy`(2egv_Wm<8#m-Am3CbLn>uzPdK725WH2^Oas-(S>hvH}Z8e0Sx>Wxs(i3 z;fdeAmB4i9u62HrGjOVzHE0YDxQhUyxkp9buCtZu&hX$+m8pY`^ zN&fN;$!Lc|mrQRbzr=YiiY&pDMTgp!%v(+4`CLHdZX20SAEYmWjk_Pm+4>X~FK1Q% zed#L&p^7+Df8CgkQCy0!=FjmqX_WzDn!hf1qP^qLMrAXsy-EdpRZvO7HlIS!#j1qF z?ms`{!7G8sW~Ht? zMrKcGOP@k9y2KIUnO9s4?w8M^-&?sh20UOb-jBxDUN)YMj7w$3N1M{fco57frDGTw zl%PF1(a1RWV9^EgXXpSUaVAn3WbB6IOzu9!lq0nPA>6T~pXs*f?fc*kAZz@eP)v5y z-FKc~`zIiUiu3Ew9OFsjjTq7}Y7WtP|8)Rfu211@s7rUl!k9dlxlbb&DNcCV(FVz( z#LcEpBn$f^1ksbp?JKB@KL6l%xIA@b>H2=EDl|~7wp@&rTN9Petx9zX_F#@r93hL- zzk7Xupg}G30LG?jL5x>)*^|R`N2aP#;-RPO1MLWWL`_i38-yeI_?3&q)7FNEZ0L+K z6iae%mrk1kE{Tff1TFdg6B7dk?b@F_7x z@6~T*>uwBP7xbag9~*;sLbs^l1jEvPV(s?y)a*euzj!YGQ(|C zPvgCXv^1AnijNxR%If4zi1#R6#HiD5G^J<2{*6;u3>67B)#A zw*gUF84>?#8&fqBPMw@cRq;^#9l5$MFq{tK9{r3pr3CNUUmf9Y>JZ-3&psBRtn3ew zbxGK@@sbuu9MaL>bS-+1Ut>38`4`GFI32{|^dc|c>!rF9yKsJwvgPJ#QYWC8ur2An zddWLVV7@12tsCr8;G||hpA@C?16!_nT{}~?D6Hj6Eln8A2R#Kd%dzu^T4XEXL2NpA zqrRt~hN~1rjxWFJJEE#0vuZ`L?q{q#h!iVkf=(Z^rA>aIgF3Mgnpo0>$3*bbtWoQw zVVQZr;uPn%cJRQmg5%nweCNfFDuR}s4!g=H-KhfT33+@ddw>j%IrLwwPv2l*yPIwr z)N8jo+svo!4etVcE`0zk@&&(1*KLgJC|x`Og?~bpqM3XIcjrL5t;aSc?boX+?2g}* zWH;My?_1k2-w*oE5m`H$DBUiR5m0cTC0qs5+y+A3YA*u}heOX^BI@n?nqn2plPHD5 zw!}VP8Mz6?l79gz1E}#+Nfp1(OmiGHZl)|BsJQP$sd!zO^Kx-iuoqqB9o$TLNz`tU zy8JthAm+FS4l=|-F+Qb{$1>{9m;X`+TpoG&eAQV8+4$0A$~Qp%?ebsii>0!<=t}RMBmhGa3e{xae*6rw+8+!7 zvkjjD@?2aSzbgY>^u;-MNVB3O9B-?-xzeD&JGIo=V~prm3O^p!9ehdq_T`U1JK-)4 zayxOeCJv?ofW4$H@h?K9I%N|5_+B^SUB9B5e20<;Vc9!_^24UZ8%~2^tNwr|Y7eH_ z!b6mDm4Bv{jwmATN|+ebl=k(H;vPatFNJ{r`8Q?>Se2M?RJ!+ z!chQWvfMFDwpSGxq3qCRn-3JNSQYOzAxfT+?P%FN+ArrYwXaWx3Aar(J}BXWjn0vJ zf^I((wkEHm_0uT>Faj8WEbTa09>$`a*;JJmy~uef3Nf<}<13quziD463M?+r0Yn^@-<-Pb`V;+u4SU&vw!*zSo5A};GsNn*;Tl@G; zaswm8d%Jmcd7!$6WV;;#c~2OJNg0bzy!hX>b1}j-%K>)e_I7p&9o~0`jCOd!7DC3o zuml3Qax@(vZ)x7R0SR<#JHVxGndaaT#;a^}wlY;bvp9qgh;X)QXoAi55Q46I<}|NUIX-^Ru%< zuP^t62up@o*?LW6i}7i)1_(P_4g^6#c!(Z`Eo=Epn1TY($(Gecv#lq z=yyf5R;peWn9?zY0=>N-+9AuB!3OBa^mu*PW2mE6NXyq`KRan!pYw5d3njE-bSCYR zmk99T1ZcL53U`j2fegG)xXykuRPw@>e><1Vemhxz9kaEvI$u_4t_zB35R-O26Ig#3 z2U)$%`@i5ZDyG+gRVRqw0#5YWR}N>(c2X^~;tmgO7V9iy7`3WJ*L~PZ6X+LOq%(Hp|2*J7xF>wa zSMZk`F`VJdXHTFYH}?yq^Sl?uc|z!J@csg_10j2B7R8R8@V#}Z&bfkMxXSxA`wadF zLyc+XXh@mwdTQ&M&N>UmR#a}-yNATg@7Y$iNUBp?%zYomoM;~Du6F7^%m82f+mi-` z(M$o&Dv^gFrS`P6wDOZc@GXY_eUQhh+q|`wty0svU)!#}`sL&#WJ-!`7I?KcF@Mz5 zdBb$p9RNeHv&})p0@@av+`r-JdWZ_G@v)E?+!SY2+aFgJ;*_Axg2Kb zxH+2l=2-y-3CC;4juJYM%h&#_85g72!pJTO$Xn>0@=l(#(3l~!07A$Tnp zK~FK@6ZU|D=3i&QlE!jbur63Y7jlI|MY^NyFBsiFacjq!<$cO$k5OkiF?SS%A)JEQ z-sB_p3!dCUm;d4qc1M~%7h&>^*!;}QcE_3T8OqYK$a9|v?6cA1a8>Nm%WRb&7vj_F zUDtZwb}G20{ZKsm?GJRf#)iOYGlkKhrSKI26Ti18dq!8++fx??KL2E>#j~kr8SPb_ zZN15U)yE`KM{V9V9rGoZ#m$tl9%d*#?7dhGc?;+=CBk3i}P@VQ#$+snw~ zvCQ%+^g{YBxY+JhCmh5Q;D1k@E;7B*ObMmhTRE;sqMLX3u~;AaqujdFb06@So>7UI z=t8LPWT`<8^1GI+D?gcuY4EahW!k<6T z+EC+vZ*#y7b{Zg|NjA!8cixdLs4<4Xcd!eR42-6T&ORMgdRM8Ll7>c`>bfEJSLe%U zYHQOeH{6ht&$D0k>>orBzlD!c;(q7I#5eNd!-V=M72$VWGjK>581Ph5l%-M_CsW?c zT33#FvQQ)L_c)zx6GH4V$`)u2pIPpCSwaJ6O6>KplYB!0aWR3cp7}KuvN>%@h{aSy z8Q-0*UY_@1jF`pXJSwl-Ub3zF$f#howEyb`16aT?eAzvKM#Mc>HzCZx zxV+S0>sA|dy|_O-G6Dzj;d{BBZuq8`FKlaS>R5 zfGVMUWOe{97^X>@B_xG<+KyME_Z`8rjttnDyPzRvvCQ^u)bqLctAmkm>C?#YLi&CG z)212vh%*F@XShZ#W>)dEsq3YpI-1*lDPIsmIkiOOACEfyxg&sg zH_pw-VY$&>Y;A#n@W)AqK0!=ndAU4ICaYll&r4)&=VPoJ8p9MaMYT+)euCas%NAKV zxtF`=P}e~=Mv`f>Wmc)jzSG!LOc$$Tu_e8uubVe_JpNZa1c#^ZL85PCwbpz@ka5md zuvE$zA@mT^+8|UqCMA4{i2)ve0w;IkOjf)0YF6I-7PIoH#B)AjeHQ-y)^X;?BfT_i z{IK~*naq&u^xfN_-LvNI*=2%32+ZE_Bqj1V1+q5Fi$a&SCN#Vk6)8++@>w`bP*uF} zyK7sw_IaK12{}B zC_$kBU%f)$tR1`ke6=9iptOU|oA)+&FViEdGY1ZpFcDxx-+(WYE@yOyM$Kn-G*?0Y zt>nA^Rr>l&&Y9PQx7pAys#1EDUSqv{Fe&8I!+V6O0^d9yRcWecg0p}9D8(vxv~OB1 z>|m$bk`lRPlTMfd62)U9#CAqKHi*qH>zNdUMG%T{cxUN< zYf0b1=W4`Pa*;EAJ`(;LGfYfA{u-GAtbio(%Blb$Z7V_k6RI)ud4KI`?T)^x1XE#P zoPnV%E+^D%EZFsa+Fg9&x-K*%w()y?v$2kjL$GDXWX-rk=Yxg*Pa9iX)y~>ZY4N_t z9_Kear0A5oEaXJ8Ycu6HK7vbcng!<&a?TxcB83onm$r%Y7*YbKi z@7Ddh5aOGSuq^k+GuI5?ZVZpSi@~I5QUG{gc*I%HU~7)3#yM_`!+IXmT~rLTuYIhQ zi9KKo!eTCr%wPlm*Q;#rD6}X7K?_I4ppP85qEf$&->yX7RQoQ3NG!jW2+hpR-9F}V z2g2v$%E;r{?#M>q%DN_X?R4o^zHY*3_0`@0n{{x8eGfi_~QnK3v3Tz`)oSY z0zZ9V)9M>(PJMMb5xy?$Ls00JaXX?tq0 z|M4&>X=;rK`!~B6=P5blp2PZY_gpb!G?ZuWJrZG(P#s_B6f$asSK7r7H}vZ3sSRZ) zF2+8h9Hy?M7@jrHKItQV?g8*^^wE;5UMM-5ENoJk7rA|F+gYao;zaZ2MaM1&N!UfN z$Or~iR>1(n9tLgP0b?Wqjn+f?6~r4n(_SDT7t!Yh0CU*G9JYYXwD@6I)%?(!EN%#I zFs#1VezMOSxTxQW*Xd1aYO?YSC5S``+q<7f<&5N}s!g1J_B0Q39l>xwl6OwENYQ8% zTzIm|j3+H`77Tpod^$zRXR*fjubmnRS^oi8AVfvMF_=fW8@1au%YkO@ASKHBk$WAX zNXW{V3Dw+Qu1ADM#G2W$=gW&>`1Uv-)E{zTP)$x&iCWjwzrb#ag;-ZXiMO2ySV85A zunmNz8!_r)6^aTLu?wD?WL?uxU)O(?U49!D zMwS%16L%qU5v~M((fPc?qWhu4kIC$hXciY2cS#r5cNDvcGL1@PE4Hq=mPKzAkyu+^ z^jV$J&@xK1hbTPv9oFth;eDx+hq~++#(k~c`dsT}0mb$j7^9+_uXYYU+zGV~CIwmj zn)SaP>}+{S0PG@-O(EsAY)_h9kLuaxoYp}@1W~4S<&qPGjZh2wOUyXl6B50V=tL39 zI=Yai{?T*Xvqm|mb4F2&X`c{L=A(nkA?Ch25Y8+gtUa2 zo<*xh{W+OOhFgo=a0N2TDqHCXMW+7*>z+R+R{2m(2KL@$F!`d}jt>89-&}#UTC=(#Bl~sO*^~38w)ptI^KKgp9y%HNfM8BwNZp2OMrOVfY`?OKp<$Kz zN+f|6As?Wk)eWKpy$4QFTSMAEFm{Zfm zcbOLLg?6>D79Y|tH(r|NeN@rpn$2XHZHz4!9Qy9gZ`64iFkr^3MxTw;6>}R$H>jcWb;8ELelF! z)~WpT`j&<0MN$0}n&P4A9L$g(u@9Sn-zLk-o1}VOlfpN4LYiT1BObkhxg`AfyZfN1*H?D*hi-Q`dl`RN&Z8g@#{gAb1arGIwR}cwP5xIMdQ^Z_ zVNpGk3O`dZv5v>Fbjyll87tvxbzQajgmOa0YBVBf2#34W(RbRAVJD5W;8c{D3W{W2 zj&6*$XVH@`wH7KMYJ|Rn2Z= z0Zl46vee&V?GR+M-4ur8{3S=#3giOnSYUP&hKWVjPrg*!XhGSHnq?XBR_6ST{9Q_! zQaDcm@YKyH+s49MKZsj+nF;oBCB?vvqAY>!cgE<^0JvMnaRc$`Ci0)=mNXX~l@?=OVER zAMMwUFsr&~r3HYUJjDlU;WjTupX^+@fR93FLwIRNX_zQcnxi!x+B8Of;=UyAfcS3; z3S&L}aM1{F7i4eWs&Dx>YQKH|MzW=aAoqERz{r9%4;o(H4iQH9^&Wz3pz|HmNaXb* zW~cCOow8(0d|(5fH~M94isf+O#~=qc?~)yMWPM6+YnczShMFt)X;l9X$4R<1P-%po zf1F~n#@O68$5;eLs4qTr@o~5H8+%7F#XT=4TTY6Q02RXnF1#Tpb3u;VF)VQy6TA@w z!Am-=zB)HaD#s~1$?%xnl>71z^K%&lSl4pobWphs%OM`F$RrF zZ=4nsManT$4a=gGvw*|T$aSQ`gdpSK{g zXHhG7S?nPMG9T%ep*Oqwv6yQ+l|;39usAe{gl(7={(au7kWhw_bwy$pCbs zON8x{4-`1+^cHcH$&(oIOa&7ndV2&?+jLJV5;o!U1( z4x4D>s4S>>deShw4C_M3AJmg_zI0R$=|sVNyqCCWE}hfr-=@UnUPpiMvLA$p9V5iY zLZwEuU@VJdEFY%Bj9>dMsP{1KRL(~XA8JoQxvtzfQV&KGsNz!T9`5qoedD*{0}!|7 z+1ThShPyTI822Qf=*w`k> z5!ua{$vcX4Ho?0>Mb01a3dj!F2i__Qg&xGrVsg9QNz>+!Nq(9Ai8eqOr{2y*ttid5 zo}o|(sf|wz*w|wqTz@@TKXk^P)hw;CV{VvPi<3sGS*5l@r+P;gNtyB4hHwQ}oIs!0?2EEeM#Y7`@Sc5xcX2N&q zE>A`|g{DM!qx}7mjK{wIsWm`7y|?o)ktUcixsZU)d(T) zffnxa3v769{bCAR!2ZeJ7^V|7DzF4RzVX9-PRjZW0iAynX(Yy zHq}L;22XKXYgleB=DgXFVwX{E?ci__y@&>|Tj+@l$B*tT%Res>=Jmi#l?7$eRbw&aV6q^Ir(L9*Q+tuMiHEC<^- z$%$6gy*I|udAtw(PPoZ5yrFv>Br(sM_t9QE_dlbL{*K01;Hg^H)TO2<8b)*SMnk3~ zxc$T-DoZ>ooEJ@S%xab10-bqpBo0pjR)t7M=CEI-dociX38P^mTsmzv> zLylI={Kht%DLf-&)7<{MQMRM8TQ>TCy%78w&G(`+ZQuqc8mdQqCdii4XJt~>M2B7S zjYChCr5TMHdvA8v4ST*v@M!s~@7hzBW=lNfjorCwD7w?avIsCJ%6=^RSzzO%4quEv z*QzR^qVT7CSivf<6wP1@MkI9}f5ns>(cAZ6gR6+;5;pMl1My=v!m5Ou5kgh2$t z7~-W@Vp5+3qR)SYa%pxAW!62ZURKC@F}fp|EXC}SKG5d z?QDb%b>wT-oyIB92W1^^mG|6Q3rfB)cvTdi06qKU^+OuHiI1|@{VSL+SW+ZY_J7Ty zCVz|MvAQiWN$IZFqnS^+8NW$Sm3*^x`=5z?(g`NKZQJ{nuHUlh=xc48`>l_^OkMx) zMM|~*t#817<@PUId~B*;8>F;ed}8Xed3)8FY1?Cd@oF~|M+&&#E=({fO1Qye>84y9 zJ5!du=qzyJFsAs%#d4VwilDV?Ct}=h6)Ic@tp+x2m#sbCC<9uX-+rca2}O$kvA?!e V=Lp}d)&x2=!PC{xWt~$(696#V#(Dq% literal 0 HcmV?d00001 diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/AvroUtil.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/AvroUtil.java new file mode 100644 index 0000000000..4c3c27355c --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/AvroUtil.java @@ -0,0 +1,104 @@ +package org.greenplum.pxf.s3; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.avro.Schema; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hawq.pxf.api.io.DataType; +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor; +import org.apache.hawq.pxf.api.utilities.InputData; + +public class AvroUtil { + + public static final String NAMESPACE = "org.greenplum.avro"; + private static final Log LOG = LogFactory.getLog(AvroUtil.class); + + private AvroUtil() { + } + + /** + * When writing Parquet or Avro, the Avro schema corresponding to the database rows selected + * in the query is required. This method provides that schema. + * NOTE: the provided schema specifies that any of the fields can be null + * + * @param id the InputData + * @return the corresponding Avro Schema instance + */ + public static Schema schemaFromInputData(InputData id) { + String schemaStr = "{\"namespace\": \"" + NAMESPACE + "\", \"type\": \"record\", "; + // FIXME: is there a way to get the table name from InputData? + schemaStr += "\"name\": \"mytable\", \"fields\": ["; + List fieldList = new ArrayList<>(); + for (ColumnDescriptor cd : id.getTupleDescription()) { + String fieldStr = "{\"name\": \"" + cd.columnName().toLowerCase() + "\", \"type\": [" + + asAvroType(DataType.get(cd.columnTypeCode())) + ", \"null\" ]}"; + fieldList.add(fieldStr); + } + schemaStr += String.join(", ", fieldList); + schemaStr += "]}"; + LOG.info("Avro schema string: " + schemaStr); + return new Schema.Parser().parse(schemaStr); + } + + /** + * @param s the input string + * @return the input string, surrounded by double quotes + */ + public static String addQuotes(String s) { + return "\"" + s + "\""; + } + + /** + * + * @param gpType the DataType, from the database, to be resolved into an Avro type + * @return a String representation of the corresponding Avro data type, surrounded by double quotes + */ + public static String asAvroType(DataType gpType) { + String rv = null; + switch (gpType) { + case BOOLEAN: + rv = addQuotes("boolean"); + break; + case BYTEA: + rv = addQuotes("bytes"); + break; + case BIGINT: + rv = addQuotes("long"); + break; + case SMALLINT: + case INTEGER: + rv = addQuotes("int"); + break; + case TEXT: + case BPCHAR: + case VARCHAR: + rv = addQuotes("string"); + break; + case REAL: + rv = addQuotes("float"); + break; + case FLOAT8: + rv = addQuotes("double"); + break; + case NUMERIC: // FIXME: come up with a better approach for NUMERIC + rv = addQuotes("string"); + break; + /* + * Ref. + * https://avro.apache.org/docs/1.8.0/spec.html#Timestamp+%28millisecond+precision%29 + * https://avro.apache.org/docs/1.8.1/api/java/index.html?org/apache/avro/SchemaBuilder.html + */ + case DATE: + rv = "{ \"type\": \"string\", \"logicalType\": \"date\" }"; + break; + case TIMESTAMP: + rv = "{ \"type\": \"string\", \"logicalType\": \"timestamp-millis\" }"; + break; + default: + throw new RuntimeException("Unsupported type: " + gpType); + } + return rv; + } +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/PxfS3.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/PxfS3.java new file mode 100644 index 0000000000..3860dd419e --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/PxfS3.java @@ -0,0 +1,188 @@ +package org.greenplum.pxf.s3; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hawq.pxf.api.utilities.InputData; + +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectInputStream; +import com.amazonaws.services.s3.model.S3ObjectSummary; + +/** + * Test S3 access, ls, wildcarding filenames, open/read + */ +public class PxfS3 { + + private AmazonS3 s3; + private String bucketName; + private String objectName; + private BufferedReader s3In; + private String s3AccessKey; + private String s3SecretKey; + + public static final String S3_ACCESS_KEY = "S3_ACCESS_KEY"; + public static final String S3_SECRET_KEY = "S3_SECRET_KEY"; + public static final String S3_REGION = "S3_REGION"; + + @SuppressWarnings("unused") + private static final Log LOG = LogFactory.getLog(PxfS3.class); + + public String toString() { + return "PxfS3[bucketName=" + bucketName + ", objectName=" + objectName + "]"; + } + + /** + * This exists to support tests + */ + public PxfS3() { + super(); + } + + // FIXME: how to bubble these errors up to the SQL prompt? + public PxfS3(String s3AccessKey, String s3SecretKey, String s3Region) { + if (null == s3AccessKey || null == s3SecretKey) { + throw new RuntimeException("Both " + S3_ACCESS_KEY + " and " + S3_SECRET_KEY + " must be set"); + } + if (null == s3Region) { + throw new RuntimeException(S3_REGION + " must be set"); + } + this.s3AccessKey = s3AccessKey; + this.s3SecretKey = s3SecretKey; + BasicAWSCredentials creds = new BasicAWSCredentials(s3AccessKey, s3SecretKey); + s3 = AmazonS3ClientBuilder.standard().withRegion(s3Region) + .withCredentials(new AWSStaticCredentialsProvider(creds)).build(); + } + + // Used within the *write* context + public String getS3aURI(InputData inputData) { + // /pxf-s3-devel/test-write/1524148597-0000000430_1 + // Need to get objectName based on this string, which is in inputData.getDataSource() + String s3Path = getS3Path(inputData); + this.objectName = s3Path.replaceAll("^[^/]+/", ""); + return getS3aURI(); + } + + // Format: s3a://S3_ACCESS_KEY:S3_SECRET_KEY@BUCKET_NAME/FILE_NAME + public String getS3aURI() { + String rv = "s3a://" + this.s3AccessKey + ":"; + try { + rv += URLEncoder.encode(this.s3SecretKey, "UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + rv += "@" + this.bucketName + "/" + this.objectName; + return rv; + } + + public void setObjectName(String name) { + this.objectName = name; + } + + public void open() { + S3Object s3Obj = s3.getObject(bucketName, objectName); + S3ObjectInputStream s3is = s3Obj.getObjectContent(); + s3In = new BufferedReader(new InputStreamReader(s3is)); + } + + public void close() throws IOException { + if (null != s3In) { + s3In.close(); + s3In = null; + } + } + + public String readLine() throws IOException { + return s3In.readLine(); + } + + public String getBucketName() { + return bucketName; + } + + private static String getS3Path(InputData inputData) { + return inputData.getDataSource().replaceAll("^/+", ""); + } + + // Set bucketName and prefix, based on the PXF URI + protected void setBucketName(InputData inputData) { + // Remove any leading '/' as the value passed in here is in consistent in this + // regard + String s3Path = getS3Path(inputData); + int slashPos = s3Path.indexOf("/"); + if (slashPos > -1) { + bucketName = s3Path.substring(0, slashPos); + } else { + bucketName = s3Path; + } + } + + /* + * Useful from within Fragmenter and Accessor classes + */ + public static PxfS3 fromInputData(InputData inputData) { + String s3AccessKey = inputData.getUserProperty(S3_ACCESS_KEY); + String s3SecretKey = inputData.getUserProperty(S3_SECRET_KEY); + String s3Region = inputData.getUserProperty(S3_REGION); + PxfS3 rv = new PxfS3(s3AccessKey, s3SecretKey, s3Region); + rv.setBucketName(inputData); + return rv; + } + + /* + * If prefix is null, it will have no effect. Note that this doesn't handle + * something like "*.csv", since that's a wildcard and not a prefix. + */ + public List getKeysInBucket(InputData inputData) { + List rv = new ArrayList<>(); + String s3Path = getS3Path(inputData); + String prefix = null; + int slashPos = s3Path.indexOf("/"); + if (slashPos > -1) { + bucketName = s3Path.substring(0, slashPos); + if (s3Path.length() > slashPos + 1) { + prefix = s3Path.substring(slashPos + 1); + } + } else { + bucketName = s3Path; + } + ListObjectsV2Result result = s3.listObjectsV2(bucketName, prefix); + List objects = result.getObjectSummaries(); + for (S3ObjectSummary os : objects) { + String objKey = os.getKey(); + if (!objKey.endsWith("/")) { + rv.add(objKey); + } + } + return rv; + } + + public AmazonS3 getS3() { + return s3; + } + + public void setS3(AmazonS3 s3) { + this.s3 = s3; + } + + public void setS3AccessKey(String s3AccessKey) { + this.s3AccessKey = s3AccessKey; + } + + public void setS3SecretKey(String s3SecretKey) { + this.s3SecretKey = s3SecretKey; + } + +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3Fragmenter.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3Fragmenter.java new file mode 100644 index 0000000000..03b2e27ba1 --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3Fragmenter.java @@ -0,0 +1,73 @@ +package org.greenplum.pxf.s3; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.apache.hawq.pxf.api.Fragmenter; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hawq.pxf.api.Fragment; +import org.apache.hawq.pxf.api.utilities.InputData; + +import java.net.InetAddress; +import java.util.List; + +/** + * Fragmenter for S3 + * + * Based on the input, a bucket name plus a prefix, expands that to a list of + * matching paths, which is returned by getFragments() + */ +public class S3Fragmenter extends Fragmenter { + + private static final Log LOG = LogFactory.getLog(S3Fragmenter.class); + private final PxfS3 pxfS3; + + public S3Fragmenter(InputData inputData) { + super(inputData); + LOG.info("dataSource: " + inputData.getDataSource()); + pxfS3 = PxfS3.fromInputData(inputData); + } + + /** + * This is for tests + * @param inputData + * @param pxfS3 for tests, this would be a PxfS3Mock instance + */ + protected S3Fragmenter(InputData inputData, PxfS3 pxfS3) { + super(inputData); + this.pxfS3 = pxfS3; + } + + /** + * Returns a list of distinct S3 objects, including the full path to each, based + * on the bucket name plus prefix passed in via the InputData. + */ + @Override + public List getFragments() throws Exception { + String pxfHost = InetAddress.getLocalHost().getHostAddress(); + String[] hosts = new String[] { pxfHost }; + for (String key : pxfS3.getKeysInBucket(super.inputData)) { + // Example key values: test-write/1524148597-0000000911_1, test-write/1524496731-0000002514_5 + fragments.add(new Fragment(inputData.getDataSource(), hosts, key.getBytes())); + } + return fragments; + } + +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetAccessor.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetAccessor.java new file mode 100644 index 0000000000..5c0cdee90f --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetAccessor.java @@ -0,0 +1,68 @@ +package org.greenplum.pxf.s3; + +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.OneField; +import org.apache.hawq.pxf.api.ReadAccessor; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.plugins.hdfs.ParquetFileAccessor; +import org.apache.hawq.pxf.plugins.hdfs.ParquetResolver; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; + +public class S3ParquetAccessor extends ParquetFileAccessor implements ReadAccessor { + + private static final Log LOG = LogFactory.getLog(S3ParquetAccessor.class); + private PxfS3 pxfS3; + private ParquetResolver resolver; + private MessageType schema; + + public S3ParquetAccessor(InputData input) { + super(input); + pxfS3 = PxfS3.fromInputData(inputData); + pxfS3.setObjectName(new String(inputData.getFragmentMetadata())); + resolver = new ParquetResolver(inputData); + } + + @Override + public boolean openForRead() throws Exception { + LOG.info("openForRead(): " + pxfS3); + Path path = new Path(pxfS3.getS3aURI()); + Configuration conf = new Configuration(); + ParquetMetadata metadata = ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + schema = metadata.getFileMetaData().getSchema(); + setSchema(schema); + setReader(new ParquetFileReader(conf, path, ParquetMetadataConverter.NO_FILTER)); + setRecordIterator(); + return iteratorHasNext(); + } + + /** + * This overrides the parent's method, using a Resolver to set up the + * List that goes into the OneRow return value (which just gets passed + * through by S3ParquetResolver). The reason for this is that the schema isn't + * available to the Resolver, but it is here, so it makes sense to use it here. + */ + @Override + public OneRow readNextObject() { + OneRow rv = null; + OneRow next = super.readNextObject(); + if (null != next) { + try { + List oneFieldList = resolver.getFields(next, schema); + rv = new OneRow(null, oneFieldList); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + return rv; + } + +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonAccessor.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonAccessor.java new file mode 100644 index 0000000000..e15a6c6fd4 --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonAccessor.java @@ -0,0 +1,81 @@ +package org.greenplum.pxf.s3; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.ReadAccessor; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.api.utilities.Plugin; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.tools.json.JsonRecordFormatter; +import org.apache.parquet.tools.read.SimpleReadSupport; +import org.apache.parquet.tools.read.SimpleRecord; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +public class S3ParquetJsonAccessor extends Plugin implements ReadAccessor { + + private static final Log LOG = LogFactory.getLog(S3ParquetJsonAccessor.class); + + private PxfS3 pxfS3; + private ParquetReader reader; + private JsonRecordFormatter.JsonGroupFormatter formatter; + + public S3ParquetJsonAccessor(InputData inputData) { + super(inputData); + pxfS3 = PxfS3.fromInputData(inputData); + pxfS3.setObjectName(new String(inputData.getFragmentMetadata())); + } + + // Called once per object in S3 + @Override + public boolean openForRead() throws Exception { + LOG.info("openForRead(): " + pxfS3); + Path path = new Path(pxfS3.getS3aURI()); + Configuration conf = new Configuration(); + ParquetMetadata metadata = ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + this.reader = ParquetReader.builder(new SimpleReadSupport(), path).build(); + this.formatter = JsonRecordFormatter.fromSchema(metadata.getFileMetaData().getSchema()); + return true; + } + + @Override + public OneRow readNextObject() throws Exception { + OneRow row = null; + SimpleRecord record = reader.read(); + if (null != record) { + row = new OneRow(null, formatter.formatRecord(record)); + } + return row; + } + + @Override + public void closeForRead() throws Exception { + LOG.info("closeForRead()"); + if (null != reader) { + reader.close(); + } + } +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonResolver.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonResolver.java new file mode 100644 index 0000000000..a100ead7ef --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonResolver.java @@ -0,0 +1,26 @@ +package org.greenplum.pxf.s3; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hawq.pxf.api.OneField; +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.ReadResolver; +import org.apache.hawq.pxf.api.io.DataType; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.api.utilities.Plugin; + +public class S3ParquetJsonResolver extends Plugin implements ReadResolver { + + public S3ParquetJsonResolver(InputData input) { + super(input); + } + + @Override + public List getFields(OneRow row) throws Exception { + List record = new ArrayList<>(); + record.add(new OneField(DataType.VARCHAR.getOID(), row.getData())); + return record; + } + +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetResolver.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetResolver.java new file mode 100644 index 0000000000..3e0c955285 --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetResolver.java @@ -0,0 +1,23 @@ +package org.greenplum.pxf.s3; + +import java.util.List; + +import org.apache.hawq.pxf.api.OneField; +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.ReadResolver; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.api.utilities.Plugin; + +public class S3ParquetResolver extends Plugin implements ReadResolver { + + public S3ParquetResolver(InputData input) { + super(input); + } + + @SuppressWarnings("unchecked") + @Override + public List getFields(OneRow row) throws Exception { + return (List) row.getData(); + } + +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteAccessor.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteAccessor.java new file mode 100644 index 0000000000..d446b131d3 --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteAccessor.java @@ -0,0 +1,83 @@ +package org.greenplum.pxf.s3; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.WriteAccessor; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.api.utilities.Plugin; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +// See https://blog.cloudera.com/blog/2014/05/how-to-convert-existing-data-into-parquet/ +public class S3ParquetWriteAccessor extends Plugin implements WriteAccessor { + + private static final Log LOG = LogFactory.getLog(S3ParquetWriteAccessor.class); + ParquetWriter writer = null; + + public S3ParquetWriteAccessor(InputData input) { + super(input); + LOG.info("Constructor"); + } + + @Override + public boolean openForWrite() throws Exception { + String fileName = inputData.getDataSource(); + Schema avroSchema = AvroUtil.schemaFromInputData(this.inputData); + LOG.info("openForWrite(): fileName = " + fileName); // /pxf-s3-devel/test-write/1524148597-0000000430_1 + PxfS3 pxfS3 = PxfS3.fromInputData(inputData); + // pxfS3.setObjectName(new String(inputData.getFragmentMetadata())); // This yields a NPE + String s3aURI = pxfS3.getS3aURI(this.inputData); + LOG.info("s3aURI: " + s3aURI); + Path file = new Path(s3aURI); + writer = AvroParquetWriter + .builder(file) + .withSchema(avroSchema) + // TODO: Expose the compression codec via a user-supplied parameter on the table definition URI + //.withCompressionCodec(CompressionCodecName.SNAPPY) // 49 MB (was 95 MB w/o compression) + .withCompressionCodec(CompressionCodecName.GZIP) // 33 MB + .withDictionaryEncoding(true) + //.withDictionaryPageSize(3145728) // No effect + //.withRowGroupSize(16777216) // No effect + .build(); + return true; + } + + @Override + public boolean writeNextObject(OneRow oneRow) throws Exception { + writer.write((GenericRecord) oneRow.getData()); + return true; + } + + @Override + public void closeForWrite() throws Exception { + if (writer != null) { + writer.close(); + writer = null; + } + } +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteResolver.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteResolver.java new file mode 100644 index 0000000000..2c3074ef0e --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteResolver.java @@ -0,0 +1,98 @@ +package org.greenplum.pxf.s3; + +import java.text.SimpleDateFormat; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Map; + +import org.apache.avro.LogicalType; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Type; +import org.apache.avro.generic.GenericData; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hawq.pxf.api.OneField; +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.WriteResolver; +import org.apache.hawq.pxf.api.io.DataType; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.api.utilities.Plugin; + +@SuppressWarnings("unused") +public class S3ParquetWriteResolver extends Plugin implements WriteResolver { + + private static final Log LOG = LogFactory.getLog(S3ParquetWriteResolver.class); + + // Required for creating GenericRecord containing the row of data + private Schema avroSchema; + + public S3ParquetWriteResolver(InputData input) { + super(input); + avroSchema = AvroUtil.schemaFromInputData(input); + } + + /* + * Refer to: + * https://github.com/apache/parquet-mr/blob/master/parquet-avro/src/test/java/ + * org/apache/parquet/avro/TestReadWrite.java + * https://avro.apache.org/docs/1.8.1/api/java/org/apache/avro/generic/ + * GenericData.Record.html + */ + @Override + public OneRow setFields(List oneFieldList) throws Exception { + GenericData.Record record = new GenericData.Record(avroSchema); + List fieldList = avroSchema.getFields(); + // TODO: handle type conversion, from PostgreSQL to Avro + for (int i = 0; i < oneFieldList.size(); i++) { + OneField oneField = oneFieldList.get(i); + int dbType = oneField.type; + Object value = oneField.val; + + // What's the Avro type, that we'll have to convert to? + Schema.Field avroField = fieldList.get(i); + Schema fieldSchema = avroField.schema(); + Type avroType = fieldSchema.getType(); + List typeList = new ArrayList<>(); + // TODO: move this date/time resolution code into AvroUtil + if (avroType == Schema.Type.UNION) { + for (Schema s : fieldSchema.getTypes()) { + Type t = s.getType(); + LogicalType lt = s.getLogicalType(); + String ltName = "null"; + if (lt != null) { + ltName = lt.getName(); + } + /* Here, the value on the right is lt.getName(): + * + * DATE: (Type.INT, LogicalType = date) + * TIME: (Type.INT, LogicalType = time-millis) + * TIMESTAMP: (Type.LONG, LogicalType = timestamp-millis) + * + * Validated that these do get read back from Parquet with the specfied + * types (int or long). + */ + if (null != value) { + if ("date".equals(ltName) && Type.STRING == t) { + // DATE + value = (String) value; + } else if ("timestamp-millis".equals(ltName) && Type.STRING == t) { + // TIMESTAMP + value = (String) value; + } + } + typeList.add(s.toString() + "(Type." + t + ", LogicalType = " + ltName + ")"); + } + } + LOG.debug("type: " + DataType.get(dbType) + ", value: " + (value == null ? "null" : value.toString()) + + ", types: " + String.join(", ", typeList)); + record.put(fieldList.get(i).name(), value); + } + return new OneRow(null, record); + } + +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextAccessor.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextAccessor.java new file mode 100644 index 0000000000..acee1450a6 --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextAccessor.java @@ -0,0 +1,66 @@ +package org.greenplum.pxf.s3; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.ReadAccessor; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.api.utilities.Plugin; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +/** + * ReadAccessor which handles delimited TEXT data, passing each line as its own + * OneRow instance + */ +public class S3TextAccessor extends Plugin implements ReadAccessor { + + private static final Log LOG = LogFactory.getLog(S3TextAccessor.class); + private PxfS3 pxfS3; + + public S3TextAccessor(InputData metaData) { + super(metaData); + pxfS3 = PxfS3.fromInputData(metaData); + pxfS3.setObjectName(new String(metaData.getFragmentMetadata())); + } + + @Override + public boolean openForRead() throws Exception { + LOG.info("openForRead()"); + pxfS3.open(); + return true; + } + + @Override + public OneRow readNextObject() throws Exception { + OneRow row = null; + String line = pxfS3.readLine(); + if (null != line) { + row = new OneRow(null, line); + } + return row; + } + + @Override + public void closeForRead() throws Exception { + LOG.info("closeForRead()"); + pxfS3.close(); + } +} diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextResolver.java b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextResolver.java new file mode 100644 index 0000000000..f03e538f90 --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextResolver.java @@ -0,0 +1,59 @@ +package org.greenplum.pxf.s3; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import org.apache.hawq.pxf.api.OneField; +import org.apache.hawq.pxf.api.OneRow; +import org.apache.hawq.pxf.api.ReadResolver; +import org.apache.hawq.pxf.api.WriteResolver; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.api.utilities.Plugin; + +import java.util.LinkedList; +import java.util.List; + +import static org.apache.hawq.pxf.api.io.DataType.VARCHAR; + +public class S3TextResolver extends Plugin implements ReadResolver, WriteResolver { + + public S3TextResolver(InputData metaData) { + super(metaData); + } + + @Override + public List getFields(OneRow row) throws Exception { + List output = new LinkedList(); + Object data = row.getData(); + output.add(new OneField(VARCHAR.getOID(), data)); + return output; + } + + @Override + public OneRow setFields(List record) throws Exception { + // text row data is passed as a single field + if (record == null || record.size() != 1) { + throw new Exception( + "Unexpected record format, expected 1 field, found " + (record == null ? 0 : record.size())); + } + byte[] value = (byte[]) record.get(0).val; + // empty array means the end of input stream, return null to stop iterations + return value.length == 0 ? null : new OneRow(value); + } +} diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/AvroUtilTest.java b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/AvroUtilTest.java new file mode 100644 index 0000000000..d566804c98 --- /dev/null +++ b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/AvroUtilTest.java @@ -0,0 +1,68 @@ +package org.greenplum.pxf.s3; + +import static org.junit.Assert.*; + +import java.util.ArrayList; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hawq.pxf.api.io.DataType; +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.junit.Test; + +public class AvroUtilTest { + + @Test + public void testSchemaFromInputData() { + InputData id = new TestInputData(); + ArrayList tupleDescription = new ArrayList<>(); + tupleDescription.add(new ColumnDescriptor("int_col", DataType.INTEGER.getOID(), 0, "int4", null)); + tupleDescription.add(new ColumnDescriptor("text_col", DataType.TEXT.getOID(), 1, "text", null)); + tupleDescription.add(new ColumnDescriptor("bool_col", DataType.BOOLEAN.getOID(), 2, "bool", null)); + ((TestInputData) id).setTupleDescription(tupleDescription); + Schema schemaFromInputData = AvroUtil.schemaFromInputData(id); + assertNotNull(schemaFromInputData); + Schema schemaFromAPI = SchemaBuilder.record("mytable").namespace(AvroUtil.NAMESPACE) + .fields() + .name("int_col").type().nullable().intType().noDefault() + .name("text_col").type().nullable().stringType().noDefault() + .name("bool_col").type().nullable().booleanType().noDefault() + .endRecord(); + assertEquals(schemaFromInputData, schemaFromAPI); + } + + @Test + public void testAddQuotes() { + String withoutQuotes = "this has no quotes yet"; + String withQuotes = "\"this has no quotes yet\""; + assertEquals(withQuotes, AvroUtil.addQuotes(withoutQuotes)); + } + + @Test + public void testAsAvroType() { + assertEquals("\"boolean\"", AvroUtil.asAvroType(DataType.BOOLEAN)); + assertEquals("\"bytes\"", AvroUtil.asAvroType(DataType.BYTEA)); + assertEquals("\"long\"", AvroUtil.asAvroType(DataType.BIGINT)); + assertEquals("\"int\"", AvroUtil.asAvroType(DataType.INTEGER)); + assertEquals("\"string\"", AvroUtil.asAvroType(DataType.TEXT)); + assertEquals("\"float\"", AvroUtil.asAvroType(DataType.REAL)); + assertEquals("\"double\"", AvroUtil.asAvroType(DataType.FLOAT8)); + assertEquals("{ \"type\": \"string\", \"logicalType\": \"date\" }", AvroUtil.asAvroType(DataType.DATE)); + assertEquals("{ \"type\": \"string\", \"logicalType\": \"timestamp-millis\" }", AvroUtil.asAvroType(DataType.TIMESTAMP)); + } + + /** + * This exists solely to permit setting the tupleDescription field of the + * InputData instance + */ + static class TestInputData extends InputData { + public TestInputData() { + super(); + } + + public void setTupleDescription(ArrayList tupleDescription) { + super.tupleDescription = tupleDescription; + } + } +} diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/InputDataMock.java b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/InputDataMock.java new file mode 100644 index 0000000000..17c73ac4a3 --- /dev/null +++ b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/InputDataMock.java @@ -0,0 +1,33 @@ +package org.greenplum.pxf.s3; + +import java.util.HashMap; + +import org.apache.hawq.pxf.api.utilities.InputData; + +/** + * This exists so the various userProperty values can be set + * InputData instance + */ +public class InputDataMock extends InputData { + + public InputDataMock() { + super(); + super.requestParametersMap = new HashMap(); + } + + public void setUserProperty(String key, String value) { + super.requestParametersMap.put(USER_PROP_PREFIX + key.toUpperCase(), value); + } + + /** + * Helper to build InputData + * @return an InputData instance + */ + public static InputData getInputData() { + InputDataMock inputData = new InputDataMock(); + inputData.setUserProperty(PxfS3.S3_ACCESS_KEY, PxfS3Test.TEST_S3_ACCESS_KEY); + inputData.setUserProperty(PxfS3.S3_SECRET_KEY, PxfS3Test.TEST_S3_SECRET_KEY); + inputData.setUserProperty(PxfS3.S3_REGION, PxfS3Test.TEST_S3_REGION); + return inputData; + } +} diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Mock.java b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Mock.java new file mode 100644 index 0000000000..3f37cfac9a --- /dev/null +++ b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Mock.java @@ -0,0 +1,89 @@ +package org.greenplum.pxf.s3; + +import java.io.IOException; + +import org.apache.hawq.pxf.api.utilities.InputData; + +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.AnonymousAWSCredentials; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; + +import io.findify.s3mock.S3Mock; + +public class PxfS3Mock extends PxfS3 { + + private static final int PORT = 1999; + private static S3Mock API = null; + + /* + * static { API = new + * S3Mock.Builder().withPort(PORT).withInMemoryBackend().build(); API.start(); } + */ + + public PxfS3Mock(String s3AccessKey, String s3SecretKey, String s3Region) { + super(); + this.setS3AccessKey(s3AccessKey); + this.setS3SecretKey(s3SecretKey); + if (null == API) { + API = new S3Mock.Builder().withPort(PORT).withInMemoryBackend().build(); + API.start(); + } + AwsClientBuilder.EndpointConfiguration endpoint = new AwsClientBuilder.EndpointConfiguration( + "http://localhost:" + PORT, s3Region); + AmazonS3 s3 = AmazonS3ClientBuilder.standard().withPathStyleAccessEnabled(true) + .withEndpointConfiguration(endpoint) + .withCredentials(new AWSStaticCredentialsProvider(new AnonymousAWSCredentials())).build(); + setS3(s3); + } + + public static PxfS3 fromInputData(InputData inputData) { + String s3AccessKey = inputData.getUserProperty(PxfS3.S3_ACCESS_KEY); + String s3SecretKey = inputData.getUserProperty(PxfS3.S3_SECRET_KEY); + String s3Region = inputData.getUserProperty(PxfS3.S3_REGION); + PxfS3 rv = new PxfS3Mock(s3AccessKey, s3SecretKey, s3Region); + rv.setBucketName(inputData); + return rv; + } + + public void close() throws IOException { + super.close(); + if (API != null) { + API.shutdown(); + } + } + + /** + * Example of how to use this class + * + * NOTE: the try/catch/finally is important since, if an exception is thrown, + * this will keep the port open, preventing future runs (within the same JVM; + * e.g. the IDE). + */ + public static void main(String[] args) throws IOException { + System.out.println("Starting ..."); + PxfS3 pxfS3 = null; + try { + InputData inputData = InputDataMock.getInputData(); + inputData.setDataSource("/pxf-s3-devel/test-write"); + pxfS3 = PxfS3Mock.fromInputData(inputData); + AmazonS3 s3 = pxfS3.getS3(); + s3.createBucket(pxfS3.getBucketName()); + s3.putObject(pxfS3.getBucketName(), "test-write/1524148597-0000000911_1", + "contents of 1524148597-0000000911_1"); + s3.putObject(pxfS3.getBucketName(), "test-write/1524496731-0000002514_5", + "contents of 1524496731-0000002514_5"); + for (String key : pxfS3.getKeysInBucket(inputData)) { + System.out.println("key: " + key); + } + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (null != pxfS3) { + pxfS3.close(); + } + } + } + +} diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Test.java b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Test.java new file mode 100644 index 0000000000..bcc3ed0588 --- /dev/null +++ b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Test.java @@ -0,0 +1,49 @@ +package org.greenplum.pxf.s3; + +import static org.junit.Assert.*; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.junit.Test; + +public class PxfS3Test { + + static final String TEST_S3_ACCESS_KEY = "DAZFVISAXWVBOBDBHQZX"; + static final String TEST_S3_SECRET_KEY = "IhipDEysJnknxCBeX/9LGRg35l55owj/wf7KZYh9"; + static final String TEST_S3_REGION = "us-east-1"; + + /** + * Supports READ operations + * + * @throws UnsupportedEncodingException + */ + @Test + public void testGetS3aURIForRead() throws UnsupportedEncodingException { + InputData inputData = InputDataMock.getInputData(); + inputData.setDataSource("pxf-s3-devel/sample-from-spark/part-0"); + PxfS3 pxfS3 = PxfS3.fromInputData(inputData); + pxfS3.setObjectName("sample-from-spark/part-00000-f5d896f5-e61e-4991-935e-95707ff4d6eb-c000.snappy.parquet"); + String s3aURI = pxfS3.getS3aURI(); + String expectedS3aURI = "s3a://" + PxfS3Test.TEST_S3_ACCESS_KEY + ":" + URLEncoder.encode(PxfS3Test.TEST_S3_SECRET_KEY, "UTF-8") + + "@" + "pxf-s3-devel" + "/" + "sample-from-spark/part-00000-f5d896f5-e61e-4991-935e-95707ff4d6eb-c000.snappy.parquet"; + assertEquals(expectedS3aURI, s3aURI); + } + + /** + * Supports WRITE operations + * + * @throws UnsupportedEncodingException + */ + @Test + public void testGetS3aURIForWrite() throws UnsupportedEncodingException { + InputData inputData = InputDataMock.getInputData(); + inputData.setDataSource("/pxf-s3-devel/test-write/1524148597-0000000430_1"); + PxfS3 pxfS3 = PxfS3.fromInputData(inputData); + String s3aURI = pxfS3.getS3aURI(inputData); + String expectedS3aURI = "s3a://" + PxfS3Test.TEST_S3_ACCESS_KEY + ":" + URLEncoder.encode(PxfS3Test.TEST_S3_SECRET_KEY, "UTF-8") + + "@" + "pxf-s3-devel" + "/" + "test-write/1524148597-0000000430_1"; + assertEquals(expectedS3aURI, s3aURI); + } + +} diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/S3FragmenterTest.java b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/S3FragmenterTest.java new file mode 100644 index 0000000000..595ce0a0a6 --- /dev/null +++ b/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/S3FragmenterTest.java @@ -0,0 +1,97 @@ +package org.greenplum.pxf.s3; + +import static org.junit.Assert.*; + +import java.io.IOException; +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hawq.pxf.api.Fragment; +import org.apache.hawq.pxf.api.utilities.InputData; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.amazonaws.services.s3.AmazonS3; + +public class S3FragmenterTest { + + private static PxfS3 pxfS3; + private static InputData inputData; + + @Test + public void testGetFragments() throws Exception { + S3Fragmenter fragmenter = new S3Fragmenter(inputData, pxfS3); + List testFragments = fragmenter.getFragments(); + List expectFragments = new ArrayList<>(); + String pxfHost = InetAddress.getLocalHost().getHostAddress(); + String[] hosts = new String[] { pxfHost }; + for (String key : Arrays.asList("test-write/1524148597-0000000911_1", "test-write/1524496731-0000002514_5")) { + expectFragments.add(new Fragment(inputData.getDataSource(), hosts, key.getBytes())); + } + // Verify same number of elements + System.out.println("Comparing lengths of the Fragment lists ..."); + assertTrue(expectFragments.size() == testFragments.size()); + // Verify equality of each Fragment + for (int i = 0; i < expectFragments.size(); i++) { + System.out.println("Comparing Fragment[" + i + "] ..."); + assertTrue(fragmentEquals(expectFragments.get(i), testFragments.get(i))); + } + } + + /** + * In the absence of an equals() method in Fragment, verify equality only of the + * fields in use here + * + * @param a + * left hand side + * @param b + * right hand side + * @return whether they are equal + */ + private boolean fragmentEquals(Fragment a, Fragment b) { + // sourceName + boolean rv = a.getSourceName().equals(b.getSourceName()); + // replicas + String[] replicasA = a.getReplicas(); + String[] replicasB = b.getReplicas(); + if (replicasA.length == replicasB.length) { + List replicasAList = Arrays.asList(replicasA); + for (int i = 0; i < replicasB.length; i++) { + if (!replicasAList.contains(replicasB[i])) { + rv = false; + } + } + } + // metadata + if (!Arrays.equals(a.getMetadata(), b.getMetadata())) { + rv = false; + } + return rv; + } + + @BeforeClass + public static void setUpMock() { + inputData = InputDataMock.getInputData(); + inputData.setDataSource("/pxf-s3-devel/test-write"); + pxfS3 = PxfS3Mock.fromInputData(inputData); + AmazonS3 s3 = pxfS3.getS3(); + s3.createBucket(pxfS3.getBucketName()); + s3.putObject(pxfS3.getBucketName(), "test-write/1524148597-0000000911_1", + "contents of 1524148597-0000000911_1"); + s3.putObject(pxfS3.getBucketName(), "test-write/1524496731-0000002514_5", + "contents of 1524496731-0000002514_5"); + } + + @AfterClass + public static void destroyMock() { + try { + pxfS3.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + +} diff --git a/pxf/settings.gradle b/pxf/settings.gradle index e26f812590..53106f5264 100644 --- a/pxf/settings.gradle +++ b/pxf/settings.gradle @@ -27,3 +27,4 @@ include 'pxf-hive' include 'pxf-hbase' include 'pxf-json' include 'pxf-jdbc' +include 'pxf-s3' From 566d0cdb6ba6ce016d7c737df84dccab14c419ba Mon Sep 17 00:00:00 2001 From: Michael Goddard Date: Wed, 23 May 2018 11:48:24 -0400 Subject: [PATCH 2/5] Renaming the package to conform to how the HAWQ packages are named --- .../pxf => apache/hawq/pxf/plugins}/s3/AvroUtil.java | 2 +- .../{greenplum/pxf => apache/hawq/pxf/plugins}/s3/PxfS3.java | 2 +- .../pxf => apache/hawq/pxf/plugins}/s3/S3Fragmenter.java | 2 +- .../pxf => apache/hawq/pxf/plugins}/s3/S3ParquetAccessor.java | 2 +- .../hawq/pxf/plugins}/s3/S3ParquetJsonAccessor.java | 2 +- .../hawq/pxf/plugins}/s3/S3ParquetJsonResolver.java | 2 +- .../pxf => apache/hawq/pxf/plugins}/s3/S3ParquetResolver.java | 2 +- .../hawq/pxf/plugins}/s3/S3ParquetWriteAccessor.java | 2 +- .../hawq/pxf/plugins}/s3/S3ParquetWriteResolver.java | 2 +- .../pxf => apache/hawq/pxf/plugins}/s3/S3TextAccessor.java | 2 +- .../pxf => apache/hawq/pxf/plugins}/s3/S3TextResolver.java | 2 +- .../pxf => apache/hawq/pxf/plugins}/s3/AvroUtilTest.java | 3 ++- .../pxf => apache/hawq/pxf/plugins}/s3/InputDataMock.java | 3 ++- .../pxf => apache/hawq/pxf/plugins}/s3/PxfS3Mock.java | 3 ++- .../pxf => apache/hawq/pxf/plugins}/s3/PxfS3Test.java | 3 ++- .../pxf => apache/hawq/pxf/plugins}/s3/S3FragmenterTest.java | 4 +++- 16 files changed, 22 insertions(+), 16 deletions(-) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/AvroUtil.java (98%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/PxfS3.java (99%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3Fragmenter.java (98%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3ParquetAccessor.java (98%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3ParquetJsonAccessor.java (98%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3ParquetJsonResolver.java (94%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3ParquetResolver.java (93%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3ParquetWriteAccessor.java (98%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3ParquetWriteResolver.java (98%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3TextAccessor.java (98%) rename pxf/pxf-s3/src/main/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3TextResolver.java (98%) rename pxf/pxf-s3/src/test/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/AvroUtilTest.java (96%) rename pxf/pxf-s3/src/test/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/InputDataMock.java (91%) rename pxf/pxf-s3/src/test/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/PxfS3Mock.java (97%) rename pxf/pxf-s3/src/test/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/PxfS3Test.java (95%) rename pxf/pxf-s3/src/test/java/org/{greenplum/pxf => apache/hawq/pxf/plugins}/s3/S3FragmenterTest.java (95%) diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/AvroUtil.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/AvroUtil.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/AvroUtil.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/AvroUtil.java index 4c3c27355c..972e6f2c00 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/AvroUtil.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/AvroUtil.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.util.ArrayList; import java.util.List; diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/PxfS3.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/PxfS3.java similarity index 99% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/PxfS3.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/PxfS3.java index 3860dd419e..70c357485e 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/PxfS3.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/PxfS3.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.io.BufferedReader; import java.io.IOException; diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3Fragmenter.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3Fragmenter.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3Fragmenter.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3Fragmenter.java index 03b2e27ba1..e6ef60f539 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3Fragmenter.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3Fragmenter.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; /* * Licensed to the Apache Software Foundation (ASF) under one diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetAccessor.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetAccessor.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetAccessor.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetAccessor.java index 5c0cdee90f..46dd710d19 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetAccessor.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetAccessor.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.util.List; diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonAccessor.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonAccessor.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonAccessor.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonAccessor.java index e15a6c6fd4..4238c4e63b 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonAccessor.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonAccessor.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; /* * Licensed to the Apache Software Foundation (ASF) under one diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonResolver.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonResolver.java similarity index 94% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonResolver.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonResolver.java index a100ead7ef..eb0e1df99a 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetJsonResolver.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonResolver.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.util.ArrayList; import java.util.List; diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetResolver.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetResolver.java similarity index 93% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetResolver.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetResolver.java index 3e0c955285..29e34e75a8 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetResolver.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetResolver.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.util.List; diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteAccessor.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetWriteAccessor.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteAccessor.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetWriteAccessor.java index d446b131d3..832c0b7e99 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteAccessor.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetWriteAccessor.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteResolver.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetWriteResolver.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteResolver.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetWriteResolver.java index 2c3074ef0e..1eac8edf94 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3ParquetWriteResolver.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetWriteResolver.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.text.SimpleDateFormat; import java.time.LocalTime; diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextAccessor.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3TextAccessor.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextAccessor.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3TextAccessor.java index acee1450a6..26b4e17a25 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextAccessor.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3TextAccessor.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; /* * Licensed to the Apache Software Foundation (ASF) under one diff --git a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextResolver.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3TextResolver.java similarity index 98% rename from pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextResolver.java rename to pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3TextResolver.java index f03e538f90..eaa50a209e 100644 --- a/pxf/pxf-s3/src/main/java/org/greenplum/pxf/s3/S3TextResolver.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3TextResolver.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; /* * Licensed to the Apache Software Foundation (ASF) under one diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/AvroUtilTest.java b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/AvroUtilTest.java similarity index 96% rename from pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/AvroUtilTest.java rename to pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/AvroUtilTest.java index d566804c98..0bf8514a65 100644 --- a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/AvroUtilTest.java +++ b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/AvroUtilTest.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import static org.junit.Assert.*; @@ -9,6 +9,7 @@ import org.apache.hawq.pxf.api.io.DataType; import org.apache.hawq.pxf.api.utilities.ColumnDescriptor; import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.plugins.s3.AvroUtil; import org.junit.Test; public class AvroUtilTest { diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/InputDataMock.java b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/InputDataMock.java similarity index 91% rename from pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/InputDataMock.java rename to pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/InputDataMock.java index 17c73ac4a3..12753713cb 100644 --- a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/InputDataMock.java +++ b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/InputDataMock.java @@ -1,8 +1,9 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.util.HashMap; import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.plugins.s3.PxfS3; /** * This exists so the various userProperty values can be set diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Mock.java b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/PxfS3Mock.java similarity index 97% rename from pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Mock.java rename to pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/PxfS3Mock.java index 3f37cfac9a..139d3b3ba9 100644 --- a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Mock.java +++ b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/PxfS3Mock.java @@ -1,8 +1,9 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import java.io.IOException; import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.plugins.s3.PxfS3; import com.amazonaws.auth.AWSStaticCredentialsProvider; import com.amazonaws.auth.AnonymousAWSCredentials; diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Test.java b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/PxfS3Test.java similarity index 95% rename from pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Test.java rename to pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/PxfS3Test.java index bcc3ed0588..001dbc6982 100644 --- a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/PxfS3Test.java +++ b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/PxfS3Test.java @@ -1,10 +1,11 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import static org.junit.Assert.*; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.plugins.s3.PxfS3; import org.junit.Test; public class PxfS3Test { diff --git a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/S3FragmenterTest.java b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/S3FragmenterTest.java similarity index 95% rename from pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/S3FragmenterTest.java rename to pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/S3FragmenterTest.java index 595ce0a0a6..b01c16a273 100644 --- a/pxf/pxf-s3/src/test/java/org/greenplum/pxf/s3/S3FragmenterTest.java +++ b/pxf/pxf-s3/src/test/java/org/apache/hawq/pxf/plugins/s3/S3FragmenterTest.java @@ -1,4 +1,4 @@ -package org.greenplum.pxf.s3; +package org.apache.hawq.pxf.plugins.s3; import static org.junit.Assert.*; @@ -10,6 +10,8 @@ import org.apache.hawq.pxf.api.Fragment; import org.apache.hawq.pxf.api.utilities.InputData; +import org.apache.hawq.pxf.plugins.s3.PxfS3; +import org.apache.hawq.pxf.plugins.s3.S3Fragmenter; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; From 779531b811ecafd77ce4e700777d3b7e3de57e6a Mon Sep 17 00:00:00 2001 From: Michael Goddard Date: Wed, 23 May 2018 12:54:55 -0400 Subject: [PATCH 3/5] Removing the commented out 'force' line for Avro. I needed a more recent Avro, and all tests are passing, so it seems okay. --- pxf/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/pxf/build.gradle b/pxf/build.gradle index 0b580453d9..9866fc7eea 100644 --- a/pxf/build.gradle +++ b/pxf/build.gradle @@ -123,7 +123,6 @@ subprojects { subProject -> force 'commons-codec:commons-codec:1.4' force 'commons-collections:commons-collections:3.2.1' force 'commons-logging:commons-logging:1.1.3' - //force 'org.apache.avro:avro:1.7.4' force 'org.apache.zookeeper:zookeeper:3.4.6' force 'org.codehaus.jackson:jackson-mapper-asl:1.9.13' force 'junit:junit:4.11' From 6be3f1fa1e307fe9a7e0d2680880763d8c269e8e Mon Sep 17 00:00:00 2001 From: Michael Goddard Date: Wed, 23 May 2018 13:00:28 -0400 Subject: [PATCH 4/5] Correcting the package names in the README to align with the changes made in the Java sources. --- pxf/pxf-s3/README.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pxf/pxf-s3/README.md b/pxf/pxf-s3/README.md index 88a8afe420..79c4689dea 100644 --- a/pxf/pxf-s3/README.md +++ b/pxf/pxf-s3/README.md @@ -26,27 +26,27 @@ S3ParquetWrite A profile for writing Parquet data to S3 - org.greenplum.pxf.s3.S3Fragmenter - org.greenplum.pxf.s3.S3ParquetWriteAccessor - org.greenplum.pxf.s3.S3ParquetWriteResolver + org.apache.hawq.pxf.plugins.s3.S3Fragmenter + org.apache.hawq.pxf.plugins.s3.S3ParquetWriteAccessor + org.apache.hawq.pxf.plugins.s3.S3ParquetWriteResolver S3ParquetRead A profile for reading Parquet data from S3; does not support nested data - org.greenplum.pxf.s3.S3Fragmenter - org.greenplum.pxf.s3.S3ParquetAccessor - org.greenplum.pxf.s3.S3ParquetResolver + org.apache.hawq.pxf.plugins.s3.S3Fragmenter + org.apache.hawq.pxf.plugins.s3.S3ParquetAccessor + org.apache.hawq.pxf.plugins.s3.S3ParquetResolver S3ParquetJsonRead A profile for reading Parquet data from S3, as JSON; useful with nested data - org.greenplum.pxf.s3.S3Fragmenter - org.greenplum.pxf.s3.S3ParquetJsonAccessor - org.greenplum.pxf.s3.S3ParquetJsonResolver + org.apache.hawq.pxf.plugins.s3.S3Fragmenter + org.apache.hawq.pxf.plugins.s3.S3ParquetJsonAccessor + org.apache.hawq.pxf.plugins.s3.S3ParquetJsonResolver From 62361b31953de6bd63aac9cd8e8361dcf154c834 Mon Sep 17 00:00:00 2001 From: Michael Goddard Date: Thu, 24 May 2018 16:41:49 -0400 Subject: [PATCH 5/5] Solution to the NPE we get when OneField's value is null --- .../hawq/pxf/plugins/s3/NullableOneField.java | 28 +++++++++++++++++++ .../pxf/plugins/s3/S3ParquetAccessor.java | 7 ++++- .../pxf/plugins/s3/S3ParquetJsonResolver.java | 2 +- 3 files changed, 35 insertions(+), 2 deletions(-) create mode 100644 pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/NullableOneField.java diff --git a/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/NullableOneField.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/NullableOneField.java new file mode 100644 index 0000000000..734fc286b8 --- /dev/null +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/NullableOneField.java @@ -0,0 +1,28 @@ +package org.apache.hawq.pxf.plugins.s3; + +import org.apache.hawq.pxf.api.OneField; + +public class NullableOneField extends OneField { + + public NullableOneField(int type, Object val) { + super(type, val); + } + + public NullableOneField() { + super(); + } + + /** + * The purpose of this class is to handle the case where val is + * null so, rather than calling toString() on null, it will simply + * return null. + */ + public String toString() { + String rv = null; + if (null != this.val) { + rv = this.val.toString(); + } + return rv; + } + +} diff --git a/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetAccessor.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetAccessor.java index 46dd710d19..ab203e1243 100644 --- a/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetAccessor.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetAccessor.java @@ -1,5 +1,6 @@ package org.apache.hawq.pxf.plugins.s3; +import java.util.ArrayList; import java.util.List; import org.apache.commons.logging.Log; @@ -56,7 +57,11 @@ public OneRow readNextObject() { OneRow next = super.readNextObject(); if (null != next) { try { - List oneFieldList = resolver.getFields(next, schema); + List oneFieldList = new ArrayList(); + for (OneField of: resolver.getFields(next, schema)) { + NullableOneField nof = new NullableOneField(of.type, of.val); + oneFieldList.add(nof); + } rv = new OneRow(null, oneFieldList); } catch (Exception e) { throw new RuntimeException(e); diff --git a/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonResolver.java b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonResolver.java index eb0e1df99a..a6683a8b5d 100644 --- a/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonResolver.java +++ b/pxf/pxf-s3/src/main/java/org/apache/hawq/pxf/plugins/s3/S3ParquetJsonResolver.java @@ -19,7 +19,7 @@ public S3ParquetJsonResolver(InputData input) { @Override public List getFields(OneRow row) throws Exception { List record = new ArrayList<>(); - record.add(new OneField(DataType.VARCHAR.getOID(), row.getData())); + record.add(new NullableOneField(DataType.VARCHAR.getOID(), row.getData())); return record; }