From ba5bcaddecd54811d45c5fc79a013b3857d4c633 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Fri, 5 Sep 2014 18:52:05 -0700 Subject: [PATCH 01/51] SPARK-3211 .take() is OOM-prone with empty partitions Instead of jumping straight from 1 partition to all partitions, do exponential growth and double the number of partitions to attempt each time instead. Fix proposed by Paul Nepywoda Author: Andrew Ash Closes #2117 from ash211/SPARK-3211 and squashes the following commits: 8b2299a [Andrew Ash] Quadruple instead of double for a minor speedup e5f7e4d [Andrew Ash] Update comment to better reflect what we're doing 09a27f7 [Andrew Ash] Update PySpark to be less OOM-prone as well 3a156b8 [Andrew Ash] SPARK-3211 .take() is OOM-prone with empty partitions --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 7 +++---- python/pyspark/rdd.py | 8 ++++---- 2 files changed, 7 insertions(+), 8 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 af9e31ba7b720..1cf55e86f6c81 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1064,11 +1064,10 @@ abstract class RDD[T: ClassTag]( // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 1 if (partsScanned > 0) { - // If we didn't find any rows after the first iteration, just try all partitions next. - // Otherwise, interpolate the number of partitions we need to try, but overestimate it - // by 50%. + // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise, + // interpolate the number of partitions we need to try, but overestimate it by 50%. if (buf.size == 0) { - numPartsToTry = totalParts - 1 + numPartsToTry = partsScanned * 4 } else { numPartsToTry = (1.5 * num * partsScanned / buf.size).toInt } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index dff6fc26fcb18..04f13523b431d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1089,11 +1089,11 @@ def take(self, num): # we actually cap it at totalParts in runJob. numPartsToTry = 1 if partsScanned > 0: - # If we didn't find any rows after the first iteration, just - # try all partitions next. Otherwise, interpolate the number - # of partitions we need to try, but overestimate it by 50%. + # If we didn't find any rows after the previous iteration, + # quadruple and retry. Otherwise, interpolate the number of + # partitions we need to try, but overestimate it by 50%. if len(items) == 0: - numPartsToTry = totalParts - 1 + numPartsToTry = partsScanned * 4 else: numPartsToTry = int(1.5 * num * partsScanned / len(items)) From 19f61c165932059e7ce156da2c71429fa8dc27f0 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 5 Sep 2014 21:46:45 -0700 Subject: [PATCH 02/51] [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 03/51] [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 04/51] [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 05/51] [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 06/51] [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 07/51] 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 08/51] [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 09/51] [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 10/51] [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 11/51] [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 12/51] [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 13/51] [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 14/51] [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 15/51] [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 16/51] 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 17/51] [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 18/51] [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 19/51] [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 20/51] [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( From 711356b422c66e2a80377a9f43fce97282460520 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 8 Sep 2014 09:47:13 -0700 Subject: [PATCH 21/51] [SPARK-3086] [SPARK-3043] [SPARK-3156] [mllib] DecisionTree aggregation improvements MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Summary: 1. Variable numBins for each feature [SPARK-3043] 2. Reduced data reshaping in aggregation [SPARK-3043] 3. Choose ordering for ordered categorical features adaptively [SPARK-3156] 4. Changed nodes to use 1-indexing [SPARK-3086] 5. Small clean-ups Note: This PR looks bigger than it is since I moved several functions from inside findBestSplitsPerGroup to outside of it (to make it clear what was being serialized in the aggregation). Speedups: This update helps most when many features use few bins but a few features use many bins. Some example results on speedups with 2M examples, 3.5K features (15-worker EC2 cluster): * Example where old code was reasonably efficient (1/2 continuous, 1/4 binary, 1/4 20-category): 164.813 --> 116.491 sec * Example where old code wasted many bins (1/10 continuous, 81/100 binary, 9/100 20-category): 128.701 --> 39.334 sec Details: (1) Variable numBins for each feature [SPARK-3043] DecisionTreeMetadata now computes a variable numBins for each feature. It also tracks numSplits. (2) Reduced data reshaping in aggregation [SPARK-3043] Added DTStatsAggregator, a wrapper around the aggregate statistics array for easy but efficient indexing. * Added ImpurityAggregator and ImpurityCalculator classes, to make DecisionTree code more oblivious to the type of impurity. * Design note: I originally tried creating Impurity classes which stored data and storing the aggregates in an Array[Array[Array[Impurity]]]. However, this led to significant slowdowns, perhaps because of overhead in creating so many objects. The aggregate statistics are never reshaped, and cumulative sums are computed in-place. Updated the layout of aggregation functions. The update simplifies things by (1) dividing features into ordered/unordered (instead of ordered/unordered/continuous) and (2) making use of the DTStatsAggregator for indexing. For this update, the following functions were refactored: * updateBinForOrderedFeature * updateBinForUnorderedFeature * binaryOrNotCategoricalBinSeqOp * multiclassWithCategoricalBinSeqOp * regressionBinSeqOp The above 5 functions were replaced with: * orderedBinSeqOp * someUnorderedBinSeqOp Other changes: * calculateGainForSplit now treats all feature types the same way. * Eliminated extractLeftRightNodeAggregates. (3) Choose ordering for ordered categorical features adaptively [SPARK-3156] Updated binsToBestSplit(): * This now computes cumulative sums of stats for ordered features. * For ordered categorical features, it chooses an ordering for categories. (This uses to be done by findSplitsBins.) * Uses iterators to shorten code and avoid building an Array[Array[InformationGainStats]]. Side effects: * In findSplitsBins: A sample of the data is only taken for data with continuous features. It is not needed for data with only categorical features. * In findSplitsBins: splits and bins are no longer pre-computed for ordered categorical features since they are not needed. * TreePoint binning is simpler for categorical features. (4) Changed nodes to use 1-indexing [SPARK-3086] Nodes used to be indexed from 0. Now they are indexed from 1. Node indexing functions are now collected in object Node (Node.scala). (5) Small clean-ups Eliminated functions extractNodeInfo() and extractInfoForLowerLevels() to reduce duplicate code. Eliminated InvalidBinIndex since it is no longer used. CC: mengxr manishamde Please let me know if you have thoughts on this—thanks! Author: Joseph K. Bradley Closes #2125 from jkbradley/dt-opt3alt and squashes the following commits: 42c192a [Joseph K. Bradley] Merge branch 'rfs' into dt-opt3alt d3cc46b [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 00e4404 [Joseph K. Bradley] optimization for TreePoint construction (pre-computing featureArity and isUnordered as arrays) 425716c [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into rfs a2acea5 [Joseph K. Bradley] Small optimizations based on profiling aa4e4df [Joseph K. Bradley] Updated DTStatsAggregator with bug fix (nodeString should not be multiplied by statsSize) 4651154 [Joseph K. Bradley] Changed numBins semantics for unordered features. * Before: numBins = numSplits = (1 << k - 1) - 1 * Now: numBins = 2 * numSplits = 2 * [(1 << k - 1) - 1] * This also involved changing the semantics of: ** DecisionTreeMetadata.numUnorderedBins() 1e3b1c7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 1485fcc [Joseph K. Bradley] Made some DecisionTree methods private. 92f934f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt e676da1 [Joseph K. Bradley] Updated documentation for DecisionTree 37ca845 [Joseph K. Bradley] Fixed problem with how DecisionTree handles ordered categorical features. 105f8ab [Joseph K. Bradley] Removed commented-out getEmptyBinAggregates from DecisionTree 062c31d [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 6d32ccd [Joseph K. Bradley] In DecisionTree.binsToBestSplit, changed loops to iterators to shorten code. 807cd00 [Joseph K. Bradley] Finished DTStatsAggregator, a wrapper around the aggregate statistics for easy but hopefully efficient indexing. Modified old ImpurityAggregator classes and renamed them ImpurityCalculator; added ImpurityAggregator classes which work with DTStatsAggregator but do not store data. Unit tests all succeed. f2166fd [Joseph K. Bradley] still working on DTStatsAggregator 92f7118 [Joseph K. Bradley] Added partly written DTStatsAggregator fd8df30 [Joseph K. Bradley] Moved some aggregation helpers outside of findBestSplitsPerGroup d7c53ee [Joseph K. Bradley] Added more doc for ImpurityAggregator a40f8f1 [Joseph K. Bradley] Changed nodes to be indexed from 1. Tests work. 95cad7c [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 5f94342 [Joseph K. Bradley] Added treeAggregate since not yet merged from master. Moved node indexing functions to Node. 61c4509 [Joseph K. Bradley] Fixed bugs from merge: missing DT timer call, and numBins setting. Cleaned up DT Suite some. 3ba7166 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 b314659 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 9c83363 [Joseph K. Bradley] partial merge but not done yet 45f7ea7 [Joseph K. Bradley] partial merge, not yet done 5fce635 [Joseph K. Bradley] Merge branch 'dt-opt2' into dt-opt3 26d10dd [Joseph K. Bradley] Removed tree/model/Filter.scala since no longer used. Removed debugging println calls in DecisionTree.scala. 356daba [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 dd4d3aa [Joseph K. Bradley] Mid-process in bug fix: bug for binary classification with categorical features * Bug: Categorical features were all treated as ordered for binary classification. This is possible but would require the bin ordering to be determined on-the-fly after the aggregation. Currently, the ordering is determined a priori and fixed for all splits. * (Temp) Fix: Treat low-arity categorical features as unordered for binary classification. * Related change: I removed most tests for isMulticlass in the code. I instead test metadata for whether there are unordered features. * Status: The bug may be fixed, but more testing needs to be done. 438a660 [Joseph K. Bradley] removed subsampling for mnist8m from DT 86e217f [Joseph K. Bradley] added cache to DT input e3c84cc [Joseph K. Bradley] Added stuff fro mnist8m to D T Runner 51ef781 [Joseph K. Bradley] Fixed bug introduced by last commit: Variance impurity calculation was incorrect since counts were swapped accidentally fd65372 [Joseph K. Bradley] Major changes: * Created ImpurityAggregator classes, rather than old aggregates. * Feature split/bin semantics are based on ordered vs. unordered ** E.g.: numSplits = numBins for all unordered features, and numSplits = numBins - 1 for all ordered features. * numBins can differ for each feature c1565a5 [Joseph K. Bradley] Small DecisionTree updates: * Simplification: Updated calculateGainForSplit to take aggregates for a single (feature, split) pair. * Internal doc: findAggForOrderedFeatureClassification b914f3b [Joseph K. Bradley] DecisionTree optimization: eliminated filters + small changes b2ed1f3 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals --- .../spark/mllib/tree/DecisionTree.scala | 1341 ++++++----------- .../mllib/tree/impl/DTStatsAggregator.scala | 213 +++ .../tree/impl/DecisionTreeMetadata.scala | 73 +- .../spark/mllib/tree/impl/TreePoint.scala | 93 +- .../spark/mllib/tree/impurity/Entropy.scala | 84 ++ .../spark/mllib/tree/impurity/Gini.scala | 84 ++ .../spark/mllib/tree/impurity/Impurity.scala | 127 ++ .../spark/mllib/tree/impurity/Variance.scala | 72 + .../apache/spark/mllib/tree/model/Bin.scala | 7 +- .../apache/spark/mllib/tree/model/Node.scala | 85 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 391 ++--- 11 files changed, 1322 insertions(+), 1248 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 5cdd258f6c20b..dd766c12d28a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -28,8 +28,9 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.impl._ import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} +import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -65,36 +66,41 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val retaggedInput = input.retag(classOf[LabeledPoint]) val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) logDebug("algo = " + strategy.algo) + logDebug("maxBins = " + metadata.maxBins) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. timer.start("findSplitsBins") val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) - val numBins = bins(0).length timer.stop("findSplitsBins") - logDebug("numBins = " + numBins) + logDebug("numBins: feature: number of bins") + logDebug(Range(0, metadata.numFeatures).map { featureIndex => + s"\t$featureIndex\t${metadata.numBins(featureIndex)}" + }.mkString("\n")) // Bin feature values (TreePoint representation). // Cache input RDD for speedup during multiple passes. val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) .persist(StorageLevel.MEMORY_AND_DISK) - val numFeatures = metadata.numFeatures // depth of the decision tree val maxDepth = strategy.maxDepth - // the max number of nodes possible given the depth of the tree - val maxNumNodes = (2 << maxDepth) - 1 + require(maxDepth <= 30, + s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") + // Number of nodes to allocate: max number of nodes possible given the depth of the tree, plus 1 + val maxNumNodesPlus1 = Node.startIndexInLevel(maxDepth + 1) // Initialize an array to hold parent impurity calculations for each node. - val parentImpurities = new Array[Double](maxNumNodes) + val parentImpurities = new Array[Double](maxNumNodesPlus1) // dummy value for top node (updated during first split calculation) - val nodes = new Array[Node](maxNumNodes) + val nodes = new Array[Node](maxNumNodesPlus1) // Calculate level for single group construction // Max memory usage for aggregates val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val numElementsPerNode = DecisionTree.getElementsPerNode(metadata, numBins) + // TODO: Calculate memory usage more precisely. + val numElementsPerNode = DecisionTree.getElementsPerNode(metadata) logDebug("numElementsPerNode = " + numElementsPerNode) val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array @@ -124,26 +130,29 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. timer.start("findBestSplits") - val splitsStatsForLevel = DecisionTree.findBestSplits(treeInput, parentImpurities, - metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) + val splitsStatsForLevel: Array[(Split, InformationGainStats)] = + DecisionTree.findBestSplits(treeInput, parentImpurities, + metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") - val levelNodeIndexOffset = (1 << level) - 1 + val levelNodeIndexOffset = Node.startIndexInLevel(level) for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { val nodeIndex = levelNodeIndexOffset + index - val isLeftChild = level != 0 && nodeIndex % 2 == 1 - val parentNodeIndex = if (isLeftChild) { // -1 for root node - (nodeIndex - 1) / 2 - } else { - (nodeIndex - 2) / 2 - } + // Extract info for this node (index) at the current level. timer.start("extractNodeInfo") - extractNodeInfo(nodeSplitStats, level, index, nodes) + val split = nodeSplitStats._1 + val stats = nodeSplitStats._2 + val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) + val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) + logDebug("Node = " + node) + nodes(nodeIndex) = node timer.stop("extractNodeInfo") + if (level != 0) { // Set parent. - if (isLeftChild) { + val parentNodeIndex = Node.parentIndex(nodeIndex) + if (Node.isLeftChild(nodeIndex)) { nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) } else { nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) @@ -151,11 +160,21 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo } // Extract info for nodes at the next lower level. timer.start("extractInfoForLowerLevels") - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities) + if (level < maxDepth) { + val leftChildIndex = Node.leftChildIndex(nodeIndex) + val leftImpurity = stats.leftImpurity + logDebug("leftChildIndex = " + leftChildIndex + ", impurity = " + leftImpurity) + parentImpurities(leftChildIndex) = leftImpurity + + val rightChildIndex = Node.rightChildIndex(nodeIndex) + val rightImpurity = stats.rightImpurity + logDebug("rightChildIndex = " + rightChildIndex + ", impurity = " + rightImpurity) + parentImpurities(rightChildIndex) = rightImpurity + } timer.stop("extractInfoForLowerLevels") - logDebug("final best split = " + nodeSplitStats._1) + logDebug("final best split = " + split) } - require((1 << level) == splitsStatsForLevel.length) + require(Node.maxNodesInLevel(level) == splitsStatsForLevel.length) // Check whether all the nodes at the current level at leaves. val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) logDebug("all leaf = " + allLeaf) @@ -171,7 +190,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("#####################################") // Initialize the top or root node of the tree. - val topNode = nodes(0) + val topNode = nodes(1) // Build the full tree using the node info calculated in the level-wise best split calculations. topNode.build(nodes) @@ -183,47 +202,6 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo new DecisionTreeModel(topNode, strategy.algo) } - /** - * Extract the decision tree node information for the given tree level and node index - */ - private def extractNodeInfo( - nodeSplitStats: (Split, InformationGainStats), - level: Int, - index: Int, - nodes: Array[Node]): Unit = { - val split = nodeSplitStats._1 - val stats = nodeSplitStats._2 - val nodeIndex = (1 << level) - 1 + index - val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) - val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) - logDebug("Node = " + node) - nodes(nodeIndex) = node - } - - /** - * Extract the decision tree node information for the children of the node - */ - private def extractInfoForLowerLevels( - level: Int, - index: Int, - maxDepth: Int, - nodeSplitStats: (Split, InformationGainStats), - parentImpurities: Array[Double]): Unit = { - - if (level >= maxDepth) { - return - } - - val leftNodeIndex = (2 << level) - 1 + 2 * index - val leftImpurity = nodeSplitStats._2.leftImpurity - logDebug("leftNodeIndex = " + leftNodeIndex + ", impurity = " + leftImpurity) - parentImpurities(leftNodeIndex) = leftImpurity - - val rightNodeIndex = leftNodeIndex + 1 - val rightImpurity = nodeSplitStats._2.rightImpurity - logDebug("rightNodeIndex = " + rightNodeIndex + ", impurity = " + rightImpurity) - parentImpurities(rightNodeIndex) = rightImpurity - } } object DecisionTree extends Serializable with Logging { @@ -425,9 +403,6 @@ object DecisionTree extends Serializable with Logging { impurity, maxDepth, maxBins) } - - private val InvalidBinIndex = -1 - /** * Returns an array of optimal splits for all nodes at a given level. Splits the task into * multiple groups if the level-wise training task could lead to memory overflow. @@ -436,12 +411,12 @@ object DecisionTree extends Serializable with Logging { * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param splits possible splits for all features - * @param bins possible bins for all features + * @param splits possible splits for all features, indexed (numFeatures)(numSplits) + * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. * @return array (over nodes) of splits with best split for each node at a given level. */ - protected[tree] def findBestSplits( + private[tree] def findBestSplits( input: RDD[TreePoint], parentImpurities: Array[Double], metadata: DecisionTreeMetadata, @@ -474,6 +449,138 @@ object DecisionTree extends Serializable with Logging { } } + /** + * Get the node index corresponding to this data point. + * This function mimics prediction, passing an example from the root node down to a node + * at the current level being trained; that node's index is returned. + * + * @param node Node in tree from which to classify the given data point. + * @param binnedFeatures Binned feature vector for data point. + * @param bins possible bins for all features, indexed (numFeatures)(numBins) + * @param unorderedFeatures Set of indices of unordered features. + * @return Leaf index if the data point reaches a leaf. + * Otherwise, last node reachable in tree matching this example. + * Note: This is the global node index, i.e., the index used in the tree. + * This index is different from the index used during training a particular + * set of nodes in a (level, group). + */ + private def predictNodeIndex( + node: Node, + binnedFeatures: Array[Int], + bins: Array[Array[Bin]], + unorderedFeatures: Set[Int]): Int = { + if (node.isLeaf) { + node.id + } else { + val featureIndex = node.split.get.feature + val splitLeft = node.split.get.featureType match { + case Continuous => { + val binIndex = binnedFeatures(featureIndex) + val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold + // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] + // We do not need to check lowSplit since bins are separated by splits. + featureValueUpperBound <= node.split.get.threshold + } + case Categorical => { + val featureValue = binnedFeatures(featureIndex) + node.split.get.categories.contains(featureValue) + } + case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") + } + if (node.leftNode.isEmpty || node.rightNode.isEmpty) { + // Return index from next layer of nodes to train + if (splitLeft) { + Node.leftChildIndex(node.id) + } else { + Node.rightChildIndex(node.id) + } + } else { + if (splitLeft) { + predictNodeIndex(node.leftNode.get, binnedFeatures, bins, unorderedFeatures) + } else { + predictNodeIndex(node.rightNode.get, binnedFeatures, bins, unorderedFeatures) + } + } + } + } + + /** + * Helper for binSeqOp, for data which can contain a mix of ordered and unordered features. + * + * For ordered features, a single bin is updated. + * For unordered features, bins correspond to subsets of categories; either the left or right bin + * for each subset is updated. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @param bins possible bins for all features, indexed (numFeatures)(numBins) + * @param unorderedFeatures Set of indices of unordered features. + */ + private def mixedBinSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint, + nodeIndex: Int, + bins: Array[Array[Bin]], + unorderedFeatures: Set[Int]): Unit = { + // Iterate over all features. + val numFeatures = treePoint.binnedFeatures.size + val nodeOffset = agg.getNodeOffset(nodeIndex) + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (unorderedFeatures.contains(featureIndex)) { + // Unordered feature + val featureValue = treePoint.binnedFeatures(featureIndex) + val (leftNodeFeatureOffset, rightNodeFeatureOffset) = + agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + // Update the left or right bin for each split. + val numSplits = agg.numSplits(featureIndex) + var splitIndex = 0 + while (splitIndex < numSplits) { + if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { + agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label) + } else { + agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label) + } + splitIndex += 1 + } + } else { + // Ordered feature + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.nodeUpdate(nodeOffset, featureIndex, binIndex, treePoint.label) + } + featureIndex += 1 + } + } + + /** + * Helper for binSeqOp, for regression and for classification with only ordered features. + * + * For each feature, the sufficient statistics of one bin are updated. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @return agg + */ + private def orderedBinSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint, + nodeIndex: Int): Unit = { + val label = treePoint.label + val nodeOffset = agg.getNodeOffset(nodeIndex) + // Iterate over all features. + val numFeatures = agg.numFeatures + var featureIndex = 0 + while (featureIndex < numFeatures) { + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.nodeUpdate(nodeOffset, featureIndex, binIndex, label) + featureIndex += 1 + } + } + /** * Returns an array of optimal splits for a group of nodes at a given level * @@ -481,8 +588,9 @@ object DecisionTree extends Serializable with Logging { * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param splits possible splits for all features - * @param bins possible bins for all features, indexed as (numFeatures)(numBins) + * @param nodes Array of all nodes in the tree. Used for matching data points to nodes. + * @param splits possible splits for all features, indexed (numFeatures)(numSplits) + * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. * @return array of splits with best splits for all nodes at a given level. @@ -527,88 +635,22 @@ object DecisionTree extends Serializable with Logging { // numNodes: Number of nodes in this (level of tree, group), // where nodes at deeper (larger) levels may be divided into groups. - val numNodes = (1 << level) / numGroups + val numNodes = Node.maxNodesInLevel(level) / numGroups logDebug("numNodes = " + numNodes) - // Find the number of features by looking at the first sample. - val numFeatures = metadata.numFeatures - logDebug("numFeatures = " + numFeatures) - - // numBins: Number of bins = 1 + number of possible splits - val numBins = bins(0).length - logDebug("numBins = " + numBins) - - val numClasses = metadata.numClasses - logDebug("numClasses = " + numClasses) - - val isMulticlass = metadata.isMulticlass - logDebug("isMulticlass = " + isMulticlass) - - val isMulticlassWithCategoricalFeatures = metadata.isMulticlassWithCategoricalFeatures - logDebug("isMultiClassWithCategoricalFeatures = " + isMulticlassWithCategoricalFeatures) + logDebug("numFeatures = " + metadata.numFeatures) + logDebug("numClasses = " + metadata.numClasses) + logDebug("isMulticlass = " + metadata.isMulticlass) + logDebug("isMulticlassWithCategoricalFeatures = " + + metadata.isMulticlassWithCategoricalFeatures) // shift when more than one group is used at deep tree level val groupShift = numNodes * groupIndex - /** - * Get the node index corresponding to this data point. - * This function mimics prediction, passing an example from the root node down to a node - * at the current level being trained; that node's index is returned. - * - * @return Leaf index if the data point reaches a leaf. - * Otherwise, last node reachable in tree matching this example. - */ - def predictNodeIndex(node: Node, binnedFeatures: Array[Int]): Int = { - if (node.isLeaf) { - node.id - } else { - val featureIndex = node.split.get.feature - val splitLeft = node.split.get.featureType match { - case Continuous => { - val binIndex = binnedFeatures(featureIndex) - val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold - // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] - // We do not need to check lowSplit since bins are separated by splits. - featureValueUpperBound <= node.split.get.threshold - } - case Categorical => { - val featureValue = if (metadata.isUnordered(featureIndex)) { - binnedFeatures(featureIndex) - } else { - val binIndex = binnedFeatures(featureIndex) - bins(featureIndex)(binIndex).category - } - node.split.get.categories.contains(featureValue) - } - case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") - } - if (node.leftNode.isEmpty || node.rightNode.isEmpty) { - // Return index from next layer of nodes to train - if (splitLeft) { - node.id * 2 + 1 // left - } else { - node.id * 2 + 2 // right - } - } else { - if (splitLeft) { - predictNodeIndex(node.leftNode.get, binnedFeatures) - } else { - predictNodeIndex(node.rightNode.get, binnedFeatures) - } - } - } - } - - def nodeIndexToLevel(idx: Int): Int = { - if (idx == 0) { - 0 - } else { - math.floor(math.log(idx) / math.log(2)).toInt - } - } - - // Used for treePointToNodeIndex - val levelOffset = (1 << level) - 1 + // Used for treePointToNodeIndex to get an index for this (level, group). + // - Node.startIndexInLevel(level) gives the global index offset for nodes at this level. + // - groupShift corrects for groups in this level before the current group. + val globalNodeIndexOffset = Node.startIndexInLevel(level) + groupShift /** * Find the node index for the given example. @@ -619,661 +661,254 @@ object DecisionTree extends Serializable with Logging { if (level == 0) { 0 } else { - val globalNodeIndex = predictNodeIndex(nodes(0), treePoint.binnedFeatures) - // Get index for this (level, group). - globalNodeIndex - levelOffset - groupShift - } - } - - /** - * Increment aggregate in location for (node, feature, bin, label). - * - * @param treePoint Data point being aggregated. - * @param agg Array storing aggregate calculation, of size: - * numClasses * numBins * numFeatures * numNodes. - * Indexed by (node, feature, bin, label) where label is the least significant bit. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def updateBinForOrderedFeature( - treePoint: TreePoint, - agg: Array[Double], - nodeIndex: Int, - featureIndex: Int): Unit = { - // Update the left or right count for one bin. - val aggIndex = - numClasses * numBins * numFeatures * nodeIndex + - numClasses * numBins * featureIndex + - numClasses * treePoint.binnedFeatures(featureIndex) + - treePoint.label.toInt - agg(aggIndex) += 1 - } - - /** - * Increment aggregate in location for (nodeIndex, featureIndex, [bins], label), - * where [bins] ranges over all bins. - * Updates left or right side of aggregate depending on split. - * - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @param treePoint Data point being aggregated. - * @param agg Indexed by (left/right, node, feature, bin, label) - * where label is the least significant bit. - * The left/right specifier is a 0/1 index indicating left/right child info. - * @param rightChildShift Offset for right side of agg. - */ - def updateBinForUnorderedFeature( - nodeIndex: Int, - featureIndex: Int, - treePoint: TreePoint, - agg: Array[Double], - rightChildShift: Int): Unit = { - val featureValue = treePoint.binnedFeatures(featureIndex) - // Update the left or right count for one bin. - val aggShift = - numClasses * numBins * numFeatures * nodeIndex + - numClasses * numBins * featureIndex + - treePoint.label.toInt - // Find all matching bins and increment their values - val featureCategories = metadata.featureArity(featureIndex) - val numCategoricalBins = (1 << featureCategories - 1) - 1 - var binIndex = 0 - while (binIndex < numCategoricalBins) { - val aggIndex = aggShift + binIndex * numClasses - if (bins(featureIndex)(binIndex).highSplit.categories.contains(featureValue)) { - agg(aggIndex) += 1 - } else { - agg(rightChildShift + aggIndex) += 1 - } - binIndex += 1 - } - } - - /** - * Helper for binSeqOp. - * - * @param agg Array storing aggregate calculation, of size: - * numClasses * numBins * numFeatures * numNodes. - * Indexed by (node, feature, bin, label) where label is the least significant bit. - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def binaryOrNotCategoricalBinSeqOp( - agg: Array[Double], - treePoint: TreePoint, - nodeIndex: Int): Unit = { - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) - featureIndex += 1 - } - } - - val rightChildShift = numClasses * numBins * numFeatures * numNodes - - /** - * Helper for binSeqOp. - * - * @param agg Array storing aggregate calculation. - * For ordered features, this is of size: - * numClasses * numBins * numFeatures * numNodes. - * For unordered features, this is of size: - * 2 * numClasses * numBins * numFeatures * numNodes. - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def multiclassWithCategoricalBinSeqOp( - agg: Array[Double], - treePoint: TreePoint, - nodeIndex: Int): Unit = { - val label = treePoint.label - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (metadata.isUnordered(featureIndex)) { - updateBinForUnorderedFeature(nodeIndex, featureIndex, treePoint, agg, rightChildShift) - } else { - updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) - } - featureIndex += 1 - } - } - - /** - * Performs a sequential aggregation over a partition for regression. - * For l nodes, k features, - * the count, sum, sum of squares of one of the p bins is incremented. - * - * @param agg Array storing aggregate calculation, updated by this function. - * Size: 3 * numBins * numFeatures * numNodes - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @return agg - */ - def regressionBinSeqOp(agg: Array[Double], treePoint: TreePoint, nodeIndex: Int): Unit = { - val label = treePoint.label - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Update count, sum, and sum^2 for one bin. - val binIndex = treePoint.binnedFeatures(featureIndex) - val aggIndex = - 3 * numBins * numFeatures * nodeIndex + - 3 * numBins * featureIndex + - 3 * binIndex - agg(aggIndex) += 1 - agg(aggIndex + 1) += label - agg(aggIndex + 2) += label * label - featureIndex += 1 + val globalNodeIndex = + predictNodeIndex(nodes(1), treePoint.binnedFeatures, bins, metadata.unorderedFeatures) + globalNodeIndex - globalNodeIndexOffset } } /** * Performs a sequential aggregation over a partition. - * For l nodes, k features, - * For classification: - * Either the left count or the right count of one of the bins is - * incremented based upon whether the feature is classified as 0 or 1. - * For regression: - * The count, sum, sum of squares of one of the bins is incremented. * - * @param agg Array storing aggregate calculation, updated by this function. - * Size for classification: - * numClasses * numBins * numFeatures * numNodes for ordered features, or - * 2 * numClasses * numBins * numFeatures * numNodes for unordered features. - * Size for regression: - * 3 * numBins * numFeatures * numNodes. + * Each data point contributes to one node. For each feature, + * the aggregate sufficient statistics are updated for the relevant bins. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). * @param treePoint Data point being aggregated. * @return agg */ - def binSeqOp(agg: Array[Double], treePoint: TreePoint): Array[Double] = { + def binSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint): DTStatsAggregator = { val nodeIndex = treePointToNodeIndex(treePoint) // If the example does not reach this level, then nodeIndex < 0. // If the example reaches this level but is handled in a different group, // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). if (nodeIndex >= 0 && nodeIndex < numNodes) { - if (metadata.isClassification) { - if (isMulticlassWithCategoricalFeatures) { - multiclassWithCategoricalBinSeqOp(agg, treePoint, nodeIndex) - } else { - binaryOrNotCategoricalBinSeqOp(agg, treePoint, nodeIndex) - } + if (metadata.unorderedFeatures.isEmpty) { + orderedBinSeqOp(agg, treePoint, nodeIndex) } else { - regressionBinSeqOp(agg, treePoint, nodeIndex) + mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures) } } agg } - // Calculate bin aggregate length for classification or regression. - val binAggregateLength = numNodes * getElementsPerNode(metadata, numBins) - logDebug("binAggregateLength = " + binAggregateLength) - - /** - * Combines the aggregates from partitions. - * @param agg1 Array containing aggregates from one or more partitions - * @param agg2 Array containing aggregates from one or more partitions - * @return Combined aggregate from agg1 and agg2 - */ - def binCombOp(agg1: Array[Double], agg2: Array[Double]): Array[Double] = { - var index = 0 - val combinedAggregate = new Array[Double](binAggregateLength) - while (index < binAggregateLength) { - combinedAggregate(index) = agg1(index) + agg2(index) - index += 1 - } - combinedAggregate - } - // Calculate bin aggregates. timer.start("aggregation") - val binAggregates = { - input.treeAggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) + val binAggregates: DTStatsAggregator = { + val initAgg = new DTStatsAggregator(metadata, numNodes) + input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp) } timer.stop("aggregation") - logDebug("binAggregates.length = " + binAggregates.length) - /** - * Calculate the information gain for a given (feature, split) based upon left/right aggregates. - * @param leftNodeAgg left node aggregates for this (feature, split) - * @param rightNodeAgg right node aggregate for this (feature, split) - * @param topImpurity impurity of the parent node - * @return information gain and statistics for all splits - */ - def calculateGainForSplit( - leftNodeAgg: Array[Double], - rightNodeAgg: Array[Double], - topImpurity: Double): InformationGainStats = { - if (metadata.isClassification) { - val leftTotalCount = leftNodeAgg.sum - val rightTotalCount = rightNodeAgg.sum - - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val rootNodeCounts = new Array[Double](numClasses) - var classIndex = 0 - while (classIndex < numClasses) { - rootNodeCounts(classIndex) = leftNodeAgg(classIndex) + rightNodeAgg(classIndex) - classIndex += 1 - } - metadata.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) - } - } - - val totalCount = leftTotalCount + rightTotalCount - if (totalCount == 0) { - // Return arbitrary prediction. - return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) - } - - // Sum of count for each label - val leftrightNodeAgg: Array[Double] = - leftNodeAgg.zip(rightNodeAgg).map { case (leftCount, rightCount) => - leftCount + rightCount - } - - def indexOfLargestArrayElement(array: Array[Double]): Int = { - val result = array.foldLeft(-1, Double.MinValue, 0) { - case ((maxIndex, maxValue, currentIndex), currentValue) => - if (currentValue > maxValue) { - (currentIndex, currentValue, currentIndex + 1) - } else { - (maxIndex, maxValue, currentIndex + 1) - } - } - if (result._1 < 0) { - throw new RuntimeException("DecisionTree internal error:" + - " calculateGainForSplit failed in indexOfLargestArrayElement") - } - result._1 - } - - val predict = indexOfLargestArrayElement(leftrightNodeAgg) - val prob = leftrightNodeAgg(predict) / totalCount - - val leftImpurity = if (leftTotalCount == 0) { - topImpurity - } else { - metadata.impurity.calculate(leftNodeAgg, leftTotalCount) - } - val rightImpurity = if (rightTotalCount == 0) { - topImpurity - } else { - metadata.impurity.calculate(rightNodeAgg, rightTotalCount) - } - - val leftWeight = leftTotalCount / totalCount - val rightWeight = rightTotalCount / totalCount - - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) - - } else { - // Regression - - val leftCount = leftNodeAgg(0) - val leftSum = leftNodeAgg(1) - val leftSumSquares = leftNodeAgg(2) + // Calculate best splits for all nodes at a given level + timer.start("chooseSplits") + val bestSplits = new Array[(Split, InformationGainStats)](numNodes) + // Iterating over all nodes at this level + var nodeIndex = 0 + while (nodeIndex < numNodes) { + val nodeImpurity = parentImpurities(globalNodeIndexOffset + nodeIndex) + logDebug("node impurity = " + nodeImpurity) + bestSplits(nodeIndex) = + binsToBestSplit(binAggregates, nodeIndex, nodeImpurity, level, metadata, splits) + logDebug("best split = " + bestSplits(nodeIndex)._1) + nodeIndex += 1 + } + timer.stop("chooseSplits") - val rightCount = rightNodeAgg(0) - val rightSum = rightNodeAgg(1) - val rightSumSquares = rightNodeAgg(2) + bestSplits + } - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val count = leftCount + rightCount - val sum = leftSum + rightSum - val sumSquares = leftSumSquares + rightSumSquares - metadata.impurity.calculate(count, sum, sumSquares) - } - } + /** + * Calculate the information gain for a given (feature, split) based upon left/right aggregates. + * @param leftImpurityCalculator left node aggregates for this (feature, split) + * @param rightImpurityCalculator right node aggregate for this (feature, split) + * @param topImpurity impurity of the parent node + * @return information gain and statistics for all splits + */ + private def calculateGainForSplit( + leftImpurityCalculator: ImpurityCalculator, + rightImpurityCalculator: ImpurityCalculator, + topImpurity: Double, + level: Int, + metadata: DecisionTreeMetadata): InformationGainStats = { - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, - rightSum / rightCount) - } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity, topImpurity, - Double.MinValue, leftSum / leftCount) - } + val leftCount = leftImpurityCalculator.count + val rightCount = rightImpurityCalculator.count - val leftImpurity = metadata.impurity.calculate(leftCount, leftSum, leftSumSquares) - val rightImpurity = metadata.impurity.calculate(rightCount, rightSum, rightSumSquares) + val totalCount = leftCount + rightCount + if (totalCount == 0) { + // Return arbitrary prediction. + return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + } - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) + // impurity of parent node + val impurity = if (level > 0) { + topImpurity + } else { + parentNodeAgg.calculate() + } - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + val predict = parentNodeAgg.predict + val prob = parentNodeAgg.prob(predict) - val predict = (leftSum + rightSum) / (leftCount + rightCount) - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) - } - } + val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 + val rightImpurity = rightImpurityCalculator.calculate() - /** - * Extracts left and right split aggregates. - * @param binData Aggregate array slice from getBinDataForNode. - * For classification: - * For unordered features, this is leftChildData ++ rightChildData, - * each of which is indexed by (feature, split/bin, class), - * with class being the least significant bit. - * For ordered features, this is of size numClasses * numBins * numFeatures. - * For regression: - * This is of size 2 * numFeatures * numBins. - * @return (leftNodeAgg, rightNodeAgg) pair of arrays. - * For classification, each array is of size (numFeatures, (numBins - 1), numClasses). - * For regression, each array is of size (numFeatures, (numBins - 1), 3). - * - */ - def extractLeftRightNodeAggregates( - binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { - - - /** - * The input binData is indexed as (feature, bin, class). - * This computes cumulative sums over splits. - * Each (feature, class) pair is handled separately. - * Note: numSplits = numBins - 1. - * @param leftNodeAgg Each (feature, class) slice is an array over splits. - * Element i (i = 0, ..., numSplits - 2) is set to be - * the cumulative sum (from left) over binData for bins 0, ..., i. - * @param rightNodeAgg Each (feature, class) slice is an array over splits. - * Element i (i = 1, ..., numSplits - 1) is set to be - * the cumulative sum (from right) over binData for bins - * numBins - 1, ..., numBins - 1 - i. - */ - def findAggForOrderedFeatureClassification( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - // shift for this featureIndex - val shift = numClasses * featureIndex * numBins - - var classIndex = 0 - while (classIndex < numClasses) { - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0)(classIndex) = binData(shift + classIndex) - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(numBins - 2)(classIndex) - = binData(shift + (numClasses * (numBins - 1)) + classIndex) - classIndex += 1 - } + val leftWeight = leftCount / totalCount.toDouble + val rightWeight = rightCount / totalCount.toDouble - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - var innerClassIndex = 0 - while (innerClassIndex < numClasses) { - leftNodeAgg(featureIndex)(splitIndex)(innerClassIndex) - = binData(shift + numClasses * splitIndex + innerClassIndex) + - leftNodeAgg(featureIndex)(splitIndex - 1)(innerClassIndex) - rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(innerClassIndex) = - binData(shift + (numClasses * (numBins - 1 - splitIndex) + innerClassIndex)) + - rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(innerClassIndex) - innerClassIndex += 1 - } - splitIndex += 1 - } - } + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - /** - * Reshape binData for this feature. - * Indexes binData as (feature, split, class) with class as the least significant bit. - * @param leftNodeAgg leftNodeAgg(featureIndex)(splitIndex)(classIndex) = aggregate value - */ - def findAggForUnorderedFeatureClassification( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - val rightChildShift = numClasses * numBins * numFeatures - var splitIndex = 0 - while (splitIndex < numBins - 1) { - var classIndex = 0 - while (classIndex < numClasses) { - // shift for this featureIndex - val shift = numClasses * featureIndex * numBins + splitIndex * numClasses - val leftBinValue = binData(shift + classIndex) - val rightBinValue = binData(rightChildShift + shift + classIndex) - leftNodeAgg(featureIndex)(splitIndex)(classIndex) = leftBinValue - rightNodeAgg(featureIndex)(splitIndex)(classIndex) = rightBinValue - classIndex += 1 - } - splitIndex += 1 - } - } + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + } - def findAggForRegression( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - // shift for this featureIndex - val shift = 3 * featureIndex * numBins - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0)(0) = binData(shift + 0) - leftNodeAgg(featureIndex)(0)(1) = binData(shift + 1) - leftNodeAgg(featureIndex)(0)(2) = binData(shift + 2) - - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(numBins - 2)(0) = - binData(shift + (3 * (numBins - 1))) - rightNodeAgg(featureIndex)(numBins - 2)(1) = - binData(shift + (3 * (numBins - 1)) + 1) - rightNodeAgg(featureIndex)(numBins - 2)(2) = - binData(shift + (3 * (numBins - 1)) + 2) - - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - var i = 0 // index for regression histograms - while (i < 3) { // count, sum, sum^2 - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - leftNodeAgg(featureIndex)(splitIndex)(i) = binData(shift + 3 * splitIndex + i) + - leftNodeAgg(featureIndex)(splitIndex - 1)(i) - // calculating right node aggregate for a split as a sum of right node aggregate of a - // higher split and the right bin aggregate of a bin where the split is a low split - rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(i) = - binData(shift + (3 * (numBins - 1 - splitIndex) + i)) + - rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(i) - i += 1 - } - splitIndex += 1 - } - } + /** + * Find the best split for a node. + * @param binAggregates Bin statistics. + * @param nodeIndex Index for node to split in this (level, group). + * @param nodeImpurity Impurity of the node (nodeIndex). + * @return tuple for best split: (Split, information gain) + */ + private def binsToBestSplit( + binAggregates: DTStatsAggregator, + nodeIndex: Int, + nodeImpurity: Double, + level: Int, + metadata: DecisionTreeMetadata, + splits: Array[Array[Split]]): (Split, InformationGainStats) = { - if (metadata.isClassification) { - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (metadata.isUnordered(featureIndex)) { - findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } else { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } - featureIndex += 1 - } - (leftNodeAgg, rightNodeAgg) - } else { - // Regression - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) - featureIndex += 1 - } - (leftNodeAgg, rightNodeAgg) - } - } + logDebug("node impurity = " + nodeImpurity) - /** - * Calculates information gain for all nodes splits. - */ - def calculateGainsForAllNodeSplits( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - nodeImpurity: Double): Array[Array[InformationGainStats]] = { - val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1) - - var featureIndex = 0 - while (featureIndex < numFeatures) { - val numSplitsForFeature = getNumSplitsForFeature(featureIndex) + // For each (feature, split), calculate the gain, and select the best (feature, split). + Range(0, metadata.numFeatures).map { featureIndex => + val numSplits = metadata.numSplits(featureIndex) + if (metadata.isContinuous(featureIndex)) { + // Cumulative sum (scanLeft) of bin statistics. + // Afterwards, binAggregates for a bin is the sum of aggregates for + // that bin + all preceding bins. + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) var splitIndex = 0 - while (splitIndex < numSplitsForFeature) { - gains(featureIndex)(splitIndex) = - calculateGainForSplit(leftNodeAgg(featureIndex)(splitIndex), - rightNodeAgg(featureIndex)(splitIndex), nodeImpurity) + while (splitIndex < numSplits) { + binAggregates.mergeForNodeFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) splitIndex += 1 } - featureIndex += 1 - } - gains - } - - /** - * Get the number of splits for a feature. - */ - def getNumSplitsForFeature(featureIndex: Int): Int = { - if (metadata.isContinuous(featureIndex)) { - numBins - 1 + // Find best split. + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { case splitIdx => + val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) + val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) + rightChildStats.subtract(leftChildStats) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIdx, gainStats) + }.maxBy(_._2.gain) + (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) + } else if (metadata.isUnordered(featureIndex)) { + // Unordered categorical feature + val (leftChildOffset, rightChildOffset) = + binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { splitIndex => + val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) + val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIndex, gainStats) + }.maxBy(_._2.gain) + (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else { - // Categorical feature - val featureCategories = metadata.featureArity(featureIndex) - if (metadata.isUnordered(featureIndex)) { - (1 << featureCategories - 1) - 1 - } else { - featureCategories - } - } - } - - /** - * Find the best split for a node. - * @param binData Bin data slice for this node, given by getBinDataForNode. - * @param nodeImpurity impurity of the top node - * @return tuple of split and information gain - */ - def binsToBestSplit( - binData: Array[Double], - nodeImpurity: Double): (Split, InformationGainStats) = { - - logDebug("node impurity = " + nodeImpurity) - - // Extract left right node aggregates. - val (leftNodeAgg, rightNodeAgg) = extractLeftRightNodeAggregates(binData) - - // Calculate gains for all splits. - val gains = calculateGainsForAllNodeSplits(leftNodeAgg, rightNodeAgg, nodeImpurity) - - val (bestFeatureIndex, bestSplitIndex, gainStats) = { - // Initialize with infeasible values. - var bestFeatureIndex = Int.MinValue - var bestSplitIndex = Int.MinValue - var bestGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0, -1.0) - // Iterate over features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Iterate over all splits. - var splitIndex = 0 - val numSplitsForFeature = getNumSplitsForFeature(featureIndex) - while (splitIndex < numSplitsForFeature) { - val gainStats = gains(featureIndex)(splitIndex) - if (gainStats.gain > bestGainStats.gain) { - bestGainStats = gainStats - bestFeatureIndex = featureIndex - bestSplitIndex = splitIndex + // Ordered categorical feature + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) + val numBins = metadata.numBins(featureIndex) + + /* Each bin is one category (feature value). + * The bins are ordered based on centroidForCategories, and this ordering determines which + * splits are considered. (With K categories, we consider K - 1 possible splits.) + * + * centroidForCategories is a list: (category, centroid) + */ + val centroidForCategories = if (metadata.isMulticlass) { + // For categorical variables in multiclass classification, + // the bins are ordered by the impurity of their corresponding labels. + Range(0, numBins).map { case featureValue => + val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val centroid = if (categoryStats.count != 0) { + categoryStats.calculate() + } else { + Double.MaxValue } - splitIndex += 1 + (featureValue, centroid) + } + } else { // regression or binary classification + // For categorical variables in regression and binary classification, + // the bins are ordered by the centroid of their corresponding labels. + Range(0, numBins).map { case featureValue => + val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val centroid = if (categoryStats.count != 0) { + categoryStats.predict + } else { + Double.MaxValue + } + (featureValue, centroid) } - featureIndex += 1 } - (bestFeatureIndex, bestSplitIndex, bestGainStats) - } - logDebug("best split = " + splits(bestFeatureIndex)(bestSplitIndex)) - logDebug("best split bin = " + bins(bestFeatureIndex)(bestSplitIndex)) + logDebug("Centroids for categorical variable: " + centroidForCategories.mkString(",")) - (splits(bestFeatureIndex)(bestSplitIndex), gainStats) - } + // bins sorted by centroids + val categoriesSortedByCentroid = centroidForCategories.toList.sortBy(_._2) - /** - * Get bin data for one node. - */ - def getBinDataForNode(node: Int): Array[Double] = { - if (metadata.isClassification) { - if (isMulticlassWithCategoricalFeatures) { - val shift = numClasses * node * numBins * numFeatures - val rightChildShift = numClasses * numBins * numFeatures * numNodes - val binsForNode = { - val leftChildData - = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - val rightChildData - = binAggregates.slice(rightChildShift + shift, - rightChildShift + shift + numClasses * numBins * numFeatures) - leftChildData ++ rightChildData - } - binsForNode - } else { - val shift = numClasses * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - binsForNode + logDebug("Sorted centroids for categorical variable = " + + categoriesSortedByCentroid.mkString(",")) + + // Cumulative sum (scanLeft) of bin statistics. + // Afterwards, binAggregates for a bin is the sum of aggregates for + // that bin + all preceding bins. + var splitIndex = 0 + while (splitIndex < numSplits) { + val currentCategory = categoriesSortedByCentroid(splitIndex)._1 + val nextCategory = categoriesSortedByCentroid(splitIndex + 1)._1 + binAggregates.mergeForNodeFeature(nodeFeatureOffset, nextCategory, currentCategory) + splitIndex += 1 } - } else { - // Regression - val shift = 3 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) - binsForNode + // lastCategory = index of bin with total aggregates for this (node, feature) + val lastCategory = categoriesSortedByCentroid.last._1 + // Find best split. + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { splitIndex => + val featureValue = categoriesSortedByCentroid(splitIndex)._1 + val leftChildStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val rightChildStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) + rightChildStats.subtract(leftChildStats) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIndex, gainStats) + }.maxBy(_._2.gain) + val categoriesForSplit = + categoriesSortedByCentroid.map(_._1.toDouble).slice(0, bestFeatureSplitIndex + 1) + val bestFeatureSplit = + new Split(featureIndex, Double.MinValue, Categorical, categoriesForSplit) + (bestFeatureSplit, bestFeatureGainStats) } - } - - // Calculate best splits for all nodes at a given level - timer.start("chooseSplits") - val bestSplits = new Array[(Split, InformationGainStats)](numNodes) - // Iterating over all nodes at this level - var node = 0 - while (node < numNodes) { - val nodeImpurityIndex = (1 << level) - 1 + node + groupShift - val binsForNode: Array[Double] = getBinDataForNode(node) - logDebug("nodeImpurityIndex = " + nodeImpurityIndex) - val parentNodeImpurity = parentImpurities(nodeImpurityIndex) - logDebug("parent node impurity = " + parentNodeImpurity) - bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity) - node += 1 - } - timer.stop("chooseSplits") - - bestSplits + }.maxBy(_._2.gain) } /** * Get the number of values to be stored per node in the bin aggregates. - * - * @param numBins Number of bins = 1 + number of possible splits. */ - private def getElementsPerNode(metadata: DecisionTreeMetadata, numBins: Int): Int = { + private def getElementsPerNode(metadata: DecisionTreeMetadata): Int = { + val totalBins = metadata.numBins.sum if (metadata.isClassification) { - if (metadata.isMulticlassWithCategoricalFeatures) { - 2 * metadata.numClasses * numBins * metadata.numFeatures - } else { - metadata.numClasses * numBins * metadata.numFeatures - } + metadata.numClasses * totalBins } else { - 3 * numBins * metadata.numFeatures + 3 * totalBins } } @@ -1284,6 +919,7 @@ object DecisionTree extends Serializable with Logging { * Continuous features: * For each feature, there are numBins - 1 possible splits representing the possible binary * decisions at each node in the tree. + * This finds locations (feature values) for splits using a subsample of the data. * * Categorical features: * For each feature, there is 1 bin per split. @@ -1292,7 +928,6 @@ object DecisionTree extends Serializable with Logging { * For multiclass classification with a low-arity feature * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), * the feature is split based on subsets of categories. - * There are (1 << maxFeatureValue - 1) - 1 splits. * (b) "ordered features" * For regression and binary classification, * and for multiclass classification with a high-arity feature, @@ -1302,7 +937,7 @@ object DecisionTree extends Serializable with Logging { * @param metadata Learning and dataset metadata * @return A tuple of (splits, bins). * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] - * of size (numFeatures, numBins - 1). + * of size (numFeatures, numSplits). * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] * of size (numFeatures, numBins). */ @@ -1310,84 +945,80 @@ object DecisionTree extends Serializable with Logging { input: RDD[LabeledPoint], metadata: DecisionTreeMetadata): (Array[Array[Split]], Array[Array[Bin]]) = { - val count = input.count() + logDebug("isMulticlass = " + metadata.isMulticlass) - // Find the number of features by looking at the first sample - val numFeatures = input.take(1)(0).features.size - - val maxBins = metadata.maxBins - val numBins = if (maxBins <= count) maxBins else count.toInt - logDebug("numBins = " + numBins) - val isMulticlass = metadata.isMulticlass - logDebug("isMulticlass = " + isMulticlass) - - /* - * Ensure numBins is always greater than the categories. For multiclass classification, - * numBins should be greater than 2^(maxCategories - 1) - 1. - * It's a limitation of the current implementation but a reasonable trade-off since features - * with large number of categories get favored over continuous features. - * - * This needs to be checked here instead of in Strategy since numBins can be determined - * by the number of training examples. - * TODO: Allow this case, where we simply will know nothing about some categories. - */ - if (metadata.featureArity.size > 0) { - val maxCategoriesForFeatures = metadata.featureArity.maxBy(_._2)._2 - require(numBins > maxCategoriesForFeatures, "numBins should be greater than max categories " + - "in categorical features") - } - - // Calculate the number of sample for approximate quantile calculation. - val requiredSamples = numBins*numBins - val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0 - logDebug("fraction of data used for calculating quantiles = " + fraction) + val numFeatures = metadata.numFeatures - // sampled input for RDD calculation - val sampledInput = + // Sample the input only if there are continuous features. + val hasContinuousFeatures = Range(0, numFeatures).exists(metadata.isContinuous) + val sampledInput = if (hasContinuousFeatures) { + // Calculate the number of samples for approximate quantile calculation. + val requiredSamples = math.max(metadata.maxBins * metadata.maxBins, 10000) + val fraction = if (requiredSamples < metadata.numExamples) { + requiredSamples.toDouble / metadata.numExamples + } else { + 1.0 + } + logDebug("fraction of data used for calculating quantiles = " + fraction) input.sample(withReplacement = false, fraction, new XORShiftRandom().nextInt()).collect() - val numSamples = sampledInput.length - - val stride: Double = numSamples.toDouble / numBins - logDebug("stride = " + stride) + } else { + new Array[LabeledPoint](0) + } metadata.quantileStrategy match { case Sort => - val splits = Array.ofDim[Split](numFeatures, numBins - 1) - val bins = Array.ofDim[Bin](numFeatures, numBins) + val splits = new Array[Array[Split]](numFeatures) + val bins = new Array[Array[Bin]](numFeatures) // Find all splits. - // Iterate over all features. var featureIndex = 0 while (featureIndex < numFeatures) { - // Check whether the feature is continuous. - val isFeatureContinuous = metadata.isContinuous(featureIndex) - if (isFeatureContinuous) { + val numSplits = metadata.numSplits(featureIndex) + val numBins = metadata.numBins(featureIndex) + if (metadata.isContinuous(featureIndex)) { + val numSamples = sampledInput.length + splits(featureIndex) = new Array[Split](numSplits) + bins(featureIndex) = new Array[Bin](numBins) val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted - val stride: Double = numSamples.toDouble / numBins + val stride: Double = numSamples.toDouble / metadata.numBins(featureIndex) logDebug("stride = " + stride) - for (index <- 0 until numBins - 1) { - val sampleIndex = index * stride.toInt + for (splitIndex <- 0 until numSplits) { + val sampleIndex = splitIndex * stride.toInt // Set threshold halfway in between 2 samples. val threshold = (featureSamples(sampleIndex) + featureSamples(sampleIndex + 1)) / 2.0 - val split = new Split(featureIndex, threshold, Continuous, List()) - splits(featureIndex)(index) = split + splits(featureIndex)(splitIndex) = + new Split(featureIndex, threshold, Continuous, List()) } - } else { // Categorical feature - val featureCategories = metadata.featureArity(featureIndex) - - // Use different bin/split calculation strategy for categorical features in multiclass - // classification that satisfy the space constraint. + bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), + splits(featureIndex)(0), Continuous, Double.MinValue) + for (splitIndex <- 1 until numSplits) { + bins(featureIndex)(splitIndex) = + new Bin(splits(featureIndex)(splitIndex - 1), splits(featureIndex)(splitIndex), + Continuous, Double.MinValue) + } + bins(featureIndex)(numSplits) = new Bin(splits(featureIndex)(numSplits - 1), + new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) + } else { + // Categorical feature + val featureArity = metadata.featureArity(featureIndex) if (metadata.isUnordered(featureIndex)) { - // 2^(maxFeatureValue- 1) - 1 combinations - var index = 0 - while (index < (1 << featureCategories - 1) - 1) { - val categories: List[Double] - = extractMultiClassCategories(index + 1, featureCategories) - splits(featureIndex)(index) - = new Split(featureIndex, Double.MinValue, Categorical, categories) - bins(featureIndex)(index) = { - if (index == 0) { + // TODO: The second half of the bins are unused. Actually, we could just use + // splits and not build bins for unordered features. That should be part of + // a later PR since it will require changing other code (using splits instead + // of bins in a few places). + // Unordered features + // 2^(maxFeatureValue - 1) - 1 combinations + splits(featureIndex) = new Array[Split](numSplits) + bins(featureIndex) = new Array[Bin](numBins) + var splitIndex = 0 + while (splitIndex < numSplits) { + val categories: List[Double] = + extractMultiClassCategories(splitIndex + 1, featureArity) + splits(featureIndex)(splitIndex) = + new Split(featureIndex, Double.MinValue, Categorical, categories) + bins(featureIndex)(splitIndex) = { + if (splitIndex == 0) { new Bin( new DummyCategoricalSplit(featureIndex, Categorical), splits(featureIndex)(0), @@ -1395,96 +1026,24 @@ object DecisionTree extends Serializable with Logging { Double.MinValue) } else { new Bin( - splits(featureIndex)(index - 1), - splits(featureIndex)(index), + splits(featureIndex)(splitIndex - 1), + splits(featureIndex)(splitIndex), Categorical, Double.MinValue) } } - index += 1 - } - } else { // ordered feature - /* For a given categorical feature, use a subsample of the data - * to choose how to arrange possible splits. - * This examines each category and computes a centroid. - * These centroids are later used to sort the possible splits. - * centroidForCategories is a mapping: category (for the given feature) --> centroid - */ - val centroidForCategories = { - if (isMulticlass) { - // For categorical variables in multiclass classification, - // each bin is a category. The bins are sorted and they - // are ordered by calculating the impurity of their corresponding labels. - sampledInput.map(lp => (lp.features(featureIndex), lp.label)) - .groupBy(_._1) - .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) - .map(x => (x._1, x._2.values.toArray)) - .map(x => (x._1, metadata.impurity.calculate(x._2, x._2.sum))) - } else { // regression or binary classification - // For categorical variables in regression and binary classification, - // each bin is a category. The bins are sorted and they - // are ordered by calculating the centroid of their corresponding labels. - sampledInput.map(lp => (lp.features(featureIndex), lp.label)) - .groupBy(_._1) - .mapValues(x => x.map(_._2).sum / x.map(_._1).length) - } - } - - logDebug("centroid for categories = " + centroidForCategories.mkString(",")) - - // Check for missing categorical variables and putting them last in the sorted list. - val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() - for (i <- 0 until featureCategories) { - if (centroidForCategories.contains(i)) { - fullCentroidForCategories(i) = centroidForCategories(i) - } else { - fullCentroidForCategories(i) = Double.MaxValue - } - } - - // bins sorted by centroids - val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) - - logDebug("centroid for categorical variable = " + categoriesSortedByCentroid) - - var categoriesForSplit = List[Double]() - categoriesSortedByCentroid.iterator.zipWithIndex.foreach { - case ((key, value), index) => - categoriesForSplit = key :: categoriesForSplit - splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, - Categorical, categoriesForSplit) - bins(featureIndex)(index) = { - if (index == 0) { - new Bin(new DummyCategoricalSplit(featureIndex, Categorical), - splits(featureIndex)(0), Categorical, key) - } else { - new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Categorical, key) - } - } + splitIndex += 1 } + } else { + // Ordered features + // Bins correspond to feature values, so we do not need to compute splits or bins + // beforehand. Splits are constructed as needed during training. + splits(featureIndex) = new Array[Split](0) + bins(featureIndex) = new Array[Bin](0) } } featureIndex += 1 } - - // Find all bins. - featureIndex = 0 - while (featureIndex < numFeatures) { - val isFeatureContinuous = metadata.isContinuous(featureIndex) - if (isFeatureContinuous) { // Bins for categorical variables are already assigned. - bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), - splits(featureIndex)(0), Continuous, Double.MinValue) - for (index <- 1 until numBins - 1) { - val bin = new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Continuous, Double.MinValue) - bins(featureIndex)(index) = bin - } - bins(featureIndex)(numBins-1) = new Bin(splits(featureIndex)(numBins-2), - new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) - } - featureIndex += 1 - } (splits, bins) case MinMax => throw new UnsupportedOperationException("minmax not supported yet.") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala new file mode 100644 index 0000000000000..866d85a79bea1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -0,0 +1,213 @@ +/* + * 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.mllib.tree.impl + +import org.apache.spark.mllib.tree.impurity._ + +/** + * DecisionTree statistics aggregator. + * This holds a flat array of statistics for a set of (nodes, features, bins) + * and helps with indexing. + */ +private[tree] class DTStatsAggregator( + val metadata: DecisionTreeMetadata, + val numNodes: Int) extends Serializable { + + /** + * [[ImpurityAggregator]] instance specifying the impurity type. + */ + val impurityAggregator: ImpurityAggregator = metadata.impurity match { + case Gini => new GiniAggregator(metadata.numClasses) + case Entropy => new EntropyAggregator(metadata.numClasses) + case Variance => new VarianceAggregator() + case _ => throw new IllegalArgumentException(s"Bad impurity parameter: ${metadata.impurity}") + } + + /** + * Number of elements (Double values) used for the sufficient statistics of each bin. + */ + val statsSize: Int = impurityAggregator.statsSize + + val numFeatures: Int = metadata.numFeatures + + /** + * Number of bins for each feature. This is indexed by the feature index. + */ + val numBins: Array[Int] = metadata.numBins + + /** + * Number of splits for the given feature. + */ + def numSplits(featureIndex: Int): Int = metadata.numSplits(featureIndex) + + /** + * Indicator for each feature of whether that feature is an unordered feature. + * TODO: Is Array[Boolean] any faster? + */ + def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) + + /** + * Offset for each feature for calculating indices into the [[allStats]] array. + */ + private val featureOffsets: Array[Int] = { + def featureOffsetsCalc(total: Int, featureIndex: Int): Int = { + if (isUnordered(featureIndex)) { + total + 2 * numBins(featureIndex) + } else { + total + numBins(featureIndex) + } + } + Range(0, numFeatures).scanLeft(0)(featureOffsetsCalc).map(statsSize * _).toArray + } + + /** + * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. + */ + private val nodeStride: Int = featureOffsets.last + + /** + * Total number of elements stored in this aggregator. + */ + val allStatsSize: Int = numNodes * nodeStride + + /** + * Flat array of elements. + * Index for start of stats for a (node, feature, bin) is: + * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, the left child stats have binIndex in [0, numBins(featureIndex)) + * and the right child stats in [numBins(featureIndex), 2 * numBins(featureIndex)) + */ + val allStats: Array[Double] = new Array[Double](allStatsSize) + + /** + * Get an [[ImpurityCalculator]] for a given (node, feature, bin). + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + */ + def getImpurityCalculator(nodeFeatureOffset: Int, binIndex: Int): ImpurityCalculator = { + impurityAggregator.getCalculator(allStats, nodeFeatureOffset + binIndex * statsSize) + } + + /** + * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + */ + def update(nodeIndex: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { + val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label) + } + + /** + * Pre-compute node offset for use with [[nodeUpdate]]. + */ + def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride + + /** + * Faster version of [[update]]. + * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. + */ + def nodeUpdate(nodeOffset: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { + val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label) + } + + /** + * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * For ordered features only. + */ + def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { + require(!isUnordered(featureIndex), + s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" + + s" for unordered feature $featureIndex.") + nodeIndex * nodeStride + featureOffsets(featureIndex) + } + + /** + * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * For unordered features only. + */ + def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = { + require(isUnordered(featureIndex), + s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + + s" but was called for ordered feature $featureIndex.") + val baseOffset = nodeIndex * nodeStride + featureOffsets(featureIndex) + (baseOffset, baseOffset + numBins(featureIndex) * statsSize) + } + + /** + * Faster version of [[update]]. + * Update the stats for a given (node, feature, bin), using the given label. + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + */ + def nodeFeatureUpdate(nodeFeatureOffset: Int, binIndex: Int, label: Double): Unit = { + impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label) + } + + /** + * For a given (node, feature), merge the stats for two bins. + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + * @param binIndex The other bin is merged into this bin. + * @param otherBinIndex This bin is not modified. + */ + def mergeForNodeFeature(nodeFeatureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { + impurityAggregator.merge(allStats, nodeFeatureOffset + binIndex * statsSize, + nodeFeatureOffset + otherBinIndex * statsSize) + } + + /** + * Merge this aggregator with another, and returns this aggregator. + * This method modifies this aggregator in-place. + */ + def merge(other: DTStatsAggregator): DTStatsAggregator = { + require(allStatsSize == other.allStatsSize, + s"DTStatsAggregator.merge requires that both aggregators have the same length stats vectors." + + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") + var i = 0 + // TODO: Test BLAS.axpy + while (i < allStatsSize) { + allStats(i) += other.allStats(i) + i += 1 + } + this + } + +} + +private[tree] object DTStatsAggregator extends Serializable { + + /** + * Combines two aggregates (modifying the first) and returns the combination. + */ + def binCombOp( + agg1: DTStatsAggregator, + agg2: DTStatsAggregator): DTStatsAggregator = { + agg1.merge(agg2) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index d9eda354dc986..e95add7558bcf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -26,14 +26,15 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.rdd.RDD - /** * Learning and dataset metadata for DecisionTree. * * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. * For regression: fixed at 0 (no meaning). + * @param maxBins Maximum number of bins, for all features. * @param featureArity Map: categorical feature index --> arity. * I.e., the feature takes values in {0, ..., arity - 1}. + * @param numBins Number of bins for each feature. */ private[tree] class DecisionTreeMetadata( val numFeatures: Int, @@ -42,6 +43,7 @@ private[tree] class DecisionTreeMetadata( val maxBins: Int, val featureArity: Map[Int, Int], val unorderedFeatures: Set[Int], + val numBins: Array[Int], val impurity: Impurity, val quantileStrategy: QuantileStrategy) extends Serializable { @@ -57,10 +59,26 @@ private[tree] class DecisionTreeMetadata( def isContinuous(featureIndex: Int): Boolean = !featureArity.contains(featureIndex) + /** + * Number of splits for the given feature. + * For unordered features, there are 2 bins per split. + * For ordered features, there is 1 more bin than split. + */ + def numSplits(featureIndex: Int): Int = if (isUnordered(featureIndex)) { + numBins(featureIndex) >> 1 + } else { + numBins(featureIndex) - 1 + } + } private[tree] object DecisionTreeMetadata { + /** + * Construct a [[DecisionTreeMetadata]] instance for this dataset and parameters. + * This computes which categorical features will be ordered vs. unordered, + * as well as the number of splits and bins for each feature. + */ def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { val numFeatures = input.take(1)(0).features.size @@ -70,32 +88,55 @@ private[tree] object DecisionTreeMetadata { case Regression => 0 } - val maxBins = math.min(strategy.maxBins, numExamples).toInt - val log2MaxBinsp1 = math.log(maxBins + 1) / math.log(2.0) + val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt + + // We check the number of bins here against maxPossibleBins. + // This needs to be checked here instead of in Strategy since maxPossibleBins can be modified + // based on the number of training examples. + if (strategy.categoricalFeaturesInfo.nonEmpty) { + val maxCategoriesPerFeature = strategy.categoricalFeaturesInfo.values.max + require(maxCategoriesPerFeature <= maxPossibleBins, + s"DecisionTree requires maxBins (= $maxPossibleBins) >= max categories " + + s"in categorical features (= $maxCategoriesPerFeature)") + } val unorderedFeatures = new mutable.HashSet[Int]() + val numBins = Array.fill[Int](numFeatures)(maxPossibleBins) if (numClasses > 2) { - strategy.categoricalFeaturesInfo.foreach { case (f, k) => - if (k - 1 < log2MaxBinsp1) { - // Note: The above check is equivalent to checking: - // numUnorderedBins = (1 << k - 1) - 1 < maxBins - unorderedFeatures.add(f) + // Multiclass classification + val maxCategoriesForUnorderedFeature = + ((math.log(maxPossibleBins / 2 + 1) / math.log(2.0)) + 1).floor.toInt + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + // Decide if some categorical features should be treated as unordered features, + // which require 2 * ((1 << numCategories - 1) - 1) bins. + // We do this check with log values to prevent overflows in case numCategories is large. + // The next check is equivalent to: 2 * ((1 << numCategories - 1) - 1) <= maxBins + if (numCategories <= maxCategoriesForUnorderedFeature) { + unorderedFeatures.add(featureIndex) + numBins(featureIndex) = numUnorderedBins(numCategories) } else { - // TODO: Allow this case, where we simply will know nothing about some categories? - require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + - s"in categorical features (>= $k)") + numBins(featureIndex) = numCategories } } } else { - strategy.categoricalFeaturesInfo.foreach { case (f, k) => - require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + - s"in categorical features (>= $k)") + // Binary classification or regression + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + numBins(featureIndex) = numCategories } } - new DecisionTreeMetadata(numFeatures, numExamples, numClasses, maxBins, - strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, + strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, strategy.impurity, strategy.quantileCalculationStrategy) } + /** + * Given the arity of a categorical feature (arity = number of categories), + * return the number of bins for the feature if it is to be treated as an unordered feature. + * There is 1 split for every partitioning of categories into 2 disjoint, non-empty sets; + * there are math.pow(2, arity - 1) - 1 such splits. + * Each split has 2 corresponding bins. + */ + def numUnorderedBins(arity: Int): Int = 2 * ((1 << arity - 1) - 1) + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala index 170e43e222083..35e361ae309cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -48,54 +48,63 @@ private[tree] object TreePoint { * binning feature values in preparation for DecisionTree training. * @param input Input dataset. * @param bins Bins for features, of size (numFeatures, numBins). - * @param metadata Learning and dataset metadata + * @param metadata Learning and dataset metadata * @return TreePoint dataset representation */ def convertToTreeRDD( input: RDD[LabeledPoint], bins: Array[Array[Bin]], metadata: DecisionTreeMetadata): RDD[TreePoint] = { + // Construct arrays for featureArity and isUnordered for efficiency in the inner loop. + val featureArity: Array[Int] = new Array[Int](metadata.numFeatures) + val isUnordered: Array[Boolean] = new Array[Boolean](metadata.numFeatures) + var featureIndex = 0 + while (featureIndex < metadata.numFeatures) { + featureArity(featureIndex) = metadata.featureArity.getOrElse(featureIndex, 0) + isUnordered(featureIndex) = metadata.isUnordered(featureIndex) + featureIndex += 1 + } input.map { x => - TreePoint.labeledPointToTreePoint(x, bins, metadata) + TreePoint.labeledPointToTreePoint(x, bins, featureArity, isUnordered) } } /** * Convert one LabeledPoint into its TreePoint representation. * @param bins Bins for features, of size (numFeatures, numBins). + * @param featureArity Array indexed by feature, with value 0 for continuous and numCategories + * for categorical features. + * @param isUnordered Array index by feature, with value true for unordered categorical features. */ private def labeledPointToTreePoint( labeledPoint: LabeledPoint, bins: Array[Array[Bin]], - metadata: DecisionTreeMetadata): TreePoint = { - + featureArity: Array[Int], + isUnordered: Array[Boolean]): TreePoint = { val numFeatures = labeledPoint.features.size - val numBins = bins(0).size val arr = new Array[Int](numFeatures) var featureIndex = 0 while (featureIndex < numFeatures) { - arr(featureIndex) = findBin(featureIndex, labeledPoint, metadata.isContinuous(featureIndex), - metadata.isUnordered(featureIndex), bins, metadata.featureArity) + arr(featureIndex) = findBin(featureIndex, labeledPoint, featureArity(featureIndex), + isUnordered(featureIndex), bins) featureIndex += 1 } - new TreePoint(labeledPoint.label, arr) } /** * Find bin for one (labeledPoint, feature). * + * @param featureArity 0 for continuous features; number of categories for categorical features. * @param isUnorderedFeature (only applies if feature is categorical) * @param bins Bins for features, of size (numFeatures, numBins). - * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity */ private def findBin( featureIndex: Int, labeledPoint: LabeledPoint, - isFeatureContinuous: Boolean, + featureArity: Int, isUnorderedFeature: Boolean, - bins: Array[Array[Bin]], - categoricalFeaturesInfo: Map[Int, Int]): Int = { + bins: Array[Array[Bin]]): Int = { /** * Binary search helper method for continuous feature. @@ -121,44 +130,7 @@ private[tree] object TreePoint { -1 } - /** - * Sequential search helper method to find bin for categorical feature in multiclass - * classification. The category is returned since each category can belong to multiple - * splits. The actual left/right child allocation per split is performed in the - * sequential phase of the bin aggregate operation. - */ - def sequentialBinSearchForUnorderedCategoricalFeatureInClassification(): Int = { - labeledPoint.features(featureIndex).toInt - } - - /** - * Sequential search helper method to find bin for categorical feature - * (for classification and regression). - */ - def sequentialBinSearchForOrderedCategoricalFeature(): Int = { - val featureCategories = categoricalFeaturesInfo(featureIndex) - val featureValue = labeledPoint.features(featureIndex) - var binIndex = 0 - while (binIndex < featureCategories) { - val bin = bins(featureIndex)(binIndex) - val categories = bin.highSplit.categories - if (categories.contains(featureValue)) { - return binIndex - } - binIndex += 1 - } - if (featureValue < 0 || featureValue >= featureCategories) { - throw new IllegalArgumentException( - s"DecisionTree given invalid data:" + - s" Feature $featureIndex is categorical with values in" + - s" {0,...,${featureCategories - 1}," + - s" but a data point gives it value $featureValue.\n" + - " Bad data point: " + labeledPoint.toString) - } - -1 - } - - if (isFeatureContinuous) { + if (featureArity == 0) { // Perform binary search for finding bin for continuous features. val binIndex = binarySearchForBins() if (binIndex == -1) { @@ -168,18 +140,17 @@ private[tree] object TreePoint { } binIndex } else { - // Perform sequential search to find bin for categorical features. - val binIndex = if (isUnorderedFeature) { - sequentialBinSearchForUnorderedCategoricalFeatureInClassification() - } else { - sequentialBinSearchForOrderedCategoricalFeature() - } - if (binIndex == -1) { - throw new RuntimeException("No bin was found for categorical feature." + - " This error can occur when given invalid data values (such as NaN)." + - s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") + // Categorical feature bins are indexed by feature values. + val featureValue = labeledPoint.features(featureIndex) + if (featureValue < 0 || featureValue >= featureArity) { + throw new IllegalArgumentException( + s"DecisionTree given invalid data:" + + s" Feature $featureIndex is categorical with values in" + + s" {0,...,${featureArity - 1}," + + s" but a data point gives it value $featureValue.\n" + + " Bad data point: " + labeledPoint.toString) } - binIndex + featureValue.toInt } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 96d2471e1f88c..1c8afc2d0f4bc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -74,3 +74,87 @@ object Entropy extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param numClasses Number of classes for label. + */ +private[tree] class EntropyAggregator(numClasses: Int) + extends ImpurityAggregator(numClasses) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"EntropyAggregator given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + allStats(offset + label.toInt) += 1 + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): EntropyCalculator = { + new EntropyCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[EntropyAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class EntropyCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: EntropyCalculator = new EntropyCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Entropy.calculate(stats, stats.sum) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats.sum.toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + indexOfLargestArrayElement(stats) + } + + /** + * Probability of the label given by [[predict]]. + */ + override def prob(label: Double): Double = { + val lbl = label.toInt + require(lbl < stats.length, + s"EntropyCalculator.prob given invalid label: $lbl (should be < ${stats.length}") + val cnt = count + if (cnt == 0) { + 0 + } else { + stats(lbl) / cnt + } + } + + override def toString: String = s"EntropyCalculator(stats = [${stats.mkString(", ")}])" + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index d586f449048bb..5cfdf345d163c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -70,3 +70,87 @@ object Gini extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param numClasses Number of classes for label. + */ +private[tree] class GiniAggregator(numClasses: Int) + extends ImpurityAggregator(numClasses) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"GiniAggregator given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + allStats(offset + label.toInt) += 1 + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): GiniCalculator = { + new GiniCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[GiniAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class GiniCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: GiniCalculator = new GiniCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Gini.calculate(stats, stats.sum) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats.sum.toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + indexOfLargestArrayElement(stats) + } + + /** + * Probability of the label given by [[predict]]. + */ + override def prob(label: Double): Double = { + val lbl = label.toInt + require(lbl < stats.length, + s"GiniCalculator.prob given invalid label: $lbl (should be < ${stats.length}") + val cnt = count + if (cnt == 0) { + 0 + } else { + stats(lbl) / cnt + } + } + + override def toString: String = s"GiniCalculator(stats = [${stats.mkString(", ")}])" + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 92b0c7b4a6fbc..5a047d6cb5480 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -22,6 +22,9 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: * Trait for calculating information gain. + * This trait is used for + * (a) setting the impurity parameter in [[org.apache.spark.mllib.tree.configuration.Strategy]] + * (b) calculating impurity values from sufficient statistics. */ @Experimental trait Impurity extends Serializable { @@ -47,3 +50,127 @@ trait Impurity extends Serializable { @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double } + +/** + * Interface for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param statsSize Length of the vector of sufficient statistics for one bin. + */ +private[tree] abstract class ImpurityAggregator(val statsSize: Int) extends Serializable { + + /** + * Merge the stats from one bin into another. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for (node, feature, bin) which is modified by the merge. + * @param otherOffset Start index of stats for (node, feature, other bin) which is not modified. + */ + def merge(allStats: Array[Double], offset: Int, otherOffset: Int): Unit = { + var i = 0 + while (i < statsSize) { + allStats(offset + i) += allStats(otherOffset + i) + i += 1 + } + } + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): ImpurityCalculator + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[ImpurityAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] abstract class ImpurityCalculator(val stats: Array[Double]) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: ImpurityCalculator + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double + + /** + * Add the stats from another calculator into this one, modifying and returning this calculator. + */ + def add(other: ImpurityCalculator): ImpurityCalculator = { + require(stats.size == other.stats.size, + s"Two ImpurityCalculator instances cannot be added with different counts sizes." + + s" Sizes are ${stats.size} and ${other.stats.size}.") + var i = 0 + while (i < other.stats.size) { + stats(i) += other.stats(i) + i += 1 + } + this + } + + /** + * Subtract the stats from another calculator from this one, modifying and returning this + * calculator. + */ + def subtract(other: ImpurityCalculator): ImpurityCalculator = { + require(stats.size == other.stats.size, + s"Two ImpurityCalculator instances cannot be subtracted with different counts sizes." + + s" Sizes are ${stats.size} and ${other.stats.size}.") + var i = 0 + while (i < other.stats.size) { + stats(i) -= other.stats(i) + i += 1 + } + this + } + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double + + /** + * Probability of the label given by [[predict]], or -1 if no probability is available. + */ + def prob(label: Double): Double = -1 + + /** + * Return the index of the largest array element. + * Fails if the array is empty. + */ + protected def indexOfLargestArrayElement(array: Array[Double]): Int = { + val result = array.foldLeft(-1, Double.MinValue, 0) { + case ((maxIndex, maxValue, currentIndex), currentValue) => + if (currentValue > maxValue) { + (currentIndex, currentValue, currentIndex + 1) + } else { + (maxIndex, maxValue, currentIndex + 1) + } + } + if (result._1 < 0) { + throw new RuntimeException("ImpurityCalculator internal error:" + + " indexOfLargestArrayElement failed") + } + result._1 + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index f7d99a40eb380..e9ccecb1b8067 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -61,3 +61,75 @@ object Variance extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + */ +private[tree] class VarianceAggregator() + extends ImpurityAggregator(statsSize = 3) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + allStats(offset) += 1 + allStats(offset + 1) += label + allStats(offset + 2) += label * label + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): VarianceCalculator = { + new VarianceCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[GiniAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class VarianceCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + require(stats.size == 3, + s"VarianceCalculator requires sufficient statistics array stats to be of length 3," + + s" but was given array of length ${stats.size}.") + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: VarianceCalculator = new VarianceCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Variance.calculate(stats(0), stats(1), stats(2)) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats(0).toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + stats(1) / count + } + + override def toString: String = { + s"VarianceAggregator(cnt = ${stats(0)}, sum = ${stats(1)}, sum2 = ${stats(2)})" + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index af35d88f713e5..0cad473782af1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.FeatureType._ /** - * Used for "binning" the features bins for faster best split calculation. + * Used for "binning" the feature values for faster best split calculation. * * For a continuous feature, the bin is determined by a low and a high split, * where an example with featureValue falls into the bin s.t. @@ -30,13 +30,16 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * bins, splits, and feature values. The bin is determined by category/feature value. * However, the bins are not necessarily ordered by feature value; * they are ordered using impurity. + * * For unordered categorical features, there is a 1-1 correspondence between bins, splits, * where bins and splits correspond to subsets of feature values (in highSplit.categories). + * An unordered feature with k categories uses (1 << k - 1) - 1 bins, corresponding to all + * partitionings of categories into 2 disjoint, non-empty sets. * * @param lowSplit signifying the lower threshold for the continuous feature to be * accepted in the bin * @param highSplit signifying the upper threshold for the continuous feature to be - * accepted in the bin + * accepted in the bin * @param featureType type of feature -- categorical or continuous * @param category categorical label value accepted in the bin for ordered features */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 0eee6262781c1..5b8a4cbed2306 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -24,8 +24,13 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * Node in a decision tree - * @param id integer node id + * Node in a decision tree. + * + * About node indexing: + * Nodes are indexed from 1. Node 1 is the root; nodes 2, 3 are the left, right children. + * Node index 0 is not used. + * + * @param id integer node id, from 1 * @param predict predicted value at the node * @param isLeaf whether the leaf is a node * @param split split to calculate left and right nodes @@ -51,17 +56,13 @@ class Node ( * @param nodes array of nodes */ def build(nodes: Array[Node]): Unit = { - - logDebug("building node " + id + " at level " + - (scala.math.log(id + 1)/scala.math.log(2)).toInt ) + logDebug("building node " + id + " at level " + Node.indexToLevel(id)) logDebug("id = " + id + ", split = " + split) logDebug("stats = " + stats) logDebug("predict = " + predict) if (!isLeaf) { - val leftNodeIndex = id * 2 + 1 - val rightNodeIndex = id * 2 + 2 - leftNode = Some(nodes(leftNodeIndex)) - rightNode = Some(nodes(rightNodeIndex)) + leftNode = Some(nodes(Node.leftChildIndex(id))) + rightNode = Some(nodes(Node.rightChildIndex(id))) leftNode.get.build(nodes) rightNode.get.build(nodes) } @@ -96,24 +97,20 @@ class Node ( * Get the number of nodes in tree below this node, including leaf nodes. * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. */ - private[tree] def numDescendants: Int = { - if (isLeaf) { - 0 - } else { - 2 + leftNode.get.numDescendants + rightNode.get.numDescendants - } + private[tree] def numDescendants: Int = if (isLeaf) { + 0 + } else { + 2 + leftNode.get.numDescendants + rightNode.get.numDescendants } /** * Get depth of tree from this node. * E.g.: Depth 0 means this is a leaf node. */ - private[tree] def subtreeDepth: Int = { - if (isLeaf) { - 0 - } else { - 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) - } + private[tree] def subtreeDepth: Int = if (isLeaf) { + 0 + } else { + 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) } /** @@ -148,3 +145,49 @@ class Node ( } } + +private[tree] object Node { + + /** + * Return the index of the left child of this node. + */ + def leftChildIndex(nodeIndex: Int): Int = nodeIndex << 1 + + /** + * Return the index of the right child of this node. + */ + def rightChildIndex(nodeIndex: Int): Int = (nodeIndex << 1) + 1 + + /** + * Get the parent index of the given node, or 0 if it is the root. + */ + def parentIndex(nodeIndex: Int): Int = nodeIndex >> 1 + + /** + * Return the level of a tree which the given node is in. + */ + def indexToLevel(nodeIndex: Int): Int = if (nodeIndex == 0) { + throw new IllegalArgumentException(s"0 is not a valid node index.") + } else { + java.lang.Integer.numberOfTrailingZeros(java.lang.Integer.highestOneBit(nodeIndex)) + } + + /** + * Returns true if this is a left child. + * Note: Returns false for the root. + */ + def isLeftChild(nodeIndex: Int): Boolean = nodeIndex > 1 && nodeIndex % 2 == 0 + + /** + * Return the maximum number of nodes which can be in the given level of the tree. + * @param level Level of tree (0 = root). + */ + def maxNodesInLevel(level: Int): Int = 1 << level + + /** + * Return the index of the first node in the given level. + * @param level Level of tree (0 = root). + */ + def startIndexInLevel(level: Int): Int = 1 << level + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 2f36fd907772c..8e556c917b2e7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -21,15 +21,16 @@ import scala.collection.JavaConverters._ import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.regression.LabeledPoint + class DecisionTreeSuite extends FunSuite with LocalSparkContext { @@ -59,12 +60,13 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") } - test("split and bin calculation") { + test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) @@ -72,7 +74,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) } - test("split and bin calculation for categorical variables") { + test("Binary classification with binary (ordered) categorical features:" + + " split and bin calculation") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -83,77 +86,20 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) assert(splits.length === 2) assert(bins.length === 2) - assert(splits(0).length === 99) - assert(bins(0).length === 100) - - // Check splits. - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(1.0)) - assert(splits(0)(1).categories.contains(0.0)) - - assert(splits(0)(2) === null) - - assert(splits(1)(0).feature === 1) - assert(splits(1)(0).threshold === Double.MinValue) - assert(splits(1)(0).featureType === Categorical) - assert(splits(1)(0).categories.length === 1) - assert(splits(1)(0).categories.contains(0.0)) - - assert(splits(1)(1).feature === 1) - assert(splits(1)(1).threshold === Double.MinValue) - assert(splits(1)(1).featureType === Categorical) - assert(splits(1)(1).categories.length === 2) - assert(splits(1)(1).categories.contains(1.0)) - assert(splits(1)(1).categories.contains(0.0)) - - assert(splits(1)(2) === null) - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - - assert(bins(0)(1).category === 0.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).lowSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(0.0)) - - assert(bins(0)(2) === null) - - assert(bins(1)(0).category === 0.0) - assert(bins(1)(0).lowSplit.categories.length === 0) - assert(bins(1)(0).highSplit.categories.length === 1) - assert(bins(1)(0).highSplit.categories.contains(0.0)) - - assert(bins(1)(1).category === 1.0) - assert(bins(1)(1).lowSplit.categories.length === 1) - assert(bins(1)(1).lowSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.length === 2) - assert(bins(1)(1).highSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.contains(1.0)) - - assert(bins(1)(2) === null) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } - test("split and bin calculations for categorical variables with no sample for one category") { + test("Binary classification with 3-ary (ordered) categorical features," + + " with no samples for one category") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -164,104 +110,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - - // Check splits. - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(1.0)) - assert(splits(0)(1).categories.contains(0.0)) - - assert(splits(0)(2).feature === 0) - assert(splits(0)(2).threshold === Double.MinValue) - assert(splits(0)(2).featureType === Categorical) - assert(splits(0)(2).categories.length === 3) - assert(splits(0)(2).categories.contains(1.0)) - assert(splits(0)(2).categories.contains(0.0)) - assert(splits(0)(2).categories.contains(2.0)) - - assert(splits(0)(3) === null) - - assert(splits(1)(0).feature === 1) - assert(splits(1)(0).threshold === Double.MinValue) - assert(splits(1)(0).featureType === Categorical) - assert(splits(1)(0).categories.length === 1) - assert(splits(1)(0).categories.contains(0.0)) - - assert(splits(1)(1).feature === 1) - assert(splits(1)(1).threshold === Double.MinValue) - assert(splits(1)(1).featureType === Categorical) - assert(splits(1)(1).categories.length === 2) - assert(splits(1)(1).categories.contains(1.0)) - assert(splits(1)(1).categories.contains(0.0)) - - assert(splits(1)(2).feature === 1) - assert(splits(1)(2).threshold === Double.MinValue) - assert(splits(1)(2).featureType === Categorical) - assert(splits(1)(2).categories.length === 3) - assert(splits(1)(2).categories.contains(1.0)) - assert(splits(1)(2).categories.contains(0.0)) - assert(splits(1)(2).categories.contains(2.0)) - - assert(splits(1)(3) === null) - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - - assert(bins(0)(1).category === 0.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).lowSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(0.0)) - - assert(bins(0)(2).category === 2.0) - assert(bins(0)(2).lowSplit.categories.length === 2) - assert(bins(0)(2).lowSplit.categories.contains(1.0)) - assert(bins(0)(2).lowSplit.categories.contains(0.0)) - assert(bins(0)(2).highSplit.categories.length === 3) - assert(bins(0)(2).highSplit.categories.contains(1.0)) - assert(bins(0)(2).highSplit.categories.contains(0.0)) - assert(bins(0)(2).highSplit.categories.contains(2.0)) - - assert(bins(0)(3) === null) - - assert(bins(1)(0).category === 0.0) - assert(bins(1)(0).lowSplit.categories.length === 0) - assert(bins(1)(0).highSplit.categories.length === 1) - assert(bins(1)(0).highSplit.categories.contains(0.0)) - - assert(bins(1)(1).category === 1.0) - assert(bins(1)(1).lowSplit.categories.length === 1) - assert(bins(1)(1).lowSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.length === 2) - assert(bins(1)(1).highSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.contains(1.0)) - - assert(bins(1)(2).category === 2.0) - assert(bins(1)(2).lowSplit.categories.length === 2) - assert(bins(1)(2).lowSplit.categories.contains(0.0)) - assert(bins(1)(2).lowSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.length === 3) - assert(bins(1)(2).highSplit.categories.contains(0.0)) - assert(bins(1)(2).highSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.contains(2.0)) - - assert(bins(1)(3) === null) + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } test("extract categories from a number for multiclass classification") { @@ -270,8 +128,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) } - test("split and bin calculations for unordered categorical variables with multiclass " + - "classification") { + test("Multiclass classification with unordered categorical features:" + + " split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -282,8 +140,15 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(bins.length === 2) + assert(splits(0).length === 3) + assert(bins(0).length === 6) // Expecting 2^2 - 1 = 3 bins/splits assert(splits(0)(0).feature === 0) @@ -321,10 +186,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(1)(2).categories.contains(0.0)) assert(splits(1)(2).categories.contains(1.0)) - assert(splits(0)(3) === null) - assert(splits(1)(3) === null) - - // Check bins. assert(bins(0)(0).category === Double.MinValue) @@ -360,13 +221,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(1)(2).highSplit.categories.contains(1.0)) assert(bins(1)(2).highSplit.categories.contains(0.0)) - assert(bins(0)(3) === null) - assert(bins(1)(3) === null) - } - test("split and bin calculations for ordered categorical variables with multiclass " + - "classification") { + test("Multiclass classification with ordered categorical features: split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() assert(arr.length === 3000) val rdd = sc.parallelize(arr) @@ -377,52 +234,21 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) + // 2^10 - 1 > 100, so categorical features will be ordered + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - - // 2^10 - 1 > 100, so categorical variables will be ordered - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(2.0)) - - assert(splits(0)(2).feature === 0) - assert(splits(0)(2).threshold === Double.MinValue) - assert(splits(0)(2).featureType === Categorical) - assert(splits(0)(2).categories.length === 3) - assert(splits(0)(2).categories.contains(2.0)) - assert(splits(0)(2).categories.contains(1.0)) - - assert(splits(0)(10) === null) - assert(splits(1)(10) === null) - - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - assert(bins(0)(1).category === 2.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(2.0)) - - assert(bins(0)(10) === null) - + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } - test("classification stump with all categorical variables") { + test("Binary classification stump with ordered categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -433,15 +259,23 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 - assert(split.categories.length === 1) - assert(split.categories.contains(1.0)) + assert(split.categories === List(1.0)) assert(split.featureType === Categorical) assert(split.threshold === Double.MinValue) @@ -452,7 +286,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.impurity > 0.2) } - test("regression stump with all categorical variables") { + test("Regression stump with 3-ary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -462,10 +296,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 @@ -480,7 +318,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.impurity > 0.2) } - test("regression stump with categorical variables of arity 2") { + test("Regression stump with binary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -490,6 +328,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) validateRegressor(model, arr, 0.0) @@ -497,12 +338,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(model.depth === 1) } - test("stump with fixed label 0 for Gini") { + test("Binary classification stump with fixed label 0 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val strategy = new Strategy(Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -512,7 +357,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -521,12 +366,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.rightImpurity === 0) } - test("stump with fixed label 1 for Gini") { + test("Binary classification stump with fixed label 1 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val strategy = new Strategy(Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -536,7 +385,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -546,12 +395,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 1) } - test("stump with fixed label 0 for Entropy") { + test("Binary classification stump with fixed label 0 for Entropy") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val strategy = new Strategy(Classification, Entropy, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -561,7 +414,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -571,12 +424,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 0) } - test("stump with fixed label 1 for Entropy") { + test("Binary classification stump with fixed label 1 for Entropy") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val strategy = new Strategy(Classification, Entropy, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -586,7 +443,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -596,7 +453,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 1) } - test("second level node building with/without groups") { + test("Second level node building with vs. without groups") { val arr = DecisionTreeSuite.generateOrderedLabeledPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -613,12 +470,12 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val nodes: Array[Node] = new Array[Node](7) - nodes(0) = modelOneNode.topNode - nodes(0).leftNode = None - nodes(0).rightNode = None + val nodes: Array[Node] = new Array[Node](8) + nodes(1) = modelOneNode.topNode + nodes(1).leftNode = None + nodes(1).rightNode = None - val parentImpurities = Array(0.5, 0.5, 0.5) + val parentImpurities = Array(0, 0.5, 0.5, 0.5) // Single group second level tree construction. val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) @@ -648,16 +505,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { } } - test("stump with categorical variables for multiclass classification") { + test("Multiclass classification stump with 3-ary (unordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(strategy.isMulticlassClassification) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -668,7 +528,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.featureType === Categorical) } - test("stump with 1 continuous variable for binary classification, to check off-by-1 error") { + test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) @@ -684,26 +544,27 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(model.depth === 1) } - test("stump with 2 continuous variables for binary classification") { + test("Binary classification stump with 2 continuous features") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 2) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) assert(model.topNode.split.get.feature === 1) } - test("stump with categorical variables for multiclass classification, with just enough bins") { - val maxBins = math.pow(2, 3 - 1).toInt // just enough bins to allow unordered features + test("Multiclass classification stump with unordered categorical features," + + " with just enough bins") { + val maxBins = 2 * (math.pow(2, 3 - 1).toInt - 1) // just enough bins to allow unordered features val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -711,6 +572,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) @@ -719,7 +582,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -733,7 +596,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(gain.rightImpurity === 0) } - test("stump with continuous variables for multiclass classification") { + test("Multiclass classification stump with continuous features") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -746,7 +609,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -759,20 +622,21 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { } - test("stump with continuous + categorical variables for multiclass classification") { + test("Multiclass classification stump with continuous + unordered categorical features") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 0.9) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -784,17 +648,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.threshold < 2020) } - test("stump with categorical variables for ordered multiclass classification") { + test("Multiclass classification stump with 10-ary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -805,6 +671,18 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.featureType === Categorical) } + test("Multiclass classification tree with 10-ary (ordered) categorical features," + + " with just enough bins") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, maxBins = 10, + categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + assert(strategy.isMulticlassClassification) + + val model = DecisionTree.train(rdd, strategy) + validateClassifier(model, arr, 0.6) + } } @@ -899,5 +777,4 @@ object DecisionTreeSuite { arr } - } From e16a8e7db5a3b1065b14baf89cb723a59b99226b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 8 Sep 2014 10:24:15 -0700 Subject: [PATCH 22/51] SPARK-3337 Paranoid quoting in shell to allow install dirs with spaces within. ... Tested ! TBH, it isn't a great idea to have directory with spaces within. Because emacs doesn't like it then hadoop doesn't like it. and so on... Author: Prashant Sharma Closes #2229 from ScrapCodes/SPARK-3337/quoting-shell-scripts and squashes the following commits: d4ad660 [Prashant Sharma] SPARK-3337 Paranoid quoting in shell to allow install dirs with spaces within. --- bin/beeline | 2 +- bin/compute-classpath.sh | 12 ++++++------ bin/load-spark-env.sh | 4 ++-- bin/pyspark | 20 ++++++++++---------- bin/run-example | 8 ++++---- bin/spark-class | 20 ++++++++++---------- bin/spark-shell | 10 +++++----- bin/spark-sql | 8 ++++---- bin/spark-submit | 4 ++-- dev/check-license | 16 ++++++++-------- dev/lint-python | 6 +++--- dev/mima | 4 ++-- dev/run-tests | 2 +- dev/scalastyle | 2 +- make-distribution.sh | 2 +- python/run-tests | 6 ++++-- sbin/slaves.sh | 12 ++++++------ sbin/spark-config.sh | 16 ++++++++-------- sbin/spark-daemon.sh | 20 ++++++++++---------- sbin/spark-executor | 8 ++++---- sbin/start-all.sh | 4 ++-- sbin/start-history-server.sh | 4 ++-- sbin/start-master.sh | 4 ++-- sbin/start-slave.sh | 4 ++-- sbin/start-slaves.sh | 12 ++++++------ sbin/start-thriftserver.sh | 8 ++++---- sbin/stop-all.sh | 4 ++-- sbin/stop-history-server.sh | 4 ++-- sbt/sbt | 20 ++++++++++---------- sbt/sbt-launch-lib.bash | 12 ++++++------ 30 files changed, 130 insertions(+), 128 deletions(-) diff --git a/bin/beeline b/bin/beeline index 1bda4dba50605..3fcb6df34339d 100755 --- a/bin/beeline +++ b/bin/beeline @@ -24,7 +24,7 @@ set -o posix # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.hive.beeline.BeeLine" exec "$FWDIR/bin/spark-class" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 16b794a1592e8..15c6779402994 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -23,9 +23,9 @@ SCALA_VERSION=2.10 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" @@ -63,7 +63,7 @@ else assembly_folder="$ASSEMBLY_DIR" fi -num_jars=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l) +num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l)" if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $assembly_folder" echo "You need to build Spark before running this program." @@ -77,7 +77,7 @@ if [ "$num_jars" -gt "1" ]; then exit 1 fi -ASSEMBLY_JAR=$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null) +ASSEMBLY_JAR="$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null)" # Verify that versions of java used to build the jars and run Spark are compatible jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) @@ -103,8 +103,8 @@ else datanucleus_dir="$FWDIR"/lib_managed/jars fi -datanucleus_jars=$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar") -datanucleus_jars=$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g) +datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar")" +datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" if [ -n "$datanucleus_jars" ]; then hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 493d3785a081b..6d4231b204595 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -25,9 +25,9 @@ if [ -z "$SPARK_ENV_LOADED" ]; then export SPARK_ENV_LOADED=1 # Returns the parent of the directory this script lives in. - parent_dir="$(cd `dirname $0`/..; pwd)" + parent_dir="$(cd "`dirname "$0"`"/..; pwd)" - user_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} + user_conf_dir="${SPARK_CONF_DIR:-"$parent_dir"/conf}" if [ -f "${user_conf_dir}/spark-env.sh" ]; then # Promote all variable declarations to environment (exported) variables diff --git a/bin/pyspark b/bin/pyspark index 26a16dd600b7a..5142411e36974 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -18,18 +18,18 @@ # # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -source $FWDIR/bin/utils.sh +source "$FWDIR/bin/utils.sh" SCALA_VERSION=2.10 function usage() { echo "Usage: ./bin/pyspark [options]" 1>&2 - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 } @@ -48,7 +48,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then fi fi -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh # Figure out which Python executable to use if [[ -z "$PYSPARK_PYTHON" ]]; then @@ -57,12 +57,12 @@ fi export PYSPARK_PYTHON # Add the PySpark classes to the Python path: -export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$SPARK_HOME/python/:$PYTHONPATH" +export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: -export OLD_PYTHONSTARTUP=$PYTHONSTARTUP -export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py +export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" +export PYTHONSTARTUP="$FWDIR/python/pyspark/shell.py" # If IPython options are specified, assume user wants to run IPython if [[ -n "$IPYTHON_OPTS" ]]; then @@ -99,10 +99,10 @@ fi if [[ "$1" =~ \.py$ ]]; then echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 echo -e "Use ./bin/spark-submit \n" 1>&2 - primary=$1 + primary="$1" shift gatherSparkSubmitOpts "$@" - exec $FWDIR/bin/spark-submit "${SUBMISSION_OPTS[@]}" $primary "${APPLICATION_OPTS[@]}" + exec "$FWDIR"/bin/spark-submit "${SUBMISSION_OPTS[@]}" "$primary" "${APPLICATION_OPTS[@]}" else # PySpark shell requires special handling downstream export PYSPARK_SHELL=1 diff --git a/bin/run-example b/bin/run-example index 68a35702eddd3..34dd71c71880e 100755 --- a/bin/run-example +++ b/bin/run-example @@ -19,7 +19,7 @@ SCALA_VERSION=2.10 -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" export SPARK_HOME="$FWDIR" EXAMPLES_DIR="$FWDIR"/examples @@ -35,12 +35,12 @@ else fi if [ -f "$FWDIR/RELEASE" ]; then - export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` + export SPARK_EXAMPLES_JAR="`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar`" elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar` + export SPARK_EXAMPLES_JAR="`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar`" fi -if [[ -z $SPARK_EXAMPLES_JAR ]]; then +if [[ -z "$SPARK_EXAMPLES_JAR" ]]; then echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" 1>&2 echo "You need to build Spark before running this program" 1>&2 exit 1 diff --git a/bin/spark-class b/bin/spark-class index c6543545a5e64..5f5f9ea74888d 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -27,12 +27,12 @@ esac SCALA_VERSION=2.10 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh if [ -z "$1" ]; then echo "Usage: spark-class []" 1>&2 @@ -105,7 +105,7 @@ else exit 1 fi fi -JAVA_VERSION=$($RUNNER -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') +JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size if [ "$JAVA_VERSION" -ge 18 ]; then @@ -117,7 +117,7 @@ JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" + JAVA_OPTS="$JAVA_OPTS `cat "$FWDIR"/conf/java-opts`" fi # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! @@ -126,21 +126,21 @@ TOOLS_DIR="$FWDIR"/tools SPARK_TOOLS_JAR="" if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the SBT build - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar` + export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar`" fi if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the Maven build # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` + export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`" fi # Compute classpath using external script -classpath_output=$($FWDIR/bin/compute-classpath.sh) +classpath_output=$("$FWDIR"/bin/compute-classpath.sh) if [[ "$?" != "0" ]]; then echo "$classpath_output" exit 1 else - CLASSPATH=$classpath_output + CLASSPATH="$classpath_output" fi if [[ "$1" =~ org.apache.spark.tools.* ]]; then @@ -153,9 +153,9 @@ if [[ "$1" =~ org.apache.spark.tools.* ]]; then fi if $cygwin; then - CLASSPATH=`cygpath -wp $CLASSPATH` + CLASSPATH="`cygpath -wp "$CLASSPATH"`" if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR` + export SPARK_TOOLS_JAR="`cygpath -w "$SPARK_TOOLS_JAR"`" fi fi export CLASSPATH diff --git a/bin/spark-shell b/bin/spark-shell index 0ab4e14f5b744..4a0670fc6c8aa 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -29,11 +29,11 @@ esac set -o posix ## Global script variables -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage() { echo "Usage: ./bin/spark-shell [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 } @@ -41,7 +41,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" @@ -54,11 +54,11 @@ function main() { # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" fi } diff --git a/bin/spark-sql b/bin/spark-sql index 2a3cb31f58e8d..ae096530cad04 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -27,7 +27,7 @@ CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage { echo "Usage: ./bin/spark-sql [options] [cli option]" @@ -38,10 +38,10 @@ function usage { pattern+="\|--help" pattern+="\|=======" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "CLI options:" - $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -49,7 +49,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 277c4ce571ca2..c557311b4b20e 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,7 +19,7 @@ # NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! -export SPARK_HOME="$(cd `dirname $0`/..; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" ORIG_ARGS=("$@") while (($#)); do @@ -59,5 +59,5 @@ if [[ "$SPARK_SUBMIT_DEPLOY_MODE" == "client" && -f "$SPARK_SUBMIT_PROPERTIES_FI fi fi -exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" +exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/dev/check-license b/dev/check-license index 558e038afc01a..9ff0929e9a5e8 100755 --- a/dev/check-license +++ b/dev/check-license @@ -23,16 +23,16 @@ acquire_rat_jar () { URL1="http://search.maven.org/remotecontent?filepath=org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" URL2="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" - JAR=$rat_jar + JAR="$rat_jar" if [[ ! -f "$rat_jar" ]]; then # Download rat launch jar if it hasn't been downloaded yet if [ ! -f "$JAR" ]; then # Download printf "Attempting to fetch rat\n" - JAR_DL=${JAR}.part + JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent ${URL1} > "$JAR_DL" || curl --silent ${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 --quiet ${URL1} -O "$JAR_DL" || wget --quiet ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" else @@ -50,7 +50,7 @@ acquire_rat_jar () { } # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" if test -x "$JAVA_HOME/bin/java"; then @@ -60,17 +60,17 @@ else fi export RAT_VERSION=0.10 -export rat_jar=$FWDIR/lib/apache-rat-${RAT_VERSION}.jar -mkdir -p $FWDIR/lib +export rat_jar="$FWDIR"/lib/apache-rat-${RAT_VERSION}.jar +mkdir -p "$FWDIR"/lib [[ -f "$rat_jar" ]] || acquire_rat_jar || { echo "Download failed. Obtain the rat jar manually and place it at $rat_jar" exit 1 } -$java_cmd -jar $rat_jar -E $FWDIR/.rat-excludes -d $FWDIR > rat-results.txt +$java_cmd -jar "$rat_jar" -E "$FWDIR"/.rat-excludes -d "$FWDIR" > rat-results.txt -ERRORS=$(cat rat-results.txt | grep -e "??") +ERRORS="$(cat rat-results.txt | grep -e "??")" if test ! -z "$ERRORS"; then echo "Could not find Apache license headers in the following files:" diff --git a/dev/lint-python b/dev/lint-python index 79bf70f0b8b13..772f856154ae0 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -18,10 +18,10 @@ # SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" -SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" +SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")" PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" -cd $SPARK_ROOT_DIR +cd "$SPARK_ROOT_DIR" # Get pep8 at runtime so that we don't rely on it being installed on the build server. #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 @@ -45,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 $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" +python "$PEP8_SCRIPT_PATH" $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" pep8_status=${PIPESTATUS[0]} #$? if [ $pep8_status -ne 0 ]; then diff --git a/dev/mima b/dev/mima index 09e4482af5f3d..f9b9b03538f15 100755 --- a/dev/mima +++ b/dev/mima @@ -21,12 +21,12 @@ set -o pipefail set -e # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" echo -e "q\n" | sbt/sbt oldDeps/update -export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"` +export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore diff --git a/dev/run-tests b/dev/run-tests index 90a8ce16f0f06..49a88085c80f7 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -18,7 +18,7 @@ # # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then diff --git a/dev/scalastyle b/dev/scalastyle index eb9b467965636..efb5f291ea3b7 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -19,7 +19,7 @@ echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt # Check style with YARN built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ diff --git a/make-distribution.sh b/make-distribution.sh index 14aed4a4b655b..9b012b9222db4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -28,7 +28,7 @@ set -o pipefail set -e # Figure out where the Spark framework is installed -FWDIR="$(cd `dirname $0`; pwd)" +FWDIR="$(cd "`dirname "$0"`"; pwd)" DISTDIR="$FWDIR/dist" SPARK_TACHYON=false diff --git a/python/run-tests b/python/run-tests index f2a80b4f1838b..226e9e2c3770a 100755 --- a/python/run-tests +++ b/python/run-tests @@ -19,7 +19,7 @@ # Figure out where the Spark framework is installed -FWDIR="$(cd `dirname $0`; cd ../; pwd)" +FWDIR="$(cd "`dirname "$0"`"; cd ../; pwd)" # CD into the python directory to find things on the right path cd "$FWDIR/python" @@ -33,7 +33,9 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" - SPARK_TESTING=1 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log + + SPARK_TESTING=1 "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log + FAILED=$((PIPESTATUS[0]||$FAILED)) # Fail and exit on the first test failure. diff --git a/sbin/slaves.sh b/sbin/slaves.sh index f89547fef9e46..1d4dc5edf9858 100755 --- a/sbin/slaves.sh +++ b/sbin/slaves.sh @@ -36,29 +36,29 @@ if [ $# -le 0 ]; then exit 1 fi -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" # If the slaves file is specified in the command line, # then it takes precedence over the definition in # spark-env.sh. Save it here. -HOSTLIST=$SPARK_SLAVES +HOSTLIST="$SPARK_SLAVES" # Check if --config is passed as an argument. It is an optional parameter. # Exit if the argument is not a directory. if [ "$1" == "--config" ] then shift - conf_dir=$1 + conf_dir="$1" if [ ! -d "$conf_dir" ] then echo "ERROR : $conf_dir is not a directory" echo $usage exit 1 else - export SPARK_CONF_DIR=$conf_dir + export SPARK_CONF_DIR="$conf_dir" fi shift fi @@ -79,7 +79,7 @@ if [ "$SPARK_SSH_OPTS" = "" ]; then fi for slave in `cat "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do - ssh $SPARK_SSH_OPTS $slave $"${@// /\\ }" \ + ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ 2>&1 | sed "s/^/$slave: /" & if [ "$SPARK_SLAVE_SLEEP" != "" ]; then sleep $SPARK_SLAVE_SLEEP diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 5c87da5815b64..2718d6cba1c9a 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -21,19 +21,19 @@ # resolve links - $0 may be a softlink this="${BASH_SOURCE-$0}" -common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P) +common_bin="$(cd -P -- "$(dirname -- "$this")" && pwd -P)" script="$(basename -- "$this")" this="$common_bin/$script" # convert relative path to absolute path -config_bin=`dirname "$this"` -script=`basename "$this"` -config_bin=`cd "$config_bin"; pwd` +config_bin="`dirname "$this"`" +script="`basename "$this"`" +config_bin="`cd "$config_bin"; pwd`" this="$config_bin/$script" -export SPARK_PREFIX=`dirname "$this"`/.. -export SPARK_HOME=${SPARK_PREFIX} +export SPARK_PREFIX="`dirname "$this"`"/.. +export SPARK_HOME="${SPARK_PREFIX}" export SPARK_CONF_DIR="$SPARK_HOME/conf" # Add the PySpark classes to the PYTHONPATH: -export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$SPARK_HOME/python:$PYTHONPATH" +export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 9032f23ea8eff..bd476b400e1c3 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -37,8 +37,8 @@ if [ $# -le 1 ]; then exit 1 fi -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" @@ -50,14 +50,14 @@ sbin=`cd "$sbin"; pwd` if [ "$1" == "--config" ] then shift - conf_dir=$1 + conf_dir="$1" if [ ! -d "$conf_dir" ] then echo "ERROR : $conf_dir is not a directory" echo $usage exit 1 else - export SPARK_CONF_DIR=$conf_dir + export SPARK_CONF_DIR="$conf_dir" fi shift fi @@ -100,12 +100,12 @@ if [ "$SPARK_LOG_DIR" = "" ]; then export SPARK_LOG_DIR="$SPARK_HOME/logs" fi mkdir -p "$SPARK_LOG_DIR" -touch $SPARK_LOG_DIR/.spark_test > /dev/null 2>&1 +touch "$SPARK_LOG_DIR"/.spark_test > /dev/null 2>&1 TEST_LOG_DIR=$? if [ "${TEST_LOG_DIR}" = "0" ]; then - rm -f $SPARK_LOG_DIR/.spark_test + rm -f "$SPARK_LOG_DIR"/.spark_test else - chown $SPARK_IDENT_STRING $SPARK_LOG_DIR + chown "$SPARK_IDENT_STRING" "$SPARK_LOG_DIR" fi if [ "$SPARK_PID_DIR" = "" ]; then @@ -113,8 +113,8 @@ if [ "$SPARK_PID_DIR" = "" ]; then fi # some variables -log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out -pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid +log="$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out" +pid="$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid" # Set default scheduling priority if [ "$SPARK_NICENESS" = "" ]; then @@ -136,7 +136,7 @@ case $startStop in fi if [ "$SPARK_MASTER" != "" ]; then - echo rsync from $SPARK_MASTER + echo rsync from "$SPARK_MASTER" rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" fi diff --git a/sbin/spark-executor b/sbin/spark-executor index 3621321a9bc8d..674ce906d9421 100755 --- a/sbin/spark-executor +++ b/sbin/spark-executor @@ -17,10 +17,10 @@ # limitations under the License. # -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -export PYTHONPATH=$FWDIR/python:$PYTHONPATH -export PYTHONPATH=$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$FWDIR/python:$PYTHONPATH" +export PYTHONPATH="$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend +exec "$FWDIR"/bin/spark-class org.apache.spark.executor.MesosExecutorBackend diff --git a/sbin/start-all.sh b/sbin/start-all.sh index 5c89ab4d86b3a..1baf57cea09ee 100755 --- a/sbin/start-all.sh +++ b/sbin/start-all.sh @@ -21,8 +21,8 @@ # Starts the master on this node. # Starts a worker on each node specified in conf/slaves -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" TACHYON_STR="" diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 580ab471b8a79..7172ad15d88fc 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -24,8 +24,8 @@ # Use the SPARK_HISTORY_OPTS environment variable to set history server configuration. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" . "$SPARK_PREFIX/bin/load-spark-env.sh" diff --git a/sbin/start-master.sh b/sbin/start-master.sh index c5c02491f78e1..17fff58f4f768 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -19,8 +19,8 @@ # Starts the master on the machine this script is executed on. -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" START_TACHYON=false diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index b563400dc24f3..2fc35309f4ca5 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -20,7 +20,7 @@ # Usage: start-slave.sh # where is like "spark://localhost:7077" -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 4912d0c0c7dfd..ba1a84abc1fef 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" START_TACHYON=false @@ -46,11 +46,11 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_IP" = "" ]; then - SPARK_MASTER_IP=`hostname` + SPARK_MASTER_IP="`hostname`" fi if [ "$START_TACHYON" == "true" ]; then - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon bootstrap-conf $SPARK_MASTER_IP + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon bootstrap-conf "$SPARK_MASTER_IP" # set -t so we can call sudo SPARK_SSH_OPTS="-o StrictHostKeyChecking=no -t" "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/../tachyon/bin/tachyon-start.sh" worker SudoMount \; sleep 1 @@ -58,12 +58,12 @@ fi # Launch the slaves if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT + exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" else if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then SPARK_WORKER_WEBUI_PORT=8081 fi for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) done fi diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index c519a77df4a14..4ce40fe750384 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -24,7 +24,7 @@ set -o posix # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" CLASS_NOT_FOUND_EXIT_STATUS=1 @@ -38,10 +38,10 @@ function usage { pattern+="\|=======" pattern+="\|--help" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "Thrift server options:" - $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -49,7 +49,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh index 60b358d374565..298c6a9859795 100755 --- a/sbin/stop-all.sh +++ b/sbin/stop-all.sh @@ -21,8 +21,8 @@ # Run this on the master nde -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" # Load the Spark configuration . "$sbin/spark-config.sh" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh index c0034ad641cbe..6e6056359510f 100755 --- a/sbin/stop-history-server.sh +++ b/sbin/stop-history-server.sh @@ -19,7 +19,7 @@ # Stops the history server on the machine this script is executed on. -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 diff --git a/sbt/sbt b/sbt/sbt index 1b1aa1483a829..c172fa74bc771 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -3,32 +3,32 @@ # When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so # that we can run Hive to generate the golden answer. This is not required for normal development # or testing. -for i in $HIVE_HOME/lib/* -do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i +for i in "$HIVE_HOME"/lib/* +do HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$i" done export HADOOP_CLASSPATH realpath () { ( - TARGET_FILE=$1 + TARGET_FILE="$1" - cd $(dirname $TARGET_FILE) - TARGET_FILE=$(basename $TARGET_FILE) + cd "$(dirname "$TARGET_FILE")" + TARGET_FILE="$(basename "$TARGET_FILE")" COUNT=0 while [ -L "$TARGET_FILE" -a $COUNT -lt 100 ] do - TARGET_FILE=$(readlink $TARGET_FILE) - cd $(dirname $TARGET_FILE) - TARGET_FILE=$(basename $TARGET_FILE) + TARGET_FILE="$(readlink "$TARGET_FILE")" + cd $(dirname "$TARGET_FILE") + TARGET_FILE="$(basename $TARGET_FILE)" COUNT=$(($COUNT + 1)) done - echo $(pwd -P)/$TARGET_FILE + echo "$(pwd -P)/"$TARGET_FILE"" ) } -. $(dirname $(realpath $0))/sbt-launch-lib.bash +. "$(dirname "$(realpath "$0")")"/sbt-launch-lib.bash declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index fecc3d38a5fbd..7f05d2ef491a3 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -7,7 +7,7 @@ # TODO - Should we merge the main SBT script with this library? if test -z "$HOME"; then - declare -r script_dir="$(dirname $script_path)" + declare -r script_dir="$(dirname "$script_path")" else declare -r script_dir="$HOME/.sbt" fi @@ -46,20 +46,20 @@ acquire_sbt_jar () { if [[ ! -f "$sbt_jar" ]]; then # Download sbt launch jar if it hasn't been downloaded yet - if [ ! -f ${JAR} ]; then + if [ ! -f "${JAR}" ]; then # Download printf "Attempting to fetch sbt\n" - JAR_DL=${JAR}.part + JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent ${URL1} > ${JAR_DL} || curl --silent ${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 --quiet ${URL1} -O ${JAR_DL} || wget --quiet ${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 fi fi - if [ ! -f ${JAR} ]; then + if [ ! -f "${JAR}" ]; then # We failed to download printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" exit -1 From 16a73c2473181e03d88001aa3e08e6ffac92eb8b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 8 Sep 2014 11:20:00 -0700 Subject: [PATCH 23/51] SPARK-2978. Transformation with MR shuffle semantics I didn't add this to the transformations list in the docs because it's kind of obscure, but would be happy to do so if others think it would be helpful. Author: Sandy Ryza Closes #2274 from sryza/sandy-spark-2978 and squashes the following commits: 4a5332a [Sandy Ryza] Fix Java test c04b447 [Sandy Ryza] Fix Python doc and add back deleted code 433ad5b [Sandy Ryza] Add Java test 4c25a54 [Sandy Ryza] Add s at the end and a couple other fixes 9b0ba99 [Sandy Ryza] Fix compilation 36e0571 [Sandy Ryza] Fix import ordering 48c12c2 [Sandy Ryza] Add Java version and additional doc e5381cd [Sandy Ryza] Fix python style warnings f147634 [Sandy Ryza] SPARK-2978. Transformation with MR shuffle semantics --- .../apache/spark/api/java/JavaPairRDD.scala | 26 ++++++++++++++++ .../spark/rdd/OrderedRDDFunctions.scala | 14 ++++++++- .../java/org/apache/spark/JavaAPISuite.java | 30 +++++++++++++++++++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 14 +++++++++ python/pyspark/rdd.py | 24 +++++++++++++++ python/pyspark/tests.py | 8 +++++ 6 files changed, 115 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index feeb6c02caa78..880f61c49726e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -758,6 +758,32 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) rdd.saveAsHadoopDataset(conf) } + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner): JavaPairRDD[K, V] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]] + repartitionAndSortWithinPartitions(partitioner, comp) + } + + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner, comp: Comparator[K]) + : JavaPairRDD[K, V] = { + implicit val ordering = comp // Allow implicit conversion of Comparator to Ordering. + fromRDD( + new OrderedRDDFunctions[K, V, (K, V)](rdd).repartitionAndSortWithinPartitions(partitioner)) + } + /** * Sort the RDD by key, so that each partition contains a sorted range of the elements in * ascending order. Calling `collect` or `save` on the resulting RDD will return or output an diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index e98bad2026e32..d0dbfef35d03c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Logging, RangePartitioner} +import org.apache.spark.{Logging, Partitioner, RangePartitioner} import org.apache.spark.annotation.DeveloperApi /** @@ -64,4 +64,16 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, new ShuffledRDD[K, V, V](self, part) .setKeyOrdering(if (ascending) ordering else ordering.reverse) } + + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = { + new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering) + } + } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e1c13de04a0be..be99dc501c4b2 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -189,6 +189,36 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @Test + public void repartitionAndSortWithinPartitions() { + List> pairs = new ArrayList>(); + pairs.add(new Tuple2(0, 5)); + pairs.add(new Tuple2(3, 8)); + pairs.add(new Tuple2(2, 6)); + pairs.add(new Tuple2(0, 8)); + pairs.add(new Tuple2(3, 8)); + pairs.add(new Tuple2(1, 3)); + + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + Partitioner partitioner = new Partitioner() { + public int numPartitions() { + return 2; + } + public int getPartition(Object key) { + return ((Integer)key).intValue() % 2; + } + }; + + JavaPairRDD repartitioned = + rdd.repartitionAndSortWithinPartitions(partitioner); + List>> partitions = repartitioned.glom().collect(); + Assert.assertEquals(partitions.get(0), Arrays.asList(new Tuple2(0, 5), + new Tuple2(0, 8), new Tuple2(2, 6))); + Assert.assertEquals(partitions.get(1), Arrays.asList(new Tuple2(1, 3), + new Tuple2(3, 8), new Tuple2(3, 8))); + } + @Test public void emptyRDD() { JavaRDD rdd = sc.emptyRDD(); 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 499dcda3dae8f..c1b501a75c8b8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -682,6 +682,20 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(data.sortBy(parse, true, 2)(NameOrdering, classTag[Person]).collect() === nameOrdered) } + test("repartitionAndSortWithinPartitions") { + val data = sc.parallelize(Seq((0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)), 2) + + val partitioner = new Partitioner { + def numPartitions: Int = 2 + def getPartition(key: Any): Int = key.asInstanceOf[Int] % 2 + } + + val repartitioned = data.repartitionAndSortWithinPartitions(partitioner) + val partitions = repartitioned.glom().collect() + assert(partitions(0) === Seq((0, 5), (0, 8), (2, 6))) + assert(partitions(1) === Seq((1, 3), (3, 8), (3, 8))) + } + test("intersection") { val all = sc.parallelize(1 to 10) val evens = sc.parallelize(2 to 10 by 2) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 266090e3ae8f3..5667154cb84a8 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -520,6 +520,30 @@ def __add__(self, other): raise TypeError return self.union(other) + def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=portable_hash, + ascending=True, keyfunc=lambda x: x): + """ + Repartition the RDD according to the given partitioner and, within each resulting partition, + sort records by their keys. + + >>> rdd = sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)]) + >>> rdd2 = rdd.repartitionAndSortWithinPartitions(2, lambda x: x % 2, 2) + >>> rdd2.glom().collect() + [[(0, 5), (0, 8), (2, 6)], [(1, 3), (3, 8), (3, 8)]] + """ + if numPartitions is None: + numPartitions = self._defaultReducePartitions() + + spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == "true") + memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + serializer = self._jrdd_deserializer + + def sortPartition(iterator): + sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted + return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + + return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True) + def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): """ Sorts this RDD, which is assumed to consist of (key, value) pairs. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 9fbeb36f4f1dd..0bd2a9e6c507d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -545,6 +545,14 @@ def test_histogram(self): self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) self.assertRaises(TypeError, lambda: rdd.histogram(2)) + def test_repartitionAndSortWithinPartitions(self): + rdd = self.sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)], 2) + + repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2) + partitions = repartitioned.glom().collect() + self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) + self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + class TestSQL(PySparkTestCase): From 386bc24ebe3e75875b9647d9223c62d7b9dc9963 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Mon, 8 Sep 2014 12:37:52 -0700 Subject: [PATCH 24/51] Provide a default PYSPARK_PYTHON for python/run_tests Without this the version of python used in the test is not recorded. The error is, Testing with Python version: ./run-tests: line 57: --version: command not found Author: Matthew Farrellee Closes #2300 from mattf/master-fix-python-run-tests and squashes the following commits: 65a09f5 [Matthew Farrellee] Provide a default PYSPARK_PYTHON for python/run_tests --- python/run-tests | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/run-tests b/python/run-tests index 226e9e2c3770a..d98840de59d2c 100755 --- a/python/run-tests +++ b/python/run-tests @@ -50,6 +50,8 @@ function run_test() { echo "Running PySpark tests. Output is in python/unit-tests.log." +export PYSPARK_PYTHON="python" + # Try to test with Python 2.6, since that's the minimum version that we support: if [ $(which python2.6) ]; then export PYSPARK_PYTHON="python2.6" From 26bc7655de18ab0191ded3f75cb77bc756dc1c03 Mon Sep 17 00:00:00 2001 From: Henry Cook Date: Mon, 8 Sep 2014 14:56:37 -0700 Subject: [PATCH 25/51] [SQL] Minor edits to sql programming guide. Author: Henry Cook Closes #2316 from hcook/sql-docs and squashes the following commits: 373f94b [Henry Cook] Minor edits to sql programming guide. --- docs/sql-programming-guide.md | 92 ++++++++++++++++++----------------- 1 file changed, 47 insertions(+), 45 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 1814fef465cac..d83efa4bab324 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -13,10 +13,10 @@ title: Spark SQL Programming Guide Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed -[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with +[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of +[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects, along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`. @@ -26,10 +26,10 @@ All of the examples on this page use sample data included in the Spark distribut
Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed -[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects along with +[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed of +[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects, along with a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table -in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
@@ -37,10 +37,10 @@ file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive]( Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed -[Row](api/python/pyspark.sql.Row-class.html) objects along with +[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed of +[Row](api/python/pyspark.sql.Row-class.html) objects, along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell. @@ -68,11 +68,11 @@ 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 +In addition to the basic SQLContext, you can also create a HiveContext, which provides a +superset 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. +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 @@ -95,7 +95,7 @@ In addition to the basic SQLContext, you can also create a HiveContext, which pr 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. +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 @@ -118,7 +118,7 @@ In addition to the basic SQLContext, you can also create a HiveContext, which pr 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. +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 @@ -146,11 +146,11 @@ describes the various methods for loading data into a SchemaRDD. 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. +reflection based approach leads to more concise code and works well when you already know the schema +while 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 +construct a schema and then apply it to an 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 @@ -266,10 +266,10 @@ List teenagerNames = teenagers.map(new Function() {
-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, +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 column 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 +row, it is important that there is no missing data in the first row of the RDD. In future versions we plan to more completely infer the schema by looking at more data, similar to the inference that is performed on JSON files. @@ -306,14 +306,14 @@ for teenName in teenNames.collect():
-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 +When 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. +`Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `applySchema` method provided by `SQLContext`. @@ -358,14 +358,14 @@ results.map(t => "Name: " + t(0)).collect().foreach(println)
-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 +When 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. +`Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `applySchema` method provided by `JavaSQLContext`. @@ -427,10 +427,10 @@ List names = results.map(new Function() {
-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. +When a dictionary of kwargs 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 tuples or lists from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of @@ -566,7 +566,7 @@ for teenName in teenNames.collect(): ### Configuration -Configuration of parquet can be done using the `setConf` method on SQLContext or by running +Configuration of Parquet can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -575,8 +575,8 @@ Configuration of parquet can be done using the `setConf` method on SQLContext or @@ -584,14 +584,14 @@ Configuration of parquet can be done using the `setConf` method on SQLContext or @@ -805,9 +805,8 @@ Spark SQL can cache tables using an in-memory columnar format by calling `cacheT 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. -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. +Note that if you call `cache` rather than `cacheTable`, tables will _not_ be cached using +the in-memory columnar format, and therefore `cacheTable` is strongly recommended for this use case. Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -833,7 +832,7 @@ Configuration of in-memory caching can be done using the `setConf` method on SQL
spark.sql.parquet.binaryAsString false - 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 + 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.cacheMetadata false - Turns on caching of parquet schema metadata. Can speed up querying + Turns on caching of Parquet schema metadata. Can speed up querying of static data.
spark.sql.parquet.compression.codec snappy - Sets the compression codec use when writing parquet files. Acceptable values include: + Sets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo.
-## Other Configuration +## Other Configuration Options 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. @@ -842,7 +841,7 @@ that these options will be deprecated in future release as more optimizations ar Property NameDefaultMeaning spark.sql.autoBroadcastJoinThreshold - false + 10000 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 @@ -876,7 +875,7 @@ code. ## Running the Thrift JDBC server The Thrift JDBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 0.12. You can test the JDBC server with the beeline script comes with either Spark or Hive 0.12. +in Hive 0.12. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.12. To start the JDBC server, run the following in the Spark directory: @@ -899,12 +898,12 @@ your machine and a blank password. For secure mode, please follow the instructio Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may also use the beeline script comes with Hive. +You may also use the beeline script that 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. +queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. To start the Spark SQL CLI, run the following in the Spark directory: @@ -916,7 +915,10 @@ options. # Compatibility with Other Systems -## Migration Guide for Shark Users +## Migration Guide for Shark User + +### Scheduling +s 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: @@ -925,7 +927,7 @@ users can set the `spark.sql.thriftserver.scheduler.pool` variable: ### 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 +SQL deprecates this property in favor of `spark.sql.shuffle.partitions`, whose default value is 200. Users may customize this property via `SET`: SET spark.sql.shuffle.partitions=10; From 939a322c85956eda150b10afb2ed1d8d959a7bdf Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 8 Sep 2014 15:45:28 -0700 Subject: [PATCH 26/51] [SPARK-3417] Use new-style classes in PySpark Tiny PR making SQLContext a new-style class. This allows various type logic to work more effectively ```Python In [1]: import pyspark In [2]: pyspark.sql.SQLContext.mro() Out[2]: [pyspark.sql.SQLContext, object] ``` Author: Matthew Rocklin Closes #2288 from mrocklin/sqlcontext-new-style-class and squashes the following commits: 4aadab6 [Matthew Rocklin] update other old-style classes a2dc02f [Matthew Rocklin] pyspark.sql.SQLContext is new-style class --- python/pyspark/mllib/random.py | 2 +- python/pyspark/mllib/util.py | 2 +- python/pyspark/sql.py | 2 +- python/pyspark/storagelevel.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 3e59c73db85e3..d53c95fd59c25 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -28,7 +28,7 @@ __all__ = ['RandomRDDs', ] -class RandomRDDs: +class RandomRDDs(object): """ Generator methods for creating RDDs comprised of i.i.d samples from some distribution. diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 4962d05491c03..1c7b8c809ab5b 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -25,7 +25,7 @@ from pyspark.serializers import NoOpSerializer -class MLUtils: +class MLUtils(object): """ Helper methods to load, save and pre-process data used in MLlib. diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 004d4937cbe1c..53eea6d6cf3ba 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -899,7 +899,7 @@ def __reduce__(self): return Row -class SQLContext: +class SQLContext(object): """Main entry point for Spark SQL functionality. diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 2aa0fb9d2c1ed..676aa0f7144aa 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -18,7 +18,7 @@ __all__ = ["StorageLevel"] -class StorageLevel: +class StorageLevel(object): """ Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, From 08ce18881e09c6e91db9c410d1d9ce1e5ae63a62 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 8 Sep 2014 15:59:20 -0700 Subject: [PATCH 27/51] [SPARK-3019] Pluggable block transfer interface (BlockTransferService) This pull request creates a new BlockTransferService interface for block fetch/upload and refactors the existing ConnectionManager to implement BlockTransferService (NioBlockTransferService). Most of the changes are simply moving code around. The main class to inspect is ShuffleBlockFetcherIterator. Review guide: - Most of the ConnectionManager code is now in network.cm package - ManagedBuffer is a new buffer abstraction backed by several different implementations (file segment, nio ByteBuffer, Netty ByteBuf) - BlockTransferService is the main internal interface introduced in this PR - NioBlockTransferService implements BlockTransferService and replaces the old BlockManagerWorker - ShuffleBlockFetcherIterator replaces the told BlockFetcherIterator to use the new interface TODOs that should be separate PRs: - Implement NettyBlockTransferService - Finalize the API/semantics for ManagedBuffer.release() Author: Reynold Xin Closes #2240 from rxin/blockTransferService and squashes the following commits: 64cd9d7 [Reynold Xin] Merge branch 'master' into blockTransferService 1dfd3d7 [Reynold Xin] Limit the length of the FileInputStream. 1332156 [Reynold Xin] Fixed style violation from refactoring. 2960c93 [Reynold Xin] Added ShuffleBlockFetcherIteratorSuite. e29c721 [Reynold Xin] Updated comment for ShuffleBlockFetcherIterator. 8a1046e [Reynold Xin] Code review feedback: 2c6b1e1 [Reynold Xin] Removed println in test cases. 2a907e4 [Reynold Xin] Merge branch 'master' into blockTransferService-merge 07ccf0d [Reynold Xin] Added init check to CMBlockTransferService. 98c668a [Reynold Xin] Added failure handling and fixed unit tests. ae05fcd [Reynold Xin] Updated tests, although DistributedSuite is hanging. d8d595c [Reynold Xin] Merge branch 'master' of github.com:apache/spark into blockTransferService 9ef279c [Reynold Xin] Initial refactoring to move ConnectionManager to use the BlockTransferService. --- .../scala/org/apache/spark/SparkEnv.scala | 15 +- ...eiverTest.scala => BlockDataManager.scala} | 29 +- .../spark/network/BlockFetchingListener.scala | 37 +++ .../spark/network/BlockTransferService.scala | 131 +++++++++ .../spark/network/ConnectionManagerTest.scala | 103 ------- .../apache/spark/network/ManagedBuffer.scala | 107 +++++++ .../org/apache/spark/network/SenderTest.scala | 76 ----- .../nio}/BlockMessage.scala | 24 +- .../nio}/BlockMessageArray.scala | 12 +- .../network/{ => nio}/BufferMessage.scala | 5 +- .../spark/network/{ => nio}/Connection.scala | 10 +- .../network/{ => nio}/ConnectionId.scala | 6 +- .../network/{ => nio}/ConnectionManager.scala | 23 +- .../{ => nio}/ConnectionManagerId.scala | 6 +- .../spark/network/{ => nio}/Message.scala | 7 +- .../network/{ => nio}/MessageChunk.scala | 4 +- .../{ => nio}/MessageChunkHeader.scala | 9 +- .../network/nio/NioBlockTransferService.scala | 205 +++++++++++++ .../network/{ => nio}/SecurityMessage.scala | 10 +- .../spark/serializer/KryoSerializer.scala | 2 +- .../shuffle/FileShuffleBlockManager.scala | 35 ++- .../shuffle/IndexShuffleBlockManager.scala | 24 +- .../spark/shuffle/ShuffleBlockManager.scala | 6 +- .../hash/BlockStoreShuffleFetcher.scala | 14 +- .../shuffle/hash/HashShuffleReader.scala | 4 +- .../spark/storage/BlockFetcherIterator.scala | 254 ---------------- .../apache/spark/storage/BlockManager.scala | 98 +++---- .../apache/spark/storage/BlockManagerId.scala | 4 +- .../spark/storage/BlockManagerWorker.scala | 147 ---------- .../storage/ShuffleBlockFetcherIterator.scala | 271 ++++++++++++++++++ .../apache/spark/storage/ThreadingTest.scala | 120 -------- .../org/apache/spark/DistributedSuite.scala | 15 +- .../{ => nio}/ConnectionManagerSuite.scala | 17 +- .../hash/HashShuffleManagerSuite.scala | 17 +- .../storage/BlockFetcherIteratorSuite.scala | 237 --------------- .../spark/storage/BlockManagerSuite.scala | 133 +-------- .../spark/storage/DiskBlockManagerSuite.scala | 2 +- .../ShuffleBlockFetcherIteratorSuite.scala | 183 ++++++++++++ 38 files changed, 1129 insertions(+), 1273 deletions(-) rename core/src/main/scala/org/apache/spark/network/{ReceiverTest.scala => BlockDataManager.scala} (56%) create mode 100644 core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala create mode 100644 core/src/main/scala/org/apache/spark/network/BlockTransferService.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala create mode 100644 core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/SenderTest.scala rename core/src/main/scala/org/apache/spark/{storage => network/nio}/BlockMessage.scala (89%) rename core/src/main/scala/org/apache/spark/{storage => network/nio}/BlockMessageArray.scala (97%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/BufferMessage.scala (98%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/Connection.scala (99%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionId.scala (88%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionManager.scala (98%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionManagerId.scala (88%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/Message.scala (95%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/MessageChunk.scala (96%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/MessageChunkHeader.scala (93%) create mode 100644 core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala rename core/src/main/scala/org/apache/spark/network/{ => nio}/SecurityMessage.scala (95%) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala rename core/src/test/scala/org/apache/spark/network/{ => nio}/ConnectionManagerSuite.scala (97%) delete mode 100644 core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 20a7444cfc5ee..dd95e406f2a8e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -31,7 +31,8 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.ConnectionManager +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} @@ -59,8 +60,8 @@ class SparkEnv ( val mapOutputTracker: MapOutputTracker, val shuffleManager: ShuffleManager, val broadcastManager: BroadcastManager, + val blockTransferService: BlockTransferService, val blockManager: BlockManager, - val connectionManager: ConnectionManager, val securityManager: SecurityManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, @@ -88,6 +89,8 @@ class SparkEnv ( // down, but let's call it anyway in case it gets fixed in a later release // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. // actorSystem.awaitTermination() + + // Note that blockTransferService is stopped by BlockManager since it is started by it. } private[spark] @@ -223,14 +226,14 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) + val blockTransferService = new NioBlockTransferService(conf, securityManager) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager, mapOutputTracker, shuffleManager) - - val connectionManager = blockManager.connectionManager + serializer, conf, mapOutputTracker, shuffleManager, blockTransferService) val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) @@ -278,8 +281,8 @@ object SparkEnv extends Logging { mapOutputTracker, shuffleManager, broadcastManager, + blockTransferService, blockManager, - connectionManager, securityManager, httpFileServer, sparkFilesDir, diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala similarity index 56% rename from core/src/main/scala/org/apache/spark/network/ReceiverTest.scala rename to core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 53a6038a9b59e..e0e91724271c8 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -17,21 +17,20 @@ package org.apache.spark.network -import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.storage.StorageLevel -private[spark] object ReceiverTest { - def main(args: Array[String]) { - val conf = new SparkConf - val manager = new ConnectionManager(9999, conf, new SecurityManager(conf)) - println("Started connection manager with id = " + manager.id) - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ - val buffer = ByteBuffer.wrap("response".getBytes("utf-8")) - Some(Message.createBufferMessage(buffer, msg.id)) - }) - Thread.currentThread.join() - } -} +trait BlockDataManager { + + /** + * Interface to get local block data. + * + * @return Some(buffer) if the block exists locally, and None if it doesn't. + */ + def getBlockData(blockId: String): Option[ManagedBuffer] + /** + * Put the block locally, using the given storage level. + */ + def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala new file mode 100644 index 0000000000000..34acaa563ca58 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala @@ -0,0 +1,37 @@ +/* + * 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.network + +import java.util.EventListener + + +/** + * Listener callback interface for [[BlockTransferService.fetchBlocks]]. + */ +trait BlockFetchingListener extends EventListener { + + /** + * Called once per successfully fetched block. + */ + def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit + + /** + * Called upon failures. For each failure, this is called only once (i.e. not once per block). + */ + def onBlockFetchFailure(exception: Throwable): Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala new file mode 100644 index 0000000000000..84d991fa6808c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -0,0 +1,131 @@ +/* + * 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.network + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.Duration + +import org.apache.spark.storage.StorageLevel + + +abstract class BlockTransferService { + + /** + * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch + * local blocks or put local blocks. + */ + def init(blockDataManager: BlockDataManager) + + /** + * Tear down the transfer service. + */ + def stop(): Unit + + /** + * Port number the service is listening on, available only after [[init]] is invoked. + */ + def port: Int + + /** + * Host name the service is listening on, available only after [[init]] is invoked. + */ + def hostName: String + + /** + * Fetch a sequence of blocks from a remote node asynchronously, + * available only after [[init]] is invoked. + * + * Note that [[BlockFetchingListener.onBlockFetchSuccess]] is called once per block, + * while [[BlockFetchingListener.onBlockFetchFailure]] is called once per failure (not per block). + * + * Note that this API takes a sequence so the implementation can batch requests, and does not + * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as + * the data of a block is fetched, rather than waiting for all blocks to be fetched. + */ + def fetchBlocks( + hostName: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + */ + def uploadBlock( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel): Future[Unit] + + /** + * A special case of [[fetchBlocks]], as it fetches only one block and is blocking. + * + * It is also only available after [[init]] is invoked. + */ + def fetchBlockSync(hostName: String, port: Int, blockId: String): ManagedBuffer = { + // A monitor for the thread to wait on. + val lock = new Object + @volatile var result: Either[ManagedBuffer, Throwable] = null + fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener { + override def onBlockFetchFailure(exception: Throwable): Unit = { + lock.synchronized { + result = Right(exception) + lock.notify() + } + } + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + lock.synchronized { + result = Left(data) + lock.notify() + } + } + }) + + // Sleep until result is no longer null + lock.synchronized { + while (result == null) { + try { + lock.wait() + } catch { + case e: InterruptedException => + } + } + } + + result match { + case Left(data) => data + case Right(e) => throw e + } + } + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + * + * This method is similar to [[uploadBlock]], except this one blocks the thread + * until the upload finishes. + */ + def uploadBlockSync( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel): Unit = { + Await.result(uploadBlock(hostname, port, blockId, blockData, level), Duration.Inf) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala deleted file mode 100644 index 4894ecd41f6eb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.network - -import java.nio.ByteBuffer - -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.io.Source - -import org.apache.spark._ - -private[spark] object ConnectionManagerTest extends Logging{ - def main(args: Array[String]) { - // - the master URL - a list slaves to run connectionTest on - // [num of tasks] - the number of parallel tasks to be initiated default is number of slave - // hosts [size of msg in MB (integer)] - the size of messages to be sent in each task, - // default is 10 [count] - how many times to run, default is 3 [await time in seconds] : - // await time (in seconds), default is 600 - if (args.length < 2) { - println("Usage: ConnectionManagerTest [num of tasks] " + - "[size of msg in MB (integer)] [count] [await time in seconds)] ") - System.exit(1) - } - - if (args(0).startsWith("local")) { - println("This runs only on a mesos cluster") - } - - val sc = new SparkContext(args(0), "ConnectionManagerTest") - val slavesFile = Source.fromFile(args(1)) - val slaves = slavesFile.mkString.split("\n") - slavesFile.close() - - /* println("Slaves") */ - /* slaves.foreach(println) */ - val tasknum = if (args.length > 2) args(2).toInt else slaves.length - val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 - val count = if (args.length > 4) args(4).toInt else 3 - val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second - println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + - "msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) - val slaveConnManagerIds = sc.parallelize(0 until tasknum, tasknum).map( - i => SparkEnv.get.connectionManager.id).collect() - println("\nSlave ConnectionManagerIds") - slaveConnManagerIds.foreach(println) - println - - (0 until count).foreach(i => { - val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { - val connManager = SparkEnv.get.connectionManager - val thisConnManagerId = connManager.id - connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - logInfo("Received [" + msg + "] from [" + id + "]") - None - }) - - val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) - buffer.flip - - val startTime = System.currentTimeMillis - val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => - { - val bufferMessage = Message.createBufferMessage(buffer.duplicate) - logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") - connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) - } - } - val results = futures.map(f => Await.result(f, awaitTime)) - val finishTime = System.currentTimeMillis - Thread.sleep(5000) - - val mb = size * results.size / 1024.0 / 1024.0 - val ms = finishTime - startTime - val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * - 1000.0) + " MB/s" - logInfo(resultStr) - resultStr - }).collect() - - println("---------------------") - println("Run " + i) - resultStrs.foreach(println) - println("---------------------") - }) - } -} - diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala new file mode 100644 index 0000000000000..dcecb6beeea9b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -0,0 +1,107 @@ +/* + * 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.network + +import java.io.{FileInputStream, RandomAccessFile, File, InputStream} +import java.nio.ByteBuffer +import java.nio.channels.FileChannel.MapMode + +import com.google.common.io.ByteStreams +import io.netty.buffer.{ByteBufInputStream, ByteBuf} + +import org.apache.spark.util.ByteBufferInputStream + + +/** + * This interface provides an immutable view for data in the form of bytes. The implementation + * should specify how the data is provided: + * + * - FileSegmentManagedBuffer: data backed by part of a file + * - NioByteBufferManagedBuffer: data backed by a NIO ByteBuffer + * - NettyByteBufManagedBuffer: data backed by a Netty ByteBuf + */ +sealed abstract class ManagedBuffer { + // Note that all the methods are defined with parenthesis because their implementations can + // have side effects (io operations). + + /** Number of bytes of the data. */ + def size: Long + + /** + * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the + * returned ByteBuffer should not affect the content of this buffer. + */ + def nioByteBuffer(): ByteBuffer + + /** + * Exposes this buffer's data as an InputStream. The underlying implementation does not + * necessarily check for the length of bytes read, so the caller is responsible for making sure + * it does not go over the limit. + */ + def inputStream(): InputStream +} + + +/** + * A [[ManagedBuffer]] backed by a segment in a file + */ +final class FileSegmentManagedBuffer(val file: File, val offset: Long, val length: Long) + extends ManagedBuffer { + + override def size: Long = length + + override def nioByteBuffer(): ByteBuffer = { + val channel = new RandomAccessFile(file, "r").getChannel + channel.map(MapMode.READ_ONLY, offset, length) + } + + override def inputStream(): InputStream = { + val is = new FileInputStream(file) + is.skip(offset) + ByteStreams.limit(is, length) + } +} + + +/** + * A [[ManagedBuffer]] backed by [[java.nio.ByteBuffer]]. + */ +final class NioByteBufferManagedBuffer(buf: ByteBuffer) extends ManagedBuffer { + + override def size: Long = buf.remaining() + + override def nioByteBuffer() = buf.duplicate() + + override def inputStream() = new ByteBufferInputStream(buf) +} + + +/** + * A [[ManagedBuffer]] backed by a Netty [[ByteBuf]]. + */ +final class NettyByteBufManagedBuffer(buf: ByteBuf) extends ManagedBuffer { + + override def size: Long = buf.readableBytes() + + override def nioByteBuffer() = buf.nioBuffer() + + override def inputStream() = new ByteBufInputStream(buf) + + // TODO(rxin): Promote this to top level ManagedBuffer interface and add documentation for it. + def release(): Unit = buf.release() +} diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala deleted file mode 100644 index ea2ad104ecae1..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.network - -import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} - -import scala.concurrent.Await -import scala.concurrent.duration.Duration -import scala.util.Try - -private[spark] object SenderTest { - def main(args: Array[String]) { - - if (args.length < 2) { - println("Usage: SenderTest ") - System.exit(1) - } - - val targetHost = args(0) - val targetPort = args(1).toInt - val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val conf = new SparkConf - val manager = new ConnectionManager(0, conf, new SecurityManager(conf)) - println("Started connection manager with id = " + manager.id) - - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - println("Received [" + msg + "] from [" + id + "]") - None - }) - - val size = 100 * 1024 * 1024 - val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) - buffer.flip - - val targetServer = args(0) - - val count = 100 - (0 until count).foreach(i => { - val dataMessage = Message.createBufferMessage(buffer.duplicate) - val startTime = System.currentTimeMillis - /* println("Started timer at " + startTime) */ - val promise = manager.sendMessageReliably(targetConnectionManagerId, dataMessage) - val responseStr: String = Try(Await.result(promise, Duration.Inf)) - .map { response => - val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array, "utf-8") - }.getOrElse("none") - - val finishTime = System.currentTimeMillis - val mb = size / 1024.0 / 1024.0 - val ms = finishTime - startTime - // val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms at " + (mb / ms - // * 1000.0) + " MB/s" - val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms (" + - (mb / ms * 1000.0).toInt + "MB/s) | Response = " + responseStr - println(resultStr) - }) - } -} - diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala similarity index 89% rename from core/src/main/scala/org/apache/spark/storage/BlockMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala index a2bfce7b4a0fa..b573f1a8a5fcb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.StringBuilder +import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} -import org.apache.spark.network._ +import scala.collection.mutable.{ArrayBuffer, StringBuilder} +// private[spark] because we need to register them in Kryo private[spark] case class GetBlock(id: BlockId) private[spark] case class GotBlock(id: BlockId, data: ByteBuffer) private[spark] case class PutBlock(id: BlockId, data: ByteBuffer, level: StorageLevel) -private[spark] class BlockMessage() { +private[nio] class BlockMessage() { // Un-initialized: typ = 0 // GetBlock: typ = 1 // GotBlock: typ = 2 @@ -159,7 +159,7 @@ private[spark] class BlockMessage() { } } -private[spark] object BlockMessage { +private[nio] object BlockMessage { val TYPE_NON_INITIALIZED: Int = 0 val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 @@ -194,16 +194,4 @@ private[spark] object BlockMessage { newBlockMessage.set(putBlock) newBlockMessage } - - def main(args: Array[String]) { - val B = new BlockMessage() - val blockId = TestBlockId("ABC") - B.set(new PutBlock(blockId, ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2)) - val bMsg = B.toBufferMessage - val C = new BlockMessage() - C.set(bMsg) - - println(B.getId + " " + B.getLevel) - println(C.getId + " " + C.getLevel) - } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala rename to core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala index 973d85c0a9b3a..a1a2c00ed1542 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer - import org.apache.spark._ -import org.apache.spark.network._ +import org.apache.spark.storage.{StorageLevel, TestBlockId} + +import scala.collection.mutable.ArrayBuffer -private[spark] +private[nio] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { @@ -102,7 +102,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) } } -private[spark] object BlockMessageArray { +private[nio] object BlockMessageArray { def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() diff --git a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/network/BufferMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala index af35f1fc3e459..3b245c5c7a4f3 100644 --- a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer @@ -23,7 +23,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.storage.BlockManager -private[spark] + +private[nio] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) extends Message(Message.BUFFER_MESSAGE, id_) { diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/network/Connection.scala rename to core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 5285ec82c1b64..74074a8dcbfff 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net._ import java.nio._ import java.nio.channels._ -import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} - import org.apache.spark._ -private[spark] +import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} + +private[nio] abstract class Connection(val channel: SocketChannel, val selector: Selector, val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId) extends Logging { @@ -190,7 +190,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, } -private[spark] +private[nio] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, remoteId_ : ConnectionManagerId, id_ : ConnectionId) extends Connection(SocketChannel.open, selector_, remoteId_, id_) { diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/network/ConnectionId.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala index d579c165a1917..764dc5e5503ed 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio -private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { +private[nio] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId } -private[spark] object ConnectionId { +private[nio] object ConnectionId { def createConnectionIdFromString(connectionIdString: String): ConnectionId = { val res = connectionIdString.split("_").map(_.trim()) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/network/ConnectionManager.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 578d806263006..09d3ea306515b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -15,32 +15,27 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.io.IOException +import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ -import java.net._ -import java.util.{Timer, TimerTask} import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{LinkedBlockingDeque, ThreadPoolExecutor, TimeUnit} +import java.util.{Timer, TimerTask} -import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.collection.mutable.SynchronizedMap -import scala.collection.mutable.SynchronizedQueue - -import scala.concurrent.{Await, ExecutionContext, Future, Promise} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, SynchronizedMap, SynchronizedQueue} import scala.concurrent.duration._ +import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} -private[spark] class ConnectionManager( + +private[nio] class ConnectionManager( port: Int, conf: SparkConf, securityManager: SecurityManager, @@ -904,7 +899,7 @@ private[spark] class ConnectionManager( private[spark] object ConnectionManager { - import ExecutionContext.Implicits.global + import scala.concurrent.ExecutionContext.Implicits.global def main(args: Array[String]) { val conf = new SparkConf diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala index 57f7586883af1..cbb37ec5ced1f 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net.InetSocketAddress import org.apache.spark.util.Utils -private[spark] case class ConnectionManagerId(host: String, port: Int) { +private[nio] case class ConnectionManagerId(host: String, port: Int) { // DEBUG code Utils.checkHost(host) assert (port > 0) @@ -30,7 +30,7 @@ private[spark] case class ConnectionManagerId(host: String, port: Int) { } -private[spark] object ConnectionManagerId { +private[nio] object ConnectionManagerId { def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { new ConnectionManagerId(socketAddress.getHostName, socketAddress.getPort) } diff --git a/core/src/main/scala/org/apache/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/network/Message.scala rename to core/src/main/scala/org/apache/spark/network/nio/Message.scala index 04ea50f62918c..0b874c2891255 100644 --- a/core/src/main/scala/org/apache/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -15,14 +15,15 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net.InetSocketAddress import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -private[spark] abstract class Message(val typ: Long, val id: Int) { + +private[nio] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null var started = false var startTime = -1L @@ -42,7 +43,7 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { } -private[spark] object Message { +private[nio] object Message { val BUFFER_MESSAGE = 1111111111L var lastId = 1 diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/network/MessageChunk.scala rename to core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala index d0f986a12bfe0..278c5ac356ef2 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -private[network] +private[nio] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { val size = if (buffer == null) 0 else buffer.remaining diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala rename to core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala index f3ecca5f992e0..6e20f291c5cec 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio -import java.net.InetAddress -import java.net.InetSocketAddress +import java.net.{InetAddress, InetSocketAddress} import java.nio.ByteBuffer -private[spark] class MessageChunkHeader( +private[nio] class MessageChunkHeader( val typ: Long, val id: Int, val totalSize: Int, @@ -57,7 +56,7 @@ private[spark] class MessageChunkHeader( } -private[spark] object MessageChunkHeader { +private[nio] object MessageChunkHeader { val HEADER_SIZE = 45 def create(buffer: ByteBuffer): MessageChunkHeader = { diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala new file mode 100644 index 0000000000000..59958ee894230 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -0,0 +1,205 @@ +/* + * 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.network.nio + +import java.nio.ByteBuffer + +import scala.concurrent.Future + +import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.network._ +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.Utils + + +/** + * A [[BlockTransferService]] implementation based on [[ConnectionManager]], a custom + * implementation using Java NIO. + */ +final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityManager) + extends BlockTransferService with Logging { + + private var cm: ConnectionManager = _ + + private var blockDataManager: BlockDataManager = _ + + /** + * Port number the service is listening on, available only after [[init]] is invoked. + */ + override def port: Int = { + checkInit() + cm.id.port + } + + /** + * Host name the service is listening on, available only after [[init]] is invoked. + */ + override def hostName: String = { + checkInit() + cm.id.host + } + + /** + * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch + * local blocks or put local blocks. + */ + override def init(blockDataManager: BlockDataManager): Unit = { + this.blockDataManager = blockDataManager + cm = new ConnectionManager( + conf.getInt("spark.blockManager.port", 0), + conf, + securityManager, + "Connection manager for block manager") + cm.onReceiveMessage(onBlockMessageReceive) + } + + /** + * Tear down the transfer service. + */ + override def stop(): Unit = { + if (cm != null) { + cm.stop() + } + } + + override def fetchBlocks( + hostName: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit = { + checkInit() + + val cmId = new ConnectionManagerId(hostName, port) + val blockMessageArray = new BlockMessageArray(blockIds.map { blockId => + BlockMessage.fromGetBlock(GetBlock(BlockId(blockId))) + }) + + val future = cm.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) + + // Register the listener on success/failure future callback. + future.onSuccess { case message => + val bufferMessage = message.asInstanceOf[BufferMessage] + val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + + for (blockMessage <- blockMessageArray) { + if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { + listener.onBlockFetchFailure( + new SparkException(s"Unexpected message ${blockMessage.getType} received from $cmId")) + } else { + val blockId = blockMessage.getId + val networkSize = blockMessage.getData.limit() + listener.onBlockFetchSuccess( + blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + } + } + }(cm.futureExecContext) + + future.onFailure { case exception => + listener.onBlockFetchFailure(exception) + }(cm.futureExecContext) + } + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + * + * This call blocks until the upload completes, or throws an exception upon failures. + */ + override def uploadBlock( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel) + : Future[Unit] = { + checkInit() + val msg = PutBlock(BlockId(blockId), blockData.nioByteBuffer(), level) + val blockMessageArray = new BlockMessageArray(BlockMessage.fromPutBlock(msg)) + val remoteCmId = new ConnectionManagerId(hostName, port) + val reply = cm.sendMessageReliably(remoteCmId, blockMessageArray.toBufferMessage) + reply.map(x => ())(cm.futureExecContext) + } + + private def checkInit(): Unit = if (cm == null) { + throw new IllegalStateException(getClass.getName + " has not been initialized") + } + + private def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { + logDebug("Handling message " + msg) + msg match { + case bufferMessage: BufferMessage => + try { + logDebug("Handling as a buffer message " + bufferMessage) + val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage) + logDebug("Parsed as a block message array") + val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) + Some(new BlockMessageArray(responseMessages).toBufferMessage) + } catch { + case e: Exception => { + logError("Exception handling buffer message", e) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } + } + + case otherMessage: Any => + logError("Unknown type message received: " + otherMessage) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } + } + + private def processBlockMessage(blockMessage: BlockMessage): Option[BlockMessage] = { + blockMessage.getType match { + case BlockMessage.TYPE_PUT_BLOCK => + val msg = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) + logDebug("Received [" + msg + "]") + putBlock(msg.id.toString, msg.data, msg.level) + None + + case BlockMessage.TYPE_GET_BLOCK => + val msg = new GetBlock(blockMessage.getId) + logDebug("Received [" + msg + "]") + val buffer = getBlock(msg.id.toString) + if (buffer == null) { + return None + } + Some(BlockMessage.fromGotBlock(GotBlock(msg.id, buffer))) + + case _ => None + } + } + + private def putBlock(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + val startTimeMs = System.currentTimeMillis() + logDebug("PutBlock " + blockId + " started from " + startTimeMs + " with data: " + bytes) + blockDataManager.putBlockData(blockId, new NioByteBufferManagedBuffer(bytes), level) + logDebug("PutBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + + " with data size: " + bytes.limit) + } + + private def getBlock(blockId: String): ByteBuffer = { + val startTimeMs = System.currentTimeMillis() + logDebug("GetBlock " + blockId + " started from " + startTimeMs) + val buffer = blockDataManager.getBlockData(blockId).orNull + logDebug("GetBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + + " and got buffer " + buffer) + buffer.nioByteBuffer() + } +} diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/network/SecurityMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala index 9af9e2e8e9e59..747a2088a7258 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.StringBuilder +import scala.collection.mutable.{ArrayBuffer, StringBuilder} import org.apache.spark._ -import org.apache.spark.network._ /** * SecurityMessage is class that contains the connectionId and sasl token @@ -54,7 +52,7 @@ import org.apache.spark.network._ * - Length of the token * - Token */ -private[spark] class SecurityMessage() extends Logging { +private[nio] class SecurityMessage extends Logging { private var connectionId: String = null private var token: Array[Byte] = null @@ -134,7 +132,7 @@ private[spark] class SecurityMessage() extends Logging { } } -private[spark] object SecurityMessage { +private[nio] object SecurityMessage { /** * Convert the given BufferMessage to a SecurityMessage by parsing the contents diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 87ef9bb0b43c6..d6386f8c06fff 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -27,9 +27,9 @@ import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast +import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ -import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.CompactBuffer diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 96facccd52373..439981d232349 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -26,6 +26,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.{SparkEnv, SparkConf, Logging} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup import org.apache.spark.storage._ @@ -166,34 +167,30 @@ class FileShuffleBlockManager(conf: SparkConf) } } - /** - * Returns the physical file segment in which the given BlockId is located. - */ - private def getBlockLocation(id: ShuffleBlockId): FileSegment = { + override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { + val segment = getBlockData(blockId) + Some(segment.nioByteBuffer()) + } + + override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. - val shuffleState = shuffleStates(id.shuffleId) + val shuffleState = shuffleStates(blockId.shuffleId) val iter = shuffleState.allFileGroups.iterator while (iter.hasNext) { - val segment = iter.next.getFileSegmentFor(id.mapId, id.reduceId) - if (segment.isDefined) { return segment.get } + val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId) + if (segmentOpt.isDefined) { + val segment = segmentOpt.get + return new FileSegmentManagedBuffer(segment.file, segment.offset, segment.length) + } } - throw new IllegalStateException("Failed to find shuffle block: " + id) + throw new IllegalStateException("Failed to find shuffle block: " + blockId) } else { - val file = blockManager.diskBlockManager.getFile(id) - new FileSegment(file, 0, file.length()) + val file = blockManager.diskBlockManager.getFile(blockId) + new FileSegmentManagedBuffer(file, 0, file.length) } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockLocation(blockId) - blockManager.diskStore.getBytes(segment) - } - - override def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] = { - Left(getBlockLocation(blockId.asInstanceOf[ShuffleBlockId])) - } - /** Remove all the blocks / files and metadata related to a particular shuffle. */ def removeShuffle(shuffleId: ShuffleId): Boolean = { // Do not change the ordering of this, if shuffleStates should be removed only diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 8bb9efc46cc58..4ab34336d3f01 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkEnv +import org.apache.spark.network.{ManagedBuffer, FileSegmentManagedBuffer} import org.apache.spark.storage._ /** @@ -89,10 +90,11 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { } } - /** - * Get the location of a block in a map output file. Uses the index file we create for it. - * */ - private def getBlockLocation(blockId: ShuffleBlockId): FileSegment = { + override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { + Some(getBlockData(blockId).nioByteBuffer()) + } + + override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) @@ -102,20 +104,14 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { in.skip(blockId.reduceId * 8) val offset = in.readLong() val nextOffset = in.readLong() - new FileSegment(getDataFile(blockId.shuffleId, blockId.mapId), offset, nextOffset - offset) + new FileSegmentManagedBuffer( + getDataFile(blockId.shuffleId, blockId.mapId), + offset, + nextOffset - offset) } finally { in.close() } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockLocation(blockId) - blockManager.diskStore.getBytes(segment) - } - - override def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] = { - Left(getBlockLocation(blockId.asInstanceOf[ShuffleBlockId])) - } - override def stop() = {} } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala index 4240580250046..63863cc0250a3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala @@ -19,7 +19,8 @@ package org.apache.spark.shuffle import java.nio.ByteBuffer -import org.apache.spark.storage.{FileSegment, ShuffleBlockId} +import org.apache.spark.network.ManagedBuffer +import org.apache.spark.storage.ShuffleBlockId private[spark] trait ShuffleBlockManager { @@ -31,8 +32,7 @@ trait ShuffleBlockManager { */ def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] - def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] + def getBlockData(blockId: ShuffleBlockId): ManagedBuffer def stop(): Unit } - diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 12b475658e29d..6cf9305977a3c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -21,10 +21,9 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} +import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} import org.apache.spark.util.CompletionIterator private[hash] object BlockStoreShuffleFetcher extends Logging { @@ -32,8 +31,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleId: Int, reduceId: Int, context: TaskContext, - serializer: Serializer, - shuffleMetrics: ShuffleReadMetrics) + serializer: Serializer) : Iterator[T] = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) @@ -74,7 +72,13 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer, shuffleMetrics) + val blockFetcherItr = new ShuffleBlockFetcherIterator( + context, + SparkEnv.get.blockTransferService, + blockManager, + blocksByAddress, + serializer, + SparkEnv.get.conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024) val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 7bed97a63f0f6..88a5f1e5ddf58 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -36,10 +36,8 @@ private[spark] class HashShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() val ser = Serializer.getSerializer(dep.serializer) - val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser, - readMetrics) + val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser) val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala deleted file mode 100644 index e35b7fe62c753..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ /dev/null @@ -1,254 +0,0 @@ -/* - * 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.storage - -import java.util.concurrent.LinkedBlockingQueue -import org.apache.spark.network.netty.client.{BlockClientListener, LazyInitIterator, ReferenceCountedBuffer} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet -import scala.collection.mutable.Queue -import scala.util.{Failure, Success} - -import org.apache.spark.{Logging, SparkException} -import org.apache.spark.executor.ShuffleReadMetrics -import org.apache.spark.network.BufferMessage -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils - -/** - * A block fetcher iterator interface for fetching shuffle blocks. - */ -private[storage] -trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { - def initialize() -} - - -private[storage] -object BlockFetcherIterator { - - /** - * A request to fetch blocks from a remote BlockManager. - * @param address remote BlockManager to fetch from. - * @param blocks Sequence of tuple, where the first element is the block id, - * and the second element is the estimated size, used to calculate bytesInFlight. - */ - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { - val size = blocks.map(_._2).sum - } - - /** - * Result of a fetch from a remote block. A failure is represented as size == -1. - * @param blockId block id - * @param size estimated size of the block, used to calculate bytesInFlight. - * Note that this is NOT the exact bytes. - * @param deserialize closure to return the result in the form of an Iterator. - */ - class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - - // TODO: Refactor this whole thing to make code more reusable. - class BasicBlockFetcherIterator( - private val blockManager: BlockManager, - val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer, - readMetrics: ShuffleReadMetrics) - extends BlockFetcherIterator { - - import blockManager._ - - if (blocksByAddress == null) { - throw new IllegalArgumentException("BlocksByAddress is null") - } - - // Total number blocks fetched (local + remote). Also number of FetchResults expected - protected var _numBlocksToFetch = 0 - - protected var startTime = System.currentTimeMillis - - // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks - protected val localBlocksToFetch = new ArrayBuffer[BlockId]() - - // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks - protected val remoteBlocksToFetch = new HashSet[BlockId]() - - // A queue to hold our results. - protected val results = new LinkedBlockingQueue[FetchResult] - - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight - protected val fetchRequests = new Queue[FetchRequest] - - // Current bytes in flight from our requests - protected var bytesInFlight = 0L - - protected def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) - val cmId = new ConnectionManagerId(req.address.host, req.address.port) - val blockMessageArray = new BlockMessageArray(req.blocks.map { - case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) - }) - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onComplete { - case Success(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - throw new SparkException( - "Unexpected message " + blockMessage.getType + " received from " + cmId) - } - val blockId = blockMessage.getId - val networkSize = blockMessage.getData.limit() - results.put(new FetchResult(blockId, sizeMap(blockId), - () => dataDeserialize(blockId, blockMessage.getData, serializer))) - // TODO: NettyBlockFetcherIterator has some race conditions where multiple threads can - // be incrementing bytes read at the same time (SPARK-2625). - readMetrics.remoteBytesRead += networkSize - readMetrics.remoteBlocksFetched += 1 - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - } - } - case Failure(exception) => { - logError("Could not get block(s) from " + cmId, exception) - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } - } - } - } - - protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { - // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) - - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - var totalBlocks = 0 - for ((address, blockInfos) <- blocksByAddress) { - totalBlocks += blockInfos.size - if (address == blockManagerId) { - // Filter out zero-sized blocks - localBlocksToFetch ++= blockInfos.filter(_._2 != 0).map(_._1) - _numBlocksToFetch += localBlocksToFetch.size - } else { - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(BlockId, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - // Skip empty blocks - if (size > 0) { - curBlocks += ((blockId, size)) - remoteBlocksToFetch += blockId - _numBlocksToFetch += 1 - curRequestSize += size - } else if (size < 0) { - throw new BlockException(blockId, "Negative block size " + size) - } - if (curRequestSize >= targetRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curBlocks = new ArrayBuffer[(BlockId, Long)] - logDebug(s"Creating fetch request of $curRequestSize at $address") - curRequestSize = 0 - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + - totalBlocks + " blocks") - remoteRequests - } - - protected def getLocalBlocks() { - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - for (id <- localBlocksToFetch) { - try { - readMetrics.localBlocksFetched += 1 - results.put(new FetchResult(id, 0, () => getLocalShuffleFromDisk(id, serializer).get)) - logDebug("Got local block " + id) - } catch { - case e: Exception => { - logError(s"Error occurred while fetching local blocks", e) - results.put(new FetchResult(id, -1, null)) - return - } - } - } - } - - override def initialize() { - // Split local and remote blocks. - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - - val numFetches = remoteRequests.size - fetchRequests.size - logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } - - // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue - // as they arrive. - @volatile protected var resultsGotten = 0 - - override def hasNext: Boolean = resultsGotten < _numBlocksToFetch - - override def next(): (BlockId, Option[Iterator[Any]]) = { - resultsGotten += 1 - val startFetchWait = System.currentTimeMillis() - val result = results.take() - val stopFetchWait = System.currentTimeMillis() - readMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) - if (! result.failed) bytesInFlight -= result.size - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - } - // End of BasicBlockFetcherIterator -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a714142763243..d1bee3d2c033c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -20,6 +20,8 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} +import scala.concurrent.ExecutionContext.Implicits.global + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ @@ -58,18 +60,14 @@ private[spark] class BlockManager( defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, - shuffleManager: ShuffleManager) - extends BlockDataProvider with Logging { + shuffleManager: ShuffleManager, + blockTransferService: BlockTransferService) + extends BlockDataManager with Logging { - private val port = conf.getInt("spark.blockManager.port", 0) + blockTransferService.init(this) val diskBlockManager = new DiskBlockManager(this, conf) - val connectionManager = - new ConnectionManager(port, conf, securityManager, "Connection manager for block manager") - - implicit val futureExecContext = connectionManager.futureExecContext private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -89,11 +87,7 @@ private[spark] class BlockManager( } val blockManagerId = BlockManagerId( - executorId, connectionManager.id.host, connectionManager.id.port) - - // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory - // for receiving shuffle outputs) - val maxBytesInFlight = conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 + executorId, blockTransferService.hostName, blockTransferService.port) // Whether to compress broadcast variables that are stored private val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) @@ -136,11 +130,11 @@ private[spark] class BlockManager( master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, - securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, - shuffleManager: ShuffleManager) = { + shuffleManager: ShuffleManager, + blockTransferService: BlockTransferService) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager, mapOutputTracker, shuffleManager) + conf, mapOutputTracker, shuffleManager, blockTransferService) } /** @@ -149,7 +143,6 @@ private[spark] class BlockManager( */ private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) - BlockManagerWorker.startBlockManagerWorker(this) } /** @@ -212,20 +205,33 @@ private[spark] class BlockManager( } } - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + /** + * Interface to get local block data. + * + * @return Some(buffer) if the block exists locally, and None if it doesn't. + */ + override def getBlockData(blockId: String): Option[ManagedBuffer] = { val bid = BlockId(blockId) if (bid.isShuffle) { - shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId]) + Some(shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId])) } else { val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] if (blockBytesOpt.isDefined) { - Right(blockBytesOpt.get) + val buffer = blockBytesOpt.get + Some(new NioByteBufferManagedBuffer(buffer)) } else { - throw new BlockNotFoundException(blockId) + None } } } + /** + * Put the block locally, using the given storage level. + */ + override def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit = { + putBytes(BlockId(blockId), data.nioByteBuffer(), level) + } + /** * Get the BlockStatus for the block identified by the given ID, if it exists. * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. @@ -333,16 +339,10 @@ private[spark] class BlockManager( * shuffle blocks. It is safe to do so without a lock on block info since disk store * never deletes (recent) items. */ - def getLocalShuffleFromDisk( - blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - - val shuffleBlockManager = shuffleManager.shuffleBlockManager - val values = shuffleBlockManager.getBytes(blockId.asInstanceOf[ShuffleBlockId]).map( - bytes => this.dataDeserialize(blockId, bytes, serializer)) - - values.orElse { - throw new BlockException(blockId, s"Block $blockId not found on disk, though it should be") - } + def getLocalShuffleFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { + val buf = shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) + val is = wrapForCompression(blockId, buf.inputStream()) + Some(serializer.newInstance().deserializeStream(is).asIterator) } /** @@ -513,8 +513,9 @@ private[spark] class BlockManager( val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { logDebug(s"Getting remote block $blockId from $loc") - val data = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) + val data = blockTransferService.fetchBlockSync( + loc.host, loc.port, blockId.toString).nioByteBuffer() + if (data != null) { if (asBlockResult) { return Some(new BlockResult( @@ -548,22 +549,6 @@ private[spark] class BlockManager( None } - /** - * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns - * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined - * fashion as they're received. Expects a size in bytes to be provided for each block fetched, - * so that we can control the maxMegabytesInFlight for the fetch. - */ - def getMultiple( - blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer, - readMetrics: ShuffleReadMetrics): BlockFetcherIterator = { - val iter = new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer, - readMetrics) - iter.initialize() - iter - } - def putIterator( blockId: BlockId, values: Iterator[Any], @@ -816,12 +801,15 @@ private[spark] class BlockManager( data.rewind() logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + s"To node: $peer") - val putBlock = PutBlock(blockId, data, tLevel) - val cmId = new ConnectionManagerId(peer.host, peer.port) - val syncPutBlockSuccess = BlockManagerWorker.syncPutBlock(putBlock, cmId) - if (!syncPutBlockSuccess) { - logError(s"Failed to call syncPutBlock to $peer") + + try { + blockTransferService.uploadBlockSync( + peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) + } catch { + case e: Exception => + logError(s"Failed to replicate block to $peer", e) } + logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) } @@ -1051,7 +1039,7 @@ private[spark] class BlockManager( } def stop(): Unit = { - connectionManager.stop() + blockTransferService.stop() diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index b7bcb2d85d0ee..d4487fce49ab6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -36,8 +36,8 @@ import org.apache.spark.util.Utils class BlockManagerId private ( private var executorId_ : String, private var host_ : String, - private var port_ : Int - ) extends Externalizable { + private var port_ : Int) + extends Externalizable { private def this() = this(null, null, 0) // For deserialization only diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala deleted file mode 100644 index bf002a42d5dc5..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ /dev/null @@ -1,147 +0,0 @@ -/* - * 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.storage - -import java.nio.ByteBuffer - -import org.apache.spark.Logging -import org.apache.spark.network._ -import org.apache.spark.util.Utils - -import scala.concurrent.Await -import scala.concurrent.duration.Duration -import scala.util.{Try, Failure, Success} - -/** - * A network interface for BlockManager. Each slave should have one - * BlockManagerWorker. - * - * TODO: Use event model. - */ -private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging { - - blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive) - - def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { - logDebug("Handling message " + msg) - msg match { - case bufferMessage: BufferMessage => { - try { - logDebug("Handling as a buffer message " + bufferMessage) - val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage) - logDebug("Parsed as a block message array") - val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) - Some(new BlockMessageArray(responseMessages).toBufferMessage) - } catch { - case e: Exception => { - logError("Exception handling buffer message", e) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) - } - } - } - case otherMessage: Any => { - logError("Unknown type message received: " + otherMessage) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) - } - } - } - - def processBlockMessage(blockMessage: BlockMessage): Option[BlockMessage] = { - blockMessage.getType match { - case BlockMessage.TYPE_PUT_BLOCK => { - val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) - logDebug("Received [" + pB + "]") - putBlock(pB.id, pB.data, pB.level) - None - } - case BlockMessage.TYPE_GET_BLOCK => { - val gB = new GetBlock(blockMessage.getId) - logDebug("Received [" + gB + "]") - val buffer = getBlock(gB.id) - if (buffer == null) { - return None - } - Some(BlockMessage.fromGotBlock(GotBlock(gB.id, buffer))) - } - case _ => None - } - } - - private def putBlock(id: BlockId, bytes: ByteBuffer, level: StorageLevel) { - val startTimeMs = System.currentTimeMillis() - logDebug("PutBlock " + id + " started from " + startTimeMs + " with data: " + bytes) - blockManager.putBytes(id, bytes, level) - logDebug("PutBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) - + " with data size: " + bytes.limit) - } - - private def getBlock(id: BlockId): ByteBuffer = { - val startTimeMs = System.currentTimeMillis() - logDebug("GetBlock " + id + " started from " + startTimeMs) - val buffer = blockManager.getLocalBytes(id) match { - case Some(bytes) => bytes - case None => null - } - logDebug("GetBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) - + " and got buffer " + buffer) - buffer - } -} - -private[spark] object BlockManagerWorker extends Logging { - private var blockManagerWorker: BlockManagerWorker = null - - def startBlockManagerWorker(manager: BlockManager) { - blockManagerWorker = new BlockManagerWorker(manager) - } - - def syncPutBlock(msg: PutBlock, toConnManagerId: ConnectionManagerId): Boolean = { - val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val blockMessage = BlockMessage.fromPutBlock(msg) - val blockMessageArray = new BlockMessageArray(blockMessage) - val resultMessage = Try(Await.result(connectionManager.sendMessageReliably( - toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) - resultMessage.isSuccess - } - - def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = { - val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val blockMessage = BlockMessage.fromGetBlock(msg) - val blockMessageArray = new BlockMessageArray(blockMessage) - val responseMessage = Try(Await.result(connectionManager.sendMessageReliably( - toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) - responseMessage match { - case Success(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - logDebug("Response message received " + bufferMessage) - BlockMessageArray.fromBufferMessage(bufferMessage).foreach(blockMessage => { - logDebug("Found " + blockMessage) - return blockMessage.getData - }) - } - case Failure(exception) => logDebug("No response message received") - } - null - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala new file mode 100644 index 0000000000000..c8e708aa6b1bc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -0,0 +1,271 @@ +/* + * 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.storage + +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet +import scala.collection.mutable.Queue + +import org.apache.spark.{TaskContext, Logging, SparkException} +import org.apache.spark.network.{ManagedBuffer, BlockFetchingListener, BlockTransferService} +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils + + +/** + * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block + * manager. For remote blocks, it fetches them using the provided BlockTransferService. + * + * This creates an iterator of (BlockID, values) tuples so the caller can handle blocks in a + * pipelined fashion as they are received. + * + * The implementation throttles the remote fetches to they don't exceed maxBytesInFlight to avoid + * using too much memory. + * + * @param context [[TaskContext]], used for metrics update + * @param blockTransferService [[BlockTransferService]] for fetching remote blocks + * @param blockManager [[BlockManager]] for reading local blocks + * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. + * For each block we also require the size (in bytes as a long field) in + * order to throttle the memory usage. + * @param serializer serializer used to deserialize the data. + * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. + */ +private[spark] +final class ShuffleBlockFetcherIterator( + context: TaskContext, + blockTransferService: BlockTransferService, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], + serializer: Serializer, + maxBytesInFlight: Long) + extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { + + import ShuffleBlockFetcherIterator._ + + /** + * Total number of blocks to fetch. This can be smaller than the total number of blocks + * in [[blocksByAddress]] because we filter out zero-sized blocks in [[initialize]]. + * + * This should equal localBlocks.size + remoteBlocks.size. + */ + private[this] var numBlocksToFetch = 0 + + /** + * The number of blocks proccessed by the caller. The iterator is exhausted when + * [[numBlocksProcessed]] == [[numBlocksToFetch]]. + */ + private[this] var numBlocksProcessed = 0 + + private[this] val startTime = System.currentTimeMillis + + /** Local blocks to fetch, excluding zero-sized blocks. */ + private[this] val localBlocks = new ArrayBuffer[BlockId]() + + /** Remote blocks to fetch, excluding zero-sized blocks. */ + private[this] val remoteBlocks = new HashSet[BlockId]() + + /** + * A queue to hold our results. This turns the asynchronous model provided by + * [[BlockTransferService]] into a synchronous model (iterator). + */ + private[this] val results = new LinkedBlockingQueue[FetchResult] + + // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + // the number of bytes in flight is limited to maxBytesInFlight + private[this] val fetchRequests = new Queue[FetchRequest] + + // Current bytes in flight from our requests + private[this] var bytesInFlight = 0L + + private[this] val shuffleMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + + initialize() + + private[this] def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + bytesInFlight += req.size + + // so we can look up the size of each blockID + val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap + val blockIds = req.blocks.map(_._1.toString) + + blockTransferService.fetchBlocks(req.address.host, req.address.port, blockIds, + new BlockFetchingListener { + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), + () => serializer.newInstance().deserializeStream( + blockManager.wrapForCompression(BlockId(blockId), data.inputStream())).asIterator + )) + shuffleMetrics.remoteBytesRead += data.size + shuffleMetrics.remoteBlocksFetched += 1 + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } + + override def onBlockFetchFailure(e: Throwable): Unit = { + logError("Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + // Note that there is a chance that some blocks have been fetched successfully, but we + // still add them to the failed queue. This is fine because when the caller see a + // FetchFailedException, it is going to fail the entire task anyway. + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + } + ) + } + + private[this] def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + + // Tracks total number of blocks (including zero sized blocks) + var totalBlocks = 0 + for ((address, blockInfos) <- blocksByAddress) { + totalBlocks += blockInfos.size + if (address == blockManager.blockManagerId) { + // Filter out zero-sized blocks + localBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) + numBlocksToFetch += localBlocks.size + } else { + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(BlockId, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + // Skip empty blocks + if (size > 0) { + curBlocks += ((blockId, size)) + remoteBlocks += blockId + numBlocksToFetch += 1 + curRequestSize += size + } else if (size < 0) { + throw new BlockException(blockId, "Negative block size " + size) + } + if (curRequestSize >= targetRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curBlocks = new ArrayBuffer[(BlockId, Long)] + logDebug(s"Creating fetch request of $curRequestSize at $address") + curRequestSize = 0 + } + } + // Add in the final request + if (curBlocks.nonEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + logInfo(s"Getting $numBlocksToFetch non-empty blocks out of $totalBlocks blocks") + remoteRequests + } + + private[this] def fetchLocalBlocks() { + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + for (id <- localBlocks) { + try { + shuffleMetrics.localBlocksFetched += 1 + results.put(new FetchResult( + id, 0, () => blockManager.getLocalShuffleFromDisk(id, serializer).get)) + logDebug("Got local block " + id) + } catch { + case e: Exception => + logError(s"Error occurred while fetching local blocks", e) + results.put(new FetchResult(id, -1, null)) + return + } + } + } + + private[this] def initialize(): Unit = { + // Split local and remote blocks. + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (fetchRequests.nonEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + + val numFetches = remoteRequests.size - fetchRequests.size + logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + fetchLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch + + override def next(): (BlockId, Option[Iterator[Any]]) = { + numBlocksProcessed += 1 + val startFetchWait = System.currentTimeMillis() + val result = results.take() + val stopFetchWait = System.currentTimeMillis() + shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) + if (!result.failed) { + bytesInFlight -= result.size + } + // Send fetch requests up to maxBytesInFlight + while (fetchRequests.nonEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } +} + + +private[storage] +object ShuffleBlockFetcherIterator { + + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of tuple, where the first element is the block id, + * and the second element is the estimated size, used to calculate bytesInFlight. + */ + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { + val size = blocks.map(_._2).sum + } + + /** + * Result of a fetch from a remote block. A failure is represented as size == -1. + * @param blockId block id + * @param size estimated size of the block, used to calculate bytesInFlight. + * Note that this is NOT the exact bytes. + * @param deserialize closure to return the result in the form of an Iterator. + */ + class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala deleted file mode 100644 index 7540f0d5e2a5a..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.storage - -import java.util.concurrent.ArrayBlockingQueue - -import akka.actor._ -import org.apache.spark.shuffle.hash.HashShuffleManager -import util.Random - -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} -import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.serializer.KryoSerializer - -/** - * This class tests the BlockManager and MemoryStore for thread safety and - * deadlocks. It spawns a number of producer and consumer threads. Producer - * threads continuously pushes blocks into the BlockManager and consumer - * threads continuously retrieves the blocks form the BlockManager and tests - * whether the block is correct or not. - */ -private[spark] object ThreadingTest { - - val numProducers = 5 - val numBlocksPerProducer = 20000 - - private[spark] class ProducerThread(manager: BlockManager, id: Int) extends Thread { - val queue = new ArrayBlockingQueue[(BlockId, Seq[Int])](100) - - override def run() { - for (i <- 1 to numBlocksPerProducer) { - val blockId = TestBlockId("b-" + id + "-" + i) - val blockSize = Random.nextInt(1000) - val block = (1 to blockSize).map(_ => Random.nextInt()) - val level = randomLevel() - val startTime = System.currentTimeMillis() - manager.putIterator(blockId, block.iterator, level, tellMaster = true) - println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") - queue.add((blockId, block)) - } - println("Producer thread " + id + " terminated") - } - - def randomLevel(): StorageLevel = { - math.abs(Random.nextInt()) % 4 match { - case 0 => StorageLevel.MEMORY_ONLY - case 1 => StorageLevel.MEMORY_ONLY_SER - case 2 => StorageLevel.MEMORY_AND_DISK - case 3 => StorageLevel.MEMORY_AND_DISK_SER - } - } - } - - private[spark] class ConsumerThread( - manager: BlockManager, - queue: ArrayBlockingQueue[(BlockId, Seq[Int])] - ) extends Thread { - var numBlockConsumed = 0 - - override def run() { - println("Consumer thread started") - while(numBlockConsumed < numBlocksPerProducer) { - val (blockId, block) = queue.take() - val startTime = System.currentTimeMillis() - manager.get(blockId) match { - case Some(retrievedBlock) => - assert(retrievedBlock.data.toList.asInstanceOf[List[Int]] == block.toList, - "Block " + blockId + " did not match") - println("Got block " + blockId + " in " + - (System.currentTimeMillis - startTime) + " ms") - case None => - assert(false, "Block " + blockId + " could not be retrieved") - } - numBlockConsumed += 1 - } - println("Consumer thread terminated") - } - } - - def main(args: Array[String]) { - System.setProperty("spark.kryoserializer.buffer.mb", "1") - val actorSystem = ActorSystem("test") - val conf = new SparkConf() - val serializer = new KryoSerializer(conf) - val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) - val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) - val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) - val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) - producers.foreach(_.start) - consumers.foreach(_.start) - producers.foreach(_.join) - consumers.foreach(_.join) - blockManager.stop() - blockManagerMaster.stop() - actorSystem.shutdown() - actorSystem.awaitTermination() - println("Everything stopped.") - println( - "It will take sometime for the JVM to clean all temporary files and shutdown. Sit tight.") - } -} diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 41c294f727b3c..81b64c36ddca1 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -24,8 +24,7 @@ import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} import org.apache.spark.SparkContext._ -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.storage.{BlockManagerWorker, GetBlock, RDDBlockId, StorageLevel} +import org.apache.spark.storage.{RDDBlockId, StorageLevel} class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} @@ -136,7 +135,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter sc.parallelize(1 to 10, 2).foreach { x => if (x == 1) System.exit(42) } } assert(thrown.getClass === classOf[SparkException]) - System.out.println(thrown.getMessage) assert(thrown.getMessage.contains("failed 4 times")) } } @@ -202,12 +200,13 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter val blockIds = data.partitions.indices.map(index => RDDBlockId(data.id, index)).toArray val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager - blockManager.master.getLocations(blockId).foreach(id => { - val bytes = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(id.host, id.port)) - val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList + val blockTransfer = SparkEnv.get.blockTransferService + blockManager.master.getLocations(blockId).foreach { cmId => + val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, blockId.toString) + val deserialized = blockManager.dataDeserialize(blockId, bytes.nioByteBuffer()) + .asInstanceOf[Iterator[Int]].toList assert(deserialized === (1 to 100).toList) - }) + } } test("compute without caching when no partitions fit in memory") { diff --git a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala similarity index 97% rename from core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala rename to core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index e2f4d4c57cdb5..9f49587cdc670 100644 --- a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -15,23 +15,18 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.io.IOException import java.nio._ -import java.util.concurrent.TimeoutException -import org.apache.spark.{SecurityManager, SparkConf} -import org.scalatest.FunSuite - -import org.mockito.Mockito._ -import org.mockito.Matchers._ - -import scala.concurrent.TimeoutException -import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ +import scala.concurrent.{Await, TimeoutException} import scala.language.postfixOps -import scala.util.{Failure, Success, Try} + +import org.scalatest.FunSuite + +import org.apache.spark.{SecurityManager, SparkConf} /** * Test the ConnectionManager with various security settings. diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 6061e544e79b4..ba47fe5e25b9b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkEnv, SparkContext, LocalSparkContext, SparkConf} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.shuffle.FileShuffleBlockManager import org.apache.spark.storage.{ShuffleBlockId, FileSegment} @@ -32,10 +33,12 @@ import org.apache.spark.storage.{ShuffleBlockId, FileSegment} class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { private val testConf = new SparkConf(false) - private def checkSegments(segment1: FileSegment, segment2: FileSegment) { - assert (segment1.file.getCanonicalPath === segment2.file.getCanonicalPath) - assert (segment1.offset === segment2.offset) - assert (segment1.length === segment2.length) + private def checkSegments(expected: FileSegment, buffer: ManagedBuffer) { + assert(buffer.isInstanceOf[FileSegmentManagedBuffer]) + val segment = buffer.asInstanceOf[FileSegmentManagedBuffer] + assert(expected.file.getCanonicalPath === segment.file.getCanonicalPath) + assert(expected.offset === segment.offset) + assert(expected.length === segment.length) } test("consolidated shuffle can write to shuffle group without messing existing offsets/lengths") { @@ -95,14 +98,12 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { writer.commitAndClose() } // check before we register. - checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0)).left.get) + checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0))) shuffle3.releaseWriters(success = true) - checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0)).left.get) + checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0))) shuffleBlockManager.removeShuffle(1) - } - def writeToFile(file: File, numBytes: Int) { val writer = new FileWriter(file, true) for (i <- 0 until numBytes) writer.write(i) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala deleted file mode 100644 index 3c86f6bafcaa3..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala +++ /dev/null @@ -1,237 +0,0 @@ -/* - * 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.storage - -import java.io.IOException -import java.nio.ByteBuffer - -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.future -import scala.concurrent.ExecutionContext.Implicits.global - -import org.scalatest.{FunSuite, Matchers} - -import org.mockito.Mockito._ -import org.mockito.Matchers.{any, eq => meq} -import org.mockito.stubbing.Answer -import org.mockito.invocation.InvocationOnMock - -import org.apache.spark.storage.BlockFetcherIterator._ -import org.apache.spark.network.{ConnectionManager, Message} -import org.apache.spark.executor.ShuffleReadMetrics - -class BlockFetcherIteratorSuite extends FunSuite with Matchers { - - test("block fetch from local fails using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - doReturn(connManager).when(blockManager).connectionManager - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - val answer = new Answer[Option[Iterator[Any]]] { - override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { - throw new Exception - } - } - - // 3rd block is going to fail - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) - - val bmId = BlockManagerId("test-client", "test-client", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, - new ShuffleReadMetrics()) - - iterator.initialize() - - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - // the 2nd element of the tuple returned by iterator.next should be defined when fetching successfully - assert(iterator.next()._2.isDefined, "1st element should be defined but is not actually defined") - verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next()._2.isDefined, "2nd element should be defined but is not actually defined") - verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - // 3rd fetch should be failed - intercept[Exception] { - iterator.next() - } - verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) - } - - - test("block fetch from local succeed using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - doReturn(connManager).when(blockManager).connectionManager - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - - // All blocks should be fetched successfully - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) - - val bmId = BlockManagerId("test-client", "test-client", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, - new ShuffleReadMetrics()) - - iterator.initialize() - - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 1st element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next._2.isDefined, "All elements should be defined but 2nd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 3rd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 4th element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 5th element is not actually defined") - - verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) - } - - test("block fetch from remote fails using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - when(blockManager.connectionManager).thenReturn(connManager) - - val f = future { - throw new IOException("Send failed or we received an error ACK") - } - when(connManager.sendMessageReliably(any(), - any())).thenReturn(f) - when(blockManager.futureExecContext).thenReturn(global) - - when(blockManager.blockManagerId).thenReturn( - BlockManagerId("test-client", "test-client", 1)) - when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) - - val blId1 = ShuffleBlockId(0,0,0) - val blId2 = ShuffleBlockId(0,1,0) - val bmId = BlockManagerId("test-server", "test-server", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L))) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null, new ShuffleReadMetrics()) - - iterator.initialize() - iterator.foreach{ - case (_, r) => { - (!r.isDefined) should be(true) - } - } - } - - test("block fetch from remote succeed using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - when(blockManager.connectionManager).thenReturn(connManager) - - val blId1 = ShuffleBlockId(0,0,0) - val blId2 = ShuffleBlockId(0,1,0) - val buf1 = ByteBuffer.allocate(4) - val buf2 = ByteBuffer.allocate(4) - buf1.putInt(1) - buf1.flip() - buf2.putInt(1) - buf2.flip() - val blockMessage1 = BlockMessage.fromGotBlock(GotBlock(blId1, buf1)) - val blockMessage2 = BlockMessage.fromGotBlock(GotBlock(blId2, buf2)) - val blockMessageArray = new BlockMessageArray( - Seq(blockMessage1, blockMessage2)) - - val bufferMessage = blockMessageArray.toBufferMessage - val buffer = ByteBuffer.allocate(bufferMessage.size) - val arrayBuffer = new ArrayBuffer[ByteBuffer] - bufferMessage.buffers.foreach{ b => - buffer.put(b) - } - buffer.flip() - arrayBuffer += buffer - - val f = future { - Message.createBufferMessage(arrayBuffer) - } - when(connManager.sendMessageReliably(any(), - any())).thenReturn(f) - when(blockManager.futureExecContext).thenReturn(global) - - when(blockManager.blockManagerId).thenReturn( - BlockManagerId("test-client", "test-client", 1)) - when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) - - val bmId = BlockManagerId("test-server", "test-server", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L))) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null, new ShuffleReadMetrics()) - iterator.initialize() - iterator.foreach{ - case (_, r) => { - (r.isDefined) should be(true) - } - } - } -} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index c200654162268..e251660dae5de 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,15 +21,19 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import java.util.concurrent.TimeUnit +import org.apache.spark.network.nio.NioBlockTransferService + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + import akka.actor._ import akka.pattern.ask import akka.util.Timeout -import org.apache.spark.shuffle.hash.HashShuffleManager -import org.mockito.invocation.InvocationOnMock -import org.mockito.Matchers.any -import org.mockito.Mockito.{doAnswer, mock, spy, when} -import org.mockito.stubbing.Answer +import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ @@ -38,18 +42,12 @@ import org.scalatest.Matchers import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod -import org.apache.spark.network.{Message, ConnectionManagerId} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.language.implicitConversions -import scala.language.postfixOps -import org.apache.spark.shuffle.ShuffleBlockManager class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { @@ -74,8 +72,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { - new BlockManager(name, actorSystem, master, serializer, maxMem, conf, securityMgr, - mapOutputTracker, shuffleManager) + val transfer = new NioBlockTransferService(conf, securityMgr) + new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + mapOutputTracker, shuffleManager, transfer) } before { @@ -793,8 +792,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. + val transfer = new NioBlockTransferService(conf, securityMgr) store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) + mapOutputTracker, shuffleManager, transfer) // The put should fail since a1 is not serializable. class UnserializableClass @@ -1005,109 +1005,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } - test("return error message when error occurred in BlockManagerWorker#onBlockMessageReceive") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) - - val worker = spy(new BlockManagerWorker(store)) - val connManagerId = mock(classOf[ConnectionManagerId]) - - // setup request block messages - val reqBlId1 = ShuffleBlockId(0,0,0) - val reqBlId2 = ShuffleBlockId(0,1,0) - val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) - val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) - val reqBlockMessages = new BlockMessageArray( - Seq(reqBlockMessage1, reqBlockMessage2)) - val reqBufferMessage = reqBlockMessages.toBufferMessage - - val answer = new Answer[Option[BlockMessage]] { - override def answer(invocation: InvocationOnMock) - :Option[BlockMessage]= { - throw new Exception - } - } - - doAnswer(answer).when(worker).processBlockMessage(any()) - - // Test when exception was thrown during processing block messages - var ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) - - assert(ackMessage.isDefined, "When Exception was thrown in " + - "BlockManagerWorker#processBlockMessage, " + - "ackMessage should be defined") - assert(ackMessage.get.hasError, "When Exception was thown in " + - "BlockManagerWorker#processBlockMessage, " + - "ackMessage should have error") - - val notBufferMessage = mock(classOf[Message]) - - // Test when not BufferMessage was received - ackMessage = worker.onBlockMessageReceive(notBufferMessage, connManagerId) - assert(ackMessage.isDefined, "When not BufferMessage was passed to " + - "BlockManagerWorker#onBlockMessageReceive, " + - "ackMessage should be defined") - assert(ackMessage.get.hasError, "When not BufferMessage was passed to " + - "BlockManagerWorker#onBlockMessageReceive, " + - "ackMessage should have error") - } - - test("return ack message when no error occurred in BlocManagerWorker#onBlockMessageReceive") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) - - val worker = spy(new BlockManagerWorker(store)) - val connManagerId = mock(classOf[ConnectionManagerId]) - - // setup request block messages - val reqBlId1 = ShuffleBlockId(0,0,0) - val reqBlId2 = ShuffleBlockId(0,1,0) - val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) - val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) - val reqBlockMessages = new BlockMessageArray( - Seq(reqBlockMessage1, reqBlockMessage2)) - - val tmpBufferMessage = reqBlockMessages.toBufferMessage - val buffer = ByteBuffer.allocate(tmpBufferMessage.size) - val arrayBuffer = new ArrayBuffer[ByteBuffer] - tmpBufferMessage.buffers.foreach{ b => - buffer.put(b) - } - buffer.flip() - arrayBuffer += buffer - val reqBufferMessage = Message.createBufferMessage(arrayBuffer) - - // setup ack block messages - val buf1 = ByteBuffer.allocate(4) - val buf2 = ByteBuffer.allocate(4) - buf1.putInt(1) - buf1.flip() - buf2.putInt(1) - buf2.flip() - val ackBlockMessage1 = BlockMessage.fromGotBlock(GotBlock(reqBlId1, buf1)) - val ackBlockMessage2 = BlockMessage.fromGotBlock(GotBlock(reqBlId2, buf2)) - - val answer = new Answer[Option[BlockMessage]] { - override def answer(invocation: InvocationOnMock) - :Option[BlockMessage]= { - if (invocation.getArguments()(0).asInstanceOf[BlockMessage].eq( - reqBlockMessage1)) { - return Some(ackBlockMessage1) - } else { - return Some(ackBlockMessage2) - } - } - } - - doAnswer(answer).when(worker).processBlockMessage(any()) - - val ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) - assert(ackMessage.isDefined, "When BlockManagerWorker#onBlockMessageReceive " + - "was executed successfully, ackMessage should be defined") - assert(!ackMessage.get.hasError, "When BlockManagerWorker#onBlockMessageReceive " + - "was executed successfully, ackMessage should not have error") - } - test("reserve/release unroll memory") { store = makeBlockManager(12000) val memoryStore = store.memoryStore diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 26082ded8ca7a..e4522e00a622d 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.{File, FileWriter} +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.shuffle.hash.HashShuffleManager import scala.collection.mutable @@ -52,7 +53,6 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before rootDir1 = Files.createTempDir() rootDir1.deleteOnExit() rootDirs = rootDir0.getAbsolutePath + "," + rootDir1.getAbsolutePath - println("Created root dirs: " + rootDirs) } override def afterAll() { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala new file mode 100644 index 0000000000000..809bd70929656 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -0,0 +1,183 @@ +/* + * 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.storage + +import org.apache.spark.TaskContext +import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} + +import org.mockito.Mockito._ +import org.mockito.Matchers.{any, eq => meq} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer + +import org.scalatest.FunSuite + + +class ShuffleBlockFetcherIteratorSuite extends FunSuite { + + test("handle local read failures in BlockManager") { + val transfer = mock(classOf[BlockTransferService]) + val blockManager = mock(classOf[BlockManager]) + doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + val answer = new Answer[Option[Iterator[Any]]] { + override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { + throw new Exception + } + } + + // 3rd block is going to fail + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) + doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + // Without exhausting the iterator, the iterator should be lazy and not call + // getLocalShuffleFromDisk. + verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + // the 2nd element of the tuple returned by iterator.next should be defined when + // fetching successfully + assert(iterator.next()._2.isDefined, + "1st element should be defined but is not actually defined") + verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next()._2.isDefined, + "2nd element should be defined but is not actually defined") + verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + // 3rd fetch should be failed + intercept[Exception] { + iterator.next() + } + verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) + } + + test("handle local read successes") { + val transfer = mock(classOf[BlockTransferService]) + val blockManager = mock(classOf[BlockManager]) + doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + + // All blocks should be fetched successfully + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. + verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 1st element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 2nd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 3rd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 4th element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 5th element is not actually defined") + + verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) + } + + test("handle remote fetch failures in BlockTransferService") { + val transfer = mock(classOf[BlockTransferService]) + when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + listener.onBlockFetchFailure(new Exception("blah")) + } + }) + + val blockManager = mock(classOf[BlockManager]) + + when(blockManager.blockManagerId).thenReturn(BlockManagerId("test-client", "test-client", 1)) + + val blId1 = ShuffleBlockId(0, 0, 0) + val blId2 = ShuffleBlockId(0, 1, 0) + val bmId = BlockManagerId("test-server", "test-server", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, Seq((blId1, 1L), (blId2, 1L)))) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + iterator.foreach { case (_, iterOption) => + assert(!iterOption.isDefined) + } + } +} From 7db53391f1b349d1f49844197b34f94806f5e336 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 8 Sep 2014 16:14:32 -0700 Subject: [PATCH 28/51] [SPARK-3349][SQL] Output partitioning of limit should not be inherited from child This resolves https://issues.apache.org/jira/browse/SPARK-3349 Author: Eric Liang Closes #2262 from ericl/spark-3349 and squashes the following commits: 3e1b05c [Eric Liang] add regression test ac32723 [Eric Liang] make limit/takeOrdered output SinglePartition --- .../spark/sql/execution/basicOperators.scala | 4 +++- .../org/apache/spark/sql/SQLQuerySuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) 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 47bff0c730b8a..cac376608be29 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 @@ -27,7 +27,7 @@ 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._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, SinglePartition, UnspecifiedDistribution} import org.apache.spark.util.MutablePair /** @@ -100,6 +100,7 @@ case class Limit(limit: Int, child: SparkPlan) private def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] override def output = child.output + override def outputPartitioning = SinglePartition /** * A custom implementation modeled after the take function on RDDs but which never runs any job @@ -173,6 +174,7 @@ case class Limit(limit: Int, child: SparkPlan) case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) extends UnaryNode { override def output = child.output + override def outputPartitioning = SinglePartition val ordering = new RowOrdering(sortOrder, child.output) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1ac205937714c..e8fbc28d0ad60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -359,6 +359,23 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (null, null, 6, "F") :: Nil) } + test("SPARK-3349 partitioning after limit") { + sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") + .limit(2) + .registerTempTable("subset1") + sql("SELECT DISTINCT n FROM lowerCaseData") + .limit(2) + .registerTempTable("subset2") + checkAnswer( + sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"), + (3, "c", 3) :: + (4, "d", 4) :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), + (1, "a", 1) :: + (2, "b", 2) :: Nil) + } + test("mixed-case keywords") { checkAnswer( sql( From 50a4fa774a0e8a17d7743b33ce8941bf4041144d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 8 Sep 2014 18:59:57 -0700 Subject: [PATCH 29/51] [SPARK-3443][MLLIB] update default values of tree: Adjust the default values of decision tree, based on the memory requirement discussed in https://github.com/apache/spark/pull/2125 : 1. maxMemoryInMB: 128 -> 256 2. maxBins: 100 -> 32 3. maxDepth: 4 -> 5 (in some example code) jkbradley Author: Xiangrui Meng Closes #2322 from mengxr/tree-defaults and squashes the following commits: cda453a [Xiangrui Meng] fix tests 5900445 [Xiangrui Meng] update comments 8c81831 [Xiangrui Meng] update default values of tree: --- docs/mllib-decision-tree.md | 16 ++++++++-------- .../spark/examples/mllib/JavaDecisionTree.java | 2 +- .../examples/mllib/DecisionTreeRunner.scala | 4 ++-- .../apache/spark/mllib/tree/DecisionTree.scala | 8 ++++---- .../mllib/tree/configuration/Strategy.scala | 6 +++--- .../spark/mllib/tree/DecisionTreeSuite.scala | 18 ++++-------------- python/pyspark/mllib/tree.py | 4 ++-- 7 files changed, 24 insertions(+), 34 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 1166d9cd150c4..12a6afbeea829 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -80,7 +80,7 @@ The ordered splits create "bins" and the maximum number of such bins can be specified using the `maxBins` parameter. Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario -since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of +since the default `maxBins` value is 32). The tree algorithm automatically reduces the number of bins if the condition is not satisfied. **Categorical features** @@ -117,7 +117,7 @@ all nodes at each level of the tree. This could lead to high memory requirements of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` training parameter specifies the maximum amount of memory at the workers (twice as much at the master) to be allocated to the histogram computation. The default value is conservatively chosen to -be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements +be 256 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each subsequent level are split into smaller tasks. @@ -167,7 +167,7 @@ val numClasses = 2 val categoricalFeaturesInfo = Map[Int, Int]() val impurity = "gini" val maxDepth = 5 -val maxBins = 100 +val maxBins = 32 val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, maxDepth, maxBins) @@ -213,7 +213,7 @@ Integer numClasses = 2; HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; -Integer maxBins = 100; +Integer maxBins = 32; // Train a DecisionTree model for classification. final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, @@ -250,7 +250,7 @@ data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, - impurity='gini', maxDepth=5, maxBins=100) + impurity='gini', maxDepth=5, maxBins=32) # Evaluate model on training instances and compute training error predictions = model.predict(data.map(lambda x: x.features)) @@ -293,7 +293,7 @@ val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache val categoricalFeaturesInfo = Map[Int, Int]() val impurity = "variance" val maxDepth = 5 -val maxBins = 100 +val maxBins = 32 val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, maxDepth, maxBins) @@ -338,7 +338,7 @@ JavaSparkContext sc = new JavaSparkContext(sparkConf); HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "variance"; Integer maxDepth = 5; -Integer maxBins = 100; +Integer maxBins = 32; // Train a DecisionTree model. final DecisionTreeModel model = DecisionTree.trainRegressor(data, @@ -380,7 +380,7 @@ data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, - impurity='variance', maxDepth=5, maxBins=100) + impurity='variance', maxDepth=5, maxBins=32) # Evaluate model on training instances and compute training error predictions = model.predict(data.map(lambda x: x.features)) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java index e4468e8bf1744..1f82e3f4cb18e 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java @@ -63,7 +63,7 @@ public static void main(String[] args) { HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; - Integer maxBins = 100; + Integer maxBins = 32; // Train a DecisionTree model for classification. final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index cf3d2cca81ff6..72c3ab475b61f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -52,9 +52,9 @@ object DecisionTreeRunner { input: String = null, dataFormat: String = "libsvm", algo: Algo = Classification, - maxDepth: Int = 4, + maxDepth: Int = 5, impurity: ImpurityType = Gini, - maxBins: Int = 100, + maxBins: Int = 32, fracTest: Double = 0.2) def main(args: Array[String]) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index dd766c12d28a4..d1309b2b20f54 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -330,9 +330,9 @@ object DecisionTree extends Serializable with Logging { * Supported values: "gini" (recommended) or "entropy". * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) + * (suggested value: 5) * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) + * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction */ def trainClassifier( @@ -374,9 +374,9 @@ object DecisionTree extends Serializable with Logging { * Supported values: "variance". * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) + * (suggested value: 5) * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) + * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction */ def trainRegressor( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index cfc8192a85abd..23f74d5360fe5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -50,7 +50,7 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is - * 128 MB. + * 256 MB. */ @Experimental class Strategy ( @@ -58,10 +58,10 @@ class Strategy ( val impurity: Impurity, val maxDepth: Int, val numClassesForClassification: Int = 2, - val maxBins: Int = 100, + val maxBins: Int = 32, val quantileCalculationStrategy: QuantileStrategy = Sort, val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), - val maxMemoryInMB: Int = 128) extends Serializable { + val maxMemoryInMB: Int = 256) extends Serializable { if (algo == Classification) { require(numClassesForClassification >= 2) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 8e556c917b2e7..69482f2acbb40 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} import org.apache.spark.mllib.util.LocalSparkContext - class DecisionTreeSuite extends FunSuite with LocalSparkContext { def validateClassifier( @@ -353,8 +352,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, @@ -381,8 +378,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -410,8 +405,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -439,8 +432,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -464,8 +455,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) @@ -600,7 +589,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3) + numClassesForClassification = 3, maxBins = 100) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -626,7 +615,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) + numClassesForClassification = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(metadata.isUnordered(featureIndex = 0)) @@ -652,7 +641,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + numClassesForClassification = 3, maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index a2fade61e9a71..ccc000ac70ba6 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -138,7 +138,7 @@ class DecisionTree(object): @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, - impurity="gini", maxDepth=4, maxBins=100): + impurity="gini", maxDepth=5, maxBins=32): """ Train a DecisionTreeModel for classification. @@ -170,7 +170,7 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, @staticmethod def trainRegressor(data, categoricalFeaturesInfo, - impurity="variance", maxDepth=4, maxBins=100): + impurity="variance", maxDepth=5, maxBins=32): """ Train a DecisionTreeModel for regression. From ca0348e68213c2c7589f2018ebf9d889c0ce59c3 Mon Sep 17 00:00:00 2001 From: William Benton Date: Mon, 8 Sep 2014 19:05:02 -0700 Subject: [PATCH 30/51] SPARK-3423: [SQL] Implement BETWEEN for SQLParser This patch improves the SQLParser by adding support for BETWEEN conditions Author: William Benton Closes #2295 from willb/sql-between and squashes the following commits: 0016d30 [William Benton] Implement BETWEEN for SQLParser --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 ++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 18 ++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a88bd859fc85e..bfc197cf7a938 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -73,6 +73,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val ASC = Keyword("ASC") protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AVG = Keyword("AVG") + protected val BETWEEN = Keyword("BETWEEN") protected val BY = Keyword("BY") protected val CACHE = Keyword("CACHE") protected val CAST = Keyword("CAST") @@ -272,6 +273,9 @@ class SqlParser extends StandardTokenParsers with PackratParsers { termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | + termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { + case e ~ _ ~ el ~ _ ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + } | termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | termExpression ~ REGEXP ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | termExpression ~ LIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => Like(e1, e2) } | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index e8fbc28d0ad60..45c0ca8ea101d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -597,4 +597,22 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null) :: (4, 2147483644) :: Nil) } + + test("SPARK-3423 BETWEEN") { + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 5 and 7"), + Seq((5, "5"), (6, "6"), (7, "7")) + ) + + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 7 and 7"), + Seq((7, "7")) + ) + + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), + Seq() + ) + + } } From dc1dbf206e0076a43ad2120d8bb5b1fc6912fe25 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 8 Sep 2014 19:08:05 -0700 Subject: [PATCH 31/51] [SPARK-3414][SQL] Stores analyzed logical plan when registering a temp table Case insensitivity breaks when unresolved relation contains attributes with uppercase letters in their names, because we store unanalyzed logical plan when registering temp tables while the `CaseInsensitivityAttributeReferences` batch runs before the `Resolution` batch. To fix this issue, we need to store analyzed logical plan. Author: Cheng Lian Closes #2293 from liancheng/spark-3414 and squashes the following commits: d9fa1d6 [Cheng Lian] Stores analyzed logical plan when registering a temp table --- .../org/apache/spark/sql/SQLContext.scala | 4 +-- .../sql/hive/execution/HiveQuerySuite.scala | 25 ++++++++++++++++--- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 5acb45c155ba5..a2f334aab9fdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -246,7 +246,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - catalog.registerTable(None, tableName, rdd.logicalPlan) + catalog.registerTable(None, tableName, rdd.queryExecution.analyzed) } /** @@ -411,7 +411,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } - def simpleString: String = + def simpleString: String = s"""== Physical Plan == |${stringOrError(executedPlan)} """ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index f4217a52c3822..305998c150327 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,11 +17,8 @@ package org.apache.spark.sql.hive.execution -import java.io.File - import scala.util.Try -import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -514,6 +511,28 @@ class HiveQuerySuite extends HiveComparisonTest { sql("DROP TABLE alter1") } + case class LogEntry(filename: String, message: String) + case class LogFile(name: String) + + test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { + sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") + + sql( + """ + SELECT name, message + FROM rawLogs + JOIN ( + SELECT name + FROM logFiles + ) files + ON rawLogs.filename = files.name + """).registerTempTable("boom") + + // This should be successfully analyzed + sql("SELECT * FROM boom").queryExecution.analyzed + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" From 2b7ab814f9bde65ebc57ebd04386e56c97f06f4a Mon Sep 17 00:00:00 2001 From: William Benton Date: Mon, 8 Sep 2014 19:29:18 -0700 Subject: [PATCH 32/51] [SPARK-3329][SQL] Don't depend on Hive SET pair ordering in tests. This fixes some possible spurious test failures in `HiveQuerySuite` by comparing sets of key-value pairs as sets, rather than as lists. Author: William Benton Author: Aaron Davidson Closes #2220 from willb/spark-3329 and squashes the following commits: 3b3e205 [William Benton] Collapse collectResults case match in HiveQuerySuite 6525d8e [William Benton] Handle cases where SET returns Rows of (single) strings cf11b0e [Aaron Davidson] Fix flakey HiveQuerySuite test --- .../sql/hive/execution/HiveQuerySuite.scala | 47 ++++++++++--------- 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 305998c150327..6bf8d18a5c32c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -558,62 +558,67 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - + val KV = "([^=]+)=([^=]*)".r + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { + case Row(key: String, value: String) => key -> value + case Row(KV(key, value)) => key -> value + }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql(s"SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(hql("SET")) } // "set key" - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as sql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET")) } - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(sql(s"SET $nonexistentKey")) } clear() From 092e2f152fb674e7200cc8a2cb99a8fe0a9b2b33 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Sep 2014 20:51:56 -0700 Subject: [PATCH 33/51] SPARK-2425 Don't kill a still-running Application because of some misbehaving Executors Introduces a LOADING -> RUNNING ApplicationState transition and prevents Master from removing an Application with RUNNING Executors. Two basic changes: 1) Instead of allowing MAX_NUM_RETRY abnormal Executor exits over the entire lifetime of the Application, allow that many since any Executor successfully began running the Application; 2) Don't remove the Application while Master still thinks that there are RUNNING Executors. This should be fine as long as the ApplicationInfo doesn't believe any Executors are forever RUNNING when they are not. I think that any non-RUNNING Executors will eventually no longer be RUNNING in Master's accounting, but another set of eyes should confirm that. This PR also doesn't try to detect which nodes have gone rogue or to kill off bad Workers, so repeatedly failing Executors will continue to fail and fill up log files with failure reports as long as the Application keeps running. Author: Mark Hamstra Closes #1360 from markhamstra/SPARK-2425 and squashes the following commits: f099c0b [Mark Hamstra] Reuse appInfo b2b7b25 [Mark Hamstra] Moved 'Application failed' logging bdd0928 [Mark Hamstra] switched to string interpolation 1dd591b [Mark Hamstra] SPARK-2425 introduce LOADING -> RUNNING ApplicationState transition and prevent Master from removing Application with RUNNING Executors --- .../spark/deploy/master/ApplicationInfo.scala | 4 ++- .../apache/spark/deploy/master/Master.scala | 26 ++++++++++++------- .../spark/deploy/worker/ExecutorRunner.scala | 2 ++ .../apache/spark/deploy/worker/Worker.scala | 2 +- 4 files changed, 22 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index d3674427b1271..c3ca43f8d0734 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -96,11 +96,13 @@ private[spark] class ApplicationInfo( def retryCount = _retryCount - def incrementRetryCount = { + def incrementRetryCount() = { _retryCount += 1 _retryCount } + def resetRetryCount() = _retryCount = 0 + def markFinished(endState: ApplicationState.Value) { state = endState endTime = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2a66fcfe4801c..a3909d6ea95c0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -296,28 +296,34 @@ private[spark] class Master( val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId)) execOption match { case Some(exec) => { + val appInfo = idToApp(appId) exec.state = state + if (state == ExecutorState.RUNNING) { appInfo.resetRetryCount() } exec.application.driver ! ExecutorUpdated(execId, state, message, exitStatus) if (ExecutorState.isFinished(state)) { - val appInfo = idToApp(appId) // Remove this executor from the worker and app - logInfo("Removing executor " + exec.fullId + " because it is " + state) + logInfo(s"Removing executor ${exec.fullId} because it is $state") appInfo.removeExecutor(exec) exec.worker.removeExecutor(exec) - val normalExit = exitStatus.exists(_ == 0) + val normalExit = exitStatus == Some(0) // Only retry certain number of times so we don't go into an infinite loop. - if (!normalExit && appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { - schedule() - } else if (!normalExit) { - logError("Application %s with ID %s failed %d times, removing it".format( - appInfo.desc.name, appInfo.id, appInfo.retryCount)) - removeApplication(appInfo, ApplicationState.FAILED) + if (!normalExit) { + if (appInfo.incrementRetryCount() < ApplicationState.MAX_NUM_RETRY) { + schedule() + } else { + val execs = appInfo.executors.values + if (!execs.exists(_.state == ExecutorState.RUNNING)) { + logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + + s"${appInfo.retryCount} times; removing it") + removeApplication(appInfo, ApplicationState.FAILED) + } + } } } } case None => - logWarning("Got status update for unknown executor " + appId + "/" + execId) + logWarning(s"Got status update for unknown executor $appId/$execId") } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 7be89f9aff0f3..00a43673e5cd3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -159,6 +159,8 @@ private[spark] class ExecutorRunner( Files.write(header, stderr, Charsets.UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) + state = ExecutorState.RUNNING + worker ! ExecutorStateChanged(appId, execId, state, None, None) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) // or with nonzero exit code val exitCode = process.waitFor() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index e475567db6a20..0c454e4138c96 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -234,7 +234,7 @@ private[spark] class Worker( try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.RUNNING) + self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ From ce5cb325877e3fa8281ffe2076f93b4124ed0eb5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 9 Sep 2014 00:50:59 -0700 Subject: [PATCH 34/51] [Build] Removed -Phive-thriftserver since this profile has been removed Author: Cheng Lian Closes #2269 from liancheng/clean-run-tests-profile and squashes the following commits: 08617bd [Cheng Lian] Removed -Phive-thriftserver since this profile has been removed --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 49a88085c80f7..79401213a7fa2 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -93,7 +93,7 @@ echo "=========================================================================" # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. -BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver " +BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive " echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly | \ grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" From c419e4f1bd2a50c558179b7118c3fe75a94b7a5b Mon Sep 17 00:00:00 2001 From: Mario Pastorelli Date: Tue, 9 Sep 2014 00:51:28 -0700 Subject: [PATCH 35/51] [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2 Comment of the storageLevel param of actorStream says that it defaults to memory-only while the default is MEMORY_AND_DISK_SER_2. Author: Mario Pastorelli Closes #2319 from melrief/master and squashes the following commits: 7b6ce68 [Mario Pastorelli] [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2 --- .../scala/org/apache/spark/streaming/StreamingContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 101cec1c7a7c2..457e8ab28ed82 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -240,7 +240,7 @@ class StreamingContext private[streaming] ( * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param props Props object defining creation of the actor * @param name Name of the actor - * @param storageLevel RDD storage level. Defaults to memory-only. + * @param storageLevel RDD storage level (default: StorageLevel.MEMORY_AND_DISK_SER_2) * * @note An important point to note: * Since Actor may exist outside the spark framework, It is thus user's responsibility From 1e03cf79f82b166b2e18dcbd181e074f0276a0a9 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 9 Sep 2014 10:18:25 -0700 Subject: [PATCH 36/51] [SPARK-3455] [SQL] **HOT FIX** Fix the unit test failure Unit test failed due to can not resolve the attribute references. Temporally disable this test case for a quick fixing, otherwise it will block the others. Author: Cheng Hao Closes #2334 from chenghao-intel/unit_test_failure and squashes the following commits: 661f784 [Cheng Hao] temporally disable the failed test case --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 45c0ca8ea101d..739c12f338f34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -360,6 +360,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3349 partitioning after limit") { + /* sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") @@ -374,6 +375,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), (1, "a", 1) :: (2, "b", 2) :: Nil) + */ } test("mixed-case keywords") { From 88547a09fcc25df132b401ecec4ebe1ef6778576 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 9 Sep 2014 10:23:28 -0700 Subject: [PATCH 37/51] SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere. Author: Sandy Ryza Closes #2324 from sryza/sandy-spark-3422 and squashes the following commits: 6446175 [Sandy Ryza] SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere. --- .../java/org/apache/spark/JavaAPISuite.java | 25 ------------------- 1 file changed, 25 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index be99dc501c4b2..b8574dfb42e6b 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -29,19 +29,14 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; -import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapreduce.Job; import org.junit.After; import org.junit.Assert; @@ -49,7 +44,6 @@ import org.junit.Test; import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaHadoopRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -1313,23 +1307,4 @@ public void collectUnderlyingScalaRDD() { SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); Assert.assertEquals(data.size(), collected.length); } - - public void getHadoopInputSplits() { - String outDir = new File(tempDir, "output").getAbsolutePath(); - sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2).saveAsTextFile(outDir); - - JavaHadoopRDD hadoopRDD = (JavaHadoopRDD) - sc.hadoopFile(outDir, TextInputFormat.class, LongWritable.class, Text.class); - List inputPaths = hadoopRDD.mapPartitionsWithInputSplit( - new Function2>, Iterator>() { - @Override - public Iterator call(InputSplit split, Iterator> it) - throws Exception { - FileSplit fileSplit = (FileSplit) split; - return Lists.newArrayList(fileSplit.getPath().toUri().getPath()).iterator(); - } - }, true).collect(); - Assert.assertEquals(Sets.newHashSet(inputPaths), - Sets.newHashSet(outDir + "/part-00000", outDir + "/part-00001")); - } } From f0f1ba09b195f23f0c89af6fa040c9e01dfa8951 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 9 Sep 2014 10:24:00 -0700 Subject: [PATCH 38/51] SPARK-3404 [BUILD] SparkSubmitSuite fails with "spark-submit exits with code 1" This fixes the `SparkSubmitSuite` failure by setting `0` in the Maven build, to match the SBT build. This avoids a port conflict which causes failures. (This also updates the `scalatest` plugin off of a release candidate, to the identical final release.) Author: Sean Owen Closes #2328 from srowen/SPARK-3404 and squashes the following commits: 512d782 [Sean Owen] Set spark.ui.port=0 in Maven scalatest config to match SBT build and avoid SparkSubmitSuite failure due to port conflict --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d05190512f742..64fb1e57e30e0 100644 --- a/pom.xml +++ b/pom.xml @@ -888,7 +888,7 @@ org.scalatest scalatest-maven-plugin - 1.0-RC2 + 1.0 ${project.build.directory}/surefire-reports . @@ -899,6 +899,7 @@ true ${session.executionRootDirectory} 1 + 0 From 26862337c97ce14794178d6378fb4155dd24acb9 Mon Sep 17 00:00:00 2001 From: scwf Date: Tue, 9 Sep 2014 11:57:01 -0700 Subject: [PATCH 39/51] [SPARK-3193]output errer info when Process exit code is not zero in test suite https://issues.apache.org/jira/browse/SPARK-3193 I noticed that sometimes pr tests failed due to the Process exitcode != 0,refer to https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18688/consoleFull https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19118/consoleFull [info] SparkSubmitSuite: [info] - prints usage on empty input [info] - prints usage with only --help [info] - prints error with unrecognized options [info] - handle binary specified but not class [info] - handles arguments with --key=val [info] - handles arguments to user program [info] - handles arguments to user program with name collision [info] - handles YARN cluster mode [info] - handles YARN client mode [info] - handles standalone cluster mode [info] - handles standalone client mode [info] - handles mesos client mode [info] - handles confs with flag equivalents [info] - launch simple application with spark-submit *** FAILED *** [info] org.apache.spark.SparkException: Process List(./bin/spark-submit, --class, org.apache.spark.deploy.SimpleApplicationTest, --name, testApp, --master, local, file:/tmp/1408854098404-0/testJar-1408854098404.jar) exited with code 1 [info] at org.apache.spark.util.Utils$.executeAndGetOutput(Utils.scala:872) [info] at org.apache.spark.deploy.SparkSubmitSuite.runSparkSubmit(SparkSubmitSuite.scala:311) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply$mcV$sp(SparkSubmitSuite.scala:291) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply(SparkSubmitSuite.scala:284) [info] at org.apacSpark assembly has been built with Hive, including Datanucleus jars on classpath this PR output the process error info when failed, it can be helpful for diagnosis. Author: scwf Closes #2108 from scwf/output-test-error-info and squashes the following commits: 0c48082 [scwf] minor fix according to comments 563fde1 [scwf] output errer info when Process exitcode not zero --- .../scala/org/apache/spark/util/Utils.scala | 19 ++++++++++++++++++- .../scala/org/apache/spark/DriverSuite.scala | 5 +---- .../spark/deploy/SparkSubmitSuite.scala | 2 ++ 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0ae28f911e302..79943766d0f0f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -20,9 +20,11 @@ package org.apache.spark.util import java.io._ import java.net._ import java.nio.ByteBuffer -import java.util.{Locale, Random, UUID} +import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import org.apache.log4j.PropertyConfigurator + import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -834,6 +836,7 @@ private[spark] object Utils extends Logging { val exitCode = process.waitFor() stdoutThread.join() // Wait for it to finish reading output if (exitCode != 0) { + logError(s"Process $command exited with code $exitCode: ${output}") throw new SparkException("Process " + command + " exited with code " + exitCode) } output.toString @@ -1444,6 +1447,20 @@ private[spark] object Utils extends Logging { } } + /** + * config a log4j properties used for testsuite + */ + def configTestLog4j(level: String): Unit = { + val pro = new Properties() + pro.put("log4j.rootLogger", s"$level, console") + pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender") + pro.put("log4j.appender.console.target", "System.err") + pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout") + pro.put("log4j.appender.console.layout.ConversionPattern", + "%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n") + PropertyConfigurator.configure(pro) + } + } /** diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index a73e1ef0288a5..4b1d280624c57 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark import java.io.File -import org.apache.log4j.Logger -import org.apache.log4j.Level - import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ @@ -54,7 +51,7 @@ class DriverSuite extends FunSuite with Timeouts { */ object DriverWithoutCleanup { def main(args: Array[String]) { - Logger.getRootLogger().setLevel(Level.WARN) + Utils.configTestLog4j("INFO") val sc = new SparkContext(args(0), "DriverWithoutCleanup") sc.parallelize(1 to 100, 4).count() } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 7e1ef80c84561..22b369a829418 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -317,6 +317,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { object JarCreationTest { def main(args: Array[String]) { + Utils.configTestLog4j("INFO") val conf = new SparkConf() val sc = new SparkContext(conf) val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => @@ -338,6 +339,7 @@ object JarCreationTest { object SimpleApplicationTest { def main(args: Array[String]) { + Utils.configTestLog4j("INFO") val conf = new SparkConf() val sc = new SparkContext(conf) val configs = Seq("spark.master", "spark.app.name") From 02b5ac7191c66a866ffedde313eb10f2adfc9b58 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 9 Sep 2014 14:42:28 -0700 Subject: [PATCH 40/51] Minor - Fix trivial compilation warnings. Author: Prashant Sharma Closes #2331 from ScrapCodes/compilation-warn and squashes the following commits: 44c1e76 [Prashant Sharma] Minor - Fix trivial compilation warnings. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 - .../org/apache/spark/examples/graphx/LiveJournalPageRank.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 24d1a8f9eceae..c6c5b8f22b549 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -49,7 +49,6 @@ 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} diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index bdc8fa7f99f2e..e809a65b79975 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.graphx import org.apache.spark.SparkContext._ import org.apache.spark._ import org.apache.spark.graphx._ -import org.apache.spark.examples.graphx.Analytics + /** * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from From 07ee4a28c3a502121770f301316cb2256e8f0ce2 Mon Sep 17 00:00:00 2001 From: xinyunh Date: Tue, 9 Sep 2014 16:55:39 -0700 Subject: [PATCH 41/51] [SPARK-3176] Implement 'ABS and 'LAST' for sql Add support for the mathematical function"ABS" and the analytic function "last" to return a subset of the rows satisfying a query within spark sql. Test-cases included. Author: xinyunh Author: bomeng Closes #2099 from xinyunh/sqlTest and squashes the following commits: 71d15e7 [xinyunh] remove POWER part 8843643 [xinyunh] fix the code style issue 39f0309 [bomeng] Modify the code of POWER and ABS. Move them to the file arithmetic ff8e51e [bomeng] add abs() function support 7f6980a [xinyunh] fix the bug in 'Last' component b3df91b [xinyunh] add 'Last' component --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +++ .../spark/sql/catalyst/dsl/package.scala | 1 + .../sql/catalyst/expressions/aggregates.scala | 28 +++++++++++++++++++ .../sql/catalyst/expressions/arithmetic.scala | 15 ++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 23 +++++++++++++-- 5 files changed, 69 insertions(+), 2 deletions(-) mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala old mode 100644 new mode 100755 index bfc197cf7a938..a04b4a938da64 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -82,6 +82,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val DISTINCT = Keyword("DISTINCT") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") + protected val LAST = Keyword("LAST") protected val FROM = Keyword("FROM") protected val FULL = Keyword("FULL") protected val GROUP = Keyword("GROUP") @@ -125,6 +126,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val SUBSTR = Keyword("SUBSTR") protected val SUBSTRING = Keyword("SUBSTRING") protected val SQRT = Keyword("SQRT") + protected val ABS = Keyword("ABS") // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -315,6 +317,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { case s ~ _ ~ _ ~ _ ~ _ ~ e => ApproxCountDistinct(e, s.toDouble) } | FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | + LAST ~> "(" ~> expression <~ ")" ^^ { case exp => Last(exp) } | AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | MAX ~> "(" ~> expression <~ ")" ^^ { case exp => Max(exp) } | @@ -330,6 +333,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { case s ~ "," ~ p ~ "," ~ l => Substring(s,p,l) } | SQRT ~> "(" ~> expression <~ ")" ^^ { case exp => Sqrt(exp) } | + ABS ~> "(" ~> expression <~ ")" ^^ { case exp => Abs(exp) } | ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ { case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala old mode 100644 new mode 100755 index f44521d6381c9..deb622c39faf5 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -132,6 +132,7 @@ package object dsl { def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd) def avg(e: Expression) = Average(e) def first(e: Expression) = First(e) + def last(e: Expression) = Last(e) def min(e: Expression) = Min(e) def max(e: Expression) = Max(e) def upper(e: Expression) = Upper(e) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala old mode 100644 new mode 100755 index 15560a2a933ad..1b4d892625dbb --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -344,6 +344,21 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod override def newInstance() = new FirstFunction(child, this) } +case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = true + override def dataType = child.dataType + override def toString = s"LAST($child)" + + override def asPartial: SplitEvaluation = { + val partialLast = Alias(Last(child), "PartialLast")() + SplitEvaluation( + Last(partialLast.toAttribute), + partialLast :: Nil) + } + override def newInstance() = new LastFunction(child, this) +} + case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -489,3 +504,16 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag override def eval(input: Row): Any = result } + +case class LastFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var result: Any = null + + override def update(input: Row): Unit = { + result = input + } + + override def eval(input: Row): Any = if (result != null) expr.eval(result.asInstanceOf[Row]) + else null +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index f988fb010b107..fe825fdcdae37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types._ +import scala.math.pow case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any @@ -129,3 +130,17 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def toString = s"MaxOf($left, $right)" } + +/** + * A function that get the absolute value of the numeric value. + */ +case class Abs(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def dataType = child.dataType + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"Abs($child)" + + override def eval(input: Row): Any = n1(child, input, _.abs(_)) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 739c12f338f34..514ac543df92a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -41,6 +41,25 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } + test("SPARK-3176 Added Parser of SQL ABS()") { + checkAnswer( + sql("SELECT ABS(-1.3)"), + 1.3) + checkAnswer( + sql("SELECT ABS(0.0)"), + 0.0) + checkAnswer( + sql("SELECT ABS(2.5)"), + 2.5) + } + + test("SPARK-3176 Added Parser of SQL LAST()") { + checkAnswer( + sql("SELECT LAST(n) FROM lowerCaseData"), + 4) + } + + test("SPARK-2041 column name equals tablename") { checkAnswer( sql("SELECT tableName FROM tableName"), @@ -53,14 +72,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq ) } - + test("SQRT with automatic string casts") { checkAnswer( sql("SELECT SQRT(CAST(key AS STRING)) FROM testData"), (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq ) } - + test("SPARK-2407 Added Parser of SQL SUBSTR()") { checkAnswer( sql("SELECT substr(tableName, 1, 2) FROM tableName"), From c110614b33a690a3db6ccb1a920fb6a3795aa5a0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 9 Sep 2014 18:39:33 -0700 Subject: [PATCH 42/51] [SPARK-3448][SQL] Check for null in SpecificMutableRow.update `SpecificMutableRow.update` doesn't check for null, and breaks existing `MutableRow` contract. The tricky part here is that for performance considerations, the `update` method of all subclasses of `MutableValue` doesn't check for null and sets the null bit to false. Author: Cheng Lian Closes #2325 from liancheng/check-for-null and squashes the following commits: 9366c44 [Cheng Lian] Check for null in SpecificMutableRow.update --- .../spark/sql/catalyst/expressions/SpecificRow.scala | 4 +++- .../src/test/scala/org/apache/spark/sql/RowSuite.scala | 8 +++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala index 75ea0e8459df8..088f11ee4aa53 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -227,7 +227,9 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR new SpecificMutableRow(newValues) } - override def update(ordinal: Int, value: Any): Unit = values(ordinal).update(value) + override def update(ordinal: Int, value: Any): Unit = { + if (value == null) setNullAt(ordinal) else values(ordinal).update(value) + } override def iterator: Iterator[Any] = values.map(_.boxed).iterator diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 651cb735ab7d9..811319e0a6601 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} class RowSuite extends FunSuite { @@ -43,4 +43,10 @@ class RowSuite extends FunSuite { assert(expected.getBoolean(2) === actual2.getBoolean(2)) assert(expected(3) === actual2(3)) } + + test("SpecificMutableRow.update with null") { + val row = new SpecificMutableRow(Seq(IntegerType)) + row(0) = null + assert(row.isNullAt(0)) + } } From 25b5b867d5e18bac1c5bcdc6f8c63d97858194c7 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Tue, 9 Sep 2014 18:54:54 -0700 Subject: [PATCH 43/51] [SPARK-3458] enable python "with" statements for SparkContext allow for best practice code, ``` try: sc = SparkContext() app(sc) finally: sc.stop() ``` to be written using a "with" statement, ``` with SparkContext() as sc: app(sc) ``` Author: Matthew Farrellee Closes #2335 from mattf/SPARK-3458 and squashes the following commits: 5b4e37c [Matthew Farrellee] [SPARK-3458] enable python "with" statements for SparkContext --- python/pyspark/context.py | 14 ++++++++++++++ python/pyspark/tests.py | 29 +++++++++++++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5a30431568b16..84bc0a3b7ccd0 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -232,6 +232,20 @@ def _ensure_initialized(cls, instance=None, gateway=None): else: SparkContext._active_spark_context = instance + def __enter__(self): + """ + Enable 'with SparkContext(...) as sc: app(sc)' syntax. + """ + return self + + def __exit__(self, type, value, trace): + """ + Enable 'with SparkContext(...) as sc: app' syntax. + + Specifically stop the context on exit of the with block. + """ + self.stop() + @classmethod def setSystemProperty(cls, key, value): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0bd2a9e6c507d..bb84ebe72cb24 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1254,6 +1254,35 @@ def test_single_script_on_cluster(self): self.assertIn("[2, 4, 6]", out) +class ContextStopTests(unittest.TestCase): + + def test_stop(self): + sc = SparkContext() + self.assertNotEqual(SparkContext._active_spark_context, None) + sc.stop() + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with(self): + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with_exception(self): + try: + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + raise Exception() + except: + pass + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with_stop(self): + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + sc.stop() + self.assertEqual(SparkContext._active_spark_context, None) + + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): From b734ed0c229373dbc589b9eca7327537ca458138 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 9 Sep 2014 23:47:12 -0700 Subject: [PATCH 44/51] [SPARK-3395] [SQL] DSL sometimes incorrectly reuses attribute ids, breaking queries This resolves https://issues.apache.org/jira/browse/SPARK-3395 Author: Eric Liang Closes #2266 from ericl/spark-3395 and squashes the following commits: 7f2b6f0 [Eric Liang] add regression test 05bd1e4 [Eric Liang] in the dsl, create a new schema instance in each applySchema --- .../scala/org/apache/spark/sql/SchemaRDD.scala | 3 ++- .../scala/org/apache/spark/sql/DslQuerySuite.scala | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 33b2ed1b3a399..d2ceb4a2b0b25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -428,7 +428,8 @@ class SchemaRDD( */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { new SchemaRDD(sqlContext, - SparkLogicalPlan(ExistingRdd(queryExecution.analyzed.output, rdd))(sqlContext)) + SparkLogicalPlan( + ExistingRdd(queryExecution.analyzed.output.map(_.newInstance), rdd))(sqlContext)) } // ======================================================================= diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 1a6a6c17473a3..d001abb7e1fcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.test._ /* Implicits */ @@ -133,6 +135,18 @@ class DslQuerySuite extends QueryTest { mapData.take(1).toSeq) } + test("SPARK-3395 limit distinct") { + val filtered = TestData.testData2 + .distinct() + .orderBy(SortOrder('a, Ascending), SortOrder('b, Ascending)) + .limit(1) + .registerTempTable("onerow") + checkAnswer( + sql("select * from onerow inner join testData2 on onerow.a = testData2.a"), + (1, 1, 1, 1) :: + (1, 1, 1, 2) :: Nil) + } + test("average") { checkAnswer( testData2.groupBy()(avg('a)), From 6f7a76838f15687583e3b0ab43309a3c079368c4 Mon Sep 17 00:00:00 2001 From: Benoy Antony Date: Wed, 10 Sep 2014 11:59:39 -0500 Subject: [PATCH 45/51] =?UTF-8?q?[SPARK-3286]=20-=20Cannot=20view=20Applic?= =?UTF-8?q?ationMaster=20UI=20when=20Yarn=E2=80=99s=20url=20scheme=20i...?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ...s https Author: Benoy Antony Closes #2276 from benoyantony/SPARK-3286 and squashes the following commits: c3d51ee [Benoy Antony] Use address with scheme, but Allpha version removes the scheme e82f94e [Benoy Antony] Use address with scheme, but Allpha version removes the scheme 92127c9 [Benoy Antony] rebasing from master 450c536 [Benoy Antony] [SPARK-3286] - Cannot view ApplicationMaster UI when Yarn’s url scheme is https f060c02 [Benoy Antony] [SPARK-3286] - Cannot view ApplicationMaster UI when Yarn’s url scheme is https --- .../scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 4 +++- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index ad27a9ab781d2..fc30953011812 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import scala.collection.{Map, Set} +import java.net.URI; import org.apache.hadoop.net.NetUtils import org.apache.hadoop.yarn.api._ @@ -97,7 +98,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) - appMasterRequest.setTrackingUrl(uiAddress) + //remove the scheme from the url if it exists since Hadoop does not expect scheme + appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) resourceManager.registerApplicationMaster(appMasterRequest) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a879c833a014f..5756263e89e21 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -189,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIHostPort, securityMgr) + registerAM(sc.ui.appUIAddress, securityMgr) try { userThread.join() } finally { From a0283300c4af5e64a1dc06193245daa1e746b5f4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Sep 2014 10:45:15 -0700 Subject: [PATCH 46/51] [SPARK-3362][SQL] Fix resolution for casewhen with nulls. Current implementation will ignore else val type. Author: Daoyuan Wang Closes #2245 from adrian-wang/casewhenbug and squashes the following commits: 3332f6e [Daoyuan Wang] remove wrong comment 83b536c [Daoyuan Wang] a comment to trigger retest d7315b3 [Daoyuan Wang] code improve eed35fc [Daoyuan Wang] bug in casewhen resolve --- .../apache/spark/sql/catalyst/expressions/predicates.scala | 5 +++-- ...then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c | 1 + ...en 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 | 1 + ...hen 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb | 1 + ...hen 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 | 1 + ...hen 1Y else null end -0-589982a400d86157791c7216b10b6b5d | 1 + ...then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 | 1 + ...en null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 | 1 + ...hen null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 | 1 + ...hen null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 | 1 + ...hen null else 1Y end -0-7f4c32299c3738739b678ece62752a7b | 1 + .../spark/sql/hive/execution/HiveTypeCoercionSuite.scala | 6 ++++++ 12 files changed, 19 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c create mode 100644 sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 create mode 100644 sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb create mode 100644 sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 create mode 100644 sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 1313ccd120c1f..329af332d0fa1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -265,12 +265,13 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { false } else { val allCondBooleans = predicates.forall(_.dataType == BooleanType) - val dataTypesEqual = values.map(_.dataType).distinct.size <= 1 + // both then and else val should be considered. + val dataTypesEqual = (values ++ elseValue).map(_.dataType).distinct.size <= 1 allCondBooleans && dataTypesEqual } } - /** Written in imperative fashion for performance considerations. Same for CaseKeyWhen. */ + /** Written in imperative fashion for performance considerations. */ override def eval(input: Row): Any = { val len = branchesArr.length var i = 0 diff --git a/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c b/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 b/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb b/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 b/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d b/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 b/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 b/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 b/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 b/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b b/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index c3c18cf8ccac3..48fffe53cf2ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -33,6 +33,12 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { } } + val nullVal = "null" + baseTypes.init.foreach { i => + createQueryTest(s"case when then $i else $nullVal end ", s"SELECT case when true then $i else $nullVal end FROM src limit 1") + createQueryTest(s"case when then $nullVal else $i end ", s"SELECT case when true then $nullVal else $i end FROM src limit 1") + } + test("[SPARK-2210] boolean cast on boolean value should be removed") { val q = "select cast(cast(key=0 as boolean) as boolean) from src" val project = TestHive.sql(q).queryExecution.executedPlan.collect { case e: Project => e }.head From f0c87dc86ae65a39cd19370d8d960b4a60854517 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Sep 2014 10:48:33 -0700 Subject: [PATCH 47/51] [SPARK-3363][SQL] Type Coercion should promote null to all other types. Type Coercion should support every type to have null value Author: Daoyuan Wang Author: Michael Armbrust Closes #2246 from adrian-wang/spark3363-0 and squashes the following commits: c6241de [Daoyuan Wang] minor code clean 595b417 [Daoyuan Wang] Merge pull request #2 from marmbrus/pr/2246 832e640 [Michael Armbrust] reduce code duplication ef6f986 [Daoyuan Wang] make double boolean miss in jsonRDD compatibleType c619f0a [Daoyuan Wang] Type Coercion should support every type to have null value --- .../catalyst/analysis/HiveTypeCoercion.scala | 38 +++++++------- .../analysis/HiveTypeCoercionSuite.scala | 32 +++++++++--- .../org/apache/spark/sql/json/JsonRDD.scala | 51 ++++++++----------- 3 files changed, 67 insertions(+), 54 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index d6758eb5b6a32..bd8131c9af6e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -26,10 +26,22 @@ object HiveTypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: val numericPrecedence = - Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) - // Boolean is only wider than Void - val booleanPrecedence = Seq(NullType, BooleanType) - val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil + Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) + val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: Nil + + def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { + val valueTypes = Seq(t1, t2).filter(t => t != NullType) + if (valueTypes.distinct.size > 1) { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + } else { + Some(if (valueTypes.size == 0) NullType else valueTypes.head) + } + } } /** @@ -53,17 +65,6 @@ trait HiveTypeCoercion { Division :: Nil - trait TypeWidening { - def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - } - } - /** * Applies any changes to [[AttributeReference]] data types that are made by other rules to * instances higher in the query tree. @@ -144,7 +145,8 @@ trait HiveTypeCoercion { * - LongType to FloatType * - LongType to DoubleType */ - object WidenTypes extends Rule[LogicalPlan] with TypeWidening { + object WidenTypes extends Rule[LogicalPlan] { + import HiveTypeCoercion._ def apply(plan: LogicalPlan): LogicalPlan = plan transform { case u @ Union(left, right) if u.childrenResolved && !u.resolved => @@ -352,7 +354,9 @@ trait HiveTypeCoercion { /** * Coerces the type of different branches of a CASE WHEN statement to a common type. */ - object CaseWhenCoercion extends Rule[LogicalPlan] with TypeWidening { + object CaseWhenCoercion extends Rule[LogicalPlan] { + import HiveTypeCoercion._ + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => val valueTypes = branches.sliding(2, 2).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index b9e0f8e9dcc5f..ba8b853b6f99e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -23,20 +23,20 @@ import org.apache.spark.sql.catalyst.types._ class HiveTypeCoercionSuite extends FunSuite { - val rules = new HiveTypeCoercion { } - import rules._ - - test("tightest common bound for numeric and boolean types") { + test("tightest common bound for types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { - var found = WidenTypes.findTightestCommonType(t1, t2) + var found = HiveTypeCoercion.findTightestCommonType(t1, t2) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found") // Test both directions to make sure the widening is symmetric. - found = WidenTypes.findTightestCommonType(t2, t1) + found = HiveTypeCoercion.findTightestCommonType(t2, t1) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found") } + // Null + widenTest(NullType, NullType, Some(NullType)) + // Boolean widenTest(NullType, BooleanType, Some(BooleanType)) widenTest(BooleanType, BooleanType, Some(BooleanType)) @@ -60,12 +60,28 @@ class HiveTypeCoercionSuite extends FunSuite { widenTest(DoubleType, DoubleType, Some(DoubleType)) // Integral mixed with floating point. - widenTest(NullType, FloatType, Some(FloatType)) - widenTest(NullType, DoubleType, Some(DoubleType)) widenTest(IntegerType, FloatType, Some(FloatType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) widenTest(LongType, FloatType, Some(FloatType)) widenTest(LongType, DoubleType, Some(DoubleType)) + + // StringType + widenTest(NullType, StringType, Some(StringType)) + widenTest(StringType, StringType, Some(StringType)) + widenTest(IntegerType, StringType, None) + widenTest(LongType, StringType, None) + + // TimestampType + widenTest(NullType, TimestampType, Some(TimestampType)) + widenTest(TimestampType, TimestampType, Some(TimestampType)) + widenTest(IntegerType, TimestampType, None) + widenTest(StringType, TimestampType, None) + + // ComplexType + widenTest(NullType, MapType(IntegerType, StringType, false), Some(MapType(IntegerType, StringType, false))) + widenTest(NullType, StructType(Seq()), Some(StructType(Seq()))) + widenTest(StringType, MapType(IntegerType, StringType, true), None) + widenTest(ArrayType(IntegerType), StructType(Seq()), None) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 1c0b03c684f10..70062eae3b7ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -125,38 +125,31 @@ private[sql] object JsonRDD extends Logging { * Returns the most general data type for two given data types. */ private[json] def compatibleType(t1: DataType, t2: DataType): DataType = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p - .contains(t2)) - - // If found return the widest common type, otherwise None - val returnType = applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - - if (returnType.isDefined) { - returnType.get - } else { - // t1 or t2 is a StructType, ArrayType, or an unexpected type. - (t1, t2) match { - case (other: DataType, NullType) => other - case (NullType, other: DataType) => other - case (StructType(fields1), StructType(fields2)) => { - val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { - case (name, fieldTypes) => { - val dataType = fieldTypes.map(field => field.dataType).reduce( - (type1: DataType, type2: DataType) => compatibleType(type1, type2)) - StructField(name, dataType, true) + HiveTypeCoercion.findTightestCommonType(t1, t2) match { + case Some(commonType) => commonType + case None => + // t1 or t2 is a StructType, ArrayType, or an unexpected type. + (t1, t2) match { + case (other: DataType, NullType) => other + case (NullType, other: DataType) => other + case (StructType(fields1), StructType(fields2)) => { + val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { + case (name, fieldTypes) => { + val dataType = fieldTypes.map(field => field.dataType).reduce( + (type1: DataType, type2: DataType) => compatibleType(type1, type2)) + StructField(name, dataType, true) + } } + StructType(newFields.toSeq.sortBy { + case StructField(name, _, _) => name + }) } - StructType(newFields.toSeq.sortBy { - case StructField(name, _, _) => name - }) + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => + ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) + // TODO: We should use JsonObjectStringType to mark that values of field will be + // strings and every string is a Json object. + case (_, _) => StringType } - case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => - ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) - // TODO: We should use JsonObjectStringType to mark that values of field will be - // strings and every string is a Json object. - case (_, _) => StringType - } } } From 26503fdf20f4181a2b390c88b83f364e6a4ccc21 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 Sep 2014 12:02:23 -0700 Subject: [PATCH 48/51] [HOTFIX] Fix scala style issue introduced by #2276. --- .../scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index fc30953011812..acf26505e4cf9 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -98,7 +98,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) - //remove the scheme from the url if it exists since Hadoop does not expect scheme + // remove the scheme from the url if it exists since Hadoop does not expect scheme appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) resourceManager.registerApplicationMaster(appMasterRequest) } From 1f4a648d4e30e837d6cf3ea8de1808e2254ad70b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 10 Sep 2014 14:34:24 -0500 Subject: [PATCH 49/51] SPARK-1713. Use a thread pool for launching executors. This patch copies the approach used in the MapReduce application master for launching containers. Author: Sandy Ryza Closes #663 from sryza/sandy-spark-1713 and squashes the following commits: 036550d [Sandy Ryza] SPARK-1713. [YARN] Use a threadpool for launching executor containers --- docs/running-on-yarn.md | 7 +++++++ .../apache/spark/deploy/yarn/YarnAllocator.scala | 14 ++++++++++++-- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 943f06b114cb9..d8b22f3663d08 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -125,6 +125,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes the environment of the executor launcher. + + spark.yarn.containerLauncherMaxThreads + 25 + + The maximum number of threads to use in the application master for launching executor containers. + + # Launching Spark on YARN diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 02b9a81bf6b50..0b8744f4b8bdf 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import java.util.{List => JList} -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ @@ -32,6 +32,8 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import com.google.common.util.concurrent.ThreadFactoryBuilder + object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value @@ -95,6 +97,14 @@ private[yarn] abstract class YarnAllocator( protected val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, preferredNodes) + private val launcherPool = new ThreadPoolExecutor( + // max pool size of Integer.MAX_VALUE is ignored because we use an unbounded queue + sparkConf.getInt("spark.yarn.containerLauncherMaxThreads", 25), Integer.MAX_VALUE, + 1, TimeUnit.MINUTES, + new LinkedBlockingQueue[Runnable](), + new ThreadFactoryBuilder().setNameFormat("ContainerLauncher #%d").setDaemon(true).build()) + launcherPool.allowCoreThreadTimeOut(true) + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue def getNumExecutorsFailed: Int = numExecutorsFailed.intValue @@ -283,7 +293,7 @@ private[yarn] abstract class YarnAllocator( executorMemory, executorCores, securityMgr) - new Thread(executorRunnable).start() + launcherPool.execute(executorRunnable) } } logDebug(""" From e4f4886d7148bf48f9e3462b83bfb1ecc7edbe31 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 10 Sep 2014 12:56:59 -0700 Subject: [PATCH 50/51] [SPARK-2096][SQL] Correctly parse dot notations First let me write down the current `projections` grammar of spark sql: expression : orExpression orExpression : andExpression {"or" andExpression} andExpression : comparisonExpression {"and" comparisonExpression} comparisonExpression : termExpression | termExpression "=" termExpression | termExpression ">" termExpression | ... termExpression : productExpression {"+"|"-" productExpression} productExpression : baseExpression {"*"|"/"|"%" baseExpression} baseExpression : expression "[" expression "]" | ... | ident | ... ident : identChar {identChar | digit} | delimiters | ... identChar : letter | "_" | "." delimiters : "," | ";" | "(" | ")" | "[" | "]" | ... projection : expression [["AS"] ident] projections : projection { "," projection} For something like `a.b.c[1]`, it will be parsed as: But for something like `a[1].b`, the current grammar can't parse it correctly. A simple solution is written in `ParquetQuerySuite#NestedSqlParser`, changed grammars are: delimiters : "." | "," | ";" | "(" | ")" | "[" | "]" | ... identChar : letter | "_" baseExpression : expression "[" expression "]" | expression "." ident | ... | ident | ... This works well, but can't cover some corner case like `select t.a.b from table as t`: `t.a.b` parsed as `GetField(GetField(UnResolved("t"), "a"), "b")` instead of `GetField(UnResolved("t.a"), "b")` using this new grammar. However, we can't resolve `t` as it's not a filed, but the whole table.(if we could do this, then `select t from table as t` is legal, which is unexpected) My solution is: dotExpressionHeader : ident "." ident baseExpression : expression "[" expression "]" | expression "." ident | ... | dotExpressionHeader | ident | ... I passed all test cases under sql locally and add a more complex case. "arrayOfStruct.field1 to access all values of field1" is not supported yet. Since this PR has changed a lot of code, I will open another PR for it. I'm not familiar with the latter optimize phase, please correct me if I missed something. Author: Wenchen Fan Author: Michael Armbrust Closes #2230 from cloud-fan/dot and squashes the following commits: e1a8898 [Wenchen Fan] remove support for arbitrary nested arrays ee8a724 [Wenchen Fan] rollback LogicalPlan, support dot operation on nested array type a58df40 [Michael Armbrust] add regression test for doubly nested data 16bc4c6 [Wenchen Fan] some enhance 95d733f [Wenchen Fan] split long line dc31698 [Wenchen Fan] SPARK-2096 Correctly parse dot notations --- .../apache/spark/sql/catalyst/SqlParser.scala | 13 ++- .../catalyst/plans/logical/LogicalPlan.scala | 6 +- .../org/apache/spark/sql/json/JsonSuite.scala | 14 +++ .../apache/spark/sql/json/TestJsonData.scala | 26 +++++ .../spark/sql/parquet/ParquetQuerySuite.scala | 102 +++++------------- .../sql/hive/execution/SQLQuerySuite.scala | 17 ++- 6 files changed, 88 insertions(+), 90 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a04b4a938da64..ca69531c69a77 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -357,16 +357,25 @@ class SqlParser extends StandardTokenParsers with PackratParsers { expression ~ "[" ~ expression <~ "]" ^^ { case base ~ _ ~ ordinal => GetItem(base, ordinal) } | + (expression <~ ".") ~ ident ^^ { + case base ~ fieldName => GetField(base, fieldName) + } | TRUE ^^^ Literal(true, BooleanType) | FALSE ^^^ Literal(false, BooleanType) | cast | "(" ~> expression <~ ")" | function | "-" ~> literal ^^ UnaryMinus | + dotExpressionHeader | ident ^^ UnresolvedAttribute | "*" ^^^ Star(None) | literal + protected lazy val dotExpressionHeader: Parser[Expression] = + (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { + case i1 ~ i2 ~ rest => UnresolvedAttribute(i1 + "." + i2 + rest.mkString(".", ".", "")) + } + protected lazy val dataType: Parser[DataType] = STRING ^^^ StringType | TIMESTAMP ^^^ TimestampType } @@ -380,7 +389,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":", "[", "]" + ",", ";", "%", "{", "}", ":", "[", "]", "." ) override lazy val token: Parser[Token] = ( @@ -401,7 +410,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { | failure("illegal character") ) - override def identChar = letter | elem('_') | elem('.') + override def identChar = letter | elem('_') override def whitespace: Parser[Any] = rep( whitespaceChar diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index f81d9111945f5..bae491f07c13f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -104,11 +104,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { case Seq((a, Nil)) => Some(a) // One match, no nested fields, use it. // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - a.dataType match { - case StructType(fields) => - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - case _ => None // Don't know how to resolve these field references - } + Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) case Seq() => None // No matches. case ambiguousReferences => throw new TreeNodeException( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 05513a127150c..301d482d27d86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -581,4 +581,18 @@ class JsonSuite extends QueryTest { "this is a simple string.") :: Nil ) } + + test("SPARK-2096 Correctly parse dot notations") { + val jsonSchemaRDD = jsonRDD(complexFieldAndType2) + jsonSchemaRDD.registerTempTable("jsonTable") + + checkAnswer( + sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), + (true, "str1") :: Nil + ) + checkAnswer( + sql("select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1] from jsonTable"), + ("str2", 6) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index a88310b5f1b46..b3f95f08e8044 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -82,4 +82,30 @@ object TestJsonData { """{"c":[33, 44]}""" :: """{"d":{"field":true}}""" :: """{"e":"str"}""" :: Nil) + + val complexFieldAndType2 = + TestSQLContext.sparkContext.parallelize( + """{"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], + "complexArrayOfStruct": [ + { + "field1": [ + { + "inner1": "str1" + }, + { + "inner2": ["str2", "str22"] + }], + "field2": [[1, 2], [3, 4]] + }, + { + "field1": [ + { + "inner2": ["str3", "str33"] + }, + { + "inner1": "str4" + }], + "field2": [[5, 6], [7, 8]] + }] + }""" :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 42923b6a288d9..b0a06cd3ca090 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.parquet +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapreduce.Job import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} - import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.mapreduce.Job - -import org.apache.spark.SparkContext import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser} -import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{BooleanType, IntegerType} +import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ @@ -87,11 +82,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA var testRDD: SchemaRDD = null - // TODO: remove this once SqlParser can parse nested select statements - var nestedParserSqlContext: NestedParserSQLContext = null - override def beforeAll() { - nestedParserSqlContext = new NestedParserSQLContext(TestSQLContext.sparkContext) ParquetTestData.writeFile() ParquetTestData.writeFilterFile() ParquetTestData.writeNestedFile1() @@ -718,11 +709,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection in addressbook") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir1.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir1.toString).toSchemaRDD data.registerTempTable("data") - val query = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM data") + val query = sql("SELECT owner, contacts[1].name FROM data") val tmp = query.collect() assert(tmp.size === 2) assert(tmp(0).size === 2) @@ -733,21 +722,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Simple query on nested int data") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir2.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir2.toString).toSchemaRDD data.registerTempTable("data") - val result1 = nestedParserSqlContext.sql("SELECT entries[0].value FROM data").collect() + val result1 = sql("SELECT entries[0].value FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === 2.5) - val result2 = nestedParserSqlContext.sql("SELECT entries[0] FROM data").collect() + val result2 = sql("SELECT entries[0] FROM data").collect() assert(result2.size === 1) val subresult1 = result2(0)(0).asInstanceOf[CatalystConverter.StructScalaType[_]] assert(subresult1.size === 2) assert(subresult1(0) === 2.5) assert(subresult1(1) === false) - val result3 = nestedParserSqlContext.sql("SELECT outerouter FROM data").collect() + val result3 = sql("SELECT outerouter FROM data").collect() val subresult2 = result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] @@ -760,19 +747,18 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("nested structs") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir3.toString) + val data = parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD data.registerTempTable("data") - val result1 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() + val result1 = sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === false) - val result2 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() + val result2 = sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() assert(result2.size === 1) assert(result2(0).size === 1) assert(result2(0)(0) === true) - val result3 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() + val result3 = sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() assert(result3.size === 1) assert(result3(0).size === 1) assert(result3(0)(0) === false) @@ -796,11 +782,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("map with struct values") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir4.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD data.registerTempTable("mapTable") - val result1 = nestedParserSqlContext.sql("SELECT data2 FROM mapTable").collect() + val result1 = sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) val entry1 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] @@ -814,7 +798,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result2 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() + val result2 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() assert(result2.size === 1) assert(result2(0)(0) === 42.toLong) assert(result2(0)(1) === "the answer") @@ -825,15 +809,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // has no effect in this test case val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) - val result = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir1.toString) - .toSchemaRDD + val result = parquetFile(ParquetTestData.testNestedDir1.toString).toSchemaRDD result.saveAsParquetFile(tmpdir.toString) - nestedParserSqlContext - .parquetFile(tmpdir.toString) + parquetFile(tmpdir.toString) .toSchemaRDD .registerTempTable("tmpcopy") - val tmpdata = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() + val tmpdata = sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() assert(tmpdata.size === 2) assert(tmpdata(0).size === 2) assert(tmpdata(0)(0) === "Julien Le Dem") @@ -844,20 +825,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Writing out Map and reading it back in") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir4.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) data.saveAsParquetFile(tmpdir.toString) - nestedParserSqlContext - .parquetFile(tmpdir.toString) + parquetFile(tmpdir.toString) .toSchemaRDD .registerTempTable("tmpmapcopy") - val result1 = nestedParserSqlContext.sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() + val result1 = sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() assert(result1.size === 1) assert(result1(0)(0) === 2) - val result2 = nestedParserSqlContext.sql("SELECT data2 FROM tmpmapcopy").collect() + val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() assert(result2.size === 1) val entry1 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] @@ -871,42 +849,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result3 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() + val result3 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() assert(result3.size === 1) assert(result3(0)(0) === 42.toLong) assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } } - -// TODO: the code below is needed temporarily until the standard parser is able to parse -// nested field expressions correctly -class NestedParserSQLContext(@transient override val sparkContext: SparkContext) extends SQLContext(sparkContext) { - override protected[sql] val parser = new NestedSqlParser() -} - -class NestedSqlLexical(override val keywords: Seq[String]) extends SqlLexical(keywords) { - override def identChar = letter | elem('_') - delimiters += (".") -} - -class NestedSqlParser extends SqlParser { - override val lexical = new NestedSqlLexical(reservedWords) - - override protected lazy val baseExpression: PackratParser[Expression] = - expression ~ "[" ~ expression <~ "]" ^^ { - case base ~ _ ~ ordinal => GetItem(base, ordinal) - } | - expression ~ "." ~ ident ^^ { - case base ~ _ ~ fieldName => GetField(base, fieldName) - } | - TRUE ^^^ Literal(true, BooleanType) | - FALSE ^^^ Literal(false, BooleanType) | - cast | - "(" ~> expression <~ ")" | - function | - "-" ~> literal ^^ UnaryMinus | - ident ^^ UnresolvedAttribute | - "*" ^^^ Star(None) | - literal -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 635a9fb0d56cb..b99caf77bce28 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.hive.execution -import scala.reflect.ClassTag - -import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHive._ +case class Nested1(f1: Nested2) +case class Nested2(f2: Nested3) +case class Nested3(f3: Int) + /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is @@ -47,4 +47,11 @@ class SQLQuerySuite extends QueryTest { GROUP BY key, value ORDER BY value) a""").collect().toSeq) } + + test("double nested data") { + sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil).registerTempTable("nested") + checkAnswer( + sql("SELECT f1.f2.f3 FROM nested"), + 1) + } } From 558962a83fb0758ab5c13ff4ea58cc96c29cbbcc Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Wed, 10 Sep 2014 13:06:47 -0700 Subject: [PATCH 51/51] [SPARK-3411] Improve load-balancing of concurrently-submitted drivers across workers If the waiting driver array is too big, the drivers in it will be dispatched to the first worker we get(if it has enough resources), with or without the Randomization. We should do randomization every time we dispatch a driver, in order to better balance drivers. Author: WangTaoTheTonic Author: WangTao Closes #1106 from WangTaoTheTonic/fixBalanceDrivers and squashes the following commits: d1a928b [WangTaoTheTonic] Minor adjustment b6560cf [WangTaoTheTonic] solve the shuffle problem for HashSet f674e59 [WangTaoTheTonic] add comment and minor fix 2835929 [WangTao] solve the failed test and avoid filtering 2ca3091 [WangTao] fix checkstyle bc91bb1 [WangTao] Avoid shuffle every time we schedule the driver using round robin bbc7087 [WangTaoTheTonic] Optimize the schedule in Master --- .../apache/spark/deploy/master/Master.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index a3909d6ea95c0..2a3bd6ba0b9dc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -487,13 +487,25 @@ private[spark] class Master( if (state != RecoveryState.ALIVE) { return } // First schedule drivers, they take strict precedence over applications - val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers - for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { - for (driver <- List(waitingDrivers: _*)) { // iterate over a copy of waitingDrivers + // Randomization helps balance drivers + val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) + val aliveWorkerNum = shuffledAliveWorkers.size + var curPos = 0 + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers + // We assign workers to each waiting driver in a round-robin fashion. For each driver, we + // start from the last worker that was assigned a driver, and continue onwards until we have + // explored all alive workers. + curPos = (curPos + 1) % aliveWorkerNum + val startPos = curPos + var launched = false + while (curPos != startPos && !launched) { + val worker = shuffledAliveWorkers(curPos) if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver + launched = true } + curPos = (curPos + 1) % aliveWorkerNum } }