From 19f61c165932059e7ce156da2c71429fa8dc27f0 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 5 Sep 2014 21:46:45 -0700 Subject: [PATCH 01/19] [Build] suppress curl/wget progress bars In the Jenkins console output, `curl` gives us mountains of `#` symbols as it tries to show its download progress. ![noise from curl in Jenkins output](http://i.imgur.com/P2E7yUw.png) I don't think this is useful so I've changed things to suppress these progress bars. If there is actually some use to this, feel free to reject this proposal. Author: Nicholas Chammas Closes #2279 from nchammas/trim-test-output and squashes the following commits: 14a720c [Nicholas Chammas] suppress curl/wget progress bars --- dev/check-license | 4 ++-- sbt/sbt-launch-lib.bash | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/check-license b/dev/check-license index 625ec161bc571..558e038afc01a 100755 --- a/dev/check-license +++ b/dev/check-license @@ -32,9 +32,9 @@ acquire_rat_jar () { printf "Attempting to fetch rat\n" JAR_DL=${JAR}.part if hash curl 2>/dev/null; then - (curl --progress-bar ${URL1} > "$JAR_DL" || curl --progress-bar ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" + (curl --silent ${URL1} > "$JAR_DL" || curl --silent ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then - (wget --progress=bar ${URL1} -O "$JAR_DL" || wget --progress=bar ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" + (wget --quiet ${URL1} -O "$JAR_DL" || wget --quiet ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" else printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index c91fecf024ad4..fecc3d38a5fbd 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -51,9 +51,9 @@ acquire_sbt_jar () { printf "Attempting to fetch sbt\n" JAR_DL=${JAR}.part if hash curl 2>/dev/null; then - (curl --progress-bar ${URL1} > ${JAR_DL} || curl --progress-bar ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (curl --silent ${URL1} > ${JAR_DL} || curl --silent ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} elif hash wget 2>/dev/null; then - (wget --progress=bar ${URL1} -O ${JAR_DL} || wget --progress=bar ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (wget --quiet ${URL1} -O ${JAR_DL} || wget --quiet ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} else printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" exit -1 From 9422c4ee0eaf4a32d2ed7c96799feac2f5f79d40 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 5 Sep 2014 23:08:54 -0700 Subject: [PATCH 02/19] [SPARK-3361] Expand PEP 8 checks to include EC2 script and Python examples This PR resolves [SPARK-3361](https://issues.apache.org/jira/browse/SPARK-3361) by expanding the PEP 8 checks to cover the remaining Python code base: * The EC2 script * All Python / PySpark examples Author: Nicholas Chammas Closes #2297 from nchammas/pep8-rulez and squashes the following commits: 1e5ac9a [Nicholas Chammas] PEP 8 fixes to Python examples c3dbeff [Nicholas Chammas] PEP 8 fixes to EC2 script 65ef6e8 [Nicholas Chammas] expand PEP 8 checks --- dev/lint-python | 5 ++-- ec2/spark_ec2.py | 20 +++++++++++----- examples/src/main/python/avro_inputformat.py | 17 +++++++++----- .../src/main/python/cassandra_inputformat.py | 15 ++++++------ .../src/main/python/cassandra_outputformat.py | 23 ++++++++++--------- examples/src/main/python/hbase_inputformat.py | 10 +++++--- .../src/main/python/hbase_outputformat.py | 18 +++++++++------ .../src/main/python/mllib/correlations.py | 2 +- .../main/python/mllib/decision_tree_runner.py | 6 +++-- .../python/mllib/random_rdd_generation.py | 6 ++--- .../src/main/python/mllib/sampled_rdds.py | 8 +++---- examples/src/main/python/pi.py | 2 +- 12 files changed, 79 insertions(+), 53 deletions(-) diff --git a/dev/lint-python b/dev/lint-python index a1e890faa8fa6..79bf70f0b8b13 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -30,6 +30,7 @@ cd $SPARK_ROOT_DIR #+ - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?)) PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py" +PEP8_PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/" curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" curl_status=$? @@ -44,7 +45,7 @@ fi #+ first, but we do so so that the check status can #+ be output before the report, like with the #+ scalastyle and RAT checks. -python $PEP8_SCRIPT_PATH ./python/pyspark > "$PEP8_REPORT_PATH" +python $PEP8_SCRIPT_PATH $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" pep8_status=${PIPESTATUS[0]} #$? if [ $pep8_status -ne 0 ]; then @@ -54,7 +55,7 @@ else echo "PEP 8 checks passed." fi -rm -f "$PEP8_REPORT_PATH" +rm "$PEP8_REPORT_PATH" rm "$PEP8_SCRIPT_PATH" exit $pep8_status diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 1670faca4a480..8ec88d95e34af 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -41,6 +41,7 @@ # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" + class UsageError(Exception): pass @@ -342,7 +343,6 @@ def launch_cluster(conn, opts, cluster_name): if opts.ami is None: opts.ami = get_spark_ami(opts) - additional_groups = [] if opts.additional_security_group: additional_groups = [sg @@ -363,7 +363,7 @@ def launch_cluster(conn, opts, cluster_name): for i in range(opts.ebs_vol_num): device = EBSBlockDeviceType() device.size = opts.ebs_vol_size - device.volume_type=opts.ebs_vol_type + device.volume_type = opts.ebs_vol_type device.delete_on_termination = True block_map["/dev/sd" + chr(ord('s') + i)] = device @@ -495,6 +495,7 @@ def launch_cluster(conn, opts, cluster_name): # Return all the instances return (master_nodes, slave_nodes) + def tag_instance(instance, name): for i in range(0, 5): try: @@ -507,9 +508,12 @@ def tag_instance(instance, name): # Get the EC2 instances in an existing cluster if available. # Returns a tuple of lists of EC2 instance objects for the masters and slaves + + def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." - # Search all the spot instance requests, and copy any tags from the spot instance request to the cluster. + # Search all the spot instance requests, and copy any tags from the spot + # instance request to the cluster. spot_instance_requests = conn.get_all_spot_instance_requests() for req in spot_instance_requests: if req.state != u'active': @@ -520,7 +524,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): for res in reservations: active = [i for i in res.instances if is_active(i)] for instance in active: - if (instance.tags.get(u'Name') == None): + if (instance.tags.get(u'Name') is None): tag_instance(instance, name) # Now proceed to detect master and slaves instances. reservations = conn.get_all_instances() @@ -540,13 +544,16 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in with name " + cluster_name + "-master" + print >> sys.stderr, "ERROR: Could not find master in with name " + \ + cluster_name + "-master" else: print >> sys.stderr, "ERROR: Could not find any existing cluster" sys.exit(1) # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. + + def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = master_nodes[0].public_dns_name if deploy_ssh_key: @@ -890,7 +897,8 @@ def real_main(): if opts.security_group_prefix is None: group_names = [cluster_name + "-master", cluster_name + "-slaves"] else: - group_names = [opts.security_group_prefix + "-master", opts.security_group_prefix + "-slaves"] + group_names = [opts.security_group_prefix + "-master", + opts.security_group_prefix + "-slaves"] attempt = 1 while attempt <= 3: diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index e902ae29753c0..cfda8d8327aa3 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -23,7 +23,8 @@ Read data file users.avro in local Spark distro: $ cd $SPARK_HOME -$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +$ ./bin/spark-submit --driver-class-path /path/to/example/jar \ +> ./examples/src/main/python/avro_inputformat.py \ > examples/src/main/resources/users.avro {u'favorite_color': None, u'name': u'Alyssa', u'favorite_numbers': [3, 9, 15, 20]} {u'favorite_color': u'red', u'name': u'Ben', u'favorite_numbers': []} @@ -40,7 +41,8 @@ ] } -$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +$ ./bin/spark-submit --driver-class-path /path/to/example/jar \ +> ./examples/src/main/python/avro_inputformat.py \ > examples/src/main/resources/users.avro examples/src/main/resources/user.avsc {u'favorite_color': None, u'name': u'Alyssa'} {u'favorite_color': u'red', u'name': u'Ben'} @@ -51,8 +53,10 @@ Usage: avro_inputformat [reader_schema_file] Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/avro_inputformat.py [reader_schema_file] - Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/avro_inputformat.py [reader_schema_file] + Assumes you have Avro data stored in . Reader schema can be optionally specified + in [reader_schema_file]. """ exit(-1) @@ -62,9 +66,10 @@ conf = None if len(sys.argv) == 3: schema_rdd = sc.textFile(sys.argv[2], 1).collect() - conf = {"avro.schema.input.key" : reduce(lambda x, y: x+y, schema_rdd)} + conf = {"avro.schema.input.key": reduce(lambda x, y: x + y, schema_rdd)} - avro_rdd = sc.newAPIHadoopFile(path, + avro_rdd = sc.newAPIHadoopFile( + path, "org.apache.avro.mapreduce.AvroKeyInputFormat", "org.apache.avro.mapred.AvroKey", "org.apache.hadoop.io.NullWritable", diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index e4a897f61e39d..05f34b74df45a 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -51,7 +51,8 @@ Usage: cassandra_inputformat Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/cassandra_inputformat.py + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/cassandra_inputformat.py Assumes you have some data in Cassandra already, running on , in and """ exit(-1) @@ -61,12 +62,12 @@ cf = sys.argv[3] sc = SparkContext(appName="CassandraInputFormat") - conf = {"cassandra.input.thrift.address":host, - "cassandra.input.thrift.port":"9160", - "cassandra.input.keyspace":keyspace, - "cassandra.input.columnfamily":cf, - "cassandra.input.partitioner.class":"Murmur3Partitioner", - "cassandra.input.page.row.size":"3"} + conf = {"cassandra.input.thrift.address": host, + "cassandra.input.thrift.port": "9160", + "cassandra.input.keyspace": keyspace, + "cassandra.input.columnfamily": cf, + "cassandra.input.partitioner.class": "Murmur3Partitioner", + "cassandra.input.page.row.size": "3"} cass_rdd = sc.newAPIHadoopRDD( "org.apache.cassandra.hadoop.cql3.CqlPagingInputFormat", "java.util.Map", diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index 836c35b5c6794..d144539e58b8f 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -50,7 +50,8 @@ Usage: cassandra_outputformat Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/cassandra_outputformat.py + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/cassandra_outputformat.py Assumes you have created the following table in Cassandra already, running on , in . @@ -67,16 +68,16 @@ cf = sys.argv[3] sc = SparkContext(appName="CassandraOutputFormat") - conf = {"cassandra.output.thrift.address":host, - "cassandra.output.thrift.port":"9160", - "cassandra.output.keyspace":keyspace, - "cassandra.output.partitioner.class":"Murmur3Partitioner", - "cassandra.output.cql":"UPDATE " + keyspace + "." + cf + " SET fname = ?, lname = ?", - "mapreduce.output.basename":cf, - "mapreduce.outputformat.class":"org.apache.cassandra.hadoop.cql3.CqlOutputFormat", - "mapreduce.job.output.key.class":"java.util.Map", - "mapreduce.job.output.value.class":"java.util.List"} - key = {"user_id" : int(sys.argv[4])} + conf = {"cassandra.output.thrift.address": host, + "cassandra.output.thrift.port": "9160", + "cassandra.output.keyspace": keyspace, + "cassandra.output.partitioner.class": "Murmur3Partitioner", + "cassandra.output.cql": "UPDATE " + keyspace + "." + cf + " SET fname = ?, lname = ?", + "mapreduce.output.basename": cf, + "mapreduce.outputformat.class": "org.apache.cassandra.hadoop.cql3.CqlOutputFormat", + "mapreduce.job.output.key.class": "java.util.Map", + "mapreduce.job.output.value.class": "java.util.List"} + key = {"user_id": int(sys.argv[4])} sc.parallelize([(key, sys.argv[5:])]).saveAsNewAPIHadoopDataset( conf=conf, keyConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLKeyConverter", diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index befacee0dea56..3b16010f1cb97 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -51,7 +51,8 @@ Usage: hbase_inputformat Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/hbase_inputformat.py
+ ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/hbase_inputformat.py
Assumes you have some data in HBase already, running on , in
""" exit(-1) @@ -61,12 +62,15 @@ sc = SparkContext(appName="HBaseInputFormat") conf = {"hbase.zookeeper.quorum": host, "hbase.mapreduce.inputtable": table} + keyConv = "org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter" + valueConv = "org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter" + hbase_rdd = sc.newAPIHadoopRDD( "org.apache.hadoop.hbase.mapreduce.TableInputFormat", "org.apache.hadoop.hbase.io.ImmutableBytesWritable", "org.apache.hadoop.hbase.client.Result", - keyConverter="org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter", - valueConverter="org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter", + keyConverter=keyConv, + valueConverter=valueConv, conf=conf) output = hbase_rdd.collect() for (k, v) in output: diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index 49bbc5aebdb0b..abb425b1f886a 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -44,8 +44,10 @@ Usage: hbase_outputformat
Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/hbase_outputformat.py - Assumes you have created
with column family in HBase running on already + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/hbase_outputformat.py + Assumes you have created
with column family in HBase + running on already """ exit(-1) @@ -55,13 +57,15 @@ conf = {"hbase.zookeeper.quorum": host, "hbase.mapred.outputtable": table, - "mapreduce.outputformat.class" : "org.apache.hadoop.hbase.mapreduce.TableOutputFormat", - "mapreduce.job.output.key.class" : "org.apache.hadoop.hbase.io.ImmutableBytesWritable", - "mapreduce.job.output.value.class" : "org.apache.hadoop.io.Writable"} + "mapreduce.outputformat.class": "org.apache.hadoop.hbase.mapreduce.TableOutputFormat", + "mapreduce.job.output.key.class": "org.apache.hadoop.hbase.io.ImmutableBytesWritable", + "mapreduce.job.output.value.class": "org.apache.hadoop.io.Writable"} + keyConv = "org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter" + valueConv = "org.apache.spark.examples.pythonconverters.StringListToPutConverter" sc.parallelize([sys.argv[3:]]).map(lambda x: (x[0], x)).saveAsNewAPIHadoopDataset( conf=conf, - keyConverter="org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter", - valueConverter="org.apache.spark.examples.pythonconverters.StringListToPutConverter") + keyConverter=keyConv, + valueConverter=valueConv) sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py index 6b16a56e44af7..4218eca822a99 100755 --- a/examples/src/main/python/mllib/correlations.py +++ b/examples/src/main/python/mllib/correlations.py @@ -28,7 +28,7 @@ if __name__ == "__main__": - if len(sys.argv) not in [1,2]: + if len(sys.argv) not in [1, 2]: print >> sys.stderr, "Usage: correlations ()" exit(-1) sc = SparkContext(appName="PythonCorrelations") diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index 6e4a4a0cb6be0..61ea4e06ecf3a 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -21,7 +21,9 @@ This example requires NumPy (http://www.numpy.org/). """ -import numpy, os, sys +import numpy +import os +import sys from operator import add @@ -127,7 +129,7 @@ def usage(): (reindexedData, origToNewLabels) = reindexClassLabels(points) # Train a classifier. - categoricalFeaturesInfo={} # no categorical features + categoricalFeaturesInfo = {} # no categorical features model = DecisionTree.trainClassifier(reindexedData, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py index b388d8d83fb86..1e8892741e714 100755 --- a/examples/src/main/python/mllib/random_rdd_generation.py +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -32,8 +32,8 @@ sc = SparkContext(appName="PythonRandomRDDGeneration") - numExamples = 10000 # number of examples to generate - fraction = 0.1 # fraction of data to sample + numExamples = 10000 # number of examples to generate + fraction = 0.1 # fraction of data to sample # Example: RandomRDDs.normalRDD normalRDD = RandomRDDs.normalRDD(sc, numExamples) @@ -45,7 +45,7 @@ print # Example: RandomRDDs.normalVectorRDD - normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows = numExamples, numCols = 2) + normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows=numExamples, numCols=2) print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() print ' First 5 samples:' for sample in normalVectorRDD.take(5): diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py index ec64a5978c672..92af3af5ebd1e 100755 --- a/examples/src/main/python/mllib/sampled_rdds.py +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -36,7 +36,7 @@ sc = SparkContext(appName="PythonSampledRDDs") - fraction = 0.1 # fraction of data to sample + fraction = 0.1 # fraction of data to sample examples = MLUtils.loadLibSVMFile(sc, datapath) numExamples = examples.count() @@ -49,9 +49,9 @@ expectedSampleSize = int(numExamples * fraction) print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ % (fraction, expectedSampleSize) - sampledRDD = examples.sample(withReplacement = True, fraction = fraction) + sampledRDD = examples.sample(withReplacement=True, fraction=fraction) print ' RDD.sample(): sample has %d examples' % sampledRDD.count() - sampledArray = examples.takeSample(withReplacement = True, num = expectedSampleSize) + sampledArray = examples.takeSample(withReplacement=True, num=expectedSampleSize) print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) print @@ -66,7 +66,7 @@ fractions = {} for k in keyCountsA.keys(): fractions[k] = fraction - sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement = True, fractions = fractions) + sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement=True, fractions=fractions) keyCountsB = sampledByKeyRDD.countByKey() sizeB = sum(keyCountsB.values()) print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index fc37459dc74aa..ee9036adfa281 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -35,7 +35,7 @@ def f(_): y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n+1), slices).map(f).reduce(add) + count = sc.parallelize(xrange(1, n + 1), slices).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) sc.stop() From 1b9001f78d96faefff02b846b169c249d9e4d612 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 6 Sep 2014 00:33:00 -0700 Subject: [PATCH 03/19] [SPARK-3409][SQL] Avoid pulling in Exchange operator itself in Exchange's closures. This is a tiny teeny optimization to move the if check of sortBasedShuffledOn to outside the closures so the closures don't need to pull in the entire Exchange operator object. Author: Reynold Xin Closes #2282 from rxin/SPARK-3409 and squashes the following commits: 1de3f88 [Reynold Xin] [SPARK-3409][SQL] Avoid pulling in Exchange operator itself in Exchange's closures. --- .../apache/spark/sql/execution/Exchange.scala | 43 +++++++++---------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 4802e40595807..927f40063e47e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -36,25 +36,23 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una override def outputPartitioning = newPartitioning - def output = child.output + override def output = child.output /** We must copy rows when sort based shuffle is on */ protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] - def execute() = attachTree(this , "execute") { + override def execute() = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. - val rdd = child.execute().mapPartitions { iter => - if (sortBasedShuffleOn) { - @transient val hashExpressions = - newProjection(expressions, child.output) - + val rdd = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => + val hashExpressions = newProjection(expressions, child.output) iter.map(r => (hashExpressions(r), r.copy())) - } else { - @transient val hashExpressions = - newMutableProjection(expressions, child.output)() - + } + } else { + child.execute().mapPartitions { iter => + val hashExpressions = newMutableProjection(expressions, child.output)() val mutablePair = new MutablePair[Row, Row]() iter.map(r => mutablePair.update(hashExpressions(r), r)) } @@ -65,17 +63,18 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => - // TODO: RangePartitioner should take an Ordering. - implicit val ordering = new RowOrdering(sortingExpressions, child.output) - - val rdd = child.execute().mapPartitions { iter => - if (sortBasedShuffleOn) { - iter.map(row => (row.copy(), null)) - } else { + val rdd = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => iter.map(row => (row.copy(), null))} + } else { + child.execute().mapPartitions { iter => val mutablePair = new MutablePair[Row, Null](null, null) iter.map(row => mutablePair.update(row, null)) } } + + // TODO: RangePartitioner should take an Ordering. + implicit val ordering = new RowOrdering(sortingExpressions, child.output) + val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) @@ -83,10 +82,10 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._1) case SinglePartition => - val rdd = child.execute().mapPartitions { iter => - if (sortBasedShuffleOn) { - iter.map(r => (null, r.copy())) - } else { + val rdd = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => iter.map(r => (null, r.copy())) } + } else { + child.execute().mapPartitions { iter => val mutablePair = new MutablePair[Null, Row]() iter.map(r => mutablePair.update(null, r)) } From 0c681dd6b24431eb35770884e50f22ebaeaade33 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 6 Sep 2014 14:39:29 -0700 Subject: [PATCH 04/19] [EC2] don't duplicate default values This PR makes two minor changes to the `spark-ec2` script: 1. The script's input parameter default values are duplicated into the help text. This is unnecessary. This PR replaces the duplicated info with the appropriate `optparse` placeholder. 2. The default Spark version currently needs to be updated by hand during each release, which is known to be a faulty process. This PR places that default value in an easy-to-spot place. Author: Nicholas Chammas Closes #2290 from nchammas/spark-ec2-default-version and squashes the following commits: 0c6d3bb [Nicholas Chammas] don't duplicate default values --- ec2/spark_ec2.py | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 8ec88d95e34af..bfd07593b92ed 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -38,6 +38,8 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType from boto import ec2 +DEFAULT_SPARK_VERSION = "1.0.0" + # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" @@ -57,10 +59,10 @@ def parse_args(): help="Show this help message and exit") parser.add_option( "-s", "--slaves", type="int", default=1, - help="Number of slaves to launch (default: 1)") + help="Number of slaves to launch (default: %default)") parser.add_option( "-w", "--wait", type="int", default=120, - help="Seconds to wait for nodes to start (default: 120)") + help="Seconds to wait for nodes to start (default: %default)") parser.add_option( "-k", "--key-pair", help="Key pair to use on instances") @@ -69,7 +71,7 @@ def parse_args(): help="SSH private key file to use for logging into instances") parser.add_option( "-t", "--instance-type", default="m1.large", - help="Type of instance to launch (default: m1.large). " + + help="Type of instance to launch (default: %default). " + "WARNING: must be 64-bit; small instances won't work") parser.add_option( "-m", "--master-instance-type", default="", @@ -84,15 +86,15 @@ def parse_args(): "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") parser.add_option( - "-v", "--spark-version", default="1.0.0", - help="Version of Spark to use: 'X.Y.Z' or a specific git hash") + "-v", "--spark-version", default=DEFAULT_SPARK_VERSION, + help="Version of Spark to use: 'X.Y.Z' or a specific git hash (default: %default)") parser.add_option( "--spark-git-repo", default="https://github.com/apache/spark", help="Github repo from which to checkout supplied commit hash") parser.add_option( "--hadoop-major-version", default="1", - help="Major version of Hadoop (default: 1)") + help="Major version of Hadoop (default: %default)") parser.add_option( "-D", metavar="[ADDRESS:]PORT", dest="proxy_port", help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + @@ -116,21 +118,21 @@ def parse_args(): "Only support up to 8 EBS volumes.") parser.add_option( "--swap", metavar="SWAP", type="int", default=1024, - help="Swap space to set up per node, in MB (default: 1024)") + help="Swap space to set up per node, in MB (default: %default)") parser.add_option( "--spot-price", metavar="PRICE", type="float", help="If specified, launch slaves as spot instances with the given " + "maximum price (in dollars)") parser.add_option( "--ganglia", action="store_true", default=True, - help="Setup Ganglia monitoring on cluster (default: on). NOTE: " + + help="Setup Ganglia monitoring on cluster (default: %default). NOTE: " + "the Ganglia page will be publicly accessible") parser.add_option( "--no-ganglia", action="store_false", dest="ganglia", help="Disable Ganglia monitoring for the cluster") parser.add_option( "-u", "--user", default="root", - help="The SSH user you want to connect as (default: root)") + help="The SSH user you want to connect as (default: %default)") parser.add_option( "--delete-groups", action="store_true", default=False, help="When destroying a cluster, delete the security groups that were created.") @@ -139,7 +141,7 @@ def parse_args(): help="Launch fresh slaves, but use an existing stopped master if possible") parser.add_option( "--worker-instances", type="int", default=1, - help="Number of instances per worker: variable SPARK_WORKER_INSTANCES (default: 1)") + help="Number of instances per worker: variable SPARK_WORKER_INSTANCES (default: %default)") parser.add_option( "--master-opts", type="string", default="", help="Extra options to give to master through SPARK_MASTER_OPTS variable " + @@ -152,7 +154,7 @@ def parse_args(): help="Use this prefix for the security group rather than the cluster name.") parser.add_option( "--authorized-address", type="string", default="0.0.0.0/0", - help="Address to authorize on created security groups (default: 0.0.0.0/0)") + help="Address to authorize on created security groups (default: %default)") parser.add_option( "--additional-security-group", type="string", default="", help="Additional security group to place the machines in") From baff7e936101635d9bd4245e45335878bafb75e0 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 6 Sep 2014 14:46:43 -0700 Subject: [PATCH 05/19] [SPARK-2419][Streaming][Docs] More updates to the streaming programming guide - Improvements to the kinesis integration guide from @cfregly - More information about unified input dstreams in main guide Author: Tathagata Das Author: Chris Fregly Closes #2307 from tdas/streaming-doc-fix1 and squashes the following commits: ec40b5d [Tathagata Das] Updated figure with kinesis fdb9c5e [Tathagata Das] Fixed style issues with kinesis guide 036d219 [Chris Fregly] updated kinesis docs and added an arch diagram 24f622a [Tathagata Das] More modifications. --- docs/img/streaming-arch.png | Bin 78856 -> 78954 bytes docs/img/streaming-figures.pptx | Bin 887545 -> 887551 bytes docs/img/streaming-kinesis-arch.png | Bin 0 -> 115277 bytes docs/streaming-kinesis-integration.md | 94 ++++++++++++++++++-------- docs/streaming-programming-guide.md | 64 ++++++++++++++---- 5 files changed, 117 insertions(+), 41 deletions(-) create mode 100644 docs/img/streaming-kinesis-arch.png diff --git a/docs/img/streaming-arch.png b/docs/img/streaming-arch.png index bc57b460fdf8bb07eb290082a0655c1052834687..ac35f1d34cf3d0c94d039b16f2ab8aa0cedefabd 100644 GIT binary patch delta 52633 zcmX6^Ra9Kt62#rzHMqOGI|O$N!8N!Y+}(n^I|K_J++7C<8r)q1JaXTB%*ULWwPx+^ zuCA)?9dQVCdXeiqDsw`TJBE-35i4iMkBV)b{+RYBA|9Mxm=mSnh>J55 z;fn1al>yT)xoF@dv?ZcaZ9z_wIP5uGYS@*M+UQ88nGs9f?hi-B|E%gnDp=_sZ}^is z$d?!4(`Uy7CR4x>n*zU260&)4BR7nb3zfRxqVMFuDFJzN-pxy0Z#Yf`Wq8W8t^GF# z?Pv|5md34NRjEyw#VTshH$GA#*hJY4Q}vk!2Gf^{$i2zpQNe1OTM0#(PQrmv%Kn=h zSgQWhU%`k|iin3l*n}?iR-PaixY0n^=@$7$y{J7zsDnU?>^Rb@z6RlKzz8%tR%;KK zBa=dFlhScB^0uh5^w1qRB`Z3&E1daWVq}u2%j`R1O|$L8=k=yPG+b#deF=^H$bz_Z zr={wXIP}mR%cdx0T{Ip1?UR$t*@z#<4$cC_ELF%q$pgkxQZ{QWI~5PAl2$hVcp$xn zMvm6H3%Ev{+PF5WtK_^d{um$Hh`!^}Kn-%W9v@x?4f0I*`>j`rk9@ve^g{VoH2Vip zj}Ssf8fISUw6Mz}`9`&WugtW|+9G0)v4}^ngKXh51NuaKVGcbo9(vMXzE}_f_7oCRE&hzo` z8lg-7z}|0e4##n|k?J>dl93b_eXO8kkk~6xRqegm#!OsL&ZT(JR^DZUS_xf7&PgQM zY?Hww`|Cv0ov$L5kCW6W2sbggG@VP|Dz;2j4}|4Op#+wd7!M&3BGjE}At=A<2=Jeh z0{JIq5GyHtzk*>^hLwFuIcJ^BW&Phc4Wn$GUcnMLoVvu0;8zeTrGJDrw=T6Ji`MC0qo$Snl_RM`9@)%;?yPAS ztUt?Q_Rod-BMDZ=g_27Ir(eicn-YUUfejEDJZC~<1w=51o3&QvbGSgwCB)XTE(!-k zjA#&G|f$r9K@Fg9Z7%2+zh?;D+^ z;NsLAQQD2ei!j|QjA2zKr%k2v?u+dVQ(>+E5+iR@Nbt`=GO2~-FiL#!wG$VQ+1L$(Ej+;DHLfa_B&K#Zs1PL zPv!^|Unq@_`;;KrF4F2I5eU+Y^vKlI8iQAwa2_&KnD_k_l&gV?MVN;d9}j{=#3@+5 zz&FxJxzlvM@?Dr`{0zURY&yaX)WKSbNE7?1kz^)mob(T=Cz9Fo{|Y|ElHed^AmBU< z+iAA_&eKAjz=Ind;LhRiDUM~jmc+{ z;G-)_b%J5OJM_w(g5k>vlZhJ1TsCECTdqFT>T7qejTG{B;}$bV2kS0=Bv} zvKMjjCXjZM{L)dq9+%*zpQy3#M|dx2`*ANMAK4w*KX=&o4bJkCIF z9-jgs5GVuF`011O(0>(m}Rc`LBb&AZw&9qtIckwUt4Vket+FoYUWbxkrfMx1L;xhJxiB( z>PW03cN?beYIs6`>cxVUr65XMwuH+|e%4aC$jHC^AyD`omGeAiP$pCa{XS!Zs^9Lo z7I46z1(=+B`CK?WWn(uvG<>ltw2vNF{jHux{SM$JxJ#iWlXGAz4U={ijM`xRP2-HU zeb5ZY!O;Dqi%!G-0W#5KU@FBtS8Ft=a}oJkq;j3}W@0KY07ow~B| zB{;$T>(6m&oUo9mZ>|{ezj_O4U?{BVR!tlc3_7hn)qa^Ja6_somU#vPW%SiG zu*~wL6}s0B3?lE;HBBkMQ8~Yr>$0u*cI3w_iX~BCISq;kiFs^8oEn0$CoCxOf8GF0ya4hFtkxO#pjc_M zq+@BDe8aaOVDp?|E5$+h(&ZH?xb~Rqx<~So_+@`P(Ms1PrZmqIgqtN#wCf5v<~HB!DGjL)j*Bdv3l1x z8hE*K6y6EH7N1tSn@JZ8d+_ddcRU9Sx5M^dUwfN#|Niv&{E4$<=GU4bNppWA*R^Vt zESbq2*!~6vc_orUEi*uWEYh2{gYFqY9NN>Lt`2p=_G7b&-qwMUj)f&oW2G~onkJ(1 z%0r{e`jrk;xn)ZHyE0W8Wl0VJifJaxyPSW#9JJ;m>?k=WSpH-qo$m!V<1Ke7+vm{f zlBSN)2QA=n{t4n0g(fyW2J9muO`1@jb5(7sc*4%Uevsm&c%XOv=eK1un(sa)bk;gE zYJv`vnOsc{5NCgflDJtBs+s|<%m>_4^hx6zpVrmA7ZR$N|Ga>ha3zza>uU^}HQY|k8+2Lzi z;l#nDgqqH-jtW3|r|TW+NE})oxBShOq`Hzt*~zJA(21*wRzUdn?HzuHqH*O1E*hmw z+(d0{O*ek67>&8tdmCbQ@+-FvtEyxkSU*tM{vK966P9m(n%qKM=%smdbVOmIm*LiT z?4a{3OG!Jz@radRzT!6tOJtyMzuSGgz~lv8D0%CpYz;Kc$Eep!tc!YPKgoA>HKN?t zN-n~B+L=a?YL%kYs3OYE2Z#Ssr^h@~Pew&jFo=DsZD%4`a((@jAeXM|Zo(~3&JO>l zOWV?VIW_G&5L%4K*Hx#&>DH@%i>o1qI3>e`m3w%X9sjRdZ~~zbaVB6YT*-4`Y52KT zTl{thfJjW-sVv;L%d<~6iT?=|?r<2Mo5@6@rKk5UFE0K9!52bX`$|C&!_vSpbBZtw~jvvBhvwNn4xN!he9Cba+tew*)oq2ol(=DcR{P{1l8J-$? zrdJ3H&L|B6{-#R})6rQ*JgEWa36mA0{Vf`I+_6f!C1+@vTZA2F+b6-tsrM`>cXKn(( z-q;*+o;NlUnM!|<4vf=-pi<;rE2J&a>b|2y`+aT-xQc+S?HPMB8soY@uU39x)*l9) z4!G8VTxujgJWxpJVAl^!hn0*oO6%woc?AU#nc9CLIMDcL$g;#xmx2joDa^lqL<^ho zn%B8{^3KOKl(mMX#Tii`H&`rs09VD&_#|5WPNzXM6kdiu{&f^gw16ykZ2myP(Apk* z#NEXbZEQPRaW?xMjDvzD>KF^k|5GY^I1?NFjSzTD=0O7ZaqKnz#l=OTsK>e#P84eR z!0YSl;pPKcq%>21kcn!8aviICCn7W*uk0!~H|fYv>%qYtgH1C<5Sfjg07OUwwp zN+ zHNDq;8ElVN``qb6QpB$O4{JHhH;>}{P`hx-&r z)%=F#x%n@6|3Hot7QF38Pw2@(rNx%Q9*Y=fPC=D{11~vw*L(P2ajq z&|o-Ve9c|zjSsR8YdWW=CrF4_!1N}CakJ;_R98k)!M>@8ANho)44^J9yc%0rA61bh zw~+Z1MT90LF|D;;tll7$dTqoJcXij$c$#Sw+^?fw7ymYF-=o02qI6&r4K%blEJX9m z_B&{4>d*u+#;~9$Bi+W#cJ!A8rS{NG&|!)#uacc8kN9$5xT%rDtUwO{IVT z9xnCc%s>RP?{UQ?wd4q}A!}YA9Eej$#C<9DhN*=SMobUC=%Cg%!>6Q3ehX&6q@Z6U z;GOvBxfPM3WN2-O(-FwEOF~GW=^9O1PYj}^hbPx(ECypV-}~A(8lEVImoNs94-T){wjTm-zSEq=PVTUY^pD+(bmD-Jv+U7))g|RQL`o zgAKwOR#6j=ziCTn#Sdlb>D*oZ-u!#L6qnomD}#c9HbqZaYqEOXW@1FnvEB)v{uofy zzpYR^(QyW%!Z<`cSENBe(q*WN)z`d`OA9)Fw-`|x zHd6IX2eH50A3a%43J{%hZT=PfqQcX38_t?9GBxcpVE7HPA_clNJ;8$gK=$~}h;03q=3Q_{u&}McMC6M}u~Rwu(`xi;y=(ToqQ%vU0>_q^KCX1zO`W_>moysVJT z3jW6C@l)fP8xY|2`-xQj*Y(e5EDcvtQ}G$j4ZzIClMl~-Gae7tU>yFVq|wB{1TtLv zncHLvHeJ37C29Z&e!Ps|$@1_y%FEx%MxDu%?OQNU6 zm^&qHIe*>+<{w48c22=y&d$BoE!-LwE6S=UKsN=SuT#aq$Fzzm5%j#WkWz{y%R7A)rpXE2BQi(XFpCio(KwX|9n z59H;{iupM8!6On1EcW*x2(QsX&5)-gQ!BMY3AFk~9mH`smHiSq+rwD&veo#N%|`-3 ze%t$Aq`P0EQ~V;#nCT0(^lpt+0Y#k;`pJ`he0HtH zTsNM;3Zs+m`$A~mN~L9K({^1&3VKB#Y;}nlP5FlH zq>d`WT3_#;hAeakLvs#ij}41Sd{2so2ru}I>ydE9cMJa=OCs}Ud+AG(!5bB4Hu5Gm zJ}c7e-&+$W7x85$>VGcRcerf7`_=f}90^iZil9egOd91=Tsf;i(YxGCHQJU|J-T`;f*vs$4sX4x1?o>-g%Yayplv~@MVg+m)$yUtfax{`)$2gw zxjpS+Dt3q(Sa7-ybH_1lOBO;zRG2i0NKtq9`;GD*#b$703exsN3Qy(6COim9`an(Z zaja{eOnV?Sw7q61K|cLC?JME6cA)6N8!P*@|A1?&E1L2qcq{R9NVW3qvIEi)Qds%CU-MU807Dt0#+r^Cv;8!qQ(-%}_Oi=doR zmDRhAJ(@&jPGS5`>@m9Z!16B#vdOa!=mR^7CW9YjbpB;!Pkb{;NWE>~0ljlbM)7&Z&+H|{A%DhcRuxd`N{ zrbi#FK?Kkg!SSt6BN-%}=rcqOd~-g*wOu~HnmlluV}Zo=InHZNs;s-UqrX@ABmE#jU5V zN4u1&+*=4;T(5C^yH1y)Rg9Z{`Ni$Vi0aFjOs(xARP&#;;sxZi_Aq@O4z_tqS}p^S z3BC&jB@2N5kkW%~?;#Za+gQPIJ6Nt*2l+*h-N{44dz7NK%K;&tKO{=q zh=ZSD8w!TVT=Ow6#R~4in*wSGX*`ZFbdVru6lp^t0{z7dPl8@#sP3R_xQ z2Hgko`5LwBYxTx2x$}in%L&pOzi9)kcu&XnEAEM`O#yiTGU#hi;*;56)T8>`-{S`X zOFS)XoaL5>7yrSDE<+(mmugH0BUHhIg}p;obotEuim>Kcnw?=M^%qR&JqC^L8o!z~ zkSnXZgsFOWo{$cnkxAlXvOR;OZQ(jf>^kbE_v5F3CI|!6*6iP`s3?Tven6_UVB72O z%ZJTsw(3P1 zs)b(;cQpZlX}TT)-pQAs6_&$GLk_ov9mG=hj3!bnloBim+}uv9$q!>ar|!h89mMJ? z-F}9R%~NSjB>ec#0<#4!i7HiXaMGXeZ=0;=Gkc7pKhwgZN(RW5IC`A>5Ze-Lz((B> zW}>;7Fvnw!Swt5!^3TljMQ6eEfg30L^1?>9ka7%`D(CPwuee{Ort%rO6}W{)3a5oc z#=D$SU|H8ORr|Mn+J6x~^)nxr_hqLPFCC0Pk~dSK z;L>l|@7kgIVaD!qlx6;y%s?mVYo(?6o9gZdKW1I`OyAW^^!lZ1X`}$uhLis~SWEyR zxF_d@m`Gs6K}0_4_l}QYLSz$ie6Vh*t-^WSjYyQyXtl{?kOo~@*?^;9 zrJ+47yW{W9N!ACg@1y-!T-Wag)JphdCgm`Sa+bXzSb-hDexXGSR*B#mXWsN+&mf9at)%|2L7qZj}`OG(ze7JhKhuMijdpTz@-{=qm!cD$}x z?f@j&lBy3$?@^pNWcq)#MF~O2wvwV{g=H}PgAxoekR@EVRdR>O1PTNW0gfJXE!ziH zG5l>jO2r0N$#GFh@F4qJuB6Nr=A!I?W0s(oOK#@ScA6NNE&W-uW!wF#eYV2?VPkQvk#kOqeCI0BO&FF{t= zvj4O@819P6CHpP%!riXbS&wC@$^mNd--llxc~(OhxImV|c?6P^>h`%RU*!}@3`Mu% zfm0OrJJ*{SkDy>?m3MvR1|bvGlom84Bf=b?=NL5BFDXZ#@6FkOSwV+k;X3wdFgL)t z&i?k?U*M2+Qj%Bk5CnW#i!_K}TKc}LLa2!oPCt}a9!mrV6eHqzoPSlgGEX37lOv_z z*(y=j((UMgg5kZNKez0@4~QJs{)de?ijX4@4*~8@Gtn$ zRKPb_r%uu{_(pkH@;uK%gZ;P;g)%iismhT8wI;x!Q!~y z?WTnN38Sq!4TzBd%m^vUoIuNKuj(T+`D-Q_3L%X+ z>TR%GgS`zcN=HYA)<`K`bzHh8*An7f<2zCv1kL1)tor7q_98lB(I%^_)vv4E4E@*w z9566ZFgZyv4c>n)Do6+-G&B@-$Ey(pJ>Yj~6-RK;HudfQiy#tFqU*{FU zKw!8Jpl}+MP|o#X((-&0)7ds9EgC zfDHUMZ$Kf=6e*pn_7T^4hP=E#4z7Nq^o(zGU#{I?=JKOC4x?6D8;sI_x<5>~0<9M?wwAB*hU4@}mA`yL3tB^Uvj+OLNPYervZFPir3; zgfk&nZRLH$upWq9$>ge3$rEN1%3om5!a^DGZ)0}vf#f!c!b{|?d<|pX+x9K0P z!hSD}t`@lh7hl`S>xZqYF6u_||7c5-9~2I=fh^SE zVr|-OO-04ej{&h7nbS_>&kV{itT)$qX43HGu??~ye;d5#2Z*`9;7f%Cv3LYj8<1W{ zPRe6YY+EYQ+l-xZ@-YBWO?yEQiPOt3kVpzqhrDM!nkM_F1d>O8_0wog_H9+$44wc* zf;c)yw$0UESe%h}&GVHT;}mQzWJD&T=n9wa5DdXXrwra^1?4FxD6=KZ@hPY|Q}U(4 z#@Eu$Ul68_lDg={J^C6cC9BV-iBNS>hjH>xlkILD3&hx6E+jz0*AQ8E3@<%)S|wrHPq3bbI)iuF5~0|Fnh#9HUPC7E`)_ zHW&W%fw59L>o(HID$p4U_%oTrm)*%3+x%fsUA#c8Krjz0y+L~u%6}y*L>7=^kXD#e zS79$DbLHX+V8v_AgS>d{2e$qudt6NXCC1V9pP7L?Fy}M|Yy5GElnS>~wC2nGD`U|`l4AH|(kvue8m^^|M#7^XXnuzCpQAYY zNDIU(cydk;EVN1kiq@@UrAGau3ii~u!nC39Fczjzde9Jxw9Qd#z0x23LDWYKEO=^s zR?E%AMkj4h;Ag-zo)+(=z{l8a({gP`t@?!nM|~tLXln!|84WKc88_A;0+ZeCwbQ7} z+k)nqC)h2aF&NyBy)DL@f?lv*`qj{wHqMxOqVc z(Fl^Q_fk|sPTI|J=6@Sif%mb!*?iwxw)vccqac0*Pal#qN*eQBLK1cVBhqkgp++q|S;Ws7Ydrau#*)a?A z!SyOw6AOx!-xQBEP*G8lp@7O_7(cvny>y_X6#6ScrayUmluSVO(H@$dTED0M*R?Qr zspZ<$vQLeoJm?&5T$=trz8%Pfx0Uh0eR|dP!B_GHfSGT*oV;9$vD}4?{tK>!BIIoW|s)yR3jO z4Sq4WgeLi#DY-5w$pZxajAYEL(Vxv#N`&x*Sg~%v6r3Beoutw3r(4Ub*XwdWJ4mA@ z9je?(+_*EtmY=7T5+yY~0@b9G99?+fRG}$bejI&@G}!!J9%RLBH%PWK+5u0%{nUhu z3A#^2oX{ckO)xA@`gw5Xq+nXKC&(cnF0k=mGBbzWLsmu}=gA1+w2G(a3dNhCFy=T%RxW2O9skq;LN+L+eFAynB zLtaqatwoX$5MW_p*~bFAWS5xCtn8?-yeZL6nzb+2?x*FkgNi>u;+w#iF9wHs8Por^LVP5_rqS_{U>fj);pooF zEDzP?1+jx(vu5Bjwca3H`BH=#Nf~uxqO61zLA+)Hx6=R4^&DNcEqVPs(G2Wkm5@_b z0GPATXkeI9lg`(qV6|PUo@sO5We#KOA*9^QfVtibWb=a^{2(E;48()*bbUR26J%5A?p7`U`Q z1;_2!M>#g#uRGB-+efZ6`}?JfUo1T;QeDvO1WBp{a)L%}CDSdGd6-4}Xe&zF&u>NLmt^^h} zH9=D!!^%r$yUVw+dIpA$D9)mhrW(vp`(A(DkZV*4A^y^2Km6(uut?1kUGh~Hw1rUY zBDIhnsxw1;catoej@nUhWD#^OK6*P-4}tNNKWdTil6I!IgMM<+rk{(UOc<2}{z?fZ z?PKex8Fl}#+UoVW3*g0hpkAK7i3E-Z&Q2mamyz+4-bX7nOWv>OlmG2m<-by}dYrFg z?+(WUP}2)pSXjzp@1_);IR|vsXI!2L1aEY=+B3KXOiX3@K`6+nCVnEZLllfLq8&0H zW##@(<1_=7&$8a+KPqgciD5&0K_A@R*>fs?6Gh#RqDL57x+c|$*Gazva7y>0g1f?O zij`e@xngOehO%9L&I~p*GP>#DUYxS8DSvC!Kj1$XMBt^b`qxugl(Lu@eP%wm($5jp__K970M zA{0X2(H{?+tHjP^@H4(k12$34YLmZue8EYw(0_pJSglQyZ?Q+u6!s0egRq|e)0v4)YGXJaQ0_eUKHaq4!88TY@ zz}S;GY`PROj%AtD&t3VecDFoSY-C@Z#S6r@XP6sZc>cwzxJ1&*mg!xB|MOKKaXuGl%-%nv9^*RprmS3cMyC#9uhEI-rNM3Kmu$Fug^UgWSWG#3Su{ zU~D8b$uV}_4`sR3;OCn=2%E5m9||~YhI(3KkdQIzd})9@@4v7VDpm0E=`hU8;?`pW zU{?GoA>E?ygzZr6SOjmNSo1=0_c&*ZH-?>t(r=v0apnZaFXVC1Gj_D_Kf7v!yOCd+ zZV5lIC4m(}$9Ug+o)d_hBKJE&K%mj>OgG@OPD;fF3UE zk29Slj(}0>t92|rOX3OS&ICK+bD=xbPN&Ydqwl?#GfRJhKm{6ZJm71o}xYhob9+!)x5w3+$LkxwhP%@(6;(FzKi z>SH&J6iF@S)7>8J9-~8?fxUx|SqjnF5$JHqWG{E7pt zDM|tA4plpuK8SFBuAYrFa^ZPfc<_^sDoiZVSmOMzFe4&s9gQO=148{ne!A{KKU>aU zb%+_^uNfo}9Nm#!cb#rW}qPxCKmjm4-04N%`P?>Xj<7T!3i%wxz5Ef8;v8v!jJgkq>f z7<8CkB&#lnzLBJK{Z z!k1qt-l$8rTr<4q2n5@L6996GA7%=8!VS>)6p|*V6SJ0#hI@l&P-;(~?Lr@Bau2J_ zhqXRDH*+ha-$I!rf(Yq%RE1en>H^F71P6mG72pKglK+N~Qus$YK3L=3Pf{1dI2WcH@BR^Bb zX=n*r*Aici9o&!ra3KPlBuCi`{OE_9P(KKMI2o#s4R2A%)4s9a#*{CtB$D%T3Eo>z zb6^@&tf9S^F)-n~GfHPPjC|5|BM_Rg6jgD{+v26%3J2qlnhyTtIh=a%jL6&;;7@oa z{UIl#?2{&St?IL69odljuu|$p;6T^8iyv?;Ew%7725ui1y938ZL-L@cte8ZSHK?

6ng9;{A0xgwd zU$%{okBHs_|fgnRP=P z9pb+rRd8R9tmjLiQ=9HF45GsP4`)}5J&XU4pSjT*$$&OgpJ{VoC&MtrBnP1H?k)mq z!NJ01;e5hLg8YO1{5wa zBSUYhETa-;>~;z=d)LFJ^k2dWgSX`<%v=(3~|qhi|_w5hY;@&gNe zGx`OjVgMvpzEKjAKCMdMai*+hhFiUF=xNHaokx=Pwhfkv}$0JqOyc7e>9e)5> zGQj;w=+J$6lv8`1B{Mkr*6DL7$(Pd104MfSCy&$C0t0Ve0KO);J=9-ix($(bAp4>;4>Z-Tw)-tJbdW1?_O(-lk58iSG$v$xb-&>Mii_7(@EZu-WhsLx#+!9P88>vdg&~(!2#0nI9S6nuwHnT zci8NB^Rycme4Dle%3)1onFrhJ_XPfH&NrCWVa>g9DDWp?>+1&3WPSg+_o4vQhkfrT zz1Jcu?q~AY?f2q;CU46!dqjdve42PKP#k+L`yhWDGC-40nUX9lW5%$7v9sIAM$*?uZ=2;d(9ns4`y@K&}aLA_37aU$~eS`%6knd?ta+%uVyj%w^FZ zi!D$U8Y-XQF(DsCGW0ek2M298VQaaJR?Pks8?1;l^?1rc4)p26#bUa&jscd97M@;6J~U7{W7crgx2rrX3R`=gS8=@()10 zvU)MCZHh-P^B3~En+TKc@<2lsAMF@M&oug8-gd*?>d@mo3Mjh?y7tO6ghfXxwl5Xs ze_3&d8tWR-ytj>bM|l258>{w?(oBFAPD}ndh6qe!ILJO?6P#-LQ+uxPP8B;rB5651 z9M3X}wlk4EBSbgxpCO!~CscTd7uA!vu*}GGy5PC~Yr4alnY99#1#}82k}zdwj2$8c z=eHbQuk85&W}pg{M?MeIFCHRlCODLYw^*{bI*^*4K2U18I@o`!omba=J;+5Q-2V@K zULj$=dV%Y2rdx|$dk>0Xd~gvc@4I6+)TOp3Z2lO?9A7d_l1yIb9b@YnDjKH6_>wH{ zk1KOJo~@;8={CjhNP)=aZNlIdD$FDp?_AV;`(}@74j+clcDAb9?1s8(t3avEQ@=(sdqaXTqz1zN z8Y=~*Aqi~d^MFY`apS2AAJ&kDd?e6pcK?Aj~Z%GpR7<75}4@)4x zOwJ&^wpzh)(UgX~Pq5u7EuU3|BQyN<9gM=L`(-|H@&xrkuZx)>6w(;k-u@nlh*Y$o zO`p>Dvv!U^aOqcu+0L?F2Wdv4BVfvjU69B`Bmnu&;IYxUVYT6vNwFh9et)XubHwS$ zH@F$zu-wfH@eT=MFU-L49Jo0B6jtag{njepY1&wc5f}o*#PB88L%eeOWVC`KGcNfL ztQHWXix0}mw-j6SU)+sA4Ve>iLX7>IasDF$^IS%Pn4JCc6HSo$w=>nK z9l%5#?ec7$vqZGJbdgNd@S!8$T!^t9&AQCHn1@I1wL!q50+oeENr*d4@i-JwrUIAp z^v`oTqX15!FAMu9+cqLpCrE~c4@Wv*4Q{5^p}n?>7*mh1l+ZoUr(6xX`E|lYN^Vr! zP3w@GF-!J(XOmEC2^phnh=`I~{UQ}HbVT^mF)C5gkq2ZRnp34`MP2(Q!0&Xx1EIIYx&KhcJd8Jxo%OCAq)Q0@>dYmP> zbx6&o>T)7i79phtxr&NBQft-A6;S*%_mF=fapcyF;0>N4ORd>XAubtDKU!atr$^Ses>_AGLi>S+X-YX^BW6#f*1F1Io-jB}a406sP1>>$Lh)HV#wyJ@^wJehSj_ zctLdk!)j}2Xqh`FzY0VqA21(lRP?EK5`a|_c zXbH$|7cF!T>8Utk^8Xfp&_AzjICJcp9#$TFR@Q2F*yX{#M~uoqFE)!cRF?oal+u!z z^jmO|30zTi)K~#|J@3vSD`9FlZpz{#PES|hD$av1@1LoOB>|Q@=6B_Mq^$izYpd&~ zoS~*m$}JvdK3Xi5p9_PsbCfbiHj1`RH(u~6Nw3j?LeEN1=dnN53~upB&^0VcT8w8l z4QVYxQav=H3J{B|ewG*7MlU^zJb&0eA$g*W6yEBv0MIamZDsA^h|gX4enWjIdK(BU z`^%)G`X^H64%m*J(FkuL!RMl_hkVTZDb$_{w^$FuN4lLV18XdMYk_Qf;0!8d*A~uE zeTV1^6hSE1`f?pO`ZXrfTuvR!0-ra6z)vE+4_i;Q2*GwOrttDUxMs*i$-Zv)x}X zpl~k7rMF{-S*D<|v9U{8uTZ9_^4MO0;|H(Sss;)@b}31t{Mb05lFrYwUXb=X6XQe& zzsWBKdTF0*+hZBo&s9w}ox*XvtN+~xK7$HWrM>0NJidi5U%XZP3!GJ}h@QDCfe-5i5K7+QC3DU71^l@zHr5HCSQ$ zdEH}rmp2#Bpel*a#>c|3^^d0A(K=Bryqb`NHL2vYh>?c)L)lkj?F8)_;CcaGG-_#S zO~I_i;pcZV3i*{Y5_ei$f&DR}=IFg(e6V{Gncl=Z*koE;oO5qq#?zQIoo00q3rjnf z7K9N%?gt6AZnEFMS~8$P{WATUv`evHyBNe<*?7105xNa+rb&c`s;??XQ#q0v0S{K$ zP>MIK)_6x`K4vKI4a%y>BdI_QSu({#X`l{8T@uquO%*SV%vW75Yd&_v;@~j3A9jQ! zD&hqEv{-Z}Hd0El)-b<3YN#%<>t$4S|HsrjM&}i6ZNrV#*tVLav2ELC+n}+z<1}`f zq_J(=Y;4b**_}(aYPMF;(YTutE8@C3XBB9v(*0an^Q8 zP09|r$-)&2KiLd1!&xM$EEq4ED-rEB`f?fO?1X63>sMnr7hD-DvrMDi-ckmXDPyyd zu`(n_p2Eae71PnuIs)R>f&FQTe_JinP_IBC1a+M{^ zW+@_&dGftk(Jl?uoC2IRR#3lr;3IUvOo3vi0&x9D1Mdgk6W)PWEP!C7Y|%S?#V!Zc zbzL9_@hUVyp{SBu-L({a7*1&mUQH#|4H5QY>VmK06<))Cj>zB@>6m9gxH^#PRCPV- zQJG&ZhEe@bywU{=YC)SRX+gAeiY+B;NhvArwRGkD5-^@@sRb3y{0Z=}5<5`_M31^W zM@n<@uSkte&$PfWNWm>oOe~hCqUf6c13k9>0}nyT+ZxBc&kM|6@w6K6EDP7F*x$xT zW@mj-!?Gx6_T!WIkG^ZHRA0Wl>Ok1yP8#Y^iWW#2!PBg`3TVK0sCBY`vUu{7IiEec zjO`XUnZ1d0&1n|QnR4|f3n{}NJBdoE``taTY`9=vp+MUQLo&%`_yx5e0~wOP_qnf`Y{@<&V_t zSYYqt4yLj@FXgykR~7aM9mkxyhuOPHf+X2EPZ&-`&v|IX;Y`gL8PK=nrcu3*;6L*a}1Bf2Y3IzHI=4+Wl4>D7Es5CV1) z+#fbMVvpjDKS`SUaE0Uva^HsqUYq0=bGHcvQU;&0^dtT z9(VJ_WTGIDMLQv!1FN1WZ`OhXoyXz!9yof&^lclfU1;VUtBPUWVn50=gjZfm_zGlp zVoM@JgT)>g8?bcP^^y6n3wz%PwW73)<~bk4jlv+W24L-(RN{Mc4-cGuRku#T_YcSxIx_9ocj#UIC(z zhRzT93ss*ee>4q3Qy1f+yg?sR#p`K%<(}QYl4Pz9SFGr(yHe;KD0~NgC|*cu4t{2^ z#*b|8fLYtFF-=A@EDcQGyiXU3iwa$g%_~`UWoG6b{ez#B*N_1nx=Xu5ZM(T^;e(s% z{uzBi5>X9*WF=Op84%AvHr*vZPX`9LB!_P{y@`0RJ+N+UimKKE620}a7@vY4p;&gFmr3^K@KT{$gYH^I?KhTq#9Xj;Y?6}m7 zz@G_~cO{XHI{CCE-C>U0JEoBKo;53q_f{-$%{0DUN*CIhLc0M~SnR?345N&|;zi6r z$~&R1GLb3r(cD89hVl3Ad>4Nfh{<=2YSVp>@pE}0KeRN zU|l-V(HR9m7I7AyeM3^}V7Q4|NGyODF^qp?5Pf}$(o1HeWQ@;2}eitSey3^Iv`6OAG zYoEzM6N@7DVJfwfQ?z}EqUS`sNLRp-(u~xHb2deq32jR=ORR^23|~K`C|U_V9WEV4 zeGqcE0>9sLgj|ZUbtt$odqQ|k#9NLmkw7}hx>LeW4WuTh&F0Ud2)t&WJXaAzofGyu z18CO})Xm!((LA^l27>sr1G3b{>n8yrSD|zNl4%0oS%<&f_lLTfv^S*9jrq1Nkp)A# zSm1d1Bzd-=qdG?qrfbelx=^n@XB6O?&(5)o!P|D6#YhUG7TnVKz%6bl1AKn4W&YwSgm0Fy~)3|?;!4Ys3!9oJlw?wHG2a|OHL!5#?p|t8&FbW=8KscjdrEn!N zG53G1HV5{%&_IRdS2>{b^zk6GQh7#&Zx#6uc_j5Nt`qgmFb?idQhrFpPY0D^$*|%2 zlQuFQWWrBF3dHg1M}b56h;WG`p=Jx^7=%bFE5p2D6xhd$C<;n%Kz8RZ$wV%w_%FIT zqRngtjswnfu-`cHN?~g5DcQlc2D7k55fYp%XV7I<>7738bo?1lcvo}Z<`Ovr*f|rRv&Sbyc z=R?9^;I z;dV-$sh8Z*w7$O_$9&P5(pYNdNfp@d)9N(X-jM>i@n4 zP-4rvV(-m;p&stNQOSyPMV(qWTI_YcV(#4*fp=Rz9~*R5v6}eguPo23%Ux8ntRH1;007*gJ=`V<<7>cMBYZ zvy%q5B*?l{*YV)A@Q2n>Ik1F>YY(7{z?>1PZ_k2GR^3z400L~(&|4S~&%+2{y{qLZ z2DONL$M7B;LS@18o16RC@ks|0Qs<&Q9bbALn$WaQu8JUc9i55wEr^=(>+#n(s(6YS ztv1;o_{ot?t0fjzmEwETnsNui9=70oaA2e#lXf!7v46wzLAh~1{^(2*_xv?JV;SF>yngZ;o5UU54FK5y{@xwNY4Q!yh5(? zi&m5Z0e~kVX`v{{18yyPYLaMD#Qbv|xQtnd%o7RTXI^b*!{DnQa-`2g*F)9O10JrT zp?b$q`UtO#v+^vYv@&si2)BOba_$IDGzkNlYguL+9yCieFnEgJ0kh<}EN(6=HkwEa zSK-cJA2`aib_#75x5P0EI1&^o669|dYp`-5?wah$i};c%g3ZL#hBu>=5svBh46pL~ z%UL&722lASGE_G-het|{ZN88m*+tWYvBbr2uwssQ5)F@la0FqkB5u-i=Ys?# zj<9ml>!@<_oiZq;Dxl3+3PjvUhmLTVQOJwMsdECT>;@uYqZ@4s6fvM-!6CA%cmS1B zzAe_1@4H37)7KlW6gghLFhz~HE?b4i@|6)}L2Y${U58{Po*~~xA?H^mDhs})sE@89 znGrv$aSzt$ZC6XiLfC5gyzj@jrZxwGo_)-}9F%=Gw#L8>j1d%3aP|oPQPX z^!{T`{u#}2!D6IYg{hPcXHZkcg=RK{-zYRUoph@K8U%sixZSybBnu22Sn+U%rMjI5 zwMdnGxM$aT0sah9gzX5i9-)wge zq(jj0TNtWhFyVR5cL!?uXyp*zl=X?-NA`Yi|D6cmun$;kd_6T~bhB(;p?bmMoS_kB z7Eqw%8XcW=_}@n8{|8J|2>dQVTAJQvX`ASKNo)q41oOBTF5J-mA>1 zy--I7SMb!p%?D3n*ENEN05h`3K%13Y>ghteHah zTT&5?JA3!I-1!r*h{-FHwak7arQyR_(`NttxHPHttnT@PJ{6L(#sRX9xDLjNF#(oN zCyODYa87GiIcTV107b^hn&p&aAlc_3Ps{<8djSzIFiChplv8+^JtYYBNB9S+j$)~Fm;0nm9{&D48lIS@=JQl!a6j)E(g6JF7vj~;=Wg4P0$Rj4D z+n7b2Lu=LQSGM@xYrd9L*ygNSsPG4bfeQ;Qo5jSBd0(wsc z>8~Dy4^l#~kF2yUQ{R7EzPSS zQT_k#12=|zyZYIw7a-6bYl$vQvj%paCpNFI&S<6LwvH+4@^Uq`?Z{@)9>A<-=E@z| znv3lI>uXA5klU4ReAQk!jvo`ay^m0`4Ewj?deKZd%sR5)Zn`!oK*NeB@3N2kZra{X z4$2{Bqg)UeE&|LuPn7o(?B(CKYDXK(w4RW@qeObPH{c&{fJw^0Ush zOxmv|#>NVt6#RDx@^>!4EBv9gWzU54?8wq_iu$1@D{C`L!P|W+HRs(@gcyo9wMH() zQXmAB4FKYwVELGDin5hw^zmKYPe#+x0;dyZuwP+RJ#$>PYvlf1Mhj+0gj3hnF+d)| za1cIyNPu3Yy$ulc)DcVava>Gu5-OPskF7J@Pd#CXnK%#|5Y-h$$ZOO8a|pcN#EGc# z7wfli4v;Yazv$@$dCW3wmY`58%mGcg7!cP6IkRDbI1}(mwu11bBR0c_es#&v_bT-l z45oKQSv#q32ar+_ZpiR@TSZrx`3dRiyq)q=0Zs&tQ)*lI;Rig%Do5x2QAHS(W0GQ2 zlS9b?_o^kT8$!|=TszzolM`J+KaAhEjbGEowtMq}#>P)Hm3%Q}ZHu+YHZVK0fdCwO z*rc+PCz-de{@Ol4#8WfU|F`Dmgnv&*cl0-K;~M4NIQuF6;n?rG^1R7gf(99#7Dx4> z04B;)>>=lRbdY&xUu@DZqrdk4eew$iA4OSqQDVktBw8Jhu}SPyL$eb)Qp$NHGA%^@ zKX9`1`e87)LQwn60LZ*Jd&)VmmLALS#XqHxyr(Yf9EEl0HR8FwVo-f`pH++E3fj0$puzoQdnlUXdnEroyACe+p=@YuFZBCr z&7QrM>X+onx<4Y=?=t~M*BuYr;UgMfKmY25t@8l2Y~WW?z3oQF;N?o#VpzuqdCq5f ziY*jqEJM}@1U1E{Uq@Q>UOSHcOLyx|J1D7r>a5__cz#I1+OH4lQ1#X=sBn0hEP>NO zg#w1xAz#oZ=EM%YoqWhD@^H+t%^Nr>TSRI>UhQyNsx5sCw7`&BX7F-!up3E-600xB z1XB|m8vr+{Rcw_iYsB^759YSKZPGSxEox&l*nCxWbSHhl?d|2!@PB?Qc2Xvm6Vn3u^nzT9sQMBun_Fe zRc3TV-C9sMPK(SD8>$fh_Kzo2DF9a_G^@jHwv91EpX2PHVbWkW?-UKObC&txY>&uG z@?Z;xe9{F)Nx(roBKX_1hCXr{|K99leykW0ffHm^sV;Vmw6ZGFl>KTAM7Oq5;QP~ z6{WNQtOB|>^Q$YjS^w2%9U-@nxk)ljIot1(_ABeF{)6ha92(J zhKZ%_S8!>oBOfE#vE$)(`BDw2Br_l2J)A4c-P3^K_w^(~?zv^MZ70aXyw_tT!u7Z`9@W3_#ZI*Je@(n2ae999D25^WyOXl%{Flmiq1{G%NJBGD856m7*NoFDkv2p>M*t zHPiW_I-w9@CnQ;=WrmSB6Vxbp!9i}8tZ3w*Pe^VsYfPSZr{$pZr+;N41qHbr6f#|V zE`LN~c_OhmeN5{_&VXq{(3tH_PGVg;&U9lNQStN>Jv!NNsbh+(uDtWKX4JvueEqeE zXAY5Yu+kB5aIoR6OUgI>Ne*580nP~)x#1H=dcHdbl`H_}u zqoF?kf&A>K2KHI#2b4u>)6tB2X+zWazg%z(0Fvpo{llx@uLO$IY_hjnqi%l?m6=EN zFYV@EYWNp8O1|Iyj-!k?3=Gd?tp)pP@*@{#Hq+Zn)U=mBM{R>kQ9fF_VbN%z=0OAWfPC)#Z^F}{^RIO-{Td94LosC46GwH=%3Hv#Lx|)-k z)Qm)cKR@)FXT#*KW-yT##JX!+we(4g{&)EkaU&`@SJ zr`@d9T#9DubJ%?DsM!i<2?RQE3ulQLIKK*l(B{mx$(4{tnYGra1`!|!Ik}PSf3zmZ zmF(Q0rr%P*=-)VrF#H(+g}kDFUm?m*3UB%P`xj^Q{2xaJoI+9C=7f53?3w5lFWC z%9X%D2Pi|vJ4QvzO9EL%!24S&k=w^`o9B4x@h~a43;El;IA!y^r3Ea^Tf=G(XKn2= z#-Sn4u+H1t6Y4FG_wQaw{|c5P>;r6mHdA`t7{cj4OON%W=%cHa12C(jL4_(Q*10AwsjRaQqEjFNYPheb4 zp%Cnl!WpD2OynLN%&^-k4%g&8@{$t(CyDnQl;Q+MNN+bXc$6%MOR5kox^p1gwtFEN@_fT&C*AK&#`Cu8aTsITibyO2L0UXy zAnM;B9>iV#5YpFQgOK`@p8=NQwD8=0xciWJ&)hZrm?Vd=Gfv=xHtpTB?1IroJT_wyH}9-TGN>F|0Q0_D4( zu|`TZ`v*jdfCxv=9ob#}if|1~3;=^~TV}_c6?I6C)yIEr$;ZOe+Csq$ZAwxE8P51T zso@2B^x#jmb8GGA+jWrz9uj%|Q8E+6w9vy5@6_`4l%)K2=RrbODYta-ab7S3!C_(l z|0JCWrmYU<@i$a+fYB_`CU1sH%lB#@vclSpfGo8oz zIL#(dVe+&~3oXhCecvX?NeZ zHY8AGl6#z~8rp8xE=v%FAE2=Yqhh`f*LZ>^@PZQ5o)J1k8Jvzvnk7zca|MLc7cnm{ zxorBpQ;sG0^bjc+JsYRbDH2a#GRt-92bf(}8mz?{-YmL&!QRJS-cR*md@;o5&yJS> zDV7f=(9*4=;KxTPy*U||a)ntxxJe6USYN-+UVl6ocBjEn5{82>U?u&gbA3b@()AT3 z!WaCL{fykgd%Lk`LqS7f>o?c0u3zu?-B#2quNL^%)GH0!$A?p4^QV;`tap`odJv~s zu%OE~A)D_OINLfd_6LFKoRN{5MR?j)K=YC~1etMBp5uqUsESQUxfg#vZ9p*Xr_s?; z!O+fzBuW=zmOL(+fZjmL#`uC}!n=<)lDbeqp#0La`huA77N?E<_zA)HHqkRU&p-7G z8n@o??~DHpBlyKh@KstFJfKEDOLZ0Lp8Y#5>I&HlU5_AH8~p3r09!vk!2f)z1%HfE zDtE))T4B@MAt8mt7T8Rfkla3XFs2k-fl=_nQ7C&q4GM6uwQm z@rwL1!mbN5Rw+4@<4pR!IKRWfg8yyWp3Cyey~Ga zg)ZFKC}sJ3ivXzLieb$HMUIMHOOU(ZmKFG=wZGnBmdUB0ZrK4;PIF zH3Cr)s(26u9j*$O%^~u1(L-$z+R0AN_v1-V+)UXgafb#brMg6A@kAOY;=ERVOSdHv zy`jQFpzIXwf|Qe{HAH!%xdtTJMoINAam!a%d!wDScx>i5v6NL%<6IwBwvG{TpBg_7 zpCr-iH{iHm2HgOQyQ4rHX)6{sq5)f;(wcKiz5;zzVWK)isKYc~D|-YCA%TIOT^%ON z8;P+tW?AFY@?-TOM-_iGv^V-&gG@GLzU0s4NPNWsk9#R2S25Efd&v@xwx{|YhCp&V zadDFXHcWh<*QHON32vGb_Q@aDTE>D^`Y>zlx;b9p=>UnFuPR1wb!uv39 zou=9?!92C&jakptr(N|@&GLulCcBGLKLY!M9Pzo)l}SWHap|S1{ldh;-swMeE%Nv> zc5f{~2qLD^V(lOA38q@)PdJfjab6^ZlJ6NRN>Yi(H!ngkRF!hlqgo`qTVG01xm*yx zAs{ywe;5Za>y@exN9RNl5DegVM_REympyi8^C(Ukh>P>ij%;?WvTSqSeRaQ$!(Y8} z%b#k(*lfe3(9|=r+1=X>q~SahsN=euZF6!9a!i2Rw)LAX!AR0~;km*!Z^^(`ZTe2b+J4 zud@CoKVgH_b6bp1%y9d5qnVtETm?f|hleE2MHCdLpAd}%+jX>Gjv{%HPoFGhzzI6D z0_;dj+OryvdDVqhm_4xb+VcxG?w0uYf+Gl!`$DAP2(XW~TIh0RWp5?F%Olc3tphCy z_3>d0v3|5jk`{b)&aLcXD3%-|aW7l~nc-6t(E>U{U86ZU0X*c#+F9Y#Pd(u7y7V!O zm1}iix=dyhhxO4`$}()R0AOBw7vg^atQ(8Me>T)pepQ@MvHvPW+{o#Vs3|&S2N!nt zqt=a!w`v8;&ruZ70VBby@gr`_vtXQYH+qA_a@*E+rbc-a7N@U|?h{^p)+(9G8@MP< z)1H)U@SBpJ+J(`%0PiZaRS6F-0`2oLr%f9yj$uHm%cXju@lDb+p4D79E7STrKv!9o zVzzK_uSFd|SzWOqMJO`qlA$q`;d!>ynr(AU*>pDmR!Vr-<05G(%gXhWO^_c)R}~p6 zdw@49&z-HZ)PC^gkzKlVVfjD^8oxrE*wtptm$@aKscY09(fJ!P1CG;Zaqb{r1cP$k zyXlA!BpyCKwTYH)LI><7RD4i5;BP7!DRaJt`b|)8saC_x)|3sJ+^~AAYi3Z?1Trp6 zh=kY5aVrLlZZH~!(1bgEw%a1{Tc8LWdQ`!%rkRtWga{}Yi3b&Y5&VXdstz?MB&OhP z9E1j09v|24*YnAYt z=y*-lAU{F`f6<4rDj_{wk**WBeTeA}S+HKc;5u8YYO;jtF2OMM$!jv2ysM&UVd%gU zgQIm{r6B>&w>eUbF+Hgk6a*amp`bNZgz5Clh~0na_-3i!cwEtmcN&OybaG5 zg&&x+S+#?yI-Fbf$Qjv66AHhXJU#fD0WLnrd}-qvQ4&s08~R}%k~-Uzx+uMu|4G?v z?vJB-(&sH@^iNSjhLSs+c6!zULFN6oN7K${)Z{U7uz|;-5|932n?o>k0)F*>=rn$= z79KT8I00T{s4iz;0Plxj}ob5I-6-!_>VUO%tT3S-wLO#^j z&xo?8hd2Kxcono5)NRz*x?31RQ0EvE+_)$&YH`l_I1+l9tLtUISC36(l*N_qRc2;VXh%~gge#3JJgwIIGbO)mELs2TN{`L(T z8WPhAoqX)!=zgA3UU)9a^xHIWAb7fyv+v#<>yBybQpBy~YkIH^qMT6NQ)gFRaabG4 zmJ8M8IdifCus~sII|2t8^3sF`YITM*vlkQ>#5q4!2*eB<2j&MFgE0{W?Crmd@mg1) zV4i9#RqagXbSvgoN>b%?U^z>~^m&o(_)*KKa^hEkWKb#K(7+?&+`*mW=Z7s4#IAN$SmK2FA zj+?F26M7{$Gh8$?7+?&^2FXOOM5zVY^v03lzK?s=I`wG--;%c&rC{RDvR6^7@q+4# zZ`Qj=W6N;Msa7Q3p3NQ>vKJ}R9!k=1ZVa(&K{1JTj6Vr8VdKqXU$LT8i|Aq--P)NfS==Z1nV2&JX<34mw}$b*j&W`Vv@Hf zy+7w}L){1QX$wRZ1&Q>Px_ppF;R=TaxsYU|dEK2`l83|>+uk)hQyFXXAzhE%`FPfF zCi~rmSdw4Mz3n5%c$Xs$x!1Sloo({JRUoxA!?G)AgyH z=%JW}>kQ@}cV;dAL4niLBu7dE4pqdxNrEaQt0Hc8xeHsq(a4=-#Xm(LOrF~J%Y{P1%}sjIjlytRz zXs(6Z%@_}m1!0#t6d3VNz8;#Y0EyBrm6PM@b?ReLgN|a_V5OOZXytNYg~*q?ltHpp z>kr|XEXMEB8My6<$!g;03PA34n0d`XlKUjbb3=~<9vMbpW6#pNw4c(_w$RV zM1giLZquJK$C-}qPe*&j+{9sgQ-5Fj-e)-<%bH#YZy z&uWr|#-CA@Nw%8Zr;9o@*8^E@^rSq)R#9erDEL17<7ChxJ~Xrt2L5_$)~T%@b7Uq~``Y}wquUHe5&ik%1A%n&>9_U~7gzA9bGCEXRi%Bpr8&|5m zY??WSCn)CuF2N!$d^OKk=#Ynv0}I0nl5d4N98&;2Da_W1kM{GYC!EF}1bkP{UX1dO z*5&Mht3)ybZ?Jo}S#1(vtoD-+gejkwWtFV_HsjhUM|Dpa3@S zVk+4q^P;I1bi%w!6A9Rb2aIObsV|Naa2jOdN+4m(Bo(AaF5BAqL1dcJEhLerBZ`KP z>s0|OwHhD$>oF9cFtj(>o|V3WF83QTBOT83TCYHd!-z*Gx#xT$iZ17R-=}&1K!2i{mbnurn)SVD6eZyx-$2pzN@c;MVRVFnwg4L zo{~epvtXq+f$hPz5>=?CoN8Lem*gOFMdF$Cr?3vfaGy7-O5NynKVOm!`SVr|yma%C z7~EU&g2UazMS-9o{x4DrA+wGdfq>6%PKbSNve8AyQfp7=VfNqYc4txD>{l*Mfj zlBHX>?>c&;FwhQH4P)o?rYB{-T#!i+y9n?eB!Xsyn@Qm>)LUM4OThGGj_{zi%9Bb9 zSfP9Yit8pPtJlICMC-O5(&tJY2eyhEG6RS_<1zXF4;3H}+ioi43!*mP2 zOgdMBvwhbQ^euk*1@Og7mYC6xnzhcqkEVn6e`x?CT1LSpJ@B8Yb%KyQ0d}xl(t|&b~VU- zJ^hCa)FB?!iKYRm{KXstMj$dmOSLLUWw*@XKO>&6kQ_E2fis=^R)=dUoXRFP(3b@t%hYs{_!>N7oOmCk^e+wq5tG|wpx0%1V z@!|+@Xj_hnr`sYDs<`ntFvg;rE5;?QPOz$~2FnLKvtoSaU%P@({a&7+nV_zrDB2`F zRT~&K>#x~mIJ*L1hGPl9Qw{{#+!k4INycY0wW6BNVqy}LZb&Zf-_m-$xA|6;<{+dW zWzUy}pl=Dnjd!}j&j%&i95&EtKbT$BeW$h2R?;Wz0Ja;^)w7OLUq0&I%6u%{B26YR zKuhfAp3bdyd9{t%eN4Fn9kmob29@Ebm4+s5-<`|Yc|WEsOVS|;I|8lg_@ zH`nV0CMoTj3IDNk6fl0=0R00FQuC#WvcdeSKOgSa$*-LwT!-|Syn~pgg@4;x9U_<) znj64LuVJff)tzT0ur8*c@`b<#8xl{Xz#TmSjD68&yRqA+%BH#4{(7Kx*fx4YWbCml zZokSAb{->toR%S9Gz9D8;H{I!D26;)pP><$eTwZDm8nhgtczyV()=%KIIH4~+E zGKy$u?Tz%EiRoS6P!mze{W$|VFl0PIGx7dMhuctpI@F)}%W66B;u0m{t2A7grf(b zH40N5DM?DR1yu=)IVz16-ltDLBLa*M1?wpW&83o8xTt7)vPDCXszm15eyrM(hRiyHz!O2&Q+ zPUmSAu(hyPQC4@XHHAVF`F`7dFoUQ=(K;^+Pl`IURN2cy+%soQ(;o@+$<-z4&LG3sg6b)@GPJwu5XNj zO+d|1@%}gcxk&KV1Q7S~AzZR+mnQ%<}|-;h1CwK4+7thy2Y#3gf0Q2*h*rPgQ~*oMR)z@VpfEgIgYaiKzU z&_*TGrECBx*tbFM0+`^Qe;1}f9^C@n2*NCUa>qcIeuz|l#<*HNakOf!w7%DdDN zH=W@vo&542J{V}evL5>oM3^qXttr_HwRYsy=QJ)&`l%4!E#f2lc5lJU{=`JvUK6X7 z=(dVQB!*9Tk&Z2mfFyq%<@ragfkcnp@RT-lWl9a6Vdgt(3#%mEj3PEI3Gx2hA1HJlw@Xuv zB8Vlw*F7*9wpG*`V%Xu9j|DA8lgg8#4sw@!lRa#K&K-T_8ZNS~WPz%ec1_e7STA%s z#YGVX9Uv;vX4E*v3Ab#=pk}9>KXuxzS8*xEVu~+_77>N(`Zp_r9dUTe!uIkDBx7=h zT)-Kx3o0X6PDX}Ga+0Y8)?Xo|3QE@w!!alIr35W4V!dg8M*NA(DFWvnvLAJfe5TxI z-`_ySYbF6-F|%)Xt2ot^ceYX4>4^??vFU&KEj#^2%-`eb(3M zw3#XJ6iK=j(<P@ zI;yUx(#Uo8`@Yjhb+BBr$}EReZ=C2PISIgj_C*`NwX!#37cp$|-y3c=R8uc^E<@>Q z8MCuI-o1`|a(8k0$s)K^ojzx}{q=Rdl<2!_ZDyl^c4U=F`5r%I2?5WI ziiL#*lguGntjI?P=>iJ6Mk~!!>^>4@G;&fhm+MRld_08#GG5-eB~o8zeyoXPf)bEj za*dWJarkVSvS3>-7;nHscPMn}%iNs(>s#70ytT7JN2PT>7#t>xst$>f_Pm*sTD?EQ zU%{KyRtgrxrUK|Sg8)1d?1gAKp^$Gqx0R&^&z-G{`7aq*ni1uHsMU48xFn^tBR<=W+zZToI zwGX$l{C*BfiH5p@;|8-QW0N@(&_w$qIGig$-=_I9QT=_vlR(!YuyNA;p`v@V0xleT zLG7`)w#n^6c;oZ)u`CU&jiV}BRwn7wcm9``k955*6pbwwD+CGB*iC!cbwHW#C1)j{ z&kQyw&3IhWASFGWaIaN}(G=Gq@N;-nID7afI&a#_BE8oFd6=C|<^>j6+DgN#tIMG0 zMR3l+^#}2UrpCQ9L1}oa1YP&K4RgpE-TFECa_kY=`^r@|LSp-Hqe*wS5I#>s%c0}n_Ci#Kg`XD4wgm5Cpz1v*FdB^h|jtZMO3qddd>vZUj`YW)S z#OUiERb_Q=Lad8lrr%cMu6i6N2*H0hx{ifZnlO|UJr1!e5`;d){VeVOs2=24y;}b6 zGmfn=2XmRmh=Twmx_uM{jh6*3hR~(gttz|Q?U=D8A(6n0aq^M2tBT6s;$g{2+_lP{ zq@yxd@E6gVMi`k7GcCO`u z??PTtR68`FzxIWG=pb|9ri%8o=s)ZGHMEx+kxT2}1yZwcywn(@msyq-zS1O0-e)vc_YJn4amq{U#4PmFx-7vS>L1T^N86q}h*NDX2m!?=!I_$tsg-D0zH685S zskUu;2mB;KpKlr$3pX@Q<^10Gs_@+PiF$uqpV2yW(xOE!x;Jd(2r4vV%#|RV|6H*e z;mA@uCgM71w4%=$rH3X{u=8uA6%l2%O}slz4)erPQdu=a2B^mt!;x$55w;1pl{z)6 zce9sj{5iN?@e8j*Hc=<42|Qq@0r%fmf+PP91+0$MvT(^BR0etEB*5dINmA#m%5D-l zKJG*do)s%828}xs|Fck-D?&;scPBi->P3Dg%5jeGxNs?_s?BBFN(1)+d!c_Is6r=1 zB+XY9NY#oJM_1Oo2--Z_Jn!D{Hw4>&Eocn(`EeBq6>kgf0U@<=p5$u?y#cijaT)>~ z{&`blnybdsLwJLJQM`cK)T>hP8C2ZUQSDqO8U#h5a&1#(N;}C*z@iU^ovpP}f9F-m zGeM%j0Q(N;*4`~dx{k!8f#svE;hV(eVF(Lo_Yv`)w>csoL}biU1pl&4Ndj^B0`yf| z#fwj(qb_QpZVHjb#x)9lm3i2rNnQZ%JigOfYmA$w*#Q!W3+fOgGUS?-G{!A#Qj=3C z$9FyAG-}YzXqpkp-pcqA*u2QEXfmN0Z+exCWj~9pHJs1_N@Kh|C4W?Wl46`Az@{8C zK8Sf{KU?xi>qI>ec&IwbDu+ z#pyd3a1K%WmWhIhP8$Xk1SqbdMLxCCne@%NnjHU>@fp}idYWtCO52A89Tr4!M3BGW zpmi{P?*28T)yYUn2g8f!r(UX2eiWPN{7qw_&4$aCoPn;P)`%WntSjNQ#fiC5q7#^Z zKHK{RJvzybK=)}xInrR3`AnVIQkzf_G>0}=L=LMiYTWne)o%orkAap5tVPjq_|GM^ z{`a{r8XhfpgNr^AcUE*IrI{LgKD|CquvXRijtArA_qv6dUP(6^G~FWB*}D1$2XyP# znd5KqCi**}12prG7nU%c>CS~|K2QGrJA;iyCehp5q)p?lHnCZ-N`QP$rSSLL6-f&+ zjSInQG_CbZ*e<|>zO<#~iTQ@YA%4)-SX4t}o|w18bu|D()OVhA(9c3Zn6XYOR*-mA zN5;v!{EsA60y({GSKZkvtPio=0IHm{2nn6^twVjNFbVz~AA&}?9g`x5|Sv3P^W%HzF-D zAkxyIQlp>Fvl7f`0 z&Yhw!!%J+AHE!4m3YjR z0tE|BHUiqXFalXDD<{_Uw5o|QK)bpG<5IBuqlk{!gXdu1OPPdz?>fxqnVnC7cnNE} z&?V*jSJU&S0$PJpH0XIAz{J7gM@B#zUHz(xtKPN{b8E}A3)+&YD1YvL)TWH>WE?Rv zEAU6GAUOl`?TCb@iQ|OerzGpez3rfno(~+I_q}H#v9B1Jp?ilY|PonRQE17UCgYca83`dc=tneD;0hE?(btZr2fog8w zy{1T?XjUh9(C3E`^fx=1FFPclC{MDrRD>h-3z91M=okHaHAx!Cf#}Q>lnmw-Jk{WZ z!Awj?R*!bh8#%THtCNal5v83Ps;1f2QC2A4*8LEE^+GMPn#slCRMwC#iod_Y_+`MU zICectnm+9VSN1v@kT!fp)9iT9hvQ-VD+sCG!56M{G65Sv6~pf#Ud2=$Zz=<;FR5X@ z*KXPKb;-xoc@|dJ)F<*dLJ81)w2Ew|GLH~Uu@N6hH$xg7x1H#t(AGRMa7GCnO&`AC z&Op_+u`?B*i4AN zJJ$+U#&N@CKS_hv_plr&IT#n!;)O=1wsC`MpcC};U#C${&2;*Kmi~2jj8Rss&p%}3 zR=oYH8u?f*XT7yx08K=9?eT*imAOY84CnNYcNboo*lw|{MYnKr#0|Ay#7>Mv!;pDg z%Ezv>YNM(HB0J^LP|8eAtnhBQl)xO)h#_atn8^7ugV~!+MhJ@gTK3$&&^*`12rCM?LB=Tf;p#Y246P<`qgk%y!SF=`JwwF!9hT|44wc1U zKFW}LW5em#Nd)xH{!y(+SI?z6%0)Q$HIa>fJ!&!b&SIv( zS<8D&;_&TIuLfL~%-tWQo3Y~NTaq14haz|cRZU8q0u!tzUh7fu@!dQsy}*WG!~UFJ30-dAY#pp`wxURVuTAw&#CDi*?_;gYoCIU zpp9zON`*O|!PL_nydU_1k1~GhH4#v6T`VS4-BbG$rKxGKjcJc2blzQRM;-j~YNkZ? zwsZTWh$OWH=Ww-Ufjw#Yy5*j+Yq6xV;@u;iH<_c*EK!Yjon@2H5iQcKhW13YW%;G< zW}+qY<-k%mT%Yc_e)u;M7VqDvsG?-BhK%>Hp5Yr#Z8i z3^RnX-c8s!B?lfF%_JKm39--V)I}F`2%eI?4k5L%;ZA3xv7JUbVrhWok#AVAxgiW6 z4ObS_0!&z~6$!t<{t%KvPSbu~;et(VYg|gfGuCLnFB+|b2HOG&-6&V|&{eY{1%-h- z@%9|6cBc|W4?b7;fkVSqT!F=V%&xHfXf$3f>1=2REUH^Hn3PS#_~GBqY*_Lg+5T^} zKW-Q3N6CC4Jr^^K>|QswSbj^QLQmEcd&gZu8TkIEb5tA1IZ2GYO-BESuXnA1eN*BJ z`|+)^8K>%L`(2~>#fv9ITnN?ezJ>UIjEWeKCLo}&n7AVgOR=NW0+O1;(d}+(rp+mg zzGrZ1*klZkZ*B*5T@$=&T-F)D>UgAk(c3)qMy>NT^qe~Nk zS;yecq=w&14J+)mK>UaOa4BSCBN&MaQ{zpACxSCIQnL&*H5WrgDxuaMtMMnBH@9>| zr^Dq~>~LPTscSPTqBCB_(N*5FVsJ!`y%@1Ez}%}f7A4z@DM^YrPYRIa9|a+wti1iWyD63UxvZz< zrV+|$6dVR>t(Cq(R^EF2Vzv;5GuODaxBmt~v@fA#?PpOG&0FxBY)q9kr5rh$D*G(YjO?f$XWWI^dDogFx%wk?zI{9z!>h=Rv0 z)_r>r;rmL-9n0Uis&(90I{Q0m^stu}&wj)dv~L^FFspi-u(8h8%t^dra#@( zEuwo42++D;e6gE9JVR-t>9NR-=Iou)hn|M;Lfg?q-|3{{+&dis17|JQR7d*|EW?U@ zEPp9{_otS`+JvzBOce)RC587e50cRw-`PV9KO816Se8n-IA3oV`v=U#gC5Jr_LIt?V3@Xse^hlB|W+vdht84Donzqq9wHQuZyY^{PfHHNEEaJ2| zp+QMoize3Sv%B#x4pjvKGC8eP#As$=P45ffpVI`qCswm` zNyQEair_yVD*Ze&;nP>^l5y`NF)g-5g9i-+Z02S%v(vAym>WAZV2uKm4gH01lNi-2 ztq-=|C8|Y+h{IZ&?W{5(+^xlsyKXfF{=ur1SLaCnua$K1W-w5TLqTkD zc;9Xc+8!v0&aWc4`DMNuofUj7Pvvq)Tx(f~5^>~T`! zd-F8q_myR$VW=a)w=^NZc;iD_yDp>TOl<=3M}9vQ*|neWA5c&)lX8@wgWYToE#)IL z90>wh2a2aST&AW=bfsV}<-`QSlS(+x#E@}A6uqGot*kE|5)9<90T=7Y6#FxC{>cgM z$t$JQNUNYqyQx=P%niRN3=s-ZDKOU&en(#@RMtSI!F2;?b33HKk>mhjsI}y2aX8dXrkT=_^+2&K#cqHz^bxil9YtNvPa^y0Cg1SHz}sI!bYgIoZM2p$+xu>i z2Z$eY%C^LMJ1+wOOvL&Qqp@2foG>ZKr0qY;!1k8EQa<`-VUFut;oK6wA*>sOZ_{m? zR+w<3A()IJ4tp#N5d?2XjD$yrdza?@&VwYK!BHMV4Q&}}enl@PWEUwoL_Fm{T9syQ}6mhG7d2A=JATal-1j}CW#$`}R8ut)t%dU}G& z3bWW6!_k57lO<8`+b|t8Ssc-ER&uaX{zDxbA%TQ&=K;ZZilTL|@i1<1y!Us~8z(}U z3+-OrB!>^ss!>Y{$Suk6iUv~W$3DvWN28OrCpg;^QK9ceLRpNZMT6nrBN=wpwi-i= zRm~pKaDYBBd^C?+jgw>Ln>hw4V={Xu%LaV1RT@AVyVEpntfw-AgQ&IPq$)%5ch$IY zq&O5XK9D&))U6*aU<(?s+e$t+q{Lt*IYT?Zf5e?*o8EWK_+o-{x#qh;(YCjI=x8)x z?S5ZiNxVcMgOZyTiQHe1T4^!)>{jCqvRX7fn^B7m*@0xgo@nS*QI%SrLYzzGg?#AG z%_1P;axqsN|M-&MNOGtMhO%UA-<}&ZRNh(aJ$eoq8BkA0{pQ8hRyN;=g;g{&RDN_s z6CjWl&am4p6GX_BA=wo>W&y^8vTyWNwfyq}sJL2Ul|AHzf}SPH>s~l;H4uB3Znxgo z_aT211q1Mi10v@URM?$Kvf`07YUPGsQ2|vHYSsC{)@Kii2$DVYB-c(fUXq4Dm*O<~ zcT_N8*u7@ybZ?OWuYiXBa3Y&0f{x`vg+M827sfO zM#~K-_o`|Zsps_ma$|Y7*mtjeza^pn0WC@GlUwb%m=BptzFxoorqK~^E2yP{hRT(N zv*}?-{c_U@VSlFgtzINN`?{)1E^y_KL!h`qJ3mkJnG5+%1t-=6tmYytr)sM0#;}Mm zjACx_HBE8P3!5Kgs2`UUIzZL#m5XA^-V?RuW4Om%m$>M}eDycQu<-LFmP*=FKNNXj zg`8va9Y?q&v|bpxGYwnB8B_2M1F0j7JErYn-VBe*Ua*k97LdxP$8RlHYQO~Qpi#IJ zGwSv9$RQ~i&*%h6$flFW?dZzU-n41nbC<5AmfWLCmr}EFi5G29BAj2?>e$(oY%r{io=}aN^tTv^>4|b4`81NKfzjYD2I7k{% z$-bBWDrqwojOH_u+sh&xH~`+9vUk-S@)`VUq2Imor7CZI5XQPIkWJi>6(Cpj_Qz&6 zd;m!h#BV#IHU!alD4t`BP*C4M&vZtAD<%E}dm^RsuoKJ*aVzqWSlMcB#fn|3v=#9F z3Z(fW{7pGWEA4zr9HX}jv!xH?BoRxvcUD+vfczTkoE)w)DqrP{fd;@%|9yiytJqkE zk0h1jMg9igLEbIAC^7?@>fMi%Qi6N+(wh4+uBeURWHz04f^TyFtQsrC(JO~FZ#y$#Hd`wsFqP=h?&(C74D?9Zwu99-XXG-AM zlx$DHyR`aUcbA1!X9AqSu9ZM|x#Yzt?;=EYm1IPq&d@}J)Q7mWwDgfrSd9(*TRQK_}PsCH9_AoMNy77 z?|udp6BB$#;9sxhYFFGWEMrQ`bsQiB6X@$j9nwSDGQ^bM0YTDDhQ-=KXe$it8Nxck zQk38naSYC)J2tsZw}R(a(vV=AIe_t|Ct;U*4?p7}ZuA`_a6kvV5ur!(aY(kdn7KzZ zk=|h$_in&kWlbAYTN;_@Rd!wIXE|+WqS`p#+Ffx>+UWbl>6dAQ)emKYsM%)z7 zy^w2vrqdsS2!$**&F+L-wLGY^%~1w4?p59SF;kDvAiM*Z;YH8tyb4GhQ2OIGHMy7{ zlHmF=FJ#9=E%j!F zU9Fyhy=cqwc-KBJg&)Gz$*m;Y*RAeBa0lA8-?+NAQs$1=opq5MW2^hzOLT*czG8_-W^lZ zVqG!z{caalARPQov2qrK^zlqE&lQj#y{XatB-QtF=4!oG71KIp?8MI2miqRz6YO?vB9@}uDu?0zY{c*FDbHh7&L~- zg>m0SYO19>`8RqNJC7^NouTP;ea^2%r}B_yks=^OC;&R}gEq!??`GYTK9@DX1}vH7Y<8ne|gam?+F1+4*R|B0j?x45ulv%NF6 zj3LWDW0OUzz86aho@nEH;|kiFE_6+v&6TS#T|^ozPj@#aJn1QMyL6j2U5AK=57 zlb}SO0`0j`zM`TP!B7Lu-^0P9Sxbf{!iO=dW(xdX0#t1jMjN~>d zQPwcaA>GIigYR@nqh$48X&f+MB~5DG*D^)$(Br=TmK*~{_{ljH)PqgmIpQ;?y(*Y< zCJvde@cBIL@og}daOwm7KbR%LMXq6p4*#pg=;JY~`|77sae2XLS!w9a$}8>b65bP2nH;hPzNXt#3KVuI~Elc0VZY5zY5>k)qADx>ASf4 zca$5^Q~0OuN;1z9eWZxn7Ao;fY!Hgm;|vN3j&8am2W?5d$zbH@`Yy@5U}!z{Gv;^ECvml}+yQ2N1@ zWoqSMs)_`aXqpzHJJ{@>pbJ!cIPuusxn*9Tk_?~X9VO0{-k4nza@zSojFSSqFBbI9 zh?4J~VJNnQ&c(WWmo0s7T)swyEC2nf%#yCxZS+>E)D!k*b+EJ&4x{u1xlSYE%!p+E! ztISnJeN_7Xg4cKB;~Dsi{L$HiLh#E}pE``yz-8$uNo2x27KTfC7|z~QeELs{IU*FWK!C^UNC~MrK?`3;8R73U)oYYH?ra~{ndiy<;nU9ou^Y4+ zlTvoDMytXc?ztEg3jKy|dgeEUzZvoGU(G1%F`>`lV*8q;&62%Y(q}n0pjG=O2Wd3e z8)4~bGDGs{I2N)=!fD&?N^%s^p*6}8Y33tc7CXzTj~`u4jcGj|0Nh8fr%kZZeDcPV zl{beOhN)EzHV+#h2N03w#-MN})sR!IpORs2&jJ^f{n?K;)mG6JyWe8#05zQDHC2Kz zH|PYVTq*JcYLL7ESui}L&|)wO&*kih>Z38+35kKpUeY4RgeVqlE?&6R1|E?0F=ANImzFGf%_U>;58C+fb_sT`O@>Qn9V|(<$+C~XL#y+XwM() ziCa{Ner_uifBP^#Uf~5e;AUhk2cgN2u5*e>;ugKl8nLeX#NFB6Zos<8t~R=iCgJgD z!krn z7ptip{2~!vQh-AtAD_p7%e}3WyKKcAP#=S)9E z$gDKY7+4=V&OD!AZeO5u1HLl$V;Op}_QT80Ju98ZWwg}0-mcjtTX3`#yH_RDf zJ^}<-Wvi}F3N~T{oV3i8Q!T#bVO|MwJx%9{VwZIX0q5iX=cpAf(1tjz_ys7c0@Dq(rv*_O= zHPe6$!@3QdG+PXjQcjehOLk%+VgAI^=cvyDgfqPyyAeu?(QZ|^p)F9`6YT9jbEMM; z=4H-Kr1F>(g|)%BxMX5EO>&s(^JT>DlHzF_C?Mz4%=vw(sn^lZY~uuV_Ol(%S52Gl zudsCd3{R5&n{o7`E@dd2%Ur-_v-tu_Uu60|alqX|B}mT@u7ZvXxtNo@O8a_| z)hCyXdM~avZ?0-yyQMoSVOQa1c(gc|2Ki|bzC>ARUBG~3qpfD_ibtsLS!?rrBuTQ1 zQX`%_UcDtZdH-_)DGG<7w2li~FN{^2mh2^*2quVU_bcI`^Sd?$28-yiO>!uF)1(C2 zbGYiKxDaCDiJ70pE_-c-<`FFC%|theworzAb=BsW-4d~_+3A^sTZ?cMn-k8#wigG* z5pq7jSH#xcFKNFJ^P?nAkM`Rdp>RH~R-{lrpn}rrI-Q{+9p!!&D(G+*a)JuNUO&9m ze1yJ>I`L?O#2d##k~+?~VvZ{3=0PafUC`LIE`EtiSG0GjDl-8r`;c& ztLCSLPZbnOU>?=ks5;j&M1GG&j{$0$QAu6Y3DoA~^KpqsnD$Dp<5U~tk5RAM1JQ>x zLf6^wf#eKU-}E5~MKUMnS(gAsPc)c6wEL zR7ehk4Xr|pWfS(7pOn~(bkgzUW;reMJo%}rO|HEnKT3Yu8;fqJbrvN(P6M+vC~lF< zb5UBpA}^kcjUlT-$sH_b>y}DOd$p@3UunHcu$ti*j3_BJ_0#(y)5;qeuMa5qkbj9HuI7F)nJiiV8hA zR7{!P_1*nZo=k*-HjwvhL-gqlx} z+1lEw6M)R*g~i8^q)NO{`_5`j3hBD^8Yz(C0YpxFEA7eTmQB>71PF{_RUp+&LmCt2 zwTgBPDvfVghZ_#Y$T2?`vuRHZ9>4C!o`tsR83L|WLAjS$G7XCN0_xZP63mTo#xUi# zIBo0l`7!OwsoGLz1A^OBlc3&~bTE z%WhD#*cwd<8$Xe+;xk;jueK#@%54S~8(uj%tbWf(nqQ%WokaX;eS1s0zroXi1n1*o z#mTiq^?@8>xE)*XPBQStBCk;j!gIuF@rq(UOKCtxPG7YBhdTS@DZ7meZHK2eVWq@YyR%y|P)Z?RNY4?n9xp$Sxu#OXMm9FkEq$9?=Fi+(aq zqNa9tVcJa6u+5jE9$qFJcjm&ku6N}ia=-284P(WqtVIjoY=>hFRwk{C$+w4E6zocJ z!CIc!`dB4uYt}bd&eE+63uDSTawzrC%-v2-8X4n{u}>B!-1u=oh{E6)Bm;wZmzvgg zV&3BG(ogx;d@9!Jy@8uKnv~AKb6WiyE4n*aRefbweI=VO3d=Eh)XKF*fnJimOxAcR zaJNy4lL0@*I!5lW^;KpT#;f;(u59rJbi0^()iEzKv14#ynbK6LIYb_h?8eUC>0mR& zW%IiFBhcJQq$^JYJ}fYP&Hq-27E--gS7|p??M0?u(0p{y@!sj-j>Ooeo$5-lRB-8i zuD-S81lDa6R7B;6U#{~qjkR-_ZSME@s0=;+x9{9l5NWuOJ;SfbKZn>k-k_sp5>23o!w^MObTVG@AS>9YJ6K0c#@^Yxt4=Vd43bLwT zj^`#M<30_(49dw7fzkx)*d}sAXZfL2P4_GHSAn7_9#jkJ61zI9rx!_RlYE?${aX!; zv>q6L)aqwqu|D@UySta6)l9>prmg?T25uT7TkPrArTIv-&*16qk7!;Yl9ids4Z~*8 zt9sb_+kT?;NO!obPe`KlwC5sXad9=QpX&E2oR{eqMvdmd2-CtH&-%W$5;_W=1jl+d zvIOztXcWN^%j1P3hQANrj@Y?OEb@(1Dv?s5)#g}GET4rnM#Y3B3=Irdo{hSV1dj2u z@pN4m&|}vIY%G7+VjnUdwpT~rJP+2qeeRO{DpmqX>YfHl84LeTzxym9&3y%qK^}+M zi>=vQ1@0iE`Dhn&@hxXI*DX;GPCd?f+8IoFkVzQ>)3^5gz>3_faN;x4c(i=*lu-AP z@BBvg!y)&xT>;`{;!H70-%N5|S0Gu8N}(7T)^Ib3HY&*KwxL^1DCBU&I?4J{Q^t58b$J{3aMLp$8uS(Ie z`o5)ORmI;(y%GGrq)p>~84+dhSe@u=%A+M|&{eLZZ8FuXxac|t>hJn#m&7dl)orf9 z*|E~L94enH;n`2T=h2C8V&NKCd-l5*ZQ|=nYKx#mh5E}+s_|0Bw4un-&)VA?c+{?o zP8BBacp4DXz*Dka?OrP`K*#-aP~BGTKdSK@Zdhu$k4a1$4V5|@XA=WsJM;?zsHiL& zW_vREBk^w9-<}a6UZ!Fz$-;4jILl|53p}c1zWhYjVr8VszpZ?%In0voygJr|GWh6X zWit6Bh&-waH>{Q~q{WXrHK}z{{qL7vV1u`shWG(lQk-M{4viLhNDt=(YgTIs3CB26 zt0?I6<~x}Fga{h+=l-C%G!8NFah{w?SCdh&B5&Q)(r-kbLn zbdP-c+mQh??Dy~A&^8380 zV&2Gkjif|9`qm>7BI(?pf~iMNj_v*gWm3bzSC_V#^dH%ce;kSjUR-QVvoyrwt2hMT zbe=-3mIW;rRQaN5pNsMz_7UL(VLYULg+oYJl0M?4VXbS% z%lz@W8aboP;ai@{{f|B_&`u(FG*<6@!cE{0z^NIbM~NKOO!dy~f->%9U5gjGT@Pl3 zmpJ#T+g~sBAE?u(K4QKjLmz?*aZ69M3yWWDt(M8ru!!XASiTVLGd4CRan{4^gd&2{ zKQ)WI>bHSEm^CS&NX5g1xgkfBWsTQS=Htp|l1p+4hK=HK&t&KT13@&QNujfzb|wyn z{oEw7(Nts=p3>hdL%F6Sw3%t^P0J9ram!ji37AY_IQu;=hFyJMNu1;IocEzx^?2I2 z^UdLzlg2A$PSdqoh^p1Le>dvWVwp;WsgI>VK)^iI(`}V4z!q6(-s*de70m5~Z%n_;7U=SIaV*SN$f~bw%@t+Y?$8@_SSv1hbTH?)>WEJ;@1yGfusX76?^N! zKm1my0$qd|v!8QugB%jUnRT7BhW*zg-OO%DAqD^(XgXi=`xoJy(qr}0`44N*&ZAEQ z{!0q(r-6Q>-03vG2jWF!-b*-b)G*;Jmky7u--j6Z8*Y9mnL$-m`iGr@@lr2oU?%9D z^Dy#H?iQnpZ03-7zedl+Cry2R2}Q1xsU;A?OzLSTeFob(<}Ij9z27v8EyX%Lxm}dZ zG@#+{3oU|!Mks5;?}dM`>&cfO z-!5dpgAK~#D>?VD1^z8+hd3#&q3Ot0aew85t5XBQX?a{v#fRQnDWa20cXVww^tWx) zHdJG3p6BsW2((m==ng@sZYq?{l%pYb-!me1dA*y7YczAOAAXtT$tIjyyq)D7jx|A4 z6U&apR&s+of=rJj)Dm9oDllUfEA68Y*Jn%#eINW_2ZUs=mu;=z%1Q~;UPEZ8O%nbp z-$G-ZQyQws4D$VD+MAqN|CDbjD@c=Fwap}!R)5i35N&b_N`;yy(EOw~Jo#W1@^ejGxwApn&rsD6#X7Jq!W-88>rY2C66y6~5p%C;*UsKSDkUm3AwYv~5HW9Z~&{6n@s-zE9DUUUY(>(Fdi;VmT`d&|_ zk0C+KmhWnu&~#C5<1%x^94ScuTI5hlxMEeu8({o6bJb5lPMr5{VZi^NZyv|FiPD10lf09 z|6BhBgs}TLRA>M1kO#o=sFc_!)y1K}x8EOSHRwtStvXbrA5SAv1EV5gP(lb9d)u1N(F&+=4YO% zA>hv-{iID^sk_ytju-JR_x8?7!ZH8DzmFDRhP@8JK4%Mse{8b)?X=+| zLFYI0bnWT)V-=t*(pZJ`y+3`~XsI~6Fsve93B12I zLhl=sg5EbKm9h6n@xS;PR_I~9M(az=%y)_iP=5D+bA=0o>fAN?4(B?3T@TQ1Jzp`c^we<639YzY0O<8rqhYk_i5N@tVuWK+##H zO!t2_0xGM{4yxC5WPkM{5jwI@J)@;w9POe|56QHdP&<$}4GT{`4KQ>s`(sttq2E&! zRflQK5e$xyF4^MmKjx4F|McviK0*LzSh7HH*2(r~=`z0#O+Weci(@Qh*5f~sVlM)? zVkwe}MEvdwf2*OZ;{QD&^lsy)yN$Ox$*A3tuFN4QM^%vLVAZGZ4MT#Hvq=0Z*4}&W z=J)$^upTt6@z<&IPy(>mL&(8s%w>0sIodtR*RoVVVih1k za!D}+Z7c%l6=rUFV@rgkoQgH$bhXHy_|Rhc1{C_*P2{6S3!2(Pnj=9xODGu`))y`k zPv%dq8$vC=o=YcPrOvMEQR z?$WqAene0gV*wXLm5j?(_}WMCJMFW*SXcBCbb~9Vaif10jRz*|wc?{Vio`iWwJf)q zH0NIS-zxxg(L_>%Q7R&(ztq0zbJDBHjQZ+`z0jfn;20O{M<{EZ{i&U9#?6rWOrDQ^ zfV@sQ`6Sn&EF~3R(1C*x!C9wg^4dEi=x2Q+R16k6-%oZgCmIG%S%zduZFlqte&*5E z{A9`Q|E|VMnP5ZM-CX3GU4S1=a>t6~wV$zDjB@ky%{HpXn2a$Gm^PTEOI-eVIwfv6 zRZB6f)Ymsr9fjw421)XQl&WtVU-|xfHJli!57vrBFp>dUnQJp#_v~S6?t1?oBVLLK zwq0}gGiq3SNboPPxsX>}zN_eueJ0yfSwf?y*8pn5rKURl%^?Y1HD4)v+WImZ(skv8 z{_Gm5m()^yY31~8dWuo(GZUU~E{Yn)|6Y1}OF9tryH()ckeiQi`~u|&Tq301exw}m z4F^Dbaw}5@FgnCEC;oE&KuL^3tN81a3wUb@t9sWrO@k@FG>?oR4FhcuUjRH07C@J_ zAj=-VXpT}F`6eV$Kla;RQnZ+iQ~K&xrBH$gPiHA}BgF}pFk%m*pRYT2_7RiQcS4Ul z9J09T1a3&s5ek>u_-M*;EMx_&OUZos07_g0=(xE-Ef0Aw)Bfjzjj)I|daz^dY8tVX9kKF8(8X@N47$2A< z1W`+c9cb{ULX|9j(U7#qKq*XULYWxrr9@(|C>gTQx*E5gp=k3ImOZ_XQXt%x^~h!I zJCXx#_xTQF+^q4{!ZnA3;ipOdvl{5gV0pjszlwsgm#z)wGHB0i`4~yQ0aWoBm+tJG z!HyR|?j(Eee&*+4zFt@DqZxf^)0}ca*&hc8dlI#0K|XhWaJsgs?8_@1KcE8NP&Dl9 zK`()C>gJ2G7w^oJon(TOd??rI}-(&G2Tm9+8HyXEmI*k=5@U^`9hWG zB{hkFdKPGBvcG8LhDK9xr<%5$dOD6I42$=41egkx{GZQ;*1m*2)xLyT__*u2+3#=D zG_tDNuPjd)OT3xOY!2r4Hy^rn;^{?a?~*@bRziWwyS$QW#9^*dIvCnqh*z^r{o-FU zRh1zoNV}>No5r)#0TCT|giC92kZg7}_2#q*iDeU`fSDKQLmZe6%OMJDl>(?Zf#|?u zd{croqrJm(4Ki^t<6-W93)D;YkKfZCC2p2V=z)o_-GwIZ!TdV{1P7e|UMS$ncAbfs zyY_l(U)-s)a>>NiX^9_NI$G&`u0fA>)+!|#ss(rRj-so*QklON9re1Q0EtpAI?M|o zCBvtx0OeQL3iwp2E9VExHn+6u<9wX8vahYd%VBn*4&egGP+M0H5BlCy_Q0z;CoHucTt6+2M0<{eR`zb0}fmV?}F;4X#?JqSQl@ryK>BO2+7sq7#3Y zesU&~(Df(fi7@x74@f<+{nq$wuW5Ro`Prt`m+t^dh_5Vq5kFsuwqg#7gZBpHx}&t6 z>eJd6(pUQ<$ZGq^cv@KI#>T4fnA7_fcv(D7G(IjVp-_>jn!~+%7W~-6adsX?Bz;O? zz&u@D5VT-$z4>!3vyXPO@NcAWjb)$hbtatI>>Bm4M&^6oiyZGl8VJ?j*@Kq40eo2a3U>dd#+RvD@I;2v&H#yvhmmXy-gobsNwgW0kj!4Lm7e=-8ng1FCuN2aA( z>`5M7{BWpcAqPh=?F~f{fUx5j>_!L(n-Y2LHBe9N^{ z%uVz$TN80WpL=Zhp_XA^B;Z4*H?lZv>EWROCWiI+mbXD9Q<%VxJv*-bT)n=?>r81| zo6Yv0al(fqp%f1QKJdK!z@@z6K=qLBx$~lMxqLhp_#W~XpCMz-P{o8OOlq!sZb~N-_pKHQrrHaS_(b)Q*>5UP=7-t{9+%B#J;KKncM- z%2xYKAjEP#Pw=22TBoXN756{T8w$gk&U47tA4?4_Z~NLUfc65^zJm)#0!AH>40mBI z8bCTk;DD{$YRynrb5k=MGGfvX+1L5?%~%p_F4(W+A@J!7aoD6R)#IGM&MqT)TY>H9 z`~6wVuvGN5<9H}iyBr1vwv?MMK!v+CitR{j5T zyRC?zo*xb3nywoH`RS52Z{HQnp?^J4kkc1=M1cyBy@-YgeA1$gY3}1!SH2*RUpKQR zay2)YKh>taw{AVlh`6&rB#wjn3@KTv%Q=4lzPg`Xyg!Q2q-$Vh%osP{k2pD0`G zI+sHDA=N9fQFU;oefd?ud1v)zz%cv2T$tBODCSM~w<}`vscp3^X!z(7g%j&-I~!%Z zu7Rx)={Z8#8ZdV3%c=HK(cm-e55Hs_{fJguwT*QjUYp)URSe6UQoz&~Q5f1CP(4cO zptB3W8_b4q{UY6CAU|&jx+!lUpELZohM^KSqk>6e4S{FX(Z>i!+|0I8E;xNhW|RAb zM9d1GJOZ$*UjF!BaFLAn^kjI3qCMJcD#~kLmBzCkS;T`#En9S3kWI`=nY7G(Zv|@g zKLEu}Zp?l-SHxWBA1d8`hrSZSxBNP+AW@_;(Eiqt9I1~) zjIf6wh!F}qe=Q(cNYi}N+0m%P68t3vak)}H_ijTaVW5=nr~U7c$w#i(bzJM&UjJK( zw9N8rRlILb1xY{6S{|7^ey;*I)o_XbJT$Z=WDE@ZZVnG7tZ7Mk;*qtSJ!zhrBsJux z+h_yFdkjg%HJmr&&hQk75~x8ipBc1qf?XBUHxH1^F(vTCbWfD7{5qz)rZ@%xWlx#d1Wfr%2GI) zpM^c@1;Cw*^z!4kM~sjRQptNpjv95e#`iBZ8C`B1ZP(H={}12LfzFf2jSK~^M}65R z0toa=Tg4@1G_TxHkURtq7NG@#e@|Hc?4lUxwiUq&%BlpUzpBs|w3qLMq@-^VNNZ0e zdnfi)6|#Aoxu&deRmu{y0ikmF^>{sfCvp$Dx^tV3BOCX3JXhzD%OZ}uu_8seApW^*U@fpRS zHVH+tP+Vs225!o-YIG!#kyYxyrMH!2DvBxx~qD#Xjts8 z6+#N`J*B6Z4U7u=bzL9oi}z(n1UK3J})N+)R%WTz7!-N(+=B#~Lt2V^J)}~+AObHLCY12dM3avIotF|a@ z<0s@_XjrA5`eY01&dKk*oHOS#e{;UyoH>{8f0<`{x@%9S6|e0dzm91F1PBl~ zu|TVq=Wx;NZw!q+66ez>{5y94dTfidKRVy;?|FD|a;@xRtC%7{fB*pk-=jdK5=!AX zxs<$B3&ud;Oca=S?5E>?8)u@4n5Q7ny=vRLQCsP{#(|+nV)H*t?$aZ3_))8|fBaRW zIkr0b{dglDe%9A}=Wb2Ee7;p{(LjIz0RsOMDBl|BYJBjIh2>cBK%;0bi9*o&3DZbX zKpOL(?9JSj&tU%abCP5l1S&BD*m`$ zzZR{Gf782q@Smqz&oTl82oNAJ69g)iP-=m$d-I+>QET`|ZQOXUd1v{eEGriHYt^=h zan%LZZ(sUEQIrGiIOFEDOuc;4xuPLxx6b>-sCw^u1QYLR_k0*BY#en}c_JQ|;We>8nN_q6sSrGs$++8^iZe@CL} z%b5N;mv1Wp9ewCBaV^G|0#BH=z9+rF50<$vi}Ik@|DwzVz+0t5&Um;7BTM%erchuUPQHp7HZd6R3i~=C|%S@9WX*_i=Oj>v*s~ z`n!&av94PEnqMsZ>)tBFDTbLT5PdB5jT|_1ca+HgJb%m5{O_u@G&7^HR|E(Um?nXg z-itEMjXg#8#hA0_5zDj+{f4g$=rayONXm+|I1ijt`ls;|eH4KS0_ECO-6Pqj*A``QV_6p0$3xZCkz~3e&dbLq zKPPPI1pxvCI#b|Czj0amFp}!KQ<-)}bGDsbKRElnboWb#J2N7Gdrg1<0RjY0E>PKo zQY+*Q+wcA9SUI*OYHnO`aso>TR7)WG3LB1+(BG|Cw0TXnA`?v&1fqFdy>amHMOoIo zxGeJvqR;L*QJZv5%2Qt?o9ahu+QmU!g;>QvfIwvhMkBIjoUexCY;~~sZ$EIKr&{f! z`nj_{mOQHL_bxDh0t5&Um?Tj7oAy_Bt(x`4e-1sCifnb1ivMs@{9Y2MjzG%3iXZfr zyt{Ycf%(sUQk}p=LVy4P0t5&UAV7cs0Rl5kpz;Z&Hh7ePr}aCQT_2@MKZ$C{kyuJS z?PbOUI#nQQcZ?Qgb|BC5_vg-=z2nx)pKf$&EcTlK0RjZKr7i(V1(%>M0e>05R0X~T X?Xus-Yic~l00000NkvXXu0mjf`~yGL delta 52575 zcmXtfgIlHD_jk6N$+l~<-K5F3ZM!?$I5kbSYqC9YvTaXxP4za<_x+te;6B&2*IFN~ zy-$vyB95SHy^Vp|0E-Uih$tQo{lpM4Ve!8{HvZXHeSBANc%M^qhY3mXFgd=U|Kh`Q z*jq6RgvLsF(Y((*uYP|p_Xx``x3NRkrd(RbL6O#bsy9E`m9lW#iUZ?d3cQls5utF5eQO!CfEC=@F zh$L0GL81c$EzpUxLPf%yMAr-5d0FE!gvo}_PbQM!`{$lsgHqxMbMV5l`W@Y}%oKxs zG&B`2{VLj=+su|x16^9d2w)SX*7K5u3N_ZARj@}gq?36rsc%gfBs;JNMhg0!U0^9i z&w7JzCSudQhy^o&A8#vKsKBpJu>Ghzt6JP_qNm64-OFC+Vt~tXn=$~9Vz-$<0 zn;XVY8t_|u#vTnxWiT|hvPGEnC z@NXb+)@Y`pcU+#wZX@at6xT`o#-WNBq-r%ry|667HSX>5$3}D*BWB(U*(G>-A5j}0 z!jU|9pUS=W%^`YXu%WP7Md&x7edl8XwoKNOQz5D$h6$J5jcD-!L|;W ziv=&qoRky^oo4&k#^^#FHRMt9G5l@=p~dp5R#mPBlU(Y`1hh+q%cb0vl)k*vNRsIW z+a=c5zJ@zp21*}Gv#Yi=#UxNV2TwwLiLlJ5sU}%0?2p7~1U@Lnu8cx{Yo1bo?iL`< zJvHWEO8MIx46D#D?@P>~>|`Qkf3B(*b?W#EmcYi+DRSm?gi!vqKD4oWu*nK`K3L@B zHM0MTEXXlKyR6~Lhu_nPg(~b|Uk_ZTo?b~zlNs;w}W$XO^7N3M<_X+sbslRaMuxm-8#{VV}pnh z4no&#aopB^EcFGO-ykf+LkiY5^JOn3tm1D$QNNSDy!}P?iT{LrfO_E(5_S|7UTA;3%4NUS!k4e7cK~&qAMwos0$bKDO*->e9sFk+-TYqZ%dW|0AiJuHlZ_3u5HWZD z2*=s8P`O)p>4!>;_SI0yD2H8Ffop}rd88umIVz7arr9wzg^$5VFxZ{nqqw3w1~mqZ zd`cXu&Ew)NJd4jGPK@Jr0s)4@+%IKQUj7OM?grwLLL!!Fd?} zEWZAQiAzMcIHo=Jxs}D}T~bzf^RLX-^9OpTi#3~j3&J@XjidfR*&46oboTy1cBjLm z&2Q>&1qa^1dL7dY!zm0y>`Qb*Lt|Kj&Fo3qAiCHgR;hqCZ;8!4iVh~K`Pu42o(|81 z$p>f+kY5g9D$G6l3Ti)Np^&8T_8UFiI>JcefPb)z6a-XKlw$r1=>WUWXsHAje%qDb zw`w7&AfEv}Lnvc%urV_3aRL9)?}!)Cdg-$4p96vKY!>Ii!S>qLm?$_j<`>KLhZ`SX z!-xwY^>tVQsmbuo4e9WJ)O#c2IH6`weEzn*_4nlhot`yudMIy6Fj{H{ub0TT^rV!3 zDU9CqhfIHUwa`+YM1Io4gsk;FiTeu<8IfXj$=Asu^bXrZNj&&+&8{1tb(@i|3Rx|| zM*^hQvMLh9ni-=a;jjd_6G!7BAjAzMNxm?3{E4@%tQL8eKp=CS+rS`QeCq9 zB(`3xzxwRofHvF4lLFzJ7^{W-6m&BFt^w3%&O%sX#q>#0#GQE~bgF;j*keyoRiUxa zwDwofELcB4sGbB&wy^oCSplnXJK$&ElGE;B7W%{%-2WPz031)4(k$I~@NfLoPvQwJ zUQ!{F-v-7Bo&;1_0$=b8_Wn5GgM@ug9*%(=$CD1^Gvt8&bpznf^F0}; zRc>(Wawv*PgWLfH)Fr`vu0G+?01wB&Ot->cN#cMm^&_yfSfz3VD5ZT|fW$6}pJMQ; zN&Vpyt*$J30jte3SI9!f#kGp0y2iKC8`a3!w6*oM_=SVa6420qe2c|!<=Z!2pNhAF`)aQico*$ye*e+ukk9jobia_MT2l~^ ziTRJXRieTT={Q5q#Tr(z8onjncmKw_S#T>VqWpsINXz04uRdz^vIU5JBVVq)7PSlg zp7BQ9D%9vBlR;4UBY!H%J+A7M(1=kDacz1lBGchu1ls z#%CArWQ+vE9=@53XC%FUZHEoz;pzN|=Rjubx1+6$)3>TceAi#cQKyIin=^R={=5B` zm?|zvS4D9O00J0)71W~f!F2bR%0iv8)NeM?l{+vo(bFSvEj2_`l1G$Z@|CZ27!@|H z0OH{u)`f}CLehfW(B3Mjs%AK;FFupIu~Hc@90ucX?8xBPX;9G=e+KC-!mLI49Ujit z#d9}5kM90|68IcgNS(5B5^L0n`%#@cauy35oF+NnEAs8GO7a#GWx|74-^sO9AsT+TWyEZQ~rz>olB^+gG=EeP@ z+HbtXrpjr#Fhpw#2}{Can%NC7HZyOK5c^`QDW`w5D?}CgnG761+ZPue391yI3L2mS zDu@IQaIXyOAfW_H3ak;44or9Gu{t*=Er0j>ZV}y(EP;0LFUA8Azsobqb&T*_>DFF` zTpi_~R}FiG(fz=Y-eV60ku?Kj{a3k3?D&LF%av7BLMIz$Y(nSPYj4rR3=+&Pr4me* z{Kg>(bY-7+yKd(gy*?+2-?lPYHK@b@iA$vRBn%V$Vp&~vSkKj>v*DVyMv=t)DzNJ1 zsf9;xr;JR>#JlX9kBdZr1J~x+Le2NHK6^} zYP7PQ@Y2cjM>p!x3as+8?pn1KQn!jnh}O(t!7S%UbBs7sBiCiiUIPm6}IZ2cBZ zJmb@oi(TnEM1a2%!Rb#BHJO&4%Rf)@l?06KOa=*p3tkUc#2U1btsYsOj<8-Cpv()` z{-Fp;{U5M3k_HsnHSP|@cM}L=fN0y?%B!mUkd8NLya?pbAk56n)SjN6Zd_-HEFWW{ zf0}5Rcg(tF%l)u~(_kO?YB!O~Zq@yFV0X}O~;FOFm(N>3J)zSvv zC(GCU;dg5OEdA_HRiyhpP8gN7UCkPDbly7#)h@S(&{TKP#UqB86CA8vVA3fDGoXVRc+QW(X#xiYZA!{`}6|Gj_zT%-GrYaQh_S-?KH z{15dR;Ce20xN0rm$e&%3XG&(bJ+8=rN4>(L(QwLd$;t*Ce&Z6_%jG^-ID)LAkw@`= zB8)Tt)kF*(?(NCYk(|*cKP+(W=(yrwn0@ZVV~|%wl}&2@3+y z(2&kfR=O8r-PL+ z-}3iWku&rY-oRJ=;q3$g6b4IakOfKT_U=guwS`3qAL7B@UXN8%)>)A-8nI4w6Z>n; zT_LDGsR+b`8I(WR<7TipMeo*F_h=YoizoRio4 z0Zyu@w%>lJ)m$zv>*vq0(c_AD$Zk*Otz|rEDh``5T^{nO2{a&~b`j)Ur*2k7^g1+e zL9KIPf=rWK)xAdi-+^X82R1Y`;459(H###^WCRvDSXmPdVTSi)a1`M)j<=vvNnRL% z3{GY8#I52Y;EimoG)3Kt^KhH7Q3!BiL*N&M+uH(faiWp55ztcg;W6-M{M(!A?(dUk zW)x&OuAOFqmRh^jCIpwm>_y@C-(f9Gn+WsY)CEa$yaRsY#C3>)=#U6AJos+~u@Ct@ z_$Xh0$}aVtXutHTQBqbYZp^RH&Q&iaMr4)aoqFkvz8m*!OIJ_T3H0!KYJB|)SY(A+ zxrYmtRqLzELm)4;!KBLG^VP&BhP_ZM3)#2D`13OXdFNeY^Ox~=tNHnE7X?zmAmujI zk$>_gK^(3}!~>ctRii>rw2gWX)mmS6fSSLxO((Y);s?nvn|8i{eeefN!+|e@Uyp{) zYADE1@q3Qwu&&FUBYqTvfD_7mGP@H}5W^lS@>W+09tp$c=j|D1;RW3O)t5XtDeY=b zPr&irDh%|@LUl{+sTI$yQ%#-9YKV0sYpegxSZPQU;84&x;he{=nOeUc=fxt0Y`}h< zm#wNv?YGd76pKG1luR9oQNDVr+2CT~Wq)|9Rpx!d{L|GIZiSk6jAQGxInK;M(#UE< zNVu{zB#h}B4|oiwoPI|&DsR$4g$2Deo1?-u{}^4)(4KsWX#HO^&D|o$H=ll$kT`HoNUZNG)2{9vHJHsb11Do@ zvWiN*q^a;CW3KP9McZbEj{-u*1kQ{WEFan*2mFue408JxQLxjPmYcZpI&IAM7L$k| zpZ9Y{V{{GiU!%t$P6BCfQLCT2wts+^)C-fmUMC!u`vGbz0Kdzil9wD!9L$BTWA7K| z68c|El)jl&=tAJKMRAvr?6l-+zR1qoX4lhu46s4P03dqQCI^ZZLJn#}Y9PaoXSQ!-_yW5+20cs(kGnd;> zJnB`iD*SBuU67GT{WIa~#dE|A8x4_7EecC6MRj%NR^~D?v~UHQA?c-`z4c2s^9s!H zU;UN}P*=x8>(=5$Q`&MOvZKvtvK*VJ&-HFaoiclS5^;Vi9h1G|%F*i+4^irdeDvxH zDa0d-6SAOSR%Ersya#a&z;wmn%Swc(Bm~8d$V-x3yyfJUq0O^l_rp!B{gHVQ#ZPwp z@E^2%Xv;d7N`zramVxxbxfF;r-u5);4$;WnFdxIwqVc{=V?Kf`N@8Vd+niHN0ndvf(!;D><0 zggEaEM5Y-Xf|#$?;N{k=t`+!N#gR7fRKH-yg$jd_U&eEcbr8z85eLlf6>Pnih!kR^ z2M}$kRR5a!RWr3@Ed}*`XZE)Vl7LyM-ENoh-WgUz0(kf>_`=33Wa3w57ncfrugmeO zPWZk<)+4+u#Xf)!#lOI(?~7XqXny{Bs7!ErwW+`=9?KHFgBE)crU?kq6ouu+|44c` z8%Q1yHG}u_i1WIU*p*2kyMN=gb^fBqw%ojOzw zb$IH&g4quV^5d&qufALwVUAmA`R7|H=ph`Z^upp50FMCG45AbAY=eFyCm(?s)bQ(! zU2d!Uu!bmsR1b*-M$XZ>+vGRreXx!1oESLrUm)bZ8lJ0(x#uNCGyB@^d*INl<*RaQ z+G=s2A0MW6aN-tio6LkBrp3LaR+j3H#jtM95m@4r zJ6(jSWq>UniQ&;DG>u6Cb5KO`vggG|%y&cL_X0o01OjNtU9A~OJ7nTAS<(oO_REKwvP zb2vs!WvUaeJCHFv`)cA#;rA+=HtLamJ+as2Exkm+{wZduyBF$TZphfiA2{P4*~4>~ zc>}eG{D7UbKLUd64&F4wVY%BIQ zczC)-EEBdFuXzqa=sWhwb?V`2g(Wiuge@zZ>FGCzl3K-esG;#0nVFY^;fL%mrIp}| z+AtFZlRom8KcVz-VbF2|tSu4Qxy?%g&)u76D-!Z8_rufu*V9V3n7T0l?r!6eGunrD$&?@8wC*fECh%?N< z=>kd2R)5^=@VMotpt}dBeuT408FruQg3D>J45@1HQa-64T4aY%}hU&&mhaq zaq3?$`z#h4y(Y-YKpn4M$1MiCB3{Gt@Rw}?U$S$V3nj-q)YH8nLZq7-BD zm)+Q6)#{Mjp#J;~_Dkd?QoX_I=5d=kVzc1%1)SY#$>KP`!MTt82`5_K zqv-rUm%~8*8o{BN7!XDKcDQaf&2h|$ax2w&66fhVJWU9w}J#!Deb&+AI!J2NLq+u8{forY_=kAbAp-B-1PHUm_GQK( zKJ-WScV0K^+1EyzROeeos0h`Su9u!4msYo6Gz%A&B6E2~y3VC3jA5g6218R?YwTz! zj~RQtJem!!>9m|4Cv#=XP0=-_!zbm4RrIR2QxsCZDMr|Nj)0xfT|cGL=7Y#7zK;4< zGA=sn_*dU_kqX+n%`23)Ij#yyW#tC_CSZ!!OHVo=K$=;tCYsGYSATNfRTY&=56=0t zJSk}(WF{E0neyOQeQJhx-ai)r#Y9S4+i7_v92RWcGK*mS!xwp#go<|a%sz!ht! zm9_v&kSK8o{ws+zddpr+sE3L0wX*m_A*aNsmGl)C(R2-GN;;p%IaaApyV$&B(KP5bHH0jrp?I01Z?Z6FZ?db4pHfQ zhBihriw zmfpnz+X4PaG`&S6P1YwHz}mQR>d_y;D0c4lh_6v;yvc0Qy7+!v#*B-DRy@q3Vn7kT z+X!den#ZD~*nRNaN90?xmdQEee5TB7yOqXyOAdlgd|ZB@|6ugFcWe_FHk#ws90o(Y$ac710I-mAen^3kDn>B3{ZLvLLgo@B#e)wK}8Hp zP*aI64oqhokfsTw$%{SoQB}XslNRoS%;fW^FM6aMjxQ}lO~kLAp{*)i)xCR z$j|2}+$(u$+|Wcy7Xc|{gC9D@_lKxCU4^CAn!Z2+_U*W*yl4u7>nsx&>DhwQ^$a;5 z;1lQ`-IK|K({Hu%N(WY}%$rwQdh}lAMPTt?tlmh9=4QCK^672{7Im$boR2QNW=ukV z|A~M+p#e?`_iA;Bpj2b#z7g5YcB8{9XPf6PMQq>T!_6)kHSZEo6Tnge5yN&k*;pGfmCY-kcYR!q!K1eHN>G2|95&U&shem6=<1?eeuk#IT@}`);a4Fq5y9fa%1Kj!|DxJh)Sr_l0b8~I*AE4T> zAT2GsP8!A+hgv$g{Z=BYV|;%#!fl+_;*Ts--j#9S`^QX8oudzLMWEn=feC|2i;JjE zP6BX~$s8yoL;(^m1N=iy*~$OG6@faGF-ks4h5HpJQ$qwe|L>O{CK@50K;yP^OplQm z*eu?Vg8{qswciqN!T@qV&&AotXjQnQqqGtK;P+yEI@v~p4#aeE;X@(J;A`uv^i$S+ zCN?~qhn48GxMXrWtI}##v3|B+gXFuPe~!x-m8mnYqo=2D&-K-K#JQ%@7RKx0KYA%s zcDIW0XQB(Qk0avehF$mq)H)|?>4Vs3K#jRy-+sECF05SsiZH}ob|>JEWZx0D9mkt5 zQOGI9Qnb#Bje0Q^(#7^$11GE%kCDPx1H`berE2^a-wE7dVqVe3Rrz`Q^69Bzh(lHt zqDGumR3$Y9XJ@#^v3q8?>`ZE7B!fi&8Rb1oa4#;cA6}pKRbs#Z@+o=ro@-A2{<|^4 z^$S%U9lrJdyD_7U3^5JsUiDQ%{hxKj32ch*34%t9$Zc7ePVSnYpMT)F=7xuo7v$md zdrbJ7b1}P*S*zWOEj~4gEcPC#kRcSrjJFbzCz99BO#OLK`j`Z!V}vF%ltcPwVv*WB z*A%3=&?WFHwmkhK1R7C^Qp~rki7odeJJIlYfI&)n5>nUApJgpGzXx1x>xEnyQSUCm8;=*|15e!LyPWN z(~6tXoS+WL%PIdJxJkB5fYZ(~=Jl`J!qC!%fHyuxkGOKy?8$CiA^*SMKS+I*5aL!O zYrOEoFGs`R^CwI1?7(?gghXZoE6BWn?eTDc9{r3L5 zn_5xM?$PZP4vo~GRXOr=4|ZW7YkEd%k~78Vpp!%;9>@Eep4H+>m_N8ou@4AzNoqRA zQ^e;^AnN@Xq6^&+eQoT}QnF*J zV>~4mKkNj?vfFRCEMcXbFydP}~{Te_`VE6I`^cCi`U1Ofw7se6wmMp)wd6 zk+Fl>1Bd%%5dOcz`$p3kwb~=|j~9JJz>+8Er#0M+tu;Zv_*w%dX?_zwcJ>Cl5ZBsW z;rxAz>BXqaBjIKZrhuZ_HI*p9-zjv}6YAEe(f@w+X1)bWJ9{xp9XM7>o*#tyB346H z4N0-!FP5HDx%QJBkm`{dAB}6B_w!RDsbEhLkbi8j`M6NAL7^Px%EU0-PUJJB7nuD3 zhlfU3?#vP%fuv)d^QWS+82o?~#A>$co+hPZ3$czK>XDqDZ z9(_7K^>vX@H#@cy$9dzA#NVe-BG?xnXXuCsoJ1V;5}5tiypOugM&JD7M?Z2FpZh9D986v2)CgWQn3{vYKvx%OC)Y0w{bK9f_>76#tB^c&EW#vh0$2V z7zRm}6Xtp%Q1=$!%*Lv18-KP!C+!E`DLt&XBv7rU<|O3W3F9aEGTn%Rfuqc4hmP zhc}60j>q-0-o_(Fu$I`VPQBQkrB)bDm?&-AlW{{&i- z<+~DFhd%4^Ha7#E3|}byU;XW;5)~;RNrW81wVB4%pZqhc*9_W}nIC2uLOK=Y6&N^GigvG@ z;phMAMfwpOJi)<4RSLq$jQFMB z2L-Ic+~q%hbCh6PW?>ogsSiCBH~W{QbNH%ZCBzgggvGTDlv^D4Apzlt7q)vtax~Co z@H*}ivT5++-k-)E|O^sVUf#uB! zabIzyyAzmA()h?c_#qQzdW2Sc28#N({=RIZWp%*}aj4_4nhL*@Lbon{SI)^h-60@6 zFiTCvw}Q{EdXXSTE2G_rjixh_JfAaPCM%x(Lqbepn_~@ zI<8nq4n$iWpU=SRhC~@4l8~bL?5-c)&=B2;c%${vh?eniTRdd1{z9B4Xy8Tfw@7~4 z($r!N06DM7*r-7WLvX@oHwYnm3hk-CFqYh1)IRJ3GFAj2oKTZi1(cEgc@q{0cu~WH z-`@j&nhYhm{PG~_U8e5hcORH(9!hxQDAWu6)ZR4A{(37o1DsZw^1q-)W9V@i{xXe* zWu5UAB1jTCVqDKrN4n4NJA5dCwQU!oNl3O+$&4#<2~F?tfKOIcvPN;1n|UkzI@cAd zHzO*Gq2lP_9azh9a;}^bO}IROy;XcaRm4rq7Fb~9cZUTBG7BtUNiEANZ)P^=VrdEoCV)d3B`A)%uYFdEK_5TL%|$XlzDrXR%?bSal)Rqml3J5m;&Iinw9sx- zua-*Z zpII@XzA2=4-=c;6LE$6PO)BOOOJ4|i=|BlFpGsI8Q;qFQe#a65 z9==hBSbh`A-pipwH#-xgu5w8)fg|4RDr&SAcF{%0SY*+RVJbxFB*iWTWv%WNYv&-e zXIhD572rvsK7q;G!%)7RnU(1b3?WIJE&_UrESKndv&5nH+1)=#1~Y~eK^=TTLI44% za|YYte}+HgpW*M^-`^Kc-wR>=V(+<^B+KM=)_RzI^EUyFZWAVcQTQ+v&6_PjNYvXR z3l107DT$jsEel)=Mp~VJisPf4rxG9KT<~!)w)$#QdJ%qG#tgK49?~IY|9OdU%aRRu zj8qngR&8|muf<}$!>#HV$pk0 zjoKYCJ$3e%YwUFW=;T{pM40xOO*y=LMC&N|kPw7+!u~UuNUQ`#*2yV1Z}l9Z+97fbjA znXOn3%YGVIUgr7{48MqFE8u-mzT--`nF+ACM~9RJ(QN0KSR!> z(Rq!0DndofBb1G_WDfO(@xl|Gl+Gw9y-3cM5`y@S^dIdp#(-hUFd}(E{NRIh%ep9y zuf#J(m$|JK&}%)1jo3tn7ic-2Qy>9ud+eo_V*0p*BRV2KqVa7EjUL26ALL)JGDvtYQ8F0+IZiY znD5pBCz~Mui(>VHXG}v8UVbt79qKH`mDi0$ffk2SMq|0}Z+zu+HVmyup-4Ia&f=%d zoWsIOv9Lh+J61VS@?Jz9+%zq~#E~OFD!;8iw){|cM}$>+w5m3sb+>(;PhTQYCe657 zOJs-1WotJ#@GDrUN{?Ty#FaXPw;lqSNV^1F=NGTGNm8 z%ev=S`L2+|(-D>wQruhuRpb$X@>7KWNZ=7AvwylKV!EJ--^u<&UJ!dBtBK3jt~VR2 z&g0>Euxfk9F0QNZ72}PY^Pk5E24l#0hgUwPj6r3B zyTiMZttB{z=DsM7Z#$!$df6iywvKS=a{6+8AOaFu>r*tH={OEg0DR%b(Je8dGrALe zA-o87{&I^f>Y?O4Xh=rj9reE?%M5Bn%ytam;;lB-6E(ZSu-1g>!i@y03LlAKju?=v zlIoKxwGqaDtq%Gm>`bbKX8nWZ3D#6_;8Z})q#!cEghWzgSjrIJRHCyZgna%pJ-b@G z=G~1KHdIisLz?e|HLwYH>mdGsze7^<<<0~H6P@N2fdcRQCncOn(gdT4Z}Myq;rQT4 zpA5jHFSEts@tNC>$ezcWe$?8pbJ`}`PKbxAIk#)Q5Etf#ye4jf9;&pu{gLqsdd;{E zt8|GIz~8o5STk1#A_nft*>l_4%5pEtam6NHymT6O?s2wF)+RdY{0v0<-83mdyP6 zXla8^RTVozZ@(9#LPoYbK%w-wb}K6K3%YaeiuV$z!_CwG zttYtAG3LWn12}=aP=?dKjM>Zk2t=N3OOxNSc;{>O8q;X1cpnFf>bsps#zZ!LpmoGG z_>c;P?FqAzild=BU8xM~B_Jo1p>f1W21O*%lZtPY8i8S?5GTRTCr2C#XUsGew238& zXHPBdKP2pu%;k`ih%=${d^I%zB6NmD%*kA3B?M$ifE-aOQTZU(a5RxQ)N5I|MB8Z6 zX$W+w9FbwiEmpYsX>iejxmL92;b)=Y3lf+CMha2XW^W#56sQhZ4=2?g2^?@(2=c@? z42-v^9q!E~3OSs)ODK#1W4`{wVG^tt$YJ@Rkx!Bt$sZw$Bp9T4rDq?L>8LzYI?2m+ z?h7sf@FmNR3p_hD7BInGG_K4om!gT>g&<9o4tUd38RY|EJLCpVk%oX{xtr}$9=o)5 zNYn?lVUW!Dud-L2Hl3~Aoal#qr8Cl5Xp0wiSqswJ#FM1^HO(@dm=VC(C8o4`Sl_n2 zggl^lyIEhTl&@c|URCl`%a$2&tb%w7ZG|xf^@81k7N&Kt~G(|Yoq1k!*6r?m^SdQTO<&*mK#BHgs?6_KaF5f2p+4* zYdR@PT3uahWjBZuxkf2B87nYZb3%|b zQ+~#oKEg1hoaVZ^_eJ6qIk01oV}a>ozN^-!Od_I>0lk zig~1pc*&&G(|g|M=d3?2tZ6@xI##*&V`V4TgadUg9k=-Dm$0 z@^}WL1CinKa0Uyq4L&IV(@A7OAWwMgbO(8&PR?$%LT`pmjMi~j^5?UQIF(*a3ZevN z-l|gRpCrZ5yCE*ezsP%OXc>HOn8x#nDCmJwUs%x;g&r@2pPAsasHX2Wh36beF@%)a z&>YH%;!e1i6`}ZHN0F;y#;7?UV5i#k?gS`|i$95??w637MtM>L;v8asB5%nPMiM+l z;SoBq6+}`2gJA};-Beqe#*A<(R zKV`BNsK%*j*VQxtSGv)nR*G55T${AEjx7Z&nPTY_yOhod@+>7x2H?Mr!Ja_2&3VR78JpZ{IxYj zfsiI9uCOFzfoiS#t9nwe;9{GwHy_uKHcSpE;6U@a`1HOpH1x@!Mpx?(r&o-(z<=bo zCt-NC^J{SP?VhWy4-S^Uo*ud1ksPtlEQwn=xGV8CmXHKoU?0;9xIWz}8vFyk*?Yhk}6 zTHRCz3@)-#zc__ut7c=;cer(V<&%OV^WkMQH{Cm^8AwuB)XZ&8Q&$yM4-)yMii@?w z1gf?8&=82kz>*LQZca7SBDy$E81TtnITN})8(@VHf?c8omUXb`5S#phJ?(F?c^PV| zqTY!ZYf1^x_ixJ6Tb>nS0q$Ls=}bQBs`*fph%5FS0a_ek77 zH(>e#LC~D#YGm|5A))pcKRI3`NUP}YGJ0ZCDJg9f^LURUOW*gB>qSf>^;?#(okXP* zWPm=bO{l_jtBm{9%SJZrzmPC3;wtRlHgiFx3D590gm_5Z;D}|X4do~J_iAdL2&|DX z*5VEn3<*UuyyCYw#Lig#*{ozrOpkcDC@w?n8Ne3mVX%F1oi;c$kgs57B)$U7&` z9>ddMUo4_KDRr9u#2salUZc>SmO?iEFa$QbVq2Kvpcn$3k@~UqN}k=}92xv5{@~+7 zs7uV-u429SZhBn4dEdcTfzuQSXIxjCARs)mHz{?+eXwT|VxWe;S7;?ioRUwd#eIX< z&4Hcb&5BmXTv0c4Tt`Jhar_#W*jjD5@LLY4^{{AuT9OEH&e|Z-{@s&noT-pFG7xF0 zQ#FZ8jd2W%R@YFxIBz{HfkyUyP8h3I{I--7z3ongUYs9uXOOW{@~tASPA{Q&vjLAt z#JC_M(35hHRB~fP%0uxwBR^ajSrR{5crMV{gnsXIg%Lt_>PCTtnis7{lk$| ztTf^;5~Sw54+RiDYFfi+y=NjEcF8R7A6=WK!hVy`07Zr+GbSd!KRT)~8 zpJGs{+iipywKz4eZ}i^WCEr)$7XBK2m`|@rp))!*)_ppaC#Ds)t)r)=0^B4?O)CuX zHYQr~5g)RhrbcfN@PgMxOBFV@F$%#vT~^atYmDmc>nyPMUAf|yzLTuIG z^_Y14J*C`5UFgUAn@<6}<#oHCX8ewh60=)r9CR=L2&RqPcP_TdDp>lHuA7!|9dBTH zVc$R9t}|XgDe^53+~(PK9ncQNR70mOx@Yti=#)Kbfo#5Kpiwl2`O^Q=KBToQjoV=RkA1BZf;Aw)mTlVzsK&rgnXC)7&+z`$9&SUQrQ#1>rb{4aOKF zeUsg6lrTdo_{9UHGSRle)qdVhYttNTfafu^ZM~~iNdV3lqgb`j#J}>20y+#jwwFc2 z#_if5wPoiaAPm{&L3sJnPz~ommX%~9ZUJ1=6rSP3nmI)fBVnb6J+-hme)|h9z88K1 zej(aU04*acn`ICxCXr7RJ38udc6*y|={6f`shaNHL&rpK!(y~=i_=vjI6gVqx?s^u zM<)Y_F3yc76sjAkwfphIfBx?+OamqqL|_l*ku_lXO}MkzV&~n)!UW=^nQf1oEg`PK zRiL3q*pv|bSc^I2R55}rWDslOZ*d{!GAFtWUp~$5eb*U&ku6R8mo~=!UxkuiOECx( zLVY2QcPDQO*SEZ66f{ zhD5ay^*;-0DXElfl_x0--|3V^`U^FikfW(dkqJLVr8IKnx2>?hZnZBYy2(%Fml$PG zHMu5e&ai7k2&>C(ntr^3c&=JO;**e_g>2irEf{L76=OxRP0-M@nX^;_@9xgG-1i3B z2HoJdPv-HvfAJ*2gmT!($l|+-Kq4gf+kSG8?kjF;X_cn`3?=3ATUOMroe))WbyfG16M0HTIeyoy z6_-f1tJQ_%Fu50)Gn*;6II?{RFk{*%9K-!IiV#|SgkByTFDLvFQBegfA~0tmWJ#fD zUOoq@-P+MVcYO!Ssn};}41|I~fFro#^hAhGZj$pmjTaxgQ zp?f&R4O%*`By`ZA5ozk;mu;ke61uf-;5+-qx=`3du7#D)&Fq_H&ChU>Mg8wONe^5D z3q&zuil^TX6xAzEU0cH#qP9D%Re*0W8$xam9b zB8Os~@^~<)#k;0;0wN2ZLjcRqr&jBu*9V;2;_PaVB4@j{n7@sN#O6ay5p4YS7;0oK zl*C{S7G4w^#Wc8v?yJrhNs4I`6c3+=1WyjBv4`hq(P-LeDOeb>U-M)>%rV$o(z`Ul zen20StfPxiPMw8M%W&;pU z>MtNF+RuLTS^ShW{rUY4G+T6xrLv5XX!swhDJ8H69k;g)g`J z=+m~MFpCY9zUS zlcc5EA34O3lPm$Q(rj2aVxF1F9M8U-i#lTA~^?nakdg)RTG|QVa2%LA?sl??|1N+Zp_|k*o9cmP&q(kXK9S zF5))kw3`v=WCTlD`6%nyczPft;vgbY!lYQT%MomIGo4u$FLB`iMjm}GDCAr9#IgJO z6n>kHK60tkmv#ixnC(F`?pRk1dl1ncU1^m7J5~6EYzV9I#BMfZU(AE5r<8*#c%IGc zVW$fqXq?Wt&>WkKkWn3Hql0{#nh@Y5aYTMZ?aDeAlvkGD;k2{=*tdzITv!O*4J?=_ z^0!P4!78ch=n&pz&5@Aa0D$7&cmRagrn6sbqj2?yK<@om*q-|v?8bK zyLIm0;ol{ud6tZCfEaprjdzTfMSj30aU?MlJ_~HBzK)A|7ayYgXTy>JEFcem+-5lU z_AyRboNgvR!1tH#ucRop+ zxeyT&4muC4_}b}wH6RYA2=)cp0ln{Z{e^pX)>8nnWXE)*(1S+)wMou#3!d~cfRRRo z&sWF*mQ3X21i^#`7!+wNS0;>{33kwQ;3q_5E6ntHXhz|fk}Hx<2nQWkNYt(SdL>sH z%`jLE(pxx_&4+Td!e$&s+v8um>p@(PW9!+wUviz0iz|<2nhFlK%Ccr71O#mRI6UFV zQ4oFI2zQ%xhRzYEt$=1&c2|mh63jDOIz!i>K?f^U zj8t|s&ANBk$Kr}bl6?UW0U8B83xsDcDH=fzf?q$R?u8JIq28{+WbhEGkzS zY?)4LqHchAv6PQOi`U)BavI1<)<^)}q1+x^aI8^xLh>>`In9&qlGcN<;j-K^@3i8? z9qJJKcSZi;)3JRIX_izN!}reB+AezF=Ph+(2cNS6ifV8T=ygPQIw`@L5ja3#F5h2` zTsEMe=w}+BYAlX;bNQ?mS(_2v(Z(n5tQdXJKsXLvo-_0{ROmBBo-m8>W3V@86s<)l zl~iN=lZQnGW~!|UX{K&gCDDOydzmT88`rLqL}skKmxq#Mt0Y%BRWi`rM3bYTEd@w75rQ-1SLSQ@Qea0a8Hng%v{?nNP9#s(lm!hh%Ao`eM?-;SaAtjidREOXJ@ zzR3@I#!`q*lJMU7H(r6io=M6XC+9@QEnDRQI(X75dtZ(_eMY`uw%;cMUlSE_wFB0IqD%C)aUa=N&NJd*sJcu~LYB4}n1&ymQUyP5$c#Ioe1fOBS%JhRg-b&Yio35EnenW8tO zJ5COohJ+=1&Fr%#ZVTwJ@jqh7SEuC*sp#R9B!ABN0PV+d-ex*7w24N69Oc1@D?U$pS5dv zybdl4hY|2qV~x8rBr!3l*y)aNq5&ei+5h%}HJoow4jGw5dWVGVfD&gRhnUupDo+cV zy>Lf(D*Y69>d(x6l%Axr)B`~H_dZH0f57jpdy+S52|8ou015m?f%_dH31kUA3VbCb z(i&k(uV{pYjG~500WXn)Xm4Y8jG3rE`u9RVwU3}QU>-5HDN>>_QPF(iBYe#(T=9)* z0{#T9ljLnc!-5?DX!J`oEVa5dc4MX3sa7*HeYT_`U(qIPNEJPr=Enhddy=jIB?1!r z7!vT=i*t@!^_zj1K6;*?am?kP^hn3=YyrL8&_3If#V&* z3!=~@A*20W1py_w4HsgW_7ksBKxFjThI)HQI>6?pS4V@{p8ZfGfMEtycPIUrsL_jq zrEfz<&>qshy)9VpA@eKB*4gobN;k` zRGY9%H|kyMYTMz8YY(SriFAcu4+9E4J5A90b5ZC{5pm|EwURXlAy@~cJmu&GZ15zD zg68WdjL~jG6doxmAb_?#m*v)bP2)yj%oMP&AO?TK#)s^{lSg_hUJ68A zLkyZJNG6ZLq$ndEQ)qS!`~FX%0jXQyz`qsBod&RW=UTe5QqdgGFz-|K}$0#TWd5w_>8u%uIZe&u(R4NX^lBqfW-7xw0AU7^eYNk z*$}F>5kt8hE(q$oqxg{9kVVQVS-}iDZM-~QA99ZV{}23^65(xG(K=;<&9?%EE74;1dbU)1?akpsEVCs50Z8H0t5! zR0%Zr{7qe+EiDp#C=@vg*8teO(_1v;6bvDMg5en^C>vv?A{j?w{L%^3c)-6E$Q%Z)I(DwG&4~UDR{%=?6IK5U>vqM! zh?C_6kPA!N`;s@$LB*YugL(Rra&h-`D!^?;@m#(?aUN*DSa!}JcufB*)P^Ln+cGNk zX0&dGe2YVJUGk$bf`d|u$(Rqj8yLY7wrs^H??_o4&0WXjr zH0^Rx<4XC0B{D-XWHBKXKxL!HvnQbm7|i_}_@T+yx*~u!6I``BbGF z-)$cU^Cd9A2BcgztdV|iytm1FyE<4N9T4LwBA;{&MU~)#%mjXn6{~M1=H89n9Y%H{ z-ZBhof%}$Lc6`M=9aC8C>9UvC)Wmya+xg$+1IbPyKb#yTB8sog&Yc;`T*!iuO&F7x z`(=xtJV!PVOHZ1UpdG^RW0@ReaYzryq3*B&I!mKuan79NxWySZg-< z8`M6f|KCxL#s3fvlPQ8+8?Q?yts4aFog-cbmT3QeIM_ah0nGCXh>)|Sn1z=1O?R?) z>2ZCNVf9b+N)ERZtG@6!rt2lm8;;OV*M0)5M$j?_)50HYR8E^n+rYPPE6u@ z3eu07RPz7iP6THo6Y4QyUKf-t2r6{IKG68=E-h zl7vl+Vk9A`OOc(0l&RTKabCd~49zwNVZ%#{c^BCtoC8zeGV6>__uLeam2KwCN6UCk zL3xYkvZELPF)#zZ0w`rdkVj*{GPnWXlS(w*eak932q7(j`fqM-jOT7o?j(#@#mfY? z29Wv-a@@sJv5g8WUVIxSf@nPPz2g5oxh5_+j&ulWpg&--d~#4;$Zy)xdtwfN*R7qn0$bs|I2umtn|tP&yRNgF44$*s zqA7Q8$~!D2eX^gxt?_)3f;4XXINUj`ThQ(4ve*M=`U~3?Uib6SCuT)9g%P}a3i6W8 z_bnPeH~X2SaqL!fh5jY`AJ+zFHaR7xpGqk}BI_G7MX0VRtn0Fie@%b%(2 zpY81p85^WPRs?dYbN;${_q2hBra1BE+Mb{^7%DM zyfbw2pUO4mf}%hS*jS4nmjse3?7)5Y8Hi@ zqjkXtLCfE21NogYqd*2)QGt);SlswF$5ZsKw+B5I_15*o!v&lTueX@k*t1CzGEF^O zV4l=E#B%okS~XOIx~dGI$WMHD*>vj>R+sQ?HJAor>(n5bS`KnDAG&Tx%Hn%ax-{JZ zYY>_8hVp-DYQA|?@j~}O1TN%nCN$&!p0HF15S-7!S2(w;w?QDX^>hU-`ry?6Fe|;_Z)MK1mJ$wyw6u}NBX?}y+F@r2;?P`Muhx|fXkML4SnA0$@inEsvnlY zk*Mu;bsS&S^Y*b5Ydq~f;?E~e3=E7D$U|}OGQ2j3?P+mhV12owc=+x}++#2@jqh=FW27AMCk>sXe1eSPVB$S6+qS11Q6 zoU?z2rUkad^nYdI<*>fgf>eQC^;`6Lh|S+`4yF;Qm^Br%G|TBoVrLM04>A(YxWA3u z)Ux9OT)$Cr3ab#wje{ViLp@K}i`olziTSt)9B@t!cRkshxc7YKY3R0Q=)%vMd$sA4Ea(4eV!^# za&rH&WY7Qrw6@X8uoUY|wvx`SO~9wWM~Dgq=acCe6sX%%N)Uab1tsSwkMFn%p-lY% z**bH-mAvRLAE7A9n`Tm0DIy z39^hJSGd7jT~2eNkRR!g++fz(trp6xLFSr&R#*ZWawRZu=I@CG`g3y293sS^ZEa1>8CP2>rO7cZ zNYMrHY4(NF=n9noKos-hoTtbz8tP#vY58@iaLXg~T)zE>lQ+w!PuF`OsY7zy4l22r zv1a}Jpb@T)vcgv8q0ZX<0aT9eFg2(NiRuF0pEpXjgs+&YtgYd^`*;cQ42WS_`9;wi zJ)1plRR_Pwfnx7`{J;`B_YHXL!eRq?mHP&-_b0OgqW~Gp&;PyPN66Sjg`Qq*p3ajT z_}}Ckk5c<~6u+cI)vK(o(bbFMk*fa6U1A@p|LONlX+6GoE>M z`_1Kbtj!G~5xp2IWtAF}zw%J;Q_wT+Ay>t7UH-aN?0x#4%VnPB+j5q+K-#O9wpxEo zPYJZT{&?c8v9jun`axh6tZ*sU%f2-O`cOifY4cVa+y%cjlYI(wU_ah*EuAk5#nB_I zzhjl1yBWnTIJc^&ErfL;6eX4vnLT-Sj6V4L`@xF;ExN@h5>i5kkNG=6e2q>g1fC|n z*d~x~rs>N3nrCBzkSxQp!bjL7QUL2QFkVf3Ae6@CKG$;#BW+jq1r?Nv*2C&6uqjoK zh=BKwNWG>W+E3~WdmwxToo@bdgF$%KAw_FpR32lyC!nagFMX? zk1!2XZ=634AFi2*INo1fUJnZc+Yt69;#C5<@luz%U$zcnozVLk}^7VUr zeYR>jx93{FqlyzEirL;wI|3vq$u}gbE178yIvvJSXJ*Xxq7_~88t8adpo7BhJh92^``MYr1-Rd4|_>;HRBKL3p%eY##Xit@Kj zNwSZd$>99h`=Xg_YunG`|2odqz2%f94EDxQbD?#Ht_h6b9)UqiuFH&jIN#N(gAZFJ+ zLtH8AVHsHK%ZR8EO?^K-jYl}mXgzC@?m1%cn3 zT5QjsCPugO=g`nl^G9y4ZP-AFD!CS_`2WI%hrvg1_o@xq_&66E7)i=k}d83BJvlp=I#+dZr$mO(HCW%+6G11OUrHFNt1c;1ZsMPrQ6K5At`!(F{a+U#CMd*$HD z79-KSLx;JfV1;sNGhcxNA^8r$TGxN zH9v$`)dPobQewfbeD5#UBf{rGH|bfgCuYTIoZq={U-5r-+=)rvXq`hRmZTt1) z@@du>OQsR*5P028XF&Ld60BpbS6>^tz7V?hY)tXJOt`MJtSv9kEqbXdD-#iBB!2ni zuqy%bpAl|k7Ny)-xVaI8Jg-Q@zGRr^;4eWuwJrs9rdX47aKZ^cFNR`@X5|nwow$x3 zqz$R+>(p|^+!9W-uvB~@>Z~>;jTfNOiRy?@cn;p^%CLLVDfqAB^)(hHp@q%gPr}OZ z!=L-sF3GnJaKuT{krBw7LV9Q3i;R~6je|Oc8h0>T-cr_6@V4>{=}u4S30rMTsKn4Z zgofBwuaqpmMkVSJ)sGUPkMpSKJ*Fp^^j+dW__L3M#QD@ItD>-l$yFFVx&E|w?i4pc zMoH^;l|knr!mDvb~v zNcA!A3$bs|@oc2eNv99>q)^|zTuJzyG}(uqZe-zSG0jHnDicBaeMtP%w?hq$Fu_na zL#(zcxOd2qpwyCLu$}ZHh-Y~qz{bv_a;Xq10r5>ZZ>G!R^ztAMG(fO9KI`dzd}mSW z;~h=Hwl?>}v)7R_?wvT@c!N%~>8z-rAr5&?!fZncr-YheEBX>UkNHd=jJ4Zo1N(}+ zQKoV2Uga~&FCybB@{D+d*KX08Z)q@pRq2B8{C)!Hz~~e7&;MTj z=A64!fW1RCj0j8N6b+FbgjMCw+K!t>v?9-m0{+2I7z|(L(ziYEx{*0&Ho&G)>TTN*UgklbI*gFlwQUT5t(ksm? zb8=3tj2~o~MPCMVnM*YRkMg8QJ-pje@9Q8VGNo_~B_bugS_`hL%{Pd+Bbhp?stse6k;eJRgkRYQf3k-Qx(`Ie}jpe zVT}U>;cO$2BKfw`CgM@9(bGlD&T#)^I^#l4JfDU~5qtd@!^X)%uHq(aI6@ZWLiLZ$ zNQgp$?K=D;i!yjYiHUHDIZF9IVFVPFnH*LNcreab^_U+lPC&DHmY({=+k_-JS>fJb z_iT_B2uGv}T<8tI6ppB<=qYE8J}LvN4?rtGeS8>4ti3K0rxnsGv2VO6Qp^>?Gzl9- zpr>nsUPPCzODjV~!W9bA*e1EdFz{>Rpc=wn{J!GDh*RVGvNrgMo-%c619_qBRe!VD z>LJ=MEGb^?b(T}j@1yySi&)d!M;tsRkp*2iz|>Y5#zl+VPBl+AGAQPJfV9`;UHnJc zriFGyoqI17e}$$!rVZF@>Ixob_IET|z?T$Z%^{wW9Kg|3w$t0NZtTP@Ew;4AWV@TU zUUTDP=|H1AB`A{VJ)qC$m?%Ln%;|hrQ&r?$W@0|5Rt*YUuh>w>6`FF%RGkLJmn=5| z9WK{_O?LwjrKg8IcVZ>Qci!#2Xzbq$Q4Xq1($_CJ){t-as;$rP4=8xUF7I_$PuJ) z>_lMns(&+v2Xb}6I*~$FL79_6*RlEsM;E-U9tfFJG?p3-gwys>?9-LcAzxxS;2}t* zCg(~DfVIwUt7RUUknwVIN}nKoS-&Rt_}bt3bX2+ma@V(oM`*wQR#tmw+%8QAZqiLu zt800pmH#+5dNvA1(t!T7Y7S^+8tyOYG2;?szpS^uFUn8KZP<3cUM;@Y-C?m&ZLLJe zdYs&dM6nM?ij*YCzZ@23El_9%RmhhL8s}|c$!uGMRpdfA^G4~lO}mlr zdF@GK5e2J=h!+RpceZ&S*2w}@rn66G`s^2?6 zcGf)tFNvvQ-8>z-EK*Bb_ci-lMtMTEC+l~K?uXO}d>be%FHJ2uZN78>T|MRubhC}c z=@)&an?8-SH?gOnMD0r6gk(SAs<&?v=MS%J@hTXZq2%#UdNeRU#(9B`EG}-Ws&8V_ zNu4+~hEziHmC)&<(NZ8Ha@KnsRJVSvnPHK)YyYKxmg#Z$Z{B9FUX3(`Ke0CSeZ>VWa_c!b9d{s!sN4MbmXS41H#JP z&6HnvS}&@*O*D`9*QN*C9;r;?QM%HdhCPox;Y`Cyr?tZc20CWuUatu)SxG`gw%Oyov@A=f(AhK}pO;K37rH^6z6bBe4Exa>^G#jGR~o-x&h=FH#>=U%GCJ zuIt9jn_2m|jR@7p-wMZ)fh=APYRBF)sezm^X1OQAb>Qc|v*Ql31Jk>tC^k;!;q+W@ zB{mgc9o{uE7H6ap7g@ol4Xc#-dfsr<Zq)4XxgXRSoi+U zqbLczs8v=#ii_$BUofUTXHT>f`H@AV8PCW&qw0K{&(}zs2t~|Q--@8_=SdoB)4clU z=rQt@``|B=6K&WgZ^evzhG{G77YQPz*0Eyd_^J>TX}fx$p65bZ(36hiNxr7Xzg-jT z$K!KIy584HJ2wD(Bmlm87yS%seKYS(rR-$Yh_wwMZOn(piG5RN-ouD*R?!-{oTQKP zyS!jM$k?kK#XIx?)9qW2bQvfI8$92=vY0J+jYT?bT;+P|2sJWkGJwt$#@*tcJ;Nr( z^^F%182Oz@`tCdL12aPSBa_4IX2M!`FN5y&dlHOyiBsfi_dmU)7OKze_-`DVnUVk; zQs8Z`Vb$h&KiLQ3)DNt~c<=NWHdp^aWaLwzdtCx31oZvwwNXM<*zZC9s$!JEUW!_L zB6&!T+9>v0OVZdRfaZl+_Wd$v-@rF*DRY3LHG&12FhoV{ViY5NXfsgdOOyfk)VjG* zD_-1*YIf$RMQJLb{O->-@KSFg$p%*17~nHSjx=LXS|Cx7X%(pQg>SCioF{IRfy+ZtBd{7DQ>rdgD(SEITI?_L=KUcINnZ=I{Br+~>< z-!5yDzr*M*#19NsoiLoQT%C9$OQg>jx<;_%qt2^X;ZjR@ zi|g^*WhdV+89f_iy0NoAH?*Ig?(oCWW9mz=%E{JwHT~gvrJfEgsqB2&HjzO>lCC>x zI4}}oL)F?KLae!scsc5-oR&$i0peDa%;>K|gCvqY;a^#&+gNzk5UhrwLjD{Zzkc(% z6@BZ3Pz*oruPqjQp`QNbW^T|uOLNRV*%O;WvuD6EmZ`3(_^OoC-y`tAi37V@XUI7H zRdOSCfsl(}DYY$~nggY4K?5Crhe*)M@S1up%}RDQJsOV@VIlMzQPCRd41khSewWKy z$dZWPlOE5w+ZJ^C!yBg2{kB`mnDOsm)3>X{L<^q&g?{a2((mvf3P-vRLBh=@ljZ3g zbCDdG7kehtkh8Du9M1J$29<)`r`!f+y8uX3J^B9KIjwY&yW-s1G9mcg^~v*3T;IWp#%_ytHhe5y6 zMMuY|qdC1scRn?h4JAHOB^p#zN48e$FeZ_Q6Y-X$*6T}p=%&E(F4-IQ(MEDEIY7}F zQuug}izzs(z}x(qgqR z7b$kR_00{oPytYc#rr^d6`bbp^?w?vEV7_ z9&UvKKbqk{LkbXtfd$9bwiEnJ=Yd1;BiHA{;?v6u@DhLsPB^oPP_`RP?7d|CvkVl& zvht^P^S2^Sf(yIo!BtCYMB!{Gp{wk9d_kL`Ei)s|mp=x#1HU8F86y|3bH*|*L=THw z)@UQ)kp>8NsA5crXlfihdWRj(uSzTj=Q)xI!d7JQP4yiJhmr#vfIk2C)HNI43(=}0 zuM3qbfPRFo`Jj4cajgQWZU=D z?RbV>1_I}irOY|ix>NJIYc+RXk6Xk*$VVHG$=aC*k#AySP0Vcl)PsZG-_)11#CDw| zyapxB2R%O-XEN1<^ENKE$Tw>DuA%Oq(P)#~g4i28h9Ji}Jl*%oo_bFK8fx3)7ozrt zv5z7C)eb(Mg55|M0cEf4bwdE!gm~I3H6lHpQ#hpwNiM$Kit$H#dhbu z`U+i`5sc>IO9(G{c7s?dx zS&Oj-SGbgHt79up4VLXO^Kq$s{kOQj&O~L^3FMV`39r*EgPX%Yg~-AayKO`RjNW=b zYe#W)xgI=j9&xzOT>AQ`!-vlao7aE*K5gW+2Bn_vs^58XsaG_ldTuv?LSQJ827Grr zEzZ`Oo#jH844qi1-s@8PlBV9JNXJH{K+(8$3E1-K<}mnC(+Qf6ihuBXgD`acPo;Mu zSCaB5?{8H?>Atg-;p^KSp5`OVRM#u%DrqX-V)Q5qPb!TlLFj$}6m1ijb#GY0arWI4 zatpQyWPQ6&zTPNuDtp4V1IX3m$LJ4l?nDOi!q|AT(u``z)V%;&I&h9tVa?u>y_{*h z6*nwzB5l9pS5h+YOzq8WkDIf&TewuB_Fet<-e`C$(Ej5@j4mR_F%U?xF^X zx{~;lhasj9%6y8OYGbUYiX0qiywRNghW`cWz9D)Yd98IJxFwLOVFx7B28rbb^(RcA zh`UKSbs0v&Qmvw?$vkBpd3c30A!2V%&>BJoCPR)y_S+ayiwSt5hHoEY9u_3Nsbu}J2S*()d+M4w@L~X-wjdYG5 zd);E*$FKGx=?$x_Pr!DYCuy^&N&~02j@pQL#ft_gWC3a>DKUbwiYI#2vlnuvZc0B+ ze_sTtA6S^Dq1Std=um28OwvZ|s!K9fpQ-xQctBCr5}CL;^W0prZ;-+RJL;^TyStMy zMrHW?{mMMif^JxKL$}qlnLcz{@n~Dg^ffP1FWKj#%8`CcSJrI4Ay`euDEuX=UslXq zss5D+UKn^Uru-sC-m^BsDu{}N|M|NQW_70D&@(hyAGASeSWx(qS0yE;w9x=Xei4=+ z4rs0?vd}^E2r!~$K+fQN^?b0~_{?o$qOnhEcT`+R6tpr}#MpKIX2G@XEuWAtnfx>( zqtwA-rQMxs`XTr)>lxyt zoZFOnCk~bc>5WpqyUJm(DS_T&?}mBFXm~>bi7ZR}LpaO!Z5fY7IrhTOAL@J%JK_5c6gRvKBIfMj zNcTO-Gry8{us56=-xj-+E^@PoR;meC@*bG&CZ3v1fA7y9-TdJhijqgLPvc#yGP8O9YcgUaCa8eh zSq*8v+ixRh$4>@u=QiWjHwY&AX0@|9^VGQsZ?<__E_>2H&!Eko8t^=3g-2gb1i5AB z)ciNVJHh0PwXM2se5i?tr z`JH1}i|WqKH~4R`C(mh=?~+TI4UI{qcnsn0#Bt(toAOI{7<77VcJWD6Hu@Z0o$C1*Og>y~8<}%@zTyB$-7phn z5Ao(>ALV`QCv1N~66`(NDZi8$;eH6Ho3~Nl+N6D0|Giz985f7XdbE`%4GkO5u|hkZ zn7zS2QpVurXF<(P=%qHirI}y;QxE~oNBZe#>xseD5_Wh9P>R4LI!Tms=(Jk1jBecN zb2kk6ycT3-RHDXfVda!Oha0 zf=O5Vb}u?daB`E=F)~=wNNMP!`}UuttXXa>Uy#=7w~v(sC%EqOZk+r~GRLa~9`80D zqlw2Ic?CLw0ptwq2U<|<*m!5}Pc*q)n<7~g|JYD5hu;0%rd}6kmUI<` z-Jw8Q7_RY0*J4v1cgQevPlg;^^z?plBTS{UWMhA)=%(ZaQ-i>lSjA$!_~9iHC*0(f zC*n{R6KL@LbVs>KkRPbis$Ii3M3L&9sQ#?Lv~G;$2uA7)$0}QcukfEyMqd~f(>h(q zPO2qBNb#wHrMXVvJz^9{VlK3%O;pSDd5e>38&Q}Y8O0{{GquSbxnbcE9tREehbLRs z80EA~%bqk_&YR;<=D)bF%LVaMOeWA-ez<`7XbIaL;ayCLFhf`8g^y?P4!I4vV8E{|UuqT?20$gceip{_DeM?A4OA!pHm;03f{ zXw{xHouks{@;C>mPn)e0>T=MP@XnuzB(ps~X+PnVhJMu2<8PP;ly+_x6wyEHhhn}BJymI|G`}| z=kG2C_VSt$J&B(Tz{Ym{*=0N0i)5%4qoU`^I6$V=LbqA!mi>L_p~%r1xxRqHp$Kts zxiP5x0vzBsLi_`mn?+*E^#UImfIP>H8D$OCp@%d)TWPm$Kd58y-K6E^<;)<7 z(P;AaQB*4|HDyRPUBxe;4L}&+KdC;-K)U{GcYg=|z8p$7vi&lQ=B{ECM4w_Tm{-0O zb#}v=$luPu#X|4o->cmrT(+z6U-ov+RAxrA90n>W4YA8kT8&mvkkzQI81NMUOyD@^ z>ApV<*IvOE?=~{s|D<$vzR#T!HvUZq0_mPeH$M$Y>Wb=AQKTj#U2GZVaX$*VL0nac z!Pg_f-16tkm+|}_vE;xRASV4%6*$KtMUu;|SzSliMD51Rf*bYO-?;_VucIY;=g?nZ z2NK72j@3>1JcI*;CCv1tvWNf^7wIGr#n?g&JOy^V+&Q7-qD|0SPb|}7utgoR>UU+@ z97MlU$OzrLNK-dw+2V3AH>}-s8E4Py#y6CBk*%4d~@_U_c^#Sd7cxy1&*9x)(*qE}D zRa_~Ar0Q>V3W`QJQZ0kF{1W4%&dUsJ+2Ua9$*9WDW4YnA#|x9hbMMg zA%FFP>PlLrYimwWe>{}~-}`qJT&ZOlwLeb6P!0tnPSChm@N?uFMBFLAqzKvbPK!fX zDXygA48a|8$j`0zuYD+vd=(`tn<_IGIo;IX)$bjl+9Y*gdB^NCLZ~`>*elmfTL}Sw zl`j=>uDp$uWPow>Jw;iL)XE4-32a<@IS>7J<=9Mm_>~>cts45f?@25pe!&N*m= zwux;Xwa5omyVgCgyEVQ7jC0ue$_h>R5kG$@{`@i4{|zXVyX7b<6xwM<{C7q{0d6X; zI^f(-dI8fH85XC15%~ZqTOf8AvCehTyD8tjyJ$lDK*@s3J_NfXDo%EZ%YrLwP9hqF zlC#r&-Ce(lxf#}NBLcCE-tFr14)+Ew6NjDpeGGB~feDgYs*>EF5AI4U--+s~2*Qf3 zuN(nTzciUTEJWu1!P6Ygn73fUJFgJvpV7Y2)LautZ%LGZ)jO1wW={$#Og zKZ6ikXTj9aS+4OsInTsem9&eVH{v<(4)Ow9=CQqf50y}!rOk-N0)+Dd6Q_-VNLNaN!&1ICM{A; z=)movx6^S;{eW+8CFY0-*j;tw>-9evqG^1;DoVn`- zLRH6HMuQO|WENNl77yx+8cX@Yv;`03AgvG^!zoljNUe2pHgitI2Y%QE3p3bENa7M| z5;=ds0V5iiCT>KmCO|lgl%DTEihBsxA_MO0$!pM=x z$g0=w_{aHsp*88jpdpD7)vYmH1W(WS`DK=b-rV)YHGu)9{f||;7JD$-3fF>xkOOolXJGc9(CKAtxFcB z*XR%Tg)oLL868ZKHchx%#sI%y6}+8mDh0paK1V18YF+T#z%iVALRNV&l$Mknq1I8j z#0}aS2&<}MwFej*LMlVL2=3yrL{OU}U6Us3PBbRQ>gZan#=t$ojvBIz$8h>>KmZV? zmei1AE_6v@w_=-e6vwu~hvSFsOv^)E*I(2*sSb~pQPw_W=2JKIMFD}@98}Ae%8_!N z!9FVo#xgU`j-(Ip$=+Qd+D1=lw}deO#ZrjGZ+XdIX^&x2YxEL9bKa>vU;FmW*&1nZ zhfTRC&h|-+6hD%)xSTBIPPN6&QyE@7^xYiJTIhGh6TuE0op)%F;+KWYgfjGmB9Bq> zO8jJGq^Fi&G)I1yOa|g!G%Kl@GJm~Oly>8xCFzB2uISt@n`KRp%;Q9WxKI$ytN@-n zLk>2unr!@YU+9awXgLR!Vt$PvRzIC9+%H1C7K(b!a7x;a`wh?8m!WdMB3qKl>%Bd# zp=RyUB@Lpug538!3L}hOGAt6%RWWgbkIn@ZaGAZjQ2i7262Pb562h!&;V#}agUlUi zQv)dLzi6w|qjcTf)5pl_S-8xCsIFHp`p_MQl!z`0CtJ4UKinAJh-+98iedqskoFjlM%9R3~ol0o6;(cjK~-DwGk zjR>cy7JjPcWtlnt#q}uP(|wKS`<{n3<4hs#fu~D&U8jgMrQxv?+V@$te|Z(|1K0L% zOwo-p0JC#50A22vB<(jl$b66C2nx zn~?4XNol0JOIlKpZcvc8>FyQ*=`N9OkVd+Z?go+0Z{s=VdB4A~@0yvl*2Fc})Wv0n z=Z#Y9MKWXCx>gS;mc%*}(&v<2 znGi|6%fUoC@cs2h%=zc@rK#<%ELXf`$qG$#l4BAOA_mDED)-4VIaTE0O?V~EC>m%B8;w4tO?g2T&E2yXO;cnq4UNhwDWv6%2CKct`|>L zk&#*ODOuKFm;ii6dZs&;(UOll3%u1OoT4~URs7!1)lD&!8ADE+R~kxM1zSXp?oOiK~{vn4Pl zX|}*p`M{+_IvSggWNBP9oW`$y1{Yd)&N(`wJR=jC9GMbmsaje}xwNbTslBY9LH5^- zEHA^}5Oy(U4Yor5cxX4-EjR&o6Kq zjTFECh6g!WH~5ia(T`Cz2=m0XQ!5=3UHp3V|7@{dif63mTwcXUiP@XP*HX26Bgj`!VASlW3CZl*^IHZ`aEVbXFUok9!X zdOw2y(>Ut!`*ZL=F%QHqU~;G1Mov{vR~ZRiL%43C#EeBIc~@E+(I$*QK)IW4v{gCw zEpewKdQwe^<;c`A*w}4?oSp8vRcqWV68SNAfNwvhqvDZ?h3i9bprLR((#(z=OQP;G zsSx;u1f)w!t-fVBVqvkD!T$xj#V3l~j9{g%#OugQjVU{Lqq)wF8kLQ6q;$@Dm5yy8SH!v@cta5F(GQ_ z7N0K^ryX;T_hNqM9$^Xi+0gP#qO33xowr(~{gXhKkUo1`C7r zG7YF_$i|1@Rl;@FMD|hi^?HrgWER^hXOJgb_}e`zg;Jp2eaXSXZ%PCN3rrT0H#}uk z?7Q-_y=A)z4#e3EWsVPk)dvGRkwMdT&NE4^urJE7f2OA&=xZ^hSw>~`AL!M>uH$BE zUp5Bi&F)V#W7<&K<#&vNrF%YAK3bdiq?Zqa6;s*$G6}Cr5!Qt3s-feTc}qAkK7!$T zip*iJ^Dv5|m`$KjTpmRSVt!r+K@rx%OabwGL%&0*n8g5+3D#iXkP@k_S8R4P>-^PO ztW$qPRZk8=qq2C`jT5TfkKGdr7o50kX`{1S!T!34_Y{ifNuXCbR$`dMB)IiXu^wTE zmqh|1#Sn}_IyjvSR}8%mu?qrdWG6n`pDmz72O)xpu(ozXEH7Uk)=S0Qzl*YUqPIoI z?A-q&NFeG=Lvm&YAUk=j`NDc4Sw?*(j(;F(uh}%c?979PQprYZLq1N3w0vgE7>K>) zt^2-=RvFq&Y00*L;)i)11I>3KOVPWx?q!c*%~z8sH2n^pFG^#?`AuW0!%Kcan;TB7 z1IeMOpcl3zpWt*bI%g+^NwoN`4byg;r;7*GM(MbZiIUHN?bh({x{pejsiKJMb+;6OQ%jfLe^*;K55CYT!OC4j{f2bo;wxs z?1qn10!igVX5CHngr$j4sDgMYw%%GnebUBJNXMwilb7HzwO}##YD9L*hC-#^$&Ixk zh9H$AW1e>h;N98m2cQ?+L{56au>}+D&}ELA?Xp8zEkwhgl@`qpvB*l3s4cM+m*CZ4 zquK~l<|52o?XcJj4?R$>alJVKflnF~#)cT?K zU;AZpnCc%Ro*6k{tb%q-Y*UCJl}dWa{k=O%;_hmtCs&RxQoXrfb*6ZXd?r;zqZfsS z=JSf4I;)>b0pAZr*_o4)VnNBRs-=(QEx8i@0sTxLuzJJvzUzslt{*grr&BjVbLm`O z*Fe!P|ETf8NbeeGUT*a{bCU6$5K$>IE3VxA(|1bSR0%_q5r;H&d}*&QfI#5!&TCGE1YK);SbREX>`A!iHW6-f5o z+`i_DNOK>VXN^_!C#-iy>&%OKR6pW30cyuyLW{AJ#1AOlV7AolHHFICh+!Q8B)uS* zVGqklMd|a(jwb{WkT8}Ps?OJ#F@wQ3>Eu}1t z%Du#P?;sKcv!~4L#B;Yfnet`wEh{94_r;1Op|VVRg*N9%F&9o;!F(ap1qJ||#I}47 z_^GhTg{*g&$3e4Kffz>NhCC-s>-ydvv%K^Cz zhDrJvKc~0tM?c5#f1sd@6fc>*q4Wx8(T<75Hx^uykw8U!xG2hv9@Ly%@5(;(2o;>c zDU^}dfu;SiE;47xtEJ)+9z(><#8u`vM*ij<;hl=LILgikSgnjkB6UYVJUoNlF4@B< z;pl@BFK-sdrZ%R}VCG{&T6n^VO;3}w6ME(wqh^oJ)2^-O12}06Ff^DHCmlx<_ zzT4{OKX0%VrJ0+MlKpo8e|rCVIHc^>Y%X-gKbW`IkXF}QqI;FoM&m0BGHmWa>Ga6N z8hCTsYvR+P)q@T{ty^&w+E4thEUA%>Q|)-hobD4Pm08WqAXgR8S z;aqz}eE$GdeJ2s@q}T91WcQd=t=DVrfja?Kua&)92o9zDDAs%c_F)%f903Qe*<(!Q z#CVTmoN~c;=lU2euL^7!Cm~H8cb4VZU%CR)elp&c2b?ARzjZ1iQ^eAKpPl(}=?iepInhq^$7j2& zNfa4%C$eF%FzD#v7k{rt8z15l{}F^nZuhyj?Yc`R)YYxa8te9a66M|-(2)3{E<{=o z6`Wk1j7^|fL7>OpprTxVtj(%Z<%^iX66!T9xo^WEYBi4m=r`7u;z2nT+kK64%~L^` zHABvV;yb6o3&VJ0?IYl=*G5pdwc2bxp7z=&r-iC6|5twcaWVkEh z$^H1m(5ZiooX3_l)t*I({C>+x%SSDW}VkmP8W z(m*>g(?!JmdX6Py({R?z({R%Ii>3Zn40HR6c2X@Ebwh$Px-OR4pTw$7*TTJoVF4^Z zKZ-qrBoU4%6y@3T6v#E?+^nB}ncVGtIbWzfe0$Lh=xALtyafvNf@?!!P16m)K7u_T z=8opl@WSlPzrE%MzcgraeSIv_uB9MxKdK^hfvZ&aS>S~qf6?))Q(?VYdg{kax`UAG z+|?gkjimXWv>b%2mLsm{?o4xG{izsG#mXsn2L+K9ZyFImfyUv_ntO3hwEp zZ#+o?0IXi@s9?rH2MsomwA2kY335m!G=Jss6-2(DB|2(#r|bP+#zf#EfP%0gMutDI zSZS2H3O{vw$UTvJ;__M$@!nX2>vC3d*Zwq#Lr%jtlel}wf7Ai{=27Xg@v59X0k4{~ z(S>o0_x5*Yb}hC4HG;9g?YJ{i&=n=1zf0l37N-zg^p9-0Rxyt)_(Yi z1J1L-dSQ39d9ZpEaDztSW$8e!$=t$j5-h? z!899heH4*9OrU3kVDw{Hb`U~H7G_=1mfAnjuRa}@oPRpol-_#sid^&zuHeACYONcU zvm2V$KAx(IO_4D6UT<@R$h4;)=D$k`wuNEeD(H@hAhkOJW0)TPxR~4(L1tr5i&#^&pso@U9?4dbXGTFmvE&?{;s+~A*1FV|pw{w9 zoxqcCQ+&j>gQX;-6-HolQgsJcCy;J7Sq8E?h9+c(Y}4Khi%Rul~~YF8N+G^kSJ&;9iK*_$odkiJV~Kb ziOQcz^jw1D))Oxsd(%YbgRNBphw+gQ4gwkc95t{wDz>dRmP;v5PG zayzg<&M;!;ojV-0tk8sr1DN+`uiL6S+wI}^(bHdeuA|7x>c_rG=tx^z10~mrZKWE< zwJ1s5JIG|fFZuB1u`A6^7p^q@GuS1{RSgcw*GNagXtC(xgq?jb;$`gB)tDh1dr0j~ zAD8V`;eD6)=GRg*l$>c-^&~H&p52cz}6nIF-Hj>opbJKCpAz1chK-$F< z)N1Hcfzz?dKSce)uC;Tz#nn5ITM>M+n1I}d=v!RPQC|+GPx9lY}` z+JJ~iC#bmsRX#2S!jtHY+>*(byj`e{?~?UR0hk-=v24ya3)pcVJL&x|R;0nzajTe} zu79+`>_1%$UIWnjh{fU$6*83v96~a1pBdo968|KRjgGsc~vJ84kh~4{x`5Qd#(nn5Tn2v_|6)COK_^z>aHYW)@vnMcU9sy zX%YQA#@4eZ9r}JJpaWMyPzguGL;_TyDWrve0Jdu`_Tl1l{?+N&y>krEj}9~7T7=}D zj1OEp+lZ=d{270TQG8-to5Ml2Mb!e{1D6cH6Lo+Tl%)cy7|&7 zL;Z_3^7(V#FNV677aCzDqHXV0Zcn^7Z7H6`dl`r1B@mhMD_0k?gFl1F_$9o6&2lzwV=3Q6@n-RQ3 z7Zi+)NV-3R6u;5L;-oaQ#UIsv^nou`_VNMIHlda zMSWCP$lax@dOX~NB^>_xWoSOkDc@uOa??|6ea9Pl99CqlThTsab{{PA|p%r!)? z$fZ>kbLr5PirMoC#tYSni67;@*>w)zml^Uizn_ z$2eRNLbxcOj;%9u1CAsWyo`aiF`UTdBW3#B?je-;{>X^vh#wrhNp)KaTEku%+Wl@{ zPQ6L3fsXM-tzWz$$%|dEc_(6Szpk@Vl+)Jar3VB!o9LjS_)J@pGd)(-{*%Gcd7<`ITdTrOsWDfX|3bXxVsB>CZ7_hR)nUl* z)!TPr5{C^NkjI4nuhH*_0>wt~*;$>2PH6hJv=3=w21vAOdnQQW#OmIxL9L~Kfx-Iw zQyvU3>zYHrU+XqpqG_f$=qCT&{8}kxZjGW(Dx##F_q#Jj@b9E!W;yC#05)3uuT>p(F((vcq{V|R#)ZZ zB9d!UkDrPvCBIy(H6_k96a6&GNXf+9rPhv->2KSb9HPaa0lsYBp}$+S8Q%S*R!D9? zD?xc9RDim0@_t8~S$8sQ7wI19bO%??_ncS{PFKH39l$_J$bStsQC!-!2t8A6VA+J$ zpQ&t(EETfHTU_4>k$EwtF4NQm*GU%K7&Ac|wy-(^T~JSHYUK#{e4ylrz;~t%M{5dI z!!}zjNwWupkC8zIxZJ6rc=-1K73qZCRAO!ad4A2N%|zs4db{f$t`|$>vKC@abxDbK zmF58&gFY;0jgFP+fb2J;HnVcAjuc>oNK)(4(wJ+qazt|CH?b9akUK?X2!W2C=7l4b zhns;O0yuYKo*`j*hR)9{%s`g&o*qHKXz+z#vwi)Bq34m*Rl3m9gj_|X^`8kl`S0Q0 zt_x6nR5ge5NrZ!liOUDx^u{FAuSP5D%m5Pn4d#+7c0BWA0u{QO1+~Luc^{CEbuauB zbkJVDL6I_{n+^gA2eS3hC#YiJ$B}?;`Emaw1SsNN$ZW!}6Jq!JRl<>@&M+$Aw1n^x z+Px6~K|aMj2&eYK@vmk{nl}!&YFP)3b-{?_oM|;Pv>s5bsKudcb$vM15L|V%sW1JNu%mEPSmz8wU5`lNB9&J_7Xo^lnmp^i;*fV_;ByecamvWU* zcM#4GJwYlU-uh_F8!xK^YJ+$c2240B74+3^Hdu0-BKff)0q_M)t~p=G3EQG!jL-@i zaJ&&rTH{4~(~S}vh`fj(B`UOOo4Z<8f*G!w@5q*l?(JWG&ytvNU;^r~%j(d_3Dg>& zKnL)XnX%*H#W5W+{z~i`NYtXtFKU>s{A@jyKFs5&yk2Y8lN$Ol9xc`ZZMpMe!sTR* zkwpzAZ83>A6&f)KEU=C0r4{%wUS(%1UIQzBn*qSS8Y51|{b;uHMLRJC;J{Bd%%hc_ zE8*5KGuetby>a*dArIixXo(}Tq+qjp!|umg%IiK z>3SDGKzR<|3Zt04VJa%h&9? zaxC&jA4E8ELsycS(yu;Td`+~CT=YFZqSyMF;X25Ox_IU}um>;$Tbk2IU%L2^XIDAd z4BEL?)x-sIrK3;?lilLJwpj_>3jQAI-GinEeFV|3Cz#7~T!}_Mh@v*2p!<1s*H4?_ z8XQpMep`C-+xvWmklMS#=5Iv;U|!o+k&k=r6XE9vPy#55fAYSu3WSATI2De%bk$of zWaK1Rs}kkWQvu2kZq!5)dmH(W7=r^H25dYzs3hyn0{tzw8(C!Z`~5ED+D--1)tt5O z7{@9P#t_%d;)47Oq(5u1jSu@b;HTPym^;lhXC3sgdUjC)?cow&10$&u*&sPQi{$3k z60i+eip((myHdP{6Tfq3vqW#KJgf*59mp5ScF%JQi2>xjZU16`xYPbz7`q6_Q?bjg z4yXw`?+pkOH!evyT|!nwEI#Q>GA{YD@SEh?y-FU5f)oFq<64y4@5fOL3_6Nf1s>&t z5j1t;HLmH01I^z-;o2g*EA3A?Z>bOJyQ(^Ru_&d;e?EGyv&Kd69g{kfzP6wJv05`y zwTYQb2`~ql_`DC?7bz;zk-qBUQPb7EkYNuze7JXc^_$Ss<7M3fNsx_=V6CcmKJv{* zC;c%{zQ1dt`2H8v4`fSN&&R`aKqnP&4P^Y0n{Q4Ub4lj+VbXgnq?_p8Rye5k_@>2P z=xbjIeSHdy@YXebik4QgF~TTW?U$L!)qBbGAz)=~RvriJfNdJUzWIgysimo^`mFT5 zx$ZOmeoIz&J$+(RWZ=n*cAprU!YyTEa`;vy*|+e^Nu zlV+u&0E5Znn|DdqpnhQDIWID+!6Nn~r)-|2#p(^2T`gt5j|v>+X}OP+S&if9mYW5V zVONM2J%xW<+-6&aV@iHlE=<&FGaEo|@cCGGksjS~QJ8+8=^&W@t?UP+qd1a&aq^@m zHA_kQ_!^skMjB)(oQ-2=ko&t)-c{ue=FrSb{lY(x+#kDj$_J|@N&=B{LJ*OC>++@>P~xP?7h! z1^e}eOqL=IJbIoW+Qwy3Xe1G8PP*kyVoF1~r1{SnXPZGPg@%NnrSg{#e(MD?XY8qD zSmm_U$!syz8icNsLWGK?V|2b3Uswr~KLyPC3nmA43nZ`gVC4vsb*d@RR0m5msTO<0Fa4wx!)%@W;?93eNm8?-xbhi zyXU#$+ONr25MA8um@lh5Ax^$d_fE#+8D6ZMme0{YaDJmJ6KqwBy`SrM;!2PL?I1I2 z4^0$A*06_9xUFkYaO-PbvmgMLXXH&DQq2Rm>Gv(0%OA4sJ~6x_;ux{1za1tH=YTs7 zc~#GX=K9MtNuDhbkk&WZ#V*ZbeEL;?f`RJjvzJLGqw`tnYE?_aGo7vMun7g7N*P5; zJFNCriKqOhm_#omM+OFlO;1iz?9KU(#%f(HlC*Nt#{EP$#V8}oeDyBC4Td+*t;Br? z#oz0P-QLV=kEC))c+b@tnZI@Bgv6%C#&H0#?n<1f$~9%Xo1bvb zypt@U6KG6N1o3+clE)ol?o*>xh zWSDfgGd!*AlIfC+H3$>48hT1B=Vw;Yc_Qv-r1OvhL@O|zq*X7wP8*bia5 zTxkC4&_C0kQPC}pv?`WP8|7Hr^cc0kKZqm2-h4m zWLE8Gfh}kOT+BTuAX1Xv@9KlxADqI$#w#0(!F-EAF{?X+Iv@P{>?jV`{5V=0>yz*v zerV$(8{7qh4^e6)zo7r#+T5D{@T%Yz*%UD~fgZrOGv;4muNt4NYDrSZHXhDN_ZQ0c z+<2o2HfG-IsER#V??JaCb)qGBNhJAU1VFDqNguNUuP&C24h`kT46(bh!nLYGMRkjM zjdMy-r93Mk%F6I&K^z)!ZS7XS?3cFsq^!n?)zcnmH<=*01ywA~2`i|P+=pSQxNf+ikh7UpqP`Q&hF`-_F2j`>9-Mkl>4(GtG zJB$?Wei?oZAVJ&)tAdzdLTld^!3VLcq|J?2YNv1vAq^vQar8n46q_vk`uv|@>At=Z z>SP-CDAoiGbQ>`8I6A^}Sy(P`ZGT12L@GGgGC2Ld`nrldf2i^5KxQy60gsnjc60Ml zcWOFgdUxu3qs&LsoYs2wVd9a8qw|vY93cdLKSfu^>qiMFBiRWPqQg((uGKcTCdf`J z6iYeWTk4qMJG?ER^H=}MMF{TM<98K-yOtGmKt+( zg6ZZp{FWMPj&{(;xXiDs9EwSlEOjL6m^c2yDz-{>I%@1pMw-~)Aud_^;V?;%8@!XC zN^LzGkyh54`^Ms{C9U8Z-*Ub<=l%T>WmKe8;7|o1PqfCvIvCE!dd*>Kma4}1xuJB6 z4!gj{UrW8X4pkr|M20YgAPzRVf}dCZ!sTSGl#!N962_Cd)e#9zIcvKD9 zz}|GgWfjM4{G>Xtb0g`Rh8>+f^{+sTK}{5JP~^0(vJvum0Nl;lOxarxt()~A?KMKd z4wbJH9H(znXn*1eSCk!2j4o5?xYb~6!$BgEE$^DA$K5`5-s-tLB&coru$Oa<*IO53 zB~yy)uN|uefhVJhr&A?M!}=-S5r<>h0%BbBQv(C3l89vq9V`2TM4f{rz)iG^us9HxMOyOS~y=V{hv?#l?q(wd6JsG5|AKLQ|lX0jUh4 zDfT%Yzdz53JRP!QlU+} zkNhFR5(*;CxF%Jh9PcJyVXa*mD}E|afbGcm%M9QP36(Iu7{t?*Kt*Ef^7?bray;_| z3Qw4%`Hs*jg5COUxu=HcRyTOg;bQQFt#{Jw%Zu=1xFN2d;s_?D(j`s4d>lw>BuD4H za#=gG<^z#{Lc-G;YX+SWpBcbK^{(Ia6zHhIi_l5}Ww(o2AzXiO1X$pc6!e?y;f#># z(B;kE>Y{4x8A>{Lnl(7xmM2h#^~M1Rf)5ob5*6`eSMn|s4~7GxN3^+poyPk+QpOA6 z78(SwT}O(i>iWBmc^}9@Z6L*8EtW~j6wWITU_99n-WLP^*Tx0213#839uLOnrV^d7Al{d`!~1thk#ttb!&B+iu9vX@2#$n~)MdPIV8nYZ;&CfR@j2v2N=$_ay6VpNBxmjbw$%o zv|IFE0cF70f+XXrd9PCG>1qzP_Df_bxt{Y8yQj2Gp2 z!5*J=mw3DDJDeA`@haof01i&vSh2vlsQ%bZ8CRSv7K84v328G5 z3~@2nf!Iw+VhXQeMoKZlT0b<0?(%%IRWcB2`=uJ;i93UupteVzhC1|fRK^Vl;MyCv zEn*m{7aS&-9jpt?^TfikgzZ)c51bm1^^NU)ayBDd(S{a&#aFWl=PM7TeJ1d$k%oZ> z{6E>+>6@pWiCDy>H?o%tCUp1yXsEM2D6fB1z>e6=`FA^-!0}uEU{{A_BPB}$tw>8L z+d^+}E-GqVEB)u~43_%tuilAV@8@&+rcen?a1U7zO_j`9BXG)4N-!x7x$c{0^dLX2GQ`1Auu z40W`=)u8t{YDs57gCMf62-kKb1f@EI8(3|xUhF(mE7TCf9a#GaBO)41Uuk%c%;+c9 z>j#m(akROL_^A4)f#|P}{3&BIG9Eb7?b}EdztsNoVE~8|5BBUU=&SQD`ewjg?&g*I zTPyQvF;wQc)I?ja#+FMNQ53Q0sCS?zle_4q_OpH{Qw#Ln$5_FNUMlZ()fbEK9*r&yZbJP~9LKli$1B$_=I{$xNb)v1&gj&gf_Q(FrVHR5G=;EK^QH*T_)j zDcs5Rbt@v10QIy?K6{>j#P%!(=x=?g;d-JcB!k4ueA~P9T~4|A2;xVe_?EakZR+v-FEGk@M+(jQt!@) zu&3d+GRrz9g;~emJ5~8voF7O3_Tv1XSM#-FbX*i|^-B6!lCD2hDMEhph9Dw#`*Pks zL|a~7zN_3C*1{B|j_bF>O3b+t161-M_vYdx(%Azn`P?|t`GPVkOaQ9>Mq5wc=0TF{ zGaHb$erSqXuf%PIMRSx!w%J*QZnEWS)0Zx}4aLz#`}7{BUocqCn6h zzBJ5}J@!f3oT)&>)a`SGx>H zPn!G;+QY`e>XrV%bsWERhVtGbl_P9yU=B}SrPaH4HErRv-PnyhF>uH?!WcAx1j~38 z4*?D~`c?=%ZyQ5K+!?)zMp%@X;I4VB)Q2{@{@Zz8_&ElCMYi%hTWDMxHQ;b%ElCyJ zKS@I9ifs?EiOrtC(kBuRUTCyt&4lzDgCwIUqa0S1_8yMJ=N&{G_p+K4k8Guh65qG+ z7L0Do{&-zBx4YxKKGTVf`Q8*X0T9E!zW(EsyC2ZnrmZ474zK9WbpU+bk7$>zOTGs_?q$r_ zhzM4%pXTUyUS!rkZEU(HG6q1I`)T)$Wo?hMvG|7}7*HAirIN7%5q(7-9tF1q4bIhX z8Qi{h+4&no?izzQN3zc6eif23ns0*fWcU~!-x4iF3qDo#t%4)MApI{v(lO~R5TQ@U ztygqyz^RnkUcmL?0bg?nfxsmehXhOI{fX+M=XVf^L!hw|4LfObD) zT}^aiIgLhEM4(;$J60o$1#FhCVQ(c_p#|GXoa)dVw8>$ssIIo=NJqu)yxD!)7@|xb zqeA_parD)k04LMXRiG{SXW{^O)o>QLFSc=ggK`&<{CUylnD}2C#{wBEhWl(a$Ptf* zHdsi*E&ZuTJ#1V_xp9{j(fYoX2Ng{%jOJ%SJ(vj`|nzl zqr3Pw$;ZDG{m_V13A{TBfG7v>Ksf(A7rL$UfH()U z-e|C42BOAo+j(6}u3R=>`n1_$IoV57v|{n?#L~C-Fy((-*JCkvdYexDIkcl*ukhB& zDf%yg3?M?ap5c{yPLGtdOpb=_qmZP9-kylNuoQp31#M6E=)Uj%av=Vwt zZy2hMLfUQf;kwpX17O;&@pg#C2sn@H$IX3rJL&7GE#dh5=LaXkA5_`04?21g2y?NzO5nbM=4Rp-YgAsU&9SE zSv&u}`#Oqfhy7XT08}=!5?O7^-9o5U{!jQE$I}i?WY~-E{(XiSH53GFMm@Oo*YL-5 z!Kv#j*3<6P-oF3d;GgSzzKe#AHj$zH@48D-UuYr4k16Hvm>f}h3YZmLa+_%~8jBbd zE6)EZixvRoai|}$YBB7MqHury@jrbTgXXeu37EA0PL-gjh4c@Mco^AHruW+s8a6u_ zY4-k42m!ucXS+veCWjT5bBWSn|9dT{nDwAHnbOJ>5gw+D1#5jpbug!$R=@gN$fa#_CDwdVC%UoDo`$S2aQ;2Yk*H!{`5gN% zFov-8ya*BdP@9a$k7@E)5`zwapQnUC?+6E;;+@I}NW4UFs-`!)?u5jQCJn(5 zvj9&s@kFnR7+s?4qys^%;M9@*`RO203VO5G$Ora6We~SgJf_i)U}5gaT?e%RBDW z203*|4&{~Wn(|Qc>c8G6#dj=<6}gFc-s0A-7rSgO;g?^^|G#gbfrmBBPDH~MY>&(s z2k9*=;wv8G128LykB~APJHci=MUT+_9mpQ=s0fL&^hDb`M(K6$)*+3m?26mPI^A1k z>EvFm?^N-#Bl7QXXWu$9N6qbn@Cg?U99oJ9W|6Q-UiVw3-|`{6C5tKzc`Z2tIze70 zeT}Q=o+y2TK!WeT6uM7aq zt+Ph3uLbm4+9DFUva=O`iWjP@Cuh?IA6cD_2y%mp|E;u$a@i5=**U?{UX8X z-@}072of7Z@~d#Bqtn?`cH(PJs& z>6Uo`V@g-5N-4tJcOpvLc*B?Sw(rsmc?jJtOS9}1CrY!uKKiYhMoaFln%WR8S_2eAI4vt&Dx^}llvB;v4Fn@pc~HoU&(H=iS6n!conigp zkJB|&D)|!Dg(UodGTwiWrtpRoWq5U)OvGCBU5JK$zS-Dr#ec3Q{Eh3N;j$$3I7yUb z7i=t=t+{o9%%aoP9NEw-#pKk`+L9}^F|R*xe>MzBUZlDO(IBi+vGY@Chec}8d|dxs zV#=k+sh=|Fr9R#WyS6N!9x2z32we+M{9&Zho&UwO=0~v?iIzcqt;P$zAIYzEq1#C8 z3-IQa13+JvGqgc$^U!)cPA$_t#Mk?;!tBBQ^fAlIDE#ous_)o3syN-e4Gnh(T@O&} zQ~nVp)Lk+TQ2&&{RyiX;=;>2vPEg!4T~u+W&&@&fq2z}ci}IT&9e`HnBVr>ZQz2JI zdyrz`2}RXKk>Zy+abGQpV#(mIm71_!H3ld4N_FJxW{Q7u^%zJH+K67mE5g?d7}H5p zuwY?XsFZRmI>jyMD84>3(-}$rE_0!}^`ECMOAgvCwRhu=`+Q9^#G_YQiEq(vf7db< z&o}tbcR=Sq(I7$B5iL^R8wPV^qHJPQj9H@OP%*icRcJ{hd)3r6MI~ncrkZrLYlnwN_zlFcsl{7o@QP`f(j!i1IrX%65mbX`m`(x z1&N{&$XTNK(a3JI6n-pVo25cMQ}5MQDBD@zXfwaJXD0PykS##*uc$MMhi_v+7;s z?#C1D6;kmNCXR?n(DR+@VvtgVEE{$kN&glPnVdj@<*>JwFjSr)|E3;>mOo6hrBsVa zF4s=@Ph|*zdWND|mvPk5-_Q=e@wLl!S%L|-8BM!veqUX4?PBu&n?9isf;<~jCU3n~ zdjQDWJo<=ufak@$hZXAVYe8(Gg{Is6JZ{;i`hqV^<95I1V&5ET6F2wF6T(`6x;-iZ zJFo|0on$w$0?Uo)(k8VG|Bfl+-*p8%A2MU7r*U_L$sEL}2{jpr>ni*>+h+}Qn##EK z0e^j0R$t#sihVetkwaE{=_~mT8K$MzFNs*S%<^RPmwEi3O8V^r;*Ul4(06sh4Ve(y z-7J4@#l}rKu;n1E|LNQ^luz#kn^D1eY^>aC4{uVlO0JOA-XW2b(4i>-ulC7=Ihn7z zc;C-E3;$;o7;xyAO_CBWUvZ4zfp$LoEhu^W}&7SnF%6Ng=v zHcjaF6Of<)x?1}7>BzrDnfD7GE}QtrqyPFHxxgrM+0(sAUYGuyse3l%j2O|SL~4~l z4lBCjX%B0gQ=Zj<#orEZ3;(}U*A)cUvQ9305k^35Q&fWDV^>Q%_=lG$ zUV;b)WBHv89r+`^=Ko@k5s9o1XKVPzw0k5=se$!&mD}fgZazD;kv9M4rgjIRJ}={+ z5#jQ=;;AOL;1~7i@-#D1Qae^nUnw%tcA{5ow&FKV0O7Qs(G^wjgTF;-%Mh)G(%M>& z5c_XS+t2@9Q{yH>~GfxZI$enHT5;Y1tcZaVvR6~>hU+)WgZlgXQF!(xxt*YKhKzvhe zTqN6*rr!07t{U|LI$MvhTr=;bqc08W=m8b2=Bop%g`HCvU)t)R?`RfiMNE9-#RU5+ zxY7;0f}s^wcN&+yM|JAZ@ypjrY6^Chxx=N?rnBA;nw@2hf}x|q7EXGER#?h0K+C~j zRkiiMXZZ|5&IL{Xdvib(WKAw;YP>W%Q#cXNl^I6uRsLsf5%r3gB|1|dKcxfmPap=} zF5n?qazYy0lv%`yQ)E7YV7qtU_hdjV_vju;<|&ZPqRPOW*ZRJF0V+r*m~~;1l3(W` zTMdF;v%Z+Hm>Qc`)GcqqG-9SGgz4SQ^r>2J) zU^_PPs1k87J{;~#68QwHXjGvPs70Ievnf)vfI3#>G0I9u(7lEdjT1WRBHc}4sG-n) zxao<sj~Dk%1YN>Q(<*P^|_A&{th%Q6-;Px6=nkd{UyM9O-ENWUAQz?=PWJ z#ChJw=>1RBE}#w0#d*MFGS?DYeEcGnY%if+M(#;*VUhq_?HEtwUTlvN1*A|D#mC5l z3Q%I&bK;I#P)GAI#A8~UmrwPlHP(~cta&)X?>B-ieG}gAv+A^ZG6iB$)M15N0k>%= z2_44YrB4|rETzy(&bcr)NG`Gu zivT#=8=%rj=wMNW%m;J|`VAD~5W2e-E@5$F*Lw4wQn0rc-@SNEuK4hq${K&`+n;&q zCfe9N<(3O)y-E7kNaZMz$C0_A-ydA!m%wO}D*`7~x*@-upc|$N`FEjEPYOJ0M8o4mK0SMDbaoi?uX6*6zR;Tjf&C3@>&#% zn{1x#aco;k`(_+*rt_~G26Y%vVb7=&IB0^3LVm>P0H+r=oM?HkZU}wvZ>13_y|?1D zGqVsWYa^pVvFOE1vlU4K`x|Os4vL-Iky=msGyJN~>YF)dw45i+K|%sDf6um|nUNgE z);ZDc`H7yFit+L)3f)J^N-74E$1F`JZz%uzQT5o~>WLaxDhvQ_0X5 zrOA4ds#UV(D5lt0coDG7sYS)x69j1&6StcI2NL8cOt=Wh9K{nfJ=bGMQ5BS7R5Ur8 z`IKVkV(9Xd69kSLUoz%_C6`+z*Z8Gjsp_sKmoq`Oou7?A(@%IUMz-#Z=se5RFe-_B z9=Gv7HQA`-=R;9e?a@|nATSo+H__7M9mnxQ2ehd2@IO|#kEG3jP@U8Qeul%<)oVV! zMy<BSise%zvskum3Uq_SNxyMzSP8fB*pkGfE&@ zEmUWCaoggbj1~^TpSu*dkoBXwETg15P0&BM|e4;4Ifp(m6b6Td}z8@{QVckF5 z^Nb067lFo_+b%CgMxTgd>&{4Aw)#0c>Cig<;s}g?i!66Uv*ca1ytcdhtn!0&&l^K0 z-N*t01PFAXK&xZzkz#-JXOT3zvPi|%arV1D!ni6DN#EKcAa2OF)bf0wd&S_p9T>V@ zCP07yfiqN~f(fOC{c$1x^Z4&GnqWM)Wd4T!NG`=S-9Vta0_(Oex^t{7Uyg_OW1T3c z^xCrd8*i^})FS)71P-sc{gO1=csM?PaWs89=d|`CrGs$++82N4>;FWe>5G{DDofGX zetF~{CQs|FtW1CafoT;;ksvCwXs(uKrK~tNZZ2D$X~h=luaZ2NNFl|R$7 zY;gDKY-?Qt1PBlyFcSnSmr&ZYcc4DB|DzA0)cn%%z>!qil6BR7zx1ZR?ioMdG=VAz zY<%nP^S&C*9*=*U(}&{0{^;*ICdRsI^{aj}_ocm6h*JzRQy}_S>Ki$5=&mS{zc0(u zjo(&lX=X-YuLuwzFiip}y&q+q>wAjs^=akisr(_zH0`L5AwYlt0RpEMIQrmu>eV`H z>ALOr{-P+x-j0Ww=qz^P5CQMWQra7(;+k7br17dF1yU>P7{8=U<+V{VdVH1ol>cGb z{0&ccWN3D}BL&Lm7o6Y74ljsWa<{~Jc3zk9ECCvSMw}D>-uK|(r&E77TM{5ZfB=D+ zBT&hNQj7P6Z3};$73KQ4aMx!Jw>=>+jRJZ0?6P@(>z~F?^ic#R2$ZXrcaLPBUtN^N z^<`OH6Ax8aMw02WI4>Wc{G70*7X%0p=uCkl{l;bK<4CISPG#B^&DnNz{qP6xr@LM` z+?f%7*=qs>2oNA}a)HVwlv*LL+jh@&W98VUsJU_B$q6hWP%VMzD{MGQLLXZ?Z{u^- zicBb!@DtT1d?_FSj1PBlyFiD{DH|?+PT0ZMvUmkiS71_!t z6~ACo{9Y2MjzG%(8$akxd3W!?12_KVv+4vU5&{GW5FkK+009C72oRWQ0+mlFwZWqV zJgwcn=$a@^`ejr@j>J;xX)iM-(5V7ZyJNH{vjcga|6|TY=Wf6Cr_VMzH5U6#fB*rv n1TO(i1()zH0eu-@ssjH9J5p3Q-nMyY00000NkvXXu0mjf-hUcE diff --git a/docs/img/streaming-figures.pptx b/docs/img/streaming-figures.pptx index 1b18c2ee0ea3e6c07bdb24c0a5d5784c972631f7..d1cc25e379f461095f932728eefc60c0ff2d2dde 100644 GIT binary patch delta 18403 zcmYhhWmFwa7c6{mcXtB8J-D9W?(UZ0?moCn@C0`fAP_vbLvVL@ch~drymzhd-XArx z#(H|yu3g=|A5(cCsXTvBNst|XF0nKrfk2;ml0RVMrvYei$s8m^K=!Kp;@S|Wf+)7F z7mCb!(on}fXhX3^#`a6!kI{p^0?`#>zS5RStALk7hNyak{up}s(&opm3GTG;MJhRM z_1A>u;lwIiN!e+*xY`uOTT872aQ5ph8^yIO78_TcQVde|(Q(ZP-L^AUrLI^EhVGbJ ztGq(#+R>JrhWP-a$39l33;)Uxm8(5uS|(A#yaW;!7)AC z(b|>2&oDpJM4Vk4M=_@i{#q}}rT$K17RP0Zh~M6$VnE;S@j7|sUwoE7M$Ktcu)_rj z3H;?-Z~z4u8s z&gReFX{86(0?d=WJsSJpf3OCMQMBV0ck1d(;+8Ic#T~$wb8c1*j4puvsgm61_(%$~ zU+|K_0%gB3mXa+mR_i@dk3g*KP8@C$XMntjViRK_A3MWUXXL0eOQ5*uf+U*JzjNEK zh{ErZ))1FWKl)IKzGX2auW2k>RWwGoJI$;w6mxcV!Gq#({VB&1lOVLn`tMj?UUw2x zXTPL3HStg?T;QK0E75Jd8%y=5;uxl(L+-WSz;O-B!-E8pJ5I!%Qka&*480(Mb+Hh&*!Z z(@QnSn`%U3_jU8h6zI8gq%|H&7F2Nt7X2v@`?}=85S*Q2;76tfEm$ud5iVznNJJvQ zRBwK=~k}eN}a1*bj!jz6DDtg3qgJ_B4JF zC=!f9H7un?uX~xbM%zc$%S}U+=M%Vwecw!;wK^fM-3!m6IJ%NUtX~_4Mb(#W{&hU< zRq6fxo-0#rqrw)3hj}L=hJS#txu>0Y>LNZJoLe~!+X6Ii<{k{v!w0wCMV=?KXtMxu z9tz3@mr+5UCiBWv8n)aNs5SPdKaYa3?S=#$X83m(Z=S^dyRXCC1BmDOaXiv5oP+?Y z8^-$8bG;xDla53yEL2ww6~jJ9A>SvBWu2VY)9KwfKC+Z4jbR79>o&kSdg09eLW9F( zW%Kk7TYl?>zPBb)(ZKb%4xqx{`tmz!J@he3rzL&{0p}Qp<&ca;=d%4IIAnR4)!Hwi zhn4q&Gy;|II(bo``uH|*mSjP*;{4MP!9c0@=?7$tbIpOKs0FQfSCY!timlTK_yHei z{kRwzR5L#IV(~7Hbq`>qYvcRkwEV!d-B#0W&ClV;fM*Ac1y9(e^)0^4BI-m6osU0P zzFTN2tluANKC)R+R&=SH_;mWFB9wXF8XaRns@mq|NtG9J;2Y-A`WtR~`w>(`Of0(s z>2Na!$Y53;K$AyVBi==NV*8_Vdo=@U08nka;Q8h6G4YrSSsX2!ct5-0{z-)0Yi} zVswEJ9T0>@P8W-8Jr!q9FHE>EP86?3{F}r_)dj_o>nF*;afj{P2m@|6D5b2i`kg^# zZ=!&Q4_psyZjY($JFH@w(pYkPUHIof2*b}$^=3&=O~MtYfaudn8Z=gR{>nTfyx zoyVM|6DzM7D>cBAA)frx7(xqU8qvc)%5rRltWNdUdKEgY4FnJWpk4kF(;+NDx&!eu zo95Cp80DC_xJxsr5no;j2E?hoHbFwdHfkHYG1dl*hM+)f%cJ24pT2p=5Tqc z@h@+^v+Vn|%2U0$JDReu2$lO~I|#$ocA-isLk?`JzJP#rpn>aU{R75@v1Fd(=GjOM z-KbrNS$F1?1aij6O|sO_Og=4NYve08LDgBYfD%oj95e=-4&+T{iUJ2V32GShCj<*l z4=S)bykT;~{Xd2zAO=`bAG=Y)&Q(D8H-0W(B%$>qiE&c7)$7N2qaeQCi))$wRkTo2 zy0;|x7NBkL<2>H-pD6E?|HDzGY*F1GA!JgW0fqLoB|sd3cbG}AP~t&wq;e(CGCw<;gBI6;w@d5PBx;sgL;fHkIg6p`5@*o^qe9y? zqa&b8ne1x}O%Tr||IXm*EU03^w&!*flLD&K3qTt@KKP( zzFCHKE@fLVjpSQsF|lHx_2(aa&^-g;%ovh$-vASM1ARVBLc~qvOV|w92KM6M&pel9 zJ15A)a{DK6dvl&2a&8n;@UV!b{4C9uoQyU;$Z+}QSbkuoF1Cw>@cuY~;@Z{>v(^3e z)?X4Xx2EY++ezaJ@MQeB+x_>3>s5FyKTl-X?OwCb3Gs6i|1J`T4^IJYM%R%KWr4nMEv zyGZ=x*9f;~iQ<9cXj+y&=$@83YN_IwWQ3`xt%M+^eT}0$o7FeAN&PE1NDjaD?qm7~ zWU$5+`w$EWG=h?>6hNKaEQJev)Yr26;ISEF!2FF?^z=+nUKLAKPD6I7-cvQNqyCxy zH3^-8L${Q5gDW7B#m0-z)nd=7a_xjMzCnYL_Nbb`FqFK28;igEd|)l&c8bDB#@b;2 zdrU@ro3Q3u=3qIvGDG9?h4{Qrin{ZsX7qs3FZX2lAS#;Qf0{ATesAgnAHIZfw+=m1 zraz#VXbT;LnPO{vpEYVU&Z#lRg`;-+@UNtx-a+)^u1*M~>`R=B)hL5iToJO#$(eZi zO?!}45MkSz_rIkZ#fA%q%z7 z=yH2bowYBn$@aR0%;kHD0{8ixwN>2TyK*-M?|dUdrwu=p58^2W;nk7&ers*>?xxKO z#)+(p*>O%yX_U5;{XtfZz5=z;)sz42z9jBo-^qtHD=p>VL)jc|&z!BO2!jSn&s>(oZ9Q`8Q5BpM7$9%B6Dt`d1li*k zborMzq?#y{7uCAoWH z@`f&GqYz#Lyzc~+H@{(Z5;@dvsOlLuGS~$mJ;2A|m%juFFPnB)PnT2% zv3r9DpY%{We6n|TG0QCQ3~Qd_iW*5=e`L|>6;dJ+3ylBqt0p=PHi_Hxv0F_#`#sQdeM4{wm#purG)rYP>W8UJD1 zz^j;K%0u#yz6e}-w_L0NXWrKiv-?!&ex<7FYEhTJeQpGd0m$7tkxH?mV9rQf!DS8!#;LDbs`nUgC8vck&E&Yfw!)!7?*R&p_jkzVbHhKF>NMt55K z9@DAGd2)=vrb{6Y=G*#@_K@&5+Q7IdQ?qqK)a|hMZt7;&~)TUxsmI z=lf+iYlU#?T+8k_*swnF%Om&N^>4Aj$$yA!6>7f#9v4zsIX!dlF8!s>#^4xYCNI1M zCDseN&dML0&a0ijoM&}HI{5l?2_z*CGLr2TNoGS3hyDaVeATUYk+4k=wS`LsnYS8; z7u~UZ8K!vq^Nj*>^1t&pWtiS2vTgOOP)3BGVhcK zJ2KCdfIP-|zL1VxR0?>`>!m=V7iXg;aJ&rP5?+Ck(I(=*&n;na+{?#we4CMHxKyuH z-bOULr%?K8B?uMS&M=W1*6Q@0%a%9h0?8cNIq2sM;j+p7Gpc?H1WqBd-3tQViJktH~yvOf&_>mP39eb z{C=+hUk``mwWeo0Lb&Q*qbQcHy-t_jfF;8SSxh6J<3|Ep%xY3KO-;qxBWI?AQ!Qf* zl}BweXPIy@Iwv7WleQ^v>a&(K(x3Dt%l5~!k-Y**wqA-@R68T!P#s9t-<_ozJ~k}F z{;9@PQ)zOvA;7C|_=F4ZjE(nu7U+qRTgrEAr8Zd;b7b1f!nnlLO4$%Fix6c8s;9X! z`v-+7gnJS*$6n=RI zcbPU7VJY&(@Yh+{!ry{Y`gGr6xqWVJWo(v%>dVx!bR}YWj$aALNwxmD{EORpcuW3<8PyG zB`Eh%MV5D5-b((ch~iQ3*yi20x4yZCTOOrX(3c2kgawaZz zILBH?S(!*M0-kAPvB45hH}M3tmCN!c`m5zz&3qR3B*OCm%R*j1+bs({_ZrgG~Y z*`3iQ3t#`xNbrN=Jp=v0VI#uBm{KV)8(%P-u;DE;Tq9jTS!W>XB~C?w_j|_f zKm)1;<8Rk7rV_srY+To=pKuKBos8d%c>}BfhZiKxSRp69KVOICfAR;bue=loQ|)8W zS2GxQwiP)--pu-kG<`7;Tu)^xl-;1}%XQJ$CR@sEpl-SGhk>yY-n9qX?@JCs*-QBp z79Ii_e?9n|=@$9MupN^Es>7SBVbwJ?Yjtm1u}=bunRiaVZ!x=?4WM|^5S}E+E9-{> zlHp0Z%Vw92%5q-FYWON`R$2kZs!)Q2W^cu;FpU(7Bi6#$Bnby^Q7gZXN_hh&;e@yh zO7tI;Utzh*@}BZ+-H0mmW2XBm0GxqCH{8WRu3bUGRNcImhZ1MIt2ygTmA-C_%C!3 zUXy_fIaqt%T(-&%nE&b2_;Ew>5dsJ_RqLp}0|U(4M%WJ+HzBv?-nr<+Alk%y$AzPr zJvLR+y>zuV3x^fWf2g9t@d~L%A&aB;Uz%qX8n``{uxT-sXxm`3Cv>m8eNx!|lZ9%l zWYoRVbnD#oUBsm1%4_^9Eu(DYnv{~{3myL{J7Oza@^#f&y*tsx)6YdO`+F|b!b^55 z_W_{IX!!WSp`0Mx!B!~8t0$57%p)sxzB1CAe?gSolfv@#aT{gjCFQVNM86Rs`D+sTkajy;tuO0GK~K>&jI)jcmiIuP zT7V2H_65mpx=W%NiZtP|snQKU(%!UFgjXNI{)~v8~qNnkNHt{lzsQiDEd=@*JL$cg| zR0;I%Ox25QYJ#KLJBY|r_O2c==dw{~sfvx5nJY}w^ux;B(IP_{Wa=L|aeKsSQ%lva z@;0sWq`VJDIjTL&lom&fG8%7~mC}?G6xX&lftX*8YtbUXeknTXg0=Lasw4*#dhrxb zhMIWmqXTFKJ|<|inr!)*__i+>h7|L!aB0kzAsk_E72jfL9_0+fpwhw(VGO<0XDCSy zefk}H=)z^$9ev612MjzVkQ;``(P6SdX||)3wbsLwwcbkp#N9SzT;w7(>=b4F9n=#C zonm~^TvFhB>-6%BMe1;)mgWk4=0k+p4s69Ebtmyg!{Ajbo}ApcTo76zxc-9B8}gam z>%rFRJN_xwc6Fb~Kd5$@FZ`6`u)$r_I-8pw!IfsyL&jw3{x5IK=AQ1TPMGlU@lI&* zotLSe0$O|xlCE-?Ms41hi0}z##N-DiWgGY510qouP^)-BCKr)fUI#0H2X&hOtEYS2 zohT)tkdLjGgzC=>f5*E^x@Cl*Ky8PV@UI=;Tv;>3u<&NXq(*17j=`U-%9Lx1SXbPM zuBqUL^}hhPsI1~pf)=y>2Wt^D!2+aNo+Me zQD8b;uu;8c&Ll%%lAR@`(gnlNkdsIBN02Pu4i=8zO^@!o>EQt+35J_Lj8S=G>LomQ)4T~diLu>k99T|Rm+!Un_|=h`pnR{kzzG>vt`BR9XMM+ zsVpP?yS_$^2{L%MEnD^usTsx8Hy&pJZdkrnpDjVxm<8W^h^@>`^Fpm&?jwN$6i_>I zTr#gkQTRKpHHf(F>BGnR-<$S5)5d(m3gRe3+~Ee-sO_b$i9|F^?CkptM7%QJpstxfNK=%1yUFY_q)%YiHO3E8|ZETpWix? z*}keuH~WnrDlsmxC#IJY-;9)xBKuTXB~Cq&_F|ftPXy#2Dbw^$)DpZ8C z)NH>oTxZPlX@fZlk6P6&aS}@b3N%e3q2bia9PVg&@d(UWl-MFeYZ5&gwptQeqAtN1 z1zl4lW@_ZNdjZOnAAmA=Yc;K##ra(7PYN4TkQuv>3-;{P{-dr!-a#fkiBmJ(SS{ja zpAu#;sereCs?f*Y3Fj61lTXw5iyWbUZQVsABZCL!4fDKzn3m=%4;`)oN65>+q+CAZ z*hQVXsYeNA?uvb@SW|qIkTD;HU1-c_K|CChpN$U<{QV-L8=;PsLG8=ANRf7M0qa8fXm5Ju8UWzfCQb$jo$Dtdq&z1=OJV-4L3 zO@jC(!qY2|_lipHYPm-qpcHKFBCpz)L_F8;m483kgj$=RAT@TV8bp9z+>E>PLOQv; zz02@I`Vzq_(n*5a)2QnRTeBL4(TwQ7hvQ@C($Ztta{EFah%T@S&4u9;xSw?$0E`*4 z_fceBKdHQ~xr(S;*-tRD{Ls1C_xG((rLO3aq;Mr`YEYbBm0SmY-S}sSXg=5KKqj0G z^UO=vp2rt&Rx3XdZ9n ze)QIuPhl+wojyJ1^@y7PBBH9GpB-S^d&&}W2nh?l^6Sw6deR%sqn%(nb9AwlIkR)eZ>A2Tnv5Eo(-_bNn;LTWG zqW3eoj`8HqyR))f*tn5 z^R{hp+oCWp(j@Wu#L9$ud{JkT|L)&pcTOn&H(7)H#t+Nq{VsV{*va{FRJCWOCNMxX zj)2jH7gEoo&GGUYcfRvrD93EedBeo%eEmed`d-q!(W7k?f0haH5jEK@f4sN8lt6eQ z!zgEPWqRK#IoqcG1#~S4)Bjrrb@~vfoIt4gvf_bY?YsM_}@%+psSlbq>dGXvRN0!iVtPL`-&) zdv(F8v7Rlw$}Tj}ooZpCBB)ZtKEg+98+|LvIuPxn(B)(g>a!G&r&8v}v14UZ)8f0W z>;1I=vxzr++u7@y`D++i9OoP%U^zh)NnP)={Hg$veHSeEsftZ$3eY<{icI9gl-22n zum$`~XZ(H;NP%~QRba=9Q?Lg7pDyDzJylqQ1A%x+|JP-u7HzP==Y>mwv=;xyz~}Aa zs%5bruRj@82}C_zHm<#g9a3t0&`Wzg2nn~@-utfD*UeZ+ni90u*y^VqKSuAis0J|D z3@iTB!^}vFUuP2yxr){5*50M=!xf6oUr3u;@s-$;^TDP$@_XtGzK|$=>>B#i;k!isutc<$cG-PFId*nnYHr9qXDT^RY&zadxnlgmtY(vp_7s~}Z{CF3sU?QZ_@xW}$aQlp$#Ob<;rPr5~*Il#3emk}P@+`ap zs7>ZV=$oo^YfYxQBiuS5x|Y~v=d4oBOh2ezlNfBrMb-B}jL&?E%$%^VD~6oW)9p_j zgJ=2vZuw6oct;<;9vua7hEMf9c^ZwD`?5B!Ri5+9;)L3K;3|C0$4fRw!3@FZk2zIF zj*`lR8?R!uh9&=ynGYYdyk)!~LEV7$(E)$^)uy#okcxz@RJsU2a|c=d5yUk!;>K^| z8PM7jDxxy)UDBP6R9o;-@6WYWOEIC#SpQD2UWrGz6nin7#C1l1I6K4*mILS?^`0~s zny*85BO1s)uLbWqfOIX|FNQkNgb47Fo-8)J+j0Jh<6@Ieeb!D~h#mS8!|ERj!Q|KL z9mf82i{&{%U)~28*l|X3`Wac`TS-{m!4{{8;!n_<#(+(Ky?vUYT;KdB{q<3aJb0xK zieMU%6bdl0m8!8*HIvFM; zE>=U;RGL+w7V(b|g}7_bq@_l;8EX5|V}cCNO%wi}C~gs`8NpWRmLd?X{t3!Wg;oh7 zPC?nJk^6p)-BoM1PQfLrK07%GGhf55tE3ppTbQdJKwZwkf}Mx|5}EI3unG+f86xm+ zepaiaYgi1axtQ#iu*q8qN>JOu74mfyMVZ>-_ideU`FfQb8j+;v03r{r^Ex9Rckx4! zKjDI_M)UvzBd4Iz(Lpdj*RDZW#c(%wft~Pq8@j|)awrEBpr^{%6(U+{ITXSZD6G?) zJU72cdKMEe&;i;DXGk#p0RJ!JGbcPxAroP8?1&JetJ>ZmKeSM!-j<@pf?Z5m*_sF$ zlIo_h;-=_JO%%Q*1$gwj=o0M{eY*$=XdFFx*GO@(r(6rF#SI+i$dmMg4`~AHWJW{Fm5Khd^3e96A_bToIV0t z)3Av^Ja*y^m^KQm01Zo?V(q*X^zY4OhjLdN&qvj%kx`Y0u-|`t`kyU*ATCuo0Ge?kG$D z=GtSRbY=d0sV^W^QW4+7C>gnjEincX9)_r|nH33}40jXQUbb!%E1Z5I928bQieCFA z*n05W8v7ctlcBi*#No~6%?;!}Ilq-lIW4(7O4_YYT^po^Q6|o2G+y6MbJyE}=f((w zR6_&9gCHLVo4@LsS2=B447lr7xITA$goo6_*`kAC?E%8q@K1Idf6u&d4MHqAj72Yp zTnfK)K5kt|Pqq7XZk5kR3&nZsrlUAxr8i9q`4VbAj(BO7TB6U)B` zbUkDU=jdJRb7_2qn^d))0BwnRH!5=1|II~ppp^`vo&U~ooaClQUu6#k(a5Zd#I>yQ zbAWtCIiZf9vwu=4hb3NSO!@a+Eq@I2)RP}qNySHu@XMqAW6_;6r%zC*^f?q#N0JhQ zB~qj;EI%pP)HII1ypU2eP-xkx1!hu5LW1-dH!Bfe=<@|5XAIYcS6`ARo$-*({{0(5 zeP0~1K=?0t0=%qV_I4GT5Qj{5%r{_a4XV0>0W5};_fb>kAJ;G2!eW2W{C8M9yK#@2 zFY3kZb9a3HN!e(1`pdGZX#h6g+1S1R`lEwegQJbDbZ7-3+45VIrGBWHmWG>fLsb>N zlAXTV2fe%u)lUr=5=)lJ_vcbVsqW74=yv7I|3tq=1O0bW3^~n}y(CkV-raop7r(z0 zh$0*HEwR8p!8c_tp$W0MkZ7p3#1J?}S$@?u{bcF2j9IqyF?dEs^wmY(dC_v)aj+jQ z@ms$H#4t&~>9emowU4cSzv4;f#xl<5cx`u6)tSH#oa9UNkExUr;K^0#E^R-C*z-OI zhKjO@YQQk|)Pq8R_gbzCjeNy&^H>B&?!MTHmO2F*ZZ5zsL=E8;MUti(A+D?^M%PW*h*LFU!X3POiR$c_BbhOSev1yH|yJ+Yx7}~B7^9#S_gO*+j)R$3- z7+m9%r&i*2ktoNCv?npqZGQwTRl}uDiok0$9iZo%2M>mSD{M1bf=zDqRvv#-5MVB0 z%`z)TW@GjjT7sTMRFq{XK~m(_a$5AvOEpK(6q8mdYgCvn%lG;VK8-j)zP{+M3ewa> z)73csO_{C_cN(Y$1uag}8TMhO$w+iHE@Kw9E9_`THMuv=089!sjLgWpPqJ!NEdlNC z4Wx-qhp5rxvfl-6RO4;0C!zYt;>R*zZYiqA9^#T!dnWf@|P+vn|ZgnIqa8QzGHPZk$WDTfVy z(e4|DFyxnIv6NDNb7R9%&+9k_#(hWp1W;b`_4yYf^^t+twJT>uLqYIx`d3-pk*$%z z7bhrfRw99ENw*_rB?MEx;Vp*1rjYI?iV_`6&QdVmYCC8_UVYqwu zgaj)~)57?{sL{>iC;77YIz})%`nCg=XyV?~Z67qW@QM5EU;JFGj+L$*Q3g9+D@@1_ z4(A2SRUu!O$%*GzzQ^QhA3WrOwxI@*_ZVqH{4Yk5uY5#db05eVl6if3fX)qp0dq(D z*pZOW$=vpTM6i=wyc)5~y5PxY?Nu0F;Qo-{_nLl(N2_d_?<=40jhQW@bz}WklD|?% z8U4wQ5@@R|KAYUJohT|GHZi!#W%$ts+fc$2Yvj^!|IJlXpIvGVb|IOj9CHATu<|#7 zi^E-9+I+N{??B^us=*M`E&$Eipp8763U>hAmghU!)Fh4W;DjTD(s7#T)UN-_!X?;s z0{XO{JjKqj3NzHTnKQCZ>dK1dc$mO%u-UIm@PHr?l^nmmLQe{qj`9`wjXnueK8GiM z@PpEm*cnQQZtcqiy0lA`gyA9-+Xyo>1P~6v`}KWY0)A zYpa~VQ{i?OC%8S8njDV1G}13)mIn;qW)NMDpjDZPBEW=t zkw9Y6+KY^&m9iRer0gRoEBd(#8{h2^>&QXkd&wSZETHgd8U7pFmqkd6UDp@OQpHSm z_xRmi{CM)L9t4;h$jFb6epGdn$_C8$Z^`CY$*SsjNNu=D5d`l;&x&Nl02W}G57Xb+ z1QIkM!qlvj#PVMSP^%85V(`+bAk!p=s@#rRo$dcBfL^l#WFu0(bH&F9ketWZ$@cX8 zaJ2|89M*1$&SyzP+r6DYV)a8vY~G;xeI7oxyt_ii_fz#RE5|lBk$~?MUzJn`#Dk$G zJXPCkoEI5NYW0iPK}9My@ME95FGJ&R(?;;jg+t%hFuau^!K#!8Y7^}d6o2p~qk?Uz zKAW=cxHgmCHzGD3ykD9qR7gF@FOG8>T=Eq|y4n{`o(JU?o1a$%|6P+6L2W{MPT35I8 zo9S0vQ_tnBf`tCA5>1lv^}g8fZtfa!zLE!ss$_GCN0^ZLp$rq(tN2RAl6Ml1QGWaz$kEe0A#>Vp z_&HzGW(YA{?4(Y;kfE>-p@@%)M)QE%0XOlV3g|1+w{*(*U|{8dQr2+IENY#U(pJkE zUNB^R&KM`y;%;ByF6ZcH?P1zxa0Ne8SE`CMVwefsvvQKn7CLTNO<-32qf>K@>Pkev z*%jpbpD2nVnAbN5HvSb_HA{>3MVOdvbKKY{b-q4FT!B-yK5;`%qCSe6la+d`9g-l_ zn#?Qe!*0gVG%)FF|0`=`KFhI4FZaan#=xstAR7Jkrh*245j)dVl3X+G!%lIkCgTBP z?go9Y-L4z=SD%=}q~_WTFt|ZpeY`<-p~OO9ewwEtcw0Hw7&3D0%uaUiGOv8K^1dYQ zj*$H0T#6G&Q%bI*7=h2tt_(@XEGxkKG2IgPk(b0@2FRI1Q)jyjKKSrnh|qS^n$)Y6 z#J}IX{I+nWK-{L0lA83J5petyN>;(%ao{nIwbvT^F)b0UkVbSq^XL_dZO_e*fsI`S z`bO18;hf)fEPVST(N6Nh>Av;R-3z^&Iw9(cQ0_hLw9(pqNJN&s+YYyvr%hSF^VK=U zdb^*nI8d1pvV{K4iPC`EX#<6G-u`3QR;I$#$1HT(vyYK`DlXRRWaG&T*#m9#8A|Zx zh*uyGWu8lC-N4z*V(+?A*aZWQsOs6jp=Q>*~7S7>hN4#D+LLqD0RtxStKT{yfHyu4vK7D+dh!V6?TZI(^Dr6|#=aMp`Pyf2Z^si+#|!uZ0)l|#zU}lwK;wN7P>3}Ibd9vxj`JY-%zS7>2+aN# zLNa~E)b#>OTu4=IkGf2EJ$0NU5=7jLM;M|1d5GF*^XwN$t&Di4eGOjjS|gke_~@r| zkyc){(%;OJXt{kJ>4P@A+G@mjNmKmzn!9EEin{(r@c=+T#WF7D!2T8l)Kv!oJ%-*v zKw;KJ51R9v=Urfm7qp9=HOfnXBGys6a~)NL)(?7$cFs>f-#_LJ=Q(jR0VQAr|7!gq zMgpUhEtcZs(~+JIk8DMz^A0k&_C+27%Fl#=dLSSS2ePhLrU54z%SiFu9+2F>v6cRlQjSFjB?;`f(%Hm*a5x=csquwQ#zoFjW_{>+ko*Uy}Ezixl+5t0Vr z>nc?9w>qWJze-bX{#C7#Cn4=G`Jg#`G?T?-y3{`ttBER-qA@EMUG7poh%h=%B-xWu z%4kscbMcp}(S`l0?>O-F%<|6gB6amH#C?k*5+H3gw%k0&eNgZB(8GvsXL%^qoM06a z-$=kZ%wfH#-zzThzL5SYUF4N6IaMF|o0aT0T~l=!>*n8vPhE8DWVDI{FFKBK<$mDF zH>`lkoDYrlGYIl*FaZmUxY_=w9Y#p(#T7W3WZzDqz}YKjPdU+*E;n;vv$ z90tCk%Y7yZpazcmE;ap>BRZ}Wfz`hLj;AGOs5?@32#5ybb;3&=0@5XoeRG9?9CBmU zYvyVpAUKKFY(Gu%{Acul#V3lmH(Ebx2q@SzU^2HRw=Wm|Mey>x9~@#NG}G$3bx!Q1 zz1`9Ms@_KqkY73!lgDjj%Wu7vZgj-_v1&*QcxQQNsZ_!yUDR1UIBNS@c>`lXw$lA# z=v1_CvF>bje!Xhtc6ryV_}?4-!Cq0O>?2s;(EYgK5vUzM-M@6b3%lqz>A7s_S-(&# zw^!fJJ;iu-_2N}-d~cEK3;%=X1RXs-{?SFJ@8>EU3xLi_tUt)N@`&*%Exyrf0?(cP zEgos&QhivqxoSx8wOMWQebPyICtaiRspRFr##-F7`3wJRMcpggUGWsy`rSzBMZ91^ z0FcBCVChe31wi{LJ{!L^hRf0*B7i*Dz<6dni_QY?R)4G@ZvoO*u< zghHTkQ5r0Q?Wn5+EWtw1Y zJkm-Fpt>~f!9cMEFA7wa78tHQ`1XP8$4}BXAt0Fp?y&t&DVR#52cz+$P`ohIe^8mXDg@870JGU3jP>zAZ2OX{wh@ z&o(B8WK~Z7FLi796kB5k>BEJ=uxks*j2nD>+NJUhmd%nI+b9S1!k}6UC zHfV&&(D<3nY=KPZ!ya$06w46srk2fE|CO&Sv6O=Q0oWD<#|Rj8{C;76n19Cglbvj3 z__<H6Lmd%}0NK@bIAz@f&wi1O3 z0oh&bUpZI3eO#b;bH;m3ea}(jWA8EC>r3)}R9M*Th@;yPguOiguIbK=da5}vMqd>n zuuT@pgvg`J~@Tcj5>EiG%`BekNa1O=rNC9Q3Gp9bLJ&GKoVfpS-K9 z`eYUZSK=oh6Kg*Zx~~ue8Dm}!#~HaKe5fC1;_o^S$JQSiZqR6htLa>V|K0XT!j1$rs+Gk%cn}k>V0^B%f;>ru>V$4E7nv? z*+yzsa!M{8$D@HysaMPiG2%Fe1L1;Hpg#X^hcK8!bNOO~amm|vj<9k-oEy1yX;kP~ z&Xd2XUrw^f1z6p7KzVugR2AFuxVC;eT$*h@$X&YBAKdG4Ejn6K+>+0XGL5(@ThaJy z8OfIm!`z6I@#^}i+oG&NTQAHq?5STqbz=>MLdqpff%Ox3@tA;6#A~L@oISNra3Qu*0d%Yd@jpjw@{7JsQk*_b!>9e!|O)w13*{R-2v(dJgGCrdYedpBfn@U zELycq%VCY4A9$mkx93Vg0lI{4yzIHaBoFt!?X>fLSZsIaFE~}3V^yVwhqT9USZFu{ zBFq=1JEvn*O+5aA(=e zx_IY}{==yKhCRle-vh@p+l%VOc_fA5ymbG1m@&p1r58c(8*juGY{q)R?I`Qjc3(UC z#d_Quv6obj^u!I^gZ(C3tO)-E*KcE$3;W#A7m7TJOB@FG&Hug(xB0&5%D)|da@IaG z-t%#MPZ5hP@(+%e#+%??=c*UOnL0o@q=T%T-Oua7v8UV~+ZT*%B~aCu8%j?3!0WnE zg?jij-pht_sD}{bcFNYb#&((F0V6fT9f#51&T=W4-kt5Z3GN=pvNc9IAAg8Sfq)J& z#(@h|Fa*ScQ3-y$)PDEVoJSo|0QL=br3H8isMFjVA8PXeM~Xb@mL{J9RUI5-T)VN;797CQ>yJw2na{y!^4og zt`OTc1cVF$bvHU{p#|at0zXAR)kD3PoxjfuVSzv(Cnq;H59_asa{343-zqF`MBU{%SeEVF`;#tM1Xvp8N>nt~1Wr=+c2aDuz_oaP4mNVOt zWRzz^RagCLQgeui(3%(HaOL|Ehg4=Sp$l?JbOhBNJwiDS8Y zO#Tjo@|Zrs+-)|{(eLfSIq}0!L-aQzU(^ttVIPS&^-A}ex2C(m^KlP^`(NOfaL?|m zPzeUEk&=Q=mcllR!|$A8@l$Kmsh%)jKh+I1b^B;C^)%3QWPMwU|5G^l7e&UGcOzQi&YOpSLhh%RwWa|F4WQ4{GWN19^sM2;mf1jns2@w?`Z4la=$+4 z&&4-B(O@+SIv3r<-I(gcXi^Vp&Z03PzXqxA{ym(0J1 zNjyPAd^U6N`Ph<{%=%v?i2-axRhUS!tcqj+8!YEiFHx`d-~{JE?8&uy;}sEE*#`}N zG~n*dZHjqW>TKzc>#GPnBPp9Ah+W6;(kA&kYOg0;zcf}ZnPl>X=I8xHzg0@0iHO)))v+fR@3RV)ltlQ(LL+R*aP zkxo@aSG1fk*xDir(@WJLh8*s(nU2|7bJka9M%&}@aN!h%b>L_hyvYQS93yiicHQFT z{Md&bdDNX?_%m8jCZsvEI#`Qb3^!g4fbj%rvUx|Hd35BSoX)q82NI@f>BWTaqpg-6 zj{DC3!gwiI^sY_g&4PF15Slx$)gc-f(tHi@lBCuzem{(LRRJt zT+_ERPHp{@jkCSw(hw4;FROiatL&$Oftm(3RyJvB0XaTi^s4)(0RhWN%?tLeC+fc# z^hQJc2ERIZF}=lc{U*vDs@1O1!yQ3Ij7T`wJk-|*^5<=&<>ayDv+pGpk~-UB9I1>i z)HEd!5Ip}?7>_shno^kCR7|8@5xM&44haR-b+5THHwT((4i(;2_W1qa*C!)O2HT*1~-MM5?B~K&>Ipmta$p=UlN&57qF?qiPx$ z&0YaUOVd`>%^!BOZ8mJBjo#5E-x>+3xJt>aJvSA1Dykr?tt2oz_jY^Ni_v{mC+~ar zT~*y1LW}JTpq2|j^}W~X#rIS!)@$p8h6`ANAc<>H+zZ-Jlz$Y}xZP{MmFeHx;5p1# zXCBUX8?ay(a7=rRHOK>XF_*<5v(sj!lO{XFz*dVqeHnMGjEn%at ziEkzLzPu{O{;GvDW24*&eQEGW;7wJlQG7$so8)+P{lJl_4~L|hD_@K^3#4giuKr^L z_OZc!>;xoo02ONtnP`Ii`2}ySgCT6?09&NN2{Mua0h%WQ2c(s}BJgx5T5uu>m_d39 z&_zTPIr2(|h@IuamL%YSeFu*vfkVjqFXd%JXc;o+Di@rR!8Ys%NKZxq4R@ZU`=>IM zwg}y0rJh&nq5Ype5_Za4m81X@GevMO24jUT=87k27i6RXb2y&@Hkhr(3uPelf#rvV2u zg*TjXAC<@CeTdW0p(s4Sl(EP#Ic9$m#S|`2%2-O&3WjrWsFMNoaI2TE=mo6d$FpexV5tQ#vh(xABKGJBT^+Lq z1uQYjo4%seB0Qe~T(Cp1CIh%)YvE!BAehQ_5iJCR!J)nvdI}cJ1wnEqz?-hDTj(J| qfI_lEgyJcL37Nn}WhV>3BP_6agP*6Gx~z69`p)QLFmvYsD*X#N?MS8o delta 18371 zcmY(qbyOAK8$NpIZV72=B&CrSkP;B-?vj-5p*sYWZlp`PySqWUyUrmGc`l#t@2(qv zJTrUNnl(G-efFCh_M7kiH{V}OGIYJ%cD+V45J-wI@f|)rn9CffGrt ze2a~GPvP7B9XY5?j>tOZRG_xjtJ(^;5qaXso+s@B@RBt-i-pjP%#_%7aJ$457&nB^ z%&Wo9_MLW5bW~43HF(P$4C&U&rd+ezK3Y&C#E3-(m2hDJaaNS$<&qaAiWV zC%vZYC+i?+iPREb(OBq$Jlu2Q@y;*mDU_QbYo;|$xO1Y@@%|yXl|wIzNQ*;X!8*`< zdT>6fH0_CBnpZg5J+10E>u}e2II;H51%9b6tR%J{cmqrW;Y$A`OF~LHZ54WNs=SXw zPohNX2X8c~5)Aec%WE#34A}B_IOnoj`+>VX?*O-i`wwjB#D-c+FFD|VDuU0&~-RKh? z;H=`;pXATyXgM-LpSvWyzf2KR^bt$)?`QUqQ>1$7TKdo5Np&Xq%a5Z9ZJUFs6-_UF zb_zqt1r=5XnAo0>TiH%a)U7!())adzkk~(*3NpBo#1sl~qKblt z<|CdHN5YDQ0=EfIw<^K2@8rFS=M+?czF*|5g<;V<8RDOt2rT7B&;|PlMh)1H& z1W{ysr76kpJgI68^u&f?zq<_1rgL+@k$f5m zosd5xd4#USF3b@rvT&7!w(mNS!GK{7{%bP##i~ZU(&pM$3;n0D^M>zMvJgz*E0&Ig zpn$Gg;!twr0ZFNZpoeIQ9-J7MEq{hErbAFAhQ4Cl4#HAVi|Dtwo>hF?H{sW)ri-=# zbPeY%s=S5s1Di2 zqJrs;ko{5jK4opC%hv`uVp|}$UbJ}!l{t1nsvUnm%}RlC6pSGAQIz6`TTDkvUdqPL zaTHWHy{lNo^AsEN;ZYQG|Hf!h+ZW1whsFNMln{hASo>!~PngsAhaO$4{4+cNXCND6 zt=Ij@?@VI=od}OZLi}&J?1<+WQ!M5@Uy>zo@94>S1DI5OTM98uZ?vksIDMC;M03l>0gP%ioHQ> zASI3`>2oxa4+}KyI0I6Gxy;qy+Ez&?EAqdCqpClSN3c3 zhw;*sj*G>zOR*_&t)Se0H~x3;anu}++ZSkGrq)mBV;N7y5mru+O?ACYb`Y4T{?;0G z+2pR$WS#xg7xMn(>CAW?CVrouJ$;(95mmRxJ3eh-rWv-w)$0PZt^J%rTNC4Kc{^CZ zU_Wy!Mf1|_aeME8^8PAwXdu2CwKc-qKm_e_$qw^6CSU<0w087elFxGPg=eZ>f4JjaOT3FO zns`_~Jp0so6W$r7YB?z~n8N|F6)%qPQ-jOp+lJ)%|1@t!#wVdiWmEWbb9Sh^nz`J> zqz3nH(T&aKj%L0FVbNI~K6p=yuZL0Iypgy>B$G$HvKb2KVWL8-@}~G5-7S`aEE)nn zY$;mPU@2NW%F)g#QW8`VF7|V*DQeXJ%(qq#wN!{)uKFZ98dltjF+Jqp`x5^)Ci74^ zQ+n~w`I>Vi;^uZX*+}ZT+-@Q>-2Hu_l#bbNwvvSBzNjKW-^ZQY{?5V{`l?~tJ>i3Q ze*N|LFDan_xeLo)Se1RcnLNg5epdkRQVTqh_IQbg=)SF-WmQ(-c~DwfIpB*4YgjTk?5 z#y5=Yp=~cPyUN#}Q;M_td8>%^Nyo~X&ycn~F_TyjIDbN-3!X?ot~_c}C@BQ<#hPLJ zYY9QhA!rKrZ!Nk2gW^MatKc>R6$#w$y#YxLCeIk%)wO;;cRC%zTGs<;(3vrd{oLwo!NEq-ga(-^`*oMDN+J+7F6A z_95*DNhe#&Qm#c!EVp+Ap6YA5q<;GEy!CcJ(Y?jT=Vis#qQQ=$k%5bQ->Q)zp&p=i zyOPLDOAU~P5gw%z|0{GOK6boRWm}w|&%%o7BsgMlY?d(j-9QPIl3s{YcZ#v-Mfl3l zJ*V|idkR614nZ@Ll0>!Z_veDB{zb&ff!fOpmF5nES#oswy&MsV8n(Yo;Kyg$C2Q+h zX{YK!@{Jn3AUf80EBG%6BYk)zes)8bU(K-5URR|)1R)>*w-Q%5N6rXC^*yQ?L6!Ve z<==9b2?-;6U*c`K4JmEBT;sk*hC-|ZB+5Xk+5=CbZLV)415hAwm{OeEK)osH_wj}8 z_u^gs0)7lVt~<;8RBOZ7Q=9<)kDV@=)AV|b-lA8N2MZxzy#(hZQBN2|Q6HLLc+f(^ zO4epLPpv*!(U0RCV5~tu^5{>3QXjE zMi(l-diP6Xj@7!0P~tfUdMJOFoMN;USw&Mie6vpYOBRI%!hLsV_qITVd6hKOp+*24 z&*jsKv~P*whTvdanmnwY^n1MMD@THXqNWlK$|ub$0_FLP{)sKg5*U0UoaCEiG5`y= z7L`8i9Tsm$*VG6C2sDlX0?{OHeSDMn_cJxXoX{1|^{ zTn|Q}{6@Nnw4YZ<>!AGm&f@7fu8cvot!co!dK#7U`|qvz#)X#QK=F9qioY$mSQT5k z0JB*LZ`;UY&QB?0VM6qo{|Nb>eQ$<9~?d(dy8=HCWC9zV!|ZuGMEQgAh*m6V{P= z{%mXa>|w|VAPBFE+Wqmn+#&8?f~#NOM@JNY6r8Y6>*;)X*}W0^0&L9Fl4=#qp%sgP zKHrLoaEs77vEo*3DN-Vqy7+D83dPC%gj#R&_j;FDHs=xw+*MUh6*sB27TLF>S%CIA zZzZ;On|u=>a!PTr!NMNvGeGN8h^E&7b13%V(Cp4z%UyyQ_~}6FYit%p`kD!GG0-mI z-~4xi6m%6)LadNS#vumLjPBvhx+~n5hBq&1Ls_(!IfYXcgTa+Z99_Sm& z3|3`pK`p^~KTZ=_x7CWy$8a z<&ESb{5$qt6z zkapq2e><()X`CyVntB3y)dM__i ze^L1u!fbA0J7Kw53DL91({-~Ite2&SC0nZV)wAKE!WVu-{HgcHCqJX(9ffqObN1twj^f+5n&B3I)ko=u;?y|q4NS3#`Br^N73HR4?N-hVj1}=e zWdn=)JPN1z|6EsLv z>3Lu90!7fm4;s*C?@mm)bD^OmktbtnyFPCZ*J%3POb4p~oz<=$Up8imE^K)%7C1Sw z@bfH!ye}&+n(KBzi%{jW{ew{K@u#immr!0mlJRRB3=8m}*WQc3yDcltOAF5sT55zd zBIG&Ss;tdm+L}ZrDx#}{`L-4jqjb$9I2)T~5RJ$3n~^oFx6diDLIvDYTf*kNDJG8X z7WXm%I>?@-*J0O}>3;C4f-KO=PY74vJ_s#iQSnkGwu6)aUTr7hT9Yb#z~8Vvoc8_h z=X2)iVDWzf@Qxm!=Qm{o3>Ejr2r+9z!T6XlVhN1NXcbp+vRqP;I(c_mvOV$G$gyzZ z25>_7k=&mW8k%~^VK3;~dttQy-UepT1a~5Yz!d-R;|aPEwB!pg66PZzZe@M8(HB?! zGL{{n!UW0#M11qkr9#zLz6bVWgrX4f{syNs%vrbP7Lz4#3*`U04yJ^wO7%l{@>Rdu znHEtWcTV|_hjXRB z5^&7#oudR#Io0>0Ke`xv4vdJ$s=x@eZy4`oF|a5a20ONPA0kenluU0s?-#miN^mOg zqpq#Ue{vbTJ#MT5V3%hnVwQ*MzS2sy1gofv z@RiF-aXh*Py+=_yoQ{c}wga)*-KiKHinps z^=&6N@?+7H$hp1uPfGG`9?Y}s!vgI6jqXPwg5M83${esnV=O|nwRf3gnTQ||a$=4i zZDOMm0Z?&Hj3Gw^sbKo482IrKE?R1^=RfWmwD!Y4>Yg2(CcA{7eKvmMSIzug zK$zPyG-JB|(XMwmR8BqWb_Q+z9^2TCxwOT%#YI$fY;EUft=;(ucja&Id?X zROQ#UmRzCg1w@y+(wyW8`IAbpuzIDIBebe%y9Z`;~gFltJ^1d2biBS*b%YU$!_Nq$OA+-y# zf~`08v8xPJ5S`R=&j=4cuhSai-v3Bx1FjX4%#EYr4>PGJd^(c_>RL3{2)%@e=CRu! zP+_5jI*tXzUW2I+K4>tJKe?a3uF?lntrI^z@c|yMp0^>&HFhIEh_Y(9rW!~?Ps;YSMudLvNf`nK!pK$1$>o*HBZCL|qDpaBtC@GEbQ^@kX1=h0nATJVD_~ddgVI3VO{3CNluvKz&IaCDHvrq$Jeq)Sn~{uk;>HTeM1$AIGDw8ZFsOQjq+O zuJBv`?w0xIFGpQ|9K^Y9Gf{&9Np?C#?rDwizhsus?y}tLr8nhZ{hZL!CJ55>- zYXoY3b(Sh>x@WP zY6g-4iKj$cRW}G!DPji@CTS_{e=@ppACmV$*GT%>L89ZQtBNM1W${wLiO@+kK4vYF zpc9vZ9JyMVKB(#Ufl$~*uR$NGjD_S4f-L#I^?qi!b~c=6PwLY z2gKu7Xrj+)Y>)@GApgkfiDz2V60PYkr+g4`lyY9ca_Ik_#s&Akb{BgX%57{wiCl3GG%ebAQf%1W^i0{s3=%j9Tn?X(uw^Yo^k<>8+ z96aV5yF~UrMaCq@Zp0yJ`i9~}9Jw2TRpLzI2l9#|Q+PCTWM)nc-tvAou zMW$&CrF)yX-=l>mvDDb2o=qIyH& zTG^whOd!*%7Wiy~$TB7^66h&1Qw}-Co3r*pN8u*Ek^LG+J#tj?mogwO)zaDz!F(#& zZanf-SLCaqicSHk(!nXvp*4aNAp!4~PpN_ya01moq`mfL%6gpB!lS~U{qe7!NPF#j zMG2#OWk@AyGlR-nGtq^VTs1(&QjRzA0r+5@4fg|My=9igpFH6}XJDz^o|Jx&r1mLpe0R`{_T z$k=8wtFT+;iXtn!CbV@bOoh&g)ASAJ1Ak*qA2}8t&wGv}&uFEXqr2abpATMhwUHbq ziEE{_@4Wj{QZw1b^0Z46PIx(@U$A%A9VKEGdk-%Xz!H7pm8SX)|^VVwf3FH0kI z0shgyL{c`tDEG|+K@m-P5&Cj>g&NaSQc=6{k;GpzaBL^tyfDIyOSaX$ zC~7lk<~8z{Xsu=|BKcKQRLeLW!QAS&DSHuZTg%McZuJPZsNZi zJ5Ya{v?=hWhvs7l3>EM?HmUeOU>Gd^a;c(>1;he-V+=-|0BlqgV)?Nkc%nJ2 zG`N_Q)95&vz>57J&B49O*acBk#2^_5tT-lg_1cSZ(HFBvYFzM0%*@|YVy6jdRi?bW zmZ{>@q>re#mWfgc& z{fB}P1sBBF81WSwd`2hQ-!ARi|6<=u>THjP?FK-qJ~b3#@w-O6xbaXMCD8yy!%FE_X4 zkYrZJ%9|c7=cQeBL9R@K6&{aN4Q+$ehn7FVsIu-+A&^^BXayZ~<4I^xJMFU&<5 z;Y}mI!t5w=j(RvL`B+Z#gv@&P?b{;;AU&uu zpR5o_A_9alys&x$mEgps&ew zjv)xuEsAEt^<=v7ys2HNu^E5W(aoS<@@=7k#?jya6AM)Cyz#t1^JJam{>%PJ!E~rW ze2Kl@O3BW2oP*03@0?cI;K~z0APYO}VnxDM3|*civ@-5<(5D@b0Glw$c3B6ssJg-v zDZ2snhJv5#TVh;;BQDwj1(6nirFgzqHo$uh-0292n{ob#MI1^k87N;0IQ`b0$QP*?LN6;1ia^+2VS0Q{#y}!GrUAJRi#lN#h zRJgCfXF~3!XH;SH&C2dajO>W0fh+Rf7LLNTw?%n3l$*GyIV|}J({3fAhn?FloD4=3AMwQLW2UL63ZO zNbfQ|OO}&x|MC+2Y+EzR96=B|w9%ZQ5L&Q4Zjo9eDiOmBfZq~@uOw)v(d&`5M`&&g zBNJk}GTlP1LQUp$b9@%j8^oaa^h3{Mm|M|b(+DBp6pW1n{%kKL=l9Lq1I;Xq|KE=5 z8tRsB)c+oYyT~5aA$zxJKGZ)c)~(`_yC@AEsv?rb-8^;>+PXP?(r-RfuVg~Sr4PBY zr%V%9D9)EF4s1Xl59<)ED6JByjotu1`EV5!7>J%dHz;lBf?b;qO)F^^M|djOy$WVd zczjE5EvsL0cUt^S7|)&$Y`xCEOthDnqs*4S^!yZGvR&ym|fB;Np7-~0B{ zX&iE2QGGjwneCg?;$P#XN=p9N`L4by5*elOv@xo&>Sz2gJY_YHu?9`}<2o5cofaFW zArNCaTI;(Q8Pru@IGf6!*!lDfuQ%`1-1%&B(QNB@ zR(w<7Lhr1$;X#gaPDiS1I+zXQ?r%?h%tn&5;k)($xMC-^#wApi<(!4wdWC_ zm<_#?DB<7gs%&#|I)p*DzD8d)vzy9UWX*)}1l;Z>a&Br(TitfO1J<}nm~LWl<({A-iNm*UWdyp&cim1PxXD| zN}Loo@ktNllWlCY6um>g_TNe@j&9q>C4<>K0*pc7qO#-hy7%PY8zfHddO1mPmfbAL z2i-=oe1xziHz?y6E4u&ko;R(VPNAE7qAh2VmWg%1u2$kM09c1a`7xIXGW^uB0UFe= z{M!8?;ozrlkOcWduayx_n-RPqvHR%@o$X4Z8w+Y^P%+Mc4?;f+hwL|{>p$CZ<*6xM z+qKLx@kJSEe96SjEE!&U+iIr);K)5MlkMj5f94^%QFjQPXM;JxC2Tnb=b6zBMuDX6 zO!cZGV%x_SWJu$LN=97>5V1K0!f2PSIP?cpm4oZ z7VU_D|9o|3?4G?Ldl#tf*45i81xEwn_JgT_w%vrl^WxBmp7@3jbho;L8|| z5;HJibQtZELg+wb7m;;fq`y-Ca5E~_>GKR+Ua*v_3;x#d76MB(F&j4-4zVTEE9X2Q z2Tj$6b$5|yos)cLzm6%ZUgRCnA9M{1+CS_3kfG_f?Ft^?418T%xC$w}8|AA1mN7&se?0I&GnofI&MG|@J78to@ShZI1D?(*>%il@ho`#~ zpMg>wAe(*+nlf@{>TCPX>SY&OG3F^NJ-|~_TZ&ik_ogsfY*9xgFw@G<)WD}z!^n!e zM^mBVN2hqPM&-TAgXHkyOp*L_CSy3&vKz<~BdHb6EcK>`1%V%|r!_alk1%*NHLpcfAyDG*Zd(KLWi;FE$@RK zqk#ciZ3{FeaU>&y^K9UvRo9;c*mTm~Cg~zf7*uv&{veM~K$!|M11&GutR!@VE#rvs zH5p)X9!yKS$eZtEqYlm?;X4GZ)w6HHeG(x4(L)3* zgHna^aJbn%PWe-oG*Qx#4)|joI3y+$A$ysf);o8)l#6jMtrUsT6t(HQeX(?%7G0HD zV(d9vvbAh)g_s{<-b7%7AKN~~n7CQC9=m{+@_r^`e1@^9W3?hUR^3ltAlzDr;m4f; z3X-Ek=UTVwht%KnN_An+60twTR;Y*q`Hk;wqy;l_3dg9TQ!gUEbOMx+-!l8!X`E+n zzr{MhQHwGSJr$HWaV`5OYgZzbOY{}KXP(_IJe*I~B&4x>%byQD8$M)CWK91{eIQz5 zw2Mh1&RHvIet2FH>-2@)I^Z?B8Xfo*#tXi*h z(++t%JkJ}iPQU#|?>W)foe+J@jj&(ibxZ60Ke`j!Ne~hE=cI?>-P(W#KNNY@vG#6C z^es1o<>~Xw=4kA#M%5Tn)oW@Z!BHUh2p2kr=@VhOzDPr17cZB@@MewUYAIen@a`{$ z9yjPlZZ_j6D>=o2tZu>%H#on;!R@n(Sxdt~iVCj4z7>z#BZB)}HH6)YrNk3%X4u1q zplqY#C}e%Qy?IpG#O&!<8eAuFVn0r4-09*U z>^SiZ^7FtL@qz1BICV8unx`0YkaH$Rh~hX`zU0|2H*x+!Si4%vVN#wTqH0j1w$^Xd z+B~I=N7AY<=8x8Y)hpW`!L&3NonL@NxT_c`pBYUF(Czig$Dlj9B0|KzD$wf@1yIqu zpv1SFj25N;-?$vriYteKss{7r*A;;wJfdHuUeS zx;#Od682whA~Gbm##Wm}dqU>L?{SHVE?U&qiE5E*uxjI1=lpK{bPQ^{F*gv4s{P^P zhc$vh9@PHw;zKVJ1H(NHGtzJGbaK*ExIW-WrCHA2T?`1vyZ(s9wk^XPk}wZs*#FF& z^$%$1Bb%Z2>=DQXSD58Vpd0irvm-trH>WS(k~7wksj1Y)5j#X!nrj=^7 z>tkIfZZLk0Ch~bSbU2bfxeB_icpKR$LJlGnrd-ZD#%&MY!Ez*$#WU5W{lZty9?rQc zT7W%6U=q{@lG?m4-66+&DyoaO zJ}X!Q;HhFJwx_Ywe_8ag$K6!>2W#6j^%A~E9wlKfLiq&ud%JpD)Q z4*Ph20%UV!X`#0;g||*>p92Hp)%6Mcn*Q-X~f}|~^hZz4V zC|Y2OG97h?pN#CCs;yzYn{CdHA-^lJHjm2qE6;(sc+@z7Jx}1g+mb~rB{3IA$MnsxC09#)O6jrT zyI#~}xZ%0cbq?YJSshF-U4Rv{&M!x#xo#>9N$?r#e+g9d?lT0jSCX`#{*NRMuOvyl z^b$s7?igfDFD;r_Vj8&Si+?%Ip3fdyY^bRVaB99 zP5TDKRR#nc8f9ry&D80kPO}Uuzl;%XPF&esvlTlS($&-?Nr|5=X0*r8iu;S}h+34b;~S(k|8Lkgf%64e*$NCCYO-U4hm(+H&tgp!A; z)b6A|-~`y7h$x%?-43%_q#IOD6Ko50-WKfS-z|2P(2ky>tDLccmBAO$0+!3H<5?;) zsJ~*IWwD8KvW?4ZMxN8@4_iKPfBtaTK|A@yqMX5f2vw?VImfviOMR#WV{~pWJ?2q~{^ym1EZUj4jEy%$>pF zA$m?)OK#MITmIqRfJc>``kTHWYI#Z_cmV0&zusHABkvmAp6^@Mn#f^?BO}EPY}x!; z-RQoS5Bg4^8>*!Eove8JGL}eozVRZ>rLAdMq5viPX2T>>H;xp>9$QzxZhxo4^!OX$hKj&18ZXni%efIT~AqJ<+t-*PW$->iKOQPC&l; zP2)^?$NHMQ6iduPzJLKNkDt2{E2W$2!sY}d@Jr1hF@1aJDOP8qu3iC{6-ZIQ?0uZAsFbo>A4A5~`iE`V6>dAQA5 zXaKus{O*aEY-O#+>NLi4@1MO2_&PVId*xetY*h%%@OR%F!MErR(BxN1p zOha9B;|F{@gFltcN8@3J{Ie+<;QlU)~_T+j$UHHQQ_QZq0unNZl$ zkKZZuX`y_Z+zq+2qLTq}{8a3OL}XsYa+a-^e#ucIfF(B;7rh((Hk)p~kAfN998S>Y zYRkub#|#wHzYAkFRuvlxeRzh^v+k0a2)R{lVuUQXkY%pOd~!D-mEJOGj_5XOKUob& z@hV?^Mf|eiK<71k;MAQV&Y*NeXILp|qfx5|TO$>)cAr$)cYQ0zMyv3dC9tybW{}ok z3EaUrZnTC^@YIwt63`9{f|Qn&bLj{)!~J;#U#*f$(e?am&k2G z{F<5lv-#*0@7pvTKtpGmzX)%1OSAuay^Q%ziVDNJ9U8>+Rqgisw}`MWe>Uf$*8(;s z&xJ6<^nW2)W7;k)z$>Lqm}G{aUKiUkATey1{HP6D2y%wX68c|fr^>*KVo>+i9|c8V zulZR-!;pgl0>c6V;a&m%3WQf6z5?kL$geQIC^sB#xyIBzw$mt#>lydY1c0 zWp5=Ow{&E0DYQPT4rDPx-!F$275Kb`m}TP%<2ZYISNe@uY~LwERd+I-49Pq+tP7G> zy4G12=pYO~mkp|lc~1v#fe7H58-M~^Fwo2Icf&C-P<9jygggra?GqeuVEYg}r?v^@ zY8}4dPj{O5;sCum9j(rrG5iYP^P)$3>lTKRZ95ck$V7nW78#2O|LvF1fk%Vswhs`) zZH*Y$oNn@#*z8;%dB{XIZ@C8LhsEtVF&FdGtY&ye8WqMyLxz{~vCJwd$#>f{;Z4drFozU{3~0vS z=8qRcg?n+jR(|vRmQ93x$2hr@oeFDHIEvZh6PZsCrxrvR_o_j4UWRT`47<$TeI;b+Sw)g(s{nXZkcJ#cH{{vUReI z0|vLue~l8P52AgFn{&b@3n9I(t{(etLa5hKA=lGDQEn&O|F!o>T2ZaIwDEo0PoGuD zFFnChr-D)Wn6zZx3*@!Mwp)XJz5?s1VB3?F$Tp?>d&*h+e^7G7&ENELxE>Tgx-v=_KFI$)ZeI@qHmPW~)GdD0I28 z_nB&o`0VXAnYX^UOAVO+3`wNUEzJ&$fcNhZAiZ88aVcvM)o(L=Q)l-s-rlteIpQwC zAIChoLKBM%5W78swm-7jh=J9Vmlpd~j;a10oMq=iH?}Q%#GnkX-+g3$(gr*9c2Yi> zLV0(_6$VAwAw?m6LQfp?!YW%AM8_(J>u7Tw9LIhb&{pIqYs<|}sJx5S3%*Czi7GTp ztnLN$9GrGwh*>#DuAi=Sw&wfrJgMKI@m?=(I~IujE?1YGcvHV<(CU!m^ifO1~vAS!hSV`_I2!9fLba|OGB=^HQ+-P|1czINL1_t`B zPkWmR_6sQnx~Z-eWp3SPki)=?eH-;KP|xJjCuRRmf^%WdE$@pMKYVD}ju^qJhh*P! z=b7ftP@O(UHx;MC<#habawV$-RPaQHzyN1@qR9k2CZr3vI@mOv`lLTf`OSCMJDdeP^1(peq8+cYPCD5~6${i24qmJz-p)T~ zi;_YosL)Q93$zzf$`R#N zm|N_T*sbY@1DVK$d3N97GnRJKmB}$gmGvW{wi{T$k=^YtS+bZXfQEER&R&|I08yu0Y-1 zNi?}!Wndvl-ujS|!Dg#Fpwa&l5eShH8ubyJeEN`^2rhTZq5=9gZ7j6sxU{yK@+yQI zd3z58FzIbKkivTgKlB=Zm@6B_5I#Ul-3B+=_7Vs6niOwoV8;?%mv(*;<3_MYZqZ{s*idv!+ul{%Ac-ml;?vgp_4jOH*At} zbKjQF37m`8SApfB0p>p%&gT{aM2run{>0GGfPeB|ZU6AR?KZ2sfj25P;X z{M8FSOGRt;R#>=?y0_cnHxHTK+iHJ114AEiXxI)AuHzoc=7(?nz}UOyAPN4Z`N9Lv zs^76QHE5k{!`U*V(mTc-F7in1Sx6K8jVQ`fJIlr@F<;o!RNX%Jr!`P$$d2aGi*vZf zr-;4}FtzzX;iKdp+;5?jq@J?n)W%Nqn|zUTgn1^RB5-i7Uo)eor77}v=h7CsYl%vX z;7G=?!p`yBc&*+fe{N9G&*|Phl5#%7Gj3sI#t(hl@8?Gy1qibl%QR zbi=vOBk)cfI$ErIxExGGbu_De%%!U*VgU-f|cGLcCqb}8edFvGCK3z(uC9Us z2<5i|U=5v7rIK;sh3R*flplni zHHUUjuSmG6YOl4-Ms4hE-R^P>y>boCdR?%yni0}IvZxcKUSLiojA|D z5PCE`4jN)VF>bf3D%Z^1i0CS4%$@Qq?KjWb)dOU5`+qz2KYFLIRhrE%pD$Rmc1$C> zV|<6Oj_#KwZvDX4S74r~CGQCN-V{mVBdNxs8EkA&fc`h;JA0ckbGL@tv9$VbM$R#_ zK!--0CGFt+I9D|G$M*LwESmXVbHx*?`RH?d@72|rl{nM?9H4#lfqeUFt)RAa$h$-N z_bVJAjs$uG`U>}Y^J({nNcfR?+-)mGF^2Ktyn%sipiy#=k82(l=Of@Fzz%E~%Hwcq z=QN`k>dUVc_kss0{e!jN|0DnSL2=jVJWI>?skD)h{p~h3W7jBK-+rgw*x%IBir)*l z+FO3r340#=frHZ-xvCh^akpOm12|c{oPgFCz174GJ*s2N`ap%%rF;?3yh3-;sF!0L zYhvxUJHfUcx^o}HN8=|hFyq*I_K9r^2j5x|!f-w?+voQ#ZlD@EsxfzM_$r{J3^AovEf(=xV%SUnvLOVoayiYL z1p{r8053XJdcf6o)z*vUAuxD)w!;8L1DkfeO7nRg3>4LgPY*WFDzfTehuDq-6$ahK zC(D)fHbrFP4u%k%aC?_*tmWgcP|Mx~Xnx;1E5y{pU1p|p>pjFhN?7{2Xq)ZJD??J1 zUn0uyV6UNe80d29|CyoSMg)L=jb*c)-0cuf`n;dq82$w-1QpnE5A6KT0cC!Pqf2wG z2(f3vx%akPG+lWoFcFBJ*kbV>QOXTYj6;SVsIt(;!k*)PQtjPy4 zC?D0+&x=SZpsw+WP*)u?ezVvcHMs;xfja}96d~MSrg{DedxBAW7FeMGtl`y6uh$DLPW+h1M zuyQNzNo>#XX@Tnh({=pZtC>Xw*u?=@Pqt9nR|9)FY`Yd3))lRqex#+UP9IwrcQQ*q9&ff-r)q zBT%{e76y`b$NbMNdH?E){CyapdI6?it(Lzq&~Ds<9nH?`KPM1<2OVhyG_azG0O7eH zi&41OD9>xO2k*7!z|ql#!_C^<{i~CMqw{|ULVl@sEA4RNw$q-8X>~}>{0g<7ds9*O z8hW9@yS2CF=XJ6j-T9SNwNXuwBmVPWx~1^NxHtFhvD1ciO(NwlUf#^iydNfLiE}8A`vrWA zT3u6(so+zz2%$_uPXh}I)zDZE7)9Hf?VIFIwhJNa9`Wr03%LGX4M7iM=M7_L1Na%b zYj3$U324;$cXS<|l@4#tFGnf4Hazxu1#i|LNNAP+8bp#cqakMfh>qgo0GNz@|91A7 z;^YjsHoqXGyysFXqqGplZ>aX$XGnS;PUr|3U9_JYE?J^7AWAs*bn9MLjPRgIu z&UXRaY>O-!-C3IxQBN88QWKYd@Mgmzc+g3IRMr^OkNosiR6fAC_7ry%M)!m*gVhwHN}NT+l!t{ z9G~xDq4#mok^3H;?%{_EZ#>Sc$b!u3Y=4)>TgSvwWu0TZy`Yd+ov}V&%1~f$_^bL2 z->%3dEL^DJcr4IWq)>yk)3>VZZRE98*SF62Sc4bJlkY>NK&jrVTF>jUnrjR{=B z&mu(^&wg4dAbQ1y$1ukrC@TJQeaEuY#dh4o2-~4rMCiXYKto~}W;p3CGXPlS(&t>T>RgNrN=(A;^lJswuy5x(CD(834ayW4CR{g#0#hbTEvOHwuxf%AYbI+W6V z`6H@yx8QvFyxm$G`ll}}=FP6Zu|rkihqB?*tmUe$T~deQr#OgioZBtbYuwwp^*pPG z>NVB6Nk6*eF7K1e&GziQ@^shT$|A)hfs-zqP5c-2Mg7Q6bjn8f!LEqSu%=axhl@(a!H)2l1GpqjVny6+^{qi6Y2j_>3#+5UG%oVCr*hrAs1 zZ+Xl2Ju}%|-uoB450f?d&Tf_b?HfyYRT%4gCY|*=Y#`9`-eZsRlj>K0Rw@8@_N}N| z6r|eZa7U;5c9cq4ioWmfZx)%-K@SdGoO$#7=EqMyJIAtCZ}5|}RsN`=#F&w|F%bSqO$&YTdGnJK>Ad}qcC1F>t@OcDYcxoteH)n-ZFR5a@eQZEZAHvh=? z_Gs_MdQ@1D(laH=!EFAvSByacD;BWz_8pr% zqpn&o=j}h47gHy04y(xdH&8jW2gu?v!H*xd(q9k*U3DDJwHKJmNt{PGC{?osVnvl_LtSSL=CQuQ)2Z z?$6^KkrVBY{XgeBnE)bpI^QYhwD zDB*uvCH1AO_u}t*cdpiK|NPRqw(kdj(xj<@hb~QVz1dO0C{p|A_5U?Lz+t~VxSY3@ zofCeD$n=MGydjJ)rkmCCIx&h*uLM#|pN%IAnut$tt>YD#*2K#*{X;!3AJY-5>4ELM z!jm^d3r<&Q;5A`7V~r4~0|{KQK?vLc3EZ)T3rv=?M$g9Ql#11Yvy{D1agz?Sv zgN?kAOkW%j;tox`rc8ew;R2KGow%k))$$5V-`d10#mGASLK9F&o(n?31{dz>QO!W( z%U$7u(^oa~O39$?^aWN22vF|^QYtt7Uo)?m3}WXX1A`($4UpC81ySbL!Ye8R-;)Ha zSx}YL_(7D-1u8@A2$;URg*SnzDH`mk=?1O5+A{Dpz8FSy#XyXh+6uG@u|f{rrY(sO zWluoL5R1almF1>Elo_`Hl_8cvp(|_4fhcPNDMKtRKv#C75TfiNP? zUK44=G$Oi!9n%AV3S>~GV^D(ItPWz)zII*%#@^}g+j*TCzfL#q;B{n7m|hB^mIB*b zK3R|PoJIFgMA%#q@i+@0v{S_-5$mttewDUju09& zza50bNd<=+m}7@w?O_B62;)(1k%1d_5&(#iL6JKO`9cPq>Y=hA40m-{S0~oG#k2ap zA3hTy>PIna2FwNfPC+Yv<8+4Aht}#rc1A=3N(m!;@0DYQ z*a)`y^z8~2vrwcW0jv0PhUfw+zV0UCWrV!VwXr-3|TIbnSfG^-x@2O=&}a|8(v_N zj)V0ZcQ;%oX4_QLWWiLHc|P4NT~i+WfYTC{E%@nMiRpa(4{fR{Ay0VDFIlk+1DeM1 z#(H(2dZ;CnGd=>8p&r~F0_VUM9DP?BDZ;Wd|X#nBNqRH`A*BQ~P=oFt^2 zLQRRGBugM6x~#|}+bZlM>?2m9id^P6r(KcWmZPiGDIOpjz$BH-2D%09j#MXCCfcQ* zg8V3yQWim8AS1>+DkG90+4mo>Es!d`!o6z2MRl45lG{yv0%0-|nT4sQ5vG}@k)|O# z9D^zp9ZA7SztEJ*g^%lIC`w51L}yEUW?GsQns`?%R+tH;xQbt9laB}Qh>kk%;4LUD z^!BOu3HI^!8~>=n_AwhjvZ>;h#3aYG$Hd>VAGA*HPOfs++8BOx(4;=mI@0>5lBQaw zLZrr5$ZI&wW6bANI8-RjG22qwn%FMezRWXL#!i+_dgV^$7_oosy@sFEwMF{2Jlp>5 zKM^=d#>>R(!o$VWzyq=W=D^K#%pA$w&pc!wYO>Zv{0XhQuY35@kU2HuVocqVqA{AM zpt|O~lwn%1c)e1f+N`Wm+p0*haRS{SiBXSBb&A^ zscyZXKRr{#E5^|xS-oWiQ3c&5xh6&1W@HtKmD99Ex(+F1sw5ZkdHVT>{fm6Ep6Cz-qP3%Qu-37R z=+EiZ7*6SlbZB&7v>lpG>Yn>o1~Tmo%mRkK+wPh*PW)Iu_*q@fu<6--9>AArP`8fN zuGl_#k8)4?Lixf1ClKx(-ih+C}1MSlB!!yHD(Xm)^SYDDNQiBrnQVJ4&5+DgV zkT$(e1Gye2|}LK{M-#X$oGk!CVA zGMF;EkV+yyw>%n*Zi4s9QxZ~&XxPf9%g1d(n;x5HMJY_8>gWvE1gJWh`q_(^#oGSZ z%wK4r7Ya`&%2T`PShlCfz03cVjv>#Y2+=`ov~Ae1lCsGAqcZm8;BGH-&vBG+tpw!=ba_@B`;oEo}awd){on6wKGh0440>BwhXHG$8HL5ezj|zb!nRDxfp-8i8qgL zVl8Xvl&!UunlC*mJ_*K)&phZn@%we2vof^0*uCixGz$?6H3%p86@#E7X!t?du3E%l z&}419s(rWesDO%Ns<*uR<|*AM-GYU+g~;Y&P8&aYzs+Od zY!2ZBA%jrXhyPv7eMV;EA1k@Z#0kvoES@~Cz#GxuVb{@-*x|fd?t*PidowvTbu}BK z!Jt~h>sHtD)y|h!4N#?34UYj|Ku$pC`S-Wjg|F?oX+~#(Hpf3N9XgMl8{g7)tK+(0 z0uG-IUYIuS+XD9EK63D(qG6ve7`yyF4A%sX12*2K;mSW73XS=4zgfSTo$VGKyf;6jPFLmydcAa7nT`xx*-f8j2+3}gb@4ne+&dhaAMMS0*Xql<>~dI&J^}BKp67 z2tZ~w4%pqCw^7%2*H)0{HFI)cG%{w6?b z?e6Z(%f#g6<;Cd5#^~f~#l*tH!^6bP%EZdb0H$DY^Ko=H@n&#zBm1vO{?|O>7H(#) zHqP!gPL9O?<~1>O@^BX*CH?oH|NZx0>vXrV{69}}bo+0%zzj0|`-O>xk(ud#&kY{R z|F4%<%*nyo)xymUJij0Z|9>d|Z+-vgIsZkkZ0+Rk1eSuUjhT$2yM-%wu)E2>)d{lx z_vrt3ivQfR7%Iq+**{SQNvlb2SLT@)muNgQ%%pL7~5s7M+@?REa(QOHFXp6S6$ zd}*kah@i)`V6sz5J9}~cI23rBT;zXf-Fyqo%l+zRSbOPpk;8Y|>SiVAb}*LxAufwX z?M(bXT0V-4aWJnsNIA79_^3Q|zigFPEsgvB{^Dd(2amz6tVZDd_II(w<8;AwgaR~- zKv$)#w*1{65)KlM9|an?%d)%a&f|6<-+Iw@Y7R2+OiW|Y`_tmGd!X~Tw)FrH^kC?B zo~nxrP%s;cOI1@6&{#0^bKp75cH8^487L?J(f?UU{;qaBm&pH0q?&ePh$twVM#+I@ zwS#uZ@3NC>z0D&znz2p#WU=HABK$GudY5|=~#pldZtrx6oj3%XN zgQf4>>wT@-DJAOjxE+qzLmP>(J=pPxG9mb4kIV1vteO~Nv?a_-5CJ)FgK{*Z=Duj) zB>|R!m-}yAD{-cdE74-%{DnCk2(6|`!Iu<`ujkoCY?J(KwyVt!#UT!N;GM(F-bj@C zihJJu@eUJSiwvQmMWbq?Rjm38k1kHOo7BLyJZ&>sNf4KzX-YIvc6hDPN2S>|Of}v| zC5|ZYk)-Xa53$H7=xrXeTp}7PQC2oN3RFR~`M^3#kDeHWhS?XL`Hr#?^zkN5MUbc{ zm=s0u3JIsW4J=NQl`htFHSPT>1RX2$!)k zCoD`}dMqSwy2k;Tfnl_!QNZ5_2wI|YoXxOVixBI(qv>gy(rmPHq`7f_@R53_oxV+N zm)(MNbhlxC-ic;H_@v_(vgUJ`Gi{mWaDedmbeJav_to3D39KBv#w<*-<9s^;q(1qb z%A6W!3@y}c*}GlHJ-tbCj5;^Ql?cz%dQP2~If2i|(_Whahc|G})nFB!1}m{{3IAXd zUC&3v7cx=<->5AwPT!|}P>}?-LDSZ~#+X+E!=(MY+7Edbv?{vVjs{s&aykE^E3v=t zg+COs+MJZQu*8Ml?$lE2Jd?>Se%H=yR_qh4we^BUa`M|xzG@J>QA7+~k2^KDR>DM_ z_S(g(_2~W2xlEmRNvBsDK@_?^`>E?=Z_TTYw;?U3V0~|~sEy^?;5N;_vNCt-da`U< z_gvM8Hv3xfu?tv=!DY+F?g^RD|Ld$zWxeg%TA_VRE83XrTJ4a z8O8Is1mrpMJ;!Tfnc))PsQc3v6B&=4n)kG-k#%{G&WkeNa4%Vx|MjYWx~BaC&!J(2 zGEPYPN5dB&N+9Kz5Es?(%kUCe;E`y4So#Oidn#18v z_^N}xaL;kvI}t}))%bnv_qCpo* zewbmE>yj_xg(2av(iF+LWE>Kq=)5x!sR7vyUhx-SD5Q6HbtG8#C?$cvDfr5-ir*Cd zOLIJ<0Ai9zN0Qk3&LYKsQl~9I23X}gPQ?iI_rtAi{R7A-NE_|=7cFH^l3#J8$)8f6!8uo>nl=F$5YMnwNhEyYI~W*yh?2inK;Q5}JOm?RvHa zG4%jOs)O&jnGoJkhMA1`;0PS+2eBM|kd86=VfUtTW3^YZ*=>^AOV*V+*XNAmD_fg< zR<4)o%_{l*z($aeNgIpf&yZT*eq~4r=33jxKfOEf8`q)BI%a4;Ye;mQo(Q!NHcu!) zH`N)8nDCv~(E7O=Q9>eqWfb^i@%uA;=;kz8Y=N8Q_mQz!3xK6B>I{AgX+|uIn8Zl> z1gk>f>-p-h_)HEvO$k;}LK=JkX{2jv)|IqOnx128nL)d!qvbddGJ!3Jgje0NGGZs7 zus?3HB2oPHQ0T*@L*f~_JD~=^zP(CYS%r4YLp!nAL)M11M@Y4PaY^nl=Mzbh=8Q%L zi$KnSiiSNy7C>e>0Lbx&mX>Uge6ogH_6Akkcj)bS9?uMRx(Sn<$COaakOurwc(T?5 zntG1`4$#ri*If?Neq-s`vn)C$37a*$<{Jm~K){I)Sol=?JzUHKf8`m~*!93|L3T|A zdK{|X9Tmm5^%4ROlwhT)wAvwGp)PzD1)Wq*Ho%4vJvx{ik$9L3+F0tu#27EKAh^Z8 zWB7q^&+bq<^$5zpoZMI0fATsK9W(!&44JboS)C@W#hdF0>3P7<$Sjs)Ep+%{a_~R9 zk*C>Gll9LlMCsDv)#@%9)07ZjMWU6AI61C*(|Ie+3q0v##t}Ob$R|EJ?PRu-pfv`3 zcOIM4L1}O$h8InOVga&jv7ll!%I}bVeB{aS`gcEa0Ae79EdzkiJ>kgE*H>$Yv|Nqm z!cZYhgnX}}58o(Qg~^DA$fW{TkopAkd|;WjAfOn~hyb_`15%};zuNe6;`lCF`q!Z# z%h*Q6V0Gy~11VgdScrAsv`>U>!x`a>`{1;Z@E(x~xTlgvjv+Y#W_!UYNFzX{tWqXN}9q)=)U^V#?ns9p9s95qRMEZTsvUA4Ds&dWAkuc zhkpRa3b86hvB|P`;w9hk;snGgaITnQh&+H~16frgS@id5pFr|gawF;G$29-Kyh4G; z0F=qb>lCY}6XA4Wk31-dt+`$=`NP;7GsZ$Qi-q{Yyf2NSLBFM~2Db(pFiEX>a(f7L>vQT~M(BPKLOjYfwe3qi+mu_(wj?$3L=oOXQm;8~OKUtAA~r>|h`;L(~EZ()bcll|AorVv_#+ zW~;;wVZ7`=>V+e6AH@am{Q11p@C=2f$bUUBw)GWGEe)1&(mEMdb!UGV6ZF`@*5ktx z>K@u~f4W$!%U;X~yK`nz<2h*l?XnWbv^?<@cVJiCHx`e>^_+MDvG3po1;OSJnB`5l zyMJ}L&C)YS$@k9IF8OW93(Y7^oK+$#o+4Ot`9J)v?{8<9JGWOMw(t7S;h)W?=am zg@HzKEgr=_Uf@2$Pph^2VZ(2|$EF8T0GIbbKc#XF$rAACQ#n=$$Eb=ROHs_}_d28j zV{>0N0ID}k4~uZb1kxxnwe1UO5RjJmk7bQC%BNlQHo@X}Mrc)BvXjXdGh*RC!GTFi zzt&u%qe1Mu_x&ykZS+$4d{87B`ESfuORTD@%S-bEH-yGhAcQ`s4I$YsdJR`B1Yu}w z!vU)+;6+OwUBr7q5agx22%jN}Fi-}CA?}iNkRl7jmt9e9w{6{@`2rEShZVp}>;viZMUE`e1><^1VRz7kC zUM^T7a-{bp-2C|cJo}W|HECgPr*PalCHh`s6%%Y^;54~aarg$J3jip2MnIA+^Go`3 z06sbGH7?@}F@6<*79eWm8Wsn!^_}0pZV(N#-8ubD6zp7wB$pL1g?^1WK1ThOaGMB# z50yAE71z;u?;|Q#?640NtA%z(CyKGj11kKH*DF@MZc}6Ii4LZjrKx|lfofveXdJ&t z_6{|KOX4d|7K9*wf(2Z`#=Rac*Fc1JOtySu-!6ypYZ zD5Ko|PD6xV`lif19B9JcpaDeE{TGy*pKfV9K8t~b=!iwR(%2a>QbeCRM0PFwP< z)`xVw$=HPK5_}#iBf>v^)5bdgz-w~DfZ=kTk)eu&j2LrkSGI{~iz=5RDuhZYBbP+s zJt9`wBIgy3KJkq};VguzaIWe`am1Q$MSvEkpe=;TWp`ZDR(~e;{T^{dtWKd%!qw(C znvpZ5Xygub_54BIeIe>%p$adOYF<_}VJ(r0DS@dV!KYybU2oF=y@rthH?Mk)9GXbu zY5H2d1YXpzB7XO=Ochh*7_^Je*~v^$K`g0`tL-kyPm;5@We$-dEgljEA#388E4vR79qkJl-j_dD@OI^dH z(Jf8P(B&avNBP;2S?(em9cCxx#s#vduiGWEJ{2jk9hl5~X30x@w=y6~35cdL$SP`U zG!1P`!fXi&#Txo1d;vNMb-eAF;xpR6uKw!6WC&;CAQ=gWtj31{zOW?k&~8YQp`g?> z?nd%~(pA3mrde?+NNAe$yKG7o!0}ixxkO<$!j1$L)P+lzX9-jrwY7^!h*BFlJ;JNO zSA;Z);7~iet8_c>AJ0;Say|16i;*EmTMQM+sj1jcT_K_?Je#$$x7oM|E6s}5YcE>L z6ttbL_a#S0TQCmkm)V@Md`ZR5oGXa%NP-pwc#lq^Q z)D;+vycw=C<)9Eo(B4K)lr_S5 zydzz>^SfMqW070mq2cAj*2r`Zah(XgE0ERK-vC19oF5|QKI&UVqL}aD7e`hhq>I9R zo+|u`MER;-uBEQKUju=@j|vp|t588+FoOq2dwirpRr06No{M(bg*X{PdGK!e8^tq} zvxm+oGJhn*9Qx>aiWKNmBTRXEx2y<3i+KWFN?GzrK!of$f%LouBG(coo)HCSp&YTJ zu|^~a&_FBF_DV-9fzbmS;z4qbVEp?p_eq_L8+Zpzyag5BrqZY!-4i^+t^9~l(R3$Pr8@?(zCv3ev`rs1S!r4!C?YbG z>!Z=-li^g-gX}TWD7%I>(@|wK%x8eIr*|*U^(#SlOZfayIALq<-Ag?YjTFVo1|b0T zP73nPkGrSAkZisla_t0{{UbLjJQEfL8s)VT9Q#akpLl$z>7?+hAj~zgR#Z8Cpp)v4 zPZC6&GCX=r&qhmpo<$evrt0v^W7*~fLxlN?F2sOr-oLzvwMeIbsbNldD8i%=NDlE) zj@VKX^^RNau6!UEWpXL(gPz`?b zDQUHtA3LA>Qrp;#dH#Q4h;3v4ocWtoWRI6cH{f|kc3yuXgfF1*1+!7$#b~mp2l$Wf zDGluHjQy%*#NhuQ&?PZGT*X(8EC%(f&;J9a{6vr*|BqS!y9QAeC@{|UzZ$~7(;aqA z6N>OZsQJ2xc%a=o<4SAcC%ssGiv z;IXl_OXGtEr#O`(neZwVCM9$%W91p<`dkuYt1# zZ0Uv<^qd&L5MUW*H8mF?yKo5;5)OPIpi$Iwq1q`mb$gF`=XbYFrei#zo|Ul-wwxC*7{|j#ZuxCsXA8@h@G}Tw@%0b9oB)+VCb=9_ zXA`^q(v%F<&hZz@tj%m5Hv*u6$Gir-CPnt`@Qu~6g^#`hnt4)5%c=J?^jvASFa zv&lF5pL42?sOH3bDjB~CI|HEG^tD7^cI`3ERsW~2gWH}}k#qs`OP)o@?0%JUT#mS1U`%bIP)lGJb zikgN#7CcXT$rk;{Tq$Pp0?l*9l>=tXU0^6+Kiw>@d?#;V931$Q|HG6z-k(ozw!?}3 zfPr8#e1|5Lg+1J(%l1pbUIQ?Cum=wQu%Nib`zF?+=xV6(?dwDq*bj!_F1uc8s@gBw zYN{tPIYxgTnx1?~D~sCJ{PbU1t89Y8d?aGze;o!zZeukR^HNb+^AG&qq!{1w27?&9 z<&gP@A)f`yKJ}PoK%x zZ0E`nub}elW0rn78R2r;r|WwzTi^mdbzk{?yggXVDsW+*$mHyXLhpg`zqD1SZh?2} zu5PE9;z%~6n1N`EiiK)DY>m>o=n87ju$~fET@$^-|cQ~EJY>nQhahx;Rz-#^ITrt}< z08Z>Xl1mai0CNBko6<{_3sG0+A5Tt(4FR1X7>mQwH@^1|gXh@}MJK3hJt~ZLl^ExL zIjO8Qzdu{fc(KQ({+SEbmiezxykEVyf^FaTy^#pGl|5G-xNMdiO71S!`I6@#;0TDN zcYW{AXrwEXdXV{p;iT)eFESSgRJY@W-qUSav>U>#sRmU6+=H(4p-hob5x!3e;0V&l z1er&9tLJqL0*Ksb3ETT}*)23*fkQ03S%NlT9wO2e@N$o$o_`s_P7STtd3U+OgMKRaNTCz-%~N<^_C#5|-#mRvx&hE*Fc z5O?HuPEE<exP9koWRIFcaeDqI!1V^(uYx0nDSutfd(Kf6;T`2l;0ah z8TC|oQ)5DB3eUAoxDXo20?&l6YvIGSaafvZ#;0ADCSj@R>^juyoe$6^cG8I+uI*mO zHfsVvSDsE$ex7-wWb&6sC+KRbEPl?)*^)-KVeW-*AAA?dP1W`)PV> z`A$B}QzYx)pd|KoN6+{k3AOchf-i@CJJ;_b@n7}>nQLvUXZmIooI_o62}k^FZ3yP3(s=u(m@e?G%n>5Zzukz-*Ym(35NMrDbW{QFOF0Ub{ie7#H}J9)QU3u zJL~%398%5KxrZsS&mK^%gC%kFpX=$v-@nY@y6c1SUEb0~H!vWK;357CW~)c_%P6^w z56Mr_3aO9ZaYWwRrz6_AU2=kRL)PG>y9FQYXB>_#ad_+gs4_54u#PKjAd) z0f_vH^!#i=C#LH2bO8}{qzAF!!$j>c%XPb+a1S;DM;mN%;y|s9-Ns45)5n4K2=uM! zOC|-@L6;k*%qG=fNI2RF2#M_&&x;2z?^^sqtjfx0Hl>*5)@(CB&!mNxj;b0bqW&Sh z(tOv_R)L9uYl%_gWN*}L`;D%!UKcMFbpA`=Oo^JE7j4FH6E@+as+<-a7O3W1``kii%yzQb9aK!rMp|`Y zAL+Sd5-tt*M;=fA2tb5PB)B%HMT}Dp-7`p`kcZ(dNvjhX$A2>{4X65{l4`d@82GrI zNV_k2^|oSHTYZy_pNgVI2l^Irlpux@iGg9>7X(gnW!Mqkk--or%GQ!@6re192?Q%& z>;(Uh1;}^W@fifUE6jGuUYrdfcL}PHb9_0l-m;05jDF=38!sJea zM}6h6TcB5kIg+NZO8i!B4Nf%1TbIf}Z5aRBx%ofD8+}#vpUC?;~0HL z&@!pzHJxxphf#X^F~c2sG4K&RzA5oNe~8(7inuiW57N@6fcTRvNfrw_*Z94>0Lh95 z#iv~2Sdm328)!K(X|p!Z^OG(RL-SIf#i$#hvjjwfo}kD|!`uBTpM++y7QI2Ql%A&Q z)^76+8ZOxL^-e&$uCD6@SnGg==UGn=-%5eXG`Ptc0_jue8lV}esmJ=n+%~ffpZ`{Ht3^L z^z9a-(RK6zB*=xicSP^1Sd+U0Nr*nA9>GBeN#WnPseBr^oAkWH?_%3>!LG$jwcan= znIv!A$L3^jF`-E4??g7YijN|^@EGJWWKwbN&qDZ49rZnx8k#zTGC!^HmshApwQS^^ z%f!eGL@nUs0@ulTe`o1CT67ia^L)4ZkVTowz%Ah+KI^!hs3H5I7J*JgLTcLFIde?v zdoUD-xQ3~z-Rsfh?)&;GLYJ%AlGn|{?6}5Fbr7_ls>HwKjvzwt8N4A-$Il& z_CheuHQ=RRudC{Fg@=yt-;L#3tErgRGFpFPDXZ6a>4m}mgwQ##|FXj;8vordjb}l} zim;RpLJ229N0yYSPZ)@PC!m~f0kqPQEBVTUnYH}xApQh_e{>6>OVXt`6AMBVhSlC) zwZi>Qs6U5pq5lk>s``nSjNb|}gmF|;&tdS9AJ4h>7Q7k(g4hpEs{E&MaHI|DD%rr=Xdqh;Xlyrprp0@SlK~?{+!g z1E?WJqn<7exXEb^eG?U7Ui)O+)j!`8=M2n$JChF*e>(+{(ye6ENzba;P zv#0`7&RsBrZd7d~ij(4{d#m+a%hB?IS)$Kh<_$*loEl6=^#A&jre&(PazBF&W@DPL zi0RLLAtB~b4yLY0(_%EnU*_u8xb(5!oft=w2=?bL9N30vl-SpbyK-r*eb~=qU7N-G zBlcO((-VUQwQBGh}+~5K%fsKG#m>~`t@d!-XO3{5FFWKPSxXhzwHR)(vYqYfh$F23}d!%%SS`Y_Ca>; zdU==9u^sWfIrkD8o{`fK@~*P~F^KkfU7BYxErB($R1qn__;R1+)D3cB>Y$yB&7*ek z+B9rlwY-2YOskt;yf~!#HHT~5%=>))w8_}J-NQ|;)Z(Aq;yEsmr0LemIbydq{@d>M z*VP)8L6%Wc*7)Q@ypfwxj8Su)%)2JsN5)fz-Q*cKhKoxk+Q=O&SzWVwS!O}w|7CaQ z{^;Vg*h2s1xkjLIgwmow;Yy5&Do-#ml`G2*P^t~D2;g4k#(P!U2MX;f*axG2dxab z_OZZaX)QD6TaLAWJYM$c&+D)f6gX{ZL5AKHxkjoUk$Z<`NOA}b4Qp16q!13q@+?(tm?w(d|t_|-ZKE(1l)vO!V^A;QgKq-!6SRI914-x7N zBsZs7y^h#O*eLq-+cD8;fkCtdU}CgZA<;0P281}lyD`XrGu3HPf_WIFd`qseuv`6W zNm|pc#Z|{fdrzmaJ8AX8UpdXl(Pr6dlF(w=BCSCx@`B;lI^C*$tISU0Q*SaZC0rNF z((UeAnrj&vFy|(v(j4h%h!dR>8$ih+@-DxTckDJkNeg?T5fgpk|JPIHZYMNUFXLNncAD5tv ziIF(|d=1B{808gUC++XdgEJ90*71~x#b|L8#0hMLT&x%JT&<_yfm+V2Af=LSjQ!e^ z6ra#{-ubXGv1Lc1_jFsL6?sUYJUlY>>!ctmdrco<+7Lov4jYIC=ej(D1fasV`W962 z&parr0pM)l+L(TA{>`^<5dBIg!>bDL{MK@lepT zR?&J^nUtzOY6va~$yzHNHCGb4O{!|2z?(5`L@p6~X|AbiHT+5l-O>L1yWKn)Ije;q zV28-a{fp20_S|Ns@)Eh>MwV{)LFkn_Zz6lYnBVNT!Z4spP-~6UD#z2k#vU!Np4@Zt zO=)U9h zv$pKFC63?q_o?JPh1V(m+(n_i8ER3uV{zbOs<}q3cGn9G(~qVNRaT$V>R;JU5Gk5G zEI4$#f+l#7x6srjZUe%tPb!W8DoH-KFUdlH-uBC`QzTDaZ6sS)SOtPR=f0+uRatgH zTbIM1t2G9EaL#p^hO|8xfzVM zq=26hQ>~(}T3gPFO7o`YEs}8k;qn>>K1a%S$O{56%C+O~1l&!EY*3e)#W7*?s%Z0u zs~PUx0|9752OFGlKFT>85pGz{fKl}(F@0aAK zuT=;fg(%&qC^B!Xe!+|hxEqm&ON$e@0rGsJhGh2l2qD-0KoEcd!0yJNuvmI>4sHAg2$WGjq#wX9V=^gO zL})r=+H2r4Eg6l99WL8B>LIDB#Fv3$Xc#@=q%RgylTl4}Y`9J=VIp}36$6CIP(1|> zFhI4e&ByLK`}Y{GYKQ_#07{Npr*HI#mQm0ytl`IeKBQH# z-s^;}-tC?0JZ=KGK9^>SH*PZB_!iIMd4FOWpnl^;wPkZj@-u0i{!%%CcXvtiY9Q7ic!0nqwF{uKHreX&TqfP2^m*D|LT(N~$XH zowHjlsWIeJQ4Tf`l;XRYUW>pHrr^LT{cL8K6nMxk3vBTg-c%QtSH z5g&I0%EFwjE)npxpFW|cn4gxbl`75`nljn;03qgIXYYjLqbdHle$q2~We!BrZc-bJ zR+#NFBIgcMwJ}mDD96T$libpSy#M)N1AI+N$=La$dVS{YXujBm-+izVIM8@1Y9JJ6 z0Djw`9*YE04g7o)2EU)k_N`9W4{oyEaez;s&WYJT&J3@&Rvov=&~Tlgd5!vCr!-P0 zu2ZwFXCnf8cgegh!A~*uGFIX2tN%G-}hr?wiTC^?Cnm;<8a+8U# zZM)TF;?9p&tlud6KA-po24y@iurKE?uUrToe}KFAGlhSQQ&8eg&{>dKnZ~IN{zX-grhnK ztz)DZ`=4q*n~*4s5lPz;ju}X)l@{Fd{#vB3M^ayohxE)Y>r-Of6?ksr{*p0KMB2Kz z2dtHpTV<>JrKJruGIijcw<(p%yP6Gk5}Y@9l}dwh3*S)A(aKrt{O(^agT%w#&+vZo zHQ2EqW?g1)I@!8kB^0$uunB#T+BR-pCp!L|C^o}xgAr&eoBZpU73nn|>m?cDlxLT6 zcY^<23S=0lMk)X60*ciqooauYqNW2@GCbCw+jn>Tfj-mCYAhSm!0R@`y8FIVa_;lW zE^s388U^!SmF6NWNjgN`18hM#w$SC93tYamc)wpIJ`Z=b~IM zdSC=KqSqP6!EjPz4~|BNiFYDvIm}VTXVoq>1AI_C%AQ6H-HAE7D&3}rt6KD2A|sqc zg&3Mc+wqSt0TACw!6sl$`Y&7I!FZNbMfdJb*&L?V&8+p~)%KU)QnBi)7y{2n zMV_Zf))hZ*%fPkMjN@Y=^w?B6?yBsTW9jrthrX5Mw3(a^neCB>TsAYsM~}Pl@i;?s zvObiAFeqO<)ZsOY`Ibmmu-{lDMLzSLF~4aBj)abT6eN{OLkRyQt_VUuZjk?NM4_C> zSRi4GyncNGu4s0dWDb=vDkEl81>*F|!6+ok24#OyhnY;&MN~qP)88mhgxoI!mn^I3 zo@cU7G0yw8S*SEs5>JBnIVgKgEWahGpaqLg?+oo9+Fg>>Gmi%(TW*o|GDn8D zUN^mE1RK0KzGlAI2j0_zei~}X3G?pFD zmW~p7t&x=G`r1-<LLcCq0E8 zJoUw<>Y3!IpT2%zXX0~b4ugAnsmL29du%|l*GHDeLU%fS^U+%WGhS84x05^j<1`?~ z)UYlsypO|--pwG$&3v;{*XqubbG=x{k|sQqkDvi`N5noSW4fUcw&%6+p}b%>qg`_r zpvgfYu7!H(RJ+~`5`=P0luE;)YSg%dgI5znvY%!k345d?LO{^6WEu@F z0lvUCIwx0?4yYk??y4$F!oSOz;JekPdTJnbyV}ZiXr|Ni74vPD&rE}^M{~fl;ZUzh zVhOOOh{SD00e+h^*#f7K!52Fgo#?2)5Mr?DkQs=$_Au7`EBJN?qW?TQ*MD|N@tN;y z<}1L0M-9N6!G~e^1|3{NLpUq}k>`H8pwQ-ZnJa@&GZIb|NKKzgQXo8{)2ngd^C}7< zHF2oKe;rC^wfj5PKwU^^=6vO6 z`XLelGpb}rmS9&dqC$okE`!WtA61L3TWhEyK5Xl)CssvQ1yz`DO9>g}`2D zIi4wjpsmjfu27LIMqiXp^JW<(myiqShFync)_CZe1v zKKD0b8~N8qH9-i?h>4~|yF=pW7+yR3D_xC|ZwCg(7})l%$*3gklzokiR@9Z|QIF&# zGdTvA)!vPJ>R9E(V2nbl{6J)EA#S*jVwVJ+v7$Bpy97X$>up+` zycY590WM5xDw_zqwDlV|AZt}g*d^zcF-dGYHt}bqM=D*JSurPmjnKm@5n}tZE%I3dHCLBKU}~yX%B+$g zsS}=k&>;v`2VcJOO~*Hdp_*Ocn_5Lu{qa^xB~E7iW_p+KAezB;sL5Qw{qoT9JTZ$1E2K*T zi7H~wRtvLqU7{s%*sw&~D>(v=ewTFg7RLsQCs9xnYhXi;nJDN!mXF zqeR9Mxr0E@xdUz(;~0_Fs6e`G&ABaSyr8zp@^U||v}S;7B%3m}h-*fgwIQPhS`Hkq=0Jl_3jW~q3N6s1eSd5K7qRZ$ZKcgcM+WiO5RW&a;THAj+7z9bpNYlcvHRds1?AtW@T z4}RPa9iGLM|BI-zaEKz>+>-?QBWyE=bWZmVcwOeXpqUy66{wZm1Qu@|HF zok!qr9toeL*(kY@=DTe-{TNo=h#@?-J44HfX=rs2Vp|ek!~QtXtxI+4FLe*m-8@e0 z2j6MDikvo)@(=;?(Om5Z^Xgo9b_FO9%#Vr_HRCnZMM`&)DH3Y#zlttfmlU$?d34FwRFzKzrefpK=P_0hkJuI*+Kc)SJUx z#jg#_$RRODcCMl)T%FyYM)fD_#fd~hBSVW4%5Lu|tCV`RLLofwQWy(HJ_w|KL1!5* z2^qYHU=+$&W1RWhboQFa=M7+@eJ9Q5iWPnnH{XAMi{iWC@J?rY!?(6w3Y1zi&H)5| zrJ&O{3;I!!v}Nl7w|XIMhb-{0x~?`HL{~TEoIR9gmf5;r~;(#+tLrbJRWa3Ulq`x*D$;5;<0Jjk^=e-FcI-O+t{ItexPd zA=$^muUh)=4=!bROV5~VGca+ndw2;&0`_d$ojwq{n9f8 z^_dXlc{c>8(9mH9!NW6-oBr*T^8I<(4MU=Enfzff2}#ObExfU7d3aX)*J1A>F*m`a z?&qvCdJ>Xt9pr3&1rQjCjQpMfGRWT>1|MV|RHQ+sR>_|dwBeUBd~hVPJyVS6)y(QX zHOqv$B(h4CPlBsB!sF}WN!P3u*TT1IcPC8t;d_sc&7lj)VWI6y%Ic5F9?#oF>GWo= z9Uhn4&2`L)D;=4$JsyD_KDZyyy5q6cBgyth@R|4yDmP>}}wWjt~&1^~xLc8SeDB_rDDarx| zixv=cLbZ#mB{{NU^fCMer|GD^$`p^H`=>lnTpg0lg38UwNIXy0yukhW1e}$l2b)X* z8=}Q&diL+rFWwu)!_O^h(ob8%!Jhf@rK6kujPz3UsfJOR`TBFj(PZW-fkp7E_1pQB zLwhJPgCfCR!$k5Iwd}Cm9h~Rp!`Vr!qGuKobY4Qx?w9sQt619^|WDiOT!+0 z?6K(@LVU}V8Me=7JtMy`sP8u0+PDQPf%TVpaSQQp3O!UGb18EeyGz$=%(UUf+T)W7 zQ!I~n3qSn}rscIvBlK8};KxG=_5E-8*rqI9dA2rwczaV`kkyvbm=E*!n z8e`p^95%?g2epVJ><+|}5S9$@yaSZlDo(Z^T76`_=MFJWuDY*hyqo3_rDZ4EH;4l_ zu9-Hm^o80ZXnA8j&->__(fwj^jKL1KXobYy@eEN_(S{9QWPw@Y(@ebZ0Mb6^79u zvWgU|mbIp&Oxrd=StiNIXqc%Now&=ad~$JsAt&R9A?E5ht0p_A;%a}}cKOR?FEql( zC;18PAK1Xbj-Kyn(9y|P9~Udn#M?e0n^WTHtB^Mx$-6Wmcg_J4589Ul-otNJs9~jOV6(0v?K}%l1r)f4!qGW6)?1QD2aKh(~sx&Eqe1 zUu>`&WzHs%908mt5Sj>zW_t$byQ5EO&v-Bxk0~1wG3;MTsY2&RU)6fNb6Gn3LvnTZ zm87gHre?v1`yiDdM$h|Rc2wZVUeQBnlxQ8juV#X0WoU|fkxreyLlFb&E~qoDSsa}@ zx${!fy^#8|^#;%7(&Op`Hl}s+MZSemW!8Sh0l}ybba+lK+U@>gI>yqLg6cwDOV0gD zU}WdP)l>droNvB{Z%zecn|jNoY5J8%WN2GWpbb|0zi0NJ^)ua02WZ)=GjcOAn=o-f zj6NpPZipnND})uW3u_xv;?TT-kGJvOx?LM)VfxQ5G+zLA;Sua|e_khr*d7*nmzxm~ zv~ z0s`4U$($Z=nI;2ygy~9l#wLRT?I1Ez{SVbnn>vY7@y~eN9h*o!c|S*MmVt?O_1l$( z$k$)Pz$HA$Je9eg{Zv8t8NSepPb(rKlD}M|83kL$r6$@i9<@)ZhCR%IJB%RG2-GL+ z#7#l7h=z?n2;L7iaKKFYPMZo5oi>A)aeVyyqhtm4wEsub{Le<}SB4r+%H32E+qfcU z%Qcb40r2g^OfARg0H>Lbqj5%Y~y;L@1t9j7K?U z%O!SN&xo!Kqfp)r1GH&xRCT*U-;9~wbj>aTtbGobv zT|)xTDInDqqpzj=ERdj|Gzn*ZyQ09rV{9K&{aqkb>`x|JE; zt6n3biF>u;#rhq}IA9#*Nez}GdRArEZgM~MJaY&Fh?>-(2z5v z;G`e3#D#!)q!}weXAR_2M1&K*w0Oq(N%-O1!Q!ZYaHzp=uuCjba8HH!H$F?#zqfww zUHx|yiIm=m0l!g)D_}`uR#dNwaDQg={p7sRQDN?=^j@Z^Ydozo<*fT`%pR*Gtiu2i za)IA6^nai4cfV-uO!Htj9d(ggF=9kL%{J^)ZtS5T@ep0TPCwG^P|la}X> z`)GFT554=w`}x9BEgr6tbiIB^JA@o5lp{5iQ5d()FlUo8QiLA>JqzDG+Aon-`$rY# zZB$Y-*JvF7F}|rXf~!=nZXEvpo?{SDKuElk;n>xS9F3aZT2JkuUz;h^TZ03)6v(d{ zy9iE)+xWZk|E5d!4VoDDmJ`uGTm3U-mS`mFdnNBw=pFrqK&kMiD#@bde2|n)i`)AD zx~IFJ>!MAHXcQ#h!md=^eE{RMAO?%ym_Nh2G8XZ!lva1XTdzod`YNQ;e$3l?ktR-s zO`0k?t6RL;rK9Nce2*s2%b@ElGz$@$^vIjUA_j~5mW#WS^Vma6%+FOHZA3c`eMWY5I23T1JGrhM}z`*HH!igRA zk~c+q&a(a>AWcQWds-(a>#TS)!>PvHsVbhnm{Vb#B|qAPAu?~RAce`Q^5K(wmPUjs zm#04}mJX`MJCnk>xTMg5DG~pRFTFjoFDzHcVdJ-#C01=U>~u#~-T|Q0@ji0Op%nOYPsY5dqzMm4@K7r zUaI59Y}2lrY#CcBVd~Lv&9Dh+yw4w^S9*KC_wjjJioN$8S=o)) zv&~8~;vR*cX%_124C8QYj>RnS5*IZzR!~%a#0je#ZvYAWHJN#2QEG4m7Sg9zXBk)A zEae(<285^!DU}joaw?T^&S)vl9{ECWsU}O(4mWx5XyC{evjJxN-77alA>n4o^3vj{ z%C0uLGS`IA_2+bN6TgK|9+$*@4c^{B%y&vaop#6~4!=z+^LJky_evszM+P-;=QMa8 z|HaI*`NNgEP#ZhRf?MGwxUGG&IRP`q6ZFo-{V7M^m&=$_gctvI^iMYIGoJ>XuVF07 zg07~7t_9}mY>@WuaBkEfKJJ#>5_1~>57}e1@+xh#Ky#F)3+}X&oXn%GNk;d{60}^4GJR9!$kb! zw|O!~;mM)6dAn-zT#%A+ed=HG%)nWPa7lWU-s3hY(P+tOI*K60`UClVy^|YlFCAA) z1~gbPlGj|wor_GcYM1^39=9<+6r(w+dA4yo0~5Fk5xCtvds5R>4%RT-_S;AieQxEf zo1W9=;@enP=d+1M?RETcEOj~10+J10cW%w#{eu+oHgLU9+nTjRqdnvNW&7Rvar<7` z{x_@1?C$NDH?Pw)(x3OfnHP{$J+IW0vn+udnfZs{qz+2LA}vih0ZxX4@$xrhhzTtH zKtzs?IxB7L(2P5e+2Q6Ue<@wEdC|Ly3ru!SMEgI-a>8J-QWYb^i*0KTy~-&xTvgn3 zE$oqa`!e=ybpL&h}(nnKT$Q=Ci9*Eo)$B|y@CPL}_;iGOkP;yntdBwDLhixvD! z5B-V*0=(xX2{M=VQ!7kUmBbuCAxelB(8 zP*N1HblJ+isOvQ$H5%0t`*CNW>S=;v0EmY6PAm3jSogWg?OeWw_a;iHsPnS3(fBmh z?LJzc=IQ!?ZU}Kms9;(2LNB47N+8Z*iUX?RJCs5N8gt*k7q3UCEq#BI`dn&Tq$hkM zG4CDH=i>pd_VBVF9a=Y5kfm7qlHI=bo?c7PdS~g=BVUY_upm!>6lIm5Yff$WzT{$; zX)BkEawqc;&ppcPLL=foF5)}lPwU9OJS|J#iYl7Y!&S^HSHcfVDoN2UqIQ~| zajqsW6oFEQ+*9ex<-OFg$mFB%DBZPS4BGObPv^h>V<4(79Y-^G=8K>c5?KrF9^dN_ ze{;XN`0U#~T44oG0}P_ZgVDd^_a%5n=y0qW=x%Yt2weg>-if#StzbvVh}^=zvX$1n z&OwG+iE+xr>(V6QPs98<&g+LP-u&nvDr_~5(rui;>_yW^6W6Titxub6Q=S@^vzjP?NB zIWBDpIa!Is$_B4q>m=3Bph^r@Hhz5+LmOJP+=XwRt~t)Hx6OuvZDODN@%gp7G; z>b~}9PPflxrW9oAOJ5X|6E5`Uu&&A%D3q3DnpDe>wB^az|BuhcH(Rag4^KaGBIRm& zmnsjv`%d>99ae<3XBrpRt}+mIF!ukwGq6lWv2%zYu5+j8O_%MMl{F#rPGq+ZZy^Qb z8TktQZPM{BzApAcrqr$AVKB{oCm~?(vHqFA6|W|I1>B`iR!6YA^bZ?RRBZp%I8w>YFt8JEdx{Mr$(o7QrqT()emFY4&(3=q4ABy(I=~SymUx0;+`yc zW@Fqk7tr{P6;>SB{CW78e=qXh_F!x-8>rLlPUr}k{Ymw@{i}*^rl&IS!o+!6+eNnV zAtAb{r?xTHK1#FgxJ*RzLBpPZWAh!GJb$ryhENkVA|i!l=-kvdMNs;nv7ZqOkYl1Q@zje_n`te^Gce5F80kaX8uwn(ol(Ba=JbxB`(qHT|R#je*Vk$@6ZSmG@j{Be5L4@`8IZMo;FZ54y~+B z(zCwlgb5k`TvZaZ@!w>ckILHMi1?vFO~!&3U=DLZ$7?^QZ6` z2eM37U9z|wUP4-JH$~sAeM(TD6a#ZPKT`V?b&D;$e#OTn=8uUi=^nwE;ASd?dee4O z7t~52VG{l)&3Sfk=T%lk>(RGT_Hh=539cg8`46e2ioAwZQ4IQ*w_<#|Y~AljlG~q*>c^CCp-snBxAFWLpo9D`P<7cKk$RjJOuhh#%2!*IZOTLj|HV*{K28ca+ z4yrCYI4<1wHBpnj8~caLRE!3DYp$i4r|vwq|KslPEu!kLv&Cu>-p$5O@Qz``tni`k zyDOS_Rys?i{AqdZ_B4E�)RnUQbI-)c;5+s)&_`VUjS;%=tl_yXPM=usGmxwJ{gK z;~aH;kj^wX1skozpS?a#rF1C|3_)-(E-4Le_EmC7%fdi%%;t&UR4Eq4n+y8OWPfE0 zYE*%9@WsryJDRo`V>v$>2)%?kt@eosYyT)5-WSTthb^_Qq zUGDyW{}hbE;dah2Fd80sT1`k}Tl&VH>Rxn6PtwoB<3oXcn&%0}{ ztMhyNzl0eYOYWGB2tI|GyTn2@6K7SCnyZ>$)xu?sCvwCP&H9>HWeDjL3qFd1XwAvDx z$KY2{KOu#mDco1{PoIg7P>B54X+Zt<<;Xm@)kEVV9~iQn@$d^#h>X;e?qkwqc63y| z9Wlz1n8it$`BOL{8yev;H&TK#Z#y%*x=x99NSngcLYY5l>2Xr4J20F3c9I>-SJ|pn z`1apBK|@2yMj+!4f9lT|Cul;!X%Xc7QuntP#T6@q$zOSk-=}V){#OTeHrcP-bGBcv zJg|p~xObmi)PM7_8-?TvwMbjn&ne#I~5 zAB=1vY4H50@1)$CCj3O~x(BDC8&-z(8=h=!N>9Lygmwz?y%bN$4u3tamq0&IiEhoh zp_2yUscyTL(uip~onL^92^xYeYXM11XwUNIW?OGl{B?7ALsyk%w~@zjmvrpZIYQY+ zT5qV7E1q4Bm#yGp_ItPK5O<<57Up*aZ=m8)Y}Hu@#dkrcyb4{}ei6Hul!u3N2DqMU z@ZddSc9RB63`!R_qN{3;bXJK^m_`7DOSQH_p!&FBm<@8H@VobPZ>l&EAPp&N)tFHM z1&ONt%07xwi&}7MGTF1=bE%6LN-oeZL1Obth?euX^ye6$!k>Bi*wP*kd6M>n=t0Iz zeb00stQ8ITxf9&TRzGUziq*OpuMHa)?rx>nj`|Xrcuz|n0=Oe)8_(mbA0lOz=>YXj zJ0TufdXcu*fbPQ~WjpsVtMk4Fw?-;ZpBw$&bko<}cf>@NNiM{?YNVa7Wl@-?(`Z>` zZA$c-d6x{W?eW1wy})yvGwr#=sv}2=johv>6zEP=|(DCCw6i2P1xKrw(IXr`3qf-bC7FvF^)d z!yuW;WlxXf>MF!d3Cj{O)yTzXO~yBde7AGVg6 z?ZG6?;b-h`468pUI8nIof2Q`vh>eqS_Yrj!$vBZMa< z{XpC>y8ctJT-mYtX@^N&saCC2UaV}jmA4kas*46L3+4tya2%>e`GBrpXd4AU5FvA# zR2*;f@^}ISE?>(F-Q%R=o`qki3a;K-d3{NPfmccHaBGD1aP5Nm>f0*j9a#VSTLuc{06TeE~ReLQ(|x(S^3~|&hNoCFnTOC`{_jRo}-9-xYxkkp{ALAZ>(v0eG>`LZ58;X)i9x8RFp-g8FBsmtKP z>>I;my*t968!?CaY6HtaYT%vra_8g9&<}tm2UTqw2lcgS-UVW;xsII6XyK-4;wv|Z z(u@4Qm~>tEHUEoyrJ-8a{A?;RL;@pGR(7-QR!*UwSkX zvq9%)0|Ja*hGaLrqj2FKrt11fiqGM|4e%*9+#=K-9XXW_n|{k`Iws zcLqr2dvbnXaKg_67wy{Locu8HehA>uHI=pQYvzMI3|cBX-5k=I(wY@XdDczmEK#}| zm3U^;mf6>Ma@ntbMH@_hYbi|AS?Nj2VfF^Ve8mGj&Vim^c<`DDIR}6TR_uM9A}LW) zUBe+|S1QmWYbhS%HIygw42#TIJ2^WQ6?gkfGtHyOBLr0Y;(dw4?M*_ZA*XN*m@Glu zL9r1xRjDl;SSC#iGjb_9N}GiDY22nuI;G75qes&Nbhm6DQtb}mO_S%9QsxAO-^~sJ zsa%T5O!sZxuo7C6wi}ar05fsDfb~86&%z%K0HwJ*Qn{$etPY19E`{yyUVUF zJiTnD{BsBk3mb{_T)VHCXsC-|vR=+tkLpeV{!~ZgJ51UZm_-MC~4r zL=DFaJ5B8S>j{{OQjJJ{D?aYg3I(duFLl~hI?5u)gu2Gzap}UcfA(y4rwgN~gm|&P zlvf6=GzulX?eKfnslA5)2xl$BuKm!+fpk^;LK|<2kr4(!Q~3DPs~a#~A2_igA#|yF zV^%Ya;!b8OY6mI9wIe@%!t;wg`jLwa$ILEN+0r;DsmIA=1{Op*OI{CE|63`kp5A4X zVOneha>s@0wyd5PS@9Itgzp_+P9Z{ zVu>YNqb6qoFttRb`%=RpP6r0g1CEV3(3j*!)I!lS#!G+m$v=u;paO0p1eZyfKihR&Fi#N-Y~fA=KMoWW6KsiyjU0SmPx2JoqwRedYAj+-auw zK3>B_3j{sX$R`wlIDJXqy-#_PYTw>}?LP|#WJ^;H9ierPYtr5qMdf* z&DEn*^K!gjt`;1!j0Y3-=+YYU%cpr@SSw=?MNLqudsmj~bx%DyE9(z>yG4I#o*}(P z@lW_KAcN)$_^tTIPmcYIM6Qmkb-Rhv&_X$ECE|=R+lGfh3{RGzWZTvU-0eQA3qbZ) zcRO6svG1fCm6#7ps!ml^UwL?b`w9`p<3X$LijkOL$v&k+uC3)3A1gPH?1!bqWfreo z=c=<|KK_OnCk1*xXu+A}ya2Q}%EVIHxA_Zfk%Nh6ul6zDG#UcwoGLA)%TBDD9R6xR zB-9~0x4i{S2PTek`k}&3vH1(Ya@eR-sLpjiB<@qAW9!%s7Q0GV;I!tWx^DE>^8T8g z6lyZ^pw%{QKmCk~3A_mktKLb(SHcdpTt5|EH^SJEhKz&ur^OTKf7NoO6jOe9-GI|{ zh<&4RK}cM)BW}&sXHfK`$X_)>6{DmS6TYv?)9R`3_F?$bf0F)FldnKOEumGphQTB*kUFwxVev6yTd!Ym z6^i=0XM<}DAD3JEx4rOhs~sgdQY(8bg1Pm?-N05%&RMbE&P9AWjj%yVaNV;Sk)4@WMHJL!fBWhp)i&)gk$iqvc*tdh zsJrmOFvH`E+88n`FA=-k#H8QV1WX1Zg)l8Pw)GK{60j<~(m0RMzVN=fk2J#4RTNoa z|3&Y%D5m7Sl?YXQnM#YIO0ueP$BZgANz_r44Mbb8uGWm-Heb{~X(TF!!8Zr8ycHS= zDoRT=7U&9O+Mdydb6Fph{l;EoY43$i*_2X_zJ|(Alz}N2Ev-vQ8QyR~QeFOv@a|T4 zVF)>Qj7!;PC@P%(p3aDwlzyMBC+@Poyt*S@yoZ|vIgP@_YaD?BJ8@>RwR7R`2xr{IM^ zCB=E@_VEm%dv?!o22RAu_p#kR{}9}n1z(G|3xAdOH1*9W6Sskof!)I;cr*qduVZ1G z|Ixm6LyXV*Dxz@)w2wR$0xpFAVRx++1wqGg+`r$Se?ZCI7%O3EOBgNCE1{7|NT`+< zhVCe-{VL%WydZiR=h2)t|Cjt%#j4*_kRVscVsT$krrxi%zfe*xyJoW)EeB5U2Z+ri zOFM!ld`3gHZq^Ffl#e=HzV2>L3p&H`7#nGuLG*sc4rjk+Fu{ents?e1N8W@_!D&Al z%_t&xRX5)Y|FbNZ%yOl-W@*=!@8osdkvX|k;lEM@?&K1_l}OnLZfcEbbBnB{GM8E2 zMWA(p_Ha;~+A>v|5q2&L#7eR^K|0ofVR#0O$6Nw3XzjuU+C-Az`pdYL!kf2eESH4f zs&(VimJeUiZ*|(t|0H<+C!i7c<>M?k!I#HbKXFsLLEj6= zG*P=66L;dmAz5iNSiYZ(W`J#Ul;0-J4QwmpB@g7?N$TGnmr>oT7(5UZ>|5au-4=cH zxw5`jCH9Fy{*X}=@xH*fUg72Uo`}x>6>{EQ37^*}&kFAzdo7)kGzZ;ck*s6?G!|cv zE((PzVwB6eOOGC~Dtxt=XBWN`9jb32%S@348Gm?*>}XXudOEO$t#Y5Yp}He1Id9%4 zxZl#PCU^UpV!sM$m?~f-;Y#~R1~y38@n*F^db@FOWq;J@Oh!!bY!~rOy>Wz_4CO~S zZ^t*_nLi6Ii%(dcy^1dO>Rb5~4o+1PDseUF_F_YmQYDj-D`jgl$3cx_1EmhO6wm** z$m;9dW@yIRpLRYCfL#b2a9NYP7IrlKzJ>TDJQduF(=nbk!OLbb8-Ivkn{b=^e{~fX6{Kc!dBNg({^guL z-ZUod?W)}B+`}>Bn_%|w*?6;yRjV3NbHvYq)7WHGDZU4)6+4{G=r~~af#~O6_rAh? ziJN)mjva$>{&4!KUTyP3FkqiA!fQ_l9r7#=Y1YbrUnad2bZ|9g#k&Dy_35l<=yK!5 z3@|_dtkf1B8meuWb?zXl1a9V-h}uF?e#>=yQBcR%SOkWnGS5lVKIkLcFc zz4N2O@*(X-)}Vry=n+#@3g`s5vCA)%s)%37@j3enK#S+1tW~7+GdoVdB>SFk9XngY zi}r$4!Q2E;o!k#8Hj6Sm|JdX}9}-5IT=<#Ko_}O>yf$-*;Z9G)At@SC6IZXCfK~#d zr!J;-m0dvId)_Cy4GwKLa;Pib84G3ieMde;52F)ohy z2&Ms-MPz*@x+5J=V&PkK^K$LKRL(7SEdy^~fOvWEMUpNRUwR_XzBbyBX!Cyn+NK2U zKpqW%NKQ`=UH)DH$?*n--3(WK8boD!%59=TM75cMLU*8?Vx>StPdmG+PFar5Yb^b1 z3YV`EeMQ}MBP!@ltt5QA=rL0GzN8P2G5+=9OK#v1hM2?ln!4zQPujFDY4RT52{&z4 zRi!Hja{O@ktwFNmGo|3N*$GJBEDrdP%oaonghLq z2Qgs0pHTOx{L&(=6K@q?J^@jRA1sRDdUe%Dc`*n456n-#dZr@1dnlp6;$t^hnXPVg zFN3yroBm*qnX{V9*k5hFwp69bJn?KL>RT2;=-Ek^h$T?mw1t^_xtL>t8c|=ya@-7A z-nb?l@aj{cFB)NFZ0a3q8vG{gcY20&MCz5SRHyrlF z6on`7_B^^c=-;02M1aFdxkrbBl|UgGe`GfuYHfuDW`M>{<)?S0ozGEq_m8Gp9twByxZ8i%r%BTB1fi*rgp3zpc$IBS9WKHr*4qm+hgrIbDs7ESK8 zd&A^wh*x62Qo52JIyV(vJdLU#Wep<`PJD(&PSYlQeKX2`)Sz6w_Ukd{-XYSm{n|?) zOz=v#>|U>|=}c%}oQH-5aSe^t*&SrRyD?z2;Y##$P1w{Te&Iw{98kOWe&#Av&%xFR zk%KiGc{Sj~HPe+e)VwyooYwfzygE-?><_bb#r&|=N7SuIiO^uqzaLH))PCY3CCbFo zIJo?|F^859q44DOl(YJZ`TOZR7yiU0MkC59~*bo%@Dvq4TVu4)mAxz ze%$r=SN#l-q0g4IHC8SrHs=ej1&?~Yf3g?)v-~=y#-|yn*nVISO&Ou4Ouw-~%n!!E z@=6GxWwAOdEZ!rn(I>F?$_+H7gIl)-2lGC*&S`sI8suRa$3fm>-V*m3RI9;}K(;_9 z;hVd8pZ9;zuq+ninz>cp)2j{`+Ojpj~lAp7q$+f0ymq&@v2lvKMHEY4%&jY-!szM7}$z8F`+5cHm+U zWTqO~$=P0=BbiTByuul>j9syxw(+>yU+1%S45=b6KHH$xCGGgM`_X<=Vg~?_>p1@X zV&8>M+b>C*_8u5r@?EKhA?%_FQj$&p% zM9K}dqxIO^N~1SGadb8U;FeA%313r85^)B@h)%27A;6S`&GGXg<^9?W+e*9pa{qLVtT zX`uFT_Yjnk+t)~|#v4es*G#53+Nq)2?5N^Fw>`*FTNbJtYh_%|wrPJLfRT%>ZB-et z2uvelv!3VJS!r+2obx{JL>qe2{{TNxt;~r2VVLrz)IrBafYE9o(Z`8=rus#t!}dpS zz(A6jmkc@&*%WDYzsNNnO7J_BXf(RV=_n#&`}m2(yw>#Ys}zciT7J!-GPaNCV#g02 zGyu)gE8z>={CMS7tXJhNTwh#dN;esMvSm^ot(;>z>(^X_-?FrDNTL>RqI8>o9b%~u z+Xn_Cp0=f@4~$PtC1w<`cR!*J$q~IrmiQ$6L7{V2dqz3Zk-+Yx`+Dw%5X&AE=x9UX zUcR3{{sx)*r}-aaxG4t^?KD;D~u#}hCSR!1Gbw7%uw24q-__a)g3?XHN|PRV}IzgeIV3r?h58{rfpvh64B$`f+@h5&I| znL~Z`tfNJ5Ju~IL9t&d86K2Md3WxLkEUo5GM-Hdcek_rcf{`u>71gWo}cE@pA6s;GJ6e`V%k|4=JqSW@QINA%TM z!@8pjz$*OTjHQYi+m&w>#N)M=>T{Omt&5r84Hyp80OnHEPpc@#d~D!Xq2y`U%r8Zv zim?kKLH8$6>ql0Z!t@rF(Fh)3ypRzzj93oV7ndw8%idG$hOLqZo+=;d}?<92V@4`RVDcg$Nm(@ z%4U-Dy>sw~tv1-I^9%FPMGNGACTaRr5@b_Kcha6^`*`kHk?# zbYVwFbBv%-06!SLIj7K<023af$3G&u#+wMP=7BkaXmMtq_FPDcSi?g)z zeKE}sZA?=I@Uywdl5>OUp|lU|*x&r@v0@nY(p=YnJwZ2z{wU_88ljnEOREEwT2~9S zqZ@dL-rW(oxhRQ|c|y!`#}8z9VL|$!RebH~$A(OwGXN)+`@-g_|4+cMy)0jmR1Bq6 zxSqdWsC+Leh>+T5yAG_ZWw=X)qm|p=>v6vG8Gd!AesX4eNbZ~$vi#^cq`n1HnyJus zUntdfeu|c{T^yLRS>MvqtCiDYkxnL#a4(4vs7hCTC3F#oDz2wsE={Jbg~#i;hB@J!iMG)mi`O{cPP$jFO}w3tX1E7nK8JeSFH+8jug=OP z&mL#_@vckr4<3(J9v=?%TTFh%iHg2|s~pqss3--Uv=#|ui7RYC$ z7xE2M=b}{Ab;M9h4+9wHQg8=Qh&2u?2hqMj`^vt#u)4kweCmZyzS@B7uO?)-QSUo% z@rz-%!|JX-`rSy8YVk0TFBt}ixB5@xp@wt;g6F?XrlpjI4lX@BzstOjHGt?sB}%Fin&S3CVj0M4pTzT~wCI0jvHnP9A~kM!q`kHIpB9*B1M zSx0V<)NTK}$QrKT3^=ezQb_8kJgB9%8c0~1qhgTr!R(2$wysoT#+ho8au~*mOQu6|N z1e121j9p;xMgb2Q(a1Yy9(t0FwGH36@S@l9i;G$!U~GS7W|dkCRSeJ=M&skd-3WkD zH3JC;D*{=7S)Hmna|~80vvg0hPoMR=xn7dd3%y)M84Yjt1MSDf#r+0sYL+9icpBN{ z-&A6Ozb_9K3T_4Qo?DZ}ZxEb&OKO}Gj7fHpm*i|KCBI85d;Bt@-Us~%qW)luA~E+_ zMTWx@Cu{%!$Y2f9qf2YOQNhUUoB=abeglPVgv9!Wu0 zq0INiUOh=G^W{CSx^K=-wH*Y1b<38pvt{vp)OgVZb;VPm z%_uP|Rp8k?5~dn56hNhn!V@f)qwp6Lad@iO%%n)4)FZosM3nFUF+}ZyC@mU!0+93| z<+H7z^iKAhc@V7Aq#(NYo39SHDowA2-38<{fkrE`@JNnNJLY>HLFu|?{HU}~OP+7~V z)E-Qwtq$Gpv{&{MD@oV<$BrL3Rsi*^&H#SFZa)6L+1MVS;cPW*OlP?3eY=nxWLiR+ z$2prqlip5W&@BUrPp}6Xy@lTrAg`8O?>i79Kbq97*rce$n}l3koA$U`>hXr!b}Z!# zClyfI!nUNpeDDoo6V|LLnPe4c#6Jrlqnx}6N!jQ9c`zzCvNiNOoD;Et8u8II_@LUl zj(IVg-C{{R+#C9URuF8r;2BGS{PT}6`Csm&zTfSty41RhnGoYJ6%F4U0s{b1{H;4) z!CT8W@`d8Kc|lr22Bz4ipAAMqLSnG}A&^utjNse-P^=Sp(g8rQ1wLb+QrAx8ISOT}xf2(#ApvL^^;!jqxN&u}2D925GSHx)3nq!b`yy~sk!0U%c8PlF zMs7uLh+0FJ14N;uXZd-ey@ZG(M4nAI1ObWn_ZtG`kdf=4qC{oci90)TgYlPM zd@hGp*!=qy_WZ@Bz%)J1r^CG9hDELwCz{#cIWQlUf25Bt>nJ#?Am#p5Dz>PezbwyVqrnZ0DRW zOUrd2@oag@oWm6f7pC;dP5u{q&r#{}VK(W+QjQs&xlV*~ShY>l4vmt@MMt*ZI@(u9 zq!o`xJoZ0&E`PlihdrCl*(>vf2S}U`+?;5Yso(0~nT)thXFq`N%JD9Muj?IB{N!<@ z%HX8o_>Ikp&A_H>xC0mtVF^2Cc-HX?BZ}>eEiJKsO1o5x(_c60r@bwh#)F6Y!x+;0 z!4y!GM)m$=VXWW}(&$%puU_hc-CZdXg&q+Omeqkv3Vmb_VQ#yH1;YS~%&6MXld z<&$4+8WrpLr(^Ye)3`xg)pOe6S8S;sf~rbkwf!#C%(?Bb%f?~h+hZe+g) zz{vnxzUZY6Ch8Uelh$nLt$`0JHVha!@SRspaXD32A~;LIYO95DG*FadE2hkKL+-iF zH#>yP2XcOAj>!vPRsr{dRrNR68#!;uTW))aclIBBKqXAd<&O1<+#J$1MN$-!Om{8)eXJJ*Y#KSnGP#4 zW%hDw4kl|s#;+X}^}ToJ2uE66gz?KBDD#)SYeyPO<+++?7F9?2bS%@8cx0hddIxU5 z&?r@ACjSp#U;PzT*seW*w17$~DN2`szyMMzAPv&pB{g(+halaZ(m8aOba!{>kVBo# z`+na#znwpTS!?Eb*!#Y(T+R+)B1Aq8&bnz*YcuRbxm$3Y>CBu^;_O9j2%2iy8ewJ@ z9CKUGV_C^9o94IvPz!QU&y!5l!!M|aOD@$H9tO*xm;b>~MzTGnO@FV2@vTc>FDLq? z$oa3SLPG);EpvPG`_+J^e%s2gU9-U2Y=)3TYJ^^g4m+W?W~=eC(OE}ykRm5o#Y^)2 z<_^{%QlpPv(1%^ZN;t1vi5IYAD;Hci~5t6rwXqkf^As@ zZ1xow8d~ulxQ!Q#ud^rNSh|dzi}ogahfsSp_?w z)buz8hSwWGW*}|p5VNOhlmENcW-8t&bclNenYp17O zcEbj>9`l(0a`8nq#I_<W=TH>T=a>?-@x!8{9&S)1XOVHzh)fir=TxL1K?oGTUO_sP% z>L550BRXk3yQq8T2n>lO9k#O#zzZ{2JAq&np^oy=s{U%H9Q0QOgt`StT|IPaeX|8a zhs9=&Ym=hVQipbXsgo=0ipO(%%V-iUo}Hi+ZhYKpV5F9>A|X{zq<8?2R51Q zi+E`YU*YK2_^|k=q*l!na+KVmdV3rHga$4&4S4A(`CS9cprYCmahO0rzUL+k6xH6f5T#&%9`k|nuX$R0gYSTfY`wwMsDEtrX(uW6=tXjD~kdaxv zn_QW>zgk&g(jnoypZ?jmqIy~}XA?T)?^~_Jx#?uo0h<^a>A*tVC8b+0KhFWK19`?>gH*0z2%A@h4G@cz6t_tv-iu;T%aNo4KBVU-aIHwrFv7Fj0xmN6=Th-4brc$AVsc8XuL#j_~ zNG~=&o4N(v?ruI-Mx~7(m+Ae!kw%duu^RT+nX0d{EsF%-c!VHC&+P)frK&digIsA94o8zUPxWtCK z!B1}9{%XRP(Jm=(fv5X2ED=6oqV3xuH=))vvMQkxekE;`*Jf|&)!zu3AT{O96hKs1`` zG!9xS@_I3Bp_*pI-xdsT-_r?-aa|dkeHaq>1%I(>U^XWXZvtSAX--U$Yk;DaudZ$T z$=(YFnLu1G_YL_0o69%%jyL(<`uGX#WmmKMIE@=2B^&1N9qA6ej`M=ojc1gBg&T{6 z#4hV*dnR8EHuT=H}_p#sKy@_D-YDQ$Rwem!g%sMqw22;jD=Y%yR zfiR;0cn?Kj{t02uS_k_84I#6kz8`?kW5}<3{KM1Z9RMnM9U-zYzJYmxhNOWKn;}8m zWp59{u;IX2FA@k#rfcE1)dYrgDin!pJ+JZ5c4-zqB}8RZRxp*yjRi4;02j`b^7#9l znFfeS`kWe<%5n|3W~#4#1-yK#2NY|V%fNue`7)q+T@p>DcXQH)kx~G_gr|S9<>yuX zCV8Fmm~Odz>q7qB#N5~usGX1mLK)6uEuv(&f!+5`va~D@L}lqY?Kp7t`@ldxQdprL zY8cg0#{>ZLb|4yHR;dk2ZUN%N0JLKp;3cb##Z9|$xvmLxD2LtwY4tuo%2J-Ev{&- z(CegXD)>9phPx2ar_mS99KaDJ;y){ZH97%;XvPgN6m7>UYoXUfGbr?kZ^nIz*+`{QPX~5Q8A@8(U61V z=x^QQT8kLW~$H1Q0-G^1|(%@B@UEj`2J zlajSyVbKxEA~0*1L%K06P%8bvAi9LC5B9qV`IQWycqj9^ef52QCkqj8NMb@1`G>GP z+q8v&HU92}FmA0KrVct0{InBCLIAB!-_Qw2Zk;XJP~i(WcoVRzk%I&VYP?40i-bP= zHz$2s?a>av?s_~exNguBrU`_Du0Z^N^`sx;X0fsbzR!ROzgo1F!JK^q*?UtC4a(!%33iOV3dMCiPha{5yg3~A3=#b9Vfd;eUai)~;0$&JJh|7YC z?F$6e&hED*kQrHJxA*f-%Q(e4fG_<3D4ds|`+#aFeE$uHZ!)BDmIME?+2>rcgWeLe z&Ye15t>HlxHb^Bf{)^voS?>I0GXG2;wRmb&9kB;|Gdz(eQ9Gf^+_r|Ii2d~`vB0#ppyI+9i5Pp`6 z$d+1>{L!)pt_SWYm>Y%8#;7t}oT$8W`Xz61IjgUn;CDH>NE@ill8`#;xw&a2pQtHm zKlF_SP@Ts?a0392s>pjl&gvX}Q+AQB4`&h5;eJ~#(B1NkB$=$(-sz{*+E19JS)W@^ zm;m?S7*_r$xmWep02a^sw1hKM6|KbzAc%ic2y4>a7q7!n2Ys)XKVsy1o{6rQoNzba z*Y{0yTP}DG?T@s^A$@M;NVbChRc&y~g$z8-EIJ=>7-IK+k-^6BZ?HQthF>(~>{@XB zx?sQZ8j+VaPm!N15EV>x7^6p}UgA7A#~;0rLy+!h@0+gV{_?aIm_As76u9%}TU`$Q zU6+ZT%GbQ$AaQ)NlZFH4Mhnl&{V8WuF5Lq>jx=esIpE~fYb`a4BnL&uvxG5=WQ)nM zkA!9cg-pg6t|O8;!V!C-pkTs`1Mp>a$73BykioiOFoaA*fWJ@D^@wSrHm3+!2sK9Kz+EX zc9=g+Xt@t$9Skt@(3DstX*}7DM>uVo%1iJ#Wr|tgn9%eqknwN?nS;0>RwBRO8Db0E zLy})G_dGKV_Ri-e5cMGR_t8Q01z4|SbWPvNEdq6tK&I;#eI`RQ4MAz(u~Cvrf&CCJzyRA+0c{uLNg>BczsueAr zcm1ZTo{?C%a;A|f{SV^|Rfp(HGyhDNo?_q)FY+C?4;_&UfO<=-#+v%om6F9?=d&3A zVsLgDOf@`racu9+dvG2a@H1=Ky!}8K|QSufbXzl7Be%%Zd); z4?PG#2FC;rrP0af&0dSNqIK)zCpT;%*^GWeyeDxztT@i@YDK6PoaK)`Qp6lfuZm4P zs-LEtxXyLwf7zDV6tA6X$b90K#`JC}`7R)$iX^a**TO1K+I7Ob>QZ3H`*J$5@Sl>v z6U=ZxIgvHXq|t6p{CP_rpu%va2d6}x0j8KXqr5q@k)`6B;-R^G6~VkMH|RMg`&HkL za1*qP#M;Dt;(n9bD4nPAV568>FVC1-n*C^@`rkV)+Kwhi{!fhxOYMgWRI~5kKK`MD zfcE{@{#1#D$IY2B`&KU3VWVuXM!(0CuQ1d?$@4L`#P#9fAm;C~#Dq~2LH=hZ++!S~ zhuQD!_iJZHo_{xLS4ug0$(BY_+M=azd?Ai^jV#F+^F@oJnA$a*7%u z*pLnn6(N>)R4*-u5@@JZw@RXuIM@gszp#s+r)==ES^Dg`oH?K88)(yQsU=hY>Dfa|Rkz8d-Sp}znIuh7k ze!5z>UYB#Z!kU^cFDjC!o0}WhO^}v`9lfL7d&0M)=1nvbIdQ4hW*H%QrYo>yTq+1S zn>Ig`#%(xqqy;oy?<>cH_eVD2>4xGv{II+Fk?;3h@TUTD!iubhn+L*?f)7UlCHWT6 zIcj;<%=A6iXwQ}(ut8-HYuK1$zXCa&9?ys*?ei7~0?SAs zBwK3EC-#VPmb)-pu8*|d_t>?pQ>*^7G%T&p@-3;oBN15HALtkobu>E5XJF*Y*7bdm z41~!LGQd6AC$!V`d-yn11^;;t4l6oVZ~3h`nQ(u2L6n0e+KMK4V0-Xt0$6(!P(}Dm zr`_(F!@ji=f%&;E=K1vJqVroUFSFtR==WZ|?i+@}DAGH(CllwHciU0Y#)&15_5Kg! z1tZsHo}Y2I_>?`*YD{N!JYiciW?h>3^0}8-Hj#p1#B(dq4=I(RJ^aelNyp1uem-_F zm~DMj7Gns~3NPYH+u8hK-r6pgOi)-1>fH(EZQZUcD!Iyx67ExTXHODY2S3vbx;D8h z_Gm<7P6YGvcUaemmXa!0WWMAfJt;X$1#%x;2az-)G=88`p97r@qIAC3e&c(qZPR|Z z$R_RK5m4NI$YaUbF3aBg@^TIC2n=gdTpyn#T6>B0?J{Ig+AYtR=JSBDG5LAs!n%Jj zPUIlxWVZ+PAB#rasE+;u$s!eIwX?aVYXdlId zH1&3aYuN@x8NOzBR2t!cHD9vH^me56$Xsx1j;Jwa!MxqRDFHg#T=sM+r!)X{Zl@xX zqsQO#!7glB65$=fdX+Pr1(lu9?Xy6Xgqf8TWAEvZ*mQNd(f6n&#L)t9gTM81Ad%F0 z4*x3KFmKz7RzSbcS;tJ^)kbj4<2wDh1Slh_j8|u!x9%<0KFy&gUKKKc9&xIpmoG4^ z{;l9GY{lUe4Fs1$^SvNbayg?A6*KkP&6i4d34ZyY*9NpN+Psk=NL9IyQ ze~BuZi)jr8V?eU#iR4?e5K?FvP%Y}}m{VV1IVim{?=5N^AyZo3jUW2;ST3qc<@T{y zQt6{{&6oF6VnMiF=L217$o1(i>tmd*G5_ELs4{8(lzetk=%l2$-{p#(xc|JdmgfQe z`OE(j59nXae-<5X=}|17g;ulWUJFck2cBji%#sTXx>I{0sUTz=0(H30#Ih=?T`|OF z>QBk$Ks*aq&;`h)e>|Umk$rA`A6aK6RIIZ(hM~Q&DBZ1^>7IFHJhErY`Hz5n0zak= z$Ka51=cIl5?Zcum<)hg{>r}*mXFjvXQ#X6Rs6h~`xH<7qDleD3TxOc#K6@tH4SUoA z=OhRD=Js0}frsH0=N;08K!KZ8B+OrsxqfwkoAu4&%dwt_^&Q60OGUZ7=GaNSE&J&n z{uzQ(c+wEP)epg;LEMM*U2dNSpb~inOMuYqoJIKIVcctVALLg9z>SAZ+0CCH2PTYF zv5~iM-kv4=z50r;j=j3gu=d5gb&1Dv`SEU=Q)6^$2K!TxTj9~q?(m1hQy)xQ$&GJ^ z$c{Nu`DELDqNs1bD=S~r4K2>^U{swfH=0#U^KIp14~?RxmwTKe{XkG~z-z;cqc!^% zWWC+uJ%dc0c)v~@uw;s}?5h9wT7=7Pv}9=_ZmDQQli-yiKnp!HCRdte2OgQfalm?9 zF^KedUFmK{h1_!UaDez=^}V}}Vm&E?`DUC3nbY%^8^(O z73g4Lpcb4KeYhjZpDZ|+b(I!>GCijf>AWY7(K9dhJL&ueK0 zUxGMi*eK~(1-&i+p}s;acLubRC5YBx#r+@m#QwmBz}8vZKtt<57hM@BRplPH z6F`!TE{(x4RY1mjWKn)2;;-pq)1zJ5^57J?D2g2d9o;Udhah$aJK-NG!|}_+6o#b0 zm(}D9Xag}+?j+%1n0lZ7q4Qmge!IFu9o0ENcLeCp8r+u@)|>w#4MCym1fg7B9r*Fp z*1i(AYc_4SkiBG%mA9o6?hT|t+A0?aaig`|kFy^Z4GiyT+Z=w)jDH5AD$7lt8byNt z{dK3?n&nz?=?$L>s+-MnuMIMFacrcBWf31V59Yu1%Ql}4tKC+AwT;Sk!gLP^XXjHF z7DIvn2XS)2g_qZFl*vqMWkJ$&Lj>o8Pw)%GbkODmq8a? z*d+J7Ha1(8rBmLXph{2lsEmEu+=yU130=x!s-0aNHrthza`I@nRRs~PVo{w-`X%*B zAadTGxHz_E;L+Mo?9D|f7Ae{L%6ou6xj7aHn2^ixypN`G`9Z5hQe|x*6+slOCF4TW?O>v;`k9 z53G4hc#2wzXxD~-N(6P?NgmY|LU>Ee8dM6@t-Py8tJOdFh|7(ujhKqII7Mr0lPO>e z1j8&0#DAPDNh|3n^#YSC7JQBRk*Vf|{JQa>T+WYQwG>2CFydDj#PSmmzFg@)@E0&N z=rac)OhPJb-z8+g1svHGoMgLVvba#+=u()JruS_Ec*A;lAW;jhw#Nd&XT)^e3$ zUfxyesRqKbydeeR1c&9sKt7w5Q4*PZfnuKf!%+P;Cu{21dW^my<|VUItG_?6L_AQS zE?JuXQUXkjmex(q3Nh*{Nw74@kmUJPMfE;QayyavprZr?M4KcuV-z5@FP)Lqb*9u1 zI7!R;d`6wM&!2`*nDu$HDp*?8^tmPb-Mxyeum6nYNz{561s>H_EOtJ<<||KiSR8J9 zy~fA9Yva4;Y}lKz-AB}u<=l~DYE%a?_9=+i#jf~sg_Pgh)2d_d`GXyP7HrsW+lMY?pPe=U2$=>N#WWQrE6N5nO^8J;br>4mrOz{j4M1#GB z2oY2ge$S1I#$QUq5aJU@Q-o{c#_!BYTc(~uPTplSiyR&#rdg^t3NABsQm(kZ~ zF^YC_Zf1GuXS)$R9Cke)Rx5wo1n*f`7GvUd_4(Co$e`pLZ-0>Nl9xs`zz+*IcW80k zw6PYXQXP)2%@CgbvIFP`KC>mM6R| zza35ALW@p0e?!V*)$W753=lBpFub;PfJyL3+YNK(_%&Is|BTq}@d3{tvaJxIk+$>i zD}#B22b-y`Af) zN7zDT$s+D>BWbN~|Ls|Cdh=e{$klg8>Rs!ePuo6Tu9 zS?!%GUb(Nw4Sm^ubHWmZ$wgb8&iSfOl%5|pS2iy5@G)gdA^lQd!gZ}_9<29T+n2H) zao)m3RYz1@bTccsASBnf^GZlkm+Q@t2i42t3L70oFE+IW+vyTZG6JRdZ1c6k#OcO| zh;{4Zu%nRoqyso}makmnDn44d#&&H} zM`mXn$Q{wtc(2sdfMK{5>djxwidk1-emjbf85H!DT0zb+0$IV?7aLF0wXS3P;<3k!~Jqf$&)c;*vF>Vz8K4eMbj`F zOdg-LoBblT!XNSi8$X%LIv6?vO-)JF_P~~(J)gZzvNUr{_%doQCaUNsf%iRIbTJMi zTphX+AiH>-Pg&hQoJCSvy+{2dYenbS9fE&F_S%GJ{7m7fCO0&m`3oVoG~AO?)0DMS z5mi*6BVo=!N@QnsL=cvKYsBf{zIz1p(q0RDtkvBBSD+ov_mEWHk0oI5mZM11IpaN5ZB7RaVY-E zSYHTkr(8hK(3YUAJJJJ%9FKIty}F>`U4#0E_;>Xg;CMH-+^v6j$SkD;4r*&WY9OQ_ zgg+a4QYqGFKuA`Faqd)+@(3AX^53uwJiNS5{Uzfu|tZh+_kHhFE2 ziA8(Yh@Uz^Y;)Va&L3VW@yiWVZg>k*k@DZ3gIcM}df_yOJ+;qbGYK z+B6J|ctFg^k~G01*!mcMwM)~wi;i6^x&?W%MT8#|?z0XEJ8J_hT^u5Hlmy#fB3179 zu5wx~&MtT6j&%nm$HQ&6x(b=KsE@IjjB&BGFXb zH6Y1n&CcmF(6JhAjz}ODMrT-pS~K+`bNRKOV|HF4DO4EZWQ%u$fXbnLzp_h~oB_dG zK<+(jv755?+2L?D%)wz1U5F-8;~>8=RaVfDDutzg%eP;dG|Fwd)~ob>7M{+q$Cb4r z3BBJfO2Cpm&0*0HmP=kTZ)0 zLW@qmQmscxZ6LVx;QH?1jok>Z zNgbTM{+RvVgOX)F5GrqrI~afSP)eLm7ki4CAG^u-I1@U3<0X2n0K}c znb?PYB3FX&U}Q^_(6fOkEGhn7<>0h*f}0c25-?P*+nGfW*(IZvp@3O_v+4@oFUS`6 zZ{))8Sw~FK01FBuf#G8WYN)*yDy`&5k{jJOEAGB7cjQ|S{J5xeC;qYe0WbS$uk@G_ zkD6xJ2~n5$vzBWJrxlTri)F52XL-B3Oh0>?^;cfupA@N96kgRz zeFpo{Vw?Aq_VvX5SZj%Jpw4jeWaQh4^?UovB|H;&J?3I#3R)emOIZqr{SxGUj(JFZ z;A4w`WV)e^Gpc*(qa^bP#rTWYYt=YMMJaS}M32^k08&%`Qd8x z2RRn7r^W?qi)eV0kX@K{tKCWVNdYL(|0_u$9*_$o-jn$5)sRc|l<$QNTdqW!p(O$j zcshFTkoHzOm1l}x-bfn%fv#A(3#cVH1ImNuV%h6YS+}DV%(U^@V2O@V$`=H(OK(tq zT}WJ{rGVZYE!1)WqU4&4_)jRSKtJk=4=Fy~-$ti1y(|?9Y3=TD$1<&Eb5BL=px?6C z0vHgx;xIMjqcV)qiCon3c6%vWQ*XU$uQbk5xVGFtF&;=%nW{hFH$3V#B;707t;S%^ zZFU~xpSNb8stPK(#?@j}<=+l}iW7_J;>4lk4{4&m3oNg+neIW z4HapN75btRRmLL<4w6o?nP5N|Ery@q)L(*}aDVw}4zOV*zN=)4t@utTZP^_Zc#8UU?Q$c{nyLHwhl^^bL>~R zjwFTeGjKX~sgy~^#S{@D5Ggb|@EEPjyu`T87lF0?qER6py7N;ogXcpXhIE3Uua`uM zqu98g6~g0(2di70OWv_3$)}=I?$XfUrTk}g1KB-^;}=CzvI~!_pJQOKola21$IN>u zC>1BmR|)THE5gF(6ioaX5A=mm^_pLXj;=3KOF9M7&xC;Mz1dz!n;9~LNXylLLS0da zm66Ej3e6nRu~6?0dz>dM9Jv_?wk-pR-f2&M`Si-^YR-6Js!Ce2j)!{<5#-Cf<@8Fl zo=k3lMZt|)8;nzp6K_KmQ{`Qhg1$|7$v|r>l!Q5mmr>8`(S!THXZGNM)#-nK{mzK| z@wU^m*E2T*r!<&;Y(jMMq~V>YX8b}f#W-+Y>1L`i@xQAdZdr&vaIYp}4)al?ZT>)U z?X&)nr&jZ8u5tjbfe;>|Q$-D-#KT%O=^Sf+({F{TFeyY#iuw{!?=pc@MlOhy*&Mmo z0tlPE5P+rM~4%rL+hW*0Pljt-Une!8T<5yyvq~s*lSaT-V>=$ zSVWA(DWxBL@9o+_eBZ46$FIm^J|@)lG8%_4rF|2^x1`ERXj zqx`eouB`mepL$wuXWe0Y7#ym57(57I-KUC<-iarkL&V@d)}YViC`Tb87?8>%A9{o4 z#HNI!vyA5@EQ<$WlK0k6BW;x`(q#)cIsGgno{4+@dJk_9} zjXU60TGaal6}PuM!Vls$vUhMy;S^l`1~oeE zh^`ns3%;y2ahCov+!&CMq4rx9_%1Q~4o1f9u)>@!uWeW6rBpAJX`l(-sZTRoj&eDX zEXop1`?XCte#Hdk`ot8vXaht0i$NIsIK(v!0*Vs8h^Nyn5+m;vH)gEw7bEX9SO+w$ zbF`*Eb_v(PHRJi$TW3CLTxV(LHSQ$xE4Mb}cwa1E;^u9_IV9_u!gM{{iKas3;#nmh zj){t$74hOVRT4Z_Plo|+VYa;f1?eyV7|hXYx$Nau6gCI}BjLS&yipEl_*pQw*E5I*8i``L!9(wP)J_jsmmRN|C9bdKA z0l@+niQ13+>8ij=MG1k&PtnV`GGBBBGN2X<^9XwY(^k#HKPz!2Ux9!Patpaw`Sxy- zV+Z;eAA~yY8RxZYOw+6M=V}ko$%hlE%z>n%silg?&wyUgUp*1zR<-w){(nkJ+)Q-o`VDfC zD%843xNUOyDDarag$Xqc<}ps&U*VI#b5}%tDz^_NJ^_$&)nmPrV?y!71JqEv+vb~o zN!!WSsH(4`n5CK5yz+6wdD#>Qo~7oD+78GEn!fJU2m|155=+32wSwOdlwnc+6}sj(k7Lv{MY%dKDm<*yoY(Vsyd@kPxa~eH zceg7!aWnH@u<2ox?u=rPqkY~K&IPh>v^uGRLuzy^YKUX_^Y!sjB_>vLc$|lGMP*wHW zCsQ5%=Yz>sL$&M0@-jR;_)iNb(@8XGX?ns?)TxzTKnYquwdQQJ{&}KBDpa9~VuS^- z4lv$BD)uH0)-!tlewsh7w0wBhJxDe`T(O-rT}ReG4IfBNrDh$)NEjw`M~^&x?|n~Q z5%MFNMh1#9z8mU-(C-7Lbh~z;`?RC=;4$N#w>O!qSy!hlP#RaVWtIwgZ*QNa6mPYF zj1_j*kEO#VFYSFm6Xv=_OPep$si_RJ(M0}ShaPKz@F>$l^|WP4rEH1mj@o+A;Evq7 zz*S8yOYh$H@N_A#{`9RZ64T$M$Nge>7Wu6)rqdi3BXN{xwH)*%@mF6#e-N1cgl#0U-pWJ4Dxa2lkl#e#Fza~%C-uOVA1825&dJL4z;ytPMHFX%&WqWN%Mi8{H zU%gQr{z@Z16Ng`yz`dauSfEi1e<+V$QTIf63PwZ<>%6g}_UX#=j*M>0L8hz*dErVs zYxugp4k`R3#~xweTx|3nVVF9(Slt51#&q0%h6kSPVm_$8C11G+noMSVGOuy|DUG)x z*`<`9K0ld4U9`gg5HLG+)J3SpffMF39&@p4UV04dHhYuuD989l5o(R|7ro@Jewr06 zocg8p_Nx%gDCWjfk^ywY#U^SBAXg@FJ)b@XH3G`S=C-xE`F^3v=nR0Ea?g>*9FG_0 z=-(kRei{vUYvrI`V>&^N&yppq zIggJV`Ax`NO}v!em)vpPK7qtu@#4>o*((j;j`xgQdhdVVLbBV&T-}6Ai@|e{qX=l- zFoB`3jm#IL{oS%U%2_CHs-behVahXsa!4PyM0?WwR8jx;ng{D}#q+6C!xP50{q%Sv zV7tUYnB{@l4<&XJ@*}RkQIRXn2HlfNV-Hyo%%|8MbjBM)SqlIF)e4Qmrue`U%K!BeC!u| ztnSST8NTUPgyGmD@4IT6E$ap}u6OucOx-0K(0Yv-sy5Qs723W~&JUB0RGU7&Uo6J& z5b_DdjRUD59g`V0XxNp`>4gq;P=}JpAV~8=k3{#_BN>uD{!pVZ^~RV&j3rww_aXUW z7gi5drzzTXJwFpoWN^JAl$&5%8^8aC$42~b-y?Ob?2g=Rkpg2^eSFG?_z}HJ(YSvf zrU*C^{=?ENmJgD5VVCSr6()VaDN1*N!eM2DlcrYl7EQ&__#Z8STwmJwfzYK`v9vG! z?R0)OEHa9Wu|stEDN%@AaFAgo*NWRjNHMs)#PjKq&N{Z|tdX7?4Uzt_E~sm?zK>Ik zzKsqA6?Hf;B);Ips`bH=z@vIL%-Jn!s$|a-#-o{hVA;@XcK|>6`GDh|qP*g?5LQjJhk>*i zOrx{}BSCT#*bt|E)o`PBEM?V$qb`OtNIF4DeElO+xo)-K7)UykTDMEiVwk^sa<-=M zL3FkJC9#HHTI>{SBd^ZbH(JAvA;3ctPiuRu+ex(ec6Y2*H-0L+V>_^<^>#a14f>>1 zsR;$N55+AA(LL8@bIVs%q_cBx!hKi)u_29RE?al4|IQgIA08T2X^6P( z&ajkXiP4b#lNkM~jLcM_;t&vFVbpCNRJeL>ur4l~THGP|2Ln>gR3YXAHVb8fo+@+g zxH{ZwF6>w$pd}MsSOV$$tx7LK^gdBC1ZgVq2A`K=mZE!F&r90QzQ0pH4QC!E?GQ7y zyAeWMebg;I9v{AFF)i2KAtUKYQ+D_$M#uAX2nfTX^VMGyh77A)R@CoV8wM7qB&fr2 zs@gmv)TS&RqgOSc;1-4D+ZH*mXq+X4t&cl}zR@R+lIMWg+teGe+N;ENxce2eC4ki&F-H$8^Kbz1-nQO(s0%vZP^xMq#gg~p@QEUS!uWSGlMiIuTr0~gmg!f|PKQPHGv%Q(7iJwD zD2DlwNtT~U_A_>PX$nTBZ#3rd!q++u-M|}RE2O85)+5gcFi;2O*&MOxL$R92u1y^O*K)TL`^{coZRZT*=ay6; zPBFfn1a?Zd0C3F(piQkJEJ2Nl5XiV3kM)@U6MweC8wa`i7shAwaxo)Zf3YLBL*?4O zM$a?%qY_H>X8QTc?|L%z37C4HTX_RIhOfZ|{jKN&ttj|%-@BiNuyZ5$;s3~5AuI0J z!oO&}KCNaemuXG&W20JCVrkJ8I*Cu*YT8+V#3z;Oz`0~dPO}fWffR8r35$3o$QP%5 zZudY_2L1fkB9q^A$YV40Dnx&^H9-^}^XRQwi>R7zyh4O0*_f~FJ6o~^Jn~%K0Rv?{ zobewP#>AIJ<5GbITP{l;Pg$~eaeemb^&U)tp9BnDwtMUeLvAi9!{HCin8;Y2!c*8Oo|<=jQi z7^|Qx82rd+>K!tR*ckj;Wjpen-WHf1A8U2Dt&+7YCF_w}cjF*K@%49w6eb@gGA{)2 zj@#Ke|B@6^huL-#_lq$CJ4kr!RlWudAu?xmD)hrq3lL9}kSCQK@PHZRglvwmRqW4~ z&8VqZJ+g^mIlh%%27#xVanWnv{^I|oNHlXYgU`u4V#2_DJXKa&x@e6)VI`;8FW=gr z7ARRNVENx)KJowh<;yAhuTnlUoAj2&^G~?WY$dNR+n+VY+vuZ7?Cd1Iv7hol8}0on z5NIKtP=5SJ_cvh=)4Mh?Fj)67+{s$ODu+ltnSJWd22nWgG1%;{3SOwZuX^5R-Sqa? z>x%^w(!4_~e@oE{y}@M0(x8-PCFgZ0eo#RG1_jh~I)eP-k5ZOQe^|V- z^29vgyj4!1?!46wmm|P=^ zQ%5*RI}KzEbg7CQb|pmGBy5Y(lvL%Z?{dz$UoTHL5I&=c@kirM}cJ${&T?7_r$SZz6>Z{Gy%T**AEl0qsU^z{n zkfch_pIIF`xM)A4i9*Wokls1{$9fOomRWbXEzN-r1m3x)x0+2d0uP_Z4`@I%KW!BU z|It^ma>QDSiajG{-PU>&!9A` zsh;k39*nXxC45-4SXKA7$~QnW#+*&1+9ZTPmfQXLw=7uK#x+9qfI+9VM2E(;X?58?@miz&`N;x-Em`? zdv}3*wFyWR)z7UY4XkkbfJ2Z-zG+_Y^%YHoZHBHP1v21BQ?6~nnS~0axE?laju}XF|yn#pBFs8e+Hs?t8Mn+t{h16XxZw?5+&Ta!)9zl&v~$)NyAMcWDz6+L~U<@zLT2C@ZJCP zJj4o+NH{p?rH8VTF7;%73rqrnX5x90BOcu*q`0;Q=NJ{8kelUP4FK8w*iA-*OTS?a zY)jc}aAKs{T;@lqqAj7k#SVSmlVD#Pb~@{VRqb!?t*##Dq1so8 ze}f7~L4i-@z;vZVK)=J8cm!Q2eN~1_FH5@>_wmTs9i8XkT`Mb!dA#!gt{1*k2hK zi5yQ;muV5(iKw$(Tma;OOnf#OZ9^ng;w?U1|0A76PahO58b$5N6Aanlipua3D~`9wA<5r!tA#fBP6g)P{VWV)DU<;mz@SkE?xppfb}@uNV&(j0wqTN z&}yRnxI3BF!jLptF%TWTlj#M`u-88#m3GYc#o_DW7vDG1saJW9mo-Q}WU(08 z{D=YK{?@Pq&nGh(((Ck74QjnKL<7MJ3kEh*naw@tetUOLN}4M__VvENE%Z^Yy@wYl=ciU3nYwPto^$yWhYF z7RSCUzEuxd`+9P_lP=>wwq2QU=7+oAm(`A&igzbzC)xx!Q%21I=8+~D^Cu9oac?UV z{`0IWE{4Xd1D7Jvg&Mb(MuXIqB~(ME?n4^Urwfq?vLkdSv2KZ-OtR_+Bi$3%$()x$Fmvh#^&e#as>!S%kgCB8=w5}y6M2~)*#k(6k#3Dmrl?M7+28bCs zTOe6Y^|m|nJZJ8-xkcY(-Ra`5Hs79zs=-0upR$*3Rt#OCG=jaj?h<9fp#*6`!PRww z@tk@Z0YoQDSJ;xT-pdYCRzG-E@MGXheW!|z@@QDLt8EUxVZZI>8WbiS#$!l3kk(G) zxYJ}m?&|{trOiV#D{jvz7DLrm6O?+ThS{y3E%HpQ#V3B^(3*7v7h_-|ylTx8E%a=E z*)hRZJ>7i&M7uYDNxh{%%z53p>i_vKo{c@+1&<9?(c*^NjeWVt6c+Pw_~~v$4z;7t zMjF{R{cG9Mt`HpF7i>Z5x9-9^C!Ndze3x{OE z)T&|5TQ`~KQvYdMVhHSb)i68b;(@k8euhEnGkR6B;MaNTgiospL)Mf#lzQ?|t`9!) zHYs2VDw|kV>y|MR*34z6s{=Req{1&lw$=aq_rVv2_YW;1OdFh;IWmDK=u$81G*9(F zXjXfIXqAZ)IS_r@LYBcmC^N@2G93MFE0(Lh65%KUyj@G08q$UZCh+9&Y(|4;(-YRe zmVnb|54DPI5Z_mT_=n_PiIU`_%qKEOHX3dbMH8K^wbFryq@qS@xK~ltfx(-S$dx)0 zX~gH4+HCX3C1b;IN^wQ`An!aa|f zya!aIRpv_csuMNx;`hPCA8Eswn%v--%%1rcVH=Orh7z%GlqYnFKfA8k<^J8izY_R^26Zb}$*n(t?9 z83jgMn@{(Lel37hzO~Hnou0i;+x*WX_%mbn-;ww#oq^u>a)1Edn1<@b`1k0@<1~;1cbP|Gz8%dEzEac~VBEW*{HNXTSKqVXzzr znaqQy-^&sFoiSc>+<=%_fkF1cZ?eegt-#{z8Ghg(7EA5Nx7z4(xbvk)bsr^4s|=gf z2*POmrDJ6%$xNAnhf=dWaXTR#!r0>yFa;EnbGlb)RT+7!MPbQ5J=|!PyJ&Y*pNI0& z2{L4oZPlzp9g!;vH!zE@JTLdw!@)`{!*G0u1V+O-)Qi_q}oc`m=2?_W~671@y>BJJHzIPcXY2GlS&vPLk-s&69VXOA`@Me zH!&jDDK|i`;Fq@Nc5Qt8w7DFhae06J#ee9C(T{p^CixNhXkqJ-7Kf7W64`n21q+-) zCh?al!z4yzcC)QO*|H7#BIb9uG=(~S0dz_-_w-;zP`+0XFL~lH&6CK_m)=}sI&jpi zqDUHfdK|JJ7C3H7`G5Goa;uO4*qI!1bZ7uCo zZo@_z$vYDOC}cB@#;&Zn=FdWt61zagjhB;j3~Z!q#+9J^iI&%9ny^qVPx60gOLi@2 z=*h({EUU`N-sbn0l*WK$g;{;2p|f}@aIwNqnCaJp-L{LW?eaW#$wY3tm@&6Bz3fFD zs0Ro=h55TZxkL@oQ&PDyy&@JgWLNnBS!F?Y`EQ2R)UTpxr;vXs5(d3xwZ#6lfFPuR z`&SxlrFdIi{7mbw>UL;)1fm(BDs75Cc4VYPcY+C(DoP^h-gdkjz$A$hrZAIY!^e`X z#VkTX1GDvzpOJ#mB8f}9OI=<1G|sr*1i4$wV3ereBM}7 zv)99t{850$cWw+vB^sTB)M{p?dAb0*ElbdHL844_uCb8=-tN_DyB-KzYrNI|eCTBR zy0b=c`Yq?++_}3l4>byy=(8tYJ>Dr=~hUCk2&(Y#;XW zVitLqKP`qdAz<8AE)msJ0`?Q9`6uML|48Ir`EjlG^PskCU!Y@3G?!{z-Kc0mwzH3+ z3DQ*Pqo8{lC}J2|9m*(+?l)&=VC65M$uX4|`@sJnPTRni&4|PjP=;D7oT`$q>>ndBQlW768j zU2i>2T2?0HdV8q4qfbv7k`NU1$i*TeWc72q_1~BZo^0?;n9{si@HR8dQ(lD7aMJ|{ zk_~-kz7n_lPgr=x%X4EW(?33e21k&03P*K^mX}~1^!$Asf95s^=Aa3`pt9~TP9bD>H_#6 ze10%~bQi;GUNSl@d|VIztubrcC=usibv+f8=|ulyEG%+nOFg##sel*eCXZ5hR&j4J z!+`nS!$3QqnFpiHpIzJ_cdVUwFcBc3ieCMdnl?EMQ~Kx;@2@i9_Q?J9{n~7He7C$9 zVJt1Y$de*<163qw^L1oxwv1UP)5io$k<@hdK1nG}vkg;BJg&rjhlA9xq3Txt*zF40 zA&gKj#JS;_EIG65B1WCGF1diI*Vp#K+6wMQhuPb=t2lO~>0QomHp)xyqxOUc5IfB! zo-ALy+t_``v}B!(TcD;2MJD(-M${rp60kE_W#}~52!3^6YV>y@2WKio*e6h4a#dfo3MYU`yNnQ4!|@w1Je-|Dp2Eg3rjVZg6uA^RAb=8boEMFzt_r1J>; z`(KdkolQm7KmFJ|d~c>5Bkmh7%(m$-4&zi$C%a(nK+dnnAYmX&3v>S8!NI1p3nwAB zB1jPHha)sF*NF5R)*lbkwNgGU#ddrLv&H=FbcUKW>51`oBuzZq(xv+UY9+luJ3#j7 ztEO-%RpJ&>)e2XMh3i`L{XvOFW)wS)iq!A}ruJcCJ#ifh0De#|`_#Hx`-v(Yag`EQZgXRWxK51IIs&je!Xz1T2ls1==|a}HoF5=T^nZ# z`4iN8(?;1e%!9uq;D)CCNq2d>yV^NDGuv6EbB&3cv!WV&qC@sY%p^EGLz2o&d%v5A zucQto>`@2cjJ4z3MfrN?{>vWlXzP1+Ec^#yBdNFHh!{vv6m_wl;|a2B>V15M!Oia? zX3S5YOok2KKJqWnL>KP^87ZxHr14*Wb>2Hkim42iXw2v`+&wXYWL>qifx-koY7-^F z*4PHyHK>Zb=hX0>m7aC<|EGNvqKWLbQ^C-CLf!pn4iRZ=2cdxhl29s`xdaI+Z`j zPO*I8DY@-s#2ND9%-^K?Ttq|t<{r6i=(mv%ZwHR2vwNF*^K0-Q`|H)ZT5URwL{fHr zjy)Hr>W?!lg&hGdBMdLMUMwq$Eu4yXGx5JpC>YZyQPwQ=`{fn}Kw%(-8e7;bjaxeh ziBXG%^x;qCR=rV62|k>n|Fjx4BE|H`7h^i1(090(6yqNW3O>Jv;e5C0s%b>N7k1!3 zBKhRMY0#mOU$;XUeEug$Ep~xRbGn(z9vMr zqc@9tRfT0YDGdWgal<)iaC_;`Xjl~i@tc8Z&im&oHxQ7_VUX6xqc3E&iM)Grcp>i- zoU{qgf>O8_I#GND#Wa}!#LP5LG>mpw$yG+EwJRvsEI98|sT_DR3xUn42T z2FVCOjQEjk+Wj=ot>v&1tP!^BHD&$tN*La)6Al_+G2sm`)ZwtiE^{1}8qeW16|QBs zulq*^DSS@2FVzdruDL_c^InLsJhskWjyXnlMGHn?XF0$or}R+|qs3fHDyk<#fVQ+_k3lP=Jg#xAH=p24 zPz#Nt$)NmmSMBswd$u)en`-7)xloybJrG<=wY#13ZWnUQ?i-nL~-=F2#{R#!r zP?m`wnUfJC{ka|NDoC)%r9+1R0n$9J+#pgDKHI(xbEqmJ&SiMT6VLDNIoAvDhNkMjb0DdbOBH z&3HvOS&A_R#r{E=t+AlREJAKhIsdLPm_XV$>7xnJYxa0HbYe};I$hMs58y(mFCJQ>NH(%h=A`u5WVHlaa)Ygptt{YL8PF&z(nMzRNg3Uy4rdR%tujxG@l{ zYgmF@vAX>SpP3P64sDg76|al?PR&Ng{e+*KAeS5I&vi$nC{;j(TFY4dJ~ep`H*|nw zxxgW8jP}4+_8kLG$@h`plA84I0?QPpXNWP+$YkpiwhtP&iaZ*92Pb)!#dy>t)$D&T zx>{du=yNLAIC1}|55j0Mu<; zzo8@!4}F0z{`TY>X{E*8b%;GE5H>pPey&;2tU#G~X&ej%-P83*NGRt{V&cmHX@`*D_cOLJO+3n(BqtC~*LkYm6Oug*pH$wXmwt3H>TbhoW2W*#IwC zLrQU+bt6b8a<)2`-aCa_b1~ko0HcFKi6GSLZrHV> zC)aL-lbY|K29_TT~g9d*-KY+kio2=*G1Gv6xYd z*W>M3;?0=6HO+4nty7i@I`u%uus3J{i-d#Znz4fY0jZVNIUpHQ1S1}1igt#eQ;_T!rkniuPI*q5 z{0oB}S?)@I!{?np9J+5i8xpF6haOnCHuU}qAN0vbCX%x9XqZ!p%eXUaXdzx&K-f4b zlNr@SQ!`aaFFhLF;C3MXls6Wz8)~;DBLRLR7>gSDvqcS#dRQ}l5epMC9^$lfa%vmR zfX}uS0t#fIHTB&4HFdKKSIieuJ=tIK;?tLM-X>eaRr4~!Ta&(*$^s>$ z7|USa=@&MVcAa(%*>+6iGG9AwBbl3{@tjX)S=0c&(>|=gG`JK9TbN_etii%h(UVn8L=NYl+f1ar%H;oFiut zhKLEhWng95cgqd(nD+b~ZIaWM_D_)~+S|RfvXRfR?_dbDDf%ko!%5=%TKX@bx`S2X zlOk|ITbB+}*S}--{##4W1NzmwqJbs^9NzS)gpYnk2$5lhz`)v$JHS^7fA@{62z!RC z<0nT$ugT;lmqT;qC-87-AWAMNYJ11*7hV56EW-N_S}Z@Q(<_MOH@1sV=I^>mpc=>y zxanAG@dY1~7ujc_c554kYCt0Sx)SzhyIZHJ(;ZNMM}Vmh4MWqo1JPP>nd)9JK7A*3 z=Wkv#XUCMaTOx^L-qFmo^o4Y77?oM6T1+#+Og0IT{cvsH24Qz=O zfy-Qs;tlg^YT3V59a1zE{ux4kOF7(LXuHtCrHovY^<0@{kF`_VGX(asgA@Qoo?Yi= zT1}Z981I05s9e7=JB&_uhG1RsC6xsI_ex6r8M9L7sPV9@+Gf`KNa<`hYl*Z7#d*63 z$;?6iyM}x#1QLP_8A@XR7=XjD4LDrCng=AGt26bLs}(CLx`C&*j2- zyD$nR<)ba>bY^$gk_-oC)51aC6PHeN#n{z5$LG2F{8d{-=tiyKTjf4$j9e8Xw|#CGS;kBnJy|;s}+5lu(C=1sc0& z%BRNqpEP>o^~P^Ty5KBD7*ML|=;vFB8(etDCP0VadSWVsbWyJ%`5cZ1$mg*Sp>nV>)gm1cmi=#Ge} z>R`&f87@1#5*aHM!~IceCKFmB!aWfi(f2Um;h6~?5fPx9PXn?bjM<%phC359MzZ2@ zj98*EH;x0VnZksW=9xA;XHu&Zgw<4;P)Fg3&~wxC8{}>e$KK33?>on|?;wbdJwocW zzB=k)H*st%W7?-9r>Awjp8U|2oe^S=^ke?cKM{#VDaRH2)lbQGJKh^@m|=B5ur+Kt#hD*FB3xG1H09)i%-? z#UT7gv)#`m#&BokqC#6(?LG|*dJv7E9(ES}s50G~kHMf+jMU`+>gb2WkzrsOyWy~| z*YRHs<(QjJ8#G;8?dApd1z)@t?#$aho;s0%pxQieO=|6HYxD%%qA)Jks>#ef;DFOS z-`_cjZdoHgdgT;4a9Zqy4-ixlYBBn04-u{W0FYM<4s})paa!`f>|sd_6BTBQ?4ve)qy> zDsdZE#t6J^s}RHcRM%<{dYB{LU%Fg#3mn4t-8b0TAfo-IibFX@U1v*Ygl-|k+WF;T zx&iZz2MHOjDJK%`X*p6`ExkT2YL|wqSsiw*HIeZcqZ?v6Bs1;a4>KL%QogTEqj1D%Op>+afWjC+_PKw$m%n+m4<@5P*r(jW>xcW62o z{+)yHsz)hRj3bnpmB~Dm{F|X($wkNCXi|Af3IE^5DigKgq=zAVg^oL;8DDKjPh&f9 z1{g80Rvc}|lh-%R|9-YKUK1y2aom=F$l>OyOpM1(B+mP|jR7r2<1rHlBN4I^l)lq6 z9rcH~QUa!GP`a)Q)VYcUsL3Nh>7FjrgB?^G3ad+fc8&X%hg7L0frb)|F5e6_nExJZ zhv`YDP`}sNZe{*?jKBLs06fzkV-#ty#aqq)Es0R$F_m~w71w`|^)or@o?Co9B|WPP^Z_V~ZZvNn&vjtLb644{%X=U)2;a$yR1(I$%`XmTJ+_R(^SkV*ls~Va#U!s_Tb*DVDo;e>YXswEN_)9QFuUetneN zKgq{1>IovyW!iw{ZXm=;!oA)IE}2KD3L#i@nBKGq7r*ZDQqNRfol$zX8>s%8xX_=G zMo2jRh9bfd*b4P z5U~@;+TLR=6Z`(ZO9%|(9h@ShDz*19;63<(fSdH5uOd^sGJ)|GlEro)LI-1}duJJs z)>%ca2?|n!trm?7K&MP2zZP05#2{oySp20z<03zZZ5Qx8ASmb#wHVN{){;i32Kgz> zVU1Z>u^gJ`n_)j8`NnY&E8YT!RovG(P{+$SvWNqs;F_%i=ibwDs( z)t098R2E4~FDCoKb>&oUh(ij*%!7D+m(6m5-dnBHwXc6d`Wzhc3L?>8t>P#d9c}wF zLa9H#0c19By++MZzlJz8BjKvA-|ZKn!dY)v%7GxTt(|pl9U7r)2ly5CZblFpz@MeS zO6fPUlSD-*Qu$U0O5EaYG_<*N!UvQG-oGv^QZ9k0H8w{@r==`WC!`u3g$=;&d3O&5 zof=-<;r745A*B`JVD%(*ZjZ+DwBlEI^5s`}{W$^h*a4zM5o*TLNnAJ2ZYCvJGdynw z^5Kruduvp|@=Y2Hs$D8amVr8mdlD+R9GDSd+tN%Lxez&Oh)&@rwDhF8htE5XtVrzu zU}i~AUC=}Q4gP1d6JAQsW$t3Kqe?nqueiUyAT$v{oj^l-2NhGf!v2&x3Rap* zpDnx?{WGcYL2}N{m{!IuJkQBJzpEWZhVqnyefqo+(xs$`YkT-l+7kyHWGc zqY|k1wQ)eM>Bs>30bsoNls%uVv8?+ljBI>RhC!}B9J;gvKmAZD_6C+GDAbXf@v~GS zDE)ljMDEu<(2){{xiQs;?Mq-0p!%*ss*Kj|l$9|jOmYvxWcSKbRnf(kTxqZ5hg?*!&Lx2(WKlCD?QtmrmR7soT&Gb{ZR8Y+2$u zzc%QefotkL=QuwrJk2Q*5{8`V|IJjsi z7rz{YmJlT^FOHLrQ^{#0%}@O3@|WtW9vj8jIHNX)y@CVN0@qrgaw5= zx#An95-WXbMb9&Y+FW+MG4N+H3Pb0e5ixL)J=b1ERA~N`fZ*tJuyO0`Qc|*9T<{+P z!236pX28Ao&!dgat%q=?Ww%{+vVqpZ=576@ys_=~n!()L=p%#@GK zYzz4MH@?s>>0Pl}wBzL2Hn_>bXI!!hE-WGr{gU=Yk9hg@f2`ui{gxK;p%3^7js?IOzK;J9eg16#MOqtBVG-XwaGQ;4cyxQ*jT?db zFOOk!2S{fsB^-Dbt?klr@nNj*_+RM>Y0N^O%3ZQY=DaKohGl#Ol@6@=tuu`fz)!KS ziBhsRPOQVf)V+EBwt$#UUJv;HV7!Ar$e@ps3=t14-3XBIIEO+%`aTN)SRM~Vf=;QN z1~PSfRMSN!wdzrUdcFDV4jguZo7b<{S(QhN0-SZ&eNjEJRS+GWfwvil(X;;xa!?W( zqA9CTi5W+|ZSY(5W|A;o@xl^4BEt-hm~80wS~m&=Q=K`G{b~hbx{a~MUxu{#A?BIK0mWgGHM-AXLE_c;x?PZb zGI~rj*C0|A?&6V@tLT52T(i&5v}#^46gz=_bv!RwE!CiklKT7Gg=N!CM1&7?vIP0{ zO+=v|0_fRKR3rQ>13mNb;Z_l-#~J6CYBoS_iS3qV4W z$T`4NB<6!qhgRZsL(@TI~QX-+8w-&l*2g&%m^ z7SxZVd`Op7`Ega;H+x=fZfW(8qG`wQCBK8}CX*()-M-a=MSU@qG%t?5Bfgto$)N$} zrM^x*r_=8oeXhZAjrP&sT#;-CvXVtsy?3yn&G3HyjCns#OrWH^Z{^{X_u(Kdt0vp< z8?{gO%7NNX3$Kp`ft|XVdM-0~Trgjo6?>w^+B;FMJMT8d|L%+}7;;QK9^3ykyu@1A zT&A4WPF<(LxUDS}txh~n<%G^skly)|)NXro%`tqDyZS9@4pt z0u_J0V-AbKd*!gl)by1Xb`?`CD(*eCMPxYe1fcKLpp6B{kO`$mb=$5UwGc;px~Llj z|0^3*5w0jFzv?^oxuyZNLhYCnJ=@8xU$fOR=jpkQ1=7t^$Phsn1s3&W7V2p{Uvlon z))$AAr^|~|&?LW>y^D^^_V&>hFK0E<^}+&)ezDB2I98U-pA_u-%G26^85|1u*epA; zW2REVeTBZ?eQ)L5daz9dg@SkM)i|q`T~`*tOBTsY-+yk=eCp|IdKN4RxvYQL9l@O^ zXQb6nx@_US82!T*{;pcq=M(d#i5xpP2y^JTfeVyQ>wEiZm_#=6W;_+EEXH7%!F`wS zXlid;7vr3~g%M}0305zyXmT=f;|u3IyY&&ReNZG7J?Ir-I1t;PJe@mawssmrvFNqk zb(FTnMOvOuk9&#&>{Thy=8s+Kj$Y^FhN5gIOU4}E%Z(WWykEwn_!hNV>cf6BgGMV1 z{$Fj-+2}6`Ou<#9A2bf8#yL2QFhyxQF}upG|$4yuKChR#h-t7>?Q^?0I!z3 znV1@*^_fK3*ctsRdHo_8mR$R}#MNj2bKOFg>uSz|KP9I3u8(rC;JjHMyS(bN<(pFV zo(edHxJU*#)o<($YM5))4PfoxUw+q_F{t|L}_1w2=NkSdiON>f_FVK3n@4d{v-vVe;yFnTYHC~PSJDZr=#2yZfL|s#6+~H6 z*V$SGEuwOx7e-TylQ`Szw^V6rMNElLcXY;RP}OALT0dVr`ON}55H=#Oq7maLYWSP^ z`oRfoA$XebGfAvY9;{?$7NY07&Kz)s0mCENK#CFm3cOBgT0dUT+N`_%+*XGTYgZp9 zRCCnDh~7EUAg<~d70$cx1J{MMHvD6B6!>Vt%2roO?X@MEgYS2Dg&W}dqG5u7ZCiUL z$d~PB0CTLq3VPt|0=SuMY=eF)=h^HlP02WdH!p!FF@!DCwQiQ&XhqHmR|x*Z))O72 z&dlB1+mZ+xRv^27_EU2N==p$GnJ?CUd!7R+BJ^iq(8YJq=s`&-UX|x+*Vw$Df%jAy zh4Ys^O$WBV4^ePmGHcG3b0s{=r_bq5Qbrjg<$`v|r&*2}R1?xSY{@@-@FzQzzU9V`bndP%r zJJZ7a%2z|($FDjb-#OQ}JT5lPDmiMIR%HmeU;~7%Gce2rdoV-aDMCzeJb}jm41LUF zH(PShy;C#XFGvOl{NtoOB8_3KugWw$WPjk*Is*N`Z9lu#vjc-mB19i=KYBeE)Exy@ z$5j9EZIVDbZ+<)wsy##fDlfCESzwvqG$yz$`{3R`XQUJJ^js7Fn`g}3W z3Z%O){g)}Q7k9yp9V;(;PD01Qk~WbevtKJbt%4cyO?nk8-lJ43nO?wsCl{hnqKMh( z1T?Z4GJXbz4kfH+g+^DT!QH3%mfdZe+}FZg;gZS=qy}#bkZ4FK!x4Uy(E8jbYFN)& z3xDB`5*oTPSR^~lboZ>#KpO+=R$R^L{n1zTU)co`4A!qvIQVt#zkj;MDR|UX+Vo=a zfXOQkF_MKKy6>)SAwkcB`hN+GZZ^z!ZvP}IekAlEQwb`30C1FcXjBg0{09hJPksH+ z%TrZsT6$2g14JX###%otrJEh`gx7*ZyS~L#De`|2x_wyhGq4#?tP7se8WvugJi2~( zQygCC)F+}mA1(v87wJLuKK_1_S3c;unuMU#fPP2`O7a3RX+G=dG;>T@!0Cx%vA9>z z5)s#=0$e`stR+ykBe68O9#j23WW+g8;gt?LEtq>#X9RVFyoP#%@IKz?3HyhJ(s*$c zQ>Ne7u^tr)(NtbMyOw77EM^@`G{}tfASwT_?i{$znnCkB=$E2JZh@>+NV++nuk_Q+}6Cs4d^bMiAy< z-Ef?1U@w?|s^FRHbi;4)^W3T}>!116vW6{ZugBJs;~Ut^>?oDH;whhZH`(Fr-^e!@ zFq8~h=QNN6+>hx;(OHxOLNxh}j$NjD>!tQPb~WKGO0-ue-O&Ez<1c(pBnT1iqLHtK190Z!!P8*U+1&=K^83Bb#*M1rXdr6-=Hpx@d)q9xLcj zL_W6T?-Ub`RlBXw(mQB5_LO>!C9SfSH%*l_Jq~9RZaw?(9FH-gXWLe-f8jfg%3Efh zKP+pJ&&JxsY0N7{ms}+bYD8%ijuE-b-4J7iR%lJQ-2HV6cHnSl*^G&iOugSRMo@Z1 zmCk?$DM%4{uOUU)24NVzn4w~50_=*VHQ_~*Ud?pn7{3ttw+f9sU~T?Ef$ zYr_VdBPvoMj)yW?zT`lmG0?9!5ME?pj5H>^c;dBZC3M^^DJd~uXR{_f2P25&G@sM= z{O^e_IvNY>Kd{ga(9f_^GIcPe>wBDWt?3|{i&^tRXb~b!&-zl>whTR#Byjjc%Fy^` zHs$4x`Jn`ha5X+gsCZ?4uj^*kLAY)~q)u12jV!=nQ!*gN`K}=S?_U?3c10i43p)_A zu2=rvutEw%Ad2=A;T|@PQ+Nuih?{^~o9$%Gs zULXRY(SjAq28U9nGCsO(?b6t|4J*0Lfy*2Xm===gYF9mmw3M<8`# zGBWrbmj0DrBlBROxPU)TEnb-Hk;OL~AyJfjhUSBunBzxJNNam7^Yn{` z&T&MEj0-l$v`CEBiHWF|4ns<=z3nAkUKq&p(80!!RDVa3C{aY{r&-6JoKKQ>tqAvd zpXfLUSp#4nWh^2r!O2UIg$_mbto2|HZ;=K zz{?~sfQ@v)sb*P6E0M=nD~E2X^c_tTFyB*lzqgZ?yg%X7EYFD6#Yh%X`iBSEtaZQ$ zL)D&4xmFPg*I(@Bv`*6Vl-rD;@Q)6nc3ZVO{NBa#IrF6kc5SX}g-wZ^>33Z@S@77& zH~Gv9Jb)H-7fLX_P;q=Oa9m4`E9{xAo$LN=0yVb$# z6XG+Y=X#jydPFaIiv>Q;eg3a+$I4(a%NX@t1`pTltY1%p?GAbw-n`rEPFAZlZt>y8 z@(T^_I~RXdmOOPS_Fo$BkGz9+j>YZ+wy7cDaV8Y`_r<32OsJ4lrs}n&md(p(j@5K` z+AaEJ?=G1qnN6ld%lV7l?Wn@!Asx>L^tH8^IG9}y*zOlxBv0#pYqy4{^bvy{N(M-uC8&M+>>|k$%6Q0 zH&}tymE+`H%<@k$lw96@)t%V}utUnMB0|?OYm?NU1LW6Mo8*te+yf0iNL#x#C-(0`g zkB3pUTYbl26IrH|&84=`?U;gDw`VTvtcLU$Pb_^v!;NvNIqou6p? zR%eF;h!%J*tXaVE(TM-9Nh{)5O~kwKc53_A04UlGF~qldpNF>TE%VwvQx6!#E#g<; z?^nv*ewWCAbw3JgfZgpBX@T$+If354cSJr0{Vg}K6ZIG2eHXEj1cgZHJEed_cEO|# zD-*GmhjVI0zQ+*d(C}Ub+4~!t)%>Jr`@7ZrNw_I4VDyJ{lOfIJlS=kyJ;@R zlMB%=L%vsO;rN_s+P>!tbG^L=+gzoBQrv9~WcIrYph3WpU^%Djq*$NndB6+Gl>GAcuyN(@4f=cV4R78=*3S$}_%~xI@l$4|LFFm!u z>hEj>Q6UpwzQeH!K&vF6V0+$p^Z`c1i2;eSeGk8M<`adpZV)4(*_R()!yU{%X2kar z#A$>F=YQNpxksB)e`RQC>iTLwT;%vEc(I72`4|l%K0nQkR0~ zfnF~^{dgt$;oKgmOY3*D*8OE9oBVG75`hK$1t0}VBkisCZo`**M_~LnmEp{v*ib-s zWd42(o5$67VDKs;!|L_oZv-&o?<$?M8Xyozva0y1bN$EPOp!*j-8GTxdGIc(QNu1=m$LN)z^G&{m%apcsjos3E8$aP&;+6XnQj@@TdRw z?!zH9z0%wpLaTXIn6;*BNmT$&9t=nYbe*R|gO@76n;|$%lOcVwA5HOouoXzXdV;?l zqCWz;O`P2??@{R}yx7B-?*|LS4h)>Vvuz%qLStUZ_r5E?$< zyR(46ytOed>B1Hn_3O>Q#Im4e%N|`J*VkTIi*zAW^XN?b>VU7iq|!Ib1*u zN{{nWK_f;K&)tP@5pkg@r>|f1!f55*Tp#ZD{_23(TNJyRm{Iz;D%)blV?sTz4hrC< zUve7-tm0*)u7Ei8tP>ZNxG<2-4tV%rv2TcwmPZFP>Mggk--{9(mI94^f;pbvvR?BQ zMzwpnk%c2NWzqc0&hg7Y2p)k=TKa3lH{}B5La0JmHu*jFFFIzml=&86BExhFKJNsa z2Q&4QFnHlbk!lP&i}GN>_r!~JF6et+@+?Cl;8;Oy{}%&QbORjALP7`UC`*46FrX5o}QhIH7#~HB+M|6LI9Y&D5@%h$3Zyz53DuDi} z8f0-js_a_?B1`1{xDEnnp$KJ5|J^d|%?lUm)%wv0PC`afc70=nmkOMz~=b--)rh{+Y+Qc0wKMkJR~ZL4w!u#H4zQ@ZR#(FJD=Yf z#~BYJJB+ZV=~Eyy?&P`R?UeFU8^u;#Y1_@5Jr`+rYAxQ+!WFT(w9U|bImH)@#!vqVMSLzd-BXgqm~I?sZ=K^-=xcy!-O^ z1^D@-&T2BAqOrckxzNk5F{H{(kqa+`i~Ov zin?x+$V-J@3Qp_eq7cH;(#`l3pUWb)(AnOWKkrtg{qgcmXz)}*XbhDJ>z{Fh&|6#k z)%_oBz`vJ`O6j)l;w6VLGr8uw%Ldo}U;EWj$XPvikR41i=4p;b!x#KLoordF{=;e! zB2`Q%-)W!NGq)}i4}DFtXr#X@93(nVF**;w}yVJBDK`i1n6srF4Ge5X5LHFq1^qqUKrAOZ}-`72l6+#y9 z_H@a|v_0WIW0#ZO@06s*gWcp;1OJtvK%UQ%emjM0%<3FmuHLSO3d;LR2Bb~Bhccjv zJ+>22@!EZW7_W3KIlu`66ldEE47_&k=7MNCm=Rf0>{}n0D_;I-)YLj{>3~Ms&E#zMuHo>+G3icup`2d8wc=o3(ShpPOKy=sKN#1C(6iR1?z!`YNUKnA_E)R$yAZ@+S(QYc`kN>D)+D>f<#GKqctc5sP~f@Z#i|xq(&C@ zF5}C6BaU8#R73$`U6pmijwMEo9?fR8udG%U8+4sFkN1R+eH3o$D}yc0tv(Y$1e^On z_hW)LUxX}Hc$Y>Z(K6|xm6Z=ga1&mhIuB7MaetHYHs^!g3y2@)`ZA8cIm1bd!msNF z<&mm906kmxGS|=u4s21@DD{Tzwww}c z-8n*kdgrzEinjJ*{ygz1Qj(J%ci3dGx)y8+NG>UB`{;NhcV#E}9?3(!5nM47`jO}q zZJ!+^MP><1qL_uwZ=Gz1M&|0!PU?qSpPns7oI__qmyKYG$LmdHHLb>Q1IL*^y~A$9 z^Z7^k(2GWKjcL;h+AGbY7Pp!^SFN7HCb}-KB24uDnNx=xzo7J60ECXoYO?}Zj9`8* zPn7-{d3P`N59mHD3AC3UypJr;fT7oi6N_u3y|vC7iG9x zB0+;hN59(p8n(@`odJ8vDxaR`-gXKxcpPPEp7Z#4hlDUnF~U;4#FjXvvh1+bPiUxq ztQe5eZ&SEFQ4RB64^KOq7lVJr6~4D&uz%~snV@(~&UEFrLH}yQUV&}Pqu-nqtKAVr zHdCqf38g&KCP~h08QU zZ!s(3X{zTk2BV5OR=j1)-hHoKhvi7;k8gxeCf-*#O*9C}5WXTIG37Dojlw7=(Xpwl z(WhFg@dPX7RL9q-r|s5PnWy(rH7Jj>`|CxFBI1PO;#L z{1C*-&H^;OoF}CeE`dFmP3_sD>qvp$AGA0Aa8VzU_e*)G%cz>w!gYTxm@BF3fVA&KM$_-rocijJ)1xV=qW{+Ir>1cba?;VtvZz z<-{UH08!9aS1XC|%;?BvO}I%&Nz-7dJi2CZkm`}73G5N05Ebmr<(?78wdp<}+$7_+ z2l&3!pNR_H%?AaJ&R%e~5iEJLu{5JY~gdYYB8Gp_4L%#t_S~K+@1n*dVk>hek z>P(GQ9#T8Ob{grIm>T-zO5>3kO>C2P$ERd4p&_VmIJMeVysg0I$wd>(OWos{qFRPE zhE}%Wwm$Fw(DapIQT1Q6Go*A&r?gU1LxZ45N`ulJ5<^JGAl)F{9fH(LcegZ1!_eK` z_wc{>egYog%*o%`d#$zCx>DHZP1U1Jo0YdT&!v%OX#W3R0CJ4F!h7%a(}ngp7xv1= zbEShED$%*pS%}&#Yh`}a!a#Z^;L?DJv4en8Z# zHvZ+OinujzG-^CJieJ6ev51d(xr||Kq*p52)(Uk1(YK*@HBx>ZrW-|-FU;ZeTl=RX zSN<4m!~(A84plTCSrKkF!+(RX)oCdx-d7zp?r+60Wu^mAOf+D=y(n@Z`xk!r=Xpy0 zsBt~Put&C7h+^PqsS#yc*LktidEW3Gg>rml^d@Y{;r@+tuS^iKt&-KvVcl|f{TbJv z*HQ-obm!ruRR>!>Y2+x0o@@Oxc8=@I5KsX{zSWy$=?QFfE~?hbkVwQd&k2F2sj#ll zhNkL<4U1jjvf49r=c?MSo!Lssek;MpvcuYCNU*{vF%w?pwr$&rAe3Ms#KU1IJt{El z>9*`XGkW%tYfN&Lp?tOAla6$J9Qm^lAZW_J*AJE{>A(ENR{qdxkfrke9F^ri@`9D(&ZpOyTO zC3W{-soea`^1%Jj)YsRC=Wzr$r!x#yUj@3R)q3Lz&(k~Ho;;N})V=Qbr62O~jqi&8 zd}xrshLT}?x$oUThbBqU4GJH$i*zo_?e2-BdlNsd$(ZX{C8oLjDs4P-pjCLD`&bO- zM|0J8TT0)H9VP5LojD|d=QK%(ZN#qgh$)3fP@lKAf2?{B1QD}FPY0rNCC&U;bE5k=MJS%r)*$43XQk$ufu?w%V=K|$#1&OF`KAz*N#N2IWo_S?{P1W9?el3DmE8kwpWh?0ZQRlbuE05nx&e!V_teh^2 z5h71ha?UHIiBpBK`&dG8`d*Jq&hyWdg8I0OOJ$nGL?TUkOE%BKU;6C+%BMQ>H8x*% zpIiPog&TmGb+?ZuN87LjUCvG4_4tN}0Fpay3iY2~jY8;iX{YTfDJV4Gk6ho|5rLTE zlWv907yiEEr7~${zgF-{JQC1<-d*eO85TNLsT_2aag?|}Rh&1^&Ho!US;YqKRhcNK zuqS%kHTvUZl=pZT4-OF@jOSi|0%Ds>LGEmeZ{F0?S_5Ze@hkRY6!}pVfg(9!y%#W% z*-d)(cx|QWkz(3M%Kc%rp z=2|z9T@}-%l}Jf-N!0aT>`}Q3d(H1~n&u33VqD6Xgh}hZ&T#~?&|Q7o^MYAYeA>)$ zC|Xe?El)&>AJP}{*Wf(buysm#;{#Qu&(_G{KbW5v>htg8$GQ6u9S}Ah&B}+aE@>OW5xm2!cbQhN_V4-gThGmx zyGYIbhK_6aCp!^DUKo@$ZF#!t4QJ}sL*Jl~`FBXZL#G{mnL)$4<6$8wI`zA)7W8E7 z?ok#Y)8PS003+<%z5rx{;cBwG|45!ND=)ToI#m!88Qoc1Do{X*`bX+=ySG?c-}&Fw zhUAQ!+WB1T?w*PqZ=TmwE&#;-fTsDSmIpTuK!6^i*Y8vHyqqHd&`ykv8=pqDsojxL z*;x}Sf{$r1bj`jg!q#EgCld^k?$mQ6;sbGtWuQLuxmJ6lf*S{WX_t4D^#crl?pYEh zmp`W7-I1B0uUvaT^XOW~U3e2yQ@x5z^mu=*JT#Tab zO+Ng?dYwAL4h$6}HF`Hv@T_h5eZqv2(W-&Vz4*QsN7lmL@wnkqBQ>jXn4+XDko-TY zh5}9~i0A}UEf&1Ai&Tl1MeSQ~sH8~i?QF^^#3AdMmZxYeRfPvp|B}1AzuZrZA#9-N zP3Ef5gP<19g%(l3d9F@P-G+M6*pNg*Puvsdc)28c4h+@p`%GRXq{q?b(zR%+ih)0BIaxGO$KjP@DcfM%h-RZS* z>iyJ^W28`^r!n04O9vfvkBJkJJO0hvE60Zi z0V@E_2lVZr!nr;8*a;`DAYXny?0*L*m)c}vJsveDRBJSG>f9}@Wwq{h$s2|m2O_& zeNnnQ^{{Hey@j6Y@uWx-IbWcwYQ$s|*CfZbn(2Q_`cqeeVe@*Z_4#BRxs&+hk%jv5 zSq&D)P?fXQ0{Nt9;i^i&exZapwG^N%33Gb>@6HF|jAErLRrW5#)Rh>mhXr2Ow;)Eg zh8+`QA_EYnk`--lPkp{hcNFVR0y^BaVFJtji9PjjMZIjL8wj&YV{5!wLdXGKexXK! ze#Z*5i#iUlIZz7+#sYC(i@Lh@8aKJ3-#Cl{D|*=I_ybopJ+Q=u)y>_? z5j64eBG1ZWP67sJKYy|FqV|pU(sJMaHK{UfoxR$+`#7vy9TS0a+VoG_*QiaGS*0kr z7$9#(S<%ze^EB?|(~TYgaT{sj?}RRB$~$edI;2T_P~?o*#XIEwOGplYRg`&Vok&!w zo%Ws*D*74;$&|~LFLXNo!u*Pjtyt-m>sZ+Qrze36n_4xCkdTDmRwvjj=HqJ?X2Dk8 zJ2yuB<>PHK<=E49Pv>8DD7nanvX8ZkG$7+b{Mzc531LCEtC}T)PF-D5%JlT*RB2SL-R?`H(v6- z1zGL0mBy0q!`KC%ZD$(tjy()+pA;JW?Zp&Opq>S~%WiIwaWoB@Qiy4dS>g)XR?<8rsUSMWsy5?KPVN zIyW-^*Rzp7TQQdYK5xGM_9L11acz^^UA=J@GMzBH#m1WOwWbyM2o`zBxQxnnfb*}B zX=Wl+rpC}>)k6I*o$Lh<%Oh$3ZhJSy-=@@gt_MH%5i;{4_lA=aXF&Bx+d1?jiYF-Bd<<) z?3Y<5?_8Rk1`q``NJpPmx7D~V2|+;tEM}P|T7$d(KQ#b4i;KGJ;1RrjRdVFjYg7p#Qqb2-12+9j8{0tv{(3$<`~8$@4UtX}B3 zH>XJ!)bgUx>;R(uq-P(agj0nfM}3CQr|kqbOEsF7xL+EBjLwcIZ5%Z?3;v2G*^g#v z{V{CsmO^beS`QCgF-x^M048kfGQICu_7>NUC=K%GvFn+mSEOxRt`R)am7}ohLHc9V zCvZ^bW8$FMG%}52)i7UGtfJs=xCKChB0xrX86<%(%LTbBvZBaWL}sn@{FxP%D@h2O zSs<_O7b0zH4txD?LM`xKkF>OWj~z+eE+p$SGX3d!YjU&|t#DE?`8^wj;WO}XE^HEK z;DbgTIa&>IM6c4EzonVVv5zm*&RVy)WB=`zQAM~}b#+_&TJ}6qyY8GihrM7+_AB$x zO@{Z8I=O&*noa#O8K2oA(p<@m`C$u=Se0CP>#%LZQ?q;B!tqnxa6%3@93|!-puZ6( z_x@3li?cGsD4$|2oetH55Y>soY+Lc_c9&5z{|(gXzesb<{YFH%FxhtqO$R-AJR3cZ zE##&L3gmJH8o3ON2a%z+EteYP5U4?LCOBwF0LwO(PxE5#xbpGr#5UFrl|UKZ0oq;z zD(c|8^+5RR%~0qU9q(HMnQxmsaUqJBAm778zbrM|*h!?+3X)?2W&)5uhAk3z?T5tp zf`o=NP@!5ZhTY^?0q#2Y2LG~a7tkx0;UZUG@~zg^#ZO(WkN!T+9i@^M6q+xPS==XD z*q?6$A6RI^g>p(wnwo743-8WP09ksQ%H)~(@_oxf27jIX^?lK>moc$Z@ZBn@pY{|s z4xUR!%hAKV%Mml*%4b2Ly&w;qu;Guo1@jq^GrwoFd7{Q>WVQETl}v;7rY^?quRtC^ zh7h0U4HOuPIQ~gJSk+7R-C)xubgrj)0Iay2uL@H(?p^Q>O1FJsE_I!U5Ba=$js9hG8Tkj1maPe&FE_)>sx0Zu4YAI{yx)nu_4734TY(D!n z6QSMkA?w}zJBx+gug>S`QL`GUb_U=7)aIlEh}s(dp*r<3j@XmYHdbQ?yU25%d;7(c zbml_rXsc13o=+L8U1U1fiIva>Y{|MNJ`o3ayQSL#dZJI{bdRj5ZQC7%2MI#dHNNH) zHia3k!IZ8W042q#?cF|+z;b+<-0!dss7L^5q;WR$6WEkRBe(qKGnbRoCg%p%!fyTM z=PHI?v~-F%p6!Q3GPtW}T=H3BC+Cd;$!Byffcg6NiqG}LU_Dep-gisfH(ul00KltS zEC3PC7=wx>fWX;{Q|fJj`vS^-+3UEF)wsMAKv<_pmrd|zrc`h*{z9ZAuxXdL@SBOk z8W7)!`9A}Q-J#vKE;$JEbkSkk_)5s$k75!2yZmf$6G)F)seBeb54C9=pLg!E!I@Ke z))fw}1sph>+^#Dk=Gc(oJQ|UnD*!gLk(xQ{b6x-|4}METa(cu3(uMjCiujg zdLJ9|y^o!zGFJ-Mot5qCrT0hor}l@>eX+1|sso<>%=>u^^bD%jmqK;tN|Sm(a_kKz zklyUErM`43Z+SV|Qyu}-zL4!?yT6~ByC+Uk%P5N?^Sr(t2fO|+!QSm~_!pOB-FY2QoQ^P26Y-1w@T^$|pYOT(xS^J^W< z#(y?J;O}-@>H4ye%tYN`gXp-Jd0hi2VtGJ`Wh(M0+4!TfZ?P*%I37!;|CUx}cYT_F z(WxJ9ifT6U1gG%LK}$idAVI`CHq&kFyF|9CDSYSVwNyrs)80zitDN$eaSH%xb8NaO z*3OS>F1l}9#zOzK z*&zUFb&-2rj&Tt`iqbFDYD1&jn=D~q1dELntkh-&s>L{2ea87Ciln@PlKAY4;v49tWDA)FO@XN}7?fKID#NOY}!ns*^on`_aQUk->kPqO!467k;hK zdNwSnddmS6spd_m24iZxv;R%nE1Lh7YEU*fG-%R77r1fY)Ufunzez$U^+oU=v!!K< zVcPtpAdY;Dl=Eka@pJ7o)3^#K5>baW50E;62#i=kj!qSpmqb*=6*TBKvf(NUNI8#5 zVR9IQTJ+J|Y$y=CobEcC41}IE(`ge>pOCLBC##LerLLS@js9m#$LsIb7VsvC@g~CF zcp=@k976kE`)X12_Dv6f>XqzB&=+lpm#oYWuU)lRU}w|=T0{k60!<3O0`c$JuuN0+ ze#3O2#MH^dO~0*9K-Y@iOcMNI^?A9@Qa{>lAi){vpz=1|rOSfi*dA|m7xF-<5tVOw z7kUtFXYm>c3n)N+E>@T6+CV=EiA+LZO9u|Q%+4hFi(863YMOJG3S z1>A3Q?3~tgZ0jXu z0#E}DQ+XV4c>wMmlF2|LZaymZ>^WnW=*uOk0tU9x2`F zJg2ZNK)mC=)pap8ZSo5t& zt(dWS)AjYPwZ%`5;zQfjkKIvXs)bYhR@6G<-lO+zJ9V+{nR_eiCpQ*ii^lgoRdr$q zIuSDwGX$|*nQ6Bh!}n6XypYoBXS>#0Mv8M#DHqV3YnRFJoy?d3lVVtoV-lzVLymZ* z%uE#viaOR05?buf@La6OUJS0}-*?st9Tr%%7Ir`T~VeQm{x!51t^x=A9-LYygJ74!9y>(0JK&`D(p%_MU={ zk#ZPxVH3z!^Zg7Il42KK3@TLZ80ZuW=Crh1 zHUhH0E|3JN=NOI1F@MXbd`*j3#Jqpdk>vki&HS#z~6qk#S&PI{)c%{vUWQC+Ij0;#j zAQoZCI>o-?inD49XyFlJ&;7LQ*F63fZvtR2k~SJ_Vkz5^z%y6l^YL z`Pp6Z)RJs=&jhe010NUigmOEo+cK{?uAGOncuW~ZrMA592WIi6v&yFE69pp13_U)g z`~_D*pwz5BwF(W(e1n_=3K|gw{?!PbM-06iRCqfrQ}S0?KuWFjmy&(Eae`rh1 ztCtOaR8vhG;8XM^0X`CCOsywM0HW}57m{9(1#BrFrH=mPuM*U>?e!K|Uv^OAduzxLC1M&Q zqidRuyl)Z$7xFyhEWrY$M%9;$SN$jj!+eriH8h<>V&0J<4I>0 zTtM=teXa1#!CCbO0U|}Alyx7W#R4ISx%F)D?J8_sMRje~IEq&>k)sB48DFX60dvUqE`uu$;oleJzA(9*};LVz)#%WC)}l6{Ok@3bd-YnSAK) z=&T&bM8J4>SNu?Ay*jmO94XKXgiqMqAGi6?Do+KG@>wTc?oGd?F!_2+vJ#huFdj(< z3@xQK+xr8}kd{=uSO=3riph+=j~9f}^JT?59}z0IMzz-SzTI^0A=2E48~9%~n%f56 z{!DqMJ@bY4{b3^CqS(`E6clDVpJ?hN^%vzPRN;xx!X6Yf__QtdOiH#IjQ8e@!(R54 z87?6Cm4WPf0;6oTn~D0OJYY+c9Vmj|O)OGpQDDMJGaR zM}SJ`en|hxXuU70?oE{NM25#vLp40`b2P%(ao3?6aMiH5;>g$LY?}@b0CjQ8OQh1O zHa$+#i_uQ&%^Jq_e0!wB^#ih?2;kb6eE8oG`xOv~UGLa}<&_gC`aUIbr%B%Z=`USfS|p#(mFYMQ`5Y`B~d*Cne}#OHd+rL8si_`M2@?*fP`V?cu7 z1JH}H5xm_u1Tf%a;7MXC_>!5S1z3v_07(a5KJP-ffQ*)IAn++|Og#MtIDz;GtZ3k9 z_%&doh;6wDwEJkgNAx4W6X_3v7y7`L){%>!>@dsa|XK76=p4uegn&bNpGap<~~Jv>7ufwFa2@f3^nnajQ*}-HfzK zTF03@S>r%nRs7A;v3DY{o``fd0Z*_PFlQ=+A_)JKQi-=;%7E*OLMug0CU`4ySUria zLRT^>NR3Jq+T`Sb4=C339)(1)hmjCYqBXoMDwMPWq0(b)1dBy*(s@~eh#tMp?lF?qeWxRzbZH_7bEq*QQ>K|@#b6PLs74ygA#y)MijFK2KJ{F@QHMW1fgyH(`a z;@>7w!a%e3n}fimXI~i?r4w51~2f1X@dtx9k!J1N& z6p0Bg*}zOK%l!Bc9>!*Sj5QIRrT$%?P>1S>hJ%7a4yEXT>7%-zK~(O{c$?pFe>qEw zv8Dev)=P4+x(8z1A+GtxLJwUHAZDchgnkV*2SP_)Gd98sN^9p?>+f;A>Ze}gs|fXn z?2qu{@p>}hXEM~z8igUbF^Y<-6}z2upb`-4Iw>QX_o6}M#n36nXNHrxnVQ9b3*|TL zf)qRl?ZWEf2KjsAMkBkB+Y+siqZoAE_#1Yv$C^&W?MaG=|0;Uhkw?kt3cM==Z2Ov?OfxKKAPzNEJH~-v^gAp#j?axG=$+nI7>1w+3iwFk1gg); zBh`*&2CQRfW*BP12hy`JTY6)xs#j!4SVaGtQ+Ez{C#^pWj?m&%sOR>s_h3a!8JqU{ zCJkd{y@E^LfCNG4*+S+b>IiyG|uUbFEWia|q$?AMNR(3h66zA7?DonjSEbk=3&VfWKlNOHx za&Z(8SP{>NS-getmmfNkVe8}70uAFQ3M6r31gux{4nu4N6UmmiYy?JkM}Vy+6BJs# z#tP$o=o}XS8UO;0=93RbbxY0|>lp25(Sg2`en}%(@W2orCvocla30NF=Mom(`(D=8 z?+-D2PKKRPwB_IbRQ7tpC%Wp8o4z7aY{Y;PEc_DBwEScXlvQ^z5qO?(L~m%O$f=2T zlIK_>LIN?hvL_^lmM#5UR}&2KR`pl1eo790DVPJ`lOnBfgK<|kGYy#b)CnfP><}U1 z9-}uVDA<>M5GyqBd&u?Q*)%^yCR)C8RX7j|_}M~TKWUs!19_*tEOKOxgfJ;Pzm`aF z$BFy%TF6f)vtO7znApE<2qTrI#=yDM@HO9^rdn@6k?^1miXC!7Q3o?-CCfXVn2QI+ z#wc_PI&Ac375>7w#uH2KHs1)UMD>ESvb7?%BN43XV4ZSr)r0@G_l%lVfBUZ^DRIEA z;22+NGw_(7U}@>g_uACU_x3z{OQ^ z%w3vWx4nGWeGj2AG-#ZvQ(WD9^EKem!>(&L_TLAW$F2E~#xEQe>x}MQS@=+i4}9 z5HsogRihYtZuYwr0n#KUV5l959ws|OC$FWg1 z^6llOz7J*@dxbo~Hfg3$F`vr_BbAiC9?N+q79ZV7Bo5~gZR5=CGuQBY;JL>W1%2(24 zUkM&$W)bQ3jtuI1-!OoHw(aAn^=iRbgZY z%eU2*yBtXxR!GncHuZ$6_rB4Px{hu-?WEcpTOfS2L<~i|(UUP!QJY>;*|ea|gpT#} zOUd4TD4lGKZ+XL3N<|DEfr))XXE!rCZdiD053-#-2o)XiJvt={XShrn%?A<_#pN~B?}HRU?*vu*>vLt8!COplNjXQ(1R|GW#E@R3O?y(6hj9! zu72xN7j9!VTbkNLx0!I%$? zXR$bFg)!r9TF|}@2kQ7F1e;WZZ^~1(Xi6r#`GuA$2H1U&&-^Ub>=-*C9mD1RHZdU7 zZVsCZeL1dU>?^^bJeiOmtuB>aKl6d+YVeEC_H49@ZmGD5ZiKp;#@7#;zru_e6KrN} zDtSZ1hQ7=}0+JAhqYzp=u4{Kjwud=_Fhr%<@ZlCcpT7-OZVE40^%dMI)1X6@{zWRB z5<-;;xu*TdL+s2IGObmm9}Er)!Ygbs*i6)07({pf3;s%!=@Ia>A-1^7^uOk^o6sVk z&>{v@82|5M3A=_9gU-7gRt5DY+laA<>5+6(ZpQE$88IKjH==G`pZ51`-dNWVy{;J< zDl-o;<*N;4(5#cfVcDol1VV?aa@&$(RZ|*+jYEJQF$>r!6v%M7;y2UC9Evm&IH6w8 zNXy^YGNN0c7^yW zlV3_!gIXnbca|kd7gbIU#icop997uqk~-rZjrBUc*~aMA*60-a+ z9Fj)8r-hib+*E`9?KtLp;Q^7k0Y;%?x0p3rQoD)RjCeiHvo8;{)qVV<<%_XoDCf;1 zRl(GmGWr>HZo7fR=W7_Ir}e?1#@myWA{=70KL7bqkD`WHm%jDB=}c7H^iFa%SFVpLj$XtfpYlwhFq{7~(-LB8$KC-m z&;aW{ey0>7`n3DX;O6xHtsnzMngtOZ zOcjAsKk4Z%EwkqL7kwnx&{i!>HpZ`*njRj%H_?=3N<|_^D$ zST`V%SUvw8UaH*q+u+mgzY&$Rb3)?~<{M>q^=XZ{*0AMaY*Iyf!i?#+>SwBjB`PV{ z>Xe(hx5`NsIRPhRIv+^R{x}VMQP~h*%`Qb@jgNqaxb6P>!apF4f=-{_DTMN2TxApE ztH0iRJyBCwp_1K{I%nwn-AutEjHc!Pgm^qu(8Cb&dzelE%HBQ1SXD^}!Z z%xxborJOkzjH6#uiNL2S{cPgMkn`Ulwnky?XUa+0t!Ymh2@76_cI*{g7ox)t+Gxi= z%Ws$p#E%GFfuUUaz99ccO^ZTnr{Z<-4@kmQh{CCP2@_*>G?6U5H(TX`C4+G9ipjc> zhwl}}!a_q;%GN9sZ^{K@Sr4>Uv|6*No1zMB?K?Y@slAgjbDa)rW>DZ$TFo|(UW%Zg zBQDGet7fN+c(Tu6Cd@n#g&pd7XHmmK+BLP(b52qiW|Y~@ z`ebFF!6#E_H&FeU&j@G&BVnjxlX{8b*~0!0|6YA&5u=@|DB0O6t9J3zn{wMR>6%k z`l#-%|@$^`#cp_9CB%Gm&Abr`@)W!i%(7;Cy}3_GMDNGIfxiR1Jw?sVHs z-dYzKM;OzDap zipJZ>2)I_Z8X{;Nf(kj>d@tx%qJB`Ao6Wn)}L49eG(NN76CFZnO@f|-0k{LPLepAdb6R`^E*t#ebh3)$Fzd(2uDV~~!oj+*5cdKC&On2EUDhf2ItxSoSgz^uCpguVwQFyM}0&n>f84kgHQYOMv z#K?2OWmARUIS`ADH?lT4aAC-iuv2$Pn8ju;^owEc-;&EjpkJp3b((U|C+Fl^q;YW4 z>OsfLeC>r>u*=TOd(z0mRm``j{5JrC*0*>6Ed#H7*Tmj-GR5h38f_ixA;ahCR^qP| zDJ0P71soGE^eyDUd5Kq`bo|R@xy<8q2cSy|sGIY^gw!}3nfxiA`v9x2>`Aa=DZ_0m z8O{)^k26{+xqFp-e$6)UZzm%cExo+g1I69ml4J^Rz@U>vJAYO0;H=_-2g=Oz4Fww` z4TV?OjmY>@gc2yom*=8frh-sB>;nqdYeU>)0}wvu{2AuX?^o%N(AZ{tkV?`9%@;Dv zBLvP3RhppsUr%b?AnXlf{ZKMRMR;&4(l>fDllsfH`&Dm8EaSj;FS!iLf;ea$533!4#0N{})@?#tBv6=eoGzQYtg^3P=nOd1b=Sk`M6@4u5@ z-fHt5l$zVj*BffQx&Aiqg8NbV`xIquDxmP#G@H?MdoM(*Z{r3Ujj*zrB))S5=Kj-` zfGGIYqR<@yFR1j@OFX>WFYG^{_#yKFlEu}I!(7S|*^hGA80H_$y@pg5LVOXxbwdksZz!axp1rmeZEHDeyh~_tn$&W2egaPwKB<~G=o-KHp z+Vn0r1eE|3zwPftSXVD#IEJD`9~-TipDTO)Ph;jbV*0rj#zF{1Dv#Rb|_-DQNL>BSehg4iD zE2PdnO{YA?E&Ki~R6}(cs8p^r zQ+P%~^GrWLV=|ZbfS2o3=I>8=FW4*dm)}wmw7D23OiVDzCm7R3`OZRnvSEc)xoJUm zY+x*|pu)1ctfx99ik*|~Sc554@eKVHPV^Y4WQ4Fs@25R7B~^DjjXp+_AJ3CRnOx}Ep}?0VYfCvzHW+M2mR7N>0gm)Tv5xEij%;oA zU41GqTYsUS)^0v&bh#RBZFy?KOic{Ucb=0pD=@F^6FKYOq7lSTXUuUvyzsV;tVJtM5sHIo3t5)cRRsicj%F&65la~b1ANgmlAw_mE%J@F&*P=!GQei6fKovG z;T?dTu=?jHXmD0PclG{6zFfW5bnn30%}47`UNK+x81{(Eh_YTGYSCd{ShFz#x6@28 zbJ2%x1TD^?@(CL3>wpfg5giC7nZE>h&wt@}tThLS9xO+a#*4s&>iA=aC_L*gh2(bW zQHn3QFy5!{aO98yY(F~(rNHm_Ae?bT+Y32U_J7KaA_$Vk=EF4thJRlhC_&6M|FPOJ zi`XsJCI0wrYhE6aW|V6#6W$9K9gU{5L`dn^hUYMIbbqlTJBz*TV7uwzzwHR-$p5=O z0#bf^r=~Hf#QP>wRJM$@+)-EX0+%MvHnN^WeWC$P{O3hhN#{tqsPZaOGOkUCqE4O` zgKk!qbRGop@Q)y-H+U(y_po@J0AB%Q%jyv@tOPlu$ca@|Nh1l(<;N9bhOSFHeg6FJ zlK|&{y2-#jTxl&6m?kLQg+nTQ^Pqfqydx>^l;dE-!BP5&XBB)q_q#(h8B#v)f^*fx zrnLWSh4HCbaRoYKy>TcT{ulZuImJ!!Zc`{#Xv;%eZMaCYjp$EK-H84Ee?{s}2d^1P zXg{_Nn;^0>oiy6=g3g!G z7l5LYAqR*Zcq1^jEvy>(^mvad_FD`fL+wEXFn%c$Xcl0W^A_=4j-S64c2_YkVZs2$ zLQQ*szzy8buRp@mX$rD)amoYY*+9Fi_=d@O*MUm?cbnLKjGs2)LmcYZ`RU}k)j(|$ z(uLmKu-CPi0V8)e=Bt9;zY-n1jWN{y@_r{*r~ECs}thtTKTlg4Z@ zCRT5getuGfc8NdY|3UrM$~8=5_&4!gOyyh}Gevq>w-jqqe!n5?pxCeI2knnl26cu7 zjv7X`9a3ie4>=7kU4l6^o3bVQs3SgiZiE1s>>zR4Gv<8YZ&3XJN$HY9Q14x z#`bNVAwvnZYSCdRU?j|gkO&o%q}`^3@5ycacc6QkkPs*}WNjk+LF#@M(`|?m>sjNH zO)<3f_t=xA&%hunj1P8XLVnO@5nno2*YaaG%T7(u@sQ`XXumlr;^@Cdtkm(j0?Xvv zYOb!*rN3OY^S=(Q=bpSub7mY8v>Yf)bevVIdJ=G8I;0$olE`t{x2V2C9pZk5N=8|c z3hwA*N2C_3>?YiQ(tfUwk+9*R2}sikS`NOhYaeaqJPeQ2aKmmuY-ja-18Dp1@Q~}Fxq{a{Qvt=tWPJo zSPPg|1*+R$#?<{xO@lxNj($-Q!@6jq%7S%jB(0Plro&Z7KD<(Q&jqVr!&1NbhDZf; z^Zvx{B!BPz8bRsZ8OPViBc)f^Ml`-?Qku-m@5N)$pTxkZHc?xdg zgBtWF8xX=i8G?Cadd!F{^;SltW$|t$h!L{2p|KwCmvGUMn7*L-irG6@6=)w`j3bat zXyTirR!Sa*qiHPV*8Mm3(IY8b)lEpNFd%XK1Dn08;E(vF$zs7Dbpon=tZebRSnB8K zB2eYI={2Z_t3I!!(&uLEmlWL&^J#}gH~_N zsHl6>O5<%|!SE+QDW7M9)r({OFtJ>(`uNi*dK$%Iqb%Ro5;knBUYx4I5Xn-9n;;E- z$6|G7-6{N~UmkSvf^rP+mJmV)_v5>FQH^4g5ypzSpprXk>{10JV5E23?A>wSqsp0s z!QwP`13eq=e-Y^3m#M{$V2ScDX_xCr0ESH#IrVhcw4epy-E#3=7$iUM=Qa*$6}cp*!^dOdXQ0H@ep=%`A>3# za;RqMx3ri_my3)3)Ua^(YS*!FTkjSRXy{J&n4$le{>+~*E69*eP{sQ_-Bi^{EWfzd zeaI&YTh+$8uvOy?3^5BV7@UZIhUA%@YE)+m!laQQ{jx>Agh~a*_9bF)X_=fAhvl_7 z%fRB^<*0@7T1=zE)nvB|y!wu^XQA@51m;HvWRC^Q&lT!r^0y`{x4V@e%G)$=kf(Sy#v$R0jpxGbi#FCc& zZc^IR{Z4ll{^hCAsytZ`oEXqeo05kWf?xAS!K%ryJtMax=hF>0XwkXLZ>`un73%Qo6bH+`QHco%hxbaSPFVxY zg-r|T_mg8s;HW@{0DM%&_FWd1zkX)ou*@3&lP2k1t!wWUm4xEg08(2c5QZp=^i|%< z_I(6y#Ku+G5cgGg6^IegnIqsc7iCcWF;LE#F;=?6)5YZ(ui{l$r(WU!iDNWp#^exZ zLRPl zGkuCpA2&+PSzH@|3stn-=h0tY%1QgsMwMbngZ9cryGu6y0ESVxCr?Z&0NTM*$R7Ep z^+in*jNzID|H&wW56T;xm;x~cnmq+rr3Ju=_RY;c(|KetCx8>WE_HZ_mo(xldhOqW zs6_0FDv76XXJ!b9fhz(BCY*f}aEb&V>5Es9C4Nw-ex*JU1czpM=d+{IfNH%vL6G% zh+pwd0vKr-ccQ%%CV>4YCGn~nZJIb+pHuWZ30%6?SH!qPZGZwOGc%62R+AgRVMZsBieQDf7j({*0u~w~ zHL%ZRnYFkGa9Y}H$6^$C0Vs6lT3lr{AO!iV0?-6tkbIZqnCPDP=uH6z;#LGs4jc?^ zmRBjoSacI+chUf_0l7xN1W{O+F;>pC=>k%-#c7$LxaU?&fmd9LI_*B3klqQL4|&!e z-Q#WMl%K7=@-kbZni>JDDuJd-iGdJ%q2dAnOD&Lx5^wB3cF!r0&aQ+M@UzOnMv79TJ(KPA2-}45BpBywUN>t8l`iwM%nZUS%UgqICGSa@K9u+J7nF+OelS2wZ>2)|B6R`=wbD6BbkHgxg~6>-=k8Q$ zRaCoKQ>=2~a4`5dn-pN+Jea zYH_xR55T77^VQv#3?igjacequ6BkP1(TVA~=u+0HYY(le5r;*IbnVjBW}}2Gu-8dT z|Ek%SM1JQ2a0G5p9In>NwN72S*~C#l1^|{2NSZ5!&`v#ufdE)5FRg6;k=r3N!6{!UlpAEG${q5fe3YwEMDvCFzyRv}F!XpeYTdB^c$CRT#D6 z($v*VsmHu|HNZ0h)NHdCCNIWwjiTe`Enyzg{zRcCx*HhF|4w9GsPI4z-) z<2=R`X#Et(3E;pDfFr{2*bWhs?%7|4J#2V~*hxbrh$Z-`klqa_CM*fp5Zggq5%g>T zHtg_=N{jkWfeV2HaXzXwa92unR9QDmQ7HaSI>U)05Lm*jrOqM#IpXTfonlpCmtcV; z?bA`KblWSbO6@UI*=gaR%$t}wA#g^t8;^jIbR7f$lHxJ54uBSMKmazCHB(d!ZH^Z< zjYyd;GZ1w;Wmav+Rog{kdC+>}a zhiHY)OGn+$Kp#h(H3!*^n1!_+C0E1Cvi&@H}E#0S;Lg!KiGtsPiuMaESsgHBHE1^ zVhc_B-r9+ETQj%9v@3&JyTr4a;M9uyr@al#rj8Qp6X4wXO)R2n{dF9du;o)AH-Mu7 zvST=WlHooHW`ZaQet-Z8Vi-gq0GhCIp9XxuP*OJ+c)-3i4tUfD>K#~t)lQY_7%QY( zQi0-cJ5EOx^3f7%l!-Ou1}+h2J4u-(0G&FCZK|{?3Gvk&rAHb;??r&CT_@?-%${gf zb0%9tVlC2<9&+##FbQC)y|^`6aW!|E08Z_6J+sW5as}R0l7XSTp{Oi86aXA=!zO<7 zR!T7l@C=Gcf*bOn@`#&LO+Iw5bt>9asgreHdJWM88VNed6JnZf-KA@zuq+W2C9HPZ zT&t0IYO#eo*osRGu!TF%vK3Zb%qC2lZ6AO2vn|ti4V$Jte8x?ZVzNMAdu6OzAZA8~ zjy7)S=ig%A~+)*O2IUE5YLU-*c00Wz+0J0>H}BiG<3F+Kv*t*oJKL6Dq|OE`uC`NKZ9k2zu`fr@ zwV5cqVbu7T3NWb%6NIE|{2Nmsra+MtsG0bS&HZYy)eikYDi}J@digD^{c;;=b#C_} zV{RM4a|1XUoEf0oNx`=Uu7zJaCbDCC-reOW}=1zQwKer zb{gd8YMFCFgE4}vum=dDsA_>0#aUS=W!ND062Oosshxlg8wJ*&Gb2$719{jePqRpd zV?;t71s0M@n@U6e5*onU3ly;n@LZ+Grhfbd*d@O;02eioga-F^i7p!aIq4fvqaR&J zX{rTQxB;xteTU`hx>FYC?Tw=t1(!+I!>>#wT~19kMm z|GufRAwxUalxfl-QW~?T*I1Ws;#~DxJ3PMh5##m>GuJlq|ooaE9 z5Ra+eI!w}@JEC_*PXQ)@E9viWcq-eams4Rf=HSqlnzvyxLK~PpZL&=nH_N8am5@^+ z*Sb0md##e_G-2I3(azP#_)sE-cBkO6i760Mpac|{JK}SjIrv(u`$}hIX_CQ+H|z6f zZ0@@ISf`B+wyGYBmmrna9Ysz62W8Y5TAJLYeP4-IqzA)5E$)F3gh(zC6(T5so4I0b zbJPK6Q3Vl>26;p(h_S@$3E;)1Pm`EQ92tR{+UlNq2j@iIC9(ksu%xXIOwlnz+H7pqrFgw_$vyd zCZn{J;Sd&eT^bSb69n-NKn;Bw)_kFt0Oti!)&IBm=1-bkWuD*jX1=-am9_6xT1yhz zR|EnC5(1%y2D+J^wr8_{@DGk~gu@Y@e`6eu7`wycarZO~&2(d6glK_;FhUaA_r3PL zva%}oeR=))JU8D`A_+Q0t7xh6o~k$R``&x*Ip;q2EZ_62%~Yyhk!5;UBhOLgoFGcm za(FgeHxKv%(4)0gBlWg6!R2YGa%|$*8vWdV>z>J2hbj24UKozif#cCQy*)arGjZ{@ zo>+1nueBd<6}ME2O>- zKx!6?t*LRq3v`dGii6NEGp34Cq@T@>1;W5QIFrt(yoiRcMTBGRq&T17e}RO z%WdW+1Ay8NIW0x(E_HwaIOx_ib8Kp=Mt{d*T;FvzF7IuR-p;1D7!kl2&(+g`?;GN4 z=>x943d{nKG_*9vrga^0Pa6*VP#pf{>uqrydW{brjlrXD#x#n?9X*>+g*X+HC@q^h z@sG1WkS^m~&BD=Xf&-?W&Kw-08lY$v9UY^W&jVluH-$q&pZ5HuQbcur=z3mCrSE13 zrV|%E-`|b~f2Hs_ue%{AQ%Ql60v|2~<_6x1+VPig!!TJ$7AW44mOux~;MwQC8x_2a zx83l~R6Y1`Yrk+xTM@t!Sg2dGfCB3R3v30qdRje#cOsEwkM@5#3JQh$GT#l|3 z=QJ!}eT0=(wwpD+foD!@C+aYFa&u@(1zj013}^U5<%Ja%O-v9bGAQsLZ zWc4XF5G)_84XMEWgcROKP-7p~_QMN3%q(Di53xxtTxHb!CkHoxlHcO9dSfyhv z(hQ^wgeO{L<1fgmp_VZ7#D9zi9J@B}y7eD?{6$1vi=%i_KfNy&2M@u;$K48>-lm;5 zVc)ZZN|vvd|6p++iwr9RI0-56POR@3ITJXOn}xzv%$w7?h()}D*WN|EoC<_$yog1V zVu>{XNI^?Ra03@X0e%H>Rc&G*m0KG$2k!vb7+L`TE)9r@&_HHghZ7>G05Az689dTn!Hz(~YSQta&q1*;Z<3p{ zrhN_N7G|s~DQhx@xfyfD6qKU~EIeR7sP7-k(Gw!zb&S*hFd2{D< z^mMkxj!Uk_i!r<`%UjJzt`{-=8w(s$%Jn>iPovw}Q`isv!j~2Bxz|0)_mGdYo&mDa=76ysd zK71S(DHF*!$kmndXLsEeZFl`I(R}gk=kC8yzN`%3Km@IwJDM#>X)!2jBsW%-V#Lzg9(3otTr4 zVnGG4@%iN>jSdY(N2})O^N6k*V%_>}F@tW=z=^|g`tTbWgtK@3PQXz=b*Uq@L?_xQ zr>hmYt>vhP<1i7per}DY&RZ%uIOmPDWx*u{N(x*c6j(ryvSEymYYzVw zkd8#x=4iX}o|t8WS{pqPaqJbiIIJ?v?=!FbB=fWYP!`V|irJ$t!=c#^x8?-yRIn&^ z!C^(xV8D&L*1{~Uq#D;)-|%0xFgB29F0?-@12}RY6y>-uf(*M`n`2#dHl`5k%q^nX z!ooQZ*P*2Yoe;zvychI7P#o4P@f6ljSTZfP;*_tg9kXrx&Yp;^pwV#_DBH;4%{u+FJ0V_cZK``h7F1DP{-F)F&rT;>23p9a zvF%i!?l6jyoavn?Awx_k7uTEo(QLDfp@Yfw=3U8LE8`e?`i}%j)m2c91}dLpE8vc zC@JuJQ($oz-#7TbalH`aRPbrm*uNbW4%8SfzcUu#!Zc3&?-3kRA4e;z=e`%yFMO}q z^rz&Y7>`Oe44I@-MH>Aq8=tOQH6$Gs!mBiIzy9}b???H@6$2atK+eY~z-11dkmi^~ zk7gc=Dph?aM(~)6oqi+UgkAC*;pjAAevK!blpD?>HkrW-Z$ok^P&UR#jnX4oG|vCY zw*YJ;Xo1_4t1xf_p%^b1Py+}NQ#24(L5_$ork?!ThRCc1ZjshXqSV5=DuqPo)x6_; zY#wW?3g+WY002Rbet8soRrzRq-Uvr&1~9NVjjXc;@eL>?|Je~l26An(Er1+*4)@6o z(&*f$P`xO*H{=(zi1bjmDL$DUaJ8UMk+~S2A{0aS*4WtA0&oFj7;*t_P6eRRT0@T} z*-U^x6t7#l3CcNi8oYxd@ESQY5rY%0F*64d^<#sz*wEk!z!4_n=t!M9^m1}5ng!G)78FwfR8m6Fy)GK@XXaX9YQN#@sCf*=;$!~-%VXAFI61m5 z@#q?;>WFKRCRg)~=*sqPWPiF?N z;ppp9**=F5J?;Fb`dhRvhC(I}`IU)t`Ss!|e>use91#LDGkY+vW#lN+X{vt>Dx1>Nxf!{^}frq29#)exC`T~l>vj?9<;sCeZLFU@gw=Ej^)Kdj7{cX4W zd9G#i&wr1_2v8(5p^8OqA3$i`#nH5RPc&}4JQ{j7qkr>q%s%;-C@o_(OPQ)yG}%m}rMOL*gSrj^P>vG+az@Q;4xB8)Pj25|pXj{O0lI z*wE1td-~TzJ9;%u>fERp+=HBzJe670s55Lk^n=6G0MHpeI~|W7jCk|V={S1iNDQ7l z8Yhn*!km08+B~#(L@y;06U@p473(N)wgdEz#Khn_i0>d$&f*ZT}LE z{?=%)yo~^wbr%6>HnLVMZKe=i&Afbn%>DA;$NZ_+Q$(eua20M)^KJhoT5kR`m=OIR zYnuK;jb9PK5!_TEnlDI`Vwb=LMLp>c2Mz#>@EjB2JSbp+4_FeA!iR}aC?YVCuj0#9 z7sUxOCwoB%Z*oC-l`ouJxxD7&%XcTuyvC$TnZPUZIkEDcqZ7BDTYxEfet{S5wFEzI zMO8TF)66*^R7hBI)2 zw*^l2Zg&+~~VrH9S#?rw&1_g`W7tmf7ee zzEnf2Pej-Dx#c+F*C+LUc(xO#rsG#{)Z&3(yb{0u_2YmY#a8XnffwYp8+Sy{nvDe1 zY>D-JpPU$p!*9O?P;82gJFd!WzQ;urqElg+1+WJ_oEd1LIec>~0YOm?FWSuEJf!6S zXAbbS8j>817T(E%98Bn31wi8siC4U$q4#@`$lg!!^vx!mL+T;C0nd|FfeL+6CTTOj zTlvU|akR9ctghZx6_OWJ18sxm2rcH2Y{JbYo=OWGkr=R{VX5{}jIy=WwAz2Upjq5Lp=G*^=XuIx@ zVL%i>=soveKqpo<%Q8vpATHxZaa49!F)BD~_#A$0oDn;_@a59cg_1EG5Y5{lk`Ck) z(P*XA0EjS@f#DdG;D8kpv@8H9QbCuOQs~a08v}R|Or;4o$6W9cHM!@M8>AZe0_?mh zAixKho5Z0YcKwaA^+2tZ#JvkB?vBOm*cswd8OOtB0v4>F_Rmk@y%;f&;xB>JJSe1s zWs`bA?z(lLmq=O#Uqo770i)%Gq*v?r*OtKQ$Rye8YzBSl{1= z?$T_W9U|@%?ZIq|N0}vnW>Y*l*BqxOTVu~;CAPtR>g-107=XA42Woz%7DM>7*>`*{ zUO6})2TySiWeg7u;OGxI>|%ekclF2G^_y9M+R*pl83AJk)EvWVts85q&TgWu^V~J! zdlvt8HTHI~8R!-u;~^XI>aIQWv4Nr3)zT0hee5zj_+aUwhw{~smcPG(Bexo!x_rhD zL;)6p`CC)%MKtZW zDOxZ2bb6X!N!>~p76CMq_{W)h_B*kN%MkT*j1O7edR?@jZ_{%57gFbPCFOnG^Q;Kq zNVK`79q1y;z^>iCjCqw@L8pcQFe=u_hd6ax!wU)hf zni!2cK6!Iow&&`2_JubvIiE<7(}DF>Z{NBY8$J``BWI#_4bSY0V?=dg;@=5?1JMyY zQ4h`yF@tVSM`s_vRoUA4Ny|Ans|DcH7s5vRPv4HqqI2D5UV)wHAE2@Q>K~Z|CUoN* zcUEbV>$#Z;Z-WferR2CQ0>&06>;Y@Z(<|QRyZv3cy?H#*!>5D(PL?dYz{QLal=hKP zFO^yzN(!uQ3d|vTnj~0e?X{nf{d3GU>bXD13>#3afEzYl8SUGyjVj`% zs#a6DJNZC~HE99AH&eg*KV$xdAD}jZV?PF|jEUxpKNYQa{!h_-(e1^6{y+tPd^4{I z;N+|Z@g$ZL^9+_c3n0Q`qK2D-4$dsxi0SF6XhVlYkxg4`8z96$mkV%PX7an9jP-&n zpb>Z|9!iiR8UXkRe$td15L46oDf3aKtz?F_K41s=K)alh2S9TYkW}XIj0`4na<;VM znmLPe=J&hxh|rpCA6y(2Y*oC~QcJ5X8T?M>>Z-`$nHhtv2q&mfXLwBTf<1dDd)aJ{ z0XNU-CRn^}#x2tzIf-&F?f0?o_gkRTy)VkE`#c4X8|}f(ZOQS*V~nOCngD< zFkJ!2sIP<=5Bjzve$sq>YH}t<#*n8%M?sHVpO5@pLgwC8X59w+z2!RIy$etFB-Ssh+fPl))ht9zyIjT=;OcmcQOA8+#&!~u3^kM zO?&Q+Ho#5O)~i?3E68VEK8O9zhgUSC*hdA{CM=^2FsV31KP=6PtdJREDizb1ueD>Y zT&=XF*Iu31HNZltVI5W18df7m2qFhZdH;eO9C+V~!4^R&f`HWD$w?Y6Om+@AD}V_n z^@44Z-Nl93Q+A(va&u}o$&QrArAe74Xc7)4p$HZn0K_~NMm@~<6EoBg_aYafC4l22 zA$N;4?S!TvEW!xTTW`Xic6v|6Tx#;!Q@@BS>hd&ZEUt)+Yx|Gt!LJXW9j^3WmxbfO6;_}OOM`s5h3mvCd zUwt!v_4spmE1ry7ZoMw9zUnf{JsZQL3o$raf$>Egq$VWh(KAr^Le78K$QGp$mJ}>9hj&C1l*vqN2 z5sd5n`}5{I$uUsEiKgyx`DxovBqZDA{-gX5c$NKuH%)))fPs+z~Rdxk%;; z&znk_Arq$)?;ISQ!ZK-r&;e61hv;JtPK8R1i|O=lCYOIjPWsZx*UV(iw45)brh+Tr zA+^lqvT&`cGBV(%LGZ(bFIz7M2fxk7l$}j+$$)hft!vi1bf8m!G9;7q^vDpAwFgqs z*r`|!dwO-`#XCp)AXPw4d&7LpqE0ynf?fo1$(=!2$N%hN5u33PO^jm#kKdmr&A6#o z)f*@E!U-X5VX&ItV!dChsyE->A4iS=R4(2TU-;~;@#6=7fo{xf-1n6`<9Yzhks~MK z?Y#$L+t$r-%_lB{ZZiO(m*dl)zBw-0wJi=DJ`%4CoQ++4zxC#8(Wx1OlQR~d{nSl} zt|sEd@e|R%rY|`njh(qM!G%{G4v6#|PqZ}#`p>#TD z67D+^L_*W^nO36?PWD&n3&--z-k0LolRrt|-%PO8R(4k{D3!~RYidUIm=GKl`fi1z z{mDtPP-b@wFcw@G1T^xv1w77T0v*If#c#lm@0fR4;ac`%I(i1f^;lm}5~v7l?66A! z$HA3@!bQjPeC!U6IqMN8-H|X7V)bVPklQYMvkOs{oIJK zp14V=;XQbfutigQe~F1V+}*W1;j&yGRXh?`;p$YnH>C(mozRt}rd>D39DZ$L=txxY zr6bpe(b$2M!H+efdt#M&wS-ax6Xs zCvFhcqft;*SY`oOnA-(i)lJ)@edBHdYfOPk7%ZG?$GXVq+6|*;0a}G6mFBKBOmwv zgmkF33L;3%y*WQKSz1RC*NO6&ts68qCs)R&qU;*nmiqhgUO7bS&@^GeRb?SiucwwA z926|^E7R@*KSqox$}S$38=TC6F--l%TZXOfj$(F#AeA`!ljGq+kmZwuRuSHy6X2je z&UmzRc4JvZY3`i4=3ha!&mfR<^V@~*lP<(d2N6B};D-+bV7?Z6_FRn4O*aF(GmZju z&I}Sja~da202Y<8Cs86^zqS|JG{&KWN8{89!z)Z-nbw#(PBv%_BCqM`={R|EAZFo^ z_3?WQuFh+(y%}2>2%r4q4RL1RbezF)p9;}NGUp|UD~sex=)2@t069+k04#_;Y4L-@ zc}S}k;AGE$x{Q_SY4p(y8-m3iuOUfaaMFA)1<~zCplSL!{gRHQ14zxdbzjh8UkB)9 zyfT4LyWoe34KO5^PI0lM^oq!qF8c&!TBQ_F{Jd}P+wu1g{3L$<^M{z{ z2HxpE*_R;9EWXrm7<=l}iFoFjXX00nJ{o`e&%PBm-F!>-=L<%3#YU!i=PjW%RQvcK z=HbuA{NBeR4nGqC*r_0*a#+>cv1r|PLyT|vR6IXeiATQoi`;zw*`NP6?+b8TCr$6B zQzL$G+OEe0d{rA%&azpoqJhvMDtQg=($byBa0&NYJBW!6b zr`oZ`5s)k_u&Z>AmP3F!vOCuC(K9hQJ{psN8YlN=w8Cfn&aSYPiCPN};Ea+ffPtI@ zUzl&3>67mQ1$A`f-ss(!8JY8(;Nv-89a*AHw!YwpX_sj>0Z>lAWFS*hx7PRq8wBgH#o|Xfg)uMy5oQU}b1e}r?xNTq0uj1 zeJ1|;ul_P#eEzvKiYPeUa4Fct*+-IV&%EQ5?#GSVbI&~$!>ANYOisk-?)*G++xZO} z^M`&*si)bsK3d?gG++6}Xv5Fi+>z&F{=k#aV=;!h_rxW?{&nd1J{vJLLo6ud)it1*sw6@nNs+0RGE1Gnywo4%DnvH#FC_@}7v0{|Qs?_vQ@=XOl2@<5tlr7NDQ2i$ZKGd#mIT$?%AlupQ+ zkjvFEUeyF<&8^&9rL!BJApOOFjEafdU2Sm>GcgOZ(C~km zmj^mYvfEv#y-jeyN@2+=8~dPwoGgbw#aA=(A*+C?0ZStTQqE6e!Yy~oC(JgGivd}M zlhV-8788iP8loR=PcaA+_))*W%i?C>%8U{ib4_1&-20`^A#H-~kts{8=kY?lj4Y`9vuY^dCiaz=UWjl1r@xG6pLsevi~UBC zYB!s-{{FS;m{c*s_~`>bh9+=US&$T8_x1JS$Lu%t*#v`cy#8AJl~K>eHSJxgf&d zc1SrUlVigy*t`^XgY?u^08c9mzxpP6Mn%7jBPf$jJTi0|5!F<*clW_raixX4jDTd( z>j#IQN_}#nbrpug;f$)D!jd$InyrUx9L?{-EP&yl_KwZqe(vLi%DIq9;bJ1VDIS*G ze933ZRfgx`n)j!JKb}h;J90Ogn+l&E0xj>imI^^Vi6)B9&N!Zj5h3vLdM+Ee5NHNI zACLd?X~a;+Q-q|I)Tt9hG$-HZzt9$k4;+u5KlDh9jt|4p>B3jdL>xYHIL5Gu>O%bW z=+7UEy>IQKb9d#={q&ikcmnRu-u-V!?Z2v|jRUr6mAPl*yWjZ%A}W;k090qm_rLwS ze~+J>jd2>~WLG-P>QydPlql2_geBSiWqW<_YF5=F97ROHfnXm>0aK2tC4hL`C1Rs0HL|NxOINW&`mO3vmlNa+_DnWB<Eb)gZA)4KZ^%{@?)$f@uKO|VI ze;hWheJL(xO#Qy)TV6fY%kX&O_)+vV(VIapC&xS^sx{yP9?nzi#;xMSb> z_Fu<~&psRXed8N3xpims4)4b_{H2&C0CrkUA*QP0u~@V7Mgl7@5`WS0uEW0?qA@{E z!j|tJT(nd=jEzjh>#yuhQI<8z#Zg)K?Ag)S`^H=GgCBf9 zP7a*JLo8G3=PCfFFzOky=!9}Cx*=^k7;S}^z*+?x*2;*AZd)L zS6+KDdO7OY@GSQa5>>nh55;S7^4CG0CPybJX~g%Wg@c!uR=hFJNVm*&e*d@_)Kh(c z*MDEgN5!@ca0J(0c0*kAXa94I9(pT=kM4`nQwPz|KI>ez%eN7SS6_;OuNdDHOljDBJ&W`x!=$7o*u_NAm<8?&z*T=QjUk~_s|Ih)h zCzDuU8MJl~&dM}kexuE6LF1tXWhi_+prOZ+)bOKpZC$--OVN(k6A38wZ zN(`Jj9cLFW=h=6}?7>$Uk*rS(s3n{|81sCb^K*T4E+P3_UA*~ykHyT6Thf_e4(7ai zVg6YW!12+Y_tI1w6d?0u7Rw_^*1*g6w+Poow;jJRrYRJO=QR(f^T<}q1OQO>N6=$3-iPhVI3T?IswVJ&jh0p{To&6fG{FK(<<17Y9V1RtA4t|=?ivJb z)022QCe#6n%=!nhKK>NXMEP#m1#Bu}ay5qFusra$5264(7)OpBNl}&Jw<$ymeLZW^ zlIskE>&TG<0FZ^)ux3LHjhu}uFTOH57{oJh$~JG<3JZ5Kj+{6|=RF*)hy#b17^Ja4 z%|WjytCjcCP3Q)>Uz!XTU`DQi6fWrIk|pcXg{z}Nk4J9qAk6DLl@x4!kQ z^H|eIt7u+-_M5(~gvYq=zbDSwa7+VY$4?)N@e@RL2k;CXd^5(+9K)poYCJCH2<+yqv5eFo6% zCItMI;$$30Ma|ea{kx)J{iP9o7sVpciff(giQmwVh!@YRSXQovT&Yt0XYc%cetw%7vzjtqffkKDR{!6ke(@EtHZQrpguDkw*xap>w;*!f)`~f=F_uJAqmR=`L98aM2 z99)stUwaMUa4OB(&nf>~1>>ig*iaibZHlX|{6u{6Q=g8juDS{^wJyQRyUTZu7VJIy z^fTG+BZuJ71DKMF!+7n2JO3wt`e*UAKmK|QogD%MjaW+-5HD;-UM)m4+qrQYzHr842F_3; z&n3X?;kqwWy4N_*N|``VZVip5ZMnavSxar=+0|z%8fu+O-#K$Wk~`hKOi+Tc-^GSL zA9epzm6^c4UODfz8Ee93`b`%;iuy151mZ?~gwS`V0J4(<2V;2u%Q5)&b1{5mA6!68 zY8gujPTg40_vH2nvWs43!2*(c1-@q9HUEouS{%+@&GY`UM9c9zf0E0?oh(owYtAh# zRGOmWakW8z92>8S*m`X=^j{J+VxQI8Sg7D`7K;eun}pZzRe zdF2&0jzsa!!DRohXFm4@&@+eH!C~|rk6`-#6yWR&anG0U#X~&-EZKn8lm335{Ph#@ zgTMPP@$xG#rz%0t!AlvO6FPAJ&;c_*t7NTd<@D5(b@u`UTQPR=S$^>Szl+Cs9@iQN zdpBaM8!zr7HUI!MiAh93RBygH?)#&!#U)o=7fo}QM-?lrxsz{2gV6w06@ zn2G~<(j?ur05KjO`Ky=S5miD$WSEG0r5EUNMF7XQ-Hao$DxwlQuFAqHK#Gi2AhR>h zsTZ=;sTJASdF@0hAZsyYH&GuzE&0X{HR%BZ$~rkO%2!9kjw=AEFSCkboPZkUCq$TB*#YxWk5y;{nFO8Y)Ju!3UDx$@oiBrdq#Spu;G1%C0=jH+P z_7OS<@7KeUQ$RXp_9f~}eKk3W^nvW`zHHO77i|A#aB{^wOP2xa87#Avm}DJ%+Sf)y z2OJyxN##ya4$b03G%(!I#Aj^4(Ww%MmZM=EP4UJX zuf>7=2U4_RJDrb?BUbv`@BIy;mVx;4SMH08FUFcjRfYFSaz9>r;f44iV*IC{ev*EV z*u^=~xfAQaP8RrfxYp`;x<)8Ql50IBM?C>Lp5gcJi|>8+J9++(zxgKsuZ`zwa$P#G ze?N-4_s7FO`)Qm;9A)`Tg{)_dfRAo;BRjGBg_F-e!w`*)jwJ~3UZ3bRWAeZL@CPx< zCgfZH{LkakOD@d~yw_jAdhEO3{%b(f%bC}53kgTd61S&!Ry#lk6{38=MUnH5HLnqUuJ$xLs}si@qtE>B*uCA&+RkhCE`;cJ?A#!3{7shJ+ zV{*Tc1T5rOg@Lg zq;1g3XY6ZTp~A~hsG_ZzYFmZq02+jRT2EGdqI4B#&kz2iGQdw^9`nP8sr?y0Gh5io z&8X=RJffWf1~>V1nzb~PA&p+4mR$HfFZQ}ful07NagioFp7;>g#G@A#;{aLk?vxGZ z-70}iktOr2%%>H8UBB}0pk<09>j>%4_yckR4 zzN*g=Avgi@838qPLrkiJ;L-XveC|aD-|vTdA$YD|w#@2PFAQgi+uY6vjFz$&TTiH5 z?C1AMD(xrMo7W}hZlJoY*fSHi$z0{geQ#1~;Z8VOXKgUGqdAmvysi;5t2%p3ah+RT zmYRHJ47`@ihtn{5>{gmkotDu6GYnKrHxA$+BbCn_4dj3(@(*S3Aqv`iRK@f)v&*gVtsTMDplJ)Bz}vPf3Bj#(tEU;rxi zOa7jhz8}MosZ&izX2S_K-)L`L`|b{1?HfmA`@-oL`DP2x!1CQ0mE7Xrh+r5sv+Um2 zj?sWKCan8wO$;jcqN14*V}eLtkP0%<`8g!A5s@QgwhI=oXgaRP>ilaEchi2wBp8V^ zRoP5e#Jsi%LEne$1TxG zBkuGU`)nS}`)u0W51!1ZsHm#YgQ(f0)_+aclcCZ;tBFMd(2ZOB&Fed$+aYFMamC~; zS+UmQb^ovrXHItNtyrydXV_^svI1;tVmjRu171sJHIO+)9t4xD{rEVS_JbuO+$rQ8$qXv&-FbvWO87>D}BofV1DWk$Ic{=(VILkJ!@g5Uqp{~F&8IiM!Ly8eeP#p%OLI{ zf4zKH4E29<;atkmQ1*DalG%cw^pX3&5?>un`taU@<9lUAy`Nc8I6B{bUF^Rale8#7 z+K!$VphC{yu2a->mq1`2mVz-ucVA92Xj`B6i#I&q^o5D=dZlb3Am8C*%T;lPdYLY1 zGWo`P@53JZI*>0gG?KlQzteGH?W#=KpRUPvEuJLByuZXDd%?)}Q|EI|SHfA&mVggg zQjU?wxh}spUom0(<{1qrB!sQ*Y~tyJ&sJ<>UpeJ4KW;+ssLbdc)UlLNomxgH2l?F_ zigp#%Z%|cqz6YstcpGl6k!&N~!Qw9?r=WG$L_1+pV20wKF~>_VTcV;eR;|b5Jl;p3 z+#MUMhaQ(A;GiLz&s@d-6nj)kLPD?W&+H|;*we>aGo-r~(_Nx61B=swm7uoT>%E|w zZCr}<1&brxG3y=IdEBWF6yx=CMJ={iB~meXxK*_cnLu@Hv;~?mF;0N)rsgY_JFR%h@;o=ygZ47V57 z*_|gh5{Mo}>RW?jWe0`jzLPk%ZA%`0DZ=-{9M{or@e0YBVI(i-xNK&_5NtedZqID< zaV8+&6K3GRi&cPMV4w9dVNSmZ^0ut9==jfcb95rA_|bHRCIxaXSdZo@1E)=dwys&| zLBu5|1mTAHL*071dAoHY;>B^h?#~6-zkU~MdC@$e$pskdcu@)hpCdTLM=qPd_sMth>vrW(MHV6gNdg(zhA_Rs>&!==WyX zkuF)QXV$-W)%rh1HgQalI!2yCnIcs;Jj|D(lEW@$7OI!w1NiVqdqXC&JVWo3tZ&tF zOG@ZFT`oGDK6F$Dg3!0eAM!%gcreg4(%W$8Si!#H|;61(Vm4?<@Y8+P>Szt5)sJF8g1ueYewX)Dn46?r-`J zawZ=WC>3*ecFNcb@hxx@m9H3yMqU)X-E$v&mdw4^sngx|0_^hIE_eJl47@YF z?TS*LOfS)7cyM`yR!dJEYp6t1c&)-&VET6RHkWvGsU&9n=9GVA!-5b-xY4k`?9V-_ zlqD%n`LcM5>3^qOKe2zY2>0}XE#M%~@LxzHkK~*isIQMLD{YGG6d{9LS?EX)GuL}F z?zfb%83foTQir}zbHfB?DXs6f+6CU@%ZoMU^0uqB-1DZD(>=TA#G39h93zTLEpyzL zhE7VGb0{rV{n29Nf>MG}#Aw2Xe7PH@453Kd<^|ZqQ@n$$_lUWXehH3rqvN7d+(#yL zb;F9Oq02BJqyl;qRTXZy(lkQWmPVn->0VrO^FF7i4ME|{EUHn5U$Ic>;znMNd(*+O z!~*xC(1#=-4~_*u**6ERA2&n2=mq5U0{zONXk=l{7_djQqHThnyA?i9+&PhzVJ>fZ z6Xsd^KyhLQhxblj`9?_3(%}T<2(Cx_Fux%3L$l99Npw`&%Ebf9o4ZIHMM#F94Ywp5SEM47GIo=u z5-Z)mD3NBshT87J($KGa*PXd=cr(&tGUlJ$noOGp&uchPmYd19MUc5Zt%G54mU^!ff|iRab~r@ z#7^xs*YJ>v@x-uiwLL|~>we{o7OAt#KD+rt*5FI z{n5{vnStlI%5-GnPuraxr!W}B3CY~7FQMmXV&^!X zd#%epqU8TX5BwyP57$zr$c=ci@xwMS4Q|;Exq><;@~dSR^9O7k1PcR`(QF(2{IR6Q zkyv{#AO~=)e{qHXw@8nx+41&t;^EQ7x+aq;1cZpBd>?L|GKw*;q-o958q0t-JAzZ3 zLDga|kyIWImXmUF!d_L+;$>?Nd@{V%QDR>8;X4RsNK@q4-#KeR)rBDD4mYpn<@@n* zD^A0wXraxvB^YS4w%*uw@o+e2wV@cd5Yp}*sA`FLC0@~IImVQxwNxpbKQAv}S#`9v zU*CH()=)uq><|TXR1S`WBGO@aJIxOGN*tD~B7T;G1@LNesThhyv>JSlX%5X3CXQ9u z><9QFi>oK1*q;6nmPlP<7NdoG;3SwNMWc-EkJmA zppH}Tu`_o+_q;yk5ca?}jTDfgVsE&KMh zQ~2l`%q+d95X)K~p>W&xx!PVQK+SBIlf3O8*5`pr{ere?QF_I}fO9~dv&xS>K|zon z9s#|4Sh|sCaF-PNq>&rLqzAd(Cc1Ahd1@kC3}qn5J9X!(R1pX!xM&*jA?!;AMR=bi zExQ8-=IvCQ3KV|DL#2ZR-I=9j&tI@`=Wp&y&>~;1yV73lPgm}>?u6MhHcEFs_E^4s zLOPfhXv#LQ++D1uDKlvIe_z)aGtBnyeL^jRciz2>`?Kw~)c@x!kX2u|E=E@Jk3H1! z@U0@iGz<|KIWl-FV>QU9{J#9={I+#`?~&+CO_A>2q4ztttMP+Ko90}+uO$B8EB_rc zztShbvbCfidM=V`wDlqW0G)K`F;6xrQ8qO|dGdFeC`=#cK-gbK6Nw4$Gte4LNWU$%83H(-G0htr zW}bFeAB<`6aSp8oBZ}xJ@eUZMTMZ0`Yp@-Es)dHXL@;&7AO1yv+p3r@wq7oy)%|6{ zHKEV2bnUB%P3slo!+mP2RPfH(Z~rY5ULRVfiV!e2A>1UsRyv2R_g&ia;gVsqC2ptI z;-22*z_N>wKF6P?crvrE5foqu9HF?%N52%A%lN=3wbdB0ZzEgMyl#=KKf_g=WX~;N zo>o{|uEL&(26fIDWjZ=r${7u>*+XIG-H%Z$N)cu z6-2QqRo5W27T8kODep1VnL1HM((l6jA{ zw$X-uHA7rRe#e-(X$OpAS3aIG2rw054Ta&q-CeZ0urgKt{gXn5_Hj=ZEhOd~Mz$+5 zlKRr)s2@LE?rn6}ZBg;8;dRyh5!ZW0;RC@mZ@xY^?v%-a2!D4Wk6f~sJ>StJB+&GN zLVZpy(X#ZqekUcpOdP$cPwWGn^QK{B?9i%UWIit;g(qIJ~}51 ziQ>bL+WJQ-+TRFsi)#N+nS>^2MkoH=)uubS;~`E1TC9(`wp*&Rzz+;$BRRP^;-PZg zMBk@krx3TkatV};WYvIObsDg++J_o{ZcQv?mUqk#1aim^I=5OC+C`){tfw26tB4Vc=dV1(DN#^NWR@jeTL4M7DxxSqvm_eE7nX{ zSo&0zKvjF;39TH}QUatbU5+TzMiMSyyGKh36nXoXxo{;%d(3PU43t2_wO8u{(qnM2 zTr{qX`&TBVxT)jVg%2@NQap-N%I!i$vhRLwNbq4?Y`(h|c83G-um;7jhY2^^FA*S# zdA&DXM%Yv4{=QX=4nel)leZWD=sf){EO_$wZz%0I1nW`g(1ogSAuBraN)a_VZ{|jO zIoJi%!^5sWM0geWW1t=Ks%KZoPgI3g{{&8VwN5Qqo6P3f1Zzxbc#g&>5&Gd&;&p`n zo5Mf+%YsIJ;EX~_oJe(*oB23g@+m1((_#EPO=ff($GAc*BhR06;x~Z%q&g?4Go@RSEb<}0ONG?yPbwUg2kl(KS>&owpf1lG~rlBXqx580*R=vuT5_6bI@D8wamC@cdhMMh9b^)+P=TUfSNrm0@41NJQ|s~aEfH`bYB zjRqYmz=&^>zS$DX&0oEe_KCpdss74SahD3$PPR+bXOg);7)CBTjxM{IOu#tu{0Zm; zPAoT2KueXR4)-mbE`&Bz8G1yOBr{rwDPVNl4GjaW!)VK_mCt2f`cB6eG1W(XOxh%X z^g~{9mPTtGwMWYU(zMUysK}+kaRrxy%F=dt z9q9_k>j-0nJW}`hN`r(YSQXw4C7W+qSE^^mTgxtXeNyhycKF@JmdL9MfAYa)-Peu* z>SB<8i`9OF8_yZ!yX)G|Dc!?ZwxuI@!2o!S)=!4mAHIdy)o(i-SQv(T4szR#u? zD-OMT0tlADDzH`ZH@Y3Kes5LmeSDpdm5aeK(Xvu|E26&elsK21_3jQ6MS*!GIbKB5 z^L{K@Y_Y=HUuE0NSRW>AWSy4sf+xUz~L? zD%G(im~Q+edwspQ)ulcdeNYR`^a8gN%h<5K`5jYr3VT$YPG`06V?-=waMa}1dabRCZzMv zYlZkB-hv(%V!`aYGo`27yH(1Z8jm&ES$d>Rfq|G$V_bBM+rAX4c_3=FL z6pfOURbx*5=OO_1fVer|Emdq^UHF4inQ47y22(7XBC?ktt68^tT)J%r5mjE*-qJ`T(-(j;9f< z`<)KwpV`;wr=maQh9-soDS(}C3afQS96!xI1Y<3#fRpX4LfX0>_8zu{?+b%EKsrB( zzfVnm<7f4kAqXyzi2=uY6*)gt2@Dr+*|tB8Fe93*Mjd`Kkbyj1>4sv9_A-}J)8sP8 zvk`+#P|3o5$)GLJc3LJ?&FZ4aOypdt|9H8_GjoT*COILb)E($>Yb_9o+?uTv#eO54 z)AZUs5S?uNdj$$~-}}%SMnE3^{=!HhgIvw^N(ZYGqMd9ol$0>RuCSZW2F}vdjsF0f z+?gbh5W6{z+3WoPAkKU?!*aig76YhvjUM7;3I)_r}s829kY?wUk*3DlDPY_ha z7Qo4Bbu$G8kW}?AEzBsm|1V900KX7sM%+?UTvy1a^xP1KMC&2A+aCseJ_DaEhXQPv z@Z|rg(ib`d6^z6JiEh(9#N6N2xg~3*UgnT0_hg1+gn7B}1_$VJC9Z5=8M>kaBbfNH z)UI6dgiFx?s#mtUhK~ml#gK~&5`;@ZVU67S$KQha?=v>0mzDV2i`?k_h%PSDFd%@1 zRM?$(n+y}wZ1Kyu;6KA7y*%L0@6}T{cYRy0PS-;{haQdeceWRi^3O*b|0JT+`_yh; zv^FZDS zu@kItT!j#LEbFMUi{ICV)Ex(h^Prmz_M7mvf_`E%bb3YQ>pKwfcaO-xAzhl-lTA2l z1BmDmia5Tks!mM;K-**2f)Q@_lom6nLljQg#T-z%&9fpdwu{}>;wZE- z`#5amD1@qAbogIjckZ3XEDaa~lJ5l`NJP=TK-=Ek-0oLDkq0cDN3tQk!Xh4K`)r5P zifPNQWrmXbV04Uq|A4jlUz4qscoF`(Z83pR2jGaNWe%+hEPL*Xl@4;D!{^2YH>BP~ zEPM)GpMW6+6KU<*uK1uRXIgl=mJAjm*IiikCu*KU|7zTKU^N{dR{TaHxe`e9$qg4| zWe!&Zybl=mBqB$2egLPzLlATn_PNJr#BnoNSrq=MI_`Bd@C%_e3bb(lQ&?n0snhbZ z3}lp(ofL7DQ2U0JIJL|G&=&)yWHBQ{a>Y4umRW?_x`RMVoQEftFmB;gxt$ikIOG_R z#KT&EJde1yF-$*^q{Q()|DDaya53=H-w33x$mwv>-!B@YM<~96<%wDj*vj9$admVk8Fv>K>W4Rp z{mDS;>+PpAy+Kl0n~eQX4AIlo1fjbxd&R5QZ8DBs#?#VT1Y~@ywKk4uFhHXi#7y5 z5~mCrp7TB4S`W5A^nYT4;P0`gy%Ek4oP|)`HCj3w^T9PSKN&K4WY z>!~SLeq&#yU=EaQU0cbv2iwr)n4bN9yU{HW8E;DfmI-&cGH)9)7YHr%849`=2`0H8 z@LgmxSAD^tzL$$LGGwtb&G9=fL)mbg6gE=I%5~r9()0JHfT+`}`GV!4W{K-0E|d3x zuwQ@G>LEj1P7j>$lmha|V7R3gw#e}Q&L7R+py>Lm8rD3I(Qi&OE5R?{hWy~bNE>O- zszH#toG)o94d@qT4Sx^`4BfJBFK?P<+{;9;cNybDW{}53yH|kv`gJh+j@Km+G#y-< zz#CrzTQnj78AXxnIli1^=fy4}&J7^ww#3?M09xz=(3RdkmkX@YD^tW|Y$K4vD9M`I zT9sshOCM^=K2U1gG!&LrCU9bb(*EAqXi!I|5cIf7oI}IfS-pws6Ts>fuR-R%fOUy` zHbzaWBb#JKaTi zZrdw}j$X7qB!i4Qj|u#OyLx<&-jo(4Z%z!BXvQghdx)bhevB&Q67%t<1Nch*)_8{o z&xDfN3;oH&=Mlc>{(MRuFAQiwlQa10R{hN2Q9XrxC3}G-C^z90syDcQrtPt zE7_X!G)8K@URh0e>tWlRpJ(wRd)o_J#|p(J9OnHZw8=0?erAgg0Q$gCfVQ4jvy6t& zHt8hEwNDUYbn)Z%f{R2!d^}%D@=Nkr1d5~Qq=X?o_wqBRzfT;#NFAwGRufzoXMzYK z^_&rDMsLtyQ-Pu$U~oNmMDYi85DasY>{9E)G4fqIdvnX~k3*=hyH{_+lBe;FOe30@R;LKSrhd2(E9nEBxW)-*aNEkG)ALW|#Q!ecLmbwj2D!R`uLYdo!^Kl8?}30`?MW zH%Rr{H?3&RdM#XByqXP(pdCa=L&@KrP}-j)6=Ji2hY+8?Ov+VgxuKmFsn~;=j=^lw z*XUD%gRz#d)cr^KWXiboOboMpJ$SZLtKR=^Ra9c?#PU8+kW{+NKR@y72w7p~p@ z)hql;g<@;Z*fc3sX*-DcrN$?HLg$K|vG_x4O925e;aTmEUx}kUEX+6`zMXH7j0%Ac zh$~(fbU#yairHIsIP_v^|(G>~ea)7%}QH0+g8kdHSs*Y9#t5;!?WdO-bg@4xz?u z%*;39%=;t4eiwZJ3g{ldr=XXrSEJX%Uxu?-XXNKm`pI@$NB;|e+|M8&pq|T_xDSL@ zgv4mn{HWhXt=ogaT0cSTke*-#B2J?8QfXWasB($ZkN12IbsrlBqcm`J zZjdOmG+nbMm40N}Dxc1M&$Pe@*^g9utIWK2hhOd$B3;d6GDu@uMZaM%UT6Ts&~24B zaBqHRaA3>Fr_OMRvm{ANWm+ERMnUaW4TPJqaGLI6G!Y*RJ5Ze~SuiVO^3_!0UCcjA z2ze_oRBFA)lL_=@ax1a(#qDry z+1aC-1`O{&MZUjhNm`0>`>h)ruFeNzM`nT8#KD+5+d4T-)+Z|#lX5S$>R;%>IJc?_ zcLKopoPS9$oL!0)x96ag*nqtaf9+0pqhIe|{ob!HZcn<-bl?rT&b&N3bGzZ;_#d%S z{i9=dBA8uTIc>I=yq^qPc%_CKN+t5(~a6tXZp3n@}tTDP4>> z!E*R%%k;yplb<8Oh(x16z5X4dzb>DxJy1M8@yLOEaiqRq2d^!ai&dCtD?KZ`>7je( zTzLyIq5WG|urQg%Fu_3%aXr9xnjrn*3T9l9SvuE}eD)#2^lt3Vn{HS)N#CM#3MLJ7 z2O9>HW=mn1-Z*t|o{)O--dd^5!2NtQTQ4EVi}Pu{Yz>>vgpHA<#uLB?;TGd6+dNgC z_z6fle`pe8QSF>6P?NhjZ0@quU}*7^WkUT>B&bK0YLvwjbjePicdGKZzP=Epgq=`y zxk2}iWN(L+@2foAuh?)l&Gb4u;=ehJ-M+C=&b4$-LVbGajQKVVt>?QkL@j>|ac<^O>hZ-d8A=; zDRQ3r1K^V@Hi^2WW^dW3)X|lwa?Y&6$9+>z67DyT;ua(8-c==y^o*vlvb#9hE+i(u zBvf7w(Qa@MSK6<5B91nF64t1`r!^12V1t*2 zJhQ#k^3u>Jei}3hswtQcF@nz&{!x&#(k6j)*cIBMoP2i7q*6!PUKTnN zkpSb00WFnoS^yK!NX7Qser2AY|3@rTg-o#u%lVNmw)d3PhWE`E6cki?J}Yb_Q*A>R z>-o54M#f?$ewC)#o+LjzF^NDzW6aI43;w5BwGlu*6lkvjtc=3Kl*>J3_3d*kKLYCu zjK*%xP6Ay_GAu>hb7EtS%APh5>xPTgkp4A(Lnu!6i~3M%9^=A{8r(A8M+O@PMHM2Z zHtz4z)720zXXGaVO~okSXZ0WYIa@6NK1vR+VWC^vujrzMgn@pgqr7bTT{01i$2u}^po61QvRl=5(X+KtQg8I^QjX|?v0jYR zYoK%Sl(TL9!P}7DFhEh{$-5duG*ZD<2Nyr5Ct+T_51VLFyP`@m2ld4ox}t%CO4UY+ z6mvld*vnG)j@V1A5S1qu$xeDb+K(xMXu?C-{4K&~UTxq`#0PbACpW+m{S%#oL380! z1vXa!)D$rF_%QmGzmZrB*$;~Q^sI(Ykd#L0E^+UrlE4aeUQReGiwNW-m>@6?23M=r zCV&c!14060adOEU`n(^8ZH0md>IFoRz(c!Fm?n~D)ii`V3Nwjlk9=+^)! z)xfCCocu$%%9FOJ+1+NaQadfAY9isl=UI}19ZHvtlbr9IlNL}zWXjV|Sp6hr-iZw` z$3}0+mEcRvFJX}8zCpV9Ou;9=26vxX;)n3_tZ9|Ymxu8Z?NA2_bQ5hMslDRxYGT`R zy-(ory2SsqtGqqr{DHJwGHv-Fq7&Wh%yqV#Yud=xQms7;f#F4qIK3At-=nL+T)pg2 zpoM)?!KvvgYdIQ#J3R~Z5;_`DLp%rLjPM&O7h3y0*h zGAtzZYzPeu8DM^~7g&X+BX9O9B83>ByzI_haCUiYWnIj`8pL-i{JTZUnP3vL1Wrym zs!mW~$#Hw)W9k)lHMYa&=863H{cMl2eVTjX-mJz_xuyq2`oKI!U3OPb{yN{xQ*qJJ*>_-#V=+C zwU)`q&l}7Z1r?V>`|pTW=aPpzra;VR6G#2`NY8OD0v<+dH?07Nhfu%dnOznmQ3jmK znXsP_YqP|&?;r)t0CZqZOQX@s($~7l)3-IvK|e^497o8zB>FjHZ#Q9?F|_aWwNc&2OVmx zgl*~DPLaR$`-!pjCp-Htv|qE&XjzGgoK_b^03<-#pLBJE2*4d-aSZzbhbYa34heKV z?r#@7dN)Q8<#|@l`pg0cf&8KzEC|CdY6W?Tg{|&-@f2JX1Sf8GLmVo>()@uKxj-Jw z768kPGVpfh)HX;)KWCdR0RFL^W|h&!^(8w^Ac!HdZq88cvsU7kM5#JA@y?H~)LnrR zOVwkoF|%(pKlF9Je8Z$Wu_>$glhjGK-=r^sQMi>Iq9GmJ%kv4<5?OQ-M-;YM0hDsI zBM|_ay7GNqyFZYb9GcebYq*O3j!WB6c2K4->+6%MeQt&$lW(C;fvPEnWO=z1aDj}FY`b{?Q^ej~Wom_~$wH=LR1BfKNThw%x zUs#<)%Df&52dD8HF-%>gNAP{4Ef#R+R?0&%q@|E!;z(e}$4em>M>QZnxBi@_=afpt z@W}AwZiT_4OBNk>0%bj%;NAnI+U=k7fXZNG{i^|1Qu+6lBc380bTa%CTf{q0Q1f(C zQzl#HDAg!!-22!s;P^giFcu&7*tfy2;WkEMqr-PtluTjtACBcn*MEq>6LDHD87{qn zB*PR7687WQ!!sAb!I&!F2%GnOevS_>^Q)GNocPL|hnbW<&78a9U*hz(K>m3bm1VEM zQq9(7z=~fe&GvbLZ!aG=bfs-a>xfT`#+z>@!D9thlVbMZ@Z+1rra-hE|MA?gN`Ery z)OYE$3;Usq2UcMi!CM~XR$f_^+vj9m_FqDNJd7$A^5o<5NtH~DDj)r@`?09|0@cx0 zE8<)7vUZL_N{(QTqPvq;S3j*jFuDs_9MveB_>T?$R`NQE$k{@E?vEmay&+qm5Yn}f-`9EF5 z1V?P)lH3Bl)W{U)g3E9bN2OZMAIH^xHTbn*Z}$KEbO$SZq{U?-rvdir|L@v|?|3MD zGFYT$F8p=pB<24dfd8@jK^16%lnAiI?RvSMmj7s~|GsT^uv`Gvra`GglTX!uZ~IS4 zO$4I~RHo+3^{5Q{|5fV4FcYjC#5u2X>ALklZT-Inn2N;^9ks4T*~vTpQ}_QGF(}1h z08RuyS&Sb1_w)Zx$;Zl=wp+KKz02q3|NC;xQ`>xphj|;WP;0zP0(=~@Qc99lpN#|m EA1lRg!2kdN literal 0 HcmV?d00001 diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 079d4c5550537..c6090d9ec30c7 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -3,8 +3,8 @@ layout: global title: Spark Streaming + Kinesis Integration --- [Amazon Kinesis](http://aws.amazon.com/kinesis/) is a fully managed service for real-time processing of streaming data at massive scale. -The Kinesis input DStream and receiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License (ASL). -The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases. +The Kinesis receiver creates an input DStream using the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License (ASL). +The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concepts of Workers, Checkpoints, and Shard Leases. Here we explain how to configure Spark Streaming to receive data from Kinesis. #### Configuring Kinesis @@ -15,7 +15,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m #### Configuring Spark Streaming Application -1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} @@ -23,10 +23,11 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m **Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your application.** -2. **Programming:** In the streaming application code, import `KinesisUtils` and create input DStream as follows. +2. **Programming:** In the streaming application code, import `KinesisUtils` and create the input DStream as follows:

- `[endpoint URL]`: Valid Kinesis endpoints URL can be found [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region). + - `streamingContext`: StreamingContext containg an application name used by Kinesis to tie this Kinesis application to the Kinesis stream - `[checkpoint interval]`: The interval at which the Kinesis client library is going to save its position in the stream. For starters, set it to the same as the batch interval of the streaming application. + - `[Kinesis stream name]`: The Kinesis stream that this streaming application receives from + - The application name used in the streaming context becomes the Kinesis application name + - The application name must be unique for a given account and region. + - The Kinesis backend automatically associates the application name to the Kinesis stream using a DynamoDB table (always in the us-east-1 region) created during Kinesis Client Library initialization. + - Changing the application name or stream name can lead to Kinesis errors in some cases. If you see errors, you may need to manually delete the DynamoDB table. - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see later section and Amazon Kinesis API documentation for more details). - *Points to remember:* + - `[endpoint URL]`: Valid Kinesis endpoints URL can be found [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region). - - The name used in the context of the streaming application must be unique for a given account and region. Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream. - - A single Kinesis input DStream can receive many Kinesis shards by spinning up multiple KinesisRecordProcessor threads. Note that there is no correlation between number of shards in Kinesis and the number of partitions in the generated RDDs that is used for processing the data. - - You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread. - - Horizontal scaling is achieved by autoscaling additional Kinesis input DStreams (separate processes) up to the number of current shards for a given stream, of course. + - `[checkpoint interval]`: The interval (e.g., Duration(2000) = 2 seconds) at which the Kinesis Client Library saves its position in the stream. For starters, set it to the same as the batch interval of the streaming application. -3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see Kinesis Checkpointing section and Amazon Kinesis API documentation for more details). - - A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL. It is used to store KCL's checkpoint information. - - If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch. +3. **Deploying:** Package `spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + *Points to remember at runtime:* + + - Kinesis data processing is ordered per partition and occurs at-least once per message. + + - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamodDB. + + - A single Kinesis stream shard is processed by one input DStream at a time. + +

+ Spark Streaming Kinesis Architecture + +

+ + - A single Kinesis input DStream can read from multiple shards of a Kinesis stream by creating multiple KinesisRecordProcessor threads. + + - Multiple input DStreams running in separate processes/instances can read from a Kinesis stream. + + - You never need more Kinesis input DStreams than the number of Kinesis stream shards as each input DStream will create at least one KinesisRecordProcessor thread that handles a single shard. + + - Horizontal scaling is achieved by adding/removing Kinesis input DStreams (within a single process or across multiple processes/instances) - up to the total number of Kinesis stream shards per the previous point. + + - The Kinesis input DStream will balance the load between all DStreams - even across processes/instances. + + - The Kinesis input DStream will balance the load during re-shard events (merging and splitting) due to changes in load. + + - As a best practice, it's recommended that you avoid re-shard jitter by over-provisioning when possible. + + - Each Kinesis input DStream maintains its own checkpoint info. See the Kinesis Checkpointing section for more details. + + - There is no correlation between the number of Kinesis stream shards and the number of RDD partitions/shards created across the Spark cluster during input DStream processing. These are 2 independent partitioning schemes. #### Running the Example To run the example, + - Download Spark source and follow the [instructions](building-with-maven.html) to build Spark with profile *-Pkinesis-asl*. - mvn -Pkinesis-asl -DskipTests clean package + mvn -Pkinesis-asl -DskipTests clean package + -- Set up Kinesis stream (see earlier section). Note the name of the Kinesis stream, and the endpoint URL corresponding to the region the stream is based on. +- Set up Kinesis stream (see earlier section) within AWS. Note the name of the Kinesis stream and the endpoint URL corresponding to the region where the stream was created. - Set up the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_KEY with your AWS credentials. - In the Spark root directory, run the example as +
@@ -92,19 +132,19 @@ To run the example,
- This will wait for data to be received from Kinesis. + This will wait for data to be received from the Kinesis stream. -- To generate random string data, in another terminal, run the associated Kinesis data producer. +- To generate random string data to put onto the Kinesis stream, in another terminal, run the associated Kinesis data producer. bin/run-example streaming.KinesisWordCountProducerASL [Kinesis stream name] [endpoint URL] 1000 10 - This will push random words to the Kinesis stream, which should then be received and processed by the running example. + This will push 1000 lines per second of 10 random numbers per line to the Kinesis stream. This data should then be received and processed by the running example. #### Kinesis Checkpointing -The Kinesis receiver checkpoints the position of the stream that has been read periodically, so that the system can recover from failures and continue processing where it had left off. Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. - -- If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable. +- Each Kinesis input DStream periodically stores the current position of the stream in the backing DynamoDB table. This allows the system to recover from failures and continue processing where the DStream left off. -- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored). In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data. +- Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. -- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency. +- If no Kinesis checkpoint info exists when the input DStream starts, it will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable. +- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no input DStreams are running (and no checkpoint info is being stored). +- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 3d4bce49666ed..41f170580f452 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -233,7 +233,7 @@ $ ./bin/run-example streaming.NetworkWordCount localhost 9999
{% highlight bash %} -$ ./bin/run-example JavaNetworkWordCount localhost 9999 +$ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 {% endhighlight %}
@@ -262,7 +262,7 @@ hello world {% highlight bash %} # TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount -$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999 +$ ./bin/run-example streaming.NetworkWordCount localhost 9999 ... ------------------------------------------- Time: 1357008430000 ms @@ -285,12 +285,22 @@ need to know to write your streaming applications. ## Linking -To write your own Spark Streaming program, you will have to add the following dependency to your - SBT or Maven project: +Similar to Spark, Spark Streaming is available through Maven Central. To write your own Spark Streaming program, you will have to add the following dependency to your SBT or Maven project. + +
+
- groupId = org.apache.spark - artifactId = spark-streaming_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} + + org.apache.spark + spark-streaming_{{site.SCALA_BINARY_VERSION}} + {{site.SPARK_VERSION}} + +
+
+ + libraryDependencies += "org.apache.spark" % "spark-streaming_{{site.SCALA_BINARY_VERSION}}" % "{{site.SPARK_VERSION}}" +
+
For ingesting data from sources like Kafka, Flume, and Kinesis that are not present in the Spark Streaming core @@ -302,7 +312,7 @@ some of the common ones are as follows.
- + @@ -373,7 +383,7 @@ or a special __"local[\*]"__ string to run in local mode. In practice, when runn you will not want to hardcode `master` in the program, but rather [launch the application with `spark-submit`](submitting-applications.html) and receive it there. However, for local testing and unit tests, you can pass "local[*]" to run Spark Streaming -in-process. Note that this internally creates a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. +in-process. Note that this internally creates a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) @@ -447,11 +457,12 @@ Spark Streaming has two categories of streaming sources. - *Basic sources*: Sources directly available in the StreamingContext API. Example: file systems, socket connections, and Akka actors. - *Advanced sources*: Sources like Kafka, Flume, Kinesis, Twitter, etc. are available through extra utility classes. These require linking against extra dependencies as discussed in the [linking](#linking) section. -Every input DStream (except file stream) is associated with a single [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) object which receives the data from a source and stores it in Spark's memory for processing. A receiver is run within a Spark worker/executor as a long-running task, hence it occupies one of the cores allocated to the Spark Streaming application. Hence, it is important to remember that Spark Streaming application needs to be allocated enough cores to process the received data, as well as, to run the receiver(s). Therefore, few important points to remember are: +Every input DStream (except file stream) is associated with a single [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) object which receives the data from a source and stores it in Spark's memory for processing. So every input DStream receives a single stream of data. Note that in a streaming application, you can create multiple input DStreams to receive multiple streams of data in parallel. This is discussed later in the [Performance Tuning](#level-of-parallelism-in-data-receiving) section. + +A receiver is run within a Spark worker/executor as a long-running task, hence it occupies one of the cores allocated to the Spark Streaming application. Hence, it is important to remember that Spark Streaming application needs to be allocated enough cores to process the received data, as well as, to run the receiver(s). Therefore, few important points to remember are: ##### Points to remember: {:.no_toc} - - If the number of cores allocated to the application is less than or equal to the number of input DStreams / receivers, then the system will receive data, but not be able to process them. - When running locally, if you master URL is set to "local", then there is only one core to run tasks. That is insufficient for programs with even one input DStream (file streams are okay) as the receiver will occupy that core and there will be no core left to process the data. @@ -1089,9 +1100,34 @@ parallelizing the data receiving. Note that each input DStream creates a single receiver (running on a worker machine) that receives a single stream of data. Receiving multiple data streams can therefore be achieved by creating multiple input DStreams and configuring them to receive different partitions of the data stream from the source(s). -For example, a single Kafka input stream receiving two topics of data can be split into two +For example, a single Kafka input DStream receiving two topics of data can be split into two Kafka input streams, each receiving only one topic. This would run two receivers on two workers, -thus allowing data to be received in parallel, and increasing overall throughput. +thus allowing data to be received in parallel, and increasing overall throughput. These multiple +DStream can be unioned together to create a single DStream. Then the transformations that was +being applied on the single input DStream can applied on the unified stream. This is done as follows. + +
+
+{% highlight scala %} +val numStreams = 5 +val kafkaStreams = (1 to numStreams).map { i => KafkaUtils.createStream(...) } +val unifiedStream = streamingContext.union(kafkaStreams) +unifiedStream.print() +{% endhighlight %} +
+
+{% highlight java %} +int numStreams = 5; +List> kafkaStreams = new ArrayList>(numStreams); +for (int i = 0; i < numStreams; i++) { + kafkaStreams.add(KafkaUtils.createStream(...)); +} +JavaPairDStream unifiedStream = streamingContext.union(kafkaStreams.get(0), kafkaStreams.subList(1, kafkaStreams.size())); +unifiedStream.print(); +{% endhighlight %} +
+
+ Another parameter that should be considered is the receiver's blocking interval. For most receivers, the received data is coalesced together into large blocks of data before storing inside Spark's memory. @@ -1107,7 +1143,7 @@ before further processing. ### Level of Parallelism in Data Processing {:.no_toc} -Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the +Cluster resources can be under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is decided by the [config property] (configuration.html#spark-properties) `spark.default.parallelism`. You can pass the level of From da35330e830a85008c0bf9f0725418e4dfe7ac66 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 6 Sep 2014 14:49:25 -0700 Subject: [PATCH 06/19] Spark-3406 add a default storage level to python RDD persist API Author: Holden Karau Closes #2280 from holdenk/SPARK-3406-Python-RDD-persist-api-does-not-have-default-storage-level and squashes the following commits: 33eaade [Holden Karau] As Josh pointed out, sql also override persist. Make persist behave the same as in the underlying RDD as well e658227 [Holden Karau] Fix the test I added e95a6c5 [Holden Karau] The Python persist function did not have a default storageLevel unlike the Scala API. Noticed this issue because we got a bug report back from the book where we had documented it as if it was the same as the Scala API --- python/pyspark/rdd.py | 7 ++++++- python/pyspark/sql.py | 3 ++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 04f13523b431d..aa90297855c93 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -212,11 +212,16 @@ def cache(self): self.persist(StorageLevel.MEMORY_ONLY_SER) return self - def persist(self, storageLevel): + def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): """ Set this RDD's storage level to persist its values across operations after the first time it is computed. This can only be used to assign a new storage level if the RDD does not have a storage level set yet. + If no storage level is specified defaults to (C{MEMORY_ONLY_SER}). + + >>> rdd = sc.parallelize(["b", "a", "c"]) + >>> rdd.persist().is_cached + True """ self.is_cached = True javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index e7f573cf6da44..97a51b9f8a24f 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -29,6 +29,7 @@ from pyspark.rdd import RDD, PipelinedRDD from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer +from pyspark.storagelevel import StorageLevel from itertools import chain, ifilter, imap @@ -1665,7 +1666,7 @@ def cache(self): self._jschema_rdd.cache() return self - def persist(self, storageLevel): + def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): self.is_cached = True javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel) self._jschema_rdd.persist(javaStorageLevel) From 607ae39c22947dad8e65cbcec310367925f62eba Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 6 Sep 2014 15:04:50 -0700 Subject: [PATCH 07/19] [SPARK-3397] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT Author: GuoQiang Li Closes #2268 from witgo/SPARK-3397 and squashes the following commits: eaf913f [GuoQiang Li] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/java8-tests/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 26 files changed, 26 insertions(+), 26 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index de7b75258e3c5..4146168fc804b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..93db0d5efda5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 55bfe0b841ea4..b2b788a4bc13b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9b12cb0c29c9f..3f46c40464d3b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index b345276b08ba3..ac291bd4fde20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f71f6b6c4f931..7d31e32283d88 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..2067c473f0e3f 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..371f1f1e9d39a 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..1d7dd49d15c22 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e48968feb3bc 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 5308bb4e440ea..8658ecf5abfab 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..560244ad93369 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..71a078d58a8d8 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..3f49b1d63b6e1 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..a5eeef88e9d62 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a5eaea80afd71..1efa9045208fd 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..fcc5f90d870e8 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 830711a46a35b..0d756f873e486 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..bd110218d34f7 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..124fc107cb8aa 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..45a4c6dc98da0 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index ce35520a28609..12f900c91eb98 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..f36674476770c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 51744ece0412d..7dadbba58fd82 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..7fcd7ee0d4547 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..fd934b7726181 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml From 21a1e1bb893512b2f68598ab0c0ec8c33e8d9909 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 6 Sep 2014 15:08:43 -0700 Subject: [PATCH 08/19] [SPARK-3273][SPARK-3301]We should read the version information from the same place Author: GuoQiang Li Closes #2175 from witgo/SPARK-3273 and squashes the following commits: cf9c65a [GuoQiang Li] We should read the version information from the same place 2a44e2f [GuoQiang Li] The spark version in the welcome message of pyspark is not correct --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 ++--- core/src/main/scala/org/apache/spark/package.scala | 1 + .../org/apache/spark/scheduler/EventLoggingListener.scala | 3 ++- .../apache/spark/scheduler/EventLoggingListenerSuite.scala | 5 +++-- python/pyspark/shell.py | 4 ++-- .../main/scala/org/apache/spark/repl/SparkILoopInit.scala | 6 ++++-- 6 files changed, 14 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6eaf6794764c7..24d1a8f9eceae 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -49,6 +49,7 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkD import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ +import org.apache.spark.SPARK_VERSION import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} @@ -825,7 +826,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** The version of Spark on which this application is running. */ - def version = SparkContext.SPARK_VERSION + def version = SPARK_VERSION /** * Return a map from the slave to the max memory available for caching and the remaining @@ -1297,8 +1298,6 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { - private[spark] val SPARK_VERSION = "1.2.0-SNAPSHOT" - private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 5cdbc306e56a0..e2fc9c649925e 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -44,4 +44,5 @@ package org.apache package object spark { // For package docs only + val SPARK_VERSION = "1.2.0-SNAPSHOT" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 4b99f630440ad..64b32ae0edaac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -29,6 +29,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec +import org.apache.spark.SPARK_VERSION import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} /** @@ -86,7 +87,7 @@ private[spark] class EventLoggingListener( sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) logger.newFile(COMPRESSION_CODEC_PREFIX + codec) } - logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) + logger.newFile(SPARK_VERSION_PREFIX + SPARK_VERSION) logger.newFile(LOG_PREFIX + logger.fileIndex) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index fead883793430..e5315bc93e217 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec +import org.apache.spark.SPARK_VERSION import org.apache.spark.util.{JsonProtocol, Utils} import java.io.File @@ -196,7 +197,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { assert(info.logPaths.size > 0) - assert(info.sparkVersion === SparkContext.SPARK_VERSION) + assert(info.sparkVersion === SPARK_VERSION) assert(info.compressionCodec.isDefined === compressionCodec.isDefined) info.compressionCodec.foreach { codec => assert(compressionCodec.isDefined) @@ -381,7 +382,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) assert(file.isDefined) - assert(EventLoggingListener.parseSparkVersion(file.get) === SparkContext.SPARK_VERSION) + assert(EventLoggingListener.parseSparkVersion(file.get) === SPARK_VERSION) } private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index fde3c29e5e790..89cf76920e353 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -49,9 +49,9 @@ ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /__ / .__/\_,_/_/ /_/\_\ version 1.0.0-SNAPSHOT + /__ / .__/\_,_/_/ /_/\_\ version %s /_/ -""") +""" % sc.version) print("Using Python version %s (%s, %s)" % ( platform.python_version(), platform.python_build()[0], diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 910b31d209e13..7667a9c11979e 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -14,6 +14,8 @@ import scala.reflect.internal.util.Position import scala.util.control.Exception.ignoring import scala.tools.nsc.util.stackTraceString +import org.apache.spark.SPARK_VERSION + /** * Machinery for the asynchronous initialization of the repl. */ @@ -26,9 +28,9 @@ trait SparkILoopInit { ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 1.0.0-SNAPSHOT + /___/ .__/\_,_/_/ /_/\_\ version %s /_/ -""") +""".format(SPARK_VERSION)) import Properties._ val welcomeMsg = "Using Scala %s (%s, Java %s)".format( versionString, javaVmName, javaVersion) From 110fb8b24d2454ad7c979c3934dbed87650f17b8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 6 Sep 2014 16:12:29 -0700 Subject: [PATCH 09/19] [SPARK-2334] fix AttributeError when call PipelineRDD.id() The underline JavaRDD for PipelineRDD is created lazily, it's delayed until call _jrdd. The id of JavaRDD is cached as `_id`, it saves a RPC call in py4j for later calls. closes #1276 Author: Davies Liu Closes #2296 from davies/id and squashes the following commits: e197958 [Davies Liu] fix style 9721716 [Davies Liu] fix id of PipelineRDD --- python/pyspark/rdd.py | 6 ++++++ python/pyspark/sql.py | 9 +++++---- python/pyspark/tests.py | 9 +++++++++ 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index aa90297855c93..266090e3ae8f3 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2075,6 +2075,7 @@ def pipeline_func(split, iterator): self.ctx = prev.ctx self.prev = prev self._jrdd_val = None + self._id = None self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self._partitionFunc = prev._partitionFunc if self.preservesPartitioning else None @@ -2105,6 +2106,11 @@ def _jrdd(self): self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val + def id(self): + if self._id is None: + self._id = self._jrdd.id() + return self._id + def _is_pipelinable(self): return not (self.is_cached or self.is_checkpointed) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 97a51b9f8a24f..004d4937cbe1c 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1525,7 +1525,7 @@ def __init__(self, jschema_rdd, sql_ctx): self.sql_ctx = sql_ctx self._sc = sql_ctx._sc self._jschema_rdd = jschema_rdd - + self._id = None self.is_cached = False self.is_checkpointed = False self.ctx = self.sql_ctx._sc @@ -1543,9 +1543,10 @@ def _jrdd(self): self._lazy_jrdd = self._jschema_rdd.javaToPython() return self._lazy_jrdd - @property - def _id(self): - return self._jrdd.id() + def id(self): + if self._id is None: + self._id = self._jrdd.id() + return self._id def saveAsParquetFile(self, path): """Save the contents as a Parquet file, preserving the schema. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 3e74799e82845..2ade15b35ab4e 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -281,6 +281,15 @@ def func(): class TestRDDFunctions(PySparkTestCase): + def test_id(self): + rdd = self.sc.parallelize(range(10)) + id = rdd.id() + self.assertEqual(id, rdd.id()) + rdd2 = rdd.map(str).filter(bool) + id2 = rdd2.id() + self.assertEqual(id + 1, id2) + self.assertEqual(id2, rdd2.id()) + def test_failed_sparkcontext_creation(self): # Regression test for SPARK-1550 self.sc.stop() From 3fb57a0ab3d76fda2301dbe9f2f3fa6743b4ed78 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 6 Sep 2014 19:06:30 -0700 Subject: [PATCH 10/19] [SPARK-3353] parent stage should have lower stage id. Previously parent stages had higher stage id, but parent stages are executed first. This pull request changes the behavior so parent stages would have lower stage id. For example, command: ```scala sc.parallelize(1 to 10).map(x=>(x,x)).reduceByKey(_+_).count ``` breaks down into 2 stages. The old web UI: ![screen shot 2014-09-04 at 12 42 44 am](https://cloud.githubusercontent.com/assets/323388/4146177/60fb4f42-3407-11e4-819f-853eb0e22b25.png) Web UI with this patch: ![screen shot 2014-09-04 at 12 44 55 am](https://cloud.githubusercontent.com/assets/323388/4146178/62e08e62-3407-11e4-867b-a36b10534464.png) Author: Reynold Xin Closes #2273 from rxin/lower-stage-id and squashes the following commits: abbb4c6 [Reynold Xin] Fixed SparkListenerSuite. 0e02379 [Reynold Xin] Updated DAGSchedulerSuite. 54ccea3 [Reynold Xin] [SPARK-3353] parent stage should have lower stage id. --- .../apache/spark/scheduler/DAGScheduler.scala | 4 +-- .../spark/scheduler/DAGSchedulerSuite.scala | 25 ++++++++++++------- .../spark/scheduler/SparkListenerSuite.scala | 2 +- 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2ccc27324ac8c..6fcf9e31543ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -241,9 +241,9 @@ class DAGScheduler( callSite: CallSite) : Stage = { + val parentStages = getParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() - val stage = - new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) + val stage = new Stage(id, rdd, numTasks, shuffleDep, parentStages, jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 0bb91febde9d7..aa73469b6acd8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} @@ -97,10 +98,12 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 val sparkListener = new SparkListener() { - val successfulStages = new HashSet[Int]() - val failedStages = new ArrayBuffer[Int]() + val successfulStages = new HashSet[Int] + val failedStages = new ArrayBuffer[Int] + val stageByOrderOfExecution = new ArrayBuffer[Int] override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageInfo = stageCompleted.stageInfo + stageByOrderOfExecution += stageInfo.stageId if (stageInfo.failureReason.isEmpty) { successfulStages += stageInfo.stageId } else { @@ -231,6 +234,13 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F runEvent(JobCancelled(jobId)) } + test("[SPARK-3353] parent stage should have lower stage id") { + sparkListener.stageByOrderOfExecution.clear() + sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() + assert(sparkListener.stageByOrderOfExecution.length === 2) + assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) + } + test("zero split job") { var numResults = 0 val fakeListener = new JobListener() { @@ -457,7 +467,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F null, null)) assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(CompletionEvent( @@ -515,8 +525,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F // Listener bus should get told about the map stage failing, but not the reduce stage // (since the reduce stage hasn't been started yet). assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - assert(sparkListener.failedStages.contains(1)) - assert(sparkListener.failedStages.size === 1) + assert(sparkListener.failedStages.toSet === Set(0)) assertDataStructuresEmpty } @@ -563,14 +572,12 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val stageFailureMessage = "Exception failure in map stage" failed(taskSets(0), stageFailureMessage) - assert(cancelledStages.contains(1)) + assert(cancelledStages.toSet === Set(0, 2)) // Make sure the listeners got told about both failed stages. assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.successfulStages.isEmpty) - assert(sparkListener.failedStages.contains(1)) - assert(sparkListener.failedStages.contains(3)) - assert(sparkListener.failedStages.size === 2) + assert(sparkListener.failedStages.toSet === Set(0, 2)) assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 3b0b8e2f68c97..ab35e8edc4ebf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -180,7 +180,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers rdd3.count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {2} // Shuffle map stage + result stage - val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get + val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 3).get stageInfo3.rddInfos.size should be {1} // ShuffledRDD stageInfo3.rddInfos.forall(_.numPartitions == 4) should be {true} stageInfo3.rddInfos.exists(_.name == "Trois") should be {true} From 6754570d83044c4fbaf0d2ac2378a0e081a93629 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sun, 7 Sep 2014 17:57:59 -0700 Subject: [PATCH 11/19] [SPARK-3394] [SQL] Fix crash in TakeOrdered when limit is 0 This resolves https://issues.apache.org/jira/browse/SPARK-3394 Author: Eric Liang Closes #2264 from ericl/spark-3394 and squashes the following commits: c87355b [Eric Liang] refactor bfb6140 [Eric Liang] change RDD takeOrdered instead 7a51528 [Eric Liang] fix takeordered when limit = 0 --- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 +++++++++++-------- .../scala/org/apache/spark/rdd/RDDSuite.scala | 7 ++++++ 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 1cf55e86f6c81..a9b905b0d1a63 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1127,15 +1127,19 @@ abstract class RDD[T: ClassTag]( * @return an array of top elements */ def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { - mapPartitions { items => - // Priority keeps the largest elements, so let's reverse the ordering. - val queue = new BoundedPriorityQueue[T](num)(ord.reverse) - queue ++= util.collection.Utils.takeOrdered(items, num)(ord) - Iterator.single(queue) - }.reduce { (queue1, queue2) => - queue1 ++= queue2 - queue1 - }.toArray.sorted(ord) + if (num == 0) { + Array.empty + } else { + mapPartitions { items => + // Priority keeps the largest elements, so let's reverse the ordering. + val queue = new BoundedPriorityQueue[T](num)(ord.reverse) + queue ++= util.collection.Utils.takeOrdered(items, num)(ord) + Iterator.single(queue) + }.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord) + } } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 926d4fecb5b91..499dcda3dae8f 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -521,6 +521,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sortedLowerK === Array(1, 2, 3, 4, 5)) } + test("takeOrdered with limit 0") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val rdd = sc.makeRDD(nums, 2) + val sortedLowerK = rdd.takeOrdered(0) + assert(sortedLowerK.size === 0) + } + test("takeOrdered with custom ordering") { val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) implicit val ord = implicitly[Ordering[Int]].reverse From 39db1bfdab434c867044ad4c70fe93a96fb287ad Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 7 Sep 2014 21:34:46 -0400 Subject: [PATCH 12/19] [SQL] Update SQL Programming Guide Author: Michael Armbrust Author: Yin Huai Closes #2258 from marmbrus/sqlDocUpdate and squashes the following commits: f3d450b [Michael Armbrust] fix brackets bea3bfa [Michael Armbrust] Davies suggestions 3a29fe2 [Michael Armbrust] tighten visibility a71aa36 [Michael Armbrust] Draft of doc updates 52932c0 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into sqlDocUpdate 1e8c849 [Yin Huai] Update the example used for applySchema. 9457c39 [Yin Huai] Update doc. 31ba240 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeDoc 29bc668 [Yin Huai] Draft doc for data type and schema APIs. --- docs/sql-programming-guide.md | 952 ++++++++++++++++-- .../scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../apache/spark/sql/UdfRegistration.scala | 2 +- .../columnar/InMemoryColumnarTableScan.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 2 +- .../spark/sql/parquet/ParquetFilters.scala | 2 +- .../server/SparkSQLOperationManager.scala | 4 +- 7 files changed, 865 insertions(+), 101 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 8f7fb5431cfb6..1814fef465cac 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -68,6 +68,16 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD {% endhighlight %} +In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict +super set of the functionality provided by the basic SQLContext. Additional features include +the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the +ability to read data from Hive tables. To use a HiveContext, you do not need to have an +existing hive setup, and all of the data sources available to a SQLContext are still available. +HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using HiveContext +is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to +feature parity with a HiveContext. +
@@ -81,6 +91,16 @@ JavaSparkContext sc = ...; // An existing JavaSparkContext. JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); {% endhighlight %} +In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict +super set of the functionality provided by the basic SQLContext. Additional features include +the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the +ability to read data from Hive tables. To use a HiveContext, you do not need to have an +existing hive setup, and all of the data sources available to a SQLContext are still available. +HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using HiveContext +is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to +feature parity with a HiveContext. +
@@ -94,36 +114,52 @@ from pyspark.sql import SQLContext sqlContext = SQLContext(sc) {% endhighlight %} -
+In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict +super set of the functionality provided by the basic SQLContext. Additional features include +the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the +ability to read data from Hive tables. To use a HiveContext, you do not need to have an +existing hive setup, and all of the data sources available to a SQLContext are still available. +HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using HiveContext +is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to +feature parity with a HiveContext. -# Data Sources - -
-
-Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. -Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
-
-Spark SQL supports operating on a variety of data sources through the `JavaSchemaRDD` interface. -Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources. -
+The specific variant of SQL that is used to parse queries can also be selected using the +`spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on +a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect +available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the +default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete, + this is recommended for most use cases. + +# Data Sources -
Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. -Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources. -
-
+A SchemaRDD can be operated on as normal RDDs and can also be registered as a temporary table. +Registering a SchemaRDD as a table allows you to run SQL queries over its data. This section +describes the various methods for loading data into a SchemaRDD. ## RDDs +Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. The first +method uses reflection to infer the schema of an RDD that contains specific types of objects. This +reflection based approach leads to more concise code and works well went the schema is known ahead +of time, while you are writing your Spark application. + +The second method for creating SchemaRDDs is through a programmatic interface that allows you to +construct a schema and then apply it to and existing RDD. While this method is more verbose, it allows +you to construct SchemaRDDs when the columns and their types are not known until runtime. + +### Inferring the Schema Using Reflection
-One type of table that is supported by Spark SQL is an RDD of Scala case classes. The case class +The Scala interaface for Spark SQL supports automatically converting an RDD containing case classes +to a SchemaRDD. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be @@ -156,8 +192,9 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
-One type of table that is supported by Spark SQL is an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly). The BeanInfo -defines the schema of the table. Currently, Spark SQL does not support JavaBeans that contain +Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) +into a Schema RDD. The BeanInfo, obtained using reflection, defines the schema of the table. +Currently, Spark SQL does not support JavaBeans that contain nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a class that implements Serializable and has getters and setters for all of its fields. @@ -192,7 +229,7 @@ for the JavaBean. {% highlight java %} // sc is an existing JavaSparkContext. -JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc) +JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); // Load a text file and convert each line to a JavaBean. JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").map( @@ -229,24 +266,24 @@ List teenagerNames = teenagers.map(new Function() {
-One type of table that is supported by Spark SQL is an RDD of dictionaries. The keys of the -dictionary define the columns names of the table, and the types are inferred by looking at the first -row. Any RDD of dictionaries can converted to a SchemaRDD and then registered as a table. Tables -can be used in subsequent SQL statements. +Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes . Rows are constructed by passing a list of +key/value pairs as kwargs to the Row class. The keys of this list define the columns names of the table, +and the types are inferred by looking at the first row. Since we currently only look at the first +row, it is important that there is no missing data in the first row of the RDD. In future version we +plan to more completely infer the schema by looking at more data, similar to the inference that is +performed on JSON files. {% highlight python %} # sc is an existing SparkContext. -from pyspark.sql import SQLContext +from pyspark.sql import SQLContext, Row sqlContext = SQLContext(sc) # Load a text file and convert each line to a dictionary. lines = sc.textFile("examples/src/main/resources/people.txt") parts = lines.map(lambda l: l.split(",")) -people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) +people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) # Infer the schema, and register the SchemaRDD as a table. -# In future versions of PySpark we would like to add support for registering RDDs with other -# datatypes as tables schemaPeople = sqlContext.inferSchema(people) schemaPeople.registerTempTable("people") @@ -263,15 +300,191 @@ for teenName in teenNames.collect():
-**Note that Spark SQL currently uses a very basic SQL parser.** -Users that want a more complete dialect of SQL should look at the HiveQL support provided by -`HiveContext`. +### Programmatically Specifying the Schema + +
+ +
+ +In cases that case classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string or a text dataset will be parsed +and fields will be projected differently for different users), +a `SchemaRDD` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in the step 1. +3. Apply the schema to the RDD of `Row`s via `applySchema` method provided +by `SQLContext`. + +For example: +{% highlight scala %} +// sc is an existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) + +// Create an RDD +val people = sc.textFile("examples/src/main/resources/people.txt") + +// The schema is encoded in a string +val schemaString = "name age" + +// Import Spark SQL data types and Row. +import org.apache.spark.sql._ + +// Generate the schema based on the string of schema +val schema = + StructType( + schemaString.split(" ").map(fieldName => StructField(fieldName, StringType, true))) + +// Convert records of the RDD (people) to Rows. +val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) + +// Apply the schema to the RDD. +val peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema) + +// Register the SchemaRDD as a table. +peopleSchemaRDD.registerTempTable("people") + +// SQL statements can be run by using the sql methods provided by sqlContext. +val results = sqlContext.sql("SELECT name FROM people") + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +results.map(t => "Name: " + t(0)).collect().foreach(println) +{% endhighlight %} + + +
+ +
+ +In cases that JavaBean classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string or a text dataset will be parsed and +fields will be projected differently for different users), +a `SchemaRDD` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in the step 1. +3. Apply the schema to the RDD of `Row`s via `applySchema` method provided +by `JavaSQLContext`. + +For example: +{% highlight java %} +// Import factory methods provided by DataType. +import org.apache.spark.sql.api.java.DataType +// Import StructType and StructField +import org.apache.spark.sql.api.java.StructType +import org.apache.spark.sql.api.java.StructField +// Import Row. +import org.apache.spark.sql.api.java.Row + +// sc is an existing JavaSparkContext. +JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); + +// Load a text file and convert each line to a JavaBean. +JavaRDD people = sc.textFile("examples/src/main/resources/people.txt"); + +// The schema is encoded in a string +String schemaString = "name age"; + +// Generate the schema based on the string of schema +List fields = new ArrayList(); +for (String fieldName: schemaString.split(" ")) { + fields.add(DataType.createStructField(fieldName, DataType.StringType, true)); +} +StructType schema = DataType.createStructType(fields); + +// Convert records of the RDD (people) to Rows. +JavaRDD rowRDD = people.map( + new Function() { + public Row call(String record) throws Exception { + String[] fields = record.split(","); + return Row.create(fields[0], fields[1].trim()); + } + }); + +// Apply the schema to the RDD. +JavaSchemaRDD peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema); + +// Register the SchemaRDD as a table. +peopleSchemaRDD.registerTempTable("people"); + +// SQL can be run over RDDs that have been registered as tables. +JavaSchemaRDD results = sqlContext.sql("SELECT name FROM people"); + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +List names = results.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } +}).collect(); + +{% endhighlight %} + +
+ +
+ +For some cases (for example, the structure of records is encoded in a string or +a text dataset will be parsed and fields will be projected differently for +different users), it is desired to create `SchemaRDD` with a programmatically way. +It can be done with three steps. + +1. Create an RDD of tuples or lists from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +tuples or lists in the RDD created in the step 1. +3. Apply the schema to the RDD via `applySchema` method provided by `SQLContext`. + +For example: +{% highlight python %} +# Import SQLContext and data types +from pyspark.sql import * + +# sc is an existing SparkContext. +sqlContext = SQLContext(sc) + +# Load a text file and convert each line to a tuple. +lines = sc.textFile("examples/src/main/resources/people.txt") +parts = lines.map(lambda l: l.split(",")) +people = parts.map(lambda p: (p[0], p[1].strip())) + +# The schema is encoded in a string. +schemaString = "name age" + +fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()] +schema = StructType(fields) + +# Apply the schema to the RDD. +schemaPeople = sqlContext.applySchema(people, schema) + +# Register the SchemaRDD as a table. +schemaPeople.registerTempTable("people") + +# SQL can be run over SchemaRDDs that have been registered as a table. +results = sqlContext.sql("SELECT name FROM people") + +# The results of SQL queries are RDDs and support all the normal RDD operations. +names = results.map(lambda p: "Name: " + p.name) +for name in names.collect(): + print name +{% endhighlight %} + + +
+ +
## Parquet Files [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. Using the data from the above example: +of the original data. + +### Loading Data Programmatically + +Using the data from the above example:
@@ -349,7 +562,40 @@ for teenName in teenNames.collect():
-
+
+ +### Configuration + +Configuration of parquet can be done using the `setConf` method on SQLContext or by running +`SET key=value` commands using SQL. + +
SourceArtifact
Kafka spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
Kinesis
spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}
Kinesis
spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Apache Software License]
Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}
ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}}
MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}}
+ + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.sql.parquet.binaryAsStringfalse + Some other parquet producing systems, in particular Impala and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the parquet schema. This + flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. +
spark.sql.parquet.cacheMetadatafalse + Turns on caching of parquet schema metadata. Can speed up querying +
spark.sql.parquet.compression.codecsnappy + Sets the compression codec use when writing parquet files. Acceptable values include: + uncompressed, snappy, gzip, lzo. +
## JSON Datasets
@@ -493,13 +739,13 @@ directory. {% highlight scala %} // sc is an existing SparkContext. -val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) +val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc) -hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -hiveContext.sql("FROM src SELECT key, value").collect().foreach(println) +sqlContext.sql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
@@ -513,13 +759,13 @@ expressed in HiveQL. {% highlight java %} // sc is an existing JavaSparkContext. -JavaHiveContext hiveContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); +JavaHiveContext sqlContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); -hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); +sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = hiveContext.sql("FROM src SELECT key, value").collect(); +Row[] results = sqlContext.sql("FROM src SELECT key, value").collect(); {% endhighlight %} @@ -535,44 +781,97 @@ expressed in HiveQL. {% highlight python %} # sc is an existing SparkContext. from pyspark.sql import HiveContext -hiveContext = HiveContext(sc) +sqlContext = HiveContext(sc) -hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = hiveContext.sql("FROM src SELECT key, value").collect() +results = sqlContext.sql("FROM src SELECT key, value").collect() {% endhighlight %} -# Writing Language-Integrated Relational Queries +# Performance Tuning -**Language-Integrated queries are currently only supported in Scala.** - -Spark SQL also supports a domain specific language for writing queries. Once again, -using the data from the above examples: +For some workloads it is possible to improve performance by either caching data in memory, or by +turning on some experimental options. -{% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// Importing the SQL context gives access to all the public SQL functions and implicit conversions. -import sqlContext._ -val people: RDD[Person] = ... // An RDD of case class objects, from the first example. +## Caching Data In Memory -// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' -val teenagers = people.where('age >= 10).where('age <= 19).select('name) -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) -{% endhighlight %} +Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. +Then Spark SQL will scan only required columns and will automatically tune compression to minimize +memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. -The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers -prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are -evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). +Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in +in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to +cache tables. - +Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.sql.inMemoryColumnarStorage.compressedfalse + When set to true Spark SQL will automatically select a compression codec for each column based + on statistics of the data. +
spark.sql.inMemoryColumnarStorage.batchSize1000 + Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization + and compression, but risk OOMs when caching data. +
+ +## Other Configuration + +The following options can also be used to tune the performance of query execution. It is possible +that these options will be deprecated in future release as more optimizations are performed automatically. + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.sql.autoBroadcastJoinThresholdfalse + Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when + performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently + statistics are only supported for Hive Metastore tables where the command + `ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan` has been run. +
spark.sql.codegenfalse + When true, code will be dynamically generated at runtime for expression evaluation in a specific + query. For some queries with complicated expression this option can lead to significant speed-ups. + However, for simple queries this can actually slow down query execution. +
spark.sql.shuffle.partitions200 + Configures the number of partitions to use when shuffling data for joins or aggregations. +
+ +# Other SQL Interfaces + +Spark SQL also supports interfaces for running SQL queries directly without the need to write any +code. ## Running the Thrift JDBC server @@ -602,14 +901,28 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script comes with Hive. +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available +options. + +# Compatibility with Other Systems + +## Migration Guide for Shark Users To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, users can set the `spark.sql.thriftserver.scheduler.pool` variable: SET spark.sql.thriftserver.scheduler.pool=accounting; -### Migration Guide for Shark Users - -#### Reducer number +### Reducer number In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value @@ -625,7 +938,7 @@ You may also put this property in `hive-site.xml` to override the default value. For now, the `mapred.reduce.tasks` property is still recognized, and is converted to `spark.sql.shuffle.partitions` automatically. -#### Caching +### Caching The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no longer automatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to @@ -634,9 +947,9 @@ let user control table caching explicitly: CACHE TABLE logs_last_month; UNCACHE TABLE logs_last_month; -**NOTE:** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", -but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be -cached, you may simply count the table immediately after executing `CACHE TABLE`: +**NOTE:** `CACHE TABLE tbl` is lazy, similar to `.cache` on an RDD. This command only marks `tbl` to ensure that +partitions are cached when calculated but doesn't actually cache it until a query that touches `tbl` is executed. +To force the table to be cached, you may simply count the table immediately after executing `CACHE TABLE`: CACHE TABLE logs_last_month; SELECT COUNT(1) FROM logs_last_month; @@ -647,15 +960,18 @@ Several caching related features are not supported yet: * RDD reloading * In-memory cache write through policy -### Compatibility with Apache Hive +## Compatibility with Apache Hive + +Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. Currently Spark +SQL is based on Hive 0.12.0. #### Deploying in Existing Hive Warehouses -Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive installations. You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables. -#### Supported Hive Features +### Supported Hive Features Spark SQL supports the vast majority of Hive features, such as: @@ -705,13 +1021,14 @@ Spark SQL supports the vast majority of Hive features, such as: * `MAP<>` * `STRUCT<>` -#### Unsupported Hive Functionality +### Unsupported Hive Functionality Below is a list of Hive features that we don't support yet. Most of these features are rarely used in Hive deployments. **Major Hive Features** +* Spark SQL does not currently support inserting to tables using dynamic partitioning. * Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL doesn't support buckets yet. @@ -721,11 +1038,11 @@ in Hive deployments. have the same input format. * Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. -* `UNIONTYPE` +* `UNION` type and `DATE` type * Unique join * Single query multi insert * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment. + the moment and only supports populating the sizeInBytes field of the hive metastore. **Hive Input/Output Formats** @@ -735,7 +1052,7 @@ in Hive deployments. **Hive Optimizations** A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -not necessary due to Spark SQL's in-memory computational model. Others are slotted for future +less important due to Spark SQL's in-memory computational model. Others are slotted for future releases of Spark SQL. * Block level bitmap indexes and virtual columns (used to build indexes) @@ -743,8 +1060,7 @@ releases of Spark SQL. Hive automatically converts the join into a map join. We are adding this auto conversion in the next release. * Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you - need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". We are going to add auto-setting of parallelism in the - next release. + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". * Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still launches tasks to compute the result. * Skew data flag: Spark SQL does not follow the skew data flags in Hive. @@ -753,25 +1069,471 @@ releases of Spark SQL. Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS metadata. Spark SQL does not support that. -## Running the Spark SQL CLI +# Writing Language-Integrated Relational Queries -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. +**Language-Integrated queries are experimental and currently only supported in Scala.** -To start the Spark SQL CLI, run the following in the Spark directory: +Spark SQL also supports a domain specific language for writing queries. Once again, +using the data from the above examples: - ./bin/spark-sql +{% highlight scala %} +// sc is an existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) +// Importing the SQL context gives access to all the public SQL functions and implicit conversions. +import sqlContext._ +val people: RDD[Person] = ... // An RDD of case class objects, from the first example. -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available -options. +// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' +val teenagers = people.where('age >= 10).where('age <= 19).select('name) +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) +{% endhighlight %} -# Cached tables +The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers +prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are +evaluated by the SQL execution engine. A full list of the functions supported can be found in the +[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). -Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. -Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. + + +# Spark SQL DataType Reference + +* Numeric types + - `ByteType`: Represents 1-byte signed integer numbers. + The range of numbers is from `-128` to `127`. + - `ShortType`: Represents 2-byte signed integer numbers. + The range of numbers is from `-32768` to `32767`. + - `IntegerType`: Represents 4-byte signed integer numbers. + The range of numbers is from `-2147483648` to `2147483647`. + - `LongType`: Represents 8-byte signed integer numbers. + The range of numbers is from `-9223372036854775808` to `9223372036854775807`. + - `FloatType`: Represents 4-byte single-precision floating point numbers. + - `DoubleType`: Represents 8-byte double-precision floating point numbers. + - `DecimalType`: +* String type + - `StringType`: Represents character string values. +* Binary type + - `BinaryType`: Represents byte sequence values. +* Boolean type + - `BooleanType`: Represents boolean values. +* Datetime type + - `TimestampType`: Represents values comprising values of fields year, month, day, + hour, minute, and second. +* Complex types + - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of + elements with the type of `elementType`. `containsNull` is used to indicate if + elements in a `ArrayType` value can have `null` values. + - `MapType(keyType, valueType, valueContainsNull)`: + Represents values comprising a set of key-value pairs. The data type of keys are + described by `keyType` and the data type of values are described by `valueType`. + For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` + is used to indicate if values of a `MapType` value can have `null` values. + - `StructType(fields)`: Represents values with the structure described by + a sequence of `StructField`s (`fields`). + * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. + The name of a field is indicated by `name`. The data type of a field is indicated + by `dataType`. `nullable` is used to indicate if values of this fields can have + `null` values. + +
+
+ +All data types of Spark SQL are located in the package `org.apache.spark.sql`. +You can access them by doing +{% highlight scala %} +import org.apache.spark.sql._ +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Data typeValue type in ScalaAPI to access or create a data type
ByteType Byte + ByteType +
ShortType Short + ShortType +
IntegerType Int + IntegerType +
LongType Long + LongType +
FloatType Float + FloatType +
DoubleType Double + DoubleType +
DecimalType scala.math.sql.BigDecimal + DecimalType +
StringType String + StringType +
BinaryType Array[Byte] + BinaryType +
BooleanType Boolean + BooleanType +
TimestampType java.sql.Timestamp + TimestampType +
ArrayType scala.collection.Seq + ArrayType(elementType, [containsNull])
+ Note: The default value of containsNull is false. +
MapType scala.collection.Map + MapType(keyType, valueType, [valueContainsNull])
+ Note: The default value of valueContainsNull is true. +
StructType org.apache.spark.sql.Row + StructType(fields)
+ Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
StructField The value type in Scala of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, nullable) +
+ +
+ +
+ +All data types of Spark SQL are located in the package of +`org.apache.spark.sql.api.java`. To access or create a data type, +please use factory methods provided in +`org.apache.spark.sql.api.java.DataType`. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Data typeValue type in JavaAPI to access or create a data type
ByteType byte or Byte + DataType.ByteType +
ShortType short or Short + DataType.ShortType +
IntegerType int or Integer + DataType.IntegerType +
LongType long or Long + DataType.LongType +
FloatType float or Float + DataType.FloatType +
DoubleType double or Double + DataType.DoubleType +
DecimalType java.math.BigDecimal + DataType.DecimalType +
StringType String + DataType.StringType +
BinaryType byte[] + DataType.BinaryType +
BooleanType boolean or Boolean + DataType.BooleanType +
TimestampType java.sql.Timestamp + DataType.TimestampType +
ArrayType java.util.List + DataType.createArrayType(elementType)
+ Note: The value of containsNull will be false
+ DataType.createArrayType(elementType, containsNull). +
MapType java.util.Map + DataType.createMapType(keyType, valueType)
+ Note: The value of valueContainsNull will be true.
+ DataType.createMapType(keyType, valueType, valueContainsNull)
+
StructType org.apache.spark.sql.api.java + DataType.createStructType(fields)
+ Note: fields is a List or an array of StructFields. + Also, two fields with the same name are not allowed. +
StructField The value type in Java of the data type of this field + (For example, int for a StructField with the data type IntegerType) + DataType.createStructField(name, dataType, nullable) +
+ +
+ +
+ +All data types of Spark SQL are located in the package of `pyspark.sql`. +You can access them by doing +{% highlight python %} +from pyspark.sql import * +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Data typeValue type in PythonAPI to access or create a data type
ByteType + int or long
+ Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
+ ByteType() +
ShortType + int or long
+ Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
+ ShortType() +
IntegerType int or long + IntegerType() +
LongType + long
+ Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
+ LongType() +
FloatType + float
+ Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
+ FloatType() +
DoubleType float + DoubleType() +
DecimalType decimal.Decimal + DecimalType() +
StringType string + StringType() +
BinaryType bytearray + BinaryType() +
BooleanType bool + BooleanType() +
TimestampType datetime.datetime + TimestampType() +
ArrayType list, tuple, or array + ArrayType(elementType, [containsNull])
+ Note: The default value of containsNull is False. +
MapType dict + MapType(keyType, valueType, [valueContainsNull])
+ Note: The default value of valueContainsNull is True. +
StructType list or tuple + StructType(fields)
+ Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
StructField The value type in Python of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, nullable) +
+ +
+ +
-Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in -in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to -cache tables. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 4137ac7663739..f6f4cf3b80d41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -53,7 +53,7 @@ private[spark] object SQLConf { * * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ -trait SQLConf { +private[sql] trait SQLConf { import SQLConf._ /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 0ea1105f082a4..595b4aa36eae3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -30,7 +30,7 @@ import scala.reflect.runtime.universe.{TypeTag, typeTag} /** * Functions for registering scala lambda functions as UDFs in a SQLContext. */ -protected[sql] trait UDFRegistration { +private[sql] trait UDFRegistration { self: SQLContext => private[spark] def registerPython( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index dc668e7dc934c..6eab2f23c18e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{LeafNode, SparkPlan} -object InMemoryRelation { +private[sql] object InMemoryRelation { def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = new InMemoryRelation(child.output, useCompression, batchSize, child)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 9fd6aed402838..2fc7e1cf23ab7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -382,7 +382,7 @@ private[parquet] class CatalystPrimitiveConverter( parent.updateLong(fieldIndex, value) } -object CatalystArrayConverter { +private[parquet] object CatalystArrayConverter { val INITIAL_ARRAY_SIZE = 20 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index fe28e0d7269e0..7c83f1cad7d71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkSqlSerializer -object ParquetFilters { +private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" // set this to false if pushdown should be disabled val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.hints.parquetFilterPushdown" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index f12b5a69a09f7..bd3f68d92d8c7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -39,7 +39,9 @@ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. */ -class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { +private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) + extends OperationManager with Logging { + val handleToOperation = ReflectionUtils .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") From e2614038e78f4693fafedeee15b6fdf0ea1be473 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Sep 2014 18:42:24 -0700 Subject: [PATCH 13/19] [SPARK-3408] Fixed Limit operator so it works with sort-based shuffle. Author: Reynold Xin Closes #2281 from rxin/sql-limit-sort and squashes the following commits: 1ef7780 [Reynold Xin] [SPARK-3408] Fixed Limit operator so it works with sort-based shuffle. --- .../spark/sql/execution/basicOperators.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 4abda21ffec96..47bff0c730b8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.{HashPartitioner, SparkConf} import org.apache.spark.rdd.{RDD, ShuffledRDD} -import org.apache.spark.sql.SQLContext +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ @@ -96,6 +96,9 @@ case class Limit(limit: Int, child: SparkPlan) // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again + /** We must copy rows when sort based shuffle is on */ + private def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] + override def output = child.output /** @@ -143,9 +146,15 @@ case class Limit(limit: Int, child: SparkPlan) } override def execute() = { - val rdd = child.execute().mapPartitions { iter => - val mutablePair = new MutablePair[Boolean, Row]() - iter.take(limit).map(row => mutablePair.update(false, row)) + val rdd: RDD[_ <: Product2[Boolean, Row]] = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => + iter.take(limit).map(row => (false, row.copy())) + } + } else { + child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Boolean, Row]() + iter.take(limit).map(row => mutablePair.update(false, row)) + } } val part = new HashPartitioner(1) val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) From ecfa76cdfe846c75e1b7ebc556167e46963289c5 Mon Sep 17 00:00:00 2001 From: Ward Viaene Date: Sun, 7 Sep 2014 18:54:36 -0700 Subject: [PATCH 14/19] [SPARK-3415] [PySpark] removes SerializingAdapter code This code removes the SerializingAdapter code that was copied from PiCloud Author: Ward Viaene Closes #2287 from wardviaene/feature/pythonsys and squashes the following commits: 5f0d426 [Ward Viaene] SPARK-3415: modified test class to do dump and load 5f5d559 [Ward Viaene] SPARK-3415: modified test class name and call cloudpickle.dumps instead using StringIO afc4a9a [Ward Viaene] SPARK-3415: added newlines to pass lint aaf10b7 [Ward Viaene] SPARK-3415: removed references to SerializingAdapter and rewrote test 65ffeff [Ward Viaene] removed duplicate test a958866 [Ward Viaene] SPARK-3415: test script e263bf5 [Ward Viaene] SPARK-3415: removes legacy SerializingAdapter code --- python/pyspark/cloudpickle.py | 6 +----- python/pyspark/tests.py | 11 +++++++++++ 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 68062483dedaa..80e51d1a583a0 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -657,7 +657,6 @@ def save_partial(self, obj): def save_file(self, obj): """Save a file""" import StringIO as pystringIO #we can't use cStringIO as it lacks the name attribute - from ..transport.adapter import SerializingAdapter if not hasattr(obj, 'name') or not hasattr(obj, 'mode'): raise pickle.PicklingError("Cannot pickle files that do not map to an actual file") @@ -691,13 +690,10 @@ def save_file(self, obj): tmpfile.close() if tst != '': raise pickle.PicklingError("Cannot pickle file %s as it does not appear to map to a physical, real file" % name) - elif fsize > SerializingAdapter.max_transmit_data: - raise pickle.PicklingError("Cannot pickle file %s as it exceeds cloudconf.py's max_transmit_data of %d" % - (name,SerializingAdapter.max_transmit_data)) else: try: tmpfile = file(name) - contents = tmpfile.read(SerializingAdapter.max_transmit_data) + contents = tmpfile.read() tmpfile.close() except IOError: raise pickle.PicklingError("Cannot pickle file %s as it cannot be read" % name) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 2ade15b35ab4e..9fbeb36f4f1dd 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -169,6 +169,17 @@ def test_namedtuple(self): self.assertEquals(p1, p2) +# Regression test for SPARK-3415 +class CloudPickleTest(unittest.TestCase): + def test_pickling_file_handles(self): + from pyspark.cloudpickle import dumps + from StringIO import StringIO + from pickle import load + out1 = sys.stderr + out2 = load(StringIO(dumps(out1))) + self.assertEquals(out1, out2) + + class PySparkTestCase(unittest.TestCase): def setUp(self): From 9d69a782bd2fc45193f269d8d8434795ea1580a4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 7 Sep 2014 20:38:32 -0700 Subject: [PATCH 15/19] Fixed typos in make-distribution.sh `hadoop.version` and `yarn.version` are properties rather then profiles, should use `-D` instead of `-P`. /cc pwendell Author: Cheng Lian Closes #2121 from liancheng/fix-make-dist and squashes the following commits: 4c49158 [Cheng Lian] Also mentions Hadoop version related Maven profiles ed5b42a [Cheng Lian] Fixed typos in make-distribution.sh --- make-distribution.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index f030d3f430581..14aed4a4b655b 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -50,7 +50,8 @@ while (( "$#" )); do case $1 in --hadoop) echo "Error: '--hadoop' is no longer supported:" - echo "Error: use Maven options -Phadoop.version and -Pyarn.version" + echo "Error: use Maven profiles and options -Dhadoop.version and -Dyarn.version instead." + echo "Error: Related profiles include hadoop-0.23, hdaoop-2.2, hadoop-2.3 and hadoop-2.4." exit_with_usage ;; --with-yarn) From 4ba2673569f8c6da7f7348977f52f98f40dfbfec Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 7 Sep 2014 20:39:53 -0700 Subject: [PATCH 16/19] [HOTFIX] Fix broken Mima tests on the master branch By merging #2268, which bumped the Spark version to 1.2.0-SNAPSHOT, I inadvertently broke the Mima binary compatibility tests. The issue is that we were comparing 1.2.0-SNAPSHOT against Spark 1.0.0 without using any Mima excludes. The right long-term fix for this is probably to publish nightly snapshots on Maven central and change the master branch to test binary compatibility against the current release candidate branch's snapshots until that release is finalized. As a short-term fix until 1.1.0 is published on Maven central, I've configured the build to test the master branch for binary compatibility against the 1.1.0-RC4 jars. I'll loop back and remove the Apache staging repo as soon as 1.1.0 final is available. Author: Josh Rosen Closes #2315 from JoshRosen/mima-fix and squashes the following commits: 776bc2c [Josh Rosen] Add two excludes to workaround Mima annotation issues. ec90e21 [Josh Rosen] Add deploy and graphx to 1.2 MiMa excludes. 57569be [Josh Rosen] Fix MiMa tests in master branch; test against 1.1.0 RC. --- pom.xml | 12 ++++++++++++ project/MimaBuild.scala | 2 +- project/MimaExcludes.scala | 12 ++++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1efa9045208fd..d05190512f742 100644 --- a/pom.xml +++ b/pom.xml @@ -221,6 +221,18 @@ false + + + spark-staging-1030 + Spark 1.1.0 Staging (1030) + https://repository.apache.org/content/repositories/orgapachespark-1030/ + + true + + + false + + diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 034ba6a7bf50f..0f5d71afcf616 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -85,7 +85,7 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" - val previousSparkVersion = "1.0.0" + val previousSparkVersion = "1.1.0" val fullId = "spark-" + projectRef.project + "_2.10" mimaDefaultSettings ++ Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 855d5cc8cf3fd..46b78bd5c7061 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -33,6 +33,18 @@ import com.typesafe.tools.mima.core._ object MimaExcludes { def excludes(version: String) = version match { + case v if v.startsWith("1.2") => + Seq( + MimaBuild.excludeSparkPackage("deploy"), + MimaBuild.excludeSparkPackage("graphx") + ) ++ + // This is @DeveloperAPI, but Mima still gives false-positives: + MimaBuild.excludeSparkClass("scheduler.SparkListenerApplicationStart") ++ + Seq( + // This is @Experimental, but Mima still gives false-positives: + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.foreachAsync") + ) case v if v.startsWith("1.1") => Seq( MimaBuild.excludeSparkPackage("deploy"), From f25bbbdb3ac5620850c7d09d6a63af888411ecf1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Sep 2014 20:42:07 -0700 Subject: [PATCH 17/19] [SPARK-3280] Made sort-based shuffle the default implementation Sort-based shuffle has lower memory usage and seems to outperform hash-based in almost all of our testing. Author: Reynold Xin Closes #2178 from rxin/sort-shuffle and squashes the following commits: 713d341 [Reynold Xin] Fixed test failures by setting spark.shuffle.compress to the same value as spark.shuffle.spill.compress. 85165e6 [Reynold Xin] Fixed a comment typo. aa0d372 [Reynold Xin] [SPARK-3280] Made sort-based shuffle the default implementation --- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/HashShuffleSuite.scala | 33 +++++++++++++++++++ .../scala/org/apache/spark/ShuffleSuite.scala | 2 +- .../org/apache/spark/SortShuffleSuite.scala | 3 +- .../ExternalAppendOnlyMapSuite.scala | 1 + docs/configuration.md | 9 +++-- 6 files changed, 41 insertions(+), 9 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/HashShuffleSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2973d002cc428..20a7444cfc5ee 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -217,7 +217,7 @@ object SparkEnv extends Logging { val shortShuffleMgrNames = Map( "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") - val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") + val shuffleMgrName = conf.get("spark.shuffle.manager", "sort") val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) diff --git a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala new file mode 100644 index 0000000000000..2acc02a54fa3d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala @@ -0,0 +1,33 @@ +/* + * 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. + */ + +package org.apache.spark + +import org.scalatest.BeforeAndAfterAll + +class HashShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { + + // This test suite should run all tests in ShuffleSuite with hash-based shuffle. + + override def beforeAll() { + System.setProperty("spark.shuffle.manager", "hash") + } + + override def afterAll() { + System.clearProperty("spark.shuffle.manager") + } +} diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index b13ddf96bc77c..15aa4d83800fa 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair -class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { +abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val conf = new SparkConf(loadDefaults = false) diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index 5c02c00586ef4..639e56c488db4 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -24,8 +24,7 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with sort-based shuffle. override def beforeAll() { - System.setProperty("spark.shuffle.manager", - "org.apache.spark.shuffle.sort.SortShuffleManager") + System.setProperty("spark.shuffle.manager", "sort") } override def afterAll() { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index ac3931e3d0a73..511d76c9144cc 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -42,6 +42,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { conf.set("spark.serializer.objectStreamReset", "1") conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") conf.set("spark.shuffle.spill.compress", codec.isDefined.toString) + conf.set("spark.shuffle.compress", codec.isDefined.toString) codec.foreach { c => conf.set("spark.io.compression.codec", c) } // Ensure that we actually have multiple batches per spill file conf.set("spark.shuffle.spill.batchSize", "10") diff --git a/docs/configuration.md b/docs/configuration.md index 65a422caabb7e..36178efb97103 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -293,12 +293,11 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.manager - HASH + sort - Implementation to use for shuffling data. A hash-based shuffle manager is the default, but - starting in Spark 1.1 there is an experimental sort-based shuffle manager that is more - memory-efficient in environments with small executors, such as YARN. To use that, change - this value to SORT. + Implementation to use for shuffling data. There are two implementations available: + sort and hash. Sort-based shuffle is more memory-efficient and is + the default option starting in 1.2. From eddfeddac19870fc265ef406d87e1c3db9b54249 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Sep 2014 20:56:04 -0700 Subject: [PATCH 18/19] [SPARK-938][doc] Add OpenStack Swift support See compiled doc at http://people.apache.org/~rxin/tmp/openstack-swift/_site/storage-openstack-swift.html This is based on #1010. Closes #1010. Author: Reynold Xin Author: Gil Vernik Closes #2298 from rxin/openstack-swift and squashes the following commits: ff4e394 [Reynold Xin] Two minor comments from Patrick. 279f6de [Reynold Xin] core-sites -> core-site dfb8fea [Reynold Xin] Updated based on Gil's suggestion. 846f5cb [Reynold Xin] Added a link from overview page. 0447c9f [Reynold Xin] Removed sample code. e9c3761 [Reynold Xin] Merge pull request #1010 from gilv/master 9233fef [Gil Vernik] Fixed typos 6994827 [Gil Vernik] Merge pull request #1 from rxin/openstack ac0679e [Reynold Xin] Fixed an unclosed tr. 47ce99d [Reynold Xin] Merge branch 'master' into openstack cca7192 [Gil Vernik] Removed white spases from pom.xml 99f095d [Reynold Xin] Pending openstack changes. eb22295 [Reynold Xin] Merge pull request #1010 from gilv/master 39a9737 [Gil Vernik] Spark integration with Openstack Swift c977658 [Gil Vernik] Merge branch 'master' of https://github.com/gilv/spark 2aba763 [Gil Vernik] Fix to docs/openstack-integration.md 9b625b5 [Gil Vernik] Merge branch 'master' of https://github.com/gilv/spark eff538d [Gil Vernik] SPARK-938 - Openstack Swift object storage support ce483d7 [Gil Vernik] SPARK-938 - Openstack Swift object storage support b6c37ef [Gil Vernik] Openstack Swift support --- docs/index.md | 2 + docs/storage-openstack-swift.md | 152 ++++++++++++++++++++++++++++++++ 2 files changed, 154 insertions(+) create mode 100644 docs/storage-openstack-swift.md diff --git a/docs/index.md b/docs/index.md index 4ac0982ae54f1..7fe6b43d32af7 100644 --- a/docs/index.md +++ b/docs/index.md @@ -103,6 +103,8 @@ options for deployment: * [Security](security.html): Spark security support * [Hardware Provisioning](hardware-provisioning.html): recommendations for cluster hardware * [3rd Party Hadoop Distributions](hadoop-third-party-distributions.html): using common Hadoop distributions +* Integration with other storage systems: + * [OpenStack Swift](storage-openstack-swift.html) * [Building Spark with Maven](building-with-maven.html): build Spark using the Maven system * [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md new file mode 100644 index 0000000000000..c39ef1ce59e1c --- /dev/null +++ b/docs/storage-openstack-swift.md @@ -0,0 +1,152 @@ +--- +layout: global +title: Accessing OpenStack Swift from Spark +--- + +Spark's support for Hadoop InputFormat allows it to process data in OpenStack Swift using the +same URI formats as in Hadoop. You can specify a path in Swift as input through a +URI of the form swift://container.PROVIDER/path. You will also need to set your +Swift security credentials, through core-site.xml or via +SparkContext.hadoopConfiguration. +Current Swift driver requires Swift to use Keystone authentication method. + +# Configuring Swift for Better Data Locality + +Although not mandatory, it is recommended to configure the proxy server of Swift with +list_endpoints to have better data locality. More information is +[available here](https://github.com/openstack/swift/blob/master/swift/common/middleware/list_endpoints.py). + + +# Dependencies + +The Spark application should include hadoop-openstack dependency. +For example, for Maven support, add the following to the pom.xml file: + +{% highlight xml %} + + ... + + org.apache.hadoop + hadoop-openstack + 2.3.0 + + ... + +{% endhighlight %} + + +# Configuration Parameters + +Create core-site.xml and place it inside Spark's conf directory. +There are two main categories of parameters that should to be configured: declaration of the +Swift driver and the parameters that are required by Keystone. + +Configuration of Hadoop to use Swift File system achieved via + + + + + + + +
Property NameValue
fs.swift.implorg.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem
+ +Additional parameters required by Keystone (v2.0) and should be provided to the Swift driver. Those +parameters will be used to perform authentication in Keystone to access Swift. The following table +contains a list of Keystone mandatory parameters. PROVIDER can be any name. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameMeaningRequired
fs.swift.service.PROVIDER.auth.urlKeystone Authentication URLMandatory
fs.swift.service.PROVIDER.auth.endpoint.prefixKeystone endpoints prefixOptional
fs.swift.service.PROVIDER.tenantTenantMandatory
fs.swift.service.PROVIDER.usernameUsernameMandatory
fs.swift.service.PROVIDER.passwordPasswordMandatory
fs.swift.service.PROVIDER.http.portHTTP portMandatory
fs.swift.service.PROVIDER.regionKeystone regionMandatory
fs.swift.service.PROVIDER.publicIndicates if all URLs are publicMandatory
+ +For example, assume PROVIDER=SparkTest and Keystone contains user tester with password testing +defined for tenant test. Then core-site.xml should include: + +{% highlight xml %} + + + fs.swift.impl + org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem + + + fs.swift.service.SparkTest.auth.url + http://127.0.0.1:5000/v2.0/tokens + + + fs.swift.service.SparkTest.auth.endpoint.prefix + endpoints + + fs.swift.service.SparkTest.http.port + 8080 + + + fs.swift.service.SparkTest.region + RegionOne + + + fs.swift.service.SparkTest.public + true + + + fs.swift.service.SparkTest.tenant + test + + + fs.swift.service.SparkTest.username + tester + + + fs.swift.service.SparkTest.password + testing + + +{% endhighlight %} + +Notice that +fs.swift.service.PROVIDER.tenant, +fs.swift.service.PROVIDER.username, +fs.swift.service.PROVIDER.password contains sensitive information and keeping them in +core-site.xml is not always a good approach. +We suggest to keep those parameters in core-site.xml for testing purposes when running Spark +via spark-shell. +For job submissions they should be provided via sparkContext.hadoopConfiguration. From 0d1cc4ae42e1f73538dd8b9b1880ca9e5b124108 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 8 Sep 2014 14:32:53 +0530 Subject: [PATCH 19/19] [HOTFIX] A left over version change. It should make mima happy. Author: Prashant Sharma Closes #2317 from ScrapCodes/hotfix and squashes the following commits: b6472d4 [Prashant Sharma] [HOTFIX] for hotfixes, a left over version change. --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a26c2c90cb321..45f6d2973ea90 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -184,7 +184,7 @@ object OldDeps { def versionArtifact(id: String): Option[sbt.ModuleID] = { val fullId = id + "_2.10" - Some("org.apache.spark" % fullId % "1.0.0") + Some("org.apache.spark" % fullId % "1.1.0") } def oldDepsSettings() = Defaults.defaultSettings ++ Seq(