From bf40cf89e37aeaf80e37a4b0ae001ba25f819821 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 11 Dec 2014 22:41:36 -0800 Subject: [PATCH 001/227] [SPARK-4713] [SQL] SchemaRDD.unpersist() should not raise exception if it is not persisted Unpersist a uncached RDD, will not raise exception, for example: ``` val data = Array(1, 2, 3, 4, 5) val distData = sc.parallelize(data) distData.unpersist(true) ``` But the `SchemaRDD` will raise exception if the `SchemaRDD` is not cached. Since `SchemaRDD` is the subclasses of the `RDD`, we should follow the same behavior. Author: Cheng Hao Closes #3572 from chenghao-intel/try_uncache and squashes the following commits: 50a7a89 [Cheng Hao] SchemaRDD.unpersist() should not raise exception if it is not persisted --- .../scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- .../org/apache/spark/sql/CachedTableSuite.scala | 14 ++++++++++++++ 2 files changed, 15 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 95d73c1711523..a66af602a1064 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 @@ -501,7 +501,7 @@ class SchemaRDD( } override def unpersist(blocking: Boolean): this.type = { - sqlContext.uncacheQuery(this, blocking) + sqlContext.tryUncacheQuery(this, blocking) this } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 042210176ad7e..cfc037caff2a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -49,6 +49,20 @@ class CachedTableSuite extends QueryTest { uncacheTable("tempTable") } + test("unpersist an uncached table will not raise exception") { + assert(None == lookupCachedData(testData)) + testData.unpersist(true) + assert(None == lookupCachedData(testData)) + testData.unpersist(false) + assert(None == lookupCachedData(testData)) + testData.persist() + assert(None != lookupCachedData(testData)) + testData.unpersist(true) + assert(None == lookupCachedData(testData)) + testData.unpersist(false) + assert(None == lookupCachedData(testData)) + } + test("cache table as select") { sql("CACHE TABLE tempTable AS SELECT key FROM testData") assertCached(sql("SELECT COUNT(*) FROM tempTable")) From a7f07f511c0783db3b2b0a19fc71814cda94aad8 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 11 Dec 2014 22:43:02 -0800 Subject: [PATCH 002/227] [SPARK-4662] [SQL] Whitelist more unittest Whitelist more hive unit test: "create_like_tbl_props" "udf5" "udf_java_method" "decimal_1" "udf_pmod" "udf_to_double" "udf_to_float" "udf7" (this will fail in Hive 0.12) Author: Cheng Hao Closes #3522 from chenghao-intel/unittest and squashes the following commits: f54e4c7 [Cheng Hao] work around to clean up the hive.table.parameters.default in reset 16fee22 [Cheng Hao] Whitelist more unittest --- .../execution/HiveCompatibilitySuite.scala | 11 ++----- .../org/apache/spark/sql/hive/TestHive.scala | 2 ++ ...l_props-0-432a3ade72afd99cfb4b886692c15e55 | 0 ...l_props-1-f8481dcbc8f2731bab8ac3894511ff9f | 28 ++++++++++++++++++ ...bl_props-10-2a2d6f2c92e32285dd4c4dd3d0faa9 | 0 ..._props-11-b4f47dcb46073bda6fb1d9f96e8b36e6 | 0 ..._props-12-184ab0f730b53d1b8b4f4e1feade9824 | 29 +++++++++++++++++++ ...l_props-2-62c728aff7df8cd2bd2c114c9076a1ff | 1 + ...l_props-3-3320f357baaadfe13820349b8d941865 | 0 ...l_props-4-f59c262efb0482b555ae867abef4040f | 28 ++++++++++++++++++ ...l_props-5-11e1ff4a0cf4ee27f5ccb5f267643cfd | 0 ...l_props-6-c49698cf69779ee8a519e2566c6b2acb | 29 +++++++++++++++++++ ...l_props-7-25f0c8b81d949d73737ee3a5398fc9f7 | 1 + ...l_props-8-69b6bc0b259beb299874e7cdfc5edb1b | 0 ...l_props-9-9461431e44ae60a529cc309d8f325dbc | 29 +++++++++++++++++++ ...cimal_1-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + ...cimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 | 0 ...imal_1-10-c20dea9d716bef1bdbdef71323b1cc5b | 1 + ...imal_1-11-f2f975b73220512d4bf2b9bd93354aba | 1 + ...imal_1-12-2c2325880ea79c8e308398d46c8565f8 | 1 + ...imal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 | 1 + ...imal_1-14-e45935cfffb9045394e804d0d1fc52f0 | 1 + ...imal_1-15-31ecaab3afa056fcc656d6e54f845cf4 | 0 ...ecimal_1-2-ee665100ca5de3a006df43e97cfa707 | 0 ...cimal_1-3-80fc87cab17ceffea334afbb230a6653 | 0 ...cimal_1-4-5dd925bba25f735bfd6442a841afe119 | 3 ++ ...ecimal_1-5-bfab296ca5693e647e33899dfeeb256 | 0 ...cimal_1-6-a402201ed5159941384d40e09dc367a5 | 1 + ...ecimal_1-7-2cfd7d00bc37a8e433ad005896173c1 | 1 + ...cimal_1-8-84cd75e494d113a48c4145298177d6d8 | 1 + ...cimal_1-9-e4e90927ac59f5920de3dc61c3288dde | 1 + .../udf5-0-2a18d9570d9b676e240cda76df818c42 | 0 .../udf5-1-f60851dc36f579e83d6848d7d3c589e6 | 0 .../udf5-2-e08fad5ccbf165f44ecabb9356e58b24 | 0 .../udf5-3-d23017942dc49be9f5a61430201371bf | 1 + .../udf5-4-1b35f4ee3febf99804db1f481af80b23 | 0 .../udf5-5-2125da6f09799cf7f10b838fc8f24e71 | 1 + ..._method-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + ..._method-0-991b98a25032b21802bc2a1efde606c7 | 1 + ..._method-1-991b98a25032b21802bc2a1efde606c7 | 1 + ..._method-1-a3b94d9f2c2caf85a588b6686a64630a | 4 +++ ..._method-2-69e6b8725086a8fb8f55721705442112 | 0 ..._method-2-a3b94d9f2c2caf85a588b6686a64630a | 4 +++ ..._method-3-9ae6daaf9783d3d6577231320727582a | 0 ..._method-3-c526dfd4d9eac718ced9afb3cf9a62fd | 1 + ..._method-4-2fc4554258492a1d92c89a8dbad6c1c3 | 1 + ...df_pmod-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + ...df_pmod-0-ed67184beaf84c0542117c26651938e1 | 1 + ...df_pmod-1-90f75e01dcee85253a501d53b8562dae | 1 + ...df_pmod-1-ed67184beaf84c0542117c26651938e1 | 1 + ...f_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 | 1 + ...f_pmod-10-cab94a0c6b36a489aab9f3f305b92772 | 1 + ...f_pmod-11-7ca6baa647215c334419d1bb8a527449 | 1 + ...df_pmod-2-8ac9813b27801704082c6e9ea4cdc312 | 1 + ...df_pmod-2-90f75e01dcee85253a501d53b8562dae | 1 + ...df_pmod-3-26d9546f030281a29a50a3e8e5858234 | 1 + ...df_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f | 1 + ...df_pmod-4-608e04ca8855780fb9e60486759b19b8 | 1 + ...df_pmod-4-7695df16d24a821224676e6bad3d66d1 | 1 + ...df_pmod-5-cf5311d51d44afb8d73f588e27d5e029 | 1 + ...df_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf | 1 + ...udf_pmod-6-3c09a8da2f5645e732c22a45d055125 | 1 + ...df_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5 | 1 + ...df_pmod-7-3665a6414590bb2aff522dfe847dbc0e | 1 + ...df_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 | 1 + ...df_pmod-8-95f85c34076952af0640b596365b27ca | 1 + ...df_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd | 1 + ...df_pmod-9-798ef5064b61d0ae403e3e11c8fd749b | 1 + ...df_pmod-9-e7280393102077442aa1d10eb69a6d57 | 1 + ...f_round-0-10b53ca1f15fd7879365926f86512d15 | 1 + ...f_round-1-2367bcc43510dedc80bdb6707e434da8 | 4 +++ ..._double-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + ..._double-0-cbe030be095a93a9ae810ce7e66bdca7 | 1 + ..._double-1-98d2ce732277c3a7fb4827d8b221a43a | 1 + ..._double-1-9d7f149dc28bd312425392c3f2abea72 | 1 + ..._double-2-69bf8a5a4cb378bbd54c20cb8aa97abe | 1 + ..._double-2-e5df309104b260ff9145229d119a774d | 1 + ..._double-3-ab23099412d24154ff369d8bd6bde89f | 1 + ..._double-3-abd1a8fc84fcd692891c1ac242492e45 | 1 + ..._double-4-293a639a2b61a11da6ca798c04624f68 | 1 + ..._double-4-3651962b1a5fac4f1dc02f0403e68471 | 1 + ..._double-5-39bd92a64ad9d5f57d477bf668e08da5 | 1 + ..._double-5-42d1e80bb3324030c62a23c6d1b786a8 | 1 + ..._double-6-5bac1a7db00d788fd7d82e3a78e60be6 | 1 + ..._double-6-b65b3f3b72ce068c2b954850fe5fc2a6 | 1 + ..._double-7-97080ab9cd416f8acd8803291e9dc9e5 | 1 + ..._double-7-ab78e74674e92847fd44db0d21c2a145 | 1 + ..._double-8-22e97175b71ca7fd8668130f5a757aee | 1 + ..._double-8-df51146f6ef960c77cd1722191e4b982 | 1 + ..._double-9-53d0629f93ae811965bb4658e1aa3cb9 | 1 + ...o_float-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + ...o_float-0-7646eca02448547eedf84a81bf42be89 | 1 + ...o_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd | 1 + ...o_float-1-eac0237ee0294d635c2b538f6e2f0a5c | 1 + ...o_float-2-39a67183b6d2a4da005baed849c5e971 | 1 + ...to_float-2-ef6719eced842e7efe970665b41f8c0 | 1 + ...o_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce | 1 + ...o_float-3-bfb661d2179679c317a7b088837258d3 | 1 + ...o_float-4-477519ea558ef60feb754d442c1b13cc | 1 + ...o_float-4-f3e0ab8ed691a386e6be4ce6993be507 | 1 + ...o_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a | 1 + ...o_float-5-75f364708c01b5e31f988f19e52b2201 | 1 + ...o_float-6-1d9b690354f7b04df660a9e3c448a002 | 1 + ...o_float-6-6bf596b8ac0a57d7df844cca1c94a0c7 | 1 + ...o_float-7-cdfefa5173854b647a76383300f8b9d1 | 1 + ...o_float-7-e4b449ba415538aac9c9ec421d8bcce8 | 1 + ...o_float-8-5b0a785185bcaa98b581c5b3dbb3e12c | 1 + ...o_float-8-c46d1c40e52bef886e56ae1e07892bb7 | 1 + ...o_float-9-c5545924be7d13b1f4a13cb2bd0c17cc | 1 + .../org/apache/spark/sql/hive/Shim12.scala | 1 + 110 files changed, 245 insertions(+), 8 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-0-432a3ade72afd99cfb4b886692c15e55 create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-1-f8481dcbc8f2731bab8ac3894511ff9f create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-10-2a2d6f2c92e32285dd4c4dd3d0faa9 create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-11-b4f47dcb46073bda6fb1d9f96e8b36e6 create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-12-184ab0f730b53d1b8b4f4e1feade9824 create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-2-62c728aff7df8cd2bd2c114c9076a1ff create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-3-3320f357baaadfe13820349b8d941865 create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-4-f59c262efb0482b555ae867abef4040f create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-5-11e1ff4a0cf4ee27f5ccb5f267643cfd create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-6-c49698cf69779ee8a519e2566c6b2acb create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-7-25f0c8b81d949d73737ee3a5398fc9f7 create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-8-69b6bc0b259beb299874e7cdfc5edb1b create mode 100644 sql/hive/src/test/resources/golden/create_like_tbl_props-9-9461431e44ae60a529cc309d8f325dbc create mode 100644 sql/hive/src/test/resources/golden/decimal_1-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b create mode 100644 sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba create mode 100644 sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde create mode 100644 sql/hive/src/test/resources/golden/udf5-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/udf5-1-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 sql/hive/src/test/resources/golden/udf5-2-e08fad5ccbf165f44ecabb9356e58b24 create mode 100644 sql/hive/src/test/resources/golden/udf5-3-d23017942dc49be9f5a61430201371bf create mode 100644 sql/hive/src/test/resources/golden/udf5-4-1b35f4ee3febf99804db1f481af80b23 create mode 100644 sql/hive/src/test/resources/golden/udf5-5-2125da6f09799cf7f10b838fc8f24e71 create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-1-991b98a25032b21802bc2a1efde606c7 create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-2-a3b94d9f2c2caf85a588b6686a64630a create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-3-9ae6daaf9783d3d6577231320727582a create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-4-2fc4554258492a1d92c89a8dbad6c1c3 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-1-ed67184beaf84c0542117c26651938e1 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-10-cab94a0c6b36a489aab9f3f305b92772 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-11-7ca6baa647215c334419d1bb8a527449 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-2-90f75e01dcee85253a501d53b8562dae create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-4-608e04ca8855780fb9e60486759b19b8 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-7-3665a6414590bb2aff522dfe847dbc0e create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-8-95f85c34076952af0640b596365b27ca create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-9-798ef5064b61d0ae403e3e11c8fd749b create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 create mode 100644 sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 create mode 100644 sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-1-9d7f149dc28bd312425392c3f2abea72 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-2-e5df309104b260ff9145229d119a774d create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-3-abd1a8fc84fcd692891c1ac242492e45 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-4-3651962b1a5fac4f1dc02f0403e68471 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-5-39bd92a64ad9d5f57d477bf668e08da5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-6-b65b3f3b72ce068c2b954850fe5fc2a6 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-7-ab78e74674e92847fd44db0d21c2a145 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-8-22e97175b71ca7fd8668130f5a757aee create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-9-53d0629f93ae811965bb4658e1aa3cb9 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-2-ef6719eced842e7efe970665b41f8c0 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-3-bfb661d2179679c317a7b088837258d3 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-4-477519ea558ef60feb754d442c1b13cc create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-6-6bf596b8ac0a57d7df844cca1c94a0c7 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-7-e4b449ba415538aac9c9ec421d8bcce8 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-8-c46d1c40e52bef886e56ae1e07892bb7 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-9-c5545924be7d13b1f4a13cb2bd0c17cc diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 7c0be4872d762..4302da5bfc7fc 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -89,7 +89,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "authorization_5", "keyword_1", "misc_json", - "create_like_tbl_props", "load_overwrite", "alter_table_serde2", "alter_table_not_sorted", @@ -100,8 +99,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "protectmode2", //"describe_table", "describe_comment_nonascii", - "udf5", - "udf_java_method", + "create_merge_compressed", "create_view_partitioned", "database_location", @@ -221,16 +219,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "orc_predicate_pushdown", // Requires precision decimal support: - "decimal_1", - "udf_pmod", "udf_when", "udf_case", - "udf_to_double", - "udf_to_float", // Needs constant object inspectors "udf_round", - "udf7", // Sort with Limit clause causes failure. "ctas", @@ -351,6 +344,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "count", "cp_mj_rc", "create_insert_outputformat", + "create_like_tbl_props", "create_like_view", "create_nested_type", "create_skewed_table1", @@ -785,6 +779,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udaf_covar_samp", "udaf_histogram_numeric", "udf2", + "udf5", "udf6", "udf7", "udf8", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index bb79ad5538046..eedb57de52ba9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -426,6 +426,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // other sql exec here. runSqlHive("RESET") // For some reason, RESET does not reset the following variables... + // https://issues.apache.org/jira/browse/HIVE-9004 + runSqlHive("set hive.table.parameters.default=") runSqlHive("set datanucleus.cache.collections=true") runSqlHive("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-0-432a3ade72afd99cfb4b886692c15e55 b/sql/hive/src/test/resources/golden/create_like_tbl_props-0-432a3ade72afd99cfb4b886692c15e55 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-1-f8481dcbc8f2731bab8ac3894511ff9f b/sql/hive/src/test/resources/golden/create_like_tbl_props-1-f8481dcbc8f2731bab8ac3894511ff9f new file mode 100644 index 0000000000000..7d8744672aa11 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-1-f8481dcbc8f2731bab8ac3894511ff9f @@ -0,0 +1,28 @@ +# col_name data_type comment + +key int +value string + +# Detailed Table Information +Database: default +Owner: hcheng +CreateTime: Fri Nov 28 00:04:15 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/tmp/sparkHiveWarehouse3490012261419180285/test_table +Table Type: MANAGED_TABLE +Table Parameters: + key value + transient_lastDdlTime 1417161855 + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-10-2a2d6f2c92e32285dd4c4dd3d0faa9 b/sql/hive/src/test/resources/golden/create_like_tbl_props-10-2a2d6f2c92e32285dd4c4dd3d0faa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-11-b4f47dcb46073bda6fb1d9f96e8b36e6 b/sql/hive/src/test/resources/golden/create_like_tbl_props-11-b4f47dcb46073bda6fb1d9f96e8b36e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-12-184ab0f730b53d1b8b4f4e1feade9824 b/sql/hive/src/test/resources/golden/create_like_tbl_props-12-184ab0f730b53d1b8b4f4e1feade9824 new file mode 100644 index 0000000000000..b55c2dcfe4934 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-12-184ab0f730b53d1b8b4f4e1feade9824 @@ -0,0 +1,29 @@ +# col_name data_type comment + +key int +value string + +# Detailed Table Information +Database: default +Owner: hcheng +CreateTime: Fri Nov 28 00:04:16 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/tmp/sparkHiveWarehouse3490012261419180285/test_table4 +Table Type: MANAGED_TABLE +Table Parameters: + key value + key1 value1 + transient_lastDdlTime 1417161856 + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-2-62c728aff7df8cd2bd2c114c9076a1ff b/sql/hive/src/test/resources/golden/create_like_tbl_props-2-62c728aff7df8cd2bd2c114c9076a1ff new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-2-62c728aff7df8cd2bd2c114c9076a1ff @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-3-3320f357baaadfe13820349b8d941865 b/sql/hive/src/test/resources/golden/create_like_tbl_props-3-3320f357baaadfe13820349b8d941865 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-4-f59c262efb0482b555ae867abef4040f b/sql/hive/src/test/resources/golden/create_like_tbl_props-4-f59c262efb0482b555ae867abef4040f new file mode 100644 index 0000000000000..8c94e5fb15561 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-4-f59c262efb0482b555ae867abef4040f @@ -0,0 +1,28 @@ +# col_name data_type comment + +key int +value string + +# Detailed Table Information +Database: default +Owner: hcheng +CreateTime: Fri Nov 28 00:04:15 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/tmp/sparkHiveWarehouse3490012261419180285/test_table1 +Table Type: MANAGED_TABLE +Table Parameters: + key1 value1 + transient_lastDdlTime 1417161855 + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-5-11e1ff4a0cf4ee27f5ccb5f267643cfd b/sql/hive/src/test/resources/golden/create_like_tbl_props-5-11e1ff4a0cf4ee27f5ccb5f267643cfd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-6-c49698cf69779ee8a519e2566c6b2acb b/sql/hive/src/test/resources/golden/create_like_tbl_props-6-c49698cf69779ee8a519e2566c6b2acb new file mode 100644 index 0000000000000..ddec982d168c0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-6-c49698cf69779ee8a519e2566c6b2acb @@ -0,0 +1,29 @@ +# col_name data_type comment + +key int +value string + +# Detailed Table Information +Database: default +Owner: hcheng +CreateTime: Fri Nov 28 00:04:16 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/tmp/sparkHiveWarehouse3490012261419180285/test_table2 +Table Type: MANAGED_TABLE +Table Parameters: + key1 value1 + key2 value2 + transient_lastDdlTime 1417161856 + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-7-25f0c8b81d949d73737ee3a5398fc9f7 b/sql/hive/src/test/resources/golden/create_like_tbl_props-7-25f0c8b81d949d73737ee3a5398fc9f7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-7-25f0c8b81d949d73737ee3a5398fc9f7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-8-69b6bc0b259beb299874e7cdfc5edb1b b/sql/hive/src/test/resources/golden/create_like_tbl_props-8-69b6bc0b259beb299874e7cdfc5edb1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_tbl_props-9-9461431e44ae60a529cc309d8f325dbc b/sql/hive/src/test/resources/golden/create_like_tbl_props-9-9461431e44ae60a529cc309d8f325dbc new file mode 100644 index 0000000000000..547d4fbdf34d3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_tbl_props-9-9461431e44ae60a529cc309d8f325dbc @@ -0,0 +1,29 @@ +# col_name data_type comment + +key int +value string + +# Detailed Table Information +Database: default +Owner: hcheng +CreateTime: Fri Nov 28 00:04:16 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/tmp/sparkHiveWarehouse3490012261419180285/test_table3 +Table Type: MANAGED_TABLE +Table Parameters: + key1 value1 + key2 value3 + transient_lastDdlTime 1417161856 + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/decimal_1-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/decimal_1-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 b/sql/hive/src/test/resources/golden/decimal_1-1-6742a91ba2b9fa9c906d30d4d0ad0972 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b b/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-10-c20dea9d716bef1bdbdef71323b1cc5b @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba b/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-11-f2f975b73220512d4bf2b9bd93354aba @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 b/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-12-2c2325880ea79c8e308398d46c8565f8 @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 b/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-13-c4c33bdb9f3c6cad77552f0f353092d3 @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 b/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 new file mode 100644 index 0000000000000..c4a17c1b14c88 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-14-e45935cfffb9045394e804d0d1fc52f0 @@ -0,0 +1 @@ +1969-12-31 16:00:17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/decimal_1-15-31ecaab3afa056fcc656d6e54f845cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 b/sql/hive/src/test/resources/golden/decimal_1-2-ee665100ca5de3a006df43e97cfa707 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/decimal_1-3-80fc87cab17ceffea334afbb230a6653 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 b/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 new file mode 100644 index 0000000000000..711809abcc925 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-4-5dd925bba25f735bfd6442a841afe119 @@ -0,0 +1,3 @@ +t decimal(4,2) +u decimal(5,0) +v decimal(10,0) diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 b/sql/hive/src/test/resources/golden/decimal_1-5-bfab296ca5693e647e33899dfeeb256 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 b/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-6-a402201ed5159941384d40e09dc367a5 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 b/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-7-2cfd7d00bc37a8e433ad005896173c1 @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 b/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-8-84cd75e494d113a48c4145298177d6d8 @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde b/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-9-e4e90927ac59f5920de3dc61c3288dde @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/udf5-0-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf5-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf5-1-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf5-1-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf5-2-e08fad5ccbf165f44ecabb9356e58b24 b/sql/hive/src/test/resources/golden/udf5-2-e08fad5ccbf165f44ecabb9356e58b24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf5-3-d23017942dc49be9f5a61430201371bf b/sql/hive/src/test/resources/golden/udf5-3-d23017942dc49be9f5a61430201371bf new file mode 100644 index 0000000000000..4cdf7737e251f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf5-3-d23017942dc49be9f5a61430201371bf @@ -0,0 +1 @@ +2008-11-11 15:32:20 2008-11-11 1 11 2008 1 11 2008 diff --git a/sql/hive/src/test/resources/golden/udf5-4-1b35f4ee3febf99804db1f481af80b23 b/sql/hive/src/test/resources/golden/udf5-4-1b35f4ee3febf99804db1f481af80b23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf5-5-2125da6f09799cf7f10b838fc8f24e71 b/sql/hive/src/test/resources/golden/udf5-5-2125da6f09799cf7f10b838fc8f24e71 new file mode 100644 index 0000000000000..bfd616764b235 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf5-5-2125da6f09799cf7f10b838fc8f24e71 @@ -0,0 +1 @@ +01/13/10 11:57:40 2010-01-13 11:57:40 diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_java_method-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_java_method-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 new file mode 100644 index 0000000000000..91e538becfc96 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 @@ -0,0 +1 @@ +java_method(class,method[,arg1[,arg2..]]) calls method with reflection diff --git a/sql/hive/src/test/resources/golden/udf_java_method-1-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-1-991b98a25032b21802bc2a1efde606c7 new file mode 100644 index 0000000000000..91e538becfc96 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_java_method-1-991b98a25032b21802bc2a1efde606c7 @@ -0,0 +1 @@ +java_method(class,method[,arg1[,arg2..]]) calls method with reflection diff --git a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a new file mode 100644 index 0000000000000..6315f678b46f8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a @@ -0,0 +1,4 @@ +java_method(class,method[,arg1[,arg2..]]) calls method with reflection +Synonyms: reflect +Use this UDF to call Java methods by matching the argument signature + diff --git a/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_java_method-2-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-2-a3b94d9f2c2caf85a588b6686a64630a new file mode 100644 index 0000000000000..6315f678b46f8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_java_method-2-a3b94d9f2c2caf85a588b6686a64630a @@ -0,0 +1,4 @@ +java_method(class,method[,arg1[,arg2..]]) calls method with reflection +Synonyms: reflect +Use this UDF to call Java methods by matching the argument signature + diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-9ae6daaf9783d3d6577231320727582a b/sql/hive/src/test/resources/golden/udf_java_method-3-9ae6daaf9783d3d6577231320727582a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd new file mode 100644 index 0000000000000..51ff65ea1870f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd @@ -0,0 +1 @@ +1 true 3 2 3 2.718281828459045 1.0 diff --git a/sql/hive/src/test/resources/golden/udf_java_method-4-2fc4554258492a1d92c89a8dbad6c1c3 b/sql/hive/src/test/resources/golden/udf_java_method-4-2fc4554258492a1d92c89a8dbad6c1c3 new file mode 100644 index 0000000000000..51ff65ea1870f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_java_method-4-2fc4554258492a1d92c89a8dbad6c1c3 @@ -0,0 +1 @@ +1 true 3 2 3 2.718281828459045 1.0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_pmod-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-1-ed67184beaf84c0542117c26651938e1 new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-1-ed67184beaf84c0542117c26651938e1 @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 new file mode 100644 index 0000000000000..0b46af11c4516 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 @@ -0,0 +1 @@ +6.89 51.7 18.09 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-cab94a0c6b36a489aab9f3f305b92772 b/sql/hive/src/test/resources/golden/udf_pmod-10-cab94a0c6b36a489aab9f3f305b92772 new file mode 100644 index 0000000000000..ab842acd48b3c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-10-cab94a0c6b36a489aab9f3f305b92772 @@ -0,0 +1 @@ +6.890000000000011 51.699999999999996 18.090000000000003 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-11-7ca6baa647215c334419d1bb8a527449 b/sql/hive/src/test/resources/golden/udf_pmod-11-7ca6baa647215c334419d1bb8a527449 new file mode 100644 index 0000000000000..0b46af11c4516 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-11-7ca6baa647215c334419d1bb8a527449 @@ -0,0 +1 @@ +6.89 51.7 18.09 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-2-90f75e01dcee85253a501d53b8562dae new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-2-90f75e01dcee85253a501d53b8562dae @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 new file mode 100644 index 0000000000000..5eb0813b60eb6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 @@ -0,0 +1 @@ +8 51 15 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f b/sql/hive/src/test/resources/golden/udf_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-3-8d6dd8a5e7a519fdc5261e4193e3464f @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-608e04ca8855780fb9e60486759b19b8 b/sql/hive/src/test/resources/golden/udf_pmod-4-608e04ca8855780fb9e60486759b19b8 new file mode 100644 index 0000000000000..5eb0813b60eb6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-4-608e04ca8855780fb9e60486759b19b8 @@ -0,0 +1 @@ +8 51 15 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 new file mode 100644 index 0000000000000..e21e4b08e7a62 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 @@ -0,0 +1 @@ +5 50 0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf b/sql/hive/src/test/resources/golden/udf_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf new file mode 100644 index 0000000000000..e21e4b08e7a62 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-5-e3ca9fe032dd6f71e33ddf367ef5e2cf @@ -0,0 +1 @@ +5 50 0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5 b/sql/hive/src/test/resources/golden/udf_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-6-c5c810e71bed8e56c1bac59b7d9c16c5 @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-3665a6414590bb2aff522dfe847dbc0e b/sql/hive/src/test/resources/golden/udf_pmod-7-3665a6414590bb2aff522dfe847dbc0e new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-7-3665a6414590bb2aff522dfe847dbc0e @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-95f85c34076952af0640b596365b27ca b/sql/hive/src/test/resources/golden/udf_pmod-8-95f85c34076952af0640b596365b27ca new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-8-95f85c34076952af0640b596365b27ca @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd new file mode 100644 index 0000000000000..48371142e9b5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd @@ -0,0 +1 @@ +6.8899984 51.700005 18.089996 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-798ef5064b61d0ae403e3e11c8fd749b b/sql/hive/src/test/resources/golden/udf_pmod-9-798ef5064b61d0ae403e3e11c8fd749b new file mode 100644 index 0000000000000..48371142e9b5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-9-798ef5064b61d0ae403e3e11c8fd749b @@ -0,0 +1 @@ +6.8899984 51.700005 18.089996 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 new file mode 100644 index 0000000000000..ab842acd48b3c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 @@ -0,0 +1 @@ +6.890000000000011 51.699999999999996 18.090000000000003 diff --git a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 new file mode 100644 index 0000000000000..49fdc0a774e70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 @@ -0,0 +1 @@ +round(x[, d]) - round x to d decimal places diff --git a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 new file mode 100644 index 0000000000000..862adeae821ff --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 @@ -0,0 +1,4 @@ +round(x[, d]) - round x to d decimal places +Example: + > SELECT round(12.3456, 1) FROM src LIMIT 1; + 12.3' diff --git a/sql/hive/src/test/resources/golden/udf_to_double-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_to_double-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-9d7f149dc28bd312425392c3f2abea72 b/sql/hive/src/test/resources/golden/udf_to_double-1-9d7f149dc28bd312425392c3f2abea72 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-1-9d7f149dc28bd312425392c3f2abea72 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe @@ -0,0 +1 @@ +-7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-e5df309104b260ff9145229d119a774d b/sql/hive/src/test/resources/golden/udf_to_double-2-e5df309104b260ff9145229d119a774d new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-2-e5df309104b260ff9145229d119a774d @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f @@ -0,0 +1 @@ +-18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-abd1a8fc84fcd692891c1ac242492e45 b/sql/hive/src/test/resources/golden/udf_to_double-3-abd1a8fc84fcd692891c1ac242492e45 new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-3-abd1a8fc84fcd692891c1ac242492e45 @@ -0,0 +1 @@ +-7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 @@ -0,0 +1 @@ +-129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-3651962b1a5fac4f1dc02f0403e68471 b/sql/hive/src/test/resources/golden/udf_to_double-4-3651962b1a5fac4f1dc02f0403e68471 new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-4-3651962b1a5fac4f1dc02f0403e68471 @@ -0,0 +1 @@ +-18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-39bd92a64ad9d5f57d477bf668e08da5 b/sql/hive/src/test/resources/golden/udf_to_double-5-39bd92a64ad9d5f57d477bf668e08da5 new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-5-39bd92a64ad9d5f57d477bf668e08da5 @@ -0,0 +1 @@ +-129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 @@ -0,0 +1 @@ +-1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 new file mode 100644 index 0000000000000..38f7ad5afa0ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 @@ -0,0 +1 @@ +-3.140000104904175 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-b65b3f3b72ce068c2b954850fe5fc2a6 b/sql/hive/src/test/resources/golden/udf_to_double-6-b65b3f3b72ce068c2b954850fe5fc2a6 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-6-b65b3f3b72ce068c2b954850fe5fc2a6 @@ -0,0 +1 @@ +-1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-ab78e74674e92847fd44db0d21c2a145 b/sql/hive/src/test/resources/golden/udf_to_double-7-ab78e74674e92847fd44db0d21c2a145 new file mode 100644 index 0000000000000..38f7ad5afa0ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-7-ab78e74674e92847fd44db0d21c2a145 @@ -0,0 +1 @@ +-3.140000104904175 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-22e97175b71ca7fd8668130f5a757aee b/sql/hive/src/test/resources/golden/udf_to_double-8-22e97175b71ca7fd8668130f5a757aee new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-8-22e97175b71ca7fd8668130f5a757aee @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 @@ -0,0 +1 @@ +-38.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-9-53d0629f93ae811965bb4658e1aa3cb9 b/sql/hive/src/test/resources/golden/udf_to_double-9-53d0629f93ae811965bb4658e1aa3cb9 new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-9-53d0629f93ae811965bb4658e1aa3cb9 @@ -0,0 +1 @@ +-38.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_to_float-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd b/sql/hive/src/test/resources/golden/udf_to_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-1-6f0f1660c78aa1318ae1da4a2afdd9dd @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 @@ -0,0 +1 @@ +-7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-ef6719eced842e7efe970665b41f8c0 b/sql/hive/src/test/resources/golden/udf_to_float-2-ef6719eced842e7efe970665b41f8c0 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-2-ef6719eced842e7efe970665b41f8c0 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce @@ -0,0 +1 @@ +-18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-bfb661d2179679c317a7b088837258d3 b/sql/hive/src/test/resources/golden/udf_to_float-3-bfb661d2179679c317a7b088837258d3 new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-3-bfb661d2179679c317a7b088837258d3 @@ -0,0 +1 @@ +-7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-477519ea558ef60feb754d442c1b13cc b/sql/hive/src/test/resources/golden/udf_to_float-4-477519ea558ef60feb754d442c1b13cc new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-4-477519ea558ef60feb754d442c1b13cc @@ -0,0 +1 @@ +-18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 @@ -0,0 +1 @@ +-129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a b/sql/hive/src/test/resources/golden/udf_to_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-5-3ea0b65c600c2a6c0a2f20b36bc02c0a @@ -0,0 +1 @@ +-129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 @@ -0,0 +1 @@ +-1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-6bf596b8ac0a57d7df844cca1c94a0c7 b/sql/hive/src/test/resources/golden/udf_to_float-6-6bf596b8ac0a57d7df844cca1c94a0c7 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-6-6bf596b8ac0a57d7df844cca1c94a0c7 @@ -0,0 +1 @@ +-1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-e4b449ba415538aac9c9ec421d8bcce8 b/sql/hive/src/test/resources/golden/udf_to_float-7-e4b449ba415538aac9c9ec421d8bcce8 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-7-e4b449ba415538aac9c9ec421d8bcce8 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c @@ -0,0 +1 @@ +-38.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-c46d1c40e52bef886e56ae1e07892bb7 b/sql/hive/src/test/resources/golden/udf_to_float-8-c46d1c40e52bef886e56ae1e07892bb7 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-8-c46d1c40e52bef886e56ae1e07892bb7 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-9-c5545924be7d13b1f4a13cb2bd0c17cc b/sql/hive/src/test/resources/golden/udf_to_float-9-c5545924be7d13b1f4a13cb2bd0c17cc new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-9-c5545924be7d13b1f4a13cb2bd0c17cc @@ -0,0 +1 @@ +-38.14 diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 754ffc422072d..67cc8865755c5 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -174,6 +174,7 @@ private[hive] object HiveShim { def compatibilityBlackList = Seq( "decimal_.*", + "udf7", "drop_partitions_filter2", "show_.*", "serde_regex", From c152dde78f73d5ce3a483fd60a47e7de1f1916da Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Thu, 11 Dec 2014 22:44:27 -0800 Subject: [PATCH 003/227] [SPARK-4639] [SQL] Pass maxIterations in as a parameter in Analyzer fix a TODO in Analyzer: // TODO: pass this in as a parameter val fixedPoint = FixedPoint(100) Author: Jacky Li Closes #3499 from jackylk/config and squashes the following commits: 4c1252c [Jacky Li] fix scalastyle 820f460 [Jacky Li] pass maxIterations in as a parameter --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index facbd8b975f10..04639219a3650 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -34,13 +34,15 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true * [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and * a [[FunctionRegistry]]. */ -class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) +class Analyzer(catalog: Catalog, + registry: FunctionRegistry, + caseSensitive: Boolean, + maxIterations: Int = 100) extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { val resolver = if (caseSensitive) caseSensitiveResolution else caseInsensitiveResolution - // TODO: pass this in as a parameter. - val fixedPoint = FixedPoint(100) + val fixedPoint = FixedPoint(maxIterations) /** * Override to provide additional rules for the "Resolution" batch. From 334480362b3a133c2fb1e9af898930fe76d7a163 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 11 Dec 2014 22:45:25 -0800 Subject: [PATCH 004/227] [SPARK-4293][SQL] Make Cast be able to handle complex types. Inserting data of type including `ArrayType.containsNull == false` or `MapType.valueContainsNull == false` or `StructType.fields.exists(_.nullable == false)` into Hive table will fail because `Cast` inserted by `HiveMetastoreCatalog.PreInsertionCasts` rule of `Analyzer` can't handle these types correctly. Complex type cast rule proposal: - Cast for non-complex types should be able to cast the same as before. - Cast for `ArrayType` can evaluate if - Element type can cast - Nullability rule doesn't break - Cast for `MapType` can evaluate if - Key type can cast - Nullability for casted key type is `false` - Value type can cast - Nullability rule for value type doesn't break - Cast for `StructType` can evaluate if - The field size is the same - Each field can cast - Nullability rule for each field doesn't break - The nested structure should be the same. Nullability rule: - If the casted type is `nullable == true`, the target nullability should be `true` Author: Takuya UESHIN Closes #3150 from ueshin/issues/SPARK-4293 and squashes the following commits: e935939 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4293 ba14003 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4293 8999868 [Takuya UESHIN] Fix a test title. f677c30 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4293 287f410 [Takuya UESHIN] Add tests to insert data of types ArrayType / MapType / StructType with nullability is false into Hive table. 4f71bb8 [Takuya UESHIN] Make Cast be able to handle complex types. --- .../spark/sql/catalyst/expressions/Cast.scala | 161 ++++++++---- .../ExpressionEvaluationSuite.scala | 236 ++++++++++++++++++ 2 files changed, 353 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index b47865f87a3aa..4ede0b4821fe3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -27,9 +27,14 @@ import org.apache.spark.sql.catalyst.types.decimal.Decimal /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with Logging { + + override lazy val resolved = childrenResolved && resolve(child.dataType, dataType) + override def foldable = child.foldable - override def nullable = (child.dataType, dataType) match { + override def nullable = forceNullable(child.dataType, dataType) || child.nullable + + private[this] def forceNullable(from: DataType, to: DataType) = (from, to) match { case (StringType, _: NumericType) => true case (StringType, TimestampType) => true case (DoubleType, TimestampType) => true @@ -41,8 +46,62 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case (DateType, BooleanType) => true case (DoubleType, _: DecimalType) => true case (FloatType, _: DecimalType) => true - case (_, DecimalType.Fixed(_, _)) => true // TODO: not all upcasts here can really give null - case _ => child.nullable + case (_, DecimalType.Fixed(_, _)) => true // TODO: not all upcasts here can really give null + case _ => false + } + + private[this] def resolvableNullability(from: Boolean, to: Boolean) = !from || to + + private[this] def resolve(from: DataType, to: DataType): Boolean = { + (from, to) match { + case (from, to) if from == to => true + + case (NullType, _) => true + + case (_, StringType) => true + + case (StringType, BinaryType) => true + + case (StringType, BooleanType) => true + case (DateType, BooleanType) => true + case (TimestampType, BooleanType) => true + case (_: NumericType, BooleanType) => true + + case (StringType, TimestampType) => true + case (BooleanType, TimestampType) => true + case (DateType, TimestampType) => true + case (_: NumericType, TimestampType) => true + + case (_, DateType) => true + + case (StringType, _: NumericType) => true + case (BooleanType, _: NumericType) => true + case (DateType, _: NumericType) => true + case (TimestampType, _: NumericType) => true + case (_: NumericType, _: NumericType) => true + + case (ArrayType(from, fn), ArrayType(to, tn)) => + resolve(from, to) && + resolvableNullability(fn || forceNullable(from, to), tn) + + case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => + resolve(fromKey, toKey) && + (!forceNullable(fromKey, toKey)) && + resolve(fromValue, toValue) && + resolvableNullability(fn || forceNullable(fromValue, toValue), tn) + + case (StructType(fromFields), StructType(toFields)) => + fromFields.size == toFields.size && + fromFields.zip(toFields).forall { + case (fromField, toField) => + resolve(fromField.dataType, toField.dataType) && + resolvableNullability( + fromField.nullable || forceNullable(fromField.dataType, toField.dataType), + toField.nullable) + } + + case _ => false + } } override def toString = s"CAST($child, $dataType)" @@ -53,7 +112,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) // UDFToString - private[this] def castToString: Any => Any = child.dataType match { + private[this] def castToString(from: DataType): Any => Any = from match { case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) case DateType => buildCast[Date](_, dateToString) case TimestampType => buildCast[Timestamp](_, timestampToString) @@ -61,12 +120,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } // BinaryConverter - private[this] def castToBinary: Any => Any = child.dataType match { + private[this] def castToBinary(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, _.getBytes("UTF-8")) } // UDFToBoolean - private[this] def castToBoolean: Any => Any = child.dataType match { + private[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, _.length() != 0) case TimestampType => @@ -91,7 +150,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } // TimestampConverter - private[this] def castToTimestamp: Any => Any = child.dataType match { + private[this] def castToTimestamp(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => { // Throw away extra if more than 9 decimal places @@ -133,7 +192,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w }) } - private[this] def decimalToTimestamp(d: Decimal) = { + private[this] def decimalToTimestamp(d: Decimal) = { val seconds = Math.floor(d.toDouble).toLong val bd = (d.toBigDecimal - seconds) * 1000000000 val nanos = bd.intValue() @@ -172,11 +231,10 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } // DateConverter - private[this] def castToDate: Any => Any = child.dataType match { + private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => - try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null } - ) + try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null }) case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. @@ -199,7 +257,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } // LongConverter - private[this] def castToLong: Any => Any = child.dataType match { + private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => try s.toLong catch { case _: NumberFormatException => null @@ -210,14 +268,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t)) - case DecimalType() => - buildCast[Decimal](_, _.toLong) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) } // IntConverter - private[this] def castToInt: Any => Any = child.dataType match { + private[this] def castToInt(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => try s.toInt catch { case _: NumberFormatException => null @@ -228,14 +284,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toInt) - case DecimalType() => - buildCast[Decimal](_, _.toInt) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) } // ShortConverter - private[this] def castToShort: Any => Any = child.dataType match { + private[this] def castToShort(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => try s.toShort catch { case _: NumberFormatException => null @@ -246,14 +300,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toShort) - case DecimalType() => - buildCast[Decimal](_, _.toShort) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort } // ByteConverter - private[this] def castToByte: Any => Any = child.dataType match { + private[this] def castToByte(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => try s.toByte catch { case _: NumberFormatException => null @@ -264,8 +316,6 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toByte) - case DecimalType() => - buildCast[Decimal](_, _.toByte) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } @@ -285,7 +335,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } } - private[this] def castToDecimal(target: DecimalType): Any => Any = child.dataType match { + private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { case StringType => buildCast[String](_, s => try changePrecision(Decimal(s.toDouble), target) catch { case _: NumberFormatException => null @@ -301,7 +351,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w b => changePrecision(b.asInstanceOf[Decimal].clone(), target) case LongType => b => changePrecision(Decimal(b.asInstanceOf[Long]), target) - case x: NumericType => // All other numeric types can be represented precisely as Doubles + case x: NumericType => // All other numeric types can be represented precisely as Doubles b => try { changePrecision(Decimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)), target) } catch { @@ -310,7 +360,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } // DoubleConverter - private[this] def castToDouble: Any => Any = child.dataType match { + private[this] def castToDouble(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => try s.toDouble catch { case _: NumberFormatException => null @@ -321,14 +371,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t)) - case DecimalType() => - buildCast[Decimal](_, _.toDouble) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) } // FloatConverter - private[this] def castToFloat: Any => Any = child.dataType match { + private[this] def castToFloat(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => try s.toFloat catch { case _: NumberFormatException => null @@ -339,28 +387,53 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t).toFloat) - case DecimalType() => - buildCast[Decimal](_, _.toFloat) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) } - private[this] lazy val cast: Any => Any = dataType match { + private[this] def castArray(from: ArrayType, to: ArrayType): Any => Any = { + val elementCast = cast(from.elementType, to.elementType) + buildCast[Seq[Any]](_, _.map(v => if (v == null) null else elementCast(v))) + } + + private[this] def castMap(from: MapType, to: MapType): Any => Any = { + val keyCast = cast(from.keyType, to.keyType) + val valueCast = cast(from.valueType, to.valueType) + buildCast[Map[Any, Any]](_, _.map { + case (key, value) => (keyCast(key), if (value == null) null else valueCast(value)) + }) + } + + private[this] def castStruct(from: StructType, to: StructType): Any => Any = { + val casts = from.fields.zip(to.fields).map { + case (fromField, toField) => cast(fromField.dataType, toField.dataType) + } + buildCast[Row](_, row => Row(row.zip(casts).map { + case (v, cast) => if (v == null) null else cast(v) + }: _*)) + } + + private[this] def cast(from: DataType, to: DataType): Any => Any = to match { case dt if dt == child.dataType => identity[Any] - case StringType => castToString - case BinaryType => castToBinary - case DateType => castToDate - case decimal: DecimalType => castToDecimal(decimal) - case TimestampType => castToTimestamp - case BooleanType => castToBoolean - case ByteType => castToByte - case ShortType => castToShort - case IntegerType => castToInt - case FloatType => castToFloat - case LongType => castToLong - case DoubleType => castToDouble + case StringType => castToString(from) + case BinaryType => castToBinary(from) + case DateType => castToDate(from) + case decimal: DecimalType => castToDecimal(from, decimal) + case TimestampType => castToTimestamp(from) + case BooleanType => castToBoolean(from) + case ByteType => castToByte(from) + case ShortType => castToShort(from) + case IntegerType => castToInt(from) + case FloatType => castToFloat(from) + case LongType => castToLong(from) + case DoubleType => castToDouble(from) + case array: ArrayType => castArray(from.asInstanceOf[ArrayType], array) + case map: MapType => castMap(from.asInstanceOf[MapType], map) + case struct: StructType => castStruct(from.asInstanceOf[StructType], struct) } + private[this] lazy val cast: Any => Any = cast(child.dataType, dataType) + override def eval(input: Row): Any = { val evaluated = child.eval(input) if (evaluated == null) null else cast(evaluated) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index cd2f67f448b0b..b030483223d9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -487,6 +487,242 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(1.0f / 0.0f), TimestampType), null) } + test("array casting") { + val array = Literal(Seq("123", "abc", "", null), ArrayType(StringType, containsNull = true)) + val array_notNull = Literal(Seq("123", "abc", ""), ArrayType(StringType, containsNull = false)) + + { + val cast = Cast(array, ArrayType(IntegerType, containsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Seq(123, null, null, null)) + } + { + val cast = Cast(array, ArrayType(IntegerType, containsNull = false)) + assert(cast.resolved === false) + } + { + val cast = Cast(array, ArrayType(BooleanType, containsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Seq(true, true, false, null)) + } + { + val cast = Cast(array, ArrayType(BooleanType, containsNull = false)) + assert(cast.resolved === false) + } + + { + val cast = Cast(array_notNull, ArrayType(IntegerType, containsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Seq(123, null, null)) + } + { + val cast = Cast(array_notNull, ArrayType(IntegerType, containsNull = false)) + assert(cast.resolved === false) + } + { + val cast = Cast(array_notNull, ArrayType(BooleanType, containsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Seq(true, true, false)) + } + { + val cast = Cast(array_notNull, ArrayType(BooleanType, containsNull = false)) + assert(cast.resolved === true) + checkEvaluation(cast, Seq(true, true, false)) + } + + { + val cast = Cast(array, IntegerType) + assert(cast.resolved === false) + } + } + + test("map casting") { + val map = Literal( + Map("a" -> "123", "b" -> "abc", "c" -> "", "d" -> null), + MapType(StringType, StringType, valueContainsNull = true)) + val map_notNull = Literal( + Map("a" -> "123", "b" -> "abc", "c" -> ""), + MapType(StringType, StringType, valueContainsNull = false)) + + { + val cast = Cast(map, MapType(StringType, IntegerType, valueContainsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null)) + } + { + val cast = Cast(map, MapType(StringType, IntegerType, valueContainsNull = false)) + assert(cast.resolved === false) + } + { + val cast = Cast(map, MapType(StringType, BooleanType, valueContainsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false, "d" -> null)) + } + { + val cast = Cast(map, MapType(StringType, BooleanType, valueContainsNull = false)) + assert(cast.resolved === false) + } + { + val cast = Cast(map, MapType(IntegerType, StringType, valueContainsNull = true)) + assert(cast.resolved === false) + } + + { + val cast = Cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Map("a" -> 123, "b" -> null, "c" -> null)) + } + { + val cast = Cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = false)) + assert(cast.resolved === false) + } + { + val cast = Cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = true)) + assert(cast.resolved === true) + checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false)) + } + { + val cast = Cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = false)) + assert(cast.resolved === true) + checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false)) + } + { + val cast = Cast(map_notNull, MapType(IntegerType, StringType, valueContainsNull = true)) + assert(cast.resolved === false) + } + + { + val cast = Cast(map, IntegerType) + assert(cast.resolved === false) + } + } + + test("struct casting") { + val struct = Literal( + Row("123", "abc", "", null), + StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true), + StructField("c", StringType, nullable = true), + StructField("d", StringType, nullable = true)))) + val struct_notNull = Literal( + Row("123", "abc", ""), + StructType(Seq( + StructField("a", StringType, nullable = false), + StructField("b", StringType, nullable = false), + StructField("c", StringType, nullable = false)))) + + { + val cast = Cast(struct, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = true), + StructField("d", IntegerType, nullable = true)))) + assert(cast.resolved === true) + checkEvaluation(cast, Row(123, null, null, null)) + } + { + val cast = Cast(struct, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = true)))) + assert(cast.resolved === false) + } + { + val cast = Cast(struct, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = true), + StructField("d", BooleanType, nullable = true)))) + assert(cast.resolved === true) + checkEvaluation(cast, Row(true, true, false, null)) + } + { + val cast = Cast(struct, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = false), + StructField("d", BooleanType, nullable = true)))) + assert(cast.resolved === false) + } + + { + val cast = Cast(struct_notNull, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = true)))) + assert(cast.resolved === true) + checkEvaluation(cast, Row(123, null, null)) + } + { + val cast = Cast(struct_notNull, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false)))) + assert(cast.resolved === false) + } + { + val cast = Cast(struct_notNull, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = true)))) + assert(cast.resolved === true) + checkEvaluation(cast, Row(true, true, false)) + } + { + val cast = Cast(struct_notNull, StructType(Seq( + StructField("a", BooleanType, nullable = true), + StructField("b", BooleanType, nullable = true), + StructField("c", BooleanType, nullable = false)))) + assert(cast.resolved === true) + checkEvaluation(cast, Row(true, true, false)) + } + + { + val cast = Cast(struct, StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true), + StructField("c", StringType, nullable = true)))) + assert(cast.resolved === false) + } + { + val cast = Cast(struct, IntegerType) + assert(cast.resolved === false) + } + } + + test("complex casting") { + val complex = Literal( + Row( + Seq("123", "abc", ""), + Map("a" -> "123", "b" -> "abc", "c" -> ""), + Row(0)), + StructType(Seq( + StructField("a", + ArrayType(StringType, containsNull = false), nullable = true), + StructField("m", + MapType(StringType, StringType, valueContainsNull = false), nullable = true), + StructField("s", + StructType(Seq( + StructField("i", IntegerType, nullable = true))))))) + + val cast = Cast(complex, StructType(Seq( + StructField("a", + ArrayType(IntegerType, containsNull = true), nullable = true), + StructField("m", + MapType(StringType, BooleanType, valueContainsNull = false), nullable = true), + StructField("s", + StructType(Seq( + StructField("l", LongType, nullable = true))))))) + + assert(cast.resolved === true) + checkEvaluation(cast, Row( + Seq(123, null, null), + Map("a" -> true, "b" -> true, "c" -> false), + Row(0L))) + } + test("null checking") { val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) val c1 = 'a.string.at(0) From d8cf67858988bed287c0125095729f7f808aabad Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 11 Dec 2014 22:48:03 -0800 Subject: [PATCH 005/227] [SQL] Remove unnecessary case in HiveContext.toHiveString a follow up of #3547 /cc marmbrus Author: scwf Closes #3563 from scwf/rnc and squashes the following commits: 9395661 [scwf] remove unnecessary condition --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 34fc21e61f60f..6008e46fefa86 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -414,9 +414,7 @@ object HiveContext { case (d: Date, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") - case (decimal: Decimal, DecimalType()) => // Hive strips trailing zeros so use its toString - HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString - case (decimal: BigDecimal, DecimalType()) => + case (decimal: BigDecimal, DecimalType()) => // Hive strips trailing zeros so use its toString HiveShim.createDecimal(decimal.underlying()).toString case (other, tpe) if primitiveTypes contains tpe => other.toString } From acb3be6bc5e0e793f769ec371dffeb474dc0cf21 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Thu, 11 Dec 2014 22:49:27 -0800 Subject: [PATCH 006/227] [SPARK-4828] [SQL] sum and avg on empty table should always return null So the optimizations are not valid. Also I think the optimization here is rarely encounter, so removing them will not have influence on performance. Can we merge #3445 before I add a comparison test case from this? Author: Daoyuan Wang Closes #3675 from adrian-wang/sumempty and squashes the following commits: 42df763 [Daoyuan Wang] sum and avg on empty table should always return null --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index f164a6c68a0de..c4923489595b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -203,8 +203,6 @@ object NullPropagation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case e @ Count(Literal(null, _)) => Cast(Literal(0L), e.dataType) - case e @ Sum(Literal(c, _)) if c == 0 => Cast(Literal(0L), e.dataType) - case e @ Average(Literal(c, _)) if c == 0 => Literal(0.0, e.dataType) case e @ IsNull(c) if !c.nullable => Literal(false, BooleanType) case e @ IsNotNull(c) if !c.nullable => Literal(true, BooleanType) case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) From cbb634ae69163ca7a8b5cb94c5fb17cb5c910cb1 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Thu, 11 Dec 2014 22:50:18 -0800 Subject: [PATCH 007/227] [SQL] enable empty aggr test case This is fixed by SPARK-4318 #3184 Author: Daoyuan Wang Closes #3445 from adrian-wang/emptyaggr and squashes the following commits: 982575e [Daoyuan Wang] enable empty aggr test case --- ...empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349 | 1 + .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 6 ++---- 2 files changed, 3 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349 diff --git a/sql/hive/src/test/resources/golden/empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349 b/sql/hive/src/test/resources/golden/empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/empty aggregate input-0-bbd21aa0c1faf4c1fe6d8a822b416349 @@ -0,0 +1 @@ +NULL 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 af45dfd6e28c2..63eb07c257501 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 @@ -319,10 +319,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("DISTINCT", "SELECT DISTINCT key, value FROM src") - ignore("empty aggregate input") { - createQueryTest("empty aggregate input", - "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a") - } + createQueryTest("empty aggregate input", + "SELECT SUM(key) FROM (SELECT * FROM src LIMIT 0) a") createQueryTest("lateral view1", "SELECT tbl.* FROM src LATERAL VIEW explode(array(1,2)) tbl as a") From 0abbff286220bbcbbf28fbd80b8c5bf59ff37ce2 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 11 Dec 2014 22:51:49 -0800 Subject: [PATCH 008/227] [SPARK-4825] [SQL] CTAS fails to resolve when created using saveAsTable Fix bug when query like: ``` test("save join to table") { val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)) sql("CREATE TABLE test1 (key INT, value STRING)") testData.insertInto("test1") sql("CREATE TABLE test2 (key INT, value STRING)") testData.insertInto("test2") testData.insertInto("test2") sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").saveAsTable("test") checkAnswer( table("test"), sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq) } ``` Author: Cheng Hao Closes #3673 from chenghao-intel/spark_4825 and squashes the following commits: e8cbd56 [Cheng Hao] alternate the pattern matching order for logical plan:CTAS e004895 [Cheng Hao] fix bug --- .../sql/catalyst/plans/logical/basicOperators.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 9 +++++++++ .../spark/sql/hive/execution/SQLQuerySuite.scala | 13 +++++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 00bdf108a8398..64b8d45ebbf42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -121,7 +121,7 @@ case class CreateTableAsSelect[T]( allowExisting: Boolean, desc: Option[T] = None) extends UnaryNode { override def output = Seq.empty[Attribute] - override lazy val resolved = (databaseName != None && childrenResolved) + override lazy val resolved = databaseName != None && childrenResolved } case class WriteToFile( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 60865638e1073..d8b10b78c6c59 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -261,6 +261,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p + // TODO extra is in type of ASTNode which means the logical plan is not resolved + // Need to think about how to implement the CreateTableAsSelect.resolved case CreateTableAsSelect(db, tableName, child, allowExisting, Some(extra: ASTNode)) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) @@ -285,6 +287,13 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } CreateTableAsSelect(Some(databaseName), tblName, child, allowExisting, desc) + + case p: LogicalPlan if p.resolved => p + + case p @ CreateTableAsSelect(db, tableName, child, allowExisting, None) => + val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) + CreateTableAsSelect(Some(databaseName), tblName, child, allowExisting, None) } } 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 b341eae512417..96f3430207982 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 @@ -137,6 +137,19 @@ class SQLQuerySuite extends QueryTest { sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) } + test("SPARK-4825 save join to table") { + val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)) + sql("CREATE TABLE test1 (key INT, value STRING)") + testData.insertInto("test1") + sql("CREATE TABLE test2 (key INT, value STRING)") + testData.insertInto("test2") + testData.insertInto("test2") + sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").saveAsTable("test") + checkAnswer( + table("test"), + sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq) + } + test("SPARK-3708 Backticks aren't handled correctly is aliases") { checkAnswer( sql("SELECT k FROM (SELECT `key` AS `k` FROM src) a"), From 8091dd62eaff28196dfb9742a4f39182704b1024 Mon Sep 17 00:00:00 2001 From: Sasaki Toru Date: Thu, 11 Dec 2014 22:54:21 -0800 Subject: [PATCH 009/227] [SPARK-4742][SQL] The name of Parquet File generated by AppendingParquetOutputFormat should be zero padded When I use Parquet File as a output file using ParquetOutputFormat#getDefaultWorkFile, the file name is not zero padded while RDD#saveAsText does zero padding. Author: Sasaki Toru Closes #3602 from sasakitoa/parquet-zeroPadding and squashes the following commits: 6b0e58f [Sasaki Toru] Merge branch 'master' of git://github.com/apache/spark into parquet-zeroPadding 20dc79d [Sasaki Toru] Fixed the name of Parquet File generated by AppendingParquetOutputFormat --- .../apache/spark/sql/parquet/ParquetTableOperations.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 232ef90b017a7..5a49384ade76f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.parquet import java.io.IOException import java.lang.{Long => JLong} import java.text.SimpleDateFormat +import java.text.NumberFormat import java.util.concurrent.{Callable, TimeUnit} import java.util.{ArrayList, Collections, Date, List => JList} @@ -338,9 +339,13 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) // override to choose output filename so not overwrite existing ones override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + val numfmt = NumberFormat.getInstance() + numfmt.setMinimumIntegerDigits(5) + numfmt.setGroupingUsed(false) + val taskId: TaskID = getTaskAttemptID(context).getTaskID val partition: Int = taskId.getId - val filename = s"part-r-${partition + offset}.parquet" + val filename = "part-r-" + numfmt.format(partition + offset) + ".parquet" val committer: FileOutputCommitter = getOutputCommitter(context).asInstanceOf[FileOutputCommitter] new Path(committer.getWorkPath, filename) From 41a3f9343831a3efc18e08595d095282a8d1486b Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Thu, 11 Dec 2014 22:56:42 -0800 Subject: [PATCH 010/227] [SPARK-4829] [SQL] add rule to fold count(expr) if expr is not null Author: Daoyuan Wang Closes #3676 from adrian-wang/countexpr and squashes the following commits: dc5765b [Daoyuan Wang] add rule to fold count(expr) if expr is not null --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c4923489595b1..806c1394eb151 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -210,6 +210,7 @@ object NullPropagation extends Rule[LogicalPlan] { case e @ GetField(Literal(null, _), _) => Literal(null, e.dataType) case e @ EqualNullSafe(Literal(null, _), r) => IsNull(r) case e @ EqualNullSafe(l, Literal(null, _)) => IsNull(l) + case e @ Count(expr) if !expr.nullable => Count(Literal(1)) // For Coalesce, remove null literals. case e @ Coalesce(children) => From ef84dab8c6aa634c4dadac7a319f1ef5aaf85511 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 11 Dec 2014 23:38:40 -0800 Subject: [PATCH 011/227] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #3488 (close requested by 'pwendell') Closes #2939 (close requested by 'marmbrus') Closes #3173 (close requested by 'marmbrus') From 2a2983f7c5edce8e4d1d7592adcf227cbd462ae9 Mon Sep 17 00:00:00 2001 From: Peter Klipfel Date: Sun, 14 Dec 2014 00:01:16 -0800 Subject: [PATCH 012/227] fixed spelling errors in documentation changed "form" to "from" in 3 documentation entries for Kafka integration Author: Peter Klipfel Closes #3691 from peterklipfel/master and squashes the following commits: 0fe7fc5 [Peter Klipfel] fixed spelling errors in documentation --- .../scala/org/apache/spark/streaming/kafka/KafkaUtils.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index b4ac929e0c070..df725f0c65a64 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -75,7 +75,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. * @param jssc JavaStreamingContext object * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) @@ -93,7 +93,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * @param jssc JavaStreamingContext object * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. @@ -113,7 +113,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * @param jssc JavaStreamingContext object * @param keyTypeClass Key type of RDD * @param valueTypeClass value type of RDD From 4c0673879b5c504797dafb11607d14b04c1bf47d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 15 Dec 2014 10:54:45 -0800 Subject: [PATCH 013/227] HOTFIX: Disabling failing block manager test --- .../rdd/WriteAheadLogBackedBlockRDDSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala index d2b983c4b4d1a..728e7f0afad58 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -51,23 +51,23 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("Read data available in block manager and write ahead log") { + ignore("Read data available in block manager and write ahead log") { testRDD(5, 5) } - test("Read data available only in block manager, not in write ahead log") { + ignore("Read data available only in block manager, not in write ahead log") { testRDD(5, 0) } - test("Read data available only in write ahead log, not in block manager") { + ignore("Read data available only in write ahead log, not in block manager") { testRDD(0, 5) } - test("Read data available only in write ahead log, and test storing in block manager") { + ignore("Read data available only in write ahead log, and test storing in block manager") { testRDD(0, 5, testStoreInBM = true) } - test("Read data with partially available in block manager, and rest in write ahead log") { + ignore("Read data with partially available in block manager, and rest in write ahead log") { testRDD(3, 2) } From 8098fab06cb2be22cca4e531e8e65ab29dbb909a Mon Sep 17 00:00:00 2001 From: Yuu ISHIKAWA Date: Mon, 15 Dec 2014 13:44:15 -0800 Subject: [PATCH 014/227] [SPARK-4494][mllib] IDFModel.transform() add support for single vector I improved `IDFModel.transform` to allow using a single vector. [[SPARK-4494] IDFModel.transform() add support for single vector - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-4494) Author: Yuu ISHIKAWA Closes #3603 from yu-iskw/idf and squashes the following commits: 256ff3d [Yuu ISHIKAWA] Fix typo a3bf566 [Yuu ISHIKAWA] - Fix typo - Optimize import order - Aggregate the assertion tests - Modify `IDFModel.transform` API for pyspark d25e49b [Yuu ISHIKAWA] Add the implementation of `IDFModel.transform` for a term frequency vector --- .../org/apache/spark/mllib/feature/IDF.scala | 73 ++++++++++++------- .../apache/spark/mllib/feature/IDFSuite.scala | 67 ++++++++++------- python/pyspark/mllib/feature.py | 22 ++++-- 3 files changed, 101 insertions(+), 61 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index 720bb70b08dbf..19120e1e8af19 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -174,36 +174,17 @@ class IDFModel private[mllib] (val idf: Vector) extends Serializable { */ def transform(dataset: RDD[Vector]): RDD[Vector] = { val bcIdf = dataset.context.broadcast(idf) - dataset.mapPartitions { iter => - val thisIdf = bcIdf.value - iter.map { v => - val n = v.size - v match { - case sv: SparseVector => - val nnz = sv.indices.size - val newValues = new Array[Double](nnz) - var k = 0 - while (k < nnz) { - newValues(k) = sv.values(k) * thisIdf(sv.indices(k)) - k += 1 - } - Vectors.sparse(n, sv.indices, newValues) - case dv: DenseVector => - val newValues = new Array[Double](n) - var j = 0 - while (j < n) { - newValues(j) = dv.values(j) * thisIdf(j) - j += 1 - } - Vectors.dense(newValues) - case other => - throw new UnsupportedOperationException( - s"Only sparse and dense vectors are supported but got ${other.getClass}.") - } - } - } + dataset.mapPartitions(iter => iter.map(v => IDFModel.transform(bcIdf.value, v))) } + /** + * Transforms a term frequency (TF) vector to a TF-IDF vector + * + * @param v a term frequency vector + * @return a TF-IDF vector + */ + def transform(v: Vector): Vector = IDFModel.transform(idf, v) + /** * Transforms term frequency (TF) vectors to TF-IDF vectors (Java version). * @param dataset a JavaRDD of term frequency vectors @@ -213,3 +194,39 @@ class IDFModel private[mllib] (val idf: Vector) extends Serializable { transform(dataset.rdd).toJavaRDD() } } + +private object IDFModel { + + /** + * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector + * + * @param idf an IDF vector + * @param v a term frequence vector + * @return a TF-IDF vector + */ + def transform(idf: Vector, v: Vector): Vector = { + val n = v.size + v match { + case sv: SparseVector => + val nnz = sv.indices.size + val newValues = new Array[Double](nnz) + var k = 0 + while (k < nnz) { + newValues(k) = sv.values(k) * idf(sv.indices(k)) + k += 1 + } + Vectors.sparse(n, sv.indices, newValues) + case dv: DenseVector => + val newValues = new Array[Double](n) + var j = 0 + while (j < n) { + newValues(j) = dv.values(j) * idf(j) + j += 1 + } + Vectors.dense(newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala index 30147e7fd948f..0a5cad7caf8e4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala @@ -19,8 +19,7 @@ package org.apache.spark.mllib.feature import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -41,18 +40,26 @@ class IDFSuite extends FunSuite with MLlibTestSparkContext { math.log((m + 1.0) / (x + 1.0)) }) assert(model.idf ~== expected absTol 1e-12) - val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() - assert(tfidf.size === 3) - val tfidf0 = tfidf(0L).asInstanceOf[SparseVector] - assert(tfidf0.indices === Array(1, 3)) - assert(Vectors.dense(tfidf0.values) ~== - Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) - val tfidf1 = tfidf(1L).asInstanceOf[DenseVector] - assert(Vectors.dense(tfidf1.values) ~== - Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) - val tfidf2 = tfidf(2L).asInstanceOf[SparseVector] - assert(tfidf2.indices === Array(1)) - assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + + val assertHelper = (tfidf: Array[Vector]) => { + assert(tfidf.size === 3) + val tfidf0 = tfidf(0).asInstanceOf[SparseVector] + assert(tfidf0.indices === Array(1, 3)) + assert(Vectors.dense(tfidf0.values) ~== + Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) + val tfidf1 = tfidf(1).asInstanceOf[DenseVector] + assert(Vectors.dense(tfidf1.values) ~== + Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) + val tfidf2 = tfidf(2).asInstanceOf[SparseVector] + assert(tfidf2.indices === Array(1)) + assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + } + // Transforms a RDD + val tfidf = model.transform(termFrequencies).collect() + assertHelper(tfidf) + // Transforms local vectors + val localTfidf = localTermFrequencies.map(model.transform(_)).toArray + assertHelper(localTfidf) } test("idf minimum document frequency filtering") { @@ -74,18 +81,26 @@ class IDFSuite extends FunSuite with MLlibTestSparkContext { } }) assert(model.idf ~== expected absTol 1e-12) - val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() - assert(tfidf.size === 3) - val tfidf0 = tfidf(0L).asInstanceOf[SparseVector] - assert(tfidf0.indices === Array(1, 3)) - assert(Vectors.dense(tfidf0.values) ~== - Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) - val tfidf1 = tfidf(1L).asInstanceOf[DenseVector] - assert(Vectors.dense(tfidf1.values) ~== - Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) - val tfidf2 = tfidf(2L).asInstanceOf[SparseVector] - assert(tfidf2.indices === Array(1)) - assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + + val assertHelper = (tfidf: Array[Vector]) => { + assert(tfidf.size === 3) + val tfidf0 = tfidf(0).asInstanceOf[SparseVector] + assert(tfidf0.indices === Array(1, 3)) + assert(Vectors.dense(tfidf0.values) ~== + Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) + val tfidf1 = tfidf(1).asInstanceOf[DenseVector] + assert(Vectors.dense(tfidf1.values) ~== + Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) + val tfidf2 = tfidf(2).asInstanceOf[SparseVector] + assert(tfidf2.indices === Array(1)) + assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + } + // Transforms a RDD + val tfidf = model.transform(termFrequencies).collect() + assertHelper(tfidf) + // Transforms local vectors + val localTfidf = localTermFrequencies.map(model.transform(_)).toArray + assertHelper(localTfidf) } } diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 8cb992df2d9c7..741c630cbd6eb 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -28,7 +28,7 @@ from pyspark import RDD, SparkContext from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper -from pyspark.mllib.linalg import Vectors, _convert_to_vector +from pyspark.mllib.linalg import Vectors, Vector, _convert_to_vector __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] @@ -212,7 +212,7 @@ class IDFModel(JavaVectorTransformer): """ Represents an IDF model that can transform term frequency vectors. """ - def transform(self, dataset): + def transform(self, x): """ Transforms term frequency (TF) vectors to TF-IDF vectors. @@ -220,12 +220,14 @@ def transform(self, dataset): the terms which occur in fewer than `minDocFreq` documents will have an entry of 0. - :param dataset: an RDD of term frequency vectors - :return: an RDD of TF-IDF vectors + :param x: an RDD of term frequency vectors or a term frequency vector + :return: an RDD of TF-IDF vectors or a TF-IDF vector """ - if not isinstance(dataset, RDD): - raise TypeError("dataset should be an RDD of term frequency vectors") - return JavaVectorTransformer.transform(self, dataset) + if isinstance(x, RDD): + return JavaVectorTransformer.transform(self, x) + + x = _convert_to_vector(x) + return JavaVectorTransformer.transform(self, x) class IDF(object): @@ -255,6 +257,12 @@ class IDF(object): SparseVector(4, {1: 0.0, 3: 0.5754}) DenseVector([0.0, 0.0, 1.3863, 0.863]) SparseVector(4, {1: 0.0}) + >>> model.transform(Vectors.dense([0.0, 1.0, 2.0, 3.0])) + DenseVector([0.0, 0.0, 1.3863, 0.863]) + >>> model.transform([0.0, 1.0, 2.0, 3.0]) + DenseVector([0.0, 0.0, 1.3863, 0.863]) + >>> model.transform(Vectors.sparse(n, (1, 3), (1.0, 2.0))) + SparseVector(4, {1: 0.0, 3: 0.5754}) """ def __init__(self, minDocFreq=0): """ From f6b8591a08835c9af19210f9cdfbaab2537135c4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 15 Dec 2014 14:33:43 -0800 Subject: [PATCH 015/227] [SPARK-4826] Fix generation of temp file names in WAL tests This PR should fix SPARK-4826, an issue where a bug in how we generate temp. file names was causing spurious test failures in the write ahead log suites. Closes #3695. Closes #3701. Author: Josh Rosen Closes #3704 from JoshRosen/SPARK-4826 and squashes the following commits: f2307f5 [Josh Rosen] Use Spark Utils class for directory creation/deletion a693ddb [Josh Rosen] remove unused Random import b275e41 [Josh Rosen] Move creation of temp. dir to beforeEach/afterEach. 9362919 [Josh Rosen] [SPARK-4826] Fix bug in generation of temp file names. in WAL suites. 86c1944 [Josh Rosen] Revert "HOTFIX: Disabling failing block manager test" --- .../WriteAheadLogBackedBlockRDDSuite.scala | 28 +++++++++++-------- .../streaming/util/WriteAheadLogSuite.scala | 9 ++---- 2 files changed, 20 insertions(+), 17 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala index 728e7f0afad58..7a6a2f3e577dd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -20,15 +20,15 @@ import java.io.File import scala.util.Random -import com.google.common.io.Files import org.apache.hadoop.conf.Configuration -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter} +import org.apache.spark.util.Utils -class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { val conf = new SparkConf() .setMaster("local[2]") .setAppName(this.getClass.getSimpleName) @@ -38,36 +38,42 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { var blockManager: BlockManager = null var dir: File = null + override def beforeEach(): Unit = { + dir = Utils.createTempDir() + } + + override def afterEach(): Unit = { + Utils.deleteRecursively(dir) + } + override def beforeAll(): Unit = { sparkContext = new SparkContext(conf) blockManager = sparkContext.env.blockManager - dir = Files.createTempDir() } override def afterAll(): Unit = { // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests. sparkContext.stop() - dir.delete() System.clearProperty("spark.driver.port") } - ignore("Read data available in block manager and write ahead log") { + test("Read data available in block manager and write ahead log") { testRDD(5, 5) } - ignore("Read data available only in block manager, not in write ahead log") { + test("Read data available only in block manager, not in write ahead log") { testRDD(5, 0) } - ignore("Read data available only in write ahead log, not in block manager") { + test("Read data available only in write ahead log, not in block manager") { testRDD(0, 5) } - ignore("Read data available only in write ahead log, and test storing in block manager") { + test("Read data available only in write ahead log, and test storing in block manager") { testRDD(0, 5, testStoreInBM = true) } - ignore("Read data with partially available in block manager, and rest in write ahead log") { + test("Read data with partially available in block manager, and rest in write ahead log") { testRDD(3, 2) } @@ -137,7 +143,7 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { blockIds: Seq[BlockId] ): Seq[WriteAheadLogFileSegment] = { require(blockData.size === blockIds.size) - val writer = new WriteAheadLogWriter(new File(dir, Random.nextString(10)).toString, hadoopConf) + val writer = new WriteAheadLogWriter(new File(dir, "logFile").toString, hadoopConf) val segments = blockData.zip(blockIds).map { case (data, id) => writer.write(blockManager.dataSerialize(id, data.iterator)) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 1956a4f1db90a..8f69bcb64279d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -22,11 +22,8 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} -import scala.util.Random import WriteAheadLogSuite._ -import com.google.common.io.Files -import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.util.Utils @@ -42,9 +39,9 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { var manager: WriteAheadLogManager = null before { - tempDir = Files.createTempDir() + tempDir = Utils.createTempDir() testDir = tempDir.toString - testFile = new File(tempDir, Random.nextString(10)).toString + testFile = new File(tempDir, "testFile").toString if (manager != null) { manager.stop() manager = null @@ -52,7 +49,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { } after { - FileUtils.deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("WriteAheadLogWriter - writing data") { From 38703bbca86003995f32b2e948ad7c7c358aa99a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 15 Dec 2014 14:51:15 -0800 Subject: [PATCH 016/227] [SPARK-1037] The name of findTaskFromList & findTask in TaskSetManager.scala is confusing Hi all - I've renamed the methods referenced in this JIRA to clarify that they modify the provided arrays (find vs. deque). Author: Ilya Ganelin Closes #3665 from ilganeli/SPARK-1037B and squashes the following commits: 64c177c [Ilya Ganelin] Renamed deque to dequeue f27d85e [Ilya Ganelin] Renamed private methods to clarify that they modify the provided parameters 683482a [Ilya Ganelin] Renamed private methods to clarify that they modify the provided parameters --- .../spark/scheduler/TaskSetManager.scala | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index cabdc655f89bf..28e6147509f78 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -249,7 +249,7 @@ private[spark] class TaskSetManager( * This method also cleans up any tasks in the list that have already * been launched, since we want that to happen lazily. */ - private def findTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { + private def dequeueTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { var indexOffset = list.size while (indexOffset > 0) { indexOffset -= 1 @@ -290,7 +290,7 @@ private[spark] class TaskSetManager( * an attempt running on this host, in case the host is slow. In addition, the task should meet * the given locality constraint. */ - private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) + private def dequeueSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) : Option[(Int, TaskLocality.Value)] = { speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set @@ -366,22 +366,22 @@ private[spark] class TaskSetManager( * * @return An option containing (task index within the task set, locality, is speculative?) */ - private def findTask(execId: String, host: String, maxLocality: TaskLocality.Value) + private def dequeueTask(execId: String, host: String, maxLocality: TaskLocality.Value) : Option[(Int, TaskLocality.Value, Boolean)] = { - for (index <- findTaskFromList(execId, getPendingTasksForExecutor(execId))) { + for (index <- dequeueTaskFromList(execId, getPendingTasksForExecutor(execId))) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) { - for (index <- findTaskFromList(execId, getPendingTasksForHost(host))) { + for (index <- dequeueTaskFromList(execId, getPendingTasksForHost(host))) { return Some((index, TaskLocality.NODE_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NO_PREF)) { // Look for noPref tasks after NODE_LOCAL for minimize cross-rack traffic - for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { + for (index <- dequeueTaskFromList(execId, pendingTasksWithNoPrefs)) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } } @@ -389,20 +389,20 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) - index <- findTaskFromList(execId, getPendingTasksForRack(rack)) + index <- dequeueTaskFromList(execId, getPendingTasksForRack(rack)) } { return Some((index, TaskLocality.RACK_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) { - for (index <- findTaskFromList(execId, allPendingTasks)) { + for (index <- dequeueTaskFromList(execId, allPendingTasks)) { return Some((index, TaskLocality.ANY, false)) } } // find a speculative task if all others tasks have been scheduled - findSpeculativeTask(execId, host, maxLocality).map { + dequeueSpeculativeTask(execId, host, maxLocality).map { case (taskIndex, allowedLocality) => (taskIndex, allowedLocality, true)} } @@ -436,7 +436,7 @@ private[spark] class TaskSetManager( } } - findTask(execId, host, allowedLocality) match { + dequeueTask(execId, host, allowedLocality) match { case Some((index, taskLocality, speculative)) => { // Found a task; do some bookkeeping and return a task description val task = tasks(index) @@ -704,7 +704,7 @@ private[spark] class TaskSetManager( // Re-enqueue pending tasks for this host based on the status of the cluster. Note // that it's okay if we add a task to the same queue twice (if it had multiple preferred - // locations), because findTaskFromList will skip already-running tasks. + // locations), because dequeueTaskFromList will skip already-running tasks. for (index <- getPendingTasksForExecutor(execId)) { addPendingTask(index, readding=true) } From 8176b7a02e6b62bbce194c3ce9802d58b7472101 Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Mon, 15 Dec 2014 14:52:17 -0800 Subject: [PATCH 017/227] [SPARK-4668] Fix some documentation typos. Author: Ryan Williams Closes #3523 from ryan-williams/tweaks and squashes the following commits: d2eddaa [Ryan Williams] code review feedback ce27fc1 [Ryan Williams] CoGroupedRDD comment nit c6cfad9 [Ryan Williams] remove unnecessary if statement b74ea35 [Ryan Williams] comment fix b0221f0 [Ryan Williams] fix a gendered pronoun c71ffed [Ryan Williams] use names on a few boolean parameters 89954aa [Ryan Williams] clarify some comments in {Security,Shuffle}Manager e465dac [Ryan Williams] Saved building-spark.md with Dillinger.io 83e8358 [Ryan Williams] fix pom.xml typo dc4662b [Ryan Williams] typo fixes in tuning.md, configuration.md --- .../org/apache/spark/MapOutputTracker.scala | 14 +++++------- .../org/apache/spark/SecurityManager.scala | 18 +++++++-------- .../scala/org/apache/spark/SparkEnv.scala | 22 ++++++++++++++++--- .../org/apache/spark/rdd/CoGroupedRDD.scala | 6 ++--- .../apache/spark/scheduler/MapStatus.scala | 2 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../apache/spark/shuffle/ShuffleManager.scala | 4 ++-- docs/building-spark.md | 16 +++++++++++++- docs/configuration.md | 6 ++--- docs/tuning.md | 8 +++---- pom.xml | 2 +- .../streaming/receiver/ActorReceiver.scala | 2 +- 12 files changed, 65 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 7d96962c4acd7..e45885338ea00 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -136,14 +136,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") var fetchedStatuses: Array[MapStatus] = null fetching.synchronized { - if (fetching.contains(shuffleId)) { - // Someone else is fetching it; wait for them to be done - while (fetching.contains(shuffleId)) { - try { - fetching.wait() - } catch { - case e: InterruptedException => - } + // Someone else is fetching it; wait for them to be done + while (fetching.contains(shuffleId)) { + try { + fetching.wait() + } catch { + case e: InterruptedException => } } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index dbff9d12b5ad7..49dae5231a92c 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -93,19 +93,19 @@ import org.apache.spark.network.sasl.SecretKeyHolder * Note that SASL is pluggable as to what mechanism it uses. We currently use * DIGEST-MD5 but this could be changed to use Kerberos or other in the future. * Spark currently supports "auth" for the quality of protection, which means - * the connection is not supporting integrity or privacy protection (encryption) + * the connection does not support integrity or privacy protection (encryption) * after authentication. SASL also supports "auth-int" and "auth-conf" which - * SPARK could be support in the future to allow the user to specify the quality + * SPARK could support in the future to allow the user to specify the quality * of protection they want. If we support those, the messages will also have to * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. * * Since the NioBlockTransferService does asynchronous messages passing, the SASL * authentication is a bit more complex. A ConnectionManager can be both a client - * and a Server, so for a particular connection is has to determine what to do. + * and a Server, so for a particular connection it has to determine what to do. * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. - * The ConnectionManager tracks all the sendingConnections using the ConnectionId - * and waits for the response from the server and does the handshake before sending + * The ConnectionManager tracks all the sendingConnections using the ConnectionId, + * waits for the response from the server, and does the handshake before sending * the real message. * * The NettyBlockTransferService ensures that SASL authentication is performed @@ -114,14 +114,14 @@ import org.apache.spark.network.sasl.SecretKeyHolder * * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work - * properly. For non-Yarn deployments, users can write a filter to go through a - * companies normal login service. If an authentication filter is in place then the + * properly. For non-Yarn deployments, users can write a filter to go through their + * organization's normal login service. If an authentication filter is in place then the * SparkUI can be configured to check the logged in user against the list of users who * have view acls to see if that user is authorized. * The filters can also be used for many different purposes. For instance filters * could be used for logging, encryption, or compression. * - * The exact mechanisms used to generate/distributed the shared secret is deployment specific. + * The exact mechanisms used to generate/distribute the shared secret are deployment-specific. * * For Yarn deployments, the secret is automatically generated using the Akka remote * Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed @@ -138,7 +138,7 @@ import org.apache.spark.network.sasl.SecretKeyHolder * All the nodes (Master and Workers) and the applications need to have the same shared secret. * This again is not ideal as one user could potentially affect another users application. * This should be enhanced in the future to provide better protection. - * If the UI needs to be secured the user needs to install a javax servlet filter to do the + * If the UI needs to be secure, the user needs to install a javax servlet filter to do the * authentication. Spark will then use that user to compare against the view acls to do * authorization. If not filter is in place the user is generally null and no authorization * can take place. diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index e464b32e61dd6..f4215f268a0d3 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,7 +156,15 @@ object SparkEnv extends Logging { assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") val hostname = conf.get("spark.driver.host") val port = conf.get("spark.driver.port").toInt - create(conf, SparkContext.DRIVER_IDENTIFIER, hostname, port, true, isLocal, listenerBus) + create( + conf, + SparkContext.DRIVER_IDENTIFIER, + hostname, + port, + isDriver = true, + isLocal = isLocal, + listenerBus = listenerBus + ) } /** @@ -171,8 +179,16 @@ object SparkEnv extends Logging { numCores: Int, isLocal: Boolean, actorSystem: ActorSystem = null): SparkEnv = { - create(conf, executorId, hostname, port, false, isLocal, defaultActorSystem = actorSystem, - numUsableCores = numCores) + create( + conf, + executorId, + hostname, + port, + isDriver = false, + isLocal = isLocal, + defaultActorSystem = actorSystem, + numUsableCores = numCores + ) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index ffc0a8a6d67eb..70edf191d928a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -60,7 +60,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * - * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of + * Note: This is an internal API. We recommend users use RDD.cogroup(...) instead of * instantiating this directly. * @param rdds parent RDDs. @@ -70,8 +70,8 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Array[Iterable[_]])](rdds.head.context, Nil) { - // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs). - // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner. + // For example, `(k, a) cogroup (k, b)` produces k -> Array(ArrayBuffer as, ArrayBuffer bs). + // Each ArrayBuffer is represented as a CoGroup, and the resulting Array as a CoGroupCombiner. // CoGroupValue is the intermediate state of each value before being merged in compute. private type CoGroup = CompactBuffer[Any] private type CoGroupValue = (Any, Int) // Int is dependency number diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 01d5943d777f3..1efce124c0a6b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -122,7 +122,7 @@ private[spark] class CompressedMapStatus( /** * A [[MapStatus]] implementation that only stores the average size of non-empty blocks, - * plus a bitmap for tracking which blocks are non-empty. During serialization, this bitmap + * plus a bitmap for tracking which blocks are empty. During serialization, this bitmap * is compressed. * * @param loc location where the task is being executed diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 2552d03d18d06..d7dde4fe38436 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { final def run(attemptId: Long): T = { - context = new TaskContextImpl(stageId, partitionId, attemptId, false) + context = new TaskContextImpl(stageId, partitionId, attemptId, runningLocally = false) TaskContextHelper.setTaskContext(context) context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 801ae54086053..a44a8e1249256 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -20,8 +20,8 @@ package org.apache.spark.shuffle import org.apache.spark.{TaskContext, ShuffleDependency} /** - * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on both the - * driver and executors, based on the spark.shuffle.manager setting. The driver registers shuffles + * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on the driver + * and on each executor, based on the spark.shuffle.manager setting. The driver registers shuffles * with it, and executors (or tasks running locally in the driver) can ask to read and write data. * * NOTE: this will be instantiated by SparkEnv so its constructor can take a SparkConf and diff --git a/docs/building-spark.md b/docs/building-spark.md index 4922e877e9b78..70165eabca435 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -124,7 +124,21 @@ We use the scala-maven-plugin which supports incremental and continuous compilat mvn scala:cc -should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. +should run continuous compilation (i.e. wait for changes). However, this has not been tested +extensively. A couple of gotchas to note: +* it only scans the paths `src/main` and `src/test` (see +[docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work +from within certain submodules that have that structure. +* you'll typically need to run `mvn install` from the project root for compilation within +specific submodules to work; this is because submodules that depend on other submodules do so via +the `spark-parent` module). + +Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: + ``` + $ mvn install + $ cd core + $ mvn scala:cc +``` # Using With IntelliJ IDEA diff --git a/docs/configuration.md b/docs/configuration.md index acee267883ed5..64aa94f622afa 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -75,8 +75,8 @@ in the `spark-defaults.conf` file. The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. This is a useful place to check to make sure that your properties have been set correctly. Note -that only values explicitly specified through either `spark-defaults.conf` or SparkConf will -appear. For all other configuration properties, you can assume the default value is used. +that only values explicitly specified through `spark-defaults.conf`, `SparkConf`, or the command +line will appear. For all other configuration properties, you can assume the default value is used. ## Available Properties @@ -310,7 +310,7 @@ Apart from these, the following properties are also available, and may be useful (none) Add the environment variable specified by EnvironmentVariableName to the Executor - process. The user can specify multiple of these and to set multiple environment variables. + process. The user can specify multiple of these to set multiple environment variables. diff --git a/docs/tuning.md b/docs/tuning.md index c4ca766328c1e..e2fdcfe6a37d9 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -111,7 +111,7 @@ pointer-based data structures and wrapper objects. There are several ways to do 3. Consider using numeric IDs or enumeration objects instead of strings for keys. 4. If you have less than 32 GB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be four bytes instead of eight. You can add these options in - [`spark-env.sh`](configuration.html#environment-variables-in-spark-envsh). + [`spark-env.sh`](configuration.html#environment-variables). ## Serialized RDD Storage @@ -154,7 +154,7 @@ By default, Spark uses 60% of the configured executor memory (`spark.executor.me cache RDDs. This means that 40% of memory is available for any objects created during task execution. In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of -memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call +memory, lowering this value will help reduce the memory consumption. To change this to, say, 50%, you can call `conf.set("spark.storage.memoryFraction", "0.5")` on your SparkConf. Combined with the use of serialized caching, using a smaller cache should be sufficient to mitigate most of the garbage collection problems. In case you are interested in further tuning the Java GC, continue reading below. @@ -190,7 +190,7 @@ temporary objects created during task execution. Some steps which may be useful * As an example, if your task is reading data from HDFS, the amount of memory used by the task can be estimated using the size of the data block read from HDFS. Note that the size of a decompressed block is often 2 or 3 times the - size of the block. So if we wish to have 3 or 4 tasks worth of working space, and the HDFS block size is 64 MB, + size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 64 MB, we can estimate size of Eden to be `4*3*64MB`. * Monitor how the frequency and time taken by garbage collection changes with the new settings. @@ -219,7 +219,7 @@ working set of one of your tasks, such as one of the reduce tasks in `groupByKey Spark's shuffle operations (`sortByKey`, `groupByKey`, `reduceByKey`, `join`, etc) build a hash table within each task to perform the grouping, which can often be large. The simplest fix here is to *increase the level of parallelism*, so that each task's input set is smaller. Spark can efficiently -support tasks as short as 200 ms, because it reuses one worker JVMs across all tasks and it has +support tasks as short as 200 ms, because it reuses one executor JVM across many tasks and it has a low task launching cost, so you can safely increase the level of parallelism to more than the number of cores in your clusters. diff --git a/pom.xml b/pom.xml index f42257265ede3..cdc2969eddcac 100644 --- a/pom.xml +++ b/pom.xml @@ -267,7 +267,7 @@ 1.0.0 diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index 1868a1ebc7b4a..a7d63bd4f2dbf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -123,7 +123,7 @@ private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorRec * As Actors can also be used to receive data from almost any stream source. * A nice set of abstraction(s) for actors as receivers is already provided for * a few general cases. It is thus exposed as an API where user may come with - * his own Actor to run as receiver for Spark Streaming input source. + * their own Actor to run as receiver for Spark Streaming input source. * * This starts a supervisor actor which starts workers and also provides * [http://doc.akka.io/docs/akka/snapshot/scala/fault-tolerance.html fault-tolerance]. From 2a28bc61009a170af3853c78f7f36970898a6d56 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 15 Dec 2014 16:06:15 -0800 Subject: [PATCH 018/227] SPARK-785 [CORE] ClosureCleaner not invoked on most PairRDDFunctions This looked like perhaps a simple and important one. `combineByKey` looks like it should clean its arguments' closures, and that in turn covers apparently all remaining functions in `PairRDDFunctions` which delegate to it. Author: Sean Owen Closes #3690 from srowen/SPARK-785 and squashes the following commits: 8df68fe [Sean Owen] Clean context of most remaining functions in PairRDDFunctions, which ultimately call combineByKey --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index c43e1f2fe135e..b0434c9a3b6b3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -84,7 +84,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } } - val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) + val aggregator = new Aggregator[K, V, C]( + self.context.clean(createCombiner), + self.context.clean(mergeValue), + self.context.clean(mergeCombiners)) if (self.partitioner == Some(partitioner)) { self.mapPartitions(iter => { val context = TaskContext.get() From 5c24759ddc25cbafbedbaafbf053d38015a7774e Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 15 Dec 2014 16:46:21 -0800 Subject: [PATCH 019/227] [Minor][Core] fix comments in MapOutputTracker Using driver and executor in the comments of ```MapOutputTracker``` is more clear. Author: wangfei Closes #3700 from scwf/commentFix and squashes the following commits: aa68524 [wangfei] master and worker should be driver and executor --- .../org/apache/spark/MapOutputTracker.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index e45885338ea00..a074ab8ece1b7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -72,7 +72,7 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster /** * Class that keeps track of the location of the map output of * a stage. This is abstract because different versions of MapOutputTracker - * (driver and worker) use different HashMap to store its metadata. + * (driver and executor) use different HashMap to store its metadata. */ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) @@ -81,11 +81,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging var trackerActor: ActorRef = _ /** - * This HashMap has different behavior for the master and the workers. + * This HashMap has different behavior for the driver and the executors. * - * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks. - * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the - * master's corresponding HashMap. + * On the driver, it serves as the source of map outputs recorded from ShuffleMapTasks. + * On the executors, it simply serves as a cache, in which a miss triggers a fetch from the + * driver's corresponding HashMap. * * Note: because mapStatuses is accessed concurrently, subclasses should make sure it's a * thread-safe map. @@ -99,7 +99,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging protected var epoch: Long = 0 protected val epochLock = new AnyRef - /** Remembers which map output locations are currently being fetched on a worker. */ + /** Remembers which map output locations are currently being fetched on an executor. */ private val fetching = new HashSet[Int] /** @@ -196,8 +196,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** * Called from executors to update the epoch number, potentially clearing old outputs - * because of a fetch failure. Each worker task calls this with the latest epoch - * number on the master at the time it was created. + * because of a fetch failure. Each executor task calls this with the latest epoch + * number on the driver at the time it was created. */ def updateEpoch(newEpoch: Long) { epochLock.synchronized { @@ -229,7 +229,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) private var cacheEpoch = epoch /** - * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the master, + * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the driver, * so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set). * Other than these two scenarios, nothing should be dropped from this HashMap. */ @@ -339,7 +339,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } /** - * MapOutputTracker for the workers, which fetches map output information from the driver's + * MapOutputTracker for the executors, which fetches map output information from the driver's * MapOutputTrackerMaster. */ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { From 81112e4b573292e76c7feeed995751bd7a5fe489 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 15 Dec 2014 17:12:05 -0800 Subject: [PATCH 020/227] SPARK-4814 [CORE] Enable assertions in SBT, Maven tests / AssertionError from Hive's LazyBinaryInteger This enables assertions for the Maven and SBT build, but overrides the Hive module to not enable assertions. Author: Sean Owen Closes #3692 from srowen/SPARK-4814 and squashes the following commits: caca704 [Sean Owen] Disable assertions just for Hive f71e783 [Sean Owen] Enable assertions for SBT and Maven build --- pom.xml | 2 +- project/SparkBuild.scala | 3 +++ sql/hive/pom.xml | 4 ++++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index cdc2969eddcac..9e6fe09d95bbe 100644 --- a/pom.xml +++ b/pom.xml @@ -958,7 +958,7 @@ ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m + -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m true diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 39ac27f820d89..ff8cf81b286af 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -256,6 +256,8 @@ object Hive { lazy val settings = Seq( javaOptions += "-XX:MaxPermSize=1g", + // Specially disable assertions since some Hive tests fail them + javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"), // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings @@ -385,6 +387,7 @@ object TestSettings { javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, + javaOptions in Test += "-ea", javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g" .split(" ").toSeq, // This places test scope jars on the classpath of executors during tests. diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index f6805b942153a..95db71c2fd954 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -161,6 +161,10 @@ org.scalatest scalatest-maven-plugin + + + -da -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m + org.codehaus.mojo From c7628771da9a7d4bd4d7abfdede37ce1568dcd01 Mon Sep 17 00:00:00 2001 From: meiyoula <1039320815@qq.com> Date: Mon, 15 Dec 2014 22:30:18 -0800 Subject: [PATCH 021/227] [SPARK-4792] Add error message when making local dir unsuccessfully Author: meiyoula <1039320815@qq.com> Closes #3635 from XuTingjun/master and squashes the following commits: dd1c66d [meiyoula] when old is deleted, it will throw an exception where call it 2a55bc2 [meiyoula] Update DiskBlockManager.scala 1483a4a [meiyoula] Delete multiple retries to make dir 67f7902 [meiyoula] Try some times to make dir maybe more reasonable 1c51a0c [meiyoula] Update DiskBlockManager.scala --- .../scala/org/apache/spark/storage/DiskBlockManager.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 58fba54710510..bb2ae9f3f4586 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.File +import java.io.{IOException, File} import java.text.SimpleDateFormat import java.util.{Date, Random, UUID} @@ -71,7 +71,9 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon old } else { val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) - newDir.mkdir() + if (!newDir.exists() && !newDir.mkdir()) { + throw new IOException(s"Failed to create local dir in $newDir.") + } subDirs(dirId)(subDirId) = newDir newDir } From c246b95dd2f565043db429c38c6cc029a0b870c1 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 15 Dec 2014 22:58:26 -0800 Subject: [PATCH 022/227] [SPARK-4841] fix zip with textFile() UTF8Deserializer can not be used in BatchedSerializer, so always use PickleSerializer() when change batchSize in zip(). Also, if two RDD have the same batch size already, they did not need re-serialize any more. Author: Davies Liu Closes #3706 from davies/fix_4841 and squashes the following commits: 20ce3a3 [Davies Liu] fix bug in _reserialize() e3ebf7c [Davies Liu] add comment 379d2c8 [Davies Liu] fix zip with textFile() --- python/pyspark/rdd.py | 25 +++++++++++-------------- python/pyspark/serializers.py | 6 ++++++ python/pyspark/tests.py | 9 +++++++++ 3 files changed, 26 insertions(+), 14 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 57754776faaa2..bd2ff00c0f1be 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -469,8 +469,7 @@ def intersection(self, other): def _reserialize(self, serializer=None): serializer = serializer or self.ctx.serializer if self._jrdd_deserializer != serializer: - if not isinstance(self, PipelinedRDD): - self = self.map(lambda x: x, preservesPartitioning=True) + self = self.map(lambda x: x, preservesPartitioning=True) self._jrdd_deserializer = serializer return self @@ -1798,23 +1797,21 @@ def zip(self, other): def get_batch_size(ser): if isinstance(ser, BatchedSerializer): return ser.batchSize - return 1 + return 1 # not batched def batch_as(rdd, batchSize): - ser = rdd._jrdd_deserializer - if isinstance(ser, BatchedSerializer): - ser = ser.serializer - return rdd._reserialize(BatchedSerializer(ser, batchSize)) + return rdd._reserialize(BatchedSerializer(PickleSerializer(), batchSize)) my_batch = get_batch_size(self._jrdd_deserializer) other_batch = get_batch_size(other._jrdd_deserializer) - # use the smallest batchSize for both of them - batchSize = min(my_batch, other_batch) - if batchSize <= 0: - # auto batched or unlimited - batchSize = 100 - other = batch_as(other, batchSize) - self = batch_as(self, batchSize) + if my_batch != other_batch: + # use the smallest batchSize for both of them + batchSize = min(my_batch, other_batch) + if batchSize <= 0: + # auto batched or unlimited + batchSize = 100 + other = batch_as(other, batchSize) + self = batch_as(self, batchSize) if self.getNumPartitions() != other.getNumPartitions(): raise ValueError("Can only zip with RDD which has the same number of partitions") diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 33aa55f7f1429..bd08c9a6d20d6 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -463,6 +463,9 @@ def dumps(self, obj): def loads(self, obj): return self.serializer.loads(zlib.decompress(obj)) + def __eq__(self, other): + return isinstance(other, CompressedSerializer) and self.serializer == other.serializer + class UTF8Deserializer(Serializer): @@ -489,6 +492,9 @@ def load_stream(self, stream): except EOFError: return + def __eq__(self, other): + return isinstance(other, UTF8Deserializer) and self.use_unicode == other.use_unicode + def read_long(stream): length = stream.read(8) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 32645778c2b8f..bca52a7ce6d58 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -533,6 +533,15 @@ def test_zip_with_different_serializers(self): a = a._reserialize(BatchedSerializer(PickleSerializer(), 2)) b = b._reserialize(MarshalSerializer()) self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)]) + # regression test for SPARK-4841 + path = os.path.join(SPARK_HOME, "python/test_support/hello.txt") + t = self.sc.textFile(path) + cnt = t.count() + self.assertEqual(cnt, t.zip(t).count()) + rdd = t.map(str) + self.assertEqual(cnt, t.zip(rdd).count()) + # regression test for bug in _reserializer() + self.assertEqual(cnt, t.zip(rdd).count()) def test_zip_with_different_number_of_items(self): a = self.sc.parallelize(range(5), 2) From ed362008f0a317729f8404e86e57d8a6ceb60f21 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 16 Dec 2014 11:19:36 -0800 Subject: [PATCH 023/227] [SPARK-4437] update doc for WholeCombineFileRecordReader update doc for WholeCombineFileRecordReader Author: Davies Liu Author: Josh Rosen Closes #3301 from davies/fix_doc and squashes the following commits: 1d7422f [Davies Liu] Merge pull request #2 from JoshRosen/whole-text-file-cleanup dc3d21a [Josh Rosen] More genericization in ConfigurableCombineFileRecordReader. 95d13eb [Davies Liu] address comment bf800b9 [Davies Liu] update doc for WholeCombineFileRecordReader --- .../input/WholeTextFileInputFormat.scala | 12 ++---- .../input/WholeTextFileRecordReader.scala | 43 ++++++++++--------- 2 files changed, 25 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index d3601cca832b2..aaef7c74eea33 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -19,7 +19,6 @@ package org.apache.spark.input import scala.collection.JavaConversions._ -import org.apache.hadoop.conf.{Configuration, Configurable} import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.JobContext @@ -38,18 +37,13 @@ private[spark] class WholeTextFileInputFormat override protected def isSplitable(context: JobContext, file: Path): Boolean = false - private var conf: Configuration = _ - def setConf(c: Configuration) { - conf = c - } - def getConf: Configuration = conf - override def createRecordReader( split: InputSplit, context: TaskAttemptContext): RecordReader[String, String] = { - val reader = new WholeCombineFileRecordReader(split, context) - reader.setConf(conf) + val reader = + new ConfigurableCombineFileRecordReader(split, context, classOf[WholeTextFileRecordReader]) + reader.setConf(getConf) reader } diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala index 6d59b24eb0596..1b1131b9b8831 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -17,7 +17,7 @@ package org.apache.spark.input -import org.apache.hadoop.conf.{Configuration, Configurable} +import org.apache.hadoop.conf.{Configuration, Configurable => HConfigurable} import com.google.common.io.{ByteStreams, Closeables} import org.apache.hadoop.io.Text @@ -27,6 +27,18 @@ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, CombineFileRecor import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext + +/** + * A trait to implement [[org.apache.hadoop.conf.Configurable Configurable]] interface. + */ +private[spark] trait Configurable extends HConfigurable { + private var conf: Configuration = _ + def setConf(c: Configuration) { + conf = c + } + def getConf: Configuration = conf +} + /** * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file * out in a key-value pair, where the key is the file path and the value is the entire content of @@ -38,12 +50,6 @@ private[spark] class WholeTextFileRecordReader( index: Integer) extends RecordReader[String, String] with Configurable { - private var conf: Configuration = _ - def setConf(c: Configuration) { - conf = c - } - def getConf: Configuration = conf - private[this] val path = split.getPath(index) private[this] val fs = path.getFileSystem(context.getConfiguration) @@ -87,29 +93,24 @@ private[spark] class WholeTextFileRecordReader( /** - * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file - * out in a key-value pair, where the key is the file path and the value is the entire content of - * the file. + * A [[org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader CombineFileRecordReader]] + * that can pass Hadoop Configuration to [[org.apache.hadoop.conf.Configurable Configurable]] + * RecordReaders. */ -private[spark] class WholeCombineFileRecordReader( +private[spark] class ConfigurableCombineFileRecordReader[K, V]( split: InputSplit, - context: TaskAttemptContext) - extends CombineFileRecordReader[String, String]( + context: TaskAttemptContext, + recordReaderClass: Class[_ <: RecordReader[K, V] with HConfigurable]) + extends CombineFileRecordReader[K, V]( split.asInstanceOf[CombineFileSplit], context, - classOf[WholeTextFileRecordReader] + recordReaderClass ) with Configurable { - private var conf: Configuration = _ - def setConf(c: Configuration) { - conf = c - } - def getConf: Configuration = conf - override def initNextRecordReader(): Boolean = { val r = super.initNextRecordReader() if (r) { - this.curReader.asInstanceOf[WholeTextFileRecordReader].setConf(conf) + this.curReader.asInstanceOf[HConfigurable].setConf(getConf) } r } From cb484474934d664000df3d63a326bcd6b12f2f09 Mon Sep 17 00:00:00 2001 From: jbencook Date: Tue, 16 Dec 2014 11:37:23 -0800 Subject: [PATCH 024/227] [SPARK-4855][mllib] testing the Chi-squared hypothesis test This PR tests the pyspark Chi-squared hypothesis test from this commit: c8abddc5164d8cf11cdede6ab3d5d1ea08028708 and moves some of the error messaging in to python. It is a port of the Scala tests here: [HypothesisTestSuite.scala](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala) Hopefully, SPARK-2980 can be closed. Author: jbencook Closes #3679 from jbencook/master and squashes the following commits: 44078e0 [jbencook] checking that bad input throws the correct exceptions f12ee10 [jbencook] removing checks for ValueError since input tests are on the Scala side 7536cf1 [jbencook] removing python checks for invalid input a17ee84 [jbencook] [SPARK-2980][mllib] adding unit tests for the pyspark chi-squared test 3aeb0d9 [jbencook] [SPARK-2980][mllib] bringing Chi-squared error messages to the python side --- python/pyspark/mllib/tests.py | 100 +++++++++++++++++++++++++++++++++- 1 file changed, 99 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8332f8e061f48..5034f229e824a 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -23,6 +23,7 @@ import array as pyarray from numpy import array, array_equal +from py4j.protocol import Py4JJavaError if sys.version_info[:2] <= (2, 6): try: @@ -34,7 +35,7 @@ import unittest from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ - DenseMatrix + DenseMatrix, Vectors, Matrices from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics @@ -400,6 +401,103 @@ def test_regression(self): self.assertTrue(dt_model.predict(features[3]) > 0) +class ChiSqTestTests(PySparkTestCase): + def test_goodness_of_fit(self): + from numpy import inf + + observed = Vectors.dense([4, 6, 5]) + pearson = Statistics.chiSqTest(observed) + + # Validated against the R command `chisq.test(c(4, 6, 5), p=c(1/3, 1/3, 1/3))` + self.assertEqual(pearson.statistic, 0.4) + self.assertEqual(pearson.degreesOfFreedom, 2) + self.assertAlmostEqual(pearson.pValue, 0.8187, 4) + + # Different expected and observed sum + observed1 = Vectors.dense([21, 38, 43, 80]) + expected1 = Vectors.dense([3, 5, 7, 20]) + pearson1 = Statistics.chiSqTest(observed1, expected1) + + # Results validated against the R command + # `chisq.test(c(21, 38, 43, 80), p=c(3/35, 1/7, 1/5, 4/7))` + self.assertAlmostEqual(pearson1.statistic, 14.1429, 4) + self.assertEqual(pearson1.degreesOfFreedom, 3) + self.assertAlmostEqual(pearson1.pValue, 0.002717, 4) + + # Vectors with different sizes + observed3 = Vectors.dense([1.0, 2.0, 3.0]) + expected3 = Vectors.dense([1.0, 2.0, 3.0, 4.0]) + self.assertRaises(ValueError, Statistics.chiSqTest, observed3, expected3) + + # Negative counts in observed + neg_obs = Vectors.dense([1.0, 2.0, 3.0, -4.0]) + self.assertRaises(Py4JJavaError, Statistics.chiSqTest, neg_obs, expected1) + + # Count = 0.0 in expected but not observed + zero_expected = Vectors.dense([1.0, 0.0, 3.0]) + pearson_inf = Statistics.chiSqTest(observed, zero_expected) + self.assertEqual(pearson_inf.statistic, inf) + self.assertEqual(pearson_inf.degreesOfFreedom, 2) + self.assertEqual(pearson_inf.pValue, 0.0) + + # 0.0 in expected and observed simultaneously + zero_observed = Vectors.dense([2.0, 0.0, 1.0]) + self.assertRaises(Py4JJavaError, Statistics.chiSqTest, zero_observed, zero_expected) + + def test_matrix_independence(self): + data = [40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0] + chi = Statistics.chiSqTest(Matrices.dense(3, 4, data)) + + # Results validated against R command + # `chisq.test(rbind(c(40, 56, 31, 30),c(24, 32, 10, 15), c(29, 42, 0, 12)))` + self.assertAlmostEqual(chi.statistic, 21.9958, 4) + self.assertEqual(chi.degreesOfFreedom, 6) + self.assertAlmostEqual(chi.pValue, 0.001213, 4) + + # Negative counts + neg_counts = Matrices.dense(2, 2, [4.0, 5.0, 3.0, -3.0]) + self.assertRaises(Py4JJavaError, Statistics.chiSqTest, neg_counts) + + # Row sum = 0.0 + row_zero = Matrices.dense(2, 2, [0.0, 1.0, 0.0, 2.0]) + self.assertRaises(Py4JJavaError, Statistics.chiSqTest, row_zero) + + # Column sum = 0.0 + col_zero = Matrices.dense(2, 2, [0.0, 0.0, 2.0, 2.0]) + self.assertRaises(Py4JJavaError, Statistics.chiSqTest, col_zero) + + def test_chi_sq_pearson(self): + data = [ + LabeledPoint(0.0, Vectors.dense([0.5, 10.0])), + LabeledPoint(0.0, Vectors.dense([1.5, 20.0])), + LabeledPoint(1.0, Vectors.dense([1.5, 30.0])), + LabeledPoint(0.0, Vectors.dense([3.5, 30.0])), + LabeledPoint(0.0, Vectors.dense([3.5, 40.0])), + LabeledPoint(1.0, Vectors.dense([3.5, 40.0])) + ] + + for numParts in [2, 4, 6, 8]: + chi = Statistics.chiSqTest(self.sc.parallelize(data, numParts)) + feature1 = chi[0] + self.assertEqual(feature1.statistic, 0.75) + self.assertEqual(feature1.degreesOfFreedom, 2) + self.assertAlmostEqual(feature1.pValue, 0.6873, 4) + + feature2 = chi[1] + self.assertEqual(feature2.statistic, 1.5) + self.assertEqual(feature2.degreesOfFreedom, 3) + self.assertAlmostEqual(feature2.pValue, 0.6823, 4) + + def test_right_number_of_results(self): + num_cols = 1001 + sparse_data = [ + LabeledPoint(0.0, Vectors.sparse(num_cols, [(100, 2.0)])), + LabeledPoint(0.1, Vectors.sparse(num_cols, [(200, 1.0)])) + ] + chi = Statistics.chiSqTest(self.sc.parallelize(sparse_data)) + self.assertEqual(len(chi), num_cols) + self.assertIsNotNone(chi[1000]) + if __name__ == "__main__": if not _have_scipy: print "NOTE: Skipping SciPy tests as it does not seem to be installed" From d12c0711faa3d4333513fcbbbee4868bcb784a26 Mon Sep 17 00:00:00 2001 From: Mike Jennings Date: Tue, 16 Dec 2014 12:13:21 -0800 Subject: [PATCH 025/227] [SPARK-3405] add subnet-id and vpc-id options to spark_ec2.py Based on this gist: https://gist.github.com/amar-analytx/0b62543621e1f246c0a2 We use security group ids instead of security group to get around this issue: https://github.com/boto/boto/issues/350 Author: Mike Jennings Author: Mike Jennings Closes #2872 from mvj101/SPARK-3405 and squashes the following commits: be9cb43 [Mike Jennings] `pep8 spark_ec2.py` runs cleanly. 4dc6756 [Mike Jennings] Remove duplicate comment 731d94c [Mike Jennings] Update for code review. ad90a36 [Mike Jennings] Merge branch 'master' of https://github.com/apache/spark into SPARK-3405 1ebffa1 [Mike Jennings] Merge branch 'master' into SPARK-3405 52aaeec [Mike Jennings] [SPARK-3405] add subnet-id and vpc-id options to spark_ec2.py --- docs/ec2-scripts.md | 19 +++++++++++++ ec2/spark_ec2.py | 66 ++++++++++++++++++++++++++++++++++----------- 2 files changed, 70 insertions(+), 15 deletions(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index ed51d0abb3a45..d50f445d7ecc7 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -94,6 +94,25 @@ another. permissions on your private key file, you can run `launch` with the `--resume` option to restart the setup process on an existing cluster. +# Launching a Cluster in a VPC + +- Run + `./spark-ec2 -k -i -s --vpc-id= --subnet-id= launch `, + where `` is the name of your EC2 key pair (that you gave it + when you created it), `` is the private key file for your + key pair, `` is the number of slave nodes to launch (try + 1 at first), `` is the name of your VPC, `` is the + name of your subnet, and `` is the name to give to your + cluster. + + For example: + + ```bash + export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU +export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123 +./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a --vpc-id=vpc-a28d24c7 --subnet-id=subnet-4eb27b39 --spark-version=1.1.0 launch my-spark-cluster + ``` + # Running Applications - Go into the `ec2` directory in the release of Spark you downloaded. diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 5f9e484212635..92adfd2d07b5b 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -162,6 +162,10 @@ def parse_args(): parser.add_option( "--copy-aws-credentials", action="store_true", default=False, help="Add AWS credentials to hadoop configuration to allow Spark to access S3") + parser.add_option( + "--subnet-id", default=None, help="VPC subnet to launch instances in") + parser.add_option( + "--vpc-id", default=None, help="VPC to launch instances in") (opts, args) = parser.parse_args() if len(args) != 2: @@ -186,14 +190,14 @@ def parse_args(): # Get the EC2 security group of the given name, creating it if it doesn't exist -def get_or_make_group(conn, name): +def get_or_make_group(conn, name, vpc_id): groups = conn.get_all_security_groups() group = [g for g in groups if g.name == name] if len(group) > 0: return group[0] else: print "Creating security group " + name - return conn.create_security_group(name, "Spark EC2 group") + return conn.create_security_group(name, "Spark EC2 group", vpc_id) # Check whether a given EC2 instance object is in a state we consider active, @@ -303,12 +307,26 @@ def launch_cluster(conn, opts, cluster_name): user_data_content = user_data_file.read() print "Setting up security groups..." - master_group = get_or_make_group(conn, cluster_name + "-master") - slave_group = get_or_make_group(conn, cluster_name + "-slaves") + master_group = get_or_make_group(conn, cluster_name + "-master", opts.vpc_id) + slave_group = get_or_make_group(conn, cluster_name + "-slaves", opts.vpc_id) authorized_address = opts.authorized_address if master_group.rules == []: # Group was just now created - master_group.authorize(src_group=master_group) - master_group.authorize(src_group=slave_group) + if opts.vpc_id is None: + master_group.authorize(src_group=master_group) + master_group.authorize(src_group=slave_group) + else: + master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=master_group) + master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=master_group) + master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=master_group) + master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=slave_group) + master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=slave_group) + master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=slave_group) master_group.authorize('tcp', 22, 22, authorized_address) master_group.authorize('tcp', 8080, 8081, authorized_address) master_group.authorize('tcp', 18080, 18080, authorized_address) @@ -320,8 +338,22 @@ def launch_cluster(conn, opts, cluster_name): if opts.ganglia: master_group.authorize('tcp', 5080, 5080, authorized_address) if slave_group.rules == []: # Group was just now created - slave_group.authorize(src_group=master_group) - slave_group.authorize(src_group=slave_group) + if opts.vpc_id is None: + slave_group.authorize(src_group=master_group) + slave_group.authorize(src_group=slave_group) + else: + slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=master_group) + slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=master_group) + slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=master_group) + slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, + src_group=slave_group) + slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, + src_group=slave_group) + slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, + src_group=slave_group) slave_group.authorize('tcp', 22, 22, authorized_address) slave_group.authorize('tcp', 8080, 8081, authorized_address) slave_group.authorize('tcp', 50060, 50060, authorized_address) @@ -341,11 +373,12 @@ def launch_cluster(conn, opts, cluster_name): if opts.ami is None: opts.ami = get_spark_ami(opts) - additional_groups = [] + # we use group ids to work around https://github.com/boto/boto/issues/350 + additional_group_ids = [] if opts.additional_security_group: - additional_groups = [sg - for sg in conn.get_all_security_groups() - if opts.additional_security_group in (sg.name, sg.id)] + additional_group_ids = [sg.id + for sg in conn.get_all_security_groups() + if opts.additional_security_group in (sg.name, sg.id)] print "Launching instances..." try: @@ -392,9 +425,10 @@ def launch_cluster(conn, opts, cluster_name): placement=zone, count=num_slaves_this_zone, key_name=opts.key_pair, - security_groups=[slave_group] + additional_groups, + security_group_ids=[slave_group.id] + additional_group_ids, instance_type=opts.instance_type, block_device_map=block_map, + subnet_id=opts.subnet_id, user_data=user_data_content) my_req_ids += [req.id for req in slave_reqs] i += 1 @@ -441,12 +475,13 @@ def launch_cluster(conn, opts, cluster_name): num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) if num_slaves_this_zone > 0: slave_res = image.run(key_name=opts.key_pair, - security_groups=[slave_group] + additional_groups, + security_group_ids=[slave_group.id] + additional_group_ids, instance_type=opts.instance_type, placement=zone, min_count=num_slaves_this_zone, max_count=num_slaves_this_zone, block_device_map=block_map, + subnet_id=opts.subnet_id, user_data=user_data_content) slave_nodes += slave_res.instances print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, @@ -467,12 +502,13 @@ def launch_cluster(conn, opts, cluster_name): if opts.zone == 'all': opts.zone = random.choice(conn.get_all_zones()).name master_res = image.run(key_name=opts.key_pair, - security_groups=[master_group] + additional_groups, + security_group_ids=[master_group.id] + additional_group_ids, instance_type=master_type, placement=opts.zone, min_count=1, max_count=1, block_device_map=block_map, + subnet_id=opts.subnet_id, user_data=user_data_content) master_nodes = master_res.instances print "Launched master in %s, regid = %s" % (zone, master_res.id) From 17688d14299f18a93591818ae5fef69e9dc20eb5 Mon Sep 17 00:00:00 2001 From: Judy Nash Date: Tue, 16 Dec 2014 12:37:26 -0800 Subject: [PATCH 026/227] [SQL] SPARK-4700: Add HTTP protocol spark thrift server Add HTTP protocol support and test cases to spark thrift server, so users can deploy thrift server in both TCP and http mode. Author: Judy Nash Author: judynash Closes #3672 from judynash/master and squashes the following commits: 526315d [Judy Nash] correct spacing on startThriftServer method 31a6520 [Judy Nash] fix code style issues and update sql programming guide format issue 47bf87e [Judy Nash] modify withJdbcStatement method definition to meet less than 100 line length 2e9c11c [Judy Nash] add thrift server in http mode documentation on sql programming guide 1cbd305 [Judy Nash] Merge remote-tracking branch 'upstream/master' 2b1d312 [Judy Nash] updated http thrift server support based on feedback 377532c [judynash] add HTTP protocol spark thrift server --- docs/sql-programming-guide.md | 12 +++ .../hive/thriftserver/HiveThriftServer2.scala | 21 ++++- .../thriftserver/HiveThriftServer2Suite.scala | 77 +++++++++++++++---- 3 files changed, 93 insertions(+), 17 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index be284fbe217a5..ad51b9cf416ed 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -938,6 +938,18 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script that comes with Hive. +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: + + hive.server2.transport.mode - Set this to value: http + hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001 + hive.server2.http.endpoint - HTTP endpoint; default is cliservice + +To test, use beeline to connect to the JDBC/ODBC server in http mode with: + + beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= + + ## Running the Spark SQL CLI The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index bd4e99492b395..c5b73234fa1de 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} import org.apache.spark.Logging @@ -85,10 +86,22 @@ private[hive] class HiveThriftServer2(hiveContext: HiveContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) - val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) + if (isHTTPTransportMode(hiveConf)) { + val thriftCliService = new ThriftHttpCLIService(sparkSqlCliService) + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) + } else { + val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) + } initCompositeService(hiveConf) } + + private def isHTTPTransportMode(hiveConf: HiveConf): Boolean = { + val transportMode: String = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE) + transportMode.equalsIgnoreCase("http") + } + } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 23d12cbff3495..94d5ed4f1d15e 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -70,11 +70,20 @@ class HiveThriftServer2Suite extends FunSuite with Logging { port } - def withJdbcStatement(serverStartTimeout: FiniteDuration = 1.minute)(f: Statement => Unit) { + def withJdbcStatement( + serverStartTimeout: FiniteDuration = 1.minute, + httpMode: Boolean = false)( + f: Statement => Unit) { val port = randomListeningPort - startThriftServer(port, serverStartTimeout) { - val jdbcUri = s"jdbc:hive2://${"localhost"}:$port/" + startThriftServer(port, serverStartTimeout, httpMode) { + val jdbcUri = if (httpMode) { + s"jdbc:hive2://${"localhost"}:$port/" + + "default?hive.server2.transport.mode=http;hive.server2.thrift.http.path=cliservice" + } else { + s"jdbc:hive2://${"localhost"}:$port/" + } + val user = System.getProperty("user.name") val connection = DriverManager.getConnection(jdbcUri, user, "") val statement = connection.createStatement() @@ -113,7 +122,8 @@ class HiveThriftServer2Suite extends FunSuite with Logging { def startThriftServer( port: Int, - serverStartTimeout: FiniteDuration = 1.minute)( + serverStartTimeout: FiniteDuration = 1.minute, + httpMode: Boolean = false)( f: => Unit) { val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) @@ -121,15 +131,28 @@ class HiveThriftServer2Suite extends FunSuite with Logging { val warehousePath = getTempFilePath("warehouse") val metastorePath = getTempFilePath("metastore") val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" + val command = - s"""$startScript - | --master local - | --hiveconf hive.root.logger=INFO,console - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=${"localhost"} - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$port - """.stripMargin.split("\\s+").toSeq + if (httpMode) { + s"""$startScript + | --master local + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost + | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=http + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT}=$port + """.stripMargin.split("\\s+").toSeq + } else { + s"""$startScript + | --master local + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$port + """.stripMargin.split("\\s+").toSeq + } val serverRunning = Promise[Unit]() val buffer = new ArrayBuffer[String]() @@ -140,7 +163,8 @@ class HiveThriftServer2Suite extends FunSuite with Logging { def captureLogOutput(line: String): Unit = { buffer += line - if (line.contains("ThriftBinaryCLIService listening on")) { + if (line.contains("ThriftBinaryCLIService listening on") || + line.contains("Started ThriftHttpCLIService in http")) { serverRunning.success(()) } } @@ -217,6 +241,25 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } } + test("Test JDBC query execution in Http Mode") { + withJdbcStatement(httpMode = true) { statement => + val queries = Seq( + "SET spark.sql.shuffle.partitions=3", + "DROP TABLE IF EXISTS test", + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", + "CACHE TABLE test") + + queries.foreach(statement.execute) + + assertResult(5, "Row count mismatch") { + val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") + resultSet.next() + resultSet.getInt(1) + } + } + } + test("SPARK-3004 regression: result set containing NULL") { withJdbcStatement() { statement => val queries = Seq( @@ -267,6 +310,14 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } } + test("Checks Hive version in Http Mode") { + withJdbcStatement(httpMode = true) { statement => + val resultSet = statement.executeQuery("SET spark.sql.hive.version") + resultSet.next() + assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}") + } + } + test("SPARK-4292 regression: result set iterator issue") { withJdbcStatement() { statement => val queries = Seq( From 1a9e35e57ab80984b81802ffc461d19cc9239edd Mon Sep 17 00:00:00 2001 From: Peter Vandenabeele Date: Tue, 16 Dec 2014 13:57:55 -0800 Subject: [PATCH 027/227] [DOCS][SQL] Add a Note on jsonFile having separate JSON objects per line * This commit hopes to avoid the confusion I faced when trying to submit a regular, valid multi-line JSON file, also see http://apache-spark-user-list.1001560.n3.nabble.com/Loading-JSON-Dataset-fails-with-com-fasterxml-jackson-databind-JsonMappingException-td20041.html Author: Peter Vandenabeele Closes #3517 from petervandenabeele/pv-docs-note-on-jsonFile-format/01 and squashes the following commits: 1f98e52 [Peter Vandenabeele] Revert to people.json and simple Note text 6b6e062 [Peter Vandenabeele] Change the "JSON" connotation to "txt" fca7dfb [Peter Vandenabeele] Add a Note on jsonFile having separate JSON objects per line --- docs/sql-programming-guide.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index ad51b9cf416ed..2aea8a8aedafc 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -625,6 +625,10 @@ This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +Note that the file that is offered as _jsonFile_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. As a consequence, +a regular multi-line JSON file will most often fail. + {% highlight scala %} // sc is an existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) @@ -663,6 +667,10 @@ This conversion can be done using one of two methods in a JavaSQLContext : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +Note that the file that is offered as _jsonFile_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. As a consequence, +a regular multi-line JSON file will most often fail. + {% highlight java %} // sc is an existing JavaSparkContext. JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); @@ -701,6 +709,10 @@ This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +Note that the file that is offered as _jsonFile_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. As a consequence, +a regular multi-line JSON file will most often fail. + {% highlight python %} # sc is an existing SparkContext. from pyspark.sql import SQLContext From dc8280dcca7b54793a3db644f74fd33460960d4a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 16 Dec 2014 14:08:28 -0800 Subject: [PATCH 028/227] [SPARK-4847][SQL]Fix "extraStrategies cannot take effect in SQLContext" issue Author: jerryshao Closes #3698 from jerryshao/SPARK-4847 and squashes the following commits: 4741130 [jerryshao] Make later added extraStrategies effect when calling strategies --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 2 +- 2 files changed, 2 insertions(+), 2 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 31cc4170aa867..ebd4cc920b1ad 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 @@ -327,7 +327,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def numPartitions = self.numShufflePartitions - val strategies: Seq[Strategy] = + def strategies: Seq[Strategy] = extraStrategies ++ ( CommandStrategy(self) :: DataSourceStrategy :: diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 6008e46fefa86..7de440773023f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -338,7 +338,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val hivePlanner = new SparkPlanner with HiveStrategies { val hiveContext = self - override val strategies: Seq[Strategy] = extraStrategies ++ Seq( + override def strategies: Seq[Strategy] = extraStrategies ++ Seq( DataSourceStrategy, CommandStrategy(self), HiveCommandStrategy(self), From 6530243a52b05007e566d458cd8f075e851e91f1 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 16 Dec 2014 14:13:40 -0800 Subject: [PATCH 029/227] [SPARK-4812][SQL] Fix the initialization issue of 'codegenEnabled' The problem is `codegenEnabled` is `val`, but it uses a `val` `sqlContext`, which can be override by subclasses. Here is a simple example to show this issue. ```Scala scala> :paste // Entering paste mode (ctrl-D to finish) abstract class Foo { protected val sqlContext = "Foo" val codegenEnabled: Boolean = { println(sqlContext) // it will call subclass's `sqlContext` which has not yet been initialized. if (sqlContext != null) { true } else { false } } } class Bar extends Foo { override val sqlContext = "Bar" } println(new Bar().codegenEnabled) // Exiting paste mode, now interpreting. null false defined class Foo defined class Bar ``` We should make `sqlContext` `final` to prevent subclasses from overriding it incorrectly. Author: zsxwing Closes #3660 from zsxwing/SPARK-4812 and squashes the following commits: 1cbb623 [zsxwing] Make `sqlContext` final to prevent subclasses from overriding it incorrectly --- .../apache/spark/sql/columnar/InMemoryColumnarTableScan.scala | 2 -- .../main/scala/org/apache/spark/sql/execution/SparkPlan.scala | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) 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 0cebe823b2707..1e432485c4c29 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 @@ -175,8 +175,6 @@ private[sql] case class InMemoryColumnarTableScan( relation: InMemoryRelation) extends LeafNode { - @transient override val sqlContext = relation.child.sqlContext - override def output: Seq[Attribute] = attributes private def statsFor(a: Attribute) = relation.partitionStatistics.forAttribute(a) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 4cd8e7d657250..017c78d2c66d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -44,7 +44,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ * populated by the query planning infrastructure. */ @transient - protected[spark] val sqlContext = SparkPlan.currentContext.get() + protected[spark] final val sqlContext = SparkPlan.currentContext.get() protected def sparkContext = sqlContext.sparkContext From b0dfdbdd181d1eea83d70dd0959297903952dd74 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 16 Dec 2014 14:37:04 -0800 Subject: [PATCH 030/227] SPARK-4767: Add support for launching in a specified placement group to spark_ec2 Placement groups are cool and all the cool kids are using them. Lets add support for them to spark_ec2.py because I'm lazy Author: Holden Karau Closes #3623 from holdenk/SPARK-4767-add-support-for-launching-in-a-specified-placement-group-to-spark-ec2-scripts and squashes the following commits: 111a5fd [Holden Karau] merge in master 70ace25 [Holden Karau] Placement groups are cool and all the cool kids are using them. Lets add support for them to spark_ec2.py because I'm lazy --- ec2/spark_ec2.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 92adfd2d07b5b..4e8f5c1f44041 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -120,6 +120,10 @@ def parse_args(): "Only possible on EBS-backed AMIs. " + "EBS volumes are only attached if --ebs-vol-size > 0." + "Only support up to 8 EBS volumes.") + parser.add_option("--placement-group", type="string", default=None, + help="Which placement group to try and launch " + + "instances into. Assumes placement group is already " + + "created.") parser.add_option( "--swap", metavar="SWAP", type="int", default=1024, help="Swap space to set up per node, in MB (default: %default)") @@ -429,6 +433,7 @@ def launch_cluster(conn, opts, cluster_name): instance_type=opts.instance_type, block_device_map=block_map, subnet_id=opts.subnet_id, + placement_group=opts.placement_group, user_data=user_data_content) my_req_ids += [req.id for req in slave_reqs] i += 1 @@ -482,6 +487,7 @@ def launch_cluster(conn, opts, cluster_name): max_count=num_slaves_this_zone, block_device_map=block_map, subnet_id=opts.subnet_id, + placement_group=opts.placement_group, user_data=user_data_content) slave_nodes += slave_res.instances print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, @@ -509,7 +515,9 @@ def launch_cluster(conn, opts, cluster_name): max_count=1, block_device_map=block_map, subnet_id=opts.subnet_id, + placement_group=opts.placement_group, user_data=user_data_content) + master_nodes = master_res.instances print "Launched master in %s, regid = %s" % (zone, master_res.id) From ea1315e3e26507c8e1cab877cec5fe69c2899ae8 Mon Sep 17 00:00:00 2001 From: wangxiaojing Date: Tue, 16 Dec 2014 14:45:56 -0800 Subject: [PATCH 031/227] [SPARK-4527][SQl]Add BroadcastNestedLoopJoin operator selection testsuite In `JoinSuite` add BroadcastNestedLoopJoin operator selection testsuite Author: wangxiaojing Closes #3395 from wangxiaojing/SPARK-4527 and squashes the following commits: ea0e495 [wangxiaojing] change style 53c3952 [wangxiaojing] Add BroadcastNestedLoopJoin operator selection testsuite --- .../src/test/scala/org/apache/spark/sql/JoinSuite.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 8b4cf5bac0187..0378fd7e367f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -80,8 +80,13 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { classOf[HashOuterJoin]), ("SELECT * FROM testData right join testData2 ON key = a and key = 2", classOf[HashOuterJoin]), - ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]) - // TODO add BroadcastNestedLoopJoin + ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData left JOIN testData2 ON (key * a != key + a)", + classOf[BroadcastNestedLoopJoin]), + ("SELECT * FROM testData right JOIN testData2 ON (key * a != key + a)", + classOf[BroadcastNestedLoopJoin]), + ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", + classOf[BroadcastNestedLoopJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } } From 30f6b85c816d1ef611a7be071af0053d64b6fe9e Mon Sep 17 00:00:00 2001 From: tianyi Date: Tue, 16 Dec 2014 15:22:29 -0800 Subject: [PATCH 032/227] [SPARK-4483][SQL]Optimization about reduce memory costs during the HashOuterJoin MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In `HashOuterJoin.scala`, spark read data from both side of join operation before zip them together. It is a waste for memory. We are trying to read data from only one side, put them into a hashmap, and then generate the `JoinedRow` with data from other side one by one. Currently, we could only do this optimization for `left outer join` and `right outer join`. For `full outer join`, we will do something in another issue. for table test_csv contains 1 million records table dim_csv contains 10 thousand records SQL: `select * from test_csv a left outer join dim_csv b on a.key = b.key` the result is: master: ``` CSV: 12671 ms CSV: 9021 ms CSV: 9200 ms Current Mem Usage:787788984 ``` after patch: ``` CSV: 10382 ms CSV: 7543 ms CSV: 7469 ms Current Mem Usage:208145728 ``` Author: tianyi Author: tianyi Closes #3375 from tianyi/SPARK-4483 and squashes the following commits: 72a8aec [tianyi] avoid having mutable state stored inside of the task 99c5c97 [tianyi] performance optimization d2f94d7 [tianyi] fix bug: missing output when the join-key is null. 2be45d1 [tianyi] fix spell bug 1f2c6f1 [tianyi] remove commented codes a676de6 [tianyi] optimize some codes 9e7d5b5 [tianyi] remove commented old codes 838707d [tianyi] Optimization about reduce memory costs during the HashOuterJoin --- .../sql/execution/joins/HashOuterJoin.scala | 128 +++++++++--------- 1 file changed, 64 insertions(+), 64 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala index b73041d306b36..59ef904272545 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala @@ -68,66 +68,56 @@ case class HashOuterJoin( @transient private[this] lazy val DUMMY_LIST = Seq[Row](null) @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row] + @transient private[this] lazy val leftNullRow = new GenericRow(left.output.length) + @transient private[this] lazy val rightNullRow = new GenericRow(right.output.length) + @transient private[this] lazy val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala // iterator for performance purpose. private[this] def leftOuterIterator( - key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { - val joinedRow = new JoinedRow() - val rightNullRow = new GenericRow(right.output.length) - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - - leftIter.iterator.flatMap { l => - joinedRow.withLeft(l) - var matched = false - (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => - matched = true - joinedRow.copy + key: Row, joinedRow: JoinedRow, rightIter: Iterable[Row]): Iterator[Row] = { + val ret: Iterable[Row] = ( + if (!key.anyNull) { + val temp = rightIter.collect { + case r if (boundCondition(joinedRow.withRight(r))) => joinedRow.copy + } + if (temp.size == 0) { + joinedRow.withRight(rightNullRow).copy :: Nil + } else { + temp + } } else { - Nil - }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { - // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the - // records in right side. - // If we didn't get any proper row, then append a single row with empty right - joinedRow.withRight(rightNullRow).copy - }) - } + joinedRow.withRight(rightNullRow).copy :: Nil + } + ) + ret.iterator } private[this] def rightOuterIterator( - key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { - val joinedRow = new JoinedRow() - val leftNullRow = new GenericRow(left.output.length) - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - - rightIter.iterator.flatMap { r => - joinedRow.withRight(r) - var matched = false - (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => - matched = true - joinedRow.copy + key: Row, leftIter: Iterable[Row], joinedRow: JoinedRow): Iterator[Row] = { + + val ret: Iterable[Row] = ( + if (!key.anyNull) { + val temp = leftIter.collect { + case l if (boundCondition(joinedRow.withLeft(l))) => joinedRow.copy + } + if (temp.size == 0) { + joinedRow.withLeft(leftNullRow).copy :: Nil + } else { + temp + } } else { - Nil - }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { - // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the - // records in left side. - // If we didn't get any proper row, then append a single row with empty left. - joinedRow.withLeft(leftNullRow).copy - }) - } + joinedRow.withLeft(leftNullRow).copy :: Nil + } + ) + ret.iterator } private[this] def fullOuterIterator( - key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { - val joinedRow = new JoinedRow() - val leftNullRow = new GenericRow(left.output.length) - val rightNullRow = new GenericRow(right.output.length) - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row], + joinedRow: JoinedRow): Iterator[Row] = { if (!key.anyNull) { // Store the positions of records in right, if one of its associated row satisfy @@ -193,27 +183,37 @@ case class HashOuterJoin( } override def execute() = { + val joinedRow = new JoinedRow() left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => // TODO this probably can be replaced by external sort (sort merged join?) - // Build HashMap for current partition in left relation - val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) - // Build HashMap for current partition in right relation - val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + joinType match { - case LeftOuter => leftHashTable.keysIterator.flatMap { key => - leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), - rightHashTable.getOrElse(key, EMPTY_LIST)) + case LeftOuter => { + val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + val keyGenerator = newProjection(leftKeys, left.output) + leftIter.flatMap( currentRow => { + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + leftOuterIterator(rowKey, joinedRow, rightHashTable.getOrElse(rowKey, EMPTY_LIST)) + }) } - case RightOuter => rightHashTable.keysIterator.flatMap { key => - rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), - rightHashTable.getOrElse(key, EMPTY_LIST)) + case RightOuter => { + val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) + val keyGenerator = newProjection(rightKeys, right.output) + rightIter.flatMap ( currentRow => { + val rowKey = keyGenerator(currentRow) + joinedRow.withRight(currentRow) + rightOuterIterator(rowKey, leftHashTable.getOrElse(rowKey, EMPTY_LIST), joinedRow) + }) } - case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => - fullOuterIterator(key, - leftHashTable.getOrElse(key, EMPTY_LIST), - rightHashTable.getOrElse(key, EMPTY_LIST)) + case FullOuter => { + val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) + val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => + fullOuterIterator(key, + leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST), joinedRow) + } } case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") } From a66c23e134a0b1ad9540626fb7436d70d577d929 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 16 Dec 2014 15:31:19 -0800 Subject: [PATCH 033/227] [SPARK-4827][SQL] Fix resolution of deeply nested Project(attr, Project(Star,...)). Since `AttributeReference` resolution and `*` expansion are currently in separate rules, each pair requires a full iteration instead of being able to resolve in a single pass. Since its pretty easy to construct queries that have many of these in a row, I combine them into a single rule in this PR. Author: Michael Armbrust Closes #3674 from marmbrus/projectStars and squashes the following commits: d83d6a1 [Michael Armbrust] Fix resolution of deeply nested Project(attr, Project(Star,...)). --- .../sql/catalyst/analysis/Analyzer.scala | 75 +++++++++---------- .../sql/catalyst/analysis/AnalysisSuite.scala | 12 ++- 2 files changed, 45 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 04639219a3650..ea9bb3978691a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -58,7 +58,6 @@ class Analyzer(catalog: Catalog, ResolveSortReferences :: NewRelationInstances :: ImplicitGenerate :: - StarExpansion :: ResolveFunctions :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: @@ -153,7 +152,34 @@ class Analyzer(catalog: Catalog, */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case q: LogicalPlan if q.childrenResolved => + case p: LogicalPlan if !p.childrenResolved => p + + // If the projection list contains Stars, expand it. + case p@Project(projectList, child) if containsStar(projectList) => + Project( + projectList.flatMap { + case s: Star => s.expand(child.output, resolver) + case o => o :: Nil + }, + child) + case t: ScriptTransformation if containsStar(t.input) => + t.copy( + input = t.input.flatMap { + case s: Star => s.expand(t.child.output, resolver) + case o => o :: Nil + } + ) + + // If the aggregate function argument contains Stars, expand it. + case a: Aggregate if containsStar(a.aggregateExpressions) => + a.copy( + aggregateExpressions = a.aggregateExpressions.flatMap { + case s: Star => s.expand(a.child.output, resolver) + case o => o :: Nil + } + ) + + case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { case u @ UnresolvedAttribute(name) => @@ -163,6 +189,12 @@ class Analyzer(catalog: Catalog, result } } + + /** + * Returns true if `exprs` contains a [[Star]]. + */ + protected def containsStar(exprs: Seq[Expression]): Boolean = + exprs.collect { case _: Star => true}.nonEmpty } /** @@ -277,45 +309,6 @@ class Analyzer(catalog: Catalog, Generate(g, join = false, outer = false, None, child) } } - - /** - * Expands any references to [[Star]] (*) in project operators. - */ - object StarExpansion extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - // Wait until children are resolved - case p: LogicalPlan if !p.childrenResolved => p - // If the projection list contains Stars, expand it. - case p @ Project(projectList, child) if containsStar(projectList) => - Project( - projectList.flatMap { - case s: Star => s.expand(child.output, resolver) - case o => o :: Nil - }, - child) - case t: ScriptTransformation if containsStar(t.input) => - t.copy( - input = t.input.flatMap { - case s: Star => s.expand(t.child.output, resolver) - case o => o :: Nil - } - ) - // If the aggregate function argument contains Stars, expand it. - case a: Aggregate if containsStar(a.aggregateExpressions) => - a.copy( - aggregateExpressions = a.aggregateExpressions.flatMap { - case s: Star => s.expand(a.child.output, resolver) - case o => o :: Nil - } - ) - } - - /** - * Returns true if `exprs` contains a [[Star]]. - */ - protected def containsStar(exprs: Seq[Expression]): Boolean = - exprs.collect { case _: Star => true }.nonEmpty - } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 33a3cba3d4c0e..82f2101d8ce17 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ + class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) val caseInsensitiveCatalog = new SimpleCatalog(false) @@ -46,6 +48,14 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { caseInsensitiveCatalog.registerTable(None, "TaBlE", testRelation) } + test("union project *") { + val plan = (1 to 100) + .map(_ => testRelation) + .fold[LogicalPlan](testRelation)((a,b) => a.select(Star(None)).select('a).unionAll(b.select(Star(None)))) + + assert(caseInsensitiveAnalyze(plan).resolved) + } + test("analyze project") { assert( caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === From fa66ef6c97e87c9255b67b03836a4ba50598ebae Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Tue, 16 Dec 2014 15:34:59 -0800 Subject: [PATCH 034/227] [SPARK-4269][SQL] make wait time configurable in BroadcastHashJoin In BroadcastHashJoin, currently it is using a hard coded value (5 minutes) to wait for the execution and broadcast of the small table. In my opinion, it should be a configurable value since broadcast may exceed 5 minutes in some case, like in a busy/congested network environment. Author: Jacky Li Closes #3133 from jackylk/timeout-config and squashes the following commits: 733ac08 [Jacky Li] add spark.sql.broadcastTimeout in SQLConf.scala 557acd4 [Jacky Li] switch to sqlContext.getConf 81a5e20 [Jacky Li] make wait time configurable in BroadcastHashJoin --- .../src/main/scala/org/apache/spark/sql/SQLConf.scala | 7 +++++++ .../spark/sql/execution/joins/BroadcastHashJoin.scala | 11 ++++++++++- 2 files changed, 17 insertions(+), 1 deletion(-) 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 9697beb132fbb..f5abf71d6c7b9 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 @@ -38,6 +38,7 @@ private[spark] object SQLConf { val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.parquet.filterPushdown" val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord" + val BROADCAST_TIMEOUT = "spark.sql.broadcastTimeout" // Options that control which operators can be chosen by the query planner. These should be // considered hints and may be ignored by future versions of Spark SQL. @@ -148,6 +149,12 @@ private[sql] trait SQLConf { private[spark] def columnNameOfCorruptRecord: String = getConf(COLUMN_NAME_OF_CORRUPT_RECORD, "_corrupt_record") + /** + * Timeout in seconds for the broadcast wait time in hash join + */ + private[spark] def broadcastTimeout: Int = + getConf(BROADCAST_TIMEOUT, (5 * 60).toString).toInt + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 5cf2a785adc7d..fbe1d06ed2e82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -42,6 +42,15 @@ case class BroadcastHashJoin( right: SparkPlan) extends BinaryNode with HashJoin { + val timeout = { + val timeoutValue = sqlContext.broadcastTimeout + if (timeoutValue < 0) { + Duration.Inf + } else { + timeoutValue.seconds + } + } + override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning override def requiredChildDistribution = @@ -56,7 +65,7 @@ case class BroadcastHashJoin( } override def execute() = { - val broadcastRelation = Await.result(broadcastFuture, 5.minute) + val broadcastRelation = Await.result(broadcastFuture, timeout) streamedPlan.execute().mapPartitions { streamedIter => hashJoin(streamedIter, broadcastRelation.value) From 6f80b749e0f23f2f5c83993b87f73c8943b99db8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Dec 2014 17:55:27 -0800 Subject: [PATCH 035/227] [Release] Major improvements to generate contributors script This commit introduces several major improvements to the script that generates the contributors list for release notes, notably: (1) Use release tags instead of a range of commits. Across branches, commits are not actually strictly two-dimensional, and so it is not sufficient to specify a start hash and an end hash. Otherwise, we end up counting commits that were already merged in an older branch. (2) Match PR numbers in addition to commit hashes. This is related to the first point in that if a PR is already merged in an older minor release tag, it should be filtered out here. This requires us to do some intelligent regex parsing on the commit description in addition to just relying on the GitHub API. (3) Relax author validity check. The old code fails on a name that has many middle names, for instance. The test was just too strict. (4) Use GitHub authentication. This allows us to make far more requests through the GitHub API than before (5000 as opposed to 60 per hour). (5) Translate from Github username, not commit author name. This is important because the commit author name is not always configured correctly by the user. For instance, the username "falaki" used to resolve to just "Hossein", which was treated as a github username and translated to something else that is completely arbitrary. (6) Add an option to use the untranslated name. If there is not a satisfactory candidate to replace the untranslated name with, at least allow the user to not translate it. --- dev/create-release/generate-contributors.py | 156 +++++++++++-------- dev/create-release/releaseutils.py | 94 +++++++++-- dev/create-release/translate-contributors.py | 45 ++++-- 3 files changed, 206 insertions(+), 89 deletions(-) diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index a3b78a3eac6d0..e8f81ccbce740 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -26,75 +26,103 @@ # You must set the following before use! JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") -START_COMMIT = os.environ.get("START_COMMIT", "37b100") -END_COMMIT = os.environ.get("END_COMMIT", "3693ae") - -# If commit range is not specified, prompt the user to provide it -if not START_COMMIT or not END_COMMIT: - print "A commit range is required to proceed." - if not START_COMMIT: - START_COMMIT = raw_input("Please specify starting commit hash (inclusive): ") - if not END_COMMIT: - END_COMMIT = raw_input("Please specify ending commit hash (non-inclusive): ") - -# Verify provided arguments -start_commit_line = get_one_line(START_COMMIT) -end_commit_line = get_one_line(END_COMMIT) -num_commits = num_commits_in_range(START_COMMIT, END_COMMIT) -if not start_commit_line: sys.exit("Start commit %s not found!" % START_COMMIT) -if not end_commit_line: sys.exit("End commit %s not found!" % END_COMMIT) -if num_commits == 0: - sys.exit("There are no commits in the provided range [%s, %s)" % (START_COMMIT, END_COMMIT)) +RELEASE_TAG = os.environ.get("START_COMMIT", "v1.2.0-rc2") +PREVIOUS_RELEASE_TAG = os.environ.get("END_COMMIT", "v1.1.0") + +# If the release tags are not provided, prompt the user to provide them +while not tag_exists(RELEASE_TAG): + RELEASE_TAG = raw_input("Please provide a valid release tag: ") +while not tag_exists(PREVIOUS_RELEASE_TAG): + print "Please specify the previous release tag." + PREVIOUS_RELEASE_TAG = raw_input(\ + "For instance, if you are releasing v1.2.0, you should specify v1.1.0: ") + +# Gather commits found in the new tag but not in the old tag. +# This filters commits based on both the git hash and the PR number. +# If either is present in the old tag, then we ignore the commit. +print "Gathering new commits between tags %s and %s" % (PREVIOUS_RELEASE_TAG, RELEASE_TAG) +release_commits = get_commits(RELEASE_TAG) +previous_release_commits = get_commits(PREVIOUS_RELEASE_TAG) +previous_release_hashes = set() +previous_release_prs = set() +for old_commit in previous_release_commits: + previous_release_hashes.add(old_commit.get_hash()) + if old_commit.get_pr_number(): + previous_release_prs.add(old_commit.get_pr_number()) +new_commits = [] +for this_commit in release_commits: + this_hash = this_commit.get_hash() + this_pr_number = this_commit.get_pr_number() + if this_hash in previous_release_hashes: + continue + if this_pr_number and this_pr_number in previous_release_prs: + continue + new_commits.append(this_commit) +if not new_commits: + sys.exit("There are no new commits between %s and %s!" % (PREVIOUS_RELEASE_TAG, RELEASE_TAG)) + +# Prompt the user for confirmation that the commit range is correct print "\n==================================================================================" print "JIRA server: %s" % JIRA_API_BASE -print "Start commit (inclusive): %s" % start_commit_line -print "End commit (non-inclusive): %s" % end_commit_line -print "Number of commits in this range: %s" % num_commits +print "Release tag: %s" % RELEASE_TAG +print "Previous release tag: %s" % PREVIOUS_RELEASE_TAG +print "Number of commits in this range: %s" % len(new_commits) print -response = raw_input("Is this correct? [Y/n] ") -if response.lower() != "y" and response: - sys.exit("Ok, exiting") +def print_indented(_list): + for x in _list: print " %s" % x +if yesOrNoPrompt("Show all commits?"): + print_indented(new_commits) print "==================================================================================\n" - -# Find all commits within this range -print "Gathering commits within range [%s..%s)" % (START_COMMIT, END_COMMIT) -commits = get_one_line_commits(START_COMMIT, END_COMMIT) -if not commits: sys.exit("Error: No commits found within this range!") -commits = commits.split("\n") +if not yesOrNoPrompt("Does this look correct?"): + sys.exit("Ok, exiting") # Filter out special commits releases = [] +maintenance = [] reverts = [] nojiras = [] filtered_commits = [] -def is_release(commit): - return re.findall("\[release\]", commit.lower()) or\ - "maven-release-plugin" in commit or "CHANGES.txt" in commit -def has_no_jira(commit): - return not re.findall("SPARK-[0-9]+", commit.upper()) -def is_revert(commit): - return "revert" in commit.lower() -def is_docs(commit): - return re.findall("docs*", commit.lower()) or "programming guide" in commit.lower() -for c in commits: - if not c: continue - elif is_release(c): releases.append(c) - elif is_revert(c): reverts.append(c) - elif is_docs(c): filtered_commits.append(c) # docs may not have JIRA numbers - elif has_no_jira(c): nojiras.append(c) +def is_release(commit_title): + return re.findall("\[release\]", commit_title.lower()) or\ + "preparing spark release" in commit_title.lower() or\ + "preparing development version" in commit_title.lower() or\ + "CHANGES.txt" in commit_title +def is_maintenance(commit_title): + return "maintenance" in commit_title.lower() or\ + "manually close" in commit_title.lower() +def has_no_jira(commit_title): + return not re.findall("SPARK-[0-9]+", commit_title.upper()) +def is_revert(commit_title): + return "revert" in commit_title.lower() +def is_docs(commit_title): + return re.findall("docs*", commit_title.lower()) or\ + "programming guide" in commit_title.lower() +for c in new_commits: + t = c.get_title() + if not t: continue + elif is_release(t): releases.append(c) + elif is_maintenance(t): maintenance.append(c) + elif is_revert(t): reverts.append(c) + elif is_docs(t): filtered_commits.append(c) # docs may not have JIRA numbers + elif has_no_jira(t): nojiras.append(c) else: filtered_commits.append(c) # Warn against ignored commits -def print_indented(_list): - for x in _list: print " %s" % x -if releases or reverts or nojiras: +if releases or maintenance or reverts or nojiras: print "\n==================================================================================" - if releases: print "Releases (%d)" % len(releases); print_indented(releases) - if reverts: print "Reverts (%d)" % len(reverts); print_indented(reverts) - if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) + if releases: print "Found %d release commits" % len(releases) + if maintenance: print "Found %d maintenance commits" % len(maintenance) + if reverts: print "Found %d revert commits" % len(reverts) + if nojiras: print "Found %d commits with no JIRA" % len(nojiras) + print "* Warning: these commits will be ignored.\n" + if yesOrNoPrompt("Show ignored commits?"): + if releases: print "Release (%d)" % len(releases); print_indented(releases) + if maintenance: print "Maintenance (%d)" % len(maintenance); print_indented(maintenance) + if reverts: print "Revert (%d)" % len(reverts); print_indented(reverts) + if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) print "==================== Warning: the above commits will be ignored ==================\n" -response = raw_input("%d commits left to process. Ok to proceed? [Y/n] " % len(filtered_commits)) -if response.lower() != "y" and response: +prompt_msg = "%d commits left to process after filtering. Ok to proceed?" % len(filtered_commits) +if not yesOrNoPrompt(prompt_msg): sys.exit("Ok, exiting.") # Keep track of warnings to tell the user at the end @@ -123,10 +151,11 @@ def print_indented(_list): jira_client = JIRA(options = jira_options) print "\n=========================== Compiling contributor list ===========================" for commit in filtered_commits: - commit_hash = re.findall("^[a-z0-9]+", commit)[0] - issues = re.findall("SPARK-[0-9]+", commit.upper()) - author = get_author(commit_hash) - author = unidecode.unidecode(unicode(author, "UTF-8")).strip() # guard against special characters + _hash = commit.get_hash() + title = commit.get_title() + issues = re.findall("SPARK-[0-9]+", title.upper()) + author = commit.get_author() + date = get_date(_hash) # If the author name is invalid, keep track of it along # with all associated issues so we can translate it later if is_valid_author(author): @@ -136,9 +165,8 @@ def print_indented(_list): invalid_authors[author] = set() for issue in issues: invalid_authors[author].add(issue) - date = get_date(commit_hash) - # Parse components from the commit message, if any - commit_components = find_components(commit, commit_hash) + # Parse components from the commit title, if any + commit_components = find_components(title, _hash) # Populate or merge an issue into author_info[author] def populate(issue_type, components): components = components or [CORE_COMPONENT] # assume core if no components provided @@ -153,14 +181,14 @@ def populate(issue_type, components): jira_issue = jira_client.issue(issue) jira_type = jira_issue.fields.issuetype.name jira_type = translate_issue_type(jira_type, issue, warnings) - jira_components = [translate_component(c.name, commit_hash, warnings)\ + jira_components = [translate_component(c.name, _hash, warnings)\ for c in jira_issue.fields.components] all_components = set(jira_components + commit_components) populate(jira_type, all_components) # For docs without an associated JIRA, manually add it ourselves - if is_docs(commit) and not issues: + if is_docs(title) and not issues: populate("documentation", commit_components) - print " Processed commit %s authored by %s on %s" % (commit_hash, author, date) + print " Processed commit %s authored by %s on %s" % (_hash, author, date) print "==================================================================================\n" # Write to contributors file ordered by author names diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 76a10c32886d4..18e16bcb90514 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -19,6 +19,7 @@ # This file contains helper methods used in creating a release. import re +import sys from subprocess import Popen, PIPE try: @@ -47,20 +48,85 @@ # Contributors list file name contributors_file_name = "contributors.txt" +# Prompt the user to answer yes or no until they do so +def yesOrNoPrompt(msg): + response = raw_input("%s [y/n]: " % msg) + while response != "y" and response != "n": + return yesOrNoPrompt(msg) + return response == "y" + # Utility functions run git commands (written with Git 1.8.5) def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] -def get_author(commit_hash): - return run_cmd(["git", "show", "--quiet", "--pretty=format:%an", commit_hash]) +def run_cmd_error(cmd): return Popen(cmd, stderr=PIPE).communicate()[1] def get_date(commit_hash): return run_cmd(["git", "show", "--quiet", "--pretty=format:%cd", commit_hash]) -def get_one_line(commit_hash): - return run_cmd(["git", "show", "--quiet", "--pretty=format:\"%h %cd %s\"", commit_hash]) -def get_one_line_commits(start_hash, end_hash): - return run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) -def num_commits_in_range(start_hash, end_hash): - output = run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) - lines = [line for line in output.split("\n") if line] # filter out empty lines - return len(lines) +def tag_exists(tag): + stderr = run_cmd_error(["git", "checkout", tag]) + return "error" not in stderr + +# A type-safe representation of a commit +class Commit: + def __init__(self, _hash, author, title, pr_number = None): + self._hash = _hash + self.author = author + self.title = title + self.pr_number = pr_number + def get_hash(self): return self._hash + def get_author(self): return self.author + def get_title(self): return self.title + def get_pr_number(self): return self.pr_number + def __str__(self): + closes_pr = "(Closes #%s)" % self.pr_number if self.pr_number else "" + return "%s %s %s %s" % (self._hash, self.author, self.title, closes_pr) + +# Return all commits that belong to the specified tag. +# +# Under the hood, this runs a `git log` on that tag and parses the fields +# from the command output to construct a list of Commit objects. Note that +# because certain fields reside in the commit description and cannot be parsed +# through the Github API itself, we need to do some intelligent regex parsing +# to extract those fields. +# +# This is written using Git 1.8.5. +def get_commits(tag): + commit_start_marker = "|=== COMMIT START MARKER ===|" + commit_end_marker = "|=== COMMIT END MARKER ===|" + field_end_marker = "|=== COMMIT FIELD END MARKER ===|" + log_format =\ + commit_start_marker + "%h" +\ + field_end_marker + "%an" +\ + field_end_marker + "%s" +\ + commit_end_marker + "%b" + output = run_cmd(["git", "log", "--quiet", "--pretty=format:" + log_format, tag]) + commits = [] + raw_commits = [c for c in output.split(commit_start_marker) if c] + for commit in raw_commits: + if commit.count(commit_end_marker) != 1: + print "Commit end marker not found in commit: " + for line in commit.split("\n"): print line + sys.exit(1) + # Separate commit digest from the body + # From the digest we extract the hash, author and the title + # From the body, we extract the PR number and the github username + [commit_digest, commit_body] = commit.split(commit_end_marker) + if commit_digest.count(field_end_marker) != 2: + sys.exit("Unexpected format in commit: %s" % commit_digest) + [_hash, author, title] = commit_digest.split(field_end_marker) + # The PR number and github username is in the commit message + # itself and cannot be accessed through any Github API + pr_number = None + match = re.search("Closes #([0-9]+) from ([^/\\s]+)/", commit_body) + if match: + [pr_number, github_username] = match.groups() + # If the author name is not valid, use the github + # username so we can translate it properly later + if not is_valid_author(author): + author = github_username + # Guard against special characters + author = unidecode.unidecode(unicode(author, "UTF-8")).strip() + commit = Commit(_hash, author, title, pr_number) + commits.append(commit) + return commits # Maintain a mapping for translating issue types to contributions in the release notes # This serves an additional function of warning the user against unknown issue types @@ -70,10 +136,13 @@ def num_commits_in_range(start_hash, end_hash): known_issue_types = { "bug": "bug fixes", "build": "build fixes", + "dependency upgrade": "build fixes", "improvement": "improvements", "new feature": "new features", "documentation": "documentation", - "test": "test" + "test": "test", + "task": "improvement", + "sub-task": "improvement" } # Maintain a mapping for translating component names when creating the release notes @@ -176,8 +245,7 @@ def get_jira_name(author, jira_client): # Return whether the given name is in the form def is_valid_author(author): if not author: return False - author_words = len(author.split(" ")) - return author_words == 2 or author_words == 3 + return " " in author and not re.findall("[0-9]", author) # Capitalize the first letter of each word in the given author name def capitalize_author(author): diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py index ef4625b003cb6..462c21142f75f 100755 --- a/dev/create-release/translate-contributors.py +++ b/dev/create-release/translate-contributors.py @@ -37,8 +37,11 @@ JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) +GITHUB_API_TOKEN = os.environ.get("GITHUB_API_TOKEN", None) if not JIRA_USERNAME or not JIRA_PASSWORD: sys.exit("Both JIRA_USERNAME and JIRA_PASSWORD must be set") +if not GITHUB_API_TOKEN: + sys.exit("GITHUB_API_TOKEN must be set") # Write new contributors list to .new if not os.path.isfile(contributors_file_name): @@ -62,7 +65,7 @@ # Setup Github and JIRA clients jira_options = { "server": JIRA_API_BASE } jira_client = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) -github_client = Github() +github_client = Github(GITHUB_API_TOKEN) # Generate candidates for the given author. This should only be called if the given author # name does not represent a full name as this operation is somewhat expensive. Under the @@ -94,7 +97,14 @@ def generate_candidates(author, issues): # Then do the same for the assignee of each of the associated JIRAs # Note that a given issue may not have an assignee, or the assignee may not have a full name for issue in issues: - jira_issue = jira_client.issue(issue) + try: + jira_issue = jira_client.issue(issue) + except JIRAError as e: + # Do not exit just because an issue is not found! + if e.status_code == 404: + warnings.append("Issue %s not found!" % issue) + continue + raise e jira_assignee = jira_issue.fields.assignee if jira_assignee: user_name = jira_assignee.name @@ -123,9 +133,10 @@ def generate_candidates(author, issues): # In non-interactive mode, this script picks the first valid author name from the candidates # If no such name exists, the original name is used (without the JIRA numbers). print "\n========================== Translating contributor list ==========================" -for line in contributors_file: +lines = contributors_file.readlines() +for i, line in enumerate(lines): author = line.split(" - ")[0] - print "Processing author %s" % author + print "Processing author %s (%d/%d)" % (author, i + 1, len(lines)) if not author: print " ERROR: Expected the following format - " print " ERROR: Actual = %s" % line @@ -135,30 +146,39 @@ def generate_candidates(author, issues): candidates = generate_candidates(new_author, issues) # Print out potential replacement candidates along with the sources, e.g. # [X] No full name found for Github user andrewor14 + # [X] No assignee found for SPARK-1763 # [0] Andrew Or - Full name of JIRA user andrewor14 # [1] Andrew Orso - Full name of SPARK-1444 assignee andrewor14 # [2] Andrew Ordall - Full name of SPARK-1663 assignee andrewor14 - # [X] No assignee found for SPARK-1763 - # [3] Custom + # [3] andrewor14 - Raw Github username + # [4] Custom candidate_names = [] + bad_prompts = [] # Prompts that can't actually be selected; print these first. + good_prompts = [] # Prompts that contain valid choices for candidate, source in candidates: if candidate == NOT_FOUND: - print " [X] %s" % source + bad_prompts.append(" [X] %s" % source) else: index = len(candidate_names) candidate_names.append(candidate) - print " [%d] %s - %s" % (index, candidate, source) - custom_index = len(candidate_names) + good_prompts.append(" [%d] %s - %s" % (index, candidate, source)) + raw_index = len(candidate_names) + custom_index = len(candidate_names) + 1 + for p in bad_prompts: print p + if bad_prompts: print " ---" + for p in good_prompts: print p # In interactive mode, additionally provide "custom" option and await user response if INTERACTIVE_MODE: + print " [%d] %s - Raw Github username" % (raw_index, new_author) print " [%d] Custom" % custom_index response = raw_input(" Your choice: ") - while not response.isdigit() or int(response) > custom_index: - response = raw_input(" Please enter an integer between 0 and %d: " % custom_index) + last_index = custom_index + while not response.isdigit() or int(response) > last_index: + response = raw_input(" Please enter an integer between 0 and %d: " % last_index) response = int(response) if response == custom_index: new_author = raw_input(" Please type a custom name for this author: ") - else: + elif response != raw_index: new_author = candidate_names[response] # In non-interactive mode, just pick the first candidate else: @@ -175,6 +195,7 @@ def generate_candidates(author, issues): print " * Replacing %s with %s" % (author, new_author) line = line.replace(author, new_author) new_contributors_file.write(line) + new_contributors_file.flush() print "==================================================================================\n" contributors_file.close() new_contributors_file.close() From b85044ecfa825ff68c8e57eeffa4d9f214335e66 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Dec 2014 19:28:43 -0800 Subject: [PATCH 036/227] [Release] Cache known author translations locally This bypasses unnecessary calls to the Github and JIRA API. Additionally, having a local cache allows us to remember names that we had to manually discover ourselves. --- dev/create-release/generate-contributors.py | 18 +++--- dev/create-release/known_translations | 59 ++++++++++++++++++ dev/create-release/releaseutils.py | 4 +- dev/create-release/translate-contributors.py | 64 ++++++++++++++------ 4 files changed, 116 insertions(+), 29 deletions(-) create mode 100644 dev/create-release/known_translations diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index e8f81ccbce740..e65c5d8233098 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -26,8 +26,8 @@ # You must set the following before use! JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") -RELEASE_TAG = os.environ.get("START_COMMIT", "v1.2.0-rc2") -PREVIOUS_RELEASE_TAG = os.environ.get("END_COMMIT", "v1.1.0") +RELEASE_TAG = os.environ.get("RELEASE_TAG", "v1.2.0-rc2") +PREVIOUS_RELEASE_TAG = os.environ.get("PREVIOUS_RELEASE_TAG", "v1.1.0") # If the release tags are not provided, prompt the user to provide them while not tag_exists(RELEASE_TAG): @@ -35,7 +35,7 @@ while not tag_exists(PREVIOUS_RELEASE_TAG): print "Please specify the previous release tag." PREVIOUS_RELEASE_TAG = raw_input(\ - "For instance, if you are releasing v1.2.0, you should specify v1.1.0: ") + "For instance, if you are releasing v1.2.0, you should specify v1.1.0: ") # Gather commits found in the new tag but not in the old tag. # This filters commits based on both the git hash and the PR number. @@ -84,9 +84,9 @@ def print_indented(_list): filtered_commits = [] def is_release(commit_title): return re.findall("\[release\]", commit_title.lower()) or\ - "preparing spark release" in commit_title.lower() or\ - "preparing development version" in commit_title.lower() or\ - "CHANGES.txt" in commit_title + "preparing spark release" in commit_title.lower() or\ + "preparing development version" in commit_title.lower() or\ + "CHANGES.txt" in commit_title def is_maintenance(commit_title): return "maintenance" in commit_title.lower() or\ "manually close" in commit_title.lower() @@ -96,7 +96,7 @@ def is_revert(commit_title): return "revert" in commit_title.lower() def is_docs(commit_title): return re.findall("docs*", commit_title.lower()) or\ - "programming guide" in commit_title.lower() + "programming guide" in commit_title.lower() for c in new_commits: t = c.get_title() if not t: continue @@ -182,7 +182,7 @@ def populate(issue_type, components): jira_type = jira_issue.fields.issuetype.name jira_type = translate_issue_type(jira_type, issue, warnings) jira_components = [translate_component(c.name, _hash, warnings)\ - for c in jira_issue.fields.components] + for c in jira_issue.fields.components] all_components = set(jira_components + commit_components) populate(jira_type, all_components) # For docs without an associated JIRA, manually add it ourselves @@ -213,7 +213,7 @@ def populate(issue_type, components): # e.g. Bug fixes in MLlib, Core, and Streaming; documentation in YARN else: contributions = ["%s in %s" % (issue_type, nice_join(comps)) \ - for issue_type, comps in author_info[author].items()] + for issue_type, comps in author_info[author].items()] contribution = "; ".join(contributions) # Do not use python's capitalize() on the whole string to preserve case assert contribution diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations new file mode 100644 index 0000000000000..b74e4ee8a330b --- /dev/null +++ b/dev/create-release/known_translations @@ -0,0 +1,59 @@ +# This is a mapping of names to be translated through translate-contributors.py +# The format expected on each line should be: - +CodingCat - Nan Zhu +CrazyJvm - Chao Chen +EugenCepoi - Eugen Cepoi +GraceH - Jie Huang +JerryLead - Lijie Xu +Leolh - Liu Hao +Lewuathe - Kai Sasaki +RongGu - Rong Gu +Shiti - Shiti Saxena +Victsm - Min Shen +WangTaoTheTonic - Wang Tao +XuTingjun - Tingjun Xu +YanTangZhai - Yantang Zhai +alexdebrie - Alex DeBrie +alokito - Alok Saldanha +anantasty - Anant Asthana +andrewor14 - Andrew Or +aniketbhatnagar - Aniket Bhatnagar +arahuja - Arun Ahuja +brkyvz - Burak Yavuz +chesterxgchen - Chester Chen +chiragaggarwal - Chirag Aggarwal +chouqin - Qiping Li +cocoatomo - Tomohiko K. +coderfi - Fairiz Azizi +coderxiang - Shuo Xiang +davies - Davies Liu +epahomov - Egor Pahomov +falaki - Hossein Falaki +freeman-lab - Jeremy Freeman +industrial-sloth - Jascha Swisher +jackylk - Jacky Li +jayunit100 - Jay Vyas +jerryshao - Saisai Shao +jkbradley - Joseph Bradley +lianhuiwang - Lianhui Wang +lirui-intel - Rui Li +luluorta - Lu Lu +luogankun - Gankun Luo +maji2014 - Derek Ma +mccheah - Matthew Cheah +mengxr - Xiangrui Meng +nartz - Nathan Artz +odedz - Oded Zimerman +ravipesala - Ravindra Pesala +roxchkplusony - Victor Tso +scwf - Wang Fei +shimingfei - Shiming Fei +surq - Surong Quan +suyanNone - Su Yan +tedyu - Ted Yu +tigerquoll - Dale Richardson +wangxiaojing - Xiaojing Wang +watermen - Yadong Qi +witgo - Guoqiang Li +xinyunh - Xinyun Huang +zsxwing - Shixiong Zhu diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 18e16bcb90514..26221b270394e 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -57,11 +57,11 @@ def yesOrNoPrompt(msg): # Utility functions run git commands (written with Git 1.8.5) def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] -def run_cmd_error(cmd): return Popen(cmd, stderr=PIPE).communicate()[1] +def run_cmd_error(cmd): return Popen(cmd, stdout=PIPE, stderr=PIPE).communicate()[1] def get_date(commit_hash): return run_cmd(["git", "show", "--quiet", "--pretty=format:%cd", commit_hash]) def tag_exists(tag): - stderr = run_cmd_error(["git", "checkout", tag]) + stderr = run_cmd_error(["git", "show", tag]) return "error" not in stderr # A type-safe representation of a commit diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py index 462c21142f75f..f3b1efdd42785 100755 --- a/dev/create-release/translate-contributors.py +++ b/dev/create-release/translate-contributors.py @@ -67,6 +67,19 @@ jira_client = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) github_client = Github(GITHUB_API_TOKEN) +# Load known author translations that are cached locally +known_translations = {} +known_translations_file_name = "known_translations" +known_translations_file = open(known_translations_file_name, "r") +for line in known_translations_file: + if line.startswith("#"): continue + [old_name, new_name] = line.split(" - ") + known_translations[old_name] = new_name +known_translations_file.close() + +# Open again in case the user adds new mappings +known_translations_file = open(known_translations_file_name, "a") + # Generate candidates for the given author. This should only be called if the given author # name does not represent a full name as this operation is somewhat expensive. Under the # hood, it makes several calls to the Github and JIRA API servers to find the candidates. @@ -83,17 +96,17 @@ def generate_candidates(author, issues): candidates = [] # First check for full name of Github user - github_name = get_github_name(new_author, github_client) + github_name = get_github_name(author, github_client) if github_name: - candidates.append((github_name, "Full name of Github user %s" % new_author)) + candidates.append((github_name, "Full name of Github user %s" % author)) else: - candidates.append((NOT_FOUND, "No full name found for Github user %s" % new_author)) + candidates.append((NOT_FOUND, "No full name found for Github user %s" % author)) # Then do the same for JIRA user - jira_name = get_jira_name(new_author, jira_client) + jira_name = get_jira_name(author, jira_client) if jira_name: - candidates.append((jira_name, "Full name of JIRA user %s" % new_author)) + candidates.append((jira_name, "Full name of JIRA user %s" % author)) else: - candidates.append((NOT_FOUND, "No full name found for JIRA user %s" % new_author)) + candidates.append((NOT_FOUND, "No full name found for JIRA user %s" % author)) # Then do the same for the assignee of each of the associated JIRAs # Note that a given issue may not have an assignee, or the assignee may not have a full name for issue in issues: @@ -135,15 +148,24 @@ def generate_candidates(author, issues): print "\n========================== Translating contributor list ==========================" lines = contributors_file.readlines() for i, line in enumerate(lines): - author = line.split(" - ")[0] - print "Processing author %s (%d/%d)" % (author, i + 1, len(lines)) - if not author: - print " ERROR: Expected the following format - " - print " ERROR: Actual = %s" % line - if not is_valid_author(author): - new_author = author.split("/")[0] - issues = author.split("/")[1:] - candidates = generate_candidates(new_author, issues) + temp_author = line.split(" - ")[0] + print "Processing author %s (%d/%d)" % (temp_author, i + 1, len(lines)) + if not temp_author: + error_msg = " ERROR: Expected the following format - \n" + error_msg += " ERROR: Actual = %s" % line + print error_msg + warnings.append(error_msg) + new_contributors_file.write(line) + new_contributors_file.flush() + continue + author = temp_author.split("/")[0] + # Use the local copy of known translations where possible + if author in known_translations: + line = line.replace(temp_author, known_translations[author]) + elif not is_valid_author(author): + new_author = author + issues = temp_author.split("/")[1:] + candidates = generate_candidates(author, issues) # Print out potential replacement candidates along with the sources, e.g. # [X] No full name found for Github user andrewor14 # [X] No assignee found for SPARK-1763 @@ -169,7 +191,7 @@ def generate_candidates(author, issues): for p in good_prompts: print p # In interactive mode, additionally provide "custom" option and await user response if INTERACTIVE_MODE: - print " [%d] %s - Raw Github username" % (raw_index, new_author) + print " [%d] %s - Raw Github username" % (raw_index, author) print " [%d] Custom" % custom_index response = raw_input(" Your choice: ") last_index = custom_index @@ -191,9 +213,15 @@ def generate_candidates(author, issues): if is_valid_author(new_author): new_author = capitalize_author(new_author) else: - warnings.append("Unable to find a valid name %s for author %s" % (new_author, author)) + warnings.append("Unable to find a valid name %s for author %s" % (author, temp_author)) print " * Replacing %s with %s" % (author, new_author) - line = line.replace(author, new_author) + # If we are in interactive mode, prompt the user whether we want to remember this new mapping + if INTERACTIVE_MODE and\ + author not in known_translations and\ + yesOrNoPrompt(" Add mapping %s -> %s to known translations file?" % (author, new_author)): + known_translations_file.write("%s - %s\n" % (author, new_author)) + known_translations_file.flush() + line = line.replace(temp_author, author) new_contributors_file.write(line) new_contributors_file.flush() print "==================================================================================\n" From 3b395e10510782474789c9098084503f98ca4830 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 16 Dec 2014 21:16:03 -0800 Subject: [PATCH 037/227] [SPARK-4798][SQL] A new set of Parquet testing API and test suites This PR provides a set Parquet testing API (see trait `ParquetTest`) that enables developers to write more concise test cases. A new set of Parquet test suites built upon this API are added and aim to replace the old `ParquetQuerySuite`. To avoid potential merge conflicts, old testing code are not removed yet. The following classes can be safely removed after most Parquet related PRs are handled: - `ParquetQuerySuite` - `ParquetTestData` [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3644) Author: Cheng Lian Closes #3644 from liancheng/parquet-tests and squashes the following commits: 800e745 [Cheng Lian] Enforces ordering of test output 3bb8731 [Cheng Lian] Refactors HiveParquetSuite aa2cb2e [Cheng Lian] Decouples ParquetTest and TestSQLContext 7b43a68 [Cheng Lian] Updates ParquetTest Scaladoc 7f07af0 [Cheng Lian] Adds a new set of Parquet test suites --- .../scala/org/apache/spark/sql/SQLConf.scala | 4 + .../spark/sql/parquet/ParquetTest.scala | 127 ++++++++ .../sql/parquet/ParquetFilterSuite.scala | 253 +++++++++++++++ .../spark/sql/parquet/ParquetIOSuite.scala | 287 ++++++++++++++++++ .../sql/parquet/ParquetQuerySuite2.scala | 110 +++++++ .../sql/parquet/ParquetSchemaSuite.scala | 164 ++++++++++ .../spark/sql/parquet/HiveParquetSuite.scala | 119 +++----- .../spark/sql/parquet/parquetSuites.scala | 6 +- 8 files changed, 989 insertions(+), 81 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala 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 f5abf71d6c7b9..f5bf935522dad 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 @@ -188,6 +188,10 @@ private[sql] trait SQLConf { */ def getAllConfs: immutable.Map[String, String] = settings.synchronized { settings.toMap } + private[spark] def unsetConf(key: String) { + settings -= key + } + private[spark] def clear() { settings.clear() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala new file mode 100644 index 0000000000000..b4d48902fd2c6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -0,0 +1,127 @@ +/* + * 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.sql.parquet + +import java.io.File + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag +import scala.util.Try + +import org.apache.spark.sql.{SQLContext, SchemaRDD} +import org.apache.spark.sql.catalyst.util +import org.apache.spark.util.Utils + +/** + * A helper trait that provides convenient facilities for Parquet testing. + * + * NOTE: Considering classes `Tuple1` ... `Tuple22` all extend `Product`, it would be more + * convenient to use tuples rather than special case classes when writing test cases/suites. + * Especially, `Tuple1.apply` can be used to easily wrap a single type/value. + */ +trait ParquetTest { + val sqlContext: SQLContext + + import sqlContext._ + + protected def configuration = sparkContext.hadoopConfiguration + + /** + * Sets all SQL configurations specified in `pairs`, calls `f`, and then restore all SQL + * configurations. + * + * @todo Probably this method should be moved to a more general place + */ + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val (keys, values) = pairs.unzip + val currentValues = keys.map(key => Try(getConf(key)).toOption) + (keys, values).zipped.foreach(setConf) + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => setConf(key, value) + case (key, None) => unsetConf(key) + } + } + } + + /** + * Generates a temporary path without creating the actual file/directory, then pass it to `f`. If + * a file/directory is created there by `f`, it will be delete after `f` returns. + * + * @todo Probably this method should be moved to a more general place + */ + protected def withTempPath(f: File => Unit): Unit = { + val file = util.getTempFilePath("parquetTest").getCanonicalFile + try f(file) finally if (file.exists()) Utils.deleteRecursively(file) + } + + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + * + * @todo Probably this method should be moved to a more general place + */ + protected def withTempDir(f: File => Unit): Unit = { + val dir = Utils.createTempDir().getCanonicalFile + try f(dir) finally Utils.deleteRecursively(dir) + } + + /** + * Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected def withParquetFile[T <: Product: ClassTag: TypeTag] + (data: Seq[T]) + (f: String => Unit): Unit = { + withTempPath { file => + sparkContext.parallelize(data).saveAsParquetFile(file.getCanonicalPath) + f(file.getCanonicalPath) + } + } + + /** + * Writes `data` to a Parquet file and reads it back as a SchemaRDD, which is then passed to `f`. + * The Parquet file will be deleted after `f` returns. + */ + protected def withParquetRDD[T <: Product: ClassTag: TypeTag] + (data: Seq[T]) + (f: SchemaRDD => Unit): Unit = { + withParquetFile(data)(path => f(parquetFile(path))) + } + + /** + * Drops temporary table `tableName` after calling `f`. + */ + protected def withTempTable(tableName: String)(f: => Unit): Unit = { + try f finally dropTempTable(tableName) + } + + /** + * Writes `data` to a Parquet file, reads it back as a SchemaRDD and registers it as a temporary + * table named `tableName`, then call `f`. The temporary table together with the Parquet file will + * be dropped/deleted after `f` returns. + */ + protected def withParquetTable[T <: Product: ClassTag: TypeTag] + (data: Seq[T], tableName: String) + (f: => Unit): Unit = { + withParquetRDD(data) { rdd => + rdd.registerTempTable(tableName) + withTempTable(tableName)(f) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala new file mode 100644 index 0000000000000..111a459e6d27d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -0,0 +1,253 @@ +/* + * 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.sql.parquet + +import parquet.filter2.predicate.Operators._ +import parquet.filter2.predicate.{FilterPredicate, Operators} + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Literal, Predicate, Row} +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD} + +/** + * A test suite that tests Parquet filter2 API based filter pushdown optimization. + * + * Notice that `!(a cmp b)` are always transformed to its negated form `a cmp' b` by the + * `BooleanSimplification` optimization rule whenever possible. As a result, predicate `!(a < 1)` + * results a `GtEq` filter predicate rather than a `Not`. + * + * @todo Add test cases for `IsNull` and `IsNotNull` after merging PR #3367 + */ +class ParquetFilterSuite extends QueryTest with ParquetTest { + val sqlContext = TestSQLContext + + private def checkFilterPushdown( + rdd: SchemaRDD, + output: Seq[Symbol], + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (SchemaRDD, Any) => Unit, + expectedResult: => Any): Unit = { + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { + val query = rdd.select(output.map(_.attr): _*).where(predicate) + + val maybeAnalyzedPredicate = query.queryExecution.executedPlan.collect { + case plan: ParquetTableScan => plan.columnPruningPred + }.flatten.reduceOption(_ && _) + + assert(maybeAnalyzedPredicate.isDefined) + maybeAnalyzedPredicate.foreach { pred => + val maybeFilter = ParquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.foreach(f => assert(f.getClass === filterClass)) + } + + checker(query, expectedResult) + } + } + + private def checkFilterPushdown + (rdd: SchemaRDD, output: Symbol*) + (predicate: Predicate, filterClass: Class[_ <: FilterPredicate]) + (expectedResult: => Any): Unit = { + checkFilterPushdown(rdd, output, predicate, filterClass, checkAnswer _, expectedResult) + } + + def checkBinaryFilterPushdown + (rdd: SchemaRDD, output: Symbol*) + (predicate: Predicate, filterClass: Class[_ <: FilterPredicate]) + (expectedResult: => Any): Unit = { + def checkBinaryAnswer(rdd: SchemaRDD, result: Any): Unit = { + val actual = rdd.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq + val expected = result match { + case s: Seq[_] => s.map(_.asInstanceOf[Row].getAs[Array[Byte]](0).mkString(",")) + case s => Seq(s.asInstanceOf[Array[Byte]].mkString(",")) + } + assert(actual.sorted === expected.sorted) + } + checkFilterPushdown(rdd, output, predicate, filterClass, checkBinaryAnswer _, expectedResult) + } + + test("filter pushdown - boolean") { + withParquetRDD((true :: false :: Nil).map(Tuple1.apply)) { rdd => + checkFilterPushdown(rdd, '_1)('_1 === true, classOf[Eq[java.lang.Boolean]])(true) + checkFilterPushdown(rdd, '_1)('_1 !== true, classOf[Operators.Not])(false) + } + } + + test("filter pushdown - integer") { + withParquetRDD((1 to 4).map(Tuple1.apply)) { rdd => + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[Integer]])(1) + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + (2 to 4).map(Row.apply(_)) + } + + checkFilterPushdown(rdd, '_1)('_1 < 2, classOf[Lt [Integer]])(1) + checkFilterPushdown(rdd, '_1)('_1 > 3, classOf[Gt [Integer]])(4) + checkFilterPushdown(rdd, '_1)('_1 <= 1, classOf[LtEq[Integer]])(1) + checkFilterPushdown(rdd, '_1)('_1 >= 4, classOf[GtEq[Integer]])(4) + + checkFilterPushdown(rdd, '_1)(Literal(1) === '_1, classOf[Eq [Integer]])(1) + checkFilterPushdown(rdd, '_1)(Literal(2) > '_1, classOf[Lt [Integer]])(1) + checkFilterPushdown(rdd, '_1)(Literal(3) < '_1, classOf[Gt [Integer]])(4) + checkFilterPushdown(rdd, '_1)(Literal(1) >= '_1, classOf[LtEq[Integer]])(1) + checkFilterPushdown(rdd, '_1)(Literal(4) <= '_1, classOf[GtEq[Integer]])(4) + + checkFilterPushdown(rdd, '_1)(!('_1 < 4), classOf[Operators.GtEq[Integer]])(4) + checkFilterPushdown(rdd, '_1)('_1 > 2 && '_1 < 4, classOf[Operators.And])(3) + checkFilterPushdown(rdd, '_1)('_1 < 2 || '_1 > 3, classOf[Operators.Or]) { + Seq(Row(1), Row(4)) + } + } + } + + test("filter pushdown - long") { + withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toLong))) { rdd => + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Long]])(1) + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + (2 to 4).map(Row.apply(_)) + } + + checkFilterPushdown(rdd, '_1)('_1 < 2, classOf[Lt [java.lang.Long]])(1) + checkFilterPushdown(rdd, '_1)('_1 > 3, classOf[Gt [java.lang.Long]])(4) + checkFilterPushdown(rdd, '_1)('_1 <= 1, classOf[LtEq[java.lang.Long]])(1) + checkFilterPushdown(rdd, '_1)('_1 >= 4, classOf[GtEq[java.lang.Long]])(4) + + checkFilterPushdown(rdd, '_1)(Literal(1) === '_1, classOf[Eq [Integer]])(1) + checkFilterPushdown(rdd, '_1)(Literal(2) > '_1, classOf[Lt [java.lang.Long]])(1) + checkFilterPushdown(rdd, '_1)(Literal(3) < '_1, classOf[Gt [java.lang.Long]])(4) + checkFilterPushdown(rdd, '_1)(Literal(1) >= '_1, classOf[LtEq[java.lang.Long]])(1) + checkFilterPushdown(rdd, '_1)(Literal(4) <= '_1, classOf[GtEq[java.lang.Long]])(4) + + checkFilterPushdown(rdd, '_1)(!('_1 < 4), classOf[Operators.GtEq[java.lang.Long]])(4) + checkFilterPushdown(rdd, '_1)('_1 > 2 && '_1 < 4, classOf[Operators.And])(3) + checkFilterPushdown(rdd, '_1)('_1 < 2 || '_1 > 3, classOf[Operators.Or]) { + Seq(Row(1), Row(4)) + } + } + } + + test("filter pushdown - float") { + withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toFloat))) { rdd => + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Float]])(1) + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + (2 to 4).map(Row.apply(_)) + } + + checkFilterPushdown(rdd, '_1)('_1 < 2, classOf[Lt [java.lang.Float]])(1) + checkFilterPushdown(rdd, '_1)('_1 > 3, classOf[Gt [java.lang.Float]])(4) + checkFilterPushdown(rdd, '_1)('_1 <= 1, classOf[LtEq[java.lang.Float]])(1) + checkFilterPushdown(rdd, '_1)('_1 >= 4, classOf[GtEq[java.lang.Float]])(4) + + checkFilterPushdown(rdd, '_1)(Literal(1) === '_1, classOf[Eq [Integer]])(1) + checkFilterPushdown(rdd, '_1)(Literal(2) > '_1, classOf[Lt [java.lang.Float]])(1) + checkFilterPushdown(rdd, '_1)(Literal(3) < '_1, classOf[Gt [java.lang.Float]])(4) + checkFilterPushdown(rdd, '_1)(Literal(1) >= '_1, classOf[LtEq[java.lang.Float]])(1) + checkFilterPushdown(rdd, '_1)(Literal(4) <= '_1, classOf[GtEq[java.lang.Float]])(4) + + checkFilterPushdown(rdd, '_1)(!('_1 < 4), classOf[Operators.GtEq[java.lang.Float]])(4) + checkFilterPushdown(rdd, '_1)('_1 > 2 && '_1 < 4, classOf[Operators.And])(3) + checkFilterPushdown(rdd, '_1)('_1 < 2 || '_1 > 3, classOf[Operators.Or]) { + Seq(Row(1), Row(4)) + } + } + } + + test("filter pushdown - double") { + withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toDouble))) { rdd => + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Double]])(1) + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + (2 to 4).map(Row.apply(_)) + } + + checkFilterPushdown(rdd, '_1)('_1 < 2, classOf[Lt [java.lang.Double]])(1) + checkFilterPushdown(rdd, '_1)('_1 > 3, classOf[Gt [java.lang.Double]])(4) + checkFilterPushdown(rdd, '_1)('_1 <= 1, classOf[LtEq[java.lang.Double]])(1) + checkFilterPushdown(rdd, '_1)('_1 >= 4, classOf[GtEq[java.lang.Double]])(4) + + checkFilterPushdown(rdd, '_1)(Literal(1) === '_1, classOf[Eq[Integer]])(1) + checkFilterPushdown(rdd, '_1)(Literal(2) > '_1, classOf[Lt [java.lang.Double]])(1) + checkFilterPushdown(rdd, '_1)(Literal(3) < '_1, classOf[Gt [java.lang.Double]])(4) + checkFilterPushdown(rdd, '_1)(Literal(1) >= '_1, classOf[LtEq[java.lang.Double]])(1) + checkFilterPushdown(rdd, '_1)(Literal(4) <= '_1, classOf[GtEq[java.lang.Double]])(4) + + checkFilterPushdown(rdd, '_1)(!('_1 < 4), classOf[Operators.GtEq[java.lang.Double]])(4) + checkFilterPushdown(rdd, '_1)('_1 > 2 && '_1 < 4, classOf[Operators.And])(3) + checkFilterPushdown(rdd, '_1)('_1 < 2 || '_1 > 3, classOf[Operators.Or]) { + Seq(Row(1), Row(4)) + } + } + } + + test("filter pushdown - string") { + withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toString))) { rdd => + checkFilterPushdown(rdd, '_1)('_1 === "1", classOf[Eq[String]])("1") + checkFilterPushdown(rdd, '_1)('_1 !== "1", classOf[Operators.Not]) { + (2 to 4).map(i => Row.apply(i.toString)) + } + + checkFilterPushdown(rdd, '_1)('_1 < "2", classOf[Lt [java.lang.String]])("1") + checkFilterPushdown(rdd, '_1)('_1 > "3", classOf[Gt [java.lang.String]])("4") + checkFilterPushdown(rdd, '_1)('_1 <= "1", classOf[LtEq[java.lang.String]])("1") + checkFilterPushdown(rdd, '_1)('_1 >= "4", classOf[GtEq[java.lang.String]])("4") + + checkFilterPushdown(rdd, '_1)(Literal("1") === '_1, classOf[Eq [java.lang.String]])("1") + checkFilterPushdown(rdd, '_1)(Literal("2") > '_1, classOf[Lt [java.lang.String]])("1") + checkFilterPushdown(rdd, '_1)(Literal("3") < '_1, classOf[Gt [java.lang.String]])("4") + checkFilterPushdown(rdd, '_1)(Literal("1") >= '_1, classOf[LtEq[java.lang.String]])("1") + checkFilterPushdown(rdd, '_1)(Literal("4") <= '_1, classOf[GtEq[java.lang.String]])("4") + + checkFilterPushdown(rdd, '_1)(!('_1 < "4"), classOf[Operators.GtEq[java.lang.String]])("4") + checkFilterPushdown(rdd, '_1)('_1 > "2" && '_1 < "4", classOf[Operators.And])("3") + checkFilterPushdown(rdd, '_1)('_1 < "2" || '_1 > "3", classOf[Operators.Or]) { + Seq(Row("1"), Row("4")) + } + } + } + + test("filter pushdown - binary") { + implicit class IntToBinary(int: Int) { + def b: Array[Byte] = int.toString.getBytes("UTF-8") + } + + withParquetRDD((1 to 4).map(i => Tuple1.apply(i.b))) { rdd => + checkBinaryFilterPushdown(rdd, '_1)('_1 === 1.b, classOf[Eq[Array[Byte]]])(1.b) + checkBinaryFilterPushdown(rdd, '_1)('_1 !== 1.b, classOf[Operators.Not]) { + (2 to 4).map(i => Row.apply(i.b)).toSeq + } + + checkBinaryFilterPushdown(rdd, '_1)('_1 < 2.b, classOf[Lt [Array[Byte]]])(1.b) + checkBinaryFilterPushdown(rdd, '_1)('_1 > 3.b, classOf[Gt [Array[Byte]]])(4.b) + checkBinaryFilterPushdown(rdd, '_1)('_1 <= 1.b, classOf[LtEq[Array[Byte]]])(1.b) + checkBinaryFilterPushdown(rdd, '_1)('_1 >= 4.b, classOf[GtEq[Array[Byte]]])(4.b) + + checkBinaryFilterPushdown(rdd, '_1)(Literal(1.b) === '_1, classOf[Eq [Array[Byte]]])(1.b) + checkBinaryFilterPushdown(rdd, '_1)(Literal(2.b) > '_1, classOf[Lt [Array[Byte]]])(1.b) + checkBinaryFilterPushdown(rdd, '_1)(Literal(3.b) < '_1, classOf[Gt [Array[Byte]]])(4.b) + checkBinaryFilterPushdown(rdd, '_1)(Literal(1.b) >= '_1, classOf[LtEq[Array[Byte]]])(1.b) + checkBinaryFilterPushdown(rdd, '_1)(Literal(4.b) <= '_1, classOf[GtEq[Array[Byte]]])(4.b) + + checkBinaryFilterPushdown(rdd, '_1)(!('_1 < 4.b), classOf[Operators.GtEq[Array[Byte]]])(4.b) + checkBinaryFilterPushdown(rdd, '_1)('_1 > 2.b && '_1 < 4.b, classOf[Operators.And])(3.b) + checkBinaryFilterPushdown(rdd, '_1)('_1 < 2.b || '_1 > 3.b, classOf[Operators.Or]) { + Seq(Row(1.b), Row(4.b)) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala new file mode 100644 index 0000000000000..10a01474e95be --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -0,0 +1,287 @@ +/* + * 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.sql.parquet + +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import parquet.example.data.simple.SimpleGroup +import parquet.example.data.{Group, GroupWriter} +import parquet.hadoop.api.WriteSupport +import parquet.hadoop.api.WriteSupport.WriteContext +import parquet.hadoop.metadata.CompressionCodecName +import parquet.hadoop.{ParquetFileWriter, ParquetWriter} +import parquet.io.api.RecordConsumer +import parquet.schema.{MessageType, MessageTypeParser} + +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.types.DecimalType +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD} + +// Write support class for nested groups: ParquetWriter initializes GroupWriteSupport +// with an empty configuration (it is after all not intended to be used in this way?) +// and members are private so we need to make our own in order to pass the schema +// to the writer. +private[parquet] class TestGroupWriteSupport(schema: MessageType) extends WriteSupport[Group] { + var groupWriter: GroupWriter = null + + override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { + groupWriter = new GroupWriter(recordConsumer, schema) + } + + override def init(configuration: Configuration): WriteContext = { + new WriteContext(schema, new java.util.HashMap[String, String]()) + } + + override def write(record: Group) { + groupWriter.write(record) + } +} + +/** + * A test suite that tests basic Parquet I/O. + */ +class ParquetIOSuite extends QueryTest with ParquetTest { + val sqlContext = TestSQLContext + + /** + * Writes `data` to a Parquet file, reads it back and check file contents. + */ + protected def checkParquetFile[T <: Product: ClassTag: TypeTag](data: Seq[T]): Unit = { + withParquetRDD(data)(checkAnswer(_, data)) + } + + test("basic data types (without binary)") { + val data = (1 to 4).map { i => + (i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) + } + checkParquetFile(data) + } + + test("raw binary") { + val data = (1 to 4).map(i => Tuple1(Array.fill(3)(i.toByte))) + withParquetRDD(data) { rdd => + assertResult(data.map(_._1.mkString(",")).sorted) { + rdd.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted + } + } + } + + test("string") { + val data = (1 to 4).map(i => Tuple1(i.toString)) + // Property spark.sql.parquet.binaryAsString shouldn't affect Parquet files written by Spark SQL + // as we store Spark SQL schema in the extra metadata. + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING -> "false")(checkParquetFile(data)) + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING -> "true")(checkParquetFile(data)) + } + + test("fixed-length decimals") { + def makeDecimalRDD(decimal: DecimalType): SchemaRDD = + sparkContext + .parallelize(0 to 1000) + .map(i => Tuple1(i / 100.0)) + .select('_1 cast decimal) + + for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) { + withTempPath { dir => + val data = makeDecimalRDD(DecimalType(precision, scale)) + data.saveAsParquetFile(dir.getCanonicalPath) + checkAnswer(parquetFile(dir.getCanonicalPath), data.collect().toSeq) + } + } + + // Decimals with precision above 18 are not yet supported + intercept[RuntimeException] { + withTempPath { dir => + makeDecimalRDD(DecimalType(19, 10)).saveAsParquetFile(dir.getCanonicalPath) + parquetFile(dir.getCanonicalPath).collect() + } + } + + // Unlimited-length decimals are not yet supported + intercept[RuntimeException] { + withTempPath { dir => + makeDecimalRDD(DecimalType.Unlimited).saveAsParquetFile(dir.getCanonicalPath) + parquetFile(dir.getCanonicalPath).collect() + } + } + } + + test("map") { + val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) + checkParquetFile(data) + } + + test("array") { + val data = (1 to 4).map(i => Tuple1(Seq(i, i + 1))) + checkParquetFile(data) + } + + test("struct") { + val data = (1 to 4).map(i => Tuple1((i, s"val_$i"))) + withParquetRDD(data) { rdd => + // Structs are converted to `Row`s + checkAnswer(rdd, data.map { case Tuple1(struct) => + Tuple1(Row(struct.productIterator.toSeq: _*)) + }) + } + } + + test("nested struct with array of array as field") { + val data = (1 to 4).map(i => Tuple1((i, Seq(Seq(s"val_$i"))))) + withParquetRDD(data) { rdd => + // Structs are converted to `Row`s + checkAnswer(rdd, data.map { case Tuple1(struct) => + Tuple1(Row(struct.productIterator.toSeq: _*)) + }) + } + } + + test("nested map with struct as value type") { + val data = (1 to 4).map(i => Tuple1(Map(i -> (i, s"val_$i")))) + withParquetRDD(data) { rdd => + checkAnswer(rdd, data.map { case Tuple1(m) => + Tuple1(m.mapValues(struct => Row(struct.productIterator.toSeq: _*))) + }) + } + } + + test("nulls") { + val allNulls = ( + null.asInstanceOf[java.lang.Boolean], + null.asInstanceOf[Integer], + null.asInstanceOf[java.lang.Long], + null.asInstanceOf[java.lang.Float], + null.asInstanceOf[java.lang.Double]) + + withParquetRDD(allNulls :: Nil) { rdd => + val rows = rdd.collect() + assert(rows.size === 1) + assert(rows.head === Row(Seq.fill(5)(null): _*)) + } + } + + test("nones") { + val allNones = ( + None.asInstanceOf[Option[Int]], + None.asInstanceOf[Option[Long]], + None.asInstanceOf[Option[String]]) + + withParquetRDD(allNones :: Nil) { rdd => + val rows = rdd.collect() + assert(rows.size === 1) + assert(rows.head === Row(Seq.fill(3)(null): _*)) + } + } + + test("compression codec") { + def compressionCodecFor(path: String) = { + val codecs = ParquetTypesConverter + .readMetaData(new Path(path), Some(configuration)) + .getBlocks + .flatMap(_.getColumns) + .map(_.getCodec.name()) + .distinct + + assert(codecs.size === 1) + codecs.head + } + + val data = (0 until 10).map(i => (i, i.toString)) + + def checkCompressionCodec(codec: CompressionCodecName): Unit = { + withSQLConf(SQLConf.PARQUET_COMPRESSION -> codec.name()) { + withParquetFile(data) { path => + assertResult(parquetCompressionCodec.toUpperCase) { + compressionCodecFor(path) + } + } + } + } + + // Checks default compression codec + checkCompressionCodec(CompressionCodecName.fromConf(parquetCompressionCodec)) + + checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) + checkCompressionCodec(CompressionCodecName.GZIP) + checkCompressionCodec(CompressionCodecName.SNAPPY) + } + + test("read raw Parquet file") { + def makeRawParquetFile(path: Path): Unit = { + val schema = MessageTypeParser.parseMessageType( + """ + |message root { + | required boolean _1; + | required int32 _2; + | required int64 _3; + | required float _4; + | required double _5; + |} + """.stripMargin) + + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) + + (0 until 10).foreach { i => + val record = new SimpleGroup(schema) + record.add(0, i % 2 == 0) + record.add(1, i) + record.add(2, i.toLong) + record.add(3, i.toFloat) + record.add(4, i.toDouble) + writer.write(record) + } + + writer.close() + } + + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawParquetFile(path) + checkAnswer(parquetFile(path.toString), (0 until 10).map { i => + (i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) + }) + } + } + + test("write metadata") { + withTempPath { file => + val path = new Path(file.toURI.toString) + val fs = FileSystem.getLocal(configuration) + val attributes = ScalaReflection.attributesFor[(Int, String)] + ParquetTypesConverter.writeMetaData(attributes, path, configuration) + + assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE))) + assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) + + val metaData = ParquetTypesConverter.readMetaData(path, Some(configuration)) + val actualSchema = metaData.getFileMetaData.getSchema + val expectedSchema = ParquetTypesConverter.convertFromAttributes(attributes) + + actualSchema.checkContains(expectedSchema) + expectedSchema.checkContains(actualSchema) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala new file mode 100644 index 0000000000000..daa7ca65cd993 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala @@ -0,0 +1,110 @@ +/* + * 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.sql.parquet + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext._ + +/** + * A test suite that tests various Parquet queries. + */ +class ParquetQuerySuite2 extends QueryTest with ParquetTest { + val sqlContext = TestSQLContext + + test("simple projection") { + withParquetTable((0 until 10).map(i => (i, i.toString)), "t") { + checkAnswer(sql("SELECT _1 FROM t"), (0 until 10).map(Row.apply(_))) + } + } + + test("insertion") { + withTempDir { dir => + val data = (0 until 10).map(i => (i, i.toString)) + withParquetTable(data, "t") { + createParquetFile[(Int, String)](dir.toString).registerTempTable("dest") + withTempTable("dest") { + sql("INSERT OVERWRITE INTO dest SELECT * FROM t") + checkAnswer(table("dest"), data) + } + } + } + } + + test("appending") { + val data = (0 until 10).map(i => (i, i.toString)) + withParquetTable(data, "t") { + sql("INSERT INTO t SELECT * FROM t") + checkAnswer(table("t"), data ++ data) + } + } + + test("self-join") { + // 4 rows, cells of column 1 of row 2 and row 4 are null + val data = (1 to 4).map { i => + val maybeInt = if (i % 2 == 0) None else Some(i) + (maybeInt, i.toString) + } + + withParquetTable(data, "t") { + val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x._1 = y._1") + val queryOutput = selfJoin.queryExecution.analyzed.output + + assertResult(4, s"Field count mismatches")(queryOutput.size) + assertResult(2, s"Duplicated expression ID in query plan:\n $selfJoin") { + queryOutput.filter(_.name == "_1").map(_.exprId).size + } + + checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) + } + } + + test("nested data - struct with array field") { + val data = (1 to 10).map(i => Tuple1((i, Seq(s"val_$i")))) + withParquetTable(data, "t") { + checkAnswer(sql("SELECT _1._2[0] FROM t"), data.map { + case Tuple1((_, Seq(string))) => Row(string) + }) + } + } + + test("nested data - array of struct") { + val data = (1 to 10).map(i => Tuple1(Seq(i -> s"val_$i"))) + withParquetTable(data, "t") { + checkAnswer(sql("SELECT _1[0]._2 FROM t"), data.map { + case Tuple1(Seq((_, string))) => Row(string) + }) + } + } + + test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { + withParquetTable((1 to 10).map(Tuple1.apply), "t") { + checkAnswer(sql(s"SELECT _1 FROM t WHERE _1 < 10"), (1 to 9).map(Row.apply(_))) + } + } + + test("SPARK-3536 regression: query empty Parquet file shouldn't throw") { + withTempDir { dir => + createParquetFile[(Int, String)](dir.toString).registerTempTable("t") + withTempTable("t") { + checkAnswer(sql("SELECT * FROM t"), Seq.empty[Row]) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala new file mode 100644 index 0000000000000..34d61bf908483 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -0,0 +1,164 @@ +/* + * 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.sql.parquet + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +import org.scalatest.FunSuite +import parquet.schema.MessageTypeParser + +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.types.{BinaryType, IntegerType, StructField, StructType} +import org.apache.spark.sql.test.TestSQLContext + +class ParquetSchemaSuite extends FunSuite with ParquetTest { + val sqlContext = TestSQLContext + + /** + * Checks whether the reflected Parquet message type for product type `T` conforms `messageType`. + */ + private def testSchema[T <: Product: ClassTag: TypeTag]( + testName: String, messageType: String): Unit = { + test(testName) { + val actual = ParquetTypesConverter.convertFromAttributes(ScalaReflection.attributesFor[T]) + val expected = MessageTypeParser.parseMessageType(messageType) + actual.checkContains(expected) + expected.checkContains(actual) + } + } + + testSchema[(Boolean, Int, Long, Float, Double, Array[Byte])]( + "basic types", + """ + |message root { + | required boolean _1; + | required int32 _2; + | required int64 _3; + | required float _4; + | required double _5; + | optional binary _6; + |} + """.stripMargin) + + testSchema[(Byte, Short, Int, Long)]( + "logical integral types", + """ + |message root { + | required int32 _1 (INT_8); + | required int32 _2 (INT_16); + | required int32 _3 (INT_32); + | required int64 _4 (INT_64); + |} + """.stripMargin) + + // Currently String is the only supported logical binary type. + testSchema[Tuple1[String]]( + "binary logical types", + """ + |message root { + | optional binary _1 (UTF8); + |} + """.stripMargin) + + testSchema[Tuple1[Seq[Int]]]( + "array", + """ + |message root { + | optional group _1 (LIST) { + | repeated int32 array; + | } + |} + """.stripMargin) + + testSchema[Tuple1[Map[Int, String]]]( + "map", + """ + |message root { + | optional group _1 (MAP) { + | repeated group map (MAP_KEY_VALUE) { + | required int32 key; + | optional binary value (UTF8); + | } + | } + |} + """.stripMargin) + + testSchema[Tuple1[Pair[Int, String]]]( + "struct", + """ + |message root { + | optional group _1 { + | required int32 _1; + | optional binary _2 (UTF8); + | } + |} + """.stripMargin) + + testSchema[Tuple1[Map[Int, (String, Seq[(Int, Double)])]]]( + "deeply nested type", + """ + |message root { + | optional group _1 (MAP) { + | repeated group map (MAP_KEY_VALUE) { + | required int32 key; + | optional group value { + | optional binary _1 (UTF8); + | optional group _2 (LIST) { + | repeated group bag { + | optional group array { + | required int32 _1; + | required double _2; + | } + | } + | } + | } + | } + | } + |} + """.stripMargin) + + testSchema[(Option[Int], Map[Int, Option[Double]])]( + "optional types", + """ + |message root { + | optional int32 _1; + | optional group _2 (MAP) { + | repeated group map (MAP_KEY_VALUE) { + | required int32 key; + | optional double value; + | } + | } + |} + """.stripMargin) + + test("DataType string parser compatibility") { + val schema = StructType(List( + StructField("c1", IntegerType, false), + StructField("c2", BinaryType, true))) + + val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString) + val fromJson = ParquetTypesConverter.convertFromString(schema.json) + + (fromCaseClassString, fromJson).zipped.foreach { (a, b) => + assert(a.name == b.name) + assert(a.dataType === b.dataType) + assert(a.nullable === b.nullable) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 6f57fe8958387..4bc14bad0ad5f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -17,103 +17,66 @@ package org.apache.spark.sql.parquet -import java.io.File - -import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} - -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} -import org.apache.spark.sql.catalyst.types.{DataType, StringType, IntegerType} -import org.apache.spark.sql.{parquet, SchemaRDD} -import org.apache.spark.util.Utils - -// Implicits -import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.hive.test.TestHive case class Cases(lower: String, UPPER: String) -class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { - - val dirname = Utils.createTempDir() - - var testRDD: SchemaRDD = null - - override def beforeAll() { - // write test data - ParquetTestData.writeFile() - testRDD = parquetFile(ParquetTestData.testDir.toString) - testRDD.registerTempTable("testsource") - } - - override def afterAll() { - Utils.deleteRecursively(ParquetTestData.testDir) - Utils.deleteRecursively(dirname) - reset() // drop all tables that were registered as part of the tests - } - - // in case tests are failing we delete before and after each test - override def beforeEach() { - Utils.deleteRecursively(dirname) - } +class HiveParquetSuite extends QueryTest with ParquetTest { + val sqlContext = TestHive - override def afterEach() { - Utils.deleteRecursively(dirname) - } + import sqlContext._ test("Case insensitive attribute names") { - val tempFile = File.createTempFile("parquet", "") - tempFile.delete() - sparkContext.parallelize(1 to 10) - .map(_.toString) - .map(i => Cases(i, i)) - .saveAsParquetFile(tempFile.getCanonicalPath) - - parquetFile(tempFile.getCanonicalPath).registerTempTable("cases") - sql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) - sql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) + withParquetTable((1 to 4).map(i => Cases(i.toString, i.toString)), "cases") { + val expected = (1 to 4).map(i => Row(i.toString)) + checkAnswer(sql("SELECT upper FROM cases"), expected) + checkAnswer(sql("SELECT LOWER FROM cases"), expected) + } } test("SELECT on Parquet table") { - val rdd = sql("SELECT * FROM testsource").collect() - assert(rdd != null) - assert(rdd.forall(_.size == 6)) + val data = (1 to 4).map(i => (i, s"val_$i")) + withParquetTable(data, "t") { + checkAnswer(sql("SELECT * FROM t"), data) + } } test("Simple column projection + filter on Parquet table") { - val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() - assert(rdd.size === 5, "Filter returned incorrect number of rows") - assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value") + withParquetTable((1 to 4).map(i => (i % 2 == 0, i, s"val_$i")), "t") { + checkAnswer( + sql("SELECT `_1`, `_3` FROM t WHERE `_1` = true"), + Seq(Row(true, "val_2"), Row(true, "val_4"))) + } } test("Converting Hive to Parquet Table via saveAsParquetFile") { - sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) - parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") - val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0)) - val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0)) - - compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) + withTempPath { dir => + sql("SELECT * FROM src").saveAsParquetFile(dir.getCanonicalPath) + parquetFile(dir.getCanonicalPath).registerTempTable("p") + withTempTable("p") { + checkAnswer( + sql("SELECT * FROM src ORDER BY key"), + sql("SELECT * from p ORDER BY key").collect().toSeq) + } + } } - test("INSERT OVERWRITE TABLE Parquet table") { - sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) - parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") - // let's do three overwrites for good measure - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - val rddCopy = sql("SELECT * FROM ptable").collect() - val rddOrig = sql("SELECT * FROM testsource").collect() - assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") - compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames) - } - private def compareRDDs(rddOne: Array[Row], rddTwo: Array[Row], tableName: String, fieldNames: Seq[String]) { - var counter = 0 - (rddOne, rddTwo).zipped.foreach { - (a,b) => (a,b).zipped.toArray.zipWithIndex.foreach { - case ((value_1, value_2), index) => - assert(value_1 === value_2, s"table $tableName row $counter field ${fieldNames(index)} don't match") + test("INSERT OVERWRITE TABLE Parquet table") { + withParquetTable((1 to 4).map(i => (i, s"val_$i")), "t") { + withTempPath { file => + sql("SELECT * FROM t LIMIT 1").saveAsParquetFile(file.getCanonicalPath) + parquetFile(file.getCanonicalPath).registerTempTable("p") + withTempTable("p") { + // let's do three overwrites for good measure + sql("INSERT OVERWRITE TABLE p SELECT * FROM t") + sql("INSERT OVERWRITE TABLE p SELECT * FROM t") + sql("INSERT OVERWRITE TABLE p SELECT * FROM t") + checkAnswer(sql("SELECT * FROM p"), sql("SELECT * FROM t").collect().toSeq) + } } - counter = counter + 1 } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index 488ebba043794..fc0e42c201d56 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -37,7 +37,7 @@ case class ParquetDataWithKey(p: Int, intField: Int, stringField: String) * A suite to test the automatic conversion of metastore tables with parquet data to use the * built in parquet support. */ -class ParquetMetastoreSuite extends ParquetTest { +class ParquetMetastoreSuite extends ParquetPartitioningTest { override def beforeAll(): Unit = { super.beforeAll() @@ -112,7 +112,7 @@ class ParquetMetastoreSuite extends ParquetTest { /** * A suite of tests for the Parquet support through the data sources API. */ -class ParquetSourceSuite extends ParquetTest { +class ParquetSourceSuite extends ParquetPartitioningTest { override def beforeAll(): Unit = { super.beforeAll() @@ -145,7 +145,7 @@ class ParquetSourceSuite extends ParquetTest { /** * A collection of tests for parquet data with various forms of partitioning. */ -abstract class ParquetTest extends QueryTest with BeforeAndAfterAll { +abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll { var partitionedTableDir: File = null var partitionedTableDirWithKey: File = null From 0aa834adeaf4e31d63198e4197dc592790d3d0cc Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 16 Dec 2014 21:18:39 -0800 Subject: [PATCH 038/227] [SPARK-4744] [SQL] Short circuit evaluation for AND & OR in CodeGen Author: Cheng Hao Closes #3606 from chenghao-intel/codegen_short_circuit and squashes the following commits: f466303 [Cheng Hao] short circuit for AND & OR --- .../expressions/codegen/CodeGenerator.scala | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index ab71e15e1f573..48727d5e90140 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -314,20 +314,20 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val eval1 = expressionEvaluator(e1) val eval2 = expressionEvaluator(e2) - eval1.code ++ eval2.code ++ q""" + ..${eval1.code} var $nullTerm = false var $primitiveTerm: ${termForType(BooleanType)} = false - if ((!${eval1.nullTerm} && !${eval1.primitiveTerm}) || - (!${eval2.nullTerm} && !${eval2.primitiveTerm})) { - $nullTerm = false - $primitiveTerm = false - } else if (${eval1.nullTerm} || ${eval2.nullTerm} ) { - $nullTerm = true + if (!${eval1.nullTerm} && ${eval1.primitiveTerm} == false) { } else { - $nullTerm = false - $primitiveTerm = true + ..${eval2.code} + if (!${eval2.nullTerm} && ${eval2.primitiveTerm} == false) { + } else if (!${eval1.nullTerm} && !${eval2.nullTerm}) { + $primitiveTerm = true + } else { + $nullTerm = true + } } """.children @@ -335,20 +335,22 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val eval1 = expressionEvaluator(e1) val eval2 = expressionEvaluator(e2) - eval1.code ++ eval2.code ++ q""" + ..${eval1.code} var $nullTerm = false var $primitiveTerm: ${termForType(BooleanType)} = false - if ((!${eval1.nullTerm} && ${eval1.primitiveTerm}) || - (!${eval2.nullTerm} && ${eval2.primitiveTerm})) { - $nullTerm = false + if (!${eval1.nullTerm} && ${eval1.primitiveTerm}) { $primitiveTerm = true - } else if (${eval1.nullTerm} || ${eval2.nullTerm} ) { - $nullTerm = true } else { - $nullTerm = false - $primitiveTerm = false + ..${eval2.code} + if (!${eval2.nullTerm} && ${eval2.primitiveTerm}) { + $primitiveTerm = true + } else if (!${eval1.nullTerm} && !${eval2.nullTerm}) { + $primitiveTerm = false + } else { + $nullTerm = true + } } """.children @@ -373,7 +375,6 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } else if (${eval2.primitiveTerm} == 0) $nullTerm = true else { - $nullTerm = false $primitiveTerm = ${eval1.primitiveTerm} / ${eval2.primitiveTerm} } """.children @@ -494,7 +495,6 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin $nullTerm = ${eval1.nullTerm} $primitiveTerm = ${eval1.primitiveTerm} } else { - $nullTerm = false if (${eval1.primitiveTerm} > ${eval2.primitiveTerm}) { $primitiveTerm = ${eval1.primitiveTerm} } else { From ddc7ba31cb1062acb182293b2698b1b20ea56a46 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 16 Dec 2014 21:19:57 -0800 Subject: [PATCH 039/227] [SPARK-4720][SQL] Remainder should also return null if the divider is 0. This is a follow-up of SPARK-4593 (#3443). Author: Takuya UESHIN Closes #3581 from ueshin/issues/SPARK-4720 and squashes the following commits: c3959d4 [Takuya UESHIN] Make Remainder return null if the divider is 0. --- .../sql/catalyst/expressions/arithmetic.scala | 11 +++++++++-- .../expressions/codegen/CodeGenerator.scala | 19 +++++++++++++++++++ .../ExpressionEvaluationSuite.scala | 15 +++++++++++++++ 3 files changed, 43 insertions(+), 2 deletions(-) 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 61c26c50a6662..79a742ad4b2ad 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 @@ -122,9 +122,16 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" - override def nullable = left.nullable || right.nullable || dataType.isInstanceOf[DecimalType] + override def nullable = true - override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) + override def eval(input: Row): Any = { + val evalE2 = right.eval(input) + dataType match { + case _ if evalE2 == null => null + case _ if evalE2 == 0 => null + case nt: NumericType => i1(input, left, _.rem(_, evalE2.asInstanceOf[nt.JvmType])) + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 48727d5e90140..90c81b2631e59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -379,6 +379,25 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } """.children + case Remainder(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = 0 + + if (${eval1.nullTerm} || ${eval2.nullTerm} ) { + $nullTerm = true + } else if (${eval2.primitiveTerm} == 0) + $nullTerm = true + else { + $nullTerm = false + $primitiveTerm = ${eval1.primitiveTerm} % ${eval2.primitiveTerm} + } + """.children + case IsNotNull(e) => val eval = expressionEvaluator(e) q""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index b030483223d9e..1e371db315aac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -179,6 +179,21 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Divide(Literal(null, IntegerType), Literal(null, IntegerType)), null) } + test("Remainder") { + checkEvaluation(Remainder(Literal(2), Literal(1)), 0) + checkEvaluation(Remainder(Literal(1.0), Literal(2.0)), 1.0) + checkEvaluation(Remainder(Literal(1), Literal(2)), 1) + checkEvaluation(Remainder(Literal(1), Literal(0)), null) + checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null) + checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null) + checkEvaluation(Remainder(Literal(0), Literal(null, IntegerType)), null) + checkEvaluation(Remainder(Literal(1), Literal(null, IntegerType)), null) + checkEvaluation(Remainder(Literal(null, IntegerType), Literal(0)), null) + checkEvaluation(Remainder(Literal(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Remainder(Literal(null, IntegerType), Literal(1)), null) + checkEvaluation(Remainder(Literal(null, IntegerType), Literal(null, IntegerType)), null) + } + test("INSET") { val hS = HashSet[Any]() + 1 + 2 val nS = HashSet[Any]() + 1 + 2 + null From 770d8153a5fe400147cc597c8b4b703f0aa00c22 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 16 Dec 2014 21:21:11 -0800 Subject: [PATCH 040/227] [SPARK-4375] [SQL] Add 0 argument support for udf Author: Cheng Hao Closes #3595 from chenghao-intel/udf0 and squashes the following commits: a858973 [Cheng Hao] Add 0 arguments support for udf --- .../org/apache/spark/sql/UdfRegistration.scala | 16 ++++++++++------ .../scala/org/apache/spark/sql/UDFSuite.scala | 5 +++++ 2 files changed, 15 insertions(+), 6 deletions(-) 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 00d6b43a57812..5fb472686c9e1 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 @@ -72,14 +72,13 @@ private[sql] trait UDFRegistration { functionRegistry.registerFunction(name, builder) } - /** registerFunction 1-22 were generated by this script + /** registerFunction 0-22 were generated by this script - (1 to 22).map { x => - val types = (1 to x).map(x => "_").reduce(_ + ", " + _) + (0 to 22).map { x => + val types = (1 to x).foldRight("T")((_, s) => {s"_, $s"}) s""" - def registerFunction[T: TypeTag](name: String, func: Function$x[$types, T]): Unit = { - def builder(e: Seq[Expression]) = - ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + def registerFunction[T: TypeTag](name: String, func: Function$x[$types]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } """ @@ -87,6 +86,11 @@ private[sql] trait UDFRegistration { */ // scalastyle:off + def registerFunction[T: TypeTag](name: String, func: Function0[T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + functionRegistry.registerFunction(name, builder) + } + def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit = { def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index ef9b76b1e251e..720953ae3765a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -31,6 +31,11 @@ class UDFSuite extends QueryTest { assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4) } + test("ZeroArgument UDF") { + registerFunction("random0", () => { Math.random()}) + assert(sql("SELECT random0()").first().getDouble(0) >= 0.0) + } + test("TwoArgument UDF") { registerFunction("strLenScala", (_: String).length + (_:Int)) assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) From ec5c4279edabd5ea2b187aff6662ac07ed825b08 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 16 Dec 2014 21:23:28 -0800 Subject: [PATCH 041/227] [SPARK-4866] support StructType as key in MapType This PR brings support of using StructType(and other hashable types) as key in MapType. Author: Davies Liu Closes #3714 from davies/fix_struct_in_map and squashes the following commits: 68585d7 [Davies Liu] fix primitive types in MapType 9601534 [Davies Liu] support StructType as key in MapType --- python/pyspark/sql.py | 17 ++++++++++------- python/pyspark/tests.py | 8 ++++++++ .../apache/spark/sql/execution/pythonUdfs.scala | 2 +- 3 files changed, 19 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index ae288471b0e51..1ee0b28a3256d 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -788,8 +788,9 @@ def _create_converter(dataType): return lambda row: map(conv, row) elif isinstance(dataType, MapType): - conv = _create_converter(dataType.valueType) - return lambda row: dict((k, conv(v)) for k, v in row.iteritems()) + kconv = _create_converter(dataType.keyType) + vconv = _create_converter(dataType.valueType) + return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems()) elif isinstance(dataType, NullType): return lambda x: None @@ -944,7 +945,7 @@ def _infer_schema_type(obj, dataType): elif isinstance(dataType, MapType): k, v = obj.iteritems().next() - return MapType(_infer_type(k), + return MapType(_infer_schema_type(k, dataType.keyType), _infer_schema_type(v, dataType.valueType)) elif isinstance(dataType, StructType): @@ -1085,7 +1086,7 @@ def _has_struct_or_date(dt): elif isinstance(dt, ArrayType): return _has_struct_or_date(dt.elementType) elif isinstance(dt, MapType): - return _has_struct_or_date(dt.valueType) + return _has_struct_or_date(dt.keyType) or _has_struct_or_date(dt.valueType) elif isinstance(dt, DateType): return True elif isinstance(dt, UserDefinedType): @@ -1148,12 +1149,13 @@ def List(l): return List elif isinstance(dataType, MapType): - cls = _create_cls(dataType.valueType) + kcls = _create_cls(dataType.keyType) + vcls = _create_cls(dataType.valueType) def Dict(d): if d is None: return - return dict((k, _create_object(cls, v)) for k, v in d.items()) + return dict((_create_object(kcls, k), _create_object(vcls, v)) for k, v in d.items()) return Dict @@ -1164,7 +1166,8 @@ def Dict(d): return lambda datum: dataType.deserialize(datum) elif not isinstance(dataType, StructType): - raise Exception("unexpected data type: %s" % dataType) + # no wrapper for primitive types + return lambda x: x class Row(tuple): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index bca52a7ce6d58..b474fcf5bfb7e 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -923,6 +923,14 @@ def test_infer_schema(self): result = self.sqlCtx.sql("SELECT l[0].a from test2 where d['key'].d = '2'") self.assertEqual(1, result.first()[0]) + def test_struct_in_map(self): + d = [Row(m={Row(i=1): Row(s="")})] + rdd = self.sc.parallelize(d) + srdd = self.sqlCtx.inferSchema(rdd) + k, v = srdd.first().m.items()[0] + self.assertEqual(1, k.i) + self.assertEqual("", v.s) + def test_convert_row_to_dict(self): row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) self.assertEqual(1, row.asDict()['l'][0].a) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 2b4a88d5e864e..5a41399971dda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -132,7 +132,7 @@ object EvaluatePython { arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) case (obj: Map[_, _], mt: MapType) => obj.map { - case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type + case (k, v) => (toJava(k, mt.keyType), toJava(v, mt.valueType)) }.asJava case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) From 60698801ebc4c4947cfc7f46762a7cca2ed40452 Mon Sep 17 00:00:00 2001 From: scwf Date: Tue, 16 Dec 2014 21:26:36 -0800 Subject: [PATCH 042/227] [SPARK-4618][SQL] Make foreign DDL commands options case-insensitive Using lowercase for ```options``` key to make it case-insensitive, then we should use lower case to get value from parameters. So flowing cmd work ``` create temporary table normal_parquet USING org.apache.spark.sql.parquet OPTIONS ( PATH '/xxx/data' ) ``` Author: scwf Author: wangfei Closes #3470 from scwf/ddl-ulcase and squashes the following commits: ae78509 [scwf] address comments 8f4f585 [wangfei] address comments 3c132ef [scwf] minor fix a0fc20b [scwf] Merge branch 'master' of https://github.com/apache/spark into ddl-ulcase 4f86401 [scwf] adding CaseInsensitiveMap e244e8d [wangfei] using lower case in json e0cb017 [wangfei] make options in-casesensitive --- .../org/apache/spark/sql/sources/ddl.scala | 19 ++++++++++++++++++- .../apache/spark/sql/sources/interfaces.scala | 6 +++++- .../spark/sql/sources/TableScanSuite.scala | 6 +++--- 3 files changed, 26 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index ca510cb0b07e3..8a66ac31f2dfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -100,9 +100,26 @@ private[sql] case class CreateTableUsing( } } val dataSource = clazz.newInstance().asInstanceOf[org.apache.spark.sql.sources.RelationProvider] - val relation = dataSource.createRelation(sqlContext, options) + val relation = dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) sqlContext.baseRelationToSchemaRDD(relation).registerTempTable(tableName) Seq.empty } } + +/** + * Builds a map in which keys are case insensitive + */ +protected class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, String] { + + val baseMap = map.map(kv => kv.copy(_1 = kv._1.toLowerCase)) + + override def get(k: String): Option[String] = baseMap.get(k.toLowerCase) + + override def + [B1 >: String](kv: (String, B1)): Map[String, B1] = + baseMap + kv.copy(_1 = kv._1.toLowerCase) + + override def iterator: Iterator[(String, String)] = baseMap.iterator + + override def -(key: String): Map[String, String] = baseMap - key.toLowerCase() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 939b4e15163a6..02eff80456dbe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -36,7 +36,11 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} */ @DeveloperApi trait RelationProvider { - /** Returns a new base relation with the given parameters. */ + /** + * Returns a new base relation with the given parameters. + * Note: the parameters' keywords are case insensitive and this insensitivity is enforced + * by the Map that is passed to the function. + */ def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index b254b0620c779..3cd7b0115d567 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -25,7 +25,7 @@ class SimpleScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimpleScan(parameters("from").toInt, parameters("to").toInt)(sqlContext) + SimpleScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext) } } @@ -47,8 +47,8 @@ class TableScanSuite extends DataSourceTest { |CREATE TEMPORARY TABLE oneToTen |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( - | from '1', - | to '10' + | From '1', + | To '10' |) """.stripMargin) } From 4e1112e7b0f52e7f59cc42025f300fa8124eb9b2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Dec 2014 22:11:03 -0800 Subject: [PATCH 043/227] [Release] Update contributors list format and sort it Additionally, we now warn the user when a duplicate author name arises, in which case he/she needs to resolve it manually. --- .gitignore | 2 +- .rat-excludes | 1 + dev/create-release/generate-contributors.py | 8 ++--- dev/create-release/translate-contributors.py | 34 ++++++++++++++------ 4 files changed, 30 insertions(+), 15 deletions(-) diff --git a/.gitignore b/.gitignore index 3b9086c7187dc..30b1e12bf1b03 100644 --- a/.gitignore +++ b/.gitignore @@ -51,7 +51,7 @@ checkpoint derby.log dist/ dev/create-release/*txt -dev/create-release/*new +dev/create-release/*final spark-*-bin-*.tgz unit-tests.log /lib/ diff --git a/.rat-excludes b/.rat-excludes index d8bee1f8e49c9..1bf97f0f8b0d7 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -64,3 +64,4 @@ dist/* logs .*scalastyle-output.xml .*dependency-reduced-pom.xml +dev/create-release/known_translations diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index e65c5d8233098..8aaa250bd7e29 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -192,9 +192,9 @@ def populate(issue_type, components): print "==================================================================================\n" # Write to contributors file ordered by author names -# Each line takes the format "Author name - semi-colon delimited contributions" -# e.g. Andrew Or - Bug fixes in Windows, Core, and Web UI; improvements in Core -# e.g. Tathagata Das - Bug fixes and new features in Streaming +# Each line takes the format " * Author name -- semi-colon delimited contributions" +# e.g. * Andrew Or -- Bug fixes in Windows, Core, and Web UI; improvements in Core +# e.g. * Tathagata Das -- Bug fixes and new features in Streaming contributors_file = open(contributors_file_name, "w") authors = author_info.keys() authors.sort() @@ -223,7 +223,7 @@ def populate(issue_type, components): # E.g. andrewor14/SPARK-3425/SPARK-1157/SPARK-6672 if author in invalid_authors and invalid_authors[author]: author = author + "/" + "/".join(invalid_authors[author]) - line = "%s - %s" % (author, contribution) + line = " * %s -- %s" % (author, contribution) contributors_file.write(line + "\n") contributors_file.close() print "Contributors list is successfully written to %s!" % contributors_file_name diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py index f3b1efdd42785..86fa02d87b9a0 100755 --- a/dev/create-release/translate-contributors.py +++ b/dev/create-release/translate-contributors.py @@ -43,14 +43,12 @@ if not GITHUB_API_TOKEN: sys.exit("GITHUB_API_TOKEN must be set") -# Write new contributors list to .new +# Write new contributors list to .final if not os.path.isfile(contributors_file_name): print "Contributors file %s does not exist!" % contributors_file_name print "Have you run ./generate-contributors.py yet?" sys.exit(1) contributors_file = open(contributors_file_name, "r") -new_contributors_file_name = contributors_file_name + ".new" -new_contributors_file = open(new_contributors_file_name, "w") warnings = [] # In non-interactive mode, this script will choose the first replacement that is valid @@ -73,7 +71,7 @@ known_translations_file = open(known_translations_file_name, "r") for line in known_translations_file: if line.startswith("#"): continue - [old_name, new_name] = line.split(" - ") + [old_name, new_name] = line.strip("\n").split(" - ") known_translations[old_name] = new_name known_translations_file.close() @@ -147,16 +145,16 @@ def generate_candidates(author, issues): # If no such name exists, the original name is used (without the JIRA numbers). print "\n========================== Translating contributor list ==========================" lines = contributors_file.readlines() +contributions = [] for i, line in enumerate(lines): - temp_author = line.split(" - ")[0] + temp_author = line.strip(" * ").split(" -- ")[0] print "Processing author %s (%d/%d)" % (temp_author, i + 1, len(lines)) if not temp_author: - error_msg = " ERROR: Expected the following format - \n" + error_msg = " ERROR: Expected the following format \" * -- \"\n" error_msg += " ERROR: Actual = %s" % line print error_msg warnings.append(error_msg) - new_contributors_file.write(line) - new_contributors_file.flush() + contributions.append(line) continue author = temp_author.split("/")[0] # Use the local copy of known translations where possible @@ -222,10 +220,26 @@ def generate_candidates(author, issues): known_translations_file.write("%s - %s\n" % (author, new_author)) known_translations_file.flush() line = line.replace(temp_author, author) - new_contributors_file.write(line) - new_contributors_file.flush() + contributions.append(line) print "==================================================================================\n" contributors_file.close() +known_translations_file.close() + +# Sort the contributions before writing them to the new file. +# Additionally, check if there are any duplicate author rows. +# This could happen if the same user has both a valid full +# name (e.g. Andrew Or) and an invalid one (andrewor14). +# If so, warn the user about this at the end. +contributions.sort() +all_authors = set() +new_contributors_file_name = contributors_file_name + ".final" +new_contributors_file = open(new_contributors_file_name, "w") +for line in contributions: + author = line.strip(" * ").split(" -- ")[0] + if author in all_authors: + warnings.append("Detected duplicate author name %s. Please merge these manually." % author) + all_authors.add(author) + new_contributors_file.write(line) new_contributors_file.close() print "Translated contributors list successfully written to %s!" % new_contributors_file_name From 3d0c37b8118f6057a663f959321a79b8061132b6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 16 Dec 2014 23:00:25 -0800 Subject: [PATCH 044/227] [HOTFIX] Fix RAT exclusion for known_translations file Author: Josh Rosen Closes #3719 from JoshRosen/rat-fix and squashes the following commits: 1542886 [Josh Rosen] [HOTFIX] Fix RAT exclusion for known_translations file --- .rat-excludes | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.rat-excludes b/.rat-excludes index 1bf97f0f8b0d7..769defbac11b7 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -64,4 +64,4 @@ dist/* logs .*scalastyle-output.xml .*dependency-reduced-pom.xml -dev/create-release/known_translations +known_translations From cf50631a66500561ba44347711cdb6e963d9478f Mon Sep 17 00:00:00 2001 From: Saisai Shao Date: Wed, 17 Dec 2014 11:47:44 -0800 Subject: [PATCH 045/227] [SPARK-4595][Core] Fix MetricsServlet not work issue `MetricsServlet` handler should be added to the web UI after initialized by `MetricsSystem`, otherwise servlet handler cannot be attached. Author: Saisai Shao Author: Josh Rosen Author: jerryshao Closes #3444 from jerryshao/SPARK-4595 and squashes the following commits: 434d17e [Saisai Shao] Merge pull request #10 from JoshRosen/metrics-system-cleanup 87a2292 [Josh Rosen] Guard against misuse of MetricsSystem methods. f779fe0 [jerryshao] Fix MetricsServlet not work issue --- .../scala/org/apache/spark/SparkContext.scala | 2 ++ .../apache/spark/deploy/master/Master.scala | 4 +++ .../spark/deploy/master/ui/MasterWebUI.scala | 2 -- .../apache/spark/deploy/worker/Worker.scala | 2 ++ .../spark/deploy/worker/ui/WorkerWebUI.scala | 1 - .../apache/spark/metrics/MetricsSystem.scala | 26 ++++++++++++++----- .../scala/org/apache/spark/ui/SparkUI.scala | 2 -- 7 files changed, 28 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8e5378ecc08de..66e07d10132b8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -344,6 +344,8 @@ class SparkContext(config: SparkConf) extends Logging { // The metrics system for Driver need to be set spark.app.id to app ID. // So it should start after we get app ID from the task scheduler and set spark.app.id. metricsSystem.start() + // Attach the driver metrics servlet handler to the web ui after the metrics system is started. + metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { 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 7b32c505def9b..1f9f35d32059d 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 @@ -129,6 +129,10 @@ private[spark] class Master( masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() applicationMetricsSystem.start() + // Attach the master and app metrics servlet handler to the web ui after the metrics systems are + // started. + masterMetricsSystem.getServletHandlers.foreach(webUi.attachHandler) + applicationMetricsSystem.getServletHandlers.foreach(webUi.attachHandler) val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { case "ZOOKEEPER" => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index d86ec1e03e45c..73400c5affb5d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -41,8 +41,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(new MasterPage(this)) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - master.masterMetricsSystem.getServletHandlers.foreach(attachHandler) - master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler) } /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ 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 eb11163538b20..6863b625514c6 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 @@ -163,6 +163,8 @@ private[spark] class Worker( metricsSystem.registerSource(workerSource) metricsSystem.start() + // Attach the worker metrics servlet handler to the web ui after the metrics system is started. + metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } def changeMaster(url: String, uiUrl: String) { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index b07942a9ca729..7ac81a2d87efd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -50,7 +50,6 @@ class WorkerWebUI( attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) attachHandler(createServletHandler("/log", (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr)) - worker.metricsSystem.getServletHandlers.foreach(attachHandler) } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 5dd67b0cbf683..45633e3de01dd 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -76,22 +76,36 @@ private[spark] class MetricsSystem private ( private val sources = new mutable.ArrayBuffer[Source] private val registry = new MetricRegistry() + private var running: Boolean = false + // Treat MetricsServlet as a special sink as it should be exposed to add handlers to web ui private var metricsServlet: Option[MetricsServlet] = None - /** Get any UI handlers used by this metrics system. */ - def getServletHandlers = metricsServlet.map(_.getHandlers).getOrElse(Array()) + /** + * Get any UI handlers used by this metrics system; can only be called after start(). + */ + def getServletHandlers = { + require(running, "Can only call getServletHandlers on a running MetricsSystem") + metricsServlet.map(_.getHandlers).getOrElse(Array()) + } metricsConfig.initialize() def start() { + require(!running, "Attempting to start a MetricsSystem that is already running") + running = true registerSources() registerSinks() sinks.foreach(_.start) } def stop() { - sinks.foreach(_.stop) + if (running) { + sinks.foreach(_.stop) + } else { + logWarning("Stopping a MetricsSystem that is not running") + } + running = false } def report() { @@ -107,7 +121,7 @@ private[spark] class MetricsSystem private ( * @return An unique metric name for each combination of * application, executor/driver and metric source. */ - def buildRegistryName(source: Source): String = { + private[spark] def buildRegistryName(source: Source): String = { val appId = conf.getOption("spark.app.id") val executorId = conf.getOption("spark.executor.id") val defaultName = MetricRegistry.name(source.sourceName) @@ -144,7 +158,7 @@ private[spark] class MetricsSystem private ( }) } - def registerSources() { + private def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) @@ -160,7 +174,7 @@ private[spark] class MetricsSystem private ( } } - def registerSinks() { + private def registerSinks() { val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 176907dffa46a..0c24ad2760e08 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -57,8 +57,6 @@ private[spark] class SparkUI private ( attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) attachHandler( createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) - // If the UI is live, then serve - sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } initialize() From 5fdcbdc0c9f91be9380b09643a5db0f96c673ce8 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 17 Dec 2014 12:01:57 -0800 Subject: [PATCH 046/227] [SPARK-4625] [SQL] Add sort by for DSL & SimpleSqlParser Add `sort by` support for both DSL & SqlParser. This PR is relevant with #3386, either one merged, will cause the other rebased. Author: Cheng Hao Closes #3481 from chenghao-intel/sortby and squashes the following commits: 041004f [Cheng Hao] Add sort by for DSL & SimpleSqlParser --- .../apache/spark/sql/catalyst/SqlParser.scala | 10 ++++++++-- .../spark/sql/catalyst/dsl/package.scala | 2 ++ .../scala/org/apache/spark/sql/SchemaRDD.scala | 13 +++++++++++++ .../org/apache/spark/sql/DslQuerySuite.scala | 18 ++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 7 +++++++ 5 files changed, 48 insertions(+), 2 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 a2bcd73b6074f..d4fc9bbfd3118 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 @@ -85,6 +85,7 @@ class SqlParser extends AbstractSparkSQLParser { protected val ON = Keyword("ON") protected val OR = Keyword("OR") protected val ORDER = Keyword("ORDER") + protected val SORT = Keyword("SORT") protected val OUTER = Keyword("OUTER") protected val OVERWRITE = Keyword("OVERWRITE") protected val REGEXP = Keyword("REGEXP") @@ -140,7 +141,7 @@ class SqlParser extends AbstractSparkSQLParser { (WHERE ~> expression).? ~ (GROUP ~ BY ~> rep1sep(expression, ",")).? ~ (HAVING ~> expression).? ~ - (ORDER ~ BY ~> ordering).? ~ + sortType.? ~ (LIMIT ~> expression).? ^^ { case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => val base = r.getOrElse(NoRelation) @@ -150,7 +151,7 @@ class SqlParser extends AbstractSparkSQLParser { .getOrElse(Project(assignAliases(p), withFilter)) val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection) val withHaving = h.map(Filter(_, withDistinct)).getOrElse(withDistinct) - val withOrder = o.map(Sort(_, withHaving)).getOrElse(withHaving) + val withOrder = o.map(_(withHaving)).getOrElse(withHaving) val withLimit = l.map(Limit(_, withOrder)).getOrElse(withOrder) withLimit } @@ -202,6 +203,11 @@ class SqlParser extends AbstractSparkSQLParser { | FULL ~ OUTER.? ^^^ FullOuter ) + protected lazy val sortType: Parser[LogicalPlan => LogicalPlan] = + ( ORDER ~ BY ~> ordering ^^ { case o => l: LogicalPlan => Sort(o, l) } + | SORT ~ BY ~> ordering ^^ { case o => l: LogicalPlan => SortPartitions(o, l) } + ) + protected lazy val ordering: Parser[Seq[SortOrder]] = ( rep1sep(singleOrder, ",") | rep1sep(expression, ",") ~ direction.? ^^ { 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 index 70dabc4e6c2e9..fb252cdf51534 100755 --- 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 @@ -246,6 +246,8 @@ package object dsl { def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, logicalPlan) + def sortBy(sortExprs: SortOrder*) = SortPartitions(sortExprs, logicalPlan) + def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = { val aliasedExprs = aggregateExprs.map { case ne: NamedExpression => ne 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 a66af602a1064..7baf8ffcef787 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 @@ -216,6 +216,19 @@ class SchemaRDD( def orderBy(sortExprs: SortOrder*): SchemaRDD = new SchemaRDD(sqlContext, Sort(sortExprs, logicalPlan)) + /** + * Sorts the results by the given expressions within partition. + * {{{ + * schemaRDD.sortBy('a) + * schemaRDD.sortBy('a, 'b) + * schemaRDD.sortBy('a.asc, 'b.desc) + * }}} + * + * @group Query + */ + def sortBy(sortExprs: SortOrder*): SchemaRDD = + new SchemaRDD(sqlContext, SortPartitions(sortExprs, logicalPlan)) + @deprecated("use limit with integer argument", "1.1.0") def limit(limitExpr: Expression): SchemaRDD = new SchemaRDD(sqlContext, Limit(limitExpr, logicalPlan)) 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 1a330a2bb6d46..e40d034ce4dc0 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 @@ -120,6 +120,24 @@ class DslQuerySuite extends QueryTest { mapData.collect().sortBy(_.data(1)).reverse.toSeq) } + test("sorting #2") { + checkAnswer( + testData2.sortBy('a.asc, 'b.asc), + Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) + + checkAnswer( + testData2.sortBy('a.asc, 'b.desc), + Seq((1,2), (1,1), (2,2), (2,1), (3,2), (3,1))) + + checkAnswer( + testData2.sortBy('a.desc, 'b.desc), + Seq((3,2), (3,1), (2,2), (2,1), (1,2), (1,1))) + + checkAnswer( + testData2.sortBy('a.desc, 'b.asc), + Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + } + test("limit") { checkAnswer( testData.limit(10), 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 bcebce7603f13..ddf4776ecf7ae 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 @@ -42,6 +42,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { TimeZone.setDefault(origZone) } + test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { + checkAnswer( + sql("SELECT a FROM testData2 SORT BY a"), + Seq(1, 1, 2 ,2 ,3 ,3).map(Seq(_)) + ) + } + test("grouping on nested fields") { jsonRDD(sparkContext.parallelize("""{"nested": {"attribute": 1}, "value": 2}""" :: Nil)) .registerTempTable("rows") From 4782def094fc5b5030a944290d2301f887f77a02 Mon Sep 17 00:00:00 2001 From: carlmartin Date: Wed, 17 Dec 2014 12:24:03 -0800 Subject: [PATCH 047/227] [SPARK-4694]Fix HiveThriftServer2 cann't stop In Yarn HA mode. HiveThriftServer2 can not exit automactic when changing the standy resource manager in Yarn HA mode. The scheduler backend was aware of the AM had been exited so it call sc.stop to exit the driver process but there was a user thread(HiveThriftServer2 ) which was still alive and cause this problem. To fix it, make a demo thread to detect the sparkContext is null or not.If the sc is stopped, call the ThriftServer.stop to stop the user thread. Author: carlmartin Closes #3576 from SaintBacchus/ThriftServer2ExitBug and squashes the following commits: 2890b4a [carlmartin] Use SparkListener instead of the demo thread to stop the hive server. c15da0e [carlmartin] HiveThriftServer2 can not exit automactic when changing the standy resource manager in Yarn HA mode --- .../sql/hive/thriftserver/HiveThriftServer2.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index c5b73234fa1de..6e07df18b0e15 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -27,6 +27,7 @@ import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListener} /** * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a @@ -44,9 +45,9 @@ object HiveThriftServer2 extends Logging { val server = new HiveThriftServer2(sqlContext) server.init(sqlContext.hiveconf) server.start() + sqlContext.sparkContext.addSparkListener(new HiveThriftServer2Listener(server)) } - def main(args: Array[String]) { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") if (!optionsProcessor.process(args)) { @@ -69,12 +70,23 @@ object HiveThriftServer2 extends Logging { server.init(SparkSQLEnv.hiveContext.hiveconf) server.start() logInfo("HiveThriftServer2 started") + SparkSQLEnv.sparkContext.addSparkListener(new HiveThriftServer2Listener(server)) } catch { case e: Exception => logError("Error starting HiveThriftServer2", e) System.exit(-1) } } + + /** + * A inner sparkListener called in sc.stop to clean up the HiveThriftServer2 + */ + class HiveThriftServer2Listener(val server: HiveServer2) extends SparkListener { + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { + server.stop() + } + } + } private[hive] class HiveThriftServer2(hiveContext: HiveContext) From 7ad579ee972987863c09827554a6330aa54433b1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 17 Dec 2014 12:43:51 -0800 Subject: [PATCH 048/227] [SPARK-3698][SQL] Fix case insensitive resolution of GetField. Based on #2543. Author: Michael Armbrust Closes #3724 from marmbrus/resolveGetField and squashes the following commits: 0a47aae [Michael Armbrust] Fix case insensitive resolution of GetField. --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 10 ++++++++++ .../spark/sql/catalyst/expressions/complexTypes.scala | 8 +++++++- .../sql/hive/execution/HiveResolutionSuite.scala | 11 +++++++++++ 3 files changed, 28 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ea9bb3978691a..3705fcc1f13f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.catalyst.types.StructType /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing @@ -187,6 +188,15 @@ class Analyzer(catalog: Catalog, val result = q.resolveChildren(name, resolver).getOrElse(u) logDebug(s"Resolving $u to $result") result + + // Resolve field names using the resolver. + case f @ GetField(child, fieldName) if !f.resolved && child.resolved => + child.dataType match { + case StructType(fields) => + val resolvedFieldName = fields.map(_.name).find(resolver(_, fieldName)) + resolvedFieldName.map(n => f.copy(fieldName = n)).getOrElse(f) + case _ => f + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 917b346086dcb..b12821d44b673 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -92,7 +92,13 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio lazy val ordinal = structType.fields.indexOf(field) - override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] + override lazy val resolved = childrenResolved && fieldResolved + + /** Returns true only if the fieldName is found in the child struct. */ + private def fieldResolved = child.dataType match { + case StructType(fields) => fields.map(_.name).contains(fieldName) + case _ => false + } override def eval(input: Row): Any = { val baseValue = child.eval(input).asInstanceOf[Row] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index ee9d08ff75450..422e843d2b0d2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -27,6 +27,17 @@ case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveResolutionSuite extends HiveComparisonTest { + + case class NestedData(a: Seq[NestedData2], B: NestedData2) + case class NestedData2(a: NestedData3, B: NestedData3) + case class NestedData3(a: Int, B: Int) + + test("SPARK-3698: case insensitive test for nested data") { + sparkContext.makeRDD(Seq.empty[NestedData]).registerTempTable("nested") + // This should be successfully analyzed + sql("SELECT a[0].A.A from nested").queryExecution.analyzed + } + createQueryTest("table.attr", "SELECT src.key FROM src ORDER BY key LIMIT 1") From 62771353767b5eecf2ec6c732cab07369d784df5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 17 Dec 2014 12:48:04 -0800 Subject: [PATCH 049/227] [SPARK-4493][SQL] Don't pushdown Eq, NotEq, Lt, LtEq, Gt and GtEq predicates with nulls for Parquet Predicates like `a = NULL` and `a < NULL` can't be pushed down since Parquet `Lt`, `LtEq`, `Gt`, `GtEq` doesn't accept null value. Note that `Eq` and `NotEq` can only be used with `null` to represent predicates like `a IS NULL` and `a IS NOT NULL`. However, normally this issue doesn't cause NPE because any value compared to `NULL` results `NULL`, and Spark SQL automatically optimizes out `NULL` predicate in the `SimplifyFilters` rule. Only testing code that intentionally disables the optimizer may trigger this issue. (That's why this issue is not marked as blocker and I do **NOT** think we need to backport this to branch-1.1 This PR restricts `Lt`, `LtEq`, `Gt` and `GtEq` to non-null values only, and only uses `Eq` with null value to pushdown `IsNull` and `IsNotNull`. Also, added support for Parquet `NotEq` filter for completeness and (tiny) performance gain, it's also used to pushdown `IsNotNull`. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3367) Author: Cheng Lian Closes #3367 from liancheng/filters-with-null and squashes the following commits: cc41281 [Cheng Lian] Fixes several styling issues de7de28 [Cheng Lian] Adds stricter rules for Parquet filters with null --- .../sql/catalyst/expressions/literals.scala | 9 ++ .../spark/sql/parquet/ParquetFilters.scala | 68 +++++++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 129 ++++++++++++++++-- 3 files changed, 183 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 93c19325151bf..94e1d37c1c3ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -41,6 +41,15 @@ object Literal { } } +/** + * An extractor that matches non-null literal values + */ +object NonNullLiteral { + def unapply(literal: Literal): Option[(Any, DataType)] = { + Option(literal.value).map(_ => (literal.value, literal.dataType)) + } +} + /** * Extractor for retrieving Int literals. */ 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 6fb5f49b13668..56e7d11b2feea 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 @@ -50,12 +50,37 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.eq(floatColumn(n), v.asInstanceOf[java.lang.Float]) case DoubleType => (n: String, v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) + + // Binary.fromString and Binary.fromByteArray don't accept null values case StringType => - (n: String, v: Any) => - FilterApi.eq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + (n: String, v: Any) => FilterApi.eq( + binaryColumn(n), + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) case BinaryType => - (n: String, v: Any) => - FilterApi.eq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) + (n: String, v: Any) => FilterApi.eq( + binaryColumn(n), + Option(v).map(b => Binary.fromByteArray(v.asInstanceOf[Array[Byte]])).orNull) + } + + val makeNotEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { + case BooleanType => + (n: String, v: Any) => FilterApi.notEq(booleanColumn(n), v.asInstanceOf[java.lang.Boolean]) + case IntegerType => + (n: String, v: Any) => FilterApi.notEq(intColumn(n), v.asInstanceOf[Integer]) + case LongType => + (n: String, v: Any) => FilterApi.notEq(longColumn(n), v.asInstanceOf[java.lang.Long]) + case FloatType => + (n: String, v: Any) => FilterApi.notEq(floatColumn(n), v.asInstanceOf[java.lang.Float]) + case DoubleType => + (n: String, v: Any) => FilterApi.notEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) + case StringType => + (n: String, v: Any) => FilterApi.notEq( + binaryColumn(n), + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + case BinaryType => + (n: String, v: Any) => FilterApi.notEq( + binaryColumn(n), + Option(v).map(b => Binary.fromByteArray(v.asInstanceOf[Array[Byte]])).orNull) } val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -126,30 +151,45 @@ private[sql] object ParquetFilters { FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) } + // NOTE: + // + // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, + // which can be casted to `false` implicitly. Please refer to the `eval` method of these + // operators and the `SimplifyFilters` rule for details. predicate match { - case EqualTo(NamedExpression(name, _), Literal(value, dataType)) if dataType != NullType => + case IsNull(NamedExpression(name, dataType)) => + makeEq.lift(dataType).map(_(name, null)) + case IsNotNull(NamedExpression(name, dataType)) => + makeNotEq.lift(dataType).map(_(name, null)) + + case EqualTo(NamedExpression(name, _), NonNullLiteral(value, dataType)) => makeEq.lift(dataType).map(_(name, value)) - case EqualTo(Literal(value, dataType), NamedExpression(name, _)) if dataType != NullType => + case EqualTo(NonNullLiteral(value, dataType), NamedExpression(name, _)) => makeEq.lift(dataType).map(_(name, value)) - case LessThan(NamedExpression(name, _), Literal(value, dataType)) => + case Not(EqualTo(NamedExpression(name, _), NonNullLiteral(value, dataType))) => + makeNotEq.lift(dataType).map(_(name, value)) + case Not(EqualTo(NonNullLiteral(value, dataType), NamedExpression(name, _))) => + makeNotEq.lift(dataType).map(_(name, value)) + + case LessThan(NamedExpression(name, _), NonNullLiteral(value, dataType)) => makeLt.lift(dataType).map(_(name, value)) - case LessThan(Literal(value, dataType), NamedExpression(name, _)) => + case LessThan(NonNullLiteral(value, dataType), NamedExpression(name, _)) => makeGt.lift(dataType).map(_(name, value)) - case LessThanOrEqual(NamedExpression(name, _), Literal(value, dataType)) => + case LessThanOrEqual(NamedExpression(name, _), NonNullLiteral(value, dataType)) => makeLtEq.lift(dataType).map(_(name, value)) - case LessThanOrEqual(Literal(value, dataType), NamedExpression(name, _)) => + case LessThanOrEqual(NonNullLiteral(value, dataType), NamedExpression(name, _)) => makeGtEq.lift(dataType).map(_(name, value)) - case GreaterThan(NamedExpression(name, _), Literal(value, dataType)) => + case GreaterThan(NamedExpression(name, _), NonNullLiteral(value, dataType)) => makeGt.lift(dataType).map(_(name, value)) - case GreaterThan(Literal(value, dataType), NamedExpression(name, _)) => + case GreaterThan(NonNullLiteral(value, dataType), NamedExpression(name, _)) => makeLt.lift(dataType).map(_(name, value)) - case GreaterThanOrEqual(NamedExpression(name, _), Literal(value, dataType)) => + case GreaterThanOrEqual(NamedExpression(name, _), NonNullLiteral(value, dataType)) => makeGtEq.lift(dataType).map(_(name, value)) - case GreaterThanOrEqual(Literal(value, dataType), NamedExpression(name, _)) => + case GreaterThanOrEqual(NonNullLiteral(value, dataType), NamedExpression(name, _)) => makeLtEq.lift(dataType).map(_(name, value)) case And(lhs, rhs) => 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 7ee4f3c1e93eb..0e5635d3e99dc 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,12 +17,13 @@ package org.apache.spark.sql.parquet -import _root_.parquet.filter2.predicate.{FilterPredicate, Operators} import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} +import parquet.filter2.predicate.{FilterPredicate, Operators} import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil +import parquet.io.api.Binary import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ @@ -84,7 +85,8 @@ case class NumericData(i: Int, d: Double) class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. - var testRDD: SchemaRDD = null + private var testRDD: SchemaRDD = null + private val originalParquetFilterPushdownEnabled = TestSQLContext.parquetFilterPushDown override def beforeAll() { ParquetTestData.writeFile() @@ -109,13 +111,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(ParquetTestData.testNestedDir3) Utils.deleteRecursively(ParquetTestData.testNestedDir4) // here we should also unregister the table?? + + setConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED, originalParquetFilterPushdownEnabled.toString) } test("Read/Write All Types") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) - val data = sparkContext.parallelize(range) - .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + val data = sparkContext.parallelize(range).map { x => + parquet.AllDataTypes( + s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0) + } data.saveAsParquetFile(tempDir) @@ -260,14 +266,15 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) - val data = sparkContext.parallelize(range) - .map(x => AllDataTypesWithNonPrimitiveType( + val data = sparkContext.parallelize(range).map { x => + parquet.AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, (0 until x), (0 until x).map(Option(_).filter(_ % 3 == 0)), (0 until x).map(i => i -> i.toLong).toMap, (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), - Data((0 until x), Nested(x, s"$x")))) + parquet.Data((0 until x), parquet.Nested(x, s"$x"))) + } data.saveAsParquetFile(tempDir) checkAnswer( @@ -420,7 +427,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("save and load case class RDD with nulls as parquet") { - val data = NullReflectData(null, null, null, null, null) + val data = parquet.NullReflectData(null, null, null, null, null) val rdd = sparkContext.parallelize(data :: Nil) val file = getTempFilePath("parquet") @@ -435,7 +442,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("save and load case class RDD with Nones as parquet") { - val data = OptionalReflectData(None, None, None, None, None) + val data = parquet.OptionalReflectData(None, None, None, None, None) val rdd = sparkContext.parallelize(data :: Nil) val file = getTempFilePath("parquet") @@ -938,4 +945,108 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA checkAnswer(parquetFile(tempDir), data.toSchemaRDD.collect().toSeq) } } + + def checkFilter(predicate: Predicate, filterClass: Class[_ <: FilterPredicate]): Unit = { + val filter = ParquetFilters.createFilter(predicate) + assert(filter.isDefined) + assert(filter.get.getClass == filterClass) + } + + test("Pushdown IsNull predicate") { + checkFilter('a.int.isNull, classOf[Operators.Eq[Integer]]) + checkFilter('a.long.isNull, classOf[Operators.Eq[java.lang.Long]]) + checkFilter('a.float.isNull, classOf[Operators.Eq[java.lang.Float]]) + checkFilter('a.double.isNull, classOf[Operators.Eq[java.lang.Double]]) + checkFilter('a.string.isNull, classOf[Operators.Eq[Binary]]) + checkFilter('a.binary.isNull, classOf[Operators.Eq[Binary]]) + } + + test("Pushdown IsNotNull predicate") { + checkFilter('a.int.isNotNull, classOf[Operators.NotEq[Integer]]) + checkFilter('a.long.isNotNull, classOf[Operators.NotEq[java.lang.Long]]) + checkFilter('a.float.isNotNull, classOf[Operators.NotEq[java.lang.Float]]) + checkFilter('a.double.isNotNull, classOf[Operators.NotEq[java.lang.Double]]) + checkFilter('a.string.isNotNull, classOf[Operators.NotEq[Binary]]) + checkFilter('a.binary.isNotNull, classOf[Operators.NotEq[Binary]]) + } + + test("Pushdown EqualTo predicate") { + checkFilter('a.int === 0, classOf[Operators.Eq[Integer]]) + checkFilter('a.long === 0.toLong, classOf[Operators.Eq[java.lang.Long]]) + checkFilter('a.float === 0.toFloat, classOf[Operators.Eq[java.lang.Float]]) + checkFilter('a.double === 0.toDouble, classOf[Operators.Eq[java.lang.Double]]) + checkFilter('a.string === "foo", classOf[Operators.Eq[Binary]]) + checkFilter('a.binary === "foo".getBytes, classOf[Operators.Eq[Binary]]) + } + + test("Pushdown Not(EqualTo) predicate") { + checkFilter(!('a.int === 0), classOf[Operators.NotEq[Integer]]) + checkFilter(!('a.long === 0.toLong), classOf[Operators.NotEq[java.lang.Long]]) + checkFilter(!('a.float === 0.toFloat), classOf[Operators.NotEq[java.lang.Float]]) + checkFilter(!('a.double === 0.toDouble), classOf[Operators.NotEq[java.lang.Double]]) + checkFilter(!('a.string === "foo"), classOf[Operators.NotEq[Binary]]) + checkFilter(!('a.binary === "foo".getBytes), classOf[Operators.NotEq[Binary]]) + } + + test("Pushdown LessThan predicate") { + checkFilter('a.int < 0, classOf[Operators.Lt[Integer]]) + checkFilter('a.long < 0.toLong, classOf[Operators.Lt[java.lang.Long]]) + checkFilter('a.float < 0.toFloat, classOf[Operators.Lt[java.lang.Float]]) + checkFilter('a.double < 0.toDouble, classOf[Operators.Lt[java.lang.Double]]) + checkFilter('a.string < "foo", classOf[Operators.Lt[Binary]]) + checkFilter('a.binary < "foo".getBytes, classOf[Operators.Lt[Binary]]) + } + + test("Pushdown LessThanOrEqual predicate") { + checkFilter('a.int <= 0, classOf[Operators.LtEq[Integer]]) + checkFilter('a.long <= 0.toLong, classOf[Operators.LtEq[java.lang.Long]]) + checkFilter('a.float <= 0.toFloat, classOf[Operators.LtEq[java.lang.Float]]) + checkFilter('a.double <= 0.toDouble, classOf[Operators.LtEq[java.lang.Double]]) + checkFilter('a.string <= "foo", classOf[Operators.LtEq[Binary]]) + checkFilter('a.binary <= "foo".getBytes, classOf[Operators.LtEq[Binary]]) + } + + test("Pushdown GreaterThan predicate") { + checkFilter('a.int > 0, classOf[Operators.Gt[Integer]]) + checkFilter('a.long > 0.toLong, classOf[Operators.Gt[java.lang.Long]]) + checkFilter('a.float > 0.toFloat, classOf[Operators.Gt[java.lang.Float]]) + checkFilter('a.double > 0.toDouble, classOf[Operators.Gt[java.lang.Double]]) + checkFilter('a.string > "foo", classOf[Operators.Gt[Binary]]) + checkFilter('a.binary > "foo".getBytes, classOf[Operators.Gt[Binary]]) + } + + test("Pushdown GreaterThanOrEqual predicate") { + checkFilter('a.int >= 0, classOf[Operators.GtEq[Integer]]) + checkFilter('a.long >= 0.toLong, classOf[Operators.GtEq[java.lang.Long]]) + checkFilter('a.float >= 0.toFloat, classOf[Operators.GtEq[java.lang.Float]]) + checkFilter('a.double >= 0.toDouble, classOf[Operators.GtEq[java.lang.Double]]) + checkFilter('a.string >= "foo", classOf[Operators.GtEq[Binary]]) + checkFilter('a.binary >= "foo".getBytes, classOf[Operators.GtEq[Binary]]) + } + + test("Comparison with null should not be pushed down") { + val predicates = Seq( + 'a.int === null, + !('a.int === null), + + Literal(null) === 'a.int, + !(Literal(null) === 'a.int), + + 'a.int < null, + 'a.int <= null, + 'a.int > null, + 'a.int >= null, + + Literal(null) < 'a.int, + Literal(null) <= 'a.int, + Literal(null) > 'a.int, + Literal(null) >= 'a.int + ) + + predicates.foreach { p => + assert( + ParquetFilters.createFilter(p).isEmpty, + "Comparison predicate with null shouldn't be pushed down") + } + } } From 902e4d54acbc3c88163a5c6447aff68ed57475c1 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 17 Dec 2014 12:51:27 -0800 Subject: [PATCH 050/227] [SPARK-4755] [SQL] sqrt(negative value) should return null Author: Daoyuan Wang Closes #3616 from adrian-wang/sqrt and squashes the following commits: d877439 [Daoyuan Wang] fix NULLTYPE 3effa2c [Daoyuan Wang] sqrt(negative value) should return null --- .../sql/catalyst/expressions/arithmetic.scala | 15 +++++++++++++-- .../expressions/ExpressionEvaluationSuite.scala | 2 ++ 2 files changed, 15 insertions(+), 2 deletions(-) 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 79a742ad4b2ad..168a963e29c90 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 @@ -38,11 +38,22 @@ case class Sqrt(child: Expression) extends UnaryExpression { def dataType = DoubleType override def foldable = child.foldable - def nullable = child.nullable + def nullable = true override def toString = s"SQRT($child)" override def eval(input: Row): Any = { - n1(child, input, (na,a) => math.sqrt(na.toDouble(a))) + val evalE = child.eval(input) + if (evalE == null) { + null + } else { + child.dataType match { + case n: NumericType => + val value = n.numeric.toDouble(evalE.asInstanceOf[n.JvmType]) + if (value < 0) null + else math.sqrt(value) + case other => sys.error(s"Type $other does not support non-negative numeric operations") + } + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 1e371db315aac..4ba7d87ba8c5a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -1037,6 +1037,8 @@ class ExpressionEvaluationSuite extends FunSuite { } checkEvaluation(Sqrt(Literal(null, DoubleType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Sqrt(-1), null, EmptyRow) + checkEvaluation(Sqrt(-1.5), null, EmptyRow) } test("Bitwise operations") { From 636d9fc450faaa0d8e82e0d34bb7b791e3812cb7 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 17 Dec 2014 13:39:36 -0800 Subject: [PATCH 051/227] [SPARK-3739] [SQL] Update the split num base on block size for table scanning In local mode, Hadoop/Hive will ignore the "mapred.map.tasks", hence for small table file, it's always a single input split, however, SparkSQL doesn't honor that in table scanning, and we will get different result when do the Hive Compatibility test. This PR will fix that. Author: Cheng Hao Closes #2589 from chenghao-intel/source_split and squashes the following commits: dff38e7 [Cheng Hao] Remove the extra blank line 160a2b6 [Cheng Hao] fix the compiling bug 04d67f7 [Cheng Hao] Keep 1 split for small file in table scanning --- .../apache/spark/sql/hive/TableReader.scala | 13 +- ...l_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc | 500 ++++++++++++++++++ .../hive/execution/HiveTableScanSuite.scala | 9 +- 3 files changed, 517 insertions(+), 5 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/file_split_for_small_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index f60bc3788e3e4..c368715f7c6f5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -57,10 +57,15 @@ class HadoopTableReader( @transient hiveExtraConf: HiveConf) extends TableReader { - // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless - // it is smaller than what Spark suggests. - private val _minSplitsPerRDD = math.max( - sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions) + // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local". + // https://hadoop.apache.org/docs/r1.0.4/mapred-default.html + // + // In order keep consistency with Hive, we will let it be 0 in local mode also. + private val _minSplitsPerRDD = if (sc.sparkContext.isLocal) { + 0 // will splitted based on block by default. + } else { + math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions) + } // TODO: set aws s3 credentials. diff --git a/sql/hive/src/test/resources/golden/file_split_for_small_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc b/sql/hive/src/test/resources/golden/file_split_for_small_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc new file mode 100644 index 0000000000000..b70e127e82d05 --- /dev/null +++ b/sql/hive/src/test/resources/golden/file_split_for_small_table-0-7a45831bf96814d9a7fc3d78fb7bd8dc @@ -0,0 +1,500 @@ +0 val_0 +0 val_0 +0 val_0 +2 val_2 +4 val_4 +5 val_5 +5 val_5 +5 val_5 +8 val_8 +9 val_9 +10 val_10 +11 val_11 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +17 val_17 +18 val_18 +18 val_18 +19 val_19 +20 val_20 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +67 val_67 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 54c0f017d4cb6..a0ace91060a28 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -44,6 +44,14 @@ class HiveTableScanSuite extends HiveComparisonTest { |SELECT * from part_scan_test; """.stripMargin) + // In unit test, kv1.txt is a small file and will be loaded as table src + // Since the small file will be considered as a single split, we assume + // Hive / SparkSQL HQL has the same output even for SORT BY + createQueryTest("file_split_for_small_table", + """ + |SELECT key, value FROM src SORT BY key, value + """.stripMargin) + test("Spark-4041: lowercase issue") { TestHive.sql("CREATE TABLE tb (KEY INT, VALUE STRING) STORED AS ORC") TestHive.sql("insert into table tb select key, value from src") @@ -68,5 +76,4 @@ class HiveTableScanSuite extends HiveComparisonTest { === Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")),Row(null))) TestHive.sql("DROP TABLE timestamp_query_null") } - } From affc3f460fc6172b6cea88a8779d6d40166c1c6b Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 17 Dec 2014 14:12:46 -0800 Subject: [PATCH 052/227] [SPARK-4821] [mllib] [python] [docs] Fix for pyspark.mllib.rand doc + small doc edit + include edit to make IntelliJ happy CC: davies mengxr Note to davies -- this does not fix the "WARNING: Literal block expected; none found." warnings since that seems to involve spacing which IntelliJ does not like. (Those warnings occur when generating the Python docs.) Author: Joseph K. Bradley Closes #3669 from jkbradley/python-warnings and squashes the following commits: 4587868 [Joseph K. Bradley] fixed warning 8cb073c [Joseph K. Bradley] Updated based on davies recommendation c51eca4 [Joseph K. Bradley] Updated rst file for pyspark.mllib.rand doc. Small doc edit. Small include edit to make IntelliJ happy. --- python/docs/pyspark.streaming.rst | 2 +- python/pyspark/mllib/__init__.py | 27 +-------------------------- python/pyspark/mllib/feature.py | 6 +++--- 3 files changed, 5 insertions(+), 30 deletions(-) diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst index 5024d694b668f..f08185627d0bc 100644 --- a/python/docs/pyspark.streaming.rst +++ b/python/docs/pyspark.streaming.rst @@ -1,5 +1,5 @@ pyspark.streaming module -================== +======================== Module contents --------------- diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index 5030a655fcbba..c3217620e3c4e 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -32,29 +32,4 @@ import rand as random random.__name__ = 'random' random.RandomRDDs.__module__ = __name__ + '.random' - - -class RandomModuleHook(object): - """ - Hook to import pyspark.mllib.random - """ - fullname = __name__ + '.random' - - def find_module(self, name, path=None): - # skip all other modules - if not name.startswith(self.fullname): - return - return self - - def load_module(self, name): - if name == self.fullname: - return random - - cname = name.rsplit('.', 1)[-1] - try: - return getattr(random, cname) - except AttributeError: - raise ImportError - - -sys.meta_path.append(RandomModuleHook()) +sys.modules[__name__ + '.random'] = random diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 741c630cbd6eb..e46af208866a2 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -53,10 +53,10 @@ class Normalizer(VectorTransformer): """ :: Experimental :: - Normalizes samples individually to unit L\ :sup:`p`\ norm + Normalizes samples individually to unit L\ :sup:`p`\ norm - For any 1 <= `p` <= float('inf'), normalizes samples using - sum(abs(vector). :sup:`p`) :sup:`(1/p)` as norm. + For any 1 <= `p` < float('inf'), normalizes samples using + sum(abs(vector) :sup:`p`) :sup:`(1/p)` as norm. For `p` = float('inf'), max(abs(vector)) will be used as norm for normalization. From 19c0faad6d04349952ec25ce5ae94b718d7e8518 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 17 Dec 2014 14:27:02 -0800 Subject: [PATCH 053/227] [HOTFIX][SQL] Fix parquet filter suite Author: Michael Armbrust Closes #3727 from marmbrus/parquetNotEq and squashes the following commits: 2157bfc [Michael Armbrust] Fix parquet filter suite --- .../sql/parquet/ParquetFilterSuite.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index 111a459e6d27d..b17300475b6f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -87,14 +87,14 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { test("filter pushdown - boolean") { withParquetRDD((true :: false :: Nil).map(Tuple1.apply)) { rdd => checkFilterPushdown(rdd, '_1)('_1 === true, classOf[Eq[java.lang.Boolean]])(true) - checkFilterPushdown(rdd, '_1)('_1 !== true, classOf[Operators.Not])(false) + checkFilterPushdown(rdd, '_1)('_1 !== true, classOf[Operators.NotEq[java.lang.Boolean]])(false) } } test("filter pushdown - integer") { withParquetRDD((1 to 4).map(Tuple1.apply)) { rdd => checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[Integer]])(1) - checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[Integer]]) { (2 to 4).map(Row.apply(_)) } @@ -118,9 +118,9 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - long") { - withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toLong))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(i.toLong))) { rdd => checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Long]])(1) - checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Long]]) { (2 to 4).map(Row.apply(_)) } @@ -144,9 +144,9 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - float") { - withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toFloat))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(i.toFloat))) { rdd => checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Float]])(1) - checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Float]]) { (2 to 4).map(Row.apply(_)) } @@ -170,9 +170,9 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - double") { - withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toDouble))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(i.toDouble))) { rdd => checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Double]])(1) - checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.Not]) { + checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Double]]) { (2 to 4).map(Row.apply(_)) } @@ -196,9 +196,9 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - string") { - withParquetRDD((1 to 4).map(i => Tuple1.apply(i.toString))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(i.toString))) { rdd => checkFilterPushdown(rdd, '_1)('_1 === "1", classOf[Eq[String]])("1") - checkFilterPushdown(rdd, '_1)('_1 !== "1", classOf[Operators.Not]) { + checkFilterPushdown(rdd, '_1)('_1 !== "1", classOf[Operators.NotEq[String]]) { (2 to 4).map(i => Row.apply(i.toString)) } @@ -226,9 +226,9 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { def b: Array[Byte] = int.toString.getBytes("UTF-8") } - withParquetRDD((1 to 4).map(i => Tuple1.apply(i.b))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { rdd => checkBinaryFilterPushdown(rdd, '_1)('_1 === 1.b, classOf[Eq[Array[Byte]]])(1.b) - checkBinaryFilterPushdown(rdd, '_1)('_1 !== 1.b, classOf[Operators.Not]) { + checkBinaryFilterPushdown(rdd, '_1)('_1 !== 1.b, classOf[Operators.NotEq[Array[Byte]]]) { (2 to 4).map(i => Row.apply(i.b)).toSeq } From 8d0d2a65eb3a7b1865f7fa7cc18b146fc6474620 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 17 Dec 2014 15:01:59 -0800 Subject: [PATCH 054/227] [SPARK-4856] [SQL] NullType instead of StringType when sampling against empty string or nul... ``` TestSQLContext.sparkContext.parallelize( """{"ip":"27.31.100.29","headers":{"Host":"1.abc.com","Charset":"UTF-8"}}""" :: """{"ip":"27.31.100.29","headers":{}}""" :: """{"ip":"27.31.100.29","headers":""}""" :: Nil) ``` As empty string (the "headers") will be considered as String in the beginning (in line 2 and 3), it ignores the real nested data type (struct type "headers" in line 1), and also take the line 1 (the "headers") as String Type, which is not our expected. Author: Cheng Hao Closes #3708 from chenghao-intel/json and squashes the following commits: e7a72e9 [Cheng Hao] add more concise unit test 853de51 [Cheng Hao] NullType instead of StringType when sampling against empty string or null value --- .../org/apache/spark/sql/json/JsonRDD.scala | 4 +++- .../org/apache/spark/sql/json/JsonSuite.scala | 19 +++++++++++++++++++ .../apache/spark/sql/json/TestJsonData.scala | 7 +++++++ 3 files changed, 29 insertions(+), 1 deletion(-) 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 ffb9548356d1d..00449c200704a 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 @@ -263,6 +263,8 @@ private[sql] object JsonRDD extends Logging { val elementType = typeOfArray(array) buildKeyPathForInnerStructs(array, elementType) :+ (key, elementType) } + // we couldn't tell what the type is if the value is null or empty string + case (key: String, value) if value == "" || value == null => (key, NullType) :: Nil case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil } } @@ -400,13 +402,13 @@ private[sql] object JsonRDD extends Logging { } else { desiredType match { case StringType => toString(value) + case _ if value == null || value == "" => null // guard the non string type case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) case DoubleType => toDouble(value) case DecimalType() => toDecimal(value) case BooleanType => value.asInstanceOf[BooleanType.JvmType] case NullType => null - case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], struct) 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 f088d413257a9..8dce3372a8db3 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 @@ -193,6 +193,25 @@ class JsonSuite extends QueryTest { StringType) } + test("Complex field and type inferring with null in sampling") { + val jsonSchemaRDD = jsonRDD(jsonNullStruct) + val expectedSchema = StructType( + StructField("headers", StructType( + StructField("Charset", StringType, true) :: + StructField("Host", StringType, true) :: Nil) + , true) :: + StructField("ip", StringType, true) :: + StructField("nullstr", StringType, true):: Nil) + + assert(expectedSchema === jsonSchemaRDD.schema) + jsonSchemaRDD.registerTempTable("jsonTable") + + checkAnswer( + sql("select nullstr, headers.Host from jsonTable"), + Seq(Row("", "1.abc.com"), Row("", null), Row("", null), Row(null, null)) + ) + } + test("Primitive field and type inferring") { val jsonSchemaRDD = jsonRDD(primitiveFieldAndType) 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 e5773a55875bc..3370b3c98b4be 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 @@ -43,6 +43,13 @@ object TestJsonData { """{"num_num_1":21474836570, "num_num_2":1.1, "num_num_3": 21474836470, "num_bool":null, "num_str":92233720368547758070, "str_bool":null}""" :: Nil) + val jsonNullStruct = + TestSQLContext.sparkContext.parallelize( + """{"nullstr":"","ip":"27.31.100.29","headers":{"Host":"1.abc.com","Charset":"UTF-8"}}""" :: + """{"nullstr":"","ip":"27.31.100.29","headers":{}}""" :: + """{"nullstr":"","ip":"27.31.100.29","headers":""}""" :: + """{"nullstr":null,"ip":"27.31.100.29","headers":null}""" :: Nil) + val complexFieldValueTypeConflict = TestSQLContext.sparkContext.parallelize( """{"num_struct":11, "str_array":[1, 2, 3], From f33d55046427b8594fd19bda5fd2214eeeab1a95 Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Wed, 17 Dec 2014 15:41:35 -0800 Subject: [PATCH 055/227] [SPARK-3891][SQL] Add array support to percentile, percentile_approx and constant inspectors support Supported passing array to percentile and percentile_approx UDAFs To support percentile_approx, constant inspectors are supported for GenericUDAF Constant folding support added to CreateArray expression Avoided constant udf expression re-evaluation Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #2802 from gvramana/percentile_array_support and squashes the following commits: a0182e5 [Venkata Ramana Gollamudi] fixed review comment a18f917 [Venkata Ramana Gollamudi] avoid constant udf expression re-evaluation - fixes failure due to return iterator and value type mismatch c46db0f [Venkata Ramana Gollamudi] Removed TestHive reset 4d39105 [Venkata Ramana Gollamudi] Unified inspector creation, style check fixes f37fd69 [Venkata Ramana Gollamudi] Fixed review comments 47f6365 [Venkata Ramana Gollamudi] fixed test cb7c61e [Venkata Ramana Gollamudi] Supported ConstantInspector for UDAF Fixed HiveUdaf wrap object issue. 7f94aff [Venkata Ramana Gollamudi] Added foldable support to CreateArray --- .../catalyst/expressions/complexTypes.scala | 4 ++- .../org/apache/spark/sql/hive/hiveUdfs.scala | 35 +++++++++++++------ .../sql/hive/execution/HiveUdfSuite.scala | 13 ++++++- 3 files changed, 40 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index b12821d44b673..9aec601886efc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -113,7 +113,9 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio */ case class CreateArray(children: Seq[Expression]) extends Expression { override type EvaluatedType = Any - + + override def foldable = !children.exists(!_.foldable) + lazy val childTypes = children.map(_.dataType).distinct override lazy val resolved = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index ed2e96df8ad77..93b6ef9fbc59b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -158,6 +158,11 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr override def foldable = isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] + @transient + protected def constantReturnValue = unwrap( + returnInspector.asInstanceOf[ConstantObjectInspector].getWritableConstantValue(), + returnInspector) + @transient protected lazy val deferedObjects = argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] @@ -166,6 +171,8 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr override def eval(input: Row): Any = { returnInspector // Make sure initialized. + if(foldable) return constantReturnValue + var i = 0 while (i < children.length) { val idx = i @@ -193,12 +200,13 @@ private[hive] case class HiveGenericUdaf( @transient protected lazy val objectInspector = { - resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) + val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors.toArray, false, false) + resolver.getEvaluator(parameterInfo) .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) } @transient - protected lazy val inspectors = children.map(_.dataType).map(toInspector) + protected lazy val inspectors = children.map(toInspector) def dataType: DataType = inspectorToDataType(objectInspector) @@ -223,12 +231,13 @@ private[hive] case class HiveUdaf( @transient protected lazy val objectInspector = { - resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) + val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors.toArray, false, false) + resolver.getEvaluator(parameterInfo) .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) } @transient - protected lazy val inspectors = children.map(_.dataType).map(toInspector) + protected lazy val inspectors = children.map(toInspector) def dataType: DataType = inspectorToDataType(objectInspector) @@ -261,7 +270,7 @@ private[hive] case class HiveGenericUdtf( protected lazy val function: GenericUDTF = funcWrapper.createFunction() @transient - protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) + protected lazy val inputInspectors = children.map(toInspector) @transient protected lazy val outputInspector = function.initialize(inputInspectors.toArray) @@ -334,10 +343,13 @@ private[hive] case class HiveUdafFunction( } else { funcWrapper.createFunction[AbstractGenericUDAFResolver]() } - - private val inspectors = exprs.map(_.dataType).map(toInspector).toArray - - private val function = resolver.getEvaluator(exprs.map(_.dataType.toTypeInfo).toArray) + + private val inspectors = exprs.map(toInspector).toArray + + private val function = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors, false, false) + resolver.getEvaluator(parameterInfo) + } private val returnInspector = function.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) @@ -350,9 +362,12 @@ private[hive] case class HiveUdafFunction( @transient val inputProjection = new InterpretedProjection(exprs) + @transient + protected lazy val cached = new Array[AnyRef](exprs.length) + def update(input: Row): Unit = { val inputs = inputProjection(input).asInstanceOf[Seq[AnyRef]].toArray - function.iterate(buffer, inputs) + function.iterate(buffer, wrap(inputs, inspectors, cached)) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index 5fcaf671a80de..5fc8d8dbe3a9f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -92,10 +92,21 @@ class HiveUdfSuite extends QueryTest { } test("SPARK-2693 udaf aggregates test") { - checkAnswer(sql("SELECT percentile(key,1) FROM src LIMIT 1"), + checkAnswer(sql("SELECT percentile(key, 1) FROM src LIMIT 1"), sql("SELECT max(key) FROM src").collect().toSeq) + + checkAnswer(sql("SELECT percentile(key, array(1, 1)) FROM src LIMIT 1"), + sql("SELECT array(max(key), max(key)) FROM src").collect().toSeq) } + test("Generic UDAF aggregates") { + checkAnswer(sql("SELECT ceiling(percentile_approx(key, 0.99999)) FROM src LIMIT 1"), + sql("SELECT max(key) FROM src LIMIT 1").collect().toSeq) + + checkAnswer(sql("SELECT percentile_approx(100.0, array(0.9, 0.9)) FROM src LIMIT 1"), + sql("SELECT array(100, 100) FROM src LIMIT 1").collect().toSeq) + } + test("UDFIntegerToString") { val testData = TestHive.sparkContext.parallelize( IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil) From ca1260891adb87f4985d3cfc515b4756644630d0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 17 Dec 2014 15:50:10 -0800 Subject: [PATCH 056/227] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #3137 (close requested by 'marmbrus') Closes #3362 (close requested by 'marmbrus') Closes #2979 (close requested by 'JoshRosen') Closes #2223 (close requested by 'JoshRosen') Closes #2998 (close requested by 'marmbrus') Closes #3202 (close requested by 'marmbrus') Closes #3079 (close requested by 'marmbrus') Closes #3210 (close requested by 'marmbrus') Closes #2764 (close requested by 'marmbrus') Closes #3618 (close requested by 'marmbrus') Closes #3501 (close requested by 'marmbrus') Closes #2768 (close requested by 'marmbrus') Closes #3381 (close requested by 'marmbrus') Closes #3510 (close requested by 'marmbrus') Closes #3703 (close requested by 'marmbrus') Closes #2543 (close requested by 'marmbrus') Closes #2876 (close requested by 'marmbrus') Closes #1281 (close requested by 'JoshRosen') From 3cd516191baadf8496ccdae499771020e89acd7e Mon Sep 17 00:00:00 2001 From: lewuathe Date: Wed, 17 Dec 2014 17:31:24 -0800 Subject: [PATCH 057/227] [SPARK-4822] Use sphinx tags for Python doc annotations Modify python annotations for sphinx. There is no change to build process from. https://github.com/apache/spark/blob/master/docs/README.md Author: lewuathe Closes #3685 from Lewuathe/sphinx-tag-for-pydoc and squashes the following commits: 88a0fd9 [lewuathe] [SPARK-4822] Fix DevelopApi and WARN tags 3d7a398 [lewuathe] [SPARK-4822] Use sphinx tags for Python doc annotations --- python/pyspark/context.py | 4 ++-- python/pyspark/mllib/classification.py | 4 ++-- python/pyspark/mllib/feature.py | 12 ++++++------ python/pyspark/mllib/stat.py | 4 ++-- python/pyspark/rdd.py | 8 ++++---- python/pyspark/sql.py | 2 +- 6 files changed, 17 insertions(+), 17 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index ed7351d60cff2..593d74bca5fff 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -407,7 +407,7 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): def binaryFiles(self, path, minPartitions=None): """ - :: Experimental :: + .. note:: Experimental Read a directory of binary files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI @@ -424,7 +424,7 @@ def binaryFiles(self, path, minPartitions=None): def binaryRecords(self, path, recordLength): """ - :: Experimental :: + .. note:: Experimental Load data from a flat binary file, assuming each record is a set of numbers with the specified numerical format (see ByteBuffer), and the number of diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index f14d0ed11cbbb..00e2e76711e84 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -41,7 +41,7 @@ def __init__(self, weights, intercept): def setThreshold(self, value): """ - :: Experimental :: + .. note:: Experimental Sets the threshold that separates positive predictions from negative predictions. An example with prediction score greater than or equal @@ -51,7 +51,7 @@ def setThreshold(self, value): def clearThreshold(self): """ - :: Experimental :: + .. note:: Experimental Clears the threshold so that `predict` will output raw prediction scores. """ diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index e46af208866a2..10df6288065b8 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -36,7 +36,7 @@ class VectorTransformer(object): """ - :: DeveloperApi :: + .. note:: DeveloperApi Base class for transformation of a vector or RDD of vector """ @@ -51,7 +51,7 @@ def transform(self, vector): class Normalizer(VectorTransformer): """ - :: Experimental :: + .. note:: Experimental Normalizes samples individually to unit L\ :sup:`p`\ norm @@ -112,7 +112,7 @@ def transform(self, vector): class StandardScalerModel(JavaVectorTransformer): """ - :: Experimental :: + .. note:: Experimental Represents a StandardScaler model that can transform vectors. """ @@ -129,7 +129,7 @@ def transform(self, vector): class StandardScaler(object): """ - :: Experimental :: + .. note:: Experimental Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the @@ -172,7 +172,7 @@ def fit(self, dataset): class HashingTF(object): """ - :: Experimental :: + .. note:: Experimental Maps a sequence of terms to their term frequencies using the hashing trick. @@ -232,7 +232,7 @@ def transform(self, x): class IDF(object): """ - :: Experimental :: + .. note:: Experimental Inverse document frequency (IDF). diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 1980f5b03f430..c8af777a8b00d 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -55,7 +55,7 @@ def min(self): class ChiSqTestResult(JavaModelWrapper): """ - :: Experimental :: + .. note:: Experimental Object containing the test results for the chi-squared hypothesis test. """ @@ -200,7 +200,7 @@ def corr(x, y=None, method=None): @staticmethod def chiSqTest(observed, expected=None): """ - :: Experimental :: + .. note:: Experimental If `observed` is Vector, conduct Pearson's chi-squared goodness of fit test of the observed data against the expected distribution, diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index bd2ff00c0f1be..c1120cf781e5e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1964,7 +1964,7 @@ def _to_java_object_rdd(self): def countApprox(self, timeout, confidence=0.95): """ - :: Experimental :: + .. note:: Experimental Approximate version of count() that returns a potentially incomplete result within a timeout, even if not all tasks have finished. @@ -1977,7 +1977,7 @@ def countApprox(self, timeout, confidence=0.95): def sumApprox(self, timeout, confidence=0.95): """ - :: Experimental :: + .. note:: Experimental Approximate operation to return the sum within a timeout or meet the confidence. @@ -1993,7 +1993,7 @@ def sumApprox(self, timeout, confidence=0.95): def meanApprox(self, timeout, confidence=0.95): """ - :: Experimental :: + .. note:: Experimental Approximate operation to return the mean within a timeout or meet the confidence. @@ -2009,7 +2009,7 @@ def meanApprox(self, timeout, confidence=0.95): def countApproxDistinct(self, relativeSD=0.05): """ - :: Experimental :: + .. note:: Experimental Return approximate number of distinct elements in the RDD. The algorithm used is based on streamlib's implementation of diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 1ee0b28a3256d..469f82473af97 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -420,7 +420,7 @@ def fromJson(cls, json): class UserDefinedType(DataType): """ - :: WARN: Spark Internal Use Only :: + .. note:: WARN: Spark Internal Use Only SQL User-Defined Type (UDT). """ From 3b764699ffc9c74b9597c855a0e8c04ac24fa3b7 Mon Sep 17 00:00:00 2001 From: Zhan Zhang Date: Thu, 18 Dec 2014 10:01:46 -0600 Subject: [PATCH 058/227] [SPARK-4461][YARN] pass extra java options to yarn application master Currently, there is no way to pass yarn am specific java options. It cause some potential issues when reading classpath from hadoop configuration file. Hadoop configuration actually replace variables in its property with the system property passed in java options. How to specify the value depends on different hadoop distribution. The new options are SPARK_YARN_JAVA_OPTS or spark.yarn.extraJavaOptions. I make it as spark global level, because typically we don't want user to specify this in their command line each time submitting spark job after it is setup in spark-defaults.conf. In addition, with this new extra options enabled to be passed to AM, it provides more flexibility. For example int the following valid mapred-site.xml file, we have the class path which specify values using system property. Hadoop can correctly handle it because it has java options passed in. This is the example, currently spark will break due to hadoop.version is not passed in. mapreduce.application.classpath /etc/hadoop/${hadoop.version}/mapreduce/* In the meantime, we cannot relies on mapreduce.admin.map.child.java.opts in mapred-site.xml, because it has its own extra java options specified, which does not apply to Spark. Author: Zhan Zhang Closes #3409 from zhzhan/Spark-4461 and squashes the following commits: daec3d0 [Zhan Zhang] solve review comments 08f44a7 [Zhan Zhang] add warning in driver mode if spark.yarn.am.extraJavaOptions is configured 5a505d3 [Zhan Zhang] solve review comments 4ed43ad [Zhan Zhang] solve review comments ad777ed [Zhan Zhang] Merge branch 'master' into Spark-4461 3e9e574 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark e3f9abe [Zhan Zhang] solve review comments 8963552 [Zhan Zhang] rebase f8f6700 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark dea1692 [Zhan Zhang] change the option key name to client mode specific 90d5dff [Zhan Zhang] rebase 8ac9254 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 092a25f [Zhan Zhang] solve review comments bc5a9ae [Zhan Zhang] solve review comments 782b014 [Zhan Zhang] add new configuration to docs/running-on-yarn.md and remove it from spark-defaults.conf.template 6faaa97 [Zhan Zhang] solve review comments 369863f [Zhan Zhang] clean up unnecessary var 733de9c [Zhan Zhang] Merge branch 'master' into Spark-4461 a68e7f0 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 864505a [Zhan Zhang] Add extra java options to be passed to Yarn application master 15830fc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 685d911 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 03ebad3 [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark 46d9e3d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark ebb213a [Zhan Zhang] revert b983ef3 [Zhan Zhang] test c4efb9b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 779d67b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 4daae6d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 12e1be5 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark ce0ca7b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 93f3081 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 3764505 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark a9d372b [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark a00f60f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 497b0f4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 4a2e36d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark a72c0d4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 301eb4a [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark cedcc6f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark adf4924 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark d10bf00 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 7e0cc36 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 68deb11 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 3ee3b2b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 2b0d513 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 1ccd7cc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark af9feb9 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark e4c1982 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 921e914 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 789ea21 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark cb53a2c [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark f6a8a40 [Zhan Zhang] revert ba14f28 [Zhan Zhang] test --- docs/running-on-yarn.md | 8 ++++++++ .../apache/spark/deploy/yarn/ClientBase.scala | 19 +++++++++++++++++++ 2 files changed, 27 insertions(+) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 62b317129b72a..b5fb077441468 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -139,6 +139,14 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The maximum number of threads to use in the application master for launching executor containers. + + spark.yarn.am.extraJavaOptions + (none) + + A string of extra JVM options to pass to the Yarn ApplicationMaster in client mode. + In cluster mode, use spark.driver.extraJavaOptions instead. + + # Launching Spark on YARN diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index f95d72379171c..5f0c67f05c9dd 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -352,12 +352,31 @@ private[spark] trait ClientBase extends Logging { if (isLaunchingDriver) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) + .map(Utils.splitCommandString).getOrElse(Seq.empty) .foreach(opts => javaOpts += opts) val libraryPaths = Seq(sys.props.get("spark.driver.extraLibraryPath"), sys.props.get("spark.driver.libraryPath")).flatten if (libraryPaths.nonEmpty) { prefixEnv = Some(Utils.libraryPathEnvPrefix(libraryPaths)) } + if (sparkConf.getOption("spark.yarn.am.extraJavaOptions").isDefined) { + logWarning("spark.yarn.am.extraJavaOptions will not take effect in cluster mode") + } + } else { + // Validate and include yarn am specific java options in yarn-client mode. + val amOptsKey = "spark.yarn.am.extraJavaOptions" + val amOpts = sparkConf.getOption(amOptsKey) + amOpts.foreach { opts => + if (opts.contains("-Dspark")) { + val msg = s"$amOptsKey is not allowed to set Spark options (was '$opts'). " + throw new SparkException(msg) + } + if (opts.contains("-Xmx") || opts.contains("-Xms")) { + val msg = s"$amOptsKey is not allowed to alter memory settings (was '$opts')." + throw new SparkException(msg) + } + javaOpts ++= Utils.splitCommandString(opts) + } } // For log4j configuration to reference From 253b72b56fe908bbab5d621eae8a5f359c639dfd Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 18 Dec 2014 12:19:07 -0600 Subject: [PATCH 059/227] SPARK-3779. yarn spark.yarn.applicationMaster.waitTries config should be... ... changed to a time period Author: Sandy Ryza Closes #3471 from sryza/sandy-spark-3779 and squashes the following commits: 20b9887 [Sandy Ryza] Deprecate old property 42b5df7 [Sandy Ryza] Review feedback 9a959a1 [Sandy Ryza] SPARK-3779. yarn spark.yarn.applicationMaster.waitTries config should be changed to a time period --- docs/running-on-yarn.md | 8 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 48 +++++++++---------- 2 files changed, 29 insertions(+), 27 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b5fb077441468..86276b1aa9cea 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -22,10 +22,12 @@ Most of the configs are the same for Spark on YARN as for other deployment modes - - + + diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 987b3373fb8ff..dc7a078446324 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -329,43 +329,43 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def waitForSparkContextInitialized(): SparkContext = { logInfo("Waiting for spark context initialization") - try { - sparkContextRef.synchronized { - var count = 0 - val waitTime = 10000L - val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10) - while (sparkContextRef.get() == null && count < numTries && !finished) { - logInfo("Waiting for spark context initialization ... " + count) - count = count + 1 - sparkContextRef.wait(waitTime) - } + sparkContextRef.synchronized { + val waitTries = sparkConf.getOption("spark.yarn.applicationMaster.waitTries") + .map(_.toLong * 10000L) + if (waitTries.isDefined) { + logWarning( + "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") + } + val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", waitTries.getOrElse(100000L)) + val deadline = System.currentTimeMillis() + totalWaitTime - val sparkContext = sparkContextRef.get() - if (sparkContext == null) { - logError(("SparkContext did not initialize after waiting for %d ms. Please check earlier" - + " log output for errors. Failing the application.").format(numTries * waitTime)) - } - sparkContext + while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { + logInfo("Waiting for spark context initialization ... ") + sparkContextRef.wait(10000L) + } + + val sparkContext = sparkContextRef.get() + if (sparkContext == null) { + logError(("SparkContext did not initialize after waiting for %d ms. Please check earlier" + + " log output for errors. Failing the application.").format(totalWaitTime)) } + sparkContext } } private def waitForSparkDriver(): ActorRef = { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false - var count = 0 val hostport = args.userArgs(0) val (driverHost, driverPort) = Utils.parseHostPort(hostport) - // spark driver should already be up since it launched us, but we don't want to + // Spark driver should already be up since it launched us, but we don't want to // wait forever, so wait 100 seconds max to match the cluster mode setting. - // Leave this config unpublished for now. SPARK-3779 to investigating changing - // this config to be time based. - val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 1000) + val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", 100000L) + val deadline = System.currentTimeMillis + totalWaitTime - while (!driverUp && !finished && count < numTries) { + while (!driverUp && !finished && System.currentTimeMillis < deadline) { try { - count = count + 1 val socket = new Socket(driverHost, driverPort) socket.close() logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) @@ -374,7 +374,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, case e: Exception => logError("Failed to connect to driver at %s:%s, retrying ...". format(driverHost, driverPort)) - Thread.sleep(100) + Thread.sleep(100L) } } From d9956f86ad7a937c5f2cfe39eacdcbdad9356c30 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Thu, 18 Dec 2014 12:15:53 -0800 Subject: [PATCH 060/227] Add mesos specific configurations into doc Author: Timothy Chen Closes #3349 from tnachen/mesos_doc and squashes the following commits: 737ef49 [Timothy Chen] Add TOC 5ca546a [Timothy Chen] Update description around cores requested. 26283a5 [Timothy Chen] Add mesos specific configurations into doc --- docs/running-on-mesos.md | 45 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 1073abb202c56..78358499fd01f 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -2,6 +2,8 @@ layout: global title: Running Spark on Mesos --- +* This will become a table of contents (this text will be scraped). +{:toc} Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache.org/). @@ -183,6 +185,49 @@ node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). In either case, HDFS runs separately from Hadoop MapReduce, without being scheduled through Mesos. +# Configuration + +See the [configuration page](configuration.html) for information on Spark configurations. The following configs are specific for Spark on Mesos. + +#### Spark Properties + +
Property NameDefaultMeaning
spark.yarn.applicationMaster.waitTries10spark.yarn.am.waitTime100000 - Set the number of times the ApplicationMaster waits for the the Spark master and then also the number of tries it waits for the SparkContext to be initialized + In yarn-cluster mode, time in milliseconds for the application master to wait for the + SparkContext to be initialized. In yarn-client mode, time for the application master to wait + for the driver to connect to it.
+ + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.mesos.coarsefalse + Set the run mode for Spark on Mesos. For more information about the run mode, refer to #Mesos Run Mode section above. +
spark.mesos.extra.cores0 + Set the extra amount of cpus to request per task. This setting is only used for Mesos coarse grain mode. + The total amount of cores requested per task is the number of cores in the offer plus the extra cores configured. + Note that total amount of cores the executor will request in total will not exceed the spark.cores.max setting. +
spark.mesos.executor.homeSPARK_HOME + The location where the mesos executor will look for Spark binaries to execute, and uses the SPARK_HOME setting on default. + This variable is only used when no spark.executor.uri is provided, and assumes Spark is installed on the specified location + on each slave. +
spark.mesos.executor.memoryOverhead384 + The amount of memory that Mesos executor will request for the task to account for the overhead of running the executor itself. + The final total amount of memory allocated is the maximum value between executor memory plus memoryOverhead, and overhead fraction (1.07) plus the executor memory. +
+ # Troubleshooting and Debugging A few places to look during debugging: From 3720057b8e7c15c2c0464b5bb7243bc22323f4e8 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 18 Dec 2014 12:53:18 -0800 Subject: [PATCH 061/227] [SPARK-3607] ConnectionManager threads.max configs on the thread pools don't work Hi all - cleaned up the code to get rid of the unused parameter and added some discussion of the ThreadPoolExecutor parameters to explain why we can use a single threadCount instead of providing a min/max. Author: Ilya Ganelin Closes #3664 from ilganeli/SPARK-3607C and squashes the following commits: 3c05690 [Ilya Ganelin] Updated documentation and refactored code to extract shared variables --- .../spark/network/nio/ConnectionManager.scala | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index df4b085d2251e..243b71c980864 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -83,9 +83,21 @@ private[nio] class ConnectionManager( private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) + // Get the thread counts from the Spark Configuration. + // + // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value, + // we only query for the minimum value because we are using LinkedBlockingDeque. + // + // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is + // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min" + // parameter is necessary. + private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20) + private val ioThreadCount = conf.getInt("spark.core.connection.io.threads.min", 4) + private val connectThreadCount = conf.getInt("spark.core.connection.connect.threads.min", 1) + private val handleMessageExecutor = new ThreadPoolExecutor( - conf.getInt("spark.core.connection.handler.threads.min", 20), - conf.getInt("spark.core.connection.handler.threads.max", 60), + handlerThreadCount, + handlerThreadCount, conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), Utils.namedThreadFactory("handle-message-executor")) { @@ -96,12 +108,11 @@ private[nio] class ConnectionManager( logError("Error in handleMessageExecutor is not handled properly", t) } } - } private val handleReadWriteExecutor = new ThreadPoolExecutor( - conf.getInt("spark.core.connection.io.threads.min", 4), - conf.getInt("spark.core.connection.io.threads.max", 32), + ioThreadCount, + ioThreadCount, conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), Utils.namedThreadFactory("handle-read-write-executor")) { @@ -112,14 +123,13 @@ private[nio] class ConnectionManager( logError("Error in handleReadWriteExecutor is not handled properly", t) } } - } // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : // which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - conf.getInt("spark.core.connection.connect.threads.min", 1), - conf.getInt("spark.core.connection.connect.threads.max", 8), + connectThreadCount, + connectThreadCount, conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), Utils.namedThreadFactory("handle-connect-executor")) { @@ -130,7 +140,6 @@ private[nio] class ConnectionManager( logError("Error in handleConnectExecutor is not handled properly", t) } } - } private val serverChannel = ServerSocketChannel.open() From 59a49db5982ecc487187fcd92399e08b4b4bea64 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 18 Dec 2014 13:55:49 -0800 Subject: [PATCH 062/227] [SPARK-4887][MLlib] Fix a bad unittest in LogisticRegressionSuite The original test doesn't make sense since if you step in, the lossSum is already NaN, and the coefficients are diverging. That's because the step size is too large for SGD, so it doesn't work. The correct behavior is that you should get smaller coefficients than the one without regularization. Comparing the values using 20000.0 relative error doesn't make sense as well. Author: DB Tsai Closes #3735 from dbtsai/mlortestfix and squashes the following commits: b1a3c42 [DB Tsai] first commit --- .../mllib/classification/LogisticRegressionSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 4e812994405b3..94b0e00f37267 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -178,15 +178,16 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M // Use half as many iterations as the previous test. val lr = new LogisticRegressionWithSGD().setIntercept(true) lr.optimizer. - setStepSize(10.0). + setStepSize(1.0). setNumIterations(10). setRegParam(1.0) val model = lr.run(testRDD, initialWeights) // Test the weights - assert(model.weights(0) ~== -430000.0 relTol 20000.0) - assert(model.intercept ~== 370000.0 relTol 20000.0) + // With regularization, the resulting weights will be smaller. + assert(model.weights(0) ~== -0.14 relTol 0.02) + assert(model.intercept ~== 0.25 relTol 0.02) val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) val validationRDD = sc.parallelize(validationData, 2) From a7ed6f3cc537f57de87d28e8466ca88fbfff53b5 Mon Sep 17 00:00:00 2001 From: Ernest Date: Thu, 18 Dec 2014 15:42:26 -0800 Subject: [PATCH 063/227] [SPARK-4880] remove spark.locality.wait in Analytics spark.locality.wait set to 100000 in examples/graphx/Analytics.scala. Should be left to the user. Author: Ernest Closes #3730 from Earne/SPARK-4880 and squashes the following commits: d79ed04 [Ernest] remove spark.locality.wait in Analytics --- .../main/scala/org/apache/spark/examples/graphx/Analytics.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 828cffb01ca1e..9cc47327f9a48 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -46,7 +46,7 @@ object Analytics extends Logging { } val options = mutable.Map(optionsList: _*) - val conf = new SparkConf().set("spark.locality.wait", "100000") + val conf = new SparkConf() GraphXUtils.registerKryoClasses(conf) val numEPart = options.remove("numEPart").map(_.toInt).getOrElse { From d5a596d4188bfa85ff49ee85039f54255c19a4de Mon Sep 17 00:00:00 2001 From: Madhu Siddalingaiah Date: Thu, 18 Dec 2014 16:00:53 -0800 Subject: [PATCH 064/227] [SPARK-4884]: Improve Partition docs Rewording was based on this discussion: http://apache-spark-developers-list.1001551.n3.nabble.com/RDD-data-flow-td9804.html This is the associated JIRA ticket: https://issues.apache.org/jira/browse/SPARK-4884 Author: Madhu Siddalingaiah Closes #3722 from msiddalingaiah/master and squashes the following commits: 79e679f [Madhu Siddalingaiah] [DOC]: improve documentation 51d14b9 [Madhu Siddalingaiah] Merge remote-tracking branch 'upstream/master' 38faca4 [Madhu Siddalingaiah] Merge remote-tracking branch 'upstream/master' cbccbfe [Madhu Siddalingaiah] Documentation: replace with (again) 332f7a2 [Madhu Siddalingaiah] Documentation: replace with cd2b05a [Madhu Siddalingaiah] Merge remote-tracking branch 'upstream/master' 0fc12d7 [Madhu Siddalingaiah] Documentation: add description for repartitionAndSortWithinPartitions --- core/src/main/scala/org/apache/spark/Partition.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala index 27892dbd2a0bc..dd3f28e4197e3 100644 --- a/core/src/main/scala/org/apache/spark/Partition.scala +++ b/core/src/main/scala/org/apache/spark/Partition.scala @@ -18,11 +18,11 @@ package org.apache.spark /** - * A partition of an RDD. + * An identifier for a partition in an RDD. */ trait Partition extends Serializable { /** - * Get the split's index within its parent RDD + * Get the partition's index within its parent RDD */ def index: Int From f9f58b9a01c4c7eaf0ce5055d6870e69a22297e3 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 18 Dec 2014 16:29:36 -0800 Subject: [PATCH 065/227] SPARK-4743 - Use SparkEnv.serializer instead of closureSerializer in aggregateByKey and foldByKey Author: Ivan Vergiliev Closes #3605 from IvanVergiliev/change-serializer and squashes the following commits: a49b7cf [Ivan Vergiliev] Use serializer instead of closureSerializer in aggregate/foldByKey. --- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index b0434c9a3b6b3..fe3129b62f71e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -123,11 +123,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def aggregateByKey[U: ClassTag](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) => U, combOp: (U, U) => U): RDD[(K, U)] = { // Serialize the zero value to a byte array so that we can get a new clone of it on each key - val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) zeroBuffer.get(zeroArray) - lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + lazy val cachedSerializer = SparkEnv.get.serializer.newInstance() val createZero = () => cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray)) combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner) @@ -168,12 +168,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { // Serialize the zero value to a byte array so that we can get a new clone of it on each key - val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) zeroBuffer.get(zeroArray) // When deserializing, use a lazy val to create just one instance of the serializer per task - lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + lazy val cachedSerializer = SparkEnv.get.serializer.newInstance() val createZero = () => cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) From 105293a7d06b26e7b179a0447eb802074ee9c218 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 18 Dec 2014 16:43:16 -0800 Subject: [PATCH 066/227] [SPARK-4837] NettyBlockTransferService should use spark.blockManager.port config This is used in NioBlockTransferService here: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala#L66 Author: Aaron Davidson Closes #3688 from aarondav/SPARK-4837 and squashes the following commits: ebd2007 [Aaron Davidson] [SPARK-4837] NettyBlockTransferService should use spark.blockManager.port config --- .../apache/spark/network/netty/NettyBlockTransferService.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 0027cbb0ff1fb..3f0950dae1f24 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -60,7 +60,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage } transportContext = new TransportContext(transportConf, rpcHandler) clientFactory = transportContext.createClientFactory(bootstrap.toList) - server = transportContext.createServer() + server = transportContext.createServer(conf.getInt("spark.blockManager.port", 0)) appId = conf.getAppId logInfo("Server created on " + server.getPort) } From 9804a759b68f56eceb8a2f4ea90f76a92b5f9f67 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 18 Dec 2014 17:37:42 -0800 Subject: [PATCH 067/227] [SPARK-4754] Refactor SparkContext into ExecutorAllocationClient This is such that the `ExecutorAllocationManager` does not take in the `SparkContext` with all of its dependencies as an argument. This prevents future developers of this class to tie down this class further with the `SparkContext`, which has really become quite a monstrous object. cc'ing pwendell who originally suggested this, and JoshRosen who may have thoughts about the trait mix-in style of `SparkContext`. Author: Andrew Or Closes #3614 from andrewor14/dynamic-allocation-sc and squashes the following commits: 187070d [Andrew Or] Merge branch 'master' of github.com:apache/spark into dynamic-allocation-sc 59baf6c [Andrew Or] Merge branch 'master' of github.com:apache/spark into dynamic-allocation-sc 347a348 [Andrew Or] Refactor SparkContext into ExecutorAllocationClient --- .../spark/ExecutorAllocationClient.scala | 42 +++++++++++++++++++ .../spark/ExecutorAllocationManager.scala | 14 ++++--- .../scala/org/apache/spark/SparkContext.scala | 10 ++--- .../CoarseGrainedSchedulerBackend.scala | 8 ++-- 4 files changed, 59 insertions(+), 15 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala new file mode 100644 index 0000000000000..a46a81eabd965 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -0,0 +1,42 @@ +/* + * 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 + +/** + * A client that communicates with the cluster manager to request or kill executors. + */ +private[spark] trait ExecutorAllocationClient { + + /** + * Request an additional number of executors from the cluster manager. + * Return whether the request is acknowledged by the cluster manager. + */ + def requestExecutors(numAdditionalExecutors: Int): Boolean + + /** + * Request that the cluster manager kill the specified executors. + * Return whether the request is acknowledged by the cluster manager. + */ + def killExecutors(executorIds: Seq[String]): Boolean + + /** + * Request that the cluster manager kill the specified executor. + * Return whether the request is acknowledged by the cluster manager. + */ + def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId)) +} diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 88adb892998af..e9e90e3f2f65a 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -60,11 +60,13 @@ import org.apache.spark.scheduler._ * spark.dynamicAllocation.executorIdleTimeout (K) - * If an executor has been idle for this duration, remove it */ -private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging { +private[spark] class ExecutorAllocationManager( + client: ExecutorAllocationClient, + listenerBus: LiveListenerBus, + conf: SparkConf) + extends Logging { import ExecutorAllocationManager._ - private val conf = sc.conf - // Lower and upper bounds on the number of executors. These are required. private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1) private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1) @@ -168,7 +170,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging * Register for scheduler callbacks to decide when to add and remove executors. */ def start(): Unit = { - sc.addSparkListener(listener) + listenerBus.addListener(listener) startPolling() } @@ -253,7 +255,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging val actualNumExecutorsToAdd = math.min(numExecutorsToAdd, maxNumExecutorsToAdd) val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd - val addRequestAcknowledged = testing || sc.requestExecutors(actualNumExecutorsToAdd) + val addRequestAcknowledged = testing || client.requestExecutors(actualNumExecutorsToAdd) if (addRequestAcknowledged) { logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " + s"tasks are backlogged (new desired total will be $newTotalExecutors)") @@ -295,7 +297,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging } // Send a request to the backend to kill this executor - val removeRequestAcknowledged = testing || sc.killExecutor(executorId) + val removeRequestAcknowledged = testing || client.killExecutor(executorId) if (removeRequestAcknowledged) { logInfo(s"Removing executor $executorId because it has been idle for " + s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})") diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 66e07d10132b8..b1c6a90d182f6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -64,7 +64,7 @@ import org.apache.spark.util._ * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ -class SparkContext(config: SparkConf) extends Logging { +class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationClient { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() @@ -365,7 +365,7 @@ class SparkContext(config: SparkConf) extends Logging { if (dynamicAllocationEnabled) { assert(master.contains("yarn") || dynamicAllocationTesting, "Dynamic allocation of executors is currently only supported in YARN mode") - Some(new ExecutorAllocationManager(this)) + Some(new ExecutorAllocationManager(this, listenerBus, conf)) } else { None } @@ -994,7 +994,7 @@ class SparkContext(config: SparkConf) extends Logging { * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def requestExecutors(numAdditionalExecutors: Int): Boolean = { + override def requestExecutors(numAdditionalExecutors: Int): Boolean = { assert(master.contains("yarn") || dynamicAllocationTesting, "Requesting executors is currently only supported in YARN mode") schedulerBackend match { @@ -1012,7 +1012,7 @@ class SparkContext(config: SparkConf) extends Logging { * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutors(executorIds: Seq[String]): Boolean = { + override def killExecutors(executorIds: Seq[String]): Boolean = { assert(master.contains("yarn") || dynamicAllocationTesting, "Killing executors is currently only supported in YARN mode") schedulerBackend match { @@ -1030,7 +1030,7 @@ class SparkContext(config: SparkConf) extends Logging { * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId)) + override def killExecutor(executorId: String): Boolean = super.killExecutor(executorId) /** The version of Spark on which this application is running. */ def version = SPARK_VERSION diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 29cd34429b881..fe9914b50bc54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,7 +27,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} +import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} @@ -42,7 +42,7 @@ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Ut */ private[spark] class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem) - extends SchedulerBackend with Logging + extends ExecutorAllocationClient with SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) @@ -307,7 +307,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * Request an additional number of executors from the cluster manager. * Return whether the request is acknowledged. */ - final def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { + final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") logDebug(s"Number of pending executors is now $numPendingExecutors") numPendingExecutors += numAdditionalExecutors @@ -334,7 +334,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * Request that the cluster manager kill the specified executors. * Return whether the kill request is acknowledged. */ - final def killExecutors(executorIds: Seq[String]): Boolean = synchronized { + final override def killExecutors(executorIds: Seq[String]): Boolean = synchronized { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") val filteredExecutorIds = new ArrayBuffer[String] executorIds.foreach { id => From f728e0fe7e860fe6dd3437e248472a67a2d435f8 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 18 Dec 2014 18:58:29 -0800 Subject: [PATCH 068/227] [SPARK-2663] [SQL] Support the Grouping Set Add support for `GROUPING SETS`, `ROLLUP`, `CUBE` and the the virtual column `GROUPING__ID`. More details on how to use the `GROUPING SETS" can be found at: https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation,+Cube,+Grouping+and+Rollup https://issues.apache.org/jira/secure/attachment/12676811/grouping_set.pdf The generic idea of the implementations are : 1 Replace the `ROLLUP`, `CUBE` with `GROUPING SETS` 2 Explode each of the input row, and then feed them to `Aggregate` * Each grouping set are represented as the bit mask for the `GroupBy Expression List`, for each bit, `1` means the expression is selected, otherwise `0` (left is the lower bit, and right is the higher bit in the `GroupBy Expression List`) * Several of projections are constructed according to the grouping sets, and within each projection(Seq[Expression), we replace those expressions with `Literal(null)` if it's not selected in the grouping set (based on the bit mask) * Output Schema of `Explode` is `child.output :+ grouping__id` * GroupBy Expressions of `Aggregate` is `GroupBy Expression List :+ grouping__id` * Keep the `Aggregation expressions` the same for the `Aggregate` The expressions substitutions happen in Logic Plan analyzing, so we will benefit from the Logical Plan optimization (e.g. expression constant folding, and map side aggregation etc.), Only an `Explosive` operator added for Physical Plan, which will explode the rows according the pre-set projections. A known issue will be done in the follow up PR: * Optimization `ColumnPruning` is not supported yet for `Explosive` node. Author: Cheng Hao Closes #1567 from chenghao-intel/grouping_sets and squashes the following commits: fe65fcc [Cheng Hao] Remove the extra space 3547056 [Cheng Hao] Add more doc and Simplify the Expand a7c869d [Cheng Hao] update code as feedbacks d23c672 [Cheng Hao] Add GroupingExpression to replace the Seq[Expression] 414b165 [Cheng Hao] revert the unnecessary changes ec276c6 [Cheng Hao] Support Rollup/Cube/GroupingSets --- .../sql/catalyst/analysis/Analyzer.scala | 95 +++++++++++++++++++ .../sql/catalyst/expressions/Expression.scala | 13 ++- .../expressions/namedExpressions.scala | 5 + .../plans/logical/basicOperators.scala | 83 ++++++++++++++++ .../apache/spark/sql/execution/Expand.scala | 79 +++++++++++++++ .../spark/sql/execution/SparkStrategies.scala | 2 + .../execution/HiveCompatibilitySuite.scala | 7 ++ .../org/apache/spark/sql/hive/HiveQl.scala | 79 +++++++++++++-- ...ing_id1-0-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...ing_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...ing_id1-1-c0f14def6a135cc50cba364e810ce28e | 0 ...ing_id1-2-3dfbff77a9b56d44312814d3fb0d07fa | 18 ++++ ...ing_id1-3-d113d984a30ad7b98c50a46158605a51 | 12 +++ ...ing_id1-4-903a499840102e2cb722dd8b83820391 | 18 ++++ ...ing_id2-0-20539c642c514a590caca2f11395007e | 0 ...ing_id2-1-d1f3215e349f056c8de60b87a6a9855e | 0 ...ing_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e | 0 ...ing_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 | 1 + ...ing_id2-3-b3cb95405b1200603f40aaab24be7586 | 11 +++ ...ing_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 | 3 + ...ing_id2-5-7dd97bda7e2a900dfc509a6133620b82 | 53 +++++++++++ ...ing_id2-6-a7dc16cb82c595b18d4258a38a304b1e | 1 + ...ing_id2-7-b3cb95405b1200603f40aaab24be7586 | 11 +++ ...ing_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 | 3 + ...ing_id2-9-7dd97bda7e2a900dfc509a6133620b82 | 53 +++++++++++ ...g_sets1-0-91128722f50ec00b51e0bf6fe5695cd1 | 0 ...g_sets1-1-3673d61d0944adeba77438d882839de4 | 0 ...g_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5 | 0 ...g_sets1-2-5c14fabebc5b4c526c459a6e867ec61a | 6 ++ ...g_sets1-3-9f501f293fe180bf6322e93d8dea025a | 15 +++ ...g_sets1-4-505ac6611b332d6cf4a364739075d49c | 15 +++ ...g_sets1-5-a33cc72bf5adee428eea079847034b62 | 11 +++ ...g_sets1-6-5f0c2e1d5489c867261e575625349542 | 14 +++ ...g_sets1-7-9b3bda02733476012e2cda434d936423 | 5 + ...g_sets1-8-1273ad6760f8c3ddad07819362dcc324 | 5 + ...g_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 | 1 + ...g_sets2-1-91128722f50ec00b51e0bf6fe5695cd1 | 0 ..._sets2-10-f39e49610430c91e5af3876d15fbdfe3 | 15 +++ ...g_sets2-2-3673d61d0944adeba77438d882839de4 | 0 ...g_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5 | 0 ...g_sets2-3-59adefab34d80e8e185b2ad03877d381 | 0 ...g_sets2-4-9f501f293fe180bf6322e93d8dea025a | 15 +++ ...g_sets2-5-1163c486fd7e2c4346805fb035e2f268 | 0 ...g_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc | 15 +++ ...g_sets2-7-39db88427f92cb770b6daa38610c04e7 | 0 ...g_sets2-8-e464ec3d5461bda47eac3d1ef8617786 | 0 ...g_sets2-9-74126b100714164d13cbb3bff436c2ff | 0 ...g_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc | 0 ...g_sets3-1-38373b67d392924967a4695689d2164e | 0 ...g_sets3-1-e8964b2aaeb388064c9fdac5ec687824 | 0 ...ng_sets3-2-c1942a377b1a440d4ed3dd05fed445d | 0 ...g_sets3-2-d95cf9dfae402d369f338b8516845e02 | 0 ...g_sets3-3-b89ea2173180c8ae423d856f943e061f | 1 + ...g_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 | 1 + ...g_sets3-5-6623f95d90d929a6d2c8171a0698d4fd | 0 ...g_sets3-6-aec59088408cc57248851d3ce04e2eef | 16 ++++ ...g_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 | 1 + ...g_sets3-8-f32c4a191759237733a10cd721b49966 | 0 ...g_sets3-9-aec59088408cc57248851d3ce04e2eef | 16 ++++ ...g_sets4-0-d8ae5a3e613dd2dda392995b90d47565 | 1 + ...g_sets4-1-6ad4e855adb49babfa3ae6abac190be3 | 1 + ...g_sets4-2-1789808269c8bd0f6259227f07da1a6a | 0 ...g_sets4-3-3673d61d0944adeba77438d882839de4 | 0 ...g_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5 | 0 ...g_sets4-4-b51d813d0c5a410d8cf9765d85005a01 | 0 ...g_sets4-5-9be98faf8588a3c8e7436f14c638e438 | 13 +++ ...g_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 | 1 + ...g_sets4-7-98f7522ce136cdffb0c14163e613c250 | 0 ...g_sets4-8-9be98faf8588a3c8e7436f14c638e438 | 13 +++ ...g_sets5-0-d8ae5a3e613dd2dda392995b90d47565 | 1 + ...g_sets5-1-6ad4e855adb49babfa3ae6abac190be3 | 1 + ...g_sets5-2-1789808269c8bd0f6259227f07da1a6a | 0 ...g_sets5-3-3673d61d0944adeba77438d882839de4 | 0 ...g_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5 | 0 ...g_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9 | 0 ...g_sets5-5-af3569757b9f52fb9b1ead33130e1b4f | 15 +++ ...g_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 | 1 + ...g_sets5-7-6544a382d851f916616c4386fdcf0ed8 | 0 ...g_sets5-8-af3569757b9f52fb9b1ead33130e1b4f | 15 +++ 79 files changed, 750 insertions(+), 11 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8 create mode 100644 sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3705fcc1f13f3..1c4088b8438e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.catalyst.types.IntegerType /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing @@ -56,6 +58,7 @@ class Analyzer(catalog: Catalog, Batch("Resolution", fixedPoint, ResolveReferences :: ResolveRelations :: + ResolveGroupingAnalytics :: ResolveSortReferences :: NewRelationInstances :: ImplicitGenerate :: @@ -102,6 +105,93 @@ class Analyzer(catalog: Catalog, } } + object ResolveGroupingAnalytics extends Rule[LogicalPlan] { + /** + * Extract attribute set according to the grouping id + * @param bitmask bitmask to represent the selected of the attribute sequence + * @param exprs the attributes in sequence + * @return the attributes of non selected specified via bitmask (with the bit set to 1) + */ + private def buildNonSelectExprSet(bitmask: Int, exprs: Seq[Expression]) + : OpenHashSet[Expression] = { + val set = new OpenHashSet[Expression](2) + + var bit = exprs.length - 1 + while (bit >= 0) { + if (((bitmask >> bit) & 1) == 0) set.add(exprs(bit)) + bit -= 1 + } + + set + } + + /* + * GROUP BY a, b, c, WITH ROLLUP + * is equivalent to + * GROUP BY a, b, c GROUPING SETS ( (a, b, c), (a, b), (a), ( )). + * Group Count: N + 1 (N is the number of group expression) + * + * We need to get all of its subsets for the rule described above, the subset is + * represented as the bit masks. + */ + def bitmasks(r: Rollup): Seq[Int] = { + Seq.tabulate(r.groupByExprs.length + 1)(idx => {(1 << idx) - 1}) + } + + /* + * GROUP BY a, b, c, WITH CUBE + * is equivalent to + * GROUP BY a, b, c GROUPING SETS ( (a, b, c), (a, b), (b, c), (a, c), (a), (b), (c), ( ) ). + * Group Count: 2^N (N is the number of group expression) + * + * We need to get all of its sub sets for a given GROUPBY expressions, the subset is + * represented as the bit masks. + */ + def bitmasks(c: Cube): Seq[Int] = { + Seq.tabulate(1 << c.groupByExprs.length)(i => i) + } + + /** + * Create an array of Projections for the child projection, and replace the projections' + * expressions which equal GroupBy expressions with Literal(null), if those expressions + * are not set for this grouping set (according to the bit mask). + */ + private[this] def expand(g: GroupingSets): Seq[GroupExpression] = { + val result = new scala.collection.mutable.ArrayBuffer[GroupExpression] + + g.bitmasks.foreach { bitmask => + // get the non selected grouping attributes according to the bit mask + val nonSelectedGroupExprSet = buildNonSelectExprSet(bitmask, g.groupByExprs) + + val substitution = (g.child.output :+ g.gid).map(expr => expr transformDown { + case x: Expression if nonSelectedGroupExprSet.contains(x) => + // if the input attribute in the Invalid Grouping Expression set of for this group + // replace it with constant null + Literal(null, expr.dataType) + case x if x == g.gid => + // replace the groupingId with concrete value (the bit mask) + Literal(bitmask, IntegerType) + }) + + result += GroupExpression(substitution) + } + + result.toSeq + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a: Cube if a.resolved => + GroupingSets(bitmasks(a), a.groupByExprs, a.child, a.aggregations, a.gid) + case a: Rollup if a.resolved => + GroupingSets(bitmasks(a), a.groupByExprs, a.child, a.aggregations, a.gid) + case x: GroupingSets if x.resolved => + Aggregate( + x.groupByExprs :+ x.gid, + x.aggregations, + Expand(expand(x), x.child.output :+ x.gid, x.child)) + } + } + /** * Checks for non-aggregated attributes with aggregation */ @@ -183,6 +273,11 @@ class Analyzer(catalog: Catalog, case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { + case u @ UnresolvedAttribute(name) + if resolver(name, VirtualColumn.groupingIdName) && + q.isInstanceOf[GroupingAnalytics] => + // Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics + q.asInstanceOf[GroupingAnalytics].gid case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = q.resolveChildren(name, resolver).getOrElse(u) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index bc45881e42748..ac5b02c2e6aed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -284,6 +284,17 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => +} - +// TODO Semantically we probably not need GroupExpression +// All we need is holding the Seq[Expression], and ONLY used in doing the +// expressions transformation correctly. Probably will be removed since it's +// not like a real expressions. +case class GroupExpression(children: Seq[Expression]) extends Expression { + self: Product => + type EvaluatedType = Seq[Any] + override def eval(input: Row): EvaluatedType = ??? + override def nullable = false + override def foldable = false + override def dataType = ??? } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 7634d392d4111..a3c300b5d90e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -187,3 +187,8 @@ case class AttributeReference( override def toString: String = s"$name#${exprId.id}$typeSuffix" } + +object VirtualColumn { + val groupingIdName = "grouping__id" + def newGroupingId = AttributeReference(groupingIdName, IntegerType, false)() +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 64b8d45ebbf42..a9282b98adfab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -143,6 +143,89 @@ case class Aggregate( override def output = aggregateExpressions.map(_.toAttribute) } +/** + * Apply the all of the GroupExpressions to every input row, hence we will get + * multiple output rows for a input row. + * @param projections The group of expressions, all of the group expressions should + * output the same schema specified by the parameter `output` + * @param output The output Schema + * @param child Child operator + */ +case class Expand( + projections: Seq[GroupExpression], + output: Seq[Attribute], + child: LogicalPlan) extends UnaryNode + +trait GroupingAnalytics extends UnaryNode { + self: Product => + def gid: AttributeReference + def groupByExprs: Seq[Expression] + def aggregations: Seq[NamedExpression] + + override def output = aggregations.map(_.toAttribute) +} + +/** + * A GROUP BY clause with GROUPING SETS can generate a result set equivalent + * to generated by a UNION ALL of multiple simple GROUP BY clauses. + * + * We will transform GROUPING SETS into logical plan Aggregate(.., Expand) in Analyzer + * @param bitmasks A list of bitmasks, each of the bitmask indicates the selected + * GroupBy expressions + * @param groupByExprs The Group By expressions candidates, take effective only if the + * associated bit in the bitmask set to 1. + * @param child Child operator + * @param aggregations The Aggregation expressions, those non selected group by expressions + * will be considered as constant null if it appears in the expressions + * @param gid The attribute represents the virtual column GROUPING__ID, and it's also + * the bitmask indicates the selected GroupBy Expressions for each + * aggregating output row. + * The associated output will be one of the value in `bitmasks` + */ +case class GroupingSets( + bitmasks: Seq[Int], + groupByExprs: Seq[Expression], + child: LogicalPlan, + aggregations: Seq[NamedExpression], + gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics + +/** + * Cube is a syntactic sugar for GROUPING SETS, and will be transformed to GroupingSets, + * and eventually will be transformed to Aggregate(.., Expand) in Analyzer + * + * @param groupByExprs The Group By expressions candidates. + * @param child Child operator + * @param aggregations The Aggregation expressions, those non selected group by expressions + * will be considered as constant null if it appears in the expressions + * @param gid The attribute represents the virtual column GROUPING__ID, and it's also + * the bitmask indicates the selected GroupBy Expressions for each + * aggregating output row. + */ +case class Cube( + groupByExprs: Seq[Expression], + child: LogicalPlan, + aggregations: Seq[NamedExpression], + gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics + +/** + * Rollup is a syntactic sugar for GROUPING SETS, and will be transformed to GroupingSets, + * and eventually will be transformed to Aggregate(.., Expand) in Analyzer + * + * @param groupByExprs The Group By expressions candidates, take effective only if the + * associated bit in the bitmask set to 1. + * @param child Child operator + * @param aggregations The Aggregation expressions, those non selected group by expressions + * will be considered as constant null if it appears in the expressions + * @param gid The attribute represents the virtual column GROUPING__ID, and it's also + * the bitmask indicates the selected GroupBy Expressions for each + * aggregating output row. + */ +case class Rollup( + groupByExprs: Seq[Expression], + child: LogicalPlan, + aggregations: Seq[NamedExpression], + gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics + case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala new file mode 100644 index 0000000000000..95172420608f9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala @@ -0,0 +1,79 @@ +/* + * 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.sql.execution + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{UnknownPartitioning, Partitioning} + +/** + * Apply the all of the GroupExpressions to every input row, hence we will get + * multiple output rows for a input row. + * @param projections The group of expressions, all of the group expressions should + * output the same schema specified bye the parameter `output` + * @param output The output Schema + * @param child Child operator + */ +@DeveloperApi +case class Expand( + projections: Seq[GroupExpression], + output: Seq[Attribute], + child: SparkPlan) + extends UnaryNode { + + // The GroupExpressions can output data with arbitrary partitioning, so set it + // as UNKNOWN partitioning + override def outputPartitioning: Partitioning = UnknownPartitioning(0) + + override def execute() = attachTree(this, "execute") { + child.execute().mapPartitions { iter => + // TODO Move out projection objects creation and transfer to + // workers via closure. However we can't assume the Projection + // is serializable because of the code gen, so we have to + // create the projections within each of the partition processing. + val groups = projections.map(ee => newProjection(ee.children, child.output)).toArray + + new Iterator[Row] { + private[this] var result: Row = _ + private[this] var idx = -1 // -1 means the initial state + private[this] var input: Row = _ + + override final def hasNext = (-1 < idx && idx < groups.length) || iter.hasNext + + override final def next(): Row = { + if (idx <= 0) { + // in the initial (-1) or beginning(0) of a new input row, fetch the next input tuple + input = iter.next() + idx = 0 + } + + result = groups(idx)(input) + idx += 1 + + if (idx == groups.length && iter.hasNext) { + idx = 0 + } + + result + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 1225d18857af2..6e04f26c84259 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -270,6 +270,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil + case logical.Expand(projections, output, child) => + execution.Expand(projections, output, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 4302da5bfc7fc..1e44dd239458a 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -403,6 +403,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby11", "groupby12", "groupby1_limit", + "groupby_grouping_id1", + "groupby_grouping_id2", + "groupby_grouping_sets1", + "groupby_grouping_sets2", + "groupby_grouping_sets3", + "groupby_grouping_sets4", + "groupby_grouping_sets5", "groupby1_map", "groupby1_map_nomap", "groupby1_map_skew", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index cd4e5a239ec66..5939276f6d65e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -393,6 +393,42 @@ private[hive] object HiveQl { (db, tableName) } + /** + * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) + * is equivalent to + * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 + * Check the following link for details. + * +https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup + * + * The bitmask denotes the grouping expressions validity for a grouping set, + * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) + * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of + * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. + */ + protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { + val (keyASTs, setASTs) = children.partition( n => n match { + case Token("TOK_GROUPING_SETS_EXPRESSION", children) => false // grouping sets + case _ => true // grouping keys + }) + + val keys = keyASTs.map(nodeToExpr).toSeq + val keyMap = keyASTs.map(_.toStringTree).zipWithIndex.toMap + + val bitmasks: Seq[Int] = setASTs.map(set => set match { + case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 + case Token("TOK_GROUPING_SETS_EXPRESSION", children) => + children.foldLeft(0)((bitmap, col) => { + val colString = col.asInstanceOf[ASTNode].toStringTree() + require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list") + bitmap | 1 << keyMap(colString) + }) + case _ => sys.error("Expect GROUPING SETS clause") + }) + + (keys, bitmasks) + } + protected def nodeToPlan(node: Node): LogicalPlan = node match { // Special drop table that also uncaches. case Token("TOK_DROPTABLE", @@ -520,6 +556,9 @@ private[hive] object HiveQl { selectDistinctClause :: whereClause :: groupByClause :: + rollupGroupByClause :: + cubeGroupByClause :: + groupingSetsClause :: orderByClause :: havingClause :: sortByClause :: @@ -535,6 +574,9 @@ private[hive] object HiveQl { "TOK_SELECTDI", "TOK_WHERE", "TOK_GROUPBY", + "TOK_ROLLUP_GROUPBY", + "TOK_CUBE_GROUPBY", + "TOK_GROUPING_SETS", "TOK_ORDERBY", "TOK_HAVING", "TOK_SORTBY", @@ -603,16 +645,33 @@ private[hive] object HiveQl { // The projection of the query can either be a normal projection, an aggregation // (if there is a group by) or a script transformation. - val withProject = transformation.getOrElse { - // Not a transformation so must be either project or aggregation. - val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr)) - - groupByClause match { - case Some(groupBy) => - Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView) - case None => - Project(selectExpressions, withLateralView) - } + val withProject: LogicalPlan = transformation.getOrElse { + val selectExpressions = + nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq) + Seq( + groupByClause.map(e => e match { + case Token("TOK_GROUPBY", children) => + // Not a transformation so must be either project or aggregation. + Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView) + case _ => sys.error("Expect GROUP BY") + }), + groupingSetsClause.map(e => e match { + case Token("TOK_GROUPING_SETS", children) => + val(groupByExprs, masks) = extractGroupingSet(children) + GroupingSets(masks, groupByExprs, withLateralView, selectExpressions) + case _ => sys.error("Expect GROUPING SETS") + }), + rollupGroupByClause.map(e => e match { + case Token("TOK_ROLLUP_GROUPBY", children) => + Rollup(children.map(nodeToExpr), withLateralView, selectExpressions) + case _ => sys.error("Expect WITH ROLLUP") + }), + cubeGroupByClause.map(e => e match { + case Token("TOK_CUBE_GROUPBY", children) => + Cube(children.map(nodeToExpr), withLateralView, selectExpressions) + case _ => sys.error("Expect WITH CUBE") + }), + Some(Project(selectExpressions, withLateralView))).flatten.head } val withDistinct = diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e b/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa b/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa new file mode 100644 index 0000000000000..76280c6f3a1c8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa @@ -0,0 +1,18 @@ +NULL NULL 0 +NULL 11 2 +NULL 12 2 +NULL 13 2 +NULL 17 2 +NULL 18 2 +NULL 28 2 +1 NULL 1 +1 11 3 +2 NULL 1 +2 12 3 +3 NULL 1 +3 13 3 +7 NULL 1 +7 17 3 +8 NULL 1 +8 18 3 +8 28 3 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51 new file mode 100644 index 0000000000000..b18af4e5dd637 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51 @@ -0,0 +1,12 @@ +0 NULL NULL +1 1 NULL +3 1 11 +1 2 NULL +3 2 12 +1 3 NULL +3 3 13 +1 7 NULL +3 7 17 +1 8 NULL +3 8 18 +3 8 28 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391 new file mode 100644 index 0000000000000..5a7ac193cb11b --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391 @@ -0,0 +1,18 @@ +NULL NULL 0 0 +NULL 11 2 2 +NULL 12 2 2 +NULL 13 2 2 +NULL 17 2 2 +NULL 18 2 2 +NULL 28 2 2 +1 NULL 1 1 +1 11 3 3 +2 NULL 1 1 +2 12 3 3 +3 NULL 1 1 +3 13 3 3 +7 NULL 1 1 +7 17 3 3 +8 NULL 1 1 +8 18 3 3 +8 28 3 3 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586 new file mode 100644 index 0000000000000..66ac2d65ce245 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586 @@ -0,0 +1,11 @@ +NULL NULL 0 6 +1 NULL 1 2 +1 NULL 3 1 +1 1 3 1 +2 NULL 1 1 +2 2 3 1 +3 NULL 1 2 +3 NULL 3 1 +3 3 3 1 +4 NULL 1 1 +4 5 3 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 new file mode 100644 index 0000000000000..8c1e9630ebfd7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 @@ -0,0 +1,3 @@ +0 1 +1 4 +3 6 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82 new file mode 100644 index 0000000000000..42fc2290c2cba --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82 @@ -0,0 +1,53 @@ +0 0 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586 new file mode 100644 index 0000000000000..66ac2d65ce245 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586 @@ -0,0 +1,11 @@ +NULL NULL 0 6 +1 NULL 1 2 +1 NULL 3 1 +1 1 3 1 +2 NULL 1 1 +2 2 3 1 +3 NULL 1 2 +3 NULL 3 1 +3 3 3 1 +4 NULL 1 1 +4 5 3 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 new file mode 100644 index 0000000000000..8c1e9630ebfd7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 @@ -0,0 +1,3 @@ +0 1 +1 4 +3 6 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82 new file mode 100644 index 0000000000000..42fc2290c2cba --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82 @@ -0,0 +1,53 @@ +0 0 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 +1 1 +1 1 +1 1 +1 1 +3 3 +3 3 +3 3 +3 3 +3 3 +3 3 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a new file mode 100644 index 0000000000000..7967c04c92149 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a @@ -0,0 +1,6 @@ +8 1 1 +5 2 2 +1 1 3 +2 2 4 +2 3 5 +3 2 8 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a new file mode 100644 index 0000000000000..da4a754efa0e0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a @@ -0,0 +1,15 @@ +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c new file mode 100644 index 0000000000000..da4a754efa0e0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c @@ -0,0 +1,15 @@ +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62 new file mode 100644 index 0000000000000..2d1b73d564955 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62 @@ -0,0 +1,11 @@ +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542 new file mode 100644 index 0000000000000..fe6c4d001180a --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542 @@ -0,0 +1,14 @@ +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL +1 +2 +3 +5 +8 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423 new file mode 100644 index 0000000000000..24d5fc2851703 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423 @@ -0,0 +1,5 @@ +1 +2 +3 +5 +8 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324 new file mode 100644 index 0000000000000..8ba0d6d25a6f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324 @@ -0,0 +1,5 @@ +2.0 1 +4.0 1 +5.0 2 +7.0 1 +9.0 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3 new file mode 100644 index 0000000000000..cabc9bb1d918f --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3 @@ -0,0 +1,15 @@ +NULL NULL 46 +NULL 1 8 +NULL 2 28 +NULL 3 10 +1 NULL 6 +1 1 6 +2 NULL 18 +2 2 8 +2 3 10 +3 NULL 16 +3 2 16 +5 NULL 4 +5 2 4 +8 NULL 2 +8 1 2 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a new file mode 100644 index 0000000000000..da4a754efa0e0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a @@ -0,0 +1,15 @@ +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc new file mode 100644 index 0000000000000..b20db4c79aa70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc @@ -0,0 +1,15 @@ +NULL NULL 23.0 +NULL 1 4.0 +NULL 2 14.0 +NULL 3 5.0 +1 NULL 3.0 +1 1 3.0 +2 NULL 9.0 +2 2 4.0 +2 3 5.0 +3 NULL 8.0 +3 2 8.0 +5 NULL 2.0 +5 2 2.0 +8 NULL 1.0 +8 1 1.0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef new file mode 100644 index 0000000000000..b2d08949e9795 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef @@ -0,0 +1,16 @@ +NULL NULL 3.8333333333333335 12 +NULL 1 2.0 5 +NULL 2 5.2 5 +NULL 3 5.0 2 +1 NULL 2.6666666666666665 3 +1 1 3.0 2 +1 2 2.0 1 +2 NULL 5.2 5 +2 2 5.333333333333333 3 +2 3 5.0 2 +3 NULL 8.0 1 +3 2 8.0 1 +5 NULL 2.0 1 +5 1 2.0 1 +8 NULL 1.0 2 +8 1 1.0 2 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef new file mode 100644 index 0000000000000..b2d08949e9795 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef @@ -0,0 +1,16 @@ +NULL NULL 3.8333333333333335 12 +NULL 1 2.0 5 +NULL 2 5.2 5 +NULL 3 5.0 2 +1 NULL 2.6666666666666665 3 +1 1 3.0 2 +1 2 2.0 1 +2 NULL 5.2 5 +2 2 5.333333333333333 3 +2 3 5.0 2 +3 NULL 8.0 1 +3 2 8.0 1 +5 NULL 2.0 1 +5 1 2.0 1 +8 NULL 1.0 2 +8 1 1.0 2 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438 new file mode 100644 index 0000000000000..33060f0d51729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438 @@ -0,0 +1,13 @@ +1 NULL 1 1 NULL 1 +1 NULL 1 1 1 1 +1 1 1 1 NULL 1 +1 1 1 1 1 1 +2 NULL 2 2 NULL 2 +2 NULL 2 2 2 1 +2 NULL 2 2 3 1 +2 2 1 2 NULL 2 +2 2 1 2 2 1 +2 2 1 2 3 1 +2 3 1 2 NULL 2 +2 3 1 2 2 1 +2 3 1 2 3 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438 new file mode 100644 index 0000000000000..33060f0d51729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438 @@ -0,0 +1,13 @@ +1 NULL 1 1 NULL 1 +1 NULL 1 1 1 1 +1 1 1 1 NULL 1 +1 1 1 1 1 1 +2 NULL 2 2 NULL 2 +2 NULL 2 2 2 1 +2 NULL 2 2 3 1 +2 2 1 2 NULL 2 +2 2 1 2 2 1 +2 2 1 2 3 1 +2 3 1 2 NULL 2 +2 3 1 2 2 1 +2 3 1 2 3 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f new file mode 100644 index 0000000000000..da4a754efa0e0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f @@ -0,0 +1,15 @@ +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f new file mode 100644 index 0000000000000..da4a754efa0e0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f @@ -0,0 +1,15 @@ +NULL NULL 6 +NULL 1 2 +NULL 2 3 +NULL 3 1 +1 NULL 1 +1 1 1 +2 NULL 2 +2 2 1 +2 3 1 +3 NULL 1 +3 2 1 +5 NULL 1 +5 2 1 +8 NULL 1 +8 1 1 From b68bc6d2647f8a5caf8aa558e4115f9cc254f67c Mon Sep 17 00:00:00 2001 From: Thu Kyaw Date: Thu, 18 Dec 2014 20:08:32 -0800 Subject: [PATCH 069/227] [SPARK-3928][SQL] Support wildcard matches on Parquet files. ...arquetFile accept hadoop glob pattern in path. Author: Thu Kyaw Closes #3407 from tkyaw/master and squashes the following commits: 19115ad [Thu Kyaw] Merge https://github.com/apache/spark ceded32 [Thu Kyaw] [SPARK-3928][SQL] Support wildcard matches on Parquet files. d322c28 [Thu Kyaw] [SPARK-3928][SQL] Support wildcard matches on Parquet files. ce677c6 [Thu Kyaw] [SPARK-3928][SQL] Support wildcard matches on Parquet files. --- .../spark/sql/api/java/JavaSQLContext.scala | 4 ++- .../sql/parquet/ParquetTableOperations.scala | 4 ++- .../spark/sql/parquet/ParquetTestData.scala | 36 +++++++++++++++++++ .../spark/sql/parquet/ParquetTypes.scala | 12 ++++--- .../spark/sql/parquet/ParquetQuerySuite.scala | 26 ++++++++++++++ 5 files changed, 76 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 4c0869e05b029..8884204e5079f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -133,7 +133,9 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { } /** - * Loads a parquet file, returning the result as a [[JavaSchemaRDD]]. + * Loads a parquet file from regular path or files that match file patterns in path, + * returning the result as a [[JavaSchemaRDD]]. + * Supported glob file pattern information at ([[http://tinyurl.com/kcqrzn8]]). */ def parquetFile(path: String): JavaSchemaRDD = new JavaSchemaRDD( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5a49384ade76f..96bace1769f71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -621,7 +621,9 @@ private[parquet] object FileSystemHelper { throw new IllegalArgumentException( s"ParquetTableOperations: path $path does not exist or is not a directory") } - fs.listStatus(path).map(_.getPath) + fs.globStatus(path) + .flatMap { status => if(status.isDir) fs.listStatus(status.getPath) else List(status) } + .map(_.getPath) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index c0918a40d136f..d5993656e0225 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -422,5 +422,41 @@ private[sql] object ParquetTestData { val first = reader.read() assert(first != null) } */ + + // to test golb pattern (wild card pattern matching for parquetFile input + val testGlobDir = Utils.createTempDir() + val testGlobSubDir1 = Utils.createTempDir(testGlobDir.getPath) + val testGlobSubDir2 = Utils.createTempDir(testGlobDir.getPath) + val testGlobSubDir3 = Utils.createTempDir(testGlobDir.getPath) + + def writeGlobFiles() = { + val subDirs = Array(testGlobSubDir1, testGlobSubDir2, testGlobSubDir3) + + subDirs.foreach { dir => + val path: Path = new Path(new Path(dir.toURI), new Path("part-r-0.parquet")) + val job = new Job() + val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) + + for(i <- 0 until 15) { + val record = new SimpleGroup(schema) + if(i % 3 == 0) { + record.add(0, true) + } else { + record.add(0, false) + } + if(i % 5 == 0) { + record.add(1, 5) + } + record.add(2, "abc") + record.add(3, i.toLong << 33) + record.add(4, 2.5F) + record.add(5, 4.5D) + writer.write(record) + } + writer.close() + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index fa37d1f2ae7e6..0e6fb57d57bca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -437,10 +437,14 @@ private[parquet] object ParquetTypesConverter extends Logging { } val path = origPath.makeQualified(fs) - val children = fs.listStatus(path).filterNot { status => - val name = status.getPath.getName - (name(0) == '.' || name(0) == '_') && name != ParquetFileWriter.PARQUET_METADATA_FILE - } + val children = + fs + .globStatus(path) + .flatMap { status => if(status.isDir) fs.listStatus(status.getPath) else List(status) } + .filterNot { status => + val name = status.getPath.getName + (name(0) == '.' || name(0) == '_') && name != ParquetFileWriter.PARQUET_METADATA_FILE + } ParquetRelation.enableLogForwarding() 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 0e5635d3e99dc..074855389d746 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 @@ -95,6 +95,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA ParquetTestData.writeNestedFile2() ParquetTestData.writeNestedFile3() ParquetTestData.writeNestedFile4() + ParquetTestData.writeGlobFiles() testRDD = parquetFile(ParquetTestData.testDir.toString) testRDD.registerTempTable("testsource") parquetFile(ParquetTestData.testFilterDir.toString) @@ -110,6 +111,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(ParquetTestData.testNestedDir2) Utils.deleteRecursively(ParquetTestData.testNestedDir3) Utils.deleteRecursively(ParquetTestData.testNestedDir4) + Utils.deleteRecursively(ParquetTestData.testGlobDir) // here we should also unregister the table?? setConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED, originalParquetFilterPushdownEnabled.toString) @@ -1049,4 +1051,28 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA "Comparison predicate with null shouldn't be pushed down") } } + + test("Import of simple Parquet files using glob wildcard pattern") { + val testGlobDir = ParquetTestData.testGlobDir.toString + val globPatterns = Array(testGlobDir + "/*/*", testGlobDir + "/spark-*/*", testGlobDir + "/?pa?k-*/*") + globPatterns.foreach { path => + val result = parquetFile(path).collect() + assert(result.size === 45) + result.zipWithIndex.foreach { + case (row, index) => { + val checkBoolean = + if ((index % 15) % 3 == 0) + row(0) == true + else + row(0) == false + assert(checkBoolean === true, s"boolean field value in line $index did not match") + if ((index % 15) % 5 == 0) assert(row(1) === 5, s"int field value in line $index did not match") + assert(row(2) === "abc", s"string field value in line $index did not match") + assert(row(3) === ((index.toLong % 15) << 33), s"long value in line $index did not match") + assert(row(4) === 2.5F, s"float field value in line $index did not match") + assert(row(5) === 4.5D, s"double field value in line $index did not match") + } + } + } + } } From 22ddb6e0338f4d101389a0655424a8fde6c4cff4 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Thu, 18 Dec 2014 20:10:23 -0800 Subject: [PATCH 070/227] [SPARK-4756][SQL] FIX: sessionToActivePool grow infinitely, even as sessions expire **sessionToActivePool** in **SparkSQLOperationManager** grow infinitely, even as sessions expire. we should remove the pool value when the session closed, even though **sessionToActivePool** would not exist in all of sessions. Author: guowei2 Closes #3617 from guowei2/SPARK-4756 and squashes the following commits: e9b97b8 [guowei2] fix compile bug with Shim12 cf0f521 [guowei2] Merge remote-tracking branch 'apache/master' into SPARK-4756 e070998 [guowei2] fix: remove active pool of the session when it expired --- .../sql/hive/thriftserver/SparkSQLSessionManager.scala | 9 ++++++++- .../thriftserver/server/SparkSQLOperationManager.scala | 3 +-- .../org/apache/spark/sql/hive/thriftserver/Shim12.scala | 6 +++--- .../org/apache/spark/sql/hive/thriftserver/Shim13.scala | 6 +++--- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 6b3275b4eaf04..89e9ede7261c9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -27,11 +27,14 @@ import org.apache.hive.service.cli.session.SessionManager import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager +import org.apache.hive.service.cli.SessionHandle private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) extends SessionManager with ReflectedCompositeService { + private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + override def init(hiveConf: HiveConf) { setSuperField(this, "hiveConf", hiveConf) @@ -40,10 +43,14 @@ private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) getAncestorField[Log](this, 3, "LOG").info( s"HiveServer2: Async execution pool size $backgroundPoolSize") - val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) setSuperField(this, "operationManager", sparkSqlOperationManager) addService(sparkSqlOperationManager) initCompositeService(hiveConf) } + + override def closeSession(sessionHandle: SessionHandle) { + super.closeSession(sessionHandle) + sparkSqlOperationManager.sessionToActivePool -= sessionHandle + } } 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 99c4f46a82b8e..9c0bf02391e0e 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 @@ -36,8 +36,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) val handleToOperation = ReflectionUtils .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") - // TODO: Currenlty this will grow infinitely, even as sessions expire - val sessionToActivePool = Map[HiveSession, String]() + val sessionToActivePool = Map[SessionHandle, String]() override def newExecuteStatementOperation( parentSession: HiveSession, diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index 9258ad0cdf1d0..5550183621fb6 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -69,7 +69,7 @@ private[hive] class SparkExecuteStatementOperation( statement: String, confOverlay: JMap[String, String])( hiveContext: HiveContext, - sessionToActivePool: SMap[HiveSession, String]) + sessionToActivePool: SMap[SessionHandle, String]) extends ExecuteStatementOperation(parentSession, statement, confOverlay) with Logging { private var result: SchemaRDD = _ @@ -191,14 +191,14 @@ private[hive] class SparkExecuteStatementOperation( logDebug(result.queryExecution.toString()) result.queryExecution.logical match { case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => - sessionToActivePool(parentSession) = value + sessionToActivePool(parentSession.getSessionHandle) = value logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => } val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) - sessionToActivePool.get(parentSession).foreach { pool => + sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) } iter = { diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 17f1ad3e4690e..798a690a20427 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -69,7 +69,7 @@ private[hive] class SparkExecuteStatementOperation( confOverlay: JMap[String, String], runInBackground: Boolean = true)( hiveContext: HiveContext, - sessionToActivePool: SMap[HiveSession, String]) + sessionToActivePool: SMap[SessionHandle, String]) // NOTE: `runInBackground` is set to `false` intentionally to disable asynchronous execution extends ExecuteStatementOperation(parentSession, statement, confOverlay, false) with Logging { @@ -162,14 +162,14 @@ private[hive] class SparkExecuteStatementOperation( logDebug(result.queryExecution.toString()) result.queryExecution.logical match { case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => - sessionToActivePool(parentSession) = value + sessionToActivePool(parentSession.getSessionHandle) = value logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => } val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) - sessionToActivePool.get(parentSession).foreach { pool => + sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) } iter = { From e7de7e5f46821e1ba3b070b21d6bcf6d5ec8a796 Mon Sep 17 00:00:00 2001 From: YanTangZhai Date: Thu, 18 Dec 2014 20:13:46 -0800 Subject: [PATCH 071/227] [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references The sql "select * from spark_test::for_test where abs(20141202) is not null" has predicates=List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202)) and partitionKeyIds=AttributeSet(). PruningPredicates is List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202)). Then the exception "java.lang.IllegalArgumentException: requirement failed: Partition pruning predicates only supported for partitioned tables." is thrown. The sql "select * from spark_test::for_test_partitioned_table where abs(20141202) is not null and type_id=11 and platform = 3" with partitioned key insert_date has predicates=List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202), (type_id#12 = 11), (platform#8 = 3)) and partitionKeyIds=AttributeSet(insert_date#24). PruningPredicates is List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202)). Author: YanTangZhai Author: yantangzhai Closes #3556 from YanTangZhai/SPARK-4693 and squashes the following commits: 620ebe3 [yantangzhai] [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references 37cfdf5 [yantangzhai] [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references 70a3544 [yantangzhai] [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references efa9b03 [YanTangZhai] Update HiveQuerySuite.scala 72accf1 [YanTangZhai] Update HiveQuerySuite.scala e572b9a [YanTangZhai] Update HiveStrategies.scala 6e643f8 [YanTangZhai] Merge pull request #11 from apache/master e249846 [YanTangZhai] Merge pull request #10 from apache/master d26d982 [YanTangZhai] Merge pull request #9 from apache/master 76d4027 [YanTangZhai] Merge pull request #8 from apache/master 03b62b0 [YanTangZhai] Merge pull request #7 from apache/master 8a00106 [YanTangZhai] Merge pull request #6 from apache/master cbcba66 [YanTangZhai] Merge pull request #3 from apache/master cdef539 [YanTangZhai] Merge pull request #1 from apache/master --- .../spark/sql/catalyst/expressions/AttributeSet.scala | 2 ++ .../scala/org/apache/spark/sql/hive/HiveStrategies.scala | 5 +++-- .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 9 +++++++++ 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index 2b4969b7cfec0..171845ad14e3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -112,4 +112,6 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq override def toString = "{" + baseSet.map(_.a).mkString(", ") + "}" + + override def isEmpty: Boolean = baseSet.isEmpty } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 5f02e95ac3c34..4ebd59db83515 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -210,8 +210,9 @@ private[hive] trait HiveStrategies { // Filter out all predicates that only deal with partition keys, these are given to the // hive table scan operator to be used for partition pruning. val partitionKeyIds = AttributeSet(relation.partitionKeys) - val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.subsetOf(partitionKeyIds) + val (pruningPredicates, otherPredicates) = predicates.partition { predicate => + !predicate.references.isEmpty && + predicate.references.subsetOf(partitionKeyIds) } pruneFilterProject( 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 63eb07c257501..4d81acc753a27 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 @@ -411,6 +411,15 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") + test("predicates contains an empty AttributeSet() references") { + sql( + """ + |SELECT a FROM ( + | SELECT 1 AS a FROM src LIMIT 1 ) table + |WHERE abs(20141202) is not null + """.stripMargin).collect() + } + test("implement identity function using case statement") { val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src") .map { case Row(i: Int) => i } From 7687415c2578b5bdc79c9646c246e52da9a4dd4a Mon Sep 17 00:00:00 2001 From: ravipesala Date: Thu, 18 Dec 2014 20:19:10 -0800 Subject: [PATCH 072/227] [SPARK-2554][SQL] Supporting SumDistinct partial aggregation Adding support to the partial aggregation of SumDistinct Author: ravipesala Closes #3348 from ravipesala/SPARK-2554 and squashes the following commits: fd28e4d [ravipesala] Fixed review comments e60e67f [ravipesala] Fixed test cases and made it as nullable 32fe234 [ravipesala] Supporting SumDistinct partial aggregation Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala --- .../sql/catalyst/expressions/aggregates.scala | 53 +++++++++++++++++-- .../sql/hive/execution/SQLQuerySuite.scala | 13 +++-- 2 files changed, 58 insertions(+), 8 deletions(-) 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 index 0cd90866e14a2..5ea9868e9e846 100755 --- 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 @@ -361,10 +361,10 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ } case class SumDistinct(child: Expression) - extends AggregateExpression with trees.UnaryNode[Expression] { + extends PartialAggregate with trees.UnaryNode[Expression] { + def this() = this(null) override def nullable = true - override def dataType = child.dataType match { case DecimalType.Fixed(precision, scale) => DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive @@ -373,10 +373,55 @@ case class SumDistinct(child: Expression) case _ => child.dataType } + override def toString = s"SUM(DISTINCT ${child})" + override def newInstance() = new SumDistinctFunction(child, this) + + override def asPartial = { + val partialSet = Alias(CollectHashSet(child :: Nil), "partialSets")() + SplitEvaluation( + CombineSetsAndSum(partialSet.toAttribute, this), + partialSet :: Nil) + } +} - override def toString = s"SUM(DISTINCT $child)" +case class CombineSetsAndSum(inputSet: Expression, base: Expression) extends AggregateExpression { + def this() = this(null, null) - override def newInstance() = new SumDistinctFunction(child, this) + override def children = inputSet :: Nil + override def nullable = true + override def dataType = base.dataType + override def toString = s"CombineAndSum($inputSet)" + override def newInstance() = new CombineSetsAndSumFunction(inputSet, this) +} + +case class CombineSetsAndSumFunction( + @transient inputSet: Expression, + @transient base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new OpenHashSet[Any]() + + override def update(input: Row): Unit = { + val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]] + val inputIterator = inputSetEval.iterator + while (inputIterator.hasNext) { + seen.add(inputIterator.next) + } + } + + override def eval(input: Row): Any = { + val casted = seen.asInstanceOf[OpenHashSet[Row]] + if (casted.size == 0) { + null + } else { + Cast(Literal( + casted.iterator.map(f => f.apply(0)).reduceLeft( + base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus)), + base.dataType).eval(null) + } + } } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { 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 96f3430207982..f57f31af15566 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 @@ -185,9 +185,14 @@ class SQLQuerySuite extends QueryTest { sql("SELECT case when ~1=-2 then 1 else 0 end FROM src"), sql("SELECT 1 FROM src").collect().toSeq) } - - test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") { - checkAnswer(sql("SELECT key FROM src WHERE key not between 0 and 10 order by key"), - sql("SELECT key FROM src WHERE key between 11 and 500 order by key").collect().toSeq) + + test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") { + checkAnswer(sql("SELECT key FROM src WHERE key not between 0 and 10 order by key"), + sql("SELECT key FROM src WHERE key between 11 and 500 order by key").collect().toSeq) + } + + test("SPARK-2554 SumDistinct partial aggregation") { + checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), + sql("SELECT distinct key FROM src order by key").collect().toSeq) } } From ae9f128608f67cbee0a2fb24754783ee3b4f3098 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 18 Dec 2014 20:21:52 -0800 Subject: [PATCH 073/227] [SPARK-4573] [SQL] Add SettableStructObjectInspector support in "wrap" function Hive UDAF may create an customized object constructed by SettableStructObjectInspector, this is critical when integrate Hive UDAF with the refactor-ed UDAF interface. Performance issue in `wrap/unwrap` since more match cases added, will do it in another PR. Author: Cheng Hao Closes #3429 from chenghao-intel/settable_oi and squashes the following commits: 9f0aff3 [Cheng Hao] update code style issues as feedbacks 2b0561d [Cheng Hao] Add more scala doc f5a40e8 [Cheng Hao] add scala doc 2977e9b [Cheng Hao] remove the timezone setting for test suite 3ed284c [Cheng Hao] fix the date type comparison f1b6749 [Cheng Hao] Update the comment 932940d [Cheng Hao] Add more unit test 72e4332 [Cheng Hao] Add settable StructObjectInspector support --- .../spark/sql/hive/HiveInspectors.scala | 346 +++++++++++++++--- .../spark/sql/hive/HiveInspectorSuite.scala | 220 +++++++++++ .../org/apache/spark/sql/hive/Shim12.scala | 87 +++-- .../org/apache/spark/sql/hive/Shim13.scala | 130 ++++--- 4 files changed, 659 insertions(+), 124 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 0eeac8620f01c..06189341f827c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -18,9 +18,7 @@ package org.apache.spark.sql.hive import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector._ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} @@ -33,6 +31,145 @@ import org.apache.spark.sql.catalyst.types.decimal.Decimal /* Implicit conversions */ import scala.collection.JavaConversions._ +/** + * 1. The Underlying data type in catalyst and in Hive + * In catalyst: + * Primitive => + * java.lang.String + * int / scala.Int + * boolean / scala.Boolean + * float / scala.Float + * double / scala.Double + * long / scala.Long + * short / scala.Short + * byte / scala.Byte + * org.apache.spark.sql.catalyst.types.decimal.Decimal + * Array[Byte] + * java.sql.Date + * java.sql.Timestamp + * Complex Types => + * Map: scala.collection.immutable.Map + * List: scala.collection.immutable.Seq + * Struct: + * org.apache.spark.sql.catalyst.expression.Row + * Union: NOT SUPPORTED YET + * The Complex types plays as a container, which can hold arbitrary data types. + * + * In Hive, the native data types are various, in UDF/UDAF/UDTF, and associated with + * Object Inspectors, in Hive expression evaluation framework, the underlying data are + * Primitive Type + * Java Boxed Primitives: + * org.apache.hadoop.hive.common.type.HiveVarchar + * java.lang.String + * java.lang.Integer + * java.lang.Boolean + * java.lang.Float + * java.lang.Double + * java.lang.Long + * java.lang.Short + * java.lang.Byte + * org.apache.hadoop.hive.common.`type`.HiveDecimal + * byte[] + * java.sql.Date + * java.sql.Timestamp + * Writables: + * org.apache.hadoop.hive.serde2.io.HiveVarcharWritable + * org.apache.hadoop.io.Text + * org.apache.hadoop.io.IntWritable + * org.apache.hadoop.hive.serde2.io.DoubleWritable + * org.apache.hadoop.io.BooleanWritable + * org.apache.hadoop.io.LongWritable + * org.apache.hadoop.io.FloatWritable + * org.apache.hadoop.hive.serde2.io.ShortWritable + * org.apache.hadoop.hive.serde2.io.ByteWritable + * org.apache.hadoop.io.BytesWritable + * org.apache.hadoop.hive.serde2.io.DateWritable + * org.apache.hadoop.hive.serde2.io.TimestampWritable + * org.apache.hadoop.hive.serde2.io.HiveDecimalWritable + * Complex Type + * List: Object[] / java.util.List + * Map: java.util.Map + * Struct: Object[] / java.util.List / java POJO + * Union: class StandardUnion { byte tag; Object object } + * + * NOTICE: HiveVarchar is not supported by catalyst, it will be simply considered as String type. + * + * + * 2. Hive ObjectInspector is a group of flexible APIs to inspect value in different data + * representation, and developers can extend those API as needed, so technically, + * object inspector supports arbitrary data type in java. + * + * Fortunately, only few built-in Hive Object Inspectors are used in generic udf/udaf/udtf + * evaluation. + * 1) Primitive Types (PrimitiveObjectInspector & its sub classes) + {{{ + public interface PrimitiveObjectInspector { + // Java Primitives (java.lang.Integer, java.lang.String etc.) + Object getPrimitiveWritableObject(Object o); + // Writables (hadoop.io.IntWritable, hadoop.io.Text etc.) + Object getPrimitiveJavaObject(Object o); + // ObjectInspector only inspect the `writable` always return true, we need to check it + // before invoking the methods above. + boolean preferWritable(); + ... + } + }}} + + * 2) Complex Types: + * ListObjectInspector: inspects java array or [[java.util.List]] + * MapObjectInspector: inspects [[java.util.Map]] + * Struct.StructObjectInspector: inspects java array, [[java.util.List]] and + * even a normal java object (POJO) + * UnionObjectInspector: (tag: Int, object data) (TODO: not supported by SparkSQL yet) + * + * 3) ConstantObjectInspector: + * Constant object inspector can be either primitive type or Complex type, and it bundles a + * constant value as its property, usually the value is created when the constant object inspector + * constructed. + * {{{ + public interface ConstantObjectInspector extends ObjectInspector { + Object getWritableConstantValue(); + ... + } + }}} + * Hive provides 3 built-in constant object inspectors: + * Primitive Object Inspectors: + * WritableConstantStringObjectInspector + * WritableConstantHiveVarcharObjectInspector + * WritableConstantHiveDecimalObjectInspector + * WritableConstantTimestampObjectInspector + * WritableConstantIntObjectInspector + * WritableConstantDoubleObjectInspector + * WritableConstantBooleanObjectInspector + * WritableConstantLongObjectInspector + * WritableConstantFloatObjectInspector + * WritableConstantShortObjectInspector + * WritableConstantByteObjectInspector + * WritableConstantBinaryObjectInspector + * WritableConstantDateObjectInspector + * Map Object Inspector: + * StandardConstantMapObjectInspector + * List Object Inspector: + * StandardConstantListObjectInspector]] + * Struct Object Inspector: Hive doesn't provide the built-in constant object inspector for Struct + * Union Object Inspector: Hive doesn't provide the built-in constant object inspector for Union + * + * + * 3. This trait facilitates: + * Data Unwrapping: Hive Data => Catalyst Data (unwrap) + * Data Wrapping: Catalyst Data => Hive Data (wrap) + * Binding the Object Inspector for Catalyst Data (toInspector) + * Retrieving the Catalyst Data Type from Object Inspector (inspectorToDataType) + * + * + * 4. Future Improvement (TODO) + * This implementation is quite ugly and inefficient: + * a. Pattern matching in runtime + * b. Small objects creation in catalyst data => writable + * c. Unnecessary unwrap / wrap for nested UDF invoking: + * e.g. date_add(printf("%s-%s-%s", a,b,c), 3) + * We don't need to unwrap the data for printf and wrap it again and passes in data_add + */ private[hive] trait HiveInspectors { def javaClassToDataType(clz: Class[_]): DataType = clz match { @@ -87,10 +224,23 @@ private[hive] trait HiveInspectors { * @param oi the ObjectInspector associated with the Hive Type * @return convert the data into catalyst type * TODO return the function of (data => Any) instead for performance consideration + * + * Strictly follows the following order in unwrapping (constant OI has the higher priority): + * Constant Null object inspector => + * return null + * Constant object inspector => + * extract the value from constant object inspector + * Check whether the `data` is null => + * return null if true + * If object inspector prefers writable => + * extract writable from `data` and then get the catalyst type from the writable + * Extract the java object directly from the object inspector + * + * NOTICE: the complex data type requires recursive unwrapping. */ def unwrap(data: Any, oi: ObjectInspector): Any = oi match { - case _ if data == null => null - case poi: VoidObjectInspector => null + case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null + case poi: WritableConstantStringObjectInspector => poi.getWritableConstantValue.toString case poi: WritableConstantHiveVarcharObjectInspector => poi.getWritableConstantValue.getHiveVarchar.getValue case poi: WritableConstantHiveDecimalObjectInspector => @@ -119,12 +269,44 @@ private[hive] trait HiveInspectors { System.arraycopy(writable.getBytes, 0, temp, 0, temp.length) temp case poi: WritableConstantDateObjectInspector => poi.getWritableConstantValue.get() - case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue - case hdoi: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(hdoi, data) - // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object - // if next timestamp is null, so Timestamp object is cloned - case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone() - case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) + case mi: StandardConstantMapObjectInspector => + // take the value from the map inspector object, rather than the input data + mi.getWritableConstantValue.map { case (k, v) => + (unwrap(k, mi.getMapKeyObjectInspector), + unwrap(v, mi.getMapValueObjectInspector)) + }.toMap + case li: StandardConstantListObjectInspector => + // take the value from the list inspector object, rather than the input data + li.getWritableConstantValue.map(unwrap(_, li.getListElementObjectInspector)).toSeq + // if the value is null, we don't care about the object inspector type + case _ if data == null => null + case poi: VoidObjectInspector => null // always be null for void object inspector + case pi: PrimitiveObjectInspector => pi match { + // We think HiveVarchar is also a String + case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() => + hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue + case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue + case x: StringObjectInspector if x.preferWritable() => + x.getPrimitiveWritableObject(data).toString + case x: IntObjectInspector if x.preferWritable() => x.get(data) + case x: BooleanObjectInspector if x.preferWritable() => x.get(data) + case x: FloatObjectInspector if x.preferWritable() => x.get(data) + case x: DoubleObjectInspector if x.preferWritable() => x.get(data) + case x: LongObjectInspector if x.preferWritable() => x.get(data) + case x: ShortObjectInspector if x.preferWritable() => x.get(data) + case x: ByteObjectInspector if x.preferWritable() => x.get(data) + case x: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(x, data) + case x: BinaryObjectInspector if x.preferWritable() => + x.getPrimitiveWritableObject(data).copyBytes() + case x: DateObjectInspector if x.preferWritable() => + x.getPrimitiveWritableObject(data).get() + // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object + // if next timestamp is null, so Timestamp object is cloned + case x: TimestampObjectInspector if x.preferWritable() => + x.getPrimitiveWritableObject(data).getTimestamp.clone() + case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone() + case _ => pi.getPrimitiveJavaObject(data) + } case li: ListObjectInspector => Option(li.getList(data)) .map(_.map(unwrap(_, li.getListElementObjectInspector)).toSeq) @@ -132,10 +314,11 @@ private[hive] trait HiveInspectors { case mi: MapObjectInspector => Option(mi.getMap(data)).map( _.map { - case (k,v) => + case (k, v) => (unwrap(k, mi.getMapKeyObjectInspector), unwrap(v, mi.getMapValueObjectInspector)) }.toMap).orNull + // currently, hive doesn't provide the ConstantStructObjectInspector case si: StructObjectInspector => val allRefs = si.getAllStructFieldRefs new GenericRow( @@ -191,55 +374,89 @@ private[hive] trait HiveInspectors { * the ObjectInspector should also be consistent with those returned from * toInspector: DataType => ObjectInspector and * toInspector: Expression => ObjectInspector + * + * Strictly follows the following order in wrapping (constant OI has the higher priority): + * Constant object inspector => return the bundled value of Constant object inspector + * Check whether the `a` is null => return null if true + * If object inspector prefers writable object => return a Writable for the given data `a` + * Map the catalyst data to the boxed java primitive + * + * NOTICE: the complex data type requires recursive wrapping. */ - def wrap(a: Any, oi: ObjectInspector): AnyRef = if (a == null) { - null - } else { - oi match { - case x: ConstantObjectInspector => x.getWritableConstantValue - case x: PrimitiveObjectInspector => a match { - // TODO what if x.preferWritable() == true? reuse the writable? - case s: String => s: java.lang.String - case i: Int => i: java.lang.Integer - case b: Boolean => b: java.lang.Boolean - case f: Float => f: java.lang.Float - case d: Double => d: java.lang.Double - case l: Long => l: java.lang.Long - case l: Short => l: java.lang.Short - case l: Byte => l: java.lang.Byte - case b: BigDecimal => HiveShim.createDecimal(b.underlying()) - case d: Decimal => HiveShim.createDecimal(d.toBigDecimal.underlying()) - case b: Array[Byte] => b - case d: java.sql.Date => d - case t: java.sql.Timestamp => t + def wrap(a: Any, oi: ObjectInspector): AnyRef = oi match { + case x: ConstantObjectInspector => x.getWritableConstantValue + case _ if a == null => null + case x: PrimitiveObjectInspector => x match { + // TODO we don't support the HiveVarcharObjectInspector yet. + case _: StringObjectInspector if x.preferWritable() => HiveShim.getStringWritable(a) + case _: StringObjectInspector => a.asInstanceOf[java.lang.String] + case _: IntObjectInspector if x.preferWritable() => HiveShim.getIntWritable(a) + case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer] + case _: BooleanObjectInspector if x.preferWritable() => HiveShim.getBooleanWritable(a) + case _: BooleanObjectInspector => a.asInstanceOf[java.lang.Boolean] + case _: FloatObjectInspector if x.preferWritable() => HiveShim.getFloatWritable(a) + case _: FloatObjectInspector => a.asInstanceOf[java.lang.Float] + case _: DoubleObjectInspector if x.preferWritable() => HiveShim.getDoubleWritable(a) + case _: DoubleObjectInspector => a.asInstanceOf[java.lang.Double] + case _: LongObjectInspector if x.preferWritable() => HiveShim.getLongWritable(a) + case _: LongObjectInspector => a.asInstanceOf[java.lang.Long] + case _: ShortObjectInspector if x.preferWritable() => HiveShim.getShortWritable(a) + case _: ShortObjectInspector => a.asInstanceOf[java.lang.Short] + case _: ByteObjectInspector if x.preferWritable() => HiveShim.getByteWritable(a) + case _: ByteObjectInspector => a.asInstanceOf[java.lang.Byte] + case _: HiveDecimalObjectInspector if x.preferWritable() => + HiveShim.getDecimalWritable(a.asInstanceOf[Decimal]) + case _: HiveDecimalObjectInspector => + HiveShim.createDecimal(a.asInstanceOf[Decimal].toBigDecimal.underlying()) + case _: BinaryObjectInspector if x.preferWritable() => HiveShim.getBinaryWritable(a) + case _: BinaryObjectInspector => a.asInstanceOf[Array[Byte]] + case _: DateObjectInspector if x.preferWritable() => HiveShim.getDateWritable(a) + case _: DateObjectInspector => a.asInstanceOf[java.sql.Date] + case _: TimestampObjectInspector if x.preferWritable() => HiveShim.getTimestampWritable(a) + case _: TimestampObjectInspector => a.asInstanceOf[java.sql.Timestamp] + } + case x: SettableStructObjectInspector => + val fieldRefs = x.getAllStructFieldRefs + val row = a.asInstanceOf[Seq[_]] + // 1. create the pojo (most likely) object + val result = x.create() + var i = 0 + while (i < fieldRefs.length) { + // 2. set the property for the pojo + x.setStructFieldData( + result, + fieldRefs.get(i), + wrap(row(i), fieldRefs.get(i).getFieldObjectInspector)) + i += 1 } - case x: StructObjectInspector => - val fieldRefs = x.getAllStructFieldRefs - val row = a.asInstanceOf[Seq[_]] - val result = new java.util.ArrayList[AnyRef](fieldRefs.length) - var i = 0 - while (i < fieldRefs.length) { - result.add(wrap(row(i), fieldRefs.get(i).getFieldObjectInspector)) - i += 1 - } - result - case x: ListObjectInspector => - val list = new java.util.ArrayList[Object] - a.asInstanceOf[Seq[_]].foreach { - v => list.add(wrap(v, x.getListElementObjectInspector)) - } - list - case x: MapObjectInspector => - // Some UDFs seem to assume we pass in a HashMap. - val hashMap = new java.util.HashMap[AnyRef, AnyRef]() - hashMap.putAll(a.asInstanceOf[Map[_, _]].map { - case (k, v) => - wrap(k, x.getMapKeyObjectInspector) -> wrap(v, x.getMapValueObjectInspector) - }) + result + case x: StructObjectInspector => + val fieldRefs = x.getAllStructFieldRefs + val row = a.asInstanceOf[Seq[_]] + val result = new java.util.ArrayList[AnyRef](fieldRefs.length) + var i = 0 + while (i < fieldRefs.length) { + result.add(wrap(row(i), fieldRefs.get(i).getFieldObjectInspector)) + i += 1 + } - hashMap - } + result + case x: ListObjectInspector => + val list = new java.util.ArrayList[Object] + a.asInstanceOf[Seq[_]].foreach { + v => list.add(wrap(v, x.getListElementObjectInspector)) + } + list + case x: MapObjectInspector => + // Some UDFs seem to assume we pass in a HashMap. + val hashMap = new java.util.HashMap[AnyRef, AnyRef]() + hashMap.putAll(a.asInstanceOf[Map[_, _]].map { + case (k, v) => + wrap(k, x.getMapKeyObjectInspector) -> wrap(v, x.getMapValueObjectInspector) + }) + + hashMap } def wrap( @@ -254,6 +471,11 @@ private[hive] trait HiveInspectors { cache } + /** + * @param dataType Catalyst data type + * @return Hive java object inspector (recursively), not the Writable ObjectInspector + * We can easily map to the Hive built-in object inspector according to the data type. + */ def toInspector(dataType: DataType): ObjectInspector = dataType match { case ArrayType(tpe, _) => ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) @@ -272,12 +494,20 @@ private[hive] trait HiveInspectors { case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector case DateType => PrimitiveObjectInspectorFactory.javaDateObjectInspector case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + // TODO decimal precision? case DecimalType() => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) } + /** + * Map the catalyst expression to ObjectInspector, however, + * if the expression is [[Literal]] or foldable, a constant writable object inspector returns; + * Otherwise, we always get the object inspector according to its data type(in catalyst) + * @param expr Catalyst expression to be mapped + * @return Hive java objectinspector (recursively). + */ def toInspector(expr: Expression): ObjectInspector = expr match { case Literal(value, StringType) => HiveShim.getStringWritableConstantObjectInspector(value) @@ -326,8 +556,12 @@ private[hive] trait HiveInspectors { }) ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, map) } + // We will enumerate all of the possible constant expressions, throw exception if we missed case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") + // ideally, we don't test the foldable here(but in optimizer), however, some of the + // Hive UDF / UDAF requires its argument to be constant objectinspector, we do it eagerly. case _ if expr.foldable => toInspector(Literal(expr.eval(), expr.dataType)) + // For those non constant expression, map to object inspector according to its data type case _ => toInspector(expr.dataType) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala new file mode 100644 index 0000000000000..bfe608a51a30b --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -0,0 +1,220 @@ +/* + * 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.sql.hive + +import java.sql.Date +import java.util + +import org.apache.hadoop.hive.serde2.io.DoubleWritable +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.scalatest.FunSuite + +import org.apache.hadoop.hive.ql.udf.UDAFPercentile +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.io.LongWritable + +import org.apache.spark.sql.catalyst.expressions.{Literal, Row} + +class HiveInspectorSuite extends FunSuite with HiveInspectors { + test("Test wrap SettableStructObjectInspector") { + val udaf = new UDAFPercentile.PercentileLongEvaluator() + udaf.init() + + udaf.iterate(new LongWritable(1), 0.1) + udaf.iterate(new LongWritable(1), 0.1) + + val state = udaf.terminatePartial() + + val soi = ObjectInspectorFactory.getReflectionObjectInspector( + classOf[UDAFPercentile.State], + ObjectInspectorOptions.JAVA).asInstanceOf[StructObjectInspector] + + val a = unwrap(state, soi).asInstanceOf[Row] + val b = wrap(a, soi).asInstanceOf[UDAFPercentile.State] + + val sfCounts = soi.getStructFieldRef("counts") + val sfPercentiles = soi.getStructFieldRef("percentiles") + + assert(2 === soi.getStructFieldData(b, sfCounts) + .asInstanceOf[util.Map[LongWritable, LongWritable]] + .get(new LongWritable(1L)) + .get()) + assert(0.1 === soi.getStructFieldData(b, sfPercentiles) + .asInstanceOf[util.ArrayList[DoubleWritable]] + .get(0) + .get()) + } + + val data = + Literal(true) :: + Literal(0.asInstanceOf[Byte]) :: + Literal(0.asInstanceOf[Short]) :: + Literal(0) :: + Literal(0.asInstanceOf[Long]) :: + Literal(0.asInstanceOf[Float]) :: + Literal(0.asInstanceOf[Double]) :: + Literal("0") :: + Literal(new Date(2014, 9, 23)) :: + Literal(Decimal(BigDecimal(123.123))) :: + Literal(new java.sql.Timestamp(123123)) :: + Literal(Array[Byte](1,2,3)) :: + Literal(Seq[Int](1,2,3), ArrayType(IntegerType)) :: + Literal(Map[Int, Int](1->2, 2->1), MapType(IntegerType, IntegerType)) :: + Literal(Row(1,2.0d,3.0f), + StructType(StructField("c1", IntegerType) :: + StructField("c2", DoubleType) :: + StructField("c3", FloatType) :: Nil)) :: + Nil + + val row = data.map(_.eval(null)) + val dataTypes = data.map(_.dataType) + + import scala.collection.JavaConversions._ + def toWritableInspector(dataType: DataType): ObjectInspector = dataType match { + case ArrayType(tpe, _) => + ObjectInspectorFactory.getStandardListObjectInspector(toWritableInspector(tpe)) + case MapType(keyType, valueType, _) => + ObjectInspectorFactory.getStandardMapObjectInspector( + toWritableInspector(keyType), toWritableInspector(valueType)) + case StringType => PrimitiveObjectInspectorFactory.writableStringObjectInspector + case IntegerType => PrimitiveObjectInspectorFactory.writableIntObjectInspector + case DoubleType => PrimitiveObjectInspectorFactory.writableDoubleObjectInspector + case BooleanType => PrimitiveObjectInspectorFactory.writableBooleanObjectInspector + case LongType => PrimitiveObjectInspectorFactory.writableLongObjectInspector + case FloatType => PrimitiveObjectInspectorFactory.writableFloatObjectInspector + case ShortType => PrimitiveObjectInspectorFactory.writableShortObjectInspector + case ByteType => PrimitiveObjectInspectorFactory.writableByteObjectInspector + case NullType => PrimitiveObjectInspectorFactory.writableVoidObjectInspector + case BinaryType => PrimitiveObjectInspectorFactory.writableBinaryObjectInspector + case DateType => PrimitiveObjectInspectorFactory.writableDateObjectInspector + case TimestampType => PrimitiveObjectInspectorFactory.writableTimestampObjectInspector + case DecimalType() => PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector + case StructType(fields) => + ObjectInspectorFactory.getStandardStructObjectInspector( + fields.map(f => f.name), fields.map(f => toWritableInspector(f.dataType))) + } + + def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = { + dt1.zip(dt2).map { + case (dd1, dd2) => + assert(dd1.getClass === dd2.getClass) // DecimalType doesn't has the default precision info + } + } + + def checkValues(row1: Seq[Any], row2: Seq[Any]): Unit = { + row1.zip(row2).map { + case (r1, r2) => checkValues(r1, r2) + } + } + + def checkValues(v1: Any, v2: Any): Unit = { + (v1, v2) match { + case (r1: Decimal, r2: Decimal) => + // Ignore the Decimal precision + assert(r1.compare(r2) === 0) + case (r1: Array[Byte], r2: Array[Byte]) + if r1 != null && r2 != null && r1.length == r2.length => + r1.zip(r2).map { case (b1, b2) => assert(b1 === b2) } + case (r1: Date, r2: Date) => assert(r1.compareTo(r2) === 0) + case (r1, r2) => assert(r1 === r2) + } + } + + test("oi => datatype => oi") { + val ois = dataTypes.map(toInspector) + + checkDataType(ois.map(inspectorToDataType), dataTypes) + checkDataType(dataTypes.map(toWritableInspector).map(inspectorToDataType), dataTypes) + } + + test("wrap / unwrap null, constant null and writables") { + val writableOIs = dataTypes.map(toWritableInspector) + val nullRow = data.map(d => null) + + checkValues(nullRow, nullRow.zip(writableOIs).map { + case (d, oi) => unwrap(wrap(d, oi), oi) + }) + + // struct couldn't be constant, sweep it out + val constantExprs = data.filter(!_.dataType.isInstanceOf[StructType]) + val constantData = constantExprs.map(_.eval()) + val constantNullData = constantData.map(_ => null) + val constantWritableOIs = constantExprs.map(e => toWritableInspector(e.dataType)) + val constantNullWritableOIs = constantExprs.map(e => toInspector(Literal(null, e.dataType))) + + checkValues(constantData, constantData.zip(constantWritableOIs).map { + case (d, oi) => unwrap(wrap(d, oi), oi) + }) + + checkValues(constantNullData, constantData.zip(constantNullWritableOIs).map { + case (d, oi) => unwrap(wrap(d, oi), oi) + }) + + checkValues(constantNullData, constantNullData.zip(constantWritableOIs).map { + case (d, oi) => unwrap(wrap(d, oi), oi) + }) + } + + test("wrap / unwrap primitive writable object inspector") { + val writableOIs = dataTypes.map(toWritableInspector) + + checkValues(row, row.zip(writableOIs).map { + case (data, oi) => unwrap(wrap(data, oi), oi) + }) + } + + test("wrap / unwrap primitive java object inspector") { + val ois = dataTypes.map(toInspector) + + checkValues(row, row.zip(ois).map { + case (data, oi) => unwrap(wrap(data, oi), oi) + }) + } + + test("wrap / unwrap Struct Type") { + val dt = StructType(dataTypes.zipWithIndex.map { + case (t, idx) => StructField(s"c_$idx", t) + }) + + checkValues(row, unwrap(wrap(row, toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) + checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + } + + test("wrap / unwrap Array Type") { + val dt = ArrayType(dataTypes(0)) + + val d = row(0) :: row(0) :: Nil + checkValues(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) + checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValues(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValues(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + } + + test("wrap / unwrap Map Type") { + val dt = MapType(dataTypes(0), dataTypes(1)) + + val d = Map(row(0) -> row(1)) + checkValues(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) + checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValues(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValues(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + } +} diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 67cc8865755c5..2d01a85067518 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} import org.apache.hadoop.hive.serde2.{io => hiveIo} +import org.apache.hadoop.io.NullWritable import org.apache.hadoop.{io => hadoopIo} import org.apache.hadoop.mapred.InputFormat import org.apache.spark.sql.catalyst.types.decimal.Decimal @@ -71,76 +72,114 @@ private[hive] object HiveShim { def getStringWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.STRING, - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String])) + getStringWritable(value)) def getIntWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.INT, - if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])) + getIntWritable(value)) def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.DOUBLE, - if (value == null) null else new hiveIo.DoubleWritable(value.asInstanceOf[Double])) + getDoubleWritable(value)) def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.BOOLEAN, - if (value == null) null else new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean])) + getBooleanWritable(value)) def getLongWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.LONG, - if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])) + getLongWritable(value)) def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.FLOAT, - if (value == null) null else new hadoopIo.FloatWritable(value.asInstanceOf[Float])) + getFloatWritable(value)) def getShortWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.SHORT, - if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])) + getShortWritable(value)) def getByteWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.BYTE, - if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])) + getByteWritable(value)) def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.BINARY, - if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])) + getBinaryWritable(value)) def getDateWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.DATE, - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date])) + getDateWritable(value)) def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.TIMESTAMP, - if (value == null) { - null - } else { - new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp]) - }) + getTimestampWritable(value)) def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.DECIMAL, - if (value == null) { - null - } else { - new hiveIo.HiveDecimalWritable( - HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying())) - }) + getDecimalWritable(value)) def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.VOID, null) + def getStringWritable(value: Any): hadoopIo.Text = + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + + def getIntWritable(value: Any): hadoopIo.IntWritable = + if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) + + def getDoubleWritable(value: Any): hiveIo.DoubleWritable = + if (value == null) null else new hiveIo.DoubleWritable(value.asInstanceOf[Double]) + + def getBooleanWritable(value: Any): hadoopIo.BooleanWritable = + if (value == null) null else new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean]) + + def getLongWritable(value: Any): hadoopIo.LongWritable = + if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long]) + + def getFloatWritable(value: Any): hadoopIo.FloatWritable = + if (value == null) null else new hadoopIo.FloatWritable(value.asInstanceOf[Float]) + + def getShortWritable(value: Any): hiveIo.ShortWritable = + if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short]) + + def getByteWritable(value: Any): hiveIo.ByteWritable = + if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte]) + + def getBinaryWritable(value: Any): hadoopIo.BytesWritable = + if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) + + def getDateWritable(value: Any): hiveIo.DateWritable = + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) + + def getTimestampWritable(value: Any): hiveIo.TimestampWritable = + if (value == null) { + null + } else { + new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp]) + } + + def getDecimalWritable(value: Any): hiveIo.HiveDecimalWritable = + if (value == null) { + null + } else { + new hiveIo.HiveDecimalWritable( + HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying())) + } + + def getPrimitiveNullWritable: NullWritable = NullWritable.get() + def createDriverResultsArray = new JArrayList[String] def processResults(results: JArrayList[String]) = results @@ -197,7 +236,11 @@ private[hive] object HiveShim { } def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = { - Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + if (hdoi.preferWritable()) { + Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue) + } else { + Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + } } } diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 7c8cbf10c1c30..b78c75798e988 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -22,6 +22,7 @@ import java.util.Properties import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.common.`type`.{HiveDecimal} @@ -163,91 +164,123 @@ private[hive] object HiveShim { new TableDesc(inputFormatClass, outputFormatClass, properties) } + def getStringWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.stringTypeInfo, - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String])) + TypeInfoFactory.stringTypeInfo, getStringWritable(value)) def getIntWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.intTypeInfo, - if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])) + TypeInfoFactory.intTypeInfo, getIntWritable(value)) def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.doubleTypeInfo, if (value == null) { - null - } else { - new hiveIo.DoubleWritable(value.asInstanceOf[Double]) - }) + TypeInfoFactory.doubleTypeInfo, getDoubleWritable(value)) def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.booleanTypeInfo, if (value == null) { - null - } else { - new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean]) - }) + TypeInfoFactory.booleanTypeInfo, getBooleanWritable(value)) def getLongWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.longTypeInfo, - if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])) + TypeInfoFactory.longTypeInfo, getLongWritable(value)) def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.floatTypeInfo, if (value == null) { - null - } else { - new hadoopIo.FloatWritable(value.asInstanceOf[Float]) - }) + TypeInfoFactory.floatTypeInfo, getFloatWritable(value)) def getShortWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.shortTypeInfo, - if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])) + TypeInfoFactory.shortTypeInfo, getShortWritable(value)) def getByteWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.byteTypeInfo, - if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])) + TypeInfoFactory.byteTypeInfo, getByteWritable(value)) def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.binaryTypeInfo, if (value == null) { - null - } else { - new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) - }) + TypeInfoFactory.binaryTypeInfo, getBinaryWritable(value)) def getDateWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.dateTypeInfo, - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date])) + TypeInfoFactory.dateTypeInfo, getDateWritable(value)) def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.timestampTypeInfo, if (value == null) { - null - } else { - new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp]) - }) + TypeInfoFactory.timestampTypeInfo, getTimestampWritable(value)) def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.decimalTypeInfo, - if (value == null) { - null - } else { - // TODO precise, scale? - new hiveIo.HiveDecimalWritable( - HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying())) - }) + TypeInfoFactory.decimalTypeInfo, getDecimalWritable(value)) def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( TypeInfoFactory.voidTypeInfo, null) + def getStringWritable(value: Any): hadoopIo.Text = + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + + def getIntWritable(value: Any): hadoopIo.IntWritable = + if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) + + def getDoubleWritable(value: Any): hiveIo.DoubleWritable = + if (value == null) { + null + } else { + new hiveIo.DoubleWritable(value.asInstanceOf[Double]) + } + + def getBooleanWritable(value: Any): hadoopIo.BooleanWritable = + if (value == null) { + null + } else { + new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean]) + } + + def getLongWritable(value: Any): hadoopIo.LongWritable = + if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long]) + + def getFloatWritable(value: Any): hadoopIo.FloatWritable = + if (value == null) { + null + } else { + new hadoopIo.FloatWritable(value.asInstanceOf[Float]) + } + + def getShortWritable(value: Any): hiveIo.ShortWritable = + if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short]) + + def getByteWritable(value: Any): hiveIo.ByteWritable = + if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte]) + + def getBinaryWritable(value: Any): hadoopIo.BytesWritable = + if (value == null) { + null + } else { + new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) + } + + def getDateWritable(value: Any): hiveIo.DateWritable = + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) + + def getTimestampWritable(value: Any): hiveIo.TimestampWritable = + if (value == null) { + null + } else { + new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp]) + } + + def getDecimalWritable(value: Any): hiveIo.HiveDecimalWritable = + if (value == null) { + null + } else { + // TODO precise, scale? + new hiveIo.HiveDecimalWritable( + HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying())) + } + + def getPrimitiveNullWritable: NullWritable = NullWritable.get() + def createDriverResultsArray = new JArrayList[Object] def processResults(results: JArrayList[Object]) = { @@ -355,7 +388,12 @@ private[hive] object HiveShim { } def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = { - Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) + if (hdoi.preferWritable()) { + Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue, + hdoi.precision(), hdoi.scale()) + } else { + Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) + } } } From c3d91da5ea8b85ca75444ec606f2e1eae376c4b2 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 18 Dec 2014 20:24:56 -0800 Subject: [PATCH 074/227] [SPARK-4861][SQL] Refactory command in spark sql Remove ```Command``` and use ```RunnableCommand``` instead. Author: wangfei Author: scwf Closes #3712 from scwf/cmd and squashes the following commits: 51a82f2 [wangfei] fix test failure 0e03be8 [wangfei] address comments 4033bed [scwf] remove CreateTableAsSelect in hivestrategy 5d20010 [wangfei] address comments 125f542 [scwf] factory command in spark sql --- .../sql/catalyst/plans/logical/commands.scala | 17 ---- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 13 +-- .../apache/spark/sql/execution/commands.scala | 85 +++++++------------ .../spark/sql/hive/ExtendedHiveQlParser.scala | 3 +- .../apache/spark/sql/hive/HiveContext.scala | 20 +++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 16 +++- .../org/apache/spark/sql/hive/HiveQl.scala | 13 +-- .../spark/sql/hive/HiveStrategies.scala | 39 ++------- .../org/apache/spark/sql/hive/TestHive.scala | 8 +- .../hive/execution/CreateTableAsSelect.scala | 38 +++------ .../execution/DescribeHiveTableCommand.scala | 23 ++--- ...eCommand.scala => HiveNativeCommand.scala} | 76 ++++++++--------- .../hive/execution/InsertIntoHiveTable.scala | 18 ++-- .../spark/sql/hive/execution/commands.scala | 39 ++++----- .../spark/sql/hive/StatisticsSuite.scala | 12 +-- .../hive/execution/HiveComparisonTest.scala | 5 +- 17 files changed, 172 insertions(+), 255 deletions(-) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/{NativeCommand.scala => HiveNativeCommand.scala} (64%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 1d513d7789763..5a1863953eae9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -29,23 +29,6 @@ abstract class Command extends LeafNode { def output: Seq[Attribute] = Seq.empty } -/** - * Returned for commands supported by a given parser, but not catalyst. In general these are DDL - * commands that are passed directly to another system. - */ -case class NativeCommand(cmd: String) extends Command { - override def output = - Seq(AttributeReference("result", StringType, nullable = false)()) -} - -/** - * Commands of the form "SET [key [= value] ]". - */ -case class DFSCommand(kv: Option[(String, Option[String])]) extends Command { - override def output = Seq( - AttributeReference("DFS output", StringType, nullable = false)()) -} - /** * * Commands of the form "SET [key [= value] ]". 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 ebd4cc920b1ad..7a13302229012 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 @@ -329,7 +329,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def strategies: Seq[Strategy] = extraStrategies ++ ( - CommandStrategy(self) :: + CommandStrategy :: DataSourceStrategy :: TakeOrdered :: HashAggregation :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 6e04f26c84259..2954d4ce7d2d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -304,17 +304,20 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } - case class CommandStrategy(context: SQLContext) extends Strategy { + case object CommandStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case r: RunnableCommand => ExecutedCommand(r) :: Nil case logical.SetCommand(kv) => - Seq(execution.SetCommand(kv, plan.output)(context)) + Seq(ExecutedCommand(execution.SetCommand(kv, plan.output))) case logical.ExplainCommand(logicalPlan, extended) => - Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) + Seq(ExecutedCommand( + execution.ExplainCommand(logicalPlan, plan.output, extended))) case logical.CacheTableCommand(tableName, optPlan, isLazy) => - Seq(execution.CacheTableCommand(tableName, optPlan, isLazy)) + Seq(ExecutedCommand( + execution.CacheTableCommand(tableName, optPlan, isLazy))) case logical.UncacheTableCommand(tableName) => - Seq(execution.UncacheTableCommand(tableName)) + Seq(ExecutedCommand( + execution.UncacheTableCommand(tableName))) case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index afe3f3f07440c..b8fa4b019953e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -26,34 +26,20 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.{SQLConf, SQLContext} -// TODO: DELETE ME... -trait Command { - this: SparkPlan => - - /** - * A concrete command should override this lazy field to wrap up any side effects caused by the - * command or any other computation that should be evaluated exactly once. The value of this field - * can be used as the contents of the corresponding RDD generated from the physical plan of this - * command. - * - * The `execute()` method of all the physical command classes should reference `sideEffectResult` - * so that the command can be executed eagerly right after the command query is created. - */ - protected lazy val sideEffectResult: Seq[Row] = Seq.empty[Row] - - override def executeCollect(): Array[Row] = sideEffectResult.toArray - - override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) -} - -// TODO: Replace command with runnable command. +/** + * A logical command that is executed for its side-effects. `RunnableCommand`s are + * wrapped in `ExecutedCommand` during execution. + */ trait RunnableCommand extends logical.Command { self: Product => - def output: Seq[Attribute] def run(sqlContext: SQLContext): Seq[Row] } +/** + * A physical operator that executes the run method of a `RunnableCommand` and + * saves the result to prevent multiple executions. + */ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { /** * A concrete command should override this lazy field to wrap up any side effects caused by the @@ -79,43 +65,41 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { * :: DeveloperApi :: */ @DeveloperApi -case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])( - @transient context: SQLContext) - extends LeafNode with Command with Logging { +case class SetCommand( + kv: Option[(String, Option[String])], + override val output: Seq[Attribute]) extends RunnableCommand with Logging { - override protected lazy val sideEffectResult: Seq[Row] = kv match { + override def run(sqlContext: SQLContext) = kv match { // Configures the deprecated "mapred.reduce.tasks" property. case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) => logWarning( s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - context.setConf(SQLConf.SHUFFLE_PARTITIONS, value) + sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS, value) Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) // Configures a single property. case Some((key, Some(value))) => - context.setConf(key, value) + sqlContext.setConf(key, value) Seq(Row(s"$key=$value")) - // Queries all key-value pairs that are set in the SQLConf of the context. Notice that different - // from Hive, here "SET -v" is an alias of "SET". (In Hive, "SET" returns all changed properties - // while "SET -v" returns all properties.) + // Queries all key-value pairs that are set in the SQLConf of the sqlContext. + // Notice that different from Hive, here "SET -v" is an alias of "SET". + // (In Hive, "SET" returns all changed properties while "SET -v" returns all properties.) case Some(("-v", None)) | None => - context.getAllConfs.map { case (k, v) => Row(s"$k=$v") }.toSeq + sqlContext.getAllConfs.map { case (k, v) => Row(s"$k=$v") }.toSeq // Queries the deprecated "mapred.reduce.tasks" property. case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, None)) => logWarning( s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${context.numShufflePartitions}")) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${sqlContext.numShufflePartitions}")) // Queries a single property. case Some((key, None)) => - Seq(Row(s"$key=${context.getConf(key, "")}")) + Seq(Row(s"$key=${sqlContext.getConf(key, "")}")) } - - override def otherCopyArgs = context :: Nil } /** @@ -128,22 +112,19 @@ case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribut */ @DeveloperApi case class ExplainCommand( - logicalPlan: LogicalPlan, output: Seq[Attribute], extended: Boolean)( - @transient context: SQLContext) - extends LeafNode with Command { + logicalPlan: LogicalPlan, + override val output: Seq[Attribute], extended: Boolean) extends RunnableCommand { // Run through the optimizer to generate the physical plan. - override protected lazy val sideEffectResult: Seq[Row] = try { + override def run(sqlContext: SQLContext) = try { // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. - val queryExecution = context.executePlan(logicalPlan) + val queryExecution = sqlContext.executePlan(logicalPlan) val outputString = if (extended) queryExecution.toString else queryExecution.simpleString outputString.split("\n").map(Row(_)) } catch { case cause: TreeNodeException[_] => ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_)) } - - override def otherCopyArgs = context :: Nil } /** @@ -153,10 +134,9 @@ case class ExplainCommand( case class CacheTableCommand( tableName: String, plan: Option[LogicalPlan], - isLazy: Boolean) - extends LeafNode with Command { + isLazy: Boolean) extends RunnableCommand { - override protected lazy val sideEffectResult = { + override def run(sqlContext: SQLContext) = { import sqlContext._ plan.foreach(_.registerTempTable(tableName)) @@ -178,8 +158,9 @@ case class CacheTableCommand( * :: DeveloperApi :: */ @DeveloperApi -case class UncacheTableCommand(tableName: String) extends LeafNode with Command { - override protected lazy val sideEffectResult: Seq[Row] = { +case class UncacheTableCommand(tableName: String) extends RunnableCommand { + + override def run(sqlContext: SQLContext) = { sqlContext.table(tableName).unpersist() Seq.empty[Row] } @@ -191,11 +172,11 @@ case class UncacheTableCommand(tableName: String) extends LeafNode with Command * :: DeveloperApi :: */ @DeveloperApi -case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( - @transient context: SQLContext) - extends LeafNode with Command { +case class DescribeCommand( + child: SparkPlan, + override val output: Seq[Attribute]) extends RunnableCommand { - override protected lazy val sideEffectResult: Seq[Row] = { + override def run(sqlContext: SQLContext) = { Row("# Registered as a temporary table", null, null) +: child.output.map(field => Row(field.name, field.dataType.toString, null)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala index 430ffb29989ea..ebf7003ff9e57 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala @@ -21,6 +21,7 @@ import scala.language.implicitConversions import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.{AbstractSparkSQLParser, SqlLexical} +import org.apache.spark.sql.hive.execution.{AddJar, AddFile, HiveNativeCommand} /** * A parser that recognizes all HiveQL constructs together with Spark SQL specific extensions. @@ -52,7 +53,7 @@ private[hive] class ExtendedHiveQlParser extends AbstractSparkSQLParser { protected lazy val dfs: Parser[LogicalPlan] = DFS ~> wholeInput ^^ { - case command => NativeCommand(command.trim) + case command => HiveNativeCommand(command.trim) } private lazy val addFile: Parser[LogicalPlan] = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 7de440773023f..56fe27a77b838 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -39,8 +39,8 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperat import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types.DecimalType import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.execution.{ExtractPythonUdfs, QueryExecutionException, Command => PhysicalCommand} -import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand +import org.apache.spark.sql.execution.{SparkPlan, ExecutedCommand, ExtractPythonUdfs, QueryExecutionException} +import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DescribeHiveTableCommand} import org.apache.spark.sql.sources.DataSourceStrategy /** @@ -340,7 +340,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override def strategies: Seq[Strategy] = extraStrategies ++ Seq( DataSourceStrategy, - CommandStrategy(self), + CommandStrategy, HiveCommandStrategy(self), TakeOrdered, ParquetOperations, @@ -369,11 +369,17 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * execution is simply passed back to Hive. */ def stringResult(): Seq[String] = executedPlan match { - case describeHiveTableCommand: DescribeHiveTableCommand => + case ExecutedCommand(desc: DescribeHiveTableCommand) => // If it is a describe command for a Hive table, we want to have the output format // be similar with Hive. - describeHiveTableCommand.hiveString - case command: PhysicalCommand => + desc.run(self).map { + case Row(name: String, dataType: String, comment) => + Seq(name, dataType, + Option(comment.asInstanceOf[String]).getOrElse("")) + .map(s => String.format(s"%-20s", s)) + .mkString("\t") + } + case command: ExecutedCommand => command.executeCollect().map(_.head.toString) case other => @@ -386,7 +392,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override def simpleString: String = logical match { - case _: NativeCommand => "" + case _: HiveNativeCommand => "" case _: SetCommand => "" case _ => super.simpleString } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d8b10b78c6c59..b31a3ec25096b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.hive import java.io.IOException import java.util.{List => JList} +import org.apache.spark.sql.execution.SparkPlan + import scala.util.parsing.combinator.RegexParsers import org.apache.hadoop.util.ReflectionUtils @@ -286,14 +288,24 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Some(sa.getQB().getTableDesc) } - CreateTableAsSelect(Some(databaseName), tblName, child, allowExisting, desc) + execution.CreateTableAsSelect( + databaseName, + tableName, + child, + allowExisting, + desc) case p: LogicalPlan if p.resolved => p case p @ CreateTableAsSelect(db, tableName, child, allowExisting, None) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - CreateTableAsSelect(Some(databaseName), tblName, child, allowExisting, None) + execution.CreateTableAsSelect( + databaseName, + tableName, + child, + allowExisting, + None) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 5939276f6d65e..3f3d9e7cd4fbe 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable} /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -44,14 +45,6 @@ import scala.collection.JavaConversions._ */ private[hive] case object NativePlaceholder extends Command -private[hive] case class AddFile(filePath: String) extends Command - -private[hive] case class AddJar(path: String) extends Command - -private[hive] case class DropTable(tableName: String, ifExists: Boolean) extends Command - -private[hive] case class AnalyzeTable(tableName: String) extends Command - /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { protected val nativeCommands = Seq( @@ -239,10 +232,10 @@ private[hive] object HiveQl { try { val tree = getAst(sql) if (nativeCommands contains tree.getText) { - NativeCommand(sql) + HiveNativeCommand(sql) } else { nodeToPlan(tree) match { - case NativePlaceholder => NativeCommand(sql) + case NativePlaceholder => HiveNativeCommand(sql) case other => other } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 4ebd59db83515..d3f6381b69a4d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.hive -import org.apache.hadoop.hive.ql.parse.ASTNode -import org.apache.hadoop.hive.ql.plan.CreateTableDesc - import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ @@ -28,7 +25,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType -import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan, PhysicalRDD} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation @@ -177,25 +174,10 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite)(hiveContext) :: Nil + table, partition, planLater(child), overwrite) :: Nil case hive.InsertIntoHiveTable(table: MetastoreRelation, partition, child, overwrite) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite)(hiveContext) :: Nil - case logical.CreateTableAsSelect( - Some(database), tableName, child, allowExisting, Some(desc: CreateTableDesc)) => - execution.CreateTableAsSelect( - database, - tableName, - child, - allowExisting, - Some(desc)) :: Nil - case logical.CreateTableAsSelect(Some(database), tableName, child, allowExisting, None) => - execution.CreateTableAsSelect( - database, - tableName, - child, - allowExisting, - None) :: Nil + table, partition, planLater(child), overwrite) :: Nil case _ => Nil } } @@ -227,23 +209,14 @@ private[hive] trait HiveStrategies { case class HiveCommandStrategy(context: HiveContext) extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil - - case hive.DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil - - case hive.AddJar(path) => execution.AddJar(path) :: Nil - - case hive.AddFile(path) => execution.AddFile(path) :: Nil - - case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil - case describe: logical.DescribeCommand => val resolvedTable = context.executePlan(describe.table).analyzed resolvedTable match { case t: MetastoreRelation => - Seq(DescribeHiveTableCommand(t, describe.output, describe.isExtended)(context)) + ExecutedCommand( + DescribeHiveTableCommand(t, describe.output, describe.isExtended)) :: Nil case o: LogicalPlan => - Seq(DescribeCommand(planLater(o), describe.output)(context)) + ExecutedCommand(DescribeCommand(planLater(o), describe.output)) :: Nil } case _ => Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index eedb57de52ba9..b2149bd95a336 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -20,9 +20,6 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.session.SessionState - import scala.collection.mutable import scala.language.implicitConversions @@ -37,10 +34,11 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan, NativeCommand} +import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.hive.execution.HiveNativeCommand /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -161,7 +159,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { abstract class QueryExecution extends super.QueryExecution { override lazy val analyzed = { val describedTables = logical match { - case NativeCommand(describedTable(tbl)) => tbl :: Nil + case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil case CacheTableCommand(tbl, _, _) => tbl :: Nil case _ => Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index b83689ceabb84..fe21454e7fb38 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.ql.plan.CreateTableDesc import org.apache.spark.annotation.Experimental -import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} -import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} +import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.MetastoreRelation @@ -44,28 +44,23 @@ case class CreateTableAsSelect( tableName: String, query: LogicalPlan, allowExisting: Boolean, - desc: Option[CreateTableDesc]) extends LeafNode with Command { + desc: Option[CreateTableDesc]) extends RunnableCommand { - def output = Seq.empty + override def run(sqlContext: SQLContext) = { + val hiveContext = sqlContext.asInstanceOf[HiveContext] + lazy val metastoreRelation: MetastoreRelation = { + // Create Hive Table + hiveContext.catalog.createTable(database, tableName, query.output, allowExisting, desc) - private[this] def sc = sqlContext.asInstanceOf[HiveContext] - - // A lazy computing of the metastoreRelation - private[this] lazy val metastoreRelation: MetastoreRelation = { - // Create Hive Table - sc.catalog.createTable(database, tableName, query.output, allowExisting, desc) - - // Get the Metastore Relation - sc.catalog.lookupRelation(Some(database), tableName, None) match { - case r: MetastoreRelation => r + // Get the Metastore Relation + hiveContext.catalog.lookupRelation(Some(database), tableName, None) match { + case r: MetastoreRelation => r + } } - } - - override protected[sql] lazy val sideEffectResult: Seq[Row] = { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data // processing. - if (sc.catalog.tableExists(Some(database), tableName)) { + if (hiveContext.catalog.tableExists(Some(database), tableName)) { if (allowExisting) { // table already exists, will do nothing, to keep consistent with Hive } else { @@ -73,17 +68,12 @@ case class CreateTableAsSelect( new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName") } } else { - sc.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd + hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd } Seq.empty[Row] } - override def execute(): RDD[Row] = { - sideEffectResult - sparkContext.emptyRDD[Row] - } - override def argString: String = { s"[Database:$database, TableName: $tableName, InsertIntoHiveTable]\n" + query.toString } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index 5d98834c6fb33..bfacc51ef57ab 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -22,11 +22,11 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} -import org.apache.spark.sql.execution.{Command, LeafNode} +import org.apache.spark.sql.execution.{SparkPlan, RunnableCommand} import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.SQLContext /** * Implementation for "describe [extended] table". @@ -36,21 +36,10 @@ import org.apache.spark.sql.hive.HiveShim @DeveloperApi case class DescribeHiveTableCommand( table: MetastoreRelation, - output: Seq[Attribute], - isExtended: Boolean)( - @transient context: HiveContext) - extends LeafNode with Command { + override val output: Seq[Attribute], + isExtended: Boolean) extends RunnableCommand { - // Strings with the format like Hive. It is used for result comparison in our unit tests. - lazy val hiveString: Seq[String] = sideEffectResult.map { - case Row(name: String, dataType: String, comment) => - Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse("")) - .map(s => String.format(s"%-20s", s)) - .mkString("\t") - } - - override protected lazy val sideEffectResult: Seq[Row] = { + override def run(sqlContext: SQLContext) = { // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil @@ -75,6 +64,4 @@ case class DescribeHiveTableCommand( Row(name, dataType, comment) } } - - override def otherCopyArgs = context :: Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala similarity index 64% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala index 6930c2babd117..8ba818af5f9d0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala @@ -1,38 +1,38 @@ -/* - * 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.sql.hive.execution - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow, Row} -import org.apache.spark.sql.execution.{Command, LeafNode} -import org.apache.spark.sql.hive.HiveContext - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class NativeCommand( - sql: String, output: Seq[Attribute])( - @transient context: HiveContext) - extends LeafNode with Command { - - override protected lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_)) - - override def otherCopyArgs = context :: Nil -} +/* + * 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.sql.hive.execution + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} +import org.apache.spark.sql.execution.RunnableCommand +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.types.StringType + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class HiveNativeCommand(sql: String) extends RunnableCommand { + + override def output = + Seq(AttributeReference("result", StringType, nullable = false)()) + + override def run(sqlContext: SQLContext) = + sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_)) +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 81390f626726c..ca0ec1513917f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -21,7 +21,6 @@ import java.util import scala.collection.JavaConversions._ -import org.apache.hadoop.hive.common.`type`.HiveVarchar import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils @@ -31,14 +30,12 @@ import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} +import org.apache.spark.sql.execution.{UnaryNode, SparkPlan} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.HiveShim._ @@ -52,10 +49,9 @@ case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], child: SparkPlan, - overwrite: Boolean) - (@transient sc: HiveContext) - extends UnaryNode with Command with HiveInspectors { + overwrite: Boolean) extends UnaryNode with HiveInspectors { + @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) @transient private lazy val db = Hive.get(sc.hiveconf) @@ -66,8 +62,6 @@ case class InsertIntoHiveTable( serializer } - override def otherCopyArgs = sc :: Nil - def output = child.output def saveAsHiveFile( @@ -134,7 +128,7 @@ case class InsertIntoHiveTable( * * Note: this is run once and then kept to avoid double insertions. */ - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + protected[sql] lazy val sideEffectResult: Seq[Row] = { // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc @@ -256,4 +250,8 @@ case class InsertIntoHiveTable( // TODO: implement hive compatibility as rules. Seq.empty[Row] } + + override def executeCollect(): Array[Row] = sideEffectResult.toArray + + override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 903075edf7e04..6fc4153f6a5df 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.execution.{Command, LeafNode} +import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.SQLContext /** * :: DeveloperApi :: @@ -32,13 +32,10 @@ import org.apache.spark.sql.hive.HiveContext * in the Hive metastore. */ @DeveloperApi -case class AnalyzeTable(tableName: String) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] +case class AnalyzeTable(tableName: String) extends RunnableCommand { - def output = Seq.empty - - override protected lazy val sideEffectResult: Seq[Row] = { - hiveContext.analyze(tableName) + override def run(sqlContext: SQLContext) = { + sqlContext.asInstanceOf[HiveContext].analyze(tableName) Seq.empty[Row] } } @@ -48,12 +45,12 @@ case class AnalyzeTable(tableName: String) extends LeafNode with Command { * Drops a table from the metastore and removes it if it is cached. */ @DeveloperApi -case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] - - def output = Seq.empty +case class DropTable( + tableName: String, + ifExists: Boolean) extends RunnableCommand { - override protected lazy val sideEffectResult: Seq[Row] = { + override def run(sqlContext: SQLContext) = { + val hiveContext = sqlContext.asInstanceOf[HiveContext] val ifExistsClause = if (ifExists) "IF EXISTS " else "" hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") hiveContext.catalog.unregisterTable(None, tableName) @@ -65,12 +62,10 @@ case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with * :: DeveloperApi :: */ @DeveloperApi -case class AddJar(path: String) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] +case class AddJar(path: String) extends RunnableCommand { - override def output = Seq.empty - - override protected lazy val sideEffectResult: Seq[Row] = { + override def run(sqlContext: SQLContext) = { + val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) Seq.empty[Row] @@ -81,12 +76,10 @@ case class AddJar(path: String) extends LeafNode with Command { * :: DeveloperApi :: */ @DeveloperApi -case class AddFile(path: String) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] - - override def output = Seq.empty +case class AddFile(path: String) extends RunnableCommand { - override protected lazy val sideEffectResult: Seq[Row] = { + override def run(sqlContext: SQLContext) = { + val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD FILE $path") hiveContext.sparkContext.addFile(path) Seq.empty[Row] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index a90fc023e67d8..ff4071d8e2f10 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -22,10 +22,10 @@ import org.scalatest.BeforeAndAfterAll import scala.reflect.ClassTag import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.catalyst.plans.logical.NativeCommand import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.hive.execution._ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { TestHive.reset() @@ -51,19 +51,19 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { assertAnalyzeCommand( "ANALYZE TABLE Table1 COMPUTE STATISTICS", - classOf[NativeCommand]) + classOf[HiveNativeCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", - classOf[NativeCommand]) + classOf[HiveNativeCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", - classOf[NativeCommand]) + classOf[HiveNativeCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS", - classOf[NativeCommand]) + classOf[HiveNativeCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS noscan", - classOf[NativeCommand]) + classOf[HiveNativeCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 COMPUTE STATISTICS nOscAn", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 44eb4cfa59335..8011f9b8773b3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -24,7 +24,6 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand => LogicalNativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.hive.test.TestHive @@ -142,14 +141,14 @@ abstract class HiveComparisonTest // Hack: Hive simply prints the result of a SET command to screen, // and does not return it as a query answer. case _: SetCommand => Seq("0") - case LogicalNativeCommand(c) if c.toLowerCase.contains("desc") => + case HiveNativeCommand(c) if c.toLowerCase.contains("desc") => answer .filterNot(nonDeterministicLine) .map(_.replaceAll("from deserializer", "")) .map(_.replaceAll("None", "")) .map(_.trim) .filterNot(_ == "") - case _: LogicalNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") + case _: HiveNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _: ExplainCommand => answer case _: DescribeCommand => // Filter out non-deterministic lines and lines which do not have actual results but From ee1fb97a97d5ac18cd2ad8028e84ecbd988fb811 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 18 Dec 2014 21:00:49 -0800 Subject: [PATCH 075/227] [SPARK-4728][MLLib] Add exponential, gamma, and log normal sampling to MLlib da... ...ta generators This patch adds: * Exponential, gamma, and log normal generators that wrap Apache Commons math3 to the private API * Functions for generating exponential, gamma, and log normal RDDs and vector RDDs * Tests for the above Author: RJ Nowling Closes #3680 from rnowling/spark4728 and squashes the following commits: 455f50a [RJ Nowling] Add tests for exponential, gamma, and log normal samplers to JavaRandomRDDsSuite 3e1134a [RJ Nowling] Fix val/var, unncessary creation of Distribution objects when setting seeds, and import line longer than line wrap limits 58f5b97 [RJ Nowling] Fix bounds in tests so they scale with variance, not stdev 84fd98d [RJ Nowling] Add more values for testing distributions. 9f96232 [RJ Nowling] [SPARK-4728] Add exponential, gamma, and log normal sampling to MLlib data generators --- .../mllib/random/RandomDataGenerator.scala | 69 +++- .../spark/mllib/random/RandomRDDs.scala | 363 ++++++++++++++++++ .../mllib/random/JavaRandomRDDsSuite.java | 99 +++++ .../random/RandomDataGeneratorSuite.scala | 52 ++- .../spark/mllib/random/RandomRDDsSuite.scala | 43 +++ 5 files changed, 622 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 51f9b8657c640..405bae62ee8b6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.random -import org.apache.commons.math3.distribution.PoissonDistribution +import org.apache.commons.math3.distribution.{ExponentialDistribution, + GammaDistribution, LogNormalDistribution, PoissonDistribution} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} @@ -88,14 +89,76 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { @DeveloperApi class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { - private var rng = new PoissonDistribution(mean) + private val rng = new PoissonDistribution(mean) override def nextValue(): Double = rng.sample() override def setSeed(seed: Long) { - rng = new PoissonDistribution(mean) rng.reseedRandomGenerator(seed) } override def copy(): PoissonGenerator = new PoissonGenerator(mean) } + +/** + * :: DeveloperApi :: + * Generates i.i.d. samples from the exponential distribution with the given mean. + * + * @param mean mean for the exponential distribution. + */ +@DeveloperApi +class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] { + + private val rng = new ExponentialDistribution(mean) + + override def nextValue(): Double = rng.sample() + + override def setSeed(seed: Long) { + rng.reseedRandomGenerator(seed) + } + + override def copy(): ExponentialGenerator = new ExponentialGenerator(mean) +} + +/** + * :: DeveloperApi :: + * Generates i.i.d. samples from the gamma distribution with the given shape and scale. + * + * @param shape shape for the gamma distribution. + * @param scale scale for the gamma distribution + */ +@DeveloperApi +class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] { + + private val rng = new GammaDistribution(shape, scale) + + override def nextValue(): Double = rng.sample() + + override def setSeed(seed: Long) { + rng.reseedRandomGenerator(seed) + } + + override def copy(): GammaGenerator = new GammaGenerator(shape, scale) +} + +/** + * :: DeveloperApi :: + * Generates i.i.d. samples from the log normal distribution with the + * given mean and standard deviation. + * + * @param mean mean for the log normal distribution. + * @param std standard deviation for the log normal distribution + */ +@DeveloperApi +class LogNormalGenerator(val mean: Double, val std: Double) extends RandomDataGenerator[Double] { + + private val rng = new LogNormalDistribution(mean, std) + + override def nextValue(): Double = rng.sample() + + override def setSeed(seed: Long) { + rng.reseedRandomGenerator(seed) + } + + override def copy(): LogNormalGenerator = new LogNormalGenerator(mean, std) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index c5f4b084321f7..955c593a085d5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -176,6 +176,176 @@ object RandomRDDs { JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size)) } + /** + * Generates an RDD comprised of i.i.d. samples from the exponential distribution with + * the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or 1 / lambda, for the exponential distribution. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + def exponentialRDD( + sc: SparkContext, + mean: Double, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { + val exponential = new ExponentialGenerator(mean) + randomRDD(sc, exponential, size, numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#exponentialRDD]]. + */ + def exponentialJavaRDD( + jsc: JavaSparkContext, + mean: Double, + size: Long, + numPartitions: Int, + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(exponentialRDD(jsc.sc, mean, size, numPartitions, seed)) + } + + /** + * [[RandomRDDs#exponentialJavaRDD]] with the default seed. + */ + def exponentialJavaRDD( + jsc: JavaSparkContext, + mean: Double, + size: Long, + numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(exponentialRDD(jsc.sc, mean, size, numPartitions)) + } + + /** + * [[RandomRDDs#exponentialJavaRDD]] with the default number of partitions and the default seed. + */ + def exponentialJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(exponentialRDD(jsc.sc, mean, size)) + } + + /** + * Generates an RDD comprised of i.i.d. samples from the gamma distribution with the input + * shape and scale. + * + * @param sc SparkContext used to create the RDD. + * @param shape shape parameter (> 0) for the gamma distribution + * @param scale scale parameter (> 0) for the gamma distribution + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + def gammaRDD( + sc: SparkContext, + shape: Double, + scale: Double, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { + val gamma = new GammaGenerator(shape, scale) + randomRDD(sc, gamma, size, numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#gammaRDD]]. + */ + def gammaJavaRDD( + jsc: JavaSparkContext, + shape: Double, + scale: Double, + size: Long, + numPartitions: Int, + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(gammaRDD(jsc.sc, shape, scale, size, numPartitions, seed)) + } + + /** + * [[RandomRDDs#gammaJavaRDD]] with the default seed. + */ + def gammaJavaRDD( + jsc: JavaSparkContext, + shape: Double, + scale: Double, + size: Long, + numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(gammaRDD(jsc.sc, shape, scale, size, numPartitions)) + } + + /** + * [[RandomRDDs#gammaJavaRDD]] with the default number of partitions and the default seed. + */ + def gammaJavaRDD( + jsc: JavaSparkContext, + shape: Double, + scale: Double, + size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(gammaRDD(jsc.sc, shape, scale, size)) + } + + /** + * Generates an RDD comprised of i.i.d. samples from the log normal distribution with the input + * mean and standard deviation + * + * @param sc SparkContext used to create the RDD. + * @param mean mean for the log normal distribution + * @param std standard deviation for the log normal distribution + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + def logNormalRDD( + sc: SparkContext, + mean: Double, + std: Double, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { + val logNormal = new LogNormalGenerator(mean, std) + randomRDD(sc, logNormal, size, numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#logNormalRDD]]. + */ + def logNormalJavaRDD( + jsc: JavaSparkContext, + mean: Double, + std: Double, + size: Long, + numPartitions: Int, + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(logNormalRDD(jsc.sc, mean, std, size, numPartitions, seed)) + } + + /** + * [[RandomRDDs#logNormalJavaRDD]] with the default seed. + */ + def logNormalJavaRDD( + jsc: JavaSparkContext, + mean: Double, + std: Double, + size: Long, + numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(logNormalRDD(jsc.sc, mean, std, size, numPartitions)) + } + + /** + * [[RandomRDDs#logNormalJavaRDD]] with the default number of partitions and the default seed. + */ + def logNormalJavaRDD( + jsc: JavaSparkContext, + mean: Double, + std: Double, + size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(logNormalRDD(jsc.sc, mean, std, size)) + } + + /** * :: DeveloperApi :: * Generates an RDD comprised of i.i.d. samples produced by the input RandomDataGenerator. @@ -307,6 +477,72 @@ object RandomRDDs { normalVectorRDD(jsc.sc, numRows, numCols).toJavaRDD() } + /** + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from a + * log normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean of the log normal distribution. + * @param std Standard deviation of the log normal distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Vector] with vectors containing i.i.d. samples. + */ + def logNormalVectorRDD( + sc: SparkContext, + mean: Double, + std: Double, + numRows: Long, + numCols: Int, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { + val logNormal = new LogNormalGenerator(mean, std) + randomVectorRDD(sc, logNormal, numRows, numCols, + numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#logNormalVectorRDD]]. + */ + def logNormalJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, + std: Double, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): JavaRDD[Vector] = { + logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols, numPartitions, seed).toJavaRDD() + } + + /** + * [[RandomRDDs#logNormalJavaVectorRDD]] with the default seed. + */ + def logNormalJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, + std: Double, + numRows: Long, + numCols: Int, + numPartitions: Int): JavaRDD[Vector] = { + logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols, numPartitions).toJavaRDD() + } + + /** + * [[RandomRDDs#logNormalJavaVectorRDD]] with the default number of partitions and + * the default seed. + */ + def logNormalJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, + std: Double, + numRows: Long, + numCols: Int): JavaRDD[Vector] = { + logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols).toJavaRDD() + } + /** * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * Poisson distribution with the input mean. @@ -366,6 +602,133 @@ object RandomRDDs { poissonVectorRDD(jsc.sc, mean, numRows, numCols).toJavaRDD() } + /** + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * exponential distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or 1 / lambda, for the Exponential distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) + * @param seed Random seed (default: a random long integer). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Exp(mean). + */ + def exponentialVectorRDD( + sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { + val exponential = new ExponentialGenerator(mean) + randomVectorRDD(sc, exponential, numRows, numCols, + numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#exponentialVectorRDD]]. + */ + def exponentialJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): JavaRDD[Vector] = { + exponentialVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions, seed).toJavaRDD() + } + + /** + * [[RandomRDDs#exponentialJavaVectorRDD]] with the default seed. + */ + def exponentialJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int): JavaRDD[Vector] = { + exponentialVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions).toJavaRDD() + } + + /** + * [[RandomRDDs#exponentialJavaVectorRDD]] with the default number of partitions + * and the default seed. + */ + def exponentialJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, + numRows: Long, + numCols: Int): JavaRDD[Vector] = { + exponentialVectorRDD(jsc.sc, mean, numRows, numCols).toJavaRDD() + } + + + /** + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * gamma distribution with the input shape and scale. + * + * @param sc SparkContext used to create the RDD. + * @param shape shape parameter (> 0) for the gamma distribution. + * @param scale scale parameter (> 0) for the gamma distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) + * @param seed Random seed (default: a random long integer). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Exp(mean). + */ + def gammaVectorRDD( + sc: SparkContext, + shape: Double, + scale: Double, + numRows: Long, + numCols: Int, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { + val gamma = new GammaGenerator(shape, scale) + randomVectorRDD(sc, gamma, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#gammaVectorRDD]]. + */ + def gammaJavaVectorRDD( + jsc: JavaSparkContext, + shape: Double, + scale: Double, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): JavaRDD[Vector] = { + gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols, numPartitions, seed).toJavaRDD() + } + + /** + * [[RandomRDDs#gammaJavaVectorRDD]] with the default seed. + */ + def gammaJavaVectorRDD( + jsc: JavaSparkContext, + shape: Double, + scale: Double, + numRows: Long, + numCols: Int, + numPartitions: Int): JavaRDD[Vector] = { + gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols, numPartitions).toJavaRDD() + } + + /** + * [[RandomRDDs#gammaJavaVectorRDD]] with the default number of partitions and the default seed. + */ + def gammaJavaVectorRDD( + jsc: JavaSparkContext, + shape: Double, + scale: Double, + numRows: Long, + numCols: Int): JavaRDD[Vector] = { + gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols).toJavaRDD() + } + + /** * :: DeveloperApi :: * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java index a725736ca1a58..fcc13c00cbdc5 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -69,6 +69,21 @@ public void testNormalRDD() { } } + @Test + public void testLNormalRDD() { + double mean = 4.0; + double std = 2.0; + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = logNormalJavaRDD(sc, mean, std, m); + JavaDoubleRDD rdd2 = logNormalJavaRDD(sc, mean, std, m, p); + JavaDoubleRDD rdd3 = logNormalJavaRDD(sc, mean, std, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + @Test public void testPoissonRDD() { double mean = 2.0; @@ -83,6 +98,36 @@ public void testPoissonRDD() { } } + @Test + public void testExponentialRDD() { + double mean = 2.0; + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = exponentialJavaRDD(sc, mean, m); + JavaDoubleRDD rdd2 = exponentialJavaRDD(sc, mean, m, p); + JavaDoubleRDD rdd3 = exponentialJavaRDD(sc, mean, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test + public void testGammaRDD() { + double shape = 1.0; + double scale = 2.0; + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = gammaJavaRDD(sc, shape, scale, m); + JavaDoubleRDD rdd2 = gammaJavaRDD(sc, shape, scale, m, p); + JavaDoubleRDD rdd3 = gammaJavaRDD(sc, shape, scale, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test @SuppressWarnings("unchecked") public void testUniformVectorRDD() { @@ -115,6 +160,24 @@ public void testNormalVectorRDD() { } } + @Test + @SuppressWarnings("unchecked") + public void testLogNormalVectorRDD() { + double mean = 4.0; + double std = 2.0; + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = logNormalJavaVectorRDD(sc, mean, std, m, n); + JavaRDD rdd2 = logNormalJavaVectorRDD(sc, mean, std, m, n, p); + JavaRDD rdd3 = logNormalJavaVectorRDD(sc, mean, std, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } + @Test @SuppressWarnings("unchecked") public void testPoissonVectorRDD() { @@ -131,4 +194,40 @@ public void testPoissonVectorRDD() { Assert.assertEquals(n, rdd.first().size()); } } + + @Test + @SuppressWarnings("unchecked") + public void testExponentialVectorRDD() { + double mean = 2.0; + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = exponentialJavaVectorRDD(sc, mean, m, n); + JavaRDD rdd2 = exponentialJavaVectorRDD(sc, mean, m, n, p); + JavaRDD rdd3 = exponentialJavaVectorRDD(sc, mean, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testGammaVectorRDD() { + double shape = 1.0; + double scale = 2.0; + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = gammaJavaVectorRDD(sc, shape, scale, m, n); + JavaRDD rdd2 = gammaJavaVectorRDD(sc, shape, scale, m, n, p); + JavaRDD rdd3 = gammaJavaVectorRDD(sc, shape, scale, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala index 3df7c128af5ab..b792d819fdabb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.random +import scala.math + import org.scalatest.FunSuite import org.apache.spark.util.StatCounter @@ -25,7 +27,6 @@ import org.apache.spark.util.StatCounter class RandomDataGeneratorSuite extends FunSuite { def apiChecks(gen: RandomDataGenerator[Double]) { - // resetting seed should generate the same sequence of random numbers gen.setSeed(42L) val array1 = (0 until 1000).map(_ => gen.nextValue()) @@ -79,6 +80,26 @@ class RandomDataGeneratorSuite extends FunSuite { distributionChecks(normal, 0.0, 1.0) } + test("LogNormalGenerator") { + List((0.0, 1.0), (0.0, 2.0), (2.0, 1.0), (2.0, 2.0)).map { + case (mean: Double, vari: Double) => + val normal = new LogNormalGenerator(mean, math.sqrt(vari)) + apiChecks(normal) + + // mean of log normal = e^(mean + var / 2) + val expectedMean = math.exp(mean + 0.5 * vari) + + // variance of log normal = (e^var - 1) * e^(2 * mean + var) + val expectedStd = math.sqrt((math.exp(vari) - 1.0) * math.exp(2.0 * mean + vari)) + + // since sampling error increases with variance, let's set + // the absolute tolerance as a percentage + val epsilon = 0.05 * expectedStd * expectedStd + + distributionChecks(normal, expectedMean, expectedStd, epsilon) + } + } + test("PoissonGenerator") { // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced. for (mean <- List(1.0, 5.0, 100.0)) { @@ -87,4 +108,33 @@ class RandomDataGeneratorSuite extends FunSuite { distributionChecks(poisson, mean, math.sqrt(mean), 0.1) } } + + test("ExponentialGenerator") { + // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced. + for (mean <- List(2.0, 5.0, 10.0, 50.0, 100.0)) { + val exponential = new ExponentialGenerator(mean) + apiChecks(exponential) + // var of exp = lambda^-2 = (1.0 / mean)^-2 = mean^2 + + // since sampling error increases with variance, let's set + // the absolute tolerance as a percentage + val epsilon = 0.05 * mean * mean + + distributionChecks(exponential, mean, mean, epsilon) + } + } + + test("GammaGenerator") { + // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced. + List((1.0, 2.0), (2.0, 2.0), (3.0, 2.0), (5.0, 1.0), (9.0, 0.5)).map { + case (shape: Double, scale: Double) => + val gamma = new GammaGenerator(shape, scale) + apiChecks(gamma) + // mean of gamma = shape * scale + val expectedMean = shape * scale + // var of gamma = shape * scale^2 + val expectedStd = math.sqrt(shape * scale * scale) + distributionChecks(gamma, expectedMean, expectedStd, 0.1) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index ea5889b3ecd5e..6395188a0842a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -110,7 +110,19 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa test("randomRDD for different distributions") { val size = 100000L val numPartitions = 10 + + // mean of log normal = e^(mean + var / 2) + val logNormalMean = math.exp(0.5) + // variance of log normal = (e^var - 1) * e^(2 * mean + var) + val logNormalStd = math.sqrt((math.E - 1.0) * math.E) + val gammaScale = 1.0 + val gammaShape = 2.0 + // mean of gamma = shape * scale + val gammaMean = gammaShape * gammaScale + // var of gamma = shape * scale^2 + val gammaStd = math.sqrt(gammaShape * gammaScale * gammaScale) val poissonMean = 100.0 + val exponentialMean = 1.0 for (seed <- 0 until 5) { val uniform = RandomRDDs.uniformRDD(sc, size, numPartitions, seed) @@ -119,8 +131,18 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa val normal = RandomRDDs.normalRDD(sc, size, numPartitions, seed) testGeneratedRDD(normal, size, numPartitions, 0.0, 1.0) + val logNormal = RandomRDDs.logNormalRDD(sc, 0.0, 1.0, size, numPartitions, seed) + testGeneratedRDD(logNormal, size, numPartitions, logNormalMean, logNormalStd, 0.1) + val poisson = RandomRDDs.poissonRDD(sc, poissonMean, size, numPartitions, seed) testGeneratedRDD(poisson, size, numPartitions, poissonMean, math.sqrt(poissonMean), 0.1) + + val exponential = RandomRDDs.exponentialRDD(sc, exponentialMean, size, numPartitions, seed) + testGeneratedRDD(exponential, size, numPartitions, exponentialMean, exponentialMean, 0.1) + + val gamma = RandomRDDs.gammaRDD(sc, gammaShape, gammaScale, size, numPartitions, seed) + testGeneratedRDD(gamma, size, numPartitions, gammaMean, gammaStd, 0.1) + } // mock distribution to check that partitions have unique seeds @@ -132,7 +154,19 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa val rows = 1000L val cols = 100 val parts = 10 + + // mean of log normal = e^(mean + var / 2) + val logNormalMean = math.exp(0.5) + // variance of log normal = (e^var - 1) * e^(2 * mean + var) + val logNormalStd = math.sqrt((math.E - 1.0) * math.E) + val gammaScale = 1.0 + val gammaShape = 2.0 + // mean of gamma = shape * scale + val gammaMean = gammaShape * gammaScale + // var of gamma = shape * scale^2 + val gammaStd = math.sqrt(gammaShape * gammaScale * gammaScale) val poissonMean = 100.0 + val exponentialMean = 1.0 for (seed <- 0 until 5) { val uniform = RandomRDDs.uniformVectorRDD(sc, rows, cols, parts, seed) @@ -141,8 +175,17 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa val normal = RandomRDDs.normalVectorRDD(sc, rows, cols, parts, seed) testGeneratedVectorRDD(normal, rows, cols, parts, 0.0, 1.0) + val logNormal = RandomRDDs.logNormalVectorRDD(sc, 0.0, 1.0, rows, cols, parts, seed) + testGeneratedVectorRDD(logNormal, rows, cols, parts, logNormalMean, logNormalStd, 0.1) + val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) + + val exponential = RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed) + testGeneratedVectorRDD(exponential, rows, cols, parts, exponentialMean, exponentialMean, 0.1) + + val gamma = RandomRDDs.gammaVectorRDD(sc, gammaShape, gammaScale, rows, cols, parts, seed) + testGeneratedVectorRDD(gamma, rows, cols, parts, gammaMean, gammaStd, 0.1) } } } From d7fc69a8b5c92b2bbb71f95027e5283968cd0679 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 18 Dec 2014 21:41:02 -0800 Subject: [PATCH 076/227] [SPARK-4674] Refactor getCallSite The current version of `getCallSite` visits the collection of `StackTraceElement` twice. However, it is unnecessary since we can perform our work with a single visit. We also do not need to keep filtered `StackTraceElement`. Author: Liang-Chi Hsieh Closes #3532 from viirya/refactor_getCallSite and squashes the following commits: 62aa124 [Liang-Chi Hsieh] Fix style. e741017 [Liang-Chi Hsieh] Refactor getCallSite. --- .../scala/org/apache/spark/util/Utils.scala | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 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 9c04e45a58479..d16233a0bc760 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1024,13 +1024,6 @@ private[spark] object Utils extends Logging { * @param skipClass Function that is used to exclude non-user-code classes. */ def getCallSite(skipClass: String => Boolean = coreExclusionFunction): CallSite = { - val trace = Thread.currentThread.getStackTrace().filterNot { ste: StackTraceElement => - // When running under some profilers, the current stack trace might contain some bogus - // frames. This is intended to ensure that we don't crash in these situations by - // ignoring any frames that we can't examine. - ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace") - } - // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD // transformation, a SparkContext function (such as parallelize), or anything else that leads @@ -1040,27 +1033,34 @@ private[spark] object Utils extends Logging { var firstUserLine = 0 var insideSpark = true var callStack = new ArrayBuffer[String]() :+ "" - - for (el <- trace) { - if (insideSpark) { - if (skipClass(el.getClassName)) { - lastSparkMethod = if (el.getMethodName == "") { - // Spark method is a constructor; get its class name - el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) + + Thread.currentThread.getStackTrace().foreach { ste: StackTraceElement => + // When running under some profilers, the current stack trace might contain some bogus + // frames. This is intended to ensure that we don't crash in these situations by + // ignoring any frames that we can't examine. + if (ste != null && ste.getMethodName != null + && !ste.getMethodName.contains("getStackTrace")) { + if (insideSpark) { + if (skipClass(ste.getClassName)) { + lastSparkMethod = if (ste.getMethodName == "") { + // Spark method is a constructor; get its class name + ste.getClassName.substring(ste.getClassName.lastIndexOf('.') + 1) + } else { + ste.getMethodName + } + callStack(0) = ste.toString // Put last Spark method on top of the stack trace. } else { - el.getMethodName + firstUserLine = ste.getLineNumber + firstUserFile = ste.getFileName + callStack += ste.toString + insideSpark = false } - callStack(0) = el.toString // Put last Spark method on top of the stack trace. } else { - firstUserLine = el.getLineNumber - firstUserFile = el.getFileName - callStack += el.toString - insideSpark = false + callStack += ste.toString } - } else { - callStack += el.toString } } + val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt CallSite( shortForm = s"$lastSparkMethod at $firstUserFile:$firstUserLine", From 283263ffaa941e7e9ba147cf0ad377d9202d3761 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 18 Dec 2014 22:40:44 -0800 Subject: [PATCH 077/227] SPARK-3428. TaskMetrics for running tasks is missing GC time metrics Author: Sandy Ryza Closes #3684 from sryza/sandy-spark-3428 and squashes the following commits: cb827fe [Sandy Ryza] SPARK-3428. TaskMetrics for running tasks is missing GC time metrics --- .../main/scala/org/apache/spark/executor/Executor.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 52de6980ecbf8..da030f231fde8 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -145,6 +145,8 @@ private[spark] class Executor( } } + private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum + class TaskRunner( execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) extends Runnable { @@ -152,6 +154,7 @@ private[spark] class Executor( @volatile private var killed = false @volatile var task: Task[Any] = _ @volatile var attemptedTask: Option[Task[Any]] = None + @volatile var startGCTime: Long = _ def kill(interruptThread: Boolean) { logInfo(s"Executor is trying to kill $taskName (TID $taskId)") @@ -168,8 +171,7 @@ private[spark] class Executor( logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var taskStart: Long = 0 - def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum - val startGCTime = gcTime + startGCTime = gcTime try { val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) @@ -376,10 +378,13 @@ private[spark] class Executor( while (!isStopped) { val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + val curGCTime = gcTime + for (taskRunner <- runningTasks.values()) { if (!taskRunner.attemptedTask.isEmpty) { Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => metrics.updateShuffleReadMetrics + metrics.jvmGCTime = curGCTime - taskRunner.startGCTime if (isLocal) { // JobProgressListener will hold an reference of it during // onExecutorMetricsUpdate(), then JobProgressListener can not see From 5479450c45038ddb4485cd676f945a961f2420d8 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 19 Dec 2014 08:04:41 -0800 Subject: [PATCH 078/227] [SPARK-4901] [SQL] Hot fix for ByteWritables.copyBytes HiveInspectors.scala failed in compiling with Hadoop 1, as the BytesWritable.copyBytes is not available in Hadoop 1. Author: Cheng Hao Closes #3742 from chenghao-intel/settable_oi_hotfix and squashes the following commits: bb04d1f [Cheng Hao] hot fix for ByteWritables.copyBytes --- .../scala/org/apache/spark/sql/hive/HiveInspectors.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 06189341f827c..e1147478130dd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -297,7 +297,13 @@ private[hive] trait HiveInspectors { case x: ByteObjectInspector if x.preferWritable() => x.get(data) case x: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(x, data) case x: BinaryObjectInspector if x.preferWritable() => - x.getPrimitiveWritableObject(data).copyBytes() + // BytesWritable.copyBytes() only available since Hadoop2 + // In order to keep backward-compatible, we have to copy the + // bytes with old apis + val bw = x.getPrimitiveWritableObject(data) + val result = new Array[Byte](bw.getLength()) + System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength()) + result case x: DateObjectInspector if x.preferWritable() => x.getPrimitiveWritableObject(data).get() // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object From 8e253ebbf8c5867bd6b626ca8f20699d71aa6ddb Mon Sep 17 00:00:00 2001 From: scwf Date: Fri, 19 Dec 2014 08:29:38 -0800 Subject: [PATCH 079/227] [Build] Remove spark-staging-1038 Author: scwf Closes #3743 from scwf/abc and squashes the following commits: 7d98bc8 [scwf] removing spark-staging-1038 --- pom.xml | 12 ------------ project/SparkBuild.scala | 2 -- 2 files changed, 14 deletions(-) diff --git a/pom.xml b/pom.xml index 9e6fe09d95bbe..4be8c220223f8 100644 --- a/pom.xml +++ b/pom.xml @@ -230,18 +230,6 @@ false - - - spark-staging-1038 - Spark 1.2.0 Staging (1038) - https://repository.apache.org/content/repositories/orgapachespark-1038/ - - true - - - false - - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ff8cf81b286af..f52074282e1f7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -214,8 +214,6 @@ object OldDeps { def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq( name := "old-deps", scalaVersion := "2.10.4", - // TODO: remove this as soon as 1.2.0 is published on Maven central. - resolvers += "spark-staging-1038" at "https://repository.apache.org/content/repositories/orgapachespark-1038/", retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", From 336cd341ee449098a1db594592a44f5ab9200fa0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 19 Dec 2014 12:51:12 -0800 Subject: [PATCH 080/227] Small refactoring to pass SparkEnv into Executor rather than creating SparkEnv in Executor. This consolidates some code path and makes constructor arguments simpler for a few classes. Author: Reynold Xin Closes #3738 from rxin/sparkEnvDepRefactor and squashes the following commits: 82e02cc [Reynold Xin] Fixed couple bugs. 217062a [Reynold Xin] Code review feedback. bd00af7 [Reynold Xin] Small refactoring to pass SparkEnv into Executor rather than creating SparkEnv in Executor. --- .../scala/org/apache/spark/SparkEnv.scala | 26 ++++++-------- .../CoarseGrainedExecutorBackend.scala | 32 +++++++++-------- .../org/apache/spark/executor/Executor.scala | 36 ++++++------------- .../spark/executor/MesosExecutorBackend.scala | 10 ++++-- .../spark/scheduler/local/LocalBackend.scala | 11 +++--- 5 files changed, 52 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index f4215f268a0d3..6656df44d3599 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -177,18 +177,18 @@ object SparkEnv extends Logging { hostname: String, port: Int, numCores: Int, - isLocal: Boolean, - actorSystem: ActorSystem = null): SparkEnv = { - create( + isLocal: Boolean): SparkEnv = { + val env = create( conf, executorId, hostname, port, isDriver = false, isLocal = isLocal, - defaultActorSystem = actorSystem, numUsableCores = numCores ) + SparkEnv.set(env) + env } /** @@ -202,7 +202,6 @@ object SparkEnv extends Logging { isDriver: Boolean, isLocal: Boolean, listenerBus: LiveListenerBus = null, - defaultActorSystem: ActorSystem = null, numUsableCores: Int = 0): SparkEnv = { // Listener bus is only used on the driver @@ -212,20 +211,17 @@ object SparkEnv extends Logging { val securityManager = new SecurityManager(conf) - // If an existing actor system is already provided, use it. - // This is the case when an executor is launched in coarse-grained mode. - val (actorSystem, boundPort) = - Option(defaultActorSystem) match { - case Some(as) => (as, port) - case None => - val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName - AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) - } + // Create the ActorSystem for Akka and get the port it binds to. + val (actorSystem, boundPort) = { + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) + } // Figure out which port Akka actually bound to in case the original port is 0 or occupied. - // This is so that we tell the executors the correct port to connect to. if (isDriver) { conf.set("spark.driver.port", boundPort.toString) + } else { + conf.set("spark.executor.port", boundPort.toString) } // Create an instance of the class with the given name, possibly initializing it with our conf diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 5f46f3b1f085e..c794a7bc3599e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import scala.concurrent.Await -import akka.actor.{Actor, ActorSelection, ActorSystem, Props} +import akka.actor.{Actor, ActorSelection, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} @@ -38,8 +38,7 @@ private[spark] class CoarseGrainedExecutorBackend( executorId: String, hostPort: String, cores: Int, - sparkProperties: Seq[(String, String)], - actorSystem: ActorSystem) + env: SparkEnv) extends Actor with ActorLogReceive with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") @@ -58,8 +57,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) - executor = new Executor(executorId, hostname, sparkProperties, cores, isLocal = false, - actorSystem) + executor = new Executor(executorId, hostname, env, isLocal = false) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -70,7 +68,7 @@ private[spark] class CoarseGrainedExecutorBackend( logError("Received LaunchTask command but executor was null") System.exit(1) } else { - val ser = SparkEnv.get.closureSerializer.newInstance() + val ser = env.closureSerializer.newInstance() val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskDesc.taskId, taskDesc.name, taskDesc.serializedTask) @@ -128,21 +126,25 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() - // Create a new ActorSystem using driver's Spark properties to run the backend. + // Create SparkEnv using properties we fetched from the driver. val driverConf = new SparkConf().setAll(props) - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - SparkEnv.executorActorSystemName, - hostname, port, driverConf, new SecurityManager(driverConf)) - // set it + val env = SparkEnv.createExecutorEnv( + driverConf, executorId, hostname, port, cores, isLocal = false) + + // SparkEnv sets spark.driver.port so it shouldn't be 0 anymore. + val boundPort = env.conf.getInt("spark.executor.port", 0) + assert(boundPort != 0) + + // Start the CoarseGrainedExecutorBackend actor. val sparkHostPort = hostname + ":" + boundPort - actorSystem.actorOf( + env.actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], - driverUrl, executorId, sparkHostPort, cores, props, actorSystem), + driverUrl, executorId, sparkHostPort, cores, env), name = "Executor") workerUrl.foreach { url => - actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") } - actorSystem.awaitTermination() + env.actorSystem.awaitTermination() } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index da030f231fde8..0f99cd9f3b08a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -26,7 +26,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal -import akka.actor.{Props, ActorSystem} +import akka.actor.Props import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil @@ -42,10 +42,8 @@ import org.apache.spark.util.{SparkUncaughtExceptionHandler, AkkaUtils, Utils} private[spark] class Executor( executorId: String, slaveHostname: String, - properties: Seq[(String, String)], - numCores: Int, - isLocal: Boolean = false, - actorSystem: ActorSystem = null) + env: SparkEnv, + isLocal: Boolean = false) extends Logging { // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -55,6 +53,8 @@ private[spark] class Executor( private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) + private val conf = env.conf + @volatile private var isStopped = false // No ip or host:port - just hostname @@ -65,10 +65,6 @@ private[spark] class Executor( // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) - // Set spark.* properties from executor arg - val conf = new SparkConf(true) - conf.setAll(properties) - if (!isLocal) { // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire @@ -77,21 +73,11 @@ private[spark] class Executor( } val executorSource = new ExecutorSource(this, executorId) - - // Initialize Spark environment (using system properties read above) conf.set("spark.executor.id", executorId) - private val env = { - if (!isLocal) { - val port = conf.getInt("spark.executor.port", 0) - val _env = SparkEnv.createExecutorEnv( - conf, executorId, slaveHostname, port, numCores, isLocal, actorSystem) - SparkEnv.set(_env) - _env.metricsSystem.registerSource(executorSource) - _env.blockManager.initialize(conf.getAppId) - _env - } else { - SparkEnv.get - } + + if (!isLocal) { + env.metricsSystem.registerSource(executorSource) + env.blockManager.initialize(conf.getAppId) } // Create an actor for receiving RPCs from the driver @@ -167,7 +153,7 @@ private[spark] class Executor( override def run() { val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) - val ser = SparkEnv.get.closureSerializer.newInstance() + val ser = env.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var taskStart: Long = 0 @@ -202,7 +188,7 @@ private[spark] class Executor( throw new TaskKilledException } - val resultSer = SparkEnv.get.serializer.newInstance() + val resultSer = env.serializer.newInstance() val beforeSerialization = System.currentTimeMillis() val valueBytes = resultSer.serialize(value) val afterSerialization = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index f15e6bc33fb41..a098d07bd8659 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -25,7 +25,7 @@ import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} -import org.apache.spark.{Logging, TaskState} +import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.{SignalLogger, Utils} @@ -64,11 +64,15 @@ private[spark] class MesosExecutorBackend this.driver = driver val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++ Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue)) + val conf = new SparkConf(loadDefaults = true).setAll(properties) + val port = conf.getInt("spark.executor.port", 0) + val env = SparkEnv.createExecutorEnv( + conf, executorId, slaveInfo.getHostname, port, cpusPerTask, isLocal = false) + executor = new Executor( executorId, slaveInfo.getHostname, - properties, - cpusPerTask) + env) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index a2f1f14264a99..b3bd3110ac809 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -41,17 +41,18 @@ private case class StopExecutor() * and the TaskSchedulerImpl. */ private[spark] class LocalActor( - scheduler: TaskSchedulerImpl, - executorBackend: LocalBackend, - private val totalCores: Int) extends Actor with ActorLogReceive with Logging { + scheduler: TaskSchedulerImpl, + executorBackend: LocalBackend, + private val totalCores: Int) + extends Actor with ActorLogReceive with Logging { private var freeCores = totalCores private val localExecutorId = SparkContext.DRIVER_IDENTIFIER private val localExecutorHostname = "localhost" - val executor = new Executor( - localExecutorId, localExecutorHostname, scheduler.conf.getAll, totalCores, isLocal = true) + private val executor = new Executor( + localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) override def receiveWithLogging = { case ReviveOffers => From cdb2c645ab769a8678dd81cff44a809fcfa4420b Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Fri, 19 Dec 2014 13:56:04 -0800 Subject: [PATCH 081/227] [SPARK-4889] update history server example cmds Author: Ryan Williams Closes #3736 from ryan-williams/hist and squashes the following commits: 421d8ff [Ryan Williams] add another random typo fix 76d6a4c [Ryan Williams] remove hdfs example a2d0f82 [Ryan Williams] code review feedback 9ca7629 [Ryan Williams] [SPARK-4889] update history server example cmds --- conf/metrics.properties.template | 4 ++-- .../org/apache/spark/deploy/history/HistoryServer.scala | 9 +++++---- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 30bcab0c93302..96b6844f0aabb 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -77,8 +77,8 @@ # sample false Whether to show entire set of samples for histograms ('false' or 'true') # # * Default path is /metrics/json for all instances except the master. The master has two paths: -# /metrics/aplications/json # App information -# /metrics/master/json # Master information +# /metrics/applications/json # App information +# /metrics/master/json # Master information # org.apache.spark.metrics.sink.GraphiteSink # Name: Default: Description: diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index ce00c0ffd21e0..fa9bfe5426b6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -158,11 +158,12 @@ class HistoryServer( /** * The recommended way of starting and stopping a HistoryServer is through the scripts - * start-history-server.sh and stop-history-server.sh. The path to a base log directory - * is must be specified, while the requested UI port is optional. For example: + * start-history-server.sh and stop-history-server.sh. The path to a base log directory, + * as well as any other relevant history server configuration, should be specified via + * the $SPARK_HISTORY_OPTS environment variable. For example: * - * ./sbin/spark-history-server.sh /tmp/spark-events - * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * export SPARK_HISTORY_OPTS="-Dspark.history.fs.logDirectory=/tmp/spark-events" + * ./sbin/start-history-server.sh * * This launches the HistoryServer as a Spark daemon. */ From 7981f969762e77f1752ef8f86c546d4fc32a1a4f Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Fri, 19 Dec 2014 15:24:41 -0800 Subject: [PATCH 082/227] =?UTF-8?q?[SPARK-4896]=20don=E2=80=99t=20redundan?= =?UTF-8?q?tly=20overwrite=20executor=20JAR=20deps?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Author: Ryan Williams Closes #2848 from ryan-williams/fetch-file and squashes the following commits: c14daff [Ryan Williams] Fix copy that was changed to a move inadvertently 8e39c16 [Ryan Williams] code review feedback 788ed41 [Ryan Williams] don’t redundantly overwrite executor JAR deps --- .../scala/org/apache/spark/util/Utils.scala | 170 +++++++++++------- 1 file changed, 107 insertions(+), 63 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 d16233a0bc760..5e1cb0c7a7712 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -385,16 +385,12 @@ private[spark] object Utils extends Logging { } finally { lock.release() } - if (targetFile.exists && !Files.equal(cachedFile, targetFile)) { - if (conf.getBoolean("spark.files.overwrite", false)) { - targetFile.delete() - logInfo((s"File $targetFile exists and does not match contents of $url, " + - s"replacing it with $url")) - } else { - throw new SparkException(s"File $targetFile exists and does not match contents of $url") - } - } - Files.copy(cachedFile, targetFile) + copyFile( + url, + cachedFile, + targetFile, + conf.getBoolean("spark.files.overwrite", false) + ) } else { doFetchFile(url, targetDir, fileName, conf, securityMgr, hadoopConf) } @@ -411,6 +407,104 @@ private[spark] object Utils extends Logging { FileUtil.chmod(targetFile.getAbsolutePath, "a+x") } + /** + * Download `in` to `tempFile`, then move it to `destFile`. + * + * If `destFile` already exists: + * - no-op if its contents equal those of `sourceFile`, + * - throw an exception if `fileOverwrite` is false, + * - attempt to overwrite it otherwise. + * + * @param url URL that `sourceFile` originated from, for logging purposes. + * @param in InputStream to download. + * @param tempFile File path to download `in` to. + * @param destFile File path to move `tempFile` to. + * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match + * `sourceFile` + */ + private def downloadFile( + url: String, + in: InputStream, + tempFile: File, + destFile: File, + fileOverwrite: Boolean): Unit = { + + try { + val out = new FileOutputStream(tempFile) + Utils.copyStream(in, out, closeStreams = true) + copyFile(url, tempFile, destFile, fileOverwrite, removeSourceFile = true) + } finally { + // Catch-all for the couple of cases where for some reason we didn't move `tempFile` to + // `destFile`. + if (tempFile.exists()) { + tempFile.delete() + } + } + } + + /** + * Copy `sourceFile` to `destFile`. + * + * If `destFile` already exists: + * - no-op if its contents equal those of `sourceFile`, + * - throw an exception if `fileOverwrite` is false, + * - attempt to overwrite it otherwise. + * + * @param url URL that `sourceFile` originated from, for logging purposes. + * @param sourceFile File path to copy/move from. + * @param destFile File path to copy/move to. + * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match + * `sourceFile` + * @param removeSourceFile Whether to remove `sourceFile` after / as part of moving/copying it to + * `destFile`. + */ + private def copyFile( + url: String, + sourceFile: File, + destFile: File, + fileOverwrite: Boolean, + removeSourceFile: Boolean = false): Unit = { + + if (destFile.exists) { + if (!Files.equal(sourceFile, destFile)) { + if (fileOverwrite) { + logInfo( + s"File $destFile exists and does not match contents of $url, replacing it with $url" + ) + if (!destFile.delete()) { + throw new SparkException( + "Failed to delete %s while attempting to overwrite it with %s".format( + destFile.getAbsolutePath, + sourceFile.getAbsolutePath + ) + ) + } + } else { + throw new SparkException( + s"File $destFile exists and does not match contents of $url") + } + } else { + // Do nothing if the file contents are the same, i.e. this file has been copied + // previously. + logInfo( + "%s has been previously copied to %s".format( + sourceFile.getAbsolutePath, + destFile.getAbsolutePath + ) + ) + return + } + } + + // The file does not exist in the target directory. Copy or move it there. + if (removeSourceFile) { + Files.move(sourceFile, destFile) + } else { + logInfo(s"Copying ${sourceFile.getAbsolutePath} to ${destFile.getAbsolutePath}") + Files.copy(sourceFile, destFile) + } + } + /** * Download a file to target directory. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. @@ -449,67 +543,17 @@ private[spark] object Utils extends Logging { uc.setReadTimeout(timeout) uc.connect() val in = uc.getInputStream() - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, closeStreams = true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - tempFile.delete() - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } - Files.move(tempFile, targetFile) + downloadFile(url, in, tempFile, targetFile, fileOverwrite) case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) - var shouldCopy = true - if (targetFile.exists) { - if (!Files.equal(sourceFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } else { - // Do nothing if the file contents are the same, i.e. this file has been copied - // previously. - logInfo(sourceFile.getAbsolutePath + " has been previously copied to " - + targetFile.getAbsolutePath) - shouldCopy = false - } - } - - if (shouldCopy) { - // The file does not exist in the target directory. Copy it there. - logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) - Files.copy(sourceFile, targetFile) - } + copyFile(url, sourceFile, targetFile, fileOverwrite) case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, closeStreams = true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - tempFile.delete() - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } - Files.move(tempFile, targetFile) + downloadFile(url, in, tempFile, targetFile, fileOverwrite) } } From c28083f468685d468c4daa5ce336ef04fbec3144 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 19 Dec 2014 17:02:37 -0800 Subject: [PATCH 083/227] [SPARK-4890] Upgrade Boto to 2.34.0; automatically download Boto from PyPi instead of packaging it This patch upgrades `spark-ec2`'s Boto version to 2.34.0, since this is blocking several features. Newer versions of Boto don't work properly when they're loaded from a zipfile since they try to read a JSON file from a path relative to the Boto library sources. Therefore, this patch also changes spark-ec2 to automatically download Boto from PyPi if it's not present in `SPARK_EC2_DIR/lib`, similar to what we do in the `sbt/sbt` script. This shouldn't ben an issue for users since they already need to have an internet connection to launch an EC2 cluster. By performing the downloading in spark_ec2.py instead of the Bash script, this should also work for Windows users. I've tested this with Python 2.6, too. Author: Josh Rosen Closes #3737 from JoshRosen/update-boto and squashes the following commits: 0aa43cc [Josh Rosen] Remove unused setup_standalone_cluster() method. f02935d [Josh Rosen] Enable Python deprecation warnings and fix one Boto warning: 587ae89 [Josh Rosen] [SPARK-4890] Upgrade Boto to 2.34.0; automatically download Boto from PyPi instead of packaging it --- ec2/spark-ec2 | 3 +-- ec2/spark_ec2.py | 48 +++++++++++++++++++++++++-------- ec2/third_party/boto-2.4.1.zip | Bin 860195 -> 0 bytes 3 files changed, 38 insertions(+), 13 deletions(-) delete mode 100644 ec2/third_party/boto-2.4.1.zip diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 index 4aa908242eeaa..3abd3f396f605 100755 --- a/ec2/spark-ec2 +++ b/ec2/spark-ec2 @@ -22,5 +22,4 @@ #+ the underlying Python script. SPARK_EC2_DIR="$(dirname $0)" -PYTHONPATH="${SPARK_EC2_DIR}/third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" \ - python "${SPARK_EC2_DIR}/spark_ec2.py" "$@" +python -Wdefault "${SPARK_EC2_DIR}/spark_ec2.py" "$@" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 4e8f5c1f44041..556d99d1027b8 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -21,6 +21,7 @@ from __future__ import with_statement +import hashlib import logging import os import pipes @@ -29,6 +30,7 @@ import string import subprocess import sys +import tarfile import tempfile import time import urllib2 @@ -36,9 +38,6 @@ from datetime import datetime from optparse import OptionParser from sys import stderr -import boto -from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType -from boto import ec2 DEFAULT_SPARK_VERSION = "1.1.0" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -48,6 +47,39 @@ AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH) +def setup_boto(): + # Download Boto if it's not already present in the SPARK_EC2_DIR/lib folder: + version = "boto-2.34.0" + md5 = "5556223d2d0cc4d06dd4829e671dcecd" + url = "https://pypi.python.org/packages/source/b/boto/%s.tar.gz" % version + lib_dir = os.path.join(SPARK_EC2_DIR, "lib") + if not os.path.exists(lib_dir): + os.mkdir(lib_dir) + boto_lib_dir = os.path.join(lib_dir, version) + if not os.path.isdir(boto_lib_dir): + tgz_file_path = os.path.join(lib_dir, "%s.tar.gz" % version) + print "Downloading Boto from PyPi" + download_stream = urllib2.urlopen(url) + with open(tgz_file_path, "wb") as tgz_file: + tgz_file.write(download_stream.read()) + with open(tgz_file_path) as tar: + if hashlib.md5(tar.read()).hexdigest() != md5: + print >> stderr, "ERROR: Got wrong md5sum for Boto" + sys.exit(1) + tar = tarfile.open(tgz_file_path) + tar.extractall(path=lib_dir) + tar.close() + os.remove(tgz_file_path) + print "Finished downloading Boto" + sys.path.insert(0, boto_lib_dir) + + +setup_boto() +import boto +from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType +from boto import ec2 + + class UsageError(Exception): pass @@ -452,7 +484,7 @@ def launch_cluster(conn, opts, cluster_name): active_instance_ids.append(id_to_req[i].instance_id) if len(active_instance_ids) == opts.slaves: print "All %d slaves granted" % opts.slaves - reservations = conn.get_all_instances(active_instance_ids) + reservations = conn.get_all_reservations(active_instance_ids) slave_nodes = [] for r in reservations: slave_nodes += r.instances @@ -541,7 +573,7 @@ def launch_cluster(conn, opts, cluster_name): def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." - reservations = conn.get_all_instances() + reservations = conn.get_all_reservations() master_nodes = [] slave_nodes = [] for res in reservations: @@ -618,12 +650,6 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): print "Done!" -def setup_standalone_cluster(master, slave_nodes, opts): - slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) - ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) - ssh(master, opts, "/root/spark/sbin/start-all.sh") - - def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") diff --git a/ec2/third_party/boto-2.4.1.zip b/ec2/third_party/boto-2.4.1.zip deleted file mode 100644 index 49886b89aeaeaa87524c7808563008b0b2e9cc92..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 860195 zcma&N1CS`)wyjyVZQC|>*|u%lwr$(Ct9H3+mu=g2Rqy}4?!IyEd3|1IL}p}UWXxPM zB6H^W##~EY3K#?m;GgZ2r~cP}o&4ViXaF1lLpx_XS|&OcI!1aGWk>*E?(QZz(SRj6 z7k6j?K#(Uu000n_zqiZ(zdWt|qyE zp7uY&QPJ6Z7(vld(K)%2t!VrAMkjU*X-2CP=8r*kc~tvo}r8^YipLT$^-B z?lvA*bL2ScV(EC#fLR!%jB4%5b`GTL=(v6v=T|~YYGZ*W@;ua|An)Sg@AiGWXO8x= z;=NY*zCSmMza4Az8S+RbBLUJ(ZFq{XX3jjMxlE85!H{4lS)H?<^dymkl#xB`XM%${ z<0?}?BGnQUPy|qqXWI!N$hEyE&txQkhdkm5u@E4FT-a?llaO`Fc0eAEPLl04>c62u ziCaKL4v8Kik^F^6{Eq4o`Rl<*mTBaH21;HMK0NRh0|+j@5OUufzd}Rap8-j%K@855 zU>R?%%%FLoPnyDwqxK;D#*-wv9lB6I@QE=D9Y+s@MnHD<#3U1OdLN@i*kA8d+Yp6? zV*yYmD~P~?j`3rGZfgT8pn+|aCLkhX7fOjCH~oPb+7Ue-y z7R>0GmiXS|~F@ zq6*6kY3M7Q#?);RKIknx8vW3L@59HNyOVTUA8;oJ?~OrsduE#+QvMJ4!cm^Fs~Fe< z|G3@3nN3{uw|jOR8gx8MR?^+zfiX|u+?_LrhaY!tFR@*K6PH%v_^UxTNW)}Cx*{ZY z92&*70So^2;L5xHTOXAU(;b92^b(}z^|BeqfL}R_|v76oZeRfx(_A;p9b9+PhTa}gIk%Yhkw+bTIgQU!y`|xQ| z8{@zi@ZMmnzX5-~Ngfm-yy+^+2hyvjMiKY_A|!2}!^=5=J+Dl`ITKs)xJ6aiBKvB2^p&aHVR8=u@GEf{u$J8 z7a{7wNZL*iZ^D_cs$|LMP#d0KMhb^FQVbk6eoP@t2{E2G5N@~!T2tZva&Gx3{Q}N$ zrK<;t`>td2rM}KrzJ|x|zN;iV$U7}xeew;`x5ei`EclYJ(dY5R-U%nTe@t@fHN%*@ zH1rkdthXA`p3T$H(lsn2ve^S-uLO|a{?NYekqBYR?U<&p>ghxEN7);;*cBy~2rH2_ zW{xW(Um%|6oeU`yn^Q0^Dxoh=0V*NT3gIP-v+j+Sz*hUb2QAKOe>-XJkk$dd_~(Mv zArkGD7nAUG(&P3ZNzQUA6lGAu9;d_yD)~=vUn9iIQ5p z7&cNV#A&?VkUAl$5$7Rba9;ap zZc+C_np5T8)#!y>iLAqHq5Nif{9{APk&)SBl-84#MPscF>`Lrn4GzwX;@?Zu`g!^S?bL~T#Uft2}$rjG0_e4bEL9~}3oSL@OAoH*^H{EkbeD=EXY9pV zpz|Tgw^d|9JJgVU4M5E!fO0Jk`>2<|^#@5tJg7nMFu5Q<&Lg&%AvRi`T9>+I(^ofv)dpMFL_tRm%tU(5nO;0=8Ip|4S;# zL0v%AF4;J9zy40P7{;43RA{|x8fqbwhcMU=s0aI$&?p-1Y@}X6v-i$A0d)Ucr6$mq zme^EFUlAoK2=!>Z1MiZSJE@lByC7WXUWkR)b>UQyC~2%|dKvM`kNy?-Rumh9K3mJ3 z?I2e?LN%`tVCoc@a`l!Lc)pKzW}T9~;F|=ddQF2CF@njeYEZNIvVE*Er^(&bGKu7} z-ApZFllZ%-iJz&E732?4ph;OYteuh4f;X3FSdK5M>thc|+)uI|EG9ymrO*f1`JD5@ zRq|$S^o*GmbkK`jJjAE?1ZCe7_kH`{M;^;->z|2RN=j>QdV>X&0pmn1a4?2M=_0*P zH|Bq0lvGn{3K>D>cERD6=Zh5`i_*MYK$aDl%hPb;StJFyZFt(0Q2dv*+Y3*O%a@{4 zGGVi)xY#;nuBz=Mk?3Ztw$@Y)yOz{iIq0JO`_(FGbWy?eUFJmM251awta~>tI|*hw zq|X`>la@SC(1e|iE7xJq%_dYjtRk;w^p~&DdJ|sU*2%qOFXLTis;B4)tS7 zmfnP>+(^;lR$aDX_@v4-B+h%>axiq#gCaTCse6!hip>R3nrPq=m*`+LMyAk(yqP9a z&?0a4pK?tp!KacM2=oRL76Gw<>Ze*jRv%?ZT{e?9I;hxl9@e2{WtgJ@>8sMKZI&MK zw>ngI;_}ONQG|BVb7Mt$?=8>a*I#RJpTTDi6^kUPce4R~UQ*-@8z|(y3lEiIX%HGr zo4W%sEtbi)oHb(xzxIL-d#}_CS_Ut3lr{9?o0BZ7dWd$6xR;;h?0V!bP#D7s>`Pt2 z1_>4!0oEzw9g^pDV~|6q`osV|7mB21eq~N=7D{(K=Z|L!L<10R!4$x=kF7$VX!c{JHrw?h!p@DmFgLEe`wq#XSAEk z-cENvd}He%&+^nA(J4wL*qPm5WYC^i#}$ih&~Y=K@$&AFen&Aa!Vb@#s8$Cbwsd^ki_&{hqwy(+sBG5m?f_AyN^uDmsX5$a*3k z=Ai`-HPMSc+4WTDvO~MOdsI(W9$s_0TpFj;XjSv){xaH9uh>$)HNq=XwZ_+oe9huZ zz&l)ICD=^g9#Uu}Bw-I$$u}hdruL#NK zcI~+XOL(3}4M~s`=>hI`h23U+nQd2k7NYB#R^IA&JF{CntlJ6M%@;%|E@=h>t5S(E z39-=;wUw1mMNog$!LaJlp<4guq8}j}=l4s)zBSB4s+3XvR5{dLErzB#iks+XP@z+`s2AIm(6)|4+P9=CpNS*Ub81}%ORvW!@RUbd z0oHQFs^bhcwoE{e zc9@QiMg~i^HIac>v6LTXCsrx0K!U#KX{wt*Pw&?5bW;NI<{)^S2TJVQ-Q0Y?cEMZ+phCMbM>fEvzR^NO^ zAvUvfm0n)P&dJnue8HAnx-o}}85)f7bzeH>XJWx3x@eYO(3|(xE ztxfbSYz)l)BhjWL>sW0tpoHE$P){p^v2k3tPWscg#x^pgf(h11B7SqoHjgC6A6Bo` zi2w%A*BWOlHfs9&@Cb_&OO-g_yi>p;dmoS-Re7rlFiw)7)!8V= z$0YlM+r&JQGfuEUna;U%ueMHxle9V2?1F_MbDhhmL_2ZqKlpc75%!mE_2-EYHe2;da8G zZj+@>12MlQsem)ieqeyIDF|XXW=sS7faz`k2t&i?@O~#;N&!(|R>uoexN^0mV$FVw zxhC4ejnM&%}`UcuTDU20v7*bJXG(~@jneyOwAkSNK+&T0kltuad>l<75ixdDA z1OQ;*udDOF@g|H+|Bv28j_O};f(_yOTF*fVK^V%K)Rj@6VHZeb9T0fgx_?O5Zx_qkY$GE9>jLJO4p0vJX1ON|y4jz@dW=V!Q| z4@et=m>(#jC4LOyySW=skslhj$OeD*wmCAX4{*}ycbXU!dN$QTWRFaq#SUz>mE`7)FGJA&(Lk@qC<7XwrEVg@{dMbuxlJd-!0qNp0l1{LHF*t7hSi!}>}$7ST0otWBKWv^qyY7Bpb)l-1#Kc@Eog+s3@HuBtttEHbwlim1dR0dQE)1}41}04zd5Gpj z#Uipc7#rBb(r`a(l2A0mEz}dH8QyZ(=F94x3^M&Wo5eGl>Og7qeQsj=_!Nj&{9-v` z#v;s)4w_(sqCN+cWVwkeXn}3*6RWFXk4~mj`ix~<YUjf;s z*MANYs#emKRpaORQ|{GYXLqLIUY@;;|j73}s! z5qwY7q@5xG#u?94EJ=%bIP{UK{41`sJtK*XFG0J2>eh6~NBeI6)DB=2DB(vH_D@g0 zOnEmkTlu}2$V)J)9}WAX;>-WlCQ;Mq7zha9_%;M@s} zeY@XSP75mmifDF(^zm^64$yo~$~u$?(o_lLLxu%PjWvpYfHh*_HoVhT6vBU@izC^P zDu5qZ4#bwdW%g#Esi{`k&@n^LYi|>xf~l)Ov6I-4R1=V#W7-2I+O!QN8hvXzB^5mv zh?AzM5++cl#eQep>1ZLMS?NRo0W-%)a-zwNeuj}o(j4|4Ye`y6@>gs^($8l%j~DO_ zLGbRQ{^)C_TZ9Hu|5-Z}Bv)hu*RdO?5k;g{U5P6h3x0~7Nrs#dqPh&lfiF4~1n*I{ z&LYXxf%uYmJO`n3C!_)T$zpdMwLb`M?`_y!ipu>;gP0|@Kb~{DCExwiH<-8WZ5{CO zLS#RPknsu7y=HjcVg7VnbT+w~#Xmqybm9x5WOpngj{U}EEOXpJI5)Ma{}?TGGMA+| z(0$qkicFtL%q$(h8W4gkynSkN4pQ$9iROE1h-6fp>-8YvpRk$19cLXK&1T=X2Ndz! z%#3RZFb7>3vA-xPgCkC9WwtPSZt>0(EjlOTL3j;hx5zJltGi+vBtkb&$40~x{6{q5 zWQOt~fjk#+n%0RCnk~YeH)O-r?q`P`C z(H4Q zj9`dO@|i|Y56m9dvlsfbQvOY+Fd>}Gz2=&0!B%>}>FN-hl zgUSmAo;~&|*8RIVgQ(K|NDr$m7b{jv+mI~C;7VZ0$Rn2V&)1=i=ZCX`djGog6YCyI z{>y+|q9jBrT-VB`?Gb{l3A$~;sI_-(1r&MST6zc%lnrru&;Hq-+14uEAD0ryEpjRE zdaRQB>V%({ck}$0McYE&$KyH`YHmly#|zuauI%sd1{}4q7z*Ubt~#+81@&wSNx+Z_=ba z9;(yX!DM+vXNPnD0R3z6TtY3zYW>R-&B*`OAN!{bW#s1KY+?PM3TTh|);|Kx@8z#R zLsX~2IWaP4Yr%?~b}KaTa#RUNrn4m z*t^GM6(x$=^6R~E`3=3ak=9#PM5CNn@l(4^9!(K-AFJ3~MxEM2vWxLcqQp`m$CwfDQ$*o0O1>MlIyeRm6BDKqY!KuR z1n^`)F>r)OLInqYq*cdaiQv=E`$ji9_9+Do;1}Cz2M*L6{*+0OY)p|xvOCrL5e4Oq zEV=>{N3&ba{nDAIjB`~}%Ywj}YP(UD1EtXb6&-f!-O-1pylR6#QfFk>r|T++U6kWSlPkufDjfY1EFpjo0scR=WRG>G(iq$5mQ@Xa(hVT%q%1u zk~VJgc%IsJndu$aRl{U)J=Q)psK!!U{&4iLfN8E!9yb3AXxZF8#K9QBqiS7!m7g#V zg1DAcx2l9K1EuHP(c%92iBlvl6&;`6eq4p^#spN=%R=5RTJ$-7s4~@r-8_y?DmMr) zXi`!4*|58>8s%r3I)qI{n$p;8{%~>MP1O>vVG%5Ox9B)CE6CDrx%aW#s`|BGg`#&~ z`aHF8gdOH$#`;+S>kw9Y2eq)=nKfrGam!^A`oo^bG)zTiTyI6Q+@d@z>II0-92V_+ zbJ)4A+|3@wEIj5Tw`MXDv|Gd;&*)6TW`t%p$RigeIH@f-DIF1$k{G;tMy`M=SXKKO ziDC}z7V`%z;6mMTJD&*&5ztj@c@KVR50!j~@sQ_+d66c!A3W%fCBo`B!*zHla5UV- zl{|#UFIV*~?vvcEZt(^2PMa*W?&Jd+9i>dGlmfGZ;w}n^ZSi05bPwQ0E3m8AI^lOW z9|qAL1Zif2bWFl~H@xmL{0U|#rQ0mQ*+T6;rh91groC4oyIxb=moh!c7;8pZdJ+&i zcr9?h7y!iRNryzEECJh-M5`=O(+$>*Tkbzw^|o(&nrn@mG`lF~9O)dI?Qe`1VAq7f zzcU$E6cQJ%*sZ{CV^rh9UbF&<+O88k{oh@MTtZ_6-oSlAbnKurQlS9nS^GwRKSNgq zHzyPrZlYRR1#kl`J7&@$z-YV7-z4nTU`x*ZKv%_}sy1pQ%0GhdIhCZ?NPp5%x6s&j zwA{^qyfIZb)aifsNwBEt!Bx`LGSJml?CYzf9#)J!0W;sM`u>`Pez}z)zySFmsgHcB zTYd;HsUss~ZXGNr9$JEB0MB$~b?(vVbq(FjhDkmvDR|i-Sv2CUPsC3W z1EnBgPQG6VQ{O$&tJr;~2x#+bHjl#dPaE;sZI~cBzMJgyroKX#@?}w6(T;IoWHA{0 z!lL@JNe}bP64br7P!57NGGypATR@r=$InRPS-P6pg&iIK1bSH^qC>BG++nN<2?3|E z_LFIPyxLokG_cYwjjtq;b}5$0-a_x5-A1deLn5!Vc3qg=u`dEOHU|KI>ux@}Fhu)HDYPv=bg9s7=Rl`JD`P+_DW5!6{3MBu4#YQPWRK#41ZZgVlzB6F%2)D+S61b^-3FwA2h`ujCgAQFs26=bBxKIeb*|Xz^8X>;#=EoBu?DfP@ z$FOPU2Swfq(#4Pt56ZIlK0y>313iysg|UqF4ibW~qX&tc{TLjo!^L{-AuN>9^xXQS zQB8+uKyiRtaX>p^rol`=z?AiS9E1dN%bA=`SvmAiRxAO7x*V`4!1zFm(r?*XO0 z`ef{Z|ATSDS@H&x)T>Zap3bsonmAn7HOG$RAe(ER!y(hAU%vsh@(E8cltTjs382Iv>Z z{SGj1Yee9);6#1df$y+upic8~3l!86imJa_Hj}G(DzOR;xs>>gSNr;dD~W${fnses zIfNyX*(&xhmmDP)*2bxns^QG$fy5HpY^&%1q>vErl6`(xf!7N2d_51=myqHT)^Z~1 zT#e5iu+XkDWO=qw)5+4zz*^ITodHD%l%KxE zboNI|WcoOcA#ZLU^O>6gWr!N>S|q{YBwz@HMzFdlG;HlzMc2(ESg9ksqobg zV1^rGsp2J9?y$}TvQSr=H@=S8BCpB&tVd0xo%aj%JZGHofcRSD`T%H8`$NlN^DCnT2`0Zo_q$T@2P>05@M9BCoywL5TdFSsXC>pDHg4c!InRJX ze@Ev@Y6X0%abL%^4&}KyRDNrN-sG;v9-Z&r&xWO~SA{*Dp9a33zkhZsXXtL^Pk?+_ zf?KrAPE~x@rpp#=fSPgX4D;m4;vP6s>S2YrmGEmjNX$o{jF3NnM5B^|cuu4#j)_(` zFs0F`G9s3k_g0}Z*Jz-NXbd6AIlDSm-}qfyh%YOG%xMf;cF-w@K|3W`um(2xvVtSk zgn_UcY?>8=?sPt1j|2N$EiUR$Wu1IxLjm*AYN$HrI1G^FFv|pslgaN0A-f>ZP#mPNZPt+DsM>}oGo5HA2w==TMy@1naaf5CSN1lc=qU%FLGtD%KX zW9l^dGbfz1D=OhsMl0)u9(%>RVm#R=7!6872RNxt>w*y>y#=l9j1Qq}uTPa$fz%s! z%9e&JLTy`Ynp@jyRO`Xgt^>78Wox=OnKk!Rl@Uvp%CkOZ;v1?i0ZVy~`f4fHxvX%& z*tWU?cfYkbY|_gBnqoM}SGQ5D@(0;@FT&f!?T@1lWEhzt-|Y%KD$A!Lmvd&Z2)xJ| zk^@-&mX@<)YIO_cM2xgW%CXsR<(}N$_qAg^s)TG`sc`-D42%5&{8t@fZmG!=2Mho} z{ujtG|6PZenm8Mo>zP_u|Ht;8jbbFui?%1LgV6?a=7Ef^^}_%^Nji|ALEtuT8>Ny_OCDL-3g z=}~m==FX2mm8heL*4;D-tL95wln5nBRjATTWk>wzI4zc&bZ)z%wlj$HAPIN(f&yBO zX^K>fakQ|EiP9LclLRzXr`vgkvN-~{&TXul7*)UW@4$g`Fk#8XcWrSncaadq!b4Du z3Zx@@!yLM1g=&vU6d9Mn*_&dcIOZlnRBFwA+*b^pw$nd|?z1Er^NDGp!=UBXVNiVi zj-bt-Lt>~x`esEiSNZXS?k;B}RJvL8O?&TqvyMyc$__5zR1i2|F|a0VlFJ#ghZ+zP zku5a_d$)GXe(uc}MT=Q;mIyi{CQCCh96qwa21`Mg1b?j6NEvvl-r%WQDlpt>HH|7z z&6uwhol)7>7K?a?V8Ztaw$ouIgKu4u5*2K3PDOvgPZyLIUC}5o&&aafSEJBXbpT6fvxVVVXCsO2!%oa2rkfS9iPuNLK`84pH|R zC;Y%dBb3bOVrUB3#i2m*zS-~dhJ1&hF{&1zTkB|gY28#gB+0{g8yX0h=lGu(62wM7sNk4DrqTtHuKjG$3!HZyk_uRw)=TWySdj&5X`)1zECaI&G>DnK<&rX$cy#EYV%RQMsI4n(V?=4*MAw<2N8sGfe zBypm4EhQmmca@sISMqodCHG&N@5tldC4;HtXe?=B&l7v^N!u6S39*sqH;=Dj?qZsX zs+^$LQaX{%U+d@^gzjVZ!&7IizDkqjImN5^uW~Am=5xUlOJFD03<@^=H4-t1W|Xav%XO81|m+R#_=nlunbeeFO~H z99?hp>-%v?!1kFEYZ8@ztiP4Qt(cj%_D$7pvfxdtE-z(fs8?OXyNIpDPWt(;>FEF6 z!q@BF_cr=V8u4fV0C@lYgyuH2X~YW?dCWBjU3`GhbU-PQZ?JNVz5b5~9ceKnx^JQbqA}q?eOwYlD93L65dwq) z$z+bWI1kZQlVOs5tduxJo&KbmI=TRvx-lW0QG6Xibdcc`4FW_w?_eq_4(5FcL?X@j zG6I2Ocqu|nIAZt-bpt$9=piCV_K5)!0g z3>Moit-=#S=)zz(L`hC4cn;iP8e-Yz6DE59CKl3Ukf$`Z6)s7v#Aec!c^gVMiL(z3 z`tyDw=VePZV~TW%S_-ifiHskkf_z*ey(7wscQftH~9t{0-o|X5g_zNS?G$VQc+%#5edLeXIGz1 zSDd@kyW3g5j$zXXFkYcngMl;G-Yh=pz>4nA#^0c&ZTfP({dU*J<^c~MJtu!m!&#pC zd%Ezu;I2E9vhD_Fb3UO~#yB&k4KBAkoPu-YO2lE0xtIgHoVv3R>8`BYq~V-S^k~{$ zPLPE|wLn_;wx5r$rv_J_cfSX^@}_Z=4Z~-V-5f6pNKU3R{&-PfzVyEZLxW_-T;8ce zHRI1h!Pnw04)x-3L!nKzV{TmMfC_y~f2TCx=|?{1k~obj{v{J0$3wc`8?R2xZe_^x zpQcey(P*psgp~RxNy-+c0lnBVpSqbL)5dQ~Cg*2HMt9-mp>L)byVXtXt>yz^D7rXRb!6wgq>4 z4!z(V9R}L_=}S!@u)xXbTudZuZr+#_WlM<_)T5^`AfZ}~1q4CD^;4OU1Cgp$hpR%$ z$WQN0B8ltQ^;=>58}6|?tR-u*g>0p2?&>7-N+%?26qy)5)g*F`4pXWg4zfJG-hLGj zaLPYw!2P;}XVjkoJp5C9H&P?_0L7;Knl<4y0F0jua&B3FNa{5EbL$ac$lL*QdX42& zf06S_bA;?BkIM)ymOWCY_iI;hpbLMjT3TO-D}X~+d>2t#a7%(l@;B*QX|UG2il(R2 z{&9SjVKr!y!9=5^g)ef7e)L}t$5mruR5OZK46UA|E|FFzcZY$}3~gx^19ci$Obo$i z46l%UVF)lw-}2d9Gg1U-)sIHw#S@oEJ zmkc2ZC$m%~T|Hi0AQ7yY7S=<;#beWM5KtV~awVft&JqG1Vd;J#kKp4i zwa!j~=_#HC(X?f!WxvEp#GlwZ>m4VPB7ve)9O0*a<$B@*jK(yK12Be0`slDjejBOx zEfC;Ir|FG4Z1N|b+&pZPV9gR-A7EBi4pjbUEWokvkXu3}Y)5cDyE_V57+P{yxRQOl z)b9-ctP_3Iu4oKNe!OG}F)lWnz33Q)Pce5T0d8kmUKJK7ft**fcaSs2QsIHkbtlw3 zISQ*_+Z*4%5k3IF%7d(pW{q^DF6JI64W#)SE&7{8h7JlWv=1oSU)&Dqi+csp3yh306kqNIwJdS zVP)u0Xyiqvwivb)=f{P)1G3~=w}|6i;9DU}nSg;a>?=1xlSk+?N6mcMy<(HJJxw)@ z;&6@@`P<_OwijOUfGH-Zd&ZY3df`QHui^9Ks)c)6%H0Hd$>zFC87&$&3}_P}0Jh08 zXQ#gOfO}J~^lmRauyfOmoqhbXPtndLcDa)LNEMofbx>f_y|T;UOS8(Na${G%Xwf-u zdDZTFAqgYa@*!)3=YlfFY?D8zBMipaP7atJeK?Vx%YEq_tbHzXr%p9vDCx&{0OAPE zCs`ITEvnpYNMt}kc|9<+YX!z87>0)`LD);cK*N<;FNG_|OJ0g9g+*r{6=)@C%$Q_z z1!+F04x0U3&$%*{a&-xssK{gFWT`dxiQK4n}AKY!oUPhgp4KC%$xv+ELGQ!k# z9cZ}dI_K(P>vV|RKtug@p{P*ws%#OOOxo&^a(xxrK9%R|8luf6IDr_;u z_Dh)Zhhr;uvcs%4B`v=_g(BoB6-+)Qk=I&g{5*BUGn`!F4ChVz#@Gn zwAO)AW*)ZzlM@1Fzd1|Ofd2X{IpA{!@jin?)izm(_jV|M4@mBJ%=?ZX=}~g?LeF*i z^RIAX@HK|l^mmb6)?Yxu`FH7JWnpct_y6dZ|0G;&l4PPb7!X2lzEPEE3z&wWHVy*i zu1X$f=kbIPBx$$}ZN|+HOFnM6-@^}PaT!v@;#lf*WAgZL(H_H^8o+IK^nKyg%&I1& zn;rmfajUkwAUAPLj}(M5UOB-JF6+|oseN1rOn730d%ZjMbm4=IMCQ)$bf-8ZjTkAs z=6K|;2b|UWlz$CidmQ!z*gCT2L0)1W0gF8kHD#_&V+^2((T=fG*{0J@1m^VEYm6*%ymJI2b;e zf){kkb~b~X#h7{N%OuI<->JkoP5PF}?K-PiVgl2wnuip4)Iy%Igr?R|LoGC-QXRR| z1$xQ*_Gc!e1D=r@C7LD5+Ukg}j$8nVO`0UD>|!jY2>!4J@bBEvZg191x%=#dvAqzj zW697a3Ljs~M`sgfd=vUSYxW%TVJZBqKo3{qmJ&bOHsplY+^U}m9Z|mB<;CMveg6+i zHMtZiErA07P^I|aYFKLn7u&y>`v2!usL`}>+86yt!=~?U2$B$Nc;)RwsSU4R)>5&x z1DMg@aa9Un{c4DJ#u`uW>HP7Pn~B~A;J#Z*SPV3M_v!TN)Z%-)W=T6cBMO%lc^pdA zX^kyfhU|(?(!r`;#%$`Q^YU|jChpQM3ovHClP$K+F&Q5|t3WUNJ;NSxd&QxX(<Q!e7D2?Er3wKc8(PA+D)s~Y1yD7*$j zK(78U+6LGx$Ba6aaVi*|!WrNgv=hFgza$_6SXRG1AiN z?J$ZV{sQj-cnzGALH!Vujl{;Ch)DX>39C#xl7WN@@KB9Qjfo0lKRMol8DaidakK5h zePIz|MDy4JWJJ*5jWN2+{hP2mS{1?@GmJzL!)|<27v~M|jEsJfa2jv`{Vqv9(n2hJ zABM?F<5W2DJbx-x+lit9VZ`@0IO%HE3X7BkT*d%45RNEPS>##oNGsa@L--UK{)t}b z0-}@Yr7PFla0ix7!?S(piEEsX!*L$Dn-f_0nLW=>iVR=l-mBVk6Lar*!yLa|S?*=K|gz_B}%*~)YbGx*w@a0AOw_FVMXR&;4v-W|v1u6EKu z_FEnA^IS%pK!y*<9$^=*?8)@e)|35&?M-Q0(i()IcEIB*reAj8!l@}s?#6C3E89CL zJORrGPj{~>OqkoO;RIha!T9aqlXy}HR@#ctN$F^9DEpus$yBlo0Kba-5c*trDaY3${+cW?AvXH4W4t{%UMNFjs6Fi*Hag?Ir_xZrm$;YMH+14CWF1 ziuuE*YspeaSMIu2+wAhs*4oixM@|!CX5K+GO8iyk5#q%Qa zL48n(BaZCVjv0pnnDAvy8MPjVYwGH;Iis+Nl@pP#v?X8x00fE~SNj;Ie<$Bsf5oxg zLo>B|D54;|5y`V~&>#^J2jbf(**g{E34d%oYMMQBH3`0?zVAhmI_Z#n{BfZr=__G` z^EoeVG4rOxf`+Ze`*1VEE{hX4ti>YIT_D^c?modz+5l}5MJipCSUUiIlgp-t;gt+- z&>Jb(DkU5Pa9z^*>*z*b!UNq*-_HHGrsnx(nfJyI``M5C;^+1DYj|a$u1sh0yLW@H zQY%Kd!HSq?L3%zWY}WDyaG}h<7$BJ~!sZ7NJRu|uRnXx=YYVGs8M@r&J?4Hd$GXYw zNEFhBwq2PzRC$QK5V~3Mk1`nq1X9E3G&LR6RQ6&%THiBLVU3Pp8=Og%!J1;Y*hTev zkrQ-KI9)TCHdVXa#r%3x0v9Ah9d0CK`A6No)$?tS%g=|}8~4|FW=fv`WZ}{lKc>j; zEyS3{RptZH0H)CLWvTs@3f4)4%l6ebs|~7&qD_W6Yx`Haz&N?g7{tw&L34RA2tmQM zdj~pwF(!d4g;XINDYs#}(LQJV8sd#AQ%#3+iF`}OQb)D8+jF&E<}t@DVKs^dGT2+7 zUCAV`ocI}EWDfQ#Zb_L>{Wq9rJUOZcY>()JKKyg7iMa5OL11tJUqp31MOrd(JmAH% zR%to3ndMpr3cJMJ-}SWKE$YqBJOX-!jhZ3p<LtG_{C@M<{bR9JhxqzMaIR691ygH-`uEk~CNw$3sR0z1Kat%X2pq;_YvfI%)sP@_UheJOH$E>RWC~tk>=O=-UD<4HkL9eR}|MZUMr83 zFnpodMfp}XIwb`3jI0glT+emAlZV?aDo)ul3OHJbJPjhJEyPY#221NFEXut6=J@?i z8}T7Ts|wWHDR+9)AE68`Zf|kZiHS`!WqDUrSmMe$f21qX3$0H+hh!BFbify{xFp1ooqCkFAOAWBO{q%m&Z<%2$;wuS_{P} z*NAe^!3MOQJ1qN~w*X0kJEM2fg$QoAu$!mqwAGM~&+CvXYBZb=E-2hxLZ(kBXce)< zEI{!IJgrp{PUsvkLbDZqOWT&_gf>Pyauqj4UPNYgLIDt#1Xcs2mm@HjI3@iFopi|+ z=MHyivt-DQ@U*tjfP~*xBRAqLARSrgXF=l~!qAHl_|Qolp?0k+(&S6emN?nCULLEk z5(3t^>J_UI?(5r|@u8nCf{C+2Bnx_m^@C%33l(`Y1;mWp3$9$fMMwuLv-I(o$S|G- zqixF%(f<8_5Oq>FHoF5d$LEI9`$O!;F|pBJq%!fvr?O7!MSc$)XZE|8d3Yn(c07Y^ zR!n@kg9Hff)?sCfwx8=JA1q%a+c|r0WFK;HZ?+|DBqZFXf#gC478X8Td%=7Nq!Zna z4MlM%feCQ&n>a1QQGcR>3-Gm6jOt0vTc=3Iz(20NfKpqfhQWN2Mm8j?q{Drfj`|9d zihOLIiFxmcvZXt3v6J#HCZtjc(0g1)PpiZ~bayhk!!)|I^pLCDAr^;~Kt8)$ka8h; z%t}(x=|E5QEuKAXmT0dei0F&ybw{>b<$Mgs;l-13;C}%l_PM9%i5m|zx|4+16Wn5-T_ss+EPw>uAK!a znKGZ;%O?RU1YcfZ>TUQGQCFEJvm6xoibuw39zNc21J8w`bnsg?=a+KaX)~6r9=fV` z9Y^^2pmV?`vp{0c>e&WFTXL`#S{hW6LJFnIbb24}*5U%rIBH>T$+P)rHIepk!)&Y0 zw6G}YgNlU#2rEk$CmA|uiBeLO>1}lwUJ7D~7qDTikEFT$oBP~6o_-m3df^s`$enZj zPE@Aawu^~{+Z>!EPECfgl0H417DV%TFVKr}_djX{Lb2lo;N_3eT#?SuxM3OdxR}@> zXAt$-3Kx?7N_XY@?W8P07Rf6N>8D6>wm~Y~^b}gl^BJ!qkUQn37{a zMd?Kiy+1*IJPXRyHJXk@UCCZCVIY%j72y_6Uh}jnEZKSRm7-&V^l7xQY;*dixH7+B zI=H{o_ER8~ny|AbG8To=;Sox`TBWqW@pNfaTyEhUvSj`Yy9dKlTU0@NSNAXfwy76y z-7GWiZR6~kM(}x056c5vQ#BtwyM)PeP?^RYeXXa!v^4g<;0%jb8r@au(>h&L5VcFP znFFYJ!?`M!Iqe9Q7xtAO(yLh|VM>cP|JCAoUM+Ge$A34Y`*+auDH39p9R|Z@=C6vV zOzhebwHGMVVNwnDcb3icaR6CWYV0GN_eqVnmHE8J0<7>}+*7H_SGJ_WTUWNB_U0Tu zYP*#4-eok))Wz1IO?PqEXSfHKu_Gp{pvn`|`H!kIEhaHU#aFAIT216F1|Yo0&vIgr zcRtfqtm0Fj%kE+TooWfdH_M4d=&2o{IsFz=i`guhbk4j)Fpnpyx5@H>t!uB{Ph7Bb zs+8ZnLO$$i6);FRX+Rq!X(6C=k%ypDZWT?PynMvtL-SI*Ubm1%oJa9^-o2;HCU;D} zp1qP)VN`q_Lqob*8$%0E+tN3MeLUjXZx zb!PvdKNI+0LyKr^NmJdfOnMf}5_&mG%5~Ny>TlTW>`{V=m5Mcs~Cx{xO@}HeNwVj!NWX-%6n@B)@`7W9+Bi`!q6dBjXw`?~^y1Xcuk=#i5Rv>qA z&WOkU!}a*K?UjTLPY+A&YLHFfZr%3=Fg2% zP-b~!UR)Slb;u(?j`V<%SsL(|hG%D#N>U3f)zoN3rQXRpkMQzVe0oj!Lzv*V{?DiFPQ9AKa1K z0;{szKgtg(v__Xlp~2SyU4cLyTerLZm_~J^{Njib%~wWN8HnRqg+Qa*kdkkA9dKIS z&aM%zR{^hwM7bOFLWr3=61~$=T8RMBrCQ{m0r##b8*7x9Wz9%3N{4)|%5k;w*+_Xj z3+Jg@jO14HkFt0I)g!vRMM99Ru4IEy8Bc07<`=hYu!=5}uLYLrGYx`Vb@OZgisW&{>pwT0swZk&Gfq%e&r16xM>GwI3_+wA3#%^A(O3 z@_BlxUCulk(v@JeY@-gv%X3DoDK+K z(YI;&hB-Se0cP$pW_xRrM%SYI?EH7uIjgENd^ddi`AmNMg=%B;^AyW2Q)7JP$VP@W zXT~0j(EZNv9s38aO)#jPk=yz_PK>mia?ETqM3V-Osob4wW79!bVh~P5J&8HKO(@%x ze8LXNArL!*u^{u7a3)BiI0MIs@~ANK{uBPYU2#QLwXVkq?$=I5l*9v_V&$!CGu>mEiZZiE4TEZKqCe7mDZD-jQ{2`NqMol}P+^AvPZwPks)T(qY9j#BlF z>Y3QKfOIET#o{zs{l~Qm>PjQ2EBD=AMzQ|yE`K=`hD&_871^!+;kw(+(HE__rr_?< z=V+mdwfO`kXR9A4KIio{MDKr5v_>bg{ak;7di!TGi~2tVwX-wJ|E6fBs%poti=p|w z)NwjC56Hy?k|82T5&RgXDutR~;V> zer&6jY#K0JyZ6|wkPj-7@+liq*cxi3_fD#*ZkQL2U6sX$G$ho}amRwDLd>r1#-c=r z?>$OnEP3g7znWjO)jhM)$rC!6b36iQ>d-C)B{E*p=Aymw`S(sZ-am!%vHkwMoRn=B zl9!P=3HZGN742PhdU*K!DQRD$g8rW7#6NN<@@)rJ!tCd>fsXV|;Pkr*w^@D{vPh

cNDMR3Jc$7 z5?ppxz|jx}uD|xWl_Seb&jQNQHl8$0H(jSpOZ7uMVLk&1+`3QTq_U>91+vl#mr#>g zKu&_*8G>Rk3(>-=LJqye6NPYGs$I{Xp%RVefc}uKCbU(DNm9t!Gs#_vYMTB6eD-*) zy?proLo1jQvKl;$g3KP{dVtd#%p>aLHZ2o9ThkJW#>|qhO|&suchQ3ijC~>3b53ql zMGj%|J7_UlIIYDpzvafjCWTf&&_!t$(XN#$U|YgM`Ox?Oeogkz{oEABM2a7)mF)Rv z%q@F^-!s+2C-bi=Ed9&fC}3?JULYGbe&jo>s9;6WUW#P503P{vdpzAd}h`PmTjIv&1xgE;m>T!DvMkdoLsfugIfrN2Z zt%WyV`vZ(S6UbO7z^b`!4!+6G@uuCLZHq zkC?s^vo;7xk1Y2?`mOBL+)6#z9zyWL<8aaVe3w;g^H%POQ)6k7d-|$ft`_00um5VG zi<7%#6H@w zEdF>vxqpo2|HHTJ;Av=UsqbuRXJTvkzuTE=)D+|Q+5cZZXeB~1kh?EWWH@ceM!A4O zQukG5Wnlv6kZJ_U1_1qEPkt^?XWN!CL6n+_?X1rZ$3Z4EfnTP()C^)*X4CQS7IWQ5 zv&X=+LSAXVYS!2X=&$Fq2MR5--@FQyKKuU#w zXERM087P&d4tYR{(TQ*5s^>G=W60_<3ma;&K65>89m&(m4K!uDfbokak$Ig(HZ7B6 zEXSRL_XjBJIzhMtoS4erA`xjJ`pCtxSLKKhik3=}DLpLnrXW__!n3L-N`k7W{s=gn zvLd8#jcFS5AkXSZC{@?O!-V_cKFAieFv-+N)<7ua(@%cuL@r)LBvdyh!$)L3QG+F! zbNw>|u>T(ELTgD>i8fEZ)?o$bXsBR#MnTmufxOFI>-$MIAz{aQ(SXEfxB0=uN@g%e z1L1ECVA8|i-pHA(+$myg4mx5q#bjSh`dZmn8ggfoNiAHZR$IqjvS$cKB-j*EC!G&Y0xQ z(5-?XtE9~-pc=cQ5Ox>hlisx5od(7z7^L#_HNhs3CR(7YMctTW{r83kqRsvh&xV)6 zLojol25oVDhxT8(%%CjKWB<(V)p^O1{K{e7e_7}cv zR>Uxd1e#3-#;HK3ciU~90WY_0GC>aJDGY~9TTjonL%rU7EF`Ui!L>2ly&bht#f}{0=|yLnZW3B*yH>` zt}!LN95?XlnTijbnQ2>tViqIOQ_5mri!7m{Bczj+$1<}#_rXTyW5>kkS?p39k}rL? zy+|>)X1XkI>fj0}n=Up=ADp|to=$64dsZPvXOJIBu*Gp^GMOgJt`Ggn>$=T06-|6@ zh{>%Gljd5i!0=G3w!@S|jFMw>?x18Oy_!?3CXhG?O4U@q(7UJtlM6^(bL?!YSZMFJ7ueFSiYT~*E6#zwOWl0|Hxtcm5jAHp@RLZ)OhE3<2h)>Q0 zbN~kl?&0#Y9!?=7L?6Z5kVkUV^%T{K2wM_4*Gs`44)|p1lUBTq&;P>2RE}MF+z`=; z-@*UNK7Bd;XMgPfUpCOL^OJbPe(D@4lK&CO;biaXV#@ZvmOEFPw)O|?{|6UJ4MmF? z$L`k1z_bgxAUmf^n7&bZwnbz|$cSP3pjPNZif$kEamOL7ghICw*{!MLEr3UuR+70l z#Zp)N-0Q7M{dS%gtuDip@K%(EZe*XIlj=Yee)uKhB>Jtz!whIcd@)tDdVbI7lO1-> zQJe!Ny#$T;U_-Gku(^O%wtNzluA}< z^iP+uD6Dv|9t*nAB%7=DzY8s7RifrLMJDQ{RJ&j`&6-nTk;KZKG63+D5i`9riJ1+9 zqyuC3q6a08$3#&nCnkEOk^U*C9NG)IJCkV&kzl7XS#WkSh$cp4rO`wTg-WWAu~ga3 zDQTW!1polQ=!ryAFZi zFP`c?vU?pg1Uq%!FB3Kc)#J}a2a)o~W{Rw6pn#9)qh|r*tKdR{fDw{2^vAx${OlIr zrUa8#T1`t0qh(@7S?fV%C32;*LK2cCC?pUm;*97siNhqBxm30exi%Q-5*YBt$&F z+MM7Uce&EB$Y2(_>LvZM!}Id?@^y0W1QKrk zs|1fb-M)+dTl|qyu4Cm3;a?G^FUZqW#r(RICb4R|?Mq4X9c+;L4dvh${*9-414<~f zWSHC|AXv_c7({w|qVDbDyT-*i9o#|(?RF;uaRqKB!1rQHjk{2RLoh?I z!TfY$0-B%*gH)MW!XH1B1zQCXwZv5zCIo1XDrM`1r|Ew~A|N!+P+(Bt;|WY=Pt9YE zMu8oKGl%9?V{_+$@LC2+%Q&JEs>s+l_Obe|bD&UR&=vJ3K93QaFvB~vdw zZD__S!c#+Mp>x=;=_YXs2v9Uvf{HqSl zPIaL!%n=@Q9*Ga%va-irrSO1rFPlH|I8O~j^0CCZ6cEJMa@roZdZubyq_{|FUM}>o zQ)iX>ZGl(c_GuO#g6TCo7O}DUY)7nexjy@^K)bgRA}r#Ky`1-E%&fDa-)F~D zFDur7RG(?KY2+EyeDr=31x(i5A(%&7g$&BN!V)r(A#OKr4TeO8xec*MQ-v+o{9NMp z`+fiNn0OT{dNPeiQ|-Ls;AN-=-}ILaIUcRUU}{HWiVJ>{nS72iZ>DYR!Q1fBFvoDi zwt_A2!Fq(f;aV*Y1GHo(u$Q3-pXFAZBOvD7&a?@7gCAp4l@u28Wkh0oW^;mU7jITm zUgUl8dyQ^e|2n^fB%96$%z1EMp?2m$35l&zltn{A z2xTn`33`kl8#k&SB4L?57~RHg156;$;cw z0ec(Tq>_LTZzUx|60@<@X5>MR6Y}UaC4muYniYu5PV;WkPHCTi``oH9lPV;qtD%V*u%LT^vxEcIgBk;+*`eyi&9U zKOZWkz10-8vVS4+d=^MuCDr_uJRbq3ngS`z&ruKj-YXb3Z7snyH}xpq1rcoo+2QTj zpi6El6CwdJ<9nLa8#87wI#ii4`qi0>3P&k4>cHxb7Sh!X$b9?(OGyzh1Cgs2p<*hn9682>5e$)G(`#)Gcl6W#CAfRW)|Is<) z%wp?m<6`OH>hizoIR769{aU}@RdKcFNqXUW>k9k{WP4~0C!0L*O0NzIMlj7X8&^|F zl&Zr0<@=UU9NUf@+Gx<<+A+({bV9qyXn!)Iy=^Wzwz=C!j4v$%r5;HGf^3rsD~BIKv~B_^ z#=J=yZ#N~_V^*6|K%wJcaxo|U!XvR6qP7W@FTE`5_UmB-mZfY<8MX#+!Gat>V7^B3 zOj-ZLYRHr>Nz1dp0GW7FW<3T>j~&SHJWwqJmy$->AWQZv*!aB0rxOkD>)ZmM)O{1W zez&1J1gfs@qn5oDSOATOWJUeJ4rNN-e2I(>__Uf0jfo0Vi;`%8#**q) zv`!uY#yGZG<_@|4o7L;cP=HLl6T%mH7#?42#{jn+^h`ecmxx|)uXX^kmc5jY7z%Z) z+O*(@-V)3ne>gKtuC{p^U)K-)?pBjKOd2j@01pz59NM7u=rJlxH?%>ey{SXbFLSKS zu3f+JHqCF2xd^+QN9(Y2-Gtb5w2}QH4H5Nr==GMkrPDb6B>sGH`Cd3%`mZBb9uwA~ zBeQ9(bE9evt*GhZq3Q**=|X>XHpA1m$wa-$y@_m<-u`b=*qTiTq1#RA!$aE-tBt3V zICMIV;N#M%wF|=B4SEFC?RO@F`N>TkxwG4cpPzSAhEN1-eNUdQ{!GKd9)b`Wb@Zr`WW|G;AodCbONfS*H zm=9l9Zl$0@{_f73lbf?_Q2ve%!%6#BPb(q1&W!ywGO7~s@g^mE zh9~cmk3|2GGZ6Gvuf7bdCw*x$jRT3SafhrNWaPZ>IR&xPK{4muKrQ`Nssm^%Tv2R@ zc(N~Qs{{xpQSu%JLPl+vUOU0p!4e|Hzt@5#t4LlZBX5qiqZOmsWZ@xGYt%uq$0Wy7 z#|%o^O@*zng8b>#d(s#bj}gIVO-kq5Dn@8F)DE0`2K@Mxo$Qn76=!-9n;`iE&KPZs z8XtJB-~eya_Xi)DMHCs&PS$ozLot`g0%x48t!AwXm$pqJfuKX$kves#^c|kWrL)vKOPg)6wB4`Fu2jFYIc8HmPp!yn2rJ z4d_kYf6%l%JomeOG%KCv_CYQ;9#sp6o&7kZ);&^fm$C0oKHq)5WC4974e&yL7I43= zszqtzvB3Vo2vNjXNQGom9^>AHcF*rbEtcU}X+d^Fh7%3jYMp?LA zL!tE8L!0>`;h{D2w);)6l2V@!`$IQZg@D>lqwCHB&xG3)IC9#x2%Q(a{**j#B!K6IkLW zh^Irj4DSrWZE~A`3Om7!6k_~pfJHzFgKS;Br?eZMDQP2J$!5sp?b(HJOj22rNYD{s zrBxzQogrQTS;&w3(QAgPk06wofw`>3(X=ef%VHy}W}OG^uUJQwC5ZAe2v_G9#6}NhISu)whj~ zqv9!;_S|P&Mqyi=!gQd&>3gVbB!BHlOX_XoVnY10szyPJTr4y^73#6B9C+m$!!u!Yq_;X2>}t=9DnTr#73TSR@oSmhqO;UC1hot zuksCOej;Rl3*z!ucgB=xwVpN($;pdnH{{d;+(B)#>i~MyC0)s9n&^}hQ@qP)9BC7W zD|8`=VkYpVQtDgoTbZ}mBwvdSCWXpjj=v z3N?6ulD<{Q%EC=XbvIT(Q(?Z&N{glbT7~GH3kI^NO@K}7ghRZr$eJLcx}qf%Fxt+= z8t}qvK&k^tNoA4hvnZZE<7+-4MFnl`6C@$LLIHeH37D6Ilb#kV=*5H%Qg?TiXm>F=wb{(z zW{SM{ZB;!~|BzJuyWrN+)?z5iFl*J2VGR}TX+b*|tBSTsdQN(5gCoNPV>+*>XM*zD z2j7K-Sl8m!BS#SdS*M{}0RdRQJFRt`G!}Fj+cm702=(G!3`jiOIoxWyb|xhP^gc~o z)$kNHci<)ZeifNh-!)L8&RINiYWpB=TcCYSBJ0od`U;nPzGqdBBx;`D@xT}E!6reaFU4!z_j%GA?8z3j&=+7LC+OlaR;bso|LJ@k*i+{R*Y zhLLj%iYrKjuA8I>QJ>vH3#5E0fpv!BN4c&T(Bp-eB(DmE-2tlb;lhojGCL)2D#Hh- zR}0kV|(e8p8wjaf)XEH^{Bm%F3KpV{>?8^apfkViz>?fgR6sq-O=|d_0B#R;A=kr!VONFELZ1uR;31t4Xi4om8@#t;4x1vjIfVJ z@<`@V&q$YuI%}JzdHSj1-8lHsQh`B>wM%9p6^j3dv-)kC92vqY0UEpIf}e+u*M+10G`G z9BpBvC*mi*iM~f!zn$Hi+4~2$q{$B|IV<_L-NZJw@T{cTPO;4H&Sd$&Q>X1JC7#q% z@j?uH<3Ns?ogZ>DGvyBDEV0)Fc$bJ3I?NhD`X@3BFZLuDjOICl2s2LaOcY9+%~%RF=bE3!&id6~H_x z`N)mm@8{v_{NEl6#{P!Ge_G_~)CtwZst^-3xVf(boM)B}2HLO*QP&K1Bwr-5^k!|S zVT30z$_F>;F6*tbyfg>g1n1&(AS+VoLPL*mtR-nLFDwkJKG!bV%(x*zTIinp7^%1& z!k_>yFV_*~|1#FfK&F{x6c&7D<)R^1?-FEgciU66+^=*BXRmwm{wZWoL#BY-^ilyc z?^8$e@TZJvMgyl7py!E?h`Xq?K}8xxvzmZwZ>jxmI;Y_8T9MWt9NxLuBtPc2 z*|e1D?$*k;k#{wuiInS`W*#pQiSvMS;k-hn!YjsjVS!gQSgHVh3$eYv3`Hw-mY0T%tZZ9&NLW-HEO_E+$&WFn_+MTVx}*(E z^4Ab$`K#{So_lQQa(p{FKK7IsU~kpn^Y~t*;SdwYvJ$*)4(Hs-b%USlByntQQ@!yT z=n0j@4WLv77V(dcTA-jflNE0@kD}_r-(*g~uGF5@$Hd83Y{|Spf*sTh)kTi#Nj>m_ z9vQT{ZCy{^M1k;rE4ATF@rA#;_c?A}@ZLJK1(O|3i?J1~XH9i}TcIp< zCSVit>u#H52>gIhxxEOmyEN}G7IA&^vf_1A|K^NNIRHEe;yP7t}vYjySDBj>U zx|>(JTImUtr)zA-dB03;5pN|*mJNv|9#Tm>7X%2g%lak}HC&~=`0fjfhPhFv^^Y>!@s4v?!h?4T`Vy@~bJZkA7YPhhQ)-vs=PUc%i({B@-z*Q}$Gy zRpkWdN345*gTdL$E8U$hV}kI;AfNh8(q*Ld#uEEeT!X)#dNxo zDF*X8F3sgdrTSavYoQOK)33HwPnFVI6WuC1eaYaD)x+h`!$8hNadKJeOHeLcxdIDc z6GAXTw|>OUlRc*|c2+|`mAmm#&s1`C1!O%A#3vI?Q;yY3Gqbp=Uf#9$SmBC?$-L{= z@JY#V3&EF~V>L=u*>4irU+2Fgp3D9wI7bP6WG0Ti{|(FMqiIgAc-P#_vv|mqNzPK& z%&)5i);0EvFD^uxIOXu=W7r;b>O}*nAihCmX7^RI$CeXolN9>Wk=9u&N&00{sZFK0pXMfnMNg49*MFK&#?zi zH&b}K=`RKr=>Zn#LFd%^kNaP)zZ#=k4;0#_9M`?#J6xCF{|h;cRt@7F2o4A+nivQO z?Z0o1IJ5j;B!`-xS;~K$Ui*<8o(k)OpQIasuMq^h1F|`5=7*qd%)LdCpqfQCHYbv# zq{h7(_`maro`g1>vIcV%|Dtd`IGAzP&CK#YvDCc7Vn2yKP9@oP#1)OqUNXzIO4Tk? zS$|Mmt}HJ4yWH;ZpY{4TXlK4#KPOg2JfzetvI0(D)Sp{0t5kdMSO~8}*ClF9e=$|8 zX;C(;Hm2??QU@VuQxj=)nS5terJmNan(?eRicrFbc_cbvk6R3LCip-JH=p4MaO;&a z=s+hx##rNhB!MP^H{{!jq}->-=}ahCEX%Ecd`@f5Eb=fxf(M1mi1ndXuNpC+RGL#M z3zbQ|hb#am5|{xW0!axXO~g*g69E~M$<}9v;ged2+wKd{pJ`(qCD|ZLbzj#DoYVht z=p&YHD6kyFM0OO)_p8ppFqEJ~xyQ|=!G?#={qaC418F*EF6uAenW>XtS!Y&2LBbCH zEQ{rFJs|FtH47b1a*`$vyoOF)jVcmKa`+n3J0igJMC6AmPI-x-l_jiA2yrB-VCLGN zv4x|+CT_@s2?TWmiZY40Twv@I+G9x7j^})1z11S*N!U9(^Zc>p#BF$tUlu!eM%9aG zmVeFMZeP|j9K4lvhI4@qyKrXuo4T*qaK26O#Z#!`vA@53xv*vfpV>`}w7qZ~Aghfg zPiBW#-5rvA0L_eVt_AqnOb&2KpaXh*a57c<5btiB_I1#H_zUUqaKFl=blJ9Z++f> z&mp$4wsbyZN}akt4-<#WvQD5a;)mYIlTXD+KDQvE`o@%g+^FGXhb=F<97k%Tv54zW zK?W9f)~ZVuwNZ86kr8PraHu;{XtEOT4keR0Mi($=K8alN>wM3QNMh(egp%muDSsOZ zNcBSs^YDm01}CLO3ST@QmvHq{O=gC<_h+UmNuHPzFXBmfzX^zE#nKn&Hxs)^T|Z%+ zlu_8@McsI2p>06KE37Y2Nw+v*K~1*aL2Zvqgyv|VPm==qn-Nul6bsOzgsWRq3ZZe# z9V(+7Sm0Zlxlr+&=08pX2&nj_2Qo;Gw3AU9faQ&_*OK_a@2CjOi5fS`l0PJxT~s<^UAR-eAkjC1gTQ(dSdhKbbIUre;d`XjaPf zJj<3dPEuWns{wH^zff?u+)L4h1M3}jLi0+1))S-DLVZRGrDadN&^#R=Q_!mW$_B}Z ze2RkKFEt=D_-lvdQP4Nld=t;c>Qx9p`Ieglp3C{mwrNkI0$RP4Mff5)=|ZiQ#C-WV zE&c>-kss@{k5Xv>ncZ?=-6bRnaBa&l4a){3Zk8^ot=@(K5auo^}m-zQCgASYhWPt=E4~h z338s1)1)2`frt_2!L1m(_M$?gJsxVOLq$9KxdIZROKTrsBOA&p8DK}hW|CHYZ3ici z7ikkyD+kbV+W+Q)95jC)?|<+NxI{mb;%3#QH! zvTZl`zl(&wtrdj|ndA}7+<=i5QUs+Y`cTF!=w0IC^;)gat2oKmN`nWfa@jB_WTccV zlMXTL1w&7$euZK%8&c~{)`(E{hQtN7A;nY{Po(xZgWo{+p!wp+%>;Vege*qitJmB@ zd_}-j_cg)Jc>0sU^HwTp!tHEIJZ`a00ihObX}%jII3uVG;@{ zK9a#t+YuUqZdk<4liD+z?_+FD2xn+Ak|xCx_%hHSQ*y_=5m;%o?1~0jlX4z(7hk6^ z5&&#%Y;{?rbUuqF_6u&-IzBBp{}4Yb@~tQ8$Y{XW3;Q8Fr6HqxT7n?V1qrn5zRD_> zj28c{c)=t4545y}!fDCCI;3C7fJqu8yWV($MVpu#LN(+yE{m+tmyJ4EMBxKV&d=ni z#1k}X!$V*#^Vt3o51c2c%4!Jo{X5K7E*ifm8g`-#T(?6n{Ql`W7Qda2+67;QSxYIF z4C#?4uFmOex&vxU;gKMMFGS)gKQy(ZBQ&mwm`aoqsM1#)SGcUJ^#+aGvcJWkjo& z?4P~`#Fr7Af!TjCya`sUs|Ci<3`dMDNrA4$#qxeE@i|a3Q&Hv<%+lv^R}V8_V15sa zIy7`>Oh!`zUTYch^i?kYteC{tSe0To_gQ(fKWL!6!&a0Z%)Dpz;dQeH^ImL+OB-OeYdtgCc1GNO(|C^m zO2Ewo*Iv<4;Hc`X7PWI+p?p#*lPO8xtZDq!s4r7>rRFZ1cH=hPl7G2Ow9->clQ!=Q zG(x9gI_o>ufxG7_0UK>{V?bc~VrKQ`JE_n4#wX;-HO67vQ&p(R;H-w{VbDK)bwh$l z-vD{(P)FMa*zmpHAFy^!N}KO^=R2_jy5$jV;jYobCWbW~bl+hI=>O0xuYY$jpxa}2 zvmwQ0;|nWcR78C6CQOvX-%+IPgiZ5#W6Sug$W8j3GSOn(}xlv02>1zqFD zW-))pm&w;9@H&@lJnBggVrKQ=fj}J(S(2sxAf0K-35uAWmF4r>5Elk^4N6M$KF7p@ z*O#dAKlPBSk2~K(VQBc!jc2Ln+#XkBOLaUekloatqJ7Psh4<#hi@!`~UM0SYQ`j>Ge5st`aGu9PdQxHxdWJg&JerstB{Qq?8Zjp_qVxc{TK(&iuJ@m zZZg%FEDiu|*5&(1=%Z9$nGyvz4Y@Q+WLw^Qdu`kY=ss^Zql9iwI^Be<;%R?J$4`Yj zdCAz@b6StMDtNaL&foO6&iA5f=b<4NLA637?~u4Tv=s;SZj`M7XYp!ziFOQam)M7{ zkS!2_b!^jG4(!noYg98HGSXbEXo3WzaH&Vn7tD{hzKnIVXfW% zQ7iC|WLg!ics>nhFor)cVJ2(+~378QL_;krk- zFKH`SvbNE_*H?I-utmN$F@G=*$$~ooGZ3`dAQ%4dXw9b8hi`8)c^73sd^m@>xcRbe zcw)ctb*9bijk?}t8dv|i^>?k4l3tnp{=Ix4mnolH|F0ph5*XK0%yHH@7}(ki zAO16MrTLfJyFKeS=zpGKK3L)+{)e#y>c?jNKX8bhP5$p3;w8;1`*jJNe>3__uINh= z0v#V$g+(+{5*NQgkQ3ahD1{&p`*{xr)n zxdK++j;Q4rOy6ZqAe&SzFIuoFFB3-PwMjl*wLjmVuk?o;vPZQxD}$$omoqNU@9zny zlL3mF_NnC*xm1hVRa>SPY*3ryKA6^cRVqN%G|m9-h%z3T*cfi3WNE6uV%2Si-=x0> zD3MDcfvs?$tV{j^cPssn%)LCu`Vt6G6M6mk8{)vUeyrGon)-2;obo#?&H*R*WR# zzt9;UV;rI)S;RqVJT|(X)o%H{3^lTT`G6KfW-JObVw|Ucv_1zNXs;qfOm?0*)vacU zmTdPPmD>&^(yp{aN@I3Fqzb{TtvN=bd1T>U(nge`A!*K!9f)#?x6C>ziC~Ka&yE3i!w(I3W*VX^)Mo-)a0fV6I1r|7B&7kp>xRk?) z@s0Q!U#9lA(c{OK4Qqbzf(v=jMyDg=fF&b#+)Z8ZIOH9EM)H|ZOz?&QaAP!8lgm)h ze9qWboY`Q!2VNKAh)KP{T?vU@&@foaFgz6E(3U|fRM^Wwzs%;=>~-k_a!5PGnIt11 zGjQ?LjaBop58K@65t@OJ^&clzeCFORzSh*x2Qn#k7ePF&W;xR@#GsA1cKl5Qz64el zCh*Ug{-B?~S}31C?iKA2^3}c_suVTJhclPf%;QR+VXbiJ``45>9#aOmb7TLsjo1Ha zCHgX%u?^(J(hxLXSCN4VUOwK+QBJ*Bv4YpTOutH11{$!h+_6Wc=~p63e)SH2zVi}0 zg|k9lmd)G9>AVkgjyXM%0u2E_N#E40M7g687?s3O`(=AjXlMT-k}n&`J+}a#F-joJ zLe|G7utT!;&z7g4OD3q*0`UiD&5Qh$dLsWDlt=gGOUGhdAa#oa13K@Gm++k-rN0>x znO37lT2W6`Jisj1)P1KqvKI?N$g#)A^c5*paXL(%_}OVI>6te=OGE-I#@7+Sfg$<$ z5p+upz1eVkje>0@`I(xAK{I*qR|CpVT?d%;4*~vLfK7-~xbuW^W?|BgfE~Jd6%|W% zq2EUYTUD6c+l{kQo=HP zi|=T_FmLQ{Tb=ElT-OoH$M@}XL-=kAV22#j3$o7is1Qw4jAo})Bpi2@8QwhtI0E zjl{)cQ<#Lz^?d}YGH#a3V8ho{aO?NrfnU)!OI(?b(Kr!+x##*CJ5b^2&JL`B)tCzJm(e%8hm*$G7$uUv z4)770KTfBH35Scm4N$B9(wvnGP?RxHgRHUZbCV^ZU7 zTwv8ZuXZ!1oxmZD8Jz>XymR{=Se~bD6%t?bS4y|L{anIz60`gr<3%10vG>TLset+K z<_)4Zmsn4Rn8K3KM_y^vrVBN4&7A{oKb=8d9L(l3Z-n{`y!IXg%w4>a8%H|^*`D;; z!hBP)r1YtF94<45WIIPa+f|DXpkxx&&&Y}RTX8*Iw3PPYgWQT+So_xrmp#CU<>gSb&B zMa>;Lta!c5y3Gv3SMRm+Rv`O4&O~@mSXkJ+Jkoe{l^QcX#wqS=q|rUoBomTVNKfts z0&$}&{|J{+<&w&(tuiX!N$LYQd6T&Ty~Tv(Kz(m=TNyF6I$E^P#oyCG%%`|U`?_;B zNp6Gf!i&s?N~A{ZH&Z1U`BXmZ$chl7AMM$k`(2cVV_$78f%}7&YgwsY0QVw(+YP3d zPN4IGQhT6Zye!c~K6?o!)l$kIka-0|(InJi3d^(>8_O9|x6NTaFFDS7iPQ=>h3fGq zVf>7&a2;aYm>iafouq*RCOtGC?OH(qJDWJ{j1>i2jr`{xLS@UHzhn7t+U3E45OW;= z286#NmDuk&8s=Yj`!d5(IogXK=4WUZ9UO|HcXOFKquCif=4MYd*;WxNel{6ikGi!? zX!jm1wepY{0*f5*5hC&UU$gv?gJBy7R=H1nh-S$K_j^lK>V-Y?_%)C*dOnZN1fMxx zz33NR-u5DQ(v7(gf-OR4i(H$&Jf|JK?m;KOD4>&1`q2kcaC%ho<*X|`)Gcv2I>wxE zi!FyWBj`Ferd?&h{mTtwz1N`R=rr~2tagC-moJAXWqocy zqH6s}7ZLg7(55OV^4H^|R%TZbA&Tesh64V%W7zmHbH4pbt~({5z*%nI8A^dpE|Koj zxl&}cwxsM}=lZ?B;&!F2PX}Ue^db*b@5wSvy4EAb>+|F-xCIj2UKB4ePq=rGQ%&RZw37+4`|sno5$L z)gOMpSu#^?&*tAOVn~vQfRi4Ew{wC~+?u$PMox@j^~zQ=Rg4<-zYJA1Nt3BW{I-i5 z$;mqvy-G@pssQvTwGox>gM(%j;*DyWL|5q+9=qo>y|UYfx{R=(!YvKCaG`&7Cqzo z;Oega+O;M%A&}PgEQ_`*U;0|yf9aUp_sZ&LRI)tio-2hjmh1#TK_6rKLr-FqQD`9j zVx>hjk2sqo*6e)v*322JD-Pzc5ZdiEuHma3zt@OCtQr(SQJ|?)QXq1d-}Mr(bPe7L zr`C(Ry!Y(OOcEqtWM8e3$^=LrAFAqO+l45kfU|OFvlA}j&pf5;kT@98UA{!X$cDHr z+)LNsY!qOT#>=HlQuv|8tG#>+uD$V0^MwAuTFNH<`cg7uBEcauEG3@Axs2fc}5Q7om zjTA#?WC>B*H{S>`tyX+P#3#tq1{*9;!Px4HN}mx{N6Lm57Kdj47$oSpm?LO|;;Uz6 zj0uRNrNPpa7I5~$+)$YVf4h_~x(}zljT2wQ$iwA8lYd=kohP|cKoCcc?HA?gxv&pL zfF6scg(VR$INXQemjfaA@?nOA2ZJ5ZhiH+UJf@w`i~H4?p&VN|Gkr?2%8E&7XVKE< z3%v)qGTV`SU9y@s)l-}mm@;)1d)BooN-F7VJ@pVzp-OXBQ*BYdIj5stwOZQusk+rw zXb#5Doc5k&{m$D(V9-joE$oFT7>+LcwbuO|7ysNLC=k%fy*U#ac&D%o>dX zzrwW6VRHN>89z%dP{ z%YT3cBz#|bhTDc0;o`7X7qa^k@dN_?BleoPutE2&O$<9}Qb!ncDcsCRe>#FzoWX;8swQf-O8N7DBP5Em4C2e_O82)7ehjrO68G04s`Oa>pwZzvb7hidZ2gI z+}Vbc|Fxm5>9&&^Wk>zj%`d$5d?l5i8jhWac@>yw^+3boC5Z@>qG~ig%csXmeYK~v zbldH`sQ=ZmDOT59S^RrYuI5;+Ma)huqM?o^yk8P2sK4)VHTRmCUHEnQ@-6!exyM;; zf`{K#7_->roptDSte2hJ)kJ~3zXaXtlcv*jaAN1mN6nyNU9WU9o)$HwV9Yi|Dy349xNW}`D-b96 zq;=;)IM|+EV61t~Cia?lKQnEqiQ{6qbU{a>W73`$G3L7LadU!@K^CjGry!g4h6=Y) zEt&ZfpBx(c^BEUZ&9!;SrkIMGu;D7EA{>%Hy&e@Ls zKMm1bT}|?00n73(?$7{$AkY6OpZ^&<{=e0U(tnj&?;rah|DnwP82_K~7rp$^XZN|4Y$p9xJCqw#40MYWrN5h>3|O_pb4sb2+W~wTFMdZBtT? zCrW&Ip=5*+n32j53$ck4e!Vwf5S~HTyg^^l8sk5cr!;uxWi3@BUP+hMXTXnOQzpTE`iHT zlAzc%A_M4!(_jTwYWo0hq`w(q5lQ_kfjtJ@A&=518@7MrK{Qa9`4OiwEIEOPn;l-&~QMYl)W+d zr`s`z`w7ZOBiR6jl(dJw4e)5EFazpKEpUL0vfsqcvhdSsQ6kYC49t0wlK_NcnF7N#k3KrfEhT!a=J0 z!A)q2($iDTXlO0IdS=Dc%nswa4f0^ct_JTeKKm_xldqq}?z&b@@xc#Y)mDwb5C(US z-@sj6Mm*E_>uuLhs* z2A9j>gE&AJ(t;+N2<_gdBe(~`9=gxd?zp^o2Y7NkYUs~Xa&%1eVr5hPy zk{);V3a90~(7W9@G6_3@-u~H4W*kTaP7go%Sb?7LzTBAbv~>>pmHkZDq zk`k`kVtwYIIQ;In6zmh7;%B}XtRe}N3$-Fr#xeuV*@XSr@t4(`R#@Ot^g=$YYpbNPn z$^m5Keo%}WA$1`J&A|e$uC-$Q`{~`K>wT~wyxZ4X%^@D{Tbk@>+Yt7&X&Y9@SY{5T z0+3?Cj0cTp6Vgd2J4>Bl(R0|7T*!i8s{aj{Bwzq@Om5H^MRMgZL3lHRRaW?M&3d80 z&be@ zp#Wi`{e`|6$zLnfdiHrn4L!_Iu-x2CD!d}8A~mxD;g*pT9`HZpj-txPFJv+8jw4DV{AOHhZ|m-A#b1iA+aW<4>x%^yDdukI!_Wh zge*?OmGFiVh<0?`oeRDm+Z7MZ|7)_9pG%tohCC2YGZpKyylgJ}M_uxFl>6CE>9?Wa z`;cDY$06q?u1eWYyXgHeP5Dbky~(nBHn;l4PLk8aF;urF#Zx1UFr_Q15c5clTrk;F zrld+mHJ;gX3j)g2g-v0+euSb&Ma<#nl6(63CklmwnJ8%Iw{;iAF~(sGG2emW#$6Wx zdjv~5Lj^R#jg?WzPH)k%Ine4lxYN!R(NH)T0Q1~*j^E?)2?J1qwUWKUtb;xMY=WID z7oXhSzIE{z?E{>1{rj<1UhT`{$o;AYnWG>|(ywDHhu3<6e@tTacRlQS77w6B?3!#|<{HdK=UQy^ak&SeFwZS= zH{U%mvGya1k;S^kJFr&KFT>dsfE2C+GPy&SLdBmjWPAqc2bC~CzBd6l+hCHIQ=sDa z-POL=*$u3|A3#9tKSF1UX#}D8+7F zeWkQp?~fz0S`=6F$7x-pUDZC)7$>nd3T!QAX|t}@!pl8PYZabBm4ad~`EK{lD!@uH zl>x@S4pL>+RU2pS0is+|$Uo0FDtBZyUVCf8lvQO`+JudZlk=g#Iwi5*r1VZKs*jGR z($~{xwvwdXIf@u!29eTu$GIkP^6bdSenBPaUDGvN9|$Y0PL~Yy@d%~0M$^Oz=B^JS zp9KL}_}OZ&yT2c+hirpd_S+`~T&HMZ!zI6_S??muF&?i|kRqrhSc}R?@oLD;yNDEQ zEw-i{?Nn8X*X|bBy^LKvV*DuTOrwv6e@R@*Iz{m5O}=fn4>9RSSBI6aLHi1Sn%mxQ zh)3A$(cN#+AsN)RCskXKL_4&>_(t<|8U9iT zsrg)fjMF;QM7f}vbh(?e$`c%)K4}3koVk3~{IJYl$~0~gFMdxxoUK0FP3hNslA0<` z8OMArhF4FAYcrP#+*rh-)4K|P5!@`CkU@aTCZHV7 z(HfDzn6NWnoYKa$L;Gzo2^{cK2`u=WdoB^lo$TUu0mP7 zIh}7rKl8jl;*bm67UA<0;5&A#&hQc?P7$VMPG>mpShI;Sgh8r=2w2|i<41yEG&4?d zY+$fd-npO;I8f>GsClz!eM#`NPN1|- zqLTLe$>jbEFVZ;;*Yh6Mget$8vaZ@(#@?$loQ2PEcR2uYN0q)wcg;AU`(r(ErjkL* z_iq{PdW}-*fM0(4S=*u=*+XWohQ+)}CGIy_S@CXrGa#lv7TRm+H?bA?d!rS&%bk0! z&Wn4_Zyo<1rB~K&$r&gU&o82Mv^i4_N;dA2h1XY6_A~aFJH_*by3o!=NmZ_;WvrUv z7c&?#P8h}QYUbN(=Eet{`wOG+QgF}jr25rB|493|K=s@mH*|^nEaL^(Yl;!g#r+O= z%@B;pP!Ppugw+>xFOHr`6OF)+J600t%E+J~N=>65*5=qNJ5zxzumGnqy!!#8^$M}*EIswGyBJSe-#Pg0I}_L$-xANo7A$*?i!1p-eK}fbzvb!y`@XQ^Mzr1 zLp|4!+>Znq5L3zrSMri$V}&bLYK8H`UDXT&6n>B_cE$SO43zF^Zqu-zv4@v7(OZ(? zJqR=EeURU?laEN!Zyy#QqHBLuJ5xxHr%~D5WxDq6&1%3;&OKZ-H5inT4)#q1%~ug# z`R@n|zy$H*hJc7@d=W~q0zopP6cte8tl+>9j}$O`RYir2x@m}rGgE=42(nju;=2h; z36&Ko74gimzOscN`$HRV5&of(NAKVv=8C&fdMX+EDeZP9(qS5huQ)}^YkKRh$@bphi~xHl)7if|3|a ziNFH#6)Vs+bD22`9dK*lk>5+Zg&i_x7R4)o(S~HJ^pGGZ9!pVyYVy?9#Byet?=^VB zs0um`NI3r-k_cKF+)|k{Qz7O|8C$}*X&|el?+i>4)*XonD$S^ZEsAI@3-A%N++2&9 zwlJx*_4v)FM*S*EwF#-ri|x2eCW{SO3w%=5nwJkmN(2fK(h5R!t(a!_d*w`fT6h$| zxp`9mxW-y##gm!9-v$CDZ0Rgc3jhnaZs|+5`TeXl@3ff?eNkn*8A*3HoHA7CCEVrl zEc0S=l*vvj(5DrJdkTEL`Fw3cb2*T`>2?CK5(ah%6~$4I6%tj16u>Qm)xiMGWGVV$ zi^OI65*75qz_Yo}Bkeu_lRbc-@bdDKngo_%714H>g4_&GjR*sIUtS1%T9BUsV}QV0 z{S70e64ejlOMk^-xoX|hFpuC>RY7Xk0Zg+xn@?G3^1O;BU<}3lCJIlylRa~u(zJ(n zD@;1)6<#n)1bR#tJxW znRTYv8z1rpEf zDCL!Pj@Y@}>=m#)<*MRMDL59N%?SUi8 zj)K(CjusMh82ul(pSh#UP_c?(Jza+Aa_NBs3hNDfC^YrfWK-yMV||)g_M?LpWk6+J zt~o%#hg=MBqBjONBv%3KM!C42@xRRQjVd%qGS#K#8w{2(9(Wjl$W;suzGNl@e+xrQ z!VP+qpXF4BP~D-@!RdLzIbkYPi>vD-U5Hdz{DXx>DI~Q0O0x)xmAx*)a+g6Q*&jnH zVKz;`#fK|PdOJbaY5d_wH;QqS`G-14-M%SRPeEfU2?~!&696DBp<34|G$kyOP>c$) z!Kr~;`&Ilwjxgcy)J+FcoEe~HD2a>rZh8~m8b!}8YdLFv_W zn=`#W4da9&fENwCa(K6KUi9g_(mN^PhKX{wH7IM4>ks&kRN(H5kmN$>UCk$A@ zc~Y<@M|oSPv;|m5igkG8n1~vUs|dt7o#KI)(n~gmv#tSoz;&4SA-!53882acWTMA>pGB{OUVm0Hkpp_wv;#4AGr6h+sQMl~#S?9kb3 zp?-*sL3-BrRY|5_M49XfcX&WRaZ)8$ugqVi{)sKQHPr`F?NKT?awc(SGkuE88K4sT zA@2w2d?gO&QQq*1h0EM4!DdOxGLqov`T0<78H7sX2(VFTLZCT$35?ui`F$PXd^>*b z_uH!M{WX$@v-RV(?0x@uYpuSNpKjsnTPdC2-+_!?c{Jz_JCo%{hRo(hAP*iufKDcJ z3dh~W$s!))?CV>qkb6AQFMRv`nAY>9hAmj$pb{$wBldW(V4TJ^pcUvEFa}L!mBGG) ztClcoez_D0o?eO$>gB6rTBz0c!ToRJqbTQ4DHwe&X=%-lyC~nMb%PZbYBw# zY&LxRo3B6qri`zK5dxkrT_|>5P9}ryhH!f=l)ljL*c4oP0~pV(9^&dm`~>jrx7jraDg!oxMB#g!3K^D#du zgEWAba7=DhX7Sa6OI6WSXt;S6;{J^?jRP5JbtEkc^u#Kzgnn5uQwE622h8j3vW?Io zcV&igJP1+RB`SikB6{_bm*D~~u!)FPmkm*2M4S{ARqMBYp1+CD@q`>ZBEIOUJVI8i zKmTQtLM4*2aRw0pKwRYi^gjKEx@h3yZ2rHLMd<&FvZ!t2v^j$KQ>*7t6b&_3Kp-u( z1;=a|no1G%CqhYb!*=Akp1__2F$f?8ScEa>rPa;t7LLQ8Ky-bk*e;WVBw)L{yL;R0 zb>{(=I}W{yBz;E7_2J=-)C+lZF}}(6h|_BqgWjet0A%!vOrjgp>yRwrveY`YG-_pj zUiruQz}QoHa>Y1j%4u+!Xc`2&T4dt1utF0slsk6{m@tsTFhHL<_wGJY7MWeDeb+cC z*a=NOo!u#k(yo!k)b_6-bS{K`_-F;Th#-pzoy0@FMX{_Ify7Xf^EnyLn9m6mqE`Sk zQU<=)dbW5)mSI#9!BOD)c{K3vkHGyEdpTiVCm}RZ{Ah;naH=3zR8umkcCqAMK{6SF zhrvk@9)T!wa6gYiO+G+lgg`@vzi)o6EL+^SeqdfUSL@h_0fDLP3HSkmh#>GGO$a)W zo;#0_+}W&RHbeg)LJ3Gd|8x6Xf_Z%jXeJw|;DYwjha=kTBrlMWY$HivD5t|aBSzfD zSB4iQpWuTePN9@Rmxz;>!x8uz`+T7w4p0bpEIx;eOspSDS$@+JsJO8r%wJDZQ#xnE zsg&?YTe@(&t)I9nxb>Ys*PS<_r-9e4 z;CEiV<2!C)YOQu*FccrxJ9p%+UgI0xd24wc-O%=ny<5-RL4Xn+*G)Im+b*rZ@}1p} zm#ygAJJ>H+9A2%$uG`(`aDrENshP?H9XWuJo##)pb$e!?F0WXyE4V*P#%bFI@FY#r z2wSFZ*FW`MKQA5IW^Q($ZmmIZ6b^ueUfVk+_?j16&Yr^HnxSUZk8r;J`TZ)NADLLz zy&<|!>z8om3%)+v35EsOr>oaDvPn9B?KbQbviLedKll<{>J*XVAAc%^;RQXhe>m}7 z?Yzq0-fisG!t!L#)cn$!dvD>mWLIw7z|8|Ef39Jn1c`xWX(Iq0;a@8F^Ft<@v2)3ja(Z z1I&n55{O~U>^4ENwoG^K(D;_DPhSzQWK(Hosr6tJb%~8&#%1NW?%?7h1Aw$kB8eOY zmhxPsf5=-@+h~Y=5~@h{DG<~-Mc`S~DRliE2LxexJHQR2d|I~nl`W__S_ATFC)wAj z7bGqD47I;EvC-PqKyuT_S#4&V$ej-`b3SJODQ^$%j{L>V# zSk1l@_VoB`9SW-VCh>%RpzZ+=0M6iOq69m~ni$9&-h8Un*;`}b!@I@X^keWdp zg3}BdCv~8&eaa008QPxMk>NJ~tm#23vIbX`7+64kQYH>cM@J^^BoKniEt9 zXSfxeJEVv{br-de(ia7tKCD6l3lF0MhvsWl?l4pPI-s_Z2u`Yx@G(+ZMoh3#55yh6 z+{_K911^81BIB7a*m+RCAX$sCExkJSS>*ChvXfLz>?oU5c?85#Ltl|-DJ-tXu$$3p&%TO><7L7(-by?dqAj88(qQ@hCBm z0V)&|oM0BmdS)k$Uld7^=ADQn>*aT4tDHZelkxQ}DYV`Q$cE%wYM~>vtCnynh~OVd zQxr!xZh(=e&nz|>BsvrcjO~w#+`#lXfx6#Ch#!M7xaHcIVMvl%J_EhP$={<7stb|o zQya4Zo}UvJXDbXu)cd{N3~9Wm!o7HxG8&`bU@H-adMe^~iTT?T-(6dWW8-ACJ2F2JVLwxfHl+ z3d)H#ylip%YoOBGm1uRsYsa!;|8T^N3y(~{U{{| z_`ndg{qgPKdC>_Q6)o%x{#P&|{<1t&5S@ZGK%b%D5j=9EhN+Uatfe_JQ~%@NH|?hMB@~x!gKERipZG%{Dj0gZMsW5ig{lp6 zSD>x;ChC1GBJ#if_zDbk{fOA4K;b@x0UoVc#XzmA)|dBlIloFVUs%ih1mqul7OFW{ zYT;3WG*Y&rg<%a-^#&xLN*$buSzxuceN;i1Bq3ZNCK&R2{_s4K7Csy>D$E>41-yJr zMsy=i4x0)?i8v%P$7Gv^C$Sz;)WKbg#Wm=QttZ~Bgx?jNSc)eTic%u@&|KS}8xf$P*cLK>Dj*T&%YIAMwO)jC1ir2j-WIG1Fj!?bT_xW@Qm zuAp8~xoq8)N!gpfJeOCDOrE^kGSc8%t{}NmLrF(>&PLH%v3E4z_~_h|UXM2Pezcu* znpD;F&nF#YNYvE~$gG-w@})ABE!U_)`PS$vay2h32R{6XYA$bk!=C=?FC&%8R)nBI`au}V zV8ECRi4d9sNYgpJ2L_oXkqDlrD$ehFHTYyCNs+qC=raw^u7654yVIw%61o zj~Q#&I|Prm@0;2}Y1-&Jf6jrq@MP*_O3=+SYS1Q!@{99uoC@7j)GPfZv^~IEx*}uE zTr~I1@nj@IReBj!Yt+<45j4chnHaT*)|Eir7VIko!XV9aks`n-Ce(dnpraEPrUupu zY%h+oZEGK)5ldsi5j{%bxUXn*@YvBKyv;kEv|x+)$uGYtBCyQNE-Wp~ubvGXFo`R? zD@?&;yjc|2m(Eoy;h8^M^^=r(bEFe?WW!Knz-^z!s?&%%S|o?n(?VwAe2P&i|P>0w@JF* zZqdxFbv)3c_##XmQ@y!$D28?K!YQSf+e^2){X%^b!LbLqYngxH0~TLPUDU z+CyZwygxFxC9_*0CZYx$4~$l6WE3f$`6E?v>|U(bzq0{fi*?sKqq7dTe$-_wz~DP< zJ+Y&iO7rx;*W2$$(*5q!N%EDsUDFZrY}}03V~%xCjPiFzjR+3~Xp`4U5vuN}NI4j8b^+Oe1>g?~u8Dvxs!#eB8Ej)izP$;6P>34|-J4CQV{N zqBQw*tVw6|90_DP%}N8i3jLH;{{|b@cp4q)^G*+Ob|8?Y1)$!?Odzz&Q9>xeD7F)7S`Es<|y8=ZXp;gF#{K7lL< z{Cq57NN^ur*3}03C_IMx(`QDbk$$D;v0jKEyY07fBlr^*J>m&kll~n!i4u~VrAjVk z<*H@+A)%JBXJe?%XMJpHT*MJ}dQZ~;eg*XXGrp*P z+LG4v@rv9G9*3e7K|QDWvg~*Bp=Lcym&|ipp~IFgD#Bxn>3kv4EVl2IwF_kYBQY-?MZJ*84Nmo#+8r>Xeac)eP_x`wVG`AqQ@(Ph0 zsw#M;Qc)^-xx}1W!x{rT0&sHdy(Va)qxnDVXR!$%JLX4e)Fx;wv|IXq#-k_O6Fb-S zi-tMiv1X(YHG#>r&%Tbqs|yX%crjc$(4fJQ5S35O`$B8(C3F@G&)kqiPG7&s5}W3v znIB?iLk4`6ZWZ@jWma1?Ut*H!$QA?SJd$S$;nnz{Kq(T3LphF1@Szkg`AgvfhordN zfWJ(^4r(C#k+RHbKYCD&!(=aL$d+x+2DPD1=t4&9{uwd{hM~&eI`%xS>u8KyBwEB^V+)u=xtQAr?e z>NJ~~%Fe*OF|XZpDvM1i$?N@jO6jrisU{7P<;&WVy)mcmOkxr{4^1wRa(%EbH6q8N zhK`$%lUg7>PC1KRAg+>RF|#x=)6a)jEf-@IN7A3z4{vAzN|-P4LCpc_A~1-aNdX{6 z<+_z`Y#3-9Ra#z{>#iZWDmyo4{yxA#Raz>>x|8g{nXhV0xuJDgSd%-RIsi;w^$FnU za|B$vC>Eoa*+XHF*~C8TZ}4p0r@QK0_gp-k}zT)wwAr-g&TH)b-JI(90) zosi%%O#^k`nZ0Z1jPbJ6_!mtiNPFhfszm;lIZ)kXm(`voN2P!;_wl`KiuRVFYl$^X z`BE*FdsVi8gZ*(YG98c*?nAU=DAbaDgXn}1tybbw_3VC38AuZ6s63CR52Slo))aTt zfTlyR0Hfw6dod0ZJ}0>bNb1n!<%k?GgwiJI&8rOpjpg}Ibr~LS$U4=I35dvn_&W2j z^h1%^2WvDM9%!#k5`k<#3OR~x%OyjfEyna}rznO`k+`~;gV?1#vT6Rb;i)4DbWrj+ zn;?^(!8w1Z6&lS{F0W>&28gL<;96Ntp7FkS>kIP6O`=UU7=$6<1x0i!BoAm7!G!5u zG%r+-5n!S7aoy@nL9Z|xBrjO_ke@a z-P?sd+I-Qyy8aQR!7ntvESb{9b7Mm3<@_HYVZfk#YS4}2W+ji~j=@_SQBB7EXuo3*nZ+WF)39y7J+;9j;RMVNW#5r>_-s_Vwvo)@9(!8eDPSl@I{r?K&6PKt~yOI*S^&gpK=gc=-;bzTlMWR!DVL@ zAbGJXc_6(dc%tIzG0wX+>r|61ba?ZLyY1C`OIuOU{_QMWcuR>29K*JDzuwX&W?(ZK1l|80pq)Px>-s75 z*??DQ!@Lo*lDMn5xO}){io&}*2#tGli@8XR8!;S2<)D!EA8ys@K`oRcyNYQYHV)# z@dp||zl-hZX0MhCrp*-m1tmVH+tm54X7j#Rc0abv=2j?8@p3>dpP-&s6a@$!i~Ek` z!TXUIG%6(UV2|I#_c7g5{d}w{20&*eWob5t-&-cw2-4EK43e-q{E0TG68;BgqbckKSUpsr+Y8KR+i^?MzSFhUxUN2EYs&XR414T2j(!6F8oHflUGaTwvQY>zQW-^A**<3N5 zoC=*JlKi72(nIXwRdccusA8#D^-^1MtI{yl?xeDEm5#3O`KgVC1s z**c;qy)qlUbn1^7v%C^PJc;Tl?JKoii5=$@Fw!opQs10ur@GhE|0U+;MOm zfXY;&*`)k6f{@aRU}VPWW6&s!Chc++F{cKk*DJ9<5xzMa$}@6(9@-PmkR5$?rpg-i z_Zk`)F-D=<8(-MI$|+M4<|_A4XdT*&!D-U|L*Gl+1+%`?*C8f_I3Aq>SjkG?Z-^rnBudN zOS4r|0)?W>sE__`1j^@4Ob*41Avv3#AB9P2*57XHIu)4*SA zzaK&)HhH>2y>WVX1-&aa8?*HLO$Z0`)nY{pc@A zX`y~^U9$XjuF_mmMw!=^g|SDh1UYy-!Bus@(JTN$-%UV*2x(V*UG@%1t58MO`MfIW>3+x?J#W61UG4omCR;SbR-Ttet36H+YP9Vdr6EDzt>X=-WdJwDjQ zk^dfFH~GE4{=S;jhNUg!=~Z_4VXlnw_Y&m}ysh*WFK>iAs22O8j?f!z=&hFG)pW5w zyrRF>L~yx>{|@Wr^-kAqe?Kin?fRv@a>VM5|AT$J;neBsecUttrMUiJgAENyhq}^>747WvGZE&XztE` z-ekGqTRp1!E5914>>U^N(b97uyMv#lo%g#$ppka0z#F~4&e~{e=kobAyKxt#)LM6J zfXyF6lAcHF`&l^(lW>W{ANKWNchuY4H+p!(j{p(4<9wLK<>%KQb?^^=l7Qys;H4+} zjRV(C_2xu%`Lf&L_d9Ebsryu=zFJ$eeA>&K_0@yR^F$YppZQr*Se-IC8e}DH_xMF& zV)7W`BP4JSOza_u7pYKPWVcIW(me9-hV|Bz*a3U}b39d#Nppi;$Xg0T@6FaD3s{oF8Uk8*l$2 z8~@ZEBRAy6)oyE-ncH05k2PiKt=Jx#N^H(*{u&hL6=N;W-t%!I_EHXy%2gX`Y(g(; zbx>XX5%m*9va|6?4AHB>&>Yr_IQGK*H0&=hx#*mPt-mC7 z2^k9by^Z?;==08dp$slntKOllM4mj;YDmCPWL+$xjALief@n%w}6p#>R~DhzKW zr1u7C{o6pA06s8$-TT0wAp-|74AsOFL*_K}Kdct!OdLLyK-uvfb4 z)eY)f(aD1?!7-HRIe6h+U!t7Zr-D4{Rt>O+BB`xn5CmQ3wACQonD6^1Ex8#Vdds0? zWj9ZL>9|X;I%l*T<-)QJGgY6>)I#%-2SCS1H8e-7Y|;4gp9yhUCTADar$wsgt^*}5 z{vV1_%y_Pc%TT} zWLR}~gE;{2PK3xl`==PY4}C&V2dQw4E6j?0>X6PV33#;rm~|CQWB(#as-jq$N)q#i z#UY>?E#dVA&3>8LP=4n++Jhxxwu@8F22dqCk<-AQ3SQO^Cak3A6e5Dfl-`K4S*M!# z%wf{kAKD3@R)2LKVwV2Cgvb!dEN9>r9hq+YI=KL}w|Cv{CeY^TFl&6|XoL&^^|_ z8dQ`6`R}cJD}o09*jbf`{%qtMNjsWXx0uzed#JjNP!Bli;{9KHoxExPWd#UHc`?Fw zBo)WNcC^EdHza~HR)YN{m!m9b#5pZ_aCFWW_$+YMB5}MIS|5|guMa<8kFTyxZzNab z_Pv3sE^T0X+u-61Vu3h$?vrx|M;`Iel~y>bx>`ajIp@%-I#i(`mOPV)VZ7E$1fK|r zTw##0s$w53;_leG#C?A)n;sbQgn`LbFs`7Wp5b8jzJIK@1f&>R1V7x)B9W{wY)F3<208t`}11z^2Zy0 z{>4&(rUlfo#+urH$rw}DlH1CW(Z%e_HyIFVZ56YVAbWWWu9s{Vx;^VsZ-8wBB{)Sg zHNiv@$r-LApr-41yUZ0@r6ZXy$-r2hVJq>`G^kW1MH_caSTrbD+rLeT5HV0u;paNZ z9C|)0HTwFdGMT5DYkCHZGxETX*0^RkF!G|Zao!Gy*p}h`)0xS>B~C^F_4j8&fM_O9 zQR&67Mt#QP!%-9Q7OMooY8?pzvJO8m0qJ(JFz+y&3J<<-RR7|F-Y!x+Tx3bvKk^nn z?qFbJSZ5-*fq=D|O7ld);^tD(_p^(_(1Q~UeGzwc>fmaR%Xw-uV<+}xmd&+qum2D{ z4XXAQp}2S89yp8=WTmG?{###-@DzarQS~QI(Q@razIR|BEt>0YBCe!5>n6+>2^#f+X21ocgOXW*ENS zw2zIju23lSL}KdxecUFQ(z1Si3Ppa}EMz>Wr-6%r;XW0OKxKB>i8uG`PaAo++UW89 zOG6unkDL-SkmfFzuw+ZJOfJ>%MDYgi)k-4diAzo>iDpPUZ+Ut7MzKl=C3&a@?XJ3a z*5ICcAMF~as&Bx&1-fH_Y2zb=W+|FN+DJauI)4G1p!YuQT&$W=bG$Pxn*O|d2*4wu zsLl&j9a7f331u*O2rQW<_S0O2!n@5ko%oI!fs}*{MNkjTp}yMzSCTMxMC=TzC#2S< z>dt59JL0$34sdsM3{E5EYp!1zSta==a8->Ph$*=r7hLA(hQ-(g05PC}!6P$!QL`^L zY|x}kmxWQRJD29xoG8R`-L9mYuFx#}pcOQ4KpDXugJMp?_z_@xFx+9^Cw`cz){~$24+LwW>{@#h|*hWnk?yKH!zHuI&Zm(UOZZQCCP}>gYXK06ZP# zG+_HJX+LGDdGNwN;F^$QBB9X=B^Nj_N5$m37T0j+pVO=V&LX#y zqR*>DY(jKhqPS>0O4SC-DiZbpDtrcVx2CiuAn0pL`?j3W-E?=5NZ36&F?&t@7QxTR z>_bs>1iHD`R?|Yb|Df?8qfZ^(Cb)xo1zk*bM=YzlxlE6^E$R2^hvza%caFP#!d7|tqaPJdc{>sSKIGc$ z8~T(goHvnd+<95IhXY`wLBjV%p-UH?@+J5UG>d)FU1W31omNWzo?Ol z?Zy!XtVONN7IT)pjvozpdKfOx9{@>A%kPsY9HRIyTW1P|U9W+&zs~NPh2fV^LZ^;0 z>g>`Oqf)3OAfvc48a(92sF!>ImQc=@(AAyB1Xv{0Y|G4YjMKN{G-&K8uyi%{D&#`K3$pRze-(!Vy<7oY8$Wj zmSjw?E$^-diHt~uy4$PpGvlUd5BwSj=(?B zdp*h6cU6J^gR*l95+#Vzbn~`t8@FxS#%_p7&%tlmXMZM%p zB~Jb)zyC;bT^hK?xC+_&4hG#OSwgsW#Y4YkW?cG#>D35#7%ajy9Xg<@uu_LDngfPo z6!vBMz)TV&Iwng-)6tEv-)Kv*(1rTV+1uM>*|yvq&2tudGt8Nob+p0R%WxnAH%n3e zxz>V9Na_o0%#DM!T^}@u-*0lSzz$H}M_{V)E7Hl7b9Ny}B3h<{y@wtWg=Tjlye#U3 zUAchIj{D03d zNqYbn)cF^fOHPQ*GE*_Av2-~~(sx>!j4FD)EW`U~JhZl5C+dv5G2lxIanA!Wt)+Lv zoQ??I>1y13&1ZF`O2f`o9Twg@rmTl4V8ks{O$K*jW<_5EJDmDTjGm6@QEe` z%n(pVclkw7^`<)qmNsTh-G_%`p+_VycT)KC{W@|8o{0I^v9p=;kR3)m_&fXR0?0umY`AL*AG1U*um zWY(r%NaLXP80OGZt%ey*LI@!4$~?AezPtEp;viBd=wQrYbWxf2^-Jg_4Lc&D9wiqN z!AAL328Vi07uYM3iGEf!{NUc>Ph4e!t@Ky*LC{YQ&9ZNCGSQ<{`AF+iG;K}gvhAV* zvNp^`z&0jDBY&LzSpSldQhE1JngZG4@ ze$N72f28;0X#mF=Z;Y`l*c;OTIk8T0rIjy4I$0^X7ERtRnYyew{n2HHK)CTpuRWfzK~Ieuc$Ow%;S?I;)Lfa@%|MRPYobVT*?84Ak)aalEF~o z!R%`o-tNJ{ijmrPPB_V&_BC{I2to`D%+}{7HQT2yT^;Pv&Qc7%WZ=RA3QZj4L?t|U zJV=3(jE>6h659e*rhT(RM<-896KhkTvx&ENQ4D3!x6>)BvK3h_#JNdidlz)Y(RB=F zv8zrPl0csZn$dF7*sg!dQZK27shzp5_ec=rk98U44-ux&}lk??s{ywnq|tR6hrP6L7*%jT17bbqH<0OZ&iL9V&lyLH=c!BwSExUy6*)0 z5J2@@Xz#uh5J_Ytdvo^SgnDr1JiJM5;@Qmp1nr`hartX>OU_e}z%AJwrRjQ`M8fP$ zO0I5J+UOUBg-Jp`6QlN;80%rgqee%*ACE+qc2&2tBKX;2WcAsP2n`Q#Ta zri8aH5>j7z8AavTNyfF?*3e~is6^TUiR}r_T(M=ljSMk)Q@Oic3fbry9^BdAv)TI6 zw(;9JyMA$p`c!xEkz(_uML=AVpu08NSVE9JVH@PUo}L4zKig&m<-SiINHrx)gmI;nIdaRkQHo{CV|(B1Y4xd zKEo$g89wn*Y<4k22e!RgUij73veRZ#@C#z=oSK~QI(gIUWCh@QHeHiOayZxQpnn>U zw<2HSVu44}_Sti?o>=_umGgYXU-2MQHaq$JoCa4GYHA`m-!>B%J=kNT_4v$0Bkt1b zS2j$d$g1h3BvCiK@E4{{J1zRgv^ZuQS5$n>VR1B01|+#pL%g4WJ;I+Ktm}u#DDin> zWM9WC!*8_>8=Bp@t2B@KulF=8RiQM1}OOH z*G_lVS`C?)ufwEVw5|*AQ`+n{e%}y3Z?EIuErx^H$Mg8O>;E|a`_yWHcy7}WB6Q#~ zj8``piZ)LCvloqLDV$3H`B~IRKb7*1gi8s=bYfU5OciE-K(Sr(M!J~D7OVC0a0%G$ zJ==|3GOo*`InIjSn3(Rcwv|k4F^F=1S8`~~etH0wI6Qv?prbKkJNba5UXf9Ij_}su z!@*!&B@`!6>7r69rK{%lgZ-^R)u-lYXYIK~?G6p~8hSZ1lw1RhP&qe8k6j|mC5x=d z9n{Pzaa{T}kN0BL3-caEUJL?Xi)G-<`BAG|*+SFJ!%8x5Xx`luwuw1rt7>6$mB&V0 ze6i0_y1a;hxR5S+I$%c|!q=f9!1QG2o#ujRVmbB)=!Cygg&%^MA(0)H>ARq|!;u{Ay0DxeMsRsi~6RX(p^Rpv5@710(WGpPtSE zUyH`FT|y-;M?&qb^ku`#aIpB= z7AoFqLi*+RkD*Ul4LkmdFvxxAMx(3D&}2a)On4YZOzth%>?yU*mDh!vAfVWxxOWxQ zeXPbr3z>B8m`ae*Tp<9_cvn&=T>-S^>1~GtLdWBvId^w-)Gw9U)XD;qcmCUapNK*Z zBuAsa6{m;C2f4AQe*i(=j(D|I4`(p1Yt(3iOe&s|al{v(T5K>?%Giiu!YK?8m3kazj%wtMaBkc<&Ar9Ag>B0eRmMoU1FhMpKBr4_)3fi3MpZ6VGtx( zR1@v=3CQHTGb}r`I(6H(rY5lnCr|(>|IwV&bj~+ZYoAZW#mp*?*X~N3q3-nsV#B@T z!sHi^@^8j}%!7C^9bMVjxJm?Awf`tL8 zyIWC<5=D-^>);iZ$cc;oah)!@nMEbV@`$}Q#5J$7YD`fS-i@oQ*`jBxa{Ui<(*U4} zZcm%t?FS%bF9*md#w+#LZ`m7>n;D(OIcJat5)seS#lw7E5pPS09WIdB&^@>XLZH%S*KY*1n9D2bExu45U;G3df`3F6c!@8w_rfE+B*>q$|Emw?weU z`8ZY*+1$SEe>Da-h!az1qP6`R64sBC>qHZ8y*gztg+jh+qt>Jd>n4x z5Cf`a|AR4U`Hg4%yz%?u%}SGxUsy;tkY*L@#0v0AkhnTcLUuDYXPlErRfy&oco=2| zPsFk16htfz8poijh`MGoXwB1->(#VOFy9SpN_ki-uv<@2^{iczbD+WQ$K4f}p?}Ea z1L%~dnL1mpAh$4zJz`t(Tho=6$-I%S6Dl6GhHY(wgxd&4T|eobx*CUoRCqvlKbT3y z7#NlG4Q5PtMbVpSmA{l>FLc$=rxH|YvT=t^g}40kZd*cyE{ibW%Nnvup^{^dH=8M< zd8B1t=QwmA2ubImvd+)ugA2EWJCAy|EYt1M9+J+V^<~zc z*9dRf7-l--u>m=uS`-TtT{!Q*QtRsAr_M*Xa}eDFXw>n8lNX8jcDZ~$(#ptm3n@d~ zjY2cV3pj3g?~{A!l6NVY_LT!+bcoA|HBP!1Qc+X@;>piA3FPI!AuYsZejhKMsUJqN z1I^IH(7>p;vGFx`b3(ELBj4l(J?OU26V-fJCRL{eAsc|LMSUI_X-FTu9S%CgTq$Om zrVWxFE$#k1o6VBU2wLGYBJY<)v~t#9OdsfU0YF1KSQ(2bto)F;0cA@ zF-u;5$^BY+E$P7giqI285wC(=Bf1vAC&Lc?{!>H^Ww!NyXT@X_cNb$Y1}Ql1yCq70&XAKnLpxpZ|8QfuVwZCN18_t21t6`11vb*qtRz*)qX z@(NQ;0bAba5vt4Eow4aWNc5P#w*N)e4oiz4y}ZJzuVZc#PFvhGfXGEqkZ`N?C8HIG z3|O)N(9Et{`FUTaOVyK2!1Pk;HiEpdkt3>*HYXy;kx6ov8edkM0Hw1^p4{3pfbg-D zvjcEl=1C!z9_N)RBz6T6l|V$`=8I;T!<6lg71dk*h72kIq=(!&3d|Zy*}3YDt113+ z#g4|92EjdS74aqwbMwFrs96S!Ghvb;#;@tg;_+$HbMm`#iAfSFyT+Hj<$*KFxFto-hdl# z9*jR~w1npJ(pZpJ?%9Nhq!vkJTz-?`hBUg&WE55^4uL%f*nn`t+JLwNEFsX+O06Kj z8_zk4e!_B+xl6BUU>sJn5*fQN?$)FRqfA(jOEb=3O9?csGYGn(el%g5{OYk0A~Ce1 zL{RKfRGnE1-x7|Y4d>r%cOZuq2Ej|9@t0KYp@{QCd@+v^CGGF%E=I&6k0F{ByXpwq ziswbc9p_$YGP@m_n&D^LC;CVQT9~A!18b~ZU>rKDl$^qPEx9aYxvzA}tzQx#bMI}- zEud%xhi?Ga9KW``2iCx5C?V3729AQjX>gFzc(W$VAPa)}uh(0nv?u3c3I=;?$_557 zT+hkYk4)`7UUxKCGNz=oh6QWk*_)%V&=m`)jQhyyzko2+iE*UMw~y`N z$q%6{Ks}C;Kn$>W1uSqt_t{aK1@1*Nd^Z`zo_uJ{^#B5qDc8m$kmuRd!^zFEwR`Je z!}i`&da!#_%vL&GFsb9DO-a{nOWhbDR2MPSbL1!dNDaSS`$%3Yu1@BDI{HuB@L&U& zL#G0JaS7F`)sOBsxQhzWz zHdiue@;lj2hI~s&L>KPoAPDOo>Z7u<#4(G3VreOi3+TlMbRUoUKSmY_wNMHHyQEpl z-rZO~HJoQ^82ttNmj*?C*o%Cw zt0J5&aWzM*nef8Qt*dQv8)O&1;=979g-L}BKsI7h=K|tb&2gH;>WJv$E9X413!!C` z)oy>47Hz3J^g0jhG@|$5yhfxlei~wilnBzCWMI}S>h^e@;s*W82NfJ_c=1o;HM(rIN$J3RY_2BTiJF(?qf z)V5PF)AK@eYu13CMLa~n>Mmiuw?^NA8SjS8&epE z%;?qO#gOelh%*EHaC|)(25y;1gQMFvGDdIK(&|XvN#;YOS{aP9Hkywp@&{F(lw}$M zws*&$L*12I^&Im-#Ow`@LZl0(GSoY?MBLe|h6}ByRt0BS^gm!|ciP)<^TrxB%QqSA zN3$}(4csy|bwGMCQtUP$M(2Gse!SN!x^pBF2y&;X1LT;JVptB`IvCZ(+=L$KOCW-y zs$lzMP^W(jL7JuIqhfw9qUAAf;ePRxCBbv)9YMpU>kx=Q<1-d$+OQ)VWj93GUIeav z88Of@5K>9lD8e>7+!}z{%zMlxx9&=vk_bM7*{`uYD}@7wlU#d=^?y#n4Z89&N88QA zYB7n}?HL^4jLV903zRc)VG8OTIH&uI!(9hgAg6W80_ApKHMHq+>boQsxSwcisn|3d z>CkO8vloBFDYPxF4l)-vsN~@$c2hAgEmpemy_~}6Iy+jJp!JDsm|_lA44n2=$=!h# z7Sl&+g_XkHGlQES&ie8&#CUm(Jj{x5nxCUKgGHLPVV-)H46}1sBVouZu9pAp5ydAH z%wwyIqFhy*obA2J-F_DF=F*xuh0q`o`8kCB)B&IQd00j$uNZnoR#XlM*_H4Z`Q?GX zX*EO_f7C?3*E#M#fIb~ALR-1Fn2uRcj3U~w&~=vzx*QoNrxdk|tn+L_ha8}6HWHA_ zQzh`AxsR!sebnx`yP?t6rPM}NJoY<}(qsTwNI03oU)lU5!@b}=)N)e}Ak0As+Q1j8 zPU&Whb!PQr~fX;5M!%!-2Caqw9}0UUhd>HHz7uRRHcf|F3$4e=4H?yLN85pTQuXx&V)Lk7v)qFVW^_rr#04r-^h0AhsZN5{NKqmFtt)L2jqJRiwJpYCoMl_y5!$2*lrm}3yb1dt zf+$j^y%V;^;R=>Dezj^o`-VB#QXenXfsU*+W6LJvKCrSsWYZ<+3Jxg5LjMHGUO+08 z9w%=i+*kdcJ%8^QY_cD3((P~UP-OVLt(~{HaR!?KR;m_mZ;5~m;=iDEf@ptua3#|u zY#%!B#_?2HCbn>>FEoI0H78DC*lO9OsaCGOURjW5n6a3=(*ZGw>a>eSOHc1;_B8tX z)iBu7(@YxsHZV_fBpZ(j|LrpE7u374GzAujvC{$dj;j3TXoCD>Yl7_RBPUqg@gcS` zsBZY!Dx;$=$%d3zF1S$A3)kdd?kA{Y3QI-_NRqCWB(@p2zR;ApQ&&LP-<+)fC{Xiw8Z z)HM~YnZ(ARmux8a)<~pi#*5Rc-dg5>ca=O@dIL`WZfB<?08hF zRhc*lzg2vW$?+N5S(2P68KxxaE^LwV;<^drY<@2`Q2fu;-=R0=KGL%@&%8*={1%^+$Ns8Y@9*n}utvCqU4&Kf z)Rli@(s|h`xCvO!JQG1{T`*nQ0w7~E*}@u5A$p~Q`Y~L4_nbZUEiC+;{swWU)VDwS zqs*le|QM{A>9V?yyjKdy#G|Rp~{#hj-mNqLtHVdZeZ1cj{3qrRvAc@8$zzj z7SooeyiU*ofq?@Lmy;P=!44rkG=06V;42@qN7`GRDVHoRRb4L6de{99j6Lv!8r4rm zoSBRv(VZsO8RT8~zkhD{Nleq&sua(eW8`_pF?huj-;FQ}WQ}G%MVz0%uD>L1e0k5e z@t&@^)!=Vo^xhcMVjDXe`c5A=Br|U!BC`0R>q>b@4B>asaH-6gbGO9BV6Fb-{A~`p z^lw2IiQ?SQ?xR`-FW0~%H?7l4z}%MSBd>!M=81n~EOJVdfcD*z%jUTB+;)&-%%SRF zOk(shWIb|r8OP3WNOpE18yPKn72$Mw;poVj@Vc0+Xh{ks;!}B%ZgweqvE>}8}ru$WH`#amiBy0;7NzAE%zfoE$u+PB4EMv-5` za>+Zn(9q@1M_eKWmj=<7m^~|fNu7*nLAM%bpEKPMFYjsj?#vbqy7I!@MN+w@mIX?d zbYmpeq`3({TvPhX=4S|owh>=wdN&>&1%Hak9%i5RPdm~KryEt(QvfRKPfT1hpe4{z zb!~f{FzPj}xWlUblGeZ2X{uE%JhJcM`U4HEq|L#0m4bBI1z%a9bUE?DhkR(?K_jB= z&E}S&`(%OC$-ccjU{Xr~7W$WKq;jN{>g1a<%^4I2+ZRS0OV$r*XyN!$-vR{!+(Dq1vq8!O zy^6LCS z#gUSDI}3l_yzT}XnHQ05dqgsl4F?SlV%40CX!9$z9?L*qMjs>zddoxJ9}_Ti}|B zDnzthY$GE5!Sp<(~FX zk#U~Ed(NY$le>H9aKw9b=C^9gDz>cmw-LZu&S<5T1jtXe{8gdda_2jmpdzYPZ#vRb z`Y-36B6pWuG2uFe_}pCbVH95*Ie4oFXb)Et$@Il5#i3E=XZe8^=; zfegH$4U0l30CF1wMvL4dD9wP`B!z7?12`OiQf<6i-c!Lqpxy3>TxJTQ_j-(AO0M`w z`PyM#X=ws>+PLH9q^or=;fhoIU0y_ZH|8IO`-lezEKf-)vD1{mYrfw}eF8Bp$wgk- z*&>YCZs{Huif;)=op5L{{E;;qSfPFvKOWRGIqV%tv&&yZyqmPzx>$t4J$h_iz^i1o zbfw+oSUZsB)%=lS_a{Nq^z(r;bZdhTRM}Q?rF3!H%d_(AGp^9hr3uyG_@#MNb$*3#YB0)psA?4DfKRZU$ z#DGl0F0XxLlxlB%B~m##ur8iSI0&x=Acc7KMAypB_u6;g?z0EX_pzQ5lP_3tXD${% z*~f}rX{5K*1%O1G0afo1?8|7<@podU55oE@Ls}~%K_cBk(K|^HxANpYlt{Y;a^5PW zy`p)a;Zn*h3El|^xIUab{86t5fCx2su+VEukWON$3LX8zX3<*|Pol)#;RK;m!87Gkq?3qv7gairKV9T7v24Jluux#`X#lFd$kWY zXIYLgjbTHa=iU%gz$`+WLr@v3t{3yqI#=LW_dTU+MzlC5=G>S{GA~-jQDk0xvjg&D zLJ-J$_ZKr>xH4;J_XhqFmw(xFOlQNR+7Cqb1Egz|{@(jnbbBn~vwrv7gz)Wj*9K>` zCACtOXzO@Ofn9Q9CH2CJA?Vz(i9g3w#(Dv=nKgHA9gW`Rm!AUv47tOjaEqroVvai? zQWa{zkdNJDEKtsy4Ol(7w z$-{gw-x2ytpUG{PC{)7e2u@pqu+aF1 zZ(SNynk{D`#Tdg%4d__RB2~QTKhM3!)S{`$Kqp*0~z>lV5)VJ{79k=W?xC z`Z|#ek8ijk%7Kk89HBOsjT1wtqYHz2Y;>BBmINC5AcCWhB5Gv1V}TOdLS2{%C*FS6 zd}i#o`^A8F^IFH?DaXY1)cz0xQ#5+;@^%GUc z^$)w>3%+1(dna2voJ;n=Qk*l){yz;decC`W`aM&#>f6=#XZVL}0^z@LlolGc)=scf z$r8lz(u#~5Y=1O9w2t$yLal6jR$P-aVJo|_K&PfWWda1syK`dAtU$2xPu8qX+;Df( zI6fMml6|?A`y!RQzX;O%{DMz(My1#;P96k+Te#~Au-J<1Egp$WDBxRY@AMBU)9YhbfFxIgiLE!Ewe!&U zZ^c<;OUZXYo?#0v1E6xlYRZJdtqYSXu|4uJ#4=JS)`%vx5R;Gd5X|JSB6VHB$uTHy zy+;?ya`Q6RkemsCSiGc;&1K)kMOn(uGJ(ljE%nzIuqdq%oHZ^Xij&>d)%9m(#p?pi ztpjBi*2-2FpM1B3m;A#5BPRZpg|QQ@@-S3sZS6cmgY4X}f*C8yaJ>+gh1y5wqP=2n=poIWp=wBHolKkidosF2CVY81S_9Yo88RYu zR&Eng{o{f1g4Z+l*h7)GF{2`3=+w0w!wVLgr>xSGfuBx140i{<3vr2D-2==CQDyk_ z^|h+sLHS1RY9kBYk4Pk%}(qRrg2mupn4R(;na)z z6`l_@Q(bHE5MVkRBqVWTEdlQ7wyOP+f7G-$=vt-ex_AQo{z@#p9)jz+z4)~sLF`Rp zak18`$2UGHKCSHO!Trd{4XHCxXp%PQu;`Lsp0m#sb0f#)b^7k8ZTq-@kxj8K!l(|i zbfK1`5Qd82RwyMR@fmW{Ok;ZRmtqjtgyB} zl|Fo}okNPeJJ8GOp@dNv?_If%2n{&}L8|)%*eMW_5I7JQ@>l|{d)T!E>d3_3wkNj6 z<%)sMz&Q+)D54Xk0OE@@p3*Df*bI0Ya3(uzo<_~)-qe=r)4L)v2e+|kk4zgqwbO_x zw~RSRSM1aNt#HLmL*WdN#;J8G^DO^=UaM2zQb4ZA)e&qapRPBSMTP3$sBSAOJ2?^E zjpw4851Ig#K5Z>YiIt%5cz-8j9zkD~g5Kh~0eZ^mc<0e}RZxya-+?>}=yo=U@h5~< z4~CeNEQ4ID!Wc}} zRN1MXUSW-j-x{y*vAC$XyQN0tMs(JkdPB)UO;@>`@k#eDzjUj=37h>CLQ8wJOHelL zYU+zXAjL?9I{d`j0g|?_?C@UgH})2)1K#ac^srad3aX@VmQ9plqkJAf2PZu?=~G4! zdUo!`5Gm{H=(dCKjdO4s?*JgDsTK@8_=1pmE|a$k_ZV!2hG0-BP%qVB?9SUsoDgh^ zQYq=N6QeGLs+&s=h%BTqK=RJ=e)AQjGNJazF-!E30K|z%PONQ;6zgO{lQM(GJLS_| zYJpo$J@;>AfjFD|Q_xqk#cmwYQFVCu&PcaDhj!nz|3NDL*F^7sWg1{$Y2##QY-4Yw z=V)eQ{r>^>c3lnX-~WQWsGtA<6#o_2`=2}hAM(9Uiha==EXbW7l(^2c5V@D6+{eX9 zXUy_y4CL$?-1CS=bxA^gh32m}oP}8&f8l#;`R%tSJNF9k{xUucoh#g&2|mPqzkVJ) zka>FLWj#yF*38TNb3gCO>EYqcsh%J+ke#U**;yhTOcsGnS)+|Ek9aV5gt#knQ|Hb% zuemJES}-QODpgU&p>?dczb}_AZIFmND?KaXE(`1=VAnELUe4>q?pjR9mLcw2UpgW>WW$Q(vGeNbh{)uUiPPf9!jKe1co5I^HutHc!G9ligK#JP4g$ ze=)u@OnOje^&AeT0JC)y21mcGt23;S@i`iw)n`>-`>Y-p^r5)k#;K@_& zkj?e)wYy`p-OH6%C>7$J8ID&NfLCSu9Lc`HB_buCB2b6Rx6qXT+RV6HiH28hS%v7Q zP;UCOCp^`ocrYO2>tYO^{Ko;-j@8)Q>2%5~Dn%8TAm$hk1*-$L&yb5trz;RE?FP+@ z)4odxNz`dVwnhrIWJMRRdT zWB)rr9;>#^grGEEXk+P4*`}O&=R!`Ws)1C`4QK@gU6TK+b&f-@mbT%W-c*E&JHL_q za^eF{4ZlW;{JqGq8p^4&l%WRI0$O%Ruj^O#Z14(bsu}Qp;&WN4oAamw|3|=T3Q9m2 zb19UzB}n3GG|CfFVx8zE7Ak&uvst!i-=i-LNEaQOxQ@MZkua88_Pv(Cg=0$hA69>f zUE#nxE)KxCjpQM%*^*a7X4;5dF3JdX-LzbK`5%?9VajAV3Re_{)lmDCz`$5g^Kr3& zaepm0f&LzC5PUta($%fm^dk0EfkpBkjPFKB2fX2NtfB#h>&L)5*vN|MeF!F(RNMhgd6v|oEp*Od zQI8U}FR1?v7Gop+2Aog;0I5s>0960|tTM2%wl*^OkD2@57=B!7c-U-=BK+v&=;z_* z22zAO455?)wB(X-=n+ZMkdVRn`{s@&HKK(tUeR2YhIqbAQ|U~ESdyejdZ-}z8PS+{ zbKYd-C6O6o-@s*BHOD`G^1SU`UeeC&j_8b`Bzz}#KDr*d#ak9_#$@yE6t)bc+GM+A zJ7&jPjJrsIqlyMX)E8w|!x|2BiOuWsEoc5itO@IRdJTpRjSNG>8O^o_guat-r_!Gb zcJ)0Bz!sxy0U+LQ>U>}Wz4vt$802{Xhy$t1xilr_4&$s=69YXPqXzxHCOkNMUJ?O( z66h!FEg<$0x=!634kONl97~HK2gbi2AO>IuARz-^i#F=O_hpFXTNU!fCBN{!-1n`z zOQ1M<9-}bS=fnp@ngM`hQ6)jd1^3)t0B7gVIiV->4<^W`i`6;P94|GsE#XdA29h(U zY5Le=*;Zl(GN?kr?Hg9K1Eoioy?R4>BB{oIEn zN0tmA2qziM9QVgvONpMw4XXl^s5CX^#RNcH-B@5cAPc^T0HI280$7S@wHM;VD$KFJoIPh z_gx-!0Ji<6w)jX>04(Dz7Sz@cGRn?kFR)JByKDJEA^2u4*NT8S)l`-(*#oomqEu~m z|M;TTOr9xPnQr@{!nW1&AwKAv7uDttyPB`705j0nOs0@W2_Ey^kTZCc69eY9)KE1+ z-jaS|(O$hllix=`SRz{`A(WZL+#20R+XT{`3P8oDe~Cx+4T+m(hQT`%xI zu?TWU6aOU#ANmWXVf?}&iY=^&l94&;+3{*3@TRB9H8;w>)e97{Y$tPFih`xgfx{c< zE;dX)Xhb6fv0y>?B8p>lZrxJBBmIwO;zAH5nu@$92pLcLt@?^pF6{QrEpH{=Dj18x zV;GbQ9a#VMVzf(o?DjdN0nKD)+S~JaU$me`j@0prBu|Pt-Hr1dUez3i73iO)#3182 z#M`AXRyGRgN2>6>kW_jc7BHyOxzQqFF1*?4Rj1ELoC}%m2@ZIvu1!}gC=EzvV7#l0 z`&OMnn&d>23C&__ifUtS>rq8YF?SQ`dcP}d)1cNUrrhbPSOtW0BA>K48>;1OsaiuM7M?Thw?ge*YlZ+T4;r5&vp{pHi%4<}}^v9}sOa z1Z*{(Or)M13c zutBPoO%PCkQn*uQizZOmgAAA9Wc(@zotbP@MPQF{7Es@Vs;?_?3P1LtzQw?ksLFJI z;M-oo2FrM}We(6%p50%t;f?c%CN+g{Ch_WvXm=I_WV7gau zdU)Ip!wEVrhWo6@I3Bv22;?BnpUYG&r+X9KgCgy;j&OxYP{%4u`0M~=^+_X>dv_v^ zeCBGE0tD}rN3qRLsuE>dW+rYM_V``rOT%b%jkg7q`LzohJ`D2lowmuQptud3oO%GC zXj8^AK_+54x`NV`$G@tKomVIWhHHfe1S}C$V|$nzq~VT70QsEx&g7e!?}RfEiqjVg zK_gH@GK(M4Ud&OXHv3$QsxKNh%M(inyMQO9W(0?Mh2%yois`pJn)0uB@f{{lolRm5 zwE?s-!Q;3>T#4^)B0r~GSrbiSg#SF)^r;Z1>|Ft^MBHdV_o70obK{>a+z1Bpj7ltd z3E}$R;FWa)z5WchRVqlL+8WD1L#VZ+m$0jNt#s$I1V6|i@G`1}MqCwLdL?2z5tDye z#PCgLpIKG`4?NSd@Y+oejeI07&?2^U?4~vD9$j_B<#1M~y+x z3Ogv)G~UrgvJP;PW9+7Bqxq{R>1f%5=3B)CwQSIts?_vQ=u)6lJnpS-499Um)i7|JM|qlo8JdZ zqOJ%G@{4#HmVS|{qE%P#PH?}xmNH<>QloKV7Ey=v@E9+2baqVoSJvqlkiP;l>y9fk zgbT~-`IYCm$$yf|;Z9*VBlaP#WE2TF)-*I1kG}FSG*3>V1mUW^+jqGK+7(VXL{#|& zvDoC`Ii)irALM_thp*Jd_h?(BlQ*XB?Rl%6GCOt~1@~lx&S@vVwGIf4&5k@opVuWO zH*K)Zm<@TaSc+aA##>qndH?(Y2S6YZ!Ga5XLAh}PsTzKe_w@%um421@W7mac6+2De zI+3tv;xB2IdOG*!!D_sg4pTGJ4lxO*);Xj#mY4O(}m=5`iR=D9ja?4DDLl* zapRG3(+KmfW8KXR@pRnL{`oWK35Fb9Y%tY=<&XlZ{?FTrS=Mh{|1*&4B;gIbXBFAX z)VY4hP_QWD*9D5~&xX+04z49_l?|SS8a7T;6R)Z*cus>7DsHv;>0liB7->VOcYJju z?|doG-=sPP_QVUeE2na3N-8-gT$~!@TpY7MGbyjC1WAkPoqypdKlJ$Y&xY9~rRBZZ zMNIFWyG;s}TAy1k;@f}rTD9DswbF=^ao}w^kUwkjKa<7R=1rv+uI_7Y`R=fqes``z zgb7Ym>3niSfy zNwB(+WKyKN3=}@*&@AOo84F_>$2XgmG^xE4)@qQo6lcn`f0)(gHH){`=|M*y&$L&7 zK21jTd7aYvv^f(Sa_PyJ#X9NAiYeix*n_f9t1BMmQ-bHRKd;NB%Cj=>z~cil?>1fB z2YYzWS8Ci``cFAB?!y_A3DhHe-HXtv8sjP_TAqDCoS$I-IT!~vR&NmeJ>Hg+0|1cy z_rbV>qn?4qe;0?VS=kV>!hcWK)W#}PA)+g11Uo(Wp~bwpG{L)K8Ut ze!Ls4STr;Q!#9s8g~_u> zCpS)-)uf@KyLeg_7yGMTYy0KCCHP6!kA)0khuVOD`&WjAsByXhx<)P2GzXFKGsd_+G)dPf@L*kSA&{hh1{rJ&*O^c*I8CGS2_)r>_3N~Py z(Tzg_{f(dYn{ND=CK|y{YU>Typ|#@v3Ym5D9Y9Wj0>j|UT2`*?+xSIm=ubI~Ved@< zmV7uy-wnKCcJ2L!bOM;VuMCX<;Kmy9R|lpDnW^Sn6nFw0i`PLYa(GTz+JueRUKWFZ zj^Emuy%MIeS&|4yj6prab?=?^IxvNFP}|{4mu&W*G~8b* zJoeY3aL+J?3*W?7j0G~t29{|lZ`>T9bk%bt#-T0oXUrI5u5_U5hajW>t6!Lww8sHg zlW2>7wT~5;gM)UguSdzr?iicYJiNZ?Ze6dFo$&nAzKzpfZW;lj< zveIJ?A?CGRaHvr~d;}W*NhzBrL>nR5^Mr_be`o=q9_iOqKqInYL+WLiqxsoYkh?mj z>>9!v!!&@v>WfNnK|*f?GttFfN;1AyY3nFu10@fQrAC`NdPV6ZPc%_6V_z*{dyK?^ z?D)h8*${fb{qv}=7+R;r0ALO6PpqL%RIf4>f7*p&*PA0aP&vQ=&UG;}gUx%KROeO-P9_v!$gD7NmKGhy|g9&D=dO@#b zL2WlxJX{5_M98QO0KI*imC|u8BD+kGL)6BhSo?z0X@!DnFwfS*DrAZs7Kds=j8Wa= zw4WOA%p7cHtMPC2?7^8Dcz&8&jVoBSBsC2S=tbR+OJ{(-sMf?0O%(nFc%XcmF5Xe% zH0Lf#4}Q^1dUIJ*+jqakTFSTT*FXg7eFan$bd#fJ=}Lf1#YkRAEgnS|)br z`2iY2kP%%MJ;+5%K{VSr5HN)8)SNQQWs{BmMs0ob%!K$}M==|nmY@AB7bxVpm*`F# z*@3&+a=xWXt9E4;wXH=6r8^8wJnFB)krlhvb?=Dcv;giOFwm!FHPLH(&>R>8)e59I z#RRer-E?rbo&%*3@PK|x67!GD6k`|+k6Diqmo@P5>2x;Q&ebutCYOPIJ)5^;lD*CA zCVzfy@m&7U!%_9&wr_=a)rijF2MvjbHwp9{hXfMS(i<4+E=opmL zIPDcL=4thr<}k~oxd^}~9_I-ej;V~qZ`HeiIl#)8=+7-DDPw~DCCTryiIk(@8=|HW|4 zT3tb;;w9)T#*jvp=>ok3Hp}t|677fL<`?}pAqpM`@jFiXBb?5DN3Nz-4u;T=0P=2v+F(V19h^;kalmN z|GOHE^?%(_?QN_b|38#y5Wv^&PzV43cADQ|@c-`gf2cxVssGrVvmyRu_4KcpgNh|K zdz36_k;q68cts@(fId3qWM{{ULm#pXR$6IKR6YCj)IChSNtm8xD|0BK=+8u+OpGuh zk)4#vws;&@F9S7Qvb^7&pWmR}-sGJ2p*&-H^-qqd<~G4^WK2vb*{&5?I<2wUWEP0_ zv(TK-JjG4SrPI}~7N87tv#86~zSFjCvnp-dwr$(CjY`|LZQHh;S&38K zAKr*Q5pO>yV*i9a=3INPF|GkYheZ@lpU^#GEy5`-AN<9$fjklrKo9Ebv#Lh~n`k{J zv=GSxdEiW?%Sgh1Lpa6JtOJXjAauPQ`UVWsez5QiDMnXM-si^(=P|QkNff4!(9SzX zLYXPZP@}&1hdaVK;Cl%>P1f_l|I|vB|`6G^<5b86AWiov!2!{V3rgKfP1btd_3of~hZQj|N z&5MeGL<_zkh4PT|6`ZNr*=y%<#Y zvY@k!!G{*T2X|HT#2GSp9~jc8LemLP%BOZ!s4`lop|e|OGtiDU)BT5O>GB$wTIIFZ zXx63fhFJ=yb6DPvSyg{Ku}v#OJ7sV zP*al?Ko~|l?+q^^H+4!!2`n}C!r&d7f^pg#|Fz}fNpg{-pN*1*lrIC}B&mEs0tf%z z65Vt2y!&YUd$tWc$urYKzI)8D^+G4$97$$|ZaW%EQC!z`rVcslA)BcsX(DJNAF6{a zA@stJ%i$6_7&$;KUlx!{w}f4=js!Qo3d)5@8ZOVgw0)-{-5T5h<~eSdYH~inZfEiP zcvMI7fgV+DK+|R^s832HlZ|7r&%SJ#qg*fGON&H^6r>~|G(pdeo@54Bi1LJW=OE;= zKUL|ZBwBe(k0qY97tC>Hp5g$-+^ela+E{8;U_D1^ndxpQ$a&Oyy=5P5H5K5M4<=t| zfpRQr8Czfg~3+bq25 z%7xyRH=`TJXslUUEGP7FcUDnpzm&LSvptZGuz^v3c&5kmv|3r&)D7F2XK#>WVe-aB zCDOLU(Ond1lzpH%aeRM-eKHWZ`sH!B4Z)`cJ5+Hu8f-9sZTssGv;rm>-30q;-W8oV znr#8m{)-IGh=I)t*-zv4gxb!Ma_yH|NgE;QvO>bchL{L;@Q($_xqnT>sHTM5qSWYx zUv6x6P4?Uhovb+7nbQskFGUcKmJ58y8B7WM!R_z7zkIMPm=`rP5QeZI$6;-;BWO=S z0fY?5jCZt{Po`QdNQ&){Xy+A2j;m>IS*#Xdiz#@nrklc)33&%dt06s}OeGZf6vtb` z+;;s|Bd&DF4ij(H7IX~w#Y2T(LqCT~;XkrX zsz(nsc~0?5A8m?%({i{&Kl&7L2y3CQq44uHh697E6(GkUwIQgFfDqzJS-dg>Te{C4 zwN%ywL}=}An;iL;WnTkpf=9>EctbP5oF8<~0f?A*fnuQ?jz*=x(nB6hyzLlkr;A_{ab-stR z>t0}MKQD-Xx>`3(=%kQVRn~4F$kfqdE0heYaMURl8FLel3tF$uYbs4*=Mz0#04FKt ztDWN;&Rrp`BIlGU15+8b=({=$p~(HS27npR=c9)rnOk_ugwQtl>n`N({$R8i<1I^^ z5wFihAI@}o18w^Q0zF;BudvqZ!Pf#@?TK~lnUL5FtuzyV#V_+Kz@uNyQ2$LwPILt& z@r}P^W`4nV+FrGyCm8x6KwhdUMZnyzZi^Q^N;U zmy0LW`6MKwLH%p7Wa_sMd|&+>#`6-NM~hD(6q#r;`5s&Pqgc^_VY6hSW4Gw%+fsFX zD(QXZm5Y*($eC>7=P$!NdaLNnrBoj#ab-u3 zw%6x^duiNeCkcVXK{2U?FrcJoN@*Wn^8FFo^GLb)Bp%hBzx*EDwE0kdC388KIOkGF zLrU4}YAA&MSVG)zeaLs7w7ZOJIEM$gWs8N43qpPD8J)00pqOhT^+}-Gmn2mO-akS$;tJex#Fc(J#W$1 zNXO7PB1y(qemJxNxC2l%h+OFh?QDfIlNnpv5YJcx>XE!2kzW1v|HD=bELE>CIcF zP|moDm4sj?3b}kCz5#Z1AHQReirKrxT>n}i3yW*<<_UC#V($?9U?3Z4Vg|ae{e{myjN3C7oFFu~lIZczW zIx}^@89DHB8&=H0XRGnOc-p_eA5Jm_V$RPFKEzz#BV*`&js3E;vtQQU_-acI*I@~D zWY>>AFRXuuwkAa-HD)c254&BFHFdRhnVKa%j z^iZ%(U6@YnV+KBidCRk9`eQr zK_K7+_ncw3?Ejhu1Jam2c_DZI?w9DJR4~poFfVG#|1kkq(oaCQ*o|AX0mMnn#{tVj zE8G)W=>{T-jgP$!a_5HscAHF|MOlx{@w8So&TkEEAJA*dI+@o2R#Ixg3*hv?&^6n{ zWnPX>>o+%_wdHs4bPIb_1aKb4B^hU|>`5<;Uo>K5Z{TwU)GV<0DQccDeB=V)|Tk6;+9A9CBTW{?n6OdpJ52BI5IKtwP& zpgUm1?0GPpa|9Lo`<90`V4V{pRfiZs0KaJi$Z)@8#6bLXj)}yBxjF$eJJ7b*W(K0| zw$YwyWTrdu#`TDRS77WVFapRYF zXy;)J5eR?w@74|BHxxCMI>dzyDic1~FZ`}J1X_ZPpTue-_-O`10F`D-7f;s!6Ym)h zY`7K495Km=$mggB*UxW}0NvAQ&|!cyvES``UkjXFB&}N((}$=_at#nl4ar*bf&`%- zmFU0;jzE8~NGiVNmkZJgH>$@0hdS1QM8QAJ8V#q>YsktPr@F5e(5;LUV}|iokC3Bl zG%&yPkokb-@OJ#gNf1UpPYdG7dRX+EfG)-Ro#l!_dDfuVbCj!u5sy-+PJVKsh;pTVgN zrR>yQYgk%@)2}y{%22TYcBtU%xmA8(2M8QfusP>BPKzaz0uVrN@8d^YN%b#i5@}5ZJyT45*;@gzhkgN+pmx!@c@ut7w9}BJ6K-_ zI`Sw{g5|H4g!hW@QlF|wtRd1ZIFIkIG1Cg?_8cD|8nBx(DAAU5zrtIN_QYS#utgS& zIP-j~vlQT>ae;Qaa05ll8oYLlA$g|qWHxJx>ftWEAD@wYGBDdb3xGE zgv`28jl|d5+Z>kq;Kx-gz_qVKE?UDW=e^sX{ovcRBXT3vB|sYTklFlc(#ht%fC+s$ z0oG+h5_vE}5*d>!3;v)Q6QRv6I$s*RNCvjO$HD+_sXiN-(*2x}BbK9-j%gSMnh}R^ zCgq}Xd61`cHPm#kO6jVCm;kUMGkHT7EORdF1RJ?0)<{`p2WMuLlJ3YrOft3I6bHr6 z=umi(ugqBEnxSQ-bEh#$xNc0_h@FT`S&J+2>bLY6>1#sjq-XkLVrk$s<@mUsFy(j& z2ZLD|#obNoyfFBH%avb&n{X7#3qkWi;CTLXFW`VOff)U&+rg^E;tFFG>tYM`Xu{SG ztt-`LQ5wm!mHDzw4~puR#x>!lVmfWcZQl&2l@}eDCFmA@XA{=0srOCKGNaxLUS5@0 zj@w-_1>K~tmWYd0CB?5Qgr%jpJPp%ovh4#eK1_6`DGaZ`{i{PgG?3v}Uq?AgCS z35SF|oxK)YbA&UwAmOt>!Skm-m4JgYr?Wdh0?x4p@NUJ!n$4)`#%s^(eOLc(8T0s> zZpw_qPtqhzFk28v9Wn3rDkxS`8k0+m%p#Q^&Td%hQerPYAFy ziHe+OGh}xS<(%GzvH;oN;+=3M>2r?H`$}Uhx06R#&*>*)a!7)##l|F;z3ui!_jc6P zg^aH-Xo4T4f{tC{(5&e@tRPb3k^_0McIuC8wsAG1!n*qZb`tA&Y5Ke?jjU~3M zX@^54G3m?`M2)!>hc2a{BRT#mV+E8*$iaO}?t#&}@qhn}a9eWwc;b0*W8)ss+3CB0 zKi^0?k>_5?rEVJ-X9M~P;}rP+9-qL}-l-i?;$G5mZ&1BEym-%i>AGRff={=j!VcTc z=*{nK(${LGMASQxCSJr8{F>`fbhxH-aj|h&S*`h=ku{s39O+8sLcEg)f(s2NI5 zr(`-H%SDu>c9ScBT-z|$;Tu+X!&GYA2E^u<5rdHCOX^@vhEBJ#fkxUb7kbo9zvXB- zDDNrn%0xk+D{dtUQK~pzl+pdxZGu5|Tcfj2k_LB#tmXg$vbDle<}zH%eH-b_co~6| z-b?AAcxp10Gtb<&h=69juF&!r*GZo&#AS9@h_xyHKL zl)H+{RRc0Sr*U^oK8LB9kwq2bQ$Sfun=+7-@fkW_5jF2U;(cP(saA>QlWov!cpJ%XC+3qQmk(I*b>t=0u=G za$Ew;r{2LoW}+bNSS4?f=^~2Qbq;dpQ<7{@9@UoIGjT`JOdyypn%SX6B1JMxX8lx`DqGj$CR8VdtChkU~HT~nrPUsitf2ukjmoHPe(YjNt9m-xIvB7rSH0E zGCLUa8ms@jz?JhmHzi2#7{vZPXpZh2v)T9ioj_i}N;OU;wm`$9;>(!zM3kf>5`ibC zELr7<%k;!RQ`-6>kH@Fje|=oNkZq4uMy*k>H`I*`FsvsAS?1^WGJ=CHM8XtSr8e=w ztqYh^6l#V}#FV8~a}=2~b2;I`P_U(*fq*$Sp@XOpo*D=w;LYmp6Kefb4esKuaDWg9 z<^HBL66k|B?50(m#ZRb}Gb;ljwN>T=Q1Ppul`&bpHFT2sQ+TS9^{}TLt*hGwQXJn3fuC5t)O%$B>2R2kjwhsHB93S zjE;sjQ7izTrdf0FlpNd1A=K7C;rhh*f|%RUtSz#OC6WXSG)S<*gdySc^)0#X-2d_wp=C`?{3OCagd zLZ#;dD1qoK3KpRAQg&hZ(~&><_SVKwTQdorK76}F3iB2O&7F4ex!uiq#kxn3zT)>> z`}f6&?a&3jtW;A`5Zb4wL4IO25?l3Fz+Z~we?W$a=4F6|lHno3DpYD|5G4 z)_(D@Bw2SP1Y#;k48FI%xt1J4LJeNyu^uEGaB~R#_9d@Nd%1ETSn7GsPd9($2nIC0 zaq{qim%~#J94iOKoTn9!mFm0vfzATgFlrj;i;b}cL7{?XfhQ^dm zSb@hwt{OvxQS2SNp2RVsJGy{<&`E)ao2mMGxPG|gAPRdNIoM=PLm-gD4N*#*RD$jM zY@cJ>MY)+!3$<%kK&$L#HV1vVrG>s1u3U@5FPz0O1+ATnhRB|VnES^j#9BTSD2!iF zUbhg~swej>c+Fo^OB*%cnAfhvOgl!{qn#f69*a`hj^1{r>~~8MbYHCL3n%g%vpt>l zyjQm7TJBtIUM>tKy-6rmUj>rmfN~U4le=k>(t1#v8RXR%8oITUtxh})_G~`=p2!IF z8sK`?&U7_z`8Q^hOlGofRJ30{g^oH|RNKjKh!~aEKuQwRT0YeR);=;Xnr!9CTD5|H zkU}1)Ktp1X&!USF+ulNvns)d8bFl?< zVER1`VNRKB2J*hRVtC4l)1~oje$68l$pNZM%X(4dX^n8ag}>T0^-3+9JWlsa?S{b0 zxkeklFKMS=W_n-kep`O7TPOXHQwc}#q!`pVuSPb{=%Zq@z1EFM5xdRJ#GUCP1B0iy~OE_f+tQ5Q!HCq;n(~Lt+@h ztu0-PKvq#kvIV?~j_|&|eu*1Fj^4;IoPNb}t_Wlr`bv38kn9vV^BE&!cmPIWoSL}xCG<8V!j z3_IS?FjQ;lqk>KTB}t)+gUI)bC;$t^FOaCt56r%ZVWqMe|J&162ocohe6F}dkBTNj z&jumkiK8)*)h4p5P@NM)UFmM)lxZ1#`4cJ(9j8Si;Zj<8S-vIvg!1xdjW+moG@uD0 zGAiE-&*S`vx(Cnp_P7|Yl_n%7b^j3#?N!d{A$gZ2HsctO$>3QrsrFR`RO%kIJeza< zp22X@w_cO~RiPELgCcJk6^=}35S!IS`KfDruQ^)vJ7RCVwZ|rZgMZn&?E(igWu;dY zzdL?BURTo&fC{Yy`}j914j5+`DGr8XWvgRp{aKO?B!3gsKjWX z@i2=DoQcUU572ix2(C;>UZGlf!a}u$sz@>A_k)0f97W~a4;I)tFDXwrZ~Ulx%Np}f zTW*CRl?SkSgG@;(7~126S|mb|!wu9+?@J=xn!98Nz^o-1e1Pk>MEwE!DwN|Cs znC>tI10_a9h24xNo_}|76>($cA8>LC=Qt|(RvaVnKUr;kS-+UUzd$aow3 z7l^|q=}3+~1RZ4gSXT)wYD!v%DW|lZBW!{Q z;j%U$=Ze^9mU75(O&{X$l>1is>+gQzcY97pib=ne)n^kRdCiq$BvAh+#K>N?|tJbRF^Tg@Sf*Zkh z-|vZ9?FHX8k&I9BdB5GgfZ!X)DeXo%J(mcK3)2V3NH3kV3=41iMzVzlY)xDHnzYm` zG(UHHcdid5oKSX(6@CW6b;!_EC-1|jEbpGdioG|w(+imEqg~(#_=zoHE(?6{r?y30yUKo-!b+ZCe$#RH{3{k-IzR zkkY)P2SY33ALg~2hW~7Fdr>F0sKj=}aV?CMAL8%4KPFun6l%xdibcVu<)1}E$0Ls# zOp_lxAV`cM98Zw)w-Y#j4yqLYwfe3R9>3e5^Q=yz`l7>vg?8kyV?*dWD7vUW#y9sm zB8zF-m(#Yk-QnqGtBY)<8y0MEx;0y(&0r@|Nu%LsTTUR%>sAlpRCu*MH#|dVD;Bs* zK*rem@sd{}(DdE*W{58!wbQK9+{i?owd9Itf_V`_tku^GnxIdpnr&duvl7}nz%I^Q z2F0NNM(S6&DV6@9_6_dI9d}K;^E0AI3JMZ}$>)99RjBU)>{81DQ3*r7Lz1uTjwp({ zyp9mg4;z9q4Oy&-l}R?sCPkTrA+zE}!Kpa8gCwWWpXxISz9I}=W#Pg345>8MCf&0?(<71%&H|Jhxp+)k@_tu)`Yp;s!-lYk!w z8|m#hen(|fv(k#15IW`Ds5Sxu3P7f3U5xuasANEU+ES?uvkdydXv_bp`wB-Fvi{{K z9Ytb&&u3!=sOX4s!!%FB;6AbJ-Tj^c*f>eiaU!F@Md%=rnabE`lVXpQQ$5Njzf&Yi zrusZk{SCz_!H(Nx)7g6qcY0+J+irGn2aV~S^su0yaV5yM?O?ilgu2*01$EJI^t!}f z=G#lKq$tRpRrQ-#zaT$0#c$Z*OQ7V-b)?>eI#PJ;S`mDId)%_M|I~k_0QR_&u%|#3 zGWv4LIjU)29cN$sau;27%9RcMkp%d4KOQ&NB>$y!p7t-h5O)ZBgG}f5Up$-+T}0q@-raVN_M!|dz3LseoSICl4_~jpB*U|Enijjx z&U%~qH9Rgmac*pEe-|AMufECKrRgoIuLQVCr};M&N~<<{7Is#G)NCZV{!#Ss{yAY^ zsmbih@O3SeR}A{BeX1dNuFAocKnQ>?xez%T9~qie{9Bc|%&>KW{v}Rv5iIYT5hA&^ zX~fiel+t@CXPGco70Ogxfa7>$?rAkboe8=bht zA)zFX$X7$mm)DeiFJug!3R7CSHGu@HWLv^bL}mK@cjWX|nt=BCgluU$J%5=ft1~K1 z9Z11$x-qg}7GY4In>)8F7zb;Y;lul=1DmMjVk_6OKe*=-Vf?=$r#av;lbA6v1$roQ zvgE1R6NU;rzsDt#n^K2^(jPf=>X0v^lJP~FX<5;Lh%|J06zU06`Q{lCW|aDHzBIP zn`-)}&u5NHPO)MoHSLVKC^)942XSJ&3kpWmy6ej1Me`gWsQZ^`H_%1}S_h#fmf*^+ zF&|QB4yBt&S1D#TqIDwJlA`rZLZWCm0~d^v#dYSHz0FW79QD6fjnPDyMCvzu@O@EA z<{&2(z~a#w*GmcsodGk66!ja=$Z_ZI&td?HtJ_I_+nUd4fVarAjPMOmNHjFM3$~7A zMSN_Q{pu0;K&;6rS))twQBBpdvQA{?(Y48ZK3Su$!ukvwtYp}`>?9@f7L_Lb`M-+e z%{b+P1^)RXBar^1LC(V1#Maru+2cR>Bd^tL>^4~uzi;#$oD|kZP&>843}7zcBk9-% ztjFVzOu@i0qlMRTjTaEhP1kF86B?@1*ljlSLBOXNaTVNNx3-QpTKIIwLdj^&TC6i!7>!Duadj1+(O2S}_fWbf<@ zl2d5U4+S!pq6$Obryk*01|ajl@myKZ&+~6l53nEN5Qd`~;cBYcAc;pRZzp6IcmfMc zA~6Mq$A`DADb$27;47snK>0KmoA1IQuxoWTjiCeA-8sc-R!nhI}lz&UfIQ~6gHFC5P7&rnG%n#ZW%;&ch zkZx}o@)jkAkZ6=HWQ^WD(jGQNW1rKMe8&b6<5_f!o^cEN`WMuefta= zk?ftnC?9hc&uCk89u1_BJ|$<6K_XcqY&{@hiAb653mHhCRCgA6QOQRjZYN70qT{&} zLlGA2k)UhT9wkC^1sOEF60yy0VZu)p(B?w?XgnK%$F&zy2mkhRyO7Se3TwO3v_BMG z{iX(9g0yD%0DQw*--3Kzcq+kU@1_QzgoYnM?9rz>t@gLpuw~Si)V#vi%q={Uq<6xw z39gN?(Ie>Bpjftrph(v^Sm?Do$)w}>)ajpy;u8%aE2MI_c zZ8&vn_~&GY9Eu1%Mc?*$#KC|<*IvJGu{WuuivZN%MGf_){~)6;rcc?p8T2pduH8B_ z1HF|IV6wQ`5)0&IS&epyxa0IDjXm?=y$`(>0MgLO(S<24aoC(`>p<9+aHoL%3@k$9 zVVdsYU^P8uBMPx>(ScQ-JkdB+_?1R;NHR;Xr!OxCWy7wWGx90b7tV*UX6WD66rwjB zq>G(VvUN*t7`X8IwnBO1Yv&}heAfkrIZUsDkU48m?+Gc5A*1TM#-yeAp{ho)Rz*zgWtNtt zQy2JL2+IYcNU@8wJN~YihL?6|SnLc@Y_D2WolGwlg&HwBlliSc+Nf>sQiG3@WPaPw zZ=Do1I{gt8sv(PZBkppUlx~`+o1?@kk(*e|&5?E?mWcKQRo*e)E-Y;UD@>Q&q4DM^ z*tSPYz79R>vzf`YAqOd$*eF96V4KiFX=;;qh;on3J0IVo#r?~!e=w1 zwn4L0CcOWbkdQ`7x5^`%yD?iibHxQ2b79!U^2IWCzL9Y8!!JrhYWK)%Y58aR@pTee zPqHl$C^v^-oMF#F@#oNAo>X3U59}gviG20LX>`wII6jqnIqgZ8-{wR)S;MK3BbMDC zN8bNH0GlRy1OY-{k3xQTc_bTO?!*!i&$*pdI@A#r-E6&ipu7h*@=hyXwr>?tTqd+n z%Pw0wB8^eG9lc#&e_kz3cP%?ht-t)&v|~HNqI^|8iPjT(K`Go6!*~X8-o$_W>VupHbPsYnk{0L3ciLihv)A^k`7 z{~OXu3Pb(4n{=W1keO)zHKaemHO^r=h~vpPSE&D$*bYzoe&rMzOSxvlt0R&Ud;A;F zgI@bE*v9XL(YLFa8~YsHH?MxZ?%RILrTv;Ck>u-z@l7k7y;siz(vf9rN5mJV4?U}% z^PU1uL~Qkn4ynjPtAdfoB-@jO%fuIRlF$O{kpnJ-w<2NAq5>b{?eWmi>xvK#*rl(i z0IFaT+R9ac;<}_A2o)i|LY^}E2oNIG#DF1*BA$6mTO>+;S@kOL$4bFM{Y*^0I4IOG z)g9<{tU1o5i>i5|paoV~^g@7qIx5m9U_zEKg3ZAs1PE3r(f}reeQg!qrh!uT-p?R# zGRg$G(j}}wT?YUSo*FU|a1bP4R2AXwnL$dH7_4CUW@lLA!~ENt%FL3*U$P_m`I>VR zR}{2CHALD#9kZGe#AD=Smw2gb@Su{`AT7~)(m6~6dl)yXgE{R?h*;eeVE|OsR4CKp zP5WbjEpZh?2d~`4pD?xHKu(w+cvpbr_qZ=*jCHsJmIryv{9$-!F$xJf2YH!B&2Po8R&PT2VE3wx{#{U-sTXE)jpP&r9iX|jc3A(*e3uwHt_IKD zP%rM^ls14JY2iAbo6A|dUB`(tjem_!=VHwPoE^p(xdPc8y6tr~QhD*F%lIHn!UI2k zcE5ZdrRlPwa6v9teXcwQ>1M_5J?@3EiXCe_B;M$DD(7MBl*%!Hb@9C?GIImC^YQC zDBG-x-_I5!A2_aN&0#9Dt%3d+pR3ZQ#3P6+EWP9Ot5AI6hQmdJnq2{EP0->7wq0CdZmRnGi!m*47Ni#0Q zErylWW6|0|c)2LBF!-59a+cb!l!S8mJig55XM!AKNec@c%@=rcx2i>~ZEMY5;ex&6 zFn6CnS-O14-Q74nt?tH7bP>@Ch)))*Jd$l@KEGRq)Wg zkzwPl+_UMl46=*gz-oTPT&*UHoc# z{0f*PUn+@m)JwIP}M6J*;z z-98W`PyXR;7MiskK2h=ySiKuHRa$<|_amOKqk88WLf2|5UJGMT{N)`Du6COEFf{lu z%TJIk$4hzd-;1G;jcb?0=8LSlgMIS=gHWhdtt=tYde;j_Px&rrnHSQw)oD zJt(PFjsPTG&q_)OwRo*$f*j_~HZJ$xUSwd>HCgrS`QkDmUO`t5bfMhvHZXpi2n{xG{KOtcSOT{A)OJ{RnQTK+k@5vh4S59~7Wf<@`O<^p zVcSeZ(>c@luR%Iv79V7U2Jg!pFMPxS>^V)9fR}urF0=m$F*Uz(1ygtKQ%iM*@BU8I zZAYJj}m z5$kr4GaEc)n%xpbn2#bSmRvPD(prOego8wPj(oP~9!zUaL|fknX`X~ZZM}t*@PVB! zmr_eHn|D5!TSs{4d*i5zvM>2~Xf-RCiTn7Rd!63>gP+N{L_-2f?-9Al1f5jQI2$d; zcKYERdeVIag^y8}SET941lm+HkiA4EJI%XC<{+DO(>nON{xT@y_U_^`EqgB~h95je+0I3S*KQBg zy6^Hg`j1m*O78qj*!TL^G{o8w-~T+_=?S+$r~MP;^FKk7{_ldcGqn7NbNUZ@^iejj zD`ZFg%+b4Fs*o%&8NZO_w5Yt4U?!?=1|=rkyetj(hw33?Z}@Y4yt|8Op4sFUhB)@p zmEke_(pgfzMUb+l8Y=pp?)(05+&cT_T^z#bA%c)$*)SzW6)o~NyZ zOpPnQIUWw`pX0^bsi=Aadk`i}a+qU0@qt=N>n1)SnMG{FsThfnjT|M$yd^n+Dhwft z1ueXZc3P^0eGkk;b5h$_zew;T49`y5w91gmvc|%gI`*xyD6Sp5RS;~}5*hA|y_D z4EcqnjJSpii{7L&=3zvVFe^;G&{abh<^RKa$`(gALVayx0T5F&i9Io<1Sdlem$*3gn_`EEG*XR-v5mCa|<)(~2_C4tW)OWh3<+hVW%(a_EK zr)B;tvF0~v{l3qaI}rI27uTe;2EoQ2GoBVSfvfq2C?!@MFkR_M zm9A)H0&>$lcn$Wr{3C`>Mwk3(j)z_&K*7;GOrYwV7Ex6CjSu_%4M=aH2=-#15loor z&VW~)tzo2poE9dGCIz9V=hLRfoR@nkRg~bRH+G0cbbip#!4vA&5b`b&gK0!{KX)v8 zY&D`~KzdoRfbp3dcgsK;4p2B=E%MLw+|(;@-MS*lC+-bAc<}(#wm6B-SLNRz0*R}u zo<%laT-XO}w~T@p^H@fv9z0k`xqd0l+Ig5MPo9BoO1ur+2ZAm3C1WmMjEPVXsjndM z*Pp!?CfCVQkpls~65~QO9y_1We8_?Lt;M zTHPq>$v$mStbNPoR|lrlg*KBoa4G9Ll)3W=6dv!d!dc^YO%qnrKOkJ6YHk>^AA*Ko z&*G$__ts%MOSv&OZob^SD;M2|In6LGyN7=pHqGu3{-@~Iu&CEU{|P=D<3Eeu(c<40 z{tu$RR=2S`5dAN31m-a+nF8LJj;xAEk~n<&5H+STS6YrT6jo!fjZJN5r)=!^$4pJk zuaTlHs#0I`YH@@046_`M_4Qx3d7}$xrE*eGrTbXkZEbJ&pM%4HVK5c2==bUOBoM{b zB{N*P{*-#Jn|_urdbSq+{T=|&zyz76Oi25;A}0G@b)?x- zs{b-*!inubFtU(!V8N6Ad`KXqL4+Ctl5PB8tVuvJ<`1Ai#3AOI;}c}SP%ZHcKopd` zVf>Kj3jid8nE_hz2-u>Awmc*xd+IdM=eRDx(=%zD%p^kKOhWvCXY?-lcH<0C+2A8) zWlC`95Cv8-{O|_!aC2!G8fgAeVahe>Jc7tdpkG;wAs=#0P8O_DW$M_Br3bkAbA7Z) zZA43=;>g7j-_Dp20tMpv@<+8kJN8SzClo6LX^d`(cWP(C5tE$JjhK%J}AVpqmvcs)uh4Smy(gVBp_JE-k&<>V+V*7&0Ej zh2@2wct+A>#07#ur$I7#ziE!u3v!7cjZrd6Kn*cYyGO_^f+)%cy@!oY;F)GNvZUt3&PGPo-mOp}ye*L0wic*{0 zk~y;qpezg}q7*E%c@wL&)J!u~I1sl?MY1|q71Ll6`1bdy3>7tUgI?R=IL^`>EbPZmrexYS-*iKcj_jSEzhecmd z+tB&N%Ay;$Jk2sjlM&8$BkgBU8GQ@N(E|siam*~>FUmXekVUZYfr@P<9sOoN3qe=7 z>(~Jtzgmr=;(n~?M0ZEnbpH6)wFqq%0=twk5qSHWZ^@>dM+Y{{iF|T3AsP!9!&*fz zn(#bQ@F88I<3|Z(zSdN=)xNmspO zg_$5;^-D*^UI%TRO@OI8>h5APwOD#mGKQFYD@u-b^(H???C>-ct{ut-z~JzBVd4+N zM*wo4hlnKol6*3$aGY|ds-7Wk*tvBtXx>B8%s@2>PblBh7@_W{n*?YlsGl?bg_OVKbgvEoaO;r6tX6)^PFW zWU%pluhad3es(s2`>U#l@8ehH&VGqyU&r@3v0jAt`~9JbkQGfsnj^W!7%}Rw>f4V? zH8(9uMyLDjUzD$og7*+kwaZTI-TS+XzO?ir+X`Sol=lic1*Q`pHmhlgSvm~X6|eZ) z?#v#|-;2x(M&UIEBrE(njB& zK9rg_XsnBTOvEoTEL_KrqDnFvLzD`>J7`EMSo4HS(oRfx%d98=%yM^fU7P`OL7!yJbY{9hv^nGff z)7#kJ--D*jW4o4nRb@3%=O^yI%TyR>$DxJ2)z>!VUNRrgE%F3Httj$XLRL6vYdq71 zwg(lBY`&W$7rXy+$MpfvJp}$c*iu6N$1hDM3o~02N2mX=5Ch$B6%Tg+RoJmY>SyadiOb5*X=s1$SYu0lHEql6png)!wwRw5R|;TkMdavaUI! zgN%SV&bLV}ITbQvDy+hb5c}1>rxeaQ{LwV47RlS8-}TWe-!3L*uDo&%iV3NuM$J6+ z`lfNbbk+7Ow-hT^^)hp-$rR+YIaxvYH3-|IFgpNqi6B%+>v+@N7SU82#2YF-tS}^s z`-K8VoWJ`io#DzIJ-kThfLXoJa~>Bt$`ip6!kl9gN1ETKs7`)w##*e@Mod*x3+!&Z z&aG?Wj|COcKoM@nUqJCTRA2#kn*5z_5h?BfY+VnFHLQa!Gq9X2g7AMRd#4!DqIO%b zcGJxADgU@B`}IgEwoDg zOcSdgRRU`(kT`~&dYBVDO`pO8LT{uIG)ga|4=9yjY+Vm20XU&jAEgvQTmp_$XP%-@ z)}E)pIy~J9_Q*sFPoAAf{|GY%sX}Zo7Z1?}X8{+ul@lkd^a**AzLM<*B$mPpaeck36Txz$E0j9_eo8@p{vcV;Mm(dkvRb-f z6Xb(5q~s2Cn8;2&N1^&z{=PZFV5gPD`KVtQxp{C3KJJB@ehuwo|G|+#;In#pYMs?{ z^bxFw`zqF4S`P;;#y7#v$d;OaKtF{iQ!lbdC-%}K|LvR8S|~$n_Xs9-eV+y04fKCs z@c;MlaP0rx%hk!mz|qM3|3_Yn>?qe(*YDUs2KzsVHU9gr{~rTe{wgc{QdIChCu&ir zLT8jRHQvJe1i`bKXoyKcS1SLItPn$AX$MDHMJ{W4FvxWDnzwTavp)4XXf9>ZSE%I6&czoQ}9fdWI&j$7n6^;5MJr&Dzi_<{MsJ!Gb z^O(CTBv@lr^t_~?GC&k0nBu_*??Op|SW$Gr+Y&}amI9ci!Kj%?)+}~a2FoYVH`@7%x1h$6UVyxMeG2QEIMn6_l_?-~Zs3#|LgAcoM@X}y*_RdY z{@0pf>w$f|;uT=BO zC|_+xJxF($f+>>^E%-40CbiOuHP)!p`sl`PxE{B|_bbN9K!o%7(>ny|LhwBUxzsuC z@ZU|+1Jpt##30S;qSS-rk^kl_hxgIkh9}v_2n?Z8h-`9N6L>J7m`CN1uCh5iY0a64q8qO#o71gWfO%j614MP?gB2Oq(A<+!7KGZRc;C+UG5KU?X zklOqi-6z^MqLjB_3qfSJI|mQ zY*F69(PDL543fdSYIHoE(bA@|p39pCSap7$8ul+eg2fnCURlr$ztUiBfCUZNnM5fS zp-oB(8F5YLSZQ@RL?GA7A zZF|7JM&_DNbR|?}Z|Wxeh(ESnq~pPCt@QlsgX=`OcF{tMPX83G!fPAZVwvLm()NgH zQn{E;Rd`WXV#4)$yWer>I?1?^=Htka0?W7B{a43%!y9^q1R_mp-^bpKIOn-K;7L(9 zk&&u8QjQY%^FIQ;2#*P~;eP>z$1kJ(zm;lbf~u3!vAPXC*ipK^X6ufR=;*DOIA&VablYL z!N|-M?;cjP(+y$Fqc9KcX>VZd1I|9X)Ep>RFg`RtAX;%~_tDuV1s3gMq0lvSag+Kp z@uTPFE zL{1VrnuuG)`y#D{*Ak;%S>6%_Fx}bWVg@VcFHyg7m1KOOK}nNg%-8~J zPrdjY7NXu=lD6a7>L*Ut4Y8$BoSHlxY|&i9RJ*gkXIGE-nULskTlk3r0YS{Ju&)jCN?C9N`Gb5b6g^_k&r$YCGSq_Zt z?hH9dYTsi9KY4g5(-M6t#~m~*NvkI;x!}b{ddt&Eztc#VXu9ZAGdoTZB3&R+udjsY z2@ImI(sS}VPZEER*rn$nBuF>vZP0E(VZKRB`A`DYH981&_Is874^>^DiPrfkxuk<) zZ&Wbh`lP+ELA*zkX^s0imJ!G}?h+?ajzNVE86%WKh-K=wYHR7Syou@HL)Mwa{6YJT zycpGxHAo}~EJu_2W9^w5JC0k3k{$aVM>FTdK>%6X{KPo?7m>n5(c}n2I0`M+g*1if6$G9Dr3!b*XQ+e(UV+IHVSf$qxs>fq zsSENv-g}y=&8j7*|JL2y34w1&qHw{F$ZfY~{+_<@y&lV;kgRtpHqoXJMiaPzuZ3=g zN|7wlnh2k$jO!8_)?_U}-Z+g*?qd4>xHnh(V?nDLU4xyj zM3bWlgWvzH$p+#dSVoM$;ubNdtuNnpurmioS6Tm|Lg6`%>@*Frv5c;n=o?8<&Is0{k2(v zbJ=r*5wU}{gyiPy!b(5$lYcUz+F2})@X>0R*)y(mm~j7%_TCH~-_dY~9nmDLivNF+$O*%p(mj{U{$SG>Oh2bnm+g^Uum+pdVe!_jLd> z{hlj*g{hi6j@)wDU9!ru%Z0IMV?$eIv=3?$@B#u$8ZH0nd)j*$oW&J!PhW2nB7+i8 zBOjl%GB}JPMp1WIaVHmM0d!%s1P%LO+K*Jbki*Zvne%nt7&M(>8tb3SZsZ)LAc84P zlLwYLu5;8)3T*4|qt*E$XI;~5M=c%6Mj3y!y5(CUZQ9_ zbCMY-23Q*2J$!zh9+z zqS%sN3)Ku}j4jBaKMg z2GDt5Yqd#Go+&TpjN~t?l-*4PuOfvn;AA8 zoy}6UaSdBo^FQKMMTu{#M?qJod#MH!@4LN+?2jDne7T?8`+~E7n`k2OYU&9gh4Epg z0d_JvQfPIu%-}rLpMokZ9WS~sEc^KQ;fSm2pUpq5 zL?j$n%tvn0TLyJ@)GlbdXqSzvo2ia%^OxitrZuv<-G-@(WK_>MbWYPOS{7Xxoo;o^ zv0nw+-R!RgR_pdtz7+R9ij|($S!CX-Nz#@>uZ9FnV@>kjY3!4XtbO4TGC_z(szqRW zy&tXx+ET^F@l0i_l1YX>Gbto9P2!%`3XpvfAX2KG+7Y-5AO~+e8)1rlL59_BlP2B9 z*z_d5nEmG{_=-b%g&iol@3hGHHh`ri#$TqkP`2$CQ( zX_J3)^=-qpB)TXmrzG>8DUb%|f%1qe6nBAY+5M!ei;DxmC5brms*p$hJD*qE#g71c zKp1?C}verh(mc{Q{B!ya7=mJ`*QlVu1sbSW7iw!lK&4 z-nYutLlOP!QB~4`3j7Jq!H}l&Q3FSz}OF)eWDfn;Q>rjZ( z?qWMnjaN>r-JXa(MqbY=M?`0ZM*nn_hPZdhG7i>-ppU*V^kj*E6W8u8!Q=(5oXiNaJsdh(&y~7+cu2BQu9L zRO|rT0hIEp+q+S0cI-bjzw~UNeF9ImEa{PdH*_IL&(~jvTTfHqY3x9YY6le$5b~Hh zG1VpBdA}h;r=D&A!fbY-XlTOruee2NwQNMY#;?bQ#Qq5-Q9x2ay2z2NN$(@+KCoCZ=rWnr6VDUcQECSVPOO-^0IZA5(l32;abS+oE~XEHXkhJ~c-Zu> ztErIPOm}qJpV|zcyzse*b#vCz0EP9`QE_!EOV0}%60>bfQWYI@#Mp*5shkizmk4$*>qfF&1%%dn(r2Pf@?a3YVB*ncF zK>DzMN!?%>lnYW&B%&j%O8Pf6&AE_J>rEk~7e0vwx$8h^8Jkl$f?OcSY6A_N6MOh; z8_g}TU8&+5lmwlR!2#;2g*w&tVGrw#)$>?Zy`>N^AoT5>011t8tZb3L77I%G@zN86 zZVNWpsm0rRkNySMTeTH6;IkwwN= zuRBw4*bm5P5Tg^^ALT7uKEVE)0{M5iKyCuR!0 zsMMT^5`4uD02w4ON!4k}8SwBAJ~2dUy@hNLfz9N@tXD8+3B44;qI#18W&#YqLyp=* zq8XY=1|FI;w?77Fz7M%UnFtoQ0GZ-J7#pnv#ieVD$5ihvB0^{=Dh&(fcR@3wiBs|K z*?&!$fc`7yo>rHb8R97TT>3n}^n$e~nC3ooo1Cw#h2=P=55w(0A0k8>yzd&R4mY4E zJ`QLTp z@A}$S{_h&loIz}wI}z9OdJ0y4orOr>3oX#K^2bASM4h|K31JDGQw&c~VA_uUF;x(+ zsTLpLkW>x+Cx%Nw1QxFW31;LoFN?|Ko>ngKsROlB1WOJ=Z-6@Ene7&}H1^T^Y8>|* zLwWabY^18OVrV^72*3whHU3{F;CU3u5n&jDk>n+97t5p`R6hzr77AiRz%f)STI^j zqkk^5EM+OaaMitCrY)`=S2r8;FuLP}ogPiZiBHwe>be4(;JppvpIGzrWcATJAHPd1 zRMbURb??Y@n3Rkh)P>2=UbgWQKSwZ94x=GL*^EFpL^D?A1ttuzm_L{Hp7gb&k_LD6 zD(%aavLz=u(poM}QnSU189>FO3rIZ;9aobtEIWSSoo(Acc+Pe09kZSC3d3J~^U~da zJlzOCg{1Fq_*M=TMjjW0p}knRIDUbtnza96IelBI&P2m&(bIH>3St{G z#SqmmTDM|7^0l7TthkT{i_$VAwg9PZbh5SAriCIIbixZe7D2`2cXQ@uVEAB8=%vck zlPqcIBzLT8IfJL1Y%o$KrPfW;uDsdAht&!)W{nepORl3(Q?_&0``$Or0%{i`bZlPiPv0OB*bk(KO?{nZQx+>4~ zZ?7>dsk^ru_yISy=BDXdW zx&JGt-!0s9D~1Ca_NVSOEM*BvSFGehV~GWS@b(MFM~)dI3x$sxb8s%?y!$g%YgtKR zf*u9%Ah$XbC+KN{M70sGCAo1wuNxSXb7EgTvo5e1(Q6$h!wXd0=@reri>K91Qe_0> z@QGWQzbtC1zP#Wz+^lwC@eQ#@@?t1{%~4yGn)ApXoXjR<%a9Bv7EW^J5nZ=hk!fLm z%Dm_kRYEbg=alAnsGNJjz(X?-^grLmUJd>>vs`?o$+M~gk?vS$?Q1n)FHri}g!nhK zx9B-{o6gtSgG+x_6!f=sWx9)?mB$`pOYdPV@MsK>5Zi+vM_wi{Y!F}nAmG(6z@wLF z(YMXGN@F?DF_T2Hfqwk1XyEI5=wVCF5 zlovAYY55?gcc88!6$!IXq~Vk<5$D@X$*OX?O!NL`6%CInF75X!ky~bNvsuGOf#51C zA~S*xe4kWwqY}?Flvdbeb1h%irE?1Hf+@G2s#mxrsKapK8AoPTZhcM9+k2)dHO2h& znB*>PfdG9&Wrmq1swi|vf8D&$M1bm%koDC|2?Fy*Q zipUv46pII5eTytRhS2Q(nDeQuRZniUU|zcN&h*b$Mo6j@TO%>Y!9$eBN{Ut1w%QTN^Vei5Pgu)d$ zx^)N4hgzO0d%eSs@#nOORSC4Vn=qJ%E6wf;zzQY$R{L*#8N52f6U_dp3mizUCWlnX zLvf90+5PpQ@P>+wJoiT$~$N*4u`azi>#UUbGY z2hcuweK)bGOT(B=Fw)8hXy5i*aMO56Kf*@t<~o*tnfgvK9fY0lTs!wz_N~#TI^7|7 zmltK0fodrmZBMJpYrCQv2F+&eJ#$7>;^z*g@#f_crLPKemzgN|GI@`+<9XfV0VYF; z;|-I~(2kCw7J-@N_O}5D$YvwLCF%)PoH2@uOx<%E#7=jHFDI7u( z8mM(w6flaIo_ICL5SOxUaPTmi4Y)EmDg4GXjnbHLe?EXN3-s#HE+@d>qiHm-M6j3m zOzOCpFO!bEVeGMq;xIq3xz;#>QTnAMdY0B;0w}PJ->Ce{eEPK|0HPi^4mAW#rIVl< zf!YO>5BJEl?w>FPO=C*DgMYfX``!`gwiutB%KEIX3ESiNX(KxUbdvd*5+4vjo-D|M zHRpb!20YZ~erV?5XYIki&46Qffvaju2N`hM7S&p<@l89R&Xy_KI=nd0-eKv4bK`t_ zuy^-mjEdKO0a#4cTDvM-Ku@SP?G+8lo>@Uo{CchvAh&HA47u z)&U`wvM5sanbQqc3(tjqOe)^pNJy4x#|S=jkf< z{jy#c4!f(TvJe>#O`ftPVHG$<&My|a)Twh&Pa$;?%4;-S#kO=obBRnL(0JV$Q(6Z` z@NN~j8G62t7Nad%s{7nUMT8+sXe@lI4!rPe0hCv@umV(|Uq67?#o%e9>P$YzVFA*){(ZQ&Jp*c>>!X1$;( zx7J#T`T`JmNp@iJvdG%fNnv7_%Qhy(oHOUG7#%A|9})V2*fCh~aSQ*knDnKzwMfVx zs@^*t5~aHz&HKM5U920n#oZ3mU35o2$})kuV#y#1HV2)3*#tMOC`}KcmSQF9?lKDn zSQvdeTR33{JB?SpY?*s?JzsqSSQC4Cld)x>mr_qFLeC~6EVaJ>wG#e++y%aRyF<#T z004!7|CueZHgGp_WcUxunJXPnrw#UmpPgL5;xei=4vo651}B#CDOHN*Ec2DvtfI+F zWl9nuWMT)Qc@ zNUt&5Clucr;Tpx_=fzp`>4KZ@pr5lw1D%^wwvK4yy+WjH8@k$df+wVj2oA?z;HRR( zbLcXHaQI0{?zTy(y#zoLvN-E!ilc*#uNzM2pHJV`4(+(Gtz+%Mp~bf{IG&EPiYes;?6PnoK2(hQ@$2Wq{^! zpFiyMhu=Xa%mBiP+y`(ew;;VLp^cDoF^3c7?I!N3&h{pGzY0JzQfb(W_InjP(X{Q- zhlS6+3kC9fI~f@xj1w5uf})WUK@*-gvhOz|?%Tw|)Xa$Sxq$I3`9kns7z?!qvXg z%hv7WyU^=0sL4NCdJ5Bdr28P!=V~vZWohuS3%jKC3|wfrZTe{U^@qHO^|7pb_&$YX zZ*bDzG`VRHRQP%P5^?R_?!xQrzfyxWm+P9?UMhv*8DLrd_|pwgxVLdV z1oC&)8?F$&c7-Rd=;OWM-)cX-EM5E81S;2suFFmyGms0+BM{+*UL0}n=|~GtJLF}@ z?ao=LuXk`luC87*=xq}S5LQ(EGT(D(7wQGRc3y+-3t(%0d48M3_V$KTD*Kc*;-LRq+m{`^!EHPIvEJnrA-z#Mbjc~y zy--P62OsPY8^YW#!|~pBH*xFe;aA-LXSM z@r|!>e_{ZO8TL2x%kwifoPLDPZE{%5%Y>01Z)xIn$2Z*WH2bz%B?N$%mz%SroAYK- zhne9p5}m41`@Ym4;j#?V>rNiDJD&#@ZGH}H4%Kt?5B_j9M0fJmD7aXII1(2gUrf+y zycHt~=D!UhS?Awr8NWLEMvf|mPuYFO!r($ZcBBIaYQQ_UPPjaFpvR6xqic4-@jJ3Q z%#H;$HGjIrzZBSMBnQpyYN&H0>%@pPi-$CT7*@RIEWgzk3b*db^Lr#OGs>So^K%jb{=nCpGI*E8|291cVb zI1j|SQ8>8w5W)#}5yzACqA;y^8S~j6p)F)nu^dQ##SZP-r)?ptcGCL{)j%zAVqO$% zoZ#O87(r3%e7#zvH5qmuM7><#EBki|V!4fAsW>7mO*kStKvp7kQGMhFPfyPYzKBH- zVazWTz^Vm@IrI=ERE~d!z>ZVpUZytu)@pN_EHYl=-+Pl-t-$tFv&5gzQG#B?nh)0; z4QYq3a>(e5w&^Hllw3M6zP?HJ2tj@ZEM;8K{ zGH>Bfbj?>nD)cd%amCT}7ZL1HmX|OF3ka%@%TWGG)|bGX(N#qn_&3x;wCmYcS7BV0 zmJ*~vwM%Eb-`0&TCRfJ_)Du02w=hlyHtBe!2_Dz~><4@D4>&rgFU}rEbyXe3*Y}#f zkx$4FUjH`h%>nUnb&*3BdVCyN8ap{auN`q`jdkna!T*> z3bhe>LFm|u94Cs0<5M#~CsgtDZHPh(@UU^|jZSGu6F_`EnAAVHCj#_o6(4ns)WLGk0!u7Ike(2JhD|pRlvpmi66) zgEIu^oV%u0;WnNAsi%uC0#x;pq;3Cz&HYf z!VC`0{zwCc1aU)mq1e+>o~FlR*o$_Nh?GgC zc{6iOr>?6=B%(MV_0-t=!;g$B8tF@MmU3`$8EhQHAV+%Fq8v zR3D3(Is(bOPm8s{I*rp8>yv@ut-O686si<*$u8djWl*ybKyFgNNNOFwG$F@E8z#5#=-fqImRAD3`|v zMN4UeF>pQYEr#tI@mdEE93qqx{p+f$sA%#dh0E)+ZqJe0TT-E^=q$YN*$QV-Q3ZGp z-NXT3>USGaY8l}O#o#^9irBRE&!)KkUx6jAE!Ew_s^pTgf8~V3X6zZ zhUjUI6`x#f!I&Y>_TUfZ-v+d~;4wLjY%NM%-M*0wm*A?(g-8741l0kr*ocWXh#!u) zo^~#JTgE~WcJj8VyJ~k<$EI7;eCN%g-8vl<=Zsbn`I5S-_GZH-)WWQPy%wlZnhePC zq`%Q3i`1^sI^{?5mqFo@hrzaCZp2y`|8aFnJJ5l8;eWP5sM3T=vS5B35 zfZLp!*QglORjWeMMA*=4vh1cpsqWA03*)^`x}{o4IjxcjX|wG&G+>2z@Nr@Wny9A% z;zf`K*{kY{D-_y3|0utMo%WitDoq~Flx5#NEo+2XLVjF`F}xQz^gnt)Z8B^T{#ktl zH#hT|klNMAp?()OQZ;>`Tn6Isz7b8;ZFT)&RI%;#C~yj#)3f&@|#NV zD3~tC=M6jSgfrEW6F`-uNd{>@y(4SL8BP;3!ZMasgdBes6%h2vL4csy%M1P8Nxz5;d>=)nNXiYblz>8;=KEEHVYAjjWz2 zH78LM!wa@%0Bu}M-kBjJ%c|YKgCRT43)=3>fiY4G?&Ep?)w1LJ!bi6A(=5^CGd6AHYe~&P_Cvby}ZJHrJXaImGS&(dX!L0P8kdMzXPUTr10}@C%tZuQ^ z2Wj#duvVmCtcoiOKEOe}f}VrIUha($DN-~uUO`K59$LB-*!KsQ@}RD!*bpsGeJC#} z^Iyg&o9w85%R2Tg!RVj|}2=y}x};9XkF6S0Ln|qCq0#o-aP#&#IKBXw<02P$t zVdTEealUd=3dbI!Z1xRV3}wZYTh=?4G01T`zHGseNoC$tLP~?0t~z4Pr?#YNLurya z!!wtUTs2mCNumKHhtH;2`6uT_7OQF=9Mk1?E01Da(8Y#h6x*6}XO4!LaZMvpaJMwb zO+D_VRK5ya^e@4C><$c-jXw|YQf~9JT*g|-a}P@E#@-I>?#~avCu{I0mC;Ohzv-!l zYhyJZq*b#V)C!AdbE7%egDS&yL30FdQYlmrc877Lors2)}tITh6gR31=JQ&@{twowX1$)pz9TlLPycA#em zENy?Jydy~M?&z^p1KsoXmd#CgRV#<^5@czUB*#;jAw09njQ}FN!{s$9a)%i}w`Wjc^W2@?eDXB~K#eV7A;Zob14a`2IbE zSCc-A=-`!+QW$7feii?BDP)Nd$ywT`Pui>3vvXob)CrQ+IrZP&DF7lxWMxLVRdEOA z+4N8D#1E8eNAQ7KSoS>KZn*wgN`X#s5$uIvkU{kR*JTO7S z=Fv*{BdA-==?O}w_vYWD$vZrT5}F-*fTzf_WI%HF$e6C)#WLV#%`5`J%pAu_jf%Z+Qj06;&rdPTAsJ#r#jp{=>(W^ z1vYhQv+)9yqS4uCl{gHW_{Ttky*PDXnX_)mP{rAb|C+~PJ9~fKK=E* z&2ZbzN3rKAO1DhKXRE-6P?=F}9GH~Evu1f_db93bx@dC5<^(nkxJY zHrY`lsMQThle+&QuDmGKPN$mkCJ|%*^p>u|`@<5ULtn=7lwo+~MT5@^V-sO}cfpIo zDYUhw*CWoHicM~-TaKAt^A)K)se26 zs~$ck+v$x+h!1>j1aAt!OS^E_%n;w?y6prj<`1tn10; zEW|~wLqoH#jDwgJhh+~g;Nai3)YPKGj;tls24xpsl&e-b7rm-}C5K$IF;?no%XEz*tCzxFn})Q)6Zqn*f} zj;(8)PsC44PmRd#wXg9o^_?wNa@!)9iCQfLV69Hd_Yz+v(k64Yzrp-gpA{;>1&$DF zR)uXtq!p%>!OzZV=q6m;q(Qk5Gfxm}d~NjpJ_kfz(22h(bh|t-$G+p}o)ZJm7nwY7 z3L%!2avFabg*@Qr`KJ6s|7ZN==~x~L{ckYr$N$Nu_}>D*|DvoJ{{v+mt@6JDaputC z)HxSMB)!E68s|cQNetSTEQitt;ctS_*QV(&Yj%vtkl%}Sf`FTEcwVxcz8pl5+W08z zXkcLO3AVi6JcOJK?h|m+$}yOfF&lciy0@DL>XbJwwwJ^#wqw4CE*xlGi6GCtq!r`o zGgeKBDEpvSjEFr{M8ahjlkBIK3eo9$r@4XN%PANBWhwBRVrCK(dR@RWF3Xv z#}-p!N5u`F;-9rGh)0_Y3Q(U?(BmBC8|q4c<5IAk^dpEm@k^Q}X%Yh<)7iLmu}rN+ z;14rEOAbeG{^`j2o*q40)~5l=Wc+>>v_l{d!?R3# zt|elGCR{9c-Ytd4JzP#eGLrZkRr-tKsSEkwJ1i&Ql0nxrC~PpY-YmlGP-N!O6JAE_ z@TX4vlNm%YOP*dY$Uk-sKO<7Z)V1($==OwIUcg>^D3NaKWr>zdJN!sh~@wi1i%i~3+0WX&F zN*q%UJb>U#vL6m%3dXH$$+>GxEq~N}%f<9BqT#C$(%i zM<4#&{#o*yQ}X`2Y@JvCA%9R+hwNB!!52W?BD)VnM#|ohAZRHu30XZX;=uZ$8BD4w;x&jpM{U< z2RUvpyKO6xRLyP7momF9-~VB73c)3aM*HAQcO=(?|e5de-dPCKCUIL{L$Dg~QPdorpt6aw^NZpROGarp~iw ziFOgRDg}5{A}7tn>eXBF~V6+n6U(LVT)Fci6U^wd2pa&)Gb4jzMXXSoM{xQ2xEAt zWd0oFqxjWw!(>~?cqyGjN@5Bg_~1Lz0Gv2Q#6iodN>s!+tQ7*6!acCHb)*YsMXC3s zOYLTjLCG2qLdEx3co2|C8Gj%duzcw-r0-4vzb0XB0+BwS=eFi}3vngUd>Z`GpvWdW zwX^msgn*{aLkMK4;|N9I*uUT80|7bkZ;a7nMeJLOtpm*A$ZsWJS%Vbh0V3~&YYa+w z5}LiHMo=ci6-YL*kCc8$P!`0&2kAq@%?>>+0D0#Qpdyk3eTM%Zi5}#98+s{ECFLgJvq;bsm z!7udPFWTHsA#5+PE!(%*Zcx*E=9Jf6V0Mso3kVgB{oJmC`?%Ua$2>lqc3qy3X>4DM zNUqz?;A8u+0XPgUpJlEOZa1%npQo{6NNA#)f{(LU+i$anysZNkco}!nt$4RU0i-wC zAD*hG-2G5`oqr9&AD*{a4EKhox(;|A2df@-JBh5k7y!FwjJJ6-06Fk>IzHj&aXvfJ zJqighe3&q0y+1Ge=zKL9d>=_|<=DH~^5Hk%3c(Ns1%PM5hWwmPF*%TTU2ypYd+^9? zZDy)2Hia-nFfU~sB@ad7Z$QxieJy1m&@glBKI1WYX9IZY4pZe#gKyJ%ryiK?9O$P+ z&-pe_C_?9DK6Vz>dc3eHubQ8MsELM}`KMa4M~x5sOj1LM@k zo%&Nf#sjJrJgbYthbUS}h!X|$KUW>E%W3quLF~iiC|k$n9P87+_6v$Lahi7m`4JN` z?jjy`bOTgo;2NfSXn)wW6%#}eBj_XP;D+_c#w9L3-$ACy_L@KT%O@BX5Y+=HNTpH- z1T)F~1mHn0O2q-8F88i$AXXmvePnkhN1MY&M=#TRS@qK98}<;;Op@fx5Pq`pOCejr zaKkKP*prII8EY-0E@Q@&QUoCyNAJ}b%cT5Eg3N(nW=Sx!2nk6xdF8mH7)7mXB*4~I zCaYn{_t_>#4oMZhpQysY@R$YvEJkIBZtkxg5XeKduRS0oODxTm z;A@UBOW!ccHm4#u-KFaVX2>)EqA+csCB3gV$qBHA2ZxAle#6 z8+Bi){f{!SAfTGU3`SmaLi}je5ur_|0!A&^I8X!B?97QXbFNX}d3@|9+>*l;4<-y2 zvE9B%a6NVM9V~dNlr-0y;&fIf`<5^gX#BSVSbh|Lm~kUfL|_-DU9JPRks- zcZQ^zj7?}pnpe$s`1s?NJBS-WtxzYTng>rju8d^JQ;Y~#@`VFnyB|kDytB#%Y1s87Y(TKYlDo8zy5z5s%80(A@7Pb zq1oJHidyDI0g|@p&;(u+yQoz^VVL2ak4An4l$6Y7Y8#2BD_Q}8DL3f{^)J;i&bQ%x zEim+5b=vfZqrXH=R?1&^{}*NF6r^d?tl6?{+qP}nwryKqxy!a~v&**G)n(h5 zo|%h@IB{bBnXC8iz1oqHJ9j>7)m;f}-{(A+_r;e^|C=MCq7Dxkq$%*#D+kh1eC1DH@#Bgt0J-EvrRIsx6qh`BRkj@pIjnCf-R;k{OmcB zC&OmVDx3(9Z2$VzHNEv18tOufFeIBj4JgFChe>vG7i}eZn@ZmNS(Hb38d_E$P{Vhi zoKH+Y>egf(-NtWAs9n^%Xw$MCYsi(W>}o%wD7$cx4cXC*y^+^LYSW#3ab-8Sd@FBK zyvnhlzI+oF!VQZeqGQ)YLQ2P=D2vtk3ex;Xp%!d)z4Ft?PrA*z7 zIVh@|zxkX~FP4D_)XJ{HETGRp`=0XwX>ry397;Fjv0WOy*}vY>Be;A=4?g+$5w3E+ zDk;%Jzg$%>r;B}C+kJ7EA?mZm3ti-eXK*DIlUip$zDRfhdupyap}`>G*HTkN@j;!09)<@H)`#OX!XD{>%{%_$a>Uv8v2{h9KKAMaFW=SAz)mr*QMt!o?sF^74F4sgmyh3fqR#KkVcnJzfG61$ zm$EfDF;VOA{P*zr@KhUhgURhXs<;m~Qr|+-A%6t=q10&B=L)iqwDUC72@A|lA}nO4 zV}FAGB|fk4CyTO>F^o`h8q;nFwa>?_9e^r*?Y_bDCuhX=*dgaI%`-DL6P3S-UJ(rH zAl2=2Ji#FoRDnT^v}QZ#SJc}5pfuM$fooWle$ue1!R56GfwbgeEK(xu>MvY`_f@`9 zYy4^e-!<5D!azc^T@?ST3+fG-Y)2yC1|^+A$f4nZ2(rU0XaMoc4Z|eSYGu%avH)^o z)7<#-&G!9Brt``H{ncd2Jde@GC>&aY7XT~_%8(F+@}4s=5Hggve;^uA>PlSvdtOeL zr523To`L;ihyq~>XHe@xAH~?DuA&EPRMj=$pss`KBq#u8t0`I z$&rpD-+*Bk!4jkSU^b?o<@+=IDP@XadJ4^qRUH5R<@v3vhd?=Hj5>;=uOmjFa7AmXK1auNm z`j!|xbpZs`>UgX{JQvQ67mBNmb4Zbc;gUNbDk|xaLAV&gn*K4U{7(0kXgQa^zqIi{ zlji)KbK8wyr23CK*9k0=Khv4qkV%q4cI_Xd*G zflDiWaL5##2Ucm1DYXDlzb)Z7hX*;S{>l1t*~=Yl>IfQKvhfI02EVr4gxpNDAB_ zX|5(qEnbmIkf>t2Z9Z%Ssuuoa{B zM;qoN|Q|;0>mTZlRvp}EhwvHm?e&n(GxZVN4E}2ew*Vz?}QllqxloU^N>FQ zX|D;To2H&=u^-Lkf3;ehksmRh%)qKq>3Jk9%m29VcIbUXEvpr9(*iHDTisSG2>a$F z-wNF}X#xJirpRME+Se3s<{m;{3W{wfb||4bj>f#*Z)VZgD{JioegydWH`AA|XDOY- z_{V8{3yhW$$d%#3(`@r0^Q7*~(jJ5yp zkjQ1$O|VP(eZVF)f8){YrFLs9w|Cjfv7AHlroXJ;us@5W0=w2Yum=J3NimPkPcB=wySIGS>#MQO)b4exrQY<0 z3El3i)l|+wFYG$H@T}oL3jO*GAKbzM%Z(mwC(54_Piwt7*q3=UjJ5tfDUD-6lR1kl zBr;<EHO$6y9mi@N}#^6@?;m}LUiRE+TG2m4zZmlh9BXG2;zWr;axfq+nbsOjO zJrpf#E&TGU_`ZkO6Q)|&8-3=)=L+op6(Lq>pMFI;84AUu$e=wcs9%2iAgM)pWFaB} z#c%~@;*nvmSo5J%xaaiZR%Q`A5qk&^@bl`CBuH zVnco@_*8I4w_Jb;JMU@?m2SHn`5kjXBgM{w(X>xzyTT$9I?<)(2x$1Tmf|!$F+dSy zuDh&_k@#3MOvO_O{=8eHgWc|Gyi;6sa(R3Ec&zgJZ}9T-F7)WE3%kpF#F`-AqbBqv zQODbkrtvDDc=_*b_a8NdQg>4yb$(N09ju=NDZZ?J5qa<}NWb6mOw4FQrx=o3Bx_1k zdq!GEr9Ed|CBh27sM>F8S5=;=%VjUAd+ciN#YAk(e}oK0zH^|lhLMf)+pDLRaOKtr zW3*ftnN#~eE&HMGsbKz#+qj=SvvyG8!amNem3trlH-rDIa7i@0EJmRCVM1 z)OU)#?%hY#y%JHm1l8Ala20C;pMz(2b@6!}y(dsWv1^yj1Z7HwrC2g^ZG_WW0dK5S zgVd0EKAVLH@IxM1R9a@Pb&gQcEOAclWi?NK7NbUb8>TX5{4%FmGlb{-8QC8iMdX@d zC_>y5E|<-sh*TiQc$P~=S*dH6bxTldbWLZLq|w}sbm)$uz0N9M`iT2Fx}v`BovU8y z#leH2#*FV2mM9SfQer$+wH2n*@o=B8xSnK1FrSqhhT~=`@_WU)8Mw=9PW%=N{)Q0> z>W68ns)Rb{QP#hXrSFB191^Z{(nw6HU+cSYC8cA6hMWJHK>reATOjnS08+o18!tOj z(dg6r{DB3bNt;hdqZwI)%|}2h2yjYFG>Qy0E0cxwZi`h;m7ZaorKozYr>w z%kP{$Xc(3!P*ZHRjBHyvM?Xgv_^+`7sjFHhlYa&rbvPg(hW{`Wb#-)iF*P@Kb91pa zad$KS4;^Z%hOOe36x!EpeMeChh`LmF#d)Mu>MEH+BL^clM4L{nQ3K_Kn|%U1YeZAR zPv1%W!+?;dZWtJA-^;e^od0^gRY5gmv4*XrzgEq$D5@hmNpZCrHXY`j;`iw z`jaXAz-S=bs|bj6f>P!adE+PaU~MQqAmYLObKY|cTTUg>rS3pBhxZFNYh6~BV2x?5 z)W}*s^>U(ogby)t_jMTpH&ya&9 zv83n*F?U8+=*$K{8K7V#bOFT(hmFV%yW7Vb+o!{6Hd4Otlr5&p1j&>DLa-tjmS*d~ zf@GdyJb1H14*6%uS+e4ZMyAdQTPleh?Wn1c#Rf+)NNIt(1ozy?88ey(9aJ|f8VLIe)E`QgC7=0b#365j*DXb-@UNULkX))D zNy4rgq0Ld~Ye4A?SRNbibHn4}X>{mAW!%1*AYVhn4;jI+rQuvB%=kh=TE{z!TiYg2 z4%l1lOMN)-T@xY&ARr=s?R74A()|6_wozppir^T21k{pd;mVUW=uOH6j0{ZiU3i}a z!Q82%yASTTy(7raArtzRIdvbUF{In5ch^#yD=tSwove-RVf1x7wXh_?{=gt+)3wvSfN zsLsC4ET@eQZRI{l$xTzB^@dhhpa!?q)vCW{Sycm#sI^>ytmNHk!io?f@wBpmY?ewDx=Bn_TqSXuRUeoi+bFK zEf960DitTzo9DX_TDReGGHvrPt{b44Qcd7*`S%j2$EFtJJ)TFfPJ!W{(?mm$cs}m8 znb@6`(2ZcvPeWkZdCifTIeDxTQNRK8&0L=;ZdP`1vSIcbi_N`==I1!Z>D6lG&xy(* z4Q|nM@l1p6yObpU_wj{W<&b=$)72XjvieDfb0~_Ne;gvSr@wy~V)D_Q0l&brAMe{- zbHd}A;FaZC?G*@U;787c<4r&yFk1E>@M+4cEiJF)eA?nW;sXMYVckFZO`MB=mQKlw za64A@=VVSw-Vje6mgo~mjF{PaCyMJQ9nJR~XKwHXB{g#Ud3zak*w-1o@Y{Q0l``UF z<1uiGXOzq7{e$_(NXuofa|L?a@O^fMdA{?r-p~*bzW-vU zuZ(2LP+OE&@iC%{4>j4m#myeDU3D8PJr*V|1H#t_z5a!-(nWjQKxK zZ(pTem`8s!dZ3%gpj;Yf9==v$Fmb<@WSeJVnk{QvwVo;DIU6srx<94OUWEJbqoh%M z^ijXX>Dp9chwNz_N++3#x5~8>C9M|$wnU@iSi@uf22{*~ReIOU(x-cYf7oqnxKudC1__lVOvC76yPn+%h4RQOI$TfmEPutQ`8liBjtxL=&au3;{xwY_T9hXw__V9%4k!bgRmA zsi!BpD93NxHQfD1Vzk)9Br-sXFuY_4%6;-NIT_>4RAf}3axi~QzF9FDs?#zsy?tFz&FP_fcJb`UFhrK9GEpOj;q@{`O2@oSp z&)7wv?AQ~q>m4tTFAPD+dJ0vPb+bETA_GN?_8>X~&|9hP1dEu7ry(~s)EC6&kM1`t zv2CEotHb9fxQx-8(RNtGj61p5z(GWytbTm?@>r)wAO?$r6|9t41Nku=y7i8K#-1C9 z1B*w8$B2qd0Pl4EUCTrY6>!drq$I72Snt*86LgZoERpd3$|6GZ<-AW%Bc+KV zVV@e~iSZ)a?N=Z>NObQL;#m*Nw*&{qj^H&0DV-OH*aZtC*Zj$vFr`gn>fGwRlNgI; z3Q2DFn4%4>5q@D~08uFng~-d#&+i2%MZ-x8juvt81i-ak0E5rd=DCe-o_t)8FtVI| zkZk^nG>P`-q8}C%%VK>%D>$4Yykg8Wm$nI1+D|Ah{mP{IxDtfXmihpFZFzg|HY3Nh&g4AB8gN6G%!ulAUcuWe7iU8Nob z6DYR1cuh-?Ko3&2X+d4QB?sF6Me4k#X)F4gu&p`jX%5L6bau7&Q0S54m#&1znAY-dYuy3 z&TB?2`ew`qI_E0gEe@*Iui^u^b9*#fpD(Xh33v8B{%&z5Z!uE|qs;%%C8QvhOeRm5 zrYQG8B(p1iz;#Csusl$@2D&{sxNL!TuOT*Lla&2Td2v9heji5Ufjqmp@atqyVq#ia zzCgLU{;WU-3XI~2Ng92{kvowGeBRB0_SR}1l*`_IY;4TDKAjqO-PTKfKV@L_<;ejQ zklH1GUwzTXB?V9$ez>6%r|wPkec{ug`#4q9mB>I3EGSt zQ-18U9hJ5Zes6ShaQ*dnspSgDYB9VRSM6}}t}20$Yp{7zz= zClL)Z^ya`0tyVp}weqZpf-cHwFI3>4O3x4xvG6A*fvBF9iTJf}$fQuH7GxyD+M8TL zDT@t3(Y${o4b+jK96-kz(I)yY?Khn;djIGuqq?DT7fuCviItTE&a2$6yO@;G@RF!P zvh|??^5>k>z`K80G3@Qm&h7Jcw$=G5FePwP4BAeFgNI^z_Ju9VbC z}l zV1HB2RCH?s8u(L<{5aPEC%*Ucz<**v6*;)Ti--xvPFZ$G^p&+1O}#0xN%R_0Zc|l^ zP5KeA-q6Fd4PTYn-I+z!k_EdKd;!2hQtOE|&Eh&T3@k~kdEz7EES^>t#Ewo-LY1?c zsfiH`8G~lgHt=8_0BAkPbw?PvvE*z8LGL01bi|G)vtQZ5Ub*MtR(d30WCY!p>n}^r zQNx=_d;PdZ15CW;VcCSQ)GKiIeTQ-_3i3v};#L$`An6DEj0aoDcP!2uCX=`1#PeBS zpto{1^vM`1*V3al#8(8jTGn8;aB(ZMG_;Z}hbTF!sg?W?S*eR&dgy00MyC-!*&rb7 zLsU(KSPcyKzF4>po_S|UA~R`Xi?H!7S%zN8S(11yt_R;5g(C?-Z}ls4@ys<-DM~={ z1|@b@mujUA8#;E!!MJ@&=pKXXH`8f{)NH7eZg?Q2OtZsKwGu{A3w|1r+x1|uNog{t z1_>@IDFcYolNffT)~t86ADAGp_!$bE>EV5pl`}E?c?j`KPYMqY$)FBKs#0~tU{w>3 zBut7TdlB)-pgi0K{gQvjP*$doHiU2mTnA66zs`S~#P(%8Zvw0oEb)d4ScGHOg zhF3ajS&h*nZV{>`EC7OwPU#1Q0en)Ur~c<=q=7?(E4eQlxNkV!)PG28B6OH5!5cB@# zAKNpiocOq`oe{`293Dz5Gt!(GH<6nU9pT6L51jSHL2iPxAV~$56*$pZlRJG<$d}`o z1-HvRzUdBPqVLp!4V4evDLc!^>S5V{jUn7M*zJL3xr@rCaz5QKFrxh5WYsbYeEBJ;{Hx=MyRK=E*mzWkk4u(&S?)K zQhQT#AgN%3Mtn?^jRpB+|1R(drNf(zw6DaKRTo(G?x9D_-?*Y8hk=Xsg@xP*)uxCl zukdG?Ex`_qGmwR?(31p!N|=_<6w@@1_hvzj8baBl>miMQ_XbIpJ*XD8oe8Ta5<&66 zFwYNYHYr2E*(Yc)Y$^@kP*J-$>?A8082sKm%TUO#gb|RG9{&ZIk`f{bdMtF}WDWeW zyzNk|SN!wf3jZsSm+w|`C@4b6lJmVX4sEX2_KYO(prr$Y-CO~e+_=bv7Z-6TCE|b$0p*I&L+4aqCIHAEI+cv5gX>9ha-H- z=U$<$`%Hf6wi)-=cjePtvN=n1vVfvV3g)F;hA5B~^q4Q)fk;(AG1C+En#cPJg~07G zz~)6^bX6i%pCVDg71$2qaHwNQ>E00Jj|h8HxQ$q+$`oRLzA-Bl;g5KM~mCHa!yX}>OKfOnbYR;CLxb(3MrDwx0 zKa7_S)*yS4 zxd0x2@vTP?!jz6h2LD#OiyZ~R_JZVvrWxxl6Ex%yR_P7@j3zg^rqz~u*C(qNrc}#{*~Yv_O{TqY}hO7 z9ns%DW}@z1P;p>&>%NX+(0$_d;e8xq&C077jKn36h_cy9w0YcIr9fVckab&v8l&kh z!9AB5#eY{6+&thy4$pRkjUH|tCxMIg60ZNmoqmorg1t|HjfylB^&37D^&_*q~KD8alZb&>bs(+p zURECwTq{-NAT6_Jbx$xTUN%X;Y+| znhr8^crim!xs+1vRPkLVovyb+9Fsd%wYy@>V8J_NP|uZ64bgc^=_6gZGVLJ29cGmu1rA7C*1gqXfqWP;;gtaDb`}VZ3&C$2(KwV4HQbDv}>!* zkUw5+D6SN437@{7b0I0@-sb%PE$UC3l)#V%2KM%lj>X)VbcRdutlErl#>J39SH(j^ zg1(|B`->UVdmqs61?*nuKit#8EiF5v0f;M=&uR8-P-9)-CDiodIA zeH^>W=nG8wDB44`16S2w@DGFqq&mGx@_QAlO&IS4Hi+9gd1Wr zDYktuh?hU>!iVp#l=>`sBdR*#M=ujE?Lm)3ZN7F}fmzL&dLv5m^)@*)1ca?WV%r9~a^?u} zP99Rf98}~*BE>Z!cQs1gaRhw?35C2nSF)Xey9AYv_s3+W?ZaWS3$bLw;(!h0-3@`0XdXc&SZ%!H=uVPyX?AKN?@R}PFb zO12<(;fgoD5wFLkHMGNj9&k_KgYW{}liF0zv2csi?&OxYY@H}LpVD3qY0zK{0yIQP zw{8@kI%Ap61PD3X5EgQCpt4=0Si9)bTHEhs3}wq_U!(G4^8XA`a26Tvx*HPM#-8Es zV^XNgbofeXP87R~mC2@@g2BfpI;5>({JhgLy%k?v-1%Mq@O|$qIs|;FXG~RW4lMb8T(_wi*8;H?gqEAIgj`_rn(1r2=aERDXS=d& zY)Z2KB6_Rk#HV3mtxIch0)cnR4bXT^ak>$nktgvr+=Xtfw!>~P>q{GVU!gGLt; zq6RHXGaW-=5q->{-rfoYvfNACfgq6F$COT|#`+qxBda?j2>RI0>(-c4ik=KvMOEuo zAUuDXTIJi?-j~g|P%ezPoTpjO2MJmhR^629=^l}b3=A-GG5Goc!xQA<6{wt4T>JS% zQTHL&zuxal*5916w1;I4c&OL#!}5dicgZ__2*^`8X&#tMq&F1_UIT7_-GTQ_7nYUygkxLPhaSvdQeT>W!-J^kNy{w%z!ty7u%&LcaaE|`m( zR{P203J2E@EnM0A^Db;FC1i18Kk3Tx1gsyYIm@hXyW?HG93$W^1|K8ScI(INc_O!W z+_dNDP{&XXJWhXLVP3cSp>uXxqbQtwFvR)F2!qMm_Q)-lr9NTZRf&#jd{Z<{gdI@Y zqV<2d?!wE$me1E4U1U2Q`{$>YT=T#XFFMj770EdcKuejizQ^GS*g{+JzOB+q7xZwR z0Bj=MBlJ$IEJm@4eC`(ag=u=C>{ zp=-}Vf=+TBFWC|=78fijcTNI#asu|jFgTt2ooXaL;5iZZAM8;Br*KUoi6*At2Dhzd zn@zW({UA|t6?mNe>?9$MQw0_8VI2+SQKm2Dk?uke?$sL)i>iGx_4;S@cP~~sg@l=_ zl^{37RS?#+ofKM(8uyH%3`4CIb9pu#jsthejP*U-`p`_T1iFyJ`hW-ZkukEm8~qj% z5J+sZH=?Y1s)|-Qff`dkN6R_$wizr!voLv6o6lnqye0dW?tJm5A}EOidOqDmmJzb| z_#3}J>N%ekRMoqKf%-o5Jtbv?6kIp(Sp4DLUB&AQ%P6xK@Z;r1(}}jGE#4X{r(N z`_SE-5%-1Z`p}!OGetV)*sI(ZIeOC=WN^(tUOa@=luwFTpq9(3wPM(~rw*XT{8-;? z25xc1-`;%D&Z2-3m?!ygF;y4Om1;0U=OQl3i@xRIz6SWCm-_j~?Rh^tX?OMZ4KNxAmrq5rqeN@Uo@lK#i{cp=s2G${h>`9KDsZ+K zSTQL{i={j(H6iQ>DNL3ux9aAjpxjTS#VC|dqo-}M-*n>8QhgHTNAiK5w=+Akaxk}3w3^{lJYx7pCnu2orAA99vNz$b1|{;FjlAdl-`|)T(lMK z@{K6m*vWIHMsxZPwr9kn2*tSi@#~!HuGx!Py5pN7W7r1JZl>ERaePpF z2i8e{k8-LJ-UQ+LDeZvKbZN!1Ba#7p8mwMOaX8UXaB!1>f6x@Pf71y^xv*MTz-<$I zC*(atn$N+!?-wqkH;eCAp$&4fY0}qK3yBH`23oTakJQ|?vEkR0vzJzk-!b@7ADzJa zb?5R@p8SH?jkw3eJa7xDq}~gBFe1ICBZ)i)XMoaJ1m*&1Fl}Xt%lK*rmNOi3vKrvqx$+6#rKPtlYLu@6bdxOCHT!x(A7~6Tc3E+#Fo|edT>RSfDaIgyJz3 zf0bI}K-jrIOt&JLD+%X%upO)qvUlMdUgvXgGTo?qgedw4r8z3K827SZpE z_~{gOAs|)8j%Y~+qI*x{I^O!VbrcGI)Y}#+*%(l`me(vNkaJcE>x6y)s)z!O3k#rY zqF;Y$J0dO-`!H-Vj;f$3-UWh8wFIYrv(jf^vp3TgXJQXGVvbs!u+z@4=uMWcZu*@u zMI{D0QuUgAD`v8go(hTp3ea+)zM*75jK&gI!Wo^19?a%xDkx0u6*P4p;)EQBP%tg$ z(J;Lt_M&a22OM(xxDw^6cQA8-ZoyGX7!~qFNf8*Y^E~btE3W<+b$_H=l_j4rNkpDt z{=);t3`UmkWH=>JzTe#*oLg&-H>+&GoOAI(N6%cqZLx_An=jtmeZQd_RrV zUwi&NP0ciX0~>Itn?|~|SuvoXVLR-73r^$TV!xN{t4HNkZU@9?HqK3*dA6S zm8N(%hjhrO98n5UY|Z6GznFj9IL%n|alQ#}NJq0pNEl?)nn*B(3&rdT8w(1g^NQEI z=@lU6MD$J7d(aF3t%gac#zr|Stcz4m@Z2Wq?%nYb8agjus{>+))i|%O4K?gz{rbvu zkauJ4Pp)mC*=wULvUgZHOq)@ENa*p1oY)mEK`>dTn+@G?=TtA~xBr4^4N zb@LN+6I$29^;w^0?1L%*kHr%5Z>$fJH|D-i zdz-;^voZ0elk?b#7G;Gs$eeyHHJ)(n!nei`=Ab?-u-j#aT^0P4-q#o3usZ7plsl7H zIvRen^ZI(epP$AdH@LN%N!3sob$#EOT@Va{0&}H)k8VJ6C@7&H*cx6>PhgDPTwlU_ zFk)af!{gv}Z-5fHgNN+LX)!GL%WudXjs=fbxtsza1p-*zH#HY4-lgY-iJ z9_9W_6=nGU++FoQ8^-JP{uchUJ|e_Pjl)>s5b9lu3v_C^%V=9I$K!$@8v>7*Nb1WnwiwNd(f@x;G4Dzmy|KjX|5vZS(b621Y}6JTNzBiZa6$3*<5X`MH-{<-LF z-Tea(CP^{X)0Gz6?5U7GEWGK|WgE0cq=YH+HxihfLUOe%Xv{MI5k|Ke7GfL(t3m$6Fm-btg zs-t`ghI#Q|^bw2A@&P}Vilt&b-r=vjcCmHiLJpL#n|ko3Jm33D>TkQscEcR+qvv>^ zcwq-(C%9F0D0Xv;s!jXD2LwxtzP8T0Mr_7bQ6*02Ce_1&eyQIhLx0>x*P`W2?0ciH z*%cXNhcC1njh2LKF$vAw(VwU_<6@k>8QnpMf*XNB*bMb7i`-N;f#%DlB~RJ*g+}0c z3SXM=F61$Dv)~@E-K-;0IVC;EGWH7n{)k5~p!SQLS0yO5#vee!2v3^R2V57#Zwn?F z??8>-;F0Az-ihOYJPUR9GaVu%VoVV%A)*FBl!Pjx3lN(YvJP4)ff>*OC2yVZ#Y1{0 zeN1Glx6*wa>=mOke}X`fC|13Fwn_fpKkW)uyH!Koe?poNCht&nh7KN;()NQhwr4HH zUMX0V`qrUNiXzenmQ=4bTYK$T6tN>}ol=%Vl>I|1^F$A!DHV5ix#8=zLZO`Ts>=+3 z3uL7kV9FTvC5H}ER)!!BoWhKWJlxW|V?J)ac%R4*!9UHC+LcNo#4#SB5at8jv8qEY z@ojuq_d?_x^L%6RbTtCma?`DJ366zwZ200-j1qTS9qifm^rOa>qJl$#F47a44g z-*&V~Y>MneBfEU&U1%Y18BQ;tMz3lQ`+b!yBKqvN>+8H;)YOG&A5MK1RhSlL~D@B$0!CN+}dVQ)wW`H%h6pdQpqdAs%%PaHoMp`LX;zyQi2bW z3k_x2^e9%ta%&9=;%*=>xS78KoJ=%3Vd5hsGoGe1NhA2}L)3UAjf;&IQ|@zG@hlT| zTIfIad@O9gE)PiN06$sj48U*J)wEu5KE{(wh;0gbEpz7+@tIcS7vog~mQ-OD%VLKQ zYvR|V&URxu?fjOTNzP$_2Fg&M1ZmUOr^r6$xIqsy$fo(MI{Spy`9tY=|F0WID-4>l zD*g1A?EAukp}Hau<_1qg!`QX88b5IE|3lmk+8a%fj>E08zXXw9D1hHFU%yX6^DJa= zR=XJnIb?Dgx&j0Jm1VyxPJ}_(H zO~-$2;o_3jD)XJTap#Muh7dT1+f6Z19=G&b#trfWw>BR6Qv2*3hUCBi_{J2k5e6Y3 zsEWFI_2R@up%d(mUkt*jH@OrKprm6bpy@9ZhutoM#d`^`nvNHn?}1#5=|guRlDBw8 zl?*{`f2)~^*XPS~BJ0n-RsMse7XLPyJ1v`STyIv`!0Du=djKYA5}uI1!*SsisDvTf z&_|0{_|3))X-Yb}OR)A-<+TV{&O`OyC#a@3Pp@w3njA2YOjh5q9rq~XE?jF8iAqfI~NY8^n>;l$MbK{$gwE$n?qv;*qQXCt7_O=bmD^ia$_*y@k~?63qw>g1o~pOL787dlbN)PV8L*`52>`#Lfq zhivJ0Z*c#DZ_ySj#>)GtqNY%jj4x3N6w2hOb5KDcG@kDvAg=)g)vWgN7oXR8Dhd$4 zVFS*Rt9;M4^I$M)Q-86mu6gK;n;yCMi99`|Y%obW$<|(>g4{uCy5(v==eSKF)P8n8 z=u(d3eKAep`kZpOg%e_R7q;mXSLdDyjYP%Uyj%6JnwG?I1jA$ z75pr6XU#ZO_Qn5RF^~1%uZ#ZAJ^%j?y?m-fLT@Ap5RmFWwD7$T+PYI&TJzG)A&RS+AnH|-kQ}8|$z-ep>VIN+(XSU?4mW#B z;`!E_uWRnH?ZssR!`uze;Oi{k_xIOV`k^&XoveAc$CB*=wE~8)zX^9bK1Ct$`A9FTp+){@qL2&lk4QqJArs48x|#V$YuF*uiAoy-yl@QP@4 zC*w9~l~=Bk^t3#*JBaWBu6Va3rF;3uI&gVgD106S7fDx7m2^KODNWM_hm1hk3|odV zx%&hkpC&zFDB9k$@*RXlcF#juG2LH-0z(H)@oHsde>^{(kQ?{aQXhPV$S9Q;WYXqL zMlIP)(9e}6aFrP+-Q`acYbp#Ya!_dvPRpq*uZ0-m2q;8VM7D+Os19vo2l$G+KLp- z0;6j7crr>52W*}~u;r_qtulu4TML~HfF+b!`zL!hD%x_mq{0NkRhywX1F301E|V#P z?ll}0_4^aa3h>l*a1&yYb5t-=!TjCG-*-4;s3Z#t_J1rK()-KFMgPwC%Ri(U+5hE? z+p}1i+c}xL{3m!&xcZjEmIRXjiKbj?q!ci1Da2+0L~KY|MFzlu=&U0Eh04X+Uep<| zV~z*>^_r(G(`H9TWgN4ao5^Zf=ObEZgQz2LNe3T;tlty(6Pb{J_$bYuxUv_GDP7*A#t!EL7V}+1lD^XZ9e(oe1-4 zrr!;ksGRCQ-5TU0-N{^)$c zMNqm^Ih%wrYNO2*jaXWAE%PtyB3+o~>;3o}hygpom^YLwqgv5deKLCQH~5;z*8iaF z9l}J5f-K#%ebcsW+qP}nwr$%s?oHdaZQI#d)uXPjUsb>98Jy8Ran_26{V&bT9Jvza z9v~>V)|`;u`g{afEyCQ0d1GN6rbrM!v zY?X?5?vn>R1@&%&C&?x03e~vmYeANh!_~ke1&LLi<3u}!GqjE8R0X_)Gbg6VF$?z<0q0_Q6)?e zifQ1y>-|epmn=l)ME#7d1xwMs`&{0Ui3+*7oeN?~CHxwsNVE$N>^)Z9(1#6+&QI}` zdhJ@q%#kQ|VCz;#+BkiOWX4TG$$l*CpbPhD$fzc-JX42O5XLD0>{(9nM)uAeaL5|F zq$kQ4enY5Jlg@0CTV$F-6V_0(O@&8LaDWoFKJRO6u(oi*f(r=gzzdJX0m5i|+(GEP z(mul1QEMbk0X_%IyG;hb*)5qvTR^;By2<^e#Eh^ilhoZ)qIzPl#_p(v(K7r9rRhFu z4I1xh1V_NEl%Rkk#&ZYu0Xk?v6yBZEIi>;&b|kS&l& z6f@D(ARL=&MW8@-57D?$MsJ z<5BZuFm9hwR#pf`(aX_F`*R+xUj{3|+KrLhD-bf=r)nYnXkqhGnIP#T%< zZ*C8PcXPw@`FtUpoRGy(zeu|5#JT}vZLCURkYw|JR&;gr3p5(Rq6&czla-tWCzssD zDS=l7wv(M5pVklD)~dx^!VVx(3_zh+PK7Xj?CZXlHEaj4jiyBw6Gr%ZYPF_rTd^;o zGT*zHiu6FXmd@1S5)yofQVw=UoULUZjB8eBw*h7Gfy842O6N(*TP5$TUvzCj#u@XXrz_0AB ze{M*=BvsQ2(cC<(>Ra>tT}))PT4;wwUHvb0L4!x_aZh8talvg+ZMb5IUd2|3Z7`wD z@wSd(ETVT^j})18`?ILlA-8=2mIWOe&tK^=g+bleuOy&^`rlkT4KeQMAq#nTjGAlh zxjd8^Fr+GkM&k@VPzwm^Ael^W(aJg!7#GaxS3?-q z=&cD}Ds?0rMhXN@|4%VTmR%8k$Q0`qu|_yCDg;iGdxl|&JSP9=JS@ZDYneM@)2Y9G?)nSgg#pI|hi}C;Jv5Q-Ddqjk6oc-Cq05w8DOHzUlC^ zube(j%z-%e>}dljTZZZq3&vaQY9eg}+uUc(?CB;IcnghU=YePL|T7w6CIZ7+X5!4f*2`l@n;u@q~vqFI+H zDl;NNWs6h3GWTnyQv4JAE9<_ zbmJcc^|>4mz}^8byxzBaadm3XbZ&ZkV2TzFBk1loF^&kzqpGYxbe5~}SxF1C_xo5a zXVt!k-u_!~YF%4brS0!W|{Z^)g{~$d6cdz_^v>!t%3U>SK2)-w3(WgRZlrnWb z!utdvvzll~$-!4DWlAf=p|7;R21Mktt_OpRrvAK3`R^74MdO#xjyF>-3>v-aMo{X- zcz=F-;J+PPI|EW1g+AaF*b`JrtG0oxESGD-NUC@hJ&|L|`;!FO_@{lyC;ObVtV$QREUeI4tK*6C4 z6;~a!lcQJFJP4$vS(~*{16ZsCClnBIdF%-rrV9 zY%Qmcd$bB1OJy9TD@4lnpVsocGH&v`b`dg|`ZYB!Iw!fXko4pU*o76y@*frAcQ+0^ z9!?mMbwvvrsFowgxtEmnY_s4Y%`4}ifDoe;sHhXeZyG2mE)cznOzI^|xg%O-CCTwA zm@Sgy@Lj1VN^L_MFXr*WlV#|w(P#I$t%M48(gtLmuX2L&?N4GR8Q$n2+jw^jtrDGC zJf>%@KXC-cW6XX~^9br-56o}lqN`hcTAtt`w`%K zcn{2zjLMRV`b{2#f}19HzY;?I6X0d^KZ+Tv5y$v(1oT?OZ!zo8NRP#Hz}RdGhZ?>& zgD<6yQ@|)C$L{48cVg8Q7JzyX;goAckdc<>G`(XU65peacETe%zzx=O2oPK|LJ*A%-DTE|Mhr zLY75a)3UIhD`^}@8_KC0!mQI~r2=&ng7rTLaz(#>^z2fs&JRiNTdD_tef;ow@kLmE zLh?ImJf6_(1Z>D?xyW<1?>n5mjmg{ujv=v@?m2lCvVqrP7^GWUwG~qTR19?wYZlXC zr?$kV9k83eo~!hGe7%}fN(&51qqXjMwDLuGpU!DR+?^rv^T%iSyP09t7t__<^@x2N zB*uKCIyM|jyOXk`!RuCW-&MfywS)O;z}7F*l2*uC6zBb(;H8seJl`i>{CcLUC}f6w~ud3PLr?T+xx=sC8OKd?CU1<=RDMq?ACBeiYpE5 z>LO(Rc6TDp@0m`k_HEeNC@Itsx`1yF&y&84IU8IGpstY{ za}?NSj)6fFMiO@iADx`0(c~_Z&||EWXI%f=N7hA2yO}t)NCP(%!Z6{IQ$DITR{J`yOE356s9G%in0#6tfy1=|b+4KATsjK6PWbNxBN$q3uGUYXKkcUr0 z<8atFE-_j#B}tvSpB!28n1_%na@6QwHh0Fvr2-gPoxdAVlS=xD0g_Vsf}*Kocj<@p zpd=V74I>4GMtl$QT353;3GqHl3FCx8{A3Em0ev?aATeIToq7w1d@l83(kO8#IPpFA z9Y?%>;2@nUqpDzo*dUqHd3j0j*I98h$Cag605W0-!!5Cm?wnG2szd>KQpdPpo&q(@ zeGwr)Hvbw-c)yCer-A>ZG)}!ptcQIyYE2@(_N`t8SsLmC`D7X@sKGZLe_vmxynDg} zwBE3F#9ODRuTz9K5Q*@o=d??-CybXQM++1`sm}Vx0;5KYS+FrpBz34!MNJ^c*aGhl zna_}%`zv7tdoj7kzVn9!b})C_r#9U_GSMKnN3y43DfhlGf$1OUI&n=p=Qm0{C}~65 ziBAUs`)&~uB}{$84!o7Sz~XOt|p$aO~@_xgS@Tog)$|tzz?OU z>&>&LS64~&y|tnewQz0g=Uj~K0wejNM$%`>s6+RZbFQD;#7EL&^4bKPw?>o{B6 zuB^L30R$iZxXrl~aJlJO3>mN6XtTr5XeK=D3NBoS-L!+b#etd-508cM*XbN~)U{l$ zJ!EZ<1A-uXBm<^p`g&{*u&_Uyy5KN-9~%)$BMWYqfV1oi^mcG3A>15yM9kDW5b+9R zb1JJmd<5EMWo7R-Hnx%|CgHTy@+JK+KlmL?KYIBbUSyu99iTz z#1GOE_?V$|HrZpyp8;*rA~E1_M_5$)Ku~f5Fs7=OMnTPI%6e0K8|JJ=ZAyGe1F4@z*$hErPl~q z+H;&tx94wxZuN>4VpNwPRmS;wmBuc0D@`F)=_pXC7IlyCo|4ckFhi&dIe$De6wkt- zxi?sYEkzVetdHSj_<<2(3^I5^G`)H!UAWiFS=ucXzlh?YGpF)rELku3>rhiQ(4^cF zTezBDh@6Gi&b=FCW~h(6jMjaJ=-mhW*6Cd187xmUaEPyg#XDjf_Iw9ey>&?MiYAk% z3r#a&gMuAM;@2e6pCOwXO~DLln%cU$!kt9td=WS8demkZvNCkHUmnlWR1FSb-6U9^ zI2E_}$XS%htxD*){u;Rmxor%*t}iT*#?&M*jvpCp@PK)<0D7=!0ps4*>n`XHu6e9v zp&208k43M!o2oZ2pqs6)uHQXvTH%GX98fB41J{kRpK?3DRL_?&*{k_JM`g6*>N&0K z^KZ30tbjCC6)b>MmN>i^DI=6lVUp97bWZd#?@*+zmm*Cnwqx4CtVcNLN~8kFBTIC> zm>~V7NLHms&EqxKk?_&#Ki2+-S!gIv(>Jnm(kxHo!@IJ^@dFbh?b(MZoCflMYxKz8 zJ5!RB$<+*1ofMc`zq6QSk^}mAYIX4~GNG4$@_u3q86D@x1|qhFz^zu=p5IKWk=638 z8)693Ip>EU5j6XBfrLWD+B2Eoqn45Q=ns(b9CcCAaz>+1z;!Ur!?iQ;@E5pYB!fXZ z?C0XUdj!{dALAH6au!!Xv~}++e*>znA}(wDQ;jgs#Yc&7YCT`0fgD~>CwXT&8jKmO zVVIJo8iUJ@zXn#a&F2$qS3!)=jw*914|I}!8#L`x%eP)1wXWP$sv5@U1_i99isz~| zG1N{eM590y^h~=M%w^h?CkvGoajLzCMo|AhY(9gyLEt|*F@e1XW&ptW^tBp9!5V>V*TZmX-XZZ|sQX803s?UD91 zbL%UAUJS&FkeE1^qb9~z92ywcI0wI^ON*T!@~e+$xKIDfEgMfWFL0=Qj=$Zn zyPBa*J^;42XHK#$F6>%l8#a7&Jg>Nm88@7tz9PoC72{f7ucd-~QYl)`VJPqUlx;&- z98g_ke{CxLas~ZJ;*a;-|24Ue?s(f+I$JE|^YwoGrH>o@RB8JOd8}JtU^>ZIKG_n2 z%GdF(oT6-@&wFV_0xNj^FVW5{v%?1MzfI-7Uj`G&f2T8y>}-uK{vQY)Eh;y5`=SV6 z(|Qd1$RXmiXGccGzV!7H)qztOPG=RB)kC@is>4!lXzkidKV5ccKyMBT9nY2|{r_@3 z9*(CjFS*XEmV#0rLP}&E1l0JtdkPbphXQRoY}wnI>s?)ZJUDh}{z=ZzY29=mqtd>i zEjdHCz>r%;(o)FGWA3YvSdG=t>lD`4Qn>hrDjEcDhc`aKxf>Ov&7)UfCWfL4My&x) zvDj6U16MX96jM^7YR`!9t0LOrnv*mK1Z8T_Vy#U^09ODonOUI$Fjh8d)q+k+?GKmt zBT`f-4|!5xPSiv}`1Z?>(1Lt!Bz4Js^n+7^)=9AT{&{?jXq~5BT>sBzGEYZTLrMIO?=n^=B#2$qhn7$S)1mJr3ETr%KRq)IY@d94KMmj0Yc1! zFmKxI3dAWSGUPY!36YrH0bxycGb%qpx=do$C9gDw$REtIu@UoLuthjB^)}$YagG0sGZCrh z>8E)W9EA)%BYb1`1^6%19L7oi;FC~@6y4DquEy+t+jEBRYn}Ho$ueS!7Lfy6r@qE} z)rZSqZ-$qAaZwNGGCK!~td;rO7E*&TZUL|tJEWT`_;JE0VahEyFdE=9061e0u~85< zAVxLO5*o#=icteFcR+^;;n8OkMkRZ~NcV5YU+E`^`k{*!`Gf6|ce+?BO6uo%HkqHM z1Dt%vzRBc=y(l00KDnEi5z0- z)`?UtQCec#!<72SxhCIiMqQxpS+s3@w~fI5UC>fBuQCERVueknpXzX_nW-4;bi#>Z z9XWH_DsLSY688(FpQavz3Nxl~HC;U!xivRfsXw&~7ASLrXa1akg3)z~RVhCybyDxa z$J>W7k6mz=-qnXvguVuw$Fqt!*u)3td10DvSJI~D24X{&o8!L~hCjLf0&P%KPCn7B zcFuL9*RA9Me=#>;AUyq?%wEXI;!$Cn;Ys5j!8#|n|Aa77(KTf44!A$@(4$e)mYOUE zQ3}v1nW@_m{JLmSmmtbtenE&z5&*Pvh@S*zpUc+23`&dgQ6b;ta-~z|&w7!Ye~hr+ zUXWY;MzyGOJ9phiAan=mX4g5Ly}pT$yDo<*X+U^iIRP(!fcVg|2XOcPwkm17a=Wnl z=QF?hFS1=rjO7V9V#d3pa%k_KZdr2xMz8*~;WD@3HJNb#=e$Occc;@aOKnGSYZ@c= zAfto5Vla+Q?MK^6PY9SZXQlh67LOIJ8v)A=?Pj(|&#WFk2Wq7 zaqY^&ysfOD19HQIVW-Eo9dCpTc)%EuNQ0RjN*{FX-2|E?^WxEq=L@4Sxx!Gq-}%UB)o!*qS9g=^0Xj#{h@ z;AWN9BR|ikcF`s^Xh*Fw)tX)|eq5+2c4sDry2{w|GkJR6oRK7r6{~l9UNR2FZauNNQ ziKgV-T?i}P>8^1{^cZ%Ac$39!>^A=u)0kNgP2YANC1u$2BMc)qir-iUVI`H`6cGUKoFPG5ZMv*V!|{ z=INgf24sUuQ=T%V`VH3Ox(O~Z>m0#Q8rZYSma%@Sw&hH}>z&yl&`$6qSn`J=1d-Io zG@1^l>H2kyJOx#3gyxqd8{aTNTbq!YOT$*WLO#v_Nu4bx-RxYe#h=m%3eg-f$$tKq zf9r?)M4-*Dhwu3H@c94TA+m5bvH2g4&y|La-9bCzFLRq=%6AoH)}roXj^CfMp%HPq z!Kbn5Y+oL|S$O1ptWawH?(f$3%XGYF5j9!c3XQ$ZjJh?o!|UY6z+SvBFq@%Q{Mv%% zfQTdHP0!DPSNn`}V<@~dYP5yARToaD*NZWp4?*5`PS;r$s`cuQ$R3&Dfq*hGF9c+U zYOiV)^-TKlI5@;Twqa%&{&D1lS#EBZKqp07mUM+Qay*5)(^wj~dt&u0_me&e2Y5rE z@zmj;7k+5{A;4B`%5419OcN1RZa|@85E>$lE>V0ERnHk>u{-2+MM450MRAeFjaxw` z8O%A}+Fijqj%mKU6sKvrn8M^H>_(tFc4eXZnMch_X7SqQNhHBLdGawgiuC~U-}o1s zMnz~q@e9uWYvgx1eng{6`9&x589KnLXBiRH0!hm2j=09H^VRtIn z2NC2L>vrC@*368>7E%RE5XJ*0%{MEC2cE%wnpegFksTQ+P%yc3Z)&;bo}_op;-B^re+ zjDDlg3wX6o6&)%+d&5Qc;5I{hUxj< z)`NwJ8iDL_ZUr6@r-0x8GQ^S?FnpOXW{kWaXY1T`<$R+}3%~`ir^rFt?hbSL$-~*r z4E61g*_+vs_1WOb1!v%!>Kn|^!H9&5#19@@*-7rLNfHXB3PSR5p+c>U&mLwVch(op zunYlVre<$B=e_?(QXH%X9N09&&*8i06@dQAr4sgRWf`s5z&w4 zW(2b+T@1~9($g1r50v4#E2%>C*-04Xc<#rzOULPW7~ZKPi$9<+_M|e9KMFJ?*EI%~ z`NrWuHz#O0zpE~6eQKF z>vt_j4S_D#rI;0TuCGu*C9R}u@iNtvFz!qw&xZkcGvU=z$8t5V+9<7i+9#X3g5{hR ziU%E68BeiIx)F0WZMi?ox`||J*HyFGn@m}L7RYgc{JffZ37Sa!WScI^zT~xcpE}Gl zInm+B`JN|yKm)1Jt*btITx$jjKH2xwJX%CxV_X)x(Oj!vK_#s{WqFQ6rQz~=-A>-) zuw_LlNEoZ2rHMT!Y529DA3X#+fxr91TsoniZ`V`?z zZ{V{j>D5ujJ3ms+FyH@O#CMF*dOyR?>#y?h1iUrwrqyej7w-zb)O@Qb>9AE|H|E=UNG5&y^+)*XMRI$HaNr zI+|u4+6j7<6WJZChGMSMlpluP-+#({7BGf*7Sb0FoHpbhsa!Pu-Z*-(ql_?Od)%Kg z4@W40AjJHflTJejV{GoU5;o>?^~G@2fJ4Soj=ydDq@&e{x?Wd8W;{80kBu;C4tBBk z%;&Qz2#W`Y$fS*&Z&6T~1m=W8wWT^;LSK_3LZaAJ_95V5yNt-K`H~~6Bdf_56SR4i zFRIaiJ&qw5C{+oFZ;vQ4cWZ{s`NNctA|IDUb2|7#E|36v${~8g`3Vxx-5?ta_S>dYWac(wr1gj3#nli4oL5%_yB*!;8DgxP0tn z9vzO1;B?kF*VO5=^1bk`iCq=8Gav_ICe&Ed?8_o(RI4HlBP3gm+DVD}hHU`{6x9CP z1i$HI-pIzJG2MALVq4*YlF^zllRRQo8y7lI%dv(m)&$PpTtidHgm-!rNfT!TDrh=R z9&+g84Si-!Y0-hU+@Pmt15TLZy;(So3WwL%R6^yrIcOy>t9aW{lu$Y?6Lp)a$yLvg zmXC~yuF2-(kec;X+XFY9BV~Lx@ns%>az@HS<&u?j5wyq@+8{H#ifrS#nneXlXw4_> z%Esy?QY1QU9h*A2_cIrPDV22cdKc?)^YvCp6qnXAMT?tEJGPl;_N*r0v>TtOPW%n8 zsHm)MH9k0oIL#V6#qLnB%82&0xSt*>)wIi>FO&&}V8UA+u5XzkO6wuXWoO($P_;B_ z@O-UqX_iXMnYLQt=|@==N6NVsn#KG2`#jz9RTkeQ70!iZ0#iv%7q!grsnl%xlRca1 z$vB8qh0-wVw5z1F4%jnaz4acttDlGx8ME$%i}yoC9qLV1BkaZ$ z;R~n4(G)XKB^**!MwqawWJ#XB9=uBZ-`Mf928;!H0)DI)TBEQNQ<>%;nmbf#VbIwU z%v{}^tcKSWZAYRsl2D&$JhtFqQ6_4WG2+&656eVe5wY0DhVtht*Vdo77cGvvfvRImp@b22&Ybx#mK+;qem7#- zm}yq%gbL~VcfjlS>C?-;PWl_wT~OLDwje?!)h1T2OY^*LfLNz{{iC#I!~E)tDk99c z5jR}o5?&-Rl3uA_>vt^i$XF377+okXU6F^RRAKB%7Sv_ed!t!;AzwisAyPglS$gkE zm0E#j#)uv+LRlh?+OOwvrU)mT3PHCW86%CtN!*r!Hf+#|jLZHOA8p=*f?i>qu;o<& z;(^ayf|LgWDIEX$ra7FEHWwZ*M>#j6iuCnjC!O~Nj7uJ`OWcqCSxlZe=#yTP$o;26 zSp)NZ&;)>xZyb0OkQP(`xmJh?ib4Eov5!?P9ej)v;er?f?Xm7~%;tWBT)8R$LeK?5 z6@8a=W;VaPFOA`8&|6oaM^Z3v7fH}J*NKE^VOAVV(FROZQ-}Ngi&EnsQ(z;i2d0t! zSXUs(_yNNY#aHxifE{6kcrkegnah6+Cz#n>CPlLY`v1vX6U#~e6U#P^*O(AQT~iKa zm9acBKx&&mjl&LnywYHjl7LI;!xBU{N-E|y_Zb$V8Qf@S@8HbyL6;dZbR?=RQ{$7J z9yT-da}K)s0KPL|^zwT0CFN(p(V6x|QadqpeoK1&IJ~#UAO86zX4={Io>@|o-qg~O z8gb`n=jwLyWoduoZ|9#JElF!jZ_AkKbN$;VfNdIk4PEWRl9Sunzc`&b3`Azl_+myl zNm?Uek8+~NjkuU|X65klW5|rWcv-lp%3=s+2fmmXJ9h_Oi+@O&>lRGQYRQpBB9l<%?%lasYj&NRpuyN4kBxb)AIo z^9lZuHA||JLi&6>cVrz}0X$~ONy*`pwjBm$$dt7e2R&J+avKaz4(C;y1mmF_$%Lm^ zZ2=ZP{rJdoAemWncLuaAws3A68;kea%e`5Qv`Xo$k66z9Ff;76} zO##4>#f=rt4F>b47o}>sdX+r-NmDs7tM_m^$`NdZ>DWIcw&a!P?&K`c9xgg*`dYY&)+d@Q)@P6k*G=$y?EOP>@zjAUg8TrSNZ-CQDDbOCAq=I3;)%)Jtm=4l1l zTIt`=Vmd4h@ku<3ncYkS;MtAlQLzkD1h3g9EuK@?c8U!hjSvXLY03%~_3aA$18;;L z&?P{df13}1H{_geDa`_i{G9ws5EcZR0S-E{=w7;i_t@DxB1c}&UWA-%KjANrbP=46 zeg{PN00NWtp#dl9fabZG!b2lr(-TDF1k$^Q`@%O|Z9Z&~Nca2sc{PGi&k#6gR~Ruw zu1Vg4mXpJFZG-p*JqPp=x>bTo?K`>4z_7<46ni{IF>i}3JhpI zQI(@`CJ-=!_f;vL;H17R=Tm5}|!5M-0&8EZ=}Z)h- zxxJA(z?lFW)GEeJ8qyQ>hMnQWn)i|*K^xDwUDa{&A^-51v^hPndx&J4pGS2jL|CQK zmSWZV87vLG$dOUPLDC$q849m`VPN&cQ-_%q&0d;=4GZ;eX6?H4pKi(L&>$+Vmw-iy z1-GmEx!Hp{-6~<0dhi56pm~pGFYw$#aTjU=Ctq=o0kAb3HXL;8b>ii5F_I8Z?l~Sd zo9JNhEa#$P-fEKhylNA&^qQo7l=`|b^052aduO8(yDHVHAj>{|a?pi}YE0alXarq-4v?Kl=6hDQTB2IHb9 zuQGt*==nkQZ8XS`U6=whi9xBkZ6HIn!gm~iCk+y3e`S{3*e>SFdRYC+GbC(zu)>xj zFRUBZ=v}=h4XjYU7gehnzRr(2p~|W!-wO7cE;&cr-jeX)()wT!Kd*TqeaUuoCOwj| zGS6KqR@Fxi*)9Ymyr?+5sPbcC-DDu|Pw{Z62#W?nY1I-YN*3#*&O}}Q4|2g4KkV4$ zAI@wA5nABFr*4%8yQN#Lr9V4RHt19$zLMlEHpY#If@t71-kx@UL6HgZEmz<9)nXkz)-Z=2A8}@<6h(k;NHDt`T4wk+Ig3R>blyM3Xmw3Yk-9 za?Du7oebR=Ks9$Y^}cX7czLre|4Qf+$m6t1W*QavbhGxOo&F>7dtD=j?}6DXoHg5F z3}s&=HR&*6JLT zW&r>sF-!7ByNArbxyqRFlV=v$Hc{VoLA9*T*QjgD0-D~#Cw^>~iv5PaEYa6NIsNrRY${z1|IEF8u{V64r zu@N>Q%L?J{i3&XG80E(Ya!U1k>LpotQo3c6S>q!Cyy{PiyfYz}9?5eUj4PH^te4=Y zpK2-C-1r00p#_6VfF$Lg_C1pM#6GKtNNz~2r~OK6 z>8tY{jvVK1I~qgwZns{?PE|8VgpaMFb&|e&g~t5OzJg2d(yEGTNKsyL8HH)r+>0Yr zB*0#q(UKA9+IBvt(SfUepXM0gTtWAP!s?Z0cDX|t; z<_CBS8Ujnb*-@zTx9eUID{t$ZsIzdwlTX2z-ZUBY~>lY@k)!enS&eZ^BIPsAM z{bPdGN!U?Qmz@$vI_Us{G-xMHB&S{md)imZV-#_fMiv8-Nk?vuol)iX~zS zt4b=9)!0n&<6*KTxT^qYKT{^lGrh{_bcxP2(5~AwI&bBj)tv~0^#igcEpQMVI%&oI zYVD-?1ikX#lC?a~^t9g_ypJkG0j`pF#iGcCs|ftv<93INWBiOLm7y#kcncmep+(&&x(UOB)S0o5MZ zg3nMaE08(BWw`R3^#0Ya0dR%0o(j;e}6bgZXybV|=7ZNUi_M z&C5lRcdVGbN`a8?Brj*H_cIE8!w)x9`g+e-0>^|>ZOB}Fqn%4}IlCN=+(otLvs(T6 zUdtYVDS$m?vnkp~-^9*E?fju_WAn7=I60sVRRe!%+qeg)p%wkd))KWOvO!QvBy(|Q zE93opg6q6UzXwG&C*izG-phAO)KrqnRrH-wwAi7Vxr1wh`OMDS)T|17Fj zRb70h(D_z&5?E2U3+4!&)(o98ap%%C0YYkzt^KgttTz_nqzE@1f>*c=#l*6%`~}i- z&rKy(lBbtayhA&dys|(t&e1R6=xs3)sejlV&NPY3a5A8*R;*c9m@t#7sGl%N%rS~u zQ~}>UA;}fj0ZxKf?pkvz=X~gfDzSsSKW3?>l^eYk=fRpcEJ8QgBb}!g0}XDpS~G-_ z?qCc@FLia{XIZWxvfz?j7XN$O$i75e8LHP>YWc6!jMa>qMJ)FuYSeBtUhi}*k1ux7 z)-g};bmxY=vw|*(n83)!nUZr}i)$8ObLdbJYBqh|OFUMGn@=X$vK>~fbj;;8oEAZ; z=DBKu`GiMtsYaF%A+ndbg zkH=?Xs(2wk-!TwMl1jep%!i7?!W1o>^Hk?3aaXvZR!f4BzUokGp`>l3P8dZP#W*bN za*7X!!m%4TQ~^g4g)5FYS|kR70z=4nhi`x4JpuYMCW@6+THBbzZjxp(jU09B;BqN~ z#Wxa`Q~7}}%6paJkeg}GW4%wxd1~#D1nrmCpIt%ls{^$f9mZ(bp`^Y7K#gTia~1cP zKw66uiDGGm6u{ot(O7=tY#|@d#=CxH0)@~faUa8BF^=!?sI=@>U0_$CkK{n!V;2It zkQNKj`SJ4@TH0eZG_x*(nwa}62%X|)M;~7~#Z7WhvR0)b0xlwrW0avZB_{Th{F zwNOUcorB!5?W&CSf{Nst9PlpKLnCp{bUSt}5`WdTz9ykJJk>gSKw=nws9=K1e)uoi zvQ}}Yrk&ePR|l=pzrl2F{iWw$UVuMVyKk88Tnz+*dx9GJcedWJ$WvR?vu~9hFtT*H!t)h|Z5&pm)4q)5*-joD*C~2Xl zJsiJN>aIv_a{;}$K(mp~D%jetAg0rn;+Up|t#l3vIUe<18G7_H7FMR`zNL`|S16-ueHfCCT_dwIq40k}%sEcfP2cb2vc{kql3WT%MKpbgZ3%*1D); z?Z#932k`Z$;h2m~)l>g`^yFPH8xhBZh8b~nkd-QGq${*()~NE4ykWU!*gk7CYokF+D|I6&j}4LZ{w!jus*` zG#4nK+nw-(>np(So8ye>A_$O%!kM>mL<3vcG`Vobp}wXq?6PiXU>{YnW{JJI`9pE0 z+d#V~97M&FK|mh7;l5@$?ULz7`1@iyNf5R4<+6^h10g`VD%|{2`P(W6GO20Z1j2a? z!Xg*ov3SDgp}zaax?9|cm_pWV^qOAg7%`USW}a9!UxZ%jh67eQTr3flzbPNo%vedb z**m2U2z@YG>brm+zX#aJ7-K#G4IdU!dYE*oT*++^M__P*_g|MUug3z`%i3=0R&<>o zruI|a>cKY6r|+hz9avJ@N*x)DX0%t)*v#cS$3={Ka2S=k$Kj;DqYDGONn71GyIQ*s{y~&{Tv*YWky)i56+h-P{+Zk8C%-^E z|I=p^JN9BYuQ~%{qvvY)Qr!RUpeKcZ2#k-@@K21gRVXoN_^I9z)k0r7=PSe4RfByN+H9Noa6Lx#5ljohDNwF|DzuWa&ZVEN0g%OLlXLeB7YJWp ziWL7ph*T3mPY&?gmc_TrNb_z_(tH65TYV45d1jBl$Mv@Y6oiF_pr|G|){gSF81hIr z4KhGbNXf%fxXz@f5{Kd=g?yUz>16En(>OaPQ>sy4K|Cl%gL1>rvpO)_RmU+KV3dgN z`&Lh=x<7?kv6fkwleS2!<#$q5mxbMN*=Rrj>*xqY0|>q8rE?})@tlIlKw%8BnVv|G zXNhFo(2>kmsJAVQPx;lqeJ`H44BPMrT5F;DLwX90zdE({+pP@hWXXzwUoI0(JQdg6 zFul{Z2$(E(Vv!)9V(F${mOHRU^$lBYxhfw*Vat2aB7Izf(6{;^{v@Ds@^?w$#pciu zH0ZP3nN9^K0`~rFpnl}mYx48Mw5!A$j-IK#ab%A+Z9p5f8_C*XYy74z%Y6gNF<2Bp zqfz5oNHBa&(co^Rgff6I;1cy&cK#&qfN=^ip#r*(4m?G4_e48M^2!#2a+5du)aP3p zl&5s6PPs--g?Yw840=|G>G_7^D1j8!rKHYtjqwPtIO>Zg->rRkz2qnL+-3TH0<<^v za;Lzazw7OZHV>oeg#9>yHe%e9_+U{PrdHhhOV4Z9o?PG)s#Q}JDfn?n=0+~Wm&o`cb0>Y8tH2anU6iA= z^9SNOg87+434rZ#(t`i>k4L7AhRLfOrpX`P57LvV12j4c)M9QpIccp&92%(rQg#(Njy9F~;= zxEwtt6yp|d==0Bn)6Pz3vjl0DL?Y-0*L3DmKlbr-|`r92mNK+O|| zFRkCo5b#~09Ev!7_7n*?~xsMT3{0&(& z0I*uBw&oK!gB)KC7l9*Y$5Ogg0o;iQpMR+ULYZcq4M7_ABdJ@K%U^Z{2BjAzy<-js z?%W}++-oz%*tZ_PlJpxHTg({5SIU#TwhKR z&Kxm=xknm9m_Maav7VmR}<3~8ERC8m&N)+aQ!pJau;Bf^chC~5TYrbRk_>UJBwbkqAH6kfB55$4fgQT z4a$Q$5Yws}L{g{iwCUh$hg>sf!Yl%51gwI2IySMOC{%=w)sc1T9o+5Qcx*)#CYgwG z1XeINozaX>=lT%huJWTL2?kObb`v0mfwkL#F@^!jmX<)hSwB-mVpp9k2`qL8u0nx` ziYHJ@>^3|Aoc80cMfT2)aXihAtX>FDdX|}l(08)y+1Y(EkK%40e<-lGyHozP=1}niw|m; z8-WR!0ulsYk1@2|Y#dcVk%3b%l0{9jO$=jd@31681Nq+9Dw?WFNB0xQG1Kf(>OHPy z(gZ@y7;5^xim7FC;~5>RV2P+9t@3_`KduF13JxkR^{D^2^o96~^%An5(9qUp8~`>O zdXHeA+3E4ZW9nijj#faLXm0{4C-h*I;rnrP{-T&Lbz>3Y8E1-j=gzb8M)c!`P>@*4_8$ht@Qp-NvQ%WWD z%GIIrDp!O+Sru%U5vC+rM^(UG#p#FV)#2u)RyCK%R({sPCAY>Dx!99ju7M`0K> z?fpRxlMq>DRW_St)+lL;yOkd_iZ_x>O;pjtul>y0=+WdXU|kOd=4&XW>XFLefpvl4 z6DRqDv8(>v4RvW`%EM9n#`e%;+N$1vB-BAbWJUug6(Fl{vmfKe<9z$`bEVv&W1dyL zPjgM9XhKWCTxK}DF_n={UC}sWqMI(u)-X+XR=cVe5_5W3Hg#JZ111=Csm+uyUER!b zAJyIvc@_zkFdNBXzvA;B)$~&jUNN}ZN_5dH1e?k)99JDfYS&!X)D65RUJeIBw94O(38Q!GLzCFRbSk>eYBa#l!hi!s&hlrFZ8!P27`IS_)wZE_ z-e?7(%kR(PfUAPZwlMW$Jgcfakk&uk&|Jy~a>0_EC8#kAk~NJ>;1cg+Zxw>Kf5?BoThW-@&JA*A z`SzYtz3Wn8Sggdei>)!a3q2?q#`lw20jLlY?c)Zc8=>2ev>7V}c#NXvFBa1g7%}XN znv{X*oN|h%B_uKO6z9tMW+fApL>KCOVe7mfd*|!Nqz$f3?$ces>k7ky`jN`P?R@vu zO&pJ-lFS&L7RAy!>NF2)gtjLSlrYG1lGGuaUxi%&1Er@Fx(X#5zcRlY(k_fOZI3QW zIOZ%tR|GC~!(s@&R%yYnE473w=s&iDDw{+=aVV`bP0EG-rZ}ocp+6l*oX0Hn&Wf4! z%zj~PC{e*oH_gLUQuAU~I3@VjK+cs-t~EmF#EVa|ww|BHE|26tLc&#JZfQ(2*ZDlt zURW1*3Jzi zJ2rDs2QFzzis1GH-6xu1qq6Z|M(U2FvY^w4a3(eCe%p?fRe*TvC>YVchaw^Bpa~vD z``#zZ-UBv@0{c6M1RyVcWZ=x&`!NnppbUk^Za!&scYnwHML+UlCpo6v_Hyl*sXEJD zr+rVSy#|-vVqIAL-h9hsEk>xa*5&zah+vrt8Qtm>j%FA~O#j_!;L2S4u%YpoehX5L zh=#;AM4qsg`w>sdZ|ay~5WD;>a*e-Igs3>nPAuoD)Y9Hgo}3$+qlV5j1%kG%L?KX| zYkaLmsjPMGd~v5DH>Cc@hlg4KRw2!!6Q*r3)KP;HJ@uvYib&2E2$FK2&jKupuGo|K z#HZ@put+KdAd$3OLU?+lBC{6SU-h1oe{AIjGA`sUmwA56<#MQ2Tr-ofg`G>7_Mil3 z-*QiqA|a*d0ve58?w5z+Rk89^SdQ)Q&SpRHiTgwYfFsr~3PbC{xHc5A&;K=dV91_%k6_u(~`uBbEpn6WSdVquh z7wwRJ$EtfAP&Gn7%H`4`zNp8C-o05WUOgu`JH{x!y6fFc$uj|ih#a5GlMpEIF5x0j z7i1}}jk3%Jx4C`%-hTfo-Kw!g+dnDH#3R((;G-tFx0}i-NH>E$MFU!T3YiS-$kpE^ zl%SPvvCo@=B0TVk$b&kw=n={^osb3sq4Y^79Gl0s)d#bxr z?1`6Q{&h?2B?{+4DIZj1{}?uS_H((UBl&U2Y@esR~2?KmbST?&Xpu}vrWUg-$xHG z(I#p9p%TKAHJ(N*1xv<;JA9aGOHq)n?JfOFLe+4#K;lNN3{Tym4Z~2+SD&ul)6e};XoIW}()7RFU z>bX>R16_NNRc}Y5|q3Pp5jb|q;foGO0*Td4LfUBU8xhp~2y6!w|7c5LOFmibQ6NA-m03vJ7%+5aG2T!oXT-Ud-p zH|WNW?-%fAhb^}K01{kXIx}fBjB8aIIZbCvmn;d*^eYY>3or3b1=ZxwcTkOJkHF>+lE4HAj_D9ipIl z1c>9AgqhzD>{lJN}C6cT9t3=d%` zoLIflAJO)ad68q4)^zXB+?u7TI;+;KwtZj^Lc57CQZ+OgeZDF_rz}$`MbTs%(P`E( z0YP&iE0qd&@)v#0-D0QX@>bD^BfWg;E-;Ncpk@CJzlOxE;1k3zcNTA6Rcn5?v%R%_ zKXf%9wjEKA9_rf(fI27t6anpSmMSVew_>?8qP!0O>ueNE;~g(AeijhHC{_?(!*hFB z$5G06)EGK$og|ynoTr!j^XiUDg|0cgnMKEz&biI5eL2C+^H_d-5>G%Eg_&EZE?V3# z7zT110vnYlLI5>t7C)yF7@lt)rlzCtO=Fj=%dW_n zptZ=jj;eWUyv@WhmO;w8pDrH-ma#p)LslbH2WEA?jA`VO$9qBrl6*%6g!;lQ_-xzl z#eF@Z))zL=glz=n$Zxp>%L;nVhOEvl?|>3$&rXfpaCV~pY;LdzU5vXtdU1r59*HP4 zu$xbl+j6m}bsvBL>gIb+411CU@xes@TuZ?dxd{mTEXk57FFGwC?-?$~TLXYMW|I>h z!ORRAlfckmv0>_q!1G1)LxAo95`kvK&##H&T_zI&1o-QWQHZXTeaEI^mZ&DKeHJ{i z9{yn6`@_CHomh|Ez`y;PBB3A8Qs8cMOQ9=n$z=AsMYB7@RE}R)-oYI9 zWEmb{C-lq3e7Vartkd?raDSG zQ_pVl7)M0tzOE4-{)hQPEE#cRso&Yd{~${JFMFSk(fmH1zVS*2Oh|c{RGH24SD-egCSt=Tu5tUT1#MUigu%!o9k# zfd3W1fUTksAx+=wfZtU^9ZD`EJkG2{&vmasupe30vY<7A6%rmt)n}_ys$%1D&sWF} zrXirt&~gNi22o?L+mkX5(!nCLQmhRiR0^A*SV@K`HZ|1C*vH5PUeq!pAqm$p+hnbU zWgd!nZwtd#cAB~&vz2>UK|0D-C9oPgsM`rrH;!z{|x)NhC2 zfr0&(=B)kTTW)A5m;|d#-tICmua73S=atwVM3=qabv2NjKS$&Ng@MRz@Da0+v2Bmp z=Xo#zcpJZQO&Uog(&)CVvG3*fFFsuG+Za6l9w74|MQC5%JLN!WLWxbk4?SVqUvzl# zHW&QQ0I1`#S-9vg77-H+;yP$%cDb)T9tY#}=>yGV4+^7@xdQxHW0U(*z4Ye<%)G!C zP;`i7nJt34kZT;>q1)X8q!+?lw>k@BjJEHkzyxkrd`Vdt#F0_9DR>FUeI8k!4}^K$ zPdH_MD9Kj_s6~u4xcA6K;rpaK54g7aJO#)RM;1~9DxzZ%G%#ji`*2}UKW{h;p$pG_ z+^1+Bb^(?OmeRbiML{<%%}{i-#HvGUH)31tZ8)xgJiAtFf8(Bh=!jK}aG2=K%<7|4 zz^3U_cbhSRE(TE2#iM=tZ5oCUc$>V1USHsOw%upHZt*q%gG^ayhutFae0Cn>(6Noc zAYAE1kd4_<5RNmoPjIBG?-vAcFcKKk829o%YgeQgcDj?$a!G|;I_h&*ME?CY`#Wg3|DM_;T=5g~fkG0lQhz;w`H!b*@ zCtLWMexj$H$1-Qgi|2r0>Mvmv=nvlWyUVtIPI#}A^Ou-Jjzn4KTZ!1&uPbLbP%rv7 z*rRX?eQ)IGpDFCx{YSQI-;WJv*^eG*#DbSAVWRVuJYp%tJ>SCCQHL3m+Een)$Fk|U zD9VeFI6Vt-PW!mpVGdRAT)zT3bG<7>e~Ue(`b*azNod-U-sU;A&ncilm+$|9$jMN! zBEF3P0H7%R-$1Hd42^95I~wPewyiyOJL2zPh#(1t33|mn&|{PWe#M#-kxGKDWY41m zgo#yJ!&;SF>tT)QzxJQ*+4wE=8nu>1^+&paJ)$ePaOS}bb2Eu|Qf}#moW$0PXk2je z%c~1^`*raQQ0wR1%Jhfm>PDKYUN2W`|7RrEkMmR0&AR@upX8x#f_0a53OFwsMH#cq zyJab{M024R3~Uj^tObtgQXyzp@`0kXU3bzYF)P}nbjj%zp+85l|ETcHi7p#bkb+35 z(W=@zP<2yDqer<@p-LDDG|hBIn0iccvVkPTn%1$RT^c4@bRgryqpfp5NF>GvNXFQS zW<120h4>2A9b3>aevFPsx;Pk-LxaR+Ms~?H?jmhXTjcJeDGQiMf%g|c6%mdI4*}AG z2okZ+%m~3ID&Sh$!|+I~(`*wJYEJbHlcgU_NuD|;h5q`3tEC%cn0pkQi4quLLHg?T z_h|Ly?h+C9&bKGd&&|nDX0QPhThlRrxaHXR#28em))>jqOqDYP-c4ouqY8@2NBE?O zkSHbFjBg1dg(tc!`Zh*5E8Lnokw+V?@a~fYPpP44B{XSSyhzn@CTl>M^E*K_u-96a zO2FmxW0Rl}%O+@o-*hrg6B8uv_t{c&Q&oiW>W;WRCZQ$LtTkQYyCynA}HxX&HwPPU*jydLcSs%9G12lI7907J?O`j1rUp?qg z*434-r85_XaD-b)nm;s(%m~8|Px6FkKETBGi^Dfy`C!0^n+^%KwqP{935z$mwc}~B zDHvJzV0d3MJqeBrej%Ao{`D+bjfoj2g81&VdaBOx;pjp$&pVQ^w<@tGL-;|d7%)|wAWV zh489TbX+n_BZ@g!Z8NU7k`k2tej92bH!g5&CT?3=l^H@}vQ;qd~198sy6MEBMnyw=fSlPzYNNyxz^OqfIHKm`SRn(h97sbM`=d$eRkN(EcIGycjW3eVw=f= zzsdyW$43Y>rl*QHQR7CYLT9lv*CXW8X!+SKLL?h?1b$8tg_OcO9W5qf5 zbU6#qFs1h`z*SIEx2x<4i)p5%1d2ft696TvgvX5r-9gMB*_a$8B`{1l$zg(Ba{nI6 z8`W}rqt>NHpzvqcjTPGR z3Dj;F1|k)2(ZC@Ft?i)M=PuZG`%54Zh_&wm1S$8l}fSe)AjIHj6+bEc+@@@m{^+mrrx%?zC>1N`Ay=6 zCIR~ssk{k8*O}sDjU1^8X;r!K}M)dw* z!RDnM=$t`~zp;}nJ`MPk@`qns-CKYtuWsoqX}*b)n`1XAeR`j8M?t#cUP)u5^iq_uAk?F1!yU#aHf{^&5x zxt;z}nXO>@^_fe5jN)2X23T@x7Dl{h1k$^!K`JSa=X~a;08zL^1s2>|EMdJDz~1=! zsHL+8{HLh&ZI(?wr*uo)`ll7y8s?^%rcE0V(xAel)|^iGs1_Fs8!#5*Mvb&u#VWQ4 z7HOH3;1M6GS@?@nNp^ac*SN~I6HQ+=MI16wY@MAqHt~GMGnHXVq8rtN94~maBF(_EVNY{-pPvEQzqJjK2p#9ypT|( zK*&6qd@B2E1EsTQ&TRHOkMKPSeT}aLG2Cf`DEsw`w50CuUP9djIN6M9VBj8mJBhI@ zZ2Ae?N4L9$-B4fXReO4wd6u{8d1KC)?=rOmngbb>5`XJ9<{sjx4qabu9eDK3c$ z2zOd6ADKU=GtHmN>;ig39@qeU(8Ky%A&K={V|gK>z3+Qs>!|ce4y)cFi+*>X#)Qi! zqa+qYE(h+m{!V<%b&EsFLO7J z^F7T+_-QVgk`XdR@T|DAc33pK3ro7Gqsj6@X-O+c^KfNGh-H2-v97MX;Ww=Pof~sB zJ5$!D@U#Y~OYx^d>x_b4fzss5VsT_uw++$-U1x~W%d|Ktoi1>N{sId`f)AO2cUae| zaynV#IM=HGN*sa~P{AptTzmB?@Bg{))KAaQKv9ADG%BdkskDpWZvU;F%Q{&md#s;) zbp{{Yc6qM?hTWMvqw?!@?fq?Cy}rp>8oJN@$#zfGFfZcj;@j_AS)_r6uAI*M-<1(EpgCgU~ zrdhqzc0=j!!!0^UAhUFb-u!5#Y6cK*WXvL`?; zaf)r|h36kW%fw^Fc_?+NZ!fKTse5rO$76@uDglMtihp~6QTK)V*8dzW$`(--c-H1IV0qrUxHQP?6ST3s#x`~xX2|1%F^j(qkH7u9!tTV8HG=9Y552sV?KwC#fd`xIG@>jl4@>ABzXFmm|NR4A z8Z*|y2ZQ=orZM&)Zh(ODT>;8h0to9HU>91E-|>HLvgKqyM0BZMrd>^OA4r49tr1eTS&tpAgAen;e@M)OEG;eKT^hJ$D_k=S(uP=QR+r*+ol} zX$}%NNCW%9s6rWrRjKN{n@;@s_0Iz?Yh*03(VY17q8h~G_fXq5Ow?y&zfLq;mqw)Q zE#k%8+g)hi7vu^d=3*B58!$|{Z{CEUQAW%Y6ZyGRpyHc&RK=`8<1C`q*fu36Vbz^*!G#QOwcbLm4!@?S%kx*vPJY@^Z9ORuuqdqDPiAxD55@&8cK%@H>d2bBmK5FJLgSPf zneI5mKR<23Z&chNzuDbJCU=i0?LhF4Kyx<$W-1n4x*A;rNB8?%eDh-~-jt6c0wDJa zWaG^Ac~lL%AHHb4rO#(N#sYHcGSPfROqJq8H;g8=Ds>w$^=G3SMWYQ^Q6gs7Xip`M zM1SJAiT*alrS<5#ev*aHJ6$X0bk$iq0M@9y0CAlZ2n`w0VXtoAd9=|8=c0MdRVr-+ zz=fWZ1endCzC;RTMf<6cJJ0G4&9kRibZ$?hM zc)?9tK;DJJvHI?iO&M>>;oX7Lf-WjXwt_|Y_%i;XYyweWV8`%3$#|6!D_$c`aiD`IuPbsh3q?%8<>!Mt8CrId&j(tQm6 z?F>H`;qmjnuO9taIqo0x6ItL+|T_6J(uUe8cBoAQ$)dD4} zt5RuGKI-g_=c7+2G3Drxtzkh^Q?B-@5H_y;Z~|GX_qL+;rQfQW(stb-KAi^Ayt#gv3U@-u3pW?(HKkR zKsI54yUOIqKtD~kg4P)-Bmt_yNQ)gTfpzM& zCJWLrDWtV*zlao-Z2doptB`|;U2gQaX$&_e+;*kwkI&>ym?b^8COwBl0sX&un}Z2fbk)EL+I09|p0;ddm<1Y>wnc~1n z4Gvv{$I{A-B@;Z(?a)Qi!|m(TJ3}~VMuxe?@>RI-wm2NEI&H*H^}oIVU1*Z}c-=1# zg0}5_RWsO!I=IFBx#$fRzSC9AI4vZ1MAOF2PeK{uNaoD4#@Ra#EM@+wx@+)9gcJA1 zk8Fd4Fv;&wvsM%`{nJ$rOmLJhHu)kN;02lz53al+M7M)OGbjGoBzYIuS(DU7lH3ta zgNF?G^}u1p5r;*J?)E{Zu2^!NI*uQtIooVs&o`PV0V9}G z5;;3%k;qcT#5e>@Zxr7VAc1vVj*QoUJ>CY1GcKT{TQ8*MI*9`_q>;sdkv*2rAseO4 ze!eHOyO0cklRdioECKjC_oUIrkUu+9i9Ewh0(zEy4t^1h6!{D>%yj*>Tv<#I=!ksG z@W8VqR*r3bWxfHBwKt8YW9B%a$oY;Xd*2(`*y}eh-Wlr>8(ih#n}%Gj^DVyqly9-~ zVoR*EMwNUl+m`i=2i8uVS`C)=J%h4K|1w8!A%$PAA>9R|eUkNRlclGAini^*ygZhP zeY76pSQIuu)7`s+pj5>S>KRJ0yo$uQbTHefrUy0009E?D;XKWn(_r=Rj0Nc67-u@@ zL!L4V)$xpPXaGc&IvW`LC>v;kgQYJrcLC@O6n%`;tnKOLv4y?c z+v3m4F1jYAp+BOwi$+EkX|^2G#)DyFo6hG{*ksjO=gaSciA4`_K(J97X7DjZ5u~`IZe{$KNS?Le783 z(NnjvdHe?b4;1Kc-R<}K;HeY%&zt{eLHVEHk4%l3{trZz!P-eZL%)SROlSZA{QqLN z|4mCwx0*?Y~|8@tTMVU;0vD(6l5T-h29j>-U z?Vn?Q*0So2@(uE|X``ZTrHgDuNwdx2eTrI4c?=B?6*i@0T5_#uVQLBX<2SZ5-1ir>5Qq0VVKgWkZN%;TtWK&T6%k= zr&S~r6|DfvE&ihRQZ%b}55ro|6g!IE@29AB1KkIh*v?nNKGMMGLn}93(~aWcP>W;h zP$1nYk^R-cs8#5CtCT-O=L^(rVRl)(iPBj=i}0y;6sW187Euh@JM)cL*rr=fmaqbx z|3}?!#mhMtF;ew96KiO}3l(d)G24G~>lT@f-gTHz^*wYl=b}R)TbsK{cFKS-^<8sg zh>LM)gW#AB?RxBa&ElovKf-!)jb8%*(mAahlQl=|iU2Is^xFAIV$Zb49H=9@0+xIO zI6owXjAic7&Y&YhX;0q%*2^j3)b;*Hv_ZnF;s6w>6$l4elVuIm=sC@PyGOs>)V50#ef&8FYX;{}~$6Mom zy*rZ{j2Lmcq?Vhf=Bfg)K-56i1(cve2QHW@g`w$nA#B=HP%%{9)TcxkzyEhy>pYD! z6JVJy_zQudEoCKaUIY&pfXO=2+#|(oRrEm@r?+*h)Fyun_3JO``24@`?we3!)|%1= z@%D@adUm9YU8!?JD~6C1x$$A-7D2f>$k6BdI(W`;Z9&{HTat9GCNn9g#4NEx3z%n|b2b#5=bFWSbs^dwNbwIffgIDfH{}E~hlzOToIzSNT-pZzMJf)O;;&bXUL2GWkp>-t3<(~9e~5C%pz5W=Zcm%op)qN2#N%`qAX~ywp*u_%;Z5Q~H3!*XXj}nXz&mIc?gV(j~f? zNNzlh9NRD%|7KT|L;1N=LKlbp-q+yd)re0~e*RO+zcB8-vG&nb76O0#G3O=5*W)6H z$9DJ^LXO2@`z{Z}aW3FP1||4M7yM`Y-XGxqPhG~8T*tru^=KJF0s!Fs7s1ic#Kh^> z?)TqxJcO$1+81%4{JKqbO0b#}(p{FqQ(BbpE%uvMkRk$PTICS|k-LaE8gl-=-<)h9 z@HNzhEI6I^}X;^j*lv`h~cdPE7@7QWO=A=9&4RXq`B`c#Jo14DO z*^uqMt~J@z8qA)%QjUti>IzKbZZ~ylmercqLWG#WiUMk-PP005Knw4B?#Q4BYNIecZP{?5uDQP{;G;}~rmnp?@$DF`aKKnLl!uqg^VXh<%am6a3!4n<^ zKAE()?wTswsC38L-(ywM4cr&n@~4DGtdRdaZ9zdss-_t3mG7)1JIw)QnWU%2V=cmi95HBm zzb^`60dqm|DT!_Zh)h3nHx!DV$rGLhf`I5Lcn`IQ)U}KHUC=)OdVj-3}C~yf7HWkr~h#HB6c`Djjz;7Sm&%XKWi>;7-u&T(dgupkz>IR=J-6T^5B4ww2GinAbn z3mKhTA}nm^RK+$Xk`a+;NLpAiu;M*Nu(*!^1m^9b;?+ii+&G&NOBnnC^H}B#nd}JM zJD)vD)i&RNsha?mzXQC$F0gvJ2Zap+W$|q}?m|(Rlh;$?ES;(NQ+kXA1vw7cFqli6 zucg}Q+uU%w)aIZztsWVtSI=PZ>V`8Gy3F1K_&|zzv*s;Fqi$2yjm*p~N9{R+bQb!! zKbOi2mf&pEjedZ5VC9VE$^q%uKNw61q51t5Fq6Z8wQ?dDdk3UE{zQ%J>VtQHh28R_ z@quwj*lVdg45Cai=&(AivTZ-5HG7+)hP=>~&Pt5V>a8owXMnOlDQe_w_x199(9l8Q z;B9soXLlX?-*aMa4{;hJa{j}M>0gNAt6{YRA7MT;hYg1MPEXYzA(suU zX*Q9%VtdANlRxa18!t)?ktLD74TN;hKUJdki_dkj-YfHJGRCsz*L<2UP4=X)?~lLJ zC5KfVD5@3d6CJX0G1<>`M5bCv5*^6mfO2cQ!V(G5tSC-M-Iz$ZOF7 z0H*8!04V>f==A^Y`@fmQUGZASZE_^;e5fPf!dz~<@<`TcY_ZPaXWNnA$@R{-k+GwL z#M;caur(Yams~%>-0##@ydqF4wrq`0nV_4)khKg|5Je4p)sOiN%q(6BqgAhPUeL*{ zMVZfjKb-xpd5w%etOjY8dL%YEC!34-{!_b>{3GOFBx|?$ak)zYfY=l6Z#_j+nzbb#^_+0RB9R45Q>C z6o-OSg&?<%rTX>1g~u|{878`W8ZClUJQl4XQHU>qUn>QUEs-cFA_6ip6ni-B4M&+elHl;iY;3AP3Zyd19sv@mlZvk6eOn#Qj}AMe+vn*zU$$5bom(_?amy%Cf$EeVcxAf#LUNsJUAL=*5-| z<7LB$BO41+KggmM36cRx#})#PUlslXj(Ir)%+kMOPd?bdk3QH>`kzzg|GNBLTzq_d zSVTR4k1K|{m!{01sY7!{EfE{8Z$F2}=ko)^ zl0*@T`k-?E1&%#IeknEyN*X-~-1h!de`Xw?%`*i*1@62P96*+V&Zq?UcaHm`@_2Kw z+gBa?+!f~Cq4Gp0npOyoJeq*WQ^^9K<^;?*0=(}4B}+mI@?&R>3MSC3%(a^~Eij}# zUS{W_YVMwCLiSE97+H&f#Gyv%9X#N5PA`C>8@<;Z#ZUlYeCQ_gW6+$t6Zf%!KB=<7 z|7b?wq%bI*)lV?vN4kMAbCAr>t3?r}QGuMW~r zHfdKcpvsnu7ubCkd-pKvp96j7uHNgO$lRR#!vOqfRG42Tx?h{Y2Mgdx^_R%M-?LE= zjWsivl})HHdU0_=eWrFf(U-$yhC;bMfBlE-Xn6;85l05_qav)Jr%8#OS63*&JTuO* zE6O!rjjnsZ+5Vc`ppj8?i$*#0{Mw;2aiqUWtUe}Mk)ezDvR~2NKLYS|*!O!dxHK%O zC8y){$qoQ&iKj*%%+M|7S|5Bokgb#!n~g|c-q5ducBe|tGidc3i&9}}UJu0!>VS(a zpoO|`4 zNk=!e4g*y0!*(HL&=3M~s5cb^>V?!O|7nO8YR=((-@Xf-u6WVxqkbbOVjC5mC))^g zKwJj{1@94>hktp~cyEfEU;`MaejF4)@)w*i552O{98V+Kc6dt3a@YUZvhU^u}_@M}B}?Jm$Oc776)-;HK-s=XQ`>#OJ6rfcTIB7L#Z3gsF) zL_B4~uqkwcMyjYG#3WwfnGQRmc27hq;yWKo%8GcD(I8q2u_U9@4mY008NxUvnu@yH zKM=|BzMmIsqE7wqa|f@dtMGr8fe`g)_M-4Bu2TMa#jV;>+o5xCOEl2!M9D0h7x}ok zPO59rg}7D2O+72tR;bOwb4^uCL<2wT8=%3(6bE}SORHWj5H0;Yk%Wi&k{tEA_)Mrb z+y&EY}PURy#6ft*~D0&KY2PlLTUEjGdv7? zlk~2;emkCzfpD?T46F)RFZB25B-W4~AHUjfe#e z@7bW^k?wheD0juujjs2rC*(B?fW9b@S5lK=H$`Q${ErreR57gY^1N>0C==3n>&e4np^E^F{r()&;Yn}doz?=JU% zC~N<6(23(f?Ms&h%A`b)IGpa-oN*thog_WN&WO+Eh>b|Oq%l!Y%4aE(3RWCDKxJtT ziKEg5$sG(L@5y#eR)-Is}ykrjUu3>wr8UW*|9^@Wv6ks){*FVxxL>D-lQraLqFT7rU zRDIrX`e%|H?5EA8tXNgl&a1WQhB~7fAM%GCluq*;Uh$epC z$w{V_;mDdXQ<-B#p;AYu3v~PnlB(|$xv?mEF@EvZ>hg}~eqj7Z^B9x5oR>hxX^I|X z#U`4qA4&#KCLp*GfA1m8z%5+-kjT;{#DzyltXxd1n*{rDcxBU$JG$k9Qq&g9Zy$b% zyb02;&1?z-X;z55#IDE>x|WexuI&iV$p8DL_s%3CtZOsi++CZT704ib2`_0GNT2#)+4E06I*4Zp z05|uV94Cs4v*f0qY#w#`#* zktUo$Tvgwiu>*_{DJxw0eI&SKrHqs}X%6HJmVHi5>V5W6TN#Zxei2TTo9~1}7r|SS zvqFs`9_LLd#Z-U3Ve8Ug*}(m|XM;njAcA+nNQd$>IqWz(`_F#|Q?$k=s;5&sR*Swp zDd#^id>k{YJj|{-j5}apPW0C8&%&qR?r7aBsArpqz%W{rzSKPO-BTHilkP+_L*FF_ z_z%FI$>~~Su*f`R{)sF%(F9ZzqDRqeNbT5-b4y%n&Fy@@Z*#I$+i6xmFlZCqLA>)u61FOzj82^u(Vq z+=zs|t6(QN6k|NxIAzJ8Xm4F0FmP5e%eHOXw#_cvwrzCT zwryKe=e?MTxG!en%#YZ)Ggq$sR@5gfV_VOkzvin1hkP;5SP0UmfTM_F>MBgMe+4Q$ zg0{ps^K`T+O~%8Mse5!q3BV1q(Cov;Rc;6hjG-kV3RP4v$s}gQbo3O0t#WaW-&jv0zFH#6p`4Obd^NZ10f6B& z_#5?njU~?aJn?yb^NA!fsGGZrKi>}^l*aJ8+ z?gI>%)OgTZcG`kQ+q-pJ)}{;_ziJtai(RuRX-^3^&wUqcTDPjql}b@I1;0p^+pUwb zY2xZEsEuMrV>I4cJBB;5F|w@!{m-MQw7pfD^^$U(0;s03#c#py>*E$>XMJNvF~WLg z%L+NH4LsBjRTj%=4BU>gYoddA^d%|xxp`f48tQ{T&k*bN;*xHv82K=P~~DO)#NhX|az%7Pn00bEMpce&);Y-v@h`Qg!Wc#>l z*MuZ5QaTku&oXtuS0??z|NR`+Rp~rXa;9NU9@=PAYl*z9!FgmC!xd-97?o}RTe55o z@mk=9b^@MrK}6Yx-}fMRrmR3hH(F5-QZL4o0C_?k`;gdp`rxGiVVoR2jaqvqiAD)v zlR!x^x*;85f&HBsx?D{p4Wg1z;Co#U(jZP@?+;@Y2BNSetRgaKMT-uD-4^lfY%m&t zqYf3BLkOWW5iPnA|13pDiMbZ3>hUs#XUowEmzcXs4XLl)K~g3fL+)9(9zU7e@Oz8g zVwf5FB32vmpKMwWVF!V{tzer=DqoAP=y@>J@hTi*Lz7zZ`(GbC(dJ5y(hQu~?02ER zR~&2Og>lcfhw5%Tf|uQO&uXGzhn6)W7X62=rPP0S&@a({IZdjq;@t{!T(SPM=(Lfr z%A`F3DFpPJrmNR#!*i|{->Hay82TTHcDj$Z$ z-OZty!1U=sd!}OOiX@MNa5sbo=J6k~zRgEe1FBp7fQl!AV~l`?rIiwjyGkH`N(pIy z?~a$>vV?fTj{=e0@9FCu_*Flu@=I22D{BAdqKR17@;CJWe+SNU>$xPgQnhrd!|d~! zvF14Cr__+(nRq5cUzN8>8KWnv@?<%5-^CYa`EQ6r7AF2mz?;8^$>427WHnQ{SXS$K zY@h6Wy5p014z8%$JTxv56o*st=GefNbRE?CGfZ8cj$%m*FI|GWf3|v9n;_miF$Y@% z_n43_TPkppKib}Bc=&DUp*%)$&z>SYfM#{7!B^3-YfYDBJ|)*uG)YhYtODDYmvn{< zE|`p3ny+P`UUE|KiTJhGi+K(h-k7s$X}b%m!X#zHCJ5|Aqbx}(1j6!&RYVm#w@mdh zM1Y>tON`i9fGt`@DnpcMUev=|Ec8bc!H_e^Ee0x|^>VdW!Fb&60J4ZO`L7tCh- zLu~|=(My^}3D^8k3U9dzXytQX13`erW>%~7JHC1RB&1fOCEWAh_MK7K=&7>)Foy?y zwNY5LEf{9uBF8L&nTFqM>Hbm>rpV9_z0D{OBRBko1DwtLXl0`$!~2f2>p>g=DkU+b z*h-~sCrU0)hayhY;9u|h1v$rf<~lYhc>(-ERFlc7=!6M@mv@sj|Iqq9X*4k&IQp_A z=b5!^C(zgOzT%dvk2ULwlqyzu_u>6qSdVxxb^3)xz1=-oop9|@tfAyu_6|YrCMUND zP)PdASRyqAz;hf-fIes9o4>@QSLg)v5mI`3B70GTv`8E$E95x(=$h} z!&ks8tH&*?Q>a)+%T(E`w+nP<A9IE%+)E?d*k*1~ToK>sk)BXz*s|ursFji9zKh>f$M;VSl|^_yVxg`ZvR$#qk#{EN{Nx<4?XB&$PR-p-hG zQ97ucw2m#aewmw8znHtYYoJwmaq5m-sts_9uMBYzUkF?tR~o50ZbF~HD~d8Eu+hCz z0xo7rAzj+|3X1qJMeW`O9H8n@6jev9DRZ-bbPj%a#{`#J%u6E~$CIny6jqXcoGJ9) zy-{l)8HLude87QsqEONlcO$M8(OqSQ4ByFAh8n8z^XKQAmB__?Q9?H5WtlgPk}49z zm6J0(H`edBjJAXB3A8wH9_ldBLZ(ZIp9v40`cR=QDnfID^y%`nAp!Bp$-uEaS-!SLw~}2r(2sd8@i{5Is06 z>qO*~I3b!jd;bu((~S(2%Sf6KWyzrkZ8hcIZg)%9IE_ln(^)}5qrM1WzpAj_G8hJ+ zpT_Ops;GLHcei&}*mAD|C%!nd?zgg}{Q7t({Q>M`iutTv{ABZK+Zxc=d^Q_O{5X`- zRW{(sK!CKkPAIB*JZ60&VKaDb>Yd@T5TukXmLJHKC=07#X0-d5JsSVw9Ha>L4t1cq z?0|a`X`fo0ZtLb;yJnCg+cKt&+nYA-<`XGzZ`s5@P~?5*#3myXVK%!G#eDq_8(+Ru z|9@R+-~X67{(U2$Z|G$1V(#SrFRRP0BwO1%R)pT|dkUE2RK3bE{Sdp%afv%o6n=8l8S4# z?QZ^&FNt2{c3S-^XO`CPKRkhK=1!_lv zlqmlZLTxgKL7W+&_&Zd}U#M7NMKW%q$ktp(oqA9n9+NX))D~7#v{eOBq@XuQ?l3MyZo3{d7XP)sv>=D0EE3Mr#}Qkl zK=B&DrUTQXDuiiPNevBO^p75Kv-x7m&{C#@u3myO($Xe`!lky11~LYrpo%6va-<<4 zS3wEsXYk3m=Mkwej+QP%d{8?a@1~tlTn{I4_F%Tkv?!U~!Le4;#=-=@D00m8Lf{vNM@yxj(j~H@APTp>j zK@9Ca$zGD)h`~o}8)AhJ;{y;*oQ~uJY|D5bj)>2;2XHms_+jv}<}m8)^|V>(C5Qqu z;_cY`3Oq%*73r~fIp~Y){`}lyuLP%I4>_?vZy=c5+o1`81Y!`bm3*-a$r?#~wEoGy z#f=5$A|Ow1%{Iktcbp&!;tYsub;VmXV2I8;o+|?;c0Jq5FcW}~<88E9V2UF+wimdN zEbMq;i!$HI=neJ94|Q{*uc7y0@+m~L?}artlAwnREb~t#%9qw-Fgs148AfT{1Xp4> zb{BfOjS4#_oK(V^gkOBmlA*QdW9qC(x3yzrEk3qAmMKX>qN34w*x1V0>3g zW%j3ab5YO{uj|*&mw}qCt6n5RT{U-0-(lpS7)_NRtx5mOLt;DQSpPd(OV0Ay~GKm#DE8vGSp!P{uP0Ibyr|ooy zRX56d^?pjewKeUtaEqcClu5VFq}{p`!K_C4fH?1!Scf-zZ3+9dxO@XVJ6Gr>$;MTO zWJ4tXG5N()ujWw(5|u`#MkKj3)JN_{#?=csyRa4Po@wuh68VuR3XLwfXckBJY6oec zt+BDsLwbakQ%-ezMuiQUD_P)`(@gZ+=Erm7I9kGITD`;C zv*}+9)?xUOvT~Cx2i)K-H=M}UOJlb_y}=J`|7ot6l;i|({wY+X)c^NL>wkP~`u}Hr zY3g9>Z1*pfYEi@5cAXvRCzF`JklcVyt3`8%C)1vz*Ew-PZJ5=nq-)C{uZVPQsfxOd zBx!+r!?y=hfv!-9LwtQ;6B86GZopKBoe3xAAs}jQAWn2WizF*oa;w|@{#ey^`{RR2 zy|yM%N+FHZi0n}R=8YvZ&L@iaGw5pdyaC;mZY1;8n?xw_7YUNlFuNxHX!5*JeXS_# z4`jnethT^w+(W21=$EL2o>p&w6ODPI&}gz89~2)#VG<_*(y0iK+>{>|iBcGI7&$>a zJ#0Od0n*wvgA(p1oaydOC?0n{Ef z2nRqBL0Exzq9Z?mF`<8L8UVN465eWweq-Nvn!;6V)xOw;2T9alDnB5wbE*EI-f%zC zR-^~_EC-^WlTZneJE|-yOjKBh-?`?b0OpPqSMVK9ww546YermvLsC^xiuBogukdJM zIQ)0gSOsGCZ+=H`b0kr6Yz%T4tw2!iGS_JeQl5hG60z#z0h|t{tiN-RIH4pe)opmu zfRI))u+r@l1;uVcAXxWf9R?e#?RBAVw}{7AAJ6vfcH?g2C(s+<_o=QUGT~Ehytat( zhbx{NI#2;42JPD4giMx=*gu7QFlB2z{=Tn!?b4+9F>A)&UV1S;={a%XN?%nE&12ue zsmPd8F=B4Vr~-7PQzUb&tY{K#RJCAqg%0m_+=9Te>Uf{Uo!?qOjT~V4g&Of_RSp$& zjN0)ZxkkQPo%$oF0&wHX)}Z%}c03kW@Ysbg?0Nfv188*P%lwMiy@6{~i*>~lWAtLX zWu`7=NQW72;dWr@#NEnehGqnQ3ns#rl|$-%ec4hB^y3NgEm(A8748#lIJE%$!#!hJ zM~Uu>L)V3eNjoIT-1lhU5=metq{gI3U+ZvYj>BBr1{dQ0{}|{J1%|q zwYnM-7+9^C_N0E(JiX_PZy->gs=p9#_J@b>7yZF0p&{U3BQ(iAiTYqHHAZ#8qBJRC zKEtIy=E)$OcE{31#8+JNxRfHRJCyKo-{o`BNLLFb}GVwf+P{mv^2T+yYY( za@-E80?|tA+j~YHN%804cD+l47|8!?Ae)&T;Adg!bX|vP(;9QMW?B25bC;PRLA$K7 zuJZekeGQ5>$^qNT6|*0*HV7FYq6$0PB=p!5=5EPXLuyer$D^C)$-%=zwji~-0Hg7? zr$5$AJ8N!6cOC88`9@Uy?XdOV{7txdeZ!S0%3O@(~8fkY}HyYCADf<0wNzRv_ zf+H1-DhDqw-8UZphGeN4h4_k~;rl`rUs7c(9}o(y67~Ww6(CO$=BlZ{;-ryiUSctW zUWjNXeGmwJjQJ4c8JO_6)6;Jp9~ zKVlV&>jj-+i$UuU1rxeTz6YsJ zAyE2>*IDd3TnJj*xscobxTrWox+m8B`uxrTg#B?d=jku%RRM_ILAtg5=sJD6FwSwI ztrHrGbL(34fmncRl&EddvHdb7A1w_HOtBRg5kfoKcp~(RMjiIMR+V%pYc(<8K54w2 z^oolTb9ub9zDt}dFc&oJD~gwbIRxsfzHq>2b4LG8sjWlzHo~H$`=p@OCFkWbNF7bJ zpaQ&K{%sFRUMnRboLJ|~!X$g_6vNmv!o@D_v{Gknns20Qrk(+^5N~F16jQ)#)UC-O z!GKL=gWS5C67)8Yn0PMqiYlR0{UR?c$H3zt>AbY zcu9(9bK=DM4Le5+(=HVr;@~u@5$eeU2^Z_dyz3ysgUWoB+DEjJ?Ht(bS43%csPR<0 zepUhp4arS5555R5Qx9WTz|3Aoto0$m%HdzFS@7`7dHN?IMtT;om_^u1m%5mdL^zfi zNVguj~kCY>u9L!-(o_ zVV8ux4Wx43pggJKaV95nXJ|g3;zsSs>7R+cj!WJy*So#~dnB89=9&RRtA_S&Y}Ds% zzBW15nYuZ*o+N1>T-j<{wzUzGN=SwRMGoh?{6(svjR-du^@iS=f!OUaY|I9*3h{(4 zT~u<=h%|@NeO6wxHX+Pn)PlV+GOayJYHv|MQToawutcqmagz`Mw~5T88Udvpy~Ei> z14YZ)i?M`t320+|3w=Du-O>^G+`A}ReM}IzXl^sDGQgoNrX6Ws@Gf61(d6Hyba>9N zR9>Vgq{;#i6=rqGZA#=Kb9Pn2w4Ss#Ewa}zQdWq%i|D6uC_r#mGHUH?x@y`K=#B0i&0lV?=qIt zV%B0BU-NfgEcas3bKO;{WmS1IJ>vKLWx0{uYe_-fCJZk(7kThYiu|D$7*EqUh0cK2 zXL+6@V8p_%M_Ul>7ERAbl3SGuo0`G%7~Sv56mr!rq=7sK!FoZ6(*~Ueb!7G4>_m%%pE{e2iVc8N>^VNec@BeFi}ElM9o?CQaNDceM4# z^W;2|q0Q2~69-!&0%TAVlRZKp>1!zw9TX3MMa4Ax-Om52$M$bt+{xN%P&)!sfNk>C zfJcw1a!o&5lB(h4pj~0!>U_aSPq@t#StyqtZ9sV@??1Aws{~mh=(k)~q+?C;BYQr* zCfi4*+d2yB2#S8dnNN4i4hHwXz+0Z6JOl;M(3SLD7Jw00aUy5mtqL?u>-WU?Zneru<{y(_5ke+)gkt!8X?U?GTb(I#fB7_KMLVocTooi^ua6vC9hWad( zOwS+qSjhs{g$xQT1h7g1k&{$bEVh;8T;xp%xe_B()h+$N6@F|rwdiX* zU`=&eto5rvq6#q_u}dltjOC4VTF{B;z0vY*2Vy}2hnF(U$(j*rV;UG3YZ5zO6%Vb) z76(A60m}r~2as->0qEvbiyAZiYBY}dZscR<&AnW)cfU#xW$+3RdeaOc*0CxB9Uk>c zpw#DlcO%wYGBnc10tM1_5rY9)s|G;@q1(*N$d(){m@(v+h2m0sDqa%{=H&{x)h!X7 zYR83r%ND9N5!02M+xvR^FDX&dj}5+v9VD4Z=#-@?7SDn&rj!-M$|pD41SjPB*v`~j z`{B^@Rd2t`?N0o3QSHcQ>^ZAh8CZ{Rq6GQ9Vuu1TSbj;Bo>i2F5!`%VQ zxyJE6NdS`j8N++(%;|%vcn5Ej!2eKZ3|GL&{c}?Satrfu0{LY$yAZfjV~kWt5x)nr zVt`o>V5-gxsj7&^-OGIywFe{6tcPjPuoBGGuNI4BgGiFV0r%;N7Ozk3d8ap24>3wF zSPL)7JGx?o`UXj@I?5;mEM*XJ5+>XLLJS&7FgApNc}(yU0t`~*$P1&*=Nqg{Vu!Ey z>&qJhr&eu<pF9J^V1Z0#NMHJauGy}^_)~Y0b z`j+bXQ1#o-QM)0nnndEPrEBt!tP@l%VdTax)3Kv-nKG%r=}wzLS$4Iv-Gynm1XQ+74^NvW!|q5S@2NCSRYg+^{L{NH z-@j1WWKvqhjceWj#9HSAPpp%n7cG%OWlzMuX0?CZzSJu|qrvAzFD!@Q)2fusrz}6D zcxEo&)NixEx2gU!ZzeH^CvBIfwOk7ibIk%AZ1kA}hn zsSH6@N6p`e@jh%lZpC?0#{9E}=_lck<8PreqU%?hbrW(4Q0R9h`5j~+D=K7}B=Y9+Q2W)?x#mjkbbAf5mskFiqYZDX~- zSV}Q4LkU3HU)02l|5A5}2x6HMiGrf&O?iaCRdvtHtk4>;$C;fg4(U5&C(0F#`Sy$K zcerINM9W9G#V{06p_cz-FimP)rMOcYDLf&2U6JvJ8H^ah^~{Y_F0KL~HU5L-BD9p z4s68d?dNW65WmK|@RrsbPufb4@zW3eBO2Z~Z1y&-s?iNiH%mY5n#M7t2}j@{&DO66 zIJ_)u+Ze$6C0)^jB~3cOv<;wI!7E(fOKJS07}+cQk9Xyr07ND~9glHMeC9T2*EYOB z6E)&1bT@$9ovF0xCV^wX&YRI_=wx;_#}+?OwqnO&&8U$Hb@rSYtei@|t!0BP92&sP zVNO*#pmt#2`KA_){N4FQhs%L|h>yGF$6Zl6JJG+lT>yf1=QU^$0sKKsRItEgDK`Fr zfHuc~LN~93cG#XK)lA^bQOr1OgTq~0G64LBrOJaby-~zRoj$ZKEn2q>D0x7^ z=fQrIf?*=~!_2EOd6g*AZ~C7gNNbnZ|h zd0?63N?ECCRkPiIx-5xTUCUJ^=syUCTKx%@!E`i7{6H!8%VSy!31pb%`i6u zas`XaOas(rp%E=F7vlePMfN?2?8qX~>d>=)|GwsbLiciG%aBcBm8i}QN!Hy9JN5aK zYXOXIO%bJstOpG(e`=?St$j--QZVr4e>{1>scDg)>JfeO7H!f<)gP9J3nr0y>b8oz zJy?t6?BKRQG{9sCc92@j6D$D@3*=6`oQ}4bcJ^-O3My~w2^{LrQGb%6VYCti(`Uu# zo7F+hR8=nNPU1`Q$!J9AuaqV=s(Q&#e|ON73Kvy1&WQ~Jb`-d@lghnmkA5<%pcDRF zH&5ofSkSRH5Gft|{l+05FWh6&|&o0t(8F?V ztNoP0xMI9Jj!((6RCWJ`=GhvzTTxN*)**<5s~4q?ylY>BmYNIx+YdRn*=(r+ftrSm zzHZUUdSKw3kAytRkZeSmr}AX-NO0^cLve4Cl4XkF1Fkoc{&y`3t6V%xW-wdOI215& zwxD}=rN6pW&fXyHFakf|rG72vzLzj$Mfl{eEfWP|gsTVQtA5XraO4TNnqMHMleACa z2t#e%LrNWmD~qZlbFPEFE3;?g*4CE7xb6m4(yo>uQQ9zoJbMSY@i!DkklpX}pP)1Z zdnw9IP{UK<%^>BK$q=+zCbEUcFUr^6OUmMUlK_7T}*p*xqQA091YLIv@%?m`2$%m*fpB99_l{U(9G1}h$aQeR@zsG zvAwZdq@5ovUcBrsp1heoIN}yAJ${!)sA`*yb`v&ry^N@y*Qf zqeItNAD+)Bo)qScrfyjnFKCN)<+W6*iBcEjZU4bN_0mS4qb+4)Z|X99_!iS0%V2+A zAqC)FUGSD-T%IBIH=3bXnzR(mi>B*74BhY~ATV*voD4M38h|%wnz^(~{(2LpY6@&j zzamWpH%nwXNx(%Q)%itM?zU>oj{?~BLS)<~eDL;)!j7b%ORtNB1q@3ex z4lMwh`<2M`OT^#Rp7at=c_m%;eDQ^YpcqoQ@!iMoG-%>M=2kKG5osx1e}ZR@InW-k zrw-Iq4xK#pp7%c|j$X>J6J0O>fGSh~0J4AW5$tTO%nko_bfs9%gS4$8`LgF3ygLy;~i2M#~2!P1Z|##=I{QrYd*1k-fq)fE*+#AHKM zG8JrAF7)+w_vhi{d&~7!!0@*|J(5R8uf+M}&V|&T1aEPYPyfn7(*7Ad4x z6o;%vjoe{4*c4-YAn*t5PY!QM#~r zBDFuEf?uELF@K*}T+op(RXk z=HO2n*06pHsBD~0coo=aQnEZBIX4Q_o^==(f-xC_=s9_V0KhMZ2)!1lPl9T0wMcr6 znePqumer|!u__M)kuz9@sFp`#qGnMQ#_0+_2P;Zw~E5qZdt*@J&V!-8&2NT*3|<*+IU`;DxEm_^R&uai8viN8mPlf%skaIlj zPG4w=21tQ4?F`+!nL2UR@isO**b>U3NoWz)oHr!lX;ux_AK%R|mwUSh;6c)LGz;Pw z%r^7U@iNy({{F^arJ!zSz)_!S1xH_6X(wXElakwFGMEZWC7AkbvPMJP?DrMiPHPqN zVz>8(=k+{EIX)hHo?5u_Kjct^>MLTyM1}xGbp8bf#-hzRmB%yEFK1a{mzyBTxZYxhok99#w$qD zrG3EylmQN~*5KDNu0CZ4ry1avK)2s2Oi5VwFFanzdALl^CV9bP3o0h) z9K3bIZ>s3P@8)P0jVgY-4}7Qe>$Q!hiDWiZbGfWlQyHQF)qQAl&bvVY*zNy5Q>1KHEn7bU-hY1 z@OKeEgtCp$#DR+Q;mKExw`lnqztX`DSXjvEQF2Y@2o5uS-`+TmUgUXsL}~CS_n|5$ zZrHM{UGeUNv{MGNNuP7py$|fTZ=2V-f6N?r5vfm1yGpt0Q&-^Uva>0aw;)j}JcLM6 z`o@TaIU)sLW!7WZDzydCxyk8y1~}_GU-sPvbt{tqlaZ0R>tS*d+tqx1^8>3Vd0iO*h54^es1E)bX@NN|>(8m%7eUNd- zs8*{#kYO~!M$_x!df4gv$^iYXOaCVt+)I8>Q~WyoCseC}3*=!T>?U80tQ%Q8knr}` z#gz{J656&%2ZXM8-R6b*hyWWE4mPv8IZ_AlbOnE3XgT1d0s5TbX&%~I=oMZ#Ul==z zta!nx!Tg_@0U4npnQ6|&!IV`2y`4#Km~&QE_>-&wwB%jGwH-joxjZ1Nyf)^(Rk1IO z@T&50Ud8EBq+y4)WZN)v?dO!0GHZ=~S{B9E6YKI~)pc%wxZ>98^lH|Mi{pyoF1i_* z#z-G&?-nnyvR8t)qy56)jxwm!Wp20G^5rZaXK3J&n3kSoudbhCbv}rt7B;$fUQD3d zZZWCH!@Gum$?-V(6h6^C&5JNU(n`=`YAdlO060OEA4V7&6Q}%BG(9|U+}e}m8F>bo z3LvZMfbBb->%n!H9_*G?qf=fxQF|gBA0VCdXp3l%)8O4FSa@ghnpm$(`Wu*oief5B zz&Uqx|MUO4L~Qh-_)*MYfBrD?fBnB4jP0F`9i9GVymd~TwORiS6LNizA{0qTvr{y1 zrIL$olTY1mf5GRn^hjxE)xNZ&#u49{e|!pGW+c64E#k*FH~oi~Vm^hQ}L)vP=e5`ReCt0o_UD2CLZ%Z(Ew5*nP0|}x0a%_t` z{zTmXnd=7zXkYWeTPIwW^UQ%v6i#jkps|Ra;9&n5hWjB-Ig`M+C-*r6Mxz zCG(1*HHMxUcSA<^7y{a;6^I5hdl@PXlt)f~hoe?5StK0>NKHUwjX715C#XdtXXbj ze@zQ#C&Y41@1H(Qm@Pds(>UF~z2-x9G47Xi^X3Hdu&~l=$n$^Bm_&u;8*p5B#wfm_ zW#PsP{@Ri};g^qg=GX+AO6)AM3>fA5Rl9q?R&6s)Ov z%oqH+z_GqR{xh`Tc|K+N!VQk@Or)>-9e&?%0rmkc%Ek}k>RRJt%*~Va-3IOdQr~oR380lv}Pw^W90D$sepW;7BCq~Xz#zz107^{?LZO+&cypMmx zYXo5l^XG2uLY_hdiN+u8+N|Y0Kxz-ACjN)~vHs-qIhESTC)q)VP3X^UIyJ$87wVWg za-cI6k>d!}mEI+seo780(pAln;DF(3kA*(~;x zWs;q?2Qlq$PQ%m}i|CNng!+?ZqEv@Kx@@Q4jB`I7PkU@-27Q>BL|}2Se}fias^47y z-(e7NdBa^U*$#D(L%5 zThixJ#QtQN$WJFJ<+`gQMfQGG9eMvj6HLOYUm}>DqSgF#r>Za%HdC1`$AWguNRnip zUit_%4RL|c8Hv}RJY-t^C(5Ky`p+d{ahM+VYow-rhk0pHU$$%4OgQs<;FW;KxZK|m z*RNZgv>98MTq$|{wH;;%t|$>o&4+@$+3N5%aB*fy{EZc0UVmep&38pxC{vq&^B?Ej zhZA0v0-w;u&z+(Wlih&m%RGDuoSQF=A)R*C**oPG*M_GLSzQA zdgfG24^BMBmlqrafpg`(%l9-78$YW9=3@MQ(Jq}DIrcmb9Tj3qoahL+Pw(j4%D?9@ z;vxHM$y~ZDC>@IR)>@vpbcMJZyRG0a2c3S8Sh`3iUERHsiZh!PmNt28d6n+qg`s-| z-WWD;wZjA+zWS+me}E?fmZcWS22|$D_@Ux z2C)}MdF3iv`#j+TYpQUb9|7Jx2R&svSt3P!zLu+2R=AAFU|i?sl5CwwStvCQ(?^K_*iRlc0~EsQOi+S{3XDDelX|xowUf zwN!lwSuxZQOl35M=kqg_>qDW&=C)TW$UWPIt-4$Ih7%z*5_}<$K65QKa+_X=C4BF+ zV;DpU;a++LTlSV0UpJ;Ic3x8~CxBv>3-bkXJk&)%nPo*(wma=D-{i!s_P$a=hzZOH#Lon~cZtY=RO(qey)2|t- zVw@8lk^{+O{|xLRS_P|*kNNGbk;frOqDRMfN-)3~abO%65I^aJ@wF|)HP5Aj53U7A zO9_kkwnJViJP_=H5;Ej?R@NM;$r>SI^h~~@6f_aak|r$? z#8IU@c59DIGZ9m(sFVTWIz~#6qD{Yj1d@eZ#k&7;grZ2%q;w^URFbo4H_gB(0M#60 zvq5&eBQNYr)uvvEyIVNXTO9o-Lb9g;9SJ5G?v)dx&OHjg2)%N`ZSnyr!ISx=veyvZJhmvei<8$?X0hT(hJ zPaj1#0wYNBq<hakMymN!mc5;Z+{TFPYd%=_boIlL@)e4k{(PP0phNc1MG>s>izMI5jqbE z`o*607ZI-`RTTxluV$C)Bmh6yk+P3IJtjOTA}qD6i2s1Zj!S9B1b-%Iitc^~Wt!kd zbVMXc#BFnaHwPLZ}*PDDLWV@%Xb5CytN=~K{poh^RyB25g zwa`d;&TCA9lpGfNJe;j~N4EQ1y=w1IU`XshMVgv{>4BFPD^(KpDiqsTK$`GWIg$GF z4*yuDAw68HHndi8`$71K*!Vh&D3P#La#(eMtu>P7E|`CkG3^FMM1WBa!OZNwb~0vY z3kFV^kiCY*V=tULozPi*t^;noo*L<8c@P!axD%qe%V9TA{fH~OAL%6S8?PFGIzTt! zg%)-l2WikTn)_?3>-}uIxqCF;+P6Y>pMn4SIyIVW>m;K_gmB-b#HFl}zbDLkK-G*UqEzzo2&cJHj5}y&mlX1dJtbKKN@nuhpBD%w0=a zCdh6&;68b#DY+?}DgI=o$hglsFQ;7MIxl%#D2HmQn@2F+@ zD!<$wYd>gFW{2aag1|;Rv3fNejCDwJr=!Uyu#?_Qs(&lgJh=YNUVh%zYp%Wacd4!6 zGpCLTdyD_w;HYTVXw{6F;bm>lvhF4Fn{x%fBoK%FbD7zR_V9ZsJ}!Tq?6r{VI7C<7 z)bxDAaR}P0>#W1};l}eAqU{y@KWjSdvCoCX&!F~=^#8Uu7+BdFS{fOd2C;n~@l z+x%N$uTqu&$KK#`U4zl20obm!w%H|PK*K(6bdtL)VNb(l14Kn+J##4=#&5U>YU}c#t>vM zg$zTvWw-jXqkIyqWj@k2*A@mbD3C|4K6nD2Zv~1}hk{K7qSq=9(xHF>{%X;uFk~RI zy#bO^RU9q{4Q-I;BTCLpuWimH=Y(*Z$jp!}nHIPwE?5utUTfhZd2 zCgg`LJg7Z=4U6}l?F-IUdiqBI!%dJ*XvNM*%BGjgRmx6~?PPO*uI&~~C>7nOX=e%A zTPQ>m#AT-sQlRFU-44jj4F*rd5IyUJxbgb61yI8wv>JcS4D-3yyB`!N`Z}wx7sCmU zAp*cg>)~%3W-BOl$SgMd)=WS*lhf5m27m}TJ_duW9v6S^vFGvyQD>lU*9`XSfxGQN zC?G8j3^j!7+Vs{+Q!ivEgS{PgR0tYFkXG*0NN;ZOH5&V0yH6syprt>CEl#D^zk!8_ z%jLT$W!T)&NY!fX5=Kr$b7~kA`&*(~ibjWJo?&hreSB=XfwT2V4Y&<`>N08ChLnbl zCjI*o0cs;4`?k!}@ELOYi~?jsv!&M5B$&pnmU&Y5DcC@LF2@tu#BQwm>YUUW#b@sr z%9LcF@T+ar^V3O|82i}FiS`T;D2>xJ#JnXyw$qb-yNorO%o^km+OrvE6SyO%$uOf@ zSY%fh&{EyX`Zu=0H-vJnky_em=hf0Gk->qUV>|ES`jWT2Yoo6sh7omaJI6rUtOn(YK*e`B!x#*AFN5UH&qd-`>WND1Zt_HCR3%v z=|d(Va7AYTEK#f95(5~34IXN>KuL79E>@BD6F!2pAn%Gp0#-GRJVuc6RA=bHG48Re zjHe7%%mgV)CpY)eFFAMbRmm`Iu%f5$R~%1{Lck&$cD9RT?wNpY`2m`{fN2PCu@G4) z-X|gpD}>~y9eM!e^iw!ECx#doUjgPn;Pq?ip;ki#q@B{$|JY;UTz+)hEfh_34t#mwaWRCL+vPF4p?;ON6mGv&BJt6l8v6? zE}GjDz-sLJQ{jq+R=o`?^p z7VWbN&CuG8DVx`dWku0Z>o(Vtwys}Y3(Srp(m&D`nB8qWCZ!r&a`1g`bxh-%`IhynlzW4g`=IZp!Z+KxFvyVT8{O`VJW)IzIa0<`` zE1q-_uJ6c=K}X|YVJW~klo5*jGO9%3WG}pV*{PL^8ETQBKF40ad~{kg7Rk%nU8yM5 zC^`2t$jPfbLoiBb-wNTVR=-eMc90vPnO-N)lsGqVF2x#{S+C zGVMF~m`AjlFIYE6X6*`s<&5!{%o4!XkkPQX+{33`u|<(y*ROlk^~CTL=oi4z?XZJQqv?FU6j`u`X}44iF@tc?FZeR$DHn*Tsd{kC6GMax92Nm;Y9h9$gn zA+8qC(m(sT#{%?Be(sgZI;}W%NUTl;50-yT z49mKjApwONW-G}_$eSrgN2#v5P71@3&Q}1HOT8UCOfs@bHAiatTuywxhAKA9*sJYq z7*Tf1m4Vj5Is@?F^uR;804 z@87j8>hI-z>$|Lt;ZE-IAMan6hO9)UNYS4J8fuCLGoLSwSh8wPzDe0kuH;e6{q<;Z z$%{wh(L;uCbHYT6)LxGUM6v)eyf3ofelVnr0}taNn2G!nQ$MWs=fOwxLTrz}w4Y*w z6|jK4sC5MtGBM`(HxeTL-9!fzx-v5FK#69!A~X;dN@}B}@y<)bVFft5>#-zrUavSZ zCUDZTDp=OUP{~m2Kt2r*M`?kdZ4{ zkmQLP;v6oF3?I1Ie+i@PfiqG>R1DC?9@kbNt%BH+LpDBEPG8)hP^Zz?V9vu^Bl)v z06*>zTNp)SA~Pm#~LoH zR1+*w__aLBzksaHQ2Qk9eK&xsjJ+3MLau0G3^MpisM z4RQ|;{|i@4Ctn|=Jo#ksE7NKoj@_A6?1!X@uHlWDeWZUD{EWmlLceXEkPc0v`|?kn z!M;))IUht;KWy7|z?v|J$WJ+s6Fu9oVY{$I;^VrW{4^)*#(INus}D`PHK(0tcyza9 z@S6HWf_>{4GoEqzO?)%}aX=&dmDWwQc`w(ijsVdA{(;;!0EgG|)Aaa<@c&hFJb#GE zrcQ2#E|&Io|4a8YMcwcxOa$rcTHjGleH~k;1wIlO)&MTDh;7hjB0j}TQyZ&UbOVQM zfwZD{xBk2Lmvm#vFf$IRr0m`GPM7-_>=VwYQB%s9RgEE1&fvz!OJDbhe%~P5$^s5Q zvnEriVa?O==T4VuOLMSJ&qdF5PbhYpXQc`aj3jz>$I%phH5n~iTv)v1tPmMeO@_`L zu?2AT`e3L`2!c_CrdII?3>Jylc#zE#PC&BcBWM!djIn`q1eIwxR73b=HfE+}WO^~S zI@7#$EP(Q1^WPW6I#xWX$+5z9c+YFn`x`7V%`InP}<@IX_Xiu#_D z`-U3_H?o=7k{v|VDdQB;I>DV)xFJy)11lZ3P9xug;#i+|@iBNBH37LqmN$Mof$AXH z>Gw%#II&iJN&5*!LnSs$L4xy02uv}(x0eyWJ|MS`_ zO~e6^4nFH0E9nb#?ZV-R?|trdI5AUl;S&h^`1pxLalH(W%<(8Ujx+6g494jic*o9g z1^e@I9e;O_YPaV_OxlGFaLwG1IuElV@cS<*(%+mRrF2M|PrTHD#hprHF}1LMLZKh9 zF`?M{S}9Rc=X(d}K-wMRBYc=0X*6Vwc^h`lkQlBiz9QDa_KSQBrne&u0|ndGVm%{o zIRMcjYcJt%>o0>qX(F;eAIQ6H*W1{E`0dV%`!u&@BFa!Lf`ohh`DG_6T*VG;NEX5u zSshK3E@$hgqNEiIJ+^jFUY7Evp(XQ!2Md~v3F{4JB?l!T0=U%mX^$9IQ{1^pM(kC2 zMk5X`)$KL}%|K3S_2~AWQ0619btf3hhRBF*xR)Ks-e)I-bScJ!&tZaltT_BnFE>wx zkZ}TEr$H$XT=dQ?G0&x^?rzhF?sGh?gfCT4j{F37D_E>C#ixE3Pla%h_&Uj3{1K&d~~Ue@3~Fp$%qXC)ArGev=J!43SNtXR?)B3QIKm+7*1|>vI#4Dxh_nN)w`> zm|m#k&qhsEw+5D`$$EwXcz#YN3r(KbB!Q(-%%EJ|~=YfV=~p_zHT|i^t07>)vcxTPiZ#Yhp8= zViRe-Eo<-+YKUmHw3P-cOP4y%sKkW{d$P!CYw?uFKy78$u?o88Uo13;ldlZHj0725 z-hZUldLbhK+d;qF*wL-yz~G}h7^Lde=(hl)j1r4=pBOnPmWCRmxc6q0q1}f8asU3R zW3ExL2FH{dtk^dZ|G2V}3>rVwHM449C3I=T=X2)pE5?rX)~M==wkxvZ)F6||p>158 z|JDj#!LRTA26u(|AcX`l7&~9o3Y&a>S}$-)M#TS8Jxy(s@!S)fA*DU08SG-T2NQ4# z#>t}`8{f0VL$2ax{>8F$&i9ATt#La7Wi|S`#~Myog>z^0t9$QPs+wl8SAZnfoJ71y z;Mu?|n^d21Q2NZI2JB2@j_cIZ3Fu+X%-u?rx*(mLYU-nzusuY{FelB(Upbw<5_J0~ zx|K*(V#|**&Q(nisO2zDCJ!9Dbu;w^(-5!z0$z{_g8Dg5@>xk-Y@37#nhQomD8)X< zc)LE}_6X0H{ufr4UgzTz#W*M48uVhZI^C>+ijbV~fBVU^c79Q$--5X8ngsbe1^2Rd zJ^J+P)e97bqq8goFMq!O^G^7G=U@L%5hKPn_O2%GhAzey|33iOnE>vg+8+S6i2naP zulp|t|DO`Dd9CenB@*|ZXy5U`$WSv=d)oK=L-^-WOi>f4@TJBuu|Rf?*d{a)&D)n) zy=)S{eN|WHJQG}!@j$jwLfPZa|KssW+N*ysc}WhvkQ?2cLipqC^*%JkSyrdtbit%; zxBe@4G~3)J$+QblrEcxICA0Wcm1Hm9YuHAz6RI34H7z{BL4qDd*(&du9g#n!R)IvP z+3+v4%zD98bI{{+*o`EGiAd9gH+slphRZ+;1RpBP7AQehZ39#R+#fPpCWB4{0ZTKU z?x7}}HTQ3H6%$HgOH+*h9b>82bQFRd8WceT6NGnQhWaJ(G>24fgUUD^d|G%3tl$(C zqCY~v5m2E6?ztusFta4drp_Qw{3^p1g;4uoUm!-?6s8gtr!GeDJ$Zg0G=?VxBU!IT zB$mXWmmWEF5=kVI4AkRNCVmW95bITZ3pSDRW#I-P$3$y8iNPw3f^oXWd^r*D@j~`D z(iB`C(JM_vd@+@$pc`Wn7nGxIqe2!JC{(ZHH8{O2v`DszU{fKuyU7WMR$n46LX-y7 zi~uqa#XjeYS}F-X3#cWOA*-8|bW*DN_HWT{!p5_if0u^_EG9+NT{+vIY@Jwo^_W4_ z+kK~vaYY*+S%|C?PFET?h_B z{ArN<`2{8oVP6*tLr-`3X~ooe-5B*_L&8^PHRJ>mAg*6PR|fIm?d-k>tDT)b6bO$jTlM}CsNBqEu4-iuKHxoXO z7eiUaa@_C^x zd^~hF0&>s(&7lX*9Mn$0xDvEf!fw%UjdDC-V9YYkY+|tt03D>zE;eC=0uVnSEZ1x| zd3_P(Wf);FCI!JNwM@hiN1BE<)Fsng401+>Rfd58q4>DbDONanj)7!IwfB-Kz6Y+j zv<0t3W4+J3&Mfd8Ub02@@Q>>R#02*Y_1%bMke@A4p>CN)1JAuO02AO=V;%=^QlMLF zh+@^DfU*}*l6p85<`L~rM0mw=tz&4;cpnC;czx1Ck$Xn#H?wJTWWqa+}I1n;tm`E>YtZ3>sYwXC4!y%jDL zxMMlG-$QTsdUZsGL|f{n*7{3&_%3*V&z*!cW%|Yw>kd3y5*~_!xI{#Q&!PtvXH%i6 zEaxI%D$Zmy|d)B1Bo_QB7=v}SnEiMi;utgIgjM~a3eAEEbP6RBgx@Wlua{5 zrwFkZV+mQj&biPH67j_w0a>0SuWjH^KfwB%=)@aZOKG+*yv!3T`|oqri87rKS@+tx zZc2B&!GlWn%px8rA`=4+qK9^z0^n)8rH19U z_F`UQ0TewAUMtsjR;x~H*OBD}L4b502}}veLkSF2$6o{SQHzd1dW-;1#Lbq|Uk>&d zM!?<=n=<3{Tb#!#$v-BrKZ!(~_ZjJLu_YE*J_nKX3|#njglRhrv9I#q!jaIzvEQR0tK`b&L2qxMqUrBLY=rYecXwH4eUnf=jO9~}+ zF(@bnrQZsh(3;Jgm$PZz9P2;dX@TLXG>Z&&X&0M16zI(YdIMAMphM@;H>mD`D}5;3 zN^g)x-(I{d*L|%bkz)C3Vt-9j-%bWzwz?z=eHVrGTJcnK>4W=t8{E3yR$a6H<63yB z`{iym(L7J;@EM%6LPE@9#TdUTRlMM)EZ?e)3ow?I;A2SglPatm1q zi;T`kVUNhJcnDbdv(4JsP8+U=1$m$}6Vyd+G=G{O$G*ryA}9>H{z7i23O#7tIBi7+ zZT_{`Gq!T zz(ggC7e-5C|G;3sZi=oE?_t%HJAe#nX6OFioA8M@9qC9_J-C2KW~RvuT}>t~ayjcSaEsMJj>sf&1kzzWd^9uGWF(?V;Fyw|)kohhHN@ zZ*>l900i#Zc2h3RgRsfW8PFVatBOlZ{2cx$zhBg>>voPQ#srlLH5mcc1;yq)iWGt$ zspeNCxRb9+{RsI0AfjOQy{HLjOLRhR?5E>jr|(Tyl&cc};KVYB_~y_{-G?!UEPG3I z1M1e}XW9n;zEfl7HvKTuNoM;$xkrv>Oj2IWDsjCiSw#}7i-B5cm;Bj1W;IFI}C%T{uconsE*lXqyy|qJ!+&J^%XVRP@GH-cjBZhY(T-ChFl^2N{!ciw`_Kx zjq*}|3H5#lm`}RkS#v+r5)YwOKeekGzR}Ei)yUs{CTfmY^n2+M=zhdhJ6a|C_CMx4 zYb=btsU?OP&-h}R-)&P{AFd*$oKoYrjLr{SN(n-c&m5PVcc!y1t9vc~6)l=%nZVbN zEOeyF>IK!PUa$$zWNwHm?g^%XjIu2jN*G874-(iSLkFb#2ki5hmzu8pn8})4GJ$<) zma4_hcPn}GJ!1;k)jS@;huruKztA4BSJ4a;ILN~}z+hWTA&CFTUG$K5+e9h1_%Edd zEc|u&8mbVI2^;PWtD=Gx5?~_+!|~<1Le2?*Nr>r|@>bGo-MZmVq`j@|b|ZwiJ3uFW zD6}4=5k}**NppC z*<>i@wBq0GygWh=%@A3bW`K0$ZK_9Gm)#^Bq`uSmyCi}o<~lu@uxci&>8Lr=eZMT8 zSJ7WKT@3=oAccu?Cj_3iLNB+@3lvsGzVpEI#wj06`{aT3^-_C#Xo(>_<69W~%T^+W z^S}qZl<9lDsksngdBXrRA!uCjfEPuTO{l)@A~ zgSE+B5nyjH4;4a>vv@@wQCG=uzJM-3zXPy@VtyM@n#Q+lY0~d(XglY`Ymo$tUkVTy zt_2s-3uRe1Yn-7jWzn=^KZtraacsT&nUvDegW}c{cIP()ULXRdEHu(24U&4+Y}fqX z%LAvYq~*WId(HKK*?Ef;O#f_u`_%SfuD+*` ze4ds^ix**v09(Lj1C7{AC`^T3e&9zdT&2ptx$2e)p3F`upr9^8;1E({#A%d&k{pD3 z^!myxz&vtwt$gTYL)TzW_pR`}=LZxnxFk?}t_RDo*j5tKfY%X^_>u*M>5ysz)t#$! zQ7KoV$DH>NFyYe>Gk<$}SxpfAYP`iGbTF>3Z!-s#M8d|cNv<-ZLviiF*6Ip_dkz+t z6FWAeQ0})MJm#q`B)gk(BoO9mKJ>;nHPoMx5ilBqML{y`ohP0cqJh2Dcp zCZrt1v}1;)hJXRrvah7Ghfel(&^`K!FzJUNE2^|YW{R~7CX1UQ7$`ez#*ROrDoob~ zDTtKNvL`(ITmHiB(Ys+~eI{Rh9!V^xk7Z~j42WdSTDW1J6R6DR2T^b8(W+S4s)!X7 zm&sRBKvEWP_&09pYnTv-Nw4@&W&lbLQU~w@Ly!S=8|4CC!C0oR-i4(7aDl|<>C+Uh zu#R)+w0FO*3|s zM)iYW(f)2t$v$-LgJ@RExM#)k7zCd}ohkH%)myFu0rHD7^UOhY9iY|7iwJ#ozYiPL zg!{XXnzJCR$PAUG6kgb^#dXta=_E}X4a}3MVjVh? zmTP?iEqVjt&**~6rW2^`#myRrI5Uk{(S*&otfp#|RWE~h%~}imx=Kk}O(9rLvsWnv9LdRmy~V!RK7Eb2BM=5NAle1Q0YKQBlxyw1r_Me=7*-$Zm@TITa=g}#RXuhc(APK&HzCtAk*58OT zOr~3Qb!^sf{>tuw-Fbp5v~>RBn5Kj2tTs|a3KrB|^l*hXyirzk5~iev8H(svx3cBW zxlSNo;HM#~SLs4d^>~3=Qxw>xt70?!8=Um>Tum3_S+vdq&Cim!cde#Z_zefd#rqb5p!sY9*ERug9P;sJ(V#t+ zBH`sPzvQ&U7NyJeybKDj;+vap%P1>70rlL#}+k`U8RK&oiQ4X;q4I` zak1AQ7{a#9(Bv3r)b2-V+sPr?&$X1#C+TeZHgz$-6dR8YX+JgndI9G&y@K+~dLl)J zt(q-f=Mlm=%MQ%5gUgYt8JFk5n{K#Z9bq!s<3~Bzr2hR%a+#;W^kW$?qVKBP?gcr= z+fnWS`s}Lepqtw8juUWpBia&`%-_UlTr%u`YCwh2EQ1WSbNIRjddU1ojiTXOCoQ|` zz8s!LP$yD(R{`8Jga#)j@|po;u{u~2a=$YM%6s<;$;Nk3waYcTnEb~GC|061IO;=U ztfhUeiB90lAtSM^EYKZ2UvLxDfz3lgGO^JvPLGH``n?E|6B%H{;J(-;)8y{lo#s$* zlwZMt(uUlNWpve)Ca^!fkSHW7{<2F|0oS9tS7NAv7<}`o61yx;bH`a(xr$d*cMF^x zD5-1r%}dl+UceAZBJs^XOOz~5ybK1i@$5h5s3d-iObm9yL_-Q85RlK>c-h?*n|$-j zIwd29`@N`U)^|c4Lrth!awYvME+$@?;X|R$@RMceLz2LpQ?qU7CW_Y%%nxs3D)p~+ zl08SxWIZ^fLu?_f7#T=ADAnj;aJ-2Dy*NWCo7tW+I^MS*wvb(EWI0i9RduKYQ>>>2xS;yzIe8b)2T5-lR9 zk%$oLo5M!TV><*mdh83C+1#z!(|6l=VW-wtExDsO;${CLK>6G%P;Dl61HJmi8j{DX z+rrje!*=QOPE>+aA5#Nint`vV zQ$O-FOw*7gWPoQvP^+>$F0G?fe+MDyn>dQ$Ldj!-sFaEa#q`C-oW}W&NotBjo#k;q z_g}I`hsAiY`{X~D_3DDXoGNyO2QC~_l#DWNYV9qblUI$bxM^MHO$hOt=T0hF7^%Y( zWpIfa)pRD&$pIf@$2tUTo&eNs`7bSy`&g76pY8JdY~Iq`qyTu71708>r|%EhVD8G# zxQ~g{j2W!X44KuMnL-}sF&Uw?SG(ZeYVXz!@Q3{eEb(&bm4UiZiytHaIDaTt(^LJ0 zG_FDDdFv0pFVKI4)KtyL$sl*!_4?{a&6N8u;jbxRrp&QTldlaM!KFBZ1*y^kbU#9k&6eDPqz@}`yEX)ols~@h$X?w^2X^NkaG*b7rfwSM z{3Ppi2gWa!w5*L?UI=lpz|{~`R{mPS{XT1nQH~n>RG=PK@0&Cpcih_#h^m80W=v9H z{(4E5VpL<$kgO|}HyenH$CX{_s6{G@Azsty1ggBvvP3B?>ojSAImYA1T*k(A7m;Uz zdj73$N}7~2zZW=0G_7jg&4`z5JVO0Nfr?iSXQo>@@UHz)zMjadDo2#r?KJ%4NO~~2 z?e)*BBju6hJj>w+eu#2XQtqnc!j|jX-i?u<62Aq|7sb*=M zpkYe1GzCwAh%@-t_hC-yi5IvAKKY{;swvpl=vr}1{_5_wFl6wd7f>~B322lPdSH)T z10V8&%jMWs5HglvdD!%SA$DXi;zRj-V$BscK-s%gLG-AKG_ef8sh{R>$mftkyCfxN z6@-@zm40Pjk)hFRwTg#=Wm6B$7lPdMy<02`uW3l>#_@T6{F@ntm1P`s%&365wCYs8 zuWO<;s=|s6UXOEmu{1MsClSJUE^85)<+KFTkygBp8m-SoW&dw?9cNhIK6nb$TCM>E z#yF#{A0e2ev^`fKs4u2H_V8&OH}>hMXg$~46Jo&b))8&@(*sRTf{Pnreh@py$WP}a z8e`!UM%C(;%%J8#k9MkB2g?OTMh|{~%De=7bPC7dzD=cDT{YAz?lcghe7+~ycnqv5 zQk$qfPW2W37|jo{5f-L+9DU`x)OqfS!bLv?eOFg?s%osQg@9B>BDo=xL(#E9@TCO6~4P%Ye!1rXLWuLpl!7lg2X!K6NG+!Q6tDvs}>}BQ-6lDaDdw< zJQT@!#tNb}{k3bZU`hy%G?AOYOWHR4^aox#i0}mH7i#kB>qg3F7GVRC1ADs8&|ai| zUz}Cqpt0ndKWZoQOuTJU?EqmsIo}euOTSN8+f80sr(h15rd&ne#j(5&X*WQspm(k? z5LQ)J&A7alvP_(|nW%kMFW%lgKVnKF_H;WlGCEbE{GvJbe()GPucFW^6Rpqbx;UmA z2c~94Ew(S1L7KhitG0L8nR|F|1K~| zjMNq85H=Z7uXFNX3BQ{lD$sVjKR5#KFnr@U;FU&2acnf??)pRa!Yu~|`(cv9D!#}L z&B4}Z?n9z1oFtLhkPRy0MQQN^l;tb%NAri0q~!n_UTX2k=LWrxSDalU`biraWLvk@ zBq{GlUQ?RpI*WuK7naA&JZYwcgu#xHN(LezEsf$8HA=b$o505OL?ef$=c;)f;$pyV6)cjfpa#Nq?X#G^iX zrW^-1afv5G9i{Q4u}I_KQHw0W04X0Q775ra+NzQ=t$L+VGBG9T7{ZJu83P7E0uG)?`Ju;-@$1a4286MjbBfpb4$%=2F*#R2ZE z+`7mFnO{n$6inQty3Z)@ck}cfCtmY7AMT8RC7ZbK>g*{1Sh?>@4-;1*0zMMEASW)Z zWT@Yp`XHB16k*E-0FMBNwZpeJbhe&002ur+qrlTdNnbVyO9z-i+_uLhnk6 zh9Bxajt0A)cMI5@e~7XjWDk^7(EHxqI8$N%*8_WRi+OTNw+|aG++k*fkC}|H$8i9o zZOYwk{d{PKlO}60AySYz*+HNamuiRk{uNIk6q`MU(Gp%vV|x$_3rmBx2}crzRql2R zs47)a?b{goS+51P9`_1K$a*mMhJ9JA%nt8*gFzCbfh-}UvaZ{JNCQ^?CvuHxCJ?Mz zXkMHui5YKVxSVpLl6Hpapd{1;0R#moozAEs`)S_+ATjQNJ=AADgAAfxGOQENOm!}@ znF%Q*k$jRhIM67}CGV*&Dtm>6P#ZZlS}1p;mh-`TePc8!6bj>ci%bex)B{&9FRksp z`RpOBLa+gcvvpFqY&uEqa=2C6_j-0J2?|n~=p+Q`D8{!|iO*!n?|krK_(#Cq!{^$I5nA_%ox8D0OCk_k?n z8;u|e?CD}_@ve&KN12+QrGpS8@$BSeum(C2%3J$eMn{@DKY3TDK`oq23^dyA9tWCfRGSxmVh|bnv z#6*sG(hx!mXbllX8S)3D7Ji|{VJC^>2U0)Wb7LjgxsIV7LJ#wvo2WhfGvU-l_eW^6 zmJ&NimXvoc(&)sjDlMe;L5ad)RRs0=%O>wqg}D$_N|)V4=gan_ql zHm4hzB445aB3i$CH2F%Nk~-YwT9}CQLQ=<4Y5~En02YbJIBJuu+?rAC5Dc$BGyZj%cWu$TZs_f ze;>t{5`<+Zu8cnht{d)-p_gUY5;HgV&)c$AFdEYy-jp=ZhpnUH4HYgusDxn-T|)x# zOEMDs^Si({Dz1XNBi<)t#$Z*_tT>eYx_?Fv+*$gdlD|3HVecwCzt%OmPOZ;8(nS>~ zV*}fj1nW)!@*sWtTN;4xc33}$reJl|8vZ6ttto9yfF_&;X%{a{DxyYJO4xj=j=g1t zYrBOUO-%!;ZClZ4Y2dW!j!J9F(^InOr^N9reS>3@;29=X63Y&2KgQS|Nfz?QU4u56 zv%RdB|9kTebz2MaC;Dk$1>x@JYg!+m#{L?x`*y#T24bSHD~+!h2?tRj)h%(3 zKHSdI^VYmRZ;jGnai-3T#Z|woIuUIc!9KB}k<~o4J?IwwDG^NNY$=Cd;#;L>3cXSr zF8A_>E-ovn63ESxqD|GRiNZSlur&;wL{$|iptC}xs?$PcRZ=yP1v>^0n@uY%fuKn> z5hk#zqmznEQHoO;x<{Lr+C75qILyPee!WRVsWf^y$*9h8PPFavdc!G;cZa5#0>{wm zs&Z7w5H>fmn8}FGln`rYAfK$l9WQw@Lba{rc^+NnRCjv&c{PASA-ks@8?#CmWAb1} z%s@mvQ?pS@UE_>Hcs zO7V~rkmRWQ4k)$9*N54mVqiYg&sbyht3M&G)f=w4t}7kYAy#ph+ObZ+zKbNHS$+7F z^M-ZRFr0t(NaRP{<8b%x9bWuR2mawSpy0qPrrzQG{Xq$@SQa{#nBz&P;Yn3+su?by zNw-MSHsHb);;|}dWh;*!24S68HpX7NcgTXb(a16m@YcJuX2AL|<`UxdSB~U3t49DJ z7j6y4!Hc2_IRN*MMiAI>kuKlaS*!t$LUZ@B)U)N;t{5QS*Y#444=Kh2i(*b^4rF8}96&nz|+DFPHW z(2Qn3C<(Jh3asCstpUkpM-u40H58p8%ERDAv?)H!=OejS%WCtDOJu0Dmc();L79zz z^mD(t5TYj%|3aFLgw&bd%S z{yxYrF7Dc}72-cdWL|%^Vkw-wHip7pDJ939SJ~vQMj785d{3WST&$%*yU3M#r zhA}wW@*w(nKs8Co-}IFl)fWcnE9xvToGoZ4f+u~{*abUtaw#Vcm*$o<8%bc*5CR2? zIBR#0b?wcMx`7Fo59-N0HYu4y=>GOZ`d_5#7($pU-f$t2GcBFMG$`8b4 zDTI(239*3Pi}tui1pJi^kcuWAkiVs8*|9gR+L;39sZ76~1iVh4P6*sS2XZ<&APD|& z$H6ge#+vl-`G5T$-{MqU)nPOQ=V>$PZ044)4ao#KflzYA-y~?wsYCl6Ikx@&A+zpr zjJ@mfgyu5K-^07r%L|)ku0GUs@pr?>p%tJG*}dQQ-?vx8!`0Wle*gf%KNc_A|6ybP zzx+O!7`hlb*jw8DZ#WL0WW%4m`5%7miTZR)WTG^i?oL<|gR;Q44Ejr}OGoY$r{8hEsMuKRAo3_G36huv~PGt-RS{u-mcZ`VlQo?(%V+F%h= z3=)R4_14?Y&R?!VWT7WSo$}_=M95A#Pc1gp^~D2%AmPb<&K$xoL8RJWGgR4}k1;GI81n~?$ zWZnDAbQ?vbw}% znT0Q(s$526vkDe4toc`kX%}-bz0%L611O%IPV=S{g1;~@XW>>BG`HblFpE}3BLHi6 zmm0&FLMSzhlmce`sOzH(>b)j1?f0m7`kuGLORmi_z@$b3vU$W>0fhWRguFUl{ebOJ zI;nWHk$%oCe^&3QA+dVx`&(oJCe~d}x z2bSR`8pP{iHH2l}4X~eBtz90t@s#|#)Rub`-C4C@*t*C`Y6}q zyn41TnZRAJ=l&RP0Y>$Kg$R-r$+0LxPzLc&7RNx|8nB2(x6FD+F8Kp!u~HfH7fUuq zBU8JGGjFxpN;|ui)??hAMY5_#rXh8;MT^xX+n9(4D`5;!oeswBeV{(r$34pks`9*$ zHO^V=nNK-xL;{meGS>&1&jYhW(FQmAH6xcf^>*55qx9m7NxZeF+{v8QlCG8hc&i2g zn>g!J!e3qK>D~{h(m}{YxmWH}-Evq+o;TfQSfKLS`X#(Hbj+kZqO;~m_$4OCTcAl^ zuMhd|*Crhe{pIpRUKepy((xCctg;S-i(Xq&o`d`a`%s;U4C2zjqRFTCuUh*tPaeIw zv8*?)H|u-S5`D#r)F-{nya}#Ti$}))F7U;b&Un%vfq(t1sQ;e`+|t(6&iQ{|QV&Uz zv)^L+;qAPki~i97@vvMQLIC%2KrKN&UJtlzrVTYzwBh~%?t~UspY{<_m&tS?PE*ZG zdcEC}ljjaiKhCM$;2<74fvBgY5$@^h`nCtAJez{LFS;pvS#;mjF*3LfIays^8@cfF zgtavO`9bdBTR_G%T7AI%qWx7@31wA zq`di09F=T6p9imli(+LgNkcBI@jyi@d8GJv!Ew@9et|4s%POgbh59&PIsI!ZvR?CP zz%5-PqXPF8D8=PyL`~9=ao>V@UL5iHgBI=(rnClcto&c zW|@JJe2v5mr*DkEhdQLA+>VUcKuNo!S132^_J}qZh(epF7j%HT&_MHUl`;aph|w_T z7D06uE#uXNnjTi8qc2Uze(~Qq1QB{;kxz_g+BHB2vBTzxut()EXo%KCc~Av=@Ic>7 zzy-!W7mL3O&L%oBoEW57H2%y&=ardwrIUdTn1Z@M8L8Wc%x!`)8gRB65H_I=R77Q_ zfW?ZI-RLiX=<9(@p7%;>O3LGoLj4bMf3{>ZIH|O9)iJGcl|E{XqQ`C=d9i_f(x`&+ zLtbcBSwxRo7sJujK;dr$_gr!~Y5F{9i6w8%t*wQybGCG5&8? zt!t9dPXrWF$PEBJ=x82HF~kT=t&%#)EX{pJnNIBpgsdOwXY{-+R?fRCf zD`#g>t*hv6cReAxOz&5ZUs=#^<9S>0oTW?~3Aws$A#*gkin|<&D_)dKek#_?J3*LY z2p%JwQ{eknSo=xK;I`uKwiqQMW$UaZr8uDwjRw}BXBSMbn9r>9L}%wbm=-N~6Y*QN zuk}Js)SpS@Q8VH!TS6jnIkr5#i~l{T0h5Gt1q$#45n%&m9Dm_zbZjfT?&n%X$5}IB z^bM<3%r_Jo%%uV_F25a{S9Hed$pFv27Z%Bgpx^mhCG*Iz^zUX4AeUBJRlkd?yO&r4 zHFqacn2Y2Ut@Jc2-zB+lTITL1kz}^@HVb#-vDM+-Ph*~89Aqo|zKiAcHdMmr>C`Mb9x%rE z-xP2xu)fElVSWNo!NYI02kJsHvrhAcE0W*r-uzkYpt-dFMj`o6UsY~ysRq9eZN#x^ zuAyEAzP5)&PFZkPn0G&rvg6{c@m#$!#|x*@1$#&y*8!OOT{!aZen6!%LXU5(K3wOA zdR>TXGh8GVwWZLPU0|IGTPJe@|6NfT@HZJ5KWfrJ{+}^&|3y)@rY=sF#{Wy}yGHG& z_053<`0-vzDy>7Pl%0lwz{Vp+){qI?Zp0s%jVS(y_iEeZ$9rY+_VP>lCQ{dABAf_g zqU$90PYQp3GGQM0))jhmA+z_}=i3G4=L;@fdgQZ%gs6CUV2|=x)zp-3Lykj37oInk zw?8^MBn9~)HWTz8n$u@&w?LD~Q4F#`<$Y+;KO}C;AR_@La}LGAg&=!0gDk06{zS^b zyv@~{r13`)UL=8q9&n=41k8cqNDov3^Rmb#^b~X@DB@NMO5Y?@t_-82in3BLbRxVk zE9kt;EhilyLM144i&PjeVwMCywyO#-WjBKoba(?0GIkq=1;m+^K+rhNLca>f>va3y z6fYRVvcAGpY(H#tJxL$r1zGDtfixZeV#`V;tb0y=PP~}tR+r@hRVbCt_PaSftT-yYMVH zW=?`}(&@eQH8jtEveBa@vFsP`Wi1Gl4cVDwM-yO;n{V!li5zbP>$~9zZT5gU{Zslr zuV=Hdb=HBe8Rt+|6NR{8_=PgcYqzs+dW+Ebv)rQuzMW})aHCd!nQ+R4^6oW zyLRDg^UTpPe`eAmB zR)0F`V%G%=YxX?YyJX|7QP^!iyxn0I$c6Y4%Ps1H`(0Yl^~iR2wg~I9t#z74xH^ew zx36LFVVD1dL}$dr5rsr{P5t4W@|E>30AC*ii#__{bHV3j7|$WgN#Lw(9q=Rsi5f)M zbsV7L_+*Gu&mB?*7MTbulOXgXbix@3xqffFpLUB+*RMrwC|k@_1`QBNJJKaHk=jN2 zc(Y>x)e3Uf4&$Ua!27{rav3AM8CncD2mvYxf)2)fivusqxrxJ!rOSBdmT${|wajJF zv4B_jNZ}!(z|yXKB5n6B?Q_xD5%)AE+eSw7SFGFfe>K(&Uu#&Sl4N-~2d!EYJ$S>E zp|LRiYZ9W)a}+#0ReEo5s6jM()x$t=vuK;O8hp}3EOsj_QTbTpSz^c;D&M}``oSUl z=89B!+gD)qW%Z1)zBiR8r&H8#82FUWm<6+PFBI`jm3uPlSPNBGz-a9$WLj58nJuAD zjt3mHtonHp$C2^)Xs2~m*J_i2^F=0_L?z|Jg!LgLUk6piIYANP{*K|FH7LkVK6f`i zE63<}k5LwG?Xe4J0)tVgHPmp=#nW>o0VI0T^WeCX$<*?Ngp$r2{Dvcx%3M-sq7&Z# z`;0ckLJm>gTVjdZUzO-A-#4j4D7x4ixM=8eHq!3+e3Kn{qC7}uUX-3XM3k)1m}`V_ zq)||cNB>1&#@h>c@4$qpz5VTP`fiH5prnmmmW1SR*dSIBvhh4bItGm*-h7i3gv)oysiLjLPY4>RMnJiElF z16iQd*HiWLD|64uS;uGE2(U_<|6AcMn#76ZYQy}FU@LV@ok)R%psXp`dPDAv`C}GU zGQIJueR#$GdGz-R9~XScS287Z6c$2>oGjW|!LrX=BQ`F^&gw<GRmnK0K9lQHP!HH&+{dmw2k}Z~3c2tMm2m%Ex_C;yju*)}T%a<0JbJe}v3AGhOgwe}hN<>(qB1qvlxNB4;8A}e&`d z6X6pfUjWRQCG*rI%hU-8?TD#X@yXgjq2#eD&n`+5iJTZea?w}a= z4G|Dd63PIB%qiqSC%M@hSJwVsGEbXEu0qqjlT38eJ+*%z#B3URH`Uz;RgSUT>0DDc zk||FPNiy7R8UGrWGPaFnfQU9Q5dnkM*Evhx24CPnCS4RaTefnIBSMgbEr1?h^%X2i z;hwsXZKlqlUE<9<+i-8vHglMjDk1wMPx%~c?whv`$b1ys+(b#C*;`eQVm{(QC03sY zLXv4WVxbA-j*2!v6pq3;0du*0%eb_-DGzb@Y@w>jy^9jk$blwSA=VfFlJGKWtgGzy z<1W>Mlk4N{^eV$2g|aKm+z(lzZut7I{xD?3fjiOA##5gFZ=QfPQYCsLGREXeiqP3t_MGh z0Nm&S?(aTwcRjITaC3FBgX;MCeDr$y2o%2#IUg1@q)s`%IFJ=I)WqKmH#a;`cMA%X z1L*#EXJ>E!!0>V;(IWt&=wj3-WYRbt-M!cu$f9rP`!IW<96lDs6M93X;qZt_2g-IA zAey~@CSoUPfEr<-DyfJQ3JenY91j_* z+|#4q!NLE^)1Tz^L>}C}lnKxDpcMkh2yEtGm;Vb)K(oIb zp80gFs+LESNm}5dv|fF81QsB)<*Oq7&n2vXdK}=Ds&&Gz0+koAT&A!PRC#f2);z}N zKlAfIHQ>x-lBQ!^2S!D<(o==^SyFy>pSFo(oy%+u;BFFt0VK^2pv?(iW$ zAE@VMwa=r}8d(srKo1KL4Cz5p$PpBY>upuZuX)+s5^NWkL*yur%z0F6DMpMwT zuwE?yH+-46x&7#ofY0vn;8##eu*DrMQ773&|01n6dN1KYZ%ZKP;U_MJKS7)P?&lMxVHIRu$K z%*H#1Aldji2($v`vVP6)eB3iCyDZY3vtkLAFY~gJaF9}B)fPqm`C663@59#8L(q&1 zy9$t$d@xSWmlr$F19j03DTnofJOtWCSt2X)^eC`Nb-B}f7Mi1Ektte}g_X=7R{6)2 z9|HNR@~C&H?*kZhP~t)HCk+}#7Sst)UFPYPqrSl0^@AX?32Ma>+ieRyeg+d?uOM=Q z@!!731nmUq*Hgs4`N%4xo8~Y2rO(*?HrngP7D3YNR#g)!6Q-mgllr>BscF$9!)|qr zLF+4lAZfQ7fEY0|>2^iJq}xrC#0(<0X^arJgtAf85HPs%XGJ<5Cd=w_I7$}Dc{YW% zXx0txfC&VfQoN_t;X@4oq$dn#k<3cBhRF;kR2!B5F`L*4s0=VD%GyUG;8QYPl4}6| zj~1ge34vT-ksE7JNpV5;KG>fyi(uhiU8Zw{1*_N+U=uX3PU|5L^Cuxq3r7iq^SWwc5Jx#J$87Y+hBF*V9Lw+0uQB$i?1$x?0g~$<0$A}Fk%<4Aw_`m*JKbycR zgad)Lx-l#^L2M(E~MaySKURjj#g`BmwD=8@(Y8 z_tV~HXoz~R2(|t<<3397p7=&&IAjW#Rfnm5M|FqCH-vXqDs^*%>THQAsMrCV6~y=S z#fJ4aYdF=Ae!}sj(mpi5PrqV8_=5e~$hQ8riGBOzRs5UygWFcdkLcMGwu0p%TiOBz z*ndcdvmy}s0X(Uchx}>|gB4Rg_>+DHPOOsiX*xVtx`Q0$rllW5RgtBiQj$T+Il;(? z3`(M)tIKS}IG2lblufd99Ge#T zK>-KIh|8n|=!#Kls-JR;L4nW`s%SyG8nGDqL!vSha? z`E#RKl8cv}JW`&e)`e zR0|^cVC6gF+cFA4R-#gXKl*jiYsG79enrI0kfI={^=I4;I+w@k0#qSbf`xL0OBX>5 zz!!_Ibx)!f)*rZz9)T*Xy!ID934#qu8iIcp#3QVp70FUsC4bEoP zRiIim0r-Lb{qovwBe?n+(Ct;;UEf_zfhwZlN2E4-`zbnp&QlsJP<)hr&@58kZ}hS% zo%N)Db@RiP0Co3#r+0jGdWH@SEM%@QA5&oJsaj(AS9pzCWvP%*o7M_de@`i`s2EFj z#{Dp4Oo9gz29-|IX>$;j3t&8QltaN1_{Cdx>Q02^xcUBzEXS{d|gp)DzpH$9VMWE_GzJuY}V8<~S=qDpr#}Z_aB9?w4UUqZ?sXunPQ3v76cR zRanEQ_hlJQBJvN(anu#ZaEfr%>1AG2u*4g_&O$Sq(&a~zKsP|$-;7#1+mfzhrp`8z zI%xdTufyh_(V$PR-N;eX@tKqg3|Si@7;|t)n}HL9+dz*+t9k-LVjLWrsXBEL#jR$7qVOCPp)67tHsaH!_5z(29%tUJ+z!tM)o=s>=cit>ogxIAmDslR~ zz%5%E+m;$fI9g;!l*Bb{PsjGM6e*^nXbJFTow_WhrK z6u*nlfBtd!;}21VIfj{0;LMU={DK#8>Hf&21ET>S5soGX{!4tcPmkX4qu*Tge89WHuZ9siiC~os z=D3aIgk+*~wOs&=D&$tW5f0c6Mr&OISB%J2#~pKXdBOcZxqgIKjfg%(vlR3dlZW_& z>#v3?rVt6%+^-7|0HcBl$6~D@pk1Q`K{BfZji@l3U+6XHYY0P^4XcG=2q*krf~ZUT zJ{T?jf^cYRU&|r{%dTbNk?KE6QSK-XOACj4dHLUdc4=lRrM*quT!#0VAxDRA1aFa$PVPRFt&Hr^_c zbYPni)J#GO2B8*mtzc$o?n2e1QX)Tj;R`prS6Z}5{L3Y6a!!`h-~=g5nQ}-ORj){3 z*@I(Y%tF&3VI~)Co>+xZsX#AxrgS}*VQ~HMSACLD*J5W!Jk}%dIIJAlJ&8B+9sqvm z8>NHL&Wkkp_{iWE`*QVP+6vxQgZ1AruAZo-aeKf(I%vn3MG;SD z`ZkREo*>}5wHV;|7+VBLC5NQIJRZd28!7XASfoIw&BEechC$^rE{CLrpy)SK!BQJA z#d1zj1{i!^<%@-(Cg9BYwuFIvRMeP7Y-yvLuku^7RkY%9Q*&e&ED*IjcDV#A;VF#a zm|SOE+t9NbsS~ln2o4VF1}a)Civ=384F-;$+OT*vUYlr6rWk}dA7{9DllBKP4x6Ab zP}*8z*Ifg?QY=#dKvku)1zF!BdMPt%DyywL=WynhZQ-kp})({asX^qZMn zA3&0|T_}g7Z7ImNrR~-v6xO!$6M9P^dSH8`JwqwF2!Vh^h?x8-CGM=NvXqQ^QMzkbNGQ^CplHPi1-Q`AinI(d4;U$#3wja7VbYFbJSA z*s**0xM3?E3!Zy+@M>;PO+f+wC&)~P%9w_MAm!;BFVrq5OX`7^8~bUy6p%U!iU38f zA^z8>VM*Q2f4i*XF`?x(uYq**o(pGOV$O3uy>}xvY{evBqy>9ld&w9rEewP)cEadr zLQ7!FRoHVPbo zfrWPFn6?LsbcRPAH)tu(N}+;cNuW_<&DxMl2Bqh0Q?nf2#H-&LQWf_DVUF+}9rOfT zpGidZsCX_!CZBLC>K(Sh-#de~C@l=#DIz-oqV71kRxbsUbEZXv?IeG zv$_i73RuhJA3nm9PR-y-bd1O?Ukfg(hX9(%OHu|DZU`X04e=K&dZcLeOm0bP;%<2KO71=&@BJ zvJB9i--R-U2l8BXjHNJ|uV^h+7+puFIAp}$`hvN(N69wUnwi!5RmSJFrOoGu9;`2~ zX=fMu+tmm4?lL8jj2LTLXOC1mW@ZHT{8EaDA}E@|73k@SJ`UIr>8$pHeAh{NgScf4 z))SS@k;_Kwk>F*kgdf)eU$reA2seHR;e80<^<+X@WKzH=Qoar2fc$E~ye1j1vLu-m zF-ocSfV79m!t)+2mQOPfQn6z!__-k0w0zBnRCUEoue3ZQ0+pcEpPpVar#gSLoxHjp#%^*81bqOq1|DzfxK=#@DG6Yg|q|jswe(3NcQn5F3E) zx0^J)X)=Mx5oUwFY?S*G3%_6hxwO=;)ZR0wUY#KaSg3sLj0NQQ3@9$n8Wr zD_OmIK$t=P?SYt@bme*$TX5PePcchA52b8XZu|6eWgM(+2XV7CEwkf1MLD`m$IGd- z*~p=q&RyT<{VZ_C*q}$$18==YW`YOVY&jG24L$(ybluIa$m6tiN z#h|ByG&j%HZoJ-Ao_`LRyH;KzEMs7Tx}1qbKGCZyNAUrDduZJ0V!!SJzuuz0-a@|S z;=R^_z1|{yvxT|lNSd?DSi;ETFuM3Z^>wDd6vzGB&c^@Y|&_{ACdO) z2(}R_s!zaoaF3FA1B1&{atzK?1gtScl5VW4$%jhzW-t2X4EPj>OJ#{^tiNW2>@1&W zRgN-v2y6o)Va$8ql(cP*&WPsv!AROzVL*sZcws((3VjxH_D6GzZ3BZjA;^>*s?F2k zc?w*gUBWfMFvF)mIE&^V8g@?{iXj?X=jZdr2t35Gsl(qPZ4U$m+ zv7>|36iX}fVaS|%JT!&|>9u4DyVf`tE`%-eW{aT%Gr?{@YU%#BLdnxE1*e!Fv`fLy z-AayL^g$=ema{G;JHniOw?fJ9x)=SSYo7qsu4OCee>N&|Z)-kLsE4U`;A z&_mEGP()()p@l8lMMk?)1()$TmR7kfgiLvht)=SdC@7T|PV89H1(ZHlS07s)94kFl zhcR}7#8W^gJ+{P)=Y9O&cYpLx={+3w2!%OBRN+YQcdRfFS{?RRYe8yn4F{tOuJVKK z1ga`Gqlr9#+Tg2dbp>djvV`{lj|0*49d!Ek@Yklgr}7MDiAlQD?p|lyt&ip~jNrvQ z((8fILkJ*R55_fP@>F+yJveH-T2I@BKOD{`Jwz1EwjP{sX++nptH(2{Ra%-ebPpC}DRNtr$^ z(|HM0@(IJE^hO6RLeeqWj6bkF)W;K$P;|D)t8_lPR%0MNfGy zyq}EFdF1W&+g5ON$KNexuzCH?ip_+6%#x3GXX~|38Z5@N_-;sJm&5l)qB7ELf6(0u z_s7l{S-F!i0@bF?vaI(KIvbaDyjQJIf{GF}FuQQ4R=z?urYh7DHL=|oGw~*y@1X;r zx2?Sm;d04mS*~^(ILTP98;;ujl3wdGcB-UVJ1~qEoGIqMv(w(;L~23MR$n$_3#L3qRN6?RqwDFl2W)UUnYI z`yDZRylO8?kVtW77!#EfmGUhG{qrAx0QsDc(+yqO&edu=q9ccKbs-A;qvN=Cz2H7c z9H7pJuE1G!OklO49|WZHr1y6X7XYT#C$RpKY=V@_dkbKi2hIFWWm|%L1t*aM26zzG z8(pr?_W3w?{A6O-*FiX^^je~%%UZ+jo5A4`=W>IEq&6OIud)gH)=xC!p`VCF6~uX; z3-2znv$|+xAUjznRx6-Ko2=1LuaU_Hs@|!r_EOd&M4=HAOG|AGg59GM^$wz0!;vhO zN*0qWvJfU5SCYl7r>n&_6Hk`&K%|O&MM-#fP}>RZ(T*Zq=a^8r(oE;7^RJegHHj{7bg=k ztDVLO^{{(MzIug!O79%818gmFbI(3^pfyG+C{75`R{48xB#Kjh$JCh=)!NeQ6l;G4 zBnepDoQwd-S1+rY)Z)TH+c@b&4K=2nIryS7m0>j_=s4ny28* zlq&3YNOX1t{X*^&jc^kw5D^ylv{=`UMH2KmolP)W*s3QFt#&4_77BWA48Vu3B&776 zpVG?0t$Wy%a|2uKpb?%$Ut6Z|QT7O}_<7h?l`3bpu43ptv`4|nf3;GeKCpvm`hvii z2D%5Tm<`oYv)U2Kms1!g&|=JO_|nV+YGT{29rBh%6DSymk%Gye_84v1o&wXAMB7qC z$?@DM%03U}7NF6m>=J{qcetr7v4QH9cbv}17f~O>T0NJ+Z3;-0)L`r`efN>TpkdMi z`3o9Kf2+|($ItKV1o>S$GJao5_to!>5XK>sySa*LG-?PpS1+tIYrhAwB{bD8^Cjq{ z`)OHG+;jKL3N95#k}G(`et-ED+JywV0W`gZ9RT{oBH8}>-b_`lYY6c=`4`-(Z7CR5 z6Bt`O;%j6-h_LpJ6svnrgwSsm1bWcvLdB=z;4^3oxC6#yx*<;-`+j#5h$&3~#x7RQ zYITEy0yk8EUo$Kj#`RRNpB6RI4LjjrsFsvnP)#~#JJwUO4l2GRN4;P#{TPfgii_$1 zBPa9nrC={uyyuy?3%vc*3;jTlP>5#uW2=bwoVMW>NV044hLy6=dY=L!dFH-(r_X9WDuXdr0LG`P+u8evZZ{lOoIco$*DE7N0kC+CsN4Nl@NhGF? zStO>nnMT6zRZBjJhRhE#T4}8ENFZCyK{DiEAr-J9E%4&qQZ|4zeBJ>ZXP!6)#kd4v zvlu6AfE8L8p&Tv70B(ByRb-L~kk}f#9|0)M8zHm`E(#3)xXkbxk!y9MIc=L&nLf%3 z(#H7NwjNr+O7&Ip2 zSbNB-sR^Q5zl-P6=M?b-oNveTZyPWuxUZDQr(h;3$$u zbHP@FlBN@4>Lb&5pQl=lTTy|imWydxnpe+L+thEQ zZlHnLVx?3S1`7whuCx)2q|9M^y0i&H^HvbT!F;UitCD8z+YJH@5O5Hr+;?*GatG>L zJ{6BR!mjF2gsE3|g(dMAXZdFq-|H=hCWQD}sPsL{D%989BM37qhR%q==_a5{PjPbF zvj~&(b%@(va!uq104WcgLi|39r|;V%pLk;YYeNW@GyF`-feg4 zbJL_A`V-8B(N<}-%k|hquhki83zCvEONx&vUYp>oSxC&GBfF#UvLN4>xg%Kg<`dSa z&oe&|>JIW~#4}tN*)8037WGD*11+k4$aD!ZRe~}@#U8sYYF008>0KBRAXNZ<01zU7}L1;9h@l<-BltDnhVd1pp{+&#?bF@04b zveEpSCRrvdU=L=$+v#_84+r)=SU@`oIP4d`Sbie5E*iZvd}^K!`r$XTwxPTSL6A@I z6}(ml4P|K`2CL+8AaC)#0fow&U@TQT2!{um4^<-|vqqycs0pa+5~{&Rz?dyHXB-F- z#cEjy9?sK?X(xAcaIDuqh981gWt;Mbs- zkxsJDG0`BeE)UGDKUf?}Ur(cNqEeG&VdIc4=NT$Hp1mncx!zfRjyW2OG=NWRG?n3e z1-e_6&SgNNx{xI+%O#tjS=km;I9rzT>WEiwU%wvi9UY#X9KA+Rm=INZL>bE%2=fH2 zogUBiO3mPz64MX}RVU4CP z<8O(1IeI~T<(S^dSONUn%J0l7>vOwWL{{bZaXL-g`8~ql`&9XEqmwcsT)0L7$&p{FnCOl*G91}0N5g#we7+4w;f^ax=GeB zAFx{3ba*19M=35IrGH6_Nc}q?CMl>OXCu5Vorsd12ZQ5Yv$zCBn35SMQ{QaW*o5V# z@!FG}`rYdClRjFA?kYeLL8=>Sg7zRx?azZSmE#Z=rcC@6vqd<-DdV$&E`UxOpBI{l zRrIs?QtS!10UNgTupJte z-Acf%s?fr&e$eR%*sZT(stDV zQe`9a{u2}^Mxa&Id}uXhD|e$dKmqtfbw<-%E3KK+#eGI;goo!2qcQFKRE`W^uJR~4 zRm$>JgQe2+)>9>)M5jlMpEEzk$!c|&iU@s}wTBg36s#}(t*-Lo;}GxrEG9_}I5zuw zUSRXhI436>b16!5x&0?)+1aZzbhadolpst!NZYT>v5SPwQGCz#s9(9nrBdnJueW zH2HvCh&+GhT0!m606ygwQ$5>(yNZ1(^4n$Fj}&HRrZ+;rK{PvWT*T z@NG_{kKM9=r9B;9TzJ&=-*DfhGUec=9eO+3Ir#Qm)G@TDBmx6SGN1 zUFo?<>3n+|DSPXGgpywsDV~)N6Y-ZK(KXjJNt8&E5%B*=)Of#g^@Pc)3l%;Qzfy#1 z-0LMHSS4-pFEb~6_q~B%y;#`QJY03#VFTZM?-}>}Qou{cux~QL4gXhd*!v;9tye{{ z_p3(X!N=u%9(-JK9M;<~V9M$gNstV-SF&J4Gjn73;r+EbG9Fw43}t(H1g zmC)U+x4O|5!qN41ddafN83AZI8pp;oTH5(>!$W$z6BNjT>sjn}3fx+`)Gzw2H)Ve- z%yC#(-EF!%AgK5H`S({KtgY3P;D&|H8oB(PiY=M^TdCE|;y+KXhHGZ>|FsJ-hEkW? z^r(&F`%d8>B2#nJcMTd1sEcykVwe~O&a5QulKEXPN-y++22!+F@x{(WKmw`Rg(pygZODH`{ikj3K4G5nBW(rR>C(m z8#Fxx33qJ8Pua^*WU88LG?qHW?$wd2?WrK(wUkrJN5tdhyNJ{{HKL;m^-tv&53L#u zjfVWiTOKt|I5IK3LK9@4qveVHt399iq0aAc(i`ibuR-|U=|~W}Siw#Pi1Qhw@>6)j zC;i}yRYi5&HgYV)RhccJ#xHRESC83_{@TyRVPxccb@)*JY+XB;-gyEH2K!agYGW9^ z#b|BXr;MJ*9R!`d;C2-ax$DcweRXOf_um`nNU%p$C8Nv806W(q@Da{fbOr&kN1cwf zaKuA&mvPOBup2o~Wk&)s#*L|Pn`j8AFcH_m0321+lE|nDAs)scF6tE>wBb8hoXjoo zqsW&Rm$h7r;ql29b?N6tPAcR^+P~2ZP+EXPCC z&Jj$T9XSFWBHcKm(U-}%b-s0YKMk;tx}xr%gMp0YxFZYkRD5w3%E~U>v)Uca+%-H- zk4!~5i3-3rUs|tVPSV+;65}kkbA`LG`DnB(#0W27nqK#1bQ_;MBu^^5AU7&`Po_DA zqeZ|aVw4xfa)DjN1Bar#hHlULn1i188zo%YN~8>jxU(IBR-|Q)Hy7u!Pqp_)Mx#tX zEi&(@#aWBM(z8bb4ZBTeJ`+4+iOKnwbJRqksmXK+#*u-11mFls3%9WgvoLNVCp1!h z++syzYiV~q#_(8_&Z0^$tOa#M-dMi+lyVs^ihPugmqmJq@c?0&8jB-qPA}9DH-=%X z%ydBrQe0E$Dt)z_q-)#}9@l$1O8F;R6OM-rD798?fvMk}Ho@qoQc4e_^amhS2d^`|5v3aTQTdAk2)zAUEKL%d20)4uPBKZ8hs zb;o@4r|2YKrRhhj=noO`aRzOJte~_-8ig%+njy#Cm6X=jYXAwVMwNWTyh(5J`VuFP zNK2`P;eADz4_TWAGTuzRfxe>|5^sx<;FhjiaH z8$@$|R>V%f{v_!(tw#OLzJ*tcm#NH>h3P6GiwTWAO3QA^WiaGQwv8;T4By5xC48g- zDCt$2Vr*6=MFIGjM&uJv#fp5MPxFgxL~(NHnKea0LrPj z$h9loM9~1m)fA4#pw|HDvUJc8rzsC{QeIk4)+ekpP>`}7zpKl?GD{&M62nt8o`r|T6aQ<8X|QWr8b~Q|2j>IEv()rIm2Ip@c0YX zaDdE|80-khsa~>h>snbnrLMx5R>=nP1JtA%$kXt9V^jP=yx0`qh(Fa0@f7BhA&bto zwq6(;E$A}~rs`+%^w4}!)7RA{ZNk`8_J4wtY-0+t zi}3Ol1%{z+S_)h&{-hQs2$#-C;QG;dC~R~$9pFkn0Y&ScqyLUXBu)wCM`V5}bK0>< zld;=7Qut|)k>jebQ4QqSwOq;eTxvY%HDxu&N2T7jq$c})+dtrp{L>vZZw@4=rumv#Ha_=j-31;Rzc!7v5kUzwKhqU zr)=E51L4er07@)w8)(yrAvx6;E=w8GyDUsHX%SkLY^N2Ljk~NQ3q{>lk{WQ2D*c>* zvF+_sNvDM|3?6`bB5E&OXl@9@pg+t&6(bj1fZ_X80?j)2!uTZ(Y~CA~e6UasOjM{n zy&S=Fi#zx626B}?WR%aqw988N6#FY5EVLO~Ue?9aPe3qaIqqzHOw)xrb?CS$l$*fR z=a7Q9E)lRc4dm9G5eSQ{VNy~y4-Zqo%^90tYeeJK-fusB@c~I6y#UI_Fg!AzL4hCa z*^qlSl+L*w@!V?!Py!W+5JL~p(5NQliec3;1q8M0lNyDlFf=N;LoRr)7;92)VI_YZ z)c3EvpMCT5_BU^~ zzd7wcfByWhJ)ckUBX{o@qhN>L`#t{z--f`8C)_&e>zpocRMB>OfkrFH$&E6@Fst*tA?V@>!H_5NB(T~S z;XCboByg#bE(pgnL9T_5XeBrI!&pSf^e0yWD0b6~FGZDTMTiYQYKEJ5in$>UL&8x~ z-B_t8LZbLoZ+7$=P%#Fk6w{pD@YYg!l~6b7Lhbeo_@2}*r2r!68{blrDXZGis*3CO zLiiz#$=l??fyu^UHgi9;Q4hq>&*qbSqlcR!rt4dLQ;Lmm%FUkmMr`O2f6scs2i{5w zt5dNL?zk)}3_3WGC+ZK1A}_l2S7V!E9f~xjfK9Yta689ETSh$g#c^Ji83tN#Kncf! z6qt7o?0|7beoUT1xk-RKK-wjR(^>rA+Zlr=Xm1g4Q0nviqtezW2Y^02(hLHfcWN27 z*668u8-zFV(W~TY+-RTUDsjaeJLQpY_5CKzfDEkq;x+mjF$kIPb9>%`ku@&O=(n%f z!KGGx{FT{X;MJ#jERD4XZ-SUqc(9T|Yr5dQ?i=>jcOgFXxZ(P{zx;14_EMYMcZ zJ;%8(N%?Zz)1t<9ajVi`oNpc($CZxpzAE6?d>%XLj6s6 zG9E*(>3+^SuNz;G+8D7FKNFt07Z(TlEp$`1%W_6o^4E4S-2Sw(A;(RtQI8>L6gbF8 zUxC@dlf6Ys!6X~%R&JSqnU`?F*o5YP^gwFdO78%UB7tx$d;J!NZkK|lDK~x2x5bjj z0~@@30O29tA7IZ^=PLS0=~0&nNClK|!Zk#{Rb+tO-4XNF##E z5He!BjaMULEm-tOO|`nD1@*Ao{*2}bSXF6D^zc?*H}W-f`*90It|H;N_XRg6 z2xyS8(Y&bTZD}Pp&`a=|^(r9WYF+}$%&JmF0Lh^6n~FVO%eXx;M;XU3Bblcjgab<7 zU8W?2$it|}lfz3>ym}90&)jgv-60FLM=R2Ftww84#4qVJ9Vv^RWx^}Sv4o0egx?fA zvU>VcJZC_Vyz@q}D;8+7RfA!1S--XKi{${9oGxJJMmgCW<^5|2Lmsd@Mjgf2axi~5 zPhx)_&XdS-xKT1Z&m@sV+XwDEhQGhH2pDU<)Eu$O>ZU5e`UY)&q5I8bZ`U}cA!{}# zC{>co7GyEyKA(R|=UIy0Y?h6t0HND+`I>PhV8n((!aoC#)7Yd8ty<*U3gpfYrmA0;?Eo=?}ES$pZwi^hBT%|2-$y{J^#@}q8v)QfmX z`*>|G<#-y*N$r)r=^y6RE5x|x+bI&gL)W7i_IwpdR;Jz&AD@1PO$20SXi^IZ$q9S4 zQ{^cNm6jK7PK#2ihs&hCUKpK7BY)1xxVn9Zl zKDH(WDM?8i``GX;j-GvF#~;1$pk*`6{tz29eG>&fP(&6PNdm;GjJziThg9WYL9)?9 zOu8QWTO{3U81ldiaC$L*3W!rmirnJqf^7tg=@VM~1X{D(ht6x$Ea|JRRqE;!9ANKE=_xs1dvdS;PZU;IU- z5ji)ftAjUGxRk49sthwIovX{5J{dU%cHfQok&v1cBuSOG`sKY~#e?!;Y z>ky>VyrECHY&4+bcYxGmy20zgCG+5tp>eofmy9AgqPV^#!&Y+7)VvR~x_VX@3QiN+(-T!2W8mxePT*Oj0VxTMarczL6*C95UrJ=sfI8IF z%mX$5eQ@cG?#WcW?g&Zpld2V8MxBH$@{ED}koFSJfFKq!3$GPZig|UM%8E|-{(6>q zVXkiyD`trn;eswy)Nqoq=b0jXXgN(onofk;qM<3V;@z-w7J`Ix%p@daI%20Ncm)jW zDz^0G!>10y{#vDnwgbWjlSKD7%_1_YuNp1H?_xQ|f3wldQ|UDK1T(=HZ?S{tp!)KG zLLycfC$2g{_mCl4P7{(LgAM3+2%cS^>%k6>hy{Lcuzw=Xr#V=+3M{(`oPjr|=`>Bn zWmPSK%w|nW4mLAb9Lx8d@wzT=StlFJAaq$1fTEvZ^|M@8mgUT< zaxn>4-Gi6%CU04;Ri)Tf>VY-3gZYJZ^>zn+!c!u;u9P=aA8tWs@DzocF8=0Yur+;W>gc!tx5;4S;j{u#>%$PF#6)?#2i^;Q1y7$ zgMr>nIck_ld@lH~+f`nm14|;LH3m~v;bU=*F`9=9mpN+ri>I7c5r^qP#d%P1{xww` zwU8LMuQ}IUyDjP(cc=BN=v%Nan}HsnSI~1YSUB)uCUsGYdl1F1&nMeQiES>aJtdi7 z39&E)Btk-j97tU_sjcsgRl=^8)tEN^ns({3)@1gtdVNacS(e#9Uczs=KYMhxKz!Q{A0ct4owXh%<;PFf4MkcHC3-)j%wx(J5mwu=d6p|YY$)$TfV z8|Z9qtqyt(Qw*B}(YGOAZi;`~p>;KodxuM4W+(a8&MSngNQPI-&M&6vP?bjY99DP? zo~s-2fyZa#gigKozZv(XwTOu%vOm1o#9sctuspI*Xm?Y$*pzJ+(3<}2l)$qY+Sxy= zNW6Ou)1>KK$+0Ip*IUo&jI6w>(J4P2eARR!kCQEfT~ksT10dBIh8gl4kYU5;UM-Ev zHq!dyRGz`$%s+IvRRUfh(PFM-dF2?HrFAx6e;28wbe%;q!6O|q;h#bZ6N_c@#M<^F z^>uKJJz$K9E4HvshuW@+D7Ww_Jv{ANg~qLMQU|qEpElOYX1AezPh2+7l$jBo7lz+UV4ZmC3?YqUTz8mcG-O?uCKf?~+ ziA+ym_zcNErOrV7Q+g7KQ98L1b;&Q~L!|T(Eg}5csfYvgHMtd=KJ^@YwJyiDk2_PX z9s2rRlH;trJD+4b@b&gxLn9xXvxn{1hwax;_50m^Jt0l4>)vZnuZwKF-bhiqsdjd| zJE}L>PF-^^^+6eWP=+iVAC#fap9f{=mXsl9ht`GPwjH~QGW3OZX4P(NoW54srHw-w z1-CQ2)J zI9%s+L-WS)nkO7qoyV_cllbe}g#Frgu^%>xHP}XjakVW4ReK#<@-FrhR^M=4akq8# zt=6^nS=Vs-ue-zAVWS{mdK$Z`z6I(lL(Ntwrne57kJkFgSHxgyVv9#EDawzRN~KOL z>E>`?8?4BdAZM~krez*6f@=fS-s;0#(9&F8*N#Nx>Hf^fTe4l?&(jzGbOww$wcFth zSVI-Ib>EUaj`4|WwJN_V>*+?yZ#Nxp-C(_kwOWI{|EPm6q!!#f%HRqA3#%1ottjO) ztuUx0P@L34C4b^yLo@x;3hA7ZPf0dS_}V0bDsTTuWvq4aDI59jN3+dsGs4{!2zJ3k zmAFZ@rLqU%^=pfzm(u(pNzp@+B2V0}oTP|}4et`_Gh+?l>q_Z~EGo9pZb=tpR@@Rj zZr;yJbZd#EnX7h0ELV^eEGy&-F%kvUgXd)lGG|Vv+`${nxivQ@GG&94d3SJhMpHx? zNMlbwbH%TO&n80)dug{8<+|SCz6mPHhuqyRUjvdFsNC+Is@RdL*onds;t zo#KP0LgR4TnhMjb;^w|0Vok$GQUI-}zJw%X&`9zclonSIgk|trqAhHI$s%;K6&bl^ z9TS)@iGl!4QG$|$fO>UJz#)+l1bxf}&9rX_JW&4@^|+|f=_+v%qG+`}Rg0!h7l8#T zUNfxtl_Ft<>vq(vuSXw=XZMX1wR0G&LiNgxX-N^Qj!KDl=~6JOaYtYtHV>CNxu1t6 zPmaSqFL}4U(zROb0SKK$Q*iCP|NEjmEN;Fi#FGLmap}u@ZC|f=bd)<7wboZUREi&s z4<+;5c@bGOj%7zcGWe?ol01m2(6xN3{anv=DC|nK^sbO2qzaZ*S#_dcw3sK9Ox*ptCbBoYkCGV&Kf&pXDXs$I@Yu~)e6XouUE@_Te z5N!>~jy=>{we2x)zZgR?Srn=6en}TZ({9R}e4bUg zCUwI-IO8{-l4>e$5rJ_qr}cVonlH!il4^9RfFW0?Efl=1XPTL6RorQWtkse#$SRAJ zV(N~!uof$D$4i%^< zeud?llHkvhj4sBJ5w_>a=;KvVj6vg^Es`o@Bej1+nrbGgNc2PlCEY(EnKSOQdu86YYO3y>eSM<1PX@jds`^KuCUe?Dd>P_lwLHIE z&c`rj@3Q$gzbb1D&0Va|sa(Un#!0FCet5ndeN3yyA~5}v>~mw0G!PxYNpIJ_3tq4_fBB~p_M*Beo3>9l^S`$ta40q zD67PI!>sa_#^IO-TvoK^##m~;$GjPMuOCVr{LU97!1MQM)}D+qP}|V&lZNZJQ^_iOmz+ zwr%Icwrx9^-2Y6~)V)>n-kPqiUHzr|Lt{VdS-aO-k*Vfg(v|f>%@r^0=T^B>(#SXD zI!hY>sVue&5PcW64a3l1bgPFE)hoKLD##7Oi~n>Oru|z0JSBtw6#Y?^^`k%vcP!iT z8_7XAbtlDA8?9lyxPQ#RZCzT;vwD|Noz$hF2z2^>Lk~DAk!Q&Q;IyjkG316>k%pNn zUG|#-By18l;@>VO`QKA=68@1dmzG8sK!%APZdSwAVrv*{>S8iPQXGxD7a? zlz-_ae+C2hvadfCPoVs|zXf0yv7-IRG_KQar3^Jno#@puYmhpsg?!=Nk-bMZPFRc( z*JSNTeP<{M9>Oy^f&h|tjuMm4gHP65Pu}MoA6t>#M(#fFSwj#rUdPp>+*Q)Q0 z1*nFK@h>ylW3h*weBFIjne_k6kH(D4PqW6mADdMftq~5n*wA2HkD?_SV+Oy8{qI!?sDT4=>{f|xMv0&u9whU~L>?!<8-XT=`ADW=^w#i1XP_>*-MJDEc3N+Me+^CQMTyj00T|NX(QDQof z7g?LLB<%4)I)|!=B0^AYRApuza_4bd2377Q*;@266)}9Q4G=`sQG!avYfKV?QyWFy z4%*7cRf%^@pV@O;ZhpB(pXgpN?>*+DXkV>v8PK2xel1$#bEU~f<9X*G;i7~FgkUMY zDPixm5AdK9|y(#5T_%D`iB`xI&LridlPD^gU{i zou)h16ncW5o!WW~yO+`eH5fQI-6?zgbi}o9Ap9NC*vBGUzMdbofSP8?mzV_1GX@ze zZef9w0s_O(Y#>m(D5g|!M;mZ4wJE+p<77hkXfub}Mt3UyO;tG|vpC)^KWM0>dpIP{ zoH~HR0bI1P+?X`Qd7M8WgRZL{sN%_20$nOkN*JtUgNKLUCB{Jr_y-AppRM~RVU4Zs znLN#kMt2O}gE74%au>&{6gFR#@-ZkN5&G~BPtULs;YRQYI`T&Z)Ja!yjxE8M7q~%e zkpQS{Og#87Zp&AkZ?Ul~3aES7uS2`*Yj_#frW|4D%GI7o?1ae~cxHk<0CyOnOTg%5 z6pP{WwcC;(=4{T*bGK00IKtR8DV`FKVYr`bLzD1Q?m}oJ2h?tnjt>W}L9epQ$RNPK zYbY%n15>u7EFhQsxLMMgL5@+=r39;-dP`Z=irCd|)55b&#V=-?C}N+kR9F=)zRBK% zm`Ma|iVCx3wppI|4j#HonM`Rnreo4sViHZ{ieIEoz+-giNM-Z5(`Mvq%Qtk4tdNc@ zle?}=)ib-@boqr-$UCzmE+qiLZzf3DLR*`Ga@dShypMq_OI0s&u(DMAXih+Vdlg(e z?6jkJ%JSTrmy2KA@HvDM_ffQ;PoP`O9sml$S0O-&k#u(Tc4@WNy}~qVk~BQLmWVL!POF6JCK>p(pY1Sl zpM|i(RyJjf8mx=YZ9e?|yV2E}0c5Yxk2Uu+A`lSK|A*0)sWFSak=2jY)&Jscc&RRJ zzb=N*-b{_qf~cZuT-{p4{FF;qSMTfN_Uvw+Jlgqh^{I3II2m&zdvSYrt47;M zw@O9iR47M#4g(*|8GyPusf=y1Wi2fFiHdIC_ZSq-OxV_`%2`vX+LAs|dQixWpxl{>UcsSQdCMC#QAZnd<1dT-f z*Rl(_ND@w4=O}o_#cT=WZPT-0p06c@B_1s?r5Cl4CqHGQ($rQ0+Eyi-B?2<9v6S@% zI5A5w-u`YF26QFUBBY9K&p@AVZIn*`uBEn5^)Y|+ajk%EnGhHpG;(c6@Lq^UYDttA zQz;(fS05GEM+-CI??x7zvb_Zypf(rHb4KenOYtBRdTZ7G+FGqN(YwmYZBeTi_((?6 zJ_hu=;Q6c{=PoXGAKOSQ5faT&b3;()N5&%m=_$-=30Eo?=fY*^0F5vTKN3)syDYq4 z#Vd%my&-r*rG@ok`ad5U+BPI0XKN_d7fpe)L=*6df)GgT`pK2uLBJPs`}sgjAtCxj z_9ZpXb4RIvn&Q9$ChM2=tM4~Lp{3}q2RjUCCp=}~>;O~d?_%vAfjM(@zathXd$!hJ zcnz$7*miiLf!@4tExTc;7{mH}?XdgMai<8q{F5J42#|=R0{N>VfQ%_nPG+zNvdoQN zwP;a+k{h9>odr9v1VDhEXK_;N89zZZsd9Tzu>>HIdtFb)oQS-41rAeOt&zpXv8+Klg639+ zxm?hA2Rs1H7D^0Jsn28O}uo0gP*R?oGQE)NmO15cb6#MDRe`bS?I^fg+G# z%(>BSiJCVGO~r0QGlk5`qkb1|OaBma`Vtq4%jH+s5|I}ZBT0FAEZn2hQ#jkk13Nlt zL+@8J5@}#bEB@bG0(kS2@fUhj`rYxyOEsujSOrshS)L2ZM!^DJ;%-mP8;TQh_jZi5 zF!a8KW5jsEzwW6}(E>vCQ0>xeJS2KKeDL1md8DwSg48AQ|HYMGqP@YK; zaIcjuQ)o>ZMF3y_UKDg(d|e+oF=zRMTd$T%WETA^F@OY%WH`W}bhLf+C3r0!DloXI zP8}uPmO0zgxpjB=EtP2r{vR$+>4iuR(}EkK8vbFUh&gF(6cYmnZ$&wYAqSb98H0~i8OaVtQbci$qp4|l1UY8-`W`?>ar$D}bV+Cj%`idOD!&aN! zvzEW-zkc2e)S}HsSyRQ+PsXPY z^Oq0lrSzJg`eEm$D?25J?1K66cl_*$B1V1&?pYhi;~L+Bg0KA>7i!aKiVa$kBEm1dylD_x{H5ZP~Ma2@3h0+jaZC00W zgQ~KopH#klKScB zn#ED^C7mn4-EHj&n{{bC?h)d2({*|)x9{M?_tRT^2Jbu}hn?&QyQ@Xh8J$~ABMo%_ zWo1xQAfOTr zARyBJa7BH6OFK&!ef|Hf-sxC6uaC}tRqHb!R-u&z6~`_d1#ZnjO6A+Am6DQm`EoFz zgMo-3Gea?9DQHn7eP>@~gR-_t;BQS*OsqAD8oZ@u`?h;C_>9QSKe9(&C3BragkF5E zxA=K|eqDCBUyzIklE3emhhHDtq#d6ZaM)iREc>Q!c5%I2?eMrwHmH(`CQ{I$*o?9A zZ`XJHlXoI`S|QobEP}m--!KjLNn%ffDuHr$zb%4m%w}t z%yG0JV9^f)g#F$b{mHYd^hkK;lF|`n#;nX>a!RicyPC2!I|%SfrS?Co^F4V2-aQ^IhjWKa9Jt=&b+a1!4r4E-3nm%}aKFAEO*pea zN-Cgnb#-FJMLf9MdGYquuSQ`cu^x{4mZ$V%!ZUAGbRo!1X9g2(et+kF0;nwfsuWX3ugV*xt~Gl(Zi+C zH?JR#5{UlqKwmsr(bWu!xAPSft6<$vx%z+8h`)H#5xi}fuw`e#j{jD`ngob}rPCUL zCZ>ptLBe0efCBoibBYIqa~gIH;5KIF1noFR{C_{5E%w3TL(J5Lb7`BO`k)1{z9113 z(U`yz{uXaLWO@RBK^e?l#L4ZK+jl{&zq)z&+!e}vBhGlQ%KagBsSVtV1d z{7d%6E`Q&M$e-l^0*F*Qv0OP}#Lc#qpTw=;Lfm>T0MljNQLL9>o0Z_h?Q2kSCYXLN zOv2VEi-<3XwA<)_KGPQnT&4f_(14h4YKB@5*Ps@Ve&1uQ2?ZsCTJC zZrZ_PhlgvF;u3G``sk)RJMhN+sUJ{w+{?k^ZJ55`e6w7GTKnjfLSok;rU&9Q!{1ry zMy~hG%?b4yAo-27IY7D>o!XZ}=ppC7*7IDjk^?HIDl&?e1~+9+iR1hUn$IwL#_*Q8jH3 zL3vEUp$jaRN<>*oN%rrpChq~?@An$St+nZoBq~HRmucebdT)UkVOw0u!h;v5&Wpa; z_~F@4HAJ>CLj}^+0!Ptckm$(q9VrY^=OW$@t-It11$Ck&sFs!#%YJ+N2|3eXmn6ok z(h?;rM9KKLU$AL97hf4dadu>6Wya0@%QKTwf_?IZyYrM3BO@`)c#w|6%E$TDd?rt| z1xt{SCRj(Q1=^FExrp%y)#3tz_*9#E=AFM>mV5R+Tbw}D@I;QC9v2&LSU4=?-r0Q8jY(yEW?83Zg>*eMN~WqWSE_% z&xOaj{;a(TdH4c3N_gnME=BemAPbon_@T zKV9qH){jdz|Ka*|l<>#A6p}c{{wijBX=X1`#zT!NV4Oytm0m|nv*Gdn(D7JyDy-98K6#3>=Y%Icara4CUqY*)gU!)}&VfD-fGVb`vRz zX>qik3J&-mlWrqPxW+w^mMBwFI7-RL3Pe_fWXYPz5L~VPIyn&IN$yK>pVlP79M8?R zFF0w(XMG0%??}7ozpvgc{4F5wLaKI`9}qi25TYe7@9ol)8Jb9BGHM0IjGAqZ z+;gnrrsd3IV8M8^^9H+KoDt>UQJg{6(m#%@hUV&2m7cifA1P3%sj=-6SL+g8y)#m5 zuqO3|u{ui4=M|EWiHv4W$Y;Hv(cEwuWB;Q}u?{+Cqlc^=H> z6gH+1#$?;1hznfx+M(a6TOk=yEjHek#!SlT)T7hkL48EXc!2S<5MlNo7cD($4+bgA ztbXz!xN8P!j&n86%r)vISAn3$Kp?mJUwq7M3Iu~kA@RiCVFy;37w{AcGD|+}4R>v{ zP&!O7ZRvPB$D!xSrs~-P-e>IEwm_u4m>+nsK;^BDstRd>-6JZnwl}8nG3HXsSF^|DN*m}b9a;6 zN9cxX19^mWrnKupjWn?_5ONTIi)DGDI@lemi7)l-kQDhE!c6(B4V|-u1Yze|N4jN7 z_LZ{_nVf(%ywSGsnrHZSia{!t{qgF`#b7zKSmd32M_iZt2=5^gl1_N`o=UEJzW1N` z3h31;d|?H+w%iPkcer~yS$VpKp{GYyFWLi_inoiizO4<_>L;*a#2K+=7#tgY=ulS; zT>1j?RWruw1R%PL>mdRu=wuYtmC6|A`IsTo3OWAH)b)PWTUI3~IwZlxs~+%Fu+05& zs2~OX>r@u(8dn{$u_nngRA=~l_$lkxpeNGr;G>bGnN^3OWYX!on!=+H`ip~ozJaf%&&+%T_vj1X2<5Efg7P`S5M zbPHR<@VVyYUzxZKHq8W|R75^7m-4GNt9y%qMgmJb=trf719q=Q3O=Cw&z(XRn8Gs#lgEAHO{r^mb)+a%Upa}@D`5QO1*x1U4iBP zCM6ej;}{zyn_)(w`_~2?ofyVFNu)n}x}xW$u$Q84IUxTZ4n(2eUl3t}`{bsu5DPys z6|9TP75ZtZMuuWO{>%=PlT7q+Ux48Fiv>&WMY^^%nwvGNw#d<7@Og11b#VB&lw>Dq z=vRPdLVE!V8Q4cX;4%G<#BS2`ca~${UmYSjR`?N0DvuZ;`=qV=8%)@^!X#9@ ziNGU+7POv=^5*vb=*;XLm!{Iitr1jS;a%19iQytAHuSzOUjaDWPDiAWMQ{>gL`C)_ z>k#%B>2vopsvrBkd2OeB7)x#LdUci*UpZ?^UpQZEyKqnqvWDgHvsXGHsRWj8jouBz z*u@WujfEx~csa&;r4lZVn#nTqzhcd1ghYlJt`gS7Uh6_@YT2#+b0f1`D7R1&s?jPy zHaTr|PNP{3ZTPGw6u-yNKZ*N_`geIJ30R93yyi(dlAuX$;h4rpms^8_8PAC%!<&&4 zKaUIQuW1xPWx7^B$rR2;1d4GnUcwK^d8`3TJ~nrA`t;eR^4}H2^c^p@iar$n&(Q`| zy6dd(#q*UNTK44=t#R0Mh@bd}4lgII-tpQSH1Y2+g9FAb!BO0b>c6$~wh99a4EVRZl{@bi$6-!u=Zb}V0P3@{_eVmer{ z*BTqMwKoMmTOWV~tA)Zm+asQjMv%v@5o{3Nym<1TkVCPBcUC?mDr-&kwvIF|jOoKW z#8!)QS$jW44wHBP3ZhI^(Wbc8IMrqxAnP1vE#Aw(r?TRcF*)CMfEj4VUTUch+DvwJ z(uBV_HICT zAFJ!Ko%UtOxh@{Do@P6E@SuEzs*w(;qkV|?s{1=oB06VlJ`ojZiW7>ihr2mA;hK?U zBXDJ;Ww_coWj}>x;i4T7@nYCfgN_0Qk=v$u@?1P$XMTx(hGI~C$Ny3PUFZ4%g}naJ zS!!A4Hoh3pzNl3_dEjBso!8|dWoM?XLW!oALgV^0`eV(UxQZht=i`wWIkzCdzrASn^$_OAZ#Fg3*s^S9!hpFa$9GH?yT!V9E`pc0?mw@76-K9rY#hYN=*9< zZgXxd6GybW)WhB{Z_pwjp_WX{PTht+0P@Qo5W!b>($gDco@g9r0Zt^u>>8)b#0Kgr z317Q|q*2h`Atb+Aczh*WD#GfJJ_=JwyMEW|Eq8cIHv+r5gW*1=heXm{RdJcb@!2Pq zO;4j$-1Pn!dp24rxf<6-oqIxX4$*scT0X5o5*>w}Bz1%L6nSG*xc*bOn%UtV-ARF!J6{L~V=0eZQ@J(ITbk5=t{${r zq44~=m9h*-@k9`xKGxcuS2J7>Jxo)~iw!uikZ<_U2@)j7v zCa#y23Q^mZu=E|`XXw!poiok1psKznv7!djpI95AC_@~eBD=fR(uC>s8PNlvl5uQZ zxC|z<3c!3yLX`~Fn2K%6=wqKgbj(6lOV-QmB~2?cW}`j^<9JLDp4`Gv_vTA!+jEIN zJ5XTf(2_$k4Z}7F3h{DWZ~6^%F6VwzZZ1p%21XA>O48Enr1kU4Q&lT{_M?aA$S22t z=hti%iN)rA0<*XffPg6e1HWcrYG~tPVQgV)Z2iAbY|(1E_IsRYKF8`ht^h0ASr{em*}g;UOSF1x`!7ZVsaDppD4YRDk`$UW3s5( za1_D9HaT|MG;PDxJz>+inIu{A2S$pdsi7uTKne+I5~%##S6K6Sv6pdK+%N`>VIq4_ zf(>gZq2Ma4iB%B|E!+-@36`5N66E_@aZ+{#xStf}h8OEZqU1V2{Rl#ViD_K}4~t*I zWGX%ESPj#}B$+uP;G5y7AXYAfzKW{}L(sH6^U=0keKIVra;SLJP=mXwP3$gnvd6^8 zGjs@(+c<@#>(GUgJBK!A(nSZ^SRyRVH;JLP9^1)w*ZCAy-Nc8^#nnW~pUA9!JSP(r z`W2&T6y|9_!zECdd8yq&Y+LLiNsbo+fXz2J{aFY+T!DVW8ri;_eX3NMKY$1-yr_4>KbsCKmjaLdyBzJIM-FJp0(e2p z2K(`>n*PCkLAww_3qFH$J&5m_wM<{L#i2|DAZY|Z7xOEzd$sQi&!jj4zxHRz{-E*+ zA`a||fJyfeVGR?|8OH`(YXDi?ojku!<$ecIk+dW3g!^qHN@)bmUcT?<%JTW`_!--i z;0W;>LAb3(5ji0=j|-ylUp!1YT8 zO^x;mI%6erX6~5b6ZM~$NIE=}V7~a|Iu_^YVLoeBkqtD?k(x~xZ>mrT)HGTTp36nv zI6FG-@{_g=Lm9=vb4LYSncw{t6?U#vwWIZM0wRztbdkyuX5eOI$=J;G$Fyprp83bI zN8vih0WIB=y9kTYh`uVHf#KC415K~3?4Xc_;hY{Wa}Z8f3PuEJ&>2$T+?IQaMrT?w zPb+f?vVY2(RlPdjkvB!o8kRIvF?!Yg8|0a3S(Or@;8uBUO?EdEP&ResuAiH)hEnPL z+$5Xvp|WS4)YWdxKzycZFlaM-Q?`G^g-P2+*l5^M}j8xzW`^kOQX!u#yj;ggC^qp|4 z8!J`?iAdU5hI8~+3x-o;zFJbaw}M(BingOgxO*3agerQt?ITZyP3@fp2>DxN$)kH` z&T3s>{{oCds;$jgQ;o&l`uUSt*28wExMfJ@v_;2p2EI_I(z5yQV6U`h-A?%zu3YCP zukD)pD|37vz(@y`{Co0?JmhrCyru)`_L+9|hb44PY|>WcG1g*Bw0h2P?Qd*~{vFNJ zRJ#(6M3~O}@jNh}fb4^97{DKK^+40Kb_5A{FT0fKs-!_4G@x_W;f=GWCmDB390SPA zgr+C?Zi-Xd^VChP_(a4Zind)zN}k`2c(V5B1Wqf*nB*Gx1Vm7(dQ@4D)dSQQoeQtW zMe(Nm?uZ6hQ;hWH_^9S9(Z*X2T|4zthwSUByPEE|fH*5-Uz7I+wq5qQr$nlZM@I2? zEP;RW^I)6%4>8IYm-_HdT1VbzcRsEl+ivO0vfUN%E|$ICp&_(o-xI@0FFb|6ePw7i zoe$O@E}5FVk7>;8oBn6c);gHwYX}$+kjc-o57mFD7?yUHktQd{lpG z1~<}A&4?;3Z&pgKe%h773=c&Uw$_zUKxuA~A0<;FFKyX?lHFF#)uN!~jU~W;mA)@=y1Bn&<^W2Y z_dJJHW)%k^l{fT+`g0T14hYb+pcKGO$d`F`{Z)I<dy%O{2u!S{ z!C1;`33Xrr)Bd;xEMv&XPD%LB`OOsy{-!K^31{}2|mA&Gm z@8R>9dY>$A`)7VpSltc&CP;!HEsszjyL0E#=j6tXa3Qj!2Vrw+J0UDuduA1`izujO zu659B6Ta2?_bqC63Y~^oP9l}`M%2ctdYs_IYOFqpM*pJL1)HKV57$&b+O{i<&ZPsh zsh5^ujG{ezM(HFf+~rBhD$5%+WEZCq6*$LmvK%E2AD#k~V3FPZ<_JXRuZs6+(qI4~ zBG7j~6R4itEqabNL-I860H4qGvWxpM(4Pj`*HghMfg&1VWH7N90REB~JU`?$+;8q7 zJk2C!*q2QAYX#s$tj&Z$2G#<%cM41JN!d))r}1{wmsvziGJxmrqw(rEUVI)*IADce zlTRjy!Wx92$r;P6R14Sd5)^-n4JL5=M{q2%N2B%67fwUiF0~-V*F1@Z0dw?ezqvG` z+1H#O1v|5-1tAY>GZ97D{ToVaXxP>MHo#O-VzA^V9o*xAsfLq}r>CCmrZ-ko80Fu- zg(MLf$(tww1m8ezAF=E^B@qxO$ipbPd^G)1m>SWLHVn zLqF~+H>`LFo9=W7oI#x9!;b;P(pGSKB21sautUxsd+2#5tXtT9pQFEkGvMuzI#xIK zMbQb^Ij1?+muykWaYm>yDdz74tCo2kr@CSzx6&urm&A@PQRz9Seqv8NjpxngCPVm* zU#Hk?l2l3KKe58Ccu2S@*-52NGZkY`egY6jBC2;?HsN-{F=ab3cr57{v~gZ%F(F!4 z+qq)v%y~xUZ?bl(J>AJ17Ic11-L77w%ZnE>S2lX3V4-Ef6g^K848aFRCK;LIwNRm) z7DWZe#<0_%TXBLgZ-KxUsbfeIG2{zPw80aUg#{)lkP3tCh?iK zdz!?7_!}bmU9M~dILUl7%PP;;JYL8Wj4s%aGcX4<*1qTkbZjwTZExdTFY#{l(Qo*X z@40FS=z#2KuIHjWY}t==9(jAoYgNE^MV)K4s=r>(TIw)LEWz#MHFy`T|N*>9#9Ty)krmyV4O4=#~Foj2`PA9h5iwJ6i0wd-~QP zB^qnAJmTEq=0wgvTRS^=`!Ck6<5XF`bLaT4VUkgIPPLQ{Y*O|9S~Hi=`IB65^G_Qf*wz?5tf z_I70iyWbTvetMZcM=it(RVI*Lg4qH3+BcOx)^Y@uPFrUYRLD50n`L7760fE+r+PUB zNJMC($WI4Mh;ukX_EwuS%E1T2^Lg{t(YZ@k@cB%=y9bT~vtmUMKU3UQ6nIH{3>D5} zXeJ|0Ap9?!vPJ?S7mIk((aN!xKU|EjqN!;ep)C@{1DvUK86a4TS@q7ZA~* z%s99KUs<Vzhg_?nr`THp2 zt4VkC-MAQG$V|DyOfX@8taRHWo~`5gd-=SIHKFSyM8j8DV2P)h`e6IXdG&1_$F6?K zfTJENgW)O`ON77&U|(qHMJ)20aa}yUGSG5HXpnNFpHhrS@a|f+^^U$EyqN0{m9Ert zA^@)vG>q1QQ0(P`3B_s?Y>{N|+N(y;ksnsj z(L3virE2JPdkI7Yi1#`vD!{XQCX1>NSAxHp00(AM=Wwd*3Lt{WT68hK{F+H`z_L`a za{1o;&77f%A45C1=D9_Uvxtias$0$vq1H?!v55m^b~i- z_g#OVD2M!aFzZKQeb#itJC7&A2a51_f%>vaHb8KIkuxi(9qa~RIgY7yuu-v>QJ zWJQ!|G}~;-``n^7R4915Uz1svbiAEtLE#(rM%Rt z;TY0-dyo@$8~K+15tqJ?`m2Wh2YE>SAdmm_FUbG-BJE6_{uj`Z^#eN2IR9_ZK_i&4 z?)F|gr?y87KCi_=Mg`rnPHBP?>cKHM%HFcKeZBHKB(`Ht%(g7+2X&-jXYgQ+E{cW! zvdqnR+<84cjkSL{mdAqqn{+d*GMVZ>m%2Eyzi-s>JI?BMrQLe7FQ}quMr#x&C?tWF zf4f(>ZtZ3xP}mWn{!gWeRUaP{ibj_|x;hT%!3L9Jq!%z;8q0!VU9l($J?#Aunz#po zjCC4xNL1W}P1iw_;jkH<=n$KW>=<)*)`ig;UkcT6C5R#^Vdy9A3rcaKHE4u&fvA}# zFPP~fw>bW2J*0{ewVNiLKW$udqjfZYr5a)~3uf@=!PJHoUri7L-)Qhxh)SwKH~Wk5 zIwp3FE!mrAA`GC0W`aba=l=J?2^0JLc=8g_Y_kr-yE=7>thh#{QI(a`ok%cEZB3|o>q$DFazh;zdXk4hNVS9liKvnq!q>m?lmrBGlQAe zXCuxg{ay{I=Hq8G5G6tetZGVag~u_3%_LjUWV2MCH^v|Ue@VR0ToIO_=A51O%RWfi z?E*`fu&__2&maph-xug@OU{QsA8g^02}H_hp!o72QVuC0fd<@#ySSZ3CRjyCGt9E_ z?9n>o_}Jv(MYw-GZeparlOUQ60G|&AqVq^;hCze;G9R2pftEWXOliY^sfekJ!WM|( z#hJ}Wj8TdaQ?Z}_eDs2&kYD>EwFZ5m=?>mVDTIIH#L?5QVS(}Edow+Zd}XMjiUhr^ z^jR1r5lxTqIA)Aq?N1XGDOv<0<}*b*p63044luu&Adj69Kpi>O(CkXW3W<@-fdvx_ znKps;ju|16(eiu{-XmN%AyGyZAS9=sP^+r4e;&qvCS+O1wJ4*1OB(jZvSs~>^@x_& z&m)2@1~aaY6)fTKW}HUNTn9hDuTmXdAgJBXNw3!?q$MP6vfU#B1~D}Ku;G=Vm2f;g z=W_C5=P`#@e8xl6Hk5VW8>})B!17nMXTFcMumU>-S1}CEJ|}Gy|9vXjr}9sz7{dvN z^Eb#@LiYZq8EYvq%5AhOYI{m->1U5@ZxMNro?Z5mmSKxq6ThQfbaThE$H@BjRM%DG zEm4;9!F@S)-_%wc)HZIZYaU zxnTY+I0zR{SrU&+7w8rCkP9G1wyP7~apH z2_z*8W!faK?OF}RW;GV}-`_lAV+-};kQoWUJLJhvhmH>xV+FomLB8H$+ss(x#@;0k z6v-rMOn>6&@AL!^T3h`*@AeMB+wzP?#uIKcYZD4pW+qA7Fcldr=P0#>sCTtdO5$5f zd`P&$@5Gf96#K!tq!O`W!(B+gKg(n0zrs?>~v{-xfQCb6IfkBpZl4BNBG`9siKL^8S( zgH1gu$Wbm>jdP zP%}ga>!ilU&P4@4RVKJBWl*?TR47pR`&bKXYVGP5LO@x1l7mBb!mB)vG@j5M{j@?+ zlt6BWaW*BKEo~@G7;SDC`v126X9c?0O_5}!M29owvF3)WX_PTc3Q1AC6hF(>HEBUlu@ZJ7b+sE~%)hXKyaE=qgDBb#oL`5SH> z;PpkFaKOL*F_cwZd5i*D54f!mWfzUsw++hc{zW*>zwW)9L>zra=rQ~&h+=IUw$ATcA&wm%do=Yt!%4A-d74Js!yKI?Yk` zk?7>ZZqXktkAh`EIUhy9|I3sG5W3XjLK5{IBTh&bOnacn_~Uxv6t5oy{{348^vACC z02HSOXU2wAf*8>hrv&gNyn?_iSiLX<70QcB6izl-R$5we)D{US&_K313n#dfq2e?l z5uR7rPBllgl-%z!xVzQ6@ll**oZRuovynC8BH=5h+rEq72N)AaX&xx}KBduFRy_*C z)0OdZg|n8rn~WD6*e34k%Y_IMg)NG22O$s%MaNI;pP*@qFsUvzY4LRyzo5MH4aV2W%WH znu~JJf0A*`KmobTnBiwacxwEb;tiwJpP(g2BF;^i%)07i*1uR?4)S~!z4KEL(}WQ} zN1sn{Y>4wp$bXTc;TdAp8ji=Vh*UYX<_(~S!xqb;`L+FFu@#9SEzbSkL!9|7woV_Z zE7$sE@hScHJXif{_bQmE(TOWbjv|eiLuMu6ZezOtHhq9b%0(Jxaqv;MEb|yEL2Rbn zj_MB)O+*y_5r4pqHS)4WMX%`CNJczDh!S=(?FLLO7+ye@K*K__I=2oJWsv9=G#$ z&1Rj_Zq}}>pg@t3xwQUVz7w{gF7kV{haZCLhxe)oq)M(n~G`t_x^e7^{ z9u5N%lw#zPE)Gg>vVxT28hr+NfbHX&x2NP(Wyt^xc^AN>P10z{cR&AHX z;#f;;ShdouYL(nG>taFoow50T`Bb-qW0S!jJ3{{UOSS8$sYgexGCP2DCoHZpHS?-pq^Z<3M ztiJRa)8kq4i_$}1?A;enr3V;inK(qJtG9*hO)&#W@@nc(37!^UY+IIc+I(V_ZJroV ziinL4(%tq=m_4kPEDn$Q8Gx2hGIskWftQeLZmCxj$|U*&0Z(n!@%i6aaB?$k z6Sw#ocKHf+)o1`TSI`pA8nQKR5wEqDDBe}l@mVdmBpr2g9TL2n8q@`~oKK>2j1%Ja z1@Rnj9iq~Wa+7k)*@XOKM<)M5m0HCkUTQ90wPqLhKm`+DUVH^Po8k!lpmRU}(nEJ^ z1$-#w-FK{G#m3t@I07p;Ot{8!@T_Q!?^T#V99dCE^g||h8NfD-x6JCX(`~$YmCR#) z4v*<bT@(x)&d*Q=@vFON~M)h z53U<*`v7f;n1nKeR;prR7|y$gY(eoYP)ZNSAe5~^ismatT;{$rH-o9<&1-Po^$guWQ}0rdO+_~S6h6KIyc2s193NAp z&a;O`N2E@pqnIYM#enYwB^ERnp3Z{Rqx?BkA`fI;3ua8Ie_2M`nLk4GRm`;hr9*zc zpf~poO)!LJn|5x2hgqCS7Wqp`>TGGJJpvM{R4MOIA=1VjbMQBb4>+wXB6k0tl6r{J zwDoF(5xYJNsjLK1@oUVP2{s;!E=!-W+QE{eWTEXxAu)kMI=MyNuxQHpbHG*r1AoNi z<~K5u<}K!m3w%=ZQf_lUcf2c!N!r1Qte(JPKZkse1Q+jjul)Lqz6$D1n3YJEp#Yt& zY`B8 z1_q-JgyD%-0g62UozJJnO^0>sQ-i^MV)crhZ z!{O$Umj(85?z)gY%`Bqo*XB#TA<5moncqC)qjgHXBaJY(z{i2S;fuk~?QJ)11txm) zIj|H9j9p9M=uwoqx9?(7_U(tR(r?9n@Xw`G-KJB-DJA8;iEU=@WOyouT8t_4Y`y9r zR{Ns>Pqo2umInPA%{vnnJoH(y5OKMJT0Oa?qr#O^i|qENF)wf{{qq5@pcJ7v4<3Yd zjRE7C^kUx?1^twEZN2N^qU`IB$`WtLf*7SUo;6Cs;sw^V5$d-gRzwvzj%&Lj-{b8H zK#Ob3QALNm5JtwZ*RQu zpmU`EL>KU)0Nzeqfirinn}em_I*yh9^_Dj0Z!94svkX1yAyPuzGJ1n1B=208qP4NH zF~9$zd+GCvYytdm9fffJnd@+{x3M&~GxUVaHp+c$zAib;RSn#(# z$;1lg5d@p#b|Wm1k!0LUyotEP^Z()=&A_0@;t>fG5Q|{3#*P%j-;@0H zwnFv!zdn$SzT6@F0U0=I!vJ+ceyR;@l}6Um1~zwwOxRS%CJZfD&A^$T{3u4kT|LvL z-C)?FJvZ3r3FVftPzUzTTh5K_Xk~eGIAB&2rUe3(2XCCDbZTviY!F9F6>%E8xcBnexNF4s zN@}8I35Y6JI!nR18xKjgMiCgRO1_8eb!w1JfYG)?VSLM{=q-X%)euR#*Jj^NL9$AB zx41MCg0);6F!qA@aR{I6+gx(T0itk<(<)5X8`RXHS#!UC!!x%iI_^5HkMkK8*&tej zdZ}AN0AR+>K>bgL3y^nPj0UowCkWxk4f{K!!X1``$=H_E>Yp{hs7qt(BgjE8 zM?3q}O;Hy_zk-EWVG{m`e!y^1r03A-)a#ObDy-fGrAmM#o-A}wL>Pa3u+Vf|6Q|0~ zK7THB)iLz?JvJ+&AfO!3PMD5}SE52SQv`7OzZiSR?$DxUTQ|vyZQHhO+qS)8+qTUW z+qP}nwv(HEZaW|LZf)Q9w)q!kjWMeCUQhWEHJ3#lgAQhJ#-c&m5O1=_B&xA5?y^jY zkJAN^k^iGBIM$pYju|(S(|mfa!S1cuC80)jQ%V~u2u)O)2UuolIq<|8&#;3zD|`^J zRX}r+>SKu;x^p)w#LmVOR_s*R5C`jxz{+Ae&B5R5SftfH_d%o0f8qA)5OTYD;ST=J z!JbZFtlW13vz0P1;x~<&4|V3y-`1~sf$yZ11@xLKhvEB!FKI62470`h0-w4?b;7c& zt{&OOw7*0kyK(BsJ_2L;H%A!>+BeEvHf&$n_OnKm{DYDDRu;Mb)fA*ldbbBpoP2~Vk!-Jx0yC+z95h9IzwH^GS^cCDZ zrF0EHrYfNq60Pm<3u&2^l#w8>7=zgd^Z_w z!deNyG>n~Q8()h*tGh3kJ6o*xTN5hfM0_0hc8PNj+SQ#?iq$cvyO&9|L<0~)?+&O= z1eI8he%hhGDlLuXF9zR35a@7Z3pi@0csw)|@JMZpNS|3IF>390m~#pfGlQ7Z_(Cv0 zNwc8^gNVuIpJyC2@bT2Y3BY}BfjZ!Brl-U{va1M@Hj@cyHH2v+6|cdY;G^_^Uz4|8OG1!>5 zPaOJ?RcstwAL$M30+=$G6Lbrug$ir?Ho@A-flJ>9LmByanYWa(xMF^C#g?nKck3Tl zE>SCaB^!&*l$%4Pi%#cB;&4L)EucAmf-8WIr%7Ams!}f)J#n?=Eo9-ppbuO-s7*bt zPDg~zwlSW~zE>%x_t^2n$&u7`k7det^z3!^W$~M;j>|K_&XKh&gz^d>kXM2YfiF1z z75l7Vq)o6Bit6i`yIeC=bv=b8-DR^yHEEkKNPC_QBD2T;;ZqXKD3vh(7ADGXVPgL; z66F8CVwADlr~fU+&j!$&s^c zr+1PUOGH2{%DF2~K6diw>51$}zY9xl&Y_^>My{hQM$Du@Rn4}wfn~@m6C44fV&Db- zCCH9s1h+qeH~Fc{1_5hs`Hk~h@Z#mQ%og5}_-2?t)Rn^SF#Q}#6#l~hWLPeXp8J)C zM4jjR+U!V9sV@@PclV!Ft)26 zPU(aSQ7aIz9>+ot!=BVckL190&4xePfq6r>spf@8b5F}Hi}$-nx=vF+?Qw5|28YgJ zVMX}3c(qC#1qj4XO@y+W^+Ks1t?}g6VPp-0RLc>%?H&bT$-X@kZ zb(Zv~a!~!3WqEr<`G1O3W|=8xyN|~7*Rq7ryGN`uL6yWc$U+Iys`qN|HSwB2!bGRT zBT9Q<6k(i1z#gESmg3bR`wLs)+&vB%A~Ok}eeaZGMAB?*!7)7E3U-N$22P3kHz6qC z4pZmQexf$RgGjv>PYj#Zi5~1C#NUCT!9_1(-w1c3+xX^uS8areK^jltY>7w?L4B* zL%~XdqT3j60&;6VD`0Z*IseD-dpjcCjll7oxKdur~q zEMInlR?eH&JrfE;F~*vy2IuGNU!_i1#JV@P$L@uf1&iD$$J~$(<_VJY8{=WZH8R~x zxR`CBP%ctJS`s+2`stj1`uw=?lqUO*kYU8)pWKFjMVSs;j|mbF**6$Ze{8EiI=K|0 zfJBA*je3oWtO@0~H%Q$@ffLhrc?%x>N8ug-4gr!9@DCAIwBh;JgbA%_1>lmJoT^QfnMWx^7EA$@ z)1?0Tu9GS#5z#MmqA}2$maE12-67BT((?__iFeBiB2!4`%x?N1f?@yuzbs);d3hCw z@f1T5cd?1k#HHkVpxeC$^z*{xB{tDea++Sa ziEob{ch1aMNOrQAHz?hiLa~b`Bu6 z?=Kx|ZoFVcXHjk~u58=r-5!Y(nIBYk_e^bmIf-Whn{F-JK>BD(LyCd!2ORxdF=nZ` zd(KXKTX}t*Lv-E_S2U!4($#=&!q=h=!LIJUOd0VVA8T7qyr2Zt03Q1?)n>gGUR*iU zpuHH~@pou$fO&5xb5qfV?CoGQriX^mzcfPIVG!znf8CkxZ9fhzow(jHiqqpMW@nJo zT?>f;^J;QO&Z&B1cW;jRGiBxPC(eu+0#|s+S+$hAdORNf=0=%rW82vCWNtst3F*MRYA1oj}8+I-2R1ru-hVIh_-JxNsnVMJxD7H1Lz9#QS_@KP{&>c zz_mzY0t3*aB6RSP^{}APttsgRn*+^*qGwk%Co9*eI1+Ihh|xNp=ict}Ed9=|LpH<| zCp8!^NRHHn$&idaLhQF~*0!Xv?3*Ezn1nvl;s%?JArpg9apRwcFl6By=tUt|S_VHz zWU!LDM`qefjtlAxwI>~oc!z`M0fI0Trr51^8*RugzL+PFof8+5EpE!5D2_PeZPZm8EKsNKAE>^`0RtdxzIH>VRIhll;@#*F1}l+Xnno8)F_d z296%=HT9W|r%FD&2~VHg7>T7MK#zWs_PE%H=@Z9upa1y8HjK_6!T}f1Xtn`0JLy(< zhi@iSpt{Zy7`t+0AVY z=U!@PCk2F0(MtFW2X~0fYi?!oP@A+Yl0KXP(hTd2UT~rI0zOKJzk|yy3-jD0=q+`{ zYrvymM@GI5CYDD_wM>LlOjm&c?R?~26}X+dKC`A9NUFLjdUzUSz&CAgNTHi@a+@7M zg)g^Z=WM>Y&?^$}aLqV##0!xva)}d7@!rT$qE$(*gR87WvnCg%se=RZU2VS@ zY_F+MAXtUZ02$Ps5EP`_@39J_aqfEnAcK9PW*Ow|HD%dLnAdf@9H^>*!_qBeUAs_j zmu#bmM79#FOJ=lLbg@pL9$3%M+?0(1BA5Fkj+jy61`!oh+DexS7)soMZEh%|>dkhG zY2O=laM2bq3ToLV+IOP+B(DW*>GMQPagQ4N7ZO>+zKICITRu~4aD*q6n!qaMZ37bY z{lm=Iy2SwddmE5$J-xC4-2dp$bfOKMD zt>N7`len&S+lDZ<=j`#ROW`ntniusx=rcL188QewQl6kE+s6(KcDCre_JNUz z6VZp`k6(8(bcKOkO2W8)Q(KH@T&MG!OeF7+WtN-TaDDzT>xJ6U+0!M0du6HYpb@#{ z13T9bFx{ir*L|-G<$2kXX9a8=bP2$s!9=w9y|QHjcBfBxpn4~p$1Fa)v}BqF<>48w zN~^+-$6Tp@lzyCtH&FVR8c734h2$`1@pJ|1K6$u%3b^s4VLPOTyf8#lI!2(9x@VIb zJSw7oxV==yy-CPMS{)<-X4e@>aYpM*esfl;L#7!Z@!e5`%^8tgP_Pi?D-k@Md;8*X z1442L<*;C85eI)>VHK5M`IjbSa}fIKb_ql2BK1hN;bt9;b6Ke_)u0M43-Ed+&VH2Y zCSHgd?h!)xgZWSpr3y>;dYY=03R&U2)>YKG>QLCX8%v=BDKv!b4VIL|RJpB~Eal#L zm$|tF1==WJiS~luxqPq1&@wbu0_y^+Kh(dBB%QGKv86T^s}6X9%>LlTv@=?v=CIj} z@&*RY^`y9ZT4%)P7KjmrqW*egOI{N%f}j3ef*f2FKHsFCFS)=l5NzxtJ+NA@LN%x5 zS8vsBY{V1m+tLgsfOED~n2NIN`dooLSUp?{IEy+%(w+lSve>bOln<>&L-7Z6Uz>}* z_$!Cvh(_C$8t}M6R~1iQR&qSP-uZlMoJE~ZnF!K{wNuoAhXJfr*M*QSaF>8jZ;f}?B9G!mzddB)>j-N!y09Qs+E z9gNc|Pah3&kb!s&eWtTyxw2rdCzms1*%>`LQqW`a*}c%dTaMH$r?zh2>&;$hZpLe5 zG~<@&9ojYp*s4|Pi8ZxU>#kl5WU&%yV(3N{Ob-L}me&M+l!{!cf0N{%HI9iv#i&pT zWqws_@En3dh8&^_pIdcJ=OIkb;L??RN%q@@!*`%YG1+AGLav&^FGtG$Kn(}G9rwLc z^C+6icl&j8QjccDXfacX^64EaEAc2Zl2f^4{+gf_O=$b2fv4xA2E-3scY-F>`DU?v z6)rsK3K9&r>=+_9oZ^4yMfEirpDJ6hkTv?zht|-G43(l(cB=kaAUJlGDEG8HoW>!E zi#E|;@b=$fyv$q0F>C8 z)jb(4zUBNRt9H9~>EqD|PXyr^CYd!AN9lyQ*Rirvo=MLU<0?Lc`gShUFaE3CC|!HrA_hq&*uvkoqgBmd6MAA za>JI-xwr2bPI;HUO3lwS%)eyA@Ntr}TjFzEQ7mReiduB8Ug7?G2XxvNX#MhwrHl{;0KofS zw21#Z7xEvp!6|K<|7wE1bM*S86b1pcTQ;#8nbt_XOOd;Fc;n}yfgldKh8~PwQ_Y#n0{KjLpa*^;@AawscmFZfu-Tk%))246I zNO~ib6Jj8-Y?2PA*VCP%2Zqt()Adr&_c4&ki(ywt-!?O0BVhU>BEcq-3v9u>>In#~ zmSm@{P)kixiElN2cAumDKu=?_Nk%10F{Vna8Ty&=ERh~h-$ll81VV2-!Dvh&RU-i| zjvyV~c!cQ}VXd+qoCV}A-CrXiJhMPQtMpU=fFK!2zb%wH*!VhGCbcQm zXKKXI(S}KXgmdm-r5lJmeV>C?KfnJU1etiGCpW_G2RavDJK za;Zt5mTA`O2!4$U8aV?)%GQ18B(mdAx@bwSfWr!Ef%t?=*d-hji%EN%A$jY4Z1R2+ z7pbBC!|zfJ2u3Lld$D`V!X^1SSOWy9@<%ZvejxAA<#G0Lg(Q6#XjY<`*0jcMLko!q zI3vam?o3}?nGxc&_=>Zm(p@Jn$!9U9*X|v%+`zZciC9M<TnIqQWt{!xMc6 zNYnS5iZeIXKSdK15VY0PfehR*q+};}a)D!>4$xVN6C50Hv;iEw({w{gkksfi@Yjai zJZ}JcVk3PVl+9lCWb166WsS)cgAxWv9jZL~^rYXhm-nGAP=B z-^=i!rh|zA9^VGkxj|kLK2MBswSU{8#ECI4NWG8R+Oti2Us%%vUJRK)jPqS<5BE7S znBUFwCyKoQC&tvjOonSw00^HSb}TiZ>=>~VzmRfJmrSqPmX@BL9=2i*cJx*Yz-OUh zGCKm@JQQJN=mr=~APrNK=9Yc^*;fnx!j{(^>Gi4tz-_5JfD{y6y@gfoT6TkK8%5Mb zdzqm?3>vFj1MELG`D3E@R?+6^#&a^#c720v204X2S@&3$7S*^T zTNKjxy>nt;BfJWI9rQe-u0f+?INmCNZv?=&AdiJ!vK|Hv2_aB5Q+(J+N!XY3tO22` zKd$LWz7-@(LGu9-ENese-QYK;h*P6h7g6zRH^FX_4N{ml9#wXtpsQ103hm1hSeDg#;7y7n`}{Kdxgw*@pdupG8ok#=%bv6@Wn)2D_8Wl#MEi zB~lk#!rjeWUgRYo&B7my`VAe1;YWO^-^9DhTU>^Dy=neg(2O@q`D`VSM3>T*l@eDx z?49quHTlc0Yq>@V4sd+Lo}YHSlre7~Xi-P)zDHf#Z$>2`91}C@9}ff6+c&!w9&$;l z_}Kjji@wMT=H1X1st1Q-+60O=AU`{pB0yek;!~)+bJ*YBVVbhJ+z5j69yVl(Sv=4D zT=vK+-RuOGd~dlCuv($ii08jt0B>OGqdNg~0shn^5G6)sUcEQloJ+08kD$obr=rTi z)LzyRd(HiR2hitnZTjyQxLc$)A%*jowaVT#qTD~fI_xLFElhiosp+3xp-2`g)M6*H zCNJ@={B;xJ*2K(SZo)F);X*e2u0+{ttyd^Y#?knp$^i{{W%j_oLu-B-LhA-7c@@ez^TLE^-PYVg)D;y;6P=H1z&^e!T+8; zE_3<1JYSy|I)*y%vZK`pO765&jkN`pcY+yh=Cy_=Y|?6Xf|uM`w@J)zPwE$uk1q#7 zS@;M2%-(1+(QQ`+*AS$P-eH34#PcyGA?NrYu~!vTam^DG6oy1MjOBBUvxdpn7$*=Z z2@NI-|uod0~Eod-=05~Y>KqK{k7RT|8$Epj`&;+o9f#S6SAWQSuB1X2}XrNyAH z?WO>Fu$kU&Bi9;!&8J6p*Pa2me?Ukv7OGD z+~p0qspN@pe3QPjMi+9jL6E$&5@N9xaIz`DIvr!wR@VA4sz?&?VwK3K?qVn!D*{ru zr%C4Cnp`*@#d2ay6m@x?vt$_78aJjshz&@*?wvUb`?jvx|LVmthd&3ienGokuo@P;R}%5^?`6Q zwbjTKujwYp=J}eaXM~eq4aMzRA6{vQK$|6oLg#_X2mEKzPV)TGA_8W2^Dk;9yF__N zv8TNwG}-BDi-p=)fusvW=JDALHO8!W_qFQa>GUG?AF4%{a5ww<9s?y~a}I8!6cPz^ zN&|TV*7-?k%PO0I?`LBJME0O&)}-(lX{kzqJ57m3iThcW;oZOGa6T8og3Rt44{o5{ zS<*o2r@Sv<<_-r%w>X(RRwSs$90760n{Yvsmu@0|)tR)1Wuy?31E=NCO~b9J!Y$no zgebT|k$;;JpyEGME5;2!UuK%ijUS|tm!8K> zLc&eP4~<5BAtfeF7FX2WPxNyUWZgq6GmkbrpBQ4!oE*FlS+Ys+G@UBWbHM;lE5N*X zVbfEu{^Yx|OL1VEY?Bx z23d(Kd@cq3FweA;!~s>sdGX~?mO%bVEnvL2u-PtJt=LA7Q8uGgDIB}x7t^$_2cr?0 zM7JEiT>n9nUwPiG<8d~%Mw+)nn3B4O5F5@i>kKE+9VA(M0(V=5;Qkf49Hp!cGwpl(eBX#{_jWna=?gA+J zCzq+m$rfwzOm_M1z!!hqLuUCrlG8>nIhh8%{m=_oP*!DNG~d7uhGI4K?N9yL8t03s zi=9tUdDBG{0j`}{MYUwr85cj@0Vhy-LsuYadi><|pX`E}{PnuPavCrXaUBymli1@G zS+Rz-GTJ$pW`0c?3p7yXQXhbYrt7%X0PQXNQ%M_Hqzw%J_E{HE2ARK+0_{AGCxkL$ zo6J+=i;}fCvDqx1=OOVdJG}209nvz$%{llV$~P9x3)yW7%*&3K72cfHPg^+R9gEws zql!(7yb4(%9Se9_%N;ZmuUL=Mzy?j{+m;hHM?E7SXG5&9eRs&wpNLvRoy<`plX-aB z(8zU!OfxjxV6db{uTSFY#lht>?sqqZyY{e|aFEV1Q;L&D_bdzdIoADHfB6!H-KyX$ zGf-%VKKRo!)C!Ja04@19y{N-ldx&=g9o;YO)I$wj8#{&W4F6$H3W>`spWg9QvjkmB zb_(}Vnbf=YBY^`k6X@!1rm&~8m4)#U5k?R{iLy3oo7D+?*i4wGZ^D5?s9y1p)&vtN zs;Y*;&y!tT!g2=&W|6cRNzBLe3#t0GS7s=wo_o%wN|=HiHg9e)l+coO#`g!h{?c$0`GyV$Y=U2wbvsN zWEWt778@Er^5pl5w&6k6GVIxQxiQ68l|9&UrnZH2U#w8R*J;GQub+@M72kEgYAO8e z{UlohE3Hk3nX{>>1&$x|k$@%?#oDE)52D3aJlT*PlBOU=2rHa7M#hWx%5iOhS-^Bdd{!z{?D8H z^VN2dezN)W%+gJjL_wb`@3@lE<0UU11+<3n?4n~aw25=Y7OI@?JuBuYwrZ$zY`JoY zkQ%Up>AN4XRmbuvx3d-4N;Y%y*> zl6xg`Z(Pn-e|R|rQI3T+RN+fQda{FBzI}&5X&i6d1!uPj#_gfYoOB*$Jyz#i)HT%k zF^b0~GAqojOT^0Cf8KdkN%7k#a*de9MFJ%ON)NREtRqeKt;e-&y081S94tJ&m_kXa zouPcIpf_FAK0#DE=uWT4ZV<^X=(|c-YG!-zt*BJK)1)@*45OW;h)ZyttlQUM8qw|} zqI!<<$nDA7Ab6IhnP-A4_%?(942ph1iav-d1(P(hu*A-auMwZre88neWYrH&8l9Y2 zULqyu3$)tnj{aBbpX9-_$P3cY#WAhxFXRC$H3+g>*q3|p)3laWPmTZs^EZ8qt6U#E z_T^L-`@nm1O^e(;Z4stioi;4eEabLJb6h53?Z}L#>%UfrO7))+Gv^D%Pk6FY?eex& zQ3PRc$D4Z!Kz@z~S%$ow{6?|w^Aig6{hS=xyHN6{AWb-&eH>VL3bkY=-w|x6TftX< z$t<9sxz4ka@gBR5;%(RH*LyJ}Ss0#1x*>K~Faf)`rgJpdqqz$7082$5d$4GF8no)$ zXasp{hab9pV-vle+J{XjHs7aAZ?h{Mvm8+Z@%!&Au?mH6^1roSFGkV~ zAd5TgyjQP_UrW(r)jZsAJjx}+o2QA71w{8l-WKaCbPKnM_qd<`n3nz7NO9a25bk+0 zOf>EGd;~(eB@pb%sdY7j|8)!=oUNDZuV5po(n=NoUJU&W9NGS2CQ>#^$Nyi< z#8u<@FJ=M)q`eD)u#U&^IP0L!PQM)PM)b-OtYrx2^YiYY=H~T2T=#$g zW+a5iMUbUR#~*)jko!D!HKMqsWKSefLpO+iO&xOoF!}R>jfrIlF%+sLaj2H+lr+A{ z_pH5aI%UZ@KT8C;ehY~0qzdPgUh#`4N^p|iyn-{yU&=(OpXil}UNAtXK#D$BBxjfv zfTvD*#P)n7m(zym8mFJ)+6{OxNOSf8h!j+?H3ibsrqv%+R+xP9!yA@dj(7BOCwwa$ zAe^iFr5($z9@QXIm28W;+DDCI(Fd=oALBjqkbbHy_+rtX_^n`DDUtQdK|8}31j7ow zVl)@)5LK>2uR}pD2X;HB9%Czi!shxHxsYVpQmQ z;~cyGB)}Q{WYV73s40DPQ+8a?+PybIjT1iv3-ptjxJiPj+&w(p86f?iYQ+?ju%x2i z)QT{stfM<-%^9~(;Mg<_G7e#{F~^`OB)xjD18SmH5~)%s?2uS)!U4B$?B1*shW|oL z2&o5KOqX7eOB*!Fs!0sT4MOJ|;m#;k4yI5~kEmmu_l49-a;|rH+D+t{i@xxV_VtXLi%} zZ)v;;KC~&eYqA^a7z4P)fDQf59{+Q8|l0Dm;{O!dz*ghmwJpLUxTl{m4q0>Ldq)-EdwSqC3 z{@Rf5VM^pl%!%cTaQT-k4#zm3^4!Pru7C-YC5QMa+P5FeS0fiVYUXoqPjJ#Xj`FWU z@=9V3*!0KCZa-dN%*(enIotf+6JX&4PNr7_(W^`O-{D^H19hS^&}(nop&VQ4lc+A{ zuNffYR}6{fBQ^G#g0sV)-GAd?eFJ}@WI^v8gi|L+xcO#sk!r5)NqCoy9U(4gJ}4g! z&rXb~_`ta8_vK_<11nnlcK-*bY;&o?YV-FL*vA9_Ao&l<;(rlmod07YT7ARzfF03$ zMh)HqniG(F+LjOWHV}~t#{h>bF8l0ChZa8>tB@+cfvL}`;R zR+pkttJ&8OYl|P_FVd7$C?eDqd)I&sq7Y8%iKtp6L61ldSFXM{Lja@*u@tsi2B3^w z>=f1v;6_l;KxME<2rCM6#Y)c{jT0s(m-=h4jQDxtIO6Ct7J!IhB6!rBSny7)o1}rp zMbt0qggcBwc|1arViq*D5%cNWzPPkzF zcN-31MRs3k0`DLrne3_~2v{oOH*iG2x3yy-(L!7iP1`cSXnSOFPi3Uc3L!hy)){sH zi6aR})(E3-N?kq*p)73@9`@?80<6zk-%m@fLMwR3)$btZp@ z+!1F00@c~47=*xgjAn|(d?E8FXt2ID)RZ6`4X!51wKqfV77ed^K3zbY8?+f8h>ZEG$M4KTT&UHdDwscV0x59$wi$$|gv&l~#t14o=`J~%EenSbn(z6X~( zCfGvkK4{j$nxJ$*s6OE8P?QK~ZEr5pW+2}7_Q6sI1nFU|{+&Dib-)nv=ss+3j@x6u zi2Voio#5RoGVk;;ZxZ?+4@~JBLGgRM&oe{tF?FPGIXA%B&+SZU^}H8gD#5=8LP$5J z`&#O0f4S}gLjBP_;kL!*Ns*amL#0ur)sXkz`6bbMD*~@_=pxX+Q|fn)`odzeY?G+ zo|dmRq*hqhaF5R$rmdL)&Utun{Jw$E6u#T|)Gkybz$-&aB8+9dx5P8yqG*nvC%MEm|X8%{>HubWrP1Y>mxrRh$Lv(Irkkird|Hdv2MKcLb5`G zlwdv;#Dpn6@*!hVHoNGzNn zDJjvQ)zKbxBpUA$*dTBzMw0cxbi|RXBJXLbk9Q32ji-35`N$`M+|jbrsTalmmyy~c zTp3Wfx5FYaVN27pzV_Apx$Q`(`y=0t#>YJV10t2Vcur+=a-o#Gb?=BL$o6|b?;c~t z2;)hUnd|Z2k*kjIm*@Ei75-+Zx3@oJO2Nkg=Jg>RD$WW(jN z!fWp&&6fZ6+`m`Co_Y*qmRN4zolAsre%=efRW4?$W&gI^YTgliK$~`5PY3XM6g0O( zg{tSGySqMEc8x@s{+Dq2Y%~k%=UHeUnh2Jb0kP{piS&XmjgXll7U!m1Z2SppiDdm}SjNkP1o~fupptxpy?dj_D|?kMI*s8wp|0zpJP&(!j(M z&wn`QA#$&kf*ioimeH9z9V~PqzofDkJryAqjDtUKCW|C{Um7^(%af*0B?(7#0?YYS z$~ie$Yn1m*QHskyEF5oFQJW>W#I}p1Zt|2M*5f7HBel_qP}je9?|l!G+_;#k%-Ptk zr10XoN?GW@gs-fDv#l7w4j%CXY}#vbO1E$qaw=}eUa&xxb<&uPA706}$~C`uYC1cdKj8a!ZcO$HU*{rtf*ltd(B53SHQn!pydqYm6mwJr_Q0*Oy?K zQwLmT6LUNKKT)tO!3(H|4ck_J&0-zJ-sN#W|HG6VJ>X$3{tZX3!UF){{x44lD-(Bn z1B?HJW~lnFZF%B|zuWSVv4K;-G>1nGP~qeL&@3=8c=#ufWhM&6%S0kYOkLTE6#Dk) zv1k#COkD6(#(|Ahzn_1d!ZF7$DIC#h2@M>E@a}AX^Y-)o6lvHd(0E6yV2p$`4=W$Y zizQJIh@1Wzflq{@#>7KM@s#<@ z=!e6D?&+y?$bTsOaiy)A0p=N*Le(MSN|CK-61hPMiy7B$ge?pQ5F0w-GeS(MdVQN5 zxYgKHQ=*w|;i;=&R*3}(p6#3Kvzld#{lGPQ7MV5G`d zn?U)SIw~IrRIjIL0vhVOC<)ZntCjoRRgd)u89sx@m^5l7gSl%vkTgm-HcCcV!>r^7 zvEHQA2hNSji~XmYXtCqVVH&ySH_%0`gC26l{egW)a3+v1@?q);#ONu@DrdWgbgKr9 z_$!76-ZSw)kgLrxXuhFu0zKRB)!ce8=UVK)THQ2}$O(PBzhxuA?oW2kV(oqS3U>7- z$knU-8pFu5Xi5$IYWBPsz1!hV^v+bX_NcZ0xOCML;;5rnqT8YhCyI5xyY407 zpjP+9o)T!!_)Eh9JX@TJUD)3 z_W6%SK?&0~HrxLMH&h1d`Aa{9V=Gi~Q8N>4lX>_3*7A*b69|X2v-~C63`3#I}OLEoGYI^F6DUV z6-rv#s=R3+rU8=6Ifk>9L{IM1?y?-icW(8*N;wA<#s~x1SG7PfGeY!^g68?xR3b9l zOd0DwxY{@30DETT%3?=7axp8SVB~97(+A>kli@~xDl3&1(3yKxY_32q7SwLh%6GHz zw;v^lI`zzMczyJ;qp;?46u8Z3I5w!9tV1+1uBGP0v$-8^RQ2rJUj86P&@}rn6h`zr zkFyehV_<%v9e0^)327mCv@FiuD>I{QH$W(9jw#J?` zN_{RSM{=%oW2Fnkn5T|U14gd$5@h!EPsIg%IP-Rmxx*0f>)`FaIN|0Hr&?`_i^JL6*W$fE+>mrpU*cD55*VNI>0(&IjGHd z(H+v1j~jfsyhxPtj5{5kY-97vAxYr^oiiV07x{{V&n6@1R@YpUslIaQ66Be@+7+5; zpuM5vkAmonMNOS0BTO(d>C;qT_<*ZO)6DO+KosW?EwIa4H$2sO+!?y?{GRxmK}itb zQU&skSfqT$jWZcp?}t}+TogD0eg49ud~z^glxV@qX3Ke46fR9zKOmcIhDv%7agfLM;yFp==P1#OsZ9Rr32W~v$LI})Yd%S_f2Ps zR%xUDr7+}${Dl@v*jsF_CPd;SVuX#J*LrmZ8K*v>sjqhxVsm*ubEUA5jnPsr%EH=p z;4aqms1oj;F;UkTPl)^bbS43bYbNf!tvhl_$*-n@aE*xM?U?FCWufYlh5X9$pnZah zNBiMf`}wnV*29Cqv*M;TFZY>5i%9n8hr~N~+$~TP?%~pV6GkjkrzwzgXq0NC;VG_! zuBJH-&0y1@6dVP~Z`I;#G_>byxzDo9*FmV3TDOZ@d(*_&y1taSI|^bsZ{Tm8X_Z2gPq z03S}jo1N~jI~3nf2&IyFnBTq?zCgcF+1=N*r8-uKcG-0~+W8L?WwW&8FqA$btMcp- z>~a8dgehw!aB;6%oNbu85*9L;;$>cupdo-BWj~~9r9Zx61b1W2I)2 z61NF30>ObrQ*kh%q?vYImgH##%+Rau=HMeJse-5kY$ZQ8q$iRWebZhtkU%ja^%6NN zXR&)S0i5s}*P-=91~q0cI5CSgNgeFeQV7k=s^m{eNo&34v9zuAXxe9le44%n1+%z~ zxJ5ouT@Dv5|2+2!IrLj*Ml5!8?6k6_1q#rTL;58iRyR)-_>iVbLGR$AunG1M6UR?+ z*}PrjN5n~n;xu35CIua1iB;muF(T3uOW+Nq1MyT>C3^ZNgVLvfXjhlL+n+pLf*;ph}+wKs3WIx{aDvr=h_}ym1-jGzz zfIHAB0*AmoIwNa}g$)Ok4wP=P=M9;6Z6)(amolCgsS2K%9?Ufj z)a}?QWG#LEX4bj$(ulj>ntWUpu3d4^B3BXlzcqezkSQg?V}t-y4YXdaF)9y0zp5xC zZtmVHgDaC3u?ipNdH4$9+in5OthCH_FGridAItAxPnlj|IgudMf1A{qsbdFmfplk4 zJO1?F8ji+C6TFMMMz-V6s+SIXzfdn$ckb4aSheeGl+K+J%{@M>l;uxKSQ_6zdlnYt zxtB`B(a`?1$Zn(<^=ZpbnFu<#n+}U;$UBEqmN%IL)N=hk+!rqR8NIvzVR_fxJc2el zn*fzI&h=g!A@d@jbosS@Hp;fg(P=N7=q&bVtD}iVCKbK>3J=-0nNq{d zf-eacFu4L9_-HY1e!i{*jU_*r{XTHSt)QIYSy$m!>WA62S=1WSi`RI_2U7#1|1{#&q-JBcE)M@aU4vd+4unY3gnHw znYp18I&t=!Y`lI)v0m5qBu8*_Jx))}_BQEK*$adNn`qJ|W#6%j1vx$U@ zL});EWZ>r5z!Do6WM{|u`{@h3iO$@nbp(lDo>{(?G%VV)^)zLXUqG9Q?zsG)zSRPdC9q zG9jac-qu6C5$mJhOE_DF+GEmIp9qZiaRl@Iohknc{fqV+mP9&elVAtO6oCxR|KU8I z7o#bp#!!eJ1e`Cy4yqS^eXN^a?{{R1ur+p_rT{ISkf)^$`9}Zg!(nv z>(bv5?^<6iw`YfNaYwkUHN*4O*7MfFvxC*YH+!%wTWxSkX2-AK7c)&`(35qkkhZ3t zg ze_<|TU;X$ez^pizK&!-}`UAPJc4&dSPDv9COLCQJl`;r;hBh)lOpkpP%$-NG!K)}~t>R+!A35P({cqpPmy9?*6q z;V~L~y(CnMlqUy7O(9XMlr_(y>NxoXe z0{Nz@!d)~Y4KYISH8FaT3j&kzRSs%6s*JAjz%{~`%0gRByrEQ>=c8i0CoW7XJkYL% zknEH^NL6O*UFmKQF2@xy-Wk#h4T2smZ}e~Guh@a+VnnG!?Pi)}-8aHKOeH;(bhWBZ z(XfmQ=y@S77H>!Ul_ZG#Uu`4y|zZ8@(PWsX97~sC556x!KNzqlJ>Lgft5`< zcnfUSJ)T7JVj4H=!rRW=iwcbC%QF|BHF#tZALcuPennX0znO$*0~ItuU>ZD?cC9a1CX|+j!0y>NpWQ`Z zjGzeWhbz{8I0EfEs8$^lC@IJ6kSh*8HNzN&DC{ zO0rQK&P5Pxw9d8=?fcVzU0LhitWh>`z~&B9x}yDlaQ_>1RA+HLG0h+%WOiOcBPY%( zo+Fvwot&2VEbMb|*Ol}Bv0~N}xk>`l=xrL&bQ!84xM3SjlO7ELx3`XdHNWPHR%_`& zVrI8MYVpv$MKr;ot22$SeZU3X3vYmwSexyOxV{l6x}wDRRlPteBDhU0Z$#5ZLXI7$ z3N5MsL=^$41Lf!~TJhurge0ArDETV@g39ccY`v_fugqX=LHV{=Qu+nXloM;wva{V? z&HY7f&LcQW{&FHuwf;#LzO4;8y7Lf%wN*}8mv(@*`g7zW1wu%>`pNDk9L-2q8^bPL6=9DY>7_3n;Z;a-3;C2o@f`5**c=@TjPy; zm0i|NI5ceZ_v2rVmXY!khx{|PA-EaIl|%N>pj1Uzd4E$PS=53}d`4y1a^yi#baPr^ zko{ZZ*b`d zSQWiS4%bThMx9_%O?=H2TAUKqa%GTz5MT{2^%!BZ9JbyQ*t8pGzGo8_rm8XlJG1Z! zZpcjs&h-B0G=a*N8&~pukB|7i#Qob94q*NL{n?nQByj)lNZWvzE0$km61rsfY~%)WJyB(`~;#a5KG{c^Vaw>#o|5Hay?$MU|I&t=es9 zHXIv&?N5eRRucrub7yhBKrfnWg=6j-PgW7l|EJ?QLF;6$%-UWv>!53GI{fs}N96grS8B0I zZ|}AK>;XRJ*M*$(3bZc;?@+~{;CVzgXV-2=A9#~DKRnaz6{Oq~*NKdew3jDH=hzVR zY~acZCg%i^o7V?iUs3EXSI4o$;CtY;sD+k>4zkn7T>2ZsVMb4uG}m2NjZrluz7}A| z&j)!;(6gUcOcYfk<@ivYym{R-~yB5?m5j6n1+ zD+LD=v+pg!|4Nec`=7M1o;!73rc|{Fg_6*pDr>?pB^o%$`#~p3${$(o2h1JAMpq zZtgH=r|$qbUT&o?THQs?RgyQ`T`8nr{&2fDA=&Fr$Kku;1v^OxHaR>KSq0VCB@ZfU zRz{JVlqUynk+DO4I`X?z1z=-QV<>*ZFBgeO~#{gu?O7Cr?dMTExo?PI7>Afjb(U-Fqm*jAUVor6$kp|-~myPISHp8EU68o4J z)&euVszzPGF+_W_x&ZPi8ux! zZ216Wpr5sxVIX`Xc8b@u4-*)gT2L}1m?*>I4=lemMe6^yU-e5FXO6i-bqmO%8z`hV zz{Xr%A4Lm9sT2@zFA!#zIwbd5=EUjVdn@vel`}`gG*mTij~Pr?$oBhclYd8?&|5=0Vc71HJej* zD!$UOqu+`r3vu;}Ydi~^U>D*SV=G}bH-FbaUC9v|-XOh~b)4pY)P}%*By+u%eQrJv zIr6+<{O6-I$Y$Gp``+_)eY@}e4Mgr>;%MUFVq*M%6?6Y($+!6(w=Tg zw|65XV~77UrJ`9IXJ(dE1Vwd+R7b6>=~Br0dYfV3_!=y%SI9Rb8@QUyOlAG#Inz$7 zKM!2^5FB8{?O$Qg_j!*Z;O~!^xCW93mJbvd5S%#Q`SkI{mix|qL9m5vts{Sz2+pGg zr~WIz`Nz92=R0O{4%0{ust$8NxbMjrE22RP zgBxnxqD*1L9F^QX$emEiB5sz1u7DCqHcp|gy^mX^<`-u}*vM(-@os}xs)9YHu*k4k z&;U@U&TfDLERkd~#j@jXPN)P@m|&yNjzem3n?hof2(54diVmmyCr}=C8li^WtWwN0 z^V%>XckaV0(j-f<^if(xDzbCkIlKnef(rJVJdQc>NQ2R>F#2=SkR-{em!!zG_^P#I zBS~BkS|f3oPl+IjF;|7ks7l@-5jKU5s3?29Cf%F;q@hOWoW^T?xD~L;z~(cr#=}Eg zTR4PKj$dx|?`CFupjvGq7rg1Q4UDru?{Iz~j(M`DNaM|}8lchhW~7DRw6V(9X^dD~ zXjC#yMJL}BHQ4tHda7q?N@qiZbn|9j)QO z3h9FgFNvJx72PZrC|o zL)5xXlvALAH^NHxN-1xXvip*+_)^%nF%cV;b;v5Q2fYQ70mhS|@KXPQVgqwTc_awW zTH+gLo@uynEIJ`tN8yYzr52QBYL-@j{E$_dd7r|Kn9c>WN+)RE(R`KxKX5pbo znV~<$rYrShw&DE`Ygv5&d$Zy~)d*b1l2|nWO_$n;4SVr1Cou}ccvcJ4!+7bn&VmD`y2cEg44iW*bzAOOp_>#bWRsIw{vle||lCzzOQ5 z+csBSL1$>lGBuO=`5m_8sar9k*PmUmBkOKiIxfK=kJZop=2 zs3&6_B!{Bb4l++&_66GTKr-3dp=t_PgjB}x$8#nTRd^C5Mft)YUfqZ-N%-CH4j_aj zOMU-4n4ELn8vG>K`}Xrrr>v5CEVOpe(i~uwT%%4!3s9`RX1LuKT=H+mDt`hyS z(-m|V%@C2_xL|Djoe0Az6(KA)Xs}S-4^ zVMX#6h_OOn$hsd&0r|Yq*~5JzD-45RGK8KjEW#yzlCfS8LX>mt*9=pE074xR1#FZAN9URXHNlJ1#ja=jEOv0vtdG#_!HWw!sP#u>)IbA)aKMGK2uNeh z<)EVAivCqJAmGDwq$LI&n8mD84J0I`oACC*zWIj*#H1QF2j~}deF16`h5Kh@2uLsC zGk3IN5ofo!bEpL}QL7;Di7Hv=52^p4rPn_>_A}r&A@77e;3ofOs*-&p!`|Z!?Q~8-%TZZ>>@mpX< ziUsiY+u9HCI9(YsY1yf#Ly60^%xvF)l^e(}7_>u%9=NgM-ZUT+2k4;DeN$fT=c88j zJ*@Wj>`j>?d?M_+8WZq8@eeYF+MlK@#n$h z(y;;7nc9)h2S1&t+ED_<46JmJcbR|}bxP>jm-q2R0{>UcPagf}1!x!_zHD04Yi>XH zX>_z(Gv7=2F}K6tir}e@Cv3=qWFRvt3BXg~rN?;z7tTP4UcK3#&Ab>}&oE{ACBU0t z1o(J(1Ox=r<`8eNE*3uyg}d&ajO7LcSZ)OPZkxtPXEOrHYL()DSUlr@$tU+W;AKE- zf@Q)-1G!T3Xa40E${3F>7v%3-xl04)kG)GGzzb$_i;5^U{$n+J78?@-^-kPDK1s$& zRIl@Bpj=~bM**Fjvtyk=UWd2v7IaG%xmYRE=p$KIh|OByKBj3(XvO+hJ%EE0%cbE; zosyH;7>Nti;c--n9g~{C1_<>B#7*o^gD{xAkxY1()Zhr-nF0J=aF1=4-R(s|!d|2H z;|DxH=?>t+;{Xoyz`zG{8%(V|6~9!H(&dU6xm#riCHW?DP`*%Dh7wGgCL<&uuS!JT z)+XzlHc$@dWvv1;h40XDXGw%C3OAYZzIcjx2#r72N z^7I40lJ2#zz#LnT=P9X+7f2nbpvl!|ePIs^*=%5K<~U|Tw? z9vBycq0zgCIv083?fQ$Ev!Y{L^>1bM*65@-YdS}?W1UeV)n=ghK%^MZ^*{Qcuz`}z z>jbae+{EaSCIaHUESUo^VwE+o3nYsy{GRuKC#h~G+-9vhO`o&3$nV7=LLf&;$i%2O zK^?gL8&R?pi*9!g?uTSSd?&(qLai|Nu`gPC9-q#uvtaINE`A!fjfah>%$6jA$>NZ& zbdR4^bE*WTw7XPFmq+HjV+iML@w^Rrqwh$`kLUzYY89-{p_u+*-b0)pWE}w`-lSui z*o)Wu*~FH0rx34CZL)M#ybW|UNbEK?)$foH0&-)Z#1~64#tQ|bhF?&l+g6K4tiLKPh8L9{l#zK<%I2sbRU-Am@L~bKq)-EM0?0Q9^=6yXJ4p;X1LkI2xA=&IWeQep{XNlx* zzpFi!E$$e@iON5VNsg5yTELmfjr@4@5~ab(Y-g@EmAX`><;w87)oLuQ)}ysSTR2-{ zwr<%G(jX5xUkT_yRhANN3L<;Oa9vP#8b-lP**`a1jj$wt18>A5;7cj>$h(8RBp%)Y z5szi$s^wO}ks~({7tL6f<8aCw7S^3VP>&fBJ21skPsO{9l^wW2EdwRW!pDR2uKpp+ zL6!}|df}iVeoqbVtdeYbl>2Uei_D{p%56kBXycvbPbw0OTA5Ad6AjP`HQ>>Uc^sL_ zM&fuy){o}tTdS6#PB}t`bnudIq__4p3<*aazl_Ax5CeMDvjhTNWhRhu9@my?| zxQb1FW{?khK4zP+8u`;o?X71zRvq`XOs4t}Q~G+xUxK92npqIxHkO||i-^8H-PuhsOj9Imhp z^A1q0GzwP*Z*YloWv=1-p$*|{33XM(d&ZAEJj*--&REB|)Mc(SNG z-SxgY;0${M7IBE3I_|?_UlmZwLzR4VLqS0IDEaJ=RFU-;$kLSZr=JX75CUNX2b;%Q ztxU`hLPf3F$tuNcnv`EvRiMqx==aS(e+Kfstf#|;*aX(<4y%7Whk^%&3;e+USu%-? z`a?B)^jAAlh0)kA@k%r71{(in6Gw_=G9S%@co>$2i_ZHuSUksEw~-H;u=Icrw4qOm zz_c}tUc!KGg=dfc)dyt%v+FWv^D%B#Jv`hhA7Z6)hakmkuRm!w=fu|Gt4K6;t;Wy= z014j(kx{q@<}!vko(im^EC>+kQkTp))v35miVD5FlVyuIO}XL*>iNEIvx;;wKVX@4 zFw;qpk!tLapoAU;FocH7m$)E}K#BS$YzX^zfq>?|h<-Kag+;RUC}`EafIzfw-gVvCr}N^IlKY_e6S$CQf3 znp1UemBk=u6vL|!t||iu67tCK4zBqX$*}x1t*)fgay$TvnOkOkY!%gPRZn@#t&H4` zB?e9G@^SYX(_#HHR zHHo6Z>+aqP%HlL){?qQxE9rD~w=I>#%RR5v82NI}o~KjmpRI}|^uC@m^yEv2)027j zwaj&wGC7DHRa(C5J9d_4zmcuqU@f+~3fWti$ObRv0HDPrRGg*rh^!5_s~UKF!4+vn zz9rVD#?F$rit>uf%-Z;c#;o@bfrIiW*Ssnm2=tk9Xg z=g?2MdF8-xG>Fs;d}q4_4OGV-rOVT^Zh6!!0xp=B3`japM>u-!TQrWBk}RK8&t})+ zyC#c)8cn**y**)-QH7lccgbzeLx@Y;8+`Yh6+4w|t)O*D5SnBAHcdU1yy_Fp{K`+b z6M~f%i9U$-KN|W=u&vY0*_2$MTQcVW&C`#4xt3&&9rj?G?XT`lm4Jzw8WasjlvRJU?1Pq`ZH81w@~8%G+3ukPVZBlpy(rce7=n zH0x!{dZwhGpScO6ZTrdTs>u9W6BjPhMYRmH+WnHbN|}3966Dg3#zCvEpsG|wRDL?F zUTP+Ke1zf9hLt#@HDRZS6Wg4!1fncNSM&oWXa@P z2aP^mE3_@4CFG{*Ek~OlAATmVR%z z=p|>bNy}|lu+rx0bAE^X&&|+Q7X$-`tc5X@Xg ze~I&wP+Ww>yt^k{HLWy2{IT)`DP5K%^X)e2|Pj*1@3Y2<7ss5J|a}H2qU2#Cok2)k$Mb-ubI!f@K z5PWOxt6w-Z_O64lS?gUl&SK zxon$6^28;-5;4fbfu-Wmd<)3yXuqJMDqkafx>4B22&)u)l z&{qt8_YN>zdIgxgo>d<`^AXeW0?b;KSNdBoy?DDb+wK?u!5Md=K+3L%5KuU@QRBE% z&R*~wj$JqKd~52LDkz5#`gCnBfn3CcY#{3ZD^Gu(p3F7;jVx0S@vg9+wu!4Va>C7G2EHWzN$-L_PQVK@NVB z9LhygK-~lO(psVH_I`5$FdQ#l+?Wt}y_y&q87P|xbhhjJ9(D8`0T*$buOpshVU*Dv;*1jPrL)~ay{rBGqN6!f4oq9+@CVSDXWXnb30my7 z2=`|^ui<)!kDwxl$z(-MSlV+mbuW=Y$B)UzR}&sxwr}!gh*($?W36X5BCrWGlX*1) zgNGl%oe2%up|SW5@@szHs@SD_lx)$d&W99dft3b*JyrREgI}go^^-?}!U7fj5pObfL+|xUSt`?q7svE-=xWBmcpC~ zi#90}$!NV}EZN&0r<;fgrG{mPd5_*NO@rip<0T0+6+@mtpZK-N*aidw*j;MxZz`QnX`T{4ma`?I80`(sVFAvU>~qlr(aA%)1EejHEp1X@l$_##UknsQ@) z%PfBnGqg}&rQlV8eAWRq88R0SeMakyNX-JGCR?55ApS}1a0EUmB z5qmw~Jx`pq#NDEo*L+RIcz_Hga9$6(biAU(@XyE+PG{sEh;w=H)J0xjEAxwyhC5P@ zAYS^_1WHz)=pVu&i{*JZDs;X6XTIc%@M4m~bK<)U*G-eB6PU#Ou{8!?Qc2 z$cRNX^5ggyHE6))bh_$X6T=A_&6!ifYG|F#%0{KZ7wHQo!%H+GReJxmZ{OEZw28(?qQN? zMIm_D85&z5UHs(LJ>!^&iSH^C7WT=g(CKpmb1TsNG!x(?a((5-!9A% zo_B`Fd2OU5hpWi^0C$#%l`FMr7@yxo8;Iv0@c;Rf1#oKf;(p&igrR|e82)Wf2e5N8 zHgqyHH8C-EbT)GQfg0f$2X1Uq z(Xs(dLR7tN_&a$7sGX_2Otd=~HJ*82u%|?XkPg;S0!{o0SE<^BTuCHI$83qUMMiUuBTRwO1th0JcK7>|c6z{R7kN6|$G zsROYHuwh*m)WKUF*Fn9X{uZH{DnsjX$H9J}A2rufgcp z_vVP@ZXYFK2xeV>pB~ytoAz3X1)H-2{g2i;mrf&~PSRZ2#b7ED+IB~D0Yj%leu|=B zmR8zuD`BEhvGm9&33dwdqj%iIZ-&45F~Ayc-pwOn;`SZR2$@$s}7;U`|fLx z>~T$91XK>M@h;zs9&DtEjCU#>ZdyC+Q~Ih1Vu7#mW}3VV;?cu!V`kXp?l!BTp}tQ(X-u7kF>Os$*ZYh79slAwZCjlz1cv9fAb3c9_Wj&07FNtJdgh%IvV6uONON z)8~W_A1-8Yhj7=ESKm#FEe64D0W$1q9;ng+82~UXmxPVk!`=fJOPyO#TN~#Cx;W;? zn116Q>$fO>Z)kbI(AJ_6Vz75aLu;P-;qfoOOFJKV%#FaE`&(w=&}X`w;tYh6~5l^ni6>;?|b;&m!Th7FWs+4HT&O=3%!ldx6aPoo{eqq?m#T{8;G1j=2&8IoZ^Vk z_3sEgPSk_f7RLvw$FohKbLyA|-F#~3emUxvWKh7Z+iLrO9~VuljugGaUu5P*ATq)USA{wyBR`Guup?f5;UEd75_0`!K|(5}2mEBXO_Z=A zRHj%&BxEFY?UYIh#gQa486kNa%n{c>@{;wHaAb6oKF6V7bsYUX>Ntv2l<6vDvCt-d za`tLuW1xzUKd`oM$zg$yb0h+)Nx!QTWE~o`{0YOtFxnFJo&$?t&?Ey?Ftl2-P#~%B zltaXjhpi}df(ceWV87Oi53tCWRGzm?4OX#U+}LY3^w>mft)`XCMQ+MXjkL;}eTRTf z;~{)mJ{G1#zE!?iKZYXLJPzp{M&uzvb2H*(`0#Qh^lip~&6cp|JRTwCKtiuKlKs*M zkXXGsE^A0|lL%huyJ3V;JS{pxTS-LiSw(b9yk5Gs0f3~vZE=`c$L3psKX2gmRigB` zMIz1)wr*VWxS`Z-TwOQr#n{XEroXu$5+)o4vSiBR;cRnXns;FGT(a;Q7M%q%I-AOc z)#aBevm3>I&kzJC)Gm#MayfIlIv9c+Yh$CIvKQFslbEt}X6%0TY>heL(Fpjj zZ)?-($S6bZ-t`aCBCXeEsN6ZyL*h9&ug{!)9q-p9&r8tfz#2U$3b5oo93+UA#3OFl z5G1;tBl!@yIiMZ0cf@rFUL!4RqO7N^Aq1TWJlVFb>o=PIZ^7Es_FjGXo5BiE!5M;y zet~<9{4^W<7weMJS;xr=Mq|64uV@c{Z|S08gXTP@k%zAD&sd&lD{kFLVA1cAMjtxZ z{grKCVq~-BC}tF7ShIg-0<~tz7Emj9b31sV><1znka8)N3| zzaaQq*P`?Xi=^6m>}W2fNwi!^iOhYb>D_xqlngIUBPR{8RZwL_$H}`Scl`{-F?4%q zz17u&e>Exf$D9(PvvJE0rNi9WXJnvis)%3>|zh{_>hzPjx|!pR_Du&!rM>n zG%2r#K_*!>Uf>e^htMJ+Iwfgj-Ms(s+PFNmVpAvI%S_C(+jr8Ls$w?)R61o&d#KTT zn5D*LvUj$&*fL(FUW;62gpA>yFY}!*UpEgi74EG;#mj2u&mw@Dm|T2IX8IOj{N?5i zulxj{vq;Y*dcLZRGz)H;8NJre!@IHz41MC$d+bBl5Vs&F zUa}%xiV}V`4ls8*BL+WhpmJfbNeyULlT{d(0ixi35>5h#*5jGdLtxlh41AWUa56l9 zs&Ixr(Kz!Np0gOX#uz3*F$s50Dam$@%+Qx=NS#TU<3(DQ=4?(0r92q9LZy)Mg)VJ3 z8ws@<;k_?eKgNCXnq=^XB;uj-mN_szN)^2uHt)R|T(gplY=1vi)MRKs|5&bixcZpc z{?C5V|5dzb?pw%42KDcnnSV+(z`^2|iMfTNlbwV6f7yU|tLn<1aUy=}SuxRww5VDM z@zu0Q@rBNmtQLx-<|i8%k~2A@)h)7`brr5^x9c-~fDY6YjNhixxy(imVZ<6Xsp@Os z?(cv%wl_a+Kzl!f(iz>sf6K5%VUb&|7Zy6GP0cZW<=I#3{0Vk26iw>GZ$%u2$A7hHmolZBBqnRn?V(K1D3H$ zhYg8R81w4fi8dTDr4^lKyO$ki>PdBB>h_mb0jLF<$HflrKy3w=?=So9u2IHoqZ0_a zchlAVFy;cXtVhyCm;MVkpW;%f7+Sp)8ci!=@#R2Kvsq)APeTBc?@NeAu3-a@Z|)g) zHzHP$qg^g=*U>@?a^{*L&&C}!y{Kgck9=|A@MoK*{iO_Qn!2^3e`rG#4SCv;>+7Js zo(ElTVl72w($&h9K2u1;=Cz-VNf=CfoZ~X+A~Zi9!fI>9h-W?rdlT;@B0mi6>U|HK zIe+_HCcVMgJaB=q4y#&){={r}&>Kk^#DaHRFCYj=)-Y#?|0FFcAET)4^5t5n(V}Y% z2uTvrq!v-0FidZU0sA`)%=b4+Ldh5^g@D;z9P&o-9HmZ_#XX!nCe|=kkdqio(X>%& z14$uivlB}~R7o3Lj=4E};h#zblnl`^I26(!2Kb~{cyt4V{x8Joqg4IGBNBLX<_QMm z%p{8JL*HHEJ_UUoWj2K2F){?KN_-zmzf?eJ+_{C~2qoO{RH!{8Y>o{Z^iY^QZd6DO zGO0&8a%<#0@Mjb=ws^~}BCZKEy*Fx#@#rx|1~NfFP|0$jB=T>sGei%6ruxGEv9NRZ zszCxAf5c9`qJ5RwG%%tD%}SuDQvF&m+p_`w{bKa9tL)97o42bm1ktm3a%|(KD0H7<}(7MKuCn^y&f37 z0f+F~NKN`Tw{Uoy{slY%+G%iFyZcqXx11kZ6QTi*kHOF=|V1pcYQp{Q7< zZCrvq%?mg9hFyF|^N@n6VL#w$tj1-w=iqvFPOw?|9*&rIirGJFdVUEr%w@a2xso1v ze?h)^ho8+`dmS#mI}(1jvYc9>)URxP{b$7xIFad|_?|Fvzttev|1x1Z8JhiTR-jFRVuR2Wk2x1$JaZ!VCv;8!^MjEI8vzS`S zXqo5kaQeLKPu8r0jZ2444Q=+p(fz?)ZLWpVnwMy+(29_TmS$&rSDQx5r2qV6`cNo1)xg|Axby3P49$j-;H7&1}Ih7?w~^t-Y2r9Shsv6A!X#0*$fpy%zX*h)B zE+;omOw-Q17Etqn!(1y(GvCa&pO&>l%;2O8CV00!&3Am`7KdHGEP_haa`IqCRW~Us zv)w5)tzT$r3E>3U>UXaidX_f57jb53;AP8{6k&M#`6&e~VgA-GA?;9f?MpuZ{2?b`-MA%Ri!woCg_ z@)iHa`i~Tcyyd^-txVF4B{SE{6#t|ltiPMY-&weN+FZDLY_rcNTWM^spkApmhoMP1 zRJfs)KyE@h&h9D?a-rtar%s=Rq^mn;5gxgBZ*f){ z_5s`89iD=Ya*buEiBUV}48$*pVeJ!8E$rEputXHF?uqd$cgmB}F6kl`!}=3!mgc$} zR5YjA<|TuWV5v=aJi~#`{$Zy;pmSHsQD0L}E>lZQE=`Tg^?1rxLUO;|(h^oT$gA^E z3L}I8M#;I^ao{lG#%A6WUbN*h*}|?2(A10mNNju6iTu^*w#!bP$Ou zZ`QCrc0@>^ls0Gt&5E~%FEWe{gMIJZR-pV>(S$X(mEk*y=CQKfPg-UfPnD+r8>IUSv%YO zH+Sr?=GV6=0R5}mH@LP?QP9Y|b^a1T;%9h@6FD~w+U(4SvQmd3BLsB#>~UJs+_Bk~B^+ zsIjbLp;e_svc4lfM3ZbZD#}MtQh5y{n(i}zzriqbSuN9oOoh-pRDYQg7~(N%R)`lw zn2X6`5sN`H3rHjV5d_9LNWBI(sh|v;3J}?q2_bLSh;WeZ$UaTQ+H6`R+O%9Hrj{W6 z0zlEzL%@S!k)-31_qi|J8+4X(32wIUYz8$3o}-!sABOqs`-LC9itSpDPQL?5+Pf`XnXXN!pMfiQOT@ zj9d6b&5`HXcjSCy^+`ioWjcv9)gOCKxfjI+0g zzRwLL_qsCJHxPiyG3evALKOI^`$giwLiB5upJ{y{e+SHu&lhG&+wW?yH&c@npXi8V zZ_XMd77x*ynWl_CbRkdpH4zZO-Ig77#a%`;3qBhK52p|V5*dh1j|0;3_im7Nm4y}R zRaih+6|}v#+tb0p!^4BwP#cO7N*Vl9I3QCRnN-DpA?{2>q&Kb{J)s8E9Ti%nkw{Yt za2`BcFj*k|&QSGr3-1A+KDQJx?8nMpD=$Y3r1B$ARnz844$5mPHjQrF>xZV;?<2af z4l)S43}%#9b&`1IRvdn`EL#%8PQw0V`q*(a!K&Lc31$f`$7zl#7d#NqM9T~0uWEnt zV(n&hk1YWCQij->y9`^k?ic*lK5R|^sm9d!$%!yCe0gw|Cfn0iCCe>es+&B#kpvzo z++D+hiWEoMXj4i>Q48`tOHoiIXm&4fHPGH9E&?)|F$KxiC#^hc=QM=~z`}9@#CV}Wz$f#!U#e7J&;Y8Eend~7@wS$Ft0&`rlb?)4kPDvk$Vq7kn5}v zc~LtpLm21UQK2sPhe`u`B^`J%wH~7IRp}>AF7zIEd(;(n9z^o$Si0cf#rV{{Q_Ps? zIv5CRM%I11y21vvEXAQiMYphdZV26b2Zd}RJC3 z!gkVRn{JgWJjN+P$Ii3C*J_RSfmEaQ^?mb5lXDgFXau9E4z|M~r%2Q@ZGEq7domQA z^h7%iSP&)xT5j9nj1Fkx8c-*E;G@NO zQ!(?{$xu(qI*mMjupMtCR4yk0`2+e6hfY0@&lRB&JSJwI@NNcPm zRLFkH(OVMJAt03J(D$eb^#V#AKG|GWwey`H^$ygrnptZWiFW!r1Am5h8QUqZ@O2rb zFk}PJ=W=X@8Hahi z_}(Od%GYOxVeX3@H?FeW6Tvh8d26RmybzGRK2(5&?&m3l9FDFk37Y3_k=r4>GWXJ1 zud)BTv&0#M(4?rQAQ7(`QkB)*wJ$vu@{!pVE5f?jr3+V5PEL4?x?!-#CODwY zw*In0dItDIXZY;#YlKQ?NyR0?fz^STJ>S4Qj(m#3 zS*}3w`ps(IX&pgOJZJMB;Q|8$RU1Iy{rM5AMV_wpHpKxXhWVXuTilCPt82*$p8c9| zE)tT6r++U8if_AVEVU;BX5CGdFLszPm=Eowk|bT%5~h*mAJXH>{a}>8c0y!d{8X0I zN49plRnw{y*M5GxoCwnpD11I~);gFg3r4@+?e+IHs4K=LoXC_q!ApDP>+Q6gY3Oe! zmpfDDkj{-8K)6Fj*MOtQ%C7s0|Byq%2RrBqQ9RD2%=KkY-ZVstq)hRYx7_TIs5>_j zTFRfr+4o&%&m54mq$pDooaEA{RGI*RvQ*w~b)gh?q&b}75TiOVS75vgV^oN?kySaS zN)aSe|5aa&yLlBj6@eF(-o3n1Fd9+(f%@0^2S2m&xhHGN+^cdmoLzKo)+wPE?w)Eo zppzrRo?UU~X;^xq=;k#B%O6$}13HtzWt13rcqrc%7@`Q;Ip@S#%OF0U(S&sAvD}^6euLf)O1@HRY97b2y*>~wUJDIU0tULfsjz{Qb9w!1bn>HdwtcMejz;&&62(iGQDo18tL^m8RP_4d;l*v1Ou*cJJ1 z@VMH#PbM+KWAf6yis>hf42q+TT}f~j?=1X zJ5VoJ-_Cegl4!@{v#p=z@GZf@L}tA1_QjR14D!@}i~Xdh%^EQS+${XY02Eqx`8^F7 z3u$O*be?AC@^`G%u*V*WU#%^66+ag>+E%6FK{d3&uK6`^g}vw$OKGILD$YqOLrF4) zDjd#R8#~y@QjA7}tx8m@EFX+u4EIE712&v5h;Sc7xSq;h{Fi;+h`REZ!~)^7IN}6`4&YnQB=5*kp?KH6-S8hL zsPW`PU>vQ5eWP;*yqKHtfj6FQBGWgd0g3<=5hxtlRbycEXXSv-o56mz9*@-iu>Xg$ zbBeMw%C>aawr$(CZQHhO+qRM6hzuJU%&=|S9aY^udUSQ&QMb>-8RzM|t^eP9%{jk` zgeHQ|?SLv`ED}Zy^0c!AU2NdMekau44Q)@ZjGv1^^uU3kA8Ik)dpoL#PP>osf`0=V zShl=nF$C7l9YYi%jn8{aC*Bi0%m7AA-+tOR+jYB@%uXnyz-Q8EF!aNG`D~dfhPd9{ zCV(O{zVl(g6esL)V`yMlv)Tq%?giGvkf8)u-#sW3#vITom<};iaK7NinQ&-Apb)xt zSMGJPIPoSHDOM@=(*5o2!ot2%$ZYGXK7y{Bs`zNEV>?jitRQ^`m~n?4MwJLK|8YVg z=NKE@+n?%N)!{hfB_7bbzi<3?Ef`233AVm!U@jCHXC_%u&NdL>t2d6<#=@K0J>s>Q zl_8a*z_3A`G{(bbCCa;Q3=cz#$IewOZU#mfF*#POhM+?O&m6gs#DKK7poT*!6J-si zUZXYd#Tp~n+udMfSIQZabvIR=kz(d2ae#w|mymKfI7pfF_xGUYMZ9a+P)X8WVHzg; zFVL;8s&?qy+R+cazv+G^Ws@yuO^rt(QVtrVfBLh0b85R;hpF%ts=PcYhx|~PK7ONC znhuYpQvOo9-;#)afF|Gg-Hm^T{8AWIAw|Vj)#yiMc~6Cu9Z8zl-9lE5OFBt%El%G^ zsgIg8mBf}4E3xbZAD7jH@bK<1pc+5hmaJE7@kqt6pg9Gtq{djafey?FV(-UwUyOIkEZv)vV7eyPG*Q%qt zjgK>7ZiisMVdeIe=#|~3!9O&7@Kvp&!}vru?)2a*<_SE0ReG?z`uwNn)JnTt*6qrF z?SrgYvcp!G;%#rzb2|0j8%ekE?VVKmqMVC-b;8x|j9<_V-z5rN$6dg#{5JwR%2pq9 zXOa01JRMZ-T?mbnOM2X@2YwE3)5I(;lqoJw=$Fp^YJ*@3>9H+Nk<@HV7PaM>%M)bN z&d?B8{IyY;l4-tY`vG&g{(>Vg{In=h{{W=!h7$S!n%+IBAO$fsTUI@@T!{fU4Ry@K zZP0jnZ0a0y4KS5iyZ#R%@U-?RXaZ3$`~uLG;`Ll?){BbWLy@=S;_cwRWpe(q_>neU z8BRO1E46@g-&f5_j^$S@k|=jBYbn(7dtL1hZVXgwx;u0F99E6o_2J{ZG2k9svMhF@ zRnRtGr+dp@}zzx$j}4l_oqbE6VMw=wGkJ{ zoH{F zacp80pk)D)R?U`St*5c*u3%*-fU1z5V>cR}2Iz0k@Y-20+`kh}t7B%cIEhXNf~~DW z1jSPCBP{~=K<$iF={Sgm3a@s=Sd+A(G^VsLFgB#Kz4>l@pV;UX2*1I8GA;q})E>#L zO5D&|7}ch76nI#Nk97ig5qE|Pld%^?2m#1r2p7`^cteZ9u~a6K+XJL+o3&P!RPa!i zJ;W$r8kaPB7@cFH!g(1pwetody2w!?Uv$U7;^xAMtt7RgKej6L&swGIg&2hzr*do< znjG~)1s?Q!`$TOn!h5AfkxDgt68E|+AtgJrgQ$!`Q2k2vT_o%5LPa#rHk}Hhb>YG; zYGp?mBT}wUkwA$GcK!(&WO(C7Y!Wo%>Sk!KyxRuSTND8-`6hFI$beD?`MrD_B$nFaOlWryaT{q zVvb;h*^MxUO)H@neHuSn8jZw={L2}CLjs8LYBacVnrH-!GRTwR6*(q0y%7ID1vAS4 z5;6$fMvE+i5(6}mRRrRFOc4HC(7<_eei&T=|3FQW8w3L6Z$cO?4H;vcH=CpWq5RkQ zA+kV>OJwsAp;3^)ynG<{s2jeo7-lr_As?KdjdwjBS(=IWwRsvC-V3C(WUwXD4382- zi$MNLe}OqlGe^mdCE;wy7P_Sa0Sn{@!kmPa3qKqFlh>zTZ+ZkR97bGa|5u=a5BL{# zI;;M;)|&f^H0!`>Dpr?gX$@A?N1UJPdb5*otx#Me_sQcORyx2`2%`_Jk|8*U<&)W8 zU>}Ic9^V4t69$`5ZD{OVwU~ZH*SsT-^w|n-@q@AW*U#XOFmH=zaYCs+z9qH_(93*4 zIEPcgL3Wnkzy6o?pvGu|~A?!ENc@{A=mvybdsrbL7Ro?5W{fb46})n{vFi z$mGl7f$7p8y=Li?a@)e9*OiEI6P@U;FQt=(M0EqU=*_UMPIFQ&r$-X`(a-N7*E4S} z9e*`8;uem}ddlA7dNQqF+;*O#@0|)~95?7&-Vo1=yB*egap#cVql-P2y7h77&!Qpp z=@PH3b_(J~?l*W#we;@3Den9Mn-9OD`YoAr_CMlz1!w+AruupO{CwVd8U+6N<9~gj z{M#pzGrg(`Bmgki?d8A&j6O1kZ-5pXt!Y-1pB4ji z$Zg8~D_%EYfd}&`P>C#4tRKoF*-`R80uNjkR;WOl zKY@qxl;dMWLd`BE6pIY3+7M#Tk)v%EH80WVYmYMBiI?Ou?jz9w7n{2^5=J4&Ux)k^ zkW+XeLDUbcZ$->b9HgCVNtFErt<8J!WDy}g$!aD-?}Vs%KY<4h2g8`27!Q-&_Gbc} zO|f{pok08nE)CxY^DUT8yNGLDaJDUd6sH<)T|hxGhid4J9y^Jp{NZw8SP zRe&!e-e+E&$Ml8Q!72T%epo9yb;PO9P<%q0W+pOfhHYrpjREMg=a;js7iZZFd|$Vn zrTDZ6m^O%ufi+kn|2GDAU=4ndbJdeVWabdPx5;$?T2E+u%-C6`y1-{deFTc%?#Ir? zv%Y;4K<&4)$j@e1ni??1T<)Ktz|8KuTbA}f8ihUvLmwTtAc1(-yiHLzbbjTDy;@c? z<<+JrJJrE~po+zxC*gU(c01u&!RrPs4lKfBpl3BOFjjQD(Z!g2@*q(s&*8J&oGkYn zoJE#Zirut)XD%9b3ozPs9_HjX&1dC^dF?@2PQg-rzA)cZl^k-}xZ?F4rBd?JtXW2! zwfWMCrZ(EWyvM5&NyllFSd)m#EcLLjFE~oBNfqo$JnNXKxpFy%;ehs7c`(oVUF_{w z0DhgDb+3gH{~fBYr?{URd+RxOFZq?-^6Z+m*XH_ zShgOntiOMstzL#)88r3l)v#srJ(c0I6_R%lI?9x8_!N)AN|QKQ%;3U;({eE!A~Bz!>F>X3Z`oY#Oy?5w|0t%*Wo3v)Nf6@qf9?Z?~dmwk`u&@etbwM2f?LmXoV|nr2i9m)SEFHH%mTm7TTAzXhp*&trKJU@K6SR1-Tf<*?$JlyB7=QdWjmd39}H%v%J z2$7q=NG~ofELwgsu&`0XRCMuXZs_~hV=?UXZ5><L&kOZ&2Uq3%5NG9UJbw-q9>gdJOfvvM3Jz!L$npO&W6_jZ7 zF6@3#rWJ}Nq69HT1O?Z>3Kiud=JzeskpK<1DgWe%UEqBcfa(jheFPMsihgLK2b~hW zOPvT>0VfL8bl{3odCxMB*%P6_It>o^cF=u;88s_Xc!3 z7!FcQ$Pu~$kTOje8g0)B1&rZ{GL(mMWKfTe`*At=ppyL{rlpNs8z^EH;R6Ohx{^l% z_5uY;E+ReZm?fo1Ad81~?2YO;v9R+f%`I62pgN{5X2LI4;vkGf$uo_N41W*6>}7I( zqXZt2+u)f(Dogz`Pvah&XE<8e7X`ClTjCDvtEKid8)O090XsB5D{BZE7>5}uH&hn7f-}O4z8*9ya1D^Kn z!VWK89y=)1oSseJ;#J^j5BhVD?uypQPup|a3$~yX!>+f44)! z&f;i#6V`xTn%NZch|4-Gn|td4vF%CReI%$On;wm~JsULP6tIu2*2aEcF5j$Gyqes; zMTeWf5&k5!%j1S2nb{pJ8`5lq*~xDhHef0K$;OPVIUgZZ93PjVk$*Y|2+dH&yp_A{ zWGinyKO4o)62x*=@xx})8hs-X~G$Y4yO$w!N ziYZ%63CtL1@tWip{UQw~yGY$khCLH);(1-*uYsy8Ft){Ixaf^#aE*_z(L)^a%*(GK zh~(cG!cq(hrM4_~VRWK&le5@`l7(}HC!gW&XK7oN`P&QJSf~ct$q$!k+ZlUq)~QW4 z!Y;>`ts=cteB(=|%BA7TRs(9WjzPWY zhq8)(7rIW`cl)8Cm};hcG$7Z=`M}7rrv{1RU##BC$DC%Ai-jwubu-l5WB|=hC4;1o zZc2+ZIIVM$ioTiQ3W~pwX@+~?4DVh3Q{UtM7U9FN_;E6FJ>-<^wuDg@)A3wfAR<7Z zJgf$ni;ra=gR?^e;t)9OQai1(;!Dvl^oeiE>xvk)9(6L74+>wEv96Em0``=RV!jWm zV%tTNO+J5>eGi`|yeBGJ>awych2-UjJojIxWiM^JG_3C=>6xIGJmRvamd9N9J-6x= zeQALmZdh@BM7xx8PVgJ7kH7E?vy59Er{@nsfhcc+3Mo)$4jBu0UeReJ-ruj4LD~;l zpG*B!d^V%Ivcn|ol?7nw=VP%HakJUpf-W{1J^f&AN_ltG%OQIAK@=O<{XR;0Bcx2N zau-fz-XZ0H&QdI%dQY*;n1*-0TiFd&QOk3>m=tl(tH`XQdi}90(Jc?}+KBr6X1^Ew zW+C1`!$2RQI=0;zx~Ie)Wh{p`MzkF5-V&?+euB!`Q?8w#rA^f0w5=;+D(IlKTS^$eDezmMEVY{lv&mlI zbC5q=ZXXZbG#YDn|BA(mowj0Qs}7^>iPHyUAMTtTw@9V9&8m4YCe>O8X+HBM-7Eh- zjRiDq{Gcj{5$>-qQ>3~5g#{Vwt{r$hOl{lbMwFBJ(vtFxIjCWq#Q8KYjdclIDQSFT z{|~+9|Knn3>-7I|v0E=WK05~j06_Rj==|G@-Os=MH!pU&DYE}ie!{N5P=Sw=0IWJM z8`;;&mTevl74E#8wPulUn-W9=OOOt>pGgmG*g#>{g{OI#@24f)n_Ks@nP<`ko-ctv zUk~HyRpQw)?y(ye_v{?Sy)WxVyp6$`^61u17qEG`SwS{UUEH*id6m5tuQbh3XexJO z#V&b(E+{FOGj=mkBxs#Pw;ce-)}1|6n{KPFu>~9p)nR zm)bH8g{A&lE+%94Ej;*%)|2U6pp04K=U5Vx1}m@2;#Q+in*a4l*hfDu(u?g`L_7dXP4F~T`j`qM)qIzEJI*; zXD0eCRyKGq0gP=;UzSAsqXqWrUq!P+lG_gpxDvD4pYxf~t}g+$3>YI8x>9f7arnnX zZaHUG>o}PQBvxyaXW1(^u(X=6-kEcXEL~@_2s|Xz+2mENLlfk0;wjlNFVEcvuCPj^ zk|^&w4mnXyx%g~FA5zpJX{y2<6e7Sj%B3mn89S~+s5*7-469NHiVSl+rp>6bkW99K zX3gfm5YL6DNiN}?^cQhNZ35^-J5tAnp!L{(P{%)LC<6eMYmL0}kc$91l3ETb!|I$F zjpoMbObVv8h9KYAMTx2%`sC~}W6~WQcOKlJ=^IVoh)BKeRhu%%Au)v2U0Z_Te8Pe|zzpGkZ0@ z!2jP0YI(~`fZk7R0{904{%=vEKQm-xZ|~yl;$-NcZ)p7Ah}Zh6*v7B@^a=gcVO(0N zCOGMM<5NEh2Po8pfshuiFIGkg?FGma#u+)Ue?I$;G<41WicN^%z5hKGldGL|c2%jq zo?B|jS>(Uv?;&qlUR#wvmJefBKa6j(0BeV+J?>W@qu5DmYC-PZ;aOy>->jyFx7eLh zVol{P+M9}}DJ%Lel6wIQ;AITGSmh=Snjux;P`4R4cCyBY)-B>C8p_zI0*X;Ne`rXr zt*xO}qe=`5dnpIxpPa|QPE4*cUPQPUBtP#QftR{@8eFP}_0xW!$wQL*r!WYOp&*5pSaD{%lbkp^ zOcUj!D@&iZ6G!c=y#-BV)Q$6TGNG$J-oSLCz_X7&ZG-aANu(|Vtz!*HiNt&X|i21AY^VVg5sUALD-%gU?q=kxnOnvrat)o zkzBA39$y}e6j(cI+2+=E1JUCyYPP>u3~_vL5b~tf6V5t5*~h{&%RPbw|Sc z+tq>B!P3I^h4okPdRRl$%Q?)qT zL43sV=T)Q=?|v4_7Wd2l-{YGt9Uk$31OVV9{a@g_f5z9?-pa^`kdA2Ix6ts50Br1>Q*>60cbD3Fphlu7V{YpV8n-+n(Gupkl&Nx4rv zvPMYJx(3?~T^digU6IMUST*dJ=U%)|yB~Wwf08($)wtQ(`8>@r`Gb4uR?zXt$gcr* z63f>GA_(F%2v)+%eL9TK|Hd+1J+D$7<_uFG*>1S}d8dia#JF|phsOydLaG`t$*0+0qo09zMO6a*Op4zzFveD7;{T4j3* zRJ@@t|3mu91T$1N^C6G}NJY>v#f*W#v2zPyKH#@CN{P$^gD4GfUe2p-O@dL9LSW4- zK(Z?OSD)>6X#OuTa^m6@!n^PcE$O zr`4NY{PMw77zWugn}-_@eyoA@xtF7_g-eSM=B}{J{qda5cswy@0q%SR$xs`X9Kz1< z-G?J$9=JO3=dp&l59$2P*dybMZ_P2`;lh^+w>14Xw}Y>*{&~6I zE>8=h<>DH4uM>X;Wu#0m>|O~?l!~Gu1a4L7rrW81F;n*#9*{dCn2PTmK>c1IR~h|s z;kbeQ*jxfFeR7WHzFPWZj@v*7L}{}!1gkd`lzk#b$ty3xGQgw`0DfA?<~VTBp}l{^ zf+y@yd<4k?JmK0aKe_vSPr{Y+RlFWFpxlC{hfTVm`6HNB`FikiRfdcFNT3_7z7w{P zkWb2)!+iSge7yG=WxiEL%zku_s!g*)GlLdQmt#xKF{C#%@W?|qv=@1e7U<6!M@+X3 zeO1b{5O%1cOb6J1MA@5F&WYDpum&ZcG~$2pu#LWGfSec-(8SGJ9w1qPKJ;DfJh>s8 zMrx#pd0=$trI~{^RO*Flv1#MH=(!EZO!eF_0IVFSi*<6yogjGFit>D8Uv9`7&f3OV z-;lHw38N#TqlOi z?%FoOAeIe^h4z903&hMBcCh4H*>O@W0rmJ9X&0)i<6Dc@^XNiMQ*cfuAwvV!n|Ezk zLJ@(TBB!#XS`l@wVI$N|2z~ z-uqfdA^~v7i4QOxhBE=5ocdUY-C#kLPaozVvhUmLV|2B7DW-P|edGv1ooO%9$4+@N zIUxcmp9?prnJN!5E?+fm`JJI! zX#zSHHiFKwXuItyiwo5c-Rw_jAC+uF?2Q-dn(8`Zi?@ywHz_iWzc5M7hpNYpOxf-% zaa}$7H(r6K7RkFO8auww>x77CNC8yd!@qm!*er0g1f-;@w2$U8IG)cqP)q!RP^%w4 zix2=qYSw}9gAO{K&M&s>Ol$`_z;&EoYL8eU?@`~SdVSWbfkWRZho$SvjRb}}++m)F zb;jo8T%@4Y&2050KcFk+4*_U9#5+o9=tZVFsmSGTbk5ll5UizLx_R?t(?n#RR{%pP zX7CBBDGiz?e#uh;s|sk=R0bI-PGSx5fzk?Of}~bOX)>k?4eIPz-sf^r4{DUEl>Da8 z?pqh=3n|QcA99@9Q617hqh=nlKCirNvjY(ut~7*=9Z%I=$#DK1`nS|!OwH=1Vzfnl z+mb(gaifG>moZb$mpXjYVV3TGEDzuEX1%{%dUnyHbqPPn4Z;8=AJu_|- zUBN(WJIzwCIL!uF1pbq{4DQMJ^$fUCwtD8g9Ho{5 zFHhRR8l}+T7AXl4ojxya^9Y`%1O8xhv8|hf`|w-n zM}_f0ev-xx2j(QmU*TP{LJ+`}Vv|e#%oil+I)BN~TiFl4r#i;$7(}m%XW+(_nb{$y zx<=M~cCSPtL(H@nU>1f{DULsOvcIr4Fx~@Q^ZkCGFg}>~V@)F14~R=!8g#w5t)#Eo z(Z#u&zhCjL@nGTfx0HZQe)*z}oQvjxRq+gT!)s6u74kl!KCi+Gsgi3496a~i>GIrC z%SRnKEUxNt7@db1PCf=eY?^QJf$LguQq|FEsZmn1}otTC9S{ZIx1}}UKqlKMyu=hu8OBS<^*25 z8M-#7x{YN$ULohwUl(mU1hRX+6SVasasw==b}VjbY2iU;HjYg-c+^jW&a(a7Il)iJ zs-vD|U|3b8vTIi+8s!%hJt@DCwU4%t%d6i>TzG{D*JiDc3`q@R;O+SW>@H^s-wxa! zlbcgT23u=FXcQ#m{QumzHp-)Vv>~=&R;JeBB1oo(GZLzpka}RNMyF~=YVB%SA~nO* zu0dd5?IWG!HcgKa3__3aNkrHBnq zcfK5Vilul0{b7R03?2^*R0lX~J?!7VfSm2CduLgNczbIUt#DXftCm3nRK`YtXdt47 zQ-q~@9rx|R03!*RHKFJt z15l~~kdIlM*KwW)6+LAHDMr~V3Zh0=M`z^zTQwiq|zvn1C}V#{b6~~RopQE9M;9)ID_MA5dmUKq>^5GrxUW#q|eKa9bC+Mko;zgydptJ z{f(vU`dd`Pa@Y;Op1%~3P|+dw3@fY+BtWrPFR3T8P1}Z%`XfOx_sPOR@oG`aMOi?_ zVL@nNxy1yNKeqjz+d=hq)SZA4{!}lvOH^`xM@`rMSenUE3T5Xzo>OC zNI8K!yuh6zt`fR|g3tyqxK>KG+clW2GKZI5W}GhDaZ?;e-e|pw6}GYVs1)#^_NWl! z(vhKXr6S2>Eszuz{==(NGV6Qd3Ib*er3YPVi>|cxJNZFVjTXcRvUknxFof z&vX}Ay~b#&OU^?YEZ&$?cXor7so$zL<^M8NF4O$`pq-QHcLBL6gH;*jd52bi`U}3b z<}I#EMfZGaM?^Zxp{>NrruQOoxOP9mc9TF$Vf1ddtm^F8oxNKLH$nCNBGGwLdIk&Z zB8in$yfOwDY2R{*_j5k z(LY+-s%CfPk`gJ+lJ;g-xt@f@ILKnAn&G=CnD) z$wB7%^RzFza56CiaZ+^9S8HNewq}G!Dn-`?*ZE#)=zD;&r|q{UoR?phU{=ds*NrV> zpJ;6fUTy7Oj1Tj-vcu!GG_ISrh9#_fOD9Wsuw0#LGsDc-9(s0pI;Kaf67@MpoP8Z( z`TQQ|BCp;fv9t?#RE>Dt)}W-OErXuF#wb|E_m8Ch{a@V9*$X+`XxN$|?qYe=ZJfn` z^FtFwT#0E*q$2w^JKWro&m#;gYQ1_S{MdLs-+lL;!I)o>r$ zh5-rn)t7Zq*w6uIvE!d--J~c@$bJEo>>R;k>JV51>E7O5eJ!jhLTu^tL^{T|rC|$B zFtv0#VxWF2FQzvZtbT@mS0c3JvKhj)`g7-FDegJ%3QMYDwRh^a&V^tHL<1 zV{4!w|H}NbTA@%Hx9jsWLHF40Jy+dTl$Cm?QAU>PR(yW%)QC59N{mp6sjcsymYE+d z#*ol0L96?G5zEs&Pe}6hV!f*s6|MEtpa*V!sO^$}=ulk@C|{gDP0OQmIXR28dND2S zIJffcH#LEnTI{m&x~u-m3(kiF)INA#(JpBm%AV+(RNx^cswoGKs(mMkR0U*{HZ4vu z-V`e(Hv6lmD(43}O`!sO0Kdf_YrIM_Rnf3jo3ZQ+5EHCl-)`rJuiORRQl^e-d&>EU zGo54qSH9UT64)YqD_sAz&gIvd%-}Km8WCK9?@${2?=WYuwZGqinM|TJ`_?%Z;i|ow z3n^qbpP*9Oioeu!HtZp)P;Aq3yYBd^mq+D-%``hjiu!lISw)(tF0mRGkzlT$o;3%6 zFB?3>#ACo5G9Iy_Fz*WDb`5wngcZ)wezS<0$h|LHKT4|^vbHdZKT!*)pv9ghec(q` zC@jai$Zx01F^+-c0eaZ$RstLtLdp-~ehjGh<9oElvIk1*J@Qo6b1BxD^%BuE4;7DN z_b$}4Tcb-6Y?XnMZp$ulcn>K|=Z+Az$;j@yPR0lcTQSpM2~AJ^su_XRJ?@vyZ?}ck z#+vCewfjo{xI4}aEarnXM@Mpi!ywf|t^_@l1x zye5X=f2@4L0Ut)`n9OH(CI=c(074>`blI61BQ620Lw)+UD_QsLw96eLp}|2ic{=4b zqhoz@`)gZUuRj4DyI;*3bh90t|F`EmSWpmk>qY?U$#1J-e}TYNumM*}wH)%B{iH~K z#&5fR)NjvJQJU^3&nM;T8|$}cv@1qhuKEbWIHG@$lYx^P$aaRo)QBcf$$T}SM0TVj zmJ>A$@lnDlwgm!(k^uB8A`WTv1w|wnE9ZY($qFRVFDjEHto#t(4j7{$5RvfGF^{N} zwDyxqDGIBXB<|1!NuB4IqmW{h3^;cQFC=b;{$`~^fk=^R)F{3Up_H2nxgku%5t?FF zH(LN816LYKOZr)8zbD*=6v=E`WG<%e<3qXOe^sA=17OV~w`H{xloy*?!21MF|&CC+UbE3;|WY7z#5?pl4JNncH(|dIG|*w9>h;obO;ExmzXK&(Mn)g z_Ue~|kht}ZjB8)cO%J>=xeFxb2#8Jp#+|Ua2Xl*0#`cRdeZzPJqTo+c^pj*VZ`0;Y z6w3ncN}CHofq{{1xmcmAu-*$p!#I*PLrYu3!H9zuXA@scP3EJX_I|MVM~|U%*&5P7 zNA}85(k$&8c39l8)nND!;?3nES>LZ6y~i1-3qI`{gbT!YJKGWgS0w2c21c^;2~8Zl zJrmq$?l`-TZ-x0+EoI@UM6yTjXVpSSd;3X9r(Ol;gbKP!UA9gT@4f3XYKlJ=WEn^4 z?iCH_Vly7Ex(K~8sUqN{OJjgN9gxZo#!b!=E+7|j2Ex4!quU*~U?#&bOQOAOL}^oR zbc3`#Bpe)z;cT~18n*4PZ}zc8=hxaHr`o_<@u@mE6Co}fI0uz!;RVVgL>0k3#R6@0 zN`mVLmdj5}g@8eR4FIpWc?cQ6#xldj8I!qf8t+8=J+-wk%v1DFT+w=UgZDeao9R~| z{fNj@wU5iQqAm1s`DE0lBpv^=8--dDM~B@d#x}Gb+MP~ASW5EPBp(Z^MW)pFl+22M z`RFLWRMs8qRPLeCqs-H!^)N_kishQRXH@4<)!J z&w#HH+1j{;O~isguA{LIf9|2!6f|=1b&vt+@fWgGqQrEd8%~bxW#%@ zmJbo-G2GRCQ=%3A4u@pVEDr!hkjC~i~halFevj|zB-n~A^c`XKbe_U zd-1mBq@#iw)lNcFOZwOizuD;nS#s%kDq_;ps3>1MxA|f@R8c$Ew>IUm8N1G ziBLQKm9}-@v9l`gnb;6-$&Qvz1x|#@zoPyLvgz(QjY=)eELTrRx9q2sM`n9SsZ8~f ztbNHYY;#>_Po39e66=_%3{l|aMOV@{028Z)YT&LD0{~)|KlLmtTSah3XLEw-ReOq^ zdux8$)LqXNDo^YJg#E481`Re8CWDC;$L_Xc6P;!#pkH-I(Bj8~nObSBz_W^U>7Ty! z@PknVYKz%mk)_d8tV2KKxD?S*?hC!sEXu{V+(SASWvaSAUl^N4+`-~Ki~iE;8P%y+ zn}PR<(C#JOEN>>i-hTxBLrH#3AV{xc^;FCwL+%`~P_yQ>_7Z(+KaL~~kd%@8i$!@L zB+A;P7tyI8%*_|dC3d{-C>z9j<{rhO$tXfh;rd$OQwf|z6huau|en;3P% zeTJx=o1p%%pQZqCBQ5d>lpA1*E&wwd#rvVr0Ho0UFgioNuvAGNPy!;qQNYACQW#-< z8y_r==s(TPk`-XShdLq?G69ZVe-zPz|EY+|{2xV>`9-xPQ2eKwCG8)ID3~E4A~H9I zh*&>jI4Ir|Al+}h^8U~nnj7JQ#(9WdH-C&+G0~qjZ_#cY`rYbD53aR48acmMNPP;b zEnEBZrWx}HZv~Il8!%fO;u3MOwI4N~)(5`Refq^t5`M#4)vw`})c8cZ1IT#(Fi(tv z*?k1plL^e7{=wNkNy8mO3y!;hQ;ydp%mWU>F!|uB6P$aA=ToZbD|#P!xOR#5Uh}wp zXV(hmwyQuZw`4;Ee=>Eod1<|8rk`#1?I&jfT=(UR*qRzzm68V{7zTl@EuhBIbv41@h@-@z?hx|z>HV2_XBSh4|8N>y)YP_LA3^e4u3zC|pg>JI zV6g{#5LkElqkqIauFo+rZl`Uh~7+?`+Mhr+~f%AI%f zhuT{%hhaZ&x)t)6W{t`Hm}ExBTyrSrCUuxcvLC^gX%znRCX-XZ>Bqa*gc~q#k=@M! z3EnZ9!xF2r4M=oP^xSZ4cw|E2cn;2E`e`%T@87DzV-8F>wlCNBf#8vID1I?K<2FZV z3@P6<LvZullOzN%08ZJDr?ev2d1iy}>45TeIMDvm0Quok&O9R@ z9uO4phj36(3Y-o;68R~nIHZ&Z0%AD)Yv4#pTu?a4$jKIr;FON=(-i&k9Ian-T^LD- z0bNHA@aHct%(g^h8~+YnmYREEEizs5awZsvDK+n+-u3FvIiruRxyqvb4p zd3?K?*Xs+YocrM@+1kt&pbINP5h{jB9i)w^=KAXfmbw~B&M7V5`PJH%Ea1#dpcRAX zu#uJAA3hv!3(*aPyk@3pa$ejuXyu>_djQy<9C0JPUJrz#LEcm|<=>SQd8 zbYx{Ap_%K-!&nzqV;PzK$Zc&}0lI0&YA7v%az?q8ZGblW^3PLSGLrX}qhBXRl)`nN zsto*^)5@k{*0qwL?JkF71jZtFG$Ak?re!0A;p#>KNt*pMvP${b%NRkiJh3iR?-c8g z!79{Rr#v>|MZnKS`YD)uJuW)vXPFZx^WeuK+U`|rIV@WwM)k;BzQu#*vp!h6>BLy4N zmT?rxfTvh!Rw(}krc51p%AdE2ck4j=db8YCs6>u&;fBuT{U%gZ3E=07j9~m+jJSCu z{4-m!^Ey9kWwLDj@L|)9Ms^|uoLli2<(ce>;?%8B_Tnfqs<%J0bQ>%@K)yE7Y`(Oi zk$D~;UHy3Ja$9jKbdzO75y16f!T_gfVTyo5zQNtj#1fI1%=i`Cb+Se2EP|ApM{*2R zi;WkTJG5F2e8=Q_m|KVD7zr?|2GH31p&)C-p{?Tz8#1q3E3jxhFAfHmF!Afdt|l!4 zgzwIY+7CJvu}U|W=rX)LF$KZdY+;l3z)4z0tasjj4ECeO>2wMu2!eN|Cwt>yxy6k3jbSSPSNi_&$@wcGNipVE});mx&E zu)b16clGK?eW3#>@s3Zby(t{ND{CotUsO@XlJ3;6z_rlRfNehWf*{gHHzVK)8%5ww z-|d&N*s`O$4=vfYPRfcSlKZj$VQy{H7Lw&jEPUqMNS`)M9A_dhvTaO%a=iP1tnpMl z%SIfudu6a;BO_4q(yF_=tbH5%o`28jgobOgv?mF@#9JBK#*R9p9Vo>#gdnXJ!QGI> zFa^Bms2a5pW}_&^?_cQ@uSDRGsg3Bx+jK(-cssr+A9wtVdls^8R-M=p=lwdI#zG5U+bXXlj&(FL{e%Z33_UBfpbVoqEeb9 ze5-`F*cQJ|w2_JUiQHZ{2mGA^2&apwM#mw3${ai8B;pPoDA3bH=(^!?U(|WqpDiFb zport&kS%66`E}w(i_rY`nAAym6b3(mTf=VD+VPMllYyNml-E($XA>e#ncY= zOA^~%jWACmV>~`3L>-F;Ryj{T^#2fV=YpRIz3<*0;u6an3P|F3`k3y8+Wjh&BD889 zZSyU$bu@HISQPZW%MFLDlI<@<3 zf@N%bl;5Dp!j-4Vn>rr|PRJ7#tQu{t00}1V2LfT5%Ds{1>R8xTjen^J+nTrtmPYff zNtnGAYOAHC;JX#IQf@AX1t}>jFBvEpbGE*K2vD{KC23c_xs*Q#5KB4{bYL};iXWoY z?Ixm63=W{?KG1SOtZ64$d4X1GNAGD}wVV~kVOKRXxmp=ZSk*G}!uLFbXx1a+k)MG& zY&bOEySU^%oxWw}kiV`A_#DjuzwzOHS+;Slrk@+tmDf5&l&9kCo=U^sJV-9Qc{0te zaGo!}H(37SrgwP+&ORY^AQ2t*~!r7^Q4L`(_8i$^<4>4X;ib1F5WsxrbVdo zq0X#LUmhy(c<+eBH-91&K}Vk9f9#?bkrz;#;g5<|kme7;V1+wPFoweH!LK})yye0y zo-5y1VSq#K*Zd!py;FB+-L|zG+qPzG+qUh@*tTukwmoCp$&78=K3VHrtzF(-_WJ(7 zxEihZ)>^Oise0kZax9jbThTpVZ->PR0+N*qRMmAA`DKEmQ=QMbwRSAenMX$^^ZEHU z9c)+_?=uhE@)=8a7>gl7Sl}Vde-wOuxT>C#&aD;1;DE~L0tzS1xMxrifvI4_ytZF9 ze+I}g)SVn%pTt_ z^x*inx;H1IlJHigfF=XZh1FK8X1GIrvJx;Z(`qd#|>`z9xy147xSKpOFd$JzSz6K8Sr-svII zE@;*JLFKOggf5%?*&7>1?~fHcf^Ip2)Z?vNAB0r8!n8`Zn=U|gT$Xj1fK3sdgv3{$ zMD_=qV|4zQg||%yCPv3m*mi;?K&+nkB-=f?Vh{@(eZ(bTq;qJ5g;ydYiJ;=!33!}m z8F?|(isiVH62x-_ItuOGr?&n5?9(38kVG0!+AT#6#@QxtBfD7ciZ%GM~cMP{8 z{X-0>>|>2Tu08RkB||rz=tZ{-bnkY9Bu7v;p}+!R`NLflpEgrTZV(`p$A8<`ljS&8 zcp;SZ8_61qfP_}!(+Ai3j47Z_RRbx2KDDqDqCb)OdulUjOa55}1wl%{Np8=jaQ5EZ zW-nTK4~Hkq*lFZ&Fi9gw0V)TT?&nNWZn8nL1W#0Vpy9g$guHX1WRePlkMGCwBL5YG zB+uhQ?+WFSkGZHB7{9N0@jf}xjj>J48v2k>+}^rHN9&0 z7XN9#+94Qy^wKX$k{G&s0E=Hz70lOw%d&lL4zt-Sc{z>yB@!kb9UvNG5N zc{Sf8dVR~zov9tyOL`v*#+5r+%5QQEfY(r&&ypKeLl=V9(cV&H(gI03?3W{3a*Rfv zo_gfO3t-%7cx3Aa5clchW={wSSIYn$nVr64HcLy%_`(8 z<&vo_E(_pvzm;SQ$T!$C=f#+bbX>UVWV1WhJ6Pq;@MiKm6AW7uIn&6_W{MUz0m0}e z92zn(UP);;F6cahAKjz7Whx=Ro}&{0D;#T}auZxkY-D6a>_{zj#NqM}AQ~O@E=gl! z@lSdcj~^k_6N0f(?o~#Io_;=kN?OO)xGSby0y!em#CF_jUGc9MI=*8$J+?l6cD-0t z$dmy_pfDTt0bSsnBv|082WoLpxqI3aFN6DslM``=!Rc9BlovZH(z(l3di5Cz?$^J* zj)OO1RkPpMZllO6jL`-baeFC`e|=-l9atuIqc`b>$MNy>>W6@6>2#<)>;S2NV@44V z0>1#kDVS;ADsSmmILa`DqJbBOx6Wi$`XOVHqZRU#Zcx4gvssI7Lb?!2FGNtAB0!&C zxyea9JiS67h`fX|XAd{`Qg0YM&;#em?QPsKJ%X@S2-at{i#wK9J~Kj5y*aGX+w}Cw z&EEPEBWFj03Wms!`c7dj;#f+2EN&ZvXMk^vznik-U7vDCN4Y$?eigxGDh3;c>T;!k z?13uA&5x(XKi{}u1%HHB7D1PKPL<`TGo6iGfvQ*%OEZss6C^Jz9IbpCjWVNkhsrVs zdo(89^+$7e7rFzVCW5*=s*CEOYcw~TiH@fb1x6Z!A#jE7Fdk5L^^*(XXOOJ@!!-Pwje1qey7@&SUA9f-bB4vW}827BPWMSyo<53N9 zW}Fj#joQBy9e^Y-#g;_w7QzvPYPy6&;ph*6p21MBcj!pqg z)VNwul=9rl!m4sd$-O%-n7zA7l!GO5 zhO|Tuzc~S41!+)kmboSimmw%Ad&?BQMFQoI3y4C~sq9uXx*ksen- z)}4uU&B2%daei%aFXl~?&6e3%Jrm80?zd?bN!U+O`Y1V^EAKX~LSiiPI#MUWEm2OL zpbb>Mzvw3?&XervyGzaGktLl^bo+$L+nKs$v}+<;vQ0wx^@r%P)ic_uIUZ-ib+Y(t zTa}8NWX^gmuVk?<5p3YSsOK~A^jo@tdC0JoY7En+qPv{JI%BH*VoF=XTSaAc=_;|g zt)rilzFtQ~84iC4Uan5*LA9y7IF81gCHz|K0&$fG*}JsPr}-ICu{*qw9AV^(RP^_^ zF`6T+31|by_`4#rs8G`e8|2)S!b#NKm|5 zdTdSlhMDts2o7aYtF`+P@cBfUH0(W)?KNc;t^e_b$Cbab<7vtk-|BX?(B^ANB<1mU zZx(P+0kwUPXK99OLz8m5wzSCa?^qks+%06>NjYbx>xR!yD{og-dveN-2585JeZVDZ zc}b)h-IBASDyCNT(U!eG(B8b>K3j}eB_H9BZp8s?7`5mZ-cO#EnPC?Z=p9)yvV3aG zy1fIs!TQUr(#bDRpJFUu3+FR4PGTy)1i@tOodQ~KL)TlSIapz7U49(o`>z|<4v0GH zv0N#EDQ|~pD@v}tE_Nfb4>@g8&!9oPI4S7jU(Cl(FO=nf_>6hv9DVk|AsDc7d4?<} zPa8z+Mg8;`l79UqiHUM~KAa0BqcbV{M2P)d$dQ4j`=xLY*0e_ARxYQd3rMhPEaG*TJr6wR9>=c;%@ z@87`azE<>7AzO^}(!5WcMQLO0O$9`ZlLx_en|$yyLm-e9Mlv^C4W(Rrg%9WQb>RdVuDe9EJi^IY3JicFS>-u_WI5U8& zsfzxQQrNKx&Kaon(zy3R()vxa{|IJlc=Ma^Jf)_dLk$(TS#S*P<8F$7&>x=zMS-xyyJh?i*QDmTZc+VC8xh+k>$~5| z=7n+F0xj0&J2w08s&6FiF75MmSXQXLr#0%L9Dt;Q%$w)Jx=szgUvG`1H&TC`*EQ!Y zYW(~<`pF@#ihJaM(tF18Qjm@EyTymTn!1^-0b1OYg~zqb1lF?4vlTCt@!#U&18#4<}{cv(HZ=wx5nhi0b!^QeWq=x=5bujF?H zcu5^?vco{y`)skfVq{+B6D?*JT8?qCt3DaY!=}!;W(Bop93#E5u1adk7N>5Gd`x0& zcG}%ijoVit!PL(}>Kl5ja9W3Dw21LhVFn3EwM=sV4!DB*E|87BIa-WBoBVQ5HFFw5 zwuj~>u6Bj^;1>~!>dxBM1Iy#-xAG1bwgC{SuDfGr|X=v7%Ehn@6q*fsk zjST#tQ)tv;=CT7n&k(Q#{QbJP?N*J&C%}c-(Yij}FNWw5;N$#SBY6X`a)DPTkD~Am z7_$4Jba&py$zbE@>evkPS4>%fmOer)lXkCLb190y>QYz%3I5|%s8BF=a%v7JLU&sD zpZ;-~BzGBdS;9{18ue>OAULhMXi@MpCTSBVYns{`jfmd#4eV5fO0zJT>!f)?#;q7U%bV8t&YQOMr+WnPfRfDtmi1k0WvLgo}4ED{pb6ZW?8+^HBIpzJ2VoFC|(n21O;Y z(*er%){XMH626w;QbpYPn%8){%W0j7Bd9?pcPf%#NlSFe2kbP(0^lw`@;Ahz38e;N zPpAW%&nMc8mf{P)jxT-1LU0Zq)UjQ=0}xJHQ-XuUsv z+{qu9+&`%&{{6ZC){v{D6T3!_;5AdLZV%t2=+X65uZS;AAL>jf5X0e7L74;}`Um|t z3YW;odkrqRYjPY>t*NPr=@hQ;o7>f5cm2)d?gTKmMVBo}T}>zaRREXz#8=*raS+%z zFTRU%ri?s$;q^~jezqA7!WCDyLpDY7aAhwfr- zQ7bltL{x4O3g(SFgfd$c&{4+mm+NI;>)XMZwuAuJDPQw0#=l}msb-XcNM_89g944{ z+Y7cwFTT(wo4(h0);C(`L<|1nomC<19HFM14ctG6?7An>LZx{Nf@7^s*I7u+IP8iKDiaMBO4sRnz_CS!4TTW1& z@$YmS5Q2b2;ZFW`fP~IoWT_*`MwoO(IIHFu%pppt0yiepv0syQOcJ1jRzo23 z5r3t0M=_>wHY4uL!o?U3>!G84~k>So^^-3{CkR6L;)f;xo zgw>c4pUa|H@@Cxl3#=seR-hALW%~?7=OELEoQj4R3exISARHTrVy9t4QEoQ|)Gx9L z=3FGaQBNVii2CLSIb!F9?YKkOG0?x;Td4~P5vCba`vOWUXGwxI^ySXX>zML?#Y(+= z5+UqOXGU$oOi_f#Pl-fd+U`CJBC>y?e*BLdNrdvpzK6%6Ti zB5xhGoGHFPy&GBe2^8cm%gGgQt|r;|dNA)}JRG`c=7$s)Z1H_Q`JWYO83kZBL@rI2|>mmVW8;9I8 zAVsXZP-bmGn`O_(F(ieP6f%J2X0IYx%XP;Ul?an)=hGCZA$}-iz3_P#APr8*OR5;X z2SN;1gO0zT6^SM=NP>ST zWP!1wAvDz-%~<>@KNPYl#hRF(*!3gh_<=fl&2ZtnsHq;2s@I^QV@AVN#zU#iYSIO z(1s7pdliZO;GqGx1YQl^p&Nd9pSNVK*AL~!;BGb=4+kOe`x4@DH`a7sCAWT7*Xg|N zy4ys@v0~ehKm^g=4q~xF%YEhNk0FWO=bH<-%w6{4LaqlPey}6MMzBYmfCJGuJX)*eltvzIahU_ z0r~p^d2p}-pFk8|oQ`{u-)&Bm>mUl*nl|G_ygiM!8`*fH39qpkTt{;Nj!*j-Lit;7 zm!UW7M`E6bbx-!^1hswssQEYmj}Wc90?YRG;>ThHB|>D^`}>z1E3;?q>~r!5<&yb3 z-jrUi2<6$Aqu1Z_18g#|NwLuQgg9Rmmb-d}qtJt}t$5fydCrWcd%y z-HR|G&_r;?h0FxcV$cqzUaH+Gh3m1{$E~m#(82?%In!^-KlhS+-bGA+p9mWXLi1KO zM%yk>?n@z(q{X-9Oa?czt-W~l(Y)DZkS4=aAI&LGpvPw(?-O12JV%QJW7%CvW8bb+ zXkTkz9JYIOpn1Bs4_1Os82NU^M3^o9Ikuk&D={-ss?S#H44kngWF;YWADc3K!o8kx zFyVZjobcY!nI|#pXUbUZMX9bTQ(}TmW~=T@;VtJ3RyU|@a?NMMGA4XL$qbk%Kl8v- ziQnpBTUx|!hFtxU)Yl+$cBN3t0@iYYS)*QX(|GkX>#0b6Hp>(zEaPEIG+Be1)_S(8 zz`2Tktf*L&F4qiR&q#9DW5hffs;nHRg7{L}Lj~cwUp%3#@u|}4QgVLf;HZ&>blEd{ zmxK6pPI$CMI*FpGWjLFH4|Eu|+4Vjyyyt$*_S8*T@aJ;cA~&(6*h4o7=s*9>3BD`u z*=d-hp`4~2Rx6&z$W`c5xE^l@zN^MFOTtx4g_+4V6V@ocxO1d??B^5J7SjBP4f77x z8vjXzs^^0g?Cva5X}*|3q1%LBz&^y20I5;=l-6h;ZABw(GgTw1I`5@(hTZwRhYu?H zoCT<}FZ+CN0C+D8v~#S)Xos2MUMpr=F+IL?0fgd{YTn*eJaIm8rEt8+N2ugAL>ygD|UcuTEuZ1EtXWj-sZS@OAv zx_N$d02)_jl}khHF9wAKn{sn8$9Py)NrP{GBQqM{+m7U7<(D{#I4Y96-ZGGp^L~!h zuU%aw^XeL9NTS7dEZGLv;(JW&MG2|pn&hzDBg4rDQsN5Z)%df%#0S&JoMg+#ol)`g zs0wb90HsA(7AL>Ek$fdG7L!}QS-|u9KoMMBOJOpd%Hi(``3-k=r|}lV#c>}%C`h=# zfsjY+Re2po658@QUiX}7M z(bw9lEWO*S=&Q~mS-y7{wf5TN+NrHzA?h6kyazf=sE#a`T3lwmrWa(69^8jN#_LET zI0q|y8i7GRsn?WDx}na0V4*_omq>{}wQmneQ7uK?TPX1i`@)W!P*>%;jifS$IWEkF zV>{JNI|6nH_nO=atVHbJg!$asqryHLQK@GH#zAhUSa)G6xpjy;NaXO%Vn&&%$ak@I z0kn(=4vD@U?TJ7!Rs+PCV@C$ixkyl(bs(=(n=_)kQ|*yA$sB}e}kNKr|$Q%{J}^oq_<*zRPu85 z=+BWS!~Gv#G8p&zpiC4 zkwM7`?{Z#C5^KYc3ZRPKUL;E+;Zyo^uIGgiEug!4!7?mlnw#$Qt(L*YBX;_f8Xziy zZn&LF7WH1+pv0ov`rZ#iucfhs@$#)KT*-y|dh{g^lD#;v#rj{; zvaM>o7+{$+CJ0Q-%}snhxvl`%N~IDp9`;og{^_nc&+mpW;iL^K$)_HLr>PUJsSmp2 z&ZMIgzuqrOr;q5{^#fedrUupG&*RXziicELiU_*V4_Q?xlR1hbSPM`pajPfjmx)@) zDS`>o<(nWUn(M@+pzd)|3zXRj)HtQ@ne~()iXg1gRc!%^j0*4v6#ckMs3(rOC0*qI+NOMI4Vl`spJ?(*D)o$#;mvmT!$pDp4fSE`%viBZz3A3c=+y zfoQbLVAI0rU*u!7D9#2NH=9xIGw=5FldCCLQmbG`?kRTxqF_`}M#Bza$B(`eKe$s# z{r~;lhv;!&R7eO1s+A^e-kNsE`@X16EK5LrLd9gCdYV#w&@If&cZB#oR3aIpVLapO z=uwOjUTIY(7A6^39)#LmBz?qU_e^ofWHa?e{NB!qiIlY-&@PPjPt|onC z8QXD8@51<XXam+!Mkin zJ1L}qvSM3nG(g$|eG_{#XVh=TywG0Un)6ceFMc9v&}u|tu4z_4*uFku1XjY>nb_#r zIUFrmeKxSUw2(?Ux5zg?luf5+~1Tyt=zu;H1( zLZxXhF%A>uA0ny9(WMG0`wteVpn9XN6>x75y(a$VPUOJTuPfd{{k;(y)L!icqrZII zSM0;SC0#+Sg+@5Z!wU1%Vg7r|oV%nivk%b0w_%F}&7_BdktBzdr=JbG8g9i3zXJ%T zWrXDu(lA#)6{S~*ctx_%BKIdCp8I8)i2v=|b9QbQbGlk6cQ=am7HCPl`PUZNrKx^m z0XCtToQ$8-h7MG;qFf-UJ=lX|>^YzT%|pUi28I+A^b|+boJ7V=gzA)#u63%LyX%-v zBXxY^ekOv0%S%pf?scxWe-etG{HD-?@7(@ef}re3BhTFE3M++Z$U&WUkntau^AbrE z#-S|v43p5D2_8Z?24j7dr#`Gxu_9@S6N4CXo#O^!XlC48vM>wdv!G4z2-UcXahdCD zsbSArpnxLpsI{E!0S;r;!2|o@kNsdwg%Ap;v*f=H3B`6fS}6@UaPsV+L4bNrmh3c9 z11h9o=p*JgI1>xyz2H&jBi1iIWW8L24G6C{?D{lVNtH3m?HiAG1I$2~$4H_AefTY5 zIE$f)5*dbfE7>MCV(H4i-RwuunWxJ=42KMlLEitK3s)88L_DOgy3hyJG)rd)a5O z)?m*`j#t6AZa-v1hH9W`C*Ax$%QI%~T@xLaWx=jUhT5otVHa9j+|4cNm7gy>(7xM?S_zhrdHOCYwimuq zdLfm*a&pRl0MkdJFnVqWRGnI(dV0IA?#b>f3wv(VD)vh|UwrLMjVp@L6X~ceEPJde zhkh~F+vai?-BCy}){bWu;*4`Vvx7#<>R0#hDVD2j0HF#K;{EXEi%^xH$HmkWyC3Zk9LY2^!LHQn7 zXQr%PMRGjn=+%PqNRo?rqu9;XdWKc_>%`K`rN}qpkr%zw#8R#M$~WSfeZ^MLu@@D} zOwS^nK+}2PX+d9N7>f>&m*ZF$>H}WUU@Fq7PuTN zCr%sgh-e|^d+(~x2mMy~0ii#c%$!iaY0d!~xDr@*6H%lsJZ#v&6_KCFdF3|lLS|61 z;gqxWc+zZ|)nK^X(S{1wxB+(wO9~H{~T565oEA#Wc$}cINDSS zSN1zX_e0NnPL^gC7u{#Qi9=V}O^<zM-KGiV)1<#z)$-`>YOk}Vfg z`F(gq@mSNH>CY$IO-x)I9b8{^fw{C;&1FGknqAYFRg-r?oYPL@0AL0V{ipu=A4vNZ4<$vSPyer@D&XAB{ zv30}TalFN$HBrsVvE_ACdPXTI19Euhf6FwlapWosNjne&I}EHS6?`c|m#b3*InUs9)@;>kv+j>B}}I$C=}pom?=@R8`% z6Hyn?egx;hRV*&oD2KLY`%Cbo`HyOdIZrOpmr9j`ixlPUHTZ-2|K(rX?GU;`Z}xZV zEsVM2WQ!l{4yGv=64-%-s-eba%o}AlefTuKF+d5g6L0Ms8LdCtLTy*A9n^@_#~4*c z^a6rn+JjzS*&>w+o}jk*TaLovU=iu{xNj6pO>jYX?tWNkraLGE8cPjW2lBXtNTskL zynQTuhEmCIH=T1*lgJwJxiig9=wr!X$@%^(1?&MOsZNkoQ>_-Kcjc@Xj`*stNDDJ+$nw zY#5K{?<-yg+k8xbT2y(_gqwl~#I^gAc#J9r1jl%X#xSy~pk`s=bdKZqg;@A|$Vi{n zW0n89-6Jk|o7=fgaguK1r`z3eD9=X2tntv^ugdj0O25rT-%$oY@b4`gdXuLOv9MWx z8m7OR@h(>gIINpeCMACWOTh#iMY)yWKzV-iS-?`xYnwh;s6Fu|XYh?3&^y(^9ifBu zybBkf8=F!@NyjP*3OnR1lB}*g+~emS25pJLnE~!ifJ;Q(hv~l6=yLfa7^;i-n_z%| zHb&vVhkYr=jeGUuCMNN^TaI7NF?A-N8a8n$@{6{tP8pd81$%G*9utWoTIk@)@*PF?N1n3|AxY{Px@4nXXSms{$AJ&=fB>ea^e;x`;{?E%zVsb^54-F%96 z1H6W3DVgB5`WmbYmFEK2GrKTQdqSjky@pO#wSUg|{_|)&A}kWQd+n>UnH(o z#!XeW&=YptQwuX#xXhX*#|d9KFDm-9DOAIZWmqm!YVv`Vw4(W}_hEG5#v;HJS(6LQ zZVlX&vVizR+1W6fGw$weHa)S;h>-unkL=>49&SFPcWkL7zcpqEc0Zq~TPBb73%P*4 z&j7~k#;D9mE?#}c^t|`H=2wKkXZ=y-@QF}5n1wa}Y?}_lyQ8HXK2^U7OW`>~eT+i$ z(9uvPz$3kf>d`91(h9- z%#K3RxtAJ{hj>b|TwJZ6vD}T4(FDj#ZS}zqI#)Yj#Y(BqUd<@dsTF0FXT!iKB{eQ0 znjBHKdxfc2J$qVRJvS9Aj zgPE_8T)~1Py0F{7>qm8i2!N96r^HSJCnoxMGeZ8-EMXu&y-z-~opvi%jlFVxn~C)F zj=5DILi&^4-$@N-i{lWXEzIG4%6ue6L`0?Q1 zO~*e!OI*(VBmaKTPo2vAv@!ogRsPr4{@X{~1UcJ1dW50tPn2IC=EYW8 zm*$I^kR|Gwe4`A;o-pA*abWBX*&<%=*$htWH93R8hS(i>-IAhwq>#pn^JsqJfFJu12Smt*w&ECr1-8myXVfF7? zz`DT8sBr<$=C6~7H^peaSJ!vVXAQ)JlmAsN0AB=)8_>BKs&vUbu2t668eJX_QEgpP zh$o0!w!ihTHlTCDyfjl`Id#FyYT>*Yh#6P9`w1ya>f{2o(*!@wEG!}a_dzL=H|~!1 zWNJ>k1Azc;1li>-EwGc+uBm!;p(kc{*nAm#n`9R28&+XOpAFRO=3(~4pORsw(0Rvs z8+j=2JMDGA@UMJnyeKw;6QGKs8gcLCF(pLkpu9s7qc9I_j2zUqrd~>fQR~SqwE$ zvl?Mb6B9e{AGqsFHGRI5}nIW%KVN^pXk zG<~Y_cSLK!9wa>lC03te3WtDMn7WcEj4Ikke?Fk^Ca~m5XZcHLE~qKyyxF|^mxgTQ z2lRhlLbQgW0GmIecSiPq=B57Y=l-vE=9}1-KWR{Ro<5*-&{-EX=T4Cw14R6G=bW

w{yfvYwGG2xj3etD^-eMSE_68@4@|2?->8hHR@k+VaG#EKFH!rXgppMHC6?2Zi;d z$$64n-LJbe=7YOYS@T?Y5xa3w&9X7EF+V2@&w*q~xJm0HzjO>-x0a}n(3ePZm_#ew z%%D+T#ED*ck-vKA`;pi@j#SNPaT-JVE~sDweKs+56*LDBcWf@$m1q+Ib54Aagpo81^C3g5SP)P#fE&L$5161N%wv)y*bZU7 zWVoMFp12@l5i*)(h5ZO&Y#I?NkBOh}iX(fdN~6N4nXnto1I8F$JjJqMyaiGEd;++5 zF&MuW0%~Rp(1KTa?7-F3|12`DwI6# z#$z=!Gi`yJ{IIxUacPB3|1hDLhH^v0L0V{bRp0e<*6}W}+OD;{O>$bL2X%9S z*P$$T^_zS>?Wf3oC#;;GdZUY&=1x9@e63uiEc0(O&aI2*fW?)x=6bq%O<%flT{pp;dc-^HTSYoBoPq^A6T z1ZuE1eLCD7=S~vNGp`<1YC)y5fKFvQ9G8N#HB+l)d*HYRu*|*v==jTAxBmR0$7Z$R zkDzs529m|PrqxWL4x8&VYofC}XtG~*i`{Sp{L zeB8A72f(7toS!aW86NHhq*_`LQ^YIISIXUuK53^NIG^Ekkb$XQ42OT;@OqmfvC-tK z2!K~wIeQbFn&P)px@2#%x|Ag0j9FfA7R`=0ctco%K*u4;OCazzIN6i8gWwM$c)5#1 zZ7Ve}Firq((C9#^x)>-gLhNz@>z0v7_YpuKG$;gaR_uy$fEkXSM;sb_I_=W;( z8pr_Jk1xVpyWA|y@%PhK*V^CKB=dU7ZtKoJG`_h^s+WBL#2}M; zkA=Vp_-H!V~%zjvc$63jKTmR%c*t#cx)YaFsosm!-Cx=zo`y}wHUOn z9`)ek8l3;rJ1lvhq};4hO~ni`*QV;4DO(B(t7);b<0h^zTcUKhKmY*VOXG!Cv$U5{ z)G3krstQUIP+>xpoY@2kifE1zLJueL{;>KWfqbb{`Lv4HqGgeP;wqtd{nZ9r?h%IjT^B(=e!j#B(wHg-G5Wm4 zzHi2!(2$e!{b<}6o*LpxT_Y7a5F*1}v7(%6V#xO>srM87#5?0S&{}22#t!C*lB_5@ksajo%?9*nJvSm+#wUa61sS|gvgc8mu7r)wAj-8v^N;p1^Qc%ZX1wb}ga?KZ#!81eB8*1GK9q|nELG=EH7Wz*q90`k=&gF*>up=FUjveK#bbK_kHJ;884rr8G!w z1!!_Vaer+zt0fx(VnI;SZ`5CIcw0;T5XRP428gSi^Sa}1ZX0H&mSafc1B!xM7T1GB z)(VTFJz#`PkF#9#Jh=Q<4uvjMUF$`8xv8#Th6CE2Vd!$>o8E*YWQI6QuqL3VSrU;k z^0Lcr3VU8A8psIb3u(s0k-|CaDlMd~-UCCsse#1H(9PxW zH*0C5$L%F2Co>5&;rP)(jfSos?_RRTq3q^blwEr2>woR&<5oRV~d0tVZ-~1@q%@y{QWmof3%QARrh&;y$eySu=^8l?uUOFXquClc5W8z@z?1sfVJgl+t?ha87rbjZ!~Chs@LRA>=0CMq@HdiWJtcyT zNDW;F6`A0uVz_peU>}3;wgS%6Qn#Yz4qBNDl^ZITI4Ee>^3i$dHLJoZ6?$m#=aS+& z6c8Y%Qu=iV+$02_dW{n2G%-xF+%Mj0YlIY&8o*eQ7@o)vlQZ_bMc&(IfyhQ*x=hnr z6Hk@wJ{6#XZMc4M;k4^EF#?5YB{%i4Y*)8s3D<2^+1Cm@j+q`JB8QcWWM;$0{P23( z@*`qm@Uif*-(GH$g^rmaoh^X67y7C3ODsK7d{pweTQ5LtaR#oHn=m0` zqc5BQ2bC9ie4atOhMD$Uu1L4b%q^{LpxrWZR*2H>iQ~fLCYHXle9R4b##+RGa4hu( z0M0}hDX-tg(zeovGv%a9Cqk!X=^Qz~_}TzZJfGXZ&-yx>5{VH9>n~;V^|N=NiK#*` z-Qacf)22(&eC_VB^iMG_^ZZH#{mzR6|*k!&c{rFi=-L-%6Py=8_AQT>+e$26f;fRUxiVT;nx4Ob zn*hQqLL91s|JxYCo~df5lHpFBON@|2S`B273|E4(dDLu~HY)Q@lH~TQP!&hlkSC16 z0gh##C3&{m?1X~Va?WjHa>jB2SNWwcPgCH2Z}qRJn{%nvjNIBX?q2&RAgoROp&PH* z+6Bh@3l9mRS_>^RG1L*A%yYM~Z1d&E_SCzQ*P)@@4BH3y|-bh-$>6)|oVld5H+u*5ZJTBse#z81OCnA$T|Um^SlmEkEnj zwo`u41Ts%;CrV%PqFN}BC#Tb9wN8<>%LTr`7m*k*wMTEyDfNTBXZc=PgU({6b&hPyXtoH1`+ zbgo=}0}m>n<+wJrE1+bTk={|D(|QVDT~Y(K9bFT8avILA&sB?i|C0h^eOW;9=Z6CG zK={8%+kfUp6Yumtp#eWK2Lm>&f2#K2C-eT%co<038Jsjo!@Ms*x(5 zQi8-j@Z)(JMndPeL<+nIkQHludAjY@Hoc{#WIf{+^$ttmI8|XVC6!^OP);=uA;7vg ze~%J9C?w)KjyIb11bTiH9>2UIrfth_Z*RIsFfbilXAi38UrO@+)QK1K>K|rQh4TgU zbbryrSNz)7Y)!?EbK&3+mM^&3 zjp`55q1z_0yQ#>+w_B@*OZTj-S3F00j1p)|~u0k;gK#Ng~yss+iK3SO~**RD#!N*F6Gx+x0viO3zlQ_;< zu(A@lvAW2lF99rh(Xt9h@Jj>@3E5n?wqF(WiO6JE#7d(NxU}X}sgfXpzDdE7CBnJd zQEVtV0Ppu^$uPnmLL9S5OtrI>B`;bBvilUxXgZ^Xv9kJ`Teol-Bs2Q+v^#kyfIWDm z_jeyMSLS}d4c$n^u%-)3EWnPP33swdxt6Xyii!EuE+&*xKI>|3QgRbsp`rqrs)sDj zll?|13OhtG$E#_t+s*-Z7$MF(A0PzbAGuAp3X=haxh&K+|5@>}4oFOoP2FX+z%C+k zy3ynfM_aOW(5da&*THRlG8@NwCTA`uYyxiXQ7lm%&Usib8-@D=@hg*SWHQ%w6vXFP<Tg#> zoif_1qKak8cJm$Gk{l{ykfEBbWoYf{4uK(Eg4S%Q&7=>3-`zvAQ85*d_{#8;H&KqF z>($g^Y3l43lHS#Y{JVe}*+k^5a~@(#705?Cl>jPY9zXt6k~T=TsZ-Ldn%0&;QkTt) zjWzJdGky1;jjH64GAq_QXt5%RLZZeM;k0lSfHO~+9iWCPKV=-6%{Mg>m0hUF;b1f9 z2op4vdHl8_K9q^%rr(WK!lr6X&-l3V_b!LFqTEPrUrg&C?*RD{)S@?Ewzi{XhyMR4 zd#4!DqODuEtW~zFR@q!-+qP}nwr$(CZQHhOojUg>H`&QP`Oi(}^L&|^qmSOd-dfvL zLaZ@OUW0L+ku%pJZgTSKj9gbqeVNQe9Lw{&fu#QGlzj8gU`Ol)z5%zWLprQq$#p7g_PG}fGm?-_w3 z>`%uZtW~Kg7Z%Se#*bxikMu`Hiab+gfW$?%8=^+cAy}LFoBnlCx@scnY<#`$}iUZ7|ySYsio*Fyy+H!f^Xe)ct!np1i-lT1#9$}An8c@ zpiC3lllu%VhA1YDsdk^#8(;9c77x3zuPvnlHVSypQnU5&r3Y%&+sC75mc)ls0++=aB9WPrK|8C*{l#yY%Djt&X?*+}P zQPM!UQJHLph1FqqMrvK4HwP6if?M1GYj~Dpxy5ch`T5;BSWHLO`k%$;4fa5`v&MGl zv+`S7+ijDevRB&4p!J5uoLtN>ir#z>hA)uR+SSs8U=r`qeDNWBlYp4DVntvVkBG=i0ny49vkxAO|UCxYD@PnFWP?} z`m;*vL}4hoDdFJTFJXPF+jkUck3(7jihkqYNg9U~8k|AKZBNFVYFJBuAAKl+FO2xt zVOEe2>@qqGl)j}GS(wb}DU9Y(c;pzA;AL1J#*>@KAC->CC6V?{g(S$=x%g?|I0>Jv zqTxB->4l!Eux^2n9aHjrd=LI(jX*R@jJ#WNg#bcgG*FFWi&^ESVHz# zFG0|tS-b$HJcI<=Xm(tOugvosM<_0!KbDo+n2ng&=-gkhg0ZU`zk;zTTG*is%6#9! zBk`YOTv>P0OI~r%Xt&mnE8~1HMZry4TU5b6bHy@x@q+zKJ79|E;GI*QZc;6~qw@ zkDJk)wOATitMD|16?JhhQY2m0;RBEdR7?$&?^tpfjDT*AM7mD92X>5X-Xo$8=BAI1 zKzctaSC&4FufUy0gLw;BL-f&xyNv6;s$WTd^jC2D;#Lb*36pOqg+bO8#r*U7Ou*0gjhJL zQ}OK)3lH`cU!)?^) zwvWEid+gV{>|VTXfOlA%#fVrpK;wn3nwXL0>{FFc{Y1@S987X{a@zXPz^ucwSySgd z{Dk<8?4LHcG95d8bxBRC6TJd7tX!=3-e7^;H01MlJUN7~(?Twz_+}F0ZmQwZd-T(C@`>{0HScof8--mc zV%ML(p>%xjm1Jm3E(veeU#+NVM${DNP-R_OaJ9y_h*cP`r0$Mwal0JzVSh}&I>EO@ zA1#?pEJIx$*`#mHS`oh6rFJ1}-%~<(i92fq?10u-;eRl?=lCFY(Nz4u3-M3(6zmE`A3i1f^EJ)nXhr(>@5YPw$)1Kxn0;dL16RRtJ% z;J?*TIJ|u~`)#*h081@0XX_NYIY~?Sf>h?RPUrqO(yqjkWJ0kV=%jF z?q2x~5m`J!m{y?$7OC-hrDrnKp1$|$vmF5)<#hPO@aP^;sR3Qaj<;1CcRf-AI5HdC zK{N%+LA1Plx*p-K5TCQFejM~ODm?PFUNgwxKW4`)F=fF?(b!fXr-&-@JTBk2b_9V< z98ti5R_Eqi{SxUtR!DEb6E58*5?wqFn^Ab`BMm`#W?^6`Ps=}bX0;Q|;COp)`#~)B zo*Iu&_itLqlt&7hEOD>o!qYa*eqnApj7j|Pap0_BMm!{qpKd%f(yHL3*6-f-e=?r4pdg98F8e&`;`((uwC%uVo)24ZnfDF_CsEw%WrE4T;>C0GMG?b7? z4!M7nsQx7$ilZLGj<*uNg#z;~$sf8QB4bJBnNjQsN;tv+(F75H?JPUY$ZJDpGsR|F-${8OvH1r2&k}50^?^9}=Qq2T3IG7(zyB3_2A2Or z@f}jPvN>o)_`cTRvxHw2M%iipLe&hXXpToYPwBO%L8C$qCYY@vKG1ZW=JW2D8N{*#2njC@8DfxE)?F*4HQW4OZ4Ih@%hMt7R&D7^|Z!yK0w zC#p)cmTvN?H-gK^Gv;Pmr!>rlM2TAYbs48Yg*GYUOm&nN2)h6GrGQ8rHw{h{kdA-d z6NXp5^4^5!fESlRM;t+r8^2x_WCKRVo<}6Qg6uf{>7FU%36HbuSRe;IXfsAF5dha@ z2-ze0zG?NC>VRP;A3%-@A8OwU!X7}J4_+ig)PWD6-dAw73WQ5mIl@X1z3Mh2!T#*| zZ}j21z8=vG02Df>r9)~gE5a>^ia$3`Sfm*4NGPJ;XQ2Zz;T(PmvQ!3eECZUIk9LHV z79riTc?NNB>O#4!?_KDL4dP20~+a;RIVdN_KM!*RiYKH?_j&aK$Fk}ah_EwVyN@JOuHJ{D1QT^!}7Kf1s05fKj z>qFy9cDi%vkb{jljxOVrT6=bl+Byz9G=Pp&1c_fwF>b$9=G!e2=X%ODI?OT6HMR2* zO|kD}HNdJd%d_Vmtj)-C=GNx4jd3A2`50_u2tH zeWMLu1Y3tAFXD&&C7G(M0c-PC*pTq^V7IW@vE{Gifxuye2ohk+%ia`u*Xk9EW^)ah z7;m~bf{jx>chys_y4!(kME`m7X%E63A03`R0yEO6a zXx)f!aB=Vv$y@G%n$d0^906K@2%GOfe4S|k){OOD3;je7C}+__(aF|f1)fMA&Y*O1 zSmi=ph(PUoJaR@ZtD;+?l z2%#fqrmZe#^8kGRpIkrI2Mg}5XY0(j!6!~f0Q5=%_m%?~6PCu= z03;Ynw3l=GojoMXS`u{iy2Of9CHYeF#PS7o}57y^ih_Qmex0$Vf5#9Y@PSndq zyqK9rn{a3JNa*9!;^D9sZ>bg2*$tw6Z^zpfgU?5B8@pjACO}g1_0=%d&32xsWIY$& zXO5j#;n5QV_W`M`=)D*-f5YGd z$>oO^BqI#r1wMHLbk7kS1C{c-o@I&I$jrAW z+%6=N>cwEWKW04Ne0x&-1eq7CKzk#}%`?)!o7+gQ1_~r0%GdQ1Wtx!`j0}Q9h_1>H zsl3eqhO+tMV7%(J9=ejHD5gFsdV#p9b6RvN4t7M1&O0IVuJwOmTTmSj=%f$3y3$Dt z;0J1zc9!~;e>HRdK~-qzh6))jP8ib#CZl?$OV(WE|An7IS7l$X3;}mSPYpZntdU%l zE}$aYy4sSCH`ZFfjytXYFoM9R*3IGIry868fXX9lBg^^>ss>fYq&jcU;GK)z&^m{< zM~5^jG|*m(YSCML=qEi*-}BF99Z65lEj89cx-$GjgO4V5ri%?I`=LJ!+6@r|w$;jh zdcWNFm#dP6UL@2+N#_%qyrGa;G>z=c7mXU->X%XxrM<1_Z=uPHrn4G;`3lS{qhC-y z22NdKsrEUxj;4w?F3^Aa3%!5^2tcelAcR0E_CimN#cIJFbDWq%D0r5hTIE zzQ`5?<35&hoHV;Pg)n6xkMW|PJ(3oX-A9 z9{lIu8lR)If3y}@@XpmWEkzw-23eesRvg&nevdtVsc>m<+1a+NkbKq1vnfMToOsQM zH_E+SMCsX9ig3#c_1l-bmky5-Od1R5;j?a()zpPGhWfDgCZnWUDa&88yi zK>4W1%6c;BT1>g+Fu5Og74Ssj-QlFw_J{UwqYq>!Y2+-%%V5fhfuC3(re+T&-b8?a z`n+x^?7B#RT*kw|ho;xOz~!~JdXmJ#5nogQz|F2x%B5Sr6qUWo8$_~Bo4(k0YKAf< z+>NASqBl;Zw(#g?EF0*R{;pCmQZYl)X#tdI@2F1TK7rzQYde2eZ!}=P@9Hh|Ojpto ztl@OT08;TJ0F%ZU^Y{bz^ngpY@?@zYrS(prfp8`WO>gjym}B%kZ3L`7a1m2EqFU}R z`$~WhOk6dfsa^dmN?q4?yT=m2=b+L(y$X3H))Z+D*@V*rLju+#xtkp|!DZ=ab6%@M zLRt}sZUwRzI%#j5NYQ(Ab9gex;acY&Vfv3xd1FoR!EjBJGVt^*%T$+a%H-Dxc*MRj z(7f3;qm~3bEjARign#nh_rX!QXalbgwl?XTL{aI;z2I(-4A(DO+(HQgudF(}y_;%I z0Uw^I+y4}cX+>RGxl^Md&E+T@Z>>x)5{BWG@95r@QB|~R8@+dF+F*R#>tm=w$~}2g zoVb4h#J_3zz-#$>~((&UUXG-GO-%6cfJX^H8G>;stWihJ9uTbA< zQg>}k6fruJyn>ze6!C&$TFVar2Z2v0tp5mIB9Mw!{%H9Is?5ptWRj+W= znJf{T>g+|B=G5~j(*sasN;i z)|wBkn{h(kvDz>DIx5LVI6%c+g16VJo$jzK{z-9JU1hwm3U@qFvNi>b*kN*1C7ALp zr5|g&b1eC#Q+@?Ok6+Ej>w>`SIA)A*~orjej74cz6rW;9YEGcU!+IS$$h4R(-R`LL z@lAcp1r-)q!_&zrBMR^?gw1h|8K*Zl(VIz-OCn5?kkWnPakxTnQWBwjP)-Fll9^cA z({?LURxK`3TPOZ-n!Zvtt;xm&e~5jrdYY&htbS`-+FuCSu1L?p9M$fSXE$u1HcNrv z+9*dYn?L!GRL!crY1C=DRT>^hX?b7NZ+aeJ+L6$pRUxRiRW=};S< z?U?#lyuo)^))bprMG)sT$RwQKh}w53*Xx*kA!J0Gr2Y;uGH}2q{u4g9>^-1g&wYau z1g{z|q9sKh_?UTZ((NbP*O7P|Sr%xwCg$YB5hbslpIZ8^U*T_1g&D~s8xnoO{}WX4 zDSjd^%oMB~KH3QB+t{{v4kuApZUJI2wZJ;quQ_!O5?QkRN$P^sE--LHfFB2R$9wE7 zboy49!KoshsT;_M^rA^#+=UYk!eF5`N+@@BdKst5E~`Jx_+bYpTHB&9M@7PB=x_Rq z*+d7;=cJ;Xh@`e3)41BxJFlWYChOO(Tpt^p)_M#{wXk+%pr4o1J2AGT(1jyA2F7YE z9f#Tnbk<~+Z|8NtbgW0bHo)dhCo3~H{FX+vELO2LwbTbJoc+=U<^o|Vu`GaA$3&^q z{M5WMcCFtJwy`{~v{iu_QkmYCbXArOAq<^CcWFAVqDhr+tY>i=HahOEMy~}zC3xYg z_Lk5opKRU-!org5VxD}4(E!$L+5`O5YdyfJhcENt#M;pUx`Tr-yZBbx=~sqO#BRA}*JMZXj)fpwgR1X(m z1yF%#=uV6F=^oeC*p2o(iReOj^)!4G@uqxT1DG0`OV+@$V8QqZSsOFPg${EHH_oF> zT+<*(1IA2%lla`+ix#Od4?^8!^}e)Q&&C+Y;k3sVAAr5JTHEN>W+Me}4irTh{z0?9 zG3AIlrF{|aj^9Obwo5c9CKU;N|*|R--8e;$;yN1)5~kvUn7=eQgX#HBFL@1Bvhysmk7Wb40~u+14xO* ztO8o3If~^>;}4UB)|3+B1Z}yP6Aw3D;MKF|tTiQ>$M$TJT5 z&EOko%t=3|k+tKCwsPa#|C++7 zUpSE3saCm`0HIQaAUe4_;tVq8Bzi&#qBU0|msjAf*H1+$+8nqQbBJd51tJ|Nn#4G$ z6D@Snz?nebq>oXmNZ(76B#ISG;8-Y>C!6AyBvO(0%>nvRNw&RHsi`OWHQee9iKF{4 zgKhgaKq#<=po-JJ_nSFGD!Hqfd?n)wC24WfO+Jf%-w zSp2!Az4Fi_S4&6$>lB>N;x3|RWcW5gfVlwRQN0`o;Pt2EWmFh%E&${Qh?eHxAE^{h zeBp)Y&2&@K>TE=n6&&=T{q2`sHkC@{#ns|GeeP8@wF<{6Wz!}lYZ?lcOl*AxV)94a?I3d z81YJ$NQ7Zv!3D5Eb}my%65L>}bh(s`5dwk$q0rQV<`B7xSc}mX3=%q%gV?XeY~Oj( z8swsGNxnk00Na;a`=tC{MPgg7s2Zxi=aNhNx|}s8nOj+m`AGYy9R_SQ?qg}$VgTj| z=vHukNAE4m@JG{)vg=T|iz2LE+SSp3#i%>*jTx$vq_=#;`PL;0sq85{L6dzw-`WbR zTr)mQQ)d41c;}_Kj2R5(9lVuD$`4 z3xYX6%DIt7*w$xbh6ybFB*ND)F2yzx5W)Q80S)9294$x3Ao1Ba6q4Y9|N zVusYSyk>k`#`dTYpx&}^`HC~nQ|D75Ne+o;C{2zn_W-lbGO&`xN9DiCf#&D$kXy1N zU5)mCZl_Q&E$PWz?oV4TK5rKlHWVH|CM?t<7goJdCP@Fsw{F|kpVt( z$gkDhA%8NN9w~r4kd#JT+$4?YzK{=c<2P@5Qai}qvplxj?19GMj703VkoLIXyaYK+ zD6QfRA>AUbA_LIxw_Gm!)F_#_9!>I+Me$-#Wy?QJzzmSs%_ji*DC=U3C7EQP8HNp! zov!!dqZTRt`jEvU(`2wAN(nk!L*Dx*%$^<9mB9XT@V7v5 z5mA>M(;qY!typt|_yHB|WQ=_ug*KD1th50;SFk)P_h4lFZa?yOpZOAc&G1?NH_7ZI z(iR-);Xxp574xMd1?pRJ`JD4VY2XUs5?MqvEMg2e_jJY55L`Ys{}tIT<%l5 zi{q&KbR`=bcGoD^)h>`_)u3jW{%L`;3aF=lxGO*uiG1cO<@b_wI6p>N&RPO{)5EmM zQUapFH6{5}$E_9_ZvS|U!1UTm{aAOksNyommwxlRa(PpW<9x@E)c%gd+fk#wL#!1v zEH+~4dB%x^R8t%GHz6b9HZxq@PY%^)_d(!E%Btf1Ykz6p$Jsa}h zLzu;Vl;bf8Q@@`~dH2>p)L9vdq9(-mILcE(;>T9Z@`zKL^5Ab}%Dj)jseRp?!Xbkz z)b2CEoec%9il;hdT)1$bT*vEoY?=}s^RpE7ckr4bl<*zC>zwjX+~o6FJMBQg|`K5lFEH$3H?ZQKQ>EZRzaWP(%hWcy$$iu+GC&8 z6P|-#rSs(^n@mAz-$r$C>swTfSVN8y%d|`NubySeLAO4K=51PP+#O#h2n4tf${%AN zN<5MChvXg0y(wYIbUYPoMpbF=HyDnE{1_*14{Rz+R2*M~>vKPS2ff401MHn$Rn^|N z$Tz8U<0z_Kn?%jcN~BFhW~46T#<_2FVaW+y4Il|)wck*ZECyrDyyT(WgQnTH7JSS zbhKkFZ4}&SPZClpWM+c>y73|+)@VM^!k&iE56g68d@yFk?#T-yb5UyECVSsI#$$Pu z3w4?vqeU!vDSipgc5Rv}oD}3sD}XC(|FX@S4=FnKwjHCw~8Jfx>2&Rk$QQK5%@IhLU$SvC<5&AnL6^Vj? zkpe*N*O@dWY86)mq+Z!csh5sLy$rx@_}Ira|1lg@Crq;=F#=7>h2S>|xNa;?OdkZY z8CjFUD2wMqv}HDPiZJ~>#v5eS)LN2Rw;Rp8RfIN`b9brhsU)3_ z=lx%y2TwX++BM+id;*r>w;{a~u)8{l%){^W+Qm2ft>FV8xl_ajm8~|C>E3(jk8eM? z3p*3o7HB@i>2d(+_0?iSErbDs)7Qw~t+tqJkt-=)hf#3h$s9}>fXC0>G^O5-*9^gL z&tk5eNfSBWmA^oLeuu^lcl&FoK(7CY=Y(T$!rxm16wrwuC*x^pg9 zy2&+Y73zcp(sF_1j$ZQD8J?)C33yfN;) zku!fR8;`3XkDRtrM1!C5R{X6pnv694OuN~q!kH25o2ow)8&Wb^v3@#zOu$gP5>i}h zhw)zqE3O>O>9}_AT;&9>M?9@rs)Ks%EUX9gI)F_&?ebKs8DSAvCsP`KjlFdpLX`EO zj~rB%>SQot^zKRrYgHF7WLPHNMQi-g*V_1MGmdTmuIRV@!wf@1*(f zT=(ER@_^U0yk>67y*0O+!|=$Jo=dtz_1NH+RYh?^e?>*b*TbMIewbXJ4(0~z)^(#t zI~}=#mzmUVxHU<<&?)<)*y3*ODq+%=_d#yi?sN0q9xh2uwcyy@Y`W>9-GR8C;|s54 zjKBCmVF0_ZyEe00eV$);fr7`61u(O-Bl1;p(DuQ<>G62dHHv^2YPprwd9{=)-cn}C zrkT1VFm+$MRnPdmKF3et+5U_?6`kkH~4b(;Z`w1kJtq{ z>wpKbF3U)R!?rl3WnVoua-W_I_{hDoRKs@x9l5fMLHiN0gvLFMt7PBO;nh0hho5!nZ zm_qdrQ<)=}J!sRifYj8E)3W$9TxQE0A%kwR4jg`QQM)G+9J^RIH%LmA`bqR`-6qNP zWg`pL&%9MXANXhdp3(#SMwZWkdQ+-MIcLfz0lMP12V{&e%%+Z3&6C(@OmXf8ZF;*JiPkYO}&#A#%#_z+B>$~&w!iB>si(3?RC=8irhQiw$~6{7uREG z7dyLa*$2xev!vCU2`-C>(X5ZU;MX6njyGbq!cgN$?e|?H7$# zp?q^9YssI^x4Tp=K^iuklLm~j{p0dICF1hFGzrqv0YaI=@2oPe&tw6f&CN{KLHa$E z<$_>Bhsc?l_KO-&tj005Z(os=8c*gO0WsQgN8 z!+Jv$(Q8Fzah;Cn?>~n$mK#?<;e0E7CwekAX?S%02ura|SxcCV#wwi49vdZ=n|yr_ zy58#n2tI|r^)=l53ox8X}-$djat_!7*UW~Q#cM>2~PZE$Uph`>c9d-aG-PBh5^5rJf0NQ zVe=X|LEaTmK=3vw#o8^gHjt>=KUU(D7%Tcj#&|6)p&Rkg0hJ&AFK&n@WVXK{;XH^w zO#W<)zb=-cJOnR2J|uIJDfpRrukveO`1jFxt5!QSa2tIu+#wr( zp082*u74qDjEE!Rl^d;oEGIN1G4EH>R8h1!oY*=-6C{vXd*WkB)ao)df4#C9cAjpv z^c&Ea0>jTxS77$+a}tDDkt`>|R(9y8uVn(-c$mXxU-Et&U|{7H6h~t6h-n-VTdFhCXnN>^q{!uf?sAs5UGOu! zbS1_&H~sE*+K%MSu`h?F1;Y>60%a}9&Th`Mp{eiJ({hsuqhNabZlsM!bd*fin64v> zBgx7whKzKv13H_5Np$T)8DiS6^>pO!a5G11U(T|LiC`exgAJG_+XE!Q01A+Xtwo!I zi7iXLW^0#&O$j4b5<0=-V47wd*%*BWWaITL6dLzNY1osS zp@r%{1G@UmIzRynORWTE8cDmJie|Ro&tP|LWBCL>NBzl|%CP`R{+nfVO!NJ3mc~UJ zyW7RM88G=`GSr{ag`EVR0AWTF1v)!(6*B~)`U9|K_NF6E>zuK%3Lzh1!L_q}>(bGm z+6^QAD|%v@V%StGTrkzbOn-kcmU?IJ7nk`wze*peH(v|JP(gqCg%;ubI(vU~!bpu} zKS*IkPQlyB!li_KA{0O)HV=FDwidi%N^W6z;>3iP>p7p!0a16`SMMRV6Y1jTE7e|? z6OVGQ$+s{%Wm%XoOyEh=v-p>~!akjBO`~Tq;-GGfU`K?}KAgzKKV-l&FdsXaQ9a?ISoXo=~b%)q0ag2*F+a9@>R7E()# z|Ms#KvPMSqNFu+NuBw3iLLtR-oq~mQ&iePck_BVt>Z5cR6yI$^lhfj6#LBOY-08V6v5xXXGExBYOHE;qOs5R{zW`(6+gdqPLU$sk{1 zg~d@{K8!)b()sr2mcDD>DREC-?}_e}MCY|spJ^xCtw4)a)L{`e^ff+kab2VPBmL#M zAK^H4?w1h(#Le@J_J-~gA(zw3OvKG|*M2Yeu{B~P%b_?}9I41mq~eP`weA!4K8n80 zW~m^fU_^b%EtHBaHM3oMo3)N>s70~j3|n$t0$W&#SgZK`0klEKlm=og$Hc0*DpA?>r|C(!%md*%V8HS(J=spjZXzi zyruur*2>+%!+Eiy-n23!AlSg9cvg}1s(%FTAU=oYDv>jnB4FkkDvdKFU(=!=S9CSY z*Hk!<>EdwSheoHl`IX4jvM+wwq`dD6O}WwkA>uLJ?6Q6ijNOBlkd;sHI@aMun>H5} z;pQ@uSIF@0Ap!=UVy$CRz<_DLuO-T(@l}I2i@wO5tgI;?2-3%K4TnYK?L)XLAwX0d zFpU>@wRM9`YRi|7m~R=9P_Kjg8kkZl-_hzH=|R(QU5$N3mBahVC6g?9zQ0A?%h@LC zT$PuA;&+$p;iTNj-=uXqlGdB_)}4|m+mO2{7=^iC1NHFUeNZ=25mN{lZ78k>2PAW# zg(dh>-JW#nHZ5}{RMMB=mBcjqxo{leNOPTMl9eGglVt3b5GM1#Yj)v;=w`X}bZzhh z$*Bm@zZIJrT~P#kH|(}7U9bu3s83`j5WinhYV0VrCI9iM z6re>3dElvm?NWrAh7$*jE_}TH+g}0EMNuAgeroqTiUosr0#I#czB_A*{ov5v08+Zs zb8)~16K<@55MCq(taJF1USJkMEdsv6(^kk_t{fdT)s`pBNEj3J#2TxS?D~i)#w0(4 z8sq6za-B8yVp9Ke9%(6Gq>xLE=X32@|A%rg?pz=hocIJe^O9P>sPoN#qBP?$p*9>n zjb`Q{*v~I9bo`1YME^ZmpdoL+#T75O($Od`k)mO0_{^B|P;Rk85jbI6;M)BQj4@;> zG%-#8Dmn+qJ?3?=L-D{OOpabE@be-zS}rQnN_6**Dt%yik!NtzA+4hMBZZ0=c@=C?b;Q2YoO`jyc29L4)&`u|WYObbaMU|hJwHw2Y6ziLJsL8%)zD)S4S0F`tjn!bgZ3HCqC!$lS;Owf zc^o(-mOPvlg5B_^c>e_WF__>Bld%7wroP2#%SP;tD3PW-1n z@KQr0pgJUHG=f~>+)&GMTiE@k8RvO4i}#F+0iiPrc=HXDZrK?=D6GnP5a4hyPTYC% z%oE9B$vb>}3ZTymMl{ZIwnhcPd@0`1P>GxFopG%vLHK3%C;`jKCkh>YSN9p6wv<_< z^zLN9I^S zHZ6|zsjsq-m3+Jyagk5OPe@o2>fIQ1Lzm`n&a8NKcm7+GCOna}A}N=yt-Ut~+#Vd- z-Dz7i7{-&onZq%b&O^Zqcxx{QXGS?u-%gBp)Tp`tG8*olrVKgz7w#VQ?LFACyNzr9 z_x@r1*sFUd$l8&ljagWf$R7e1u5_KwzL~v>yFd_T?eA3+m4hSX#5EKMxV0(UZfu=R z>0k$E2acXJYGK`d#KMw;vn6bfsA`dgqJXTs$SCh#IC%9=3~3Vgfjw$zdXpOAUBg#% z1_ndB$cQmp&n=Ubfr~whedB54RGyWEVWfQ!$L(&6vyfcwK_9eNf_RPv=hbr!`w{?O2HF;|2=?e*VGqG?IX%g}6Kcpz3o#h0~WSeZ^Y`HcV;WD59vN zqqFI`xYfAiG{vU#%4~wDan(k%;|c z<#;HZ$%>kmh$eM0u_jtf6F0oHe#JlxW`Fp5!0R7(l7SrV&^$)tdR#oqN9Dc3*Uj${ z)!LL0TE#8m4+US-KTobGK!T!ijwoS;C^n29?xi?wpBUKKE!kW}xANJVO{!3E@TKgk z;T%g|u$T)fg(yQ|ZmS-(^#tTB?Phjm>1rR6)3tD)tMF20Z8M%?W*W?_Fy%iB`8N0t5Syv2vUXX>&{f?&v%so?x1 zi@a&TNGaQF=b&D8m$Y!H9=z#&S^<%g>xn24hvYo^e3ipoUM)sbWC?jaghSBp;_ksH z^Zwhzmis~3St<4U`t~3L&DI_R;zl>;=JRH<cTYFY_Tda`{eOOqfA^QA#F~%H(FhGvqqsQ9h*+aRV>Sq z8)9O^CA276}lDpq!$17h1m*0 zHcE2Lf=Lt*_I;BD$^BVe!FsvwcOJ;`Sy;WY5GYX^=gcjF8#m1D# zNEZbPcBV4UxTh+dc;Q;o(+7gyzAcqmqD(y}(N-yYjh8%33Ki@ysiIZ6GAB$B(M1R& zb5g_$CSq`fO!eNVEOX=7USo|H*tRDD?d^g*aHL`|$H#We`n>GONQcb$vqWS87XXBp zLG?t&z!viqogV8a?yokaydeza#f9FWJ?#3__$$SvJ-^AhO)*m!yPYttc1p3o2p!NJ zhLy@y#pa)8nQ`;YE@tp5s z5vhyKc@1!tuhygTdC`}QR>8G3_Ef?5!8}B%1iAgHGT_fp8P@Mzx>xZH>0gVd`<44w zQg+f#^XIje!X)Ij0&OkcICmY}5;D#xbGv;$zu0uiZ=-ecb(j?U0RUOYT_#G^8Cwjg z;Ze#%)qJ@)TF$otO3ftsHE}Fy)`iUaZ}5SzxU$+mn;dn!1Em>OYA5XZ;Cu5cXmvH) z-^2C&z?+8s!ibQm0Hu>=X{eeCHX?BN@&t9dM&tWl$Q3n2s*S%TZ^-owDUtMrL)OKe zomQcB4!TAWg(~*#C0If%+&z(0pjR^kB0Hd7t8iTnvRN2a9gbYHQ1H8*Y&i_k0fh;_ zn1E7XMxC}H)*FWj7-ho+k;iJVvTY9VjzCsSK&~*ARr|^5FRBU_`VVt~K#U5RK+Fbo zF_;!cm?rdwISK{%vcbM%9bX(P(gk!t(WKRZo3*7+*@-L=ECw;`NdhxrRG6p=7@f!} zigUm<4S>}2we3=KW6OpS*WR9%@Wcwo`7LWmC*o~a1Luq4c3?q?gz~`^F$|HVjlLB1 zr*}*5Be{1{AKfL(*=d5@Gu$nM98Se)Y>upperW+U0u)ef112CP8Jb#pTWOL7Wf zxfwnf%86p8naY%el=qMIq%H}NT+G?}ye(b(*yZD4Kco$|TVSzB(ripzZqWQBy`sA0 zuFO1z(gn|BFja@VBl!?Qg7--HnRR>qXr|YhS1+STbS$0N| z)sivPA0XM%lv10dGys}ujb&}cU%7#i8DK9+kU7b^|D(&SCstBf4+dzCP!184iaCt) z(`3Q_4A|{jD5`(f!_qY{>JQeSAN5DRij(rPJ_uAHtb{CeG3hS_fSY>9ew}1lF7Cf=?Q6X#JF<$ zaK}IR0tchQox!oK1z@$ULV@w8-m5FAV)+IjvKA;bhs*{UZrFIB0WA=dD(~0 zUsr@%n&=?!h+j`}&-oLK^7C!;mWYa?%vzV=GhWF5dOywiiP1P2Ur=7z_^>j>FUvA* zu2hs2(P&C@{U#iUi5s7qUru$4HFV^4Ba|a{L?v8sbAwu7WJd=ZgR@v4U+AE(*B@ZN zQyj2q>aLbWxICoq>|G5Bi^;d^71<>Jp@GIkSlp(}xn+H5!)c0ajL(OE&33uLgi_S8 z5jWcE4Va&`x!tXuWXcNw)t%9*wz()rF+cXFxpB!xsmci$-|RT)-tP=x%3ywE^@^^J zmI0%|DkhtB6%#sd8GWi6+qt4Xv0rk2wgza{Cf%BJF^rw93IkOr5`unQ6-|n~h87fgo{?z?%&EgXI5M*~8tueg z*oc3jcd9K;zHAU*Hpf64hVMR}UqP|nhMa$ip1@?U;N#EDgSU4v+5*1;q+mV6PWOvWFp^&4{#}&&|QcB7wue3-F@K*r(_9XjN%`YzyhVc*l)?D!(6w27ppOLECaX=sspGu zu9?@pWx#WY(sAa$KMS?Fy|{IFc1lNNxUP5RcJgk5F_KEEth(S{85_UCQf=73m9I&I zy3fQbwRF@q~oa zpxvo&fVcy*}pk44pRa`Zq0i)tZ z!FQH(17S(7+c%Kp#Nc;EAium{m@RaF6Phzb`#?2JzkOTtHQKOdd-tH|wj`W~Wb4o~ zu25KvH9>o%8kkas`>Ka%R5E*IYeqEBUwFZvSEfKoTnAOq$Te%up6$&+`;p5K zm;=p?h;m=0hXGB~{!;c^Z;P%Q-?mGy{9lZ{Q;?`pkfmF;ZQHhOow9A)<|*5@ZQHhO z+g11U%#E1niSC$w{9pdJ+_`t=TI|;9 zaf)f$Z+tuR`_GT>%14b_R(t>eL2Cd2y8n8c@gE85HU@^)CVDRR)^-NQdgcbU#{YuV z{~sHUDeYgU&6b2;9Y0XZ*f1sPao3$onc3Q#=!3C{6P!)*aak`}apCy5A+x`8#A9t2 z)8DtPod6PXAgG<=(^)6oOGp93hV^e(=rrzAHAA+!VwbWby4l&e^{qlxMl)oR3|BaA zG3&hYt^dCxFtpUW-RyEno(b<|o$5F%0u`Fgx8uYo1@eU4tL3mf7|t;LT@i`Y>IAqL z0BoRT9SlOUej?{9N%%4DUFk2uLjV!9Xq=r#!aHV6+h0RE4faj@Ge4F^pBPUTg|Y86 zQOLP}{%Ava#TG=55hc1t7N|HWd@N80JRp35VV3Yhn<8;RAQHd`V<*T*0o_k)>H+hR z-(!|`ot`M-_VY%D1YiPCV46o5JPB`vI}on{Z#mHkl31f6B*0&m(*$wLAmVFnEgJ7pb^rmnp^L_eZ zayjHbmD%Ai?y6mTf3GG&R$`Av!X#QnSgD=^%-4>iUlv3RWn%p z*i9QdFW4HNHS z#=G(5Yj^rr=P$=$p{$vHn`djtAGPwPY69-Uw_{5l$Cn4E%_nZ~OJ3a?85!=uBzEO& zYfBjx-oD>l`GVv3+1Y+DdjRs+JLjiiVL3egHDuN=8SR-oeVA2^4Y1VM3&%hCyqG=t zJtMbd)^VG1l^Vxk9f6$ry08~x1$;n$J2T{OY~$}MJ2&4^dR`247d__R2aVZo8t0vv zgb}snZ+D0hQHY+}iJ9geU8~OW{3|a3WbZx)t+gK>O?f&1Eh#L{Y}xT$x#rQ-(m^lB zzQz|>5Kkt_CS4txSoA_z`~VGu4Yj-vPw3a)V|kc&8_czM#uwdBOB>#LP}0w;l-BUy z(3zvcjZv#7cf;<42P_UV$qL~c6?1HA{(`Go{fPIuxgJVBjK>z<5k`+#P%Q|{mTI49 z`C&eC%E;BYe~=|`6Bf?GDn5Cw4G+MNo_LT;zoO%U;a_y!AA^Zlc-jo!a{ooioTDGg zUB<&&v*8vO7teMhcup$?fG-z(sZ>8A%rqf-Ud3@xx*u>f6oXrjdW54p96q-o5aQGR z!yk9~va~Dupj=T?3b6dZa+K>%1HcV-$rCt|X^oAKF2c=WJJt_1gyd|di4QY}(MNL3 z;Ls8%=5PNYsplZpLQKq{al^a=DU!37Vdusu?gZEZkvrToB#7|d0J#X7#VvAZ*O9e=Y@_@< zQsa+4y0tc5pW~eIE0Or2zkW!$v%xK0gzLdUpR22YfOspxjWRCSZ~ajQk>B zK2I9M5s>EPxN}DywD5NvRs)MX!sY5e*KZp;N%EyLkUE^ks^heaTHN}Dj2C; zY-!~BQSZoA!ngg8UG3TFxjWxCYP}QV&#|R<>>6v(sZKQApxfiSPrXea*w(!6nH2-3 zf2$2}Nkfo?`Vm(jv;VTE(sa-YHWXY!eP*OC>K1h-_eCwu|pujzb zLAm*FqRm_P#4za49yE6~QSRcNm{76rWXpt#AS9}5RMFhg(n@kONMx(2NlW-UTzU|bWxQN|$&2H+aE ztg3ShDbNU_s(Nh`PB>!cbnPzsc_xhgfIzLgdkq5Z{`pZb%oN)tTXzXH)v!13512m3$y)sU;zSZYmPjfpQsvJhb!t>GDIQ|gqe;{dvk>noTgCzBM$Emk7F4HJnipI(P(7F)?5BlK&n8p=gA>*Ky8e#9p~G}5^79!X z0#}iykRM|6xt#lxER3OWxOOa(YT4;(ZWB2m2dNLuS`loOM-S0*q>$R55p%bW=bZYK z*;*Doa1+W=wVnJ>T{jg!375~-3_?-|N#%%=P{dBMD&O*S@tlE;&vgRXRuXIDLz9x~ zff5D{sS*7=0Rqc~7cIRs+v3W&CoVWLnsm1~!7i@B4LhTbvSC zWy~gVEZ5%r^4B0(Th3?wdoU{`aB_(HPXQZ?Wknhd^&gE1Vc-sPp+9g)DH_P~n^im) zl`jh4k6G>@aP6FEF_rdfA7#3F%W|T=hO$bUc?YQL+bR?Hde%c9)`9une=V z9wjQs`}@7WqtAQn_F@vr;D4651okJfxXRnE{5KL)HnCQ@dFEi73*x(&YkRP<%#eha zNO-eJM%tF|@sT9W{kgQ<`*TfU1-dF~6EoS^Eq=Jk#!j zEi3I|9wRL)fpGk*?XA0PA+^(#R(2U0w?=KUDp2q+?A0B`#JkBfw5eWMPmF)>9GqS$ zxxXq#bLMKn1PX#vM^LG0tKWi3PJLczF=Fl3D}M)JWBP!oYFc7}YcrKvZSY?33Bl;{tBbI(D@idb7V3;M-53mV974HGK_Gh%#&$Ao`XH2DJaV z42sL<=l%ISf~>GWv8lriwpH3ac@K5ZmM}*!pOawtT6j5Oc$ABZ9(N%BoLjl;Hvo0O z8yH1r^$sv&Da8j$FT}BWa?kOexI^V>#=Uzgi3Qum*L|z+o5s z{J^PvY%HxxTuy%WvzU5UHBLegLZ!FKnXsoA6rZe0Np4z3t;!&2TkO~o>$@!foUpaH?+f{Kz%pqmxYuSkr6-^?WD6)I~LUUA@?njScv<$%sQDZI}^ zkWySj@!os%+>OEIqRb>DT8dnCPgL`y%9%mPfbO!len2LWTsiL8wNISI^w6HV)KAE` zuZ?@Q@^d9w_fFL%3us51x6m0;hfLfW+aar`EE+^5K!v0cZz2z`3iZI$vzaM|-12E| z;&q8w7L^0TgqkEM2e|>luKxs|K9j*LK*}NioqFZI<5#8xtsEgoV`MtDlwLS*ZFcTE z>~ZF00G%nj56?PjcQS zJl;XR5AH^lsz@Y;?$n3)8I8%5@X|s>u0G{2;P&0xdKj@rQuqu zq2l}V8l)1}wwLFE(D=N*kb0q99raFYcdHIdT*!aTfL4Ivh7+lKL>yC;%a-tOxv+rf zZ7wff3w%uNB^&guv^E^eu4ESSVz?rb4=c|eMWTXwU$=_X)k9LRgFbB+i=mjRm-X-& zJtt@v>8w5~upK4jv|*+@Dx~q(EL?z;q}Ki@AJ4-FyF^(C-%r7B{cG*?%C3L0E?fUW z8rtc2JKVn_fw%EFvL+Z~LuV*aKCw^DxVFySU->vOSByx_0_ja&19}UK&>-XEli8lU zhyQc))nbHhn(4Y;p<3z zoie#4KcqSPLo6%^>)n7QonAM#_O|2@RHZ~i+|+@2&AZrS>Fxasx;wX)EE^GxHn94G zss1e5@_dg&m67DlhXhnXEX}2SOI>cGHS)adVJH5ISu|zu94zl2W7g{#p%9M;b+hy! zFQ3`+`E$Uf4K*c=T5BDrAT_q$lU`{R1_wHyH>9Fc3j6!AysUF?$OE7%vjsPM6RreI zNopdly^3W0uFNW*nZHx&;zWOh9yuMK+7a4_CE!SDnf<`8b^nzh1WoUm_xNyLqpl@{ zQikHnvzLdDj_NCe%N%NY;>()3ca+&rN0R!eE@lr3q~od8{Fi4}y>uJSb9531{zq*C zV9ry6v}rldc>NAJ31waWN@bA}A_ZC{$<=}$o!jJDaIdt6h{aunbrB@gGYWrwy`K^6 zw7KSat#0L~iYs-5zQ5seB}eLHJ}_-H?c9#*VFZwwl0WeZtAQOzM@+I5vt_$^7WMC* zR`G}Ob-1zFM&XdLcrS%_1AIhGW|neAgrZh2EfdVHp~r)N0}BG&fe8Vsm-^9@#pe3n z)zj9ZZlj~aVYd|1m1s;eipiQIUtX0EE%d5qN8m7MvGrw~CSDfG_&%^1wX87@Bse)C z=t!=PDn!97?lSUBl9P>CjS*ocBlj0@YxT)p#uF)1nVP!VT|q+9s0IQe$MHE2^+1=1 ziz6Li5;IX5YyU~{p|)fME^tD~b_{0WP%HsNXRKFQ4MfTkT_d1Cq*jnER0_U zW~Ld}Ym23>jJ7eX`tl%@LA6!v_XHNt&&>faTAc*cdKu#=Oq6P$JouqdJ~eKKA+wQ z7#2V(MQ^(`__UFjZ~Mfc0L!pTwBNBi$n)ew5o!7V^}qRicgQO|_EowosuyKE~hTlfG$C=YF=H2vD! z4c4PLd48x?J(FK$CDqJE=PdnZYVywczgXl)0vtxZZk4ZVh+Sz)BU0KF=hbg%Lhj|K&L8$=(b(!70R4FK!Q5PRN zF8sy{)O@3izDr;c^GC;L|+#L2e3eySd)R8Pd5`j zp*3wlJG|idI7fFUtofHC9ax1G2^Dk?*7xkAmCIUmRC}AV<+jM0a7AQBDZZBd&dqKO zzWuF=kh{OhsCyl4dL1t}0f09XlBuo80Zd_ft= z8c;1|X=MnXIl-?8`M(2@1?Aad#%gf%MGMcd>$?T1!%%5=vs{~02ux)P<>d|)Et(5m zjU}{{tT;*^Q01c}p0{RRD!9Y3%08<`S`5W{zt`Qg%=R#Cz$!UI(9V5ZKT?R&2dV%b z&>1zq!?va=^9y-$EzkI?NwG2s^Qsngp7T_99&53o-GG0yHSlgk3`b4_*IBQM_NAqb zRV1Nlpx}mlT&Ap`Ixe&k4lh!YAe6&T301F3?kd{xrLAgW0+h2ZdmOrw#q#~X1;d+enZ7R zCb5p53%JrR$A?VS*6>)yd9xexla=z>%hazs3!8u1cyRY}91QCi6Oh5#dP_5Z*mbl! zN9VhgCepW1f%smfi!j`>PisuWWXn}C4oy=n+g;L9ns!mNnK~_=A zx0(20?~DUdVp2XR5DJkr8$2__HoB8zrY?%H#T@C3pzCw=Wxpe)d5YqQB9&^9>|tL2 z;0ED5WSU`F9N9|SiyO!i5k=^VK{20G%zvfs9}HW-_L3@ zH+lZRXm1U;mV20O=55|(Js*{4#>%nFoH4X6k8iCJCY{Lh zWAWs_Zh5TEp?C7WNA^IyQLNy{lN`0#MfZN}{V(vU4uLJ;!~R1nVEmF6bsn38<6o9| zzJFQBab0qaJjg!?E*APDxUxui|K?%Jg44Objk3gdU}+D2xeB;#>Ztg?v{lz}^PEb- zVzgIXIfK7qH&xwhyK@|pT?<;PH6pVxw-2#E!}elVa3q`YudYs`+!+3oER`;x;!#7+ zB-WOc*2^NI3aG#lJBGv+gHvtZB#}EQl(7)7Zqbm62lZ{<)R2*T0t$<+0$TYjY>L@}no7r7Y6K$`A zfVvyVM>CoxSaWZw(aJ&A`-mX{C>5z+C_$B3NGu?1Nm2e{zm>a^ZL%(hC$nFbDRn7& zh!5|6rN--$qR?o>$!9UXUMp&35?1us*@@VG#H{WH;?P*|PK6~TP|7V9_krU+5<_x$ z?d<4ez-}y_Qq3jJGov3n#?v*pY4>ZU#Q+V{>EPFCE~$j6{gJE>s>A*$Z8K=9`sSSA zj}kAeO`O-u_1VTqwGMvI#d47kEH9oXd1j!gALISV0t$6&hnsDLQyLd+oJ-RjlAELsQxONQ z&y1-`qFrV!!q%1rL(L!k)IiUVfy+pvn5i^fuRNkwXyLmY8w!2cZOiFI+`#hMXc^#P z=U^?BMODs5iT(J=eHET7U@WkYYlYjPn$mc53?(88U~d#IX=%wCXi$UbuI2;=Tc3r~ z9<;VCl~Vcb8O6R^htAF96!eRoow*(HgWRS4_^GxL5NYxle=lUxvyT~}SSI7w_zul+WtdSG@264l$sgdsW1+8QGg!X{ zyScnkcF-z_?~G5azPzp``(^`Buwl@Qu<2<>EjtRNmWrv7hc6qyA`MI@?tWR^!wuNf z_o)k)SU(rJ$i{-llPenbbH$`bwU*F;@{)Tnd5*M{bRDhjWEz(`(C2Ln`eM~eC#DfC zfyS=~o7-pYw>GhOq)eom^J&)k@$GF74n9Dl;osM@Bsuqk$p4!ZWz=dq@zq~PlpRHz7Gz7ULc({|s_ zOoxF(S9T#HyxFf4erDGj{VLa6%^G&JpP#2ya(6{b9q2t-#;fh;8-2PbRV-o^;DtUh z^Lo#)<{4~iDR?SckTR&hvNfv%z?B4cI|k%U{Q%7k?bgPPp*WJDWJG$Mf^mw;yhoaO zUXbmK6>*$M`O3q-)L5BX6YArd818l00r%fck>~s!BE@4?aj*{{IIVjG*J9VS<_1+M zpo%=sR2M!GTyc)hWyh+n8W4M2`VgJ$>fjd7`jt`|^S$=N*DsZt8PVXyEPg~#$W@YSc1^E(Hr;-*71vL>wioKQZRrBu(CAiHiL}fOqlfG+Z*Dj)KRhG5bGBn!C<#ir( zr@aMr#)Pj@%OvHh;FfOr8A?hvCym&K|Atvo-v~8zt{c$}W$nvC$m`kQ+cxR4F``u> ztNe;Yke}Il$|aIv_3l_Ca7Lv~(w)1occZKY4Vd4ScSD!ov7JrL40QoY*vhtI~5A6}}&Omb*nf@|EyhoXZ zPaCE-1UaN2*^VUKqon>Y!(r^uw^v_#SFksQ z&HYz7C`>1%u)i;eY!}Mg!#e0JUggxCZGLeh^kNPImbgP=*&-KB__GJOP4MXwR--^8 zWmRk<()8m}=WN-ql)D=}p{dS&Ii{(RKgZrigZ6B4X<0&`Ykb90I5)Bc@y6(pdFq@e zUgVGO(0lx<*F7}cOF69}NDzb`*y?vj)q;UsXF(6lald##jajRK*p{az)sxw9?CC(2 z=1*%iH~M@^n{^o8B8459U&fIvnKJX6V{ER_7S{8v?C&MO&Hg^pS#}V&YbYPyDIFORFa316u%eX*`F&7bhX##M2Wu>uBTZ=l`OvqW}N^Kmcg+ z)Cp)^t~ZDND|F8PrqKUqiks+U#s6LC?<}Q_y2xBi(c0a%MxHSe<8*a30%^|*w{`@yLKQMQf^l8vqOLt>U z##2HzHeP;Sot!XXQg*gptKC-W_T8u>5Tt?_W$h=IZfoXbwDiDbET?%?6qY6IJOGvc z<8J2dY-xEbDUg&$U>FHdOZ$d4(FBE5on{$E?vPU$DiEO}6e{(bjuz=IsMPcJz&=_| zQ@s-%S(O3w%8o1kabt(}+JW75%^MnXTk5n>;*bnw=B}g{0uzjRh+6Y14gk9ZhjrXR zd|4|1EfpxqpT|4eom%VzOMpoO-hwD->bEp>7_AES=S=gn)eHLWiyVIizIPra3u7IU zHCHtw^MpISpM9!ZQe)~gmgS^rrhHWY6@SkOe^;IDNQhd6LjqQ_+cjMhO~|9^%{SQz zM*qqM`0EH^YT2E680OjBv}O^VF)+)#$w_aSLQ-R{*?x+_=I@FIadvu3f+d~8e@ zI3hEU2GexzKoY>TemvRkS+L9=)s($AY9I2n**Ft;l*Ay1U#KyZ!556*qB|16dyj-< z3L&LrzzjMfDTnl-LsQn=8zLS9vz%l2S=LGTG_qbpkYWPoR#wp=Cfp!6R(t6D7pFhv zv=PZ5NrHnpiwN2Hm_U z6g)_BFgzls{`?3B)H#p8BnLcxb_NPkl)Mi{bQcl`?{s*ssCRg-gIwYoB|f=B&H;mf zk;M=5XM`z31Z2wdm4oA&kDs}*&ZP(P89VG)%10pi8e7W(;Z%anW@N_-i-b-k%33RcdeQwdIP~ z)*>hYY7;f3T0m z>&%^U#V7Ll+L@$fZ=EMm;1joT=qDOgQ1lbR+< zq~{zzubsrC8d4c_*bF}PB0G~#x2Nd;0(|96&L{dR3k!D()Wk>VhNZXiS^=U z$+B7jXo`W0pmU5TX#_h(It(boly@9R=Wr<6G|e&s%Mo{gijUU0n5Whf#l;ju38tIi zC@{xWT*JWVlC=+YFe8>sQMr;MS&(^LcO$g#FobuP-3~~h7?rL&F5aGNbqd)Iv|88F zlTk>WT2F4UD4n)R=RI3goNEpYPd#)?7csZ6gH7)Z%hYowX|-ia@yXe`w6U+5teM(_ zgQ9C2c^ooXmjq z8B!SjDaM!^HS#=qqG%+ILB-5r7mKATn__i^XUQ?b)*8cjlP%p~QyxKrrma}n(2*!X z9#F{t{h|yd&cR8>iw{&JHaPFTkN>JF0n%1U}>O6NTabH)c(~is{<+(HB z2l|~|gMSqdb9R1Y?csgW;U2(I;#tWmH%5KagMXr5gCn326*E}s3kV*pvuraNky{S+ zdS2xmjC!ef7rN=F*)}-U?!6hUhHT`oL8>Fq^VvtLzdCJ`i!-CNutzQ%A+`(o!+rv@ zd1PvOx@%GSN8S{_c4RRZj`)BJS2Ezm@30fo2I-jm+|s*MDls8vI$S7qsY!=4xoB|E zt}V6rqgTEa<3pX*Nv?cv(+cR~&)=%bdol8u$Q?rtrV^rL3WZLJ{6vey zVw4tUlvFkY?~JQ;++r#w7O=BN_soF%4LHmDgA5yPAhcMUMRzftz8tJeC#`GZ^R-*1N>XTA3!G#_XIysL7ycl$;=(?VLq{OJe6C+`ZMO?Fhrlo-* zP8s&sC#-3a1=JFQNn(0t3NE#$xHx@A72i|-Hskf_{jf9^|L#pB{Y6$jb|FT|^P)Sq zXKNRqZj##w{xx#*cbRPHR1EHPd^YAE6ZYZ^3CYN*wM}$cb-mD4E;GKVP**59uNF$h|~TBMhkVRVmh!Cl2Hf@VEanZ+O=Em!u}ByYvF*qB*3P zr7Sc4@>wzb-)y|3Lbo2ZvflC2E3;U1x-Ro2466Cr z?)sx81`ssZ-}+OvSVZKc5`=2-dD5CP-;3#c8*zxr%xee()Jn~uyMhz=<+nQIy==_D z;{Gh5t!PXwTuHu)7vF`U`L!(qejOeh_$W(ix#Y;gwy>{J#i0+P)QcVUzyLUA+ zwl{mc#XJX$#CpZ2-5B~rWjhX);SOIzQgX%l7Mh+2RK0oF%DNuOGs&e)Lp5Xs(#cCR zs+FM!vk$5V)RCvpmLnC=k^+)=n?L>a)DI>dS`lCIcnaz0ti?pgtmc0?Nm^-;WF?4+oDsfc6?XCNZ-*m!P&vv)cx z1!F;vCh@DJDUevym;B4}1qYDzO3H*mE*i#?5)c}BCE+Y(Q|}?QkZ`yeO8-(_M9-V_}P=-YRK6erT z00{q^lKdZEOs{#Z?1_J=6_zbR9ZT><*|g;reMsQu+2NQYf>)_I(O@&?b;u%*LjS|S13R4eo|j=_xOA3 zw78p2u8?1KW`x<`2an&^*JXTdJbtdO&SW)q$u33E+*BtSpw{$JgAEZ`G*+86#qIJ| zyYJ9l5!P}hul%P&=trp89^0}1N+!zs5T>!4tYm#M&l~5Dwsjgx?}N|vV~1_I+PIBy z2sG$>TVu}vD(uM)lHWj^^BM3!@JyZn#LMzfYeKLe5yo&d<0952+SBFz-U~c)@)ZY= z1#(Zh0_Ny{AbK?nwxfqkp$ndEq6t|D5l=>t`GX70!{T@uR4O6>Ad(5xDFQqSXlGjQ z%U)6b=*=Ca%&*l4c_hkS?cw?d0*)v`zyo>fL!$q4hs^Ngf(|$!{Dpm6%tFQ~hmC0x zSqKG?NQe~k%kYYPit<-q9CI_!7wjg(Lx>&u5>_Yt37js?38LWKK~C6w*ougDf}V+p z5H{~0-6_nAkR-J4MzBw5BM?u4EL<+spD{o(%0D9~1Asi6j!u6M7e@$WS?h}`K3&XP z$CQ@D^1Z3+-+sQ6?%Xxl2JF1u{od%l(5^px0qDVJUr)Wcd)yw36O8ojWdF+E)@gLT zb$gk$y@eWjYW?&zV*rqF^I7Tc?mc(iTeZD?IN7h*{2byFn%&us+jh3=6b9h#DkMa$ zvo-VIv*rGtoV{!QuJ;N8xrFnxYq-3r1CLv01+Z=WboE>7_HMIyUH=yT{jr?`B&h*# zuHEds8I;K0l(Sn4{O1i>c}ML5$m>6S`R5H;^YzzwS|5eeU-)rfjMq27N@c&ku}Rcv zw%g1_$mHt`dhxMb=_Zr`m`k0-{Xp$&-QPW~^I8?2qQCVV1lix(1rK64V+&oiqf-X{oIjPfUY`!<-9MCiC^vID?M!0O1H))E(g0a~9L z(LGD|YyAO01gq8p`W>EM0%t7AwdpYjoj2_NUJj&BzhI2(-FhgVM=4rCEo$h{_?+DH zBi^s2J0uquw=1(D7(g^{+l!1poHiZcx9F3IUMwV@1auncxZW&Ul*F4kf1gJTlK<@a zq&CNo@#x1gM%n#6-2VKv9UDbvG$}+}tFvd}0?f_{eBc8mw$&hl5YrpBG?u6H%%MKI z4XE=Lpz)x|MJ!tyEoq6TrwjEBE&(;I+@tZi-JJOHO5&LUPN3~GeLtrY0Dd0af*yb- zk&C2%%gupy2P#I72GU1nj4?&O&1~`qDGbIhq-+(MrO%#82z!hi(Eu*1@S8~NYL1Bm z^@rvQajscqE$JX>BU0_|fuzm=o0s)m4GOw4my$|dY$uZvc>W_S=X38De;qj+oL~vN!lOiemhg$Q4K!{30h)5o6wo(YcsGjv zFK&&nDb$4-Ct$=6l+y-A#9V6P{@&pq^<4sE@Go>F3$2@zdLE8?tNwC6C=yEM3JnH- zUol(?ReXnH{g5quto6~K{R4mH)`vmet!@|7h#ff_H!nDVvP+Q@w?=lK zhos_7L2>M_`G?1jp)E?}{%oIyV0^XM^C6FZA?JsXEY^lpA*WkNcO+{6eG}gYA>zGG za~^o*t$Viveql4OYrP^7ZZiG*;lF==jwJw53_xxhL=cDZgxUx9tj@0c2awp@?#OSQ zd^a*hstx#1QVLXXL7dqdgMw595u4VBvAI`r`%hN#i%)-Vw6*ibxmSelwm(9gG#TJv zE(V#sJ;eQ}J|&HpWh5l*@R34wVcV-mn)WCOv{IOJ;tfjlVIgn7K)+LV zG9HfimOvmhfER=|)i4s`l0Ay$-jMXDvQ$*sk$o}>mGB>aABf;5AIQ$qc6-3e^lKq? zfD}*+EKUUEyykqc7OhYU_uV~@Uf>-n=KePp6h18v#Y4pTrr>XgvqyEQ2?Ab&U{Ycv zULN09;?;q-vSAE$`G}^cuSo!2M{=YM#NJ}aAij8yIl({HrvTGT41OYl7>9ZSrlek{ z0L)W~oW7c|3k<~AKJ!n)>EVudx zKaZ&v8M1)0H6(LCdeL6y_VTmXzo9!~UdT~MeL@fkYD-#d(r!y6wCA7Ik@6~b%WR!@ ziLBDJFR)GU##w=4T}{<`G)k2n7rq(n=0jK}lV^6CN-c~=KAeDZ#Kf!f;D1)43{BXL zX3g8O*2pB1G$0y5!Dio5RmnU{qSCJAc>M?$A^Nu>vm{ZMmC&f87y9*aVOC;bQkI7B zB&>t7u@k;lD_aC|gINS3z9lnL!Iz%I4zh;Pz=#A1{L14gA!KD6bd>Dk9#iX3YTOfM zBelz^@TCHY%w&#Vi{?4owF=7jXDH8>lhYh&L(ITVG9^Xh(@RxgG6})_mi2Rr3^V-1 zjJW`L=}Z(MjcUr|r!a`ZNd|yeafrFXGZ}kjFwKzQPe==-W@!2L4aR7vvgvq7TdWbg zhAain%|+a;_?8G8izF%23y+JWkOi>B$K9H*4q}UlP%CeM`(cloRD3gy|HSW^q;4va zO3jwSIJc-!Q9(wgx?<{7J^EE~x|uPN%Tt2_O-$=7wnFi~^Ms{I9%O`@C<-BP$4N}( z6^*N#e|jBI0*1WsZ&UtL>F9uk)5L`a&{~y+B`qpF?a*2rzn2|hls{7>b`<KM7P_j-C0C6hcCdg;N?U9{kucSZ9?` z<2!7)6yXkM7ttI{Q3HijA9{vYRdp#v@EViDtw-c?aU-6J-zFc}W|PUn7{oOTJgZGU zJeAo+jt(wJNEhC;rkYgE6cqi@W<@7uwtB!RkJAXs)g^QGJz0(jiC=tDc31s zS}Y?xNQUvn>JO8?%?8!P52Y%MLO4)5I_yADS(KWR3(&MSVu`Y1*yIBc0qjSz4&6}Fj(-%2^5Y}e`(iOB?xO$U) zNciRow8q4dlnQp_G`FhFLPE=>(%}qxXn`mpYjg!t&OC+M=%5*=Lxq~r)XF1ESt=Ft zim**5)gjhn$=^ULkaqGMrG~7Fgpg+;(2*!1{nUXmnoM!FC8Jr3BlP)XVW@Wm+w$3- z!RqIdqvcRd`egnQODjWXRZzfHU#sfc{-ZAnmlar`{%$o5qx728<|^s}ZhHGK^M^L^ z^>U4kH36h3m_g=XgUyZ+coL}06_ytgSVC+3ft< zM!XjjNhY}dLAy<>-X}x4-N3SJ7K>wOVI@ODjxStZOXdDI@}e} z4tmi*69_i+N+)!DulUceDig(ubA7vp+@S9F@$2X&P;;mjHa5m^Eks9%`1bRevI#Ro zX}0q2pVzM-jZiIOwkHY`Y^*G!7rS92|6YU&x>$v@cIX+v;O=UX57p0P{wjyZ%5~v zyb++?+=40^UZa1E1Rnca98J9_Z6*xmij^vOhPjHXilaPZgRLsGXHd5a@@&H_(r3p# z3;JrMPL?rQB@erPo6sY5Nz1(ug(y*rh020b0U5wj-=+7%`dpuYs0-5sTJSaIbVnIc z`zN_hOH=$W6~v>bX@FNf8E2`HUxf3VA(GhOk{)t<`)j&GA+A89$wKWXh%&(H&%`FL zO6^mvpd@z4*`v!Fir-FKe%y<>b@h41>0=F!z8|CXek`ZU3pCQ#Mq?L$ga8F~8*6c_ z%n^TKM+g4}DtVemc$(Ze=uZC)QX4%?R1)ovM6jwD0~Q>3PWWE=82$XER1RNC-UozY z{6JxhI=xZY?o=Z1p;{JV6VNEk6rEH#yuB}gJ%59E8$O5{yOC5?w4^d_9+^O+ZKmEqDWM8Vn5S>r1-0E{R zn~$(z8?f^q_Kq*|RPv>gc|?Z?fa$H`Cs>Gvh7v{SY9i5OdNi>*9&9*5y8+He#`8!? zoR`-}S?wQ)*>zu#1QUU^@E+#nrM0WFY|k7mrb4y?a)bFpO!LWe{svqWay$Ie?4fgg zCk~cS5!5L85VPohybx#nb)jdYa%N17dSa$^*5;h zsi?W=R)EOei8fKx88uIv3cZvxM1BXOxjT1OWCYD3TT>RVVotTJL|58&v)zSUeINt1 z`cW;Z7{IEdh7YQS>RGE7lzrTDSbM}hb%B~^|l`u|b(PSK&n-L_|TY}?6>ZQHhOJK3>q+qP}nwr!i8 z+ubkcj&ttU{a&L+J=DKytu^PGzt_XTa^Vita(Ka!e@%^dUW}Bn8or@j+HL&&e#~|? zfXb7l!0ZC27%%Wj)cP|mi$P_h^Y?;G`kOqGfqik~Xsh5Eb zXUnK5vfMsnlS0FicIH#*3Y-KB)(WrYU9h*}tD$)G4fI1=i6T{*cGvrOD)FGx8_%A` z&0P6!D1)nhr}PptJbyXxL7zf-8#(o}0GZay$hG>Rz4N?Mf&~^Yo(Zn&*dw( zhoW(>Ao}2onu}A~y)hRtr;=2^64r~I-_4?mqwd~iQUab)=iFcBHD>~ru&WcvPiq~j zdW?_IdwY4h2Ac#x4cGdCbD?ql144;UaZtfOKcS(7a+sQw(*BXTA575|oO>v4gcZyw z5uNKr%qcXLx`mm%+aLkFMB}`d0IR?SlDle3&2YJk3ZYgNv~+s*+jDY^ju0k}6iNDBO*%|n7~K}B;@u%eB^UZIt(0#Nr3qp z;QOS8x_Y0E-w3Lz@yH$dEqb2ns9c*V@G0-|2`K$HG#E>{<+Ry$EEr7aJ~aNj|Cup6 zV4K@^QStGcTv%D{vJz2Ojz9(B&%}u8`#kI$kT%3AMn3WYZF(AVL_{EtxxOroZ2gh) zv4V#p_FWsy!i{GG@YOcE`6CwyOHV&3k}bV6Z4h{v>4R{Eoj8ZF;s#uQvPl5}{|K4= zpU};XT;jI%eOdnhdmyk;-#L_)&d$VUV!H;xABmCeKL*{u}G{{Fw^&O{ld`zilAUOa!b@&9S~_!~Q8t?%Gwt#9`qqGqBJ z`)m$a;Dc|x!}%Rn(Pud}aDfhUSDmDklA*5K5JBSC7j1Vi#={Pr&`R}t z@lSGWzcJ#wgWIlSL~ty?Scj0kzn&*p)hJaLZ^@)c0|VCEzOSF3>1a&Xv293Kc*UT6 zVS>z}yRfK?L@$gifp)d&Gqeo*MbcG%0d8R90@P8Pczb$6Cq@{2#)ZJz{PIwav47ZL z@;L=)qlSRHTm=4=bNccr_!Ws?SOBYQx2ftAjZ4BWp9??`36w?TC1oK+Q>c{3q1tFs z)BjL#9%)H?^h;#rFVOf`E»)QsW zfcN`nQAsJ!*FDR)2=OYg!r?2sR29{^J|h`JPgld|IKE>TlCYZE_Y6+5hg(IWflsXJC%Xq3pmvVy z2NERR6C4zwYNn47=!Vrps2&h?ND%OUUz~6il8JrjH1I|`Bk(?9mo;GEk=^GSGWPW< zdh@tvoKXYNA7{kuV~}13&Lc38*{4s%XowR8>gEDVoORErYsdkC`dc_xNdHyf$&!W) z)2%WZ+>CPEiuxrTR%7MPng-x4q26Z`BD;O%%=!vU;mPFsoz)TZk5nF{1a8R05d2y; z{RJPv6fN}(BPL~vZCUI+fi3A&NHS)e-SF(lIqUJ}Td)n6I|rej(g*mJCY}EqIG2Lf z|HI?;zJ<%1U39x+Ey^F^0XKPSL<#RB@pAIc;UMaD1mlC$0cHK^Z2_w3vm~}FfHa)W zu<=c|c7~+tb2F~M?c0UN74Zk~zbbCkHe%U!zh{*X?Y~#tEF5iZeuJXyY;7Ek|HElr zQdPIz5P|nzsY&I4E0z;+en;m6F-waMJEs%lkkLa2l&&IL?yT(yuljzUrV@8_Q=y6nwq<9AKX$#IV^@>|MZ*txaP1admDck1mL`od&&e zUzMtL(TBfrY(v_;3Ni|#>k}WC+r?qnq{|i(5K@PX^QAtxw1>mOEsU>{Qga-}*A#4p zc@0j(X9S<;K8%Di8pSn+FOqNf1BeqqmTq*2;3L$WIPqm9(&fh&LKXQ#*iQlF0wHOP zFTT$ao(3RjgK?kZN<1?r2cc1iA51391-gX9CF+oIMPx@p6eA!&A4eAHLb(VePvOrd zk*@&xhb#x&e+!~9z=@e)Ol2;;JB@*6hkH0~HN7@`&?8XbPYh=moIkEWG9u}^W3oLw zRWt;#z%Lu4DLOou=;%-*j4!Ao@$tkunTcA!w917`xKGNyioq#6{*4rhcnkiGBSO50 z&4a`xM7=$Hl;utyWKJ}Em;PfoQD69TK`MVbhz2>X6V~aCsQC}(-KfM*9SUJy56~2Y zj5~PP8oWN;eI#X`bwq^JeU0@ks_J*vFC4?w?tq4eGv%2$Qw=v;R;tzU^J1t9PHa!Z z@rRPCx;Nb?h&S_%xcDpuYvq|sNTYRk(xx~%s%6@fp{vX3IA<4-{R9fHczW^le8@UjBT=h9h=Z!$9E$u6XB};j zqZ?&!fx>pMV}unGIMJM`erjv1BF~R2iPI*dLqUNei=?-4V0sndJ zB0ak})3Aw-`&+?o_{$l@&`?8081bNg>W-kO!z6-f)glVYW|fOAc7!MrZ=oAQE@~t< zDZ5z&HC*vwELCGF=BJ`747CvTAggA*DpZHa5_6yX|DqC0vb9M5_!xLvWx7L0n6D1t z?VZad5L3AsVNYBqUdw)<-*ZpzGrdQkSZIT!x}@ul4$BYJI?qO*0kt2A72{w~Cw8XlB)D7D~YC7ySbsV+2$Ep(b8ROlha zy^GG2!z3)m=^$N%T{VH{F}w0h{^??2K9t&D&&R!vD`hc>(i~9ag`?~^{8|t^=w4f= z`#ZsKYHkyM(~*+SQ^XG}(Cj3(COPT=kxbRXneh<$^7mLRwg!$X6xgzEDO#v({N{_h z&ca{{ojE93pNgSUymZ~fTHL%8@*&Rga<*1^O!?nuF;Tctrc&M_R4HRxXdFjZ?$<_W zLiXs8*)}qP6Lk|}4YEzd`59%EY=_(wkF5(MKXHD(XH+)f;2WTQNc*FEB^a|wnf!+x zc+Y?vF<_qW9Oe^u{!Z3Ar9ef9;6(az5Zn_>)MeG6QoaVe zGhecbZOD-_4BsAHg|sfM>UaTOSKMjDNS6i|?j&BfNo;IRvx!-WU__AfyQ$;1509Vs za3H%)_UA;i8@{U-Qt7JlcnKTkW^9<^ZpA zlTF!RaG`gM0M60&t&5r_eK=7|3>w;*0<3& zHU9tMMQNC3TMhg5q9_3a01*DqH~jx-N70U#wZ)`|AG+}jM`$COClTUSvakV_s%`g) zT?srkbB4LFQ2)Due6-;*97|Fb$Y=PwsL}OHsXLCg-M;PsZ<)gJ^zi)jj?v}Kt*txT zRsZ0>0N0KjDH${JA;-(ykriPrz*58TsU=Wj^@W+mux+-jeQ^&hp3*ys`qL6nyx?ui z5DNQv3o3d51lwx8=v+I1%M_2f$lp~bC}-Ed*6CuTi`|ag;^$jw`!<`i8uzH# z^M5HWXEk3&hzwJB2X+Z9y3O*?*uA+PpD3I#rLCrzc^3#F9+q=5-R2PpzpZ{+8!V@&>g@lu*{IsFYZ{Cd6PEM_x=-m5 zD1{lR;ivZf!uw1vGR4)Zj>^l~!&CpZBOeLuljz(kaPOLM0e$;}_@5I`JT%5u%>@OL zixOLjMd&zg)akndx7n@A`TljkTG#jBZ^NtOF7 zV97r~{}O$SYh#t;`EWZiNrq!dF!ZAm9TNWgvMSTAJ z*Yc0@55sHt?;%1Y_&>hnhSo-oX2w=l|DggLQvY=^vmt(8>+%H+ENoY7Q#y!7z##eGMbhyv zH|D}|)>Gt5%^Qg&gA@hQig!kbqu?Npi6>r($w2*q{^S>-GF%H6!Y2pij8&=TkKL7Q zi-Lqaga`}VrT?u1#a*>*%t-!*VWvYu-_fn z&~sq#?tvu18}-=DltMGY-UNk$??|Oc5>eZok>9N1?dbFynfuU-`~!uW6MV>_CdJ3l z2HN-6fi<1lf8yiLOyv67)McR!PDIP^RGh9Y+mGaDIfsgC7WyXi2aL-%_if6IO9fK& zWp^mD^QA9-?xU_|$R8`^K;Dj%NVQ;N4y@5cq?nIa(Wp;N9nha=`}LeU6tB-GB|?vb ze}c9oVF(&GWOvm{-rfiL_&YYzqz~KBjyq!;mRAei|}DHcMaHW`2116J_vT>4j`{K9P?M zs1M0{n5G{7Vk498tAV<&b3xrq6)A&FFBT|fDAhs$=HigDlk%&AEo6oAY*uB47L8&n`??2p6foeHC zCY~8Vtt9U_QpTv(zOB1j&-~45A#W^gskf2-5Xs;u9vZZ5bf8RVSnYnrcbi1Yn0`WE z8(NT#dVa)MJhm)&UotMZHYTZPmsmtQ66bsyeljXI9N3=Uy0BPvN5T&J`F5SA*xk`f zByA+ioG(mWDfeU$Lo#CR5Y}dca2OU6G@V@WmNDKl1V{qcw4hPoczE!{rO?XblPXDr zbSH9<&H>{Q^5k!1;@6KCUiPu2i~z*wT7L80{r|woL6ShOcQekXoTEZ#Z+?Wgvsj(M zb+TKUE06ptCVJ)m!2Wsk#~G<5Z(O1C{KG2nkS}s(AAzJ52)ODh-7~_}QnFL<*X9)ExkCc!lyMqn>)q6{nC6x81bGRPA4=+nc@2APsWGQWu(K0ODbZ599_^2}Hh3(o_Nd8Yu0+zA)87~J%xwoue zju7U|v&o;-(Jp=lxXXPppJOejZL-Q8tbM!s>dYN`jmXZ=&?>_(^&p1QBFN zu620W8Tkc@OS#1SafV78lp%XoEYt9uEp#9`U4a^F56}S{Qq*GYcjRQ-BnQ$4jwKkn zk#5*UwAgx=Q@lqJn{4;rjEM3W+wfVdVuQ=XPE1AW((RUn>}v6 zIE|ZD9xhYFnqLiP^naircD4=2zW$mxm0zlGFP8+Z6}`jki9fY${az2eJPE(5z)&`w z3tu;?Nb)9bCgbmE?UlusPrt9iFOvJrt}U1SO9@zMX(y0Wy#*-$Jm|ju&L!YCZ?u%J|3T^SOAq z^ENPM_F4e<2URa+_D*6Q9f3_I9pjIBA2p;Z=7$mtJk$LVP07-_+X2z+t92A`M~p5^ z&#_8Z*Fdx`92~jBfLpY5F=cTZn?lfxcOw%XF1UAw-0Dc$`NADA%_ys@l4pQ77l6q=50%C zz)OhC{#&77<-#kb+Ld?Mi&|x{o7NZJ|61kZ*Qa7L{@&ERQb`G|7rp8YH zp~!7g(Y8HjgZKK4n$b`Ov3|F|8IaH{gCh@QS&z*nYSdI#O%ey)R}O7xR$TY#F(gGM z+g$QeS4tXmJf37{bnYjI7XBi5tDl{n0locte}92`e*u$forY{Rtv8So%=_o%_^-*z zmNrOB=Sjw+gig3^U&3DgkE&em;2RhfaRt*cEEN%9D2NeAf}|54l-?59A{h#CXq!pr z{t_ZOy>a1y-amdJ3983*aUxe(De^dbk;vGHN_G9BG!_)JP9YF4ZTzDgBJJt1Uexip zq+ZH>!PP*$sec5RFo;RXT#Qt(e)aWZ;$U@F0K~d?)!_uSFbcuV@`nRv|MZ9*y6JYz zrk18U56kWFPtN^IOM5%#kmX2X^^F=ZI+XPFP z&2L)JBbBuTTnm)z85u-t|Im?=oU?A&lSIK5nX}qqicl9)zc1YxQ||W#5Mh?c+FbhO z^fE=`e#No+|1$VKkifD(9|_Wn_(E4<7ED8jcY|kcOYowxZ@<%rb>iU_kppxZd6R*=%w0(!o3|n1T&^VK_p|;XXncgu;a3RtYp3lOK&@11Q=%nf>7;6CC#My+Bx!DEuqB@x#5(aa|B(}Qv zt<1+q>F>THfr(rXcClI*go5K~rsAm)3q3yWt|{lcoY|(n%bO*_70^YUEfW~pzo&g2 zl4b+hT#^=p9k8Xd6$>sxjfYXe7zgW27;<#-g8Qd_sP9{YL~*g=vn*Ih>;&mPyLVxu z)e3FZ6qTM8BKtpwx5Eer1!UmPh5%M*3{fvVKL{}B(1hreD@C5DYXjQ$TyRuxo?H^)Wx45<1&snP+s`simUWQ zU0J@DFL(_5mE(C`)?j;7^Ce*)yrWe^W`Exj_J$EL8#p|Ca zYtBVpW@eD(wn2)veW$EQZ+eGM^>PFzVg!?vzW)tKuYCP2ifJ}Yc;;<913xhNDXj17 z!tXW}%lMija#w(KQ_Hoa{}+KfZ8A_7UHxRhVS0gk@Kk}lzt z`2X`@={Eqx`9EfW3~^Z|9JEI5e5l%aSd2UG&vC_D5qlghJ`FS+Nc5C%b$DcqB}ycl zX_~7m2${Eg@NU}j0uT|$=jaIY*j3IpN;_ydCrcn{Hp-xo=!UZleuAbi|jZ?)L& z_Vm<6i0lG9LRGiAh+ zLc-DAxHgFgiA7}oc}-XKyz5s{PF&%_XfPgPr%a~O5nT$8QP-{|^U)`+0mes;;E;G= z(yaq08|nq>U$(?23OD{jR>tl{LWPhYGzUgjR~VcMKKZ9~5IVth;16@b4EDLu*3XVJ z#|II^kbKM+!?-n+J@2l~#9Yx@TEQejVF(c3K=OyjHym_yUbXu57g8WvN!G#MEI$9~; z;`fo3gbR49BBH1f$Sl{C*r6vOD9QQU{uj7I%Mi++HUe6GJe*X?Me9&Jyguy43=t22 zPy<<)9fAagfYxGH{obEZ!5OKleUJl?09ETOhLjKD5eFk)c{HT74^tsEvcA+zPIaG; z;m&;gxCIZ-o1-JAel2rrYv-G-BU7h-p@E$%Cwfh`Aw;%?my6D~mAAv~HEqk=$dr}0 zp=Zj5X2j}U*X3p54V!FhyJ78VYsm_-sv#qy0*}U&b*I6xx3kSlPqpD<4Jg*DwTJ+v zZp|tBu)5?0Eob-c($hHzGK2e*%LLC9j+HOX3%K9(^7`^y-}!D}X^p$%WAFRgnFoY~ z#_w_J^7w!|lBp|un+uqZp|S2P<`y9EIA^-L+N`$$k1yQg_d1#B>6+qrs!)6Te=Hri z4HFF)4L5Na9?np=FTgAvgfhOfvN^?zkcTt7J3|AP&!Fr*tzB-dAtUtQ&OfsTKTH|2 zBbS%nETTA`ojqN+1;Ko1zK)Vc01qYj5KR2aFY2QQ!oVZDys+aDw&N znMaPi^EJ?fwSl?5@#{{_f17;8aD3{*KFZNGYHm#5BgsZ_Ai6K(_IbIJS$lqJ7kL;; zRd3cHhNVUID5uh0Z?=z|{Te9^w|Ql+}Tw?IZdQ-tzV7gP=vVw?6?y-rTII> z^#rFiGSX&vwT;L%>P7h5{fMB}<)He{3+r-YzBybCaE688-Zj?+GbG=A7Q+HPJYGLb zTzupi{MZ9e`TMf`s^(M$3lXhyO#*xtLq%kOd`EE-V=DQo`gi-hcNZ`uMF3ul*)LP2KzT0+yrB=9wmJ`G__yDSZ)Ek!qzPNwz>9yN}KXjp*~Dz@>k6?hWDWaLP7ShhTQq@5+5j}ka)ez?ur1v z`24$M4)JrfdBq6PrgKX)hZM@6$7_S9;-yI{Hz?Dfab0|!u?l@GLV@f6eO@10aDcdw zJV{?4a(ZBTz6?X;>~B}vg3l|f$QuYH1lT*qH&LZOM=b{~zjdKy9EvY(GN!CkHA-6J zN+j8cfoC$TZZc^b^~L1B_~K^XXK6)c2_yGYq4+^Nr``u?Jo^k&5d$W-)9$6_4nh4v zxnblrLvW zuiUDW7+VQ&g01iB&bDmN%-ETRM1}pGPv&2Lvr7)zbB9F_hAo{R_z3B9YZC264PkT5 z+tZ7_5@Z9@$%&D>nx8}z`5qBe(+CsPWKqysOyiLDy^O=1ziZd_JczK`^h+t?KCT~> zlMwFq`!sIs$`41e*N9JdU}ruQaL5%uZaZ6kRk`BW9GvmA@+#G`s_Yjp=TMExi-xWB?>1qjzqrN?+c5-yNZ&Ev4vK#)`C&zQI;V6 zTJoPrD&Vno)2U@Nz9ikPE#2mJ+#a$f`jIsJrx1>c0%4fBR4@-wXm<@17i4k@6|EKsX<1i(uI;nXCv@oao-Qgyfl z=>B-T8;}#pU=PQpx&=%TsOF%X3c)X>W7lo8*rpSeNKoEhsk_`Q=_DJCKnTbnPjV@_5%jP~Msg!wZFNlXbYb4kp@2rtBrQDok*p1C; zQQf)qFrrP@^f(|#%-PK|wmhjct0r8lA+09Het03?u{JX zk9{43iawRvCX42+wqobWw>u5FDiPir^up)rG01aF1jan@*KSr~qW*l4-z^zY3jzI{ z46oPn1pu3p6H#t0fNKaxDs+%s&CkvBD=OeenJv!~U!In%)f!u6G8>K@w{T}1S~<14 zV?Sm2Lc)92VNlqW%I;{$6nbNF9zn(&pky?aMCE_^LM?=0=*Bus0Wv%3q^$KtpB7qw!$f-=-p$XM1sUDZkbdQ?X1}M+u@tA zm2VOq@W37p|6xZ$PnB8XQr75yTC&CZ!%6=zr$M{*?0SUL<%#TD!ckTFs-3d52RGZ7&} zrlovwir3}Vv?JWYpX(yiE3@U8D99&J0F^%ep~LD=z6EF*4y2ns$@IQN)(KE;-nwfO z-Ma^bDw+=y6mEg*PWRCZnmckCwQ{Dt^>n7;@kY02g2!OWYMXtQ|2fL%Fhb{iN640+ zHMYeVhIN5Wv_Mj(m6`q5>3d27w?y=MbO2^&6Y&ybV2m1#32E7q9zb%DUq(@Bl!i1c zA7CV(FmeGukqpZSwGLhl9>HBGY5})}ddaFkw`i=0N0dIO*AlpgA+@dWB$9AJ8IE@s zyINGx7gMDNX$;i#N!n%*wh=5QA0J5XDkfu`Vj$+H(5V%rep9T@@<=e>%AB&FwT>m0L|J95}N@Tza*?hn%p(W$yU@gSEzqhk&;%9Ef? zTANDc#A8b*dq;76{|S$sG(6;FN_H_-j!(`apAIH6ma@XcXGO;} zf{UsyNGYMcMyq=Dvh!7qW}kz}jD;qYBTRK7du20#8n=68ol54GNuy{M3(cggIvI7s z0#&RS3_@8E&t61qL?otlm##`^X1`9$h_Q?TC@9vFFW`WqDRIU@#*OG?xJgR+X#vd` z5PeO}!c`<-=$|dsUR#L-#|n3auW&5=m@iqX?uR_3!=F@}JF8kAU%M}<2iS-SpV$1n zu9Dcl#pG|XwGeor;Oh5XM7b6b$VeV)7HVdHwHlKAPpXJRTHS;k^^3Qto(g+tq=SZy zN9?K8t6QrFoM>TA<!Vq^W;ms=iEOwT6GnGP@xqUtiWPKK=#{Aac1I;a|C*#yL|6)Ge5u(O&K~o_MM? zh18)W7&>)h@%)|Wy9LOwDlr{uGj;qSnugU*CzfH(`jgTy{`5 z9jnOh>=vSK({raHuvhaqSQB@SLxjBrmfkv~vSB6?+P%0dyWAuNu-2~T=y?psqa^U8 z;4XNgx4^lfaM{A3gwBZ!m&-H_d>nlOiuKQLz6q>^39(TFDVm>6?#bQUnup{>`!NpB zY>vy#JKol2wahMhzU>r%3*EWYVh8ORzRp;Ss~JK+1Iihp2R(~d6JFWZK)8Bl)j8H# zPi33!*Tth}boP?SoX)aM66~_tKS?#h4~h^weDxT@zOVZ@-oF?}doe0tWnx{7g$;qJ zQL(>%Vq1n607(oHibu42N1vM=(Xz2#-mb#@Gzsbv4&qZ2d3l(aqA07yWdT}LQa$ns zlBt1KvwQb@Z(R+0ByP%C>X)D!(GRgx#U)_Z9iVfgcG8@)n-njz<+^c$#cr;x35zsg z`mWo269YW1)<}v%t2LpteEEvSG<6fz#@&7I6>C}(o#Df-3p%LTM{NpgaqT^)4V(2S zehq3qhg)hZ_+X5g0XsGWpMMdW>$~>zeH|;Rb1l;Hg4=*?Q(e zYTZCKxv1&XlhnctEk??$dnYIvXz()q5c2pe!E^`#V175dv+e`hY00IbI*F7Ok0B6q8O(X^vS523hYOhBqY+4Uw47IO2j@N_hkYk*Sfv_PhrHoue%zDk#Fabwf+P9FR%W7zUkad~tylq3s8(yFk6>u7E#-0pppxFm zLm4BgZbGNDM4zYbDlTieEt?smaTOvsl`tMaShdwc^lpc%3%0Eq&Qy`#MeXb}Y{jPa zo8~+Q(OpogK1T7<$y=C;y!4d@$cLY6tWocR$AIfFGjwlZ(?ZbEM&IgmUta4v9rQR|os1C)35enidEhY(vIFiW#Q}d|<>r zTN)Lo-tL-g?+;!XFUfN_C0F5gY}B8x$x*zktFlM2wY{I%T2W`-r5uiEzID!L@k`pR zSH@SlKQi~I9}OSt0fjIHV;sFsMpxENz*bg_?bW@;%cZPrnc6^O`Xb$*=p=36S&Jxj zMF=p%oBbg_mY_>^qIRo3l6{*+7X|2&7d(=vUMV?^ENt{Pr`kOA2d|@Cp6H7nBQgn4 zAIICqL^RO`fxS2)c9j1>&xR1~_3a8R;&5bC+7ik%uN_kZ=K>&z+)@IRZQSe`SJn=M zR@TMVyXMWCE-J$HW~OUGhyd0%ky5f26a%uJhadwWk`kH$6Sxx!{%1F5*sSXsW#Q#r}CNtWU~o-hr#txw~oD8Xh9i`Z)nY zq$pY8T^Y!c3@y8BHe!pd@#ZvF>!ehvEd$uh%im)s(u?6JHc#|HHLs>0wz|bke=Ghf zg6+EQs(EuNkZ?n}qYgpZQI{`hy~oZa^WVHn!0HIz`l>)|vCGb$;oVct`-nh1hT0^N z-hh0pb4~$i-`cig7o19K`cwiYaHuR-Ma7KStb8gq-|P_UD< zMEiqP0do_%L|X@=S-0*d@|P;dPt?@>G!)zYvxlOV7*li~>dDePzKEJPs9AO({M#)j z*%UAI=_b7mXdEwrEXU~l$Y(EQ_1V$tds1UWrU3E>h4s9K0Ma_&;&`OeQ%ju{HS22E z2s*#sJK)8`R);CO9YnAbN&NB9=YNaM0t+{@lK<7V3NZlyaR29tvXj1}<$n;jhSV%= zH`+0N9bA1yWP~qlsu zdckpGhZ84Vx~HUj~y4 zzL9NAF9S4T=a_wJ8B)V~RFCAmes@eWwEc>W0*4tyL~;Fxe8=2DES1wjL%_8(c)^Y#ZBQ&3vO66u)6^a4G`HS!hIpGt9cN$heg-_5RJ`?PI9M_~^**f$8@P_& z>=~R9AhE@c$I+&LSNh61+c36v+7ntXVPs^AC0QIst`Tb|%1qJ%+NS+?ho?V}adbj# zHxV!ZX$Qu|t!IfjPPMx3Xi+iS{reSOqjK#snoLjqW3W zTEY2)=-#8tChBLCwFhoyWPx21pFqmVFBENZAw--NCDb{eko9hfwbLa9FMjTKqy%Lo z-kEh&F^g;74NYjTWiKz$EPx7SEmoKZ4%JEDg-rMgpa`&V1~8p1wxBaTpOr76SF>;j zfPldNrFs8YQ3L{J*kx~TpZ5c&4mtKoRcJslq?1cX;DZ8-A0U^^ zjjIS6j@Lg?%9HD0FZ5$QP#%@LQ%?3Tjq6axWLo2Xzr>*a`JotOOu45j?mFLrc7+F%q!3Ygd(Gqa)YD1*4mP)0#N=h zxq7t1xcO-SJdJ}USqFldz0zZe>xr31XEyo+^K zXP6bzHuHDr_m0{kijE`^%K{4X#s0wwkEE`?A1SfoVJKG7X=qS;7wQmWq>Pb8@_VF&{oDOtuvz?c0$*f^gl7WPDU$5sLSz%m;ncCd(Py%kBH9 z`^%}dmdZLRKQ8~_?G&KI?S(fKG@5~1ThFX^8Uw?apuOFm_?nyiKy5L@8&%k!X7ax8 zxE3k_Sj$NMzsNfwXcuf64veEW)@A{J*8DnXSlqm_4?JXUrq#{$z1K{DI7h}r{*_ihFxK3 zR3J3^f+XNvm%eeuQE0`0EdgCmo@G7C90VlHw){(64oaMQDoAam@*q0AISNk^5393F z%7FI>VwQ{_py@@{0GQ7-J5wtP)^hP4I8V>0E?_s3mv|DX8SB`vJ`O=Z)8Edu3ei%G z+L1ZEdsr0d29f_+q)j(v)WS{SSpm3ns z5XLl^ZtP|e{qR%TaJ0U zxw_)f?9HkQ^jB%u1&><~@UQMw^H9YWqXUC)GsGTt#fKV1*(yBS5J+$}&=nmggT)~s zb$@0)>NoGd)Lbpq=Eva z<5N|2$5*9g6}cEdRKKJGQr(8Ge`(&5w3d%q;WRy{##g81!YZ8Qm9~A6YUtd4wpr>TSDCc_KQF;GnwPt5O`UVGDLpl;X81`1y8Zj$TPHJG2OKCQl zk7>H-4ZQRMQOKQ?C!PI9RMJztqFYM#d0qTM$-bAab>0P7=vHJF7na`t;MNg7WWK^J zpUH@g$g~JpJht|o>Vi@53|lkRkF5k8FPo>1c*$6>G^h_8uWeRiN@rPXJI*L??mK$- zdTVt{$M2DZNVXl)&n^fo<3cst3p|j7X-R0AdZ(Kk>*TWW1E-;n|9pe}S0fVUrgJCv zZ}chp?}jw-|DjMbc68GHe@sdLp=eF&qgAaE338zGKC`|-kmT)pR4OX!L3yjsI z(TiJ+~9cN84?R5&CMl(92le zpM6>u^)JXhqv9x$22f9qxjmEvv2L#I{re)(7syUpKC>7&6bR<{#zI0%BCO-GWri1; z)L@F5kv1T_;m~eiA){kPl;l$65x34U%wM}3KxkT*u;01>b8jYm2G6}Y#oulQJv9dQ z=aK$i=UKizZk#prcimd1`bkaD107toSU2Bqt~l6%7tnILx#3o?GJj3r+1g?EB>Bkd z-;n02A>#f=(aGjPx{aly1kL}&-a7<|7KQ1SY3EJbwr$(CZQHhO+qP}{rp=o+I_p(W z-izv2-977w9-Y~qoU`NX`1kj%r37LSrXBK6@G*Uvr|Jij*3sm?u8L1fUf7yDx$->0 zLPdyANJj2Jj#u9cIL}P~iX^(PX3KoAvuXH8=mq+Jem6eKIP%Nz000Ag|JzuGtDUur z&3{*g9l^4)!|J%}@&mOZig!*iAtS$$FQTkmmn2-K6Q9(nv%s%zPHukUbv~-a*{=IH z9sif4gG%GMg@+)#<;dQYohf@wk?e|b%UYya^R&9>tlsVPbT@C%&(D-ip;x^rEHrg_Q{R;E+5N+);~BMt0riayTFDV}PXuC- z(-fvjtWK@|T*i@k&NzfYgm8yzBm?ozP9u9o-n*zCN!w>cnFpwM0yKw5!M_4fgW<;} ziYp0#5aUOfLqmS>>vmC344VHYFKld;rVqtyyQrg&DDr`T9tguABaonT4MERkY4$EzZ{-fx{t(yBf0b5H0Fse$+_i~ z!QhVuZJp0u-2h|#{ht2j?d-W|+~>O)8=3&2-;UWhJNfox4qjNeK7Kl_TDTT_##PX_nlaxafWUt}T8J0<`vUyXmKAkr9ev*4xN?S$ z(OO$`)2RJ0XTo$@)8op>gW-+3vFRkkj{~j#MZi?|;8J#%AFL$=0LkN%3q=u98yaBI z7g88<#Oii)ZbsHGXxeBThaNCvLM178i5 zSqp=yrrW`;gUG6PcCHoWTiJ6`(N+dril3}3rPB@YltAmk>VVx9hA%XwGelV;77s+e z5TnIuCo`W{fSHsgjG)ZLJrW5Zdmu}h1=Hj5 zOWErLKp<8O+#=E3C12j*_P1$S3RF$mHYIE2zYf?m@p`tH*da{qFiH+(D5Pm{N(BYC zaoz|L-j3k^K#4a)M3DP}+jY(M?~eB-=*S67%_cMz=>^oGU4ak0+>El+u{!RuRrQ0N zWE?TEuy=cpfyD_RgFleR_VfQp$p2Y9iQB-G_|q1O1qdgZl`zvth?+@!9LUuuz^jLk zwYY4>+#@!^yfWQ7R2l8-16c*po2D{HWzKV9Qj1GQKfZ~#>*i`5fQnNeLQJ*j-8xao z>^C4=tx}iR)6<*bhjrj@6a0w~{Q$}(53M`Wt-Cqhmg}8sGDEdzr@tQmizDjevi1<} znav3+iWugxTLDJP3G}mZyI3E`VmY9kJDk6~@qcBABJi8U2^3`DkA(_FM|Rk;%Ne*}!mwqo6-)f2hRWNB3)}%=%uA7) zH@sFZ#SuVdnQFt@38NIPnb(l)%RO#ZnG6(>!giwcc#K5>eDlt~?=B`}8Gq0Z5E)~3 zdJ)&awO5p@<({Ch0&xJCz9U#bq6-unNNXH3v#|~xkvaaGT%rEyOr^I`vS*w@Fb%+zuH$~9mvc)WYK|=E%kgs&i&$3MaON<^jcZdy51@G# z+dY$A#Zfe&OH^GWP%?)p-h|lnTCQMqrjO4!!Mr~1V!B9E!otnsLwCpIPkL8$BpMfB z+?(C|b+M$XktxzdTsmssJ-Nir?`$xJNA$9}Yuea+YLH299}F z6+sx%r&IfHlpsWOQmj%n4(!1KCtmN&hu7_VCzyLYoGXJWrL#`stmUQdM)6IZ(@buc zQw~ExYiOo7{XltHPHmcgbpg)z8p`m4EIMAc=4_MX@F@o021|)+tg0!bP&H%%*&+uR>e88BaT1(Wb8yPnuTlMn`$ymJr<44gP)}NFG5XI8PY( zl+w)S^y7a0M<$jO8BZ!9&1tlw_r*p`h06ZUy4Q^ zd7?(}*@m?cmNnac-TIANX6rSh%*@mbe@@Dc$3y&#Fu;%1ZC!eITR$SxE^8@2E*HD- zDXI^pFzq}QlXjU7xQjYnhM-3vozu2YUtF+3Rd}KGKW){)mXSEJgR;ZVxm1lBo->NT z_CwnR&oQMn=+sNlWBy%FN;$OEkSIenOgN5GrTUZ5J3sKlixNIAQw0k3vGkM@Rca6n zb_z$#hbEEgK$Q0iott$?pswMvx`g5(Pi+v2QPY>2shw`@P0hL{?^eC;^SW9r9HCz*rH?`&2*$H~U=9Oeg2q6G8DL|#qIUl@6#eOPohFMPq9X`ZYTXJj2~w;X_~twi z0~(V{`vVR^K}p*XM>SF{-L^yO?oxS74g?6ZPR`b=FJ(M5JOeeVax0yIT-QL%tbkIW ziX3MMG+nTnD?1 zHtZnXC(-u~QlT66Yzpw$y`HoeFqhi6cG+^_SdDiI0^hWRy|m~9zDfM2xi^t7rLC{) zl?Srgfbpd11nkUCaWYCaeidJmGtr>flAw#gKA!e2zuz#tDPx`0{89KkKd|61we``$ zh;hk=4)wTYWE3hk30xiSosadsZNu5BRg>Ccugg%B<4@^B%n)_rvlAv8z!*Fy!fGT+ z7CN`brJe;a^ZRdalm)QHTK2X6aS0)~>Z9u>5yZ-+RnCu1+y|iXA+-)b~fBpfN&Azy%|-s-bBWG+L;;D3vL(Hp!bBSg7-sslisbp>X53T zUbX`599fI`>UX1p$Q`F0H9y*^{<&TpvH%O(MBOk-?(e-rvFs}gUfJacC(jHo<^Juzav$5UK-yl|c?$!=V3rz2AZ`3RGwDmoIg{=J5 zd-tk`k8@?3Y2yzoNwdG>)C9kIX}hjCSH4!yzaH`|f~IgkXequsKy=NpvZ3zE;SO;* zN^p+IrEz;H_5z5NyMO{X5|JBdwS+JVQbw$#{=-4qSn4vl9?W!&^bzX*=P|s@YdkH`_<>*U<-!#6bFZB)a3#Z!n zofza5V~0LPJcMCtUX}~;2IC{9<5J%N!4;>58Oh;$XmWw$-ODG^&4_zXpSLi5g%>TD zN?-P!KlP!!ff?hW2yId?*U8aS@(vW??B4N$6Sh!5Dm zAN#!PzJZRSpm;Y07xjFqIiI3jN+6d$dS!$jU(>(|{Ao5Oxy zGm7Nq`pC+z@4A&IO{WVT2&k5)XkijyibsI#(kCfsYz_SMa~%m_(X{*stued4@8-WMgV?JB|k`55v}Ex}=Af7_Q(Z^SXdH?KU`*!qH4DnmrDBNt&#ZFIScGlViWC|TgjcY6qk?l2a4_H2T_i{0g@H}?_YxPt=dbs4t zp%DDI?{}eJF^Y8RfDT0XZk?oO;d0M+;&g7CCal~PZV+=`)qR&Rk8#t;-+Jlk85(!x zK(!+{Qp6C1e@TqsoVE);=BBy&?tCeg*|OIkOCMdLNlrUL&AQhB2{vdLV-}xC%?8?{ z{`K9*F*FYKkUD6Ck3OLG0*HvLR5;h{iV#^%D?=1hH9Gr za1)<%M*Nm^bPdF;xGph2>$`-2y~IWMkgj-idm*qScX^bwJi^T4mgjYgu0_W4ms{G zR*^i23%ou+pgPg1kIh-onV#3Szet?g=}@#AtQ9p~f$h;WP`sE1vNR7nV|&v5dJL?R zxJY3`u=GCG5m>6T^ju}FCRM;ky*z7JE$6KGtngrq9?7U!K%wPvvD{W5o=BjL`~Kf} zf6)KSNtTnji@nqT1KtYm4P0aR|H)hVuZ(&&2LHL@|83xkg54%Pa>(u*irQmIN~7Yn zLDI8nKe82pfUNnNSQ7dH#qc<*(ANtWtFsLm!F!mGfzX599s=xbm-N>wf@OG_Bm|cd)qP@0Kmf+UlC!EFn5)%ukz1 zyNXJcZ0neyQkQQf#jd%*u2{*L(hcOGiorMy?al!1UD0MIEt=bkyW3(E@D;7HmIUAg zLfRe7Jg?7LoiQH{FF2WcrJ^)zJQ;!AGk0pwtOd>F0y zuhAgH1KlHd-QOa#m^#epTOykIH?Yz7qKL@q}w=>TJ$7bHXgt;fOo==2pS^k-4*tZnPwyS2?ENu7<=TR zHo|I)XF8G_EM@D}*E39IsGBgGQ9Yxth*K`mDP@f(A^PX0apwTd-Da#5QWxH9Xe=X| zfEhJ@vwcxh<{~5&Mq*GKcT5@zjsr274C*)Tp~*_WJ_zNLtY$^_L1#Lp2B8*>zK%V= zCKfH|{m(;2vgU9_h2JJ>!!L7%^nbH$ENq=k91V<|EnNS*QDcni)_*WEeb&@qSm6?Y zMOu<@fdfj*ryiP%mz~3P=@&$4Fxmqm23OVo3Ii$RYu@3Ol&FpG=2r20`;^ zp})m*LC|U1FWchbE$-3*2=NSn!Za)Yf&Dafe-$IZH4M3V(WhGva-gL#ZsgDZkVXgR zJnTxJ6vOdK65uh6n#K=Y$;g5ZGOM2_Ar=f6P}Vd%Ds~1r;xB(Bv(c|q%Ak1_sZb=` zb0p?2WNhh%h24Lc!O<>gVgA5C!Bpta&qihppvuT^ga~xM!^$#(N(#|iA;Ju1u8iQ9 zTLK%DsF?*I(Li?>G_rsZb1>^ri$MoefDSQw278Q7Yk`m{2SpyNS!3OP(M`QFfl~CC z*Pe^Vg^2x|JyqxrDQLk!6$LSl*!w)DXZvq-zNdq%xo&&E-ixW1k4P)cDszmiI{wI6 zRUC*kE?VAppt+7N3{7vm_oKpZ1#g$>LD$dQi33yMpR2gL4!U#L;rGb4iW@{w*Z!WX zGX`)p{4Ryv$)U0OEOJhKNX_5gvUyEH_2T#_vSN5L^1Sd||KP>z?K>j_=o;MaJ}QPe ze$)KsD1BSvlu>a(0vMCzQW2oeuCI#dW2ht(+dhZJ0FH{Pj*1iacciR?X8@_ZdtomRMsBd(#KZ0G$^u zPs^)RZMQ?(>v&Jh*+c8e=zC6qOyE9t?iSIkWn9u;h4?*z_#-ghBay=iM||WDQH0Wt z1)3~j)=nl*v2_`(_Z~}s=kN<^Z7k*5sw-)*s@kwy)%6cPv3|;;lCeowr4qF7`QHw+ zAtjd9JC#>I>WUwL+rnu$iB7DV(m&!B4XU0NsWZ@#<9@6rq&Jo)C=@wDuG?k))O62u zv^c=5BEK>wyCdK))O|))X}!km6RO+1x`FI|*0r=`1NDdg1)gTLAKIU}%y@qAu1L=v zl>In6U6thuBx4&oMa|9=@?t~PV-m6e`q-UubFsGvNNeNq{TrvNvv>TLL(5AVYP6To z;-T%Tv9Q@=ow+Khr?RO%cFLB_bZ|3mu(sJxL0TLhiKoHkv`z=bN z;U!y!p$zGVw;J>rE*Il4iIxDo)8*ZCLV*koo3pX@{^#saqNuX{snLK*A9jv)3HPk` zsB>33)Q#AN2@MmAv2N9hl+NKRNYAWYjJ60l?n~(`xWLwDx7^lt#c;4J zt}(4&3+F^-V3#_%WKX6g%dvWjuqH00a$4dm1~{gU=lhc(e(43i1j(-}O^p8A?)>oV>>Fgeo5~vXvqtWF4@o{73pJGj2S6*k{{^+WddChU4;LvEQUiY99 z!;t+5;EDrmV}+$W)}pk}rUQh}po0ZLqgn=FWw03N z&WPWgC{WCEPlE~PeJ>woMX7``B@0*au5x4F{~g-2=tP z*|r{}6L2>5Ss`FVDNepb6enA~C?n80zr&V1kD z0VRWN;e7(maoPvaUYFo(aegcGAv@l&;!E(K6V_K=?HAxhcSALJ-ho)$7}7-OA4q(H z>*kg!tNOicWbF}Xu{RDEJFiBB({a7*_NQWighMz$UGz>v<9IyL+wi7v`_`N<^Wj>- ze?IqUg=Kbzp$h>mM_F!p4?zM>UT)m2Q99wyL?GC3H4k>8d!SIa(PMP{4*F}j9QYDg z2&5Hwn+?C$H3RwMUUGFr+|hnS6Zl}q?-!WE1+zKN)X{C^_}-o70NEZ-hX-`nbJ&OB z9RtitQNiTEbVGz-*nk5Fnf=nexPokX+@j5eS#WpJ@SHgfMGU|yxN?Ha(&10uLXPJw z7QD{kxcrC8+b-}h+vBjdDh|MsFbG>tTx0y=JS;-PKHR9+QuxVc8+iQrTF!Tpo~u-i z`NqT9lN29Ll_yr1Qp?W`-b&P8Le-q8(Povid5o0WZ(MU~I*Swtg)M5kaG%*gopPPF zggU=?;uI<4abD|}K=|Vdu*A_#5Ua$4J_dznQ91sxf>6n|Nj0Q|T0s~Ov{lt%xbj!k zZrWW?%70ciAg*!N?Vnp25sfI4L{<1}QP=q(X(B5xF#Jo%2Zk=SVWLU>It2Ej<;x+{ z+t+s6$MzHv@%<_GwmI@oA)!mu?a~MA!{!tZGP2EX1!)3CIqXTc$033Q@lk~TB2HK$ z_W}x1&JXE2Iv>JYyKVSFk0HgeZ_)X10^1O+}tDz*Wx9#kn@#d2^#$ zL8O&2zo8o|Wt&^|t)mx}C0O6RMMl4~vz83+WkLI6Y6sER6>f1=OSEb0>0J0(Iasy) z1b5k9@Rgpe=_hpIjI#W4*B~O3+vTf3vI%W+UVoz%*5S&j*ut=-Hif$dGM}I6+k|XA zf@S;@y&l=!qu!f!-U<6}S-=TQvKEo%Lr^Y{Nk3;f=_^1=SinMULMUtN*^=g+{- z(fGe57o=!-{Rb=XXE!e}&CtN3KqU#t4I23tu(pVK@V%wAX7aq@765B0-UILU!`= zVJuFI=dsc7qv4_8Az)fP!Jfc96_Iv47RA~Je=D#wQG|8_IXb~wXb^{{b((~ch?yro z8wCixv89;%_&ilU9yt(4oXQPf+!*Uk8XD{fp+L}>pbS4jSfgD|ow7nW1`C%-0K3De z->-q58;z%g+>n`+8`L8@S?VZbN~blgSYHs8TbP0tKID!Uh>KrR0{$Tw>z5J;NGyM) z`%4L|l1ijL!1^?u?C|c3?O5+1J~%*w2ZdVof`kS_6HF!itpGDnh3h*+E3*Gm0(m!o zQ2|P^|3L}dSKQ2V(^!0j=-<4GLPlm9K`{aNr3C7sN@fe(%b*rXI6Oz&hiPCsJj!^h z3TOj?>X^w%bCf0#%n+|v+YE#sjqzuZG!*R(XJUM+gCqjjX+Bi*a|H)m@4-iNQzb99 zj^(l*tdd2Y&)je6-z4b^m_D524Yc^l(1Jmu>$M+g)pzIY&_K#hgC`yHWbDKb9OZtZ zO4sTf{BUG_M1hKD&Pved#sJ}&22I#IyWvRpCernDWYA8Kqk~JqcV+(%Mj)O1-}O8= zI|KGt#Ax_>eUE>MS^6A+6GvbmaO2kWdSF9c3;MFAZVwyI`#}h6_@0Q;wkP=$cpph6 z;F$)ouYUaiAT+k%MG1=R?Z9*=M^SL8Ful>%!J9Z^mqmYdc;RGuz^Pav(Y zK$nA`v!=Up3PfrV8czxR$tsrXbxWd>Bd&Moq#Za#uPX8QuEGpf(;X!2m7DIb=62;Y zfFVf)NhFEAt9a=U+LQAUwiZ&QB5!tVam>H9o+kX>pQlmCXRO55niQMIny7{uSG*h zQ6@M-jK4}G!~Ca#Rg-wj70(?=F5;Zr(lS{INs%7xX_K(hECxOzJ|D!ZsgpF!X1dSr zf~oW+k5I2(PwElf&ROpSg<|t*YLRTlox;;10$qJ^%*z}YvgGA0QUT_bW)fIqe^|Yz zpDYVwMhxS)@$b(HW(HNLZQVI%rUCYtSkxA~jl-rfm2c01eU5S&+8MH?8|OCUblbJX zQ71z+gIPm;gYClZ4qwcj)@WPCwLRM;OmeiNw0+^VQHPe)-;Xq!;fcP9l9A+1T0feJ z&1RIFcoW$dx2AbkDCiId2s|44!)B4B3VOOHAbZ*eJ^uEIPc%6AMCr0G=u#gIndBPz z{1l1jBR0<6toIO$Ne2T@#Q=g)00stq00Wj*HxtY;A9Q~~_Ka_lSKGQ)V*MNr+{)DB{nemoKGK7y zS^A#<17Oj%YDBj~kpK`{$H{1}yDft+@1Iy)`Q@IZS5EAx8+K%rF`7P{_D0+eF%AQGR-?>FW4r zDaozBCiR`JgiM3J58G&%rv{wJ)zYO9eLOAwvVtFRXWb}A9VzfK7rj1C#=@bMJy-x6 zMfyCc`P+_`GE0e2yJvR)uiji68>?-HDU|K!wf2P$#X+WH1nOIztp8TYCt?rU`|(G1ZNTWHx7(!C~~hYZuIi)Bqu%WGX7H4dE%sxjA` z;|q))m95RinyvUwO&!!pkuF9rEEQIL8k7|&3!SK3YLxQ?oiyDh7aJ9%WyNy1mCn{p z1N+viEv~@EGO{VXdcpVHgt~U3yIEY?)#Pa z{X(F0nU!2iGN@Tn+9bVOhBi{~J{~{#WI+u-_-QXpKLjDM9|FAMD>_i#p7Ha(GxcAU z?hThpvT46JWGgUL2|q~V>|eVidMn%FMup4$i6p-#JVgINt^V^az z_)Xj3Jo&k(cS+W6V;_q8sGqObA~m6f-Z>Sjb?IvR_Tcc)(L?E$*E%5=Kloj-s5SR4 z{`v@QAmxkGoD_dmM^{EvmSE=I9p{+Q@a-ZE1JU71l1KSm5aaVsKUZAobr2W$We6}O z*Ns7KX}ef0g$GCVWg$~tsXPYOc%AnUbUv-F*n#yr5InRgNgsoMHx_H$aa*+jzJf~> zCZ0Y-(7QO~Tf(iSL2v?u`zl}%;>x|&3#nG;24Z3s`$Y0Z3{0)@y@POyiP62ts!%qm z2wEai0q*d45t>rF)qZa@B`(XX9j~AF40&RO{O&Lby_0e$%fM1{t;R~}?4qi`Nh0Df zDErt(ji)Fl<3L`9ER8WXH)1PncS`C?VeJ;y4= zpTfnGe%-@2~_+6Lph^heo1?7@NtvDQ@vU2OOI=dDl@$sS+ zCOmRSW9vzk_G0Ueke`>CYuXWQ$Nga@uI4fXp(6(q*S}v23<`%#lh3TJE2u*tfyc|q z%m>2fiMav1VR$uMly^U4*cS*K{H4x+i;2M^VaqkC#XGw zI}SPE6JQa0jRWjw_~^cOJ_n*nl3=*$zU8JazZ z-FP7y3MBoQK_v6-^pdE>c-;Ya`uou}GNWX+5W5m1q9`x9AGN%1V`DRbdh#0@7o*X+h0ZT>_r)z8J%#_q~ zX5{Gja%N8OCLr}^(X1Rfx-x45t~?1^m4{82P!hW`Y31nlTpby^ghOUl`Qpakta-pE zkFX)+#-EHka&vq;v#BSZ%8s85SwKl={U5zq`Sb>5m#WUauLo!AMN9bxVBs}&q-p6! zLX+h1M-sE81Ck~tj+GNZ8D#0mnyL^uvb1G*3E89?0bWZ0vUH$+QBbE*73!e(NUv_4 zxilx^Z|8=+x`WpcAX>FnS947(&cdWq8&EFxz1rF_@NMSnb`>Y*F*?ORCT0;waJnQ?8kU%?{)gjH_U+?PTo?vVw{*VXi?#Nj4pS~P2f z*5&2311eWoP zyBGr>SxEJISEHi60Ea94BFMSfUx;Kc&9o8Tul2G{Wpg!nUdoTm<35*8zQK(jI=k}{$rF6sUkzkOlW^@i;-^Rj=$_^TS;AcMJB*08HuYfo4A8N#Bm`y zx?i5UVI-duk_^rxoTC#3uKbk2Er>G2bJY!AH2+Ie<+dpuxexEP^O1?XQaVU zsB*wv7@l~G|BUEvINVy4o$c#>dxvOF4dOg&AqRs!{H(_J47~(wxDa<^4vKNUj0>Nx z2fq4a#Ws8hCCBH13tRl{d??h8R|#`Py0%e+uZ{Y_c7R@iC=+G0eoWt#?vQ)r1ZrYA zp*LDNVU!W#Tac#-UJ{TesNo-FqqQimYYn7<6C}=nLa`!^hx|x*3`!bgr;oz057AA( zPuL?K;Tq{HRcg6~CSG!s*cIROCW-571L(3}>jYN7-&YcyXec#q`Q%?u!>UW%MA1e3 z$h}YZ#R%zY^(=D~3&K#RpEABFD79(2n+8Q`jEp?;DVEyv!YigXThM9?2~oo-N7P9k zRZZO+5baI~l31E1S!gNb)Iw4>Fj`lC4WJpHy}(7$vKd)kllk69(p|3ghm`yaot?VF zK|Q>&!a|vX`BQbe^f?)w#zwqXH?w-$F}esfk~@Zw;R|*Tv z+9u%uc@PzR1YgwZvj|v}qWJ^m?;eXQt2MuFTU^OJKO)>h1^nE?Gp5C3s7`VzpuFKm zJ}j3cNRi|cGo#{2XK4o7RMW`sLfPsW36I-e^aL)g8LPdfw(tTy=gpZ^_BItMp_*Qw ztF~j7C1-=HQ8oU^huk~byyyNd=fGjDb(6qVO%55vxC=EqH$q8rs%B)yhfbKP*;@eD zjGMWg1{h|#37*`4*}r6Jn0D|e9zrm?6fVLz`#iDio?7Ui;LWH>eO;ADZTn4uaqy&O}Csl{)R01eOt+c190=lov!}P*Dd34MKa-uh) z9mPKPB4);5IgrfDU9jXxY43#Fu8Z1lW>3cvPdv)}GMC~b)o8o5loPWapSV$lN+y(4EWIHnb1O zcEy`ei>sz11W)n<{&@`kB0?m%(KIp!a3U&!HZJ~ul(LRyiXz>SrNs-z4>nA|=`?>Y z3Uh{dh%SK_>qP*+$Nb@efqXo{e|zE5x}>txi1W}CA|yfGQv_59NWd`&{p>neD*RWDW6yQtRus@sC`)Kk>*~wmciH3J@Dm-OmHL7SJWSOtd`ct%)?m89|I<=$#{~|N<-hlj26@hac*?>26-s9gS!U@jNZCShnu~mZ)w!)4|OD{|;h68()q?jq9oDoCU z9N#DsD>WH}>Us1$JWcVx2z~yZQBorE*78!Kiun=#tgwqb#O|zDoap7qm&$=Ttb{Adk z`map>snUTo|Mo1y) zJz$+WPvv3O7UBZeDegSFLel|C*#$+(mV)4{z5KHY??-jZMR7-h0*|ySZqi0PVt{(I zc8IRep8~&Y(Mx8b$dd8n&?IT5^e{nrc}H!?dR*lY;%W^O|8Tg5eF}2orCU^Fr@TRU zM=fgkazLiA^Zu3(?q0zOFkGArc$ehkp;d9N+C2KCp>Abp!iRLIXI*VKZ{{&lx&;I z7_j2!>M=v}IWS>g%U@0Xz&2ZHA=Ob}rI*nY4ujpF+un^XE@qnmzR%kouOD!!(Rkjn ze5^5spiJKRekJ~8p<=FxhT4Se-iw&!L!WJLFKns z&s(O24M=?Cj@M)`{w@|yHf`C2`hcW0_v%==Ufb0O~^n_~r)Dwv1Y%bZl)k9c|lkY);cL1;1UB#MJ zFV;0+=H@Y9zhZ}E*OimIS-fgVRbe4^$JClYXDvcwybBhy!ijHCB7)65hEZ+5Z2;N9 ziBY&|!U$nPQ;LA9V6f~Zr{+o8m@%X2QGGq%@t!)Xi#mxUDwF&qCr`;daqd(x*exfw3Xn3`Kf8QM+(lmbh7P(7N^{jDca1B`^yQkr7TSAq0JZj_~ zRdN;B8zly?gG_#hr&Kx5CepFbE3TCt4TN8QDLDUWs{F47U#S08bjI1m(dz$Y-t50J z)!O`z_5S~~+5ew5`~Pg(Y^9lC$^X~vB^eI@@LxtM|Hmh8WM^yp8{M$5v;A+%#@9Sn zPFQS-yI)lHjveA8%trrou1np^dWvW^OlxR3ka5UPr5Zx9<+Eq(Lor}zoa*j(>+sxX z{oyD!WVF4im8oX|VnmB}&Hdp<9vpjW8naAjIv%m#_NwY^U#5)G+V8x`jZvR5QjjYnd{0T!`u`&*w70V*67c1cr$Pb$QdDNi+k;6*fN*{C#@ zC1QwZ8?`AHYSIwj^RzE{$;z1JDU-B~LHm;iyAj(odZ^i2%8M(jtYhUp=go8N6#Iem z)4@DiCm+cn{2_|ruS5s9;n#B2Q!HBh{YYeNdoqqQG@petE5-nzz|w6j-3x9G>S21Wnj$_$mR(%Qt(J*qB9D}t@<+@22eD*LRRaR83QDyi%avQ z0TJi9?o3q)a4W%V$MqRjO+*|DGR-q-m4BdsG>qNC3R@sT2qMaTXyTk6v zc^-V3-(Os83-d?a-8$a)w0jbSrtL=cMbplC9UOGE9Y$SR`8%_q7O~ms#nJAapWVM* zx}6qHr5#{Ac6VonuNAQsLHg%tFT`W6pB-Qh^mL%}`GM+%&570}=od}A@Z1?PcYur^ zfCAt~whi4GKYngs?yGnEu=#qq_yCG$1D{PT97i!kJKvogI4*`I>}RyUqV@vhU#bnQ z)milKbES`{b9VtqhtBEh(W|GAKCBHbWaYBs^oUs@H`S0{~=9dBZ zLj3ULMc2|IJe;bfB{6tk4-AhYZGC*5y%&;&9xkvf?v1=&$iPv6H2d?afq;4H>e6Eu z+qc8dgY@-v_1Z(^=)vd#GZV8ozhu4OagVFEPaiB%{;N7lddm!aT-}h&{HH-v)k*F2 zv^P))l$x8;U;e(0=uVra@&Q_ZP1xly_fh4k^tG<1HXhSCqxEShdWT6cN889D(F_jA zkD9pb4@mFr_cbG`x^?;-v!?BvIn}cJe(;7js#p_HaV(;eg1hJ=&ydxv+Q@1%_PD3H zT7&V~4Crzs@FnW!r=UuueL1P*t?Q7(LE3gk1E$how)P0Csp1P;VY_6X|C zKjIU7Q!IPNVqVEe15REElRhuSpD5s51(H0D8Qfg1V6FA=ed%Hr$b_)iO}i7(ae{Xb4C_ltNvW( zAX%x^+8O8EJc3CDKVQaZh93!Vcv40D9hW7hAW>cRei^5H)==wP@DHozsAy=+M|PXj!D*W}LIL$F_LaULZs;Ut)usqtWHC9#sSR#H%po zkY;&u^-mA~Klk01SI$~=KhuNCFGECeXr%HhbN{^i< z;0w1osdQ65G;jQbfjLllBvHukEs@-s_UK5>#IEx{bhPb_10BnJs&K}05hGvh6;P(TluSvq?jFc$w;s?G(H!xw>1W0S# znPa1Pz1#LzL>hjJW`NVp?$XG9ONPUQ9hmB&pr2%4V~>lpir{VHw4#A%D9_kpCV=&) z={nH}Dv-r`ePOeDjOm00j>PnkjD;O6=IfL`Q1T3ctMF@MDjoX;(UoBjfS^A^pwPU$ z&A};$>_=npcbnj4**NAXg zb9NsCB)uHhQjh0V2z+^P0(8j{=|X$YKWBX!a}p^)zq~h(KkMS3ByKn*8K4SveIJ~Q zeYdQas;pr4zHNX-BG~7PU;|U4;iG)L4o9ly+|&vbgadWSX0;Ikjs)?o6Fzt8KSsJiZS>6^8t|M}=MWhqHYOE6q z9h=5Q1+r|mb7+18-{c=S!i8j@-YYjU54~vALTwY4;~{u$!5P4f;vId;Vc@VjSPxE0 z;xf|wZ!jWjDCcB5<4v?7?zXRFa62%l%KT*1z}ZS3_!Nnh9Dq3ZP%_LBUu7m~p2S?P zS)jsqZ`bj(z&*hSGH2&*&-$!b+=8W8_`ZKk7g+O~6ERnMx-z14qH|&C7wNuU485U# zaB(GRZusQ-+{>S*zuCbg<`!ksghLWRD-&A5Z8!$WZlW@Ne-X;)PX5X(`!&~n z6q5T9vT!KC)|E7|cQl67}YXON)rlgunEwr92 zQZ1L~GI!@@N#1}T<7`ux=Vmg)nQ^?fH@MUAha~#kh$pk3HGp~CV;zT>Q3(ma%Zw|s zSc1HJL#i^K7lWDe#&a1J+trU6nwosagmk4IBSU2GyZbtd*;i-c-(NT<&UfPgqeh0bRIGw>p!PLM2g~ zSa@$3x@+xZrdaod5JgE#qV7XFg2FggOm_p`4p6h)22UV1p8;L14;6g!r7(i60(56? zm}0*ga@=)Q#$C|XjICZK^SEnWhOHSy(ACv+wMz=h1E`T_+qOWmuu*+rQNH2+;K)2x z54K7@&bpRAE&-M1r<`w1OpL6a?7V|nM@dui_^+kkF;7r`5!GSpVUHsJ56a#tNYf}t z(5^0Y+33<&wr$(CZQHhOyQ<5!ZFbqVt-p5mzuAeI*xBnh;#{1FjLbJr=JUAuu-iHD z<4CG_TGOjb%8ofpyRXxY0lcf0|9astG?V7|*zvGwm@oV@Of?Bup{cGz>RShE&1{>} z0UA_eFt&9V2t~t9V@-!)6b?E5eOtF5p0!!XAe2eJo+wnOUIP9Ls6}pp+nnL!LMnOp zxXcN|M~df^N-M%;z`L!3gY}7NaeDjM&e2I1K8K4bH!)!tXTVG#^|{^AHZ@ z%PQBmsb+~vLOU?V8iu(nfv89h=P>yCl|B%6zB)gzDb4VJb7L55(u|bAxuW>c1-bnP z=^d-f!Shm)hW&fL>g61(Jmudctu52pddJmOxMZ`?*u90ooX8W*Zw3xxyzb%f9M)fV zh8IpU5h)-VCB0xj0bdgO`A+JEvUSy8R6*8im&ip=UJ zJWq-@e<&?W8Q^6VPjk2U!a%?%>iI#Ik7CtF_I13tpd;Z*6``X?40rmwah$IRmlDt7 zRik~Ku?9?FNeuc-`gt{Eq3H}2N=;*A|H{o6&i-UaR#tg*YUV!I^3pI(4!1ishl2Z! zG1o9HH)ZTktdZ?zWzKsJ`l4X$n`kPV`|zS!r#@xm{N!}}8{G}$uJBN3XtG&~$3aHE z^fV|F@ltg)1#LD4$FGs*RG^kIJ+*gL!&vjs%%9IHh436)YMFPCe3^-)9x4{cIN37q zbREk7pQgol()Q}pM%*?_Gc?lWJzAJIY|_Y)3x-Q`XOVK9wR^nJR;iWd*ml~0d*%Fg zJOq!&^>J|}KoE}>dolAnSTb>{PgN(S9JP}CHI(m{(p!j0FAnpi^wb!4^!??;1tG_U z5Rd<+kg9912Dir~GY(`RvXlYue&zb}q*nYX1U9R(#;UT0z`V8>#0m)3Z!uqE)BD3p zs(YK8yLfN+H6qtw%<}gnVO8(f#cf)`iL7Ip_lp!W(BR-OGr6oJXOwVjP3yqI9|Xzd zMnM=wy3##yG=Wk**d1SOQc42GUx(g0u;7arMEw3_@d_;R1`gue(z8fN6$J)xzb;^e zbJ!2~v6yS^kVUH>@)vl@#c3ibFv32yqYRSVwKd7T(mFE*i(}Yg5eLt>SLAroI9)uH zFf|i2<=U_=F?pUUkh^LxMaOa~8gxUeY|hGYhD>8P@s`K)j*^{M%w5Dox#uz(q2nIW6Bn_;PAlo`h%Jf@Jiz{Ee6rv~Nv ztKVvzph}UC+Pro(^#!q7bj87-s0+*>S#R$nHuY9fRrJKJaEnKv=tn$9PO%u)?#1ta zCDk-4u^Wr)LO)m&Cx~>>dnQ@zVb~$F=m>RxQ2Lf)^;PhkCxLL{Rq*;i-)!Q|CB@mW zni9t(V2c|gsvY9ab!fPMKKWjq$QUt>$4%B)L)C;g6K|be~ z2u?D$t#PbI4|93rShN*(3Lbjnp+{nl21=T7FlY1^e03)BlbFLi5jq3j#!cP0eL})w zVwP^INWUwWcUs-mcOXe_S=EypXOsgZ4;8Z7^W@#v4*v+fwYo13zJ%4m)LXhT)_rONR{Q;?uO!X<*Z|brghmznmmmzn(Bup14Bz&- z?c04FHfAQf2zN7I4r%gmMiIIvCW8$KHUAZ` zv?t%zEX0a;Gpa`;Z;qc?!3VupG1_94_>5mP@x}g`(wyNTQ#_ncJLudzepl3wEfO9G z`a%wpa0{OHTfgRvDPXK`$AWbP>3vtP!w7Xd-p`?-Fc9h>C%=9na*TBFFUg~iF9u*j z`E{-~c8Z~`9*=(nL{u zghn;3A1tZ4*_yYvZ=Ek-nYETl;v}1La&tABGz-=4jGdMOx&dcsPj0eHk*$^fJX%oH zZqn#&@7=-h*Nysn(g0~!)de)VEx@yvZi%u^J*)85nbcOBs1$Zb$ zBCCiNCri(TK3AO|)x@&17KEYfT5?jtleseaD)8bKTK`Gd@kgL_7l9aPut`+TA}ML|tk5eQPK% z8#beR;(cg>OT^VnW(DmO|0O8wjMc;yp|_AEGj?iv%6hPIiDr6m93-m7#R0R*e_n6T zdM|-oMfbJzI0-+K4959n@N?DmTsgjUQE1a17)9hK?3LqQyzLWD?L|kK>+lqOI_vtoeEhk=HM?ZN|}Z+bkX0 z^~=X;ewYsSxcN=h42=oIvIKP_?A$xUKy2EI4L~Q-P&`%Pmsl!efLoy#_j(mL5!Up+ z2PVi_(yPFtX>%rd1zKem<8$e&)B`2qF#`2F((mqBaX&d;%ff%kwoIs5OO{^2~*^n}lYWTb`gz_sBPS=;hMX1}ox z!NkG;RTigzu@(V9hY_YCo8LQZe2%Ybv746XcEt!X64E%gyN76_=E=hl-ImArdl;8A z#M=v970M(O!qUJnFJ`m91Mh1i-5na55HIeF#mFnA5A*$Fz3gFPXtvsmvHb{%%GU$;$=Rt`e z$Ga_mCrMCKzz-JHRj*5QONJBkwnaex5D1Q`mQj7tK0U8)P_DcU6$4t|2_qr>4;Ct& z7ZSOozb_F=UcEXxRtJWun(Pi52#S}nREX#Dt@}KcP;S?SsTgF=Ck!h970qzx(>}uX zbEiQdL!TP`+a?}FC!^)xR3H%3lU;BmIQCb;^OkT+LcDaVl$p(l zQ_P;ajSEF1?Z|122!vNFcu;Wes~)E*nYACp62=|45J=vuG3$souI7&PWYov0js}ps zbw5#q4ae_177(NETiyow|=&gLY8cC$;dc|D)j^L6*<#tsX7Dy=N zG?bF+EO3QUtIlHn5t}|`zMVmZAojN|HIo43$VRIkI2Wq|sz&QqvG=L{7GCSLB=y94 zf2t>b@61T1U+%1l07H#97MnhC33h9y>?}R@sij~i=i3E9Nh?WlO|yLRQh&u>r&=Z4 znP>8E9v67sn6bSgE#mM30<8-#9uv^`c>g6ek`}II>^8VgwIJ>W@)&R=9rNysi~)Qw zALB|k@P5}OiMn;au3Jm-18o`tVB_6S#LK#nyZc|m-$-w>RK@*XA0N6}cnDUO&eePf zzg$4pf{Sgo7VOWnS65PBVv!cIu>uIpA0%cT*{H7vc{vp_M+zyz&v&{&7o z`Snx2RSsT<;OdO;cs!K(B%lfr`X3e=gQ@2pH-YdBuIC2Y+*0bYV}(o(xoI;h2jaa2AS)&%g6H zF=N@!*Aq@W7x9Mm8grT?_gr6B3qA=p0B~0)W-tSg(`2Cbp_#_Mv_|rZbV9lC#db6M ztOS76q|JCG*Qli!IEomag}&TvLxd+vA;ss@qCoO4%R5TC`WlN*x8uC7M{UA({{inF zRsVOIZE}_4uT^%5@9VdZE$v)F89k;>?4D~hCBCe4*w(L@jWcbhO#t*?mxXHY=2q^UddB4BHr}}$*`l)hGSn4UH+r^QH<;RC zM3KocDbsl=>DS|pWiSIt_=6<(%u5wpPnFhZSJTQd z1?F>f+GH%`&YnQr;?5ojc^eeIEk;#zTtw@28 zpe^3}^{k}M7wvZL<9*KN>-1{}><;EWSRFf&!NP~#6)%{jo}%0vO1+{7MOOk#g%^x< zbJt;CBlll*l&IlP{X=(VTW3>AEGi+s2t!Iqvlt^56pn3co(4#faKzbZv$N*52mxNI zS>lg^1JBEIU(LzWyM}Y^!gMifqUuxA9JV#pq}l%f^3n|SH`T9%S*lqq?UDF0mHXR? zcnu7j3O*57&$mt{%TxEe?ZBL}RUdXF*6VQpBG5uDL6}5KE!oHtc!$ z92eu#w`KhGMtiBM)&t5m85;8(m*H~YjLdzClZMSjmTDm62$=J_R)F$xz&!T}-c`dQ zO|sJKj;xam5W3+Rzmq?e@4YnM>M!$%q+8+4kQ$r0gsyWM^5ra74cDxPq)*e#+S88= zfW7xkqpTCte}eD~QM$9a8_q4f_->kv7&YFMH>2}_sB~s`JBPAw&+djbS2Vh3NEQ`1 zp8(c$<;?WBeX*3agKWRdW5t38dK7Dp47EC|^)%pI!xd%_76{kw8K;@+N7?@9Xzf#6 z+l%KwjK#dSJRkw++ZTI&_`7 zrnsYJ8V<9yu{}=lEW>5-#p;^j&ck5dByIJqamKw!i|-=VZuI>wqUXQHbiX&n^~FDo ztoeSimCXsG-BjdmztmP;&(6CU7jb>O%a5(C(<4OOxq$^!ry=Wuk=2JO zl6v2TKTaoO@3%uaH?~8KX$!dUbD8;6$N(vQ@DaemG};q-Hjm4h3YM?2I&YMg1f`C? zFs6#lYT*E#e^uo-~*jk3f;j4*FmIqT9SajUFFl*<2pXZVx6mCvQERh_g(oGzAZ)->oUOfA&MBc(GmS zade}>kKDMGk1%#j+9q#V?RRq#gbcnX#3kv|nieEuik(}O2qAoT61X7dSPh*%nV*US zUe9vs<=G%Sg@}w4Rj{~x5KuO;LTkrwYs--GbfvX|*CtbMa`1!#=5uZeM|cLC-dvQ4 z4dpCgQg z-Cl>turLEwPy0ey#`=s1$+C3Y$Zjr}wn_bop_DRau5sm&}#E>|iAcLj#tj9L{Thhv6!WFM!t*)*0xJoQC37 z6SLoMDMBKGF6iF-$f|Y}`IM?zh$J@0XWpik$sTJfhA#2s5>2Cs(*9MfkY3xA#SG1u z5PM_b(Cc%}ck9fzHh1K^(Q@?W_CGIGPH`V-K|DQY@vCTJr@>MDHF-GDC1`$MsnwO> zEj+Eopv$8NF!N=d+fwK6Nm$`B64*qSb{$WfJIo|hz>Y|5rlkSu##qibpvRfJ!UIA!R!kAY@K+pAPU+@FDot)0>*l*Dl)TZ~b|Z#M0;hr}Y`9Q+OHTZ>`B) zhxs?VwAHJ{ar}_#5AyMxZ-@q)ShgGzfurQ+72Fr=EgJ1hr&LN$K*&fPg}D^q@V4`x z$sBP939$fhARvVwGRJ?(lK+q44jWr1a}#qz{r|yuN3`m@?79ld7q{<_yj-%-6ilHe zY2DlysAA5%#4^X+V$JUWY_l(c_Fq6>!+qm9#tC5GHZ>I%eC9GZ)S z$r-2zXr?5|$#WGE!X5BCiX4$v6QI;7=nfdE_o$}Pi2P9a;7gVe{gil?Webw<;Kj+3 z!Rvqsg5Bkc85RT@2Hms7gslPMH>{+TIVK|zP{m?o(|)Ap-omnVF?SM*j zPze-4IDG2JME009MRb1laRw_AeQ^JpB+YS#9lvO!?S+{o4N#rQqn2F#0{qmA?ekv< z;xHWhbM`xlh!cb44Nh#Ww$9AV)D-%vmw+lZjf;4N-P>&uvzTkfVFHSGp&Z#}Mn4pP zXXMhoa>=R{6Zsmrg1e&)TV--E@;~t7o4gh>g)EK59b$4Ux7!ItSm&v#QXtV8kf?qo z;>YdS))}ggd5}e0g&{@}&_>T5qiHrFEyGERsD5Q%vLDEpT}0b#N**UT_~y-_t`5LIWO8<1B;3ERIFcaaMRPfT?{}Z?lyhD@+R|+9dbR@{+Z`xw z16q0XKI0MFfE}|T2Vk*lT%!xUcb``EtH1LlRjYYH8MJ?`l+<0m+FX~|dNOt24i|=l z5YznPb_!L+Az}wHJ<*JdGCkPX1Iim~+Puc{e}b*NIqCN&c@o|(*Vh-RvF!+?(Ov_2 z=eFFRgLufncYLpew7aSX+#{T@_;_}6V%G0<&CNdBUG&q;Y}0MGA(u)vAIq$OscxC| z;#~;cM8UwYg~~(^z-`5Xh`o3*+Bd&xak$``U6|KiCul~CG^|=hO?=S9A%5c{8RLML zMEzDEuPZ|ycNcavT}KPuyI%|b+oAFuxzgrA0Uz}+LO!) zCp89h_psWQZf0p18`&uB;e18xp{3=1RSo_knb4N5Bt<2eO10SF*a|ZAm8qgO`TNgx z&#lE~CLQk5f1tuAvq)pq?82=nh`A+SoyF2j@y|lt<$KIX|3ry+gsjT@@-6LZhwSdT z%x4!uvDmh^hsUP*6`e;^*z~Vs6m;NA?p7IUf+WWb8G5}3r!|p*>_SG@GrhY|MWfHI z(1Uqn)RWGS^n+vIY-B(zu_U@^NnODlWTVT2k`cZvJkvaEc*VU0f<|2IG%bU1J;D0a zWTQ+-AZD#kO2nUZz8B&sXx?U$bHnqTn3LPLOI*fdBXn*pfav}1-F@o99Kq+^d+7G! zD^FAmzIbF~nf}M)TyJHd*V{0!x1)=P$J5b4toM^>+Q1p+(fV8Mh{xT2h(9H|NkDD2 z=4^H@ZUvAfFK+uHHu`TAB^_dX&z47tri_O(tX3?nS5YVVp|Og>GH=cz_uq^>TedW3 zri#eJj&JX|Q^H)ODZk8bVW3;g6(srVXYQUA@MbG0A{kWDn#+l(IIHN6l~$}Exo8@M zCz#rmxNbJLEiV|q-n>6&xNcK}2j}Xk-bOye0BWA~&&(l(mPLR>B;w~K;p3uE5R;qZ zIX|wAo>o5)9I7Jma@6O27wQoUXB`9da(qC{eLD{|m@Sz6d%hv{cqhe*pUW{GTB%0X zX>*YCXybrT$s%9aIAhS!2eSdX;Xf3Y{umJWOoPc)B=5s{o+NT=7P)d$e zww1(6Yw_2@i8%}vfeWvfszl9N^>TgPfu#$vJ5wKqiM`J8NgL`(G2LXh@L(==hvLlO zSN6hHV6}aMax`dq$pc6Izli1Yrde*>^6U|He${InCrJCfsBj_z@~1UL*H@$Ov{%d9 z^M_yHdOhm4VT5JFoRP_6r7Q(r-LPyunpAu-tg(((1nUQ}R7&Vuaci{1*5NEa*ydXR zpF~%mT0;x(DAlOoYD^_&&*rKFKhqYi4oum-sonYINYUMD3(Sr+K#F;7N5+*=`}PYD zzmr)QdWCqn@OV(ZGt&L?E$SpBbac;7uu-%Zq))|)@H)4(0*m67iSMAimy*b~?k_7& zAx@ptWBtm|Z&N}W_%|QZyrn}{M#anY-9bba-O2j8HGViZQ`c{DWBKZhp8a=axy+tP zhiuJqb#~YH0Oinl9rA_Q9qhDEiLU+E4Ii}Dt7=BhXy5+~LaBM#D*bV|l>Ox2|5CX3 zAGz1g!QAlwhVOG$nzPwsL-9VQtkzKWOUmE$UP+E~6YE2<6bwk9%jUsPWE=d8All5R z%)I8iZJCJ1(1=D*?|s#FoMZWpc2w`KxF?5+Zfy8+zq?Bd|NRxxM5hiZE437bK^ARZ z^86gS;6$*r>tn~+8Vsu{n%@-13;|25>h=8_e8^(J5guxYeH6Gpp&aD7S4{5_i%H8I zCeCv}i24kK&S+dNbSMKUypd{>QW1NXovzB51f4;Err~MSav*9-EiA$kJw3tjLwbH_ zBTBmjU z2i3I;`enuZQ>4_*gi~#En<+FHXm!WgzFeXt51HCZC6#d2H%t&#B8173Z=OSmiD9Vz zL>eVCK2m-pwc&-=PW)(g`=o3=w>#-2f6X$12@H6pbn!F-aJzeXnY7&ae5?!-h(=L{ z_?9#MkhF?-xmIY*UO;2fF+|8goun0uqzO~&2@8qpoGg)MRM{SJ&GldlPb0x4GeSmT zB#EB%_eaRHZ9#ZLrdYOwAo&rYi{Q*#qv(;<6X~+`Ote7m0Hk6`l^=L^ke}2lhy#TD z#9xMdGL34(TsNm{fx$gbxs@5{1cM`RA|R!p;W?^O4h zyQqB}fQG97S>C=FA}nFO7KAR3ncU&5x2)s=g%3@|M732ay%h)`n?juFy4AJiK%*xI)B#TsHbgBxu)r8c*0nB z?vVAu&B1P6rSGy#GZ5WeRjA{x;qq#~Tg6?@`NG^f9n%9y1VB}F%Fr*;JXx8_fJ#BeHt(?@yr&AbE{O|Pkud2Ti`q5vMfo7#hRd8B3onG`M zit&0e621dTCAN-`V5IQj*?=|yQ1(zg@e4SJsPI`$|7#VuGlE{g4Mf6r{ z=`=ah1!Na1RjbqI{ai4-(C(_>H+9r>FHU!`?&fYe*z~pGOu7_=`GG$t`m=VZ$fkac zul*UEuRro0c8u3a2gow1ny-;iie5FNeyjifUCAgqO7Sh8kAjh3(41-OuM>@eACX4k!fG^RpW1AKD61u)W5IK~oPWDaa zm!!$VTQg_v&LYlAPb~{hRM-}yk;>Rf}TYCqv;3erJ#gLtlg=~)kxXy6;2(~%~QUIj#LhLY4(D@UDwh(71JFO zWign|;3mvo6^G)5DcC&e$05=Cw5@{`4bRO-Hmwwhs|h~U&0(25?W0gFD#?fs84K8yl&}x`LrWo>G89(6eGjhOY$h-mrZ7W z@~t(3|Ad#nit`jw16xN`I4iz-7rMD+%R(d6(P1&x4KPUF6)WMo4UjVEv_V@zo0;ZK zwts;+AcCeLIj@VYa%d1`ZWLt)=n#_3(%ZA2Ij#9>mbm`g|CFd(%f4CwYLA}IDV1L; z^mL7E4Gr)d)^HVF&WSH16UQZNHwc84hN{%MKZ++aTfiQc59qWVR4Y1|uHpRP<<@=S zUI4)%eya%jYBxORGBhtOCs^jS*e-$0*OwM`aD*asB8aWwPpbEqIh&ktIAx1E^I5GO z`mQM!{AQQTZfT!MgsVvU^yOI09aAhGpwpd_%NB*dgi1sKsRxaWGb%*?Y6eLVy}sjOPvPcJWO!qQpdT;O^^}GD2y-d#i)k(EnXWB^If+iFwp&Y^-4g?SbqJ2?9eFKt%nUR~1*UYmkBm zlw=y?#v7OVje#_jw@k^<@B{g3I_3&%sbl8aGq6p59{UpN?@64>hug{2OhZ=uD?63L z9Jb*)cT7n;4_EctLowqcGPcj!3zbbqpF? zgB_rY#hJM_}>e5C_q3!5I`DSwR}4Bf74uknn|IY{|#s6 z|Icbaa>D=nuFw?c=MUbmo|n}ZY$jO0<9y|E?}}Pynm&(Y`rq|Oyea1;nlSz>n)SI# z%7;*!_|xv^H7jEgkwOCC+2wjSLYOL^ao{xvf5>~vrr^`@!nPUGvRdO}q{a5T5Z2en zz>rtndUb{ij$L+zMf^#1^Gc-4rNC{W%gy^XV>8n2Sn^a9|DJd~LX9eSmE=gDpp!*& zM4;AeN<|aLRw$DgaetR_JyH!=pi!z7_iZ=CKCcm&2Z3c37$2i5Q4|Ln=ovkgLE{yh zjDc1I;6fnf7C>43$tU(tsu}pk(lJ`ma!w#;=hqMNn^cWs2BMe2c)bB$iL_QkwVN5g(&`lKLC5wGV_vPMt8pkZ}%zGcxS>o)4jVBRH3%_F0MCd zFgzIo5&7U$gg%h~A}#{{BXN>F7BrCZQoaciLE)@m3Abf#It3!BiDwX>iV9*PlFz;I zXnTmF4-r&X^jAHKP;A+iGKvS!I7!WD7_{Hc#7QPzUlL=GJTMof&Md+E#;k{g4Ld{~ ziiU4L$PaABMhuM*m*JP~O&|I@S&48Yk}C=_?r$reE=?~76A+fHhW8)e&y9=SgvB)W zmZkNRdFuqT<({W^NMB&(k zCg!es_GU@_v0}o8|HJ1(Urfwgx2qR(FdVo#1@7j&1ut4wySwL@xe5CRIdAgHuXf-n zd)2`0p3cGK;!EF!HyhqBToaEQL*~k)c3zJwjf$g|A*!xV)a{qRJ^Zwp5Htro-cVE* zE)ePT`y5|97`=_iwB3eD%k;`CMq7FvdG8E_6&t}1Xr>e%1QYBI?!&bKU50e@b;_we ztbu0-fJ9Vv(~II|TNFyGCmKF=sdJAjQa zFQnqP6LP1ceH|2@)~YWqUT^Qesj!V&XXm~hT>_VPpuIHh1`S%U)8$MasnL`R=Y~4W zz6D&_%Jg%A95JmHNc)BmzehZ<7fani^oTf%e$YY@o`{uX&pHd_g2$Ou8>+A4oss^I z?)_gO1VQ9>_ZJ~vq}8PIWK5|qvZaI%#+Q|zI3QZU`4BR2^2MNrTyy#~j9LeE`|UVI zf0i$?pK!!BK86ift(q9{WI>@2zO{@3Y=rh4PUj62B(ZLaA*!*;Z7_dl)`{Ugt$tIW zyA+FA5Xg4qtr~o*62$C7n9kyZ_$ha*MP$C~(bykH_F9@uz%y>Lu35BgU{*cZf8=MG ze-i2-v7*}8q^N+`Cx-Ehv!Dh@R%zvwGrccs)i(h%Mn|(FJD>|@>Z+rN!vk;`oU@p= zo2(&iP{SeR#DrPPF^8p>R?VEga9;6Wf2NF?i$t)8g33F4%oGc9!KAXu!P{HK_KyEX zs-B~t@c}iW+d-Hohl5I74pE|d8*_~d-Yii*1lU?A6!F}gEm2DEBY^w& zd?1P*SWvobB;Sq?5kS|UjKO_~cp_5Vo716NVyMX#{qko!^COj18z(UZGWF{{%@WCA zM-5;s&tFk%9guD~IQ=~bNA`LNe~Ne;bh zSFy#U!L$5?xzT4iTRxZ0*In|ev6OkfqAT=4Sh2yoz|`HO?jo#Y-=w5pvpwMVzrgv6 zN&>zThiR#`O1>T>6w9b%&6fxPlu^S%Nm}nNpLIQbrVOoY^rW~N#HVQbJ68MT^)GKi zb-I8uP~%ingnYp+hRGt0(DpCN8eBoh5b04U0yawe=!Bwi9lfU@bUR)3ExzoOJziHZ%+yn}g z^}XtskK|{|md^L-u#OB93C3?P-sZqPWZ~gc|I3W4)>@>IDJ|@(J|a@K4bHLCURy2$ zhdPc@;kVsh^(H@z_DW~c2TwiVIjud!hXB3evIkZPmEL7QNE#@mZ`+QxkhW`J{3O}4 z?RJyOR8Y)F&8%tEG<%3k#E*J9&nvtDL&kZ8F}cq(s6t4N5D3ZEA6HT`?Fj{t`{oO> z=DqFg2c0D(TD~S_n(ph#f(VA^G)+|tV`h@&33?+VWf5 zCwr`4Ol5Y8(|UmV_s>6A${L4ni0!Tx%{}R2c>;BmKMf#&E=tUG7E_GB&JqH;L%+rx zmnoRN1w!V15?U?0{PHCXA_^?kLjtG$EtW8d9}xxP;MxQlj9a?RdFuQ(9l$LJlo`$3 z`n)B|;!Ro|n+Y?h>O54CtT#(FYVeZpG{#>+MR>9zuG`NxgGm>Q&vVwZ|K7aO zenl4|3C%M(&9!qJjd;e`L}l02Z(!lr620Bm#J(s!7=x19Qh7K?_)~;ATU{DS+#Yuvg|Lzv*16ACQhit{fUjd@5)*tU!89rAlFr8BT+{XR+z>=bOG ze0tu3ESy>6XnTTSaGou|;rAn_GPU)COs!9dK%v`6BGK`_TaAIW2d3{qN$KbyEiPK^ zI~P^4nBDgRnX8*{l+U%Su8mE0@cj^3h;EMnBo#kP47=2-+L63yPMyLMDohz?a#Xu< zcdDi&*cQ{e5%fo;Q1%m+X||)usd)eaM~ZjIl zS7pmNwYb6#s}m$52DtvBrf=TBE-EdDorvFH(gqaLgb&}(?W3QuTT zGGUf1RnK`83>dvLxjfnJXD4pYQ!?mhxUNG=ZYH5!egF>9UW-gTgz^>nMT7AA>AaE; zFMZ#o_IjQ=ErZzwWyZx5=*sb2k!Fp}eS}vj@E&4FM){Fdn2o|E{0Z=A%~$$ulRC-> zW6ifN!b18*4uS>h(_`;w&@9cEQa{rv+eThlFWoaf|}=gufn-o?`WybB{t zwt#P894P`wEKwaG;QJU2u^h7erI8k8gHrXusr#(_M{<%Lc!&to*WtaK_i9>qG* z(%m)>AJZo_@HIv!@?E4vV~T2Df5iu&?csmN@RlFfv%(dV%$!N@wGpd73myDYH>PdJ z1-kwibECB`hhhmKW(59iC(ZwbCfHOvsDk;Im?-(~7@v0_f{?ydRi`wM&AViS+}dvJ3Y3sG?A30>jlGJF@nX z)JtDZV)LD!1Qw`4G`7{BLsGB$9a$IjPuhF;BJhHiZ>qZM&k?qDS9J0lf-t_3+jjpW zp`YwHq$E8}m;L?2KV9MSjNtRZt)O3c1BuSsM5Un7wLcX(Ej){n9=cWBSNV|!vy!K| zTK)sWkdfm>b71lRQIT-2RekbtT?B@C(z3DGwkNF_>%GrMN$AK}Vsp`d}qMp+};`+jDo_4U?{b{jerd z@_nL8_>klGT@!^v#qe$>xzzslqEk?1&mQ|&&^zjwq2@OYU!SX2ht?426|7q+-{R}$BYv7n=5zEeYAC=xe}*l=UBcf7JGflm}h zwGW$<+QL%ej;D>S02^hC=)9j$KX_F{eVsRp?{suzY}JLa(Vid2rW7PR%E9FiOcKI> zveLo6>^h0hQPr$_bHpa?s=()U)nJ8Jh#*$y+rVL&4^&8qoewU}!w^s% z1<3FRvlyi8C`FA;Cf1Ji<3qP^5X*V&ccVnP2s^#jsVjE&Ue)t|3>%85Xp zg!T^HyKrRcHna~GvK`jx<-4@Yd-F%BoDUrp3lLeVYLvaL@hn9vsSgwQrc*XnOIdUT zN}Zeon-O#d9prNTUFX5`@G#KdVz;e308&@~r-MeTd$6WfN2Q~(PX&Wcmdb_|z?&R? z%79fS*^b#d^tP-#(WrjO3}s>Yrzp|wkVXjSV^d$$W+xv@X?9e_$J^+1%Ng|>RdSzM zdsjT^uZ8jtEeeg^f}iol5i~#mgYV?|qT1CuA+$))y|4Lo@BXt_n~m2{V2GsIIGOjWTS#$7Gt?D?y_!8h1nCw^+xivloSZRsEJu!(ktM zClysxMs_xPrUVNnPzB8@cJs^_&!`RYVJ&J)Jd+1~lq{OVsA4(Ndh7qPj;^O`XbvGL zJLzhbwgS(4KQ}GvW6Sr#FFz@a)Mq|PX(HBOJad_-wti$57(Mg=RbqS(e*)Q(pjl3* zNODz~9X;TCAKTdYz{!4};)&?e&1#%9J;k(ZQ*}LLvp@x!Cd*i#h$t&lo{Q6(Ubb`8 z`NenY7@II)^0nI?~@t>GWu2r;!Py z>YvCsl^C1p(lqBMlrP?Ol2y4Buq=;H?N?*5T)Kj_C#C;g-E!u@Yf`ybPSWNr88 z5QeTlQ3yTEi>)*-&6hGEOVl!XN9m0{VZ!lsehfI-!d~w`-j?<1A1;5->y#&BTkv9x zsVSVcU&j$nH!t_67~W;pZQOc;%WNLL0-%N$&)CCWVh4W@0MqKIcC!`FClMB>NBm6Xz zkht99gJL94+#Suy)SOla0v_B5vddjsU?+)PW7X6-9;?`ct4LaWUwqQwDw! z<=U%z>z8LSR8PfXge8TqKRpMTY5EZ`gl;}8l+znon<{4d zBAHAE&5)B4oZu!!m#Xv~(NeGnNk>kB*{6`g&Tkf`rYHiVg7(p$|I2q1MB=2g{3SFO z%oJnZY+mh4T_*Aa`v3IO@jXgiuAl({ZMpn6y}zGO`5$wFHnlw+_e7e1)E)&jisV8C zdY&$cRgWp>A$9g45xRfUq!hNullfrHsELH(4P7j{-?@*SS{6P0`ut2?w8zkkt`C_R z2dvKZv4U^ca^g&Fu(^Qbw$W$ag~9)^LR$6d2P8%lPOH7=ZdT|Xfk{O2Li`^A7Xj+ol$03J46%ox27AaMb|Ld=0#4La%fc%Fh@vB3}{+m7*3|Gbp4 zA9E>SVUr7)A>$G46wDnj=&58O`Z3E8;{HBboPY%4!o1h@O4>oS;(_Bb8iPK9cqY36 z{5eL;O+Wyj?Z^2Yn?*v>*awfaqka>B1nI~>p?SL)A`a6w7|Y2AdDGmKE<7rl<;T*B z7eyClHhUMP84&IY_}9~g?Pd1NQMqpF+O@vj{wEvSW#ww%SuoN21a51y8<);(&T<>( zORpuh()%9#y~)#2zkd4KwM(W4KFP+j($m{@>9Tj}YU*|AJ@RUb_JaDMvkg%D_sZE7 z$OycJsfg&`8U(0!U5)gudv-f`ClDROQ^6bKJX1EbuECMqR*!a!qI-F z>6*1o%S7$Qn)Mz$R&E#QOV7nh&)_VeZ1`&qC&aRr-JW~?=cnHoyggkQ4?BM$@6TZC zr*YdqW5(9HY;f^gf20Y7h@gmX8%P)@?hA-c2s;JXc|T(8?dyXNEvGeAKG^t>E06ns z{+xpN4W;SJ5tpQo&K4tzGk}U3LCz!I&qqCMe;W8eA|Ttn_pGS+F_KI8&u>ZhMz)zN zvnkX2)}dSxu0_uWH|zkLTNxCvxngdK05kbBI(!6=ts{0nT)Rzp~&kKA5Rloc8-k zW4ML4M=vkeyx6(DzOaeR)$X8Mi?TQO@`dIRGTbp*Mw|c6ov_$@`B_k0IM~NHANxda z%g*IU=bpqY^UF4b{S#yA>xh#R_c?8 zkD3A3^M~B1H<^yj3(w)k;$XqbmQb{|_eEhtVfYNkuWAFc#y9pW(gSvlGHD2Y7A$n% zxsgZEZ*9IH3OB$rW3tMB?X6@$_`sG7EW3-Y{7(NJPTc{tyePOPRp@R9v>%mzIVw-Z zq+*!$Mi})qrC7ig0ukg&Pm|orF%=T!9-tZp+E3y`w<@HAxH#C9*?T-i;vC@yjQB?+ zo~yT$zkC5gOb|G3aM;x__5Wk+8-fH2x^&C7ZQEV8ZQHhO+qP|6UAAr8uGh195r4!? z%q;Kbt};%Z{OBbMBQ5V`O)LfvqTnL7og?N70l4DdtZ&_TOP~?Czv>46cwzyFnLP^( zE@u)a%3k*AKn=(9hY_nrH0dVxz7gZ+&;nG&Eqc3Qe5l(K-Oro%FPhoXjTe~nz8=oC zm+2n;6!;wx%(6L)l>`AcT`Zu+UNAP^YfRWX&)nyl^$L6>-<)vlHxH0y``u^pJ0b^S z&h2(wOAh`?4Fd$MxuUU{B;qeNS1Koi2!pVjVTPH;}(AL0PE z1zPSLL$I;jV5|&j7|2B>SrU!lZUyp7%s3;jLC{f#6h0+%X>5M7mW3{6{V=5O^riFo zZzs@$u%V@A20yb~upq8b29y%hNnt&Gr2>?7Iug*n2hd}<(4)pRxQIl$#KvWWg=Lw* z2%nidd}K6Hw$a%pITdgpy3+3<00k};;^h8nhpWon=aD!JR2F#Kz4Q{gyw#=8T!cvC zd~fJr**qb?jF{y8oq$Xw7^L16DDwa#?hV}u2Uu_t-T|7S@VMidCVdhS@FC1 zKozF^a`O>y3w%pG86c1*tzy6`WPn1dLGB1_Zm?owv}4fqaFZpgewsHXNXic<$TLq~ zTiP;5YJhbU_H8$Qm4Wd$TO?46b|u2V=5X^|i=c`^TyEC&bxfa1#A~G{jTa8`D?g}d zRD&^n&_!w^d*HMuq`}Y8*M~*nqXcAS27!{%S~Uiq`1zKBw4-Lxcwbe% zw#+YPM5ne5odb9JPMeVdnC=00{8%VrB9e%#_Ht556Y~56*K7uLe}iLtaLLv|7h{H7 z!R=IYX?%UNPvxxdPSNP1t(O`a5x*M#G%hLWNQDah0yx>G2od`u%Mix_qQsjD`8*%` z&`%qYPgVV2pj4?Y; z;lvUgDpP-t1gH3tIY*Z#-G>x40Fb`3F;a#(&?qEkE*AdY8boh7>)G!=cd~DXe5v9K zA|oIVPOzePYI_Cc7fO`JnU>AB4RcggYNWW6mXl^0&*n0mm>GtZ8bq3vD%gRFk0vH0 z^*G``1z|a!5d{Nv>gXWN*qYB$LNL)3R*#4=2w67GS7~&`BOlxaJOyT79tE*Gf>04# zrk}em!~sx18&{2S^ijY7LPfOHpXJ?#eT7>Ge>6IZLxSGEq@d&0|5B##r(ZLYbW!oUg0f1Ni+=}@?^E_M)zP>>2W9u zjCl)={8pYaJIiX?#W}J}a_x`S9j2h5OqJUTlHqBr|5&_etocrKmzZ?)W~r}8PSR+a z7-0=J>jmKr=dS;XUd$IkXGV72X72K5(zXmGt(HH@;4|qt$1$0naDkCBU37|yZn}?; zp@dFRD9>Bgq_>C0Iz#5SS}?WryKD-fdp`wwT0lrtY!j2UN0dSY1E3-f#6WDj3UW1? zG3<>g9k+`LQIGnBs@Q#fJ}<8m1iJX~&g}2+w|w_!OyAgk0+Uf@(d6Tb&Ao%>Ap3td zUWhhBOGbjqw8;6Nf-@1Xn6ELmr*TNVDJ?8~>Vm@=-Y{_c624C8_)QR_$MO4r9YOt= zgGpCKgv)VVpg5pkW7{|h<>SScU5F_v^@l%^0 z+qDA}augXTczFmDu2CG(YeW~Bu0-=W5QpJ>2-gnL^5p=Q?Ol0zi+ZLj#FuD@Kn&Jg zV|>^^4CyjhaCoQ73fm~df3`~xM2XmCJRcmrV*jDcb?Hy&jlsUsQY8^WE)W-RqF@Im zHlhIo&mJ`mSj5QD2`N)V*FNdv;MJYpZ)-5VUI&y4=$179MHZeLf088S+mKocf!v^T z9;ZO*j>--=JW)`j4KQLAiLZ`oUL)y(LjxTO%tPQM?}}YF@7=WHXHG)Z8f${1We7-r z9YL$0`A`!?a7Nd?JiQ>QbVw6L-g(KvgXi%oW*W+Stik2PB2bWbqxOo6vEuUFgG@N% zY>!!<*!r5)P`S)CpHB^{w*RWG0Xo72Ny_ZpD5N2a29h4K$2V)&w}JA&h)d&O6--`g*!u+)QXiYjm~b=HZ7rwk zV8!gR>=)bn2uI9Gp7<9q0>-XGbd<&@By-Ee@?os9B;##eow(L8{_|l(#}JJcUXN;~ zjaWCSI2D=i8O#g?)n&jw2&_xeoNY{l3VfKv?pP*A8~s*7nKF$*wr|G1_;6mksALoE=fMW7-;NqRl%o_kMCL?K5oy@N{gY^kx((_^(x zA^x3Kj`^%)0gqHewfe(iDHXK?(1>U&<6*LN&;rO+_R~-vDQXQ?Rb1J7tU=+uBGU-u zSfIt!g3NZxuV6o(0FE_hJ7eA7+SW1;9kPn(4KC^gD@DTnyLQ2!n(Rv_o zb8I<-B$1>^pwfQzAxIuEjlvl)OjwnBIIc@DM1n~+w&YZes=p%y^?J@+s*+0e!a7;a zGEHidKE#1{XR0Aa^FCJD&23Brhe6N9lwbEI(r7ndx!JgfOb3&?D)tCfUU{dmzEYhG zr3LQ|s5EEgMdV;pC8;vv!3<}7rQn91iq@Xj>;LxaJ4OMMW;57rJ7F=_>h1?Sj`x;ImYa-da z`;dQWfj062h`goZ=825RT1lD?2qmXRv$e^5$yEhN9L3x9gn&)9#Ie7MQw3JLYpP_* zK-G9LR3>9yXAx!AW;%Cp!D*mZm`F&D8C+(n6Z)C3=dGT*qCalS**~GgIzNbujI=16 zVp%ZG0wZGfFc+67)-YN+Ao@9sxj;r$kJhG|k&>2KM2jkG3qA+7W4v-ICt7f!SL?!t zli6Gyc!V@?RJnB~C|Y7a5<%&aPDWJjJ1Uc4#<>arsvJPddEiAPf2_)mj({q%>Shr* zUplZJ@(oR$hc%jD?(({0A!SJu@~_@lEhUMXDr=|u5s;WOQ2RiI!6IM7OfEfcUw@)! z41awXfc))SNE?0ouMO@mRa{-iM=02aUrym&TnGW7ed#5T-EW zTGTX#1`W96%E-0M-^5yIn6O+DpJts|JkM;LUgRmCYW|xYdlL0`LxY}bUhoGIXmyRx z()C*!u4=7bRkF$niSPMju8OvTpDAm6RRhToLkXirk(lv?IYfNgj@4Vi|TT{ z*w*#t)?5BH-hfI>qS98YE4By{d=?rDX?-%fxPmWhz<^kwCmAuauH**u!?SX+hW8Z3 zY-D23N*;;?w;~UpkZ!Gly+QAjevd}oE5ZTa9mwsqGeD^M8%;o?@6Z-;*`iZSh2D^i zgquyi*6(#BhgTP%0cv}lv0Ti};W@St=V~m>h-C1b7DOa>^O#;gNb0U@;Q>eXm;{ta z>Wf&|hg4JkQb~D!ugOmjOa#WBn}EUD=EdBR=p1dJ)Voj`uN`W zxrWgp>&B_e>{Jf5d=^1tsb_w^n>T)rrr?qf=I6YyBl3!Ny`;w7qatrnA4E5#VjSXL%P>&kou5$z6(xN;M)}c) z%79ON!UF(dvgKgj9wt4a=br%2ok?$78p^M9upTxMwg+fVE7whG)Mhn?h(OntB08MZ5uUz`Q|MPlosWa?^LJ@sEKpwdqVew742PbF;YRWVcV z&DEJCbU}GFm~*iL^Z~6Z`+Uzd)PTl4zv|IsrIP&H`*5zQ9DeXGrD&m3G& zJb79Og;|F3+$X^NK_y;=@>$b)2)1MIn?0SZJHFMEN#piHE!&8UhRSj#6Y;{rir`9v zF_@RbK@~y%S(u%5^~H60)oTdh&L!bY@YW(W$tlb<#f3_7SG!g=X{?#yxR z41q4hzMs{G-nf;OwSyAI^86j{X9EeueTE({bOi#Ku4HWf>R;zAs?ILEEbTvaP{wJi zQdKBPpEH##E=VJ~DysnUN1zH{Z|ZRMudE~Sh9SbP)w#lS+0M0EUMAyU7Ec7?)^aI7 z42)w39uXy9We#Ta2!Cgg@k=1qK;;Pb^&Sb?G%Sh-LU_xO4j;(N9oPRgh0?kpZBDg! zpwTt6sYa?DW?q#YP{KagYk`AeLM$Po+uu|U9~L9_z~?}5foci=Z80^?07;4<$19hC zF#<)aNn#3QC1{haj92-rQOj9`JB|`!C;Ot`VNmmCHaxWH+*2?Mm2{*cUy}4mj&W@6 zaHD-t?9@50QCR8n)%`H#BS*07EUxZ52@z!vTCQX6g!mFI)J=-L?MaurH?NE*obdWa zCk_8o7fDWNu8-rlZbiNEq)j93H9krGl1a{OC(tzmS*)?V3^ttg!QwCMf|}P)Zh6oT zs;=66k^#MShM*)IaJ1WOg#HNuMUJ_|4uvK1UoRj!-kn*a5sSC=g!{qt$(+q&V*~(;h#s5e6BkWNh z?O|AEOUxI&gBakKX-=GCJo>?u0LRiy;H=>w?8(Gzc9CLGq){|YqSFvqY@#l{%!)P+#-I|1<;u*F2bx+73QW5;)uA+n*T+Rz!~8m0rJFo$6AVMRm#k&*yqRlOh7gMD6(KlAh3cBsa`FM^CF&i z$Hy1w7SrZnJkP!K5l40NzY0M5KUp$-hmtEPqj}%T)%05EHPGpW+D;|y6am!g1;Zm~ zsfj6A%;1g@5JX|lBS%QMb3p1HAy6fSKn-)zY2DgD#HGo>7}dIn2}m6~3=tr+{*f zX2n73aQ=aH@4Zok^^lZ8TGS5)&7AdUn6uO5nocS`6z!TE{EXiM%1inRR-FY?F(^_> z625dHAR!_K>g?xjxp7#uqt2Dm&6(ku?bff;lGU@g3hExTQkK!JRLCf~*YCheHl{G# z=phNp;V}_hs3xN^e_U+P83{qO!pWOQGq>fTWHPa7O+_jX1#j&tP>GSDw@$+_85MS} zhBH_=lS{YzEf}c?Z5CH!KV!nR;1BGRx4II*ePI?Ux*x-K5H@V34_kp>eI2o9kO+j2 zz{Y`tiS2{H-)_K#z`Oa%643t+qloUwl^X39Y4Bl0n+fr~f&Ed&42F#Uj>to>LE1?5 zV%m*@-5!ntYh!V?`)YuklEcHnc$eUxVjwa^FSSj63tC@`(dUdRHMmL$CsY9UWr!?K z`g0#xESF;e*Ug26px}KO`?s}#ix?eo%e{EG2h$6db{!*H$M2wjE~kArIuk;0Pz>!P zE(YnKqwo2Oq@nAHS^SHEVG!IPUYNysOf9(^W%sk;)F1o3AwnM|1X;*XusCyWLEdDK zm78qu(S;x!bJp;--0ZBw$dFk=>7BBBcY^eKHe{|au1~T0>Tzh*bbmk8)F@ewf`>B1 zx<%7MK#ongoN5Eb+)RvSSPx z60n|L+f{C%UcaVoT*?X2hv>SRI-21C)bjPW3FYU@EjKFWkB_x!%%I!W2HCA+&bv2Y z#0Y$+oBuQX2^{h%oLm(|<~$%<-s>e*QB%Vv1@{zQ0}^WveCg5k*zVXya?7U8CkHj9 zv+lw^(W{C~_ct>NYl0RNQ(^O+a}9}V;@6xtd__lzPumf}?>|_H0g*v!9;^6sNbd%=>zPuqgO>7)PdxUXh}8M30i&6Oza7))B9- z1XgAb;H;XL2-o9t-;epn<4waq?A|IN=al|-yPZ?5oeTPY$Tq8l<KX z?^=;T#OSc*oK_QO@<39BQkWkK$&V1R%}!;vh5gp)t;>)5;_u?jG?OR5vR&47K|j`2 z!lmASjxF`d#m*9V0Dub7|Cz>iG;y-GvvvC4ysO*Tn~n#fFWY+aTpkG05E)ZNRoX4l z1I!!x`hanNj$J4;EcqMan>*7DbRKWLWc7+~RzR;Zjtu@8s}%D-Yi2FR`Qj%jcu9Ut zvl3!ANpE+*A97&?Z}~%Kl5RyVS=UW9ynh~NKfil#-z^gVV)|Tl&EK3h;m`OIFlSzU zu{;;WzfrC)1oPsIkvR-Td-62OQjpGPwri2E&*6;|9R!0TEs}r+pAkBJ&B6Z}29%_O z2&`X6gD9Ej+~k4;wU1-B!0PA)Mu|Z0c#jRuNFw~(f~Zyt z-eh9rU1+at6AzI28>HBOKqz-WFcO*$77_XBokLPwG65(N4CFL`eQsg7 zWXjPZ!f2-WjLAT}2r03s66Z^QBzWbuI8T1at80o(QVOw50hYJDXm&{qC8a}N^n=B4 zno_mN%>i%WE6|MTvb^hkbMYM*H2dFDG283ovh(RgfMIV2jUUlcHDiYNm@iAYP#&g_ zRcl^N8vg}n*{jR-HM{{OFGr4E%#EbDBVc{mQYog^x@*#S9a&5%fhj!N_^oGi-*-q` zaQrJqtSQ}}6=Jm)N1)_!R0O>Eb^YFEdb`mej*RUov&9jJU<{!12%e5i;IjE=^TsP+ zGj~zZynQeTT3*;uqfvW%V9lxVFhp^tHwHY1#{AJ^cYVFrF?)NSxA29jag@DNm?^v> z68_Vd#c&>w3*fg~M+-S}inn9u#_S;*_@_*I=p!vJ#*2i3E!}*`+LJ{ql0Z?AY)WF_ z^d!WxAh4Bh00MWvCXAE^7_lHmc_As_PN*F|EpmiBJeO}=vRybH+0d_-tE(wkf|P=T zs{?;u4Lrk*djjyTQ!^0XDsu8M$_|1Rv+nwtHKn-Pam$oBc8wa&cvdUGn7&|Bg!f_3 zpO3HQi@GA?Q{x=ikmhM5u+1TEtyhj?ybNK&>qs+g;p98(kEaa zVDf;Cd*O^SX+q^7{iG)P^Z#*GgH*MPz2vCXzPuaK_3ldAONco*p)M^ry;7%-;+ zgD4ZMVlalZTl^PC*+mSb1lOYxihOm|E>7+rLw<{s^0@{7eoPW)*L=iRs-=*>KD)DL zbk|D^5~z!g8$75zUZu=w34tdRJP^VU>T4{ajfzMf&|PwxjC5rqgaH_R462CuD|dNK zo(804NV$5{1)!h| zNWDraXnh23ubr>lSj|{Pn@F)`n2HSak5a>yf5(_f2V$|cNqBn^WF_n<85Qn{Q>x zW0M8u>r1xEF-+7|COCo59lk%RaGjLugjI#F*h)~qk4lYx)36uvRMnh+uxsigt(6~T z%T@4+pSua{6%fvossBzF9?dzd!H|*!R8Ya&({F>!2O;;X&H^P4#p6e?ngOUTw`nUS zV75g``W2Q_y&A_*0VB~CITs6-iCG%Qe?p{bvMaF*44_6Hc4IK_SX{L?H_ZsCjj#Cp zld#NB>?PE+<}Yt;1JVdSYlxPzXSqsi&z)^mT2v&@9Mk0+AnTTN7U;%kmXjf2p@UA_ zSFCIdnP-snh5_;T9mlzY71Q__jN++%Aj zIOm1)$~%B&`B-T$F9la6PC%)1D`})8;$2>%jzNRgbV$%z)%&&?@@m%XL7}Ts2L=E2 zgt%~_S>aJ%kQ8kY&CB$H0ItX=gG$7qiKNc(*Y|BNiOAcJHwJM~kQN>u!;eL?jXw}4 zyu=|Gf@#&{VW$Ko74SqT$wbOFk|q?$i1d z02`51yoJ;l_BKtI)x=DgkJG8)z7sd=D7|@*l)8Uq2Rx8)r7$dtJ}PzC4F6G@0TPT$ zK{hB|yC0BVdE8y7J3V^0lknibLrX(}7OQf`zLt@mlI-sEa;sC-vnpXxrFj5U&`eTF z6CKj)E8G}e34S2+{Tx- zS;}qF0CU+hdR$W1fU3ql&C*?dwtm&+c@yb{m0balHaI3Q$ew7$x5_wkKV{mm51`1~ z114F|!+9LNYO$rP>Wy!^rf>mgXJrb62QGt?-O5wtq`14>!}SZfdGBa`zN1z>IAUN1 zJPOc!Amtd%TD`i4{RcLNd|j9)cAq~IK~hr6U;`dAxKb_IGwMsATd{G-WeEZ5BLdUV zIDAsXyuSiT4XC_RUGh1>^n;1&+fsGPklyrlpP@jn(d0>7XrM!+l9O1apT3COdR1_! zZ`c#@jATJAsxf~&>2?7$STna3%0XsvX` zofb)D?9!!NSrK^2uypVuOJa3YEC^uw<{!IVbdafgOd!@26ojjG8?Ji|rNc!y3=yW? zu61z?$Rj}YkLt+?R#dh*S+%4|QH-ip33XAnTU#1q1iLZ9xCXwkNk==Mtai(g&o>T( zmnZ2SS&2&+rLiAWMPBJQ+Mx>DXm5@N%LU(bOz8Uy7WQCvxa3?gr__jd3k$eG9TklA zpjznpG*jFoS*RnjLu_3YYmHVr^T9eN{pflKEUc2EDu6kk6s)rL2l@-dU%^rR$N>i@ z79a26^7_m7CgjYFlAn6c%CJ6s+y@1Y^y@2qE&Nm-ytgmIPQod-?{G=;RCV@Sa zgd%^_-=oh6OG~M7e1=uEYFd0J7v8nq=1;oIasBeD53NA*EUCJXTLw8paaWj>>!8vm z6|?KdE7B!_W1}dqb<0au1gB=*`-@=NqmP&u2jBr?JbX`2N#+<6$y`|;?ybfg>mLKb zZFA(0-x0f}Y*-Au3JbWE8NKL7kr!RR_|jybKdh386z32RDa5$MvLE#FpPzA?DS(+; zl7EuDwMcv6FEE@+D(1!w(<7W=nYRmyFncmZ1~JKQXh-7qu@di6)?YL{@ZEre7NU|W ziysPcD)Ya`DT6xqB?#!{>!y)z0)Ag<5Jf+tkd4oPS!5k@E)wLQow#|Jp|+YWq8^|i z@)sI-9mE9r?}CK8JD+(R2GG0ywlR{SDRwWu28($j{QYy~1HNU55Mk6QnD|MB>)<)N z3;R&E{P;jW;nDj169hH-xGlW+o>(ML2Z4Xc-7TEf-Gv3wExJ0Z?I z6ZG3$%xRgqabYdF0M*N0$?Wa6vk zmMMK6&a~j78?0iZn8L`nG80CfCzUmo$~O6@3V-i8ze+$z%;FTX#|VS@uTSvcivNffpYX? ztTuK@E-CLcRRN8z*@9qjJwbOsErEU(CH-|D3uTnnIm+wu3rLqy+4k|6ChPZV$5h%s zT}eCZ^*o8sO2A(L@Lpwr)}xeG)&F|@j%U{@@cM8*>TjDaFXppcK=nNWk!xH=Tz$Gf z#NE~)4vtw-zK8!M;_3v~u{i#L9*#+eM1Q*A`$r7*p$tvyT%`9{^+~4-DpJyViER zJ(m`j;?Odo7+7CtDJ#UN9In=2pRfrOuh7V0H3#QEmAi2##gi~_!clZZiW0a)=V!## z^>2c(a5R3JQf+W4f3M=x0#)I58uSFpR3X6DF;x`++W;HSzmi8v ze90(V2|jU+Y05os2BSix)opx}^7kd96Xc_iTC6D>v zfR^^wE@l@0TMuZ6ijLi;D1!HDojN;wDs-yMr#S(?MMPx81vCoo9|)j*;oM;<3cN|K z(dE9|E8J&;zY?HBNVl$Lrl+oPcIL%dLRGFMC44(SKdC!bJyj&6E_iS|FgtUKtzIV4 z3#^nvCDz9${9UR`s%U^VT9guYl=LE9Nq`zMg}m~_T0tN^4T8&+I|%@aa73{!r2f%6 z8|3qn!RRE>^BqY1#e%2Xx+|*crdySgSXdj1M8B5Hr`p@>g@Tch%COuLe`!4xTbFW5 zRIxTIvlW0qj#bq7!R2ECQ#Pt{LjbdZLk+lsIMuEAy>!ZH{iM{i7j39aCj6mHd_hql z*m6cf_hYnIvLifw7ilpyRH6&>cbC0a&J5El46JDVyLDo}E0#V_Q3tARR#ym8>8j=! zW*vMJI*w7{b}u4HRX!rFXIykBYNxk4g{BY)**>zIL*5r_TS83hLgb1$@SXciBm;=p z8eiqm_yczj3L8^hFrsYb*`r0V5{XL`0Xf%*`Va%6EoDPsi1`unGUZBC`kuFM3g+F&} zh$J+GIrCu3E7p%RyXS@;h-QuC+XLjU`ZVWs37!Y86hEg6O+2&3@={M`%oH7`$#-Ch z+G)y6MhlnIQD<_I1bA}C?^5X(d&4#vH=LOei}FZ4VB|NjaGmg#R(0T4$gExqlITna zy+Y{bpBWq=aKiJ#aeaFiAPCOGSH0JmV~;K6H>yhT#l(XH3#zKB5@@ue8h489t?0z< zEKEtk)}N;UG2zbDQYq>yPnG+dPoLAU#pn@|P*$P^!w{)gaw$OakyGAOvWxpsm$eC~ z5JbzL)=1W@ZAEEEN`~?68~ilnI;Z0`;3=4cOIA1o55vpN%zmcLVpv3F0{;}ttApKA zRR`Sp(~AE7Z01-@;<(v?{ddBZ+VM|@NSYt1m$@+_+!zP5{6D|`hKi@tS4^FMGbFab zWKnuA_jT!-1%cpC4g-cFdc%nGef2w)^w}K8Rma+_@4Z}Fb6Wd0R2#r=f&FzLHOCY+ z@?QMUxjo~Aq0i`9hu4p?rZ!u9BTX>RZgxkmT^3f62 zu@|fvlGq%< zTfIgrQ~r2i8FM6Z&8 zlJy3`GSIPf&pF(`@&9vXK2m5|HT;{GZ4>|i*#F(kbhLA|Fg9`gU)qyLG;RJRq9Fa; z=m`|7uMMo<+@L|@06hg+Qpd{o2`FyBM)K#APOONafhtH|@I3YH#wXH75aGHxwL?R( z5Sis-!i|S98}AfA%D`(hZy%fQ=@sqzeP63JhdcU(nX*eU%*@lxE@r*nwjH0Qn{K}L zxWTd(FcmptBO8kqBc> zKoGApL*`?0u8?esMT@WqAPR9MCQJj`FlQf;U=%VCgUlnmb7k9%BOIgbANeaLGRo&? z;fP5jTO)#z;CJsgzQg%rz&@g9BO<^fm_~mff(fI@_pLdBJJ~Wu8xGD9>>|$fIa1|R zSs73eu!|}Hxr5+#%}FKSnkPI89BCXmB@;rtJ0SYw`#Yk97x^e_>;o*X0k-Nhz};LT zTw5T7eEAW9rJ-f98+H*w5wQVLFw|2zp6PcFkwdNq-*=Z`8iD{pQ6Rq2k7|EX7`K-s zhp~a@GieWgOvc`95(nysh@|=(5H$lq4Is1qihO0a+26;HZk43>-r0Y$Koaq?xe}Aj zgFBU51cNb^AcO|%&VYp8ej~ED&G?W5(soV7#U-}1|8R95v4~bn+}$9 zA2^bBxRE)kw%w==u_YYmxAc4%%1l#sGU2@SVinelN?gHJi)hn}rCruj$P0#f9&Yi;UH(H13fKh?ri9fYo|-O!K4^ngmHf|?9s7*44jn&hJh3&2A(y~Lq6P;Cqz?jobBZYyt z^S~1f_TAz(y_i{IIq@GRfIXMKwY#~vEIYkB-W#p8gp=^fY9J2nm>5nhM)QgRj*HTe z`l5*P%7ca4tuY%w93{L3r%?2m&#pojUcGIGo;0JI)&suTm0O0i;D~N^`gEHZUH$!Q zYs9qrB4eKz4?u#mRym#uIInEOTxM%i-UOdk!G%vJuct&1B2?#QhNZOlqJO`m>F1d@ z4gQ4qJX!8;Av$>G)z`+LV`SMLO435k0cRg8=g4zdb=U&XvG$6%^UHeiYFf1{z1$Td zWIUWg{%GW80MiCef= zlzVz^sqx+RgGdVh)DlG)6T#_8`x3}zo1Puj{q^#FBEAT~Or_y(R(f#m zfQZ+Lqaj~4N669&p@_c@oR<$s7H1Ghu-uM<2G?PvMy<}tqRj>y0-`u)NfUP%V^cI+ zm}!=GBWEIJ0A!N!Nb-21QaYK0t<47X8j8ja4xtoP62c_&_tgI_3!y1`ij^e^M1ekdibmHZgR)F!hAlI9=z(H z=%bk4pXd6O-N>(o;590QRx<$K_^oQWqA%u$U^KglO8bC$p<+>+% zM&b-97=SXLoPrlwc*BRkoY~h+HfZPpYOaFd`&pW+f@#lBlRoqsG? z_N9}bi<1yjq0A)(GY=JviXEJD1{4;{ks3oB#^X-zcn)PwA$*&Nw~@7(=<_Br%^z|| ztS*=UaD|zv&#Y2&9%Y~AyF_tCSRm$C3bUi0u*bw!HRu(@(H}^- zApF27xQ6W-47QHNnMzVO_D?$Uid48M?c~JY@*;)}uf5w>zOw+EulP|tcOobQvqTDp zd{l5MQ@c@%fQD)_RTdOKCJN9sQP^HFy2UbIG0PE{!ftCP!B)0?A=zbV;EalkXj zQ_(^P(HyCT5$@N?WDvv&Fx{>vwKB}gw6v_NLJZ#wB&^&L^W*k49n`C=Hh-WJiOdID zIqm)oI|+o&xHQS2M^5^)KhN@7S^n&2fM<7Eq$e5l%o<`@te~EOeampjUM~s${ZR^` zxY`;qdXRuF4+`SI6(>*c$T57Z>1UJjN84PRC55XPjR&ka`~GA(jSMbkSvRM1E8VcY z&ITF@7|X;cky{MY+7nOX3}1XEHj0IGeNPj0J~aaCA`9%G!RB)rGeuwbnV+wTNwE%d z9*i|R?#v3OFshto^OR_VX+FFh?7)neYM}cffVYrIkvShR3h4*tA}IibLs$gqoh6<& zEy92@to_^v&`TTEfR^QRc_JKt&ABPOujJ=R1JQb=w6CDKxFOz}lg%#z!&S!TjZ>un{ z8V8Hjib^WKWXE{dhZ6jzR+iK)-jo&H_eS@MMYD;w+R3ldf4xjTx;~3;{~mjobnm*` zizyz*nA|cd)CD=)0BO~aKH<-@%~i)@s8XsDCBlh?EfR5mH8m%r%j)lZ$d?43K^|+g zp3k2|j$Ppj6h15%hGXZ5ms;{f`+ky$UBOC~KrQDQEB(2EmurFWH{WmWU8J*b9)t$~ z)s4qHhQEVHCfo#1-j6>E($wI?PZ^f&VrakI#`gW{Pl|}*3<)4Xikpi9+&_VToqnv{ zs)b|Wrw?}=oK|tPIL9e0=2>{I5-6SU( zuq@}|4h;YZ@(c(70P-|b{htl>3U(U|C?UI_)U{_) z)FvexLuA*}{*)_%f!XpaVo4YWl*40eLf@}kY|b|1gmW#o-!~k8w-%Q0^m5kS0&X(B zK0e;wXoptaG&AR%pNhBgmGkL>5999L{V4IPH{nN zbJQ2*EM{KCX3BQ-%uuK+4pHN_JYv*u=1=K|@=(TOorU)l0FQ0zuv3@K?<79#u!#nU z*VxE{^MIfpjOLuTly%KnOvP1RPMz`6TDDw_!|Yjjb`>^3=J0?zZh)I*ijU7NJ(Zzx z@_SgFO%G_d3G+jZiCv8ki7jQ$544)1`?)Z=aoczuc13F*_5p-|ZYuSM#G%LF9guuQ z*AUlz@+!hSk@Dama)N6&-#(>at}pU&X%atW$;C1KaAmsP9~bvOSr)C+P)Tyt%T zN%ZjiZO7(nCi8scTOeLJlh^tbBR5Lp8`1}6(Q6(?=ji2odY*8DB5VD~%(E2`+_?y1 zMt6Dh_X`>@!l{vx`gQxVM{L+rPI>ekA|Y3pmrR{C95rJxM7>ZD#a5u7aFIESuP)as z&qk)!Ju9Oy#k|*}9jW~!POQ)I0bZkIjt#`>#)inrILu@;8 z?BpLOBZ>=%_#iFnD2I->fMX~^O=?N6bt1E9Nv&!;Kwt#!L>SRFMAmyK^zApzM*Nop zB;TU%k&D*Isx6*rPhpUhtyf>qIGv$x%3>t-g0d`5HBYaUHI{_r&jy_<8({V(ZKZ&w z@ZL;g8Nn39sPUV%6CP@<@d2kzOnthx< zQqK6(z|(s)t@ocNJTAyZybS^X!0Ml`p6Gu(;f8j0&Q8vb2KN8U^RY+$UqbW*>VG9f z8@3FJ402 z#cjx8!0up`cCbkfhbVzBq*x!F1ExR~AW~G2Bm}r5kg3;##)EnXsgGq#;DWhl^kZiPJzN-VEUI9@Xcb%}tL%VLEhw0G-8MzyAjPnZ?(8%Y1vgnDHkM*n-LEXVLk{ z%l$I$R&@6D=Z(1w^whmkpZIrN55SeNfk=vBHw$?y=}(u}@ZsGGe!|e#NE)x#9l8-g z6F30;x-+}ho7-&BezZ)U+2Z(C5YnqZ-sfQ$CW(jKj{CkK(+=aU8#jRY4?9jwb^ktd zb$Wvqd?t*Ym@XhR^Npy`Z^i6QnB358@pQDzzdPReXUq~PKTT}57xW9^_Xh39SUPi8 zW;6;D?~}p0wY)>`t7Fw{D1(n2pL>c-&gTKa8!$H_1n_i?+J0>4n%_S-H}6!pXs+iE z4q%Kn?2C|NWU)BqjP%9DrXrR{gQB?h$BAu>I1Bac$`%Mw5ln=8yd&~x0p#cPYB{?| z9X!K>PdXI3MK{tV)Wr<%^Exd=R8DBEo6o@AxijYreTtmSC={Erw1Gb*7W#6+Mj@xX1Yt)RMZmf5+mTlMgtj-L zGVOt$$m@7Ybo+`eAsF4pm^!)-EEALgR{{wd$cVUL^>nL^h%FIuvaW#(X-lezCZf~0 zIb(i-Nr=GGRkD`?c1EF2(R78jrEc5BhHi5vi?@j0S|KeWh(}JQ!90XPG$<99rI8(> zoN4^{ZwG4RU)yWlPtp@=G?1e-0{b}*wUXa+p|~5_O0}mJ0<_5O`Wu4jseaKeFX6Ia zi-oE4*!(Ui<{0K;1xmovd?9D zpNH$XD}R4()Oh)Db}f4F(CRryz4zRQB2%ko>!{Ciz&8aUw9hdd`)BOjI`^*?EBo=- zCQjv%^K9m`xaxmu+}RDD67`lCTiMNl8i2UNor7p3lCuC&mVI+sLmmArroJ$GCp5y< z_8b{3WMn5k0S9YqW;SssNB5(FSKGza%A^o43-8VdD8|kde#dI@5u0tLE)TXu zFjD555jcg{cov!owIgU;J6E^yKCeS{S}M+zSgv_L19)OJKV)bL<$@cs$+=-|UKLSu z>GWE}Jqq6|`q7$N>O=&I2T)FLxhjc@(SvGKfw`o?oL!3>W6;RLpOsO;S<(fa%9P9X zqIk_|xjAkhS{VM){qCavt*vA?1p-RYxv-Hn>h&avbz(EwWRd5lt)&nM7nE@l;(1J~ zLff_#@;QdR*qtFA#%@mY=Ph8RXh}-5NgGm1pO65uD$?;*`h26=%u?0KiGR#P{JXHX z&d26$<`}~()>F6;mU3o|Iw;i8qovZ8@IW?5ZJkEDw4)2UT7au-?ki=WJwY(WV+Vb0 zQ7g+nn*3!+o|(ZCfal?+0ssw2l{xsL73s@+EQ5=MX|M7v3Qgqs}5G8?swi&>`%l z#Zk~Jm5?))quIhl$QdMidwER1yCLrSJbBGDx9;M3d;^rHpUo9EJ#*rZ)kP1Y6N*@ur1AxcY7l)?p zc~yLy%J2ws2qtLe6yf0}O1B`F*($#8E z1t2Ov;apH?28eMjTwA}ZRmi2N%apq6bz(*5Yj?5d=|vDfA0a$ zXr-va zF>ts*h?sT_00lC&3WX=`jnII?*`KSFfIOZEqQG%dYg6OA#6m$Syh;VYbhWU+E>&-- z2{IG8>H!s)aTE!{AvgJv69RQj;6a0=Oo@o?>_)K=USxA84ekJj;!KyD&Nf9NM1^u< z?c5)3tdbuk<|xr0&Pn|-N+bZ}x0q?dc}9q83>>C!OtdQWOzRqhlm}+a)swsJl>~TO zcHqkNHQkG)osi2Oa-J)E^|@YlyB!TMJcK>XmvqZ|E1EZ6`*kuw81%g(3${o}5JNuH z+m8DN+;$#E7v`q+bwb<@Ff;Ei8u}H(54^Un9!1ml(U0=S7A$E}&i+*AMc6r=|NSD~ z%8w1m*b%l5ulk~O--(5Ry? zb!m`fL1h3t2#Ygo-Rd}FqIZY`nUeUo^Pf0eqGI63mQHxU(ZsaQbE$+8ZwD!-@w@Hr zcKN>`^G1oaO5214^?XqE;uyCL(J%|3a+qWZ<^Ia?cqpbjgWjN`gxk>-C6l?#Z2@u@ ze93IarqUjNk>n^_CXy|XC08Mj+{H`AUs!#CWvO-|qLnG+Eq^@1OY2c+t|BH{&B%+T zsvgsp!^l~~@8KGdpKzGwM!|-qS1}IS#|V9zg&%(5lj>IoDp!qaPo46tr{V#XP_4GT z{{+?793hgnWVo-*o?BOsN+s$v7e5*3$<8d2!dC`N;_!9Xlhd19a$YEh9BGV%e+4=&MsX z%i?8v$C10~YXO@bukV7&U3OD&3@tzj@o??wWSY>#T#y;}jtS+Mqpbbiw(}I|F(qOcNKFSFfv?rrj?yHY!C@N&i}D@9-RGF?dPe5A%+oXh4l*p7?>y5HK0PQ|$v zX=ds(PT$Esq->N%_e5GiWA?qI#o-pwq>i}X<1k*@Zfh0G1=%LaRdh083`u7*}F)+V-P`!;d{l&1_iSn3k z!hr5@>zCLqdAfZk>BiUC-+f#KMtCbvypswhAz zGNun^Rh%J16YAm!=?P#=jzbZ-hKhnd!CoMaXqvd;?jh>?{$V?^TBL{e!I8U~+WE&U z^8Jj|w$Jja<`5o-#SKKN!m8q?`)jAxOc>&<72>%>Fs%U|q`D%#N>`$e$F>SB_7wLq z0L{!c9b)_<6_=~B$1a47+cKz>!&H!2&!Lnx>8bP7l4L2dyB!5zWG#uN6-icupQ^L{ zzE|K0XoF7fwnVrM+mKS{PpWAwx2IX9gf*-kK=7Q^FM!FG51=U~0Tq&|{3}B`krX+7 zl&X6ae746W=+W6ZMB<10$Nf`8)ZJcf{=U~vWfj7xg$8d|<6Tn%D27>)T40Z0MSN54Rb>kD~cr7az{s-Ge|m$xFHPmy|%pSZuKVPj8Ux>uXd zt~0n1S>#fp=W(Q&Yps4|1`=vcPRNKVSu@S!bcDTtk#}Z3RUfYLSV`LIrw>Z%U*Y-# zh3WUs&2A;z0CgFu`C7_%pmco?Y1*8yw&Mam+YRli=U^MUvr+D6kPYxpgY6V$8LKS@gzk4Wmv^d~OWuAVmy_g&`;iARx{I zQQ;5dftLb!n*sG=z(r~b8Mwnr(ie!DFwO*p4PcohdJHxFC1Y;D56U%~jKyDWbYdTV zOdV>PLDS7D1Yk_;C9>{+2~ZeMyKwhbI!=pXP|_mGAqb)Hfgu(b^v)abcCD3j5;Jz4 znxTZ9UF?JpWXV?I<^1JrCum_8@^y&eLDlZx_ql;1izRV}L9!-w6w$F9ypoXl4UE*bJEp%Rm603(d^sB7e{ zxTyt;>6C8LKC028t5d0ZDxXo}~8}Topzcn}ss+*1>gCa6;%g5>3mEfL zqHDmlqi3$1L_BX9&7heU{5`b5WWp{)4wZv z{d#thF($clvM3GMa?g?_?)tvx1-Q%nw8d`&6p z{l}sL+Ta@l`6;OSpAf14Y9s#tba(%|sO&xde>GLD_Crgt{YN08U@=4$K&Dbv9~j69 z_^<+&L9ES~WTPPlh(@6`TLVSBGw${3HK(L+!UYK~LjgUS$McpGeOD1*T|gC^

@+ z=PBv?+%9tJb`s66hlfZY7l+kVo2ALKj027>x!mCW#8>CHJZ~<1?Y5>!C#>A3=CSB z*5(`H9uk>1$C6Z~9tsGN0DkB9xkV|2Qm7#-0KehVntOD|Og4!@sP)Q`Lfe-oNCM4l ze2>cuP?2U$1e2lKOYO@Vra)dxYi*E-#Yvc7gS1A^3@J(gk29f7iZ{qR))%5cWT)Jb z`EvmwZ)M?ll!k<{m4gJBsLHG6Yeqjahq`A=_d8FG;SC$!T3~z=Jj9Pv7>wHu+3vf z5V67F1wq!#t!3Es1NceX!1-~u_Ln;$Ls9Ao-i{hN39tY&wgK0Skm;iX`}gOd*AE!8 zqs#cjw}3oRqOXM1@9;kqmcmS`7{94?0a&`9Lxrv8yn@jQJ8%1kIx*i^X|^z6y7&4` zT6cn;L)L<=VP5AVL4_wl1mvE3bU?ep_=6}g;=*6`c_M|nI_znpT{qp)EDzKX0Y7Rs8OiB zE!VP*D?YeXh?tm0h|Vt=Ds?&rrBoL#(PjO;NWRzic1=a&_-ql|7T>=pp|72t!N`q1 zriq>ZC5^+)p~A_get%K&Zj+D$(*3xaA%Ec{gG^40e*cw?UB*s0!JGEDqR1O03)8ME z)*64G-huGog`TPFby2v3M2hM=VU&P-t{kBuAa=(0dFGc^=4|P z=yikDS)++v@rYh1%gVLD&4a%s$zW|rBGa*Y->k=j-e1+T8lFX7D11;n)#eU&i_NdI z+@nIC?z`W-s9ubh$(spMHY{qh*-u+~6mvs$?G(nK_0j+vBGg{kT(UN*)e4nm7YnnF zwgVUVX(uvqIW%)e)g%kSHC##wq!_oC2HNWQj>O3;wUy{)lT@;Nw_)g1m2#kB`g{6W zrhXnerLO4VKDDTR=AN*p`!(_eeb) z&#U5*EyK8{Tq*s8;dE~`vrIvb1B!-`&?G5B9~wr zIqRF)!#`gYG$OBC{rmKkwNq9>be|}*AvvqWOCsl4p1-b?0i2j;ma0DGF8*6Q&p;xvbjB$GiVKC1X<_qXJbt-65>u*fwOhv&Y+m*?=#nYRp5@9sMrsB(dJ%nPtD~^(?}I>TLPavnEyGg-L}F?L~ZeZv2S0 zd(_6xum;5mRov!ul+f9f7(~qO+EOt1zCWigW z<! zMwqmJYbku9=RMLz&Gg&}LzY*xRF!``_`TF{KV6yu170S*=78wP^Dn(PZF0sTL7h(} z*z~>D4=W~YXO6hd{QBtmwY7)u2s3m6iz?`H5*q2_UC zq(6naDz8!xQ7Z2udjc);U;Y+QdMSoO(1>X^R2+>2+J$XJV>uQvlT@|j$e7;(T8H(A zDbhDgb2jev>cqxQ31EXFkJ)pQ%9!aEpoXGas>+7-QW}DrI`z|1aEMa!W<*-3CqwK! zm8K`rIJegj0?IWSnIoarVU$prP80_B$$I9DBSa>KNpp!dYfAD;@^IdW21GHE0!@1N z>7cxWW(pBJ&=cq$s;CgiFsjOa70-nSko7{z{Xr@Tc8PmGTMROI7V;Ul&~o-KyT1UC z{_2eHg3@wI^`ljZ{_y8@<12q7@uJxqZ&h+1rj}LXz6fyXCMdN5sfhYEkr;R-zcqS+e;T46%7Ny;Ey@4qr~ijIPqm7S-9FoYHdNVC zX!1F~;B64|!G}|k3mc5Z?@BpAn?y7&B#Q+W;6ARO6I$3M=56Yuzyfd+nXipF2DRt4 z`?9=OsHjio9%5j$^rwM>eyvwkgc?8plJrimcy~IS- z&5SPp#aMr~LL7~JwxqR0{Z4oy7YNYr zbmOS|%14_p8<>q@Gyb*fg#!86e_uI*u%98SrwN{~)DHA6RTlOfD_`YmhZ6p4%#tM% zo#Pv+`JS7vYq>jZ=^S0}O0!bb)9#5)+b{&8XBiwZI)_Wh0(_dB_DghRHHr^^J$7(N zHT#?#i=bfVBZhh3MLoEO^3OgLmYYP*t%@f}dfOvEX9ckSEqW zaV6fJ%&%~Yzg3=v-VzQ^B7Sw&hzfVd9pV#6;vRz5&9WMSI;qo((}5<>&S6tDwG<}Q zs(>vMz=xR8q$Ocmq{AP8lP|F)J^EFik>=wSFk?+1_R{BVzt_BzMnJ3Chq>sH!_@(} zjMf_flo})(EWu1K6z(9Q0aD-{GQkGifzd&?N#&7tOU5D5*p4iwR1s7qa+je zKp4=#{wy55#0Ts2jlSfl(ad(;dE}~qrXoh8F0WpQsgfZM|J*#VslR$6y+h=o4-Npq zGBg|qZc%;x5Fu{QWiq;;G)WwwSoAE5dtsm1*Rh&TRJnPvaElUP#H7G~D``f-C8a69v+{h*ho`zP z@~nzhT1{zTC;SB-v5KdPCOz8HU%p|Om@shE8vqhM67$BG`#^+r`N>K?BRZ}wCaCNO zEE^T8I{UV z#dkzBLOgZjC>iuDkhR}A<)&@#dGtaf9}0Z^{C)aHV4w&yLAWK}b2%K>(JUKh=1DyT zNI%usTWbeFfhhBV^g2i?dXDkTTLtlrZ$8V59o{S0d=tm}b&MN)YuAO;T$h^XY3(_a z7YdE%@tJYmqc?^A-=9Ae|C2+)#n8pp+2!9r2}$ zE$uw2lrGuPzo`&o=t(fn%&RoH!CC(-Pi zYHMD+B8#SV_apwBf{PZt?3g0pp6($2Jl!Cu#p8DYlIYSIy_c(bzPvi zfmT(bL`^lExKju&#Sz?_Spco|%xnA=wY){0EWcShz_|%#`!*4?Z7#Mb1K9~ZHxWn& z3rqQ%e*;t6fp+t*oudF1Cj%MC0s6$Rz`)fsbH8O8U%78Qu{mQlv3U!kz^&yH;el|O zPYL9lnEmAeS$r0$|IP2=yP9rkzK~ED1JN;*Qkuu@_HqF+xRLYWA0%^Ti5r?``o$(w zj7zoKo=ft0CGEg&YfkF{9e0X^s(MoWZj@L*iO<*;{-YI6L}RbfeR!dAM3J?ere~YP z2f7}?&2pWC!~KGWCpozCLcgwGzK9IR3u@3T2T8yc1|?Nz4QGxS3|VV_2W42>OK$Y$ML$HAu5YYDIVePcs;`hFu4U-A z8bH;aNo_M$N>I`l>Nc)Js1RYIL737XE=Fo}nIWlKaa38x`89>86X{6kfJEd8hA!JdCwAGJs%QGDMo6Yc@K?dGv!DY33F7XFIOxo6!OTF62U_$IA=P(NcR_$ zkD0V0F?qH^Q9|3)T)M(<8(ZNXCLM*O#m;N}*14MuW05JX=|9b;_10(Z)W$sm-A{e*9g)n;U z9f{~gAs$fu^w6XTufZ_vNhm0B(!*PwJw#X&h`T0vGyBzbxwVAdv_;Q^1g!Kxhs>p8-F|6R)6wQ5Q8`w zXc+P-Q)03bz;c@NH`YYxDKC5sL=-pw&93U-tWIvpo9sde;Po=46v8A=C;&n zCB-1;%B|m_TnG;T&P4`cG;sJ|$2hF*=;O*-0f4JSZ1_0oX|JMJGspm>FffZ41Qxco z8ElD;mqZPD+NOXFZmuYRoISJlT0Zd6DIIHCAv)Wp&H7p|{qgy=R@JeJgzSW`LTQ~l z`v@fI}X^r z_?2t@&>znN<5(`fiXb{~PY;V8nBdz+DbYk+QZYZvMg&sRqAT203d=7PJUTiFhp3}~ z)9`+%3`alN7_OD@)G%i9CP1{tfW;3=|JOanc%9IX1;dZUi6MFjrM25B+c-p|IuTdt zE!D8JvG8B-*o=9w186#g5*Nwzi5zkcnED15zA?VVn2-e07Dt|Sl1VaqT*2La=|e^g zOk`%U=Qr;`0_d$fCI>2WiNS~q+LeqS)t%hqn)~5s6BQnAhTx3oNZ|Q44xz_B^HQ6= zngTdm(g58UqD2kEn{WoB6QOc7Pp6*{H<)pwUtV><_qJ|dWPEPqUF~4w!;I1&E^_REIl;k4Pk z(ha-v_jCQ`c1W|xj_vs;hGv#VD~}8o0AP;{0D$h_sCpAar+?8BPixx5ZLL^gs0w&d1ESz zHwSaJ+bbrw+B#cWWwXJjd7o~N`|Tj)*DH8uDw$?uo_nsN?b=5A#pCJegbo;$UXE+7 zOKv@Pf?e^7Vu`e63tGFn^*YsYIpS?RtrKCoW`p_keF_Fs5r$kV4RZU)&uW);DU%K( zTrq?mXOHp2ig^G~*fdZ<;F)+LL?tQl$I6G&Q6z*YkYCkP+<3IGm&pm1u>RQ{7gups z%@&d%CbUsJ!y^MVk*LXL&mVE&Fx!N8C6eJKBD{$m#3>_*#*4L=gfoKu=x2iJ{n?Dg z6aHqS&^h8Q+~(inwVc7~5Vib>z>sf8tmBS*GxtGF}&bZ{GAli#?UPJ5n_Kkvg6L0 z3UuT&j6ZHK8%K90W!+c0p(kbKtzlPnwdXWejz1d1t-V%nF9gYqFb8(x$<6Nmb$l>& z`~{UY#SgQ7XzUAY>lDNCq6U_91y9)mmT{WqXQ$Z#$U8zQ0p`>A)Nu$pR(7-m8zB|i*}4)uTwg6Ib6nox6XO@1UU1nBuDvqZZd=7K}5h`ik0 z+1>(VgdZS=T+>5bJU5%?r2!)roknX)a*hpb+`?NmfK2*LPS8$aEQ+Z{%}XDlJoU~ zpI@_Bq}q5WX^yxQNA!^K(kpMBDxxv_PAo4hVZF(W1#lKD-wdFaA zoZP1AlDMJWaMgea@-$A_K@+V*oy>4y;ojt_UEDt+sX`4IZMS$va=THQEztX;OPXxHZG>1a^tP>rj`IDe%*~h=*+31?JGKKliS}9Ksu0 z+-hXXmlWh14Zz)ZA<)hxo7HhMQD0z)KuCxNA8z|zzwc6aeif~1Tds#-|7A$4*v&~1 zha>i#&b3-GnRWBJNaap4++S$uJ3aT&C~wiQeedz+I`{qd!rYF?x-+Kd1X6-AK&RTt zRS83G6G1Vo{My6DtcBkG^xN~KRN@$Y>a&$w6P|Ag6D%W6qiG^!g;~e^AcGm_8>~Ce zM`r-Go3_3GrGM$B9~A!Fb1!n!yYWi^;^1|dL_L=ToIh?y%4oxUfPNd)ac0@}3D;;uB zpUY%ru2h|lI*@8FnDC7QIh5@L#^)1J@rkcfq5OA9fo0y_OJ)wNrc-5xql`QPICX$I z<87Rm@Cvnrobi@bfik57Jb>f8rkKF4?CqoSs0MbSt{skWx%oyEd>Y26KAe0jVAks^ zhJ4IKWatA-{@I@PubmlGJ!x3`QzIeP{b6RJ!^?qF#AQ7d3%JV6Qk4MJ3TwhA6OB0} zxQw*Rjmxv&V+wAVY@Ym;tl;30tz=m;oCKY2E<=57*Kud*eL)1MdD8`58m(zZ0|Z;> zH5*wUXkr)xGCUbqe%3&~Sl-^Z<&8??#9|ieP#YACufTzPdijWTIz4_Zf3(^foG;&n z)-wF1L|Emq`8`r3g3jjmGbbGBB@z^X7xC+^XVhyqs-NE?E>@j3O0;8!hFc&QS=nYk z5eJfF3JT(mxlRa6o-n}EE(n^5H!B8S)(2k3QBhGT+9DxR@HOW*q(F@ zl&@|X3)JoGTVI2L&9$Kw?{!$jB0TY+4pzIY|G6ijk{wS}gx zanmSG=8MTLcVdclPJ7nqkgLPthv$7c@dA{Ujwy+7t8-47Ta^_ICE=DNw9UNqT0?)? zZ(PZ*Z_Uu^UMWGSj~GbgaJiZ`(sK?~7dE3eB_gJU&+|R=t?E%qZdZXfhbLEz1{Ks; z+Z24#AoJL@6Mx;YPU5&2v36Ju1ICd{bp%tz84Q7e9$t^n(oRe!>~$s3Bi3o}q9vU=-eZ#yu5i8;xmL2W zd;Tli{59_KUF1rQXr-xp&mt?L&KybKcK8)o+HhtdcBX#x$y@2g^w4)n*{2-TxM^^ok`65afBE3=U{tb9Nu&23$qzx+hMQJiqqr@_jiO&cztvp z32mTAXs&g1L>aj21K{?wsL72AVN{iULWY7@iS7uwu;L z9akUk*N7t&n-Cmo+olZ&64y+etE2PkdQY2lUyxz=iEU|*XG3FlO8&OkbWsDz+b*%afIQpz&nkazK}U{9Yd;$|0gR$&tcTnZRk`w} zY@slj>XqnDuO6Su-a+q$e2{JZs5<$(;6q;LMV9;UK$!SzWtkFhWZ8a1k!JwwB#P&05Z-n8=D zdrQp-Jq7mH-V6r4*!l)q3a)!tMDfKLQ6IYLhz zVVxp6A{^`7E%%nd?7|$_m}sYzn2VW~52;{r8Ml2DY6-_1d*J-VPj_}h{XPqLBmO#A zN4#z8gp$7VW%}OEF*+Z8!O)VMJ_8fx}uE+9K z3qR62dsOlgZFa*Dm?v@!;1iaUmjKE{|GE%#Oq0Bz$@2A5B$3CNF8xG?N}!lYr=8K% zQm)nhJ&Kq56B!Yz@WC@j5T0q+quC!;QQGzO)%YZwjn;!d=1xz$*+_%F(F6K;9wA(| zM)cgfC-E}zxmR){!_L*FZNhoEuu;V0CD}TLS+)a_xxu3fu?nA(Nno;Ds%*BYt$J3X zO1ARMvDaT+_(6akuyasZ zIdS!ilg0eQwm%L#{_V;nRbly;ir}Ma!KSU+_dm^GB8>{9Fn*Yq&7Y2p<=^PW{_ms6 zzv#(wemI!Fq6l9zwHOYO1gPL;6_%CauX_l`hAC-d@ltX&sI*U5Ev&j$Zn%+4_`7B+ zVA6D_VTB}Xoy@acyv&`i0!BJlF`9NbKcN|KUyEh<;4k;c->(^^6QtO0G}45@MKMl? z%jQirdSM$?Vma_R@DBvjsv4Eff)fj(SnV5&8{q@v&H@<;iO!WJ@hKiN6i{F9>2iFYm@UeVmbm_B z&K~ETF*~_p!Nw*jl=|wP)yl*kJ4bm;4qXF(GiKIHw68g~Bj^-|>({CEhDb?s22T&>HLTpg{r!|FAvo zU$uqP^KeG>YsmC?xR% z90L+DG%!#zpuquSVnY$|Sf2jEAXVkDc|^-C52E8032~d}|(x@4e|v{lw?j(QTRMxV|=8?zAKJ`4)zz zq$Vtt(s=Nyrir7oj-$(LaC6&co)q72XFOWcLoyqQ8l%q5reP9+Y;E!LpwrC4xoYeC z+zILKsq00zuJ8lmxW6Ki-2Gt5blI^aTcNSWP;Z=Nz82qcx*=IreWK?!>jKxHnvD^l^Mqm@q0zT|hKLXnKuPbNbXlz>VEzjV+FW$piVm5jyQ~lL@ZlwV1j6F%7 zB0Oxltp+-g8C_3)xh(uU40TP4jpHJYf~O=j>X-s9Xwxs&PL6Q~m8(0fH*_BvqRh;k z%C)V*)L5+oL&YRgYg#Ocz%?0C16Ae9y7kXurEw6YOrjQuJO6kghjaZ_OUDa^lN*j5 zuwD>qjfnCQ(XAU)-tr0x29l3KPz##GrwV*p3^5KJsPfnwTwqO_-@Sk8>5a_yhx5!M zRwL))C*GD#=*;{y^#-?W6szs5qQv*2Kwv*hRxaVv)C_aSOMO7uSz>VdIZPk0oU)w@m&J5pf7CYFMpj{ z=fYT*4r_O#5|94r%U2bz+=clw(ka&3iJ9!v=7bQ3Z)~iN?|>YPh!yvdKL%LPA1`p* zqq-QQ7?bi&CbKSq+-2KZFstm%vbf`xCk$_GpyzGxUDZVdZ>Sc$x@>R9ouXV@y&Aur z9DCcH+*fxUJjaFQmyZkS?s&gq`g-xo`HkUYf~ngXseInES%m(qFfGz;q_!K|jR76`zwz2IaAvvwFKnpVXVqc zE%R*LL_)Z|f;0wny0h{Cf$EBBInu^cN(eDu00nt)jEMuqNEtg z#*|jcTkVvfiRnT)Q$m8P)Q0$~={jzSbehYQfo(lUO|4w9zzRtaQWs=$2__&q>Pqei zc^e+g$fXPvj&fSJHn9XXiDi@`E^O18u(q3>=9P(CMv+KFm;xlH*et?r!roDrEM??` z$|Q*{WS}Tg2TYM7P}owjGyp&+5NZGo%)homXX8?A_(h%Lq){)KyzWe?SPD)-p^l0M z9f~fLaVPu2JjaQdK@wf4-+TXRUX)}YgBi90G!`~#v1{|ta)lU(itYh5yl@%G5_s(V zMI}6Gr_QqinU#Ek8^*n4QM|@}KSSyO2wA@%x4?N6MYxQ{=B33!yv{g(n79tXQJ-k? zVGKetq?PTS+K4++h=r1vaJoh5bLFs$ed{1{f4cqm-kz;cUlzXcv~17SS!((Z&g?Xu z&(G0Y76%C3?>pU&;)1g+I~{VHY$${c_8e?wxOS zku8nR0(TVzs*kQz)&K%6>J-|EKW)qR*TEe(np403uTD zCkA{M3l8WboS@G2hn`79A(}d~GyOUzraKOtKb}Mc1G91W^Yi(e^Y$<-aI7tPkxpb; z)22rKs6~p#d=kd7rILwf2`ygtUJJ}jegjYy62%W~REk=2s=?SPg*ZT6G2S7npxI~M z)NlknjKn#TK-=L{Pb&+B3Dru=2odwz2lblh=7_C&S0}#w-_Pli(|xR}PQdG_1>))o z3lWmoK(sicdkrWe`NCz@a?ai0RU5)f zRxm%%+$8PnSo8>P#@7k(mk_9tel{spkEW`(MfFM+qVlt}A4yG^@Zct2HHptNs*Bz9 z+fq0>B}X3-O$S(rYJvwm>#|~jz_%`stb$F9z0I9~=YP5Hd8o&p3#` zBB%P|gyfOtrRVRHzG>TD*VmycgRjjq%opVKu+vUeNL$gXwsbFUCmpqegD2-_Yhxm*f8;ec}$(2G{AAKLprqE&LZm=o-u5lsws>DDs|XsqGy%0`#|vgeR9<W`0nEVjO`y<oA8iEP>+VFdY$_lb^D?iENnV=QpI-1FLti@j+rIm@rk3Q!ee8Tu$2# zhK(2CXEAM=TYl>9RTc3|7EO#;yPnXmT@ls#e)Q;)%o08M91l~St3If`)wDPo2L?3# z0TAxF6a%hJ+uu6tqT89$#_{BL$Ii+b5u90dC%XW{YCgSh{i@Y0xvWT=pZEzlv4TJ9 zm&Ca=tFpi^;JThk@BWOz98y}2aF==GRA8{`^*f{xxNzI+&j9y4F0#ktnpqP(XoJa3 z!eodd4CWw6%dhlJACcDSRN9$%>(tg*x7A4oMZdK_Fb%0a#T49YukbcHksxo={IR1X zj-+L%i@dbz^m9%9)te1?T%8-bivj=C(VF62xH_34DR*F{!Dm}@w-w}!->I2{rJZh0 ze2$_qxzwH0Jqh<6o5d6m^%x!I@i6mqZEvVUP1R4@)i_ zs!-lxu)_X|4AZgw;PPmAW9c>%^_*dCm4Y&!f>d6a|JIXKpAYvZc7{O#Bdq;IHZ(BP zD%U>8t!Kmmc=zrT%WmNepzm>GF&_l`*8k#5ZBGunXKlY~YHKZ8fZ|=UW+ph3pT12e zR7CG>It9zGe)$!$bSS|eo z^49aE!2!pV#>H5rupKJ`{vWzrc#~+;iIJ+9oGBjR8|lJ%Oe>r*abjDjwGc_J>Ud~9JbKYXv&GkYgS)n^7vz5wByTQ;sk1*= zf*%F|0R6uqNPdivjQ)ilX;!y#+GIofywVF$Ig8jz8DrUm&xua@$-1$UeY}xxXw2xS z3=kI|2qkI*piuel0VjAY#hbY>{hI)q?R5qG4D|d4YY|~h`D$Dt!P`N2z1#gB(o~vA zy?`i1Hm-br5p*sp)WDHvLgN~ot=--3DU6qq?NHJt4mqbpwr(y#mROoO49k=njmStg zpF9C+g+L1-lSX2sC@2uka!4sh9vny+2nkkMMxba|g|5uhXC#z@gjl9eQXhVYDaE=h zy8pA>pU{!ZG`9LyB7HeR6^SVWA=f`TW*J4(mK&*!Jf57&!$bq?Uq}*X0Isg;A3W<` zK8&!oMj@O<{bFEk6Q2?Oy5`MvuALf(R8tCz%-Pw;ck8u4GR-F3;1b7b4Q`JNQ z?ZroU^>%OtxW_Xus0C7;;bnv1xj!G;)Dhk=j>q#4fjMt;cRUYYNL+UhJn@ylHf+Z; zA2Cuq&Kc}p+BGQq2^*yqZ6G7{cB zVQA=$6(j^7kDE)X6W$Ly*g)M?^_?>paKaw8J#`LsDL=?J-ynB`Id;Z2c(0t$2n0jG zjNFm{<5hZh>0mc_AcXtR@@qSoj{9!5HR<1kJ+SQLoT=jT33CJEsm>#~xkP}8AeOZ<#Zy=w54% z2xZR6@8dLl~5kZL!VP0L-r#4v5KI;hOA`$s#1RlA1`WI@G~U4Of24j>1VVeQsr5 zoQ7B|YS+@Y-r8lkvS!oq!5+g_W&KRN%ohulk|4O=m^L+pi;CVG524t!$}S*@!m$h^gvf2#S?_ zhuIBr99yghu)m*{S08Zxu6}$m0mCXb)Ds%ni=K2kN=a8j_D!6jkd<26wp1os-CXMJ z{RCd2u>kXmfwAje37((Ty2v7hJ308eAEdk0S>Wc;pVQqKh6DjpMXog zjnui_`mo3zj;5_un$x{hgK`HxBQM%g+c(XiQ`Fzqq#`HuFG@=-DLGe$4YJ5}<+E8R zt+4G!+tsJ*y`SZ`yXdp|W4|=gdACE;|4V7)8^wQAA4epIBqE#ZiKx_-n(7~%=qvYV z3z4I$3@O3R$RlYwna5_*VOOi`<;=c@w+G3!{0f>j*94@ua6oxpZ&tH=geME1?gN>| zt4xmB&(yjAPSy!-yy=l=Sn3f2(VL4*9+~u|V6+;M$@v5J4q>=8)j9i%rYla3BZbo@ zsyxG9dniWdEs;lNKcnFhlQLHah;`y=afnb^HhbfNHk&wl!@m=|$SL!I!xHmqT84cH zwoTx5*fKYyE$HYcLRy}nsM)NW>P|=nVdhqk@xqyEdQS-}-K?4EHu9vva&=%uU4JTE zaGo0A*wcuwYoc;?W!tfGV3z8PPFg&AoE?7@ej&que&Perl$j~z50o|VK22x%!)tNk zoLot@Hy-=WQdE*wt-X3gV5BIhF!Gps(ub=EsAzuKsb1K_%?_%qD?6A5XCJ z%wIxR#%G(n7E_utH7(}-$r)&!W#cD^x{T||9&BSG8MRTum^mU%!KJ7#j9wtrDeWC) z);(8Np<-H1VO~~QrnuI^;+6A2aB3VDT;*l6_Ux4sTF1C=e`O~h7pscysTG;+)B%F? z);XAFxhAQTtb0A@i|)yk#$l%L@0~J_HFd>{MYk_=;H&3DgjOtKzM8kIx2pzgBYrt1 zKrF<%ctR<9-J$Y%Iu)~l^f)5_;r0I7o$}AM(X(rEcfY4+T2K4O|8<={o0Zp{{>v`p z;rv&SkCnc&jiH(DuW! z#XDw?Oe)h4U)SmIc)pzde0>dH6+Sgzf=wAtGpw6sLQ_+*JTFPtPadvtitvO`#fyy) znabBxj0~DhGowZ|K!96jg7n5S-ZgWqhy2ci?%ie;;wiw1siuvo4`B}Z9`eDamo)*( zjjCO_kkWAJYm^QoVTDkMeQ?9o9HpshjLDH5pH%pMymZF0UZ(~8_XrXh?m(zQ*JK7R zv>K(zv&D1e{-hm+x&v_gDae2u6Gtk8_pyl~Sk>r(opb2PBaZ#`hBDO3s1+!00Y(*b z1|+AhoA~<&8Rm_URD8`b-AF2&p7@tBh@NhaF@dd4O0fLvSJgRn(P@p?OcT(!;*M&l zuU`}ZGQ(u~L+d3iYxcnbAznd`+fWWSjOyBgQ@%I2X(H<=Pi9A;(dHZB}zeW_$^$EdsK#Wgv6#}-?}c)Y>8>0-$7 zNWAPE#jd23wMGO$RY?(4eG+PfKTysT}*@qW1()89GR^xO7`gosgw+B88XKl~cFTVeFYp|#Hc%sL{ao1RqWO}{;8a!b1Rw(QBlQ z$xMoLRo}pwL{pu3%-|-fMo3*vw9~KrN5KW-aYcm5u8<(okh(}fyx7{gK+9ZCgw(B$ zj8>jwoX^PRhC!yAE9{TcNh)$Q4J2lBs#yC@2Z~5MRFf$-+c;1#SO$jF;Vm&7uoZYM z*%Jfao7p6dyM*Mxqb#WXfaOt7(RCLr_MuVvW(|q3GNo5L=~c<9iZ)wIfrnRcdW=)B zd6td2(PXqrn(sxQeGO^XT)1tLx`KvC(Ji@jy3DQ}iv^&0RJts+DWjyVX_MPoH9)n&>D)5QN_I-@B z<{3fMqY_Z?Ae&PM)j<4OaB0d#1QV1Wyx+7F&P)vk?kHrk@?@{;9|@%NoO}ZR( z(kW-TYI{?-JG%WZ~DrvzXT?wv_4S{sAuu6J4do!isu<{NQQ23!Wx{ltovUIJVe3zEr-98lUS zhS!ABhkOIbPd^7!SdjQD(Qf!$h<&DXv@DRqzV5FI>pCEWTC$e_n`Bb&LO6_Co_&Vn zcP$PAVdMHHzDt7II{XQZ+$utb8(K;Jak#P40NA4(lKXFWK~yDswS}mryqQ#Hz9A_b zv?6wfam((Gr?56Z#IoTnjgKjTUU(vGdH-&I40Ls(`?bWjAbt`giCi_2QacczB?5WMJC=@L8}sA-JIkYNrqvD@f1bw+ z=5OQ4axbrZaOzkkZU{r|f5MX%ZHwPDIg|1y7nO$m1GRcQYRP{M@=w(eqBdwhbiuop z;ysFzH-VkV>~<7^Kx3}4A~N9zVwwHP&s`~R^Jcv*D{B(mR@RE3-ybGq3W3X?uCw3V z{AAcF#$w4nYP9JUjNsBg3L;K=PC1%bUPMid1+Mw&0$rBiiR)NsV8EdV#xkj@XHAoi zZ_D#KP959Sv+t6Pz?GG!fjI4g)CarqiELe3f6cD+0x|Z;cf{m0C)t|d$^+=ma+z9h z+doKgG+yAM+>fz(h|(}(n$|PDKf&@2s!J$+ugHDQ%A7D3fYPZhAL)jbF@0Pv)l0`t zb2m)Ftt5!*n&h#q^j_jZ#f)IBxV5py$)>Ka2tZHim>YaoNgvSsmNbOX61S!EH7 z<-Wv^V|7^#`A(B8;EOHq6CGCUsJxlMC$GaS2X3DrJw3c#oQDMvIAN}S>XWTSn_E0S zzSyXnL%3_x_#uo1g=%LacftAI4?I(k0^h}FutUy`WnlDth8|P-tui-c-lclt1!V!& z;xYH3aWyk0j}x?cP_@r z0_83ChXoDx0n?kIa&qt(nWrFzauRwK_WX~GCXv^N0grN_$3ZiDuIVv)AU}y~ze|G;)2Yo$mlZ?*wSQfj0-;Gqqs>>Zzefy>m2Hi;oC*n6Z!%`Pk|qVjLl z)nmn~$KV)LQ-G^-2v?2t+#e&}VpO(&ADT^aMD3Xw4O1F*FG;jX*jd8ZKTZoJeng>` zs6sREcMaxjc?+ONoKom|rPm_pV-fkL11D-<*2CbQ6IxA-Xp~idTPS-&eq$V*d>2st zdgOh2j1G1r>_ZN!BtG;ygMt*5QDL1#Mm?OD&dMWdua6*o2mMAjYlkPP&jljxW{QRD zRR!-O$%E4`)t#1#O$c?!VBCql=@+Oxarce(3o+93Lwh*yb8)e*UCozN6Lb01;XZ`2 zV6Q^svT%jJa)EA-?469Ug6h>L}IW1bGH}7-N1|2CLDW2ZsGQSy(5tM-7G|FzaUagv? z+qpn78<;x3JX7;2EjmOMH~(m|2v|FlR8zq=uwW68dt7xng4pB)7ya;j7f4AnHHe@P zxv@Rh-3q-R0@b`743nbcQ*KG%tdk9Ja@JpbjiEz&oBlxhW-d%QEi4j90sHwfvlx;G zzNGm)1TY360syKw(zU`7uCPidF!hLtABRjW*R8(-tF%c1z5?XXb)1-6b4~JhDO?P> z!Ag1Vi|`SAKP;zAqLza390@Dz@?2PtbUNr71dkFd zLgn#r#I6LJ3UW3d3%CX$s8(dX_L*wB(ACUi?WY1{3)yfUYYTq|iCw(rVpGF{JQ~T; zDGu^sR08`3&x#uObxp3GiEV(z4ll2EWP!jH*GTVE?NF-pE=t*QJM8dGTJN?0n8m-z zzn~mg-WAr~X`)waf&1)|@@Eh#Wbw7F*9-!v4w|+1Xa>*2XyuzCo7Tay>W9>+TM5GI z>Bk{%k&30XvOU)h{DnX@(baB)Z3K_s$MpAJ`w&N^yh zLxTj;(F=1$Iq><`I5ogVcrwX0ftb~irk23%u^Vc>7myo7VQ!Z_gq4acKr=_yLVtz< z$_-2L^@TqdbmD4mN7@AW^|v5?>Fj4IOGW|vEQ>$)>8(9cM20drSyFdrK1^h=PBpw` zD)Nu!mleZ}e!7amzoACX`oKGi1(KAT|jd ze~uP+xI)vl2$Y+C`ViwcuAtYKt*I;;VHYjC5Np$)GyIDAQ)q$0f{RwHC6vJM(oPZl zoXe?P&0Imlqo2V-s)|f$eGXq^i>32s-e{Q%CO<-F)0oq)Kj>+R@3nUKET}MDhYu+@ zZ$xK!XsKsUr=76Eg}Q8c?{K_2KC8gdv%&OOh8?kbwvVOymj&^jvQ5cZ?Bn&#a1y>( zsdw_6clP)XJYn3|BC|b)`#5*y9;1G&8NIc8M1oxxVVhIqNkeG&ypckoo=N_<_wnlha*=aTtszI#$hWF)%5ka5wb)O^^RN-thlCUIM~sg-NTXbc?Ozkb>PD-P4zZsLcosOd&4_lK+X>bLdi#7$F9 zS=XEf%N*(eW|)I8P>RgQAX%0xmivOJ&rhoqyO{%w%$e?^l-ovX6P+e~$($lN{2U`4 zc+h#8N54hn48NvzZ1m!LYE4fFFo#@sb4*!aI5tM!oElWFi7sf%EKma3bsLsbxu5Z9 z>PMqk%7=Ee27Dl&ta4E+9HU3KeV$1*1-h ze7v<4@0PG{y^wzn7h4X0Ap)IxX20;c0aWDrjgtEVk@2M5gGlmuXAgpkxuD}E0j^7V z(5#6utIWd-rv2jK(UyC1Mxh!-@vGJ;!}APeioeiaLV8ydB6{2G^sa%iW;dyH+dz^aGwnYNxs|TvH5p#SDQk#5KZ%8 z0ij13XUP>G2-|o95`@Fi=dzxI%L;#M^ zzjb)R>hu`)NbRlng{7T+QUBLl8~UAzwdeHLC@}K!kst2{miCvW%YvcRZA_JWc(53n z0YeK!sKe2gPfl3R^9{VB{TN)--7~#C-oplELH}-<)qxR7?Mv?l8sT=?#)|npzWqBgYKh z`7tEk>yJcy0dJt^MBy17*rRMn^NG?*;xU$1@h%P2C)S4-%PwuorYA((ekOXe?y2Su zODdpep&iET(R7vIE=9s)*S!v8Wjx;`a6}&{Z`cC$s2N-zf4X}o?Hh0q??=U zEBdw^E_Z={Jtr9JIj*Om`K%D_gdm5qV>^CmY&l1;QLUYV#xe<3jYnjA*9%fRdl+ts zNOO)|p(!4WF#UkJUUke3mQ6Q-^Vt1i-ArOGL~)0aGnCwWIvF_F&M)CBKKEm~Bm7C- zXL{A{={2o)KCwi$*#ST_9G;|5n$BL%btjrCsj2Hd9A@V$%g4qyy4&~M6`VUg9ZQ=& zR}YMvs;4w?gmr=ME6y39x61aAe0w@5mY|a;SXMhwfhj(AS;L0w*=T=ym|aOB#UCHj=q3xz>fV|1@tC@D~&#Gj3LgSent*^BgkSqL>U+(3W>gIesc<^fVC^RXV;3 zaMyzM1~3*>`F*|Jot^F7l?Dp#Nyg5Xj2Nb=8y()$;$LI%T{v^Rlqlwu9L9Pv!G58w zS=!_20f`BCQs*ec@ZfJK59mfEpaA*Y(Iw0>hVxoK^^Oh?QWEh4GKCLO10klcdhw`l z_pmi34Tr07&@wx4J(j3vl}!w%QMKr>_|ChpFf@bjbrMrqPFKg3ox=tL-2dC2C|hwK z{;EJ#4%BuNEEvaARUW)5+^sRIff9f@g1-pdjgifV>_Bazs)VSnG>B8m zc7fLFAZ9j|Z7d1;XNp3+h35&sKB8!QE*$_*&pfWHNEiGuX4!=Xodbk#0$DUE7Khqp zEB~R-Mw{nNDXRQUJLSCZ9Z(`=xOdY)bFJYvE^2O|fs{NrY&oW%09%&j$Mo@Rz!G18 zwKG&1eMXqu_O?80EjaonHRuhST2lx)w=trhK@`ovnh%Qy%?>&JHlC@b| zU@4ngNrzDEQDTh)KGB{5XzO<6p*>)R^b&vxk$KZ^{hx#sUQ*B`wy87%Pb(2;WaAhW50zVkiDPkzukX}g>g1G zAzL8doihEdahsB-+;{U57tqem4UAoPJTSwK28HvPR$^JOuy0Z+BfOJ7v0RKC%*b0d zr%}IRyfnM%N~af+)<1CHHmXPf$do=;-K*G4L$q{<8MjI`NLdhSj$Ff46!;x%_crY( z^5$HZ8Fc`6!$^=)1_A`OdQKXFQ^X(d_oIWkm!%@D{2-0xk<-(?Gr6w%xWiu%1S{AD ze)JC`t(}()(u6V)3jGfPrcEAcgA#z`a96h~pe#UvArKZU@r5H7R-Q z7?dPKQzWthYr6#s+$rlb8`Dy^d##+nCV455KBOien*>BCRKq&#TK{^A<)SRa2Fsqx5$GS=ty44 zxnj0|1LTr&wO^7((-Cx94HMMLwUGQWhq^eVDUWR>`X`={8Y0sMbQxDJn#ve|;F->P zub^B)Wd&U&&~DW*L$0GqAC5qXy;meuSb+iu|Jz7f+nE^hb6U;C!RB$}0Q6}Ug1S3a zc>#9Ctz2JQQwloW{B=p-`MKCXKWme!YVr4vS!EysT#dauKfiU~z@r`qT@&Expd{by z@X_3{7BrN%tCQ2y*WT&rtgqKk4_`CL9s;G(26?}qa=|-OF_X?b*VKEsSd)mV)x;u& z48UqZk}cs)fF|;(vl@to9*SjP^l8PsDc$rkkp{3T)CRJ=g=j9RdxzlAJ4}-TAO6Gc zd4eoM%tv!no>b0=a`AE&LMdJHPRiVhI1h$jj^ULeuOyTOV`XMau%f;z?c^bPDT@es znKDZ(61ZJemY6j`g~SsLu<#|mMj35$R^w0FzM)1rTsLMlRZW4tS&F@KzlC49_Mruo z9XY*vJf4EI?>KtVuh89gyg%AE5&AaYsHB6LWe_a>f>Rbm_0j_?a9L8AC*96QE zJTgkTp`Z+3ug|G?l%^=XbUf;`{PGfcBtyR0Tp}V(#1L5YuUjAQa%H-fIqisWn}%GQ z=wXshjvC&-ca`PQklA4RNuRLElzj{Iz&(OK32*6h*Bx?=n<1$sF*4Jbk&Fhw=PIWwRc z?zg)*j2qg=GA|waT_@u=IoiLDYBV#lg|J=l?b|H2CP?;9Y5Lkpv2IGeYq3tkt$gg# zqQ%Gy_*MC>piXwC>Y|xCXegKxf+Ww^owi^=&70|n3#duR0A|@}#AP20)0=f+ESLVt zyD_w4v*~Wgbhoqzmf`7yB$Aw}fVonxdacCWWOseko5EmpYpG7Cm>#W7-T+S+O1WhY zV(#dxT|VJKc?!!Y2b`*=75i!lEo9Ig#BTI2%n*-Z*t3uu1V_8n8J?b^072!Oj0pK< z*p!!>jI*+y(gF!fG8h&Q>0&YKYQ8si^PizLYUQA?hl0Q~_iGoSm=!f1xsGBQq9tlV zza}ocA5njKO?k0HWr0iD>y*^`*$`dZdXq}1qG_N&d+T0gJyciIAvJgd{D?QTU(J&$5esH$6EU_h!#V58`E zl$&gkE3W$MZ=1@?|24WSB`MHYXv(rDE{&N^wvViK#)D=5Gr^`t-5M1hup^3KJHbO^ zZg@KY{D)lB^mJ$ZdYmNx`Pw4$7F{!7J$gP#x9%;SNt&f8RIzv0quGRP_O`-mX3Y)A zon0uPnn(4B$+{!-rD)=xYK0QaRh?P4sC=K0BAk?h=#$ltc@%?TCA=4Nu8?d6pdox& z%ADKh7Mw`x!Rr1A5gS@6*oTbqqdj7dIhQ$u+cWFuNT0^x2?)w>he6X8e)q>Cjnn=z zsFXNj^cp;N6XC~Y69LyigC+_O&cQvdy_td6M=O)Lrq{*#flgC^T2R7LJB7seULjU~ zMNC!+K9~Deeg~1?J=S5JE_3;P2Tk5l)C^Cf{d^!{s9(Rz59yPha6KLf5^M#5zk*ac_Mx@5yOWj0L)VAMln9|nsp;Op{PstYNS z^~N}DVf~>#+li4Euz#hC-?9f4=ZP=hG|T=lk$4D}JisVz(1IyMdqmNDwnfY5x90D z?oGrO;B9<%Z)UII*PJ+FEvcI(vuZRI5}LC{`1k?i$dthjQ&fuY*i^Qd$xepCn=>Zv z@kTzneqSzl1KT4;dZ_y9=JeQU-zYrmg-R4W-)Dhdf7r~Z_Z?PDJ%V?ZxZ61I2;>iJ zPDejFoDg49@mTg>jk%D~rWxm#xeu~=_8&k|>BwEDTOEE<);#dU)Lj5|Kint#CY(|c zc_#=?&-5Uw3KromARQt zGo*G`jY#3Tm_Z0KLQ~sPVXaHfDSa62$5?|c$Ze*S<=hB`*H3CbzAZfNdo{h()ot}3 zFdULtw{)sn3d6b02_7qd2|(9`U=kaoWbMvnRy*t_vviWlkEl;?D@3e!i_&c!g-1X1 z&$P%}H4G!2cu7`C{hQDP*)REVZfX?ZGCRp=$}qLDpi%>kZ?ht81!`ZeoPms}Qv;)2 znkjf-zLYfg46H$m=CQ|%NK9tbY4hlA$|8bI9$B0`!2Y-n@p z@NAnEz`aKO-q>vXMK0cqEwzk^;b6{C z_Ox!Lq)=#h#6TKn_KMzSYV%Ob1A1Wm)V*)m`P}v$FyDNS>@e@$>-y_p z@vweenX}6^M&7>oIAo&NiHZSV!gTQ862zdZ_ z<0qL^QB(A8Ikl@TSfm-H*Zs79v41^Idcf@?URvczU+7D3b1O#CGFiHmyi#-gHBbRc z>Ex4xXi|8eKdjQjEz28U``R%f6>F21o7VFhRf9kE0^OS^spnfh?i3~*VOkMx)^HnwCs&l}(xJo>i>v*KKb!izYr_J{U-K47O(R?C zpQbq(B7jcjHOwgex$C<1{^I;8056YsXBknL8eV$d!2u-`-l(OzXcH_*rXLp1$imx<+`SexoJv94g@ZZ{5 z`ccyn64BSFHl@XI2dxiB0r>zRbF3&iiCqD51bs;ZkvsS7<5g zplgc-L$42;gTx}sC^11uG+k0`ZsXd6pdCvM&ou7g)Prtn_*)(qV;>QpW~b}oH}(@^ zod@ErxYK(QB)fC>3%MNL!1M{gOi$OcEul%E57^Iv$unw>I(uj!g;irI=3&WQzNQfg zYX-hFjQ!3Si~eg{Oh=du;EiiIzk> z17`1g4qw5d8q*c2UY)LCk*jJRaJCH94bAr%!JUkal0#8o->O)FhT@|a1rQ}qA$jBtP7QE7&z)Bk(~wEQ!1+vq0QUyB_$__8}6o(+U6n?%XS#Z(l8l)=!UwvL* zwCcGceASKM#m%G-@>`C}f?_69mvBw#%3tpkTxQM^RGV{4I|>M ze&cD}*7i-2*9zamHr9CL7>n|2+&qf}wV%P}FMaVDBWWf3_|T)AWrwH?%OdAt+wbZ} zGeqPSlm`_nBIcnnFuA+5P7^*^F+W_qnSb6RGKJe_MN8Qok3QncYI4cY2iSg;ZQ{jA zV1bhI{APfVZ(;Dm)ie#RC?Q#&J47hZ?rel@5qjd{xdYI!N=5G;Tmk5=d3fdk#6^xO zw4^FxrufJ+KI;#neswFQqd*jTDlW*>Rb-Ek^l9lcgA?)sF9jM&<7rL9RT8%IG!`0q6Icrm+9yy9){*I@^ zi`ZtPQA1f4)RYj75_=yn#sl71a&kTwo~|_1h`bC@<@zoDOdqv`Nw+AR1z4ya_{h&Y zjLt8e6vEYR!y!(`K6)n!f(NsB{@|;!TrGspnI=3%>wXI*$ z{8L?or44zrS$(U5jr@mqyHedgrmyxGWtXuwqku@S8f~B=$_0gIa0WE@RIVY<1#o;z zp?d2hKKD@}iB?AU-lMV4WkX@1a~pO$Q1gkdn&9^9|H=gXK}0g5h5`V%r~J=bu>ND$ zWn}O_RV6lt|6S^2TEo+JlMV47Z*D+pL72DQHnSO=8Zdibc(w{#uO8>MGa3{f%{)u3 z2aWh&XWs8=Iot4rZ7X{=f-oBUlhJ*2>nHfTOvug@ar~5e%O8qh;3Vk z`mw!(gEikhvgqA4*B9@CyeK|mM+=cV1w28ga(#&be0FKYu{|#CN%f@-3wUH-a zapiYh_qnpwNW(sn*7yh51lCH?QM3HvmBV+WhPn|;vvp!5hC@cJHxB7ms%IATt!Q^%v$8P4tlCi#I z1ygU;k=q!^Bw7`m6^HJYx9|FHCL?d4>iHDcy>5kxj}#7oi#T%M!FrolCPcEqO^f!V zk^!6n=q&xOr`mwj;kjGO&EV8sR21(33_vvxG#L{2{vCo!J+Z4{oRJ$V#%Nt#)aUqI z#>ds8STkT$@q)*!&C74~6X$RrjZvD?2k%u}G~kwJ*O{0jeWv7O$wHHf-7 z>%4#-JEPro2@}zP%0nFxd9k)o5k3*u%A;SXk$_B!n_C_kfj zGd-3|kd1@RKq#Qqxi(xBjV67k7ipOk#0-v3BuW%;^u0T zI-Mr9i9d0f&5R_k>EzoG=yPK~cKNmr{0&j1>Tr;Nl#=ZD?VJKik(zLir~y_Z%=PW~ zUlpwB=lwofZ6^PC##g?D_=NU@T})g|)rrD4v2twM!S7Ky6PfJ5(r-)aOTw68T}z7S zT(IF=|IxAo2b4)-6Y9U>AB}SxQ@)W4Onr#xl9J{Y^^?yZp&gvcZbQ+4GU6x>qW(|= z%()e_QfNK0$f`Q0#Iw|A`&VURyHZBC{CV6bdw@2R-dIW-?E?q69k>2 zlB!%H&s%lcLGpiYOl#|p)|8#*H^)oGkT1d9fyLzP}gM({O zr23|fg|WVbRsklfK+GP_M>O)6I~F30GSG-~owyC4SFWEY)scobI80+KxR)=S)xM#9 zt21j&E(9khgi0&LQI1|J{}9n4%O@l9HB?kVUdDK=oS{;p=7v1L7nPHAZ`+l|$t(+6wTQB;F!kUrA(N=aV_llJ zuan^h^+~YCBIZ)451f)wP;vRs5NAjd2&ag>LQ&TRnS)-?AV{we+t$`O$x3dp@I;K- zzi3W1fw(;5Q;%H7tFD*W})nD`Psn^p@*k$qFAh*-^{kL=hi};rx0L(CO1^8>~@)vI>BZ7R9 znk$IJ9WclR^7U*O(=qO6kXok+{*i1jQTU}ej;hBI0N^z>Ac%o+cD2sUy~;QwtI8Bd37N zIPbVKyMlq#zFP^I5%&cU(dzQO^+l~SX4e5%uYFrJOQw&(JeQ|Jxdme)IV+S#VmmIU zDjms97#yywA#OqkSG`CuX_KYX;2H*P1dauKN&OEL94qIIqHE%xptVDfqAz5D=)`pGVX?LqutxJ-2q<|svCBY19-O|beS~OZo0i&vW2?t1&^B|Y zO-A~_wPLrw6)oBT`(`9;8|#R-fSIgP3%pPWd}itBplh$ zi3eKfu*qWD+J2IIX+;lBb{wL6OF(s~GUYb$fmj%SX2AW)N$?1W{$3^pwsqSM8JegIp0|^u!h&ekDgCM*h8T;Z$8xh7k^~ZK*sy04oOwLK*(9B z+x5Q@RQ%9Bm1B5Xy;xehy$icO{T;!i^s?Re-^w=~- zc6vM>VYgVhqpA=C?O=w!F15e-%uJxyZHD>> z2wuBcz0c=cMhc~_4s7XWNPb>`msoklydMpZY>^%+PL*B>^~NrjP{=22WP>y@ZSv7%eigfSDl z8Iqe9Ed!D~wBWP36${2Hi1xF+2o6ATMKPL+-nr$l<@FiWfo}s;rAb^^Q=11i?)NYu z?7>;wDJh*C6fHIpFDqa;66UIyIfF{MvnQfD9;_d%f~f#*CHufJd;V<-0Rc9PKsGa>0 zl>@C3CJNmX8CLWPD1Cb{ZDl==95H=mwI986{Kym-ywAER(;DS5{Yhzfz_3f!xWoW& zt~kX&KMpv;0D!rFpgV6xHpwa67<$bqT(sbr;tV)SRXRz1aT5dc*4)p#!r(oATfe>i z_X5nT_y)u7GrPmD{q;W^QjmW`Yv2GHzX3KD;K|_U-{|-E?<>;(p8%VYfupgZvxB*l zyQzcie;d%fR<)Kr{Iv$==rRCh!xk}T-DncYFBAW%B9%ExD=e#Ui0Kw@Id!YvekS<1 z{;fIthIXtOdz5xareuHMw~7nup%+UZ-Ku7ZZ{O10}40-R3Jsc z42y zqTk&1WJ@#q#}ri3_I@~hLy@gP<;{mVNOQDTXY0)mGw1ljwXO|hvO91W+m!?4^R?Kt zRwbwdQvJ68`bwYvOH1eO<7Q%TyRy%OjHAA3ofyk)Gb{e1jbpVM_;0DgQMyS$mYJ zxf-JYVbN?y2@mFvsHiA#Z1I}zS~27#SP2>xs!76SC>2G+l9?4r%DspggTel3JmnE* z0x+*!Hs7Wh{74&=Q@uANB71ZV58*-0so_^R(g+fvYCTpLY2oO)nEi$PZ z_oQCT#+5`6FqJY9so;dtWQGokW^gYh!$J2!dCl^tP|D-k79e&Zf^SPG%p3U~H_J)_ ze@nqEfok3#D=7Dlk!rV5NcBLHa7(4Sr7=myXio^Qt2?frJf4A2jvY*JPfZI>lO{5> zq(q<_eGSRwqu%{sTY}jSY#t8DSfDR98yev;qHqGV0j`M@0nSeOxDx7cnX#&BRbrul z3(m+)z_rBLYAw$Aqo)#dDb@S(vI|g(K}vcLA}XWkJ!Gs`3I))WE+Q+u%iTaWLpnWg z{^B#CY^_dE%ekDqYDEHCpIbqyicb`=#g4g8fKBatY#Vv5iok>eo(Jg72y@Eq(W;!4 zBT;IT4DNZ(>}vfas@g*7_Q~U6dxiN$09ZzB9^!wQ$(?8^*BPb5K1wJmuB-((&0p5S zY;^nsH~c0Hld--#TWtM&QIvt4@%bJ+obaqpK}^^IX+l}yOwW2q|AVo2iViKxwzXs1 z&W>%{wz*^5wr$%scWm3)v28nl>fDF>*KJj&>S49@HeXhobB)=@=>3bl$;8|ozq!$? zy)ftRT{gR3DHI_2R^cnEnlyr8cG=BSkyzjd3)Qqj{HyLYT;B+?uuZf5`ye8jA!>?% z;cLyeE=m*e>#iA)7}7inJRuAlf661KFBP-!NI?UGvA9Kp`^*?m<N`cc|TT5 zI)`;?rh*3gZ^ge+nxz8OD<@N$&vW&6f6BCUCPnY4U8MHI8)}yK)5otkbaX#ZP;b$~ zIL})PlP;JuR2r`L*0DL4n4S>QDi}#})Vy`>*g2n_RLtM!OROJH4f*LES3h(1Md0I& zm&wATx4`m`+3D=t?QHPY z4sQxK4@KOUIaDvzKKN*^p$`FG2Rwn6>8BOD41H+(NhW-qt^nEwu1d^})Fn8Ticf|b zT@f7}k7zqP?JrmpXFO%%nzE)FYy~SR+-*3uFdg15sMgw#YB87Y4t<#$zD=EHU0Ym6 zPFIoz*q&Uw1G_94oD97Otn;8BuWcx6DUy8w{@L$w&H&nj_v81d_!*S`SJ=P*qwP4_ z=-WA({UFZ&cN%lDih}JPJA%)#T2z@Tjbcj8)oxMcVkp0~_y*H2_V8B=U^c{S@1M8ykH}COnC1vVVscfUnp4)3MseBf50(7?qnrh5jVJ z@#N9&mrt`Nj%0KD_2z?(-tdy9Vu*(xN&OxW zyM~!`v{=6YaS959@NjfsI0MSRiF%4l31UN}hbXr09thN;DX!lGt(S{dj5 z-8^#v|BoQ3DN7jG?N6NL&hIoy8(8um40D^vEj@W{E9j5q!<(OGxuDtrFpGb!--hK$ zzs;jAC7Aq@&2`GUWkU00JW%kY79tolMJo_|4ey|{K7Dnj>&Nr)TMNbi(W>bUuQ9CG=8z*UY~FA3jlX-|;$UFF0~c{Sd?PjcXC)dSi8VZLvV{;(SdY zC6t6CknF*PxVvh53d#!RN_mX~v1LPB7$skn;mk^JKsAFyezQ({qB=NrMpFq7i zUJ&{t3Xp!06LYCs8REP-oeAv7eUeT5*gf`<(M5$uK>T%b!aXBzc)y~Uu*6bc*<%_& z@OiAL$9znBW|Q1@$e5e4q!ZlEG=8Xjli|29CZa~6hN!%_AR(*A?vijpKzpHKCdI=K z`#1WvsVin`5TfP-tnCR$vh5-`+z$YJpFZ2(WL|-2!bFyOaUc|=y}bLxv!9f1kddG| zs3!G)gwB(e{bdHSeesF@=9vrcewm%G#=_#@?d{x>pE8T1xRFOJq;_hAhwS?Y^YF~m zEQr*w>qtQpEpPf4)KBEzSNbnwZ5H1kt({F1Jq-&_r4*nY8Q$h{4>(UX?OOP+plxlK zRumc=+gz^%vntjvC7Wbv_w*f`g+Lu|P2ZR)MqqdL?>yD9!0SHHc>R?-_Va^DB4BC# z240h?`@NfO9yLE=X%EuV7tPIoQ*r|2X84$r3y45+(c~GVwYR+YWcT8(kFz-AsDod+ zpgLHu6$bO14}CVdr46)2<%;<-d87jbC+a!f#6KrY@uz+CZ}o`rAkSZ_uoXX(lZ=|d zWf>r`i+S*4X2+wmMRVH@sl(60;BSJC)Pf0=RQl5Cn)~JI-(k71-tqfeYp2tHvvIgC ze1CVi1Rb+pPMVm1Syt6Ae_9O*Toc!dlZq?5)Xk|L2|<;rkdcytE>BSz$AVtS2ji_ZA1__48bI4| z5P&$ojx*DJFSPsAov$@*ETGwjXuQ6kE=SbfULhULBe8OVY=Z0LwSFLEo7PVg@$;D- zeM>t~Xl1SQ80DIAU>1vid_tqqs)rc;B#8nl^2&m@^Z-=-&9?J}tjYPStK+Jz*nWj6 z%m1LlEP*gp{<7uAACN#8DieSr<|_}`L{_6N8&??xg4tL_qwr0>rR`SBDHoXWw+srs zG=kdBFI$)j6xM5^#URC%nYoZG2_*3ah}szE4gq!w4C}jwyxBE_)u|L$ZScJvwQsh^ zm%!yE9Z}_z_j#C_5B!M+(n6Q`+@OF3%N$l9Ub^<=T|8m=L6B87gRHF|Hufx8c&;I) zt6Vxsh0lp4M@-)T$jANdWU@Nt!^w+fY^JXHYl`h#lU?`*vQYJ^wI|r?_#vtq74Eu` znc}V(PCdpJ0y0(g&+t-&@!NZ4OqXZ$DYAh@`c9JXMY&vfmyu}F;)I^F2K)G~;(~rQB^M>a?h~Cmosr~2=)9>{% zNbh?&gmvVl_ZUzx8+k@Jyf`5e{yZ`oh6a#QM}-|#mE0J$G328kT*Y!h0;4n~x4Y-$iHY451^l#;Gc> zETplV?Yi>l(5!c^1-ESGc)EhvjDH0OX*uNRtu%9a;RDccN95-2bw2Xm#XtIuZ-kEf z^Z`%7>dWNV@CH!d$+vz*#TN^tWJN!5I(Pj;;pWYCf&rIzvMU zm;uDs$5h=fAJFg5z{xZy*8|623BTlxN|il6PZyT8W7<}I+F$H${kawEO%)k)5J<;j zpxSxksv(d>h$5{(ii2#$|3D9>XbRgS5rZcv68QVqs`MhISV;hg6P)-rS%C8=*e+WB zB99Xzj!Ouo1`tNNrC(nV28ANAQI{sctiT2RnlPSrb;NmImZ+!eoeQ*Y#!O);8*SU9j6O} z*}kN$y&SqPMrYI7_^usUpR4NJOJwlb5CB+ zSLPO;2e)IM+SIu620*ipu0GfP#HL4+pA6z5o`bJI=m1S#b)`x)vpzsn+z#h{q)(TN z40U?Ln~N@{Xz8m=?@Si^@u0Jw8I#!-G5{y%frZyVov`0mUakycQ`cyY+ksNhF!vt` z@0A@+)CQhcNZxkWG$v>s5MF@X(6GQ+jw3sHp_%>wNS!{KoXLX7BRN1=?O3bl2m6tc z^dSx;vHlQG$_lp{H`KVI@)m{Xk%uqAid z1w*;S68g8cP{s3Cpr_}o-%Wt+I#f#*psh!RygT!B%;9gHccuuY53oAv8?3mWp|jG>O8hah?YvQpxa$`QE#;_3A}b_?hVNlq#Ov?r5- z>Mc>?>ji0m_8N*}@&WrgxGq(E@OR37lD4MC#-_yy0W#oTl) zGbH2knYq6h8FMxu2E>=OT8gf%GH__czxOL=IVe1{h?EB0+&&tsD&SmK#W>v4^36`Km=J6~m$(z(^ z{Jpe_1If~bK)y16G>E$F!KY4ZQ z=R?TC&onTc-%$&ekux%{f98+xWJ9f0GF^O-tHnbN?WRcW@oMQGFm#w^INd&DhO+Z- zEK@$>Z&n7Ku6D&ZC@mqbnRCaedY;&uAg*L)2CX``PIIG6_w9jprUXvl`mwaPNN#ao zC<<~y+$pp7K};`B&527DkH3{3j}O%5&=mWWC>QnBNi$WJnDln>>0qqC zOoQ*dXYyK9&zvc0zwYL~L|d~|?`h!P(YB6&Z7*^A#)?~iTKj_e$D910zf)d?7Alyn ziYY=^3>u}*pBsOGMc@CAga9bFutxk-$+!P#)BjFFa4DNkDy zaqoiizxdT_PdPT3TyLD|ACIr+2NOXl<7_-UOR6i)rCaZx1D~HyO*q&cUHIN_a#yJm zf)}%k1NPVOG2?~z4#!A)?U=whaKIveh^Vrg_>3dzSykt+k{rj)6k|Nh50AvPv@Ge? zgE*;3(ql5ZTeQ;S*p$zSE{mPT>9KXGsvlA;=vnMg>bk+>nX-K=t3f)8*Ekl+u%Z?- zj_MNER(Q!>EaGBpyy@b7m2w|grkewwcI3rwGG?cnDxkj^(OUVezz*MnG!JnqZbK~A2+b2 zWDD~36mIY>yFH@P@|xhQAAfN+Q&sdlnmpGNGFNh`!|G!;Y6c#OO!*q^3laBnF@Z=0 zx=@Y)6=gnx<)G7>$=Dd?>(dTbJc8puy z>~hNn_R~KNi_UFR@n#oU^p#N`EjdzR+$N|NyiA)~iL#l8KgtNsbZVlUNBAL_>vYL8 zlrsC~_qg*8oPO3LRZ;gTL+nkc3}Y5>=bD^j&g7mTrn@UB45xMiS*r+-F=3h-=V>uA z{MJk7y`yH1Qlo>Ma!(FSKl&wWo*WS&x1vF5){ItJU_{RV>^yzn=7*%!(1|64$zZTu z51`N{-bf8qVyR4U65l8BmR65UGoEHbp-ajvm3pdVM^px&BaAUqfQ-OHrydo+J7lYQ zO9($*Dxw2Pjf_HCA@#&Wry*FrV|>!5jWR*(1kuGWi)bE~-iR!Bax)?TfV6y`@EdkY zJ;R@-3G4KFS3jX}dN0Z*ip`L!HLZ0%o1htbfeeudCw{IB)2?1(a@_Lf>CNMr9t$x? z)#c$WVkXGfyKAWVwEuEJbGYN3h7Ai{bcM^^C%JtSyVzShAaHtyOR2i!PkQnK^a`lQ zBmGz5jl?}=s&>Qo^=qCZ+uQ%s7E- z?(lm1<;ux@dwst?>_eLL7jd~6SLjXlPNzA%y!@#S1uk!i?V?kw=gYylsYp5;nODZz zD`fSm#l4Ai^0Q-`>FZ}9uW5VR!Ia(a&#LU2)f@#13c*$L@(!vt2;NNx#vbArg)QPL zY1sfI@<$aTZ=1*XwGt98h@z!8nq(jn#KJ{9?_dZgZ69RX+muqSHwuw1UCBcS!<1xC zLgRk-ne->c#9(r@)Nh5zOS9}Po6o!A*e_}j#xRrBQuHy$g(X< zD~l$KG3A(Mpm6hV)^~lf=hTSVDw$#^=}Bv4(mC$Td*2+a41yQYBV|Qecq3QJNa;9M zo~F#SUjfyJ;H~{

_=EGx~kF-e!AY*HR0@`D*{U$-#nv$o3$C1r5dH_Q+lQHqqA z4~=_o#3Z(OgS*5U?eX9~F!$sihp}A@51SYx-ry-dcWAUo{K5xdV_ZSVc7d)hEAYc^ z9^aW_Iygj182)^ja^CQ!pL)!{qds>~b?6x(kkj6n_ypF7n<%guwqM^vgUX2qat>?sfWKaK3ZrdF=VuVP5Re!IqH^YKnMEimDpAq4S+5Z3m) z=!vq`Ywow~uTp(pWyL=0F4$>t%>~xb8$hv|$;|B>m85`Df~Of6DvkbK+}2W7!u*Et zofq8e2<{76kn~4dHx;@R$nN&x-u6McPXsFC#95ijsa#=`LeV}D7T&dbWT5s9uDd0; z8E4gV+>N7@tI<9dz-`UJm-l{z z67cgm7G*lBV_<6;ZUH1Tyq0M7tfN_*Va+ELBW z^9pYkdwNMHF_KrA`JpIDOK81=cE4$ZB&^PX^nk4ow|KI1+r@crta_X`)HF;Io>HB5 ziy)`Bij;LF?>7deBlYWzkxAs`uc~mNcQa>lxt5iXlk=nVD^$EH>@GKV(G1!@{vEZB z+`73@I|1A8`#I}Ae{ip`J=>JCeGj?>ga&k2OPr~1!+V6h=hrEx~v`OHJHAAj8 zlU<6$-R+P{4Po))39@aXHKp~@<&pRAlu-#LYquRILip|SJY98EBQ1UL&B)sZkzQA(Jb;B!R>sKlC)%fa8{s zTT%Z1&^wJugC)w$)eC3EJWjJn$b0SSjwgqT0QB70jesUTJ zlD~w(`1~5sMiw=cR4_ZpMwl-8Kh(Rmf;QQupl?Zh?RpH;@oCK@%6H^EJ-tpl_ByY) z(JU6k8F1^-acV;p&~NX_+W*1c+5KSe#(%JP6X`kEqCrM7Xr!U~r_GAd2kx0Y6N9(d zXt^g*zsDaE7P3#cRd6!9ZbWbd1*0AtAN-T2;w-^)rF8!g4q4_`5^*6z7*JP!jG>5+ zwys7@*5NPb|ATvH_QSo4{_~sD9(_K~m&*(0hkFOOIr2F(C*vkW)(0>{M&l6`;`$j% zAV3J`F;KhMQ=Ipl>=W+6Mg!HeF|w-%P~;=6@Z60Y?1ueV!v)W#aE<(1pm7CXk?c%V zs2A>i%Lo)L7&zoS+2(K^ip#)O9qfdg*sduzfSDb-6X%xRjUTx|JJha(-4djfdXW7lWPwejO@Pf! zcvpBi{Of9@2<{UId#r1|FXRKjcTK25&q$2!v*!RtEX^P)JllBe+%GEy1TkRQKI@HZ zMhG%QPH%YNDIN7ay8zd707M@jjG)@DjUWMVvn=y6+sIbz=x~9+nC1oR_Ze1`p|}jk z%~QZ_>}cWvdL`;hLPy}!X?psrj{m`~BK z{KMTe40;X6C}B-Fcuo)6{B#4Lxzk|!l+bgj*2dd;E(~tCwF8D4BSHjNk+k{FG3p?F zjX9+IU3Z4$;7O%)G9NdG1Ugx)mc;UUL@-+|68Xg}A{wh`p#wmvUsuUng6%xu8s zWOgelIjIaS-Y%t@rAZUThfzsw2IhxxKXVS_{`KgPux} zO83YBEms-LHUAuZ2S9xy_rX{BOTBJ*8IUYpiTx6>-?@EB33k{>$`eOpR6r`(Z9$*- z?o*Y5mUWnvC+{}EB71_&gMy z<&*FVs%}ksVZ~$)aLh?gyU}#PC_U~ZIx}Athh!yL!E1bn(Ak()kHdu+5ni`Zq5Y_c zozD|#MS#g2A7c3@CmS{))~KQ`Nc9BEpJc##DG0n_%wEmd;yO-*l8hN^7im|tkcoOp z!m^{4){`?8wd*Fdqq2+0r<+3w8AUJqxRE9f%{qSZZJ_;9adiH$Why#tvWmOx;k;FJ z3KDMhe6x_O4GWk3sd$5fjFaS+S~)}`j${7dnu+}gX@%T37SO2y?Nq#2jAV|~OM#v6 z&0BFiCQIWW{Q63&9aY>iT4H;&8KA68%dR`rgaA9Urq6P}-{GNRdaO?< z%2!~6lIOlN_s5uWg@)-p$nNAWakUb=?E?Q=IMRch`}w*4Hzk^f!Pb98IVVTJShrU^ z)DwTe`uS$frLkwS*!(JkuehevcFNS)H_V>YTV3f?bm_|o$-OYtngbb>`BVOmU;T=D zgO8!~S1k9Ge9rAz^qNNmRfOertkCIraSvYBC71^23+1w#zVTURxno@9X7!8OjA?<` zCypzGjpKy0f(P$j&MVHNUDw|3iJZ_nd~Q1YfNK3K!)ZKlZ(A}P!EQ}rx%s91=+804 z#~L?^&xXh7cg5EMt6F+TaI&4P(QuD)V9Jj%MBQ=!TBU93EPSj{|CRRT@(=q-v_*7C zpO@7m&ztzfdpx|e-bxzBS)QwKg6maRE&VQk_+{N5ZuRcj%K-Mqwe7^(i&Z>J<&?)0 z+f-(O;AhEmnJ(f6yC3_*Ta9)i>br(fMT&N;iPwqiSl@Zhe#ap1eQvEUxNdUXVeRj& zyD;?19lk$*mp7bTVxm3_391M3mbc<{kiA|_Dq}=Hm@K@twltKqSS?-k)ARSa8c$PA z9M?^1?)i%6-c4GsL5fem??j)!9g<&k3oCmZ=v)2w9S{mMPqHaEq`!PY$GNQBa1O-M zj<)|%D*WrnHuQgWwXrp@b+V;XQHBHn{$Ent{?CH{7@_lvD#rf_PzCtML-_9({4ays zh@k$tvVp%6iwj2r0N4}<0HFBytLwWu+B+LNxH}mCyd3{inX}3LlMeT1Zu^sJ)`&X^ zNh5V*WU|Z~MN5|XQHpXRb?XlvoUkAga)gn@IPYWn*R!cBU>pGCxZ=VVlhURPA>w6YHlv_3n6AvE4jeW1jIk05+E}TxiE`1I? z93j6$q_MR+y0CQRO#5Gaq^MIGER5kMbmUA<^9{`0j+~-mS#f?l_tKW$W9LoM`ro8^ zW6m5c+`m`eMtQg8dbf3D`XQk6znOSAasbOFo{vp=L1gV>W_|$i`OR(~-!7pt;A{t? zF+JG>B}DP*$PjDD?|)cucj47V-aft{OEMX@0`L*louWztId%Kw%nIf4hJ4Q)FRWvu ze0)5a@CN1r&swt9#`VZvlLE8m$Xw5Yq&BWw5(kKaWN8rsr?DYRfWunN0Q>D+zjsK` z%nTWMfiBL@``HA*%gGkNPBT)u1b=0yZljr}tI9Pp?z}4IrheauA{zL-g5V zpmtKbc6GmUOID1uFoQAZrQJl|c(Yk08v&p0ON_x1(l>A zRdEvRT@`Qty-pyddpi);5c2)B1Y~881ivngPS{Es6bMEirv&e-37-gzd!*+AkPwLI z!PO7~BmnWc=|$_bUI`t-9JXUR?6(8G1P#j!V|VZG~=@X$aGtk#J;Tx0PhAF}VZXd6V&O_nV=$ioFW^9Kaq%szwzJk}PIL|ehq zq6Q!M@|z{BU&*py9ZWF-$qx^?)osl+aS7Z-Gq0T505$k?4-T6V&Aa-Q7;;|zfUm5v z*WOpDO1fBqOwb4@UGGRhD(F!*lGj2}|MOhaOt^T;7$udjYBC`VrFw`2e;+7u+e+p- z@mk_7>qNk^B?E%5q9lX4xc_@r$S%zpYuOXl{~-XQ!8xpwV^|fHEI4V0M82K(0a$nCYu1fR%*Cf)~yuz;Cv~reyes4)Whi7(_ z2!?0OTqQs+9vGgce$qbPimmzF`cx6hG2z9Ow7Bts#R55^+6v3;mm;>?&ts(32eLzZw<{ez zdNctYtpH0G#8%)i)Qwji7E0j4An`Zz(eH~H7< zb(0_gk3#_r{JapwMG%4pI9`r(Aj08-@vQ#@Nv`ketL!0xxdXuxC5vXOnc7zix89o@QGAts%3Z`Lexq65t< z5DEh2Ct{RCw~+EJ;!d$$WTf_K?nT1l?$=!#JZBxLa*+i3(acD|Xoe8{(h)+dhn3s` z`{onL0tj?!?9(A{NeTg5xNn9inx#vN`?U(XE^|vQNY2rLJx~edP|cwq|l7} z%ebVE&skarC$dA!z2QZ^|_)jjTIV4aQD=PJQHm;%-_8tk*N#@7g+L9U)p34?M>Q68%U_ z_i^@%#=E9I*diMS>K~c^^}dS_JBDul=e%lkKX!cD)h!eHC$nDG-ts6_-VQ2nw(TXui00tm(ab{V zlDmT3&fvhIvI`b$B(AjFQzm?eKUPPWqyDh=i`Z-Sj`p8F)*$j zxKjX|UVe6ct-HW60BTa`{9Cd(NC6nINi@{D#y)=Ca=_iSj)01Glounn;`4GVR2NX!+^^0HZTq=Z=Q zCZn)B0{It(RSBd<8tC}~o9I7jqh>(>o*d+L8CgTcqDwD7by%nua;-e_Llk!YHICkWKvZ%mbB{HdstN~*C*`L zIVc?nUt2-G0!g5hikHg?7WnsCV$)6qj3b6vBZTnbzl>)sH3r*lXGg_)#FOf}QwXW> z_H)MVlTp%rl$WL)V&0-pE~N9UE89KJqU*wGAUn15XHE`36PLRnSl}BBDdU;{zO6+O z1ySC?_JRja8$y>Sh>*~}2lc8(bmz{SZ08-Ur_SCGt-Hbo)NDoV;YJFxRMcnDyg__# zZKf`?WADIU<2fUbhAU1~-UZNV0H@uP@7gP0t>R}PU62I_iF0dvK+U(z=RAE$b;-J3 zap1+0|XM2g5sYjY`+@_aKe_&U1d zGs|$=^BDAyHlXm_R_07I+8}y#;>Q)x+BqL40vzu&%rFF#H-qCpjZ*3pSULA+(-wje zjIO>uJpghCyU6v&0VsmM?wD~R98geuvTMr*N7?|zODXRSx$dpml*GfbHr^hVA*ycj zdu;_pzh++0$?t-%lQA`Z_L@K*R;|R%p0+48xpDy3bD5m7v^j_l z_Yn=GZph7fvCN}H3-nTB2)gPaH}-SHAHaqxPS3pyR*qFJ@HeNbjdUb)-#r#R?cV!w z=)zz6urvwOI}fe$Lgu$#WTLd!e)I%>XHzx;@4Tc8Ub}yL zydmWQ2Zr#N=y4Mm=A@)0y1B~}#qkIoXDqKB!8Jdn+`LmGM07`ad7{Q9dlV)dmsXQi-VDv zj2e>Z0Cd3|FDHm4qj^$kgr$k?VGm~-C1%T!!#$3#M3TEMyw2-Xwks?u$+r{!oZn|u z!>EY&%eCCn+JY_2Mi1A;WylgO2$o+cyG9Fqvq(GTWfH>f7~N?@l%T8koKVX!J+_Rm z*W+dAD@rJuJ-gxf?xN8Q$QW{|R;f0BnZ)GXquJ?`FlWoG?o>e(y$MyZSOuE98S+06(VSIChEDNr-!c<$TShUN@ z%6`HBv(|+oMWM;@Gw`E`_+M#Vj>Zly=KnDpW=LJ?2N;9U{jOGbD&VqHA=6O`K>PDy zm{0mwN#EHqLVCsb(_n=xB*x3Xy-me=XvW5J#v>^lws3ryn%t(F?WNblAsYKfv#_C; zo&#Old195SQHo^EmNdRbER1`PG@acIU)X{>Wu|p6T7f3}0KU}O(u)$bCRE?n_`${@ zX)ot7tV@1>y}*BTrT|HNJY6_2>a#PhHV`99{vl4+jMLiCJFK(nE0~Gk6W5b%td_@= z!e&vBann@Ui03nPQPuE%eEhO?IKrFMDWd0rQqaECU6D%uOZ5d~^t*1CXLS*xwo{|? z&FvjD>aqsaMr^sXv!HoT;j`(R#me`D3)NBdg<$a z6I@f!5IIYi1+Q(rie>CH=&wU!VzjrRXlBa z7((C@S$Z7Jf3?A^v^p|vJ_d5>#A=fK^h5Sk+%+-e7CRh@OhGCfzfY!c*@XGJJ_pOX z_*|Fud~{-U(nD~zYd&NxQb9{96v&+3Svn*wkex3G!uS6nTvJ z*}^bVodca$3{u;@(i35k5xk$qFcV=-dL=CMM&1H0w@sCnIk*6piCkpB;T^tdLP$5wD{8pRaTpIRrI-&DgTrk#*!3@bV-tZwbUmoyBImg}xzvS7xYSkJ; zsTE$BoN#!xwf1mhCV)Htp@}CCxdBtZFq3pE&Vaf)z0zAwmu%#SIk?%Je}t@hu6|Al zJl=Yi@KGLE&}^I_VBZI8>}k^Hu@k`I9eXmyLq3hPwn5>l`;)>(;qO7kC*YKF{6hP@ z!UYZC>DOz zRaRP3b^|dkClyD4`|DY;p*;esy$pZdxsX2+Goy zoVEJ?{SW9?7LRe@^L*2b%D|-`3uRdJYGN=AcQglbb3wybQw8xd0r~l?S$sW5kpWd5 z&`uw%nr_)6H>F&^!QXs`{pP>fv__(KSL_XXQxwD*-_<*eqxr0?zkDl|U%Rt#o&0SI zJvPMnLZD#PLMv#4!kP@3Rpr}_qoKwsL4#rnR;>334d5QJvU$-{J5H1Qt&U0So!3~~ z$-b)Uxjow(I`|Z_0(h#|7WdY(n1|IRY9r6aFw_z$WIb5OkiM``!L}SS%a=q>>xWPZ zuW@6A)ie_$}Gb#Zhb6%qv+dW+}$2XU+ z3vaGqNmZazMv|(@s^9+kDC3Hig9fW3>3K+fCX_4anO}&Vhf7Ws^TynqzCwhUKl|jmT!&U3Jb%WdirO101J<)r!_=oQN3H`4Jrd9>|sO3w96b7-hI)L|Tm2VU2IGwdB!I&J`j4#XUY4m`QTiv1E{-wO)9 zRg10ZcLOqN2da}}^FawsZ6)F>$Xq`$;e_TlfH;w0Z;RNWzyMEl1DJ6W>qGyUw{W?P z6`juajU*>*hv*&b!2o5U?*D`Hy1QDK&-cW~lo?hs8=IxcZa~`G)&aL6mG}PQiW#~9 ziQJPVy9W`Aw->Ru{6d=riubC(zK^0UvcM2%hosNsOkbA;!U9M3rvpxY_jp~SZl4L} z196spfqq*J|H0!mLM=V%^f3V_Ix<|5SKPXoSG9&vZrUSg;`4R0yoy+p)|pPRZD-FY z(9>qt!95EFgDrX1;BYRGe7ON|2U>&)S9V6Pf!2*xo?xPQN}WzlWOHMl6qqelv(l>% zHEMWFigrpEk4roDsDE_rJO{qJX*9Zst_szx+KFm$zTnwHr>j+HoB=g;bkKDuNTXfY z(1h_d(z6v=av%SXGYnbvfiMX}hhI!F zqQ{?KOeyLvx4)bUQy`)X;Q|^n@8$3kQO^69`%7CSO6ry}htaaD728DS@2**n4!b5I z3cFe_aIXFS>LYvXZURbXtBRIgx9^e8Ah5aqwkNxlKjUUwmoG+@wSuLo5hg}< zAi7?6`jfGt)ldL}MtAweerD3RPClLYeY@|uQ2Gp>5qjT2u1jdbT3jJ*TO3^~ zzPy&P4BhO9cTJyrp7$+2+C3HFwZvcAemv$ptfkj-bSCC`PnWL@DPRjYy_`q)at$uMm;8nA0MK35rEPzDq8^HR) zi2I-!#1wjgGL!Z#D-S;{iqvZMYh?!pqV~0ifYB8SeW-9Xjclx@w#{mR$&EF93aSZT zF!1^xwDPG5+QM#M#N1h3P-`;s395a4+pm@b;$(kMQ%CBKlue{4X2v#>#y$ z3-kzG*J@ENbiaZ(fdYVAtl685A5ugwV3tVBsqIg?%Mk{rcO z7zh_N#s!+3mf|_lmqU_fQ3S8L=0A`=kDaIN#ZH(`*Oa1HG(0=a1EcdET|Lv`@Yv@H zemTEGa{Oc^wMe>jFkax*&}ON5`{Z2@+p)G^OTPdiEC)@~wImF&qq% zHmq9?Jh4)n^tqVabrzwo%)am&q!eWVI~vN`R=FII@oO!CPw;Sraal1jb1Iwi4wtF& zi{rXi8tR|^#E%L+YD}4pzb&3iC0mg|`DrJ9D+80h(0o@+g5gdZVlihhU|}bIY2CUf zRG=0%sH@7A>kUe+h3&=Dtp?OAARV6v<+wFdwx9wM%y)W}7)Y3ZN$a|I_on%*L#U?c zQG(n){(qFcQ=4c}lw}#VZrHYM+qP}nwr<$AZQHhON8B(gvb$d@zs$<&KX9JTIp5lI z&oLJ1AxtZGuGuE&EZ!Mz{C_Y4tN+0W3?-p4&P?#Pni}>DFm#`4&_}DyLel&PBargT z2qZ>E&bIzC0*xq{1;z;V0KbetF+XXGsfSC{R2$KAIB@ucN`?u8L5B!R;0;Dn(F+FG zVnMV=%7cqwlyf#O2umQ;o=EULH6`HuV&t&aTfKlLp@Dtg4L@IT@ z@*Sgl9r7?IBwRoA5OO?NJO|f5kT5+qpg8-Tlq-?Bz+#NGz-5RZEL=_uQa+feV<-Rl z#-_w4c;}9OeQZ(dW!aaFTz_VCRvvxhI#B~+5LY>$cXqbHyaQLoZKJPqavao-W#CZm zI^AZ}DEZ#LM*`{9pL@05{{-Lw7$I0d!T|t0G5t3juHFCpeHz)>+L{(3f;Hpd9^m`~mjIT`yc@pogUkEwn&OV;Bjwy944+Z2W#}7hUTG zLo<4|gpho%oe2Y@USsFD-9*D-J=1&CLt-&&cCjrsWmm>uW#w8{M97e3I0+KTK@no!y zz>NMp3m79DbA5bVG|oeZFVeW#cT4G;**&6*-*8HNzxAo{-orM#bE1HKO+y@=mD54L zByUbzJ_<*F{kEf^S z-QO=f-J5Rqc%XnQY2Y^u{*hmOU43{FO2y9iVv4VM-fVX-#%@RCWPY%Mlzga+VMB7N*(jm^@;pnqZ>*1jCJ~W6McDq4y_|Vfa@XMgKIe4*Ed-c!QtTR6woHJ zj8IRzt`b#J7%I5KIHx&pM`V&d-&B+@&C+3YoSZ6`U(^c^pFl!L|s#Bi=g2?UMJm&1ad zN#o5Mqg)9kHkv_n5ZvnqAbaO{oj#L3ICgOC%&-KA%M`OJ#2NZ!6q8`O#sDqoIt)+A9+ruDG>Yig z=B>%UT6@*Ta_b)_7n2j*D3tY{q^Z_TC-9aFYlaM?02uXUqcTpSgQ)Gc5-kD3LX zXi_8s2Su3TXyZz-0s`aT!enA~MqAIR+Hpv5ccPg~SWljEp-4;oTe=D(=4C!QMUDx8 zCJBUUDYA!Z409LsCsQJG%iiurWfYFinNoyKH?pcOfqYl%41@ZCepW#}Z&xk-b3T== z5mffCFZTn~9U>XZI?prH_91 zP7SH6G;ecj*1jVE+nhDpX@+KDpQT2aYUYqO`uXg8)+7)neWvyR**CVBY{ zz53#$zd@*>QvFz%$$d!v)f#66BELaYsOyUWnPfz^XlCtHV!M@;lvjmmLzKfsL2bd1 zTe-+&r3%5@Xn-bk=@nkdubMOeQG1mT5!HOc~4lzJ-$36sd+^rTr7x+{ zv&psEqzbE(OFuLg4CyoG;>qYmFDN`znC8_Ww9#7UMe9R2FfV9^tZREt(-x`b!YhVN zTA5BHMSY0BhpX)q)(}A0{C$d7S+QpSgkW?in5&#?PM(1s4G!3~<%V}{t}tvHS6>uE zK;qJ5B((Jyt*$=%t5^6-iVNvGsCALPGEcH*UWRR6iC|D2mhh$PRwWfgK>C7B$)@ad z{{x^}N^5nY?3DV)L)a{W^G^HqR--b(E&0Ua5j-4ARo_Z5UJo_a)ZuQWYaN`YmN7bF z(jg)n^{AkD(}4=g>vHHd#e9}q61tDo?a>4MmgGWhCG9l2I# zh|C|BxAxvG6wIf&5o7fJHs7e({oaOAU2;UALG#7S*SOq?*{Jn?jA1)dRXdbhcQK}f z&&h$Ql*c8tD@KP3KKO6n`E~5c5Pdl5qa_Cq$3=f&xA`YnvTJu9x3kMj&@E?789M-+ zEP*yNye%J;7LZ&uMs?Z1xoF$v+{o7%uE@h}enT7_$hYzqn|GddG+&NmN0M1l+bZ~w z*-E;+aY%;@%3DayiPUWtsrqeYPuymT)84jO5XkX&$LDXJk#|R#dJYo%~4V7DP)?_s~N-MI2=`! zl(1|JD)d?0D9ytf>`QsP61^Up-8aug^`B)$JjL&tfTqaVaZedY*<0FfdC3%C6;;`x zvrC{a9P##xBD)=FUeg254T&jZiVsHsABr2f9ssYB^{&<^$W*qzCcH&8s`w?m zhMhBURSQK^3T3#%9pwwAUgNoYCHXMQFk=X%VtCu93M8Rgp0*dLz50JO(p@p?krIKq zE<1|83rMtkv_9R5C8Z~ORcZF;gEiJFG?2x%_$<}Dw>!7cVd->RbMENTTW!4~4SdS$ zY004@ON(PkqLp5?YglyO#!azF_KR)uL%_y3lmNjN$;Y8q&|dmC>Sc2*2Z3H7S63bT|&v^vk73r-#Q~8*Y&Z%bxb32mpM1q^nvK5gkzKa(CwrxR_%n(3+(fK zXtUohP!KvB;)WkdQ1Zc3XZ{>b-{lsSmd6xo!WB+!D|ojull7Aeu7gX&!LKq@vqI2a z`mDjklLh+Jf7dT(t0lM@fvJXTY#(hK!h~pN<5R}IYWV4QckgkB#KPD`_33uZ9zyVP znU-tT7u|%!cxuekvP{_ zx-?WnLLiyEM59z4dO&+8d?3j=Z?ze(gpwAKFl?L#87&_}TeqcnAj>FZqO+VB5?#ux zjw1IULF$5X#?qx-Az#-vCZJd7Eowq^fiu+%NSjYdZdSiyR9$dYvyF-z970@bA5$Yq zdJR2&f|+0^`bGf_uV9&MARlv0D=6X!Lt>qM4M8=cW+Y{$pji*)0c9gW=}Y}X+HeLU zJZpt3Cop@bvs5q^^iL#ym1F`rDETS(#!TTpNu8o1uD^ns)j}LXBd;`^`E2SfgpBy} zg%AMpsph~gG$qSKA6lXCi^n^K7(5ECgKcMOX;w7m9ztd&V+n zv)8R8`I`5&uK!p?f%kot5P+{1wVhl9rLu<_^e?OaI#;Cxm8Mbj{1!xytM`k=k`J#-Y&HyI=%!OB< z6{-jFy3O6NJk)9OY)mLBKTqm7c7ApI>3A+feutMQ=<~GGpIec?Tk#~!6)P8aLPn+r zde@mDi-9vDy0*4}4exMlKMT{erDGDTujsb?lg3r6Sl~{3;@3t0JTySHt}YS7z2=D4 zKoc@8^H&>*3ICz_bC0&(mRi#yuYJ*wtAUH9gYBGC4AE+5^>3=aVE4bqsift6*b;#} z46btRxy$HIi3wA{;(KPVe~JF;l2OFICW-ojZoZLU*a4b(tPG6GkN19FV4x}m;;Q6>%HivWbAd9muuG6)oD;0n+FW9)@{C|C%^ zOnfjp{C*MCh@NnHLw;~Uv8Wc&Lb=mC8Lv)1!xa+bV7w0WsG1~u9+{vw2v@w{htb3F zD7>+S)cq?$x)RTmZ!yIIV!i{4T7edbvosMjgx-c?z_a9M?$!Zx9{}Oobc=gMW{9r@ zYu3g4zHqHE*vPOqRMW(`aSwXy5BbiC?tGXq5WNX%O`7`~nah^pR`UDb$#Q@Eahu;# zUZgzC+b>q}()^}mTAhf|__#G;)u@K4x_E6IjQir$3=dhueLyzD;R}k$guA($`W>wK zQo+bTQD3^5pC2bU4_jQBP1yBlqDbkeIi6e0-LxWMyiJIHrUrr;R6di}|5?>e%G{L(!N-_B6iQa*{ZTaRh>@jMz&e}ven z>>^xooRQ5OXSNBcZ8@HGz9(2(e0q+(u!C!}^g21(j(u6F^$?=B{fS54+ScoLM=vt& zJD4)CxF~wv_Li>u<^K1Jc^RT5FLD?F04$3C#+3B`1+|VQMs|+I|3gSSjqzo-+4l7E ziNcK*aGi5mgu`krnizCI9z*X3(y;IsB_v1Qgpe&yV!X+{DB3!BEk;&W6|`{Nx3h4(PXC2W zjBdl(Zj2eF64nE)kvmpI!lRZq7yn<&eQ%U^8)XYH;!!051-b+%*}Z4BsRcDT9#DR& zi7=w$0D0*ZRhi=>5UMTtW=U45h%EmltjXg|{8l478Z$;5OJNeSRl~;6RSsM@xp)>| z0*(nbL?qXrQjd(?M0ECQezqYLEd8OVX7_Lm(8u7REHo-*5(0_(4Lb{3!vI79p$Q2& zYw|fa)QpnF)IdZ`Gvet0Jd$HX5AorWHE3qRI_D5=F#o2BXUH=^ZOLHj?GjBW#<>>Y zO&)MA`BlJmREdreJ<6EP>)VDTJ1}C2H!6KlpqpnPRCd}b^VlT)SxEOZC3|vv2?0L~ zr|QBX0SPYWY5>s*9mV&-lC_Or-{Oi_N{~Kfc>+)tnaQ`b8W2Hv4=tlI1r_exb|s;l zVpGiDBT`ZSKf5S;<;>Ec;<^xX8s~vr4V7$~boVkaqQB!lUj_i#otw4#h&((kKBnSY zv`&A~T*)wHLV;B-^@-X%INhT}9X{Lsr;q))UP5fE|Lu&wH$=hZn~g94(WB`+8DlTu z&K1&|7kwu-!q1`1869e$1#W`ACljQ%KUm}a(KBOOHxU{?N2ZuDqI3|%ny3?nA`|o{ zXhY`v{wpSLca(l09CU#kup^2v)1GvVf4!J(7d_NSD-s(EbKu6kp7V!$zn>$y8z#FcwhQ1=44+Pqezl&Zy54vXH4LLmzDZrF= z9pHFM&3mr@nHK=ccbE24GgoRi6M*fJIWF^Tu&{6rZ+39fXjOZgkNZQFZl}Zq`Ft9&G^3pOy2nBk8Sc>;T6|s!3gp3JGL#2w z=sZY;aq^BCsyhKJU&3RJpFoAP=8Y?aJZVt0)>pX#{QW}nA}Q?+FMjA}{M(8wFMl{F ze&DbwSTXZhw_h>aZEHnDkXZKeo!WD5Bv&S+@R$W=_*wtPlH@s-dwrD+jI$A|`+QKSt&vV1h_3V5YA%fN2yM zfTa;|LmC^0(6F`OzzZyd@=gkwFl%6_OcQp1{i9H!aDqw~P?@*^THm$?)R!S$B&p9; z9)WIM<%S(O%Y7V7yszPuCYaxES<>^ zKLBqqN1!Z4C{qXi2Z=vqXbz~SR188d~2l=6p z@NfaKLGoI~F-qLc??4CdZI%_gWN~GSr;k(djA;SU4j|(CACqfw^|ptx@oB%VdE!0} zkHN&2q5J!qLNWfF0tSd#X!*BjiC>=6Rszo)n7oTWtPLT!Gm04S`b#>eo=iy8J-NG@ z0Wb)Hd@lX zM5mKzQmin7gNv`vlbD>aja*l$OR!|}Di;r92pum>jyDqg^E(vh7L~N~`KY6K)(GV3 z+=sr}r132-#4-c+j!=UnNzt%#Ddt(MmZ+V_Ub~P?)|n1~b`Ni|#J!TMj49LG)OHri z_q|FPSpR@M+>SkwxOH0pA%9K9)N0j)(q)q`P#;K}znbjxI3x_C5G{+bXf|_?ga+F7 z5CO_%w-U(W@0T@d7n)`MxTZNU>DB9ZXncAeH&n~yZpbR}XLCoc`}5sgh&_*So;+Ks z2Ub*g&;i!-&A0@2=I2zGc=738GPD9bUQbe8u(>?ok_i_$^^Jo(=S>AGKhS!Di*W!K zoiVDZ-u;)H5M{<^2(&pyvgG{PIi^)|x_#KSI_LXdwlAQiA`wfh$Gaq#ES7~ zsbsNJNzQBcTH7im^l>NWU(+H+5%)Q>^53c#50<*SFIlj4+MY0CTTaDY!SN%Tsr=A1 zC={qjOVhYMR=7n7Ys3?TQ%>H$6d?z|>aLHq$zn=9$S}+!8A~NL`Tjj>2vU z;4jM5htysV_+uFdFc>Pc8&(k4TAEF*J1bIfKX*x~-VMeT5Vdwsb)!>>k5(C1hgCXw z6+uqDVm2 z#m5AEOxbtE6nVozTohZZbzc-LMfr?UMAg6%zuBUxx#X{9JrNgpbY;U)i-8ZfpiY`_ z07Xq^s;s^)-bGX|Gk=q*#F%AK39qtmDBi(u)}4ms4A)pE>mYIMfCRwGFJNZx7&QCI zE6|b|rN;8L$@W7#PCl8Wg7zxCNk5b$Z84nVB=OlpR(A%tlI01rMc>~09ww!b&DsO= zu9?(Z%36VCRG!L5PyCrwK#hguIs;bY;UevA8FI!)OD;@v+FErYh&6ei9w{elx&Yw< z>Jta7wpn$axvp`SoNB-U6yjtVRrEQW@pncxD7{wmhN<6;hW)`caUp$&Ga0;9Q)Tdx z)Cf_axW>O!_^&I9YPo=T-A`jzkG_u&i zoWyVvo4ZXHM9nXWs$!$pES^o~FWNh0gs>Fg44Sz+C7kw} zaW?prTejmdz=u+r1`c@=5}oZSV$ z&iH_zdPlWjc2JzCxuy_YckdL>JtyDVno`{ye&k2HOA72@;hUN^hs`-Kn%1DTRc(Ah zCS&<%S6B9Uq;O;zZr7>F6cN#7vmrDcKuN4eh{g+-z*%>^`JwyoyIe^BMTY0}|EI=P z^QX1B;x`aX{|y9j|LceS-!`}uW$ZTb(R*&yx+LkckDAxvxUNKz7V)C01P%sWtlk@#@(NF?n%wHQoYkv*MjkI=??HrkQ_7&b;R!ZY3`2cJ6izv%;-x z;6%TSz!`J9Y@A=GKg+@jv|?Omtr)v4IZs@>i1sI7&Nwg481>l(F$Q5g7xgXmDGY_e z3GdttNN~WqOHFufqJxTB+A!fhA}EJPaxGfPwdOCTNx>s6nuI-uH-hN3Wjqt3E+ADzq15q#O~ED-E9nC*HD1ss=pc zXA(=3j!H-f2`EoY$vc=F`SBrkPn1ns!GWRqWDJ0y^P_ z1o=a1U1hg<(L4uybpQ&@#vVB!>tGbbJofAc@EpB!gd@~uhRO3;)oVu1MCBm5q-kS` zzp7Unj}1_sLAFW}4dK(qhBA^AjKJ~t+sDK*cTwp`O4qu$W6CJleCVm92(U!Z-n*gD zT|E$dhwk#%%6twpjy1z&bU*07Y;XJj34{8JSdE_x1^~c^0sw&VUw;7xM%MqsNxu8H zWWOPf@I6zDuAl;vO>nK!kywB*4oFrH2h$+4SP>alkG7U`uX*W0{&H#OdX2t`zOkhe zQ!%oW?Q!yTZ@beMH|!<5iUo+~i{|V1asv)^;eWx zGbl`)_np9G7%Tv>1(EA75;AnmP%cVVYLq?+LiQ(;$WTGwG_W)9f-Z<)XDKHtM1B(T_&SR~2XQ@?wFdv1 zBfEmYs6`X;#27F@8Bd51u$1z?7-8F2;)D$JS(3xJIQk@VF|-_uBhLzgTa9Oh zK~q?ShCtr4-Uz7>@u0HpNU-38)DyVrT7p7w)yMP84f4Rc@BI}hJpZR z9%p=(Y=EV~O`jhNIT%IfR&c!StnSP^;1}DW$qg=3J}^q$^0adI=@v09^u06ILl5y6 zmcX1Cu`%Nke(O`_#TxuqT7lZ_HqC zq*n!v+l2ISOgpYK2H(_M4Om;vlt%f-n%9B}I2IG}_b#D$C4 zqvj4lY;d8&C-5Ql&h@$v-hU%#K$$}4Z0oh8SoW{dujem7iW&xm`-Un9>f}WPjuI*I zQrVmA_H7B#7{{&1%)vW|Ri+YhLi~r_5nLj!Ui?z7uC*^*a~#)`_rz3QDH-hyddUl` zPd|2~tZ2V~L5`H}x*KJL2#R*yDn26F7G=6iYU2)#cFDV7fbIDqa%2;~wekg==-i1S zj7WO47WyE1WkmROuK!U75rsYBdVwu9nTuSlHjNu$uuoa{l2v@K(U!CbmP8r`&uM%) zZ@&);=xdvRC$1ac^k|&J{%7oYu4hIIhUIrV_kvO9PxAa3VbC=$CdgYVG!gMbXZnkt zDwG^V&e&fJ*`jykln(Gkq7Y@rl7^z@BWz6I?v&=SjatIQ4^IRdSCw~Y@9h@`zCr7q z5#~QKEe$JTJFl?>NmS|Ot7NHFvIr%;6g;!KX$e1RIcroXpDyz&%9nqLg~vK@yfiD? z^rP%H>trDsdO257{%Ig>J4-V5s)X{2U9dukKb!B|XzS&)J}B0rbqrPH>`NAJPbFJQ zITm}C*`~bJ_Kj$0m&IUf^Q;ee`8@gq1^Mh!}nzvX$f_p_7GjvzVKOT6n=rIIW9d7JWgVH z2<}m0TCAU*8j@wB%;?xtPfttse8(DOZw_Q9yo<o$&A7E z4F+B%M&wa_FVX3g(FT`I7nT0v{Tv=sgBuI}Zr)&j!9JZ^5ws&75oWcKcO;-9o^w4H za$&emZ_9?8n}pvH^l|(zPP67U6lgjOk0b-9Q<>NIxd?HAxw2g#~y_qpvC}YzUNl>-dDe}Uq&@9uc{u~20|2=H{IB)BzXPP9 zi;>m;sPDbv{&L!EN$5ST?Q2g=e56!xU0C%vtzT5K=9uU9%PwNRHevKcfRu=)P&^K^ zc{=%i+0+3bex=}$;JUpjurNvhK#kJT@2%@|yP{sR7IJHk)Y!bKLkkPzd)CFq#c7~o zYf-y`h4=Y?YR1zJy}e1bstB|nvprPJJe*ez28K>{oe_!QGA#kR~Jl{m@4s_vL_D8%&-ggcRd0sVuzHZYn(pW}*-F10gP}0@Zss`K^Kx6tlWQE3e4HZ!up1x12BJZwfB&~f!paPAGvx$pvUOy!IrF} zk8YgoKdYFW0apuR3!hH=-G#xRf~A8<3UKzdN#Fx|2mE?#&Dqf|IyGxsQ)=XXJEBtIYkb&3I~a`&}uoJQcxi>(dBOw7W} zR;e|c({5$VlnL%|XZ>fudYG)Fr$(RCgK!FpT0J}hO^k3wW#rL-(6BJUy=gdfrJ&*w zvnR)c79!999gMW=ta#W)ZnYCuf4?7!=dgz^tAG6%1hu#qr2QF8M7d`>d+oy24{g(X zofH}Rh2eLb@Ed0A9Z0w|?oNdKDp#qU+9Qpg7p##p8@F_rPY9mv{$OVen!TCWb)$Ya z!5o*CM3ZB;colKBTT5BqEqDE`FiZh0EyFDfijJTH3RdRkH;f{jI&~Hupd!9FFv6@3 zgqM}qxj5ZZ*u+fWF(QeXZ`MO0$8ohBemP(zY&b_emAK(*)MM&svzf?e{axhGX~p=j<`dA|2n-ZWsbY_k^d$W$5K9<+aBF~f6itO1v!O6nZ(;2W>67(sf`wT<%B4jwrf zKFvHmIb2ydtp%yw;E+YB!aV(jZEi~!+*Hj!bBK^jMF)QYrmWMFm6?IdSf(2iLQUx` zGrP+yQ5|?f4I8V93NUsbz4b>cegmmCOn#6S31EOm^@Nw*h$t&S)fgc>|6lT|K|a7X zh@;_bAXw9tSm_=u68#FQNf}iY4*vDu{d(j0tMNU{56|73knwtd zNAD1(0-xu47%Be2me(I!lt08rT0J6*_bz+ZfD8To{o`_VcWd2AT&3V*EBk;*Jvez3 z?w)tHp7S4@{c^R;0}$FsQVU?^%m%r)HzQ}IBzVAO|Cs--MYG}bG zEaPAqbB;HRbC(B%445(l{-VvAN{;#8?n`|+d%|w?OnUdAUi_i1!x#@#;@?3KAr7j; zb&mCW0L2g{pV56tR-BV3^I?sL875U0PYKb#6poBD(qF4Y^~y0w!ZM@25YRX19B7JP z<|4&L2j(16kYo}@=b{nxY7F}$VpIcxDe^LP*P(5Y> zr&R8X6s}i@E~JAWEXpjZ&SD0WJKIJlq^0|UsUjgmGi6LfnuRdZPZH=@Ug&D#7qe-; zUMfyNo)e05hwE5}n7&h~f_dPMi0|6OH-O(;$yK7sT=h zV!!Ga60T<8Evz4cC-^rUbP;TSLO~wE&R>YTHYoWDGEkD|kB!1y!m-{j&aVkb1rfdfb4FcCF$`-&qpAjr@XGE_q9 zn67&76%G1^Ke$E|#CD0TeLWFKU_{i&o|1g z8qlj_HQ+KZ>%IMC5~s+^*R``}<=Z+LMQ(i4Gubf~R6al&UPo*mehZKi^5sV#FvXvs zDtl%Rl*%Xes%?5wdcOy_&%3Yb>n!&he^IC3A5#7g#U(7z$5s@h7Eks$RwFF4xxpBta8Hb1?6mp)w z$h0>8<$(+#4RhI zaNFJqu-7YE=Y^#sDFmmkavcM;_zJmj_*kC>%OdHcq0_19g{hT~Hr2zX}-J>PXt1DYzPtzs{DXt9tk?^=^sf&_4khV%5anK_=!Z!RVZdUeIsk-eUd1)HQK}89 zKWPX0D4VD*lLyNO@htM=J+u^QUmF3>;2tJzp`C+FLS-fgUS=Yao$ zIi2|7qY@z@Wp&-q-CYj*IC)e`nFEq`D@-i%m2W(!o4}MF7;)78q*cH<@og>Ctj3~B z#%{Bi$2M>aX*>_^!<$gDNH7<+X1$k}dW`AJNxCFV3o?b%DezG&Q`2iN*DMi-06IIR ziMIXSwZ!1%`ZW$DrDnR7(#U(q=EeB@S;*X~E*#~Fc7xfB>!+;FUy?laz|v75XWUj{ zENcu`7C{%+Oy`aWVi|*VE@-uEO~Q7zp__!QMV<>avhSJ{w$@tU%EK^dCG4+Xz(&e#|TSHNcGMW-p8?y+t z)p~-==U<80h6U{F9kz{~(sgpB6Va#I;@GFzl?!Zu!||IYjS1et5-y^RkD*EY zoJ`wK05z`C>LUZzf9~~<>M&9hlE|~@ItIL!wCK3ILU>GV>>DXoE8bkSFd;#0QQg#W>q+`0q8K3B$ZWQGui=j4>MSi zWJFK1hX=}Xq=zLzQ_zG|AKi+#AljlLCQ~NDVF#x%55&js-zSOu*Xt? zk>TNqv*R_Bu8*-Cqtg?NOB40tSqxOj3ltId+KR?5o>EuTLv<~omX~e9%#&@-tg5S& zWOU@8Haa3*&%K-G8N2lcJE_;`KVYx~n!g>n@+nA8ayRu~7q5adnxAD%yp=HCcG;5e z{RB;zBZfS;;~~FFMJS{^^HjfzC^8y@8R4gG4ze#uzs6fUnj0&pdo(Ge2$Q?7YI7Ru z%w=U*NVQbU$Oc);6qLNmOQm5|D$B{<8h2ALDz!~zs%WOKN~_Bcu>CYIA09Fl&Nlxc zYYLVW1Kh#pT-o0r^-J5*i9f5>mVc7o>U|4Vu{m*deVU?1K6cU;#*k>NR%}TQ$oO+n zA!GgBEZEL4csHuW1{?M)t!h5`TQ;PL(l13C<)y4^eIx3L*>u}k2cykpFFahb!b3Ww zj;sQ1oZvqHp@os^1p{YP1YjW``JD3w{iqv|tX9vKvWeDtiOf3T4{ozXHEEOLF<|Ef z>P!^*ow7dKGit;1R=dUhFXMUYq7NO(cec8bZP3E@^h#n8)lbi zJb22P?23O_b)?Wj?2PBDR z15^PEhtgQ%(NkPuQx|~8?w%bJA+!mh^)T2gk9&R3RagqWv>v)}Ui2~JLn1^m<(d5) z;!1%^_I%x*z96*{s3la*ybL4VO=N_2Kmox)f6hRIZwY`R*7s>{3zn_!>y~!qggukG8`6_tN9XoIZ;}_ zXsBoBRrV5D2Te0Vec4MGh9!azrTn|cQ67cHMzXKXMG9-5!oJga^#N(-fC23Rc)qvM zFl3qstiSo_GAzTV^`-60#tc)0)XC<10i5DISpw_hqfPq@%f1~}{Y z%mnA{5eKs(Wxg@}MZHaatkBNM})Ly$d**o`3f z>0O>v0`ts{v-U-L^s@``HYH*?5&fxXpm$FcA2q3q11L6SWuXh@xJ00(a1Y6TYjATQ zFhh9ZdEX{m=?k%oyu}9kGF!H%-1J~|(Yj4Dt#mp!Qc`m|q-Di8h#S0-6D|rWc=rVh&}@C!(pCXYs+uA5ZDHbg|IXrI~GMi5XeL!Stn# z+Y~=qDldqOgNw_W!{!m_2t$rRmo(6VETrs~bVbni1c+nOn^bV%E#R2&dv$G=RFPM` zbWg31LGRl{zF$&k(VZ*o{5eVvpillK6f4A=7+DkWb$f_YYR&zM0fpVVLyO*ZW^(hMfmGqgW~ zgqPG@WIAy6dC8C`-nOBlP^I3=I`Q;C1tPw-XhoEL-zig9TTi^S<^*ljM4$d%gc47n zn&ETqN#gOr?zuT35%Him zJGouwI=dPD-kG0lW;gu!Wf&x-^aE`}+inEuy#6!P(<%Df5?8vm3s9zPN2Y1{ zU$1_{l;CsUy+bgX_Vi1)nEw{GNnb6 zG_sDwR2suYQZ2k-K&TI>e&JH-;uS6W{RG&;Rn~0+^Q;3^BG+sob+4Y)^CRl6s$1vm9WE07A}+boBq&h!Fa~r}834FxhOQ1pW@Na| z35&)pWRdW(>aD!uNC;cRRaF2rzGrh?vB+uy^gpDN-`73qF1FBO zQO7+p$AJ~aA(BaR;{6KW6P&8g)I`h|^rKp{c-v>J z;XtIYJ26h;SdBI$^{oh)>ObwIPCk4{%`2tTF{c0(7ez`EvP!}2sH#f5Qbm!QC=K^Q z)k&f`k6-O;B6(dd=XlW>D$Km8fg(Sz*13eE?-+Pf{mhA4mvo*zB*XTFk5n%HdzL&4t*Y55J;~QcK8n*@rDc79SBC5skJSL6cE$RB5y+|qM958eYgD2u4RVs!mkEf=3A zLLGj}IK@a@1U>QMf}4BfKs;LU#S^mqnfTE8$SFN{x4xWITcu_ZW(uAhq#qqJJ_B~WR z)+}J4%HS_)k9tj)jU%W)sAWKgBIh%S{iTcJORl2Q!@Q0|x3{Hx7&!mRFHN^UaIWRQ zR(2^IJqQri%1yxRB-lju@+L{%siiVV&X-bIp!j)_$L`C~LbcoKW+%P@M**vi$hD8l zyr5r>6VsN&_@ebcsL|^;oS6lVmA*<|(A#}g;Jkm@v$u0UDow(J&*O$pxJ?JtWn*bev>l42AMIP)o=@3`mJ}a zAG6t#d~Nj_Hq_7*XH6;ZZ&hYlCZnh+YWF0OCX?zUO6uH+9m;1Q5AQjIwy1P|?4W-E zB^`WCV+U}&TpBsV9Bi%3Bu`%y?CKxBVzeOPg)r}&h z?*+xn#2fDUH?W-Z$(B>7-k~`Q6##z_zzLaARRb-cwJHPbwUjj>!P5MNVQahKCVLHY zp~;$Fp8D?J10?G3L#uP3MU(8~dV{J;;t!$w>m3-v?>H#(qit4ua=hR5*G$=;8q$f} zy(30Q3p0<(NKT-G&_L!EkWvff$dH8u7vq41!eGMc4;CB!uK3O|MEq_9>FV9~+1jqT z-z^*i(n;2-Wm;^BH-mERG=a`6l#lyiXb+^R4ev?wXcr#T{V;7T%1F=Dzh?iunp;o` zzn!M|0}N)mO{VYiIzHQH2rnh1LLKVkB>Jc$!=g@fPYJDyEhfbK1h^(_U0+HI?;|7N zDyds(GqU?CnRn4iMvWRr{y+Zk--5(vDxO{o~1TfR56t|GPDQL zAfMt6le+v}7$?qLxq3H`VgvqLI%}uPzk~{z3QJTSOoCh1mf-FJfie!1&sh5f(mrnP zE`4#BaJf@Osu7OqTBn!ei&8K{Z4Zg&)OQYmhcc{|oMEvo@c?eu96Hfkt`xewl;cd` z{}=|O!GJlhHH*cqel2HXok3p@-9cZ3pOrn_>mO72^yOA(49oa5t-%?ajiAO@{;_b+ za94D~;l4%>0}q_j0_7#L@8P?`1+^Ldk~@`~cVEE?t|kQM`P8%v-pb0DHu&x$&W|Y3 zLRRuUo-A4zoq(c9?1B?~df4LeUKLVoX++&<(<3^Jvdug|q8qZanTV0?gR*Jn3EUy;5!1S~M-q zRi-W-h7nHBrqG;TZf2?U?FdA{Fmh50DPS{}%4DNV%AmBB`$P)8* zu33ie)c6)b8cx;KaGaeH3ER z#vGy<3CFL6hqz4P0wIvINMD2#J0CuYw^X66F}5=lGsmIXo*7F)A z?_HC1;Vo`?9Yo)qfXZ%CfPr_(5S}$@P3~njuLA|Jv6y5>fE; z?@+<~tO@m)^?BCB!Y6O1rl6%e{i~z2R^SiV^uIuTDs^rp3-b%{c5>N3I z*d#M@YF}}%+}!FLiL)E=`P;yavuI6`Mv~!v)p$stg@?-ReReLFTFFH*qp2@qA3SVK zklP-?ir;Q);dQ$h8a6&u+|9}B`!)#g_YMCC^?Rr*Q^l{Zj4lZ+O66_wNB|L*W-J?J z!Q1&lm>VpbqGqlZ>ebb%lbJjHiRv!_-QVlv9oow*`JW99(pN*X01r*={h5QLnb~QJ z*vraO1blF~f5aQ46>X5KQU!}0UhWvUigfhKjE`dsrBQa$T;_WZ7hNvq@zDGej#PT* z`n~?OOzu2p6XNhE#)v!_$Txi;H7+;2uIyNo;{5vxOkte~wOjMmh}yh|2)lQ8Ph3i=`H?M%8n+4BmZYVB`&$alH8+u?Y`$>-9hJPEc0P$PiahXTUrzTqI{K!fD-b^Up z=sjm{Ay67=@$LWYDNm-NTSE6odM3oASTgOH<=d;|y9$3C;wgAS9uTp3+2A-(VKJVMf9rf;KH%=J#yp5g{{eC$c3dwIh!H@n?z-H(ICnR!gX7>y`(;sn;la)~N; zvmzi(iPT)>2)11h&l5SYKANVkZv!)GNdTtcysi=86F_F*MA?cm#3awfyR{5vXnV@{ zFY=0+h$k~P6Q>t;@fqyI%q}*LVHMsG9TJ)$9SzJlvR+P{0@5FKRsfa?sPd3~O^wk? z^cDew4kOCVu!nj@^O9zYsb%|V8k1&E+)1mCpG|lT zR7;CS@-FUrvm5>dK7;Vdb2F`f{5NOE_g>svn=Qgn+vt;K+=C;Z3^xEqO(WVs^MD~( zTXHB|Nkhx8&dJqQ2L%?_%t6~o;c;21ljD1WK5|lCp-c0Nna8Zm$RpOmcFEN~8`Pot zh~bfkHbG8`gs-2p;oJR_=*fVEx<<*AVPU14Q#Z>10~V_$F|s66;`S1G_z)uADe(|P zA8s@xkEigp?9cvK$0G~@59?j7yoCpUj8PH23?d%139f4tnjsR7#iKk8JrjE7)FE4GkOm2^t1vxi< zj~&&HaWCIV$?gGHSeQVDg#)C8usYovGHA22y=K24*GfS0U{<;O?boAfW=E=|#gslx z`mYiNWPg~O3)#k4&^a7OhVHniRUh%0O{GE=9+^B?(1?lJ_%Ejey8<6E4BASs@ zWSDiJ0N^0`R7iwm`qYD2su<&$0%G&stj!%M%2DN_zLKR5QO5QVmJ%`16r*93xT~WJ zao;*cSDINal$urfSTX(W&syKDbu!x6HNg@1Yf609%FwFEORW0-K*1#{e`94qBJ&g9 zreBEOTvp5sbuhP{WSH1^UlC?N?H*Ba_vBVXnML>)V~bs)uuutnR96LnPBsNccSUoq znBQogW%q~-w6NgY*K7#r{F``l5k%CAnq60}+`igeeni2QI`(tlh*+DR%f6V8=cHuE z4D_e-IK`an=t@9p`>b@wMTLUzOw9`y8dR88wD&AOtJMOe@KR#w4mFU%1=M(WHB%C> z;|8e?fQmKpu$Km?A#rCr;C+>A_olz(C7(!bqoflDTdGS}fBzLs+_5^6PwC8$7XDHI zH`P0y&sB3inceiaZ+vkBuvs5+eTU1H%<6W=@_|M@t`j#ctXk5@+4R!*wC*(SPt$W? z@~LM5BteCL_wFmR3jJQg!GJCO(S3hZN=DO{b;YE!f4wfsZ3_@OTf4gc8k()H1H9i` zHr4z3gT&?YY(kYgy0KI13l`R55cJU-U7`uv;0UL*vkw1$>1oAI+Od3)@Kk(JfBOaWJWX&m?LKkFxs!o_R78E1eRL z(wa|Yckhe(^Okk)-T9*oJY!$HCtLA1BA+@D(*9x-!~6kCh=9yyII&00YgH7oz5t7O z6SIMI9Ovt&s=PK0himr>8-gd>O;~V7^lxb9tPHAPI>W3k zcwwE293KBbdLrMIdtQWUQ5gpM z0yiRi0#*<35s4WnF$y(=Py;|;?l7q|E!eDpgyDw}R~68i@nj(Oh8;Pug$8#(Z+*lo zzym&DT89La#!%E*Oz_Z!Cf=duxK2!&$;ocEn`;R_%eEvys!f|#v)tLWMnnaXnbh^m zm(jK1L>^k|iL%FQ%7(KRha?>=q_K>7uuWbq5vZe!VbKg#udp42TpP7jOjtG@p(Kqu0Jhi$AvGa zb%cEQ!XswqookGY{o1TnGL4o9C9!Ym%E^2Qg21L($%Uf6fZ%%iMDoblbB$9vH2n|N zG~w1v_2OJpokd<;bM@L^c~)JxnL6C#Te4o#OENew7>04;@zDSds>FJsX%}jUqGzCkv3u8YO)rX4w(G%EOg8ZP~U zBdc-`)g$)da31B$JJj}kbSUH$DK(_{vsq}yfOi+^SDDgI5NR`~C(=p$U+-4?`x-0w zk}BHK1W3Aihl!YuIiT&>d9yN7IZLOogX{^3u?i?Pw;e5btXv#8gXyK79;FenSp_N@{b4UuCvTTSo7X5!(a)f z9L|_mF?tqm&6f~Woc3*ZpV>;eRFe@eX3Hr1#@&-2MJT&b66^)7gh0iTy7x@Zjw|GA z>gd1xr!@nI&#(NXiX{X6EQ8@u@7D#os+Y`vp+xc>t>&f;E3e;-T<*i2BYG3s8nNm( z!t3T=A2Y+n`vP;5$LQ6Nt3x`rEi!3J7qnRS&;HUy>!6yD!ZH-UHHH1P_6^4v%J#?A zN3{PpHT%5M_D}N9&Yf-o{$Q~#rbho$jiMlwtc%!i;0;82_vC7RjU5p-&abg2iZ zNsFJ~+t>7uwG=bl+PHGAXA)N{MLqQf$d8Aocaboia6UND9l?X3!O=F!Fve}M=52CJ zZHfm$VAGRNwY>1P8E1hiUkI*e^j=sT8^b^S$%tB+@yQx*#fiFb)g=5 zJLv73<=a_EvF?35M6NIHPc!DM=(@3p2w#SWH!uvF#U__z*tBUP zTX>uod$v>|3$?tWxeQvXmf?*xj3V||%PxIuo)*Z{(TrIusppE(u*e4HTVNSt@$zvW zSLg~qp;&Hd!oLJypZJ7vYZjo;@rXxwrwtsfXS8)`Jr|UWxO^MYt7_+3cu@sB^%A4C zy(qbGbd+&)ny%xkAH}Z(;oZ0;`%2B^!~iHO8?3RLiA!>Ae0}5Ik@!Q+1rju-<(8S$ z`MYJPx5<-cQT=>TFIn0C&s~*Q6FXO!RVZaqr=<3j`9M{ug=uLUQ`zy5_yUafbzAt)=lK2^avXr(Y+$@WE5HeX#t*rrVYvjyfqt=+gtTf1< z{QQ`qM|SaJtn>=KIFy%YxF4n_TN@uJZ}L*8FWN4z>BP56d4j$P=Mq? zQbD+?=;Qva1dQTAV*GN{*>q%;5lz5Ah}cID9spQK9*7BK6Us>-Uv|orqHxbZitPJz zncBc~MZeCDCi57HZf}o@8KH(N%~&JWKO?ke@?mihKC0DEWY1z&+Xc zN8VcmYeM@}BxNJ0rX@1i08vTzZj@L=s7Islmx+o3)hW1Z zK;u*%bgjBco8;c~oA7y+E3pc=>R-k6ssB*LuaSmdQ3K;y`#vz=2i9yWIq(aa15TS( zdtGpS2UGWp(V2Kaus96l#euhm?p>F)V>sKGHfs}?GdmpY!&~PYcqhhf0BWR-+YL?G z@ER)iu!9e0*!Z#R3%cSChk=K%9$0nu>14-Dql+fLc0bfe6T#2N-SDtyOk{1s>5WRb z!&cX`4-nWci^Cc3Vt4dLPX$9ees6wBy3znktGPl1hrc#AM-cu8`X0>!+B z4Lh;Gt7<>-z^U%FgX(=Kyx^-PfU9 z4Vvs^$!b6&c+2KYYka^FXg6>`UTx;BV-{9@_@-qO zz5NhS_d_M}@kTTAkqYLTBPImc#)>Bt~eCB2NyX7BC^}$}|q=##B1Of!`LWb_+OCY=w`85rxYi zu%(o@W;uxY7v3RwGo%X8icCmRK#Y&lR%lJM0$w3BmG?rt-eU9hUgOV|nW~PVN;xu& z)Oe#rC~1VC{fVvFo0QySp#%a)%7XSWl1SJp6h1c7J?4)Hnyuqr3s+ZDNI2{hsn4dk za4#1xEfQNHM^avm<4|sut*ZUuDyW4$r#|a~J1T4o3cSlE#+<*M&rz+zi1(2ID5a;H?CCH8Fn_f<IKM3$sp`dMCzW>|!FK;Q1rT-oO z2ETx?|H+B+f5*R_nVE&H+5b{1j#2&3QgM4EK(^7mYRB)&XB;p|9UKgEL0ctz*09#V zYsBiRCiB?$YpMpf4l-hTJj%`OI6d7=clqX*xJBOI!VTKp29IA;SgU?_@wcU@4 z$Vmw0Dv7!h)Ze1XrELwK?UwCUmuEOrWXuCjW*`o_#nZ1EuRS*usw@x16XGsDl(L@3 z5&sDZMMXk&8$(26_J#&QG<+JCqQVW5$fFV`B99{H|7Xld3;_`dFV!o9x}a(JuZgmd z+!e*S)`aHhIkqq)DP|qc9R&}5Kd}M}vCs!0sw(iHF^V=u zVJRYaGV%)^%}LT}KP>NUY8}Lh-d8<4*o4mhdc$6S-1SK%gglXR&?)vL`7YSbt{gwc z#_m@$ME4O=!ZG1%$fuUvG|U4+N*;9KL$a4bTnIxx5A}GSc7^f^oQaw*=R(r+aQVXl&*Ai5jf2e-rJsi zpn#N@YiCQC4(e#4U@drF2fI>T&?tlGk{bR-gUjhn?kgDxr@!zqY5c4?1ha$PX0C|a z!?_BFGr(7G=6E86bT%+F(QxbV-}8{Wf>V8!_AG-}B6;Du zwJFgWGZ(Vpw-V-17p&9Z&AKmjdWx5pyrPQ?yY05@f=bVox@i%_Qw4rDi%*5k zS(1>U&8{DHK>dT~NA~~bu@qlY#drhpoUH-~y?;LJ|1iImm_NhMN+jV}H<2oV z8YdJLd?+&0F$be70!5#tu6N8jFur(!kF-njA_Rw6=#2lu4qEt^yhTHUZEvkH4L^Oa zdjyujL4u#HleyK=HT7Zf*3w-o5{tm06>`s1aWjM^>J;l1t|)XKMKx32e!Mw`Iq?UJQ^`AK;jZc64j95tmdd%-2ol|0j_Fd*Ct!i+IKqQk z#q;e<9sg8*?dubCZkODbm_S1KqT9(t1xvkF*a}sKj$cF1wga|$Cwe`7U|ESfRk-qd zbuf-r+}HVE&gX>FJwO?S)1(Ng?*4yxtx z^RbXGUK3l#st0T#ZLsh_#@V34iqC|6@=s{Kr!Y2G_Y4k|k>ac_=oWu=TCMCt(2`kU zN#h|T;5%cI27V7`wpN;0U-?DlPsUb1c4#gnW>-dT&`|ojML6$F==6UxDC%H5mr(rb zsw%|*00{nPti%7eOlRspimq%8y&tuGlqZsw=Id{5T{)%WUXj%#8C)&7B@OXNXvH}|6$Xq3_1~? ziCqQfzf--$jT4?mZjnr!`o$kBc6TUr{n~UC&ps~y83ZjJ^t&b%CC0QeAzGc)8JxBQ zY{LVI?IsQjMZStn?lH4yN4j50orx1ZkZMJ!PqtWNJ}@9Uh0z-RhgxSla#U)sj^sbo zIx*p-&f%tzFT(uVOmJyDc@PFks04{~MD0iQ0Joe6iefo4fW(>Nx*cc+1IRcId*DJ1 zvJFVHy{yYVnmSnKbnbnh2)B@?8LQQ~RRt{l5+wCEglORYT?E;{xcsF&Zt|=AAb4A&~MA~J&Oag;WSRn4aW^G^ix_rjoxlMu8!^ z@x4XpbeViT3->TQZe5-4`UT+t>GNhl^8<2DfJ4AJwNXI)cR!ZgTEC%Zz*bD}_m!i4t^qO2|84udFRm$DWVx}Q3T--J2Sa)9MzH@!$Gh~Lxw z>-~vNx9L6WK{Zu?T%aJ$Ax+r~SK@JVS0VuM)D(ygd5UdCMfmTk@8k&mqi{ez9z69W zj{p;d#I%xq8&C!*lUSd#zuIYgUnTBjA*(AWzT3g51k|?;DkU6|4TcN?>K{W`1-F|y zCBNs-;fbK$?)TaoxNo>POk~BovTg6TLL-`G0`vf`xA zY#b6jclBs}eDmrICK=Wn^_xb&l3|@eO^yW(5=5}&fU!+HdpJirQj&LgPkwNo8!;C& zuTpS(%pBYP^zTQ2K*z&X>q_gZ@g2}BYIvQno<{AcZhZ440+5ie=l7aWKrgI((t}z; zM;VNjoJdkj>q$V46dSKXqpwubwFt_NLA*7URDwBE$>028+<#1H@ldl-g*FQ*4OpdH z(+-o0{AqIiT5@wrRv_)@p(NyB#kU1{(aQ$A{9acrA=~-}yvy6E@qsOCzRSH(PfcMO za|0?Qd1c(B<5nMsL+$_kVdYeDq2YTf+tr<2J5BH~qD^GYnOqy@^e|e%*XL}}himRC zr+@2wMdtp%;};+r#v&1taEOe#Z*U3v(H9!|DWQ@{!)^=Mctr(4U1R>gdx;ty=7nBm zGs`}~NjX;a^a&6k-4lDg0=lxIzAx60(}GaCDQfg>d`|kp=P6~1Ana1+qol}CwI*RB z|DT0jYu{d%YbiCv42!z+#JRTvFZuZ&iWtrHjGQP8CRjW>d;-XipX-EXoi9qyBm)&q z-R8&&USr*gW_oxnfH*}SQ9_j2A7hAimGV*;8Lr;cZ<^t~Uo*zZ}x$shKWnpbx; zkER;6$6UDW=h|tq!;@8H^^-c};6Ppj*4r@w%MIs04oPT&`7Yvc2UYreQialnKTLSL z$}}7_I`dE2ZxgScvM96 zJ*o+;V?BGF$vHAccUDlb0;M^4JGbfheexuUhPGLdEo^&Viz6k^Ct*qla=SGE)6a}` z1&f;cu#iyZ4az%`WrDmep@iJU#cRvx{X9~Jj-Q?u)XXvqkx`XhmQ+GfmRNM*1sBc> z_4)PEQE29O^2e0$8bS^*@5Zo(syDCLgNyb{RjCK#RaHcsOFGJDVFcPE+tv=!&U$do z#%;ERC?|9f`6^IO(ErRV0t+_9C_9e`Rgw+HYSt5zO|JiWV^)JQgh*9nD@{Qxk0p8@ zfcX=m@DI_Tv|7e1MnL$Dj=2&;;c%LY);e_UG3uoXymLCt4cJ4D4P}5%yWkE7j>X)5 zhTJSQyb89eOmp;u_wxtb*fisSib+bcl&Bv&R~2{#sgvVW{Z)`a3<30o2SL(loUmyC zQjO&9+1~X9Pr*08=m*yz9_)vuOfUSjRHz)vCFJHN9VjaafUl9>UzB2L9v+waNGwUQ z=`9N^s`+4^CeV65S^m2Q63?JgFlzz$ZNEX;B`E1=YEIE|oC!Mwzkc*jGhJFh!Cs_Y zt)=xjRh3Ww0z>IC<-uunTw!oj?&JP}mo6vr;Vcc3{x;u;79jK>>P8orxAGr6kmH0! zd#{M^YCQ?$vvw&Y21u#;Y&jR)xFV}3Az@;GX*U-aS4G6sbDS3-K!vwx36-Y+9?fmFjC_dDq{Li5`W@$WR(gu(N|>k9WRc_;ws&qI<8DK1>mD715*GH{G+5gmZQ{iDJ1BXC%!{d?!fc9(Ah3I)y`9N;rvP*)*+P9p*(ObF zKtzPPrYN&a7OaqMiLe3Qlay;fA++-~iW^Ijkx(wpEcoBTEmj2}Ft^EA6_K}6$mrX- ztK6)wMrmX~Vc;mG_?|V=(D1QAn*{(^)lG^MvDeBuOt|!RKzFg3^1e*5+Kzzez2t-o zeg0c+zic3t@7~}Ryv+!(JGK~e&UmtJYX7>Iq%qQuJf2vC@_X(!2?4nV6=u!oLEOgn z8`5R*2Kh+N>U44=9OQ*v3)RN(Uc2dkR0s3=(U_na%w$qlia)o%zxC+mxrzY)axeot(8PYj z{ew;Qa=BM@s=mq_Q?{+0nv`kN=P6c_4t*^u^TKvI&x`kkTQ%7Q-N^ajjJ!K2PCeb{ zt2OIt*8)@}1@W?j6bB8d{LhQ_4uIJtVi&R#k)+kO3pkAqe9Z)e(a0YD*@R!_U#N9^ zdKiz)a8R|mvGDf2HQ{egvO+Ea8r*~;)>9?&kYb~7cs~Xt04Br%E4DF@GGX-Trb^=E zYFh5y^e{UpFnDC;`K2aUyZnv-&E+;hJv|8TnXe}4ZRLH}%rpC6vp)E(Xcs!ptHLI2;v49f|F)-6( zQ-je5TUOrKu+}2sF056iw$+eI({)i8<1S@96z7mv74KU2AG_aJMT~gsCU2Wx3?pdJ zkfui1VD+q4F13CCTZ*MpIO@<(ROrm0;Sv12ZIFEa;;a@5Y zOE;M^3F+@65eC4eTF~mXrmke=D#iU!vo2X6uyDp#2etLpD6HD5X#bjMX2?K3Dp|ic z_?6>>CJd?|y?hynFs#MHD5~O zmt(i3f9LuR;mm2E#FwNHPd1R(Z=`#g{}da8n#ZaeU*!R$^yX;CeT&s43SR>V0B%aI z%GW@wivGc3%JNpRg%iW@tX3)g1_ zIG9D@8o)SZAO@38b^D9`EMt=vTj~B5OCf&zs$!b$?&<}W1|)d{K;d0u2#9I(VnGfSyA=Dy(40HpqCkmiOCtY%hraa z7Q7k?U9(1$W=@D2za9+3PmW1kmNAFT@v|7x7-O^in7>kQmVb;6FpU}=}(! zY^p_*{W>Q5<{gr)oJe1ESc$~2K?`u+;OBext!9J3Db^Bo5L`s zxKhIbJY}dkmDa&wDmwZKMlVyoMZgT&AyCZHk%9vupn#>Rp{K{>sLCR>AEF2=u8y=eF_m?@_Ijq#>&zPs_4L5^2aH6H@DR9YRj@s z*&s&i=)-q;Y&4tp?|6SLEv)3`-a=FWFclJ4g-A{_Be%W>W}B4jTRWvWwcK=DlT(l+ zmxEFhU_t^@b3P3CRWQ~IVu@q-!$MmFTdgBc6I!+e&-v^nwYx+trp6MciZM`HSwPF& z%%zWfYbTT6?I%s++|5&Ei$W1OB}CKBSf)ZueZ?J&2S+PcY*eSPvt3qLcOQ2<6dYgO zC8-3~?wp700_8ONIhF+GRKeJCy_~V3lmTA@UI-DsDJZcAv9RAR8Sc0iyb2?oT^sxx zp_g29tR}i{^Hjd&YT1|zMyq3|mAx&+=0j!1*p1Mg@dImA;fQhxk>s)>45HJ^xMfu0 zrc;h*vUv8avTi7T%q8z?d#p$_)bCMCsY)dI!-V$Dq#~8IvXiN>%bbP=Qhvxv$K_Y! zLV@W~r}_F_S90+ZHYMCl8`ny$)!miRgsrV(^aNnGKIrEwrwYyE=kfS5TH3rW{39q~ z!z(yL?a|V_?O}~l!)fqNZhWCZ5+qtu7H6#2p2LZo1xps>iaLQ`*YpN`|9RvWuN6{G zWy;&)UNf1e!zEBFC380~AkigdR0Fg-Z2Bi_m_>Spfu_wzApxvzEY&ysM+;a^J+)z6 zsyftKJnU(JvW!TQi{KTwDkKVH(S&r9kN(l8uI)?UEe1#zfmVmBc-vwD(aGhYN`3WP zSn}*>j>(?aSVqvN*tYWXxjJwBsGw^eV!yV%`*jfo)2XZg27jW0sMD8vUHn$$qx*a- z9QXNNU<|WbRIj8?YH^&WVz)U(@dYvZZRfv+0ouxba+-b>QbfNBDWd;*18HnxZQ^WV zW8i3I;`qOP1}XR3?*BpP{-Bl|GYC8+T+^_CPjIx)gREv`YmjN+Bs6MuxWQmQw*^1C zX|1@D@HbEkitnpYtKF`wxVB0w`|3>DSRasK4777|^Yi&~eHq22DWbJ>A9u+-L9?`a zd1-Ck_;#pzd|vK+t}&o|l*J-efu0wjZCdRbR6VIiErw4<15ySR&AzC&f}W6F43$Y-j6!AZO2o=Z3YB^d0+_u8EzK_( zj#G_TtqjEjxIvT;kYCt7zC|?~ls0;)fdCC(P;?==3|PY#2&+9w7?sE#mPup-^HEUK zXSG~W^^rQO-KW)}sN9F5_$^XdU4KCX1_KJEpji6Rsgea_j%A2Qq4(jle9oY3K1b6k zq0BC|(zk5sdJ&QB_bnL}5;8%B$~Lp~DUOn|jnJhCD`C0#E5Img`Fl9E*-2;`{*V1! zhQsKKscF36SQlEx_=oe9(D)oj#(HR$Ux5hunVa9SL?W!d4HhY^UTqzf*Nx{|bP}`Z zj>ZcT35+Mi1d0EFGIKyBe*YmEo}MRjv=Rm(0Yu>$>JBFG6t(BU{2chhE4_|Iw1fk& zM?ExaPYX|GVCk>8+--QeLYCk^bROY#8dp+qrfJ0jtvX|-(b3)Caxp_r_dCOZM(`wq z@IF9_VB*~v6%3MkO`=H_6AGi~dAN8Z2z5PZR0E^x$Zucu4lrWg10l3VzfK4pl7~qd zi6^wGR1`ajVHX;|q@T#0h!t`uTn;BXOcoNUdotZC>JiYF95aqskuTy`6ZQLxM!F{c z3@w)qFF2rd-cJ*O&gTN*6~I|v*gqNm#Iv}!AV?6Lqel6G!eSyAZoOzJ%^w=aTRY#D zk8QHF`pG#mSRi<9ep@&2lcUDso?$?RWiU*%!XMo80>EEfrr$lX(O0v=?#@Mp&!%%H zyc+hW*_~|Nn|yOeA+8j@e}oqb5A4+51hhK_{1plL zDs^U+{;6y%*26h)ZeSnR0pu$jip9+j(yKzd>4uv?TtV zDBWHa)ah`@Hhipm!DeiGAlt2&`}Vtbs5OP;(k!&-8Y{^C-GAcVbn8tgq*OpEc3+*%?<>dKpga7@(?uuOW9r|@jns8EVom6u7 zHa4c&4JXpT$LXnK<>k6;*7}JYrx+*m!gQoHHj=>4;R&03SU7QYXCUHn&Rsk8ly!0s0xDPSjSnY@Q0ex*0D+)m!>1Vu^`AT2eYU(dJ%R)G0h&6 z3=n?)V}2sOsZ<|uUK*4^9jr4wg#RfaqBWTTPPkRB%`lVto}O$_RHG=yKbc%wU=NtQ zydcpOp@1QvT=7+;5ho%fG7k)*zW}$TjOZMSxR;b-^CbX(5~4hJY=&BRhzORcq=-(_ zlq!n>_7!sd4tyhjA)AuH;ffTle{A(cCICjsW-O4*{Z)#qlbn? zE{NLi^bw$I&PG!KAWyF}h@}wV(}CDQ7%=Pqlr5ZxqmobYMXx?ye_bBfGZ`LGcIIw= z@$_Kn)T<3rcJ-b&#vQDMb%tREVI}QdU81|bM;%tvN9z}@ha*YV5F`L-pSXO+zl5EY^_^I1wsvdPa_Suv8zRGp-*X>~JZ6tvS64jL}GK?=4q=-}fKZM<5cW6PADB#$(ZQIF- zZQHhO^Tf7o+qUhT*miDa?x#Cz?s~uU+CN~guIlcpry#|_yC3BEVkJCXs@-m%m|*x_ zq7#HkRF^pr<@yuIOOCsJTPwy|fe}8sMuM88u4)Hao%LBsnxsMXtc~WtAFl2*3t854 z@fqFAu16BFG2+X^QI`PL(s~dkRDH_yS9!Vah34Wf$$M^~kUnr2x-4 zB+VEIF9Th=IKX^_C$5BFDdl1XFdgkS6=HTwDpBQ1`v273>&^6d74p%)MNXgO)h@nu zLBCrqn?+6e9i0tVhx1#B?4&Ht<_w_VSFy=A1Ejnw&<&}|2m4Kc_Wjve2qHZR{95%( zjq;w|z2&5+XF()PGAd2Rvg~_Mj!ElTw%E5FB-YzAoP)wp_NE~@lQ-klPJf&`4`E

g^y68rqkYtp0*2hh0zB5*~lE3XtTA-Y@n1&2`d1J7WZR_q;@%j0m> zCzo|4jFc>37S3q=UD#aF`WUu`;v;X)LFMY8fZ9(2yiOI3BW6!#mo)F=3L!kli4i9R zOwxXg@k>ENXs_kmv&^qs+j!L1qjkMU$@eL%W~v@|Ax=0yB%gZ|{y49xRayOBd7Pr( z&rK5A_tn7SN#um4v9}`X0@oH>BuLQ&&>PgWp|09q8= zCo`y&8vJPv4k?m4P$ zy(W1e>^sPu;QkCR-E*^o_kfbGFj=PjFAQluNO^3{uhGXV zkBm`5*&KTnQ0jbi=De>)NkEnA?d*B^=}e=2+br-FP!;{E}gyEI7uE6mpgXmlvPf8uqzea+#F=7OMR5By>co`cWd?C+nLewn~; z1sa8PR1UsSfW_YnlE}F~pYY=)osBCx0cC8gURNzqCrUP=2_&kKyp~q+++zeop7?M&^;nUcbC zMfAygIPQB*YUMG1))pSH6`uzc-f*Q|Utkdhj2?gFgl#-%=?tz;VGBeJ+55(h< z?Boc`QsopnBA|fg712ry^`{IUB~jo)13cN)8_?cBnyGq#7R+;IiAw?0b`^un3d(ZX zGfxr=-~x<41TO840KuYE?P@`!!wEncJEfE8lm@|_RIHN#=D`w!MicacW`A<9Z53#; zOl%JwNjfEffP5bh{Nd1?t?X|e*Wk5A`6`woZyW=Bc*wg9IO?A%(-%`DIq~j^6EXgY zDrtadbxQ_U3MAr?u)aeWAA0k=RUqD0Q=Fg^18L4#p=LptmHqM4!|ks?^A3>?*+D)0V-)WM~{dP1GmFJmW~t{tpE&i>q7J}U==|00WPglTY< zKC9H6EU{Kcb<4|T6mHAASrRY3T%Whyq(NOKN&nQb@A{~n6Am<%s0LpY4XA;4k|u--=C~_h71JU6Fn49d z)#56-f#ay4HYQSR)Q6$M&ZP&}1Weu6elWk1X9APgaRE4@gL6WN#8n5kd|M^kbr7H9 zGN<4w-=xgn&420>pa2iKtq)?aJ9kdD)2o~ix;o-kkS;TG1J?(eDbJd!37A6Lp4q!d zyxzOOqllWZ>&nVapUg0UirUrTE z)G_e>9jr#a`Cbmj^v8+^* zW%+PVgUs?3fyF4O26wUt8zEQS9$lZWl|$R9kOx7SJ&<@96Z|-zH9MY_NeH|79Mj}S zzUhY3T8ox7$rk5(C)=Z&%qg`%#lh|bBK^P;u0rkIAO}!05{-p&Tdse+-)HsP0DK5Z zvZl2)LFL>@i4C+eKHNkb%_92s*^#ZrwY+d3=t{JJ#3RAnNn~B{IYslxn%^Ev>H>n`Z@RO-VV8lU?_M~g73}1Qd7}v@m@scJBT6bys$p%RN z%}x11!QbH3k4*^o{`xkI^bih$;1LCQo+_$+@6iPd z!hFITjB7)1fyiV@SSf{hN*PUst$o1i2fHf#SDBaiNxBVDm-<|j~Idtrgol0Y@;s{U;QGv58hzTb1l}B zK*TaR!0Vi1xKfI<;5S!>&e?Aj|ExmM5x@1lDeg|lH1~%3EU}?UdFqw3cfW~nHZ;AM z4joTQkV?ww(COUHUt0jm4%@^jh*ImrCU5iUD|i|?v&b9vR}#7@*2Gp@q9jSzoKzRX zv8yFJio4@@@>N?-%R3@?I$N^GM$dq%X*4F-8gG0^w)Wn37Uy zG)xm*Q3nAN=Z?6NVh8$<3_ydnab8|Pqvr~si63iLnL4Z23{cv=1NkbZyI6ASMUbrvKPU(*^vIMb zQ$Lu>r^F|x$3ROioO7cy3?NS5bpEtHp6}D2ctVM3q|z1ORIL&RA$P`(=9VR-L=Qhv zFUw$Uf%viNgoh-^1*Hnees$3v9!}yO2B0T!!d!Fds82It+fcqjJ^`pm&_Oh0^IPXPpdUlk%`Uk$%nly+sEB zYJfz%-O05ErlB_@+C52r?!<4ol&5UTQdpJzdPZyx8xQcVy&0*GrQ)csn3P)fu#CYZ zqcu^85U;UKOS!seYDu-NVNp*e#x-q;_Sm}FgLk_FO=9d$*+Y;&z}7ZurRpIlrOnW2 zD-AlVxpBUW0-P4kLXRonDm6%R^SS zc06fR37ZOR{gg#vnaxjhnbr*F-u!Gp%j}Ie>P7DQ(#)qQFXPU-6=&MAGikar@n4CN zhfFCv)pUO32CLl|>11bJEPayW_?EL%3pM#s>RsovvFG8AE|0!eJq$E(Asxmv_geC^ zq$-_)a+W=nW93cjOP@a1CPkX=N1$3 z8qprrnwr#gy5bX)H3>OR&kA95lhqFWSv@YUmBVoP$%0s(j1;m;>Ol(YQ8q#C1z0P- z?;cS|MJ0)d z8ZgCOQ1+S$yU*(Ly^D7eqat5o(Q_{10CO}X*z5tbRyp0C=AmZgVZ~ag)vA-K3#<2wT3`cO% z)^UAyjtr!3@rmky;YS+)fuUugkyKBTuEj?Ozn&qzdItXjIYoDBW}9{BJV3l$zVb*- zg};?ze9Cje7xbhILucD)NaEOQ*?AW7NwITXN(tn1mXu_ql`%_o0%`|LZfS`lno_ds z(}b)85uou$_~hX4pp^2mzveOCt5`gzVn40FnjPTROmZk_iI20}3rI zOu|p@VrrHzkcEvF8k}g%+PvNOY1r0oQPr~1y3cJd(anB&c+*}X$Hzj27e+;f{0>|{ z(_`W8RVlXB`uE`0xo+SOu|&dSzA4zs8d!6pI_4fE&?` z^`o2D8M{jIU|BY8WpHXu!HgrL=k0jYP=8ojv()%hm(sCI=G&t8((0vpkvU^+5?OLfj#UC6p@5orc1oW-B6oS^2~Z%-#c6gP_#)lviMQmL}yMearj>o ztEzr(Dh`TK@2F%5@{l?J1q@PPtdI`uxrtU6SD>!8a|w{PlsTq$^&D9~20jh{cc?49 zTfmk&Z(AOSw?CG^^4NAI%mYbaoa`NQJEBG#&C^MsFgK2=#>fb;I$T0E7jN%=d_HML zI%DjT;~7wq97pAoZ*h0#g1MRiP*VTYKS0bYSd8+e;{$-fz#tpJ6&~BJ4rDU;2RMg{ ztsO$rXM(?N7_6`1-v>Os+C>|nDIQH)J-44?7jQ_ML?ebK2;?}4!`YYd;Vyz_q!Vk5 z+M;bIw@+IIoQT~0Mce_>v#b2QSqTkxGD5!TLC1LLC~@n*Z7s)hXU7~^k{0Urdobzo zoI%znO?ndUtpXVp0@Ok>nf4f#91!3e@bj1D_x09wT-13ff21ZxFp(KWbPy|f!tgXJ zR<;aMlJ2^odT!BWBpdaWU1SWtJBEH*1%{IQ!`u{2^HV`A3hQgb(hGrBXw5bL)fn}! ze4to2{^mt8{tNn*Ueg9DY4+))VZtWS-?*n*XHx><&8c8=JZ6l^Bv&R4hzO_6Mf{>N zMWP|*Q@rxN&|FOX8AWWxW8gE8DxfRQ1w~uRlqxuB{HH|@*zz)!?_HgdUS-F?4ao-4 zeYcXV$LFnpNebDBRgeZ)drw0CPE_@*=1faZX$3_63bLEV0C+%2!%>LQY?1SgG6)={wQtd{g6547y$=Kaa(A-YAz zw!x)!_nwZ}G0+Z?%zJAN1SSUO_zLrDnYtQl7^_&Qa4`#R&R;`YmDLo>KOiSpWA==q z338n0rv?{uLMSJ1RXg~ULK{AA@2!T_-aG+&o0a`-iyo8YaXq9tcPK&#d3>BJfkdIY zK0>A8ot+sOW1hfkfkPM(C9( z$hhWK>;AO-9b-3Km~tN+pYwY~u?B>8#A&!UJa>Wz5<=Ymr|Zk9B=PFP;xplu$&*@T zUP8e4wReE({FhC`f&X1kyvq#$_;bTfc*t zi2&n4;m0@FoA(kQ&Io1{oj^mU>z|^ao&9Il6kL8*z{=^t3bct1(*`DQy1{7pJ*Nx~ zph{}ji`$)JQ0AwX+4nH)e-unNxA-+f=3@EeOl;d1(6i|(x1^5tpG^{;>ht1+1X?GG zYmVY=&T{A<#2Mm_rFzCzrZ=J^FQ7{EiTXIJ zkOl`M`b6;mQ42%oPLX#RYLdi`*$#5x?PJpFE(&6WRy%Tol5+XDZU{LxF;ZPyqRDiW zqOK4-z<4MZ@;t(=7{S)@@t^V$|D!+Vss#U8s-S>1cl65vHK5|!YjbbfdS43hSY9Vv zPH!#8jbNFz;+n0%ZcMk8cB|XRDw#G_Bygv3nVqrSVFd^z6~k?t<4027^IYGtF!>Y* zHvuGlNkQ1Vh@u%lskvT1%ly+=i}`()tqfP#F<#pR^bE{xRg@R-H(kRW^Re$lU~SpB z;@NRQ1tE48th8Hq5fr~-@g@_(+QrlR*<;aWgY$UvZCTG5cJgI7csky== zut3tPw87Xs;q$&}KY8O`5Tu*k2GvMW z{QOS-izL-+p_V5-ps}kZgNz?P*(2AWTvWL-!F~4tc>=c+2#@~&?pp~0XUhS^U_FTI zUtRP;*ciOD$fVUyt=8x3d<0|9dOO4hhX`%tZG0~Asx~Gp>?0h-`L3?)C3AI~KHf`) zu&vGjxVpVOM}$9WL_9iqx;f1syv!s3B>3jog!t5zjTnsW%v~>JG1tFng*7S#c^UTy z`liOh-yYDObD5$Qb5gG@0nC)J^6L@R^s4efOTyQF@pcHmsCs3;X|V_0-X_DgoN}cX z96ucKNqQ*~pe!|sx4MUbfWDX&=tGl30656B)B#{w^~cqVdoY}4e}+BXdZATLt8dj? zQQ=>xcTfpdQiZ)h1c9q0CV06#WZvtG#8BK6Egk3o+p&))o zImE&#f{_dl{AFgWYlHWdLwAaqr$ST)37iG14L+T8(cJ!Sq}`k}^|;cRv}QkOvihmb zgRmq`1NMT^1D3&~m>n_)!{1{Q)gck90i8p*K=7TTINin&6$ONf-4$^MPvT%e9!Vj_ ztwd%$x6g($M$KJ+{A-J>E`8);fcq(VVjw&2X7CX4ftHm-bINevXC+aDm4^U)=>zc3RqF{r@VsO8?4qyvQ%ZSh zPDBtAjCgdKj!x465_PRwq^IO$R4m;znNE|O|SDqbw|IKacbnKV#PBDU= zcGnCcFQLp=l3fccaQ8DCO;oYn(G0OOS4=WPg^x5X8JAi7!(%Xji!hjINY#cFWh8fF zPxm=`m~okj2@~ z3x+E6<~*8v4t?H%F*Oa^K0lvailWwhkmS#bao13|vFGj^;tb9Tm>p3&w~95#CI2u< zRJ_@MM<={rluENIjhiX63Pnw+X&D4_q~4=Gp9&VKP7i@*I963=GWa|X!eBB>v}9zm z7LcpOR?H8A4gXW9Sj^|;>zhrT;9AOUw>j$_+)n-4F_U7L+v_SS|8J*#qp%I@m91^2 zgbs7arx>tL_gFIvry3#(>6v&Ka@+GA7ww<(A}!9FTq#cdcY(5alcz2nowK=OHSu3E zP%xC!49n}YfM83gQh;VS4iJ!V)O1DMsBs)6@d*nD$>p7;M@rku&WJm6-C*3=F1VP3 zyl7Hm7^j(SZPn%niGcif1GUQGA})U^TT#5l&^>WR4i)j_f$j8(*n>>^*+B}Cr?4RV zVMQ3intg>ZK=hq&L0a4Zv`bfmIuQ7@hIY#1nQ{Mth&Fu}MGMuEkB-(a;t!svTEkalJ4 z0ii-0n`&e?o!?*C9~dwa1Y4U0N@}eE4y>KzH<%v1J>vtve!t%CCYj+G4D9m3RzybPHy%ld`l>lqP2m&tKw4ry3|UQlLdV10-vapbpeMIGWa9 znMm}ZB?9&p?z>XCiS@dMQHMSW!Y-s9r-bN4X|&{lkaaly`B`r(D%6@uT`l-tf^wv0 z$nx>waG*iU*_VtQZ5&^CV9c^V`<&C zypa-%`t<%s{FM{}#QZj&RL5M=ipyb=`!yrW5-C1+&aRpCV|}gK6btIF{0Wq|O}c^x ztPrR;!O({R{YH70&>cT}PA!G4xY1EROGP%2W))k^oz=)01Qnpao5EDqfsqLj$zoPT z-=S0Uz9h>(6>UrY@-9?{pEnr~wvjG6%72J-mz!kc|HeeveE7M(+2%N(3CY-EzB4TG zUixEPl+Kmry`i+C0uIo;fE5pyfqm{6U*Lh?&!??&lRan;1oBWzUg5SMMDRN=3vY~5 z3@wW(BbN;=t*??iX9-2^5#&qvE1Rkl$CQp`2+)}X2=OSEV~XOe!ZwVmD(*F-5AC~b z_JQ_7F7TH^IG0>%ny|li{0R&8?By+QCgqP2JRmsOcHzRTW{7Qwbk(z~Suc@n{1@*; zII|tKUDMLa7%^Jd8!PbeuCKm49U=HT-+KUNa|M^g<2Jp%cqE%y=!3dZAxioQ6>*Wv zc_kGJEx7ol=+J0EN#Ro*`uQi${SxoD)g@#=&!UExQKd1DIY`E~?cL!*&j3JK`q#qw z`StzfLCZ43Zm~|xiPI+ACoEWCeBuc`-D;WuL|nU8Jejtd<*AzW8=2xOjwJbDr?eBGiMBdmV9H2-X zg@LCr&Hz^dOJOqk5o=y{t_`HlBQgx~sXX>Z8yrTlUcI@-Fa51HN_0b4brw<*vsqs^ zO2iLL_^=_35H>9W!4I2$%xBSSD9z(WiS}<)dH_UdUVll)dK7`hwTHs^%wxOpW^O$4 z0BD|VBzC@)iP&*Lthj7JxyOw09oavWV(3yEJvC9}0j>@;W)Jh4T@&dUNI_9Qym(#F z--QW$e2Rnx@SkRlM62xzwa9CMq2Z95v@sVtamkRLYt^o8*tVNY zM{$}SVPAIW9~LB3puT-3g)zwWO*2S3OZpn%uFPlLg4LX_u+I}*;Lw}JL~&wCcjV5l zt!C3v@F!%Tdc<}Jj|ry_L_h_VZ)L8EMappfNkdJU$yZLp8~oOk3S0dfd{7}WDUNV1 z2luS5b1`cf5Wn_Z)H=D3fXbz&o0CXGYFhOyPh;arv{{pWl(@Th@PASE9>@W_&DHb zHGUJJ1KC~X1ia!LYtZJv^I~iOf}gJLuXV*W|1Zv^hu+x-Fx)gzD~-8K4IkQ>uQ$nt z@J^}o6^CYay}Ht|4Kq&;>u6p-!u;Ea48Yi0DY~PkvJuI$iq$NR>oMBe09N(3P@v1U z6ws8MKhW>}Xk0`EmYz)R`R|J#w@Zvn!&JSuN;v@vS!moYWaW0#N*XfxdUPAG1 zA66GHdt!QjUGY}sFNbMgFXf}3tHf5z(N;h83y@>2Y_rctX-yL)Jh(os6sAXm;s{id zog%3{qL_Z%_&${+WP?cl(sB{rYg|LjJG!UTRf?=Dyw5{Mqgh2ly69|K>RVXfDa+If zuUSfcF_g++RHV-Dy*ZLzT9_yMr@ zsL}8z9m6hfg*U$6)J+X-0Fr>!NTfp=_>kV1nv391;)vS);`pxZegt2t7%9V3;583au!ap4NAMgNg0w0a2CvvVfre z1vrNfU}~;+3|JR!vl@{$ph=IY#Ykl6D0Y4t7Wl=I@bwt%8gAMohiX-Yu9m0u#|-Zi zdKuN{CNhj9qR9FNZrFAaT7k}$RN6icxt`ciWy)R&L>+@dW2jFKhH~V36vtbWt#tlp(1Q>FWvnF&| zul?WUkH7s!R_YdXX}A4`1PAIvyKR9t-8_M-UIC^yb%dFHgk$)lk(i;)eNJ}U*m%Gl zcw#|LuC@xXVb~tpq-|uz6c0t)(ry`K#`d}8Izys|42-$1wvL!(kevJy9tvka^=NID zJW!u38~2}&>x8&n$X znq=-scgp6)t zu|#(U#CCL@?n-nWulc7*1Rbw8xel&zKi3?LYxX9(+f?Glz%-#t@2n;cOtWEc$pvFI zz&PA#$qa?T{^mm>&@!bH5GpY_S^dRX4VZQ?G6f+&Hs+4X9N_00DT;`~FAdf&FCN;_ zMP?`815FlA6OrfDg5WQ`Joj>Qqf2U*;I7eYlqP$Fbxi3TV@u}5_du`IfFiftZbf`! zHadiPtb`#LOcKj}RK}RaEIwO^+H)v;@<*r-D^v@isM$!$q>u4(@VHno%m%C_RdL5= z<_xBkcDcTe7C)tZuXDt_Z2Sx?7X%`e?8$foZ&o8%*OnBX#BEn_X+&< zFnm+1o6WQ7=i^-WV@Z78{_>@i4UZT2y?AnzD`+d!=;^)Iu#`*?5O2}y&jd&Cj|2Rj z#op9h7m`PvHoD_8Gahn8gan6E?(*Esn$mskwb4>>a|>!>u-kG5UN{umVqTeQgN7J_ zRRy|zWZw%k8ate_hNC*cB~dhAErq=QKquf-uWDA%WBsCL@xSG zMgsM|h&qwvo;wK>#f<^rCJfNpbj+T^R2KZYNrD0t2lHI$jDPgTJ;r?j-fY4r zhFyhHL1qmH)y0K`L)0?;O|}^@7e6cQEY6z#X?EjjyRrIj6=%4+v^VH6?yb+{P%p>v zbQI~bjT1KA!qIEcLRj!+A}Yp<&RR>SzS8z{?s~Q*geTY$Bm;>)iAPk|XBLH_h&n$; znyfAC4hv1kp&^qNiF6_pyF>%LnUmCJN6_ubmBmY}N&3vs{~)`}BhUvm!2kea5CH)2 z|Mw;hYYS5oBM&2MlmAefU9G0|>olVHzSMF^s&KA&UTv%T)F+4!k-=*224|P}L^8lU zUmD&Ha74VU-nf|p06QqDz&04`&%Dii4RB+TbqZ0fVvGFNlJkAuoq%VE|Fb)7OdZK7 zH7{JUox7=>HlmkU{aWc<;m(M1rf!rbV;qlWBSNhNu2!N~B|241gA$Ms{~nQAEs&xj zr01KHEyBb|NWM3wIb`+Vzt0C+tbp}zAW@dnrQ*knOErn0=Os{#d;O>mPcBtAq#%x; zZ{L8p&87V{k^Z8gK+qFm0?Wxe2+JwjBy<-j8&f6o6*>zXiG2Wg0u&R3AE_QIV(@>E z8Ljz%ajUP=*(CCZe$qENbbe7Ktd~dyO!gYXJ&y;Z?B`98ONSZY3{vMNxjV6ySlcL$hWOHzr~oc8_V?0 z>&-YXk$zlkHKRilOlv!mJ=7Z=Rx{)Ny%C*xWY!6C;?i1JOXra4qchVqhC5z^4$WM< zzD0{&X=XMJ2OF0pWjX zZSjS6hwj~9NSD!g*thO*zFP}1xNbP=aYJ)WrCHUegSXvUFPg*)r0-*1WZ28v01M;Q zh6fRUeX}*dN5xuS!7#tHtWz8Js#Z-S5>kV_m$l{GYZ8DGA+q@wJG)ty>M>iLf}dYA zzAbjZ^ZPSkHbJbeucv>vn$rmV?1`1}ArOWr857g;Em+?C5Th6~uJiMGQjY8MyVZZX z#ahG6;TuzkRA@Lq1@@LUe$GenA9yi3rYW&Y9qv*=vs|sH! zB2JtDPCxk(UiY~q4+T2;X6Dhw$au_4IxAi)hTz!5wQh=(C7=Vz(_7}kjzNz9?N%9z zX{rmH+}rtSZd!^Nu@dvT|O0`PP2YV{~HcYd${|9Y@sN4nE)JV=Ry%XGItKA)^$Qp%FM@cAz9S_f7kC!%b^)baww3NvVR z4V??5bZwHNdxTGrFW0a}1hL({Mt}=7Nt$IH#p|>9f=dj(@(7T(9@BnhUihOz$KwsV z*F{wArZI}|Tb5Kn)>yj?@?kTfExjkSpF@WEsFhTRxfnJ`r{nNEP{;L&ukG|AGs^d0 zl`v=g3#VeQfmu$3#&qgiU!OdJCsmfwx+f=dTzr=@TZW(NHlfF`P$p$S8hO%1_x#nG zwr_Pt1?2ECE0SdEkreYc7rhd!4ra33jPlds7OVH|ama^cONxLyuT>Ps?LW)vJ&yyG zP3!F8o>_-CF3d$@Fq%{)KJJOQB!cESf^Zt_c~}TN%&6Gz#6X#|B=tcmVbXDqKU4bA z&9m6alj-GV%C;6KA`9R0g6Sj0t6c={`g4x5sKga%vdjICxV@TyJ>k*Eo=Fpzo#oPL zwH&gB(q7>;a3%@@d#qzGk1M|VcvZHHOZ|AWX7%nwg-&5q#gk&Dh!+cl>p@fAewS_H za+b-pU}7M|o$?wy!lZ+$d(<=(3kRA}V~` zp=iUZ>WjiyDm#WoOK-}XSlCb~T8EHo%4id=Jqj&;j>Sh&>rveeU&v7=Li6W}5riAc z&$O;HJm$R}Zuc&pFV-u@&6z)WWfxHf)w7xK0sk{(8H20Y_WwS|CnEs>5dQBW%jOpr zw)nrzh5sRFU8&jF9dICg&(vbnS3oJI?Cgj+XC(=)i9$n4EyRzll&3>FuTQD)`1%vdU4gN}N5K^8l+9`GYlxCMFGUvka#uNT z=onGtlK`g>#gG8}Lh6p;SlAMAUECFF^HBS5rlkM?XEpZrO&&z?V za_SX7O2}I!>c-MFG~#LAvGV5n3KZILo`fasD?{$rZ2%ID(uE9luSL~kG@6`8iO$5UYAN@z$AGhSgB=^AAY{WH z{Vy@OBJf|tdevD4bAr9=$`Sbd(#w|Ss=nw=v``kb$L$a0@OGuwX~lP{9wcsLMW z4v*I0aBYNBKtJQSoW$mvgnu{Dcsl2UelP$fL0Zs#!2DoZUNHIEcgBEDVm0=H35nhS z+ZNY{ThFMNW%mL5ML$gY=S5ZLZx#-XNM1Yy}-0{4O8uDsj|kdH-kX)!wE z!^Rjy^|4J|5!wewBXIjJ8PneKgJh!-Hk`U0{8O^Z4@G#JqMi15_5lJ0z`d9l!LG!j ziTu65dFyY;Wg(*sV8~mzoAr(5u3|Sb0lbsoqtmzP_%QF}QI{8s{hO#qG(!VqIPQ>!UvG_tKdvjy;QV z6;So^a#Wx})8Kq=yGqM7^@ppiDs$sERXk1IC6- z9??qv=#Fm|hVlkhR*3Ge4N~j}C*RJ(jT6m4 zj-Hm)Enhk*4_krsoB|;whilueNK!sT7_PNfg?By-d%ZKhcXt5R`J6_*6l0qgY`Do6 zf4*wv9~%)=#YYvZ5gM0pFy_v8C>ID8nx@s6c9g}f1l zkf59OZm479NwTfs`*_K+ftevC1KOo6 zXyZ{XPeLgjRl?QQ_=Q{8gG?y3*T?yfaIKddUB`Uzd#;U~d}vyZ6M%}3j#75YlZ)(! ze6XEVgwnJHsS>#I!5$Cqxmq$!XYUr1wYw8H8EK%Ngx$tarn2A94V`{*W~zsIky9LTeL*{P@Y3PXBjTvAs5kgmJW z5=RtmOac79m$fN*Y0$s55_|ZT7#>JWYZ{_wklLa%cenV7dSqEq2Ppt{N)4@8hLA) z;`M6{$wv-UpKMx^gz62(I~kM7=8Y@{JUb0BaZoVhME?YEhNwpf#|M{$GiwONNhcrE zr2rjs9g30Y_u|GrhbD;-OL>YhQw1nctctAwzYoj*aq=x9zXgWTgcHiud580YpK*;V zl7&c`?8PUsg?BML z5JDl-G@t-Sg7F|ClV7%rlPD3vNJs*FHBT4+9(_zV;;fV)n64KZ|I{XySt4dNPZ>u? zgdE=^bI8rU72Ya(3LZ_8kR%TrrEgu4*5k+7R8$6W0E6n0$O&Sb9^jKxIHzNGu2ykA(vCuaPLqX|}EzFI?8lmK<)bkM0!Bf+;;SVES>l zo$b~g?x6I>ZeAYhyR!RcG0IDO7OuZodrx{6-mJ3+&BHD{c7{p#afKA!1;bwdPUqvx zffHRF9L?-M1@?lUnRg!v4f^PZF*GycEkIv(X4jax<8uRS!E9hOC&#j`U&b1c7whKpg>vBa>c)#5 zEmrjQ$=8-0eY1H)_zJ{hL<^Qx*b^^=R|5t_^7!Gf zL66#T_7WN=S_9T8wZq8;MFhp195L1FhqPH(EPOzRR8;8GCTbc&I(BtcDT15dh?>kL+vs_fb6 znoP1Ai(D23e*Nv8yM3j9qBYNZC|8?>O!0dPis;`LXXJm zA*Lwa#;Za21Th=zhs;1A#tYvl?(slEU%21^glpxT1{G^LZNr!Ce6=ZWEOZZa*5n(g&S{z)jXc1L!55 zAuM+bY!0SN%46=ztdxQwZ{hqk&{4(=}5UwK6-Z$Q_9 z&e1$Ffm!BwjcMggd|%V5i_+QSCHA$c9!WwRc4JMCAe^yVTIF?1f!lT^)Q$Xm6Z(Wa zO`lz6zKvz&2qCGLTfvV05j`xmu1lji3U_;IZZpc~Id|o&Qqu3cVtGYJ0nFoyg_hdZ zV1AtcL;;o1CBmtjipMLZjxpJ2(byxq$CXH?*9__2!3M5V|MYsjKfh5&L>kuR)iemM zNaPPza@pS3Z_f1j^1?y}2p)qgcdMAm%zEl6v$wI-k*~YVs?7LoaK5a5MeiWSp zUV$)hR;4@kAkPyKCqkq{?u;;dc79Mw!~ZheA|X2Zff&n-Q8$c34TzL(T-h8%-tkt5 ziDd0s&j2!6y?rn-gbcMN8_0v#h|gRMZyU9!@SgS^3rg7@>cHa7SSX@Z$hi#%Lg@uJ z5O+r-L)7GxqdJ0flXC3em-8YZW4U!!5F*oi&eBo05%~F)JKQIi9=FMvO5MF+hNfC0 zt2}Sa!Hery5pK-Nh{M=q{0(V*LbgZFCqcEQtDC3 zx;Tu`fz;s&LgDS>JRh6JbFWA4scMG`;e{4V(!K zs+#7_j3A4S<<#Ndq(sL~D%*4=;rbW`^h>x1XM zmu*)UL0Iceh`|Kg8IQb7auenTwCwy2#;4=Vl~~_JZFe9gmM~20ofEfqULB4Qgn5={ z#VujopuU~Bv#Da)BsJ*$co3(%?Zh>^FffDy>LYFufQ}#yMcVsV7b*)VkFeprBd7(0 z?%u^kX#Oj}wwmjlx*`g0BM;wPBYiOh=E`r9r;cD7#vIPOuJuypNexl>+5du9APs7mbON*9#M!6)JsX$WUu?z&O387$Z$xC`X-V_O7Y< z1kT@k;qosUYA|h30nl6(-TlTn z6o2ZTZ2ZY_kh-VS3zx{B|3le31c?$w+1h2>wr$(CZQHhP*|u%nvTfUT-LhYG$3N*8 z-O=yQK}JRn&M41{JZpb@ZHm1{njhi!vzSy`)5oW9s${(p4_n_AsPa{C#4NS z>}x&cjVf`irt=CZrgKQ&sx=1ln`mL=RF)f?kK=DU1B2ya0G5eHVt--F9Qt-3lDQIq zV%mg*5UossGH}n}#3DKzL|c@ayj%3@U0=zhAM-PaD@>qmHDc2zIJKAXAg`f+nzBFX za+UH=$Izub?66AG(hEW!)1+|?{wylG z3+KDwY^Wvm6Mr)|*-@+vBXRGxLKD=VG*!zht~~XZ;Z%})Cj#G8WxU@x!Re{{ zzZ7brEW=dNJ_Cb|I|&;<@LJ>=>#r^o=9|fexu2Eb^MB z>gg#>P`H^@@UMj?f>Kn*wsV$y+jY;5*j`(jX7F(^xiD4Io({S`65bMiCHX0hG!q;A zvBtA;lX$jikkVXOvHTM3xUKbB^i3F^{oBPCqlfH0^DdMdc(HjMq-7i#tP28gRB9Pr8iyo_I-5B?A1|Z{YuW1hLDn zE5rGXz3Te4sBr#!d&s1+swm1Iav=EK)?xGm6PQAGgmXjw0blG`>R_qJ zcG1j@&*F%=U1)F2`44C6(1p^WQ*n7I*S_aAtD)?ap($M5+|4#%@&f9+{qGxde_gME zsnGI3uv51Z;06L*+v;kkBWH`9J1=Km_Mld0_x;WqUUaw`ceih7juQO=k_toXw1VYye>EKEnc6~y8ddO8aXwgn;X5NC>u>gRRUxviWS^&`Je;ZV(uEDCI8l85*-Ku6wSlh*5$+ z4JS#X5riecEo=nyYqjOy3madfk@cF>dtN9jj8&~`UFC0awLsShT@ARJ7gx|NosGV> zLX||7r$^THn|Bb%~LmwY#gP65#Rg*f4ma zgIyILorf3ahZ2Po=qJR+WCP*WexlnQ3McJg;ofCNbc+Jc_AuBZgAX!fGNa2lx&_~z}AKUzqcB5Ca zN~g#}(uiOZHq#(r3}#I#Nw-xZGwe>d?Ao(1ie1VWLqke=jWj<-NpXaKRo5XoLZ0MjmL)MJTMLU!@?z4I7%AVDKUG;Of8z5@SvHS zf|Le{MmE`YC9Ze}l8yrUWI*_!Dug?rXCQ}X$-sO_4Db4S_?vXKk`~ZbgvU-1t zC=cIXN(^-qdlx4-F@VrbX3yktth1L>+8bGf0_e~zf@6sJ&|8iB*V=-JF>qaX>P+R>XR06nr%>((eI;D~0^kEoIo;uK2j$KoCU z&HyNegE+Xvz@UK$;3z<{s5!tFxJ<^-^Ygs zK)Tzk%8?<$r2@8%5WuL24{7!_s*&7;huf^SYt1vW4yR?ivg>}|JlObaMsN1}H8Ps! zp>4ufeb5G@UamWRWYpNL*}P?Zb!q6yXFN=9Z>-w@ELcsW;osV!!})Y-%-G-7p-;YC z;a=&~wqCj2J2rsUZ=x2k^r6jw_4~T4Jiy;^c(=8ShS9d>m0g)N=mMFz0SB00*|qM_ zoxHR)v37Rq@7A!vAvnV3r$rr4V=-{z=NzwqP2J%-e}lmTSUecIc}qi>zc@s)dY=N2 zHo3ROf1J3ix8Xf~l%>n}DA(<%Wu@cBBNNeA!+`#)S8wTKLyRvo*+ z#wl~GdTHBzH?MIhvuV?_jRj>rT@V8ak`3gd1_tIxiw_Edu^$6S-@W-Asyym&H)#ii zWrtlq9Y0&3;3`f;OJ_tJ?8lp0;#^0H7ewxo(E9-c`|oMqCyS2$o0c;%LXlGN{*1}6 z7YrV>fVawpJiq>%H8}Vp6JSIcIwG9)Ph{lhGet%-+wY7)I=`7>*F@%?~R{cV18 z_){x8QNtrpx!Pl;mexZM)yWgLLWQESZT@UH3cbh)6r&1k3eLcsF|DufMjBKO(nji^ zOs?ix;4E~dyRwVvTSB&lm|6?Bxy7hwVvLWYjB%_VayC;>6mN>EL^sXhC=0MEs5eRw z?mRm&R`YgnS}6~qKLi-RRLL4J*GJgr+v?qlqkKYYs|;v2UUBT_`5 zaN*#6zajwzJP;rwLhh_g$rgCo1dl@=-dTvt{CMXd_am%!$Md3C2R;ilBHO)9XMC1m zIX5&;E>RaF33LZYLL^1v#aulE+IaK*)*UvXn|faKn@=$Yx4*rg3y3J zBz$oKqbj*pqz{R_=C?UDKxV?p^L!iVfS@4APZIpg`T_Rez!>8eqrUJ1=uu2MA=Gk8 z`3C_}U}i~3qo34q3BbSAgmDqCf?hIhmN|vUs>@~4tb(UP7m;L>s}M)Qqu&4pSY8yM zeu4>r3qZ`96tk-KPpE?80Lc9q<0)|0K-tK&t@a3{X$XifvWweL*%213xBTVEZr~R3 zC}6~Zu@e9jaD^B(ji7=3Bd4Lg;E%i09F*M|nOpL66EFaBL5t8ASQ5}@#C*?rwL$F< zbHCOX9%9-Sh5K%B6$}GViUfJHa>2Y;j94`c>i#<^+`CtH7Kw?gI;w-G8Y^TLx%HF#4R9HXdIRZ{^V|mNJwd^0ITrwryB_j zuo#TWJ|mtPWZo$Q2Hh!)Ah7HycmZj@wMm$oWO(3708UIW8F~)%mLv@I!FiH8e5A8I z0q8IvtLK+;VUNk2h&ARv5Uuqx!O;y?lcj;WTI0ekY^pxTZdqwL{Ru3jtsU_u0qxVkJSRw9~+H)FPz3xveuP-#Kf1Vk7Zxr4oF0xVw(O;r&T z^{hboE-JpKAUS6LbD}2c036Sm#Cy&P6$!DOeOp=8Jx%*`^-wv}dBT+I>HSiLewuv& zQdD5)nM)l8rhNlwzR{97VhK?xyB94ZH4eSi6KZ3_gs>JN zep&OYF|V-!M;?n?BGNHg;jxMI1<0p9Gb;8@$qB+O_Yoe;4b`9&HTr}u{nK}}<#)Ai;Z-kUl zC4&l|3gyQjZoLX{uo7Y!)CQGa8$~#wSAMPwIYB2ri`K~jqFaTy&^=j7{kkQ)6%_Pw zaVZOD64Pah%y|xUYCL!exDxr=1@x77oC1crla}00**oxSHsywSZN}-CrcGC_*Cfy7 zxVpQg!rv>ESAI$>@7f&d(Xaaq>LIaPsaH847#l1Gf`%}OtDqtT4h|G~W0K8!kPcaR zGcHt!xQYjW45|dYkEfqR4E}RV?mdqUZ%w2seukoDguW-*+RT1o<9m^{V)^Y7{~|z+ z793@R9Mg0Qt5+#L9c|@l2s6*tdLfFSAH$AY6L)_pITiwYuAP27^&Qy2Wy7q~iaoYm z^&e=}l1|1FrIZh#`8{*FZ)C1~%}0oC(W^8*!y@uLZv6X-SgPfjiWl9SG4vr%zwx;b zm5@*fkNkV|9_PjG^f&(H@_Lhd)zdRPGv;_)p&82(y@jX@0Gk8MR5i9nzMFLSl|hoQ z8gM6B5}jo|qddVoFU)mCg3V`(Hd%U!nEDBysfG@I-@YRxCDAiE19)&=T&mVEIkE|r zG8iSr4B8e221TY>CWflK$A{Au>xP0n-I0p#h64Pr7O#pAhrokbzYy7~Z_QfM+L-Fy zH6U0cBdkbnoBX4cUtbA41*_5f;j$EeqfG@UEDxISsvKkH7Hsz{%!7db-cb8{`(kPE zT!(T3%?Jle>d|0O7zO=`h2NK>`;d!&f0#;~e7Ed=1}9 z;H2G|3I^?fMp%>U84@bkF#6J0wuf0PuzUFDZtyS`Cc6A1L;z_ptutE-XW3u~?v-x0 zDX(@7ep~jn;3zDnZgOCD3gwEfNc;YUy@-`!b=aV&`5&3op-Uglv^|>~z4mZ`=hF+L z>(uB^9~g2ATAdy!dL0nVzM6C_DDpa}_3^`!tZ(6bG+sTRgORc1YRgujR@J(1)zEe= zi66Wx$ zBDPv_j-JAzz2WE*LF>+v&Z9KgiP#tyun_b6++rIpzlT2Y1(lda%jFDTsR65k*O*d_ zd$anrM_OcK_&batMfP*(L?8yVXIQc#0ATk@oR9LPi4u3TvupV&IqCQQn4Y!Mz#zQ6 zRH6%gcX)c_!steUb>SWL=$xD-wd1%k|ATXtGnk5Qqx6Ni{&!P)AHj>kIsddRu-B$t z#k?2}*MN|+n-xu&@DYA`F4LTXi~)M%MOSKoR&vNW>HLOR$nNeQro=7bFXk*^4B0hB z9cDZ$WaPm=x6)06OK`HpLBfOUVZ_DFQbHjk4NG|E?f}IAjwRbB7j#=)D9Z!rh~ixD z6h80AytO>7S#!lvXk#VJDa-s~S^3RKZPpL6SwH$vAvYn%5i<4Kvog4_7cT-Kq$E2{ z*#HO_(NL6OGQ3|kLiyK6mLZlPYtHR=mgDlMm~;AJc6ej1adLq+A@4l1QRMX$3@!It zy57*elWZCamhuyFi1H~K!NLF*;&VJm3nN@>g0_%Oh%4s-D|Ke8@n8Ncn5b}RqRgNSvo$q?hRLn&}3$O^aSYN zYU>YyiH(!&d)+h}a8{cv_v6W|wM}JVD4QrPB@WC}Og;Iz-`E?0B0KkY>K}HN%!)bz zVTM^7N8O~&7c_QJ%OYxQQ;elOG2e-K8s7=<9C01qRMX}xme4^giN)`)Jg=c{_2M3M z(TCitYBx{dK%u#OySBgJ2)C|J8P6La{04B!CN*5XLvB13^ireriEz?uAi=5P&cN$51 zFKyshc|Cf9^7L)hDsx32**7E<(Nb&vlqkI`#J~xQH7GI)9o@RVV?8Ib_&3~;3F=2w zVZlj>9ZRfEasArXLYX!`Xxq+!%fLA?Kr+GJ0WBxJ#wCJ4S!Ep10~Om#Ay6#Xiy7kL zJHRlVFTBC86@NKG)a~MVd=(%jor5dElM67rWBw8g0;>zuVx7wlL|L1Al`fmo)qiIk zo6n~Fy`>hBJxv)}cTdV+jQYZRufE@!ykD?!#YE7q)6j9+bY;YJ(6wGz#t=ZLFaoo< z|F<#_q9rn%eOQ_8yhVGH?Z>6p=&r;gfR`kO;8DI?0gb4t(`(}5{cjAo%L*Y6JYV5Z z$(+drTWGhg-RBEyE&u8<8@KNb`maAOWy(XF{m;wj_X^mGDbVP6`Zj?T>{`(lg!fGd zFb~M$O}Qs?C3JNH%X95pQC0c-X+?tZUadh2GcC z@vQz6t0r#%WV5gxJ6CvF0BIB6>D`M6>nDt4v3=KXtJBb^I7fK2A$kXtM`9F7RBC36 zmKd{hOhs3dSKoKXIeZev{e%)7p{7dq;x<6n$9kTg=$ASi`Bm8{b6wB#7BS}jVmAKP zw|V?Px$MKt7R5=>S#O zrI__`?zVgraLT@jG1DQQDVL^k2x55xxDl5GTwprAWRp&wtK=~X6<;3{T-|=)ur9Vh zTXbj}ER6TZlADWJY87at(2oGk4*AH2^=J`l#&dqPe~$(!H!^eg6`CtyrHW!w`ZW|% z=oeiVUOfH$lFXRN=TQXEmNy;OEfa!@5gkw7jX2 zyc1@;n9jK!Yi|yyFY^cfpNn~6GR?LV@~xjW_uhulEjQZ^nQHHW&?1HOdQy%9gqwTY z$*@RK7a*IigVL&&TcO`sf5h!7VQ{TCPK2n9M{aGkj9SoRzoGxDmDI()FHrew-Zp>( z0AT(1R+6jp|7sffsu=zURQUA^6)LKt*#5fshVqrEDM}DEu7rY;YkK?z@f<^C2*jH- zlfAu|k!;+|I{1llB!Z3d6wlc)mb z&WH-y*~;9+XG@tY0scGZPk@THh67_~u`%W$n{j4NgKD9tIz*(<`{Xlk!Kh*>U&AV~ z)I6!ipX%g(4UwgaqY@MPcY1=9ZFc1|%{)vELT@aRMD?G<^^8dl^uJR4YXoGGV(hnI zZR2Uf@)E)67_`i>Z>BRh=`_mBwYye-9&xfO_YKEViEuU#ED$(j`V?BPOOK@pk9z{R z)A1^7&psm!v3+;82eUbM*ivZ9E2x?DgdI>QN%+F4Y zI_hc`^*G)>@>9_2RgdXuudjg7p1!Cc=DKN;cvd{O1bHxEZ}VqWFRj({*u#N($x?9? z+X2%*xn{N`ajfbL1<Lwp_jY} z0d_QulS^lXbe%sG&(GAi_P%UfvNJq#{D&ThYv4ZpJ?i0O+96@gV!8kh`(3s9n2a@N zDfC=3)~<+wHVZddrtK5Le15D9Jnk)z-cM_-2)i7&W;lw+w_)CS+Owp}&F-G`2IXn- zEWH{8ueW(?7fl0Qr6U%cK3btu?=^>FZnCS-R-!|1Y(m5;a}>3o!)W zQ}yUE;jt7)yIbKBqT~Jm$?BjO8|TIH$^oOg`_>WbYr6c$Rd+XAJ{h~J_(BS^Rp!&| zw>4ZlgP|(+{*}K$dQRZGJ$_$*<9_@CF_k6Z;zW`|D8^DM!Cqc2y)2y@dTzH|>5w$cucpz)(?YWV^(tfH zO)W5A2{P3J1v)0tD60@ogcAeBW)KSv)&Plw^-9eMN2>}HJGW7WzXGy$qZW$H8Cr?# z10ocps%8LPsaL|Hf|oJZn@(V%aE=1zDJxwbD<;OQ(!~lDX!)G!>Lc5J+Wh;zMlxJ83LEzFVFxe_6Mlf>w698rOoq943P+w(4*;X$=PxBL40v|?E&8~ zBG(ULa4E}^abP5{z^0FfeZhg59fx;+0nuN>if9Dl+WVVCgv|Zqy&-0*9sI^!hd|-+NTw`oPSXH|5Jp$!0H(|iHp(!%G?u-| z;A$#cyoM|U;9A)DL^cKq#LLfVnr?vml|8b%Z*@XE5I?-VnW3J6&x_wX!8#DxD`BRe z1ja0N2H^zA7YH~ww@3T%45ke_kj$$)Vk;@X;Zg=swhU`~cG{JcJEIQ|)g`9H%ycBs z)@7d%3tqVCeBZ1rX`z}ivAa_c{O)(c+Vn#L)uO10xQ3t_C@+Q)c2tM+h%k?kQMj#z zd-SY8d0Qmh-Imm{jY8_!6dRSf@NvsS_^8VZ~C1H4{X27xfwujpt=6rOb^8LpP zr=+@c^!BAmi-rnyvUFC6scy@$eM~q*9`rfp<+_r{K(UsGyA!oN%vt#wo}R}S@Z35> zt$>~$7Nq5(i^)Zd_De4_)ZTuoR_CtP)fF9$Or>{-vCYg3a>AeKm5FRwOc>` zm$1n)Q{-@B004NK{SOTozryzaJX`*n*V=iLBXReOS|L|jGFLg>#QL^Yu4RX6SPg0U zy(N`-y+TJSNM!6FgoxGWwZZ*mw+beGH! zH)N|)z1WD5F%dFPgQP z8pScO3XCZB<_rJ|O)~+6cl2sdfs_sT;hA7Yr0y84K-8D_taB(~>`+3R*a{#ZL#OrS zmSeYt3)rA$n%XxIX%&q_(pD`FYzVzR0oZNU3 z_xyBpql=URT)iksi&07A7(F@JxH(bf2SIeBV6@juN${_As zJv?6E_~LcsM~j(oVlNTUGnHcrnc zNNQMlv9q;68kjxEhqrsc0GU15IoWeTMKASH()V64>Ct+5(U>$ghi4`%oVXE(^XF%@ zM$VKohM%OcS=k6=0B3`pjK0u&;&;3A%SWFujQm{uogE=@z-w#As;nP_CR}(C^T#HU zEFR9Dep~`jfV2iPoPmKE=Eh;cC~la5!gmiJ=j~!=9z0xt77|wJcJT1e&xNp!np?I9 z1ZnOk#tk#cB&dt}Gz&!A+NK24uX1Phs|0op6Bpos5mto~^3&<_;pe;3Dcks}^40sE z?GOVlz-GfR{H5~1U$+%hM-4Q^?7wWwh)jB)_pM`Ttzia*rfP&-oWA}-8uIG`R?SVS zm9?=n@B^(xHQ{yfRRwZdU1!Qgm9E2K{M@2>y_+_an zv%QSS^=kN3FII#}N!SA1MdyQ&Nso*Ox6b)zP&U6A z?|)-1ivDKy(mbPSVkq_P{QtVce+TiGnrtR4&QUR?Svskhg zw+LpYv0H}+rWfJ_YMct$w&oLXFjySt*JTJsso)&s1zSz9YZE9EdXZF1^q6xFqa0|C zr9Tca&znO=sxGt@mxBe(0Y@Dl%%K6eV?=WDfO9Kj5A}do>Vb%@;AaNky^YFyr=aD_ z?GM61l1m{GQzNTh))4v<&@I?v3csc<&RMzH3Pz}klx$BcagR#mOo7eocz)n>1&O7e z)R!7N)I?^`xa_6A`0{B;`R@T#LSl?vtC14kKN;BHV($P2fNm=~F-I+4TBkgD*H`TX zI)c$XWWvonK&WHH-B0P5vPYU|Mp_fx*K~=?cFIpyv%y^Th~RTjO{NcM1Sy})rc6bh z*1-WeN}(l~rUbV?s*N2h9T#`hCakLpTB8do7DqBsr@9kdPuP!YtI)jGhlv2N)3Vk3 zKZ}FX4HQr(QAINaAny_*TjGe+)_+0pq?Cg9)H3SLEd2A5{f^R&!u>M^(EIn2J~pEk z2O+zm=b7r@In#JVvR!swrjKK*c5BSP^iS8cVFGL%zOGD9a6SGTSTF(M$LuNn&*$fI z?^82=n-J8BAN>Ra)|DoWhVh6_j<4EF69o^CH1hE7sZ~b7r+VC52Ju-n8%yid(!Q#i z(E)H^Y*^FuLZ|n~SGC_7$Ph6>(Zm*IAKjN|Iok2l{$%Rk^!SsejqzDECrH6a&1}=1 zfRy3)`j^60ATR)ddKDFnxe)@JOw@Dp;Ro*MUF>`wvL=D@O%~|+cEMbjex(KKN*>gt zJ((Jbk+FAMM}dAK&7$DKQr7+`FW`ExqDGC?5d*x4Tfv1iq~=Gm7$JyEdxrYoCTVGz zrYk(=j>Kx`TA0Lu9oGGnsL&dUvvQC8l~$+?SrUBpU$n?z13t8glsy(M8(#gTG+I*|2y%%nO|qbI|C?9qdr(U zUp{Ec?`<3v*SWZ!)v=ro>_GqwmWCsyNsVA4j^mPHSjhz`*x;FHUle^=sie-*Z(Z2( z1hd7KDjixYb#&PrJ%}#-NqxZb1@kswzKr+tp#v60_WQ$j*$H9R<-~-I z5&Db?-l>-Ac1UxF2==!YoW?CVOG`G-42I&ts^O-64VF%HaD$c$`O5fK6ORWP*o7i_ z0|}F0gzpvP6#%KmxhSRV;97car5Yq=G{{nTb+eaq4Y2Ok0a$j(u48V2=5}Nh)PH<1 z3;GcXBF6i=D#E`7Gm0Mc-?Gj>Y055Q*O)mM1@t^N$}Viz`^)F}eZUJpPE%Gs5A9~w+wlhTa8cZd00n!V@7}Nm@8B;?yD9T zZqvW=(?k;n{A18W^RATAnXkdkS%VAuuZ>wNEcMP&2+PWGkw;6@i+tT$FK(B1j9Gu1 zsw*?#shNSkG+^z>@WGaKc>`nSjPd$|daGRP=KL#D*y-~|Q$Ac}%vcck)vPNX9~M8;o<;9HRY*~-|s{u*=K zjQ8Z`2G9>&Te!n5@S@Ezi&X3bk0WpU6j}hASenAW^VO@G%IAT`T90o{}vdBD<~j?yQMflTAZV8T}F?s4dvGgYmg_oXr(L2 zsV$5vQ9y6k^ix7rBP-8@mH6qPPyVyh5)L&@L7xW$12GB#rEoxxj0z7a_MDhIa>F*J z22F$S#yKBF0uyGV5MJIY@^!L%2GwNZDi6hGCZi-tX6k3+O{AqwI(V}MVW1YV_JU!i zxrZI5k~1~*mG0UDSt#`xM(3%FK4i;I3AOZADsy&eGg5$t&S~eRqeFLLIE8>_MYnC3 z7(mf4jnD$GbPaq0RIDT&i~bf))RD9@j^ShOmV(BFHkwx8(eCfaqD_adrYMn=ATP^> z!2lTJI3&NwTxdHmy}OwGrXr5o7iX}i_TG9*S=1-W-jWh&mY zPB13hzg!LKc9J5wW@Lb$mhN_0<(lC|itDykSn=jx-dUK5WOuArr3Il=S=iW*pYP4T zB&${r<`3K4wUQyLKhKighuY$Ge# zI&9diRY7AOO{>dvE!{$y?hKBCvG#_;%yZB>g11@JL0X^-WoJ4A6yONWARG1@&6co) z8kKJSm%Co{=(1%i{Fsv4vO>{S(>@j0>0(T)+-lbk@5<#FgQjo)DN<{XSNIOug6|L| zHB0o0i!_)F7iy9wx}gkE!le`JhN_yBWlANA>&C2mv6@J&)9d1kCA?D%laPtlqw3 zTISee2O8`%aYt5_ar92)C1JG!24ZlAj=c?gYCgCB%ju%I`zE|3*KA*@o5;_D38PlZD9$WSEVQf`F3 z%r1TgCyPRhCoueVhdXH#2DU|gR3c&}d>M(k*@I^{SSEiG3YUXSTDRtT#gIcFqI!#M z-HNbqOqpi2nMV`AcbgMp zm?RxIb{Y(+|B$n-9smMcbt6{t9q5G^1cAW(rcm`}5ls-|c|-#xHMsBh?N5tuCv8!U z=4VA0H?bj?z9p6-=US$DaMMu*>INoH@4zTXG2V9OWr1pem)I@Hcjoftz{sdyTyOC~ zkmWt!tBV45De19%PM}XXAXlNLn|%!%PO5&s9Qx>2he=C_{RpUvV~NiX#%}WN8yI) zE*9?yfBPcrdSi?>0C|DczG*(8X+dVtjM*rpboj-pYNM2qcd(e&N)D^LCAfo)X32>`y&QNn?oo^%mgrD_X(j+iP^&SXXB|WjslHp02~U zVH^)P!r&cn6k8QJ(%7v)6YQ+!9>LQKTQs~S7S1$EipN}3M`XE|rap1W*QU^|iY_14 z0z4yk?{v8!Toq6DI{%pGUe*ss1PXFc66z+%I{|ETWncXD$?d4({<)s$8?;EDw3Uy8 z4Iymge^5uuS&1*&p)uNm;DSND+AESWB?hrsc4`Rfu-t4s9xI z)$^)lOn&`oEQctj)7xDgjm!-AgZR*R_b@XETpj#CCp$TA$>({8>eMoOp@ zGSm}RU#h_(iNC7lES`3-2P5>qQO{z?C)?@V|7aqZ2ZtU8!P*MT zUo=B33kK%eHHd^2@P)%>_4m7<#}veq5h7Ercv-QZAbOvk}ztGB#=z)C2D zsAWE+?-q^P-#^Mblq~z+#QWEZn!P;V3B;=wgLyk&slpCnR!F&Ftsc?&nzmJef8p z!2UVd^=&t0-8tXj>^akH)Er&f^4I9bnXI5%iXbWG=XwnWHC>z4QEYQ)vWmRe2}->^I021;|K z6ylp0P~xWRt2qqP`PyPNr}&}SVz;tK-h^nW^2mGaUksJpm^p!pajO?{wUT~Ap;pz! zMHi_Gr{YkY%srN4xj-6k)Iy3@g=;rHsXARs^keUM{tUv4!wrkyQnLphp~?KyhHxD5 zwg7uKTknw0hA{19;_G9C*&Lg-DjhD^^{(0=c478T2zzm{&kh866xNoLQsgzl|C6@% z^Z3s+*v9ASwEmDU?hBY{+18NQ6}K0)DGc)6eJ04?=D__k)sG>}{-Sg63BCffeo#kt z{PEe*9k{^_+-mn$XUdjyyFaOE-*UrUPtbWXs}ivplt(^cyN& zpW}RqRjPZKO%QST1CvqNSoIcTlB5FO*u}yA$)8oLXF1d>{>T-0)EakMl^?F*^Ib(& z!`CN3%T-?Eml+4%N@fR}WM6DheP7)949S;mZZx4qX`3sL8xg}!gC)g?=ll2L%UPVI zE2SVmDU6k;OIEIs_o9VQWwlKFI(q7j%_4C(w`>M2@aCjcEz8P|i;4FUjn0h1o0mboI!EQ={b?=gvA7-b z0;hSY=xQ6qpY(`v=63lxGbkxp(8^RfAysL9_G|lzY&wp>mhdahyr|^AGjKyLK1_Ta zm70oM!8Oyjgu(5>)jFG;F*?|)Rur+EJaYGVAmr_Z_Aq;lEx%9^)t`ktCO zK7FDcimNTw;TM^|!x+OnRx`zTYc*Nx^#JU+UZ#;^H`=?nk+p_JM}Z>2iZ~-16WZOj zy!HG(hdoGkF`T-HcxmgSLBjScMY>Yg_p{VFv+ji*{BWWl$j5vB#CTmK?$sRS7xNw~ zeyjs#iwq%9rE{a2GhO-WZR+En{Gs+tJ*3^`oA=BQ_Xw4u60Ge0`OmJq|GspK{y&s% z|9@3)KkLqNqJRJZ)P5_tME~=G|BnS+-9&x+4F;6q-8U5VYiVjzg=@pCce6ee>$hMW zkO}tkJn@R`Yu<8udue} za##G7?;1ika(HOc*h@1yb4y;dIZs<1&A*0Fjok-!7R|n?zUIvrWL+&r)LhGnJB8p{{D*rp z3&6dBdCgy0-`nnHF@NO@?<#-?*k0VBwZXXzVkhLxRyY|ZD)W1(5U#ij_3_!DP8lLz z2`Ypa^ifcYnWKH>=FB?r@1f=9=A7x=#vQ04hk-k%2qnO)@J!tbLu zUE|6EN%2TFqBA7*Ouw70{U%}P*`5oCMBlJz^oM6Y zSns`u_oBa@e7dwtVUlo#VJX$Q!`ZX@Vax43;|9b_jat-wP~%chK+8ZHZ$SK_oU;Kx zT9SIJ5|tzoMk&y-orT%Qmwzep3?g$(O9;?I6(Uqsu{=W|^^j}#oLYxSvm&#BkS|Ds zLlsC94Fe6psL4{&Yn;gpTT-hkPbiFH&yg&$94H9k*9S?R^U{5A7j&L+Dpp)t`)=^z!TJLI zuaJW4eecE9FYu%X2>^iiKY!*%HugsU3!_k@qWB-slix3+a5fl}T(Zo&RYO(d0w`py zE1`hYx=L<>6#j~0WRi9L?DsXD4p30y4et(t$n!AGzU$NK42<03z3il{iE$+SdVF)E z4)5<*lt<5y##Xl%cp2slozl|zvAH>I$f@MzyxDoa-6wQb!y;A*Rvaj0(cl*va}t9Z z3SNQ+qyp(V{3PaKfO>a=>L!FuqhLL0N}L7jU^1ZyV`v*HB9`(TY8`k7)XqqihJ#t4 z@a9L2HA(AAV@eAHV?$cslh?}nRbZ_Gp%-iuFW<)qO243HWiJSA+%olNSXe>t7J4a= zMqGflt?^n2WQJ6@**n-dUJbsrGMZq_lD*IH; zlA*Bnk7%JE`-5iDph6k0rgg%ZLE7cFQpNSLh=tnURaiE(+DZkB+{?GLF`jL1U->-l$bC=;PxPje<& zuIS-K0!L;ysm@Y+ay(oS9PrFy*rk%GwV+o`x$%J+pv_p}0o# z2=hJh**oy4QaliUx>wphKWMS_Uy(pz!441VayPl8K&ZCqt!-IV?imG z2Z9Wqg9ZGico^gLCUE9d(J>fil4rwxa?C6mVgA5jCe0zs0HRU}+y;wOgE9j=k#z`e zW0)ZOc7ejjd3fQ~IsF3^$Y@}dgSf#xfhc1cp}c9{^mpf_}v9FMX6$ru5M znlwuXG#1!Tg!M5in%e>KZaw@X{ z?dK7=NP)RV4JQ6Jh#n9T=SzZn+;Z73=AJ z44GW%AM4g|={4CyeZ~w3UTYB3M(Cl9g zdHsbZasOuxu{QPmUnC?&b>8lP1I720`ep$g!}Qz&-%UM5Yy(B8fsBL#a`E~%jFE5$ z($OTV;{CK*Ev>KbXp?(#i=)nyv5i#2<;#Hu#$acocWSnS~3a4J=n&TDKtuZkS ztTioy-aLHgcVYc7!ks=(qgvN`TtZE3-rI%scbX|e4?ffWB9`0O;Jf7`l2%Ja*6tM>Qe5^Y zBlbpsAo^AI1K8X$B2_dj_|5IO=QG&8Fd3PHLs~c=il{n0@5D35B`)|Nk`S{< z*YI<}Z{08+_uGN?Np*Llv;NAS&VDG| z*2{UylEm4PYh_n{*|EvfHud|_lZ?}5_?xkVojPyiU_j-1&biY zcci4NXOJCJ;%XlWMxNM+PfI zFw0KG3d~c{D%1>aBgeYj6Q_{jvc>{Qf7(mzL>Z$&-%4tw{s<=NA-$zzjowFy8HZ#! z(k8_lr<;99%wljd?6LZAix5Y#Rl1dgh0fGrf{+?gBr|l}*i(_&-}IZ=VexeMgMUMW z^zU}>+;Pgeini5q-JXWN1coBW5!!w1A;H-6U-9d%nD6>5d>Cl-UtRlfDIP{-nPSbJ#_3GV>Mf^Tq*Q?UJv`K@tzSeY!Fw2 z^Ra@yrv+DF^P|a|C2ZxW|9_cAqfNITu|*WLSaeVsBE)HlDdKI2Q$Xf6n0TI_=~&a1 zSxIjNv!gN}vsa`!QH|3HgDg>V?Ln^VORDT}$UX1L)hu=7`(w}PwtPdq%Ec*TE;wio zX1aG(ggr!=4WJmBRwMOVyI#L`L@0afx>V$8=9~WFzEeo;v3mRuWe<`;vurwkId;vS z=VuasiT^`73Ds2Av_fgOG_-W{dsHblcvEskAq=j7Wjs8#6D1f!F+X#u6kX9im@1t+ zaY>noqI}H)y5ymF4ZZ<$D)X}czLJhH>}mx0Bju8@6s8hoSC`NMKY->RKS7IRI>|az7)!mB~|STD(kF!;i3)uTpdu`wleT59_M#0WEvAoF9LnnAL4N^o@%uKvk7s`hDpMzWr>I z;O|xBgFc(+(dV_c;#Wp}L|)sbu&K7HFraVDxYIU3TUWd71ZtlXvqv*-)(m&(n^!K= z4n8hA|EqHQg>{BQTj9U+KE9()6=*7ci!9P?ct=qwU zOP}lE0o4#{VbqgT(h#Q0t0Fzy%r91o!@|#X(dBGroJqzWA*K*5ug`4*tTUeN?6i02 z%{bDA06$CX`$wjjTXk&{tfCRc7GLE0m-O6+?YObLI^W`#SIsp#XGF%=j&*cfltz)q zn<{V4M}8FgB~U-p{oyvvecP0x*)FvKiVSK!|ids{t%sF6}ySM<#}HZ0$Xg+I$Ho4Vdvm`mq(VP2;= zLHm=0YmuN(*<&_u$3J5>OPa$$$G17O#W@Opjc4-s>`edn1NPqt;wvm|+YOQXtzLt|fjFln3LE=2Dy>ug`ZJ+0 zYK(*|Eq!zlk{W7OWN}9dE`j+UAEr)8hhFkYb36D@B8BwBnVFZVnac*=xJc%Wllj3C z{A}gaPcN6}S=`T0XkQV4%(c+EFd>AppgY-WB`#rC6!w#q{4Fe!_LTxBAmk^1E0%UGv>}5`Eg(j;eYx> z#W{p>3Jnskp%N5T4(Flj9|6ozZUlCL>yiNMo1%$Pg!-C?5UyhUvBxZ9trqDu4s<7i zCtW7l4Ki^diJo2gKtWPEfxyt=s>}kI_;Xs#WzeGyhyDQmXdP#W!UZNVDp>=D1$D4L zok({qkpLN2ySf7fQRJeO)-2ozAoqmhIqXt8w4BHN4WcX(9NgLK25{TyqR#_Z(W$%(RXxWNN>iu(m2v-$K0I$Y5=ScCnYA0iELxfYkM|E{?ci@ z*f0wRL?U-Q4lSC}0DirQu+T#jcFibSo6$jx_V(;8sS{j!d&n}f!to^-a`%;Oqj~U& z+ZeapT|k6sUYJos89O&SPR2e-dov71w4$h)T4R?Np) z00pyn+~25k@OwMM2F%&`Cuob~24Jy&5v*FuOnh;tyKymQL>&M1FIjnP3Bvl*>dhn- z`llnw?()N4ga9CT^<;ap2&Qh^!K}(H(QAX= z1LS;k>!SE$UBb$#XS;xo>636$rWME9LvwcOompLvmI4nnXHw&zv~5 zU!AxXCJcGJku0{xUwMcF95o+b8J3J5D=L&#zI>rQRfq9%nZqO zrMOI5Dij_n6AU>u*UVvbI0|U~vi`NRNPeaSfd!!HyS|=W)A#5XmM)l>v6A+Hk|W>v zMNbZ->+HgW8*|TugMtCmmaA?%>By=l#oS1#Qo+ittuOhjuLiTaHk4uRVe$8cJWwp9QoWXCpr22rc@?2Cw5bVde~EU) zW&G@keKXFnI-QoOd{em%J1QLrmoBUQR(T{*{jAIxKBAYF75{mRHXNb)h#QWqN@2C{ z7c-sY@?l&)>;NQOk!}?m2dh|Q2%DT1Uw!T!f0LQ4<3WzTCF|~bYG{5bny(%b#<|vq zEEJU>3o&+KWS6=@tTuwIIh9QAJWgS&P>Rh_lhzPNLA7t$&o3$mPLXM05XNCvnCkP# zilW!QhUj(fxh9S~@O;~67z+5wAPfV;2uTjL%; z@O}nUWM+tdTYWG<7ZCrN2Ej-ZU^hD&5G$FVB86p282R8+1w1an(46x6agY2il3Prz z&s6~6oDu+@m07Wcfxe2FWm7>L=O6eg2=#`%!K&_c0Xwy**L-65cr&C**{4gJ5G;IO zxLopUswMTbV3+98t!rLzzquNC_M?5DJ*8*Lgrm|g(>asbiLXN0BLh(>UxFW&Ln3%D zP0nnIvV!T|rr4=dI_w?Bzy(E=nZhdv8L`W|v+cdudS>m2dU{ets2?AUd2);x;bEpF ziW{yZ%%~VyYy9C6W5of`s@Cu$Y$V?al4Y^43GP!>!Qa@g`aaC{(Bz)rfpA)+4{axq zNrj%l6QwlDr6Y#Ac)Gc*5(Pno(o_I2i#NGGgEiz1&sQb3EDT}^0P&HJ%T%sq0p#cc zQ&Q5cbhk6I&iNNd1ON$Nf)(20ud{3CC+XL1M2=w6ssXFyW3VQOq@kv;l2Tl10cNMY z7p+d#Whtuvp@UUBK>}G83))HeyPx>- zIv^-^+N!S+BLMrphI9nAr4+Z+PV8<{lc&O zsg|_!!hyMG9`48=+|}e(PwQkO8xAj`5dT<0G_~)|*?XRjix%HSI%tp^SAZzyHVW#( z0rRmE{asqC7p(8OAf-&U_gz@~1%p~xXEggbM=U(U@$G}=x}kl?D+6W>UR5iPJ&hJl z(vq2n-tYbv;!2|}QBs-mkh+<@~a6KnK$jvc>k39OX&hYU9Gny51^|V zg97+ZR-)AQxq872HOt`diAz7>%53=nQ5d=$UA*13m{n_h%Q*fHI*9SsMW5&OZR~R< z8xSmD$Ao`uGGW`+RE?T`jZM2Gk2K#^@TF^IbU-Gz_ZV#KYX+BxRiawX$Q`4a zw|yLvG=p9_iVA_ zP0R_$C7*(*G*81D1OD%k-a^EsAadZ01HuS{$?_=7vbm~#j{YOB<9giQtgLHomI0-iT&WRAoov!rF6U8$ zeWX1zT2NnNFGu<@>`moQzPECbvbYa!lqb4^oruN^$)&teT9DKphn4f-Bxpt?%SbyK z_otz(H1OuvbT2CEkjdlm4DNN+sX|-DZ7?Q?m1o5!?R&ssg^3a3JLy^z7j(qw_xmPvj<8N@X1e5} z6)u&E6*WVLm(f@W67JryhXJYc_V(Ml$v!$fUGoIV*hf0B(L}iJZHlABDeT2GO#uNn zDRz*vPj0}?lVJywx$OuddjQc7l_oee=$;M2*Q}NK6wnu-BUBFbT(qFR8(^`A#p}cD z*xaErXbvu}$LwlyQSVBc=^vMMad_#e>aR%@7+qTK7d*GZjvGq|xGCqw^LJQ6J1uVk z%D}O9gMpZ5>KjZQOsuwXJw3>7^*zOPCcGQpQ@E8uIlqi zp`a+=yk>bJyV7Y2d)JRte1)3M|F-aJDl&2gg_d6E29z?Al^|gYpCvOXJq<6&<=L*> z(H5*uaxGTiQQ_4oNk4jV-NbJdQEP@NN*7sD$CJg)teLP^^IGD)?J#k zEQIUMT2O9=Y*fK{S8fhwc{#NmA+MIcQs2vkyur+D4)N^DM690n_=sA17qFPE-_A8V zM1#`x3Pe`u$?Y{F)4!Yp2+IvI4kCB>@=JYk^MKaCXQ8m@&xV;PT7#r|osGsN#}ev1 zx@bjx48!f?uI@SC{!2Xx>>U8+AtnGox;X#<^?$!6IvN@L&-TPaZcEpVmj67Wba56U zVWim*&$Li|(Hd33wEh~aPd4eSsg($i8_pnfhMb7;oc+4t3IvBl$fIU!@5Dzch$?1}OO}M#t0s4Pqdm@}6N(gJmKMhj>XlSRdXIn>X{8;P)j$|) z8deO=*Jy+Ti03B^NPNebqJ#j51}E5L33kAz(`l40e24ub`?jlcVg9F4I8}&Y15}Am zuY)$vUw<&FWEE+oG4&{Do|vDS;dXXZ>uyXxZUIVI6M&F_R_04To@N@+zwY>g8+0J) zC4vzFxAcuG8|4+O|3p22nq&<)!ZW2H}O=MpNqkx+jA&-ZLNg) zwa~?RgzcD_SXthjs2N{k+x(Na;k2xn84*K1uAW_Dl7{(PsQGMIy4an;gZYhl3Meb) z?_$aA=#dk@^e*5b(Zj*{BR8kdyT{W6uUEIXy?qM^sVso=hxxZo$vkFmtejZjJeZrY zTT?dxA=R5$8Cj10!dUtU$$J-wEbQ*TI8aUWQ3s2K^XjQt*x5K=VVhbB(0;v#dWDKJ zkofP9K4UvT??kVz+_>6VL?@$YCkK*lmjivYqsBL~g_ZiL*xE5Qa$q;KbTb{~1aY8` z?Ym}qCvS4Q{E(SQfcUSUcNS_4PK?}~pjPA-r!|4#Tz7v=)1?{-6<;{E(ogbQUTM|P z><7ZE(nrmi(|@gfluHjBQB3B@62Q;S!q|#Q*~fXC&|onnQ2P*g)+y%k5~xF(%?*S~ zZeS8R358MyDxNV{Z3onve}Si%{`{2tB+9$z?UOfsApPAUK|VS2D6M3FzRwS7&}G4d z%s2}Nxo7Wb+c~?6tUxV=x>naPw6(k5YFcH%iniX3Tbnnng%6fLd@ibXI^J)dXW&wC zF@UR)I0NG!Y`*0XFCJYXw5X8lgFrh*)ji;W^V@UnsR;79XAgc-sfQLpU`1n&f%@ki zqy$CALq+T2_0aHCldwqq)B`NjjVDpmq%MP1lByQNj>n1Cm4~1dAnQH(#<=)AKn=uB z$2AfD0ddlqLxlJP;qb*IDM%2-cuSxF5A+3tNVf{T*X2=g7a9s|6vjC;4jb3zff~u| z0fq_n!`<-AAlm?R!6&k0*dr@wX~?8uBT!{Dgh8XT2hWSrN7ye>ZbQZx)cFd#ftM&k zQi7KISs%S}m`0Gs0q6w35(@-Eh>p}t1d%iWr|Yc|tLM+W8XOQoss5IoaVo`VE(FXl z)op3RIZ7%*eapw#&hAEpGmxsgj4lE1$)mZfMa=BGb1&8lF!!0I&Aw(4I+ba}p-*Ly ziAj9}%LD94zN5hxUqNP!eDujJhRuu{0Rajzq`&UUo$Q7WGItfX|1c?qg4nR?{iLf15LFh0h_+b>-%n)PVL<=1LmULqlC_m| z3`m$)){o(RfVH^MCPPKJuF0Mq+RvLDzOV@L!oS)goyByrRLy~Q5Qv(}5f=?5S2eDD6{rNo_Zq%%G3yTOaHR>M7`PkB_6}401 zt~AyrpsFSUNh!)}6viyH4AtTZ=KX_0^wAxts^Pi`?gl{xdEZa}*!M%dqu*w?(k&Wt zQOMfN@6_|G(LFduMC0HJ9Cm#%QI*N1hS^inS!GDUAz<~-KE3Wx{Ew{lG#(5DWsl-X z{hoPs?7q78#RGlF?T4vV>YIS)D@FiqRFvR0KWKfcU&HV^KJ>rh_j8%EL7|3nIzC!k zspi9h8{cvtUy_TM)U@O>BhST*_7MI}wX8Js9{cH7f=qyYqYM$B&f#}~q@J|P-J1&- zWW+G14$|cqV?If%6rcRl9L9nS1qk%NB+eU(L@GjD)kL+Q#`^;`#ci@gwbl4IY>t)P zJjSa`eNl5w&Me^V$)#)Srb(EywlKYo5a}heR|@&-*>pO<^nd;VEH>-~S6=B@$#be? z8o>Z!Dx=Uf#R3yq1cs;21?|(CTN^uNp;x6%@xhM~NZ;LTdTZOOc7Jp?Dn}f3m{i;; zJP`8(>F0l48B~;BJ*&>4>*X;Etyl3Vasy?bavPkZ%(+RUP?g8j+DfOMADQ} z7A1Hq=it>SRm>D0IHjH1f;mt&Q{2DnBx;=+wq~L0m#skw@bzxly1sBZs{tgacxJ1( zXp%K&hbkaT;`7FU$!hQnYL1I_attIs9HhMlG(GYd|Bc{q9X^x4z zV9E9LUX0(?4lCj}nd(QBozhJI9o88YuWRk5c3~SAxG+8nRrmM^`s5H>1o^bHvvYg> z;=j4=WUJNn{`tCK`@15AZIuyDXQ1y&g}SMaV*|sAuJ4d#u5XEqnmwOY72MsUzmTtpFqZZ3N8U6W#jejQ8T@CMu34PUC zZ&#TJnD_G**nJ!}%45wu3A*HyJ@Qkx2<*3C&kp(RA}J zbH3gOjDQ7Cojvt^pej8v*`x@|-Q_vPp)1XBBiqf8?-gL@e~N9za7}=Y3m5%v@QDq^5@O#g4q=p7>fv6K6O*7taT5U~TKr#RreaD@mTsGj( z>cA(m%@~Yz|)|=KWE>Y4N72R&5P*eiwCk5#0AqnsfV1PS3PbTK3 zbzCNak92DRNcJX5To zECiX)RleI>+n|4LvJtJ4q=q#YIhknf)&^(8lG=VTMTDZ2C=CCEXeHDWUx-a}&WqA1`oZ*M`2II8G5K{aF0OWU~@vp;%}K=T{n zF5k>re1rqbnEM9~k)w&5Mg+`#ehK=7dUv_AI+JC&0S(Zy<)Th?uSBYoE!%qMYyHam zEqea(4GZr2Ll}9VR2oy2+&b)MpID|;$%yh~C$_y9xYczMZ?s4Q;p;EP1~DxTWu&yh zt2ujZmhWOE#Qag>j?y_=K+zY-?Y4Ii0*hJ>_IiS{iakn;B^Mfl%XN8k z)r|f**c{Z^;V*z{um#`K*C|baxylm81xA19xAVTe18`OA*?axA;``xzb1V#g9e|eY z`_MsReQ}uh(v;1~-S=W6{)Y(zRaW))a@4yqt?Uu^dsIgF4|jd{`11if!h!po&N`ECmcg zd8L4wYJZ`fEVW0T|7RIiD!-CJsADLA3G24C-fxy=SJuemv&Fb;%5TrSx}9Maf5#DM zT0R+lmr0FDCaOAGD4&`YBAj)7UC8#v?$i1a;#=Pkg1%p9=hWOc;(a_}{gKk16O`bi zq;FOm`F5Szg0d*!1)`Wq9IO4tKToJhruLKdup57WN_0{@*+OQNV&hkspVVx-J`kiA3=h7T~*4H|HO0N`e7 z*cbTZk6F4jHA=nyW(~_12DCApy`}{jX6k$15Axf5LvQoxe1>vK`Xmm%z|teuzZ?U} zr9b~K+~|+-Bm(M8A7j;$QiTL}PJB5hUXAp&k%GT8>60TX4(N=BUV!G{yx$~v{pWW@ z4>e^n78o&@Mcb!E-3$%GJhgV6Hk{(K4IyRFl9=km{cIvC1ZAY+zzVs*LF7x$-pW`O)$FE{ zuZX0!4=bq#2JXhpxM?4Hb%PrDZ{=KdhJDg6-S5$6=XMp=)>0=bY9Y+$7;!4(0nj_z zZlq}QF8K$D$*4pX@nOLHv{z?;Me`R=L1BuA3IT~!_cBkEu8{Gk{ifgvPaoycA$lVp zqs@~uceD{5Mc2zKBA0+5IP?~gkd!YP4U0Qj@SoaWpzU}23J zw3@MEKE7QgIxq_1Tu3u6BICG@Y=G|G0oxp{80)b4JL);^ISfsMc+R0gSVSxsj}AfS7D6Lk&Na(=dKCD4JZX zRbfycyh4(KF&(ba)zADsqQN|b zM)BVHp9!#ierEzYov<9%V_43ZR8PfpH1KztWdeiRg5iLavW9HKEJ{i%LXJC43m~1N zgrP#tUfB4&O`WUBb5fHx$iFoX)Di?c>n$e%icgh1jD z#joLD(zk5JI$^Q+zLxbnVGdg?WHei>Z3FD#qz~-=^bq?oUAG(pcab;-{Nfq@0F-#@ zDS@Is)OZ}`4>0mgx4P#HWc3E`Bz81DlcEk4PvykJ7O3I~50cDkCeQ8x3=CLi5FNzD zxsl@WTk=si!jexFhENQx`klJ71lCig?@SYx#R(Ftz&H7;1L!Eh@qOF_O&yO#>p3m93(Dir`yyvj z%@R0n6?Cbk_nM+6T$$_J;H!GdT%8$@=W&NJuje$}Roe z%8h;A)ta3IKM|r4BBTf{5D8*~z}p&8L=(lYumwlPKQCGw+g`Y=E17Zvm9AhR>^kPO zFfz064OvW|L)wX6Ontaau53Jap)iZ-yHaNVro1-jXA_s9{M`N~^iPPTQ^zYYs4mS) z?jU|Lr5L@OsK%F;ae_tYmb^6i3@xu!TDvw z)W&rC+6IBd{kik`F@ctLonzl8ew2q`V2kmml7GXmss`O5W};GC6snmS-KYu)1NE!B>-9evRq! zGJ+bv8#u~@mP$4#dH;;2@D#V*$Z8HZm~Y1BqjXtN_+c8HuSs^y9fgeOzvp)35UJSg zCXL$^JcAut+<*d8DK>dq-_st`+5Z|`0=L3R zEp}V0cGsxu-9<`EKh4T(c;5j-wvrW&aO=ncYO^!@jq?4z+%}^p0}|4vSAVWd4gS%r+2+WL z8(i#pg((SU)>EOnRS9b>nVFif#yfAdwzTa4JxuqFDzc1J2C21Z5sphnT#5K%3^qbw znV_Lpb06SUFP%$|)d~$wAzPmK;*gK{WojseksFsX1FgwC4#BC$l!Z;#j_p1J>Jm$& zdUAQL4=Tf`c#rl%o1qec&;rnqb8oEUx@^q#M+>cJ-UE~dk;ye=&R6P-Y87ZuDTDHP z(r6pK6>oE18m<({V2G%syxgQTnFF}^f=zX3WQ7E_2OAS#e)Mc&YF8Y4wujlUx*kVVK$!h9`poS_6 zv!QbcNn3+9&BpfD8ViZ_GM5~JFG&_QP_$eUu?RAC2k;xwee?r;tqwYUtq2nSo{u?7 zC;6(tr91NZ0PVhf(07YQ*TJ3u2d12BNBrP;M;=U+{F3zJhLe%=mAgBLU0M&R;|1F} zO#Tkh4WZW{n8J{7L;>B+gL7J^{Z0A@!MntGuEbxHpwz!tCSgnkLq&&4qKX2_S>;lZ*sTfYi@$BAu3@i8f1q737 zd$4rz@_Dzt9RC+5jhf8d=KA+W;7aU& zE@;qoJ9U!=KWZf}8xc<{z|WzowLWfuzF*;6MN1x&+at!fqbM6HOFobHN8Mhqk@vrl zA~w^VIxd}S1%d@a({epco%SL(7qU!S$pEKhX(TWUG)$(h22~u8#iL~ zq9)NL6+f|x$TD85$ZQ%!a!W^YNBk*sB?Z6xuwyzeZGGqf+Xxu-i3o6|myWSSWMd?v zj3E-fp~`(Gvi>55Lt#*ENgX?p=V_rz+WKhzFcj7I14um}@6iw=l6wNmx)5YiL)|2y zf644gQeByK!&jL{u?KHK(rA}9J|C@TONSAl2cTWxg3kv|Z`aRX34lXh+?wB0Ham59 zm!{7vXO|XZq`X_xWRakH9oda%(_2qYK-G(DM^7J>XO}ddutcVLd_01AGcy(bn#>|St+sMaz81d#SIJxA9pUznLTHSfa%nyQ@qm1@wn)d zMbPuz>Cb1AFiarrzBEk%U?Nkb@Za8)-*C$A$H#Dy{;36H513V%1-eZ@sOcG;X?k@; z1^Y&lUezlyZUr-v)RXlBcla_!)j2_DU@u+-BZHoB1JvFu*>@TEXEJZghyLB2l;GxW zWRQ5DtkPM`1??BDXYXYLJ%!62+$Y>;rT?|Nv$H4D))kYECz{%veYj-Gw|bw&%uT2j zSklZ)N#q9a*X=o%OTWhnZ!h~!H@j@g$&JVfTj9g-j;E<9G0w-?@q5D(P>ma z9R$!X#bOv2RiI>u&%W(CkCoNz)~u3Al`#_T_UQR*V4eFr53q zi7gh{A3F=NOcD{Ktgdhk-^m95m>+6gG)Xr(4(Se+x<7pq=*^9y~m~l zd~YBH>P($AQVxm!L8Ut~I~fgm4l~d$v^}yJW@&uDE{P|KAXG#F5jB(n%=JL22CGCX zIW(hHV)?F})Zw0?IN>%F8uTu(8+PI}&m6}vK8c=GR8@hAG?Za7M))96O*evPuWXxz z=5Un#>RtQx5)R>0wHJJbv}V49u46{{R@AEw$gu5rXtLvMaj-|0Hbj+~u?8TiFP-~~KFh)#s#mNs zTOP>9S7ta#9a|#hn&7?JOC_K>Y!$Dmlm*2lFfd&-ZoA~6{2}q{o)M~9`x%$?mCQ5p*p4L{`o^S#UXyQlb92?)_* z3g8k+P5Ye?(!S*VFz$Zo9cZ9+D&fIu&N2%9ZI4 zYO6o~t$IHlkD3HmhDH@YKXf}Eh!FLq`3eAcih}eVwO$UOdR~3u2)i(5CosNHLf|2 ztL;1Oxqh!8;{9FRaAy#vTQQz0hnpyUHw6qo9s$lkZ&1Zqe(#R}hKzd?erHzT1l5X8 zSd$A&)hPu5Iope3`!D$_1tstHbClduqGH%%s~g%9uWFKFrC; z`-TD7+^=#3b84;**Oy9H7~@%bhjAJu&}jt+gPUbx5)E)_B;hy}?F;abmFk=XGH~&r zbfT%0<%7i%9+Pkfm^sWrl#TS{ZnT0KOBXe*_ z7P1tqF?J%A(9%)nO0vKTv^Go0W>z1zti85z0a1WT$N}$Uu0ulf$$Mw~nyBHx?KAXT zW)mJ?Avvnr0J4!*Pw2(rruGgwz~83rRafi8!roI$#@b! zFB|{7!#@DI>*#eBUimDG=cF{_J|0d0MfY{OJZoF~u{JKmPvw&1bCxP8<^edc`td5) zSQ^GxDU6ZLtYX;3UH8eHrWZtbyDY`g`M|?qD7wU{@2IDU=i?3tn2se z1b{TedThr-oEwzRy9m(VgD)zvRtmqx9#0jlz75Ur^Lj-E{^I(lY7=_5`i}z?s`$Mu zoz!kPkxtVy!T@bj1Iy^Kd9WEHsX`LD4kn^JfAi#9T2f;gHn=b=Ru05%Qfiv^w`o#c zBPUIO4Er-$kzB;J*}3FfSPar-f!ElAaWo`n#cm6{ z48PHI#WZzL#)QY;5CCYfVtcklul$2XC>u} z*n;ubue8A;@2*I=2bfq03rk&^PXSAc@9u9F8(q)lUObPT8_5P4;~oV#N6{>U;Z(%} zm}%qocdh7R$3tx0hcQz1ZR)tIsFa4Uwr5CqZZyXwDzZ)ham5Lu+5iJ*!R&goQHlQ5 z11wSr3STIM4GKPeKBYnv=T^AHEFHvj@f;GdqmUJAuw?wgENv|a>Bd=VJVAMStW5?l zzaW}OWK)a2!m0jw+7^02w9pb9UxKRbkJm6+YE4q-Y3lM1Y|J-yFDYS^kZsGYi?dN9|LG$WwEM&_aO*za#pki#Y%JG{x(Wz9 zsfxNz;Z+&4RvF*M`b%DE3$wk($gy;y(l_KUfQCt(J0~9kmTP8{5NpK++fZ%bD)@|n zVr!}2q39(r{b5$UQDd)3sfs8`tJ5r_G&O+He4v8UNGk)OUP9Ulw7ddVnFy3jm_Zt& zoQ2u>X@Bt&p{|Qjn}b!^v#{seomd-5^|MzAw_n>V^K^q@59r%-q+U5b<(d4ZDGy?e zCmx0*y)$)w1E)T6%Spc7Cy(TZhGo7exB=u%(2UDSh6)lbm%5qN?6eTWq@XcNB6>*q(KrwGQA?4h>ar9SEDro%Op5%&rySCd8O~* zU7=VWfPP}9&K-0Zgl=|NQvIVm5_KUz9g2ai(PgRN#iikLUS?GTnH?W_mjWJ0R`2_E zfH>jsC`*4rRfJ~_mpDW0)X+~C+4489BCaf!$ec;J zOibpa`PYEv**L9?AA(K*H(xY_zQogvfo5%(e9%K#adU{h*u@ynCf zEpFy0FVEo~75cbEvV3j{Z?wc>YR$ z>&}r;bZ?82&OMc5;BNuf6BPEAwJr7(1hb9y&6f%0VZs~*cg z)}waR)Op;&$<)bpV9x(Bf~;oJCVxP_=>A`XokMph(3Wmv+qP{dC$??dC$??dwtZsT zwrx9=*LaQFsJdh9->}AB_~siK)tpwCpuLe5zGgb@ zSu74;W$Qar4^PNQM@ycmqN}U7&b;=GDiIbc zKrFp)srw3fZ8g_Sy}&VKw9=phPI`@QHgB`#`FQLJRK6}FZHC+qUX%1{C$}35>%)(> zcJ;py69r{S*pA&Y6a#;CS-Hr0I`fOD4Grey;a)Yci>$~Jk7N8J^F%09A@0IIHYC}V zSj1#Kj_YjfUN9b=vY^*k$@@qW2%-DW}XH{KeQFXrv;Y`yluy#{vU)=qFfWdupZcgSPuF34X5@ghc zOlp{gXOuqrg|44MGI>=1lp(2Qcy#CFxdc6H42&9o;93+u<8m{BFYC>1(WInDPf)Mz zgT(ABPID+))Awl@kcJETlIDJPiPbpg<;9ko6usD<5UGkmuO2dbrB3yf*QOPe#Nt`@ zFl*P}O8X8R6L~9}Ow4T55--LakjgBz69Iu+-E!=7CG*_uj$x*UB8ygsEby-3RZy&2 zY2Fi32|tqO^Wt&dHxS&`&F(Vv^a)F7y~wZN3M|t?YoY*HzdwuuFh>u_E(m})YLNdK zxcqI4KC2Ph5ip+=`t+BTiAko<#{Ne8F!@B-(;c?LdQMKsa1Ll?!s4BHx=VC_I2=`8 zNUs>0`3sXR)JP;7gV{a~LHQ$~zCFFQi0Jm!#lOSq1zL*O=KP0w)=<43XA@m%jGk(Yjt7bKo5sCJ3^mNhF zw0BYa`ozu#^JF-+ea~{N-PpV=ak5mI?EYxn;ouHwx6gYAamTE*dkzfE2;{qzUkSwz!|w5~e`lHrS7G3YkJ%uT!?-P^m$cBs&1A zXaPJ+v<^l9E1C2j?M%pxG3X18qVyu>-a_>&WMayzevy+56YW>64N4OIVBL@ zUiXX)C=cZMkYGwVp3fpI1TigcEGb=yPXv zjnAer_G|6m-BlFeU=eqQx8Z@+GOoAb%|~G++r^g$^O;O3zXgxM*WOg#PGIZT+|llH z#UGCeIsiwOoO%J@)m*@}wKR22!>?_WF<8&U3D?`zu2c9~{jarBwtv?%qWtMPz0I`)zH>*$*t>gW7=R3%^f2fU-OWw#|V2-J_!jC`-=KCf4CpUKUEtt8&nMWX>}0ZgZj0vwl_o#qAWhyg(G?Z|ot^Q(1T z1C-^L19rh&XJ^qUh0V4v&MWn82KK5zY^d5d~BXgfPR*DOl{zM}(WW$YJUq9s$t9sc#{b|k1q-+Wrq&oWx`*-1=+t1#t>iMxw3&SOmMdr$F3 zT4T;)g3rIw6vWshhzch3YhdHQ7=#8oedEYo=vcLRer*8fssx^HwLhqj0tj)yr*FdY z6gy@om)DMPc?%{lcydUH;O2L#u;ET)L|2_S#vgn2DB~ZSJuL)l_5<_5&T^2^wF5~wRnJsHe=!nI z7|2u{W9OHh@}Z2Xf}!VxZ@B?}Hm8L?0q}h8t2oR!_Y-^dyDv-OpL{TD8q7f-I|LAx z$yv8BC~6Oou(-yi7ORj23kd);gX7@Vfm;v*03^o7b`5id{qXfmm8mzQGGTk}f^4ET zHM|fcyp&R5amJfvqcO33{5Q-~AyD1+Hv~yCqIYfRWlYW^{tjcIT*Ka4Z_cWt_ha1O zD41VDayYYb&-p$6TD$qgnA>I2H-}Iq$r8t4uCivedws4r;~)VJ>BmK?C@p$gQ|`ZH zR0k-^AMTCR2nm$SSXdOByrzi7$CyvS8&}k=7(S02#`5xk!(Z9vgK)w87!&FlmTvI} zL079y;!?#XdmxBgu>fkE&rX1N-1QG zH&;-ldS4OB9n3|QV^599+=y3-OG4P6HW_w_9V+Soow7A`OR6!!3Fmk47og+tQ#lX> zfw&>WZFLwp3my^`6U=jcn{X<;ch@7X5k@PSOC}C97A2it9vbUr@Rog z(EEI28Xsgjbozi6mwQtdF0=;febVd?+RE8H_ZHG#wS{z`Th^D`DC_b5Yi8H=%g4SH zfox6j(ThEW`Z!Chxc+Sb=VQ&?HeZ;6k0txSmL{`eQo44&@)~iGLbJb&*3{Rj#>g%h zAo$ZYI%qfY{2bVKwy?moS)zZHB0C1>7%rf77I^1>%(?ko^l5gV#blhb+!d`@tM9Cy$a>h`+Vy`2ac6nEU?Qg;|X2V3}#h3 zifIC+coCBjC#Oq~PmcQ{=Flk8JS0Xm9n#zXTQ0kkIVgH`flc zGd1}Zmk@!Lu7RfQJ4-Q?_D7|TDae&lp2K`lXyQ1VE;wL#JgI19X@=p_Er=@|bvDRl zgLUossTO{PE@(t0IoM{>5>$m5wC)hWM&zrh)%}gf;J?dpC?}zIpBja=i`I^tGIn4q z1KQe|_GJy;_t2O&_(gRpig;~zB48URazw@WA~wWxUTmibU*G&~{n}NQoT@9Jt4Vh_ zm11~g6;&~MYl!V+YFl7ZL)?P)k;i136B1uA<<5eJ>m@x1a~IYaDgmKP{@wEoVB zX-JA!k8*O6HSXj1m-!|6V{QF|koluVBmC~7ccX<_6;o+hqM^g-Pz!L_W^NwPUA%Z# z&dxrlfyX6SV3X~p*FiC3T#!T6%-TVa20NpS^ z)}?Fd$OK7|N}#B6^Ge!z@)bzxG`XAT?iS&sffcG+p~S%pIAG3q{hei8D;*@6y#TEO zS_VvmKZ`|*77cJEx2El6WBoA?6t+KGR-$dA@}kEX6=B2HW1{$nbB(thVMxhqymH!Q zH6mUGaiOWG+II!hZc+?Kpnz=d!MR*pQ_M+?LM=HE17#7JsUG-Hw^HpU8KSG5hS1UG zbjE1pGQKIvMi%TbY^08&F*YUw(O~MfoJ3rU-1!|f2c#)jMwh54%(6o)X79*iSsV=K z&3$cCt{u!ei;Za79;T%>KZ4dGWbjy@hO&vaolcCroQ}fpX zB@fiDlmj7F@7+nn87cbc1tU*}j!^fKh#$(!9&?aWaB71Y_T~oMs$sn&JPr0hPQT}G z^&sTA)OZl`(qE6k`h_c)T3^k%Z{)`yNw&S+Kf3n!6srGDI_k(2R}q*=DxHbMXh24t zUu(rLhqe1+5AnJCHs0Z>0-SUP3Rs0J^ZxK9k~v$gvC}hWGYLH7GQ5nNlfz1#>(g6QZ@Sm;b%npuZ%SobPE)|ew4ti zLrnNSuRJHC2s?7|eIurVb5pX*Qf@PbT+%^!^Vb51qy zmq8;(M+p0%+1N%pPP1n48Xt=1Ccs9vBEFET#4a?860I7MwFjDXZ)@w0gU)vSQCBO= zw`fAyZ`l9rB6`*9+s%;y03`MQPjIxogPpyJgVTRQVqWq5e?4a1crnQgmD4^+qPAtT zN(8N(@`0GFYC`6?u}Ttzgt}5uO40sl&2x_(yY66M5-BxvV!DnB)p6mR`6D*0AN;69 zhqjqD8@;%dgLn-7pYLwipRV2?-2`Y=>Epvt2C4o!f`5+LN2n|8l?IuEI9HT8 z1~MAJ-vp=HVG8PllS*9FvB)r4STQVrq8ln8?r(x~(7MlS$| zN`Q2^-?cNn5X?)YzwDGuI?H%eU4W7{^6|-?jq(c8S85SOGIU1U;F&;%I8c97+lCMd zfEcqdlLRkqE}VeI-@cxxhe zH1=y^f$i+;W<%iS9g9H+R{UrG%Do3%?jx1_JCpTjM8SN`p8?9Q%^Ye2+O zzxM`TcQ(Hq5A?1H0|4Fu=6O$ecE8hinU~Kj*Hv%uLX5Z--n^oZ_fGQJeg@g;e_xSo zoL9O|zWkP8hfurVI90yWlz_tc+mEkG{5!$E4Tc&YU>CHrQZn80X9Q?<-ww zXK?Q;qk)ZmOCs2uAf0|}YB;b>otKb2qXt?C00A#IHJ+pbrCryY`a-Hgj>tS59Qx2G z%;CM-&@Fl33otCtAvNJ%cf>l8W(Q5;fhv9b+Fc&dAwHR=@ppW&!+@V3BN3)p13$ab z=j4UwPe&*;2OPN6+O>P{yZT5%Glw=Q0SV|As^c;zw@ZD1{`gTOIH(t- zq_{w`fqw=z6p-j{H5$k8*IaWg*km1%MRyC7n6Uh3s6`9WUi~aK0xq>nAXj_-Lc;c| zXpjDj`ZgM3!g8MO$H;;dv-si*Wd!zEkY-DpRH|Fl-mXRhi$aMuaH>F@N8|v>Y-P&> zxz!U_+{^B4(4$`8>*{c#kZ&@2IbR2h9{>b!FV<_ckl3P*r2qVjR~no$R3`DQEoh|= zrQQCoTqbUpq7xDQy_oP4lhy<_V(3Tci4g=~a6ssvog2cS1x7Pm^#~?y?;0{c7MjTN zK7M956QipH$aAe1EFx1r?cFnfk2n7`f5z@&DNkD|BvDXa((SG0c-iPO z8>iuBAH)b`gnzy%-tkc{7SPhE!8QTBR5)(;J&JFJG(QBI+Bue|&6|B5>#n@e0?sE> z{zSmzD?Uc#7UK)$UoPk}j~z>M`Gjhuob|x9>{2dafh2@>%;aq-dLL z#`*{HQEDMbqjAiHQgm!e+}Cm2`C@gdPZ9Qt8U7bm6$7z$e6rk>3g1mAq&hT&@#QIn z*7=y~E&b|4_|bmY#ET<19z`jk3SfuC%GWC)R-Ur(o|-}I%Fdu~=__imQ>c%U<^Lk! z8q#{DNpMDhisxe_*$vL*CJw6Ko_BkjlH%=UX%n^CXupzd&_d)2{ucKDJ8jOr*%9=m z0q9*J4ztATB=#hb$HwMvm4BSDq@y}!`9WftVFnu*GvGoaBaEk}_1NLPVx4{T=RCWw zWI>ngFyo|2)`G^#HY^28HdZHT_Chf|OJ!wFWzU<6)oGc|TGI4^A%Qw))Fo87taV+> zJ{6_27wJV%En9$TTne4;__@1Iajusc5a%0yKGhBgE(O~kX8=ulKTa(R$eTpIL5y&BGFz3XlmH7 z6>`X*GlvN8f^^@KB1KZi2ds^PKc49DA-azMB5Ev7X4GW9;F6amuoNtr9Z6x zqe&w)TgIRF6KBfKiCGIdOr~2lh&ir}T9@S`@4$kxur3+q7e^gO!&GE`Oag?wd&Jav zQcZ%IAvl?e0P8{}(XW}EmvCfJP>RSAuPM20aA+aB&?GTB|l zTA?|LeP@o7&f6nJlrdyw3>;!rqsn6=h;dCrnz|0ATUFuUJrNxsmsf-nc2_nJNG>ab zlON6`nWArl963=WLRLvrD1lg85AgOm#(3--$wKev&YAH=3Gxh5El_0hjSdQl z+(aa0MAyrh9h!@IE6G4iNa3*Va`BmrbeJgQ0P8YYk~Oo6`5Z{AJ;DFE%cY7; zlbBRO39YrtWNLH}Tvr?U1X^N7B;jZ_Ukvo-QYL;TJ_%_5{Q8=nGl~biSIKw-LaBDu zPSrMg$rUcJ^^{4stGr9=yf?2Q6v?iRt%La|8=TIA7x6;7%2{SUV0Nx$fkFVQd9u4L zaE;YTa=W0hag=HBV8rLibSL6KE%r;(oax@m+S2QuJ?T5y*_YFv(aec(C5y$h&@ZQAtQE_0v^4dpgqM$Ty9kS}3eTIe#2JbUquEPkP8RS7$w_6j~XvPdVT=VYBYZd<5}j>%{DZTQ>@fAr%AZz@1=@=B%#Vv&pbM#$dwt z@F>=?tq#TzY)%sZ)^>hA-Z(Zl_9UtDGI|>2BVh1D>-&CwQkWa7iU0HbFjsm5w@3AQ zecOZOS%UgGZC0OXv1v!>*h8UB=}O!94is%TA{z`Wm7t{z$5!LDDn%@^TmiH$|LR2W za+^9`F+ns)E8Mn=eLIoT97o|)JuckIF>uzM`?UM#!udOxK`c^tJj4@5cJ3Y`vrhWu77!ow+sgMLJG7D|VB zs|gi9(E&T6BS%Nq0*lC7tD|K-%F3FeHbBDt(^CT<613LfX|RaWqxa@UX&12yT0tHB zS7_eATqke%lWsswg4yt*NJ>V_=yCF?oeG15&;@n7^FIc|nwfy#!aqRI|6 z8%(#*F6yz~yiK#XKH0HU*>-Cus1t1L?4i72*8-MN|6oNABgj!goVuq8yD=GOp*hxT6Sx+Fp9h5T zZg<+G;{UbxBjLx3^XXH2AaPFG<06?;i3+2$Knjn}!HFc$tH~KZATP^rOID!Q%A~rO zJFcw7`jW-Msfln(Dry~fF8ffxIAfB6yr#Y%u0Gkx%BZAslPw8>>QqWy0vLq`hOnb{ zt$D|$T>6IGO6FKJDy*Sz2k`5ek!~z4IMH#jupRyWw5j)*>Qfv&Z-~b=S?YB&nP^_@ z*kA5O)>Z?!-X7zACTfA@T!c1km1x96GOXTqyTJ6UqaqW;gpq4iDQlEh`H1>?jr@IE z=kq1cSBfdy!wa8+*tah<4?a@Y;HR%tq}-VkHaizM9;|?WPa}X|Bf#ujTx!H#doN_G@a?19Pu7U?lfMQ&ujwf&E}cDzP;0%l$!vLw4TT|g6GDQ6`~7t7nnea zWJ4%WB@Wc9f}+^Yw(vsvdN`|1vwn%p%BuBCYZ4O} zEnN1hSmHy5o51tASdBuTy?{H;QL)=y7rFN~O`qJr5nAzCel26POP&hkUg+V0>2}O$ zpv8sqRs9^giHyaNoBk+{R4uzeUO3-7>wPqr9s2!j54X%i zT@8>!AqFjE8Xb~Erv_txWbCKVCem`cUEJGtDACnCR5Vqae!o9i)A! zD4UN1bJ$fG#Tn7p&Zi+?c#bXI{(0`+Iq3Aq#g& ztvMwoBc<4;R>vU>?A@&aM7OmLue$KN7y-}39HPnJi?U2$pcyxtri+ru!k$%W*qBuh zQhZ_p;A$ii`)7w~Ze_HGTb|1kQ5h0T6rY|mXSRXk=gHwaAvChzOTQBud-?^@CFDM} zDc-n&xwEqi@&t!DxqSBJJzWX;(VV=s7bCrM@ih=4QTR#*!`FQ8_Pv$%j+4WUvu7!M zf8j%Az_L$p&J1}i<$WpbMYdP7xg1*ll?Fw9Di|%}q>~8n4;5@6bX{W{lh*I6Jzl@C zIef4Dxc&P(1+W+llVdDwtC5LGFM6oevnWiHOp8}c$<~<8-7{<)yRq|D2PR+_<=}w) z2K9L1Rnr$IPguPRJL&++Z6385OWzE4b3RDU>c#2KA5;kq9ihwp-B;y1|Zc@N&@TMb(0lS^`aX>K=D)lMHgyV`L z);yq~mxatDiL9PU=2PP059GlZ)v!~$&-1s|HScq#9Lv!DB!(1}>jL`WOCc!jvBjg! zrG_d)zuem60nVn6VPL9J-wfVarl;&;wZ&G5_vp6uwp6&ZwU*o+sf(HZ5}qKTYxzL# zr@Xw{RBQXa8V5}yK*LN@<2J*;N}{V=P0=w=T5kSNMDv-V)qRu&(do;_qXOQG%bJt` zy&s5h2%k@7K`Z>2_siClP11(7su&fuS?kfrR+RpQ3Dvs?{>$>+rnXsHwPb{ZC9?Wg8-K*4cz4c*&6?5r@ypX#v69KeEIZ^STISYy(lzJ8S-j$N6+gSHq=)KUv zlRs^n!X|1t7-qJ~&#@+x!@XW=YG&m%wT;6@c31@P;*^7y<&3(Bb_8y~wM-qocBn?_ z`cu@$OWwW?&h@&mm6mf%L3%+G)x>UtWT=QZ4AzLf+}f%=iU>bHzTdh5ZCxog5JsuR zm$<+j5%m=X8jS5ny!Pxl5%-{Wo1dUh_iip9 z;Ia1lQa_>pbB3pe(ex|I1pqkvy$1L{^cH`eO&tE)054f}$F5Km!RJMf@xF@#$+>{9 zX-UGJfUP}meRtq+zN`WvWT+%mW2ZrT`p3IRBMk6FOltz@W_t2vYI1t=9eC<+uk8Lx zc7dclFSoYFr~dXe8NX+aX=N;iZ>S>5NQS!_9-p4Rb;^Ni>Dl4Y;L%@I@amyr+(g1% zggbxmSX>J=RkH}qL=qwalk}guO|XcjqBh5nRFe{O+n0_uktL41QyZLv_x<9II-P&rDOyKG4XU`P(S&&5aaOmAL?* z0W#HuW$JNZ3ABha#mdUz1^P)i-CU zBs2Ekv`ACjRXJk;I#h?qrQGo*8IIOVVvb>iEc?-)bkh1VB1wt^C&TD`@j|Y%)n)5L z5QfC-pW!Itqpq8g()IYsqO6A;QGv=Kk39Pp2ZPx{RzGZBb-S7C1aCU=Xa!8mDn1BSVlax$qE`nn_co(P3z_W}Ljm04xE<`);DDv+PouSVwV2z~ zUC&+3^z%9e7oF0;V6V3lGWdq16-C!v_Dy=y}|XNfBXqT`DN z$)TV!g-24X2#O%;B$7)Rkd%)S43W$#un3qLmoy6_V$xKe4OywSmZAJ{DmqeAXuAOY-H_`r79>J{0DY| zCs*hQ@rwx$1##f<{bo(t?Ze%XvL+Wi5y68Jv{JK)x{1g^YF= zrpbB~%JN}mHE5#e!U*9D<&TIyavx*y=}s6zZ(h1?!PCv+eu;>Obengql5of!2w(#NzW6GA^Zt`3z#%u3A+HlBhXey%|x~RlEa^ra}Dk)Nh zay9%rC?mhCu>_9S>;-rLR6ri})OHv-A$_07sy7U33z9SBsr$Eh1U4J<(97n;q>~b) zhC@4&^7Y&IgyqjRJbS!z`G$SZlKcG=mvZ3Y^UPIy4p-rszV;)0L2x6OjsDX|vFpB8 zexYuB_Y9uRd8c?`2szi_`Hs#T{qK=jPQ-O3N6w1c$|yp2hLUdXY*UoaWh5HhPxf>v z2^MTBr&NdxXrU_@@=5u^sdH+6GNd&Zi%thP}UZ&8T;dIxMJx=ldW zp1*V9Sv6%r1)Gxek-J{k;#8Kmde)9vO8!)^?9k)XT=b{ zw#=S1)Wc$dm))0c%3LkKn!IFzfYB(hxf)HNRv5z&`SI)#QbchcbCM?(r#ZjX?r|i1 zh|w_#HNvxV4QSts`*L{Ri_*EeZ_|HSi$Ad@AKd$VWSMlpOa1_k71ikG#qGrIG=k^Z z84*D=qXd47Iq~u~-rx}jjpb<><1||f3I0d0;}9Dj1$q{b>!zzO)Pg3EHGGx=x{D$R zUVzHYK-CcK2I~NK9|kHCAN&^}hGx9qVMYlNBB3oDS(icWlH-{hx@!beQUDPi39WQ@ zMB^!abN%toUZZhpz=gC>IgF;(gS z?uaWqGJB+;?G>|%Kwk%{lj2llRB0X3Q?@Y+l}92>3g+td-LV`Qm3>200ujp|_ zr^QYa#LQ-#Cs?1(RRlAO+~j*qNPt?ndkjgDa!hw@%OFLRaZNRAf`Co{v8oZB4pxa@ zoEb1XnGPjh2-p0pVilCZaH{#o4Mg_VRbL||5ib}3%Mj!s?nu#mV?SPT-=!a|G4tr; z&a3|u)H7{{?`NpDB&4uTAO>cV|FBJym;m+wTc*=Iw?LGMGTCSFt?d`yTPlKk{ zg3avvYashiU&WuU4$sc#Qv5Yi7g;kYmy?|jsXCqMBJruoT$-@0FJm9)$dS8ZUop6c zR6QP-1Nze4N!wnxNxb2QV&5k>_MFDhk+%VBMt`Igecq>1c1Q7yO}UZ@L=vND8NNNh zD74mtf83PzpAgi?mkvNO!>O*wLujVjThU*hu{yCgBlr@ToSFVUbYq{~<&YTO-Pxlm zb@&G-mS;01MQ=;i9a#r+fd&u7YhP=6(mZTAAJF%GQ(i7N@u9oWm+2C0tDM;se206c z1pIw#%esOLV;C^y|H(1ZQ0Q>Uk6{8gvqdaTp1q=E?lsp>9FbOW=I_`33_X_bs!{ z9jbeuK!%w_0L(NllSw+YNw9=rzxsy+X!@lmRGwt^l7sB;WdE0{>I7lLS46q$k<=_rOy-3s% zA+aCFgwu5SX6Gtxm+9ko>{avtaKt5>#rG^FaI$1QS02_4J671nRqWJ|&e z8mY^&v>TK;1u5s=y=r}yG2|TEA-KcW{gxJHv2fI z$Kk^85{>LwT*~~X({ywg8Wf2@h=cQSUtR3O&G4s%q%Q{G2JA&lIpbpJBsXU#y2~>y znsn}kfT{U3hBZ@{1;MEXO1u_W`TO1m2m-^r*)LSLJdZ3zaV{xY2GM#k4bR?~I|e_0 z+O1p-*8)fsusPTmPt}(C9`%ETvEBW!XM(NzHQ%a=wZ+1>(>lB>iPw}#UUKKw6ln^xB4NZ&el6aNRdTz7rS?lxq zU6Vf>5Uhz|#z6d#FTm55_?q-klw`We;QSGHF?(VV;RU~GV1XEDi|=T_w-R#Kl@X)j zgv6iH@6Jc7wkYP0f0UoqjHORV?aO;^q zt}swr%j>KbWj7Mbhpm^x+O*Ec=(%^kV#w(pbWfR|o|Z-vJF!-6$6jwkSs|2tY9H5( zj#-Wq`4=aOYm3H4)3Oex$@+pb#b9$3BoIR%Eb3kOppG_+8ptCzMxB(R+w#fx<))zp+4Ql{|?(ti!1I}zlVc4(m*FBCX4*kEfyU@JbUn6#xF-s^1s0`cp8 z30HFx74di~IcO2I@DH5@(S*lE8}U9bJ&1?$hRDq-M&cE9HrVO+92C>Ii3-Mf+D3~C z&=L>%eaYXaVtLB_qOvHF1d(%&#Ufdu`$BihH#V7B5(|#Hq@GD8z_~sFGJhZeLwudQ zQp|nI?2Z^!!m56?*0q4hR@3d^>AUH?%4ib1%x0ABt~eDQ-AyQ!DQu|N#(OXTvR^fd08ie|JC7N-oK8=OZNYAc*g%dBTTSSNxC zeie80OwTUt3lW*)A~^;yWlOo3CQE#Y~sOIXd! zJoqdd4h!WlRY6?~n+IN7IyzoAJF zLN&1v1JiHy9fkL4Z@f&g=ld0z?7Xe&-a3_LbLZ{6b_#QS3ej5Urjs)O4n zBFis_eaRj|I#FOm#gc`_TChL^Vs@b=Y|F}mvXl+~Eas?b+Ho;3r=LGJV&jj~6zhli zWjN2dR#IcseyD*X_U~W|%xT+4Yb_O>LS&}6hFahnSgK}|1D%>*2x%!4}tLYoIh$e}$7R(JWR8Ww#fqR=WQn9?jg0 zUc(s1ptl20ICmC)To!#NJ?Y#l`)e>B7oYcbd;atH?H>16!wD1sAPxlpfbf5($UB)h zI_VkzS8cbtmfZ#$%J1iUHvxPi&5T8dxe4^Mu8LY-pEiKa6AH+`B?h^ssVq!c+jZ4< zXHSO2Ih<8rWLBK##Ou?|mS?;M&9ebfvdXw#V{b+mr5AETtw8m#&?o(>gG+^$s%&Yl z(p!O*`lfd4Ji&_sbyElzttM!fU#iC*>ke1Ne&88_VB1kZ#qA4*yDWdR-)Yi{>PTQw z1mG=j4C0=+TComDf`Vl4_uON$EAym|)@wtsIE_N9wew#`h~_?7#_5eS?fDuFdS;J) zC5c92&rkZ)wAsnwvA=|aeipJ@fFf(s1MwQbS|m9|7?Lor-DFfMgc1Cju?MQbw;3@z1Kin&ZhES;KL0j-!^BKBhY{QRWx-2Rod=&hba{7pw9 z(^~&1$w!UJuU4l&8-e1&I3e+DG_Nj`l_GjH4JJeOIQ(7|I&n}6@y~$KfF*_EEEe8J zTCZ;BxZoqQ`-rp@rTY}~4l~iMDjuGjBvcvbR3(oU# zr$+jzWA?9vW>cR(n&ZHWfqh|@-V)8)%@#*!b8T7<<;`?GXLl5tU33|3rp&4pAc#_K zT)R1{6cH8OGqxlcGx-?-r--oM(XFq|3j2v1MY~!wCZKQXl1fcu^<|EltHutYh@cyc z6te4_nPEe@q@LCUkbAd)*oCKn48bz^qf_C`X6`dXVBGOH4yH1M%BNr{wVLg`#|dZ8 z%&wI+pr4QD#aeYHwJ%Q4*M8lYwYSo6je;#ufM>`JWGVh5)mTSOJk1;K`ni>FxQ0}f zRSKlwWepP`>aB_@`1NRW^!jWDK7of(m7e^&M$-|})#`Vf-L^*mm_j+659y#U*_~w7Sfq_8n*-?^za~XkLqtpe!P;}DZuVzV z{5oAvvph$_#ho`ZTfYx_tpEC#$v51IY`xR5)2aLAz{G5i7@{=5|KnS3@G2((+j`Tq z2ob=BMTjAAs@?FsEm4@SCn)@>!%9kVh05WE6r04wNHy&5Wp39bUHvBzDSB-c+C$l>%5~Uo1ydH!c zKVaO*IOQeTwhrwZS1xuQG?9u$s++%3t=A(_Nm=)>(2L@7#lbi_P3)XV8HH`Onm_$+ zvQ@k8*!kQ8gjWj2iz7wKAQAqW5wfW0xC#paO5=e z!pqgEe30B*U_`zxLBt;VK5zt@aL?m90|AaX7dA(gIEpMB?K~D@a)C`@^VY?)-mCr7 z{ms(deZ>ltYTG4Zf;K!p0n0q(*lIK>=?K|!Je_e&vb^jX{wQH}jp?uEV&?Zl8#9-m z+wNj+Ju}#P%7aVNt>uzzPV4#gnvaWnfY`nwtF%emL-Y5GHJ6f@eQ7pnY1>?MX`~wi zYp%?pSmG5%=_HQqPbw{1Hef#N4>Mt%@v!Z??uQ&kG{3Xgs+kggp%(Qq+S?uP(R{^5 z53MnYw>@Q^uy{}sWwGp zLNnKvut+y8&7=1;kgpnbkl{-kBPG5;Hy2)P0|vD0zG0(|j$=n}ISouRGutieWzY2g zScdR_zS?tSsU>_wWDkwe z!CtdPICl|0KW()7Uianp)J^caV?XVGTRL-aL*Z0newdH3l|}A4N?yUQiNS&fRn(#@@qB& z5@6OY0n35LeMCuNdIEwAACKs^pvLb*iD`r9$5mCb{eobQOXN<%!0tieH4p%E^xcs? zAr{G|QKW*EEp`bd;lnx^9{l0?#~}OlvC|X~{6I($*%BGx51U6?>hY`Yv}O+ztWm)n z7L0rdb^uUd>c&xi_2#$2LA(ZhBY($>AcTY={d_YV$B5^4#UUe90prlnqrdCITc4x& z)h}qG_4Nh1J~4vCjJ`>Dg6-nGa6?BGG5+MWb#dGSXJzu_i)aIY>J90E$VgKQp-}H? zJOofDBthF`9EJ>#sP>P_z50RNA19|5kA^De!2;5VB9*d~aUXP>?jM9rm7YY(1+Ilf zpIln9efjX8pKBXv}HseJMw2t zufE=&8}bCS9*nIGJ=ycT^==Ezj5+nu4W}M}_R^KG6aiw#9z(ctW@LK@ z-5JGZ$D$+io1Osh=&J>;8G5;RGp5y7UaT!S@d1&A0PteUO`9>ypE|H;JQ^6gjfn0H zzyoCX;%IBn`UP604Mla~1Cj^6pcBJFFL36*!J>VSQ^CYUm~~{D z=b$j~B=iIE)teC`6&je^1)7y#ob8g_{=;$V#0AbA@8|EIE;)DJYi7UV9#99mom>{3a$JXptT**Gl>|)fYVPs=q&Hp#9M1VhFgF*$zyK0n5voKnQb2XpA->1IWU^D*9wTs&BD@ z1QK}CkHV$@kE~9-cQPp<*t&RsWCh6`Fwuc@|3)1V0If{a;G4YW!Uqd;|5NXbAc;%8 z)2Uo5Id}_{_mC|d;$z(i9OU`!atVf5ogv_%FJ7pL5C;6gyAXiyG6COEcNj_nzInhQ z!iJPBZ5|xl0_8o|d@yW;wj3Zf*W=aip0s%`Roy3dj6fxULH?(bZ(XL63S~Nmazea5i|5fbJ z8aVXmTs9z+a~jHOghM}sAc3N|L&T2~rA*g(W&vpSE$WM2?@{!zN3SmPkdhrJHWaHE z2_P)G@nWtX6!KG9>Dp^Z#KP9p^b<(R6^+$X!w=IdTOSh1cU?r$lZul0ozSw zqF_41PrqEPxDoVM4FbWfxq#5_C5LiFaFB4RUS8B|2)%^HkES-;;;Gr1GXjTX9HVq#?QkPd598-r$a}DW27yNZ?76^UU49;)f93_s6-OjjwHL7x$i_j zK@G!Wv&dn$x>WVCiC>kG!c;wAw01pv_B=0h(KS2!aMmp4i^f_USc^3bJCP47@j}3= zks%qtOGN*khUAetwqFzw6jBL5`DwC|@zUVp!LJY$^;Q6Q9QbcLcVuR12a&zBmV%&7 z35??!w9X-}F=`B9o3xQCe>y0i`jW-~c*-z?%*TKxp>ev(VNL%cSsQ%zzo^onIrEg1 zOo~qN`g>0T-yIVQr`BDcWF}>`oTrG_7wura%KOi?UxJo?YR*hK+fO9sc4Wbbln{F5 zP@*Pni=9{dpW9W^RK7GIl|`+vV)|ED6+^idy#T?;$`lyZM$K5>#(PWk8cEvc&5BCE zp&AIMp<>dx!{(=NeP9?&1suS}hDe4ZIiMFceG5^?6#3Y=8{pbTGNv4qzg3% z-iHEzGt;u^e%$op_bXbh2_*ie}~_WvR5oT78-yKEn4$F}Y4*tTukwrzXIw(Vrcwr$(S z$=ls$oYCJn-CtciqpqH-s#7-Lnq=Fb|(DQ1hSJ|;~vO*D?Y$J7dipHj=tVN3628}iHD>CbJ) z8!aAlOHoBJ0=x5#Q$R`>@_tlSGA3wmLCMe>C{`=b?xF-L#{y#$771SV7+~q!Mvbfn}_+&Qmz1Gu3%@nz=zWAcZkES6EUaixVWAt@><6WwZ*~m z02J92my}ppFkIE(tnH~M5Lo3{$7qwA0a#$1;e;5)OTn+42P~>R^rdJn@Ceo+>{1W9 zIpn-Xg)|z!5($Et+D@;l0}0Qffd2bw}WOa`I#X#A=H=_yoXga z;s&J=2HB0FT=;2ec!ga+wgN@InSDY-OS@E+s+fioWiq`!P#q!vQn28VC(?7az@4>L zCt^;=yA!1=&!oI!Ina%C_{)1}#AX!`MkT=j38lgJH+QRjetTkE%*7U;o{J$^!pA<1 z0M%^?yPM6#FOQeJQw}vs9UdD&;0Bvt1bA{8V?oi)f<#aKhoD$8zgK6pBps{`z59wy zup+Vq3HY2%o2Bu&fwJ;D$y~76oCqXl(%S+C*Krq`q>xUac;E_UO6NwPQJJX#*J2B1 zf^*ZWc8khPl;=B)EIqZGO5oQ)-VdJ{QGxyvm~+(xAh4UoEw0fd{{|GzTj_9xHu8G8 zbj7!$-oBBTH9wa9Y{98j_&wyQR+4o-_?ltMvE9XKby`^iU)#fb*_o2%`eYpu@P{iO zT;uOWxVoRO;`W@4AWH2Fl4k&bM{Jlgrv)GA2E%Y=!1l67dgoh`pY4^cC5VYPkQnie zL^4^sJ7To(zI+xM)T0RRqQ1R=$zP)FRojqqX&6$`hwJXo{>fFlTJ#NUmQbDt4%lVx z6k)*1X0zRn9qzx7b7Y6`oBdK%+g^d3U?z;;H>C)+7!DX9jusXVFQ-SNj5K*bb^`vGN(3RmW->vPIsTx}j+sN>L~^=;7~I z>ucUAkNl@Bv+PRPC3v3A%bj6uwr!Ir!%pZ4ocmn&@0lrhb39;r?+Lu9H3ZC0f+O*= zlgzaQZmm?=5{^XW2lGxBWGgGd2p;a+Yg!YYti`Ra!I)#OV}%dF($)7$8$R<{ST!9ez1qAh(Hsi`kokA#R*xp-;VBF+p=O2r zO~mQkY@m~hY(}uMfaapc1j)$daXk~3Y?hQ*_`hszC;3$mcOu=11Y)h zPj>>DXO?b=d+)NiMzoHofkEi&V2Pmn;pf%9R}+ z6bKK0@=Q4Esv`>E6KwlhLJ+DSHT+lbra+l)zfs z!AGo9FWkZXx639_DG3s7#|Cm?i+46Les(ig1NW=75Y<><^P$cB#0H(X@S(J_rfr2o zUCX>~-7MOM)5lxYXeus9B7|?4TIUj{2zzx$X5P$bJmU?5rz@n{BgT58Q4-AgT~KV9 znn$GMwOQ<`*a=EDG0(43^d9b=DQEU!Hc5UT@< zD=9wTWSe!2hr_QG-Ai#29JIpW6V2A-=s!%68&;Y{9bn4$s+Z3lQK(6g_b1^(%*pRr z2+X@r^;>adQd|UQyRC0e^V%jh*YuU@@&ZjbeuT4Q&!|jLG?>3SIki-uuEw@l2iG-S z=L_erlZu!@oaMlIrDZw>(fAcO?~fb~Dp z*zPCZ?M4*s3lg&+R4u92A{}qn-=F`w-Wv^dQhNRnyZu(RiZL%g$t!2Lwnd5qlex~ zh~d-Zg3qoo`zdra+Rl;3feRFu=>#E$5za(jiVUh~B(y>kB}X)CQ{Ci9F=Qt?q9T)H zgwug#BfciML0xLtR?$&5k!rz9fD1*MVe!G`9)QTVLNHQ*^uS6$xMcLf&NUISaQyo7 zpj&4@>Vg}5+D;Te*9CwKGSPwxML|QYB$ZivoW(+@EH6Y(@L-xDiIWyWOmd(Bg?~(& z^3}YQ(JpMa zGDJ=D=A`Z7qm?h|!NIneIW=rdNQtNf(=!wnS zMf7`LPT276NuuxiUlg(}kZUrW+Y8xiQCUU6h&t9xAJhWQT^QcTzaP8Sq7UDXY*(_R ze3;R}-#2VG?=NR?wqUX?UpEDaYd$U-wpBW|otf;ZF`@^$^a3nT>+z+9yhGJ*D525q4#QPa1zI{-B zq|aM=JwRP#YV@ef-e!B+aBr}vCL7RP-+4qfUEtS5=8ULBGqGtCc(tSe9^KdXsy&`B zU1@Ix1m-@Bm@>m|mp!$&jXOS1WS=oNLtjB&cC5JEP9uUhpl%}2z*G9$yK$lC{s0hN zywYz)0xjOQ089?dD-{QuDpVVCse@X@XV1RP8j-&isk|y!E&(20wk#n&GBY!8d}Sw0 z9^6?9gxc%|T%vsvDeoho6JCl$dfTj&0@`WH@zjYijDgbP+q+N+2AqP??tu^Yf_U;X za#M-STR(plY_mxNqS41ip_9a^BrIt!u?QQ8Vm9@0br~%i9(purcJa?6`c5@1ZZf61 z0)NdoZ`YXMfO%?$nq|M|6W}Yn_eu6njQW=1|1HCz1Z^Hr;*0E*2Vis@7QkvHnyuCp z%XrL~@Gqqbk*H!hfmX-MK-QrQcJAm45w9W3UM46r6~7iLSQ6Ld)Ukd#s^rtkPp(xTUE*T~Ifd0- zaQzX@=@0?&fX}Lvya_r?H#NRMEhuz!YQb0k{9*C9L!8XO;2=!Fa&~tKv5C^&Q`X)Z zC0AB|Qz1AhF?9_eWAprLpz=Fj#0F9ZvcuD5sJ?;yk2b>`>I_ZlvwUqq-4bkQc0`mt zLCzeJuP>DLv4|hgL3soe_1o*SN*c~3(!pa~q<+SnCvFG%N3l@!;W6;c3EvsurUw30 zB(ukO&w%k!_8Vb_(`d>fn$}6}0q$9(NbeM~1}HxInVT)fXt_D5nHKc^pZqPpo;LZC zog`Kj3~E*C-0Ql%wgO^vl(sMq#%4SUO#6B-;tCF}O^Qhzc2^Rl{vxQe!y>kOhpH)$ zrWYo2ZszrUmIl5Ejd$wr2n~GMqXf&08ElKaC~5_VE4~V3?a@aX)$IA$tBw(&OdLpI)&h z_L)(-Z_+*zRr!=SuD)gs<(Ek&x$6gW(RpHL!?)wKzL#R`;mFyEY7cv9OC_F~J8NuX zVoNZnwnCSpQ{VFvCG(Q{!wmCD0vBy5ZaUQzrJ5=z0i~{ri$MSJZj<5~yj>QW$>bmy zj%98a#EfQ3+Guk%VJE<2b7e+6R9L4)8F)q8_ckMldLt)}h8(Y>N4Jev28@%VUZ;qv z9-xwlL-(W{kn7bcaj4*o)uo98Va7x$+!U#^u5YT_ZeEIwuv1T6%LXUxkVk7y#EUo> z?xmbOGAp-*U2YYUX^C^#${jIhC34b$0cMCoQL(~}GJNn%WKmGH=4wWo=rtUl;#WX} zJH=WTqfrSNSM5$ z`>QCgzN}=I&7#k^HSVepiLyvPSIUPuRU0aVzWMxOB8js5{2XU5uWa7Ohh8-CBcwa~ zLeFxeyl7`}F}cDU8rY*iC*X~s)_X|T{WrU<5w(b*sQA}fLFKhS^A>|I)dtROp8CQc zNcpa2-VekN+rN3O*ZHV){4*B~}Y2F>k3<<&hSLDstrs?a??dl9!qGZtZL+xXhC6 zRTN_!P!e6KmVTl~pZ)YV3!5)He>kn0BtN^PX-oy(q$KF8Rb-wTV?WGHoII_ax9o@% z?#;r8f6y)SvoKn*8xmvZIu@7|`btERnX4I02MJf|YTP!qHifx#V8H^fVRMyBb?!v# z9Brn`N44oBRZG4A|NWQ(urSBW^~V#>^>b7~@;`KM|6z!CFm`mda&k0w`Zs;trHZ!g znFzf1v8q&JC{(%R-203~eCdIocqL}W+EKB)z>v%#m_K8$QDB{n(2{F=5EwCnZ;KQFARMGqvMBLSo|$?rfe&J?#b2@7t)N59xA~G$H+kU zV)>^aZB@l7Qb+9=#)XKZrl>6^>)X4a--QA?r=}E8ERJdX7I) zII9D7tsrYYrc}S4iF|M^>Io7$`olvOW+>%`4M_To4+1-x0MhUie6&RLmGr%S&7V5# zuNC^&Mj?7L()YBL+_9=RyN6xiGm_WNgLLbK5j2#^jwJ_m5-b%wXd4edOCn;;d>K~~ zNL!G)4SGkNOP1*U^S3DPB}p0|Qxsf;5cW;97oJ8t3nPt$OiK&FB*DQ6O}fgod7@fk z6_K9uEP=!t=K)U3tdkfSU)UHm_Op;PL5l>KsYW*+%I(R>6SsBB=@V{4#gHH>ogz5SV*qI`hS$;~~af!D%(0(83C#1=VBddc*byV7U}7=P zpzf!8cXBLFjkz(c%e?F2BTOpi#Mnkf<_nd6Gd(v2ocR^VA0Jqmw?u50exQ-Or7*Qu z`J{kXJrC%SMowUXdI+P~b^#sx{APYzD(hdB+l=|URK^eWz+E@D=icJ`pljYx&Ku2v zs!OJ zsY+^$gbk(=z7+CEs5E4;cK`2Bo;jHXr@bU8^Tt~s^9K+WKtp;ocGL#_R6yf zxvxASPgA05`#z9i`c_sT9g~(Ms>C$Tf)L&raioee%?K@`;R`j(JI1hugk%(prkH*i z%I7{;CrFe7EAy4-jU2bb@~O03H8B94{)Cp#G*tKsMV3nnyo=_oz6M@-!k?z z939cZFy$YKQEouGtY<5nvUu2S`!Li!e>gcL=`66r6en7on2eta2A~Z=gK{=k*BalqU%A=Q)Jv z9oSGsI83^YFX7Ppxiy;26?z80t3Q!o;%2VT&`gKUYhg?H4bTHw{2yG0|0F6^W1y^- zdqp^kbmm%Tdml?S7iP5fypiUU=~Oppi;s-#H+mIY3ERI(`QAH)&%YaHG~@#)q#$iu zAx*&_o)WIoxW9=xDiy1XLu}t5)*7Rd*XU)U;M(-x4PuVFO(AX5>F|g)?ioWGX#lbn zTW9|9s+0VIsCIZ|pg@MpDx_QWK$%k3XN0>C}ep zWukS1S8%Nhbqk(=TGRssjwNZ^d4=-A0#W#MQLT`hjqZV%buW^CGjGZ|%wcBEcDZ<1 z3;^DzlrIv;`n)BD{t`7}DD%{^qD;A`cjN5d;+7%}DnX#>>fB-6)zPu}*_7q`o`kr2 zFWNQ_gXta-gb=+Ie7aOMfIU5 zxnAK?Kk><=2icN!ixDVH{AKX%l}m*To=T~>s(087Ktm2L}*Gwyc5yriY3(beKt z`=r@%BWXuLv7${4BV@|LjfChW7uW?0Ib*86EMy@Vhk@-0z?}=))R=i=OJQeAv^>6o zWyU-|9Di`Dy{X6LDT@>4{r(vTW4C0aMwJHxuv_|8^@){$sjPqIQ*Ron0s|BCXIU_; z+!m&XBi%9$LLAT?g6G{0LX(M8ZPn_aZq@{j?E2OVZNbvZZ$ACrCuFzm-lv}A z6>O*JuZCc&!nZzRAHtY97M8wdM^_G~=~e(H(P(akW3J)6yY8|A!mGNEN;Ko*THbah zgmHLww)Jk%MbGEvzHzp7g{aw;5A%IY5)weklMkEf{_GnrC7p)ZP2&i+=+7)eC*%l^Za1vGt=wwOlcSe^(jFDe3L3yBs4%FM_d;ckz|~1anQO4p*S#)1@@BF)_?i{ zU648(bOU`tuYen`3af#ZE>_%lfS_;-NG=h9egHs8!q=yG)rt>jh#zbO1?rJo$s;a2 zq5AR%qpx?hQFwFP|S)B0)|0TeTL|DYxYXanw-4OXv&8gOhV6AV^%63TSAb@&Q653P~W4}ejC!Blkom@ za_`=bg|IhsW4hQEWVyg{4r5Ax^LF%D&z{p;4c;#mhAi^8k%wSN;_riJJ%`K0S>9{Y zf>&&EX;b3X4m@z86&O<0{m^s0yKpU_Bcpcw{T*60fYrf;TBm`4S?tlxXcN_zT5KIK zLngJ_m%YRcJ$k^g1F&Ckqn~!ozp8PE`Z3ykQ#WOgiQxAz#bg6UC!yJ*uS61s@!0xt zF+KsI@R|PN#0-c0SVJ3A_9A5Bznj2BDwqsj*%RnV^o1y^~fR#|6PXsD6yhMgWcs z^-eQ=&OgBFYP_R@F23c0UG)C|Rzq&_s*C@Ui$Bkl0F95&pB`lr{ky%N(g-h^Cyb}^ z8(ysZ1HM5yoW6x~*=i9|mzsdlrPicqU0g69Z4}nYMCDnbCm!)`z?w+nRoyyS7UUyG z@XO>}&%*(cM9zQYa07|7!~>0t48#W)S(}joKMROgD{fvZg4rEeMEx;Hy?O5(J`&kW zMp3LNX0WrK3UuWADj-P8brkeeha^fuED!r^+OZch!&@%e&QM)w8(@kRq?}jV5=O8$ z9cYd#z^P$z76rgGPQ^>QNN`wI)bCsX;{}6})RB1(H)-^eoUJWcevrIqEldJz$>jG$ zd{d;tBlCDF(;=P*6p18cbXjy5yPirTz3N&LVGG0@1O##}pKx%}i~#l{9V&%W-jy#j z_dV1;9dsV!DcwV^LQT&;36fUIEQ$%!AV6wZ)FPnNgCtwINVn;pksg55#lu?M8&ixR zmY~k@g;jsT$AoZf0H&DqG^SG&M_vkO;3RtcYHnceE^ecKP9Po#o)f-_F56Cq@u~Qm zfV;q-_g3XBBBLUF2r(%lspD^GIT%10K{*1iP}m4Pa9Vho?`#JoHyAMi2Ju!2gq z2w`|F-(WUJ!B@1Ru_`6@#SGR%d!MG-VTjPJADP2b6V!k<&jg|^7^A1rkQzQU%6JTi z=#iC*8J)gF3zzBz_QP%U_+E2w#%?ZRl1l_p!)SKM=r=49!oAWpdL0rKOVT3R)BT1$ z{xF1v-=&%{_>7v7b{C`BwDmBkhG9$BIpAf0AkweA8@UtNF-B}KRwnsjX&@3Ux#77rUVR4a%`)3^OONcx~mxBX;2PFFx5w z$I1ZXxgGclR=ZDmI|?xa#tTe=Vw5(cw>Di%Na`w}VzR8;7d0}B7eiy#9uNx-=uIkE z+JuX?9T_9XKjvTu)t84os(+-3g#tohX87W+mMgIhXxlp&E9v68R*a${B;t6iC~E`? z1;XMKAC-L6sK6|Pn}@7{YWzI3 zb(vn~Vj9M~aWeP--|NgV!sJ@#zSOG5gv=yqK(xH2eEKy>H^g+PKeep{vC9_Cz95ucwBWw4^CdzoApc- zjq92T8cvCKE%zu@PMqq#@XvDnLq9D%{-tL4tF1#bELA$qi&663nY4MYy&z|SFS6aQ z`RZFr;<%O|N(}Re$V{)q7`=;OR7)9LH^2Gqwtt@BsEDGrx4NfqQdWKCS=RZ_zVq44 zKGPw=N@I$&&hnK?)(NIz7|bqwyUWU>IP2$(D#eCN`ywvLGj?^$u0{kQZ=oEoFk&?g z>;6c@lSwUGSLNibvcPZZyZTRNS5V|5nta5>w z-p0i%6g&N+aB=7Le9WCXsFwbg07K6*e}Dgz3dqY%(jaJ2+gEazzjfc1#Fw9i0(k<> z_AYUjSHbzZJ~yx;@Otl%Iz#?euJ{N@7z?+Dj6umuxN**|&afFM{*_t5ga2iq|1J}xN z@^D`*Zq&BjERZAIG1~e&a;2OtD+EU{`g@vVhe{x~eQ&%qli_NwZkkxnMil1+;P(cjMu-pVqJ zaN90Pp|>18+dZGTny+dt44cif@9_}--SEg7uEyX2bHP62arTG~@|#LbwRXuGddEx*QZI|v^YR*(%Q|yKQ2W4E4cNv{QMY^XlOM#m zke0^**P7X-u7B?v;uNxpYpSdurV?Z!|J9GPxK@lVQ%u;%uOQ4zPD3$|<C7(f`(BS{pk$>YM)Cpuh60%^n-F_c2xVTBrg=(Wdvx zYSlt0vOqW(Vp7nlin4Or0Q@zvJ&Q}RLAOsVj#-|fNntenz*F1tmg9w`j5+zaG|^I9 zv5$w$UtR6b-JNr!$sifWc0=jG)MBmkx3@coO&IWcb=@Z!&l09%X^JGodcr*NV;Ef< zv&~wG`-wc|e)dt6B6iEA4FZ~QWvp`d0(ksC>ERMemIitf@}a$F*g_K34^WnDI~epu zMwIC26p7}VcM^=&4AnxtD38ul2)>B6hB8yu`B4o4LZCb{9V8wI9Dk~cL#Q^x7HZi5 zii9)rU~4Y`g2@M2A_;22CB!`aV`&7Btj7BwNSRB%T$)BRl_$2vrm`9k}WRU?G-WEMZJkh}Zy<5h@ z2-cktOi>p1d+16@C5>)vp=omfVw<_03M5-Hd=>EZ0G}oPBy67&N??Lp*Kr}s%{mGUr=`JAw zJ|U|ZE;)807T1g^FFO0dCwFWo8<-+`Ja2}qH(Y}cT_&xFkDX{s7<6Ay@z;bbUq%qi z;mi=cOFX{*Qy+yt*6^y<++>?eq@$JvcN)*8Eq(^w7FcLXlk94?S#x zMArKNBfp7MqH8EH{0OouCkv!@sM4^LB|(y_%Y}0@0z)?E&on_I*Q2d83qnZ$Sn7$K zH3>c^-rEc66K$`Ma6GW$JAHIwgb>&u;Ze(nE<8`zws<@&xFLBq$0)%YAo{!_exn^{ zuA$xjXn=tzw7O@NnGwelJw_EGFFc-;M->>E69o%)x$koR(PRTl?D#n=o=MUM8w0o{ z3;9wUy~)Q0l}*a@FREovsy3x|1Cnd&l0t#&@Pu>e$=mYVweup-YN!bJdl*xvjH>VS z=a2SfA}$QwJhJrT8n1cL-__WCSQ@F!l}fDsl`|pZO9#Fz7?;on=I>#PsBbLBxcy9K z=g_Lke>Hix;rJu?C!KuI3s1o$F8cTugJ(W*u0Fu;h8y5FDJWH>j4fU6)x4Y?Yi;yXv#v72?w>L6O@H5nMDd5VZ#m}FVV<>@|28gf z>+ZpsPfg+~RX;AGf5I|Xy1ga{ zVj@irSFmROUB!Nh-v6y|4$$o&X7665ba3WF zbPZ5GQZ0@5_3RSsSbVj00i)bh^Rh6(o5rbWYNgh|3(f3YaR}djyHKu3g9mA!8F4K@ z@XZ_5VjRZ7ID1eQEKNG1SNd5J@hb|pI(_30H4rg)+qE01TpR^{=7G4VOsJ~H^E%6@ z*833$`wowEFx=Ze2Lx&#SKDr^2UhJtk7YybqX%JJ@Aa9uMd_V>Gf7djn)jb z=fFD-DnMgr#*~T1?gK*SuaRW%6sa1MY=2PJYE#AUEdtj0_fyeEQhcZ#IHnBV`4oVe z`K%IGKp#lnWh!5K(eWCzdu0GgaBL`*riaA857-PaA0$INENUB^=mcS)Bh*aLVpH6K zfe1EaKx%fMtq(C^n{M+Mx%#F2ougMVBK8sHq*^h}Ko-ma&C2$W!Uj(j%#Gw;qSW#W zbT+MfHhFtODwXUg3029J`0<8JkVrq#E4_NLg}@zxz+HTvs9#9_Uh@61 zUvb6@TMKNpz5|x?y>X8$;9x>liS>lcYknTHq!iU=^-F*kvvX_}G|0EeGC{Ut z7+~-f^B8l))H9{b6Bs1pmBkLr6Y2T+ENzrjSZ>)>BuF)M6IB1_}5*eSx>sBVaL z1~lP8M+u7x%suZRZ|Uq?&9A=JWanmYa<~}@WC*_IXru6Zbv|GS9iT6U8Pmzo(+I1L zxdDEa!MsX#gN>{7&(TEjK^L5wiU>1qhs>CSIj zX9}~du!JaSHv*>PFRiw5HH@^ao}9n~qk0^(y431urIFRm892XUHaKR&}NIfHIrugQaG$9eAj2j^eJ>QPY}P4>v^M7lkbx( zDV<)2+8ip?UHyS-Esa&M!l2O-KHOeY(Ue&X1x#N>BBK{q~X9Z7M%;J zu8&2aL;(Awj*=}2b#{CRO?iS|;3C+oBw3dggw1BaVK2Dlb0WiOpi#0b)_?c2w5>hU zdW>QrQ`Mx#+H94_QD+j&Q59raN(}q1E$&6cwh67%@iuE_X|c~5TUXG6dE8a=>)Q4G zyKDBmkR|<7sOGy1?DuaPWqbpS!M-aa9NwNdpEC0*DBVqB;3hW%)GQ056eYi6=r6Q^ z`db-6m)JV})cT9i&G#wiDHtSLbwj$8M+e+^1@L&`FzGg$gaf%`=18pyi#v|L+?&G_ z&((_QGP41kR80H&JzN{s0Kg~4zZp3k|Fhixn=;?^|IEUTWGa~Yp9u|FlPLWEh}#YE z!c}6d=f1v-$RzR!o0s0~1<^DPM`^pAir1r6Be9^{SIGAJ1h0p8cZuM>U&6&&gPeD1 zs2D{N7fE9^w6x8K1-~OM2X{VqxZU*zvn5A`MOukDRcasOt)S8=QrwIb=|l!X1~@gW z(nOQ+nK%*Q@4_4u;kDT~6`L4^{+UTRI#@*zoV z+{TFQ7h?dwJBoJQSr-BjMHKt~sq1Tmwpy11{P6+e$~WlpT=)_5f z0~sCU&GI&p-ir{Q0oA9jRb&f<`T9uQ)h51w4fm#YV$tlrW{GC^Eq;Fdly(F9^s4T- zR~(GV?zrv#)fJ*%UtH8)2MDoyP>2$^L^{7kI~PuO?9hhk128tzdz7@GdApy|-iB)4 z)2A11H?L>!2^U+CVHW@frK-#2fL3kdCv^qjCWI>zsqFU7hCwsZzrUy74b2x$&o|w- zx(P%KI@2KT|)dDaU7u%|?ye+5Ccqe|AP%9ti3iZA$ z^qQE$L89@0czOM-= z+xRAhe@zwUXa=Ojo2i#e4$mmS?uqm>V5d{9soy@to;Uv3_c}He*CXVr+UN}u@{nhb zMHq}L6&e+_?0KquqCTGCsw~o*I?$mHP%IQkK(zGY>A--2@W7Aq!Jr&X1OFb&oPYup zjSmu#&P^+9zcSkkbpTILVAM{O{X+to%^H!*QBLlVr(solVuaNlU3;QEEd&;vuOL|? z-t|iZ{@z^aewqlz(qZF{vAk*KZ-E#dbHol>wWxf?lQeQou3OR#kXoa@q#mBs83)`6 z!Z1;c@UhdtCe*rt(4)dD>=Y{B3h8?E5EQW?n)joeE6g4a$R%BlfTS^wu4zUwUYw_~DB72%KEzG!j;&it>r~BD+=mG5I^s&O&Mdy`pcr5l+8Gw~%8HgEH5zTs>SxJqsgHvdKH3WaR zp~6iG=!o;$7A;`5{|zYlxWVZYwBblOWZ3Hn#st#JQ4t$59j%@hymsIpJWDD4U$gimwRE)9mlb{F3a+fdFUd9&9Sqtqk1D4Rd zFjJ;3;4h3tb6EEZWmoD%7i`_Es|c?+G`t6ZXMTMlxpNZ=pnJnmXD||Wr`KDd3it*)m}atB9g{xvBR%ubD1)8puDYefxI(s6=XLQ z`lxZp(B68)>Cu0j$f*b&8szg=3ENwswR9p=H#1Y)~)6p8IG^-eQR@1ETpb$9SOOQRSD@j%^PgfNSCe<>GQ|D$RIY_iW!QdDYFln57h@X|>Aho0u~1A5Jya z75FPv`tzC)>5wo+WcxNuhs~LqD4L=bdbZ*cn52+qvv^-i0F{a>N2zbrPzU92ZTNn7fqsYKYf~Wd#+tjSHoCfZFWQvO zugu-7bV{8m%|5dL-ZF;Y0fm|svqCP5RXbpPl~S{g3AE%{)D)**EYch)3tG(-%-zO( zo-d8u8|xP%*SNr1$R$xG2XW`D-t9%6L1imsP6cZ6r~q_A{O6K%*^-X@73WfO>{@(P zUk#k4AajM1Bn)O&@7UAi9I802M=Kd@Ez0DVRI=dvYx(EQxi+v*p1@OQ-BK9i+V31{ z-UjD=nRRNy)L(<#{6|z?=PB~FZX`18)7rPcm#U!pr-VSei`TJhuFWVfXUM5?w;U@* zhK&d%g`uDmILO!Bf0RNF5jV!(A#e4Xzn69gr`mJ7DxufkR@K`cN?D$|_7|Gwdz4ljJ8Xb|}bebiVAAUGYn##;?#3iCEU50I+c*_UJi9#x0E;EMwum9By85x`Wn|`QDW!+|<1>XBv6}ld-8L%p$nGcFO4k)6^ zD(Fo`eAP)FIfPoC#npJ0YEJzAnOl$|#fpp%*8-$Yd@7UOao3J#M(i@1s%;A$vrzEu z_3TU){M)CXjOjFlhfFz*ID}>Bf=64Y`n;&3{L%PY@EVv#s3}=PQaNFv5g}QPd%j&l zf`}pXPU4pWdr?NA22^NNexX-!Rw)zm!WeOV(qTFxe$h^Ey$LXWmBW3K63`w7iAIGI zDPN&f(R&rSw84r2EmWSl-VYQ6O}@a9zA=O$9+_Bj2f~9rpUj-a!nPD~O^Lx68C<}; zZn_7U{5MeW#@G-f@Ctx=$fo5>uNKZ)xzy1IO{HVUXL+2*z8vYJC|_@ng{@ugD?SQ| zWnYe@ay+VVFFE2z)ijMLRey>Lwgzx6ht|^Dyohxyw?OTkw{l-?twx%_P35F5A#-Q{ zUK?eqLS8o8Mj0jJX!ofsOE^7oLb3<)Qe8W`gmF*XQ6I)?Q@Bmc1F1g})$D+*5uZS3 zVvH%u{S4S5etc9`dF8BP>}L8vmh9{Yi_eWfPCyvy?uR7Z)3DR7u$nI01N?yN{d+1t zf=9y()uiochIbKf!0HOYca89gC_>y30qFc7@l74pgh=AMJ>9z$>rCdPVLHe>KA06( z3n^A@Pb|94S!e+#bytR7A0X6T4eTb^E%b;JlwK^V4mxSv+7RxmZY_YE)3G$^Qm!l9Z-MsP-hqy^7iQ`Wbm>mLvze6)kDN4VF~^Zm z73x&ApP*Fh{Uz-V_eb=e9*^Ns!7DXvozspQI$k4Qz=-PzRA88)9oAF{tmv;gzp0Hr zKF6bFHtgMfA-w6O3L93S&bx0|!&JJB~JCjx)rIIJRQO!xDj#sh))6!+F$Gow8-QW+2o+r?? zbCB3n+4pBlvo2f8%}HFr3}E`3=wRBu0+42hV>LvcHDefTku~AqxWX}4)s^C;{u!=o+Vn&JDx2e+H2UprqGX`pB`2=7XB@LsK)^ zpu_uBmJ5;<2C-;2NyVvBWSk>876`yMbEj& ztlewL%GWJTv16mD9P$ItMB9PgiS6R6-WqskB&6yM6l@vi`$DOevaY)a!xTPw3r63@ zwzIfhHq3P;Om1srBou+0ARplYT8;A4{`11Ur!sjjzVA$?X|7PufSyZANr5ZO4VaY_ z(L6cn-2{7v*s$zp8(fuYLx+yXFzmdllxv-)tv=;JJ8m@-e)%3>0Nd8ng0~j>X?h2F^jtv%N02 z)idmyM&Shy(OGN4n8Y{8r!V0DKC2qipG~3tn1>&JKGFZ*NW}kDq#bQ-Y)!1Kt!WKS zO#j2i2-VRb`;X`GUu=xtO4hOKEYSZ%Tk%gyrqt7QTqkQ$Tdm1ike92({PoEp15-|f z!l>Ik?5Dnc3NNq~cG_i&%R(0fc?wk)A-^8kdp3DJUufb;|3I&?b@Xy_`V-B?aSd=m ze`LfULqr+&3!OwA=D?DQ7~OG`&Gpz*8nU~55 zOuW5P=*7?mI-8~2quwOZQLzImzi9ucT@rFoSIlAgUM49gZ7=P%a)Te%uX z`jSp@KZ0;inhXBp=n%4*^moR95NULWP#)RrMpgS2d{Ouls!`(N(zi&OmSIAsHz<>z~7-y!;4^g9!~TU!9%H za&w^`l>UDxd&e$YyDe*T@33v#wr$(CZQC|>*tTukwr%gIc;2d3)w0^jJbC`Wy}qn5 z=A5JVeht7cFlWCjo34)F_F&YFspCa~trm=CsM=Bb8Qb?ecZk8Ria9cNxi7q&w2h(u zU#vB(nb4Ylj9ZW?`zNlB^*slUOFKz``wv3!A99$jLra4shk~KlnYj~X7Vri|3&y@* znK7Lc6llOa;N+L3IvHfJtW`)UCYpW0Ctz zjW^heT+4DlPIukz?OMH->=s8xn*;Ol`TmQF`27jtSe=i=6J!h@0-;dU%$hh>?1L5Q zz=+<7-sula=}1G23Ty{XPPyg*o5?_h6e%qxG0aB8MgJk026!CJjii2E3^PJK z?Kdr4Y~LP)i&jy-f*eDc1P**exPZLM&?ulh3;_C!yJ7IJAJL~z1>p;?n&gn+82rn0 zz0JC5x>ZU#tj#M_&P-36yfe-TzAk`iE+?Ffzn+Q^Ddw~3=R!m+n9_`-5zOb@+W4eR zao;V9R#`{^$u!xhLPh7cer$?D&Vhd?WF$n6$O+*}6p@x_doMyt&lhe--A#9;rVSPR zlP5nGKwoPPx2ZP5$VaqX9*i9d`|-PTW+x8{QdEM(jIBl}U~)gXX(hTf>Ys4~&7{}{ zHUtIVcsmt8BUM)t^XR7EUIN8#EB+$e)3mt-?*y&gl)cS7kddzaR(pMR2p7wSo%W)4 zwBmB!{8(F9wCC-`wrJP+#b|mB1cvFhZUlI|DjATu@%M|6N5geG3g8$hN6O6(05yK^ zw?DO-n$PYg?-Ug-itOKh&2=$@h}r4`o>#N6>WLc9&>D_-j%C;nO#B^Om6EW@bSq2rMAO*1A@*1+Gpq$c2%Dr~Nc7RGwGB#YBOb3sZB(w1s6F{BLH0xlnI zL#L#o3VV`s!`9%vros_?trgYIT$QlZh>;VH4Y{IH!o#%gXkJJiGtGl6GTmBpReIW~ z?(k&?ME0UsRkt2}<0|bScE%4C4d(1cRtB7sJgso*rV@Io1g{AeN~|@9Rq&yB8>0I^ z#{eE|Wk&2{LP#cw%(Ns;Bv5--@NNT(;QR1##tC0wpB>zt2X@qVrmL_-GS}8cDxn&D zVK~u=7kjRRb7ZwO=LUG|7phd|LpN_5g6t`GZi@P43*zf{)H+g4<;NMD>2H4C=gVF(AN(`G< zRaya-fhl5LTlNJ`(6~XB+j7guKUWlaX&(Xcq*r)0=tT^Jv(a87Yh(%@q?{2uDccN0 zw#8SQk>HsjjPuZ^J4;XWxIY})&PeDBTWZ@^U>a0x7=^ydqpA%ge{wmoJ)HS7o87PY z7`C$QJx!}Jz1w`9AO$xXv1_MI0@M`&I`y`HDbDN7cR99VY1@;z)CBDhq=GDp z;iS})la2R)zSGw0-80wDo18Vq9fWCBSPfWB_j)e_*kypoJ;K zH3IpXv{>FBWRAb0f zcc!Q|{cbU%c7eS-n{V=1F*hYXK&~%yTL$R=C;k^1`fi$+? z7s{YdcMMVBwLnJ)d$K{8sfy{d-}fei!+9Tg5i-G0%y88k>@3ppW(TbGMB%qbd=J>P zCd1#{=wk^#o#^Yxwmo@+CVGK+AyNzbq#M)P3gImLc_1MJvmbJY@q<5t`4Hc`bwNM{ z5_@w9FvO#;wH&^;&(27ZIlWQGy`f^^<3cB-J;v<6W-sL?+taEkV-hWdsAK2+rS6{sQn2SGt zaw!y_Y~WigxZcO$t-Cxx9evR2aD zR#9MMM5LlizAiyeI8}+)Z&NCKcWN+yxpn!1Gs(3ID0sQ*rZ3uQF|@R9IK@OaN2^+o{2D#u>0QoC zl5Lqhpc`i2xHnqdfWd+70Hk4|dwvf3Xlq1}!>F zP71Mckm#6VMjYasF-_slx;L-ogm&TDF<21+dhI#pR=aoGNy4eWMVh(eI9c>5AoMig zSBKA^x`wWwf}b~+)*R5la^MQFL0gGDwJt6~2=>=^#LbmFqxAsmxfINHs+^ge7V%^Q z-(bl_-l91^_^SR`oH$#deD&s?YS8gqg(J1_7CNwo_?jgZJXaMpD3B^Ur@}ozijURNuZBQPI5@~$A^mY^8^2KAKy62l!WOBTdvwH#d5hsEnc?$`w{BT9QIOCNc8Et=0iJ)oXW0wjJKdC52rCC> zr8AmNV`b9PC2g=0vjqG5xB&Keb$o*`>emBgraNm7LmZ;9Ng@}3U zDx5moS`}b7B@V;5W5d$z<4+T~h9awuMnKPFLrY5Tz;eeMr_V)7HmiX1OBP1Flrh8| zT=JJVHjtLJ$=tYBDVhxiY@2NX3kFu<45Vf18g-LSU!BQP%5_&qh~WF%e1a&7DVW?$ zRK=ewLZkU5H(I^{AJMd3h5_lAnJkGkv;3K88sb9cMJ<`HG!8hHHO7Iun2EEBUyRnL z;xJ=v7!7P;9_Kg^Oj6Gc-R@lk86sWjR)ISM5B_9gnnk{!2i?xUOV_NhmY<5<0UaI= zE6ya7)3q9k=kJc?4J+~&%tI&OA{%gqpCcTFTuDCGgP-lkk;?xH6q|-FLM|1k9!Fq>v6euswo3}c`f#uZl)7z5N8A_iIOK01 z7ZL_#2=uIXJ!DR&CDny?9O;Hb%#(nI?}Qd~fJXVo$Sb8?8Rfa_I|2+qjA+Zz1l$Ge z0zXg?E7dttw$p2XIZ%c=iG}9QcoV)l2&eC5H4so{9Kbh&xN}X=8R|R06eeK66V=Z& zTof2RbmsZ&821X}W*-*^a=@0s{?Adt9v|E{IcsYtws(z37qlEX$P170wv99=W+BB3 z>snL241pa1Qqjh(jYomON#w)ewPO{aGB2ij2W=DPK^+@kw*_u@a!zq#X-44mZDI%f zdiLb;{5BZfgt5UWU#~nC>woIJvL3QIh3@*22-HOjN_e?^q*LwO_-u1dx9yxTLznyX0AVT7^@TE-PpF# z&0IQG)NDS^)-I;QIVA!siw@}H8MLXX)ohFoo{2V5-H9YmQLdx-ki%e!=@M_2%WE@UslVQ{Az|3}z=bQK;j5LVtTC7Xe}2ivDr@!o^SKz;+HY@JWb*K& zOP?vNOAP*Y>V(jta)Tvug)b2;q7841>gBf6H}AWHj+dK3=d=&YJMrSWq&Bsax97{N zjpV=Q#AoMMW0}vjFZ&Hr3tg?l-$!XrAi7+mIl>SWoG0_T%9~Yw4rHs0fno5(&A%#ea5<&x%4X+o!?Dxt-Y70qg~0(g!(6g_>UZJNP{Ymg*swNyQWMoY8X_eQGLan4 z%p2>`$zxDEZDN>EL;`l%hU`9GVUw|`)gZdZsPKKRoBlfRK2hkT21@;CGfSiS_%cGI z8d(7^@-OiBl;v@S1j30uX#A);X@%^@HUnA6J9nbu7F<)^(8PE&*`ck&=hVBS)1L)0 zie5KRgDZEJ4_B$_e6eGSkTF@gafcEU2I6egCB>N~|q?^`W!U?}lE>7Tif!^W1X-p0_ zFcCiYpG{c$R(>xtrz;2c#FP~Hqsa^_D)fzvT`~K|AH_s~ufIRSsOz^nO`dzW)3z zTSDP2cSa_ixOkpXj{F{c$3Xre^E!qmD>I3BZuWis1z$qNNOw@FCyPs`aUN0 zbwp~J4@G0OPI1Q|M0mMvrNhh4&VENYNJft>ebMx!QE`=%^#{z0L0PV5 zh$>%;_EN+K>D@WkYG7ejK;mz&_WMBogAU}2-Joe(b!B?S%PjXq)dhZ^uf9qPG8QA?K2~YOPCg)Xd zTdLeZ8-EMhY!O>}qoO7=5@-Q2%_P}*dA@mQuY)f1#6u~$hD+i%j)5zOF>X) zso8SuPV&hvJUU$Ol2A*pndz|YsMT%?AqM7KxSI4|naQ17$B%wK^{p{p1ng(#xw0?H z`yiFcv9S1(Tw1*ssL@xoY%GC#UU1l3a0nypzNM-ir*D)+^IXrqrAk|;wu+^jzNHE> zoX*qEv$c*@`sxBxM4lvmwB0#HNJN1EhEcMwY- zmo}lw@9|4HjPMb1$RmKPSvZ3NjLT26lEi#N>cHp^jubnqqfjlw(>m77CG12R6QiY_uSBkQ-Rmlcqv>i8`?sNH3=U!b_)`mSbSTS>`)T&H*>D*foKc zS~!&uCEJo3e6_fk*zV9r86Fh#T4IDKZml^v?bt})s4MEeqwP_)mJGUz0|K4WF5C88 z6&u2+qWvq`Y1?Zi>~rxq&Yc%X`!4Xb?vl>#n>F{L)i#d}`NsNYb<}f+2V`v=&g!)s zaa2?zWkqcyNwYb@g&qS6wUUkQQS~vY<$e%p&Vay`P=e)a zg>6~eoaJ>8LwJm4(TM@Dr-Ta9f*H|&k~q;p2inM9Ns{sjMq~X%imXMG&bIi@wkPM> zr9*Mm#xbI^#(>xV<7=`=y!ifswjs9Ud!cH$CK*bs-to>8wU04Ws)Vt!(i7y=S7qx_ zV|L;bVCJ=J?Y)ZA$k}CVU88|ChIR{f5U?%r`npc=F{As`?RtO@$zF`|Gza`nr1n;z zMx4mL@#JqKa`&lC)vT#d2WdJE7kB*%^*wQto_a%})M+v@*>4Y+k-4>k8}{yVsU>1U zm#xuAr*j^ve&bRA7uP{uKRGVkFq*4CNicWObjFKkY%B$>NMDZvF?sln z1{3xcQz!amQ2M{^;XDOwk$Q+k(lNl-WVVtqTZG=N)Fk*zJbC$2Z4HyE@f4IGMR5%$q?ql`1D;zjo}~S74Qb7WT8@p zhHHkOJ@*|v}<@nay09Le$w}RqPkXv4$wT$FwRr(>n&<7fYQ}O zjkAF<&*Hy5@d43BAeEep1tSC)i7FB`b{ykGiUOw-z^`y4)@91h@x3r9y-K6yQnQjz zd|po0r~TNhSSdZ?tVLry2BItrK@pi^HQhx-C5`Ry9!%`CidvLR^PA`XI;A)b21Y>; zAcwUL3S_{8XxdLdUp@U!9MVR5fFL{iG^n{>u{ zzYzu~7epxCrB9BI`wa<)0MT7Zd4{d#;egkTDeQE&E5STQeHi!z<}ti7*2iPSnboK+;E(Txu4>-h)yt} zH$lRAI%kK<#RtoQeFUzP2QtdOL?H9)R!fIlpJw_ooYbqmagk1nj^>7$M(TUy>T9YSDmSvi44Z^wUP z=a~@jC&k$wyt8cc-#a8HX%ePb@)Few<+27Q;`p6C`93M5TJX4($kI9xBO>-p3IN)f z>cNm$-1DTYnD8XKqxP&<%Hx}OVwXmZDVIiraut#^VJ6G~Ee5cJsWQ>$wRY1w5D0J5 z(E?=W!<|&DCPLI^JLuS z{Hr{rGht$O&}BE_F!>z=W-00pASo63xv5W>?dH-JAGeSs&Y^RKsHihJPIu$#s8{B^ zoklucoU!R;R#{aO+0a~Qx}uTN@$jA$2bQ7F7AI+*`pnqtomnYt(Wkcn!i^%+%OKZC zN~>#n+`wH_=>@nuzmH0k{-Dun_n6+wp~}(z60k~Hl}|P@_9&u$CITT*-3w$0__gN; zP*LhWSt^4%pP+eg>r8L`4tyG0^SbV04iOqP{wi9uIMOJ0EmJ%wAkmb(fkg}*ulY!? zEK^X|Fd!T+I&bg(OM)hmzkB+Xl4m?(y2x+tk{ShXyO$fVKr`Wy!30 zh)GM1>REQr-z`4Ql}^T+Z!0F!n1sv3umVs$t^1si224!7gvwhm(Nlv_sqJYMN(D5- zznjMuWMqU=>n&hzMVXVY_COzL$cBl(Pvs?T%<&TaU^d%mPAw6 zSR$0F3#L5aT2NVr!PbOfQtz{09Qtp=sz|Qx?Ta%2M*pA!D3&+PvGaQsx#ie2`a)4C zwT$Jm;@aM~KDT}J-wfcT{B+=^nI2!)Sho_Xn08ZQQfJK?X|imKn1i`H@+aobrt?qP zlh08YS5R8DbLy<6ml-Pv zfuaF|E=CZJuV>5_;B|hVJVitrIghMroNeJv-8)Pr<@^<|Y_D8NwXGOOimkEZca zWr@4&>?e{r_{bR7^45AG4hS}DMYg(<1<=kb&$)Y%#WJJp@wzQZrAPhAv*}FYrSz^A zc_+AeXMqI4ZWt^M+|)6gw@OSC(Q3$VKP{O`uP&oVUXpdtZ}PPasVm`mW)NOk9v*lu zvx)ZFpu}0-KtKqq4noa2@;B-HU=eLg>zKupd6{^ zo-{F&#axbQ!GZ!`VQSGMWozB`%&6f=rV&!qk)zc{bu9&4g9wiH+}$`!C*)p>riF3t zFn=ZT{KnkSM4iwLf#wkn-b}tZHf#D1-~?=Og*>C#iyg<*j!f2^G|~NJjd?FN>mkqX zkKh!hNX~+;XKDtg^m(&E!(jX8@c4xQ|E8x5DAt#~)()m@afG$tyoMy-{_J zM2Jo7#G+ukq>BC#@{DuQbt@zGV?Wq`Q64o{{)24pzS1m3@Rd+*n4_paZ+zxJ-^C6U zVQ5Wk4zVf~pWC)Akw8Pe@5$eN#qKAlR_qmA611nmS;0U!nb$Rk!&%Y^a0)s&SfCTq0Og=B_t+wDVFfJZ~q0i}^+pitM^!R>T!uJDquJ6efnY z^m9vpz#{jnlj;SwwN^VC>!1iMhxm@6P_br4^Ye)G!NUJ&_sdXeQRq+L;Iy1BfkS&H zF<7mBbTd9uPd9Euyw=BDUN~&!!sJeD|edr`-Y7vskl- zRyA$O63XP6^q^biReIu#151w~__0aa`J@^bJ`kLKubOWwe9!P{p0YocQ>w197GrZ` zzmvb@sn-Iq79q96`4aPd7YDSCl8j1mOx2>t;Tpr)mte$GL7i`RS>4m}&!OZ7M^BMO zvQ$pL!BeV=>ZUMD!5+`W=;)a?-H+G55<)FIR*2f~>P6ip)HFK&R05Re%TXpPD>{Z6 z7LLN-j;db@B-4gA0(lT*sMgS%F+JsfRUtc(eST^h?%UVJ*%@8N7YU>-(R-QU7lO-9OkkuwNQK1iFE+_lubyk{r*^g!x}GAb+Mc#%_kjcK-}( z{vAuJD3RNCgAXdi^$S8k5&_TNFJG|*=y^5|*aDQCW1fS8z=5M&bUgg8cUj~W!55g8 zQe+|E+x?!Tj`qqDw@He;e>NsY6fwD4C`cN(Q*MZ+P46HqI7K!+Pzns_}987m#z|C4TWyqYPd& zVzc3ka!R2yzEOg|r`}nzE|Dfd_sqXc4v7%v!UZJn)StEV_Ar8Hzpth==mO8Tm52BL zM9}{ORm0{#Q8i*~f=E_={&NaHMCE^}YW#fdf2nG8{lilJQ8i9fqfQ0Sl*Md6!v5iE zU{Vv2fwyQ@sSFW;bh56>Tk&n`_85}#3A9w?FQF*^V3wx;yxp#<&Z53`>rEsF$T$GL z?A&y_-tAyVrAw&X3PH$(1tcrD-#ojh^Q`BqZ8ul*^lR+eAS7HlHpRnT73Eu+`;Q=Y zI0u)4Q!h(Nkg(B#Q)D9aYADu%FXE_s#UxQL9l;TIq!(AmT+wL;kcd`h&Wdm_t3j<6nMD1_%G`I9 z={|fo9*{IfqmUQ)Cb9=vrc0D$Ni^(82@%IM!2~JzMRJ@dVpza^#L)y1KJJ8m(~+#k zO4x5i%>pbsvY4yjKgP)OogY3=x$cF}K>cHE%DNW1O*&gOnnC)J#2HXEEdVGp{UC!g zB&pj8-pXc|DBB3o2LGtM6_7LNcZ*Zk3tYrObolGjJIJGzoWQ4MA@O#=0S~+mfotCq zqy_cC+U~;SUJ1Q!Ht!Ba<^-Dm(3LrSJ|X(xu)dLf9zK6JpWP(RMiUr82;2@Q#-Hq$ zpS*}J_@D6h5fXgig~YQy8x%$Z9@G;&Y>F*JZ`FQB#ZDKn&VxVft3rn8ToN)u#jpB@ zBM$5MaySU2Nezg@wUHV4Eq0&43O*jjX!ADDF;@G30OH1xdANuW9?Yg2AmMYhmkzpP zwGmT?%;BtgrK@I|ifW&ue-k(CTLi&@uyJNJG@&m4*<7dh+_g@^lNWOI0 zq*|cq9U#+@j>%R!Mp)NP+sjU=4=>_S12)Odobq5g*M8k3t*i+J? zjXW<$RSElR;<2aKHCf&H!Vbou+c_4O!7C2bJDJ&wDPV%UcnqvL&oPZel>EWW5LPX0No6(FR91Z ze<>H^J16XWYG@?&5TArOh;`U$R@4Q*U^%|2N(_VD6!)vJRI=_bq$68)H(GK!TK*sz zQO}%^ZYXp@_of(8Rhf_`d) zs78--Z--Dge41lA8QS>jM09UVnso0vv2f-*HVu@u`%p{MNWiCrSm4zH>(tmSuP`oB z{Q>_Ss}N{wq?*I8EEko7tN%U#1Da_qhJ%9vZqZscE}yP?&iORH9E(fcWfe`VjQTtn zcXq)gD~o-bz95%YQHignBCTv$;%gZZi?Mgv`Zp-dB%1l+%$WGrjkZ*^L*>%Md`J1O z+GQu4X2^>92O_X<&iS{(hpX{l%>SyYcyG4nX#bdnI{5#abNZ)W{;yS)E6f{PjInFa zzY+2>O9H}Xp04wp_(y=!ViTMI>?w^mO3G%d*m~6yFk5TTN7Y|DIaldlz*Kpp39cRx z5eP0L&#O2a2X}0|wUA9(q7E#v>QfaBxVqgQW7Wo&Q)`$w!GA`qa7HGUBT*S2R+Z*1 zhsIz@d22D@sSB4Cf~722P~+1o*QoaOsXjq6MFi-Wjko`ZGS2DM8E|Pe6ooP2m5NUq zlMSFB@QneYz_DZl;#U>lMTi3QdJi|LWFPXyzl}?x5Jb60G0GU@F;xsX{(dc~9%8Rr z5bECjorph+NQG@ci9Vx@&>(iomVl#R{wofZi0AgVl7b%}O<@)=c$^JWf;7{pe<#bT zz?(yaFEDtE6p8=h1IR-W51;~FA5pkI-+>>gz#x#jPHC7qJdl_$z&DZOG-2Pal%$e1 zR8UmA+4GVX^*DlmeX%%F|6o9^7w~ui`!}3Ulr5ZBIe-!QPyDT=L_jy+s$Dpe4FHTs z-ZcP-(FLv?&5fmVH-hRRxLMF+nm;VO_8pN&4#4{d00e)ZXIKjY9$BXZFWpXVzd(1_ zJh0ifbn5N7s;`~81qFK3o|+};vhK3?*4Vf0DryVbLI?sk;NttpiK$iR^6mO;+2XsV zm-W2~#aRuR1R}WH)qUp7dE30vbGoJD@?pDb*?S2qQn}J~9piUt#;2%Z@bjKNtmYhu zY`IZ!g%vye#KD2Q+p84%)aZ=EicSweh^d?`Xr`hT4Rovj;Ey*!sH!XL7FP z`d&aX33a1(g3a{{|HZmvr2u@R1#c*>>-rbz)k;?kKBR%x2-yc7F^2 z?O)S=qrXgg3{P`ICLn*~y?R}^IE2t}Ug5|IDg%1Jce*<;ME44iQm9DR*2~33(zo?C znO?nI+lGfNqppel=0ad}!yhP<=nH);hP0Kr!~~M+Aq{_&-8b9f(7&ytW&VU7#6{C? zHFdah*f=(xc>beV$#C=1LB9 zr`6w~?VE7ntl-5+;p)1dVGpkj?DAkRLG^hXt>3PK@ONii6!$S=p~B%iOmNZ zKsOtiR~I>-Xn*zo18alG00&1}zyM*k+I>4jl%CbI=^wcfg>#a^nC(ETBG1uQ7oS8G zdvY=jX$p#tAT)eoU&#XwsB3Gi=MaHEiC~#rJHblK$0i^H{U!03Kgabpu_Ly z-+@sN^rUnH$`Al}ok`ivkhl@FcQ&Bx+}u;$E&ERjNP3!3jHIK<41UD1ulYdq{k$pY zqY`+8gI7N2;hbd%Q(RK;cyTv#P(@gfzuVGwxYou{r^@?wza4s;4M)L6drJp2Cf`~( zPdRrauCp0cGnU}qb|3Zsq`B|_3}LnBs&xfT0t0fZPY3uSG*ZNgF8|41^ouEX#z-*< z)HgDi&YT_{({|4PWPU5n0|6QR(&!pBX8}@IglgyD4Ym3_GKtGq9?q&jC%eADn4cO8 zcvmmnmz}zK%&)+? zT=P<-Z_ZiMz?-%-#@6T1`({2*B+`sdEVEwy%KcNayshe`qzpd;Z)m|4py_wp0#Dx>TgmjvEQ z@b!V#DM1LAY2vttqQm)8aF3RRPStr=ztSx`h5*E_;z${bwjr>D)e!Ji;ppO$+i&CY z>pVtO5+Y1IXFwURcX4(Srh8;a~pE1cfl*O<2lhR_?M|Une1bQ*df~n{bdM{TA0ZI0Lx7 z1uq|o>Q$tv9&V}kdR!>SR8GC(p8>sn=5~lUSB)2f*oAIH{1&3TVI}F$uGsz2ene}D zbJ%0242BQFc-;f5G~j>k;=*Zb4G)S;y(Sf(c=JvTjlAkx3f47k?4LP${*Y4XWu7p` zZ`IsX{4q2)MJE?yH*mk4pRO)74O54y7&RUco8_@Wa+G@E$NyMm(RgTMInAI(V6*2d zx8rQ*_y`W4Rpj{tdie&Z2}U6L!#8mt#*4fCtSoMc31E$xQrouhK2>g^ z1DcY0%Sx>m5O6a3&aY(GK%L8noITx)HUUu{h5?Sr;HWIOZ$5J@0_>ppwRrrYRy;W` z`30;kMN9`SyJDB||?Xe#+ps;yNyTE^xh z*(WDY-h1dnW1i+uP8Pg({~7=rm8*`ZwcMvzS?_bWppKJ(+=I>Ijg;A7w7^$~Q6OQPDJ zn?G?d?LBRmEkZ>5bB3?cPryOrTwbVp>$m}aWyaWAt+H1a_t>gS4hu#EovGd*}%1O4u!IdG5#Q16yqsRet%>`tbNS`iNw(D^syEHcIE`p0^wMly?~#6);cx*8#NBXeE@V-~TkB zd)R0^NMMFtO`Z)h0c!^)acE0QQw*w=XHzNb6pGho@fvoiBVZO&*5fzr^~|nI@mEuE zQEz~NGVPnShfLQPT30|BZkfXjx%v5Yfkcv1#FR)`$

La^iEzb0=O?=&AlfYO%{x z+6Q|ynaE;tFI6~qx*a}uyIUUG_E}wz*WCodWwOTi){{p%2wD4;Msd=g(in_oEKxWB zU)a~UKNLjS6e_>AA7K;wqlcmu?krC5355B`gAL@2aOKm;zAo%jZUxNFdzko=QDIzm zPElQ{SG51FdLP8q@eW%0x;+@V9};ShJ}fJZHi4-H_yDm_D_Av_D#}2PmKzB756PMr z*M!4Ld0)6muZ>+pPjv{vEEP8^U5x=Edk#c9qfIIP{n>f|K*GZM{s>`F zc>+m;&W<5bp>n*kP#*!5SDngvDy|AOI?(~Lan1`Cgg0&6jJ|nUqU7tIM9SLpg$;y7 zem@H=TVR+1i&Wp_>qHgHA;njT-_&-c2;1p|B4P%>UGxKwMfd#i8n?OBD)uF@v@;Kh z7iqfwHRbAj<7IMb-ICu;xsN2)6W&G1ib#ztyM}_jY9F^b-=OxnXjmuzh9DtG*v32M zPQuCumAs=?_;Z97E4^3+z@N4#f3DkjOy!B?g0^wl9hM!dy z6lG$6u0kp}So%9PFZZRKhv3Kajl;1>!8D}g2;ZQ;OrD~xs)!|CF=0Lpng*AfKd$C7VqKda&eog>=SOp{UZ)VV(xnmrep{b2MMG6ZUlI`ZvCMl<{JbpUw% zY)*;%cR(O~J_VfjD{#>J8&Cc*yToNaX=sH0^UkQt+t~+6l#ZBc1L#z_xYqY3CF`8O zqV4f`Xt27Ux;T#Y?+*Lfiwo2ztk&C;M*;9&E!ErDRqX!rH}HSx1-AcBre+)folbo} zppYFQl7G{V1w&9sX;3WEtU;xX1~N&fMvpuZUVaZ>$GXocw*NJ za0NH9E|Gt!*{REWJG~|rjt`sqNw0)7{KfBpWE7^9hlqt;-+eymT=3j zFIEyPb7t)B+G>2-h#+t!g%SWXf^w1_ZWpsy@JSjY7~7JZm5KlcZGqQ3o85X$uBL5fuwz#MvU{$8=xy&+{3s{l5WG#gZnw@M;6L zAsWK==0r)i4R>1_(}sy|NBjNDT5hwWGSLX4Vy%DjK8Qpf*&i;bXWXCC3JAf&Z$uCa zXsIwHC?%kejN)FPooPhHAPsC9Y{dds^{IWX3Bb+ebe^0GH$Uati^fKDDcoIV<~0UlpC6wr=F)1)ds-fv^mrB2M5s2$KH zp;~4G_7GLxc03dBA?}*)F=p_tmkrlQM(1R!QtNE=VKla=9t`^1?)+WvVNsf|IzQf) za>g|1#?8@zINrytfmo629RU187c;}<%_=zT&$m9HX&j+YoXm9-{l3`kXFvY-^*}Y@r|G=Ny&yy`czKo-r2pJ$dH*tSlXl$K&9Fd!qz+*edLTX@tpP=*%N4 zYza=CZwkF60-Z4Z-4MerR0G!^pbBzW+*6!n{Un86R4!m^jJ6bRbkp!6&005xk=dRT zf9R-^8JsCMjDU0#yl3;X5%Qc$0N&yw3LO#fz%cSsL7Zs=Cj&^7VnU~3tU6IBzNE=7 zJUi$wWUWD`ncTr1*j0ZN783s%R2=EA;BUR9L8RZfoM^Cr5a76{(ldd(Vt0s+W;dh2Au55h$Y_E7V_Cqsh-rBns+nX?8 zv7X5Jg-;ke5DhxI2iHlh)@*E3@J6_t+#Hkh+C85>^fwxuZGUNW{K1q7d33(_7N5ct zeX$OfYc*D9WNY&LQMFW0m!C6dnfkrux5L&4R?`KThD3p?cfR39=lxHwV|cOv-ko%K zBfeke_k8JYD=HMTN_QAGq~H7=3F>y}Td}{ehv*k25vXY*qo5yG{A@r-H+M<@>rA)J zfgF?cCj{E@GuQo>g!!MEHPCl5H2Z(HivAm~e3OdxPu0TxovuNv!-Y^t+0Jg%jeExr zo)rNFPc7W2P@!BV3VEd=V&5ghq}OqyNs)C>Y@HqCZ7;iCQfE3tfAvTlkQY99ad zbD>1G7LH?TFJdf#9=dJ$>GjjgGPOX$X2ug&N8uqO3Hd{V6=J_@-m3)bcEV}#!|z^_ z##Mk^J{b?`Q0x8)r#T@x!xYAf=tRyMZz_G$0n`b&a#L`sNFdmvDxFNY?I#mTu0f)n zMHpAk^6oDlvyV0`&ytjIMFvO?LzS?FvSgs8) zT@Du>0=g128Z-h%K_`La-BCweu9FN*7|-|MA-*6UUy?akg)j~}weD}6YKLMhom!?y zidh()?BB z+ZrIb2-OdC{I_2gjs-k=4CRM9AE4lvscWOs&-_~P)hBG<`~ z!SyQ`P|>KTfFCX||5ceX;w=aOOAb zyVxJ2 zwP89l@N)Dah9eRpQl{YKgjHX%fu_$`Eq-*A1TA&#Bt8R#PT7%9oW3{A2zM{6_$)1P z`B0F*#XOQei!{Ica?;3CoD`#6+Bq<*2dCwX5^E9JJ!OM(M~4O?^x#)RU$q}iT!`%j zk8wncnW$rmzu`rInh(_=?y{V5#WjB#0a;lO_IXVeH$SwKRdm!4cOqUreKGPAq2~X^ z*f}+a0t8q(wr$(CZQHhu8{4*R+qP{dH_nZX&CJWzPR-WLTmONs>OS~RW>~3&*ShVo z_{y>?A%oZVqXHN0lOp)v{TXeCyE;ek;$wJw!0X1jI%3+=JVJ9SKOu-!)mA^!U{Z%) z#>MFMjs#{jJ{qkQ0G&E^D7&QpWhE7dQn$INGm!;H-6GWy{np6zcthPmyJ7`OO9n=b zX3;xNmFWZ4_Le}Y+L`<)d--hfGn5>%qd&3fuOH&ZBG{Z7IjW-R#C*KTRTx(z3Q?Je zpTK}ce~@m`9Z1Y320=%+KRq`a3;swLUuRCl{Q-uKc5;`Ft~=9;+zsR~ef|ueO(T zbAgAHAWb7|iO**X-o|^|Q){`@y;1-&2A+TkGS^v#1?T|-SU$LAzFDL%Hsun^+9PnA zzKc*{No;&H^XhG`^oPH}PRP-yn;Y9V`ppgn-`c8WFmbg?Ra=p@*F|cFl3){WL${m$ z7x{rYj3%M$)uo7;lfCRn}7f(LH4bExC!@5U|59fvJZ(9Jj=pY%)+Jh9rE zg^@wC+{k9AiJaaQoi5pNTggM+`g;49GSiRx)UZ0D?uhf7;XlRFm;YQ~{J-xeCb)Qh zSOfq7Q~v+|yXk-JtLTOBr!lDS!NUtgn!2unB5 zRMAuvqKtXB{d}28ND)2N?>q)>@fx?)3?De`!Y6u3(hXW*Kb%kz+Pa-~u>) zuK;%eNQLP?wXRhc1W^(IS||m270}`~!S(tR;Kw|nlRWD~r-pmz=m=D95a5~F@81uY z2Fii_>WMlk!muY!gL9U~gU%v_p+hOOK;|zIAzj<68DgD9jIc@+1FVrgp{WNkm}LE( z)B)Kmcq@8FEa}@W=-6er?|3ZxSd3H#Ab5HJoD@xI3^52|q1K!)Rw%UKB~eYmz-X@J zI}I#DU~lDsMvO}YXuY2wO%Z)kXd2rU8Sd`GgsCgb&vnFyDKmcU@o4C<(MLj`4aw7Y z79SUM-nFok3^2ToDd$I8M12V3N7@Gynufdh^KsqIp8-(1kPc&KR|oFn!HuD(v!@Fe zYCnOU&WR&C@w9Ko0NT8iprb^L{YV~uQ_A1#0UkG-HNXmG!|>H2Tnn~DXh5d!zfRaT z>%rCD-sLuW{A~A!JM%FhTv~t^2c~YcUg@WlKikdF)IdQVt~6)?fwB(+ zw#?Y$3c&ctlJDclVDTS381qW66`e2Gb&zkeGLS_z-gCcAHiAU zKOJB;6jmAbfi7|JC=41GJTgP{Y8r#Z5eWUFn}`R;dp^^ zWwL+d#&1pl^wRVC%BhVox4eAG0IM<(ozsGL%f^u5WSfeja%SJ9AY5e#NQ5rMzKOtn z1MbF8vjFCCbWt+90SUf1VRs7)375hmL+Bwmo2m;qFR>7xdfgs`KW-9?$)2qF1juOo z{_JkF`_irX0`ZC4@wJ$oF4z+z$j|BBk$=KpprA<51St%Z$N&_=!Nqx7 z37RmELrvJMg{yT06t=@LU3C=>uAsMO+uiwKfJ=9w|BliyhOVTGn0~bf%No1!#VEsU zo{*xG&{07EOQ zc3PuegT<1Fm}_G$`5Hp$Vat@b1R81?Q*Az@_AhxFdT-Hal)nr0p+d9wy-IGT`r>sw zZ~{`{XB@nWZ+^!d1RPP#o;XZ)hE@!7$}p*zG2QGDcBxn~uS# z24IurxX&CAi`sKPkUF?2xDIpK+4N-`i1eF|gsI%cgeS$b*gHAy87bNT^cgQ;5|KKK zW#8_s0f?cHQCB-&L;awyF_7E3OnE@=K;(#tGcutB?|kB7 z3?WvCe&=S_KQ**WH{ABK=Gjv|$yw!cQV9=;X{t{PiI7XP>Ng(#&n&PyA!{UN@VO#o zEoi5ycK8*OIuriwsv>iHllu)gvojl3%SJtopAMW4b0salRcK?TMVm@Za#sttM@)Q# zhPc;j+T8V^A;`;pLnhWZ!BNy_0R@eoG!b~v2;OSxWm1$hr3F#Z5{Z%0Uadd|@#2J8 z>de;4`&bs^JsRu;bkbmmK`%&9_+s$sUgNK2C<-X)PiQ}oUgQTNkvmBJoBKg1uL0j* z=pFGa=?yv*`jb@;rBygNdYA+nGUUpKwY`i}oN%gaZjjBS>aAqrxCr|r-v zlZ~c_Y>6yGNknqlb?UX8X}p6aED|H#q%$o4B34?d5e9mPO7Ekff<^kyxy5{bc~0GZ zsRgNbo6f%v5S0UoCF#B2o>_Rdlsmv1e6yKrmLWGTH_J@p{qt2Mzj4lYuwG_C1Mv9<@ z6C3%|d}h$v!U4uHTo-pPU9nCAsIl~?uGABh?9NVYTNl;ZPzYH5p;MNyktBH2s|_f~ zx|VV{m<4;-Ae){F81?BngUIr7-_6%;TV&T=?HJT?9y=_c46EEGOGzN;aLZ)W1{M!i zny0)yw0oSV;sNM>#E9GR)FvO)0<<`=qNUhM1Z^}9TRDQ_1ZgCV(}t~WS@{%lc^p{O zm!c(g0xC}b_y*rhQKD4?%Vr40i{Z*`YQjY8qSi`T0bvzMF)zB{~BP#9S=8SeN?d4!bM1Q%lt@xPXr$R5mt zP`=a$YO&G82i}T;%2v5`WL5L5cG#t+Ddgyhhvbi47*hsi>*gi`nV3%{DW{noG$O0E zW+j$~u+nDvYc)pU5&+t&43(L?Q(lP1<^pr|tVs3OovX_&(4cKWlClkN( z0u?RB+0M$L)~pPkk;)dlhNV9Oww~#fsPJ6a2j}#N3V!dN>UTb)D?8S@_cA!PQxe5Q z|Jt79)I!1wUD|!UwwteI(tr2Ant+XM3$8&)MBv^Lq*2I%45GHus&$J9B8beOzr(u} zQ#sF=B`f@Wfd9(=;y$SpHZ&u_`uhS#ov&7({Lc#STEJ1|*x#Msk zUz%aAGSG>HO~RMNs4umP%WrXTT%%kC9y2ZPWHkeoP_4iKcCwQ5aZV|WDbmcV&pj#0 zwptn&vxC-9hO>XzsWVFBy{AMw$8_yV>;O<8q~ptUFC@9j?Gotn2B>~PI8R}cLu7V9 zgTUTAlzaY?+1_)eT!B`mGF1>jyoBkS1~raOACA%!g#xcD2%WjwO~p1zl_GUu9C=sR z9m?e`J_W|@E4Yzf6@Jmy%KqXkk)lhfX$zF9gYRuLCm4*{_nS8aB?pQ>}M3P_Mc#JXPmhM@51Gaw76weK>}MsH&z zGriL6g=&_dZWKr*q=dfP#IhxTI4Nm|t^TEzaCqvT^mE6ZGFeL+{j+i6;wbSmD?)2} zJT|ef(*(-oFImBEpNbz=-mPg{a0uR9OoZ2N^((-1t>hmrb6;Ope0c6VJw7#5kElOk z1Dv^Kc2Q<~DX3etvLT}ZY2wr;BkB*kLXa)RetR``u*W88bsgZI*#8tOxH*@G`oAy%&5UEFkJdLwFsP$r(P_!^(Fv#!3)N`^JPm8#vP9>HdR=A5Azk% z!D@ZXmo$VH7J;btY^wGweBol>eGFc(cH#!Bpc-FfvMUr7HHV{`*WrX(l}WH3qYpsQ zUjMr#FhN~#+EcA_eeHJ>pBa^jR=!^M7!UaR;o>5q!^J5{;#;?Z~t1Ws9%3(s-W! z;`371e#AqPVVILMD_5G7;nUxWQ#G;=w966)6uf{&Of`}9zW1Uhc*wEEBpF`pJU52A zh)Uz)$7Hp~(N1Z9kT#e1kN+S~->^t$qrZqD2LTT2^gXPr>x&`VvZ*giN4D00dPY#Q z$_0E={(4sIEPf0(O!>(_%jSwyH%X_<8FqWt&Tje6b+B{j&$MR{H37)BpD-||8ucC^X zdC1%|J+}zmZ?T(Ds?smtDY>B%-M9+c8^%E_oaZlwXZr(_<96nGE5yB+$L^)m7}Jjs zg7ZO6Z8GQ^E+j~cPWO%`~DU2uLdSLr^ufn z=i(V;dd@`0RK%Uok|5 zhCKIj@n@DZpEQeMuTA5gNJ$T(NT-x=z%Z$Q?SI`FW58g_xpMF-mJN_)!Yj~Kl3Xxq zW42p`FnV?%LY7_AD;tZ6L5Pj=6^(B^zAPE5AbpF1oZOAR>nnm9RPRp_xaPNdP5aOY zcZN54N^ie;F@VJJQ*_^yG0C|p^hg~Uq=Uj+rXS|pY0p9+n z?d7-Wk{gFLKz*i?JY1cg4{qH3WlI%)VWfDUYF{Ddfby+#S4OrEPdM_aEW#{x5Vocy6zt3bCrcOPL&Q5u$R(C|K&atnzKLBAFyQVxN=)A1HW z5urW^SOW4wsL=~bh)nZ3EkQc&y-kip4*-uHj6>gwiEb{FJY<3>L9*>hUH4;Z@RsAE z8Bf!r2!!}79#0*no4@p^5~WI!3j%dK541qE^Em^g!xY4@0iq7Us1HJE*r8e%>~tF; zdk!M~O0m&t%n1w7au2~TUo!v&e5e@>EDQ;WwjjUaj8QHy!9Yp_cyF2siIovfL~^PD zBR--_?bRO3suXotb^3)|(|D%g4|&o&n0L%ShK{E>N>GK|#cub66Y~>nEpvi9LP6Q% z@k_EzlL(_&pV_z+gp-KoUngpLGZ@j;{1n5>-L|PW(vb2di?M+~h#{5HG}|>;C%0cm zO`Ms%7qH(%4T$hWi}Sf=kod*<=OJ``e7|0x z#q~@L;Bx+|i0i2ALFWkBBd{1R!@~AFxH`PMPoswij-L#fj|Ic58t~vlS&Sr4|2~E) zpeH=Y-}w&!qsiKOy13ESy@OGW9XG;D~`T5(~(Nf`Er*PT%w-bQffweVtTzizAoz0PiPkyLJby8D&^w?NO(Fx)j zcA<*nySJ39z$KT^p3I(*@6%^vV1sxb{@MzOZ#(KoUnH=81p7vD3fJDjz&8#Y0*PEzohP%JmSNV)%S=dy|Yl@s*E z`i*1=B| zMLSrPA6b}wgW9NfjRD^bEc4QVl$ASnkK5V)P86VWze@Y{Sh92{aBpJbxDLh&6kl72yrW8dTA*CoTNQlhC>bkMBD=TIrhx;0l~ZHHpPKq%kHsBvy+ z7A(VmWo}v^6Oq=iY@b7#X@`p_6$ZjE;oQDw2hJ)Q4HIILZash2`I4G98J_l ziY+NK%`x4MXF$$)l=(DIZMtKRF{)J(?>=W(t#<=(vzF@5Dy5)Xsf{5|&APu5AvU>H z)WLC`fZ{GU$kCOsHc5zmOf8eVjj{2>3jN)H4cepyTNOt@)Tq^jAe`?Mu$|J-9E{U+ zD2;X$LBfKwd`k0HDdmbIDJAntG_hVGJ%H~hqMzIWSbn{6?GU8(0g-z?ZorggCMn%e zKlG!LtQYEriliEKZE<8$RbbITQzgPKpnQItnAGY3}g5Q;N|$z5|c-*}=NAu0Q;2-&Y+f?AV2 zU4YPXlNocrNBdQ#=SiSgG#zt?MGMzx*drmhcooaakAmW* zo+p*yz+z9WvdGwNMG{W{O3D6dA(6vFp=PHitr0KDs7zj0(v4>wOc@o~%3GIPAXPd; zZP_TR+t&$~GdAXHl!9c#j1=0mfW6^3;MtG3E4-}vJ*VR=4xHmr=k1$c8ImJ;jg^nI8h5)m_`@5jIX3aKxA!-k!!g$^WzRw9{;-~Kpp>B{Qw}@*bI%a zVx2AwL(avva<5i-Y;5-M2F-BkX;3ZUZ7Y{p3|O*VX}wHtxw7)`g@XnwtM3GHe!<^m z@GG7X3n&15X5o+m9mP_#u2eww*~r;)b3 zJ|FkzO24Ozoi6pgJj~H!K>T|A;7B>yuBY8(w4IM|Rhg9&={Y@53~-GPQ1pEC1%tV2 zhFQx%gXk-!$-4H1CjBMU;RMR74L75y@#*E8&+E3-&fTQkYZ$H_^{62oGM(ggH7(x- ze9zC?o0ATnX!V;}>2rOO>s8}qU5sOK?`QX_rC^8^IGt-}wi5fMV!qr3&Y1EYP}DF! zzWd{UW9;hxdOe5OxyW+jag*Ktj0oSgTDndB3$yqji zN-_RcZ5ByoP72D@^oGg^k`u(-s=>{>t|{vASR`0*yEmUPtW6`S4!Ld&pw?}8Tc(1s z^l3e1yS!4Hs8vfyrsqRTuj{iL#FX#<)B!bJ8l2hA)hArcU#&bvYJ2zlA@|GL(PBS0 z7>oO1J!iL4O%csA4RqhYT$$-zzU*)3E=<%UqYH`1-Bbm_NmK-FR!FgKh7E#sTdM4W zA?n-Vs!IX)=^gJOeRpx2_2$E$H))wtk$HH^dYGfr%_hATPW`PluA5!vHDSJX6 z00l2YGuWuW|DE#}Qcs`FvuBrtW8E&Xf;aaX7uvE1KCoc$9-mg}(T`%)>|8&c!Y>pb~HpnZXzGgA} z+=*e^9O<2YkY(iB){9jhFwT0-okg0re=M@L{G7#iBvLBJA>&S8A$(Mj-BjOQn8-BU z!?BU#eR%d*t499G%Ob}*P6`8N&_831Nfft>pGjA;t+caS{`WXZ?6VkM;Mb}C{qwiB z56tXgz6&<955b?A8cGlLm_ElKV)%{z|BZy{M)(X8{l*b0`2c=@4>$lu_Ad6cOmr-C zjP(C;q<8-PKRCLYx|;qES+EmrZ+mQZq@TUKKnVni&dJ85=g8G^cm*3yM60P<685zz zRHzo|-;HSXRO%|K_XB&g@g33^LTNcA90#!l5BIR410Fw|#kmbp6nkX*4k^v8^ZGh@ z`9AK@pRazq?q$t}H6BP#`3)-Bqo$;!60a-KiTmTTw*_Z6^2Gk6#VDb_lC4M)>Gm5; z2ZmJJblN0B^%eu!sPd_WQ;Xqu4pT-65*P?G$~Pf;?It)JoCM&DrPKx_MWUSt5dcnU zj+De>NBs#M2~BV!>Gvq+Iv&08)j72w(SB?T#<{K*MCuNCo`ko^G#|rkdYzH0l+wX6 z$a8{px&%(2EznK?Y0>>JK=N~eFcN}D8?~g>z^2!Y1WrAm-_#b4cV0K%PAjIhoKU*P zJOw+@$wKxIqFkhX{2#RGTOTKvS1HpgQTsh^Q~`>{!nB$_TDd3 z56*Uzxp!m_-&IrluqF1k7iBP%qi>6M>aHH6uZ6m+vV*h6DX^bzYFW z&?Q6%Gxh%VxtsT+^V`GkPJT1@m!qz&34q{skkz7tGb@!yxfO<8;!A97F0Z#63mPK`MsNklk6}_C`PFHQ2K2kzG3Xh% zP0lN$d@+X>$^PO4MU2;d(G{qWR4+)Lk_!99-Pzgr9U&|QpSid=jIU0EK@%oPyvwh{ zFF?ycH>9ATk{4HqAOvZ)&}TWyeCS;J<~AFS{7n#~$C#UgJ#9m@52=6f^K5MmL{Oh= zy9h4tKQ0&5Gd{u;>TTy)+%|q5 znfCh7|Kv4}*K?KTCbr}G@?`%8a3HP7&Vyb+Lh+|WJr`QU5lIkBvdtEu&KLswUf_?K zqc;=Sb*nacjXV-x8Xjo#@fi9j6u}7TFJ(_7vzeL*wD_Jyw)i|vRe945y+C^m97Hn; zj4z*(bg#oL>SNe`V~XW7l>-xtQT!+I<3y#y6gNxj^hrSPaC@V`hEi!++|{s@Oq8pM zNL*R_U-XEv)P!$dc=Cl*s;~-4DYZX6Qe@{pqlUouYavWje{~_<3?-T`Ok`e-K+pCm z3Qoegxbk&8Ba!_6HoY%^QqE-7|H=^vc18!)Ry9*a;w)7p4Z?BplED!OAV$crHL*-C z{=_Y(ZMi|Vih>hpBxAOEIPPA2_*M1Bk&OgXFgP$*Lm$8;ttTkH zpO3{``n~}9cNe)bpif*2X$!gKaoC}hpzQRnPZT5qx+&=kL<3NXE6p1t{3Gf8%hV${ ze09xzNC&juP1~H6|2+ruKHP zDRnps+_VBoN{!}BUjYx6vmmWd80IVg-7)nc&B^eiKsK#p<_vX=(kZnHD&TPIp=Tj1 zXkUrk^G72o%KUrl?7*B#)c|lIJF22}A=8#%;1CZab-AdyacWob%wEy1A4OR8z*LN@5?~_QLDenb*_kX20&8=LZvrxva@9=1{w5gfLgVMigMgAHFsXPw=nxD zlk%{e=gFvH^`U)L1)nu2wWUU#)ldm|f;4f|Raa*zm=`_0p)jcDQ0JuU^z%uchNTN+ z>`eZRIk{HJ%2fc!wN$}YXNnguLLS%- zRe=8dP_FL@!r0(i5glS%nX0cEuUp2Q%@P&jWsB_nEL4fJR6t9TF0kCG-_K=+bNpk2 zi1}kzcS^5BGkMjH=+t<0P4c2TKaiX`0K#yU$7L#XmFl8jOf1df=UfEM?t(Fy_{@kX z6(op&HZxeIHrrME7~JLx=RY`_z+VEVuwKS)W0q4Er*PbGAY9d2i64?@MMguwGVR(W zr57Dr8VVIvYvq2inA(FzUxnBTo0IH@d-B!QZf{^$nd~f4NZJb@MYchiv~r%F4EU8G zuqDrf3~dmHF5323mW74YrwhJs7U$2R>vAUGTcY&W#YrnhZ#ORLV>PXpEt1_hfmVi{ zw*#kJ8ETt!-c>-li(!1Pcm?!r1MVj9AF21+0~%_y&%RfnhK;qRi}V8s_k-fQ%Rnu$4gW zTrW(tss-hRr`ObeYU7&RX}qqjyXPyHght!KG&kvd_?(MQtpxk=xXjlkHXEu zS^9Oem8UQk@1A-tl@r5u)rJZ$3`lwH(%+^6T%cRLA3H*@Wo0VV3dok1>n4EmN>nWm zAtOzSS|)I+P~KSDf7L=rKiIHK4}D51&Q6?`?6qrkkzGlK>&knt1&B7PM!0YBDI!^R z*gXC;&7`xTiwgRemZ?hYi6NoSG}a0hjVs3JqWnDvYx}PF-Tb*sI!3_xO)&NALJn2I zj`-kFgOw3kREUzYrcLnf+y^ROH~k^9{!2#G!tZ(sO5poamP`=BlS1xuR+2 z^8bGN^i_!>LZgg>5<1~ga0k3xT_*9ah*^;Cl>Fs*_pwZXYfJ$|( zM`5GFb8;spovapDt2mMCK};G7BO_1?-=*KLhr-JgZ%%b#^fMV492WtIl6^gLw zt7s&k(?`R@fNGm3(&X%r-g38+T^{{0^&Y5A3k^Hy@1(RzC8RT1SH2)At-;l@s4trG zM#S+;HemD;(uB#f=1aiVDa-bPF*9_taGLS8P2?~`< zIeUsS_MMYI+edUbrjh54mF;nr0KJfsod7h3$_}tYbp=MN_nH3Ct50&3#9bp$zCu8` zzTqcKg*Ig1Y|RTC4rmecnGV%34{ebJ7n3ztQGbn~8{z@L1HQ$)6T>xE(4ys~rG`!y zDXdBohTIu+?v8$hqSg5q;jDfxklGZ|+my0)`G?B6O#|G@2ou$Or65EL+Y4cnpvCn8 zU~KMRd0*QtGW3%8B7v5ATA)12oYwr4Gp=C$o^XLr=?L00m2six6;5x<9^XJ0Of^QA z==?(l+|#}_N^RGZB~{#YRnF#P;v$J0>!lg38S-ehc&I=6jaKwHkY^ktG=~R#2n^K}%FFI0kGrZxue9@n zTu?ix;si={gHqe9tS;~63aZ{x{X6-*AsM2Xo;~zL$OpF_S0S%$CGVV%rV+*ozU(n; zqYnL-=}@8Ii*7NJKm6BHq|FkgnQh^PGXfEGyJt-GZM=rbmPz*Up^^u7!24glv~x+d zyO}R<)2pH_EZ}J%W5KYkJ81w9wYjvy@Y*}$Wm8`3qPPW^>;k0LgD0)>^{5f&>6AOE$L`lMQ zb+Bd>dNQhpWn<%iYT@FjQaHzuKyq&BnGlK`awLZMV%Dht#K^mcsi zrFXrGlDZq!!CF6-IY#kj*N?+zKPp;hS<_DpO~TOI^ib|AeIrDb|8 z9Y8b0JeU%`aJSjL?EGmT-QCw~hxLnrqtEo52$NBf`71vHb3fve@SA9{c~l}^o>8s| zO-}BSzSWuL!5cu~q)d5vJOa?KSD`)cg%0<2%KJD6v`qj%`;s&LlWQjrR_wPW<`f{A zQ9ZbsHqB?`eTQ{<>#n^4Qji*MefL~d9AJZ>bAZ2s&}&HZXSI@8u>qZ$GpE(syug#o z<(Esn5|&iz;8escp8C8!)wVhEvtT!k)+H=2Ln{5E`@_n`9=tqE4;016Rqb+n3qsOL zLpr9fw9Qp?r=FYE^zuvKdfro4T+|Dq<)f*iXWWW}c@^BIg!1DV7OwxS2de;8f>7Lb zLg=AX)XbMJWKA-fp)#yi*@acyg408kDj*nO{7tWxej606gQPxQNja2X0fg0hKczOq z2V=iHWmLuo!h3G@3P3(nsNDVgRKik0TSEHsxoo7RENzK<$~wu(_5NRU2MmZRD2o4x zC;ZmY{#y*_ukPSvYW~XrwX`#{{~yE$R~6ac8U#YmhdRunG*Ef$)({@hm0-hpx`w6` zX@{j;0F5FVI#UJdIoS7$PjOE9Mr$Bo)KJC?qwodr%##C?(;jDMt^L`9&GXvWT$4Pr zmbAM2qXMR?YAn}jXxG-(mnY|nnIHGtt@L%7rpT3s`j@4Hxb(;bmt#GmqpExY-V1Mt zi?o=AVQ&J-xSf?BK+2UZxI_4}Nd*j^? zaT<0^m);g#k@#9%Ai%)gWxd|F2yIe*aIS{U1QZJj1@dd)p`7v9qjaEc74U4kw*Pza ze8^keLd}~6YM4lc6>A1M`zLbqEnME-^&jce`$${YYGuN1Ru8l=O@iX}qdQ`BPF%<$ zT;`nCr%0&dd>ZYRDnkQu+IIw)#F_h367!L(#xJ{uFQ*&Oxm z-`FCYH*Fqd)fmDx0Ju)hnZS{?o0_u&F&T~Y8vqmcM5B@O|EnEypSScyLS8d~xno&YCU(VW zU!f&?zh5Tr8SWrLmee^mQhF%O2v#??men;8e9KZ;a%%pEZ2RGXwO01hK2M2gd#6vi z#=%g<6(LQ%AX8BKWDp6%#Zyf2nV%aMQ7=Sr^po~gjde2PTX87E*Cckg((k4Fs3YU^ zKl!Tvw}~V<-Uwff1ORZZ|Nn^&xY#=xnw#poI$8b?_wQQA)_IE!>30q^z?qD=BITqx za`)ftFUVd&vgS1Xi9oL)58O1*-ZRuNE1gFaf8fN=DSXe8=Cm|2!jkjEBf9!26yt4eRt=N_k;bn z>o&3f0bexW0h4?qZsPHAV8-EJHq=#>71lP?<$etN*xi*EpLKBU0Axb`5?DgSh7+Bz zeQn_Kg!g^bhq+5&Ff;+L>%^PA4QT2F8~kxB?8kzs$GiO~empdEHuU8IAk+=I=swhS ziRgf}4R=GvyALxrzR&#%3=y;UVs8gB^bH(sB(e)5kocXI6+i+#)ZTdxZ{Kxn`_uXZ zSAv<~ZNZN;6Ut5jIsf+ROMnOT#P`{i3G?rt!fn+@*964()nL0=QuV{wfyX@Q*MdVd zf==9xeF8)vgwG8q=9wpZybFSXHvBxuk1qziljknnJG%gELgrMKJTxa0JptZdjp5%TM<@YPf)Ik_D`Z$LCM8aFi5 z+?I@n&*@=s|J?a`#p~DTfx$-MIgSidDWZ&@LAY|jR4+;7@A%8Y>ohzsmf|aYuwwBI zdFDP2f(wZw5*PKkL^!ZB0EqS>i6;i^9MBK4OcZN-*&G0@K*>4 zx)E5vNN`vMv=G7?q@mHlA16bo0TZx3v?lT^Y6(n->50seef`xOJvN0qz_Sn)4^}71 z8x4sOWrg-L2xWwAhS!9WBm(Z^5yL+~P8QXtqzaD4nF`t6LK>nY@u|@0(005!3Q3pI zbUEVqVoRkxqVg<2ZAfDPHAIaviQ)1ObW0sUAWsU>=t63&Fe`7X5egDDG=n>&pr*~* zpRUk7vepCIxF(xF95-bX_f%;zRCVHfR)T8v&-?GdTG1HL!<(d^Y2-ddZaJf8BHRq4 z{)bp1yrPg?DQ`37m;G9V5+HJk3M}2s{iv&q43HW!P{F8-P!2VchjrzPN%h*zg$kTX zI^bJ6kZGfgT9wJsM;F0W{)HzzOnbtup8CyiiS+GNQusl1HJ@Qu| zI}8*`@5{2{3&b6Q$me1thHk*CQFhuL3enLg6we_14gG)=8zr8HQjNZwql4_93@k+H zN8KyLJjm{{6c}kZ@m^7XWyM zMetW}hVU1;f+!I8SPe({x)LsiVixAKjhE_}K?OYmwBiqeF~VXg6a!hvCTxtVxK-Qi z14Y@!T?fsYo7f0vIM2^Q()sPX5^4d(RO59~7!w2%pOz`m@#dg;ZuqNh&6gRjP7YAa zaMwz6=J^(0R{YZ~3?-B*K07OWqkj6Vs^jA=_-L_Z`S=5m1vJINi(y3$1dcRfq-UiL zG*GW03o2&#%u!SilvecX}87s40zljawJ=ypRd@(3RrzZi|x z%Nw^kJISAB>y@CVetViX!Byxrk_V)L9=bKIEh=|aeUXWXr-)wY+hDx_2t*<0_pP<5 zEk`R>ihlezEphFuE8GdKJ8fv4n)Q9G4vdoVoSp!zC18yartFRw?C-{X`RYT+4+JyR zUCpwY=fu+=Q`VkCZk>&RVV5=ZW5Dh~ADDO7!zWZ4VE-xBDD`@Wf~?ZmFrj^cIe{oS zEE2IjN3I79x*@l*$P{HTEFB&#rd$IEGlapA#*vbtAz34;KP}RPW+>lM*Jy&TFG&+& z4U9&+tm8biKJJhSxHbm!o+%K`d8z)!ca$9LSceyT`D-xcD1BH zJ*dNCc#_1pR{fo!clX;^Sa!Gbi~Wlg(wt~t95Bz$!{(vj(E2T-Y8Gy zr4nUuCHk2DYzyrUU1ad&4hD2v&w`j^${<7i--%NecSYLL7JZ7B_l5kACeZ*e&{_n4 z0|dRlO(V&pb`TUra9u&Wb5cY~X4%prBXR9H$+5EqOqUjn{sp(}2;K-l9Xe{E6_erlK@bX)Y&gsMkuJgvtE{24vekyh_GXl&7D#d}Z6j(q`HdK`)>&Y2hE``{(14|uK zA%x5@l;Sjnw^$)mbui1WlVe4aJ%^Gx=Q7b`8ST$o7$-$ZHXaXX{)VhtNv_kn4J~Tj zEjUxD%vAOVYAvbqnpqJr!E%Vo8x89K3OHM<>mYT5DANorZ15FWG;$N*_kM#9FZB2Y zBV5+jlBE(iL}|Hy8>{-Q%Vz9me|0@yu4vhP5lW8=w~&$jDm}y7zZOmeR0yfV*V+D8 z&;u!-^A~F4W|pI?1HMVG0?aCq^@d(}7Wafn(o0O(^4&B0M@AA_w?Aa-2d=T65_y@1 zz69Z@eMiDJbH2{$rPHAB08N0*Jbx~3D`e(a8O7Yy`V!Om4LdW1w(Ru`k+exP>FaX* z@mI2^h${no72PWI_!qO8PYC9@y<$p3x2hJD*p#<5)eo`*x}`lV&qPvbFYIYoNsIKY8t;f$PpcgKBarCa6}xF1gxSMB6t#h!$SW zj7MudhX_slAIjdbITNVc){br4w%xJqq+{E*ZQJbFPCB_`+qP}z?S0-)Pu1DA_xS_s z%c`1dt~tlJh6-zMjK2F$tUvpCn;CUeSw@Y2Yf7_R)gO*!SeD7C^*p9WGPB+Eyfn0G z^45&8Jj;E7AndVg7Sq@vA2GdUX@|o29FsBlg{v9nT)ULur8kQ7n!eIK9(tI#>EeTl zs9GMu)T8;^KcByn zO(H2jYB_vjjobHfH79Xc=@%RI=gBY0n)!2(CKiL-N+`M29lg8EsBq~9 zh2LLlcOu*`hQla0lpFV^REgP$#Fy`gYEi~e%`dbe3!zjdbibjUV5cJjjCPGDlm9IjGNo&I1Or((FtD0sBesuv(5@x`NOICOcT%w_RrvK(n=euj3o}n zj?sJF*ruGuc#U-SmuVvWeYEL{tqRTe)qAx#H(ON81Ap0fM*PN+JA-*{RBjP~h7@g;1Vp>#h-zE>D|9r7D{HnjoI6W)~=|0U@)u?SA z;`>r>ThYgB33Ssyi~Jj&WAQ__r+ep`#q9K2F{}bh!x2$mL#d)IvB=#@^&zPuLB}+W zl`9UH>)Y#h!q=>fS#i=4@Z>EO#A@GT-Uc}vjyZr$MgBl*0*%qPu0xJLUh0_TM~Qx3 z(3f>Pwf3W}wP|V|7XsszFA@Gp?+gmVkjb+(!%t??GOnxp>Fx~;(IG0ztgy&dXE^J0 zLNjBuFOZ0+T^kMZp7?tF8dObp8f9V8=nR7H>G>#ghh0enn^x1v{Gc7_Ij52ZE0#9H zEWgcZ#-87hLPA)U-)Kv!ahfVpX~HMIQnZ})bn>SRMyKeH&0pZ5TBfCICluv#M_*qh zs(chg#(Fk_*v@luy4MMortf=K?bu6(&gYYC`E zwFtN~-;mWIB642cpzUyNHv=QubF-FmszApK$2w{yZ{|jeAM)uxjlODISOygrzg9zV zWyi4e?V4Nt52R#sml%ASCTcV*;^z-@N})eVl9YpiCpM;@=XAR63!Q1x={5IAFkY|U zXUkpaiSu4$mC`EU2K2CDyR=eM$=Jwn5JjTT3*71RAVsH)o;C{A2^`6-(90K1o9LP4 zorR)Is<2nMXQq4_RRltv&2gI!-(uI(zT2s_^X8gajy)Y=6y6%Jaz#1oYIaLDsbf1D z5;};-{QA5wcC=>JCrsLIXuWV|X{g-)9)$>`s!7ZEy9M60E$!V&r}WlHvk!->TMMPW zF)NB>2d(jkhA?JC+()C6blWr5tyLM<;G>b1x=L?7`E>m{7SXTdc`kI@bi-^;JF8l2 z{_;KT;)o>g_Bf(fWCl7gY_$Me-9Q$>cS1T^ z-}R z;mnwVIKfkr;cxj;9&r!w1KF8DBOO;OBJY%4n0QH0`#(x)7Nc+hn)7S zj91!3HT@e`=>m^*D~ChmL3J0|>+VoC+@TGCuE|>jmyv>To+(_2=Z~F{FOXdD)}G&P z&)Yyv)To<2-$3|FmyMZBwR!-7zj9EE>4IYroH(SW(uB{qMUczNp%e78gj5O7oCvym1FkjEo26-a#2c!U<_fKiH)zx)=+aVQ?;z`9ZJpvh z*V+!h`%v#_QtECyK08HLVdl7yb~vdEY7R|Id3kaUcc-x3wUckS@-@D1QkiC=FZ?bUSNvm#eX4GS^pta zQ&dDlWJ@-Z)UvEp9H(H*{?XOK-oDzW&URLXv5UcPVeYW-^Z|JnIyUT*WzB~bLXJ#S z^sU(Uf+@y=0GVxdW*~JeQ-d~`4<;RaDzV{}<1riOCpR^RRNe$f*JwpegO;UN;Y>{G zDp`tWXG$_Ebu@@jQvZ#I$l_S$R4dXN@d9xMZQuOp#tr1CmhEvESk~-RXtbH-&*ake z%Z%F=D!5YN%h=6|KP+}$Dk8jtV5Fzgml?Lz%tGXBPIm1Xk&ER*ZFQ>h4Uy>9PsZ3J zh1-wStd{@iYIeI^Fs5-=~|hyyiAV!RQP3NbQJ|0PVo zeL{>-ZQ_H4Wjv8QD})UHkesiZ(VeafiBo^upE_K`n&f93a>&-?`a1L0wq3a*^;9(g5d$C|WT!uKO1jMd5y` zc4+Dprc7wXF*s1;(gk26$Rmzhy2Lg{SOB@n-n$RV^)6}=O~+LzmvzNr*S!6pc2b;6 z`lVyPL|-huv#rm|aZAWwJ#Gb{{c@;jcCwW6kBYiZ&5pW%C`lf-w=5*!LD_Lh`nD*s zLEUclu;#&Ki{~6z`2VS>ORDqAx=~Z^a0(ok7Jw#o3yY-nT57hSyee}oZbuj1&$N+5 zHdMey`|tI82XSi)d&-9^58LLWJ2Pv*U6pkcugEtKy0gMRiT|HfCr`_0R?0Swvtg^b z1P9qlDX_o8RVcBJwTkX`t>E2~wtf=2#;dK=db;mhKCgFZ?HGr9g`|q*vb=E|-KGLu z`Ij9U@K3`yxMe@ZloXqvY#M80!1~-g{oi!I1LUkLS$V&*srgy^ht|G9BCliM&(BO{ zZog2t^Pku1yNbV<-d+q>lrOoy**f*HUL`7iW-bL{-t7NV0GxidkWLQ;1Z0Hq-z?(( z)vW(DkE}=2_TM7zzZL;PB~+I{aq|rI9{?_YMZD6vTw`zg(vBictY*=TZt0@5lKppt zj~)B4#V+3sJO{2qNph#x$w}6Mz1f>l*+a^)7f3nrIrg`+fsy#b$cs}!ex*#M$}yn} zA1Cjww*0t+j}1TR6^~y`T;I5%$b;2#7LRrII=!kYr>ozw($MAB8G4 zvLj3c8HQ3~qpV6Sjys=WRLo-_;JvCq$Jx~uZ77B!#5`a`l3grLhfuUI(nMN@fOs4$ zq^KkUfFM`_hLU%$3=4uKkM=LU;8Re0gbbZ|XYXf{D;;wqq-r2zP-hDWjcOGGhXPGr zlx`q(ccvymw1G?r4dEF#9~U)K$#R91$Xa6*4vn>m2177!?hO&c z5>MzZS8}#Q6ff~w(u7%JXK@gW84S7;y>61r>>wW+oytC=u2&<@p~Q5{=)BOp`x^@p_%qPf-2h$oyx!i&Dc5Xh*~Nem^{Z z;v^wpa9uYg34^ z+rwb+mmuTk&;Smbr@}Ugiy&(Z@(rsGkmBv>^oe-k2b^B@ST1B*ySwvDe-Lg{$Av)b zSkK3ypRo0^3G}@+>G4b~Ind+%X{J(u``!-SV>)cJPpeLd^($Vm@kPDf*FNNSM!U6u zAOgYiL^(4Hm`-0-X<%R~?94vVz?NVDl%D{kp7CtY$uuy0Mc9$8qdC*6)aY!@Y=0o# zM8=v?NI91G{$H?O4B0%hCs`St#$Rx^Hk34XE!cO4gHd8hLz5**m7-=-NQ2PZrE5)v zufvT()WN4$?&Y(5N=uS(u3`@l*z7|F^-7aupW>vO6h+8nS1F$t@c<&}0;)tJ zI2;Y}lWkoE2vCrjUmJ5IbIbYQa>%Bde{jW9|GshYrQbDjVe&0dl{Y(CNdHi?c&Q3n zSsiVq{$QVjG{OpNR@n1@Iz8N-6slo2;jh^gbe>O{fi0OqnjZb(V2kQ2pW<{|zxC)u z!frZ<0`BN!w>^PVJ=qg)O|Zck2Z<(dP~SiNhf8eP;hZ|;PDl-~qUn+eSaBjU+b|ON zaxkRN)3sVSTrra%r8g-**068}%K((n=M9!hUsl;M%ip@dv(*acfKt26D+gqq=DaT2uNVIKgHS zDA&CGp&0L@0r6Ew7SKjtgF1g~lqqoq=YuPs9Dk+iv$2i=K&T2be|WJ4`k^kgeb;Mi zZ8qt}#EaSv!*~ZGr!LpdbMR&PY*WVi6H*)CgHQ8!#!k;4=X)6>kO#xpTBC)<%MGl8 z($SFCvLm{{Hev`lGi+H%&i`#z*;HNVT8AcmI`2#gcG;%+nV;2pi#{!qv_Q>utwJ9L zAD~K$e`mPkTp$7nDr@HDPK~l{Zp^|b73uQscS^K?p{0$%RozMi0&%BU zt4)PK%C?&vXFDDIdxS76tzxRs$01Vcv0_jA>FS|B4Uv|+XOFevRBd0dCSj4vG<9L| z8hO@-Vz*BHQ9uy}%t=9kzDau=8g|u*#F}Q5y@TqL14Ox*MDy`oZba4T1+>qh! z8$8YfNP`0VG{Q#wpcLB1`h*Tl+p%fu({~QDfqNFnYDi!&3eA0J@Xk)1t zb8_D>mLO>9xPXWcqRJ8wVgE8X9cg;Xp<@)cbjelPENmQPr~jxjDqsrtr-Iw2596EI>~$y zoAf?JoC#DkO58YhP87=p;e0QW|EPMcw*)^da?P?vs+-JAtkV=_Km&ufs%j~nw*42S zW1Xq=UY0#HkBZ(+E%ZMZ@#9ik+1GtJFe{+ITBOYub)C4D&e;|=Zc|S+d-CTY7otMi zGzUlz5@It9Kk%+`&8%q>;xDRU`-||;sSX{QtLF&nUQbrHhUTNh6$VI~2GQXQ6ejt2v!|R~wqBaLFM@pwYpMVGz*=sZFgLa@=1DL8o+&MS__WNe zm(@G`ljSs~AAD5w$tJ+A-n}>yaqj5AOeUAjI*7ZOAPe{QbCCzBC_TYwR6UvV+iGZg zp6J+sbbDNHMpcTP@dx5R`;Ztan<&qJeFz5jf5R*JuRg@d)Wp=z#nRB`e*((4 zXzJP@w4?jZ)Df{DIx5CfY8$)&ECOgES(H{9;Mdm9_@J9bhc*+%Y9xm)7@m7ybj~-_ z=bKNlf>%)(5pPcRdL9C|DlZczPHGH`ZrZmZ#YK%V zPK24zn;a${%$d!^nezxVDV$>At&H1cP2VTE5`^*O8qHvdYK;E9jsYHn?aMSNcoB5U zh7G6{dtvf1rX*~*nKgai4{9wZpYsH0(0NRmAPV(fkyoW_mHqWP7ReI{io-}S$s|Dp zf)(H>9mz}7UjE_4QpO;3sRW{^*5;oMZ%5 zA*#}gd7Jo^Y)K~BdFw>A!~o87G0lPT*Ya_}6J^XO$z_O6t$A$(QjVhg{D=TB|A#nJ z?z2MpMj;Zc%<-$j5sdWAWG{Wjn;Pa=}+2^wd@^Yxu1^5ttA*q*nH3UiOiGz8qVI_Ut69JGnWv&C(-QLp7Gl_ z^%tSP8bKa5(E-0Yzpp?Ea&hg&5gwNHM=rOuX96d$fiLSz59)e0$}40t0y>=tw%!5R2J~*Uv}Gh8w?)6Lj~rlyyzWWgd9b-UE7i zJ@AslcVRDoYhqpIbde)?wi7}{%swoP+oE?m=VH%}F0F2(=jY^(;fH40rYI%%kBY9? zt3J>QQ{#4^no|hK&S3E`1r58C_>tS~X3f21zWzeXG|WSHc+ffO!B;<~lNCh1{1<&$ zpIIg`ZuVnJ?d|7h*IRxcdJ+>}ILq@&fuNH88t41{8?U=dNzKMTfN5 zk;eP%dWnwFN~)}X!xJZ%g}Z;p(*x}U{tkk4v)Rgo+t&>TwHt^F0XxM|$2xclYcUgm zUc47XhZ^7eT&EBb1t!yq8i^*z1lzn~l@xt^x3)$eSAMVh8k(sp5YiAo;OAN}^ zz0mN~vv>H}pxWVPJXIe4czQDzv~Kab!6i?XYP@42&|6y&B}sA86SNYXt%JMZve7EdJ`dMn|DXa|UWbXI~W4*XvCZ3$A{?&vB^!-cim+`O*~ z^VQpXN)4SEDQObJ^qenK$Wn1@TZ0Xd?PplvJJ~Bn+Q#f2l5#xVPSD*1ti`I-(iPgM z1&~BBON89t@Pz z$;D5&ypwNJ^Fp*`|LUGr<=58o=rFn}KBp+9nk+^_(8L+41mFN3E9&a>x~AGlKFvP# z=)ss@EHjjr(fQ>vGQ^P!vqABvqD$c8*h*PNgnbx^>dF@&%*t)7#k$5ZLI*`;J4pIF z5Um&!XpG$y!MDq>JeuIvCbc<~vDIj&dQ)Edp{#~Y+sW~S<)XL3`mvkFIUk!I)vaZ; zRz!dsQCf@JA|+adV!g$_1j_F$WjT^KN-5~6(`4m#1>$6!sxKS#thCBq_3*kX090FQ zgt$W$;C~914C;6jpjB9|Gw>jemV>8xoaA~r8bF?Lp**WQK^GjbOReI0D$AFo z9*UWW(|Fj0ws92O%cY^k{eR|#=kq1W!s5>xvHH#)mvNA;Oyr!?F#_j`i?a*23;_+$ zyaMxkBk6g%5>L2!izBfNYdf`Tsj1Zz7v5CANwiOT*md1A62#LM3=du@Zfs7c7acuI zB0yq#J^P&DTH*xEc~A>u5SzP@3!%A32zl{(VCBV4IXvCrU%oL7un~KKsZ}E35ChYG zP)uk>f0g`X`e3LPj-jQ%)TxCf;mLdZ9=9QvY4`8XUnvoo*e~C3AN1 z28l`M}JP%iv74gIR78V%kE~3YO2scpuF8p z3gUrF3a%coK)_&6z(7D?|0UG?&lbsSYz}?mpJtjI^1m4`|2yq}IbL>E{m=1oo3H4( z2uxuM$G-^s^qN|Aon)>-Z;==ql|jF+X)epcrEMtc)ja3p$;`+)gQ90j`nHx(B}Y(1 z|4o5Do%NzH^IGj``o-nz5qI|uyEEFJ9$cL-Lb}yEf3*uts}o>rSi%$tOZeN$q}W4s zEW$vc1wm;+&%6urbl1brT3rdrM$(*2)2o>VufUiC|N5 ztYioV$R*s_bj3uWLoB*vzavP>B=MU((u~R+2!=c2GaV88Z`uWmF3BTE(wQ`k(5d-C z|B!h^^~YgMWQYQ1do#fw16V7hhykyQp$((dLPbk5r*4V-eLH04QnKTeQaR=AkM;Fz zKZ13d7+y#vbfk!8mMkx$#E&4%k;OtLcy4@$w<0!ostVjVz?j_tNBe7s$YlEW6V^whI{BDtI#w-)yb= ziD-W24X;|R7yrh`gx3=~$d^CvAo$T4ol!txtfEn%AMrL?NPXU&?t;5Lh#@pg@aX2x z&<4cz^z=UStNMA0YG0??Tph5#Jptc#&}I?B>mK%oAEfWVxL?&o5A*pkZ^v(59J7Aj z)Dr%3v0dtXFj(waF4+pU-|@p=@veQFe#9^Kb0K!2_pvO0IO`eXSL!<;)nSSguso%| z81SYpp2$g`9H8e!^TZ0g|h&sy<6{i14;0P+J& zaA7x|FjeJpS?@rb30Rb17P9*}9C0{v(@K}M7aIpA#x}e}A*Y3@_#YoXt2LG5)rj=s z(9OmP(=e+ceaIWTBCx_)iyrq(3AoIquj9u$fq}o~AIq>c^q<6*dKUd* zL=yNMTu#5sYy!*QUCKc5=HG#*T{i@ z20(y;@c(agjeoacW9Vt>#QZ-g>~(GJak-Pe^Yn#c8tQu#Q%n%txjYgPakMj`PFDG~ zGcOGqW?RXm&!ZP>uI$$C7>;}O)$Veq>{|5jC|OY;P-UhVFvcp4mIg>X2d5W~j4>3< z!FjQB8wvG$-x7TzqUU11@Jyx9&~4PwCYm(r%J!{!r^@Q|@Iv)2=!*PHrMy!Zw5ro% z*6F4+bq?vWWcNMs(E02KqI2JJw)UgByNNM?zEyJM4(#V>vHi%3EB@hk zbnL+=c|s6OSQQAt;j6(Qst@Y=0}Kz>$!xwLImCCxRG0)!FgtyNH7pTXXS9`nMk#EY z*zE6QwKTz@oNb))Sx?oIpF5?%UV}R+9*MN$Gf=k|D!F0iR(gSAp-YiSi;Qh&tBizr zGWq~>0BqdoJY2I+$_$CcV8X<=NMPT0Ty>i1sDxes+aMFx1Y5E67e*U6>P^?>^}Ftm z%1P^lm;2@M^LzTX*h|pYY)9~oK=W$3-p|+52!Ak2TW-~N-Q_#}h0fRbce%UkoY7YA z_NLp-b-?@+{e^FP$EEAu%J$OR(Ma_6(#$2L$HRH&41+hJmoL#qZy61FgWe0mnD6O! zUT=6^ZoXIgs$tJ}9sP6dN-xOxB|g%1efG3%ulx61qw|Cv|Ms@dj%#2fERpY0mzO&% zyQ_dp-Mj4{Y(L{I{ask3PVb$L?h+VY!b0@6%W?*X`M0x1+EHQFX-3{2M0&Tmx7j!B zjtYa`*hJ(tj3>ch7J14v%K4s=(TT;{HHK-e#>Jhm9(NZ&~Gb_JUZ zwMrN%ut#aX!Jo;cxTx`x!Flqpd8&-uY6&$<(|>yr#(S2HZ;#%nBY&F?rgQXkDctn2 z(F^r?a84+9*?co8l|EVMA79#zjz`Q$^Mk8%ir6F)5e zk~n(_p$WDKz4?j07bSLM1*jd&88Io8tK{Y4i9Ez9=qui})#-O*O#xCGCr%U1?d7Gn z{AoOX_QDBIcP1Y9j&gRK*^0ntD3Jf~sd`HE^zoOjow)bWbOm<+RSRkN%J}QtN;PL; zw)wz>)D?l7+ppcx_>UELd%PWMqKgl(Vu%6A6vo;O_MkNps*0dh7nhq&W20zX^m&W8 z>AXfAOEx-8pf6eT+raU~#Q$D82&&O!ljU8E3l5?({ZTYPJL0pCR7!$_H8c(ux@$jl z^5|1#C^|}tX(&M=7eCLy3KoMYhqc$#D(+i#HJP0J-W3u`O?G#>;;8kJB zzyaiY5DYIo+y^OEoNop)|LuT?ptnaFmBDZ3jS3XhkRaL|q3bcS;*#dT{$mYh3#Z;B z2|@FSG5&c6^r2qXf^HJwl_cjgOeoRsIMPSgg4dUu-4{b)wj%*?s}`*n9;NWfN;-3J zfdgwf+4!{h0D~8!G!ArA)vx|yo!r~?Tv5;)tYof7=|M_cH8npcFO??PtZ$)p_o-&e ztE1Wyw>Sv^B7&4L!}v&QwB`p*sqj>9va*qvqG{je$)ZdF-^D2gIR@lf~Fe%?}4y`gr4rp5sV2y9lX=mcfwy=SxUGPW#piPZ#y}`O%h!gGWq3FYaU*nTuyRHf;dty+zGNO}PStQvLn= z1e91OEij)?QXQ5E2$^bT=dY)3t$n~`!WGia1{Y~=hLg!rU$jlKUoKZ&?)tG7YC;jr zF~Z;?yR$koi6Tdk@`s{y!Lmh87kg)x>DQH(CI1X4g!=sOwYQQkN|k!9@=zEjWr8uq z3F%j7Mr??L$7MW+U)(D94AKh{m{SyN^T;W8Sl&j2V$6cUEevF_XKm>SSM*@Gp^}#G z$a1?+qTD>1LbszWcT$pdRrv-yhX(g#ZX#423qR!~gjT4))(E&{ajgpOGIl?nQf7fLUHR@J$B~DB*OWDKnl?Z}p7StS#wkLkylP%|gBakEdRHj>#o=OiLzQKA) z6loBJE1}U(gkMM~nccV>+V-Pv<(A(_#54P87PYuDwpDuxRUU%HMh`g=`5A&Q_(!T` zVAZ^Vj=&ZcOnA)<|4s0F7DW+%ky+UJyHrqfMl z*wjXtum^<7#_OL&m&$;p^mshaO_GG1ScERWwCW9OPhf#K0?OzRGMBc*>0S~Ph9B-- zSP@7o*_3JfVWib@XZnvR1t4lUJ~NVuL5K0Oms)v(aP` z0vGs9$#r_`11y~sNbU;nnqQD>Q+Q{)q-ug8N+TkJMR|=x&3I>g&n-oWQonUZgXTjs499$2m=hlD3!TvY`VAG=1*8$4IuQ|^=G|Cl|AqZ$yp}8>cy`zAMp?O zTEx_iqln`AT?@kwh|`ER^$WULzlO$!&#J-;|9FKkxzx>C1BG6wDztk0od47DFa~sa z;xt_te>f>Dpxi#2L}4)iT`CB*)7r8AyegZujS4}6;gW&?JyGIm{1#P2H3&^OHrzPc zlyeC-EYx3ouuw0p(q!eTm3GQ&BWc>S#yE#5t2(Ey@WHC4jO;24y1EfmhN?w$PkI$f z#=wJ?3hq5p{Z4grdgdzT9yVIyt%lDy{sBP)BE+8!d)P1Lf-;L zBp0cO5Pl(5hbkbPR_C9;jtY|d~wO0trmA$Pl;t;NX@nR_BYxM@BoJO_BRN$TP1IwV0tisalD%luS7fy%ZzV>{CW`qJl`AG{!q+Vy$wHBgvL3c??Rj zZzv2`aYwB-yFyB`jP&sa!-p+8iA&#rGS1BB5u02dE388?N)w_YZ8RYSXb*Zy()0v5 z_{j?g`41%6D+atf5>+#1;SQ7B$tDOE*q-*lAYYc1q#$BN3OJrP0~wiis`-Ka5S>u` z*>fA5UBOgkqf8vMB1j1rH6Oya2LxtZS#PsYfoh#xqfX7As zDK&K3ZIf^;65pw~Jw$F`S<1oUvEX1o zMEsNg7*X1nEQ_p{XL;UK2q`t!crOk?QLL{x)I2Vihw7_Auto+6){iKKA(+B^*E~>& zV8&Ci)E$zd)Kr!;2V z_n5s>w}b;kVLf#+@2G5DRFr9?3-pk%bq#zD_uKSMxLcb0cj%ac^?zYRQ3g{z&b**? z-ML9fnCyeTWk~ED*0wK1XZqBk4Jxo?53wZr9PU53bdMGl_X>>mu^G+IR;}03XWM~M zC^Vf{IKj9=g8)lbfsOhE5%kq;e}jD@`DM_-#AQNc2r|DhnPM0xybLPDNP2O{%M=+F z?sVfU6KCBQ0zSDHCV!9lE=Y!}oTGYqrWpkHB3DC{T_RY_HA`!p$SsucQI5wT8+>s@Q5&wI#WgL78hOG-a9*Zn=i3seI zrqgRsQl)hbN5EJD3iH>^W|w+S3Z;Yy{hP;GUa=Pu&Wl#5kuuWgx7hx+l`-_D_B53@KSn`q7>ptsV-y0fRsh@|AKoff-HlZ~o3<_UcS<=Sa63rG32sT+n-kFD0 zsGvwpimi*roNbdL$cd$=B8npd3_2<&gMR1$ENsk`h*aTXUC$X}>O6MR(mZ@UR{h7_LMO@zFF%kc#<3 zF^WCGNgE9$sH*U+bz1fiWJ8#&wVHi~W7w!ABR4lU*zZ_dfbvTh-jI_iqxzW^$qKj~msGp_Q z0?l8Y=ot!A>{Agj03Mc0kb4hy!rbEm1%f?%x)<1wq00#)fI@RFV=<&9N>m5Qxfmdj z@rEGLzSe44Yt^9*f%(aGjqkXxdAE20tI?Yvt%!pzk2fC&?cs=kt!kv@8*z#Sb*rJI zt7`mlu4D=0DyB`x`M}s7OC^4TWh@-IJ%7{Q5G|$6C?U!=S9k!IG<+*6V;Zj-FW&-Q zoQOR9qn@U51;AGXF`ChkktN-|yazI5h316Cki$F(ek z_t)UP<0`rR>icHoTG!uUyMyOOFE6nH)Fz}cxD{BFCzT9NjEppBOEHr6jNJOx}_8CgA|% za1<~c>{c+kB{rINVv7V}?zSzd4DAoKzJGsiC+-Y(8bw&dfa^T7m9N3EtYe&1U8_Obw>JcN~<#1(@{wphA|;42oYj3QIv-|8<_jtyvD zjHDt&#b*D7%tV5e06Bq7Oe9)&(a{cAKYiEiJbBK(r-Y1Q_Hze{pSy@gpfn4eqd07D z0M>f^3?o40?()GpsGZH^8RcaX^SpJ>_%judVFS`K3)&za<=@f6RdY62aV?@&%LX#F zp_THa%GngQxLv?j*~J}#p^*FsxSw&tGeeWLeP_79E$H(mBC>@SZep5Jo(QBwsMC%| zyR@p&Es&_5DS$nbS@{A~huOxI78_0&*8%r^PE>ScAyF3Mu5zd1+4ar~U=Rr{V^GQE z81;H7*=Uqi0vU~NTc>vbh2g>x*8)n%bEEz<8sA_UG&8k9=6|;RL7evt{1EbhS5{sfcW1=77Q8Lw$!0ty|e5f2mj)cD*V@~X|)V0tF^Ujbdb zeiw-wkZE1`30=rh7d=%EOPm1?zfjo6NL*iQB*+7fz3{s+O*r8$-qS3)dP-3KB<_=l zEmrjM8`=h6?d6KzOoWafso!}&wa!R9p-90Kemw33mEC5KtF2*H-nqW~6^M7%?BO~6 z_y@EY9=>hmcvL)zEBSYqmR1}Uj(_n`BbD|C-D0H;^TETjuBMKeswUNp> zk?3e~Sd2|rckDCalP#@<^01b-j36aw>`W%sb3zve>uat9!mJ%US1sy7hARisUP6Ts zkxJDLpz#Paqh&OG9^Lb6B*X6reb>8eyvOHRul|#Bc^_7Y6G8$f*Zzxi#+o%I)k-tD zM=8B|Pqjs5UVIgG>BFwt1)=;yl_wa@EDO)lRs3|UBr<;T2+ir&k(to6XM!w=bej*-8IILw%rWElUHi?UHJB;y$4m0O8hje3)SY>p8IsnY;axH z5x)4II8~KloHO}Rw2X#+o_gRHxJbcB_FTh~QGWKta+L@sXeu{B>&dT$^hX6(JU70-3#8FRU6-MQBc}eVYWH#iXsxgwYX|?#d^AR3RI!$vGgi*09 z;W#~7S+dez9d=07&UR1;NF(UGCCXX3xKn3Gd2XOb<@H-fdMgOFt~Xj@JaC+JjFh+o})Z|4UKnJm4|x~q!fTJ zA7{_m4<=emXFHK4L1Z}`&ek3OaU2(R4O0SVw035*`GQsdLnw3p{;@rafIefDO)?#e zSQ{Lx2BAW)UYJk*9tZhOqgKmOi(zN<+RlSY0vS_#rsN71hw7f~p$#ebFL|mO?C28s zC19X;UlA%XL7>in9?R+4UOU#Rju&wb$5Ripj<;BGC_-cOXe4W|vZ4-ped0)Z&RdL~ z_juvT66Ee-okUwEC$lg0-PIsDcjfqH&fa^$c1y%{+}EW0Y^q2DLdO`yl%1(M=E)*B)49?IiB8 zikI--fi?>?W?BvL^$uuQ5W_VY4jd=Wsjp~vm@QMF7Jvs@DC;n{9*95imWEIf*PTX& z8aku2C1YP5&tkoRSh+tz@dicQ&Cy<5L%}H2*J6j5wt|K~+UUlZEmGliWHqJ75iaea zzrh0~BpncK5VCF-2ql!5P~BN?ZlH3AiW&ewqX}){rw#WWar5M zrH#*w`A1If+Ay>dWMwq{Qq0-g-liMY=E~@%S4ee^XJLHfK$R0Bdv)!}SyF`e0#VGE zw~KajwGIN87Qg-m+Qy`-Aa?uuOHOdawSvEKvm^Zu)$5n2U$(CtyxT4hpH1V`S-??h z1K$T$z>}t)u1@c9N`6$I+5tibrneVF9V>vO2didH>@5tG&mW^#bkDz@>Wb3hLS|HC#43^#}}a6uRLy&r_- zXgsLvNL?Y zjoWRA)t1?(&&1%!6SfX{aLQaq(MbNN4xCj3SyKz5VU811ZR;Xxw_#jG_HDjP5)Y5l zi}A1;v8>zjQq{dBa6qoB_&RfVgriOR5GX&`C<|4eaJNJKtO!M~q|rE@w@|KAKjU+@ z&ve~FWo3n|Gg`(x*R0-x&o<4v=9Oa-_B@7(BBMrW54tey)+Gp<@vO6rY77akv7(_n zKFR7{m=QNFqpgGE0-d)P=wp`Qm>W@AL*>u_An18$WS}ahLG!~M@zL(G;@u?AT`^Oe z`EHZOu#-gd8Uv50v87$H4mol5XXVd7{dn5e8{4%z9~j$Pna)NNJ*o-iD{oii&vptx z`eM1(7My0s9w$PRD`}*fD$u%FfaC+$I?D@BTT@@+wS9l>40BN>`b@EV;0{=7K)+Mp+qCzN2(7?6-0r z^kbIq*gV*~K4hASO`3b-lQ$E+4pniR2MFrf=wB=tR^e7t+c{4t-$s=lyHly1SZjKr ztW<{Dt7LS4HetO<@RQx^Kw6vBT*A*K+J4OI=RzAbamEX1OUMTc!_`2lR~9qAQ3q)uBuoNsD(dam?55E(i-+qNF$2?=a$e79ilEaud$m5 zoph9$v?hyj%cNLuUnPqq?RB)!?6S_sb)ndm(o6VO+HPWH2zg+(Wk{oO0?uyI=EQ|8XUYvD|pS8xhz$0iOiCD1qM@-Ykogh@d1{ z2pM>F0vwVa9%Qe4wRo{S%DSDj)OC{)3q(%bHDdJrKGT+&yy#z zP9EKPDjPsAgt(V%pVM6An@*BXu9Etp2XTe{5Lf5)aiT{1RA}ASm4IMdxg=eGcKtQh zD)Up#<~1BpIu#8ZE|1dV)`9b=PqmqrGS%QD9EI+{=@zZJfS5Y69kZ474-mqLQhNJn zo&11`Yq5)~FRY?_alI?vwQAq>>TSiyyAdg@I)`+p+NDX{HK3;&xJ}M+FryhlUQ~^9#|VB&m44b4hahwhO$}Bx0Y%dNGQd( zu*+j>Y~vd+=<26P>?stO10jex;0OsvPa%UvA&qp00}2)84V&Yv8Xpr zEYUp7vgFIsaB-O(>D+}D#%udO%ybHoMcFl%N0oiT<492 z(fRsyw@ZP(5X67JZ3U_$=~(+N_AiY>+^><*wHbFr*p+JX<0(xEAFd>U_e$+=8T7PRnvR5Y%KB-Qm&Rf0PW0A8~B z%t_?@-&9VaKWgF6f)YZQ`Awn5A}O7)mAZ%~D@ zz2Uth7kfa~o2z$e_BYKZ?bh62K?3Au_Z1Oi&S|prLgm+EMxUZ{V{9Q4Jnhx}@F8{1 z{f>9H2lwhX%*l(L_gRVR+Du$$mAi3waUrE|>|I-C@^UWU#%oh^Y3casJ@SEsllftl z*OQ5EfIH&Fn>Nj2c`3|L-sL#epzmO`hB(xnkr`(zzm7C9J58to{KCz(GkcxiNO(7H zyB`~9^#Bx?-}FTI_TptyyXvkX`vh`A6ppqinV1KgHcj(`6pg6FGXKwB!74+FtUALR zLtvEjeSgz|exahEyZFJ1n_1~nizO9^3R%Sz{R~@qAPxS^n+NM(KixsSsIJ4%E$rHW zo?S~)Nt{i()Fcu1tXitW_6w{hDHe6~yDDgi5GZG+Y2>nrnuuJzY!Tt{M(U!ho5 zs;Nv^Kq>X<8Y208>JyFkTVZt^KlsiER)uc!MHF9uaS7$U$KU_Viy@>S&d&d%dubQ| z0LcHJyqLa;sj;Q=|A67U*4nn;WJBXGm@xM zeH1Azh*ip_L{U0B8@NB6=vTtM`t`K1P+r9hgw8ki>#rPrhN>v%!8UmpLyNv0>Zo6RB=_QVQD=D3_8}NjfTA9#OP3>NaNuiZL<^tb--{G z+&u6E>Kh$=3UzH+VA2f*dt=@;@5pfFeoHMKAstXGxeNZnt16ut1lV5PmD5TbVz`(S zkyIVYQ-h*9g<4+e9!iCiOtiH2k`oe^<>765VyUBnik84YJpkbjqgpw(bf zbT|Jx64(sr$V>SnDD4e7@iwhG|S^F@Y=PQkzl zCvnZ-@mS+cF_`(%xRr>nGLctkf-Cf#fCxi0!#pTxW#8$f3XE}H($f~&mAnT}mpfjcn|5GtR*vh*xfmd3eI=lf z{#2bW=f5)Z;w6YTzDtE+1Q7~mF-)4Th0Qulu%|03%_%cF^jC>jD-CmMeV6w?bR!nx zTRmPUjK*bdjCQt9W}rX0D(ZWQOt?9S=QqZT-X-*u3=TXRV8ovCGS$Y-a%8Tav*4b= zJ`JOa1;&})q%RONujA~c9puerp7mU1D_Ew$Rk$eV)GVaDl)DO;5hr%}aox?Z*WB;D z)@0Z(?o-KhQdClEd5cP1#GNR(=iL1=tvePWu0vM9(;F;T5mRa_UA5J)D2e-8wg^iB zrUsx2gw$5F*eiy&Q*1-2C<>oMLEg=5LEVcRv$&sRNtdg>ne=Be)knt(uc+UzcU*y z#xQb){>);$rjQb%Y z%{Fdnu@Z(LZ=s`4nA?0RsBYe*&D0L5&4$K@#q^U7j(`k6-Z74f!q$O5RSI84@tYiY zgeL*+;#8JSA`IC5Ey;1ie9%s(HsqqFKPK+uwZ+z5WU=&HbY0ZMge!<9;t8x@a8kWWb_z)#W<;l;Dxn5i>8oINim3I}N5odU zw@iT~T|h@hq`~6vS73?bjFwzHhk4rPJeMgrdrP&P?~Hl)Jv6bfmXf^@83l;Zzlp;J zDRVOvTcEth@7nsOsEN}P)f~>2C6}O_pv!VE<>m#v4S-ua)!Yrjq}`7N3c0Jhl`goi z#XRlm9KuPV!4CdN%^AnPsS8`1Ga)+Du%wtviVF6BkIN927P&yA6aOcFs@7-gm8gjaXs9CUd3e9h&dF(Ro}5cINTI z<+-VPWc#kVB1VzHgV|i|oOYWvZIzF3cZ3BaM;AAF?Woj)U5-XpAQt7upy_U{#bZEJ z&riPqQ9=05n7Q$QOx}39!Qea@1JS%G&0}@SOCWb={DUu;nE2Sk>CL5A*}?BYZDGo} zdd;~nl7il(dTrf<6{1~3ogpc;+Ph`ozZh{kE1;8V5cVhAndl0`T7;jp*+Jda* zmc}_52XWb~FkavuO-=)MxzeB%ac&Aq7cj{FC>}kZfRWWNItQz9#H1QpLHKT_2_^{z z_Q{2S_~&F-{zNhmr8Jr8$bH+(Y_is9JON#fL|{XB)u}Tfy~94R&lNvi=CMzloFiS8 zROButZYJ$ZFID=pa8zo*WP;7;U zLJ=}JgKr2K;tgL)*d!6?EVUg?l#B?YRL)3%6mS7hA$U+Yxlml(naU6|<_(b`O9|!o zBL(^AI*$+!A(SMGWC16jp)33EkZhhr6I8nBO;{L7o3pDqjSU6K%`79M2` zUz0l`EN(4BQaeII+!A^w@-W(>IVJ|1&p=y})S%dQE{_h8;1wt2!301)yjU?$g~O)| zVF^(X%QBH|rI?q_kqq*zmN5VIodi~ob4H*7@ zWc6Ou&WD~V#vM_gB%c3|Yj zsSR}fkSU2}(sUj)uPb+2c6-m^$=*c}IFrgBJN$0r2|9g%6Hb0$#-u+hXHQS|Pan$N z*P$mfFpR07KT|t697)2)t#rK!!R&GYsQFvTe@fRaAQ=tV1!kM zp0s`}ku6I{zKhK6>#t2KMiF*tlO<@6M` zZp4L@^ll@_#F3}3A1fgg4?3#Dj1}FTX~jdbUrGcZG}jhoIWoWQ;sj_zVg*wd+~v(h z;h|Y8rT>xy5F0k0KSM;&1Yv?k6iOh_pPSMqOI)$9l2D2WNEj`{N26n$Q?GB_wMvI6 zWBDnTTr{u|3!o2A5RD3}4ZpjSgf6xN9FAE;^^IGP(j3iCB!O5OXNc1-^p%c+(aWp{ ze=A2^=i>hoFH^Yk#+}tlDzA|a49Ue*q#44Q?Jp~IhteUJI6%+Q9>QqWBpn$Y_Ejd) zbVLa=Uc)Fn_=oSoJ{$n0?Whh%wuxj$D8qYwRzlpelo?7bCtj=Z4;w`6Iq4juQ1-`o z5+T)`o<;p<0=iwuYq&&>6XUv9A1Px22Q)q+0da!>6G0-E>U=YU;C>Ba7NrSwD#OU; z@}DBz5sdu_?hU6Y7qjsmxu|%RBZI)t&rk0t6Y3GLru7ISarIu|$Qb1hgQJFSHwE;QrK3!*+> z=dTx0B-ZJfdke>m9rObr#Xy`p{E$D+RznC$R|(b_u`J9eQYz6`-<92}jd~#`m>xG^ zx%jYRMga%~XWa4D@ReOB(|ilQiU-aqh5k*B`{d5cA21S!>LV|LW+2&Q<&ths)Jc74 z`5uSY;nY?WNi<9R-iQO%<#6-GS4%YIybN7)4}u{bK9JH0ViD*J#Xl8{3dI0Go&iLp z;K<IZxPxi#U^F& zMK`fG;$(ImVB9TuyYOKqnQ=_1nBW$Z>O0I2&FI1v8bA;wu&6szIdS8l4FT^S+V${Z zq@mJhDWPujZITL)WqPq#9>ly4*9i=R`1+&`SEw2Fo^;0PRu9Q1HtMW`n?T@}AX2Z2 zC)9_8vkHd{y&yMgq4?1sx4#RBKuMnz?kWwLy^&&1hz@5iiq7NH2FyB`d)zL&QT)-M z_tTb30~7>Z*cvm$U>3KwYxe<#`+Im3Rr(m$@=pt*fH7W?59P|UYE2DN%pGw2uwmq| zCcMafQG7efHdT(di3dv4v15LMm11w=aY2qp;VYDlrE0Z>8AbkK55#!+x5z-+y7Igz z35Uy2YuTkBR@E;J>mX#Q;KBYG2$Da}=Lpq=#z%M31>tZsI*NpCDaD4hnB{7MmZ&Gd zBh$l)4}2@S-LoQu!n0=UzA-CGaVD^(udf_zp~~i{DhhIhX94^V8izJETYXNmjI~#& zN6GU9spk}AjYe5ibXYJ%VKnV&l-$_CAI_d2x??bmV5RZ2rUnA)HfD!S9cltJB;ANnXTj_QGvSLJX1kEI( z_UlYE&KP~+M?ro!H}-RZGe;Uebu;z{S5WNV-57zQoXqak1@G&{?Bh_H*X4FT6ui~u zwksS$0CD%WVf=7lw0hnCG3OqGNE;JgEfGfs>h}fjl*-maPENn;Pw=!sQHNbYDgz+U zi_44@sA^^R&x$3s|3vUOSh8>r5Cky1r+?SqT90#?jT zGcf_z9%w*v2-a4Lt)f-8p(oQ@U#>m4*Xp_{iZ3|W7=Lu#>9N;R1{QJ4k*$?^WQlq| zpR=C!jb0R7bz252xMd)oN%8jIf_l>CCl4da)*8(^%dBP9{GZo$MlEiYmZ*lICP|=4 zNv}k)pxWRD;?{R*YO2KKTbU{lY)et+T%niT6qO>o;mK7XLyiCaV~;+V%&Y0FdnLd; z@n>~4_ZLoalIYn{BRYH(m+!-?dm4%p$+1CK(K%e| z_iY>^y%{ZWlBs&K*m)VCin4%Mx>`ayrrY(@%U7y7jFur~b?+{xIGsBmtXeyh=6>}>`&D}fw`iMMgY(jXW080Fj!3{u6J?N`1|^V1`jPdT z-h(qme7n!>REI744Grpos zWxckcxPe;S?upHYoRE$B85_LYbI3IQ)FQ>u2P6lBO`A|ek30m8f$lcbyP(Y(im5= zq5xzpa(acaU!=7%ahU-H6E>{hq8bW&7ZaSz8y3#Xy12ubfx9?4_}JARy*#93kRktA z(0CDxZ*gouWW6Cc7)|N6)r22~95#;odDvKOoDYrLzqRkJY$91(PrT+pUY})8IZV%9P@vW@#TtNH1}T7L zuuqXvrpWrb{s6U~YfaV}Rd$vQt4F3`xVIYCbTY<(_f>nw+p-O$WzdhC69x*G*{| z`9_}MJ$$7E7aepCS0zFuDBB2?iki@I9Ph+%5Ab>M6^8Z$(EemA#t>PW*B03pCtQKr zQfKtG){W-P)rS)i$mhp^&JZe^z)6GA`eWg2SJ}y!kklOPZ3TCw7Bu3lQEV_olpZ>; znhC}%0hFIJyOPPIx8g_E-z#5)U}}o~a{_9OSp`^?tUr$@4&v5WU;q1Bb_B3ynX~q! z5x=~cMP2s!c)Lt9ec$iRyzI%=`#k;}%^Xi;jqroFdb@U7fKf1}q6$NPeK=%KW$`wXAjB;|)6Nr98t9qiIJsNtg0 zwai}H}YhDUWPlq zYh}!ZNKY4UP$cBw_bd9bAQ$Kp{$?N1 z=aqVF9s&}8+UNm-9#BfOEr{e0UkZo2e}@po zXE@vUQxfs=3>vm3j}|4~#(+%8;N2D|sx2{`c{KuJN2k3Lh~gaJgk*VHQJ=pT-uD+_ zAXUU68hVWT9(|e;6~m}QHt%1|{{YnblS=THDT}ep_`2&E6riP-?YLJ!YJw5^QX38U zq)qqr?;AcOH(+a3y!y{^xl{0S{Hp@Cgin|8A-N;Xw@*p0lLi0y<2g&P+qj|2;Ur## zyd-~-Z4r`*jjhGn5COznH*7I}4%ViugKpEM9iD(Wv6~*QLMTI*lT`}&#_IW;b=P_6 zoq^;%Wtd^N`FU_|4f@}&WSO(ketkYO{QL~&o+j2by=GUhRA{U)Xn$QzsaQuLr^(tZ zcgK0dm=1!)gX5esA^)J8*UnAGqfQQrZkAAf6hoA1mnP++dF8isM4dpf1l*#&%spSm znkU3hLw=9hK^-nO16b^z11d^>F7ctCr~Q!1jft~O-S&yZFbO+LL1D%q`P#te;_If= z?3uRc?VCig@xJWd5hLQFGnRFs>cFHsU1*O2xUXi%fJvU>X^FP>GcpR`*t1a;JGP-! zJ&c;w+)3AO=={@ugqnBem@yV1Cd-{>PAfOe=_Bb6^(Jf~jT%cwt2*qC7x5V?%tQ`g z9A8#$DL%^v8-I4eMj|(s*EjC!Vu5QhfVmi!MTNxJf`F1PB62O<=Z+Q(*4Og0r}}P{ z-CDVJfCUHBW@m!;gUVnWq*|#mC@%$prUN<*w;Qe|-9g5K%Mi!4_q!BGi>U zc9RrW#CA(0(b@Vey{ocTLAR}E10@S!T*+9r9L3?M;p$9YO#TwoxzhpMw!O^Gx;TNm z6Vdi1F8a$i!C>d>{O+vv*b7>0GJVVakD+#Z6bbbU76TWJ->uBT#gV36MaZLLgZC|DXM>jZ?QJ31jLwv4TG~TiR7ccSnasj^2f+|>PJ#W zUzpURj@PduS7P*+-nWbx!Pi;QTm{N}I@)!1*0R@ogU?sFv_}~9Nmn(F%Cgx^o=2Xo z0=7L0XEOVzCTEz0hpV?-M$#ZE^XfOAKd6zkEsBN*h}!O)xmZ_kf$nBtxfB4LlHe5M zK@;g$A89S5&YvXsYHL=UsNQ^Irbk?mf{FhuR(PEXX{(P06R1D=KAHDyPYGc=v*bg!Puf}r1`??}ExZkx_?5UHytPuS!Ww_L zKAF#Gyp)f^?K($WuqWI?W1ZjgiV^dC$hPtQp76fC`Hl&UokZa8{9G^Hv<>j)(&gin z+ZF64FkCwHep$zKrW~JdP$Aw{?TY!VC$E}6Z9w`~Jv>KcvTGz2&cU=7wB51$YHp3s z)dY=$Tlpl8p1qwYF4270oX+-myQzDH`pBgkPAOz8nLeWp>BPH=^vm zgw~;Uz`}9b&KoOYU+{i_L&Z8S){V?+Qt2zPNv14|&0Wh(QhPMN(nL&K_-a#I(cxemBNy=wI?qj-vDQZE;I;wN34Pj*lw_YsFih@IrX$ZnE|8UAAp#k!n@{ddn z|1inff;BU>VvU60D`>0^zC2CEh@4t^` z`7JEa)dwlOr~h+YY7j}LkhTll4LkILNZnDaC&c{~Md?t*`?~461<{uXx=+dWW3_qN zh9&die<>SDHSo-Htp(*8Um{`ptFDTtuslTioW2%HvsPD&wi~opGf4eT}Hk~XkY6v!LTFs4w zhU5lK>lN8nXHFs{nmtg;cytWSG4?d6Y(858T=Gg(O^cx;&znDl*fbqc$se4ay%`RbUJR_6OAlg=bhg~*@>+i&0n{*79lRgQ!>wCIFLxI z-REe`5}&pchLmVPY1K<-6mNF4t(>9W@7%^Q{{EUINbMI$AP5ku4tQcm@x97)4J8yj)$eBG2d@8A2GHMVngoeQb@2zd2;b!Bbl=B5`wdnbQ{But#w zhx?8YhokJw0(b_(5DR`zLFHl0tT>;CF$IIX9iZhF=YO`vx4E++WHy6#YBDE0fjkR0P#hZlWdaVkw1!()!o%Q=!OwS< zh#lQ&y1ke4z<2%LitQmJqNOF96vO+8h%a_wW}#P9vlN$B3foB}?_-PD50C?LDo8UR zT9iY^1^?rJCz%LLB!$+Z&+AZNmS4BC+w)AL_CfmB&ndrH?-`@dHIF$$fn4BJID@y`4Uz+Ta#pxL$JejUvU!FthqQ#$VEG4+4Rd zT|<9t!x}B1z+6s!xbybX^3lO7}>p2|Gr?$pQ1IAv6kRc1$B9v6oWlI zns2%N9F;+uztCQbn1%q$&5l6p>ym$uz)@VlDl5xp(m%-(%ulZ<*+-3S8;l7nIMAgB-T zWk;Ng3L~Rck#c^m8#=sX<#Sw`nn%|5WfrZ;8#pYtbLi$l#x0Q{|02L8`uBs?W=4CZ z^NqTuH2n_1#|!PWb_II>ZhD+W(4Qrh(P#_8h7kp|PaD1$wzX!6z&wk0XzxDrtL}!P zVDw#K9HgB1VsWEnTebVhJje19{0``oVsx444E+Y!AU*W)l;qiRJJ0vwHq9??F* z#7BM-5SOWSQ#@?!4?yP26EUIO0h7Ovc1_?yp`+lr!&HgVfhB9I#dg$-0-`Y1Yr+Bl zK{@7#lJ>Nr9m?5C0l_L{)G+N!qFH%g^f1y2S0t?tMQo#Eri^Kn*39C@*n)#1D-&G! zrAheer>hx1t9}%bliMbhl!IAzl660iA|H!&J6`IFd)!AM8WP*n!Jx9j}LO9@-|6HT786|SUBf_yQ2 z88pA0cT_4Mxe7vCnTYRl#Hi2k9Rcm}$XizzB;DdbAMd%Dyxwo8Kh~81#z*K2t#O4X zg%rgw`lvA+S-v(Olg2aiC;EQ8v=X<|vOEsMHogUoo3aINM#|!Y2eZV7BkbZRPB#nM zm;4;dMb;gI-J%^PLa7WSURg6ISBR5P`nPJs4BNrjXK?H_hGI&V|aSw^;H6eQXrhMNT!X0;3G>Efi7QA))PLI z`(wW#bPIHjX>b7%?VFZWO?;z(GmpcpWjGUlY!8;+5r@ox482RJl8cend! zNa2z@lZDC)|7YsmTO3~{3rZ14dTT^wWxwhp!*U=5MFTyLqzfwppUCM*V=THV@l8iqC2=7MhM_WKcxE9{FiN!t< zC#dMm^Y`G{8z{%h8T>P{Vv=@Bt_UrR+)DSOGStFc0ivQG^6NKqpiL}RMCl!N8}U>^ zw5sfTq%8b?cZMuW+{;MAw(=_&#pNaeONUME?G5GOL*$pxT2G%7*H+G3=@htPbuD;z zLQ#6YGjtcTorir8&BG&(GX>l_j*YgHh5V9+!mFf|H2vbw(e?`4-IySOSf)t8=jd>O z%E5@};g!9QD@KYpny#+xVC&rBW_rtJPUwf}nMNbQ3UWvTAxaM+3I!hbs%rWvZJ5{< ztV24%#VO;1f#l%I!qqxy*qtWx!yAI5orB#Wv?4;2eQ?wm@T5|))KZjGvngER0603~ zf9x`=R(4(?aTy$Jf_qA&vXh7zcyGX5Bbp&skT94JxbOEhg8Hzj7V6aGmvgOFwz|q)q)4%@sTSwy;9EMYQm`Tqw8@J}=K@A~4Y7x<5d|8qn6FE{`<2jl;b zqERT0axqE-004x4RKb7cvHIU{`yUpKUgKE*Gr~M{`GY3-IViQ??D%m5e-1QktF;fa z*|hcC7$s04;a;vLz(~r`wVnC*VJ0Hwd2u*qj_8K?4lg^Hab%-&7V9M_HhN1GWy#9P z1v5W9%JYBc=!kjN6W)HaynJWdJovDCe7=PKUXh>AY|d8AQZtpE+a{YTA|+7}V`na2 zf;%OBILZy-X_h4?n#cylmpCVExu zm3-lpv`j5i4AAfbK(ToxyaUvL3b1v-hNKD)m>?5w{)8C~?0y-yMG~|A;x`T6ysm3g zY&}n4nA`!NB5>%L+ZPR`2f{Lz$C%qBA;sLbLzD*jYGRbe!-P@VD6nuA7#8c*-G%nwN50xd4xBDOX7&Jzt#wtHH<3o_ICo;6!_2! zft>Sh+H(MNeIf#_t}zBaxV==!j|D`Az+N9yGt$R|(?S>|sXKKkRj6Oc?a;PY};uQou*#IZ@|1+b|F zagzKUFaj+<3_Tsjz`i)fNKm$XK+^vE$e*GR!)UtD9n-kzu(YAq@tL4$0Kb9$Zs;SB zJD(p0!X=6RW>{;W>+@z=LjBpga{teclCy&Swfv;>@0miWsaavjmi$umw18vb%hL{rkTTg z@B=f!py zAJl@d#&F^X;DU`|lC<^t!KXM2^}Cc=;O|4RXhK*5)srS^fJHR=aO>7C9T@iqS7BiQcDNdfVn`NQ$I?g$RDN~CM^03w_8MBM;DhB5-qoB|OL_yIHd?9gz2 zjDr62;JJV(ur*!AMAZB!65d4(-w^~#wKV@A#viU#lx;fpvC zSF5LGBa@70x&9)#!hucGf$OW1uYrg5a>0hnnu_@m`ukCD7P?l5ZBWdMeg6Hlz}6#d zpx${+{;gx#!8`Zc0M%RLRYpXcR0@gjV^<7~ugr<8H8(G9XuLTSjn7)TNjC#5)b z-^QbK_;TBbaq@zuCHT{{J68-0r39rF@FU)ipS?}ZmV|m^Htf|eGxC({6S?REe?G$# zb5rlyEc?>hB9#r?EV2~7{8w1zkwOJ!GS^pTJ%Gc5Y}U8@JhKok#R}k@-7Fk-V;&bZ zbB7c-WXe7zsed8Yts~k1DVbAXLb@Z&MVht4d8Q)%JzX$(dUGP{&}p*MBZTQ`G^EBT z765aS71hU*B4oKP$;2AkGv$Wz!mSk^DAV-vyH&D$ivcVs$x5@(d&9hZaXf))s)?w$ zu{pTDdk9fkGt+70A(^O^N_H%!Ashpjylk}&qLq{@-Zjf$Y7LcRCSUvDU_c$ZQ*3%= zzTK!Ux`PMo@0gx&_zi{)U`zN2R9ajI-u*(oB zGz3#utn#NywJy*e*&BsZBiUjb+NyIyd@=%|wx}$XAG`x-FmO6}QWnGX4*8eOdgy() z{9(NGNNg4Yxtn{^tO@PA3{jBe0+GPD1Ddj4T*;?<&Xfg^idDj`U*+Ll`pNt3+-IYM`(nz0->MTBG=%qQ%VY{-+S^@pE)-Gbb zE|OdfNb{Miv-Pgcc~guAr}t8b#pKPuvP%udz~=9qw#BQwZH$PchTeze5F7E@LD~nU zlC;4{_DiX62oe0&2~(uPyA zNrEFRT}<=_%9Gg8-j0dL&BDI-{hr@7)Rln!Yq$OFSDHXev<507o6(EN{Xk`lZA)j> z$>IAC%2oj!Vs>Mw5(0v^}a2y0?KHBP&y%k>UPB)?DIz(!Pf;1*DSpWYI}!O*bc?IJiGzR0=4;~ zidPPsFYQ~}x_0AepSq$L?{?mr;=9w8Z9v;S+#@dIvq{i`aV{}h57213FmhYiieT|o z7{+kPnp(U;)^Sgjmy zj0mS&`*a~&w%^h+dxw)w?Po&+cLhO!d|6Uwk`mI-3`*#}dB^cvf96b~NukaiQ!Het zQzVX->8r%Xx8Yf>s>FQQu$RwNJ=<%@)@fr$GA3%*{kDi29WQP?!dhj9C#lTD_XG|MJsI`z8}PUmrobQ`j*R%6&bEQm9wEQ??3Q58IOLtwra&U~@A}Pv}?FmFP%GIfW7`0^*REy*@u7UH})=Z!D$+HAoN2`?VW3?Y0#!=l)gIB zHNX)3MVi8?xVzTs{tWvye!ynFKizKhfi7;E?Tw;HhMf6p$w&I|#lU6Y za{8r$^J`x(uZ(p%k7Zg+eN|{pN`6eS#ToIi(WIiwh@&Hlc5q?#g0*n31H_@m>T{w* z;EB{uGe=*1X;SaQ-Z+i6n8K1AMD~a>O)1fO^z`-Jy&(wm>WM_&Ho-?c6$axHBJ7<0#Nmqc(BfM4d=Z;z()-tB|%1n$V=VD5Co`N^xGgs_=$5)rw&8)UkPG`TG9p99a>r)Ph!U^ zX26$jK6J3Zrh-=sv}4SB{VaQ;2mjQF--c>w4NWs|h<#a~DFXN7C!9sT3DCPgGZn22 zBQ#Xqz#Ox>m$r#_qSq@<)T;E5E2<)`^PN@CK0#{@ftI?J|1eWU&hrzOzKf#Q%z$C- z=O&63*3Ga)@~LK$@=7KG3sQq}q~+Q7YJIPWV^XcH*|p=e24}3I`srW@T}s(WT>2^WqWIK3 zXm8okD`;fd>5G!VO|zcwE>Y2G^!ND5y=B5o)aREfhE9f$ZLMz`bdN$2`3?1{%g~hd zgIBafu66wc^fHy$80tl}AI6QS0B2XNOS#?HCf*RD8o)su=moiIr|nSU;uL0P z@@wtekwW^R7^UQ|c(GTlM9KJ?AxO-El-sX=xQ0`6WKX%8QAn*>*#^p}O&RUVsK9Zr zobljUXhGqx-GZ#UI&jihYvAm0U$(xhMQ%u`Z^J`p(-#XW0R<>|@AcjW5%(w#WpV62 zq`sY5UTd2w9txdVkr<#JW(tHmqqg6=e2w|kDsfu#=@RvO&4jUEjQr4NGyLOud93oz zDyWw3KHI*QZ->0;+t?}wpDomO=9*P&aHjRPtIb)7U9qlO*Hpy~C90q%T(uxpy17xd zw$o%qj?q$|S$b(MH0H`J*k}w+xhRTcVJLh`J5doiy|&8cBERLv-(FKM2f-bNb>US; zH+Q|%zQn6;8%w*_zI!dj{50ZatEX6D9ong4uK_*!HhisS^;j+KkWWR+6q-Gj zbTYpNpT42E!^L?m5!_#b+$tyk?8qbHSWpVh&_8v&mCH@h=QX_J29gFm`ozv9~pKGBzL2|<;sVIidkUtSpW z+@-3ifZ;vybMtcj{QUhs?m?6dAE0(JtT31rFqe9H_-bFK0#&NSZ)dxLG1aXLTH;|L zU{Te5I)YP-V)jGeOAc^MAU#KxBR>sD>dnAu)v+L*EYmGSnt(#3I9ClDN`(k%rJGD6 ziMs(xTc$yUMJY^rH6KSAjG0plPq6T0MiO?Vx#j9^kx;W&{ugEM7+u+)zIn&CRk3YX zTuCanot)UVZ5tKawoy^VwvCFDO7fgv_pGOuCkvi~~#lXg6ErQuFvrLDx!x^$((8B*7Mip@2IfND+iX}`Sk!OT8qF!iQ& zA+|REqcgq+hqL}5$FEC>;vU%hS7QBTZN~zIJ6er4?Y?8bzb70vfSu5cNLa+bgjsrT z+gVI}23|?$$@}@oDtWG%e>iz~gM9cXidf!FlsFmDm|$aY5e@Ay?93^B5rmrsJHjy; zUiTnAYaw?~yqra7gXV>jt5I^1nypmgE<{>CK4P^Y``f*tOe0K+Ve|m#WFUFrpa%{) za~4%2H-*L!OfeZ=4Em&43HdlwI_{JIpAks7;v`}O-2sDGUGf|8y2Kp@axYD5V;m^w zvq9=~sOy$Y2%{k?Bymx7SU4Y<@bzymAiaapBMH^sI1-wH(nR>u4Pd;S~2%d&wY2E-MMKuSWGy2%7_S| z_%`HvsNPZt+YLFQ?!^N^UCH7|?jP2Ej(&Q>Yq@A+E0)9XtlqJkEysr(Jzx^ma%a21 zxn#Heee(U%8uT|7EK$SYdd+iNDpAs$b}ie?k5lTR zk(|05Tyxp1;}8(@)9BvC#>#F8p9|P|z`VMfkZEcY++}=sj2y+O{iP~~e*r8{5_meH z!#w|F2(V?;SKJJ?bwDnBi;Px=b9RdI$=v49zq8_|WIgL;JMr4m|L6bwjVst>C#EsMqZuJCfu5*rv@_}xu$<(ch(QC4`^9Ja7CPm0?+`Me)5XPpby7?Q}U z4S?HOO(Q0cu(ztKuiNsdT|SPt+cO%_tmQ0dRp1qcDm5A1!L6q*iytszRHOWW8|DjwvcM(nt`*?%0P{Z`cbuZTn4VW1(LGyQve=S zBZz(WiepJwOLwJQ2kV~B+F{C9`MYjMy!1O}Idi|M-Hp(^Fr{VWkY|kwyp{@Pz2t(O zm4JB@&iF!nGgJg`r}bl+(xn`oNModrwy}|6p<~F=YW8R3N}vl<9oHQ1w7mC$?pdqpD@Vt}<6(9D%2$gU=_2$FP!5Xm34-Ct=6 zh1TD%hTuwznuG%5#k%{wWLdQU>_JCNrotf!ROKQhP3|a`q~4oB=V;(AqdyWl#EPs> zfqj(sY8u}Vo&;!bufTf?hgm5DMzI)Wx{;;#}|Q!Z`y#_-e`8me>dH7hdD<+6@_DS%F2*_fJV#uAcobFz5)MLUAK4qiwkU# zcPD1MV0*+x%f1O_52S&X=@eP*BPptB_@JqHYb^NlK>^4`aLGIs zWHs|Pi@(JET#HJFdCn~2=AIt)2vc0%zUdF7wbJWe_AZE}gWHIGKW_ejSijwT@P7W1 zKT^nRCuo_0^tfAmEJz91Uc3JN)*|3^=iDM-Gus-zy_Ea1tEcgiAhr%2##k(&qLodC zsQIebKj!cO{m%-+xs1R@4OAF3VEW`gc?%0$XMm$E!1@2qTU4t8y#+Uh&#}6WVhjk4 zC~&2{l#e7y(^4n|g<;2{xk|_o31JA%s$AvATl+F>p7fKUt0JAr+m_Sx`vDxK<6Hhv z$KN!@%jxcTc|A+e)__q)ZNRwVjGaRPud%zcN3V6XUgi9JOZ6HKOUEX7No0wbJz3{= z5xPoM3qO}nfqGgPD?X)M%LzIV&cjVnN}W}*c&(NB6+aAzLJ zTQaaEkdhv-T5)!9mXnN#sOSvNBon;9IULUuX%=LcVus`0a$n@`=%y=iLSZb6M?@yx?YGHNrEoqKV!T%m770`}z7d`m z9kKd7m5ilTuYKr1b2^Wo z%~|v)q)JkWVw+|&qsmFR{*+I92!dfQ#e1K#t_9c9Lcysrgtp-;Mu*oNAX2!>Jxwr| z4DE>TYots9l1&{x1eP&f3dPqr_l-#R-^55}KO6Bq`iwNhcDXnoN@m=Q`JcYHoBmX8 zlL28M<&xAR&JMuu>eHuB=iPbl4)5X?FJcDc)PJ5F)W(webC?VncjKQcobLbYh+cSw zyeat$&Ye+Ume=e7Y}^(29gQKqhl9aLB1nQ^Xdff}2DM>Q@wIrJhIkb7qrwnIJ~-AS zsmbpnz4+(^-0Qx#P-uwAJe}><;48Gk@ST29imRaO<~p2_A-C0yfO0-JqVKX0Bx@2& z(Jw_14!L>!UJ)mZAF^mzP~_fdP~U?kp!XvayYJYdzCv$=4<`p|VobvZCx#2;hY0`b zvvQ#BFLe<@=Vda!{$Y03E>NIYGW)*by>b5V_O3V7%B{~`jJFPrQv;^z6jV8M^K?I$ zc@Gy7)0HS5>)|T>*>I^|pU)UTu3PJIC2K3{pQ2bA-a3z9de*IuQ<p3;x`&&)c^%MzgP)P0btQ(viPt6g@{3 z7{^1MKrP9|eKE*JJK7LELN${Y@c8(nU}bj?sejn8(P5_u`6GM>tqhT|JhbKOMI5_jZ|NSch67HxtZuW0|>YqKOq1$2LhR>!u*{vy^ zZaD8ax`|Fmovphm0uw^VhWG$|QN80d+UHa?#LuF@QPe2qcKTk7=1cjgXQngzrLuuf z6_!_%J*0Bn@ohq$%4?gfZeLc>m9WrjJzqCpB&j&6a=`nq`TNghj1b+LeF6F7W?=sY zZs$R_G~oQtzr;Awc2YzOKpiqh`uD5a{&G5wb}r5UXG0@vz<&vEn$>LV_QX*>uJl<* zc|jB7ie8o^Uv~VzR|UpcJ6SmMk{C19sqcT=aw+)pHeH3d0x@rO9f?MAJ;{GJIU#Ta z7i#pBd-otN%ycz)IW={iXJ_Zz89<|c3q@GSA4Lj@U+n6(ynO!Kf%x}Bvjae$g zXpDkn7?c`VaEOTrEECjd=j~WLlBp%Z0|~t~FJp>t3MF5f)kqXp8FSuk$Xcc%>ohA# ze=)PnD@ODkon~)qR-FWIsFPAimAek&urtT5 zxKGB(xlpm{O;kTM0wq_j%hWXG_=^T?cDSQ+#$4v+BcUlGsPMt;2n4iy5amGFw%_)m zFHoJUK=Sw~`T2H}xR61=dQTwg_u&T}_Vf|O^0D@RI32AE`%C5`PdjruSo6DV?3xHT zZb-QINiM@1?#bk!7>df5R`n6^5x13JBIn)AdfHPiFjIN67+wT3k`CeoFr;hxo**2f zPlT;*fU>k zDiz*0x;{i4ok}w=Zk)0Y#}tAFj*DVd@J@l*)Hk`-cbN zzGTB8nn>C&^DzWZwg@SCJMOnNM8Ud#HET?(C9-D%d6l$|Y1Tv2)88bO?(r9R*$ zBYN=MzON{1ch4@^Oj{w;=8U z@^?{P%xUxwi{j@8J{2-vv0P-ayAAOsxt%#*)kf}K;}~IGk=3{=z)yVe?%+ZRFC-(u zJJzuZg={O*3}*Qf5hJen`=g11;@7p zPmjVfbT(+e=G+NA{L~%zftwpuYg#GVTsRKD#PI?3&$bP@lL34Ph=jOcK>qD6#lPgz z$;Ak0+Ww1F`Xp}y-5Sd1_MLiknHqyqs-sV*ln;zSqzj4g_Z8bG$iGk!JP-w`b@zUZ z9C3{X3XSDv{F?M0ef)v|cq<;NV7-ZV+b!Szyn9prj1!Ov`AO#vs4$!q0?54md2U-{ z^irxO?B=^hGS#ULSrXwSW>?XLV`@F3w4gH>RL(u(Hm567dj5A7 zbl|X(wWIw1v!LSVlfSc|@vVF;tNIBeP&s0odi)rl6)hi)36bQ-yw0PwsRk@y*DlH{ zUjcK!qw&m-YLeram?2+6pjA6xQ-WB!_7qq;@wP5mlCnZ+N|^wBRvo-{kzrJBltH08 zVXI1F09jDhUo7ZJX{n`1%U7%2X%%DOu_SL3h3(!z`lVmjcztk2GKj;@svg}UE@&&m zdAKfuV*7Lyo;-i+qKFM}J_98m)MwbF)LxYK<*DVH3{252w--Db7<-5&+UE~4)kB)?hn1Mn9fX~|4d8aaPCGZKn6=|V$pqS+HnC>-2j z++b%ZwjtAc>GmMSM6Hcv@czgW;uW^sFvM&h3*_JmNdlSe=4e+C> z4xQw2bK$K?#ouv~*rK0;`i9}gkZC_N$2LHu2|T8lgza!;q7elKm#76gpvf}~MTWD~ z4;q+=wGhzVyb#O`*BsOEEU}RahqG6*hvs!W@z>6uy2IMt`tC9;_7QK=EdBVSYDkC*yvC*lYRoTRzr=*3qX9 zMGf#W=m^R@m1W;3&Xlpks{RW5a*Jx(W-e6YgCP9f-%woP6y zkPfxjJ>X1b1uks#wDJYuwTwKS1})m-6R6Mq zj*mMnHx9VX)Q2ii1MXnG*ZuW|{Lj+hI2JNe4U~p-AT7rG540F45C2scbnPm+Q9`d@ zFbh>AWXa{$E;rTAAW7tm83JG=JG&!c(sy(N6Ku2|w)5(LK3T{wRy3F$Ww`0BZ}>B< z$^UFk2{jZ5410U)^%L0mbU}u%xr57PTQ$iDl{$aCy|P_r8Wpj<&c;86N|m{4Q_^AY z!&H%Jn4+=|r^baV+JrjUA1 z_#S!+X(APoZ6+Bka1K{L`w$po}i`A;?6Z4QEvxa>8;4@iz{E$W6Zs1utSE z4TFa*H6!Jri~0|wA_6qap8YpQC<8<#sCAm`L30;FDmcB&&q<_Z*R0IS@sPmEyzl|2 zEIDJL454W}<%qitSlGzL=6w4FJ2L@`CY};ytExpH&u`ciq^lq!+m+Suq?|&xbABKV^M3OK22{yDHwr#;4(jiBW2EBdLmu zRB!0@sS!Ya&&QV~gZD*h-)6@hr#imn^kHaUZzx*`ZE8-PiBbq%V`LUSC-HxAB0Y=0 zIFW()MaU_>7~9PN*K=5?GWJ;SL^?TaVgb&H5w1dDC;NTSVshv&=KgVJDUu)XnWUUC z))QZ&6Nh%N$RCf}?kU-sLuc=&LU3h8$p;bo3F*7Ji$7>@j5KQY0LdsAEZ>|Z2Q`5% zE-?5MhYu(V*gC}28D7E$_?~pxq@l3F6kFuPU$I7Sz~dNo(mmm?SIZ(u2L7y+%=Xa= z&zxtk=ni>3pO>=3#W)K74H6+?m5!dNXQE(trBjXsX%aB#?4x)Fa3l;H03|&*5gh5njej2!$HVSZ+`B($)FwE36b91J!PWs8|6aqW#}-nzROnH zbJ>Dn7@_xL_ABUZ_BOkMD_V7(fZG!kb6t1%jn@l#({LZAoW7i{}|@Ca{*2 zhY2`)FzU~z=(9C(MCg@E&S{#={zRDe{r+tj5W7KpaPDhNX9g}zdUHGlajYWsvR-R$ z)<@CEo9L~PyL$$%KXH~0U(&c*1Qin)R|(Xzhl+I*>7)C)UC48vZ>pyUmHW|hKFURR z>VqIPgk{n?zDJspBmW1H&XsqHzOeRRm{63+|s(oeYmmr$zGeP!b(PyiZ% z-hli+3V^M#ovkgv_^-{^|3wO7fXSjW?!PCCbTNx`)-Ko8&P3r#H1N@qgMrB+AkWbY z;ck$n`*@GdxmlEMj}jVt%wRK{j`jg3o4p{{Ps4j%1tGk;{``2c|6Kk$*)}cZ{vD80 zf+tm-HQL$f(n$c<-s5DOji1iezANRR2(GTc`csWhW1?w^-Yq{qI)Iv(PPyHb(HzvG zJ1F#f2&hr9vfiC1A`L5`Y|y|ZLQD$dl`REfm#0W7BD!=8rhTjhU+r8aD;lykJ+Oxn zcjRf0v4pEG!ZB5?&+5RqP7v59EM31yoH3VZ7CQV*LCwZ#A%NaXXqc;O!U(2Hcznhd zbIGu&qYjmaX;QFpm}6aaSoKt7dQrY;0HU&b1RQ|Gs+J{Lbu1^TP*s5dFY(@TR45+B zl7VMjEncw1nDyR>k#!e2Q)NVFB{ONQ!Z^d9b*Gs(?d(=GjuE9%KP{L_hW?t9t3nxrRP2Dq%~^;#NBORG zTZhR)ffAN6j!V!pX(Fwdi`pE}mOvD3C!hy(mjE*s$I@hzu4&4*J ziZswHP^!x;uc=96WbEl=GNhjCPA@tWMr23=OWdatB4Vp#=!&#EY+F%eu#cK3L>O&% zZg`;J0OeiWcrOrpcEz1JVj#ncK!k~zvWXoBH$HWg+j|QpIQ@Z?QbO2x?`6!iYUD=w zdBy>lE<&$|`YT-oNz*mW5>jSmYNu@jDo5fFt1c2nGC?>g-?Tr@Wh&_(O(@d2%dHu* zOr)3fIyNgvfK2!W`X;FnELGZD`rdJMv{N;mt|LqkvcUAOEKxY2FFq@`|J2X;;~R+3 zLvMU&?ZzmoRW*x%UjvRY;lG?)#m)k2TN`{78ADEV?*!X>6z^siu!*T2B`>7Z&m-^? zExZMG9w&E4iO#TmFQS0^L{Tf9lvwUy+*%e-p@^#En${ZFkmGpV4TW#+AzFz_JAwI$^>Uc|f_9#%@kt9$;hjIuxoccd)p z)4AOR1WfXltJBDpZUCTN(~iF3D-7MqA-L5{)e*qJN06 z@1=-8kDo3^QwVv3c8hZ%TX#r{jStC@RB;p`UO!7i!%6d7D7jm-f4OU_08}QZD0EY@ zO4O;@kU=T;)Xp@rl5-+(Vk0zpq826uHWfbvsvHrP&V_u*iOe#-ikclx;-lnQ>N}g`O6p+<1G&Ctz43=f)_%V!DY}e#P2`5{QqP=+~mQ1tU{L_of zD4NijW-EiH3O7`V`e4I;Hc~|*Upr}%28dg1J~bjGSM1ixtCoPc#bSqAoWe5czH84= zbZ<3;I(?{C?gT8t01JxAKCLRA=uLYZiCgK z7%V9aW5RnE9@RQTZatj&fOGC5RJl))hHqWj|fgAf6FdAd$fH%K2bMrhBOFVr{t zC~=%vSR-L4>S*np+$0gs$drc6%`Rbe*f{7oIMMf;P!ev>DlpEo#~ufSura!$qU@WN z2-h9R43*30nk1F>{r0jH3hw1n-rq3R=W+E6Jf4@I?}eO{6guz=ituZ=X6dA;}h9Ke}pz2cEwiYmLU4Q z;HhgU%O7@o`-)>K)=J&sp4!idkRD~d849MhHHHuIJ3#GVvQ*ROty${=UTXnhqrA!# zF3+w_fb-%bsw*Lc24wdkm2;=dWKcG!l>@DukLH@m9C7`9sbE)%fx&+2GE3ei_ zEEKF4*3Ucp>8`wdMXDj43!|0&#Ru-6N!WHwQ-|+wS68PFofs>zyYjm0w)SqOeiG%3 zyE>WYe#87{_E2u_H<>?(+y#+DICiR~@2Y%sFh$>}wfJ!1RV?Z)^Rm@a!9xM>;C}@! ze{b7)`LPgc7N~P;q7Xa7Y|sJcX5XY0fl$~1p$G;0QUU+^N~qV z%dEhkfc$ka{7*-k0PF#_CIDMw4}EJ3TPr6wEgx&FHK zJ#gAH;9P%@BD8<2rT@5f1!)O}?{ebu|I^ty8a7UAKzco0+m_*LzQ;1)if0zuW7J0z zQy+0iIYsprkJLtby5QBX=;xF76_x~dZUMkgn7_Wco@SP}ZcIxb*M7K(EP`L&Ai#$x z$Ggmr>BFh6=5qFA*6-GDLFH^Q?lNl@l>AxE`lvXmNqsLaK2S%;v#fFPr$T9V>IIEc zZRx0F`<$Uzx7`D!yh*}z!Y?PFk*@g7n%1Vt$gX{wyLEBBMzUSzmA<%HKlKm~_e96A zakB5(*>M>c7TqF4`+A@2*;(V^_R6mJqFUwFT0bY(t+eYTHzzxH@%k*RaHQ0i=Vz5# zeU$mJUs@Frqq)8Xj#(oZi7EZ@ES`mt>oCJa^V%4WMREqFXFpA0ykPvvj~4E_8BF?b z18iP8P^LP$bT+b|Ld5jR7P&B3TE!Oo8NfDu?OHF4{*xed+! z&v!enU68+NVbxU3NVW^fWEN(6GR)-`SgRD!8|CrLD3F+6nd*U#(+iggW^* zV+Y=K8ms_^U>?MnOwv6e%M7!vQVedx7W6Vk$?9u?xAdk#FeuFS3_&Q1Io;_9B@D|L zaKB$M@pZ%aTAg2h`&$`XHi_qqGL$l+-)c^iaPEg}S7UW$H$zOE(d7Jgm9|FZr)ABp zQ0uQunTv~)s>P1*^tNQh&}oenJoIF+&ix^(Wm6yb#CJcUlrLhb_!xT6pz0ECmlGn5 z#%N}-dOmq=rB+?n%nsk{uUz81_-Q_5K^MDkdA$>SZbbF;ay)zhE-?T`> zTCBhkmCb-e8ty21GF3a17k)a($*$V>mXaIbp8h+0GuUt8QS#`ieWGd*jhL~J<8XYA z@LcBWt2?2hxe}w369wB-#_*g5!*Q$;Cxg?F4{OvS8DR6Ra#&x^vWQ99C0Aj8C{Fja_VsrxQEFgyg)SzYEl=^V5B71C;AFL6XHq{v{TQPZ z(PGh^$y#9U5kvv4>q`&Ehb7q}u8UQ#tJn!QVx^|wHKP!-#&$5VhVtZ*hR}$xaG>Jf z;p3Wr3g*;Tp)94*A8Ph@nQJn$l3%v!MY#u3NL6QSP{7sEf|p-%(C#TD9H(@qwnN>_ zOyGm{th)$m@q<`OXXC`{*LBe9K?H&1lcGV$n7W2E4V0NB*sJ!jW_H&DGsu!Pril$E zXnPS)cr&hQl$iba;5_+jm&zDUjSw{Pd&+ngY&DUuav56W5M`^VxUxzpfq=KDFMoWs zG{aD{B^qpOSvX!nzJlBiP9M=kvjJ#zbQ7zgm!P`R(WsiUzJwaXExSi9+6O-GQ|p`9 zEU=K>0-9pGgz0$19qRWNTwmwZMI|A0CUMpAFcQ@+qyc zY@8d*3mR|#wC=bAsUE0sQ3Np-Go_PweGe!54AY zt%D|J;dHtEDKaMl_#C0g%(%iO+h(=1DvhtP+k;@9J;hx8?!hEbZmihS!hdFKV`hN` zVAa0O#1UEc15iAh*gw{udX25qv}w~fIhalt!P^#wYuXBDdfKSChObp(>8yI4rGvJ$ zE5l97Ju(y9>2eK8bv?~!t9DK=+JE<-n^|QGYVXS)6aDks+J=oM8wOZWCr0&e0m(n! z*2?m#N}}JC|CpUy}1kGQ)xpA*bJf0x34^5kU0E z*1==*NKbH=+g~#O!L$I_%axbXemp(SD2ssos#|rJ*YkP*R@>|Sc6$x4p7-(RabB>; zFRD%xQhjSqKK5l;XY%!Qd=Z$8NhZeW(Z}Y1Sj$nMa;mGIuNJlXP&mWM z?^ZO0E3#$xvfJ=yf1j`bWUy)RxcZCJqYDC}7m)=_yEQq5iF7udp?EaDT3-cl*D_ci z)@k?}7(Q)$I2+F2^Em(hUAz79^0K_`_fi<>_vbcG(C_1Me{b7+IC`>=llj{+=;YZI z3To{XvBBqXf4722#zT$jtDJS7CVYO{ZAmq`u!ZDcqQt%ScYniKuef^|JYoJy_sW!O z#4qir(M!M20}%*bmiSqLp2p)!Jy{u<^jXr=W_L0uM+T?3)-QlkVdVEl8|z4a$Uqex z&1iVBli52yk7b9XhIm-;euaWme9CI?Lpc+5aUr82)qCJC4xgQ5J$k@U>HD}V-V!oO z!(>dnzA2?leol;Lar`6?wkxoh7@*;rd^tq&iMa0&M)xlmEIJ_N+0n!6H8W&M);psgD&2SF6G`YqVJ$QH?WGQp<$BdpQ_oj@cF!}r`$;d1|fs3$oC_N z)gH$Tz{Q;oJHZI77dVdZF<^EH(W`F0q=cK;Pt5%mgFi6cjL1OUHJgz;O|V!|laV!?CsULdNR)Hj z9ZocOb96s>$y$F@+TWv=-_Ut2#Tv^%bMF>q9qzwDA@k8wYA7e!`%msQn8}&&b%T%& zLgRF#4b|+JBWs{RPS;!l2FnS)1oIV)kf5?Cw;UkG<_<5Jk2T>T#BRm+EUUN-OYa>) zu5adq$Fgj#*pTV1-p5+;Vcp4?fA#5I;bXN8M#ZEO323$Y<6cmZR#~?U3HBE2-E{{4 z9C5lEv8L^>O^0lr1qk)-xa9611t{&NAg>EI4(x{jbJvU^L53DEHbWYy+FwC>)WqPJ zYl<@XHk6i_9x(PVDRv-8w5-Un!a3f5f5M}Q_O*IbkJMa=W`MsaDAio4dQ(Goy1ahp zBdtW;uN-|jHeJnq*_q@An zo^_zb#W@#S(TEU|o0DrUwJ9TcqE-_{cJCIMnso$Gw@;Qg?+lNN8Q({LuSRs9jKHACdLLscWT4Z1*aVc{QlrM-Sl5E> zHt^*B%h|7(-}(uNpa_%pG^iH~29R4IM(8kgz)l1yG0yE{#8Yq~nG6v1kQ#J6aq)J# z1ESCtALw|-X)D+sc*j91O?cbv^4Y~c&^X~TBSI^5U13dLc;@t^@vB^re*K3k6o8Ek z=Rz3?*B`kn`X2WPd9++c1MpKCRlZ+ReLL7h+fawrC*G2VU&iww_XKh(Dt!b>esd#& z6Ird+b!TS>?sp99`H6J~LEw*mlhD(`IJPvgLp)^(*{`!2lgzHjIf#K1!=Wk725A|rzc3z zj+$^oxKGqu{dF$*nE_u)-rc41$f!z#H-&u8V|X>bKIl2no&YP$+XZcnY&O1_xQ=A{ zYpYI~NvuI@@SVMz@ycL4X2!O7k*Pi#f!C*+qQ$ht&J3V8+xk87T&(lVr*tD0u4`RU z_lYsXSJ?s`H+xeLa@@6ou^`vn!h(;LKmD+kLkm2BEbmf9n~&}WhV5a86C!7P8B8pk zL|_?$yikmxMAVYz?!mSUu!9iB6a9=;z)re_N~69uBY$Py45fKv0YSjG`}ZkxJIm zN74#gFkq>}sKFc3#8A%W!N zuj}7b7XQO#IosLmTLWAH*8k0C8QD48{cqq!3kY6dwEL-!fOKaXnD6?Rl`o>^hPGw^ zYdbSWN2kBD9Qe+E%W`~@`Xkqc5JIng;l6O0+9WDaw;j}NrY1_$Y+hEQP#RNUolh2# z$*ej$vt5PHphh7t8?nV2?MDwAtO?$0O{qZXdf?_oMPZ z^)hH;iPzc1@#a6*6=I%rc3ue#I609RsuHW#5`mt(a`Ue0NsGeU5vEJjjm|ux-vWO_ z2m4TR`uRmdCsbOZC1{M6mT;&GArqE?3(Bl%@&z73l{DlTowl68MFET5hE1lqB7d@< zc350~AqshnbB3@r(*`4KH&z7-P}+E9-qEd150&Gb`%Yfh+Y{RU3R7BE5>t)tQ-a4= zT2%h@qeZP^OU$hE)z4nuj0t^b`2c?+JBec#suSLL9N{&K?*!~y>8~|?;oc;-Cn2c` z1zskK>}~;LE!=j*^k74@9t&B-qr2XT#_45IH<0mz@33A3dQR-6gfqDh@{mYOe&vlL z#RWleH91S1y;Z1+>IKBlL5%ZTQNOh03x`UD4U1}`w*LVJ|8+zV%0Gia6FXz4|A+BF z)`hmB{lKU16aN2r0{?cBjiD8Q(a7+Bd;*Js^{T+NgdYU&kwKuKm+>n&b@HeDRW@kk zLHPI)VGp7Ssi}-uRuL+a9S*POZr%wL9;{DT#PuRy-JUjXuLwUqUp_pM?kX$E+dKb& zTmw{D5(debPzj3z-#T`TpxY-~FyRIf(n!Cv?5-FXC)GPQqdH|41KmFYKXQ5Vz%Et|9);qq0=&zircI6=Wc^GSaN9s6pdo3Fnr ze*as944L)mlk_kr46-18$&xvt?+e&3$cigidqgGTu~7p=h4c!DCsCFhaGSE5s;0K{ z!?27_uxG+^3_j6~!k`ZCJx~xn2DVLN@_2K}G-T)Gn3-GgU*OA5qOlSc;KT zGhlVZ^A33l&kaMVZmxUTwxo@@O>LvAsa5(tZq1$2k}ey-z2LtHXJhSEt&;M=4Ax8w z*Fwu8FQ)@tIEL)SS7?h_6K3-Vc1mx&x>fxW!S{8Sg9l1galh{XmU1mHF^OQ}XdCI# zoq8RcQE5#Rg(kb&dx8kT&<%FwDW7zAKS!6)xjFD0?2d&#YKEGl@oB_-j5-1wdw+jp z00nX0ijsbSS?Gi6=MKfdy!;{a6mvalDj_P_3skW8)-SjijZm%5rL(Ib7~xKm(9`x! zR1&l8PcF+#q(~I2mbqIbXYE6~j=ILH1-V79py*-^a;5bfpSQWl||vN+=q|10oZE<0dV>VrsH85j`fW5nZY& z+xrk2C6Oa}T-V^(y;*#Fuy+VNR0>?5P7Rn?3%zUuXgO6DK~1lLsHd)>Zl9C#>A940 z4t{$Q!6Rb zRiDs+#aGDNj*bl)Fm0&#zIcn6tYn+r!Fmf;maNh-E8S9HEo3X(&9TrT7XUpd!*6Qs zN?Agxmljthsqi~J(tS*L>NGft!{?i|i{Q+Ye$YdYot#h2WH5_LS!*#*SN_hSyVUXH zo%}!e!e1|?JdUFeXkccQ6XD-Wf-F!HOf9Sde@g=VhnEsi66Dtk-2N$p0udEq&j-1%rRp{UTF`W$9^?0`Hnmz;tZ`egO&)VS9nwucLw&s@C- z9x%aB0zoQ`*|X=%nl+^R3eG^6Vt_F`5p`Z!TN-buCz41O!p`;$iAMbjxLPbW@m5#* zb71!>`prR%Q?f%n771Vmvx z)r&e9LNc=`-S*_7gv93B|DG&>SBX0q%{s3UowS}}_1j@CPTY=AQx_;QSYcx# zA_gB2aM0a_)ZWEr@YSIErwf&`Pfo7R@c=9t>#5beJZ=+5l}!2IP?H1rlOZU z^^2r{%rsX3L%`~uX$lcL`xB$;|2AX(*zLL>pnPqE$raa~{dRU|twND)57d!iJO=PA zYd%uXLdX(taB6Gp(#G!??Y=+2ax1}(Ea$CSw#&L`!tPsa>7Q(&?wV^C1}$n0&AwVh z!DQo~U0+D%6TsB4$rx(@NXjW1X8{YuVz1??>F%MdL+^~AHawEU59idVQO~{XYg?`4XSsg$raHYJb++_p zW-zOS_L`k_XP>E-rEHo5;9MCLJOOsIbb<|(4yhdAgSUhg+*aEi`^OVM*bMKZJN8%v z&&LiJ6FlVZ%XC|*6<9Ktk_^qR8ca6u<#*F5;il-59DLT062WmA*ol1`QpI_-t8b(N zvwT{hbjYh7ave)JTF)o?x;X9vI&0i#!5}wduKLs8@&)i5p=iTan2k|scbi2 zWFlbcazZb=WVj?;^_}DH6Y|<7+4&w5pq#n%IlDiUr%j~P))7jIEY$F$1~PG-d-Q}0 zub^k@up@E(&V1NDN7CG=}47__~?+mrcPZ_?c% zRHSHiubEmX&ww>cYET`s&gTv*M-gG8rgj^TH=PddUcms~p05K>dguzCJ4RMqevNM6 zx}U4%S&n>g^t>;XmrG2mJoCYr+0EuEojD#nBk>U(L(kJ^H`eAkzPQ zpuRJ}#@-q@S>XT07`gCSJh%Y8zd!K$m)z^$jM4wP+5hcqYeNrUhYzEJKSh#= ze|G+t)lb(q_4U1fDY5H=o_sa}5@cE&5D@DB{Jj~vINLcH8(IVOoq>PXe+ffUH2mV$ z+fd*2^8&x2x;NH^$Yw(I7a;9e&+qu#T+PVZn+GlYpcpqLi{MSv8hyF%SM# zqM%(p(UPWnT)R%3sI|+LqaSa<2(f@)^hh&XHGiQLn8ZT<7&GI(ecKe3fH0zwnb*8j zp^<@ft5$M1S+qyk!S`07ulGWb_eyoPJtYYOyxYt=}PNJ#oswZ zw!Vhq^9Vmyd$g%1+oA2gtTvHnvb8f(Xyq~qnZy1v z$Dm_3(4bb1*1`p?)i=7YkgWopRlO$ft#5$3ix93NZBBMA!sQ76EX{?jVfGb-p*YQn z^GIbDyf$bBZ7#O)UJcQr@$^-&*&h4i}*6U%et_px7>X5 zI|)rUCN@KvERI9T5i7P?$O7J^* z+ey99Tbh{)=G^{mWY|Xhn#%aS-Y-fjOYFO5s{B)jV_~vpRc4`Wh0sZ5=J-5hsmH!O z>F=^vx(E>!_lMSu18m9l-!N|)M~N%(;#>SL^%lc;8VNBGpV`D(gTNu@-w(uGIYd}u zL*QAdw(vlj*V4D5{2a`}bviIO&MB{kyGdS8<4BykKMdEDczH zgvlEUJm=gyu#JfE(j%XcwCh9LnPuxwQx>OH`qoQ?G2|XJBp4`}Airg@X)04>f|obW zx;M*iQ77yX84?f+d4L64Wl}u9-N1PXLNVe{zSp_Z`1Uo^dlmA#?VSHwyA1=aWB%bw z{GJ{Sg%X&*@HJaDbfgC64`2D1ccs6HZtWnGO4z6P7q-(Ks``XsD)QD)3Zvdx3O%Ab ziF+tGA}N?Esi`_}Cb@C|$8IRjtY%;YyJ(VK0~rC%S?gsM;?yt8asM9{$_mXXBte;$ zVWp$m^!*3KkLI^5Ctw_MX^SS0k)JagF_p?I^ScC4Y*otmKSZ3RrNGgN@!qILWaqi| z9pcx3g~h=gR6tIu!b&Pggd3snH$A2f2`WD;49RU5RAaIzF=$Ni=SQQn z604;Njo!~pLQU3S$}4lOq5bo9bsYX@A9TMso5UzSOzs3V^6HgX1PO34z23qF{ZQK%NhV;X^Xfx|jybST(2u`c2) z?%9oIS=^o0$_P%zm2`Kb7i&2lB!2$!9|K*{ENJL)@|>=j;?ddPHILn|$EE0C)cl?&kwanCj3+SS5Vf!N5L z+?|`qnlGC=TMyo0rLS7fi`EV|j!W`YRgg9Vsm|T_%Q70Teh>X23b`hI*K%d;;*X+g~;C@)08y=o1{nZ|Zo}swr ziNSZog@Jo#Xil1VLZGtzd8Nsy8qAy%+{Eilx;B5Mcj!~W zJf&Lq7@SiHqNB*LUN z6CmVYBHV7rdb!t4sa?95^Qp1>dzP!smWKb=xZKc&_9Bo-m-Fl19Oir;kxZ+%uPCT} z^|_~eR-?D8OC1{xWJ41?urQuh(B96*ZZ$&3WgGDydO5gYFg@ny{nL5-IutB}W$?zdn7?tG81K#7>6K;GwDmgA&w+P|X z$8FZ%{M@3wS3{dWI>}8H6Q+6d4>u*7yTP`56Mvi@x?m>NgfSy8=D+Rd=7(sx$@V1Y z23I;y*yb_thZhr_VdY@3S9Cw!%K-&7CP>TzZLX`h(65Sa5Z4sz?%C$j8#C!;OPfxj z#h@WILNy5RhZ-pdE1&s0xa`JDevX(tFL~?nkI;p<;_W_yPfc0IIWZ(ETvaVFV(c6! zQO`B`i?vmyu1%`z7m@WkFifi!X7_Cd@>4C#C=FVjF9*IahmVh!pS~}DH>$^;uGFY! zb&~8ZuWbF=-Z+Rg0NLoL7l{2yC~cEUP&0Du^XzTL&%%Jq``{Bnp8h{~7y(k5jhz^U zuIJvc_F0B)3q%g|JK>VT`+-IE269R9!i@psK}uH~=5y|>2M-g(OtteFO?L3ZE(To0 zd3G)wBTgi6+L>?J(#qJ6mL(Yh%H@OeV$k+|gJcV!b`O(~q8}nY@9V-12&+MP{OTo7 z^BQlIBYxJA;w1U~t8B7W3fhBS)g_FQV|uoM92NtYA_FVp^^lwZJT} z$82ypds`97+AvS}T(|(nYnzZnsWPb~`q#htjSGfJ9bvE)zeN&$CBb>p=siMgq{}Fg zaA=fC#J;X0oZah9qRh^!uBzVeAT|Morgijx2y%01t)HbIK~$^*665`C8IFHJ86j0_ z1$R)L1;~eWBY{*C{%VpiC(`RWCf3l&D zLi|kq@sS$lbx+lH%qJY>=^s-+dYeYj#xrHl$q`~IX2(f)QOxWoxV4!ZnJ?$E8mZvw z)ld5{bd#boNhe+u-yvv^60bCYo6ikY#tBY_RVHUlbmH=kd4cUsm9mT*pMX~ktvwRv z?awkS0eBE7ys6DN6q{iDa0NNZOvda%C-){URcYXxPH1^o1$XYnxLvTbcJ2i zz$71zeMawb?R=8fNk0F$MaEAqmMX2Dh3n^lXa22f%hoFsXg5K0sSnX_a|QLoxLgA@ zzh;K8Mmdb4Ndtx4xaQ95M@F|f0V6%4df!!Z+EDjUg81PKs}8Bpjf(MxOk&J&7(zPBIZ6n0vkcB_JuR%kN3iKL53tO>@=tExz!Xrvi^+O#0&ga7&-T)dCX`W z-6o!}Ba2hy_)KEy4^$%aF#qAr{OCsfrBpx4*Xrn8>jc;lmFFH}TT97xzbe=wG}P*l z388W=67s%|ta%iA;NwkDraCwWzTs@p5TWvOab$+#Q^M5!vUCP1>+(Uz*ez01ShNUa zAEC?znM8WQ)*^jdeOxG_BeBZ#mN(ODV&@L~bH-lY9LTNU>A%81 z0B(QV9t^UZe(%KQ)S+Qvdk&9+&h(c9{rYH$s?y@>AB}tZ_+Y)~B9__YX8EasD0y z{@Xp6C_WQOK#Q(n2)1CrpDaw7_2hAe+|R~c_9uFoGmxnmxLs6NN`o~^j zHF8T4S7-6q^A?Q?hnGj%urf<^8E-#54ba66&mt6^o?cg1X4YV&UP1qPrV80CD_1`- zP)XdkM`$j0FN3l;f9-&yQ0>e3pl!&23$%#0p~&bL(YDPOiSLNn11= zt(;zTQ{C_IlmVZ*(92mJGJ`2vHX#iPMzm3&AR6Id9G#ZC$!OA*Y?C4L@f1O655C7Y zwDqet;5QiZO#O|>gqp5ZN*4H}R$tG#)==7-V72o6i@VWjPZB2>{F0-l6l+B#eKa;f zum3b``d?Ev|7=^%<|a1(zjP4g|IiZu?`JzX8#r71Z_+fHq$Ig*zPa(s@2<=Izxj%P z{2)3bCnvpstE6?RkJ#?9VRXGwhGP7&(2oW8 zAIs+@X<0xrn%&g%E`*ctI_#(C-L5g-1L8fR9N+mnK8KP{Ig+!6#@`+!CJ{7y7c{xF zyEGq@cy z0fhl(Gm<;OJx)MOH?Gldg6Cg7usbnhu6r=!QrBK$hQ;3oy0G0nq|cz}kfyt1!h!@k zzm<9_k6RXn^|2M;E?;N z9ygS3UTFl>#Li5J{$P+oOyzHj`81+X%q5riDG)sRFlR%C!kGJ!jq#YaIS))ZWKR6J2rgLr3j||`P z8qTE=K)H`0n?d@{ni&6kZGgwyTcdp~D$-6c(Rf*kaiyg1OLti#kMpTeBB4`;(2(2! zJ1@bxVh)NK)_83JuoTLnaxnLyd~Lr3t_fNXLm8j#1)4+QTvjeEiMZKoh{Q8H*Z9Y@ z+chIxXAj`g-~Hgz=17&pw)y}j*;!<<^o3VcSbHWF*GpX?=F;d@E?Z}WqEkf}aPOW$ z)~Rp5rq59nXjJc}bR~p=efE z|Ei9UtK2z!ddn4FO!BN=?5{YgD^_H7255?Ab_g_nsiiI7W5S_cjr@C~WLA#UR$YHV zrM$LjtVSjIxqpQ7Mf2y7exow^oI}xO{HZCV0!c3l_NzMtf%bcyYj1TSA7dvKH5Qqy zBeND(J>ENEA}P15eIAY4XXgY)Pw&So(Yr14AjgJwec;mviIPuk)RRLC1l11c$A(CV zf~Y}3P=2FPX1y>yQO;t{ZveabT^Y<~y24LPYNtx{xV+<}ZJp+>o--gj$ZK%_Bxayz zP=D?pKS2GNe*~KPLsD`ND3sEB&fF1H?M_Q(I*f8YdS*KIsH;7AspUW#Mj6p(!x8_c z6Y5mwt~!eMhFZOXw1AO1ZI=Z!wSwe>j)kHq&u`lrIP_71^EZm($1$M`dy|xhbX3W< zsH{sJOekA65K+666_;2<)#gt$Vy&RrM-tMm9IVt+mIuG>pDAGbhzcY;zphsl$?{g3 z%O3oQkP#UmO{$QsAJ?RfW;R zvLF-Xe3-oOcw@%pJwCKkSq_!S5v(e+yNbLy5p??W&wE3qGtXvOpUmz5>^Q2FYRq*n6(cdY+4jyya3V>6 ztE<w1?-bi<=->YZ)|wPcba_?QiRRdL3`kNbe3qgUH)RVUF^vk=92vS zA133r7JYdxO*sQHO0gP8O_Git@SjtW7NZmpRz$qv)v$gku~LFGEjRd|#fjoHZpCuf z^6XF(2)n7p0u-j&+=|D`9s+N5{J;>BVf;V9v8+jn8Mey_VEQN|jUtkxEt(6s_p9m^ ze^KEwy{I%b>GOla%kt9XBtIjFSLwEZv`YU~6@jtwHMu95voIyaYLz9;at(d`iagf| zCz&`YFV$f6E-#|E9mcWi^mIuz6+BNCN|AC135FP%AR((!49}u;yis@o?M)hdfu-*o zgM$rVur`f>wzat)v+3CRp<=02i&k!0?cYs3EM!o7>u86se_ucTP;W22&hG5IZQUei zqj9NZk#5@Lv`xKW`o!=>D-BXZtdM;~*9W0c5T8GQqpr`+dV}8}2%gsT^Ex-5LHpcD zYy!MqcAJz`;EK*%J=sJWkD)TLC|QlB&tTcrq3Tx)b;{!-_tk*8==ltTFVxLfNfSDjSH)0irKo zBxgm21`J)NMC%u9A3Vu1@x_?c%*qw%1&=sO)*rTPo!_G7 zT#7d}&&Ep8cEZ94{AB4YXa~n03HF`jS45Rhx>tv0_5SP7uNvKENApiX3J8eiKYFD9 z#5r{TAOQZ|+l-(ak(iK;Iv5=vlb{`?QLIswkfNECIi#X18<(V=l#m63IY`+*BSl9C z{g<+F)t&lO|6MjDz9HVf!CXexb}q)Ij&`=r{|O5L_+BW&;za!H>Nb@70FtN?7m7ww ziKVKVc@R#Q8Y$yaMRmW>v~RZ}?hbVPwsqPBOAc(uQ+xuY-%K;(VV>e~P4CM@UyWEd zKRBFhcZipHlxKpuvh8a>w7ji}M? zB4)ZN2{@m0kD%Mt=+^z-dNsx0-0WJT{lQ%|<`MuOLF?WK@i|9iU8oIg_LC`!gFTeU z=pEaHg5XY?0Yjg{=uZvHFiu|0!zw`v4G`%FO=U1I698Hq$QrK+MR=3w*flw&Px7{^V+kB89U+hj=bZXW%-TY;S+gqlDC4$!t3F!>13(` z%Mg*mNZfS(!2zB-#x*Ohz1oJZyPYT&&n=d2O6i*3z;kUQD}42-GeqONFOyGQG-)ez z0%m#^*5y!WsnVhYap9+enz6U7Ac>2S634ZZPT7axIzrzk%sG}lANr+XAy};D3(^<)FD|5s>0d_tdfpyY1$~3X z_+{otVJk1Q=%K^vGPHkA=fXO~_L|}%#Zh<~m#B6*snk858wniDuMT*W1i{e~v#SY& z#oYq3R-3bnmSLYkw@6;3y^ttG3rt*mvfrgRcHS(46*G4Uj@l^h$s|zAwuz}QO#mXz zvwhhVf!?h__Iow<(A| zhBPn-j>;-11vA;+GMH;9rC9nc4^}`Uyw@g4LBTfTI(}d-1vCg*xqbdK!tnori2?jz z-tO;;z=;2E$;p3K1t${&Mbg9;H8#)S{bBsz^v__Xz)Tne`n(*r>}NpMTuvPa1c(&Ev~Va+isobSXt0M7apF z)XHf@p7Uf>0_V5JQ7~3zFU7uY(KRn;mfVf$?zF&Rp_U(;Gm)cHnRJkw!6v2RT@*1X zHKsVL6<->_3C;iUz#}b6@5r~?6WFK&hqJE5(Y1i97qQkOp&}OUI40d`K}9R2={S?v zv&E&TS~re4=F}z8GWD|B>iYTKy1e{re%EbdKd35)+|_4t8C(BM#c2YJf38shj(h!s zad4;PR1I5v^ckUz8D`-YIUBc=Z!)IqL)tfSiiuzCxH#ZvX69Z!%{oyWH)pOUDO8{b z2PwS*a_codxhADm7OuOi-)@O&5j#bE_eWYp9m}j1OBn^UCx};y{-+jO^f{8ZRZ*)d z!=ILhHl+cyevh9ojm|Lg-MNIMHVWfY<6zJ_KlI96chx)0phM+`{!AKcL7iTYkHpY{ znkC&5>rrYZbr4kFz!D9zqs{TyOR)QzLSvpUp9>WZ9$e62ZqtQ>WMpIVU&b0Ezms*+fn}$ z?ge_CEE#oY&<S5{#M^cAAYXdKDa*PE5 z$Hs%Ns~)~X*rdKwoHoTP+j2=r#2AtQ-GcJ5I(%$*XXdzoZ&C8?7>vq8&0-RMwuW$1 z@yCr_LR0Pmp=wb`tj^bfUym4Au@DhV4zvZ?=5IIGXrX^6rRsATjw21`azfJ0=mxNF z4Pljq;WFO}lLKHDKe(0I1PT~gQAU}*@4^NkrdwPgSpi{jTNuh!;$`k%_iFVLzNenAJ<7)`)bWZbK2dK_q_Raag z15Nv@PzayDqGaWt#K!CB81RNWtE=M#D5RJJ8=nhjogLUINIS)FZTURmLe=f1qj>}x zE>7h)^S4Fn)^UFA&6RKvHZX~`O|qahV$VMk{h`cF1!h>tSF|~DDpK(m4HcP27cdyy ztqg5|ZrgEe)@c3nBD|nyYeAJ-iaHuwG85rMVi*QvlRtLZM*ml9Q_r*)d&SFIVw<$x zuh{%D!(~VacOv^PnoRz~FCMvjqjm>Ns6Zp)P)liEkU~AH-!j}h&I!JEwvm(#oTD6T z9KAgEdfZ#?j-;ey9)fN5C}qGmHXV1Tzlzln2dt#t4;ai-N~P&47TSU$6nQbEDJe(` z_!=4_xAQy`hXAp|13$LVK_(im)=eCj{MV7qs>v0>5P{OUc7Jt!sy_GQ6BtZC&9p$k zt-GtUH^Gfzl2`q0;A(H+`{P`NfmmWGr4^|8yT+%NtrU`6!p&7ehZXif!4 zne{GldEUzcG*1s5iU!gG%Oh=-oI8o8UNxH6?<@K(oZ={&aM|HyrQZXcZ`iG2hOnr9 z=U-RUX>AV==iQ}gpvZ~nqr@N}&ZNsm@CS&~f;#NEW?>ur#@tD?Xk%!U|`|7 z((k0pFs~5HYe3Yn3f5t{xPGs_tn)o7R7hENFN_|3{md#x{NmJ zbY~jyL;(FrjRV%l+5%ES=C`)0l*%cEtZ>6z%?ZY_sC#uc-6G%7E7Sdec^8Od!{#X4 zraZ>pWMTo#HpNstr@Wr{W`clJacb+wT$7+O?Aie|6zBYn3jE}Ny*6~|Au>Z)%HYM5loTF+Eu9iJSG2 zHOM~r6Clr|V2D=pKJ%_qe|O;3dXg$*(_QE9IR>t{Td*{J`6|IjOA0JeSiu);j82@O z%MDCp%65F338Mq{SRYZ@p)ozx-_O?;ImrHUO7&!XzGX#w#yqC{VVf@f&0D6}Io=WE z(Ck}~nK0eWC!}?HCIBrBS!lD=q@DY2hy(RL#4sy9@r5fY*C{?3ynmAcdCGw=1~>Sb z$F97*@_-6Hl{d>X@BAns7<*}W>;A4x)2Fz{0Yn_a!F7TF;Ws#5ZE)0H)JZ&2AdD09 z>T4kl`(_9TV0!YQodD{7?F;?6&c8#iOxk}le>dP&`E2gfS=XePJ% z*fjtoRZ_jE6m|Q2l+M$0xinB@Kg1Y7$P+ zuO9sLl1(^DF2Db#wA4KX8ea3Lv3bL%p+7>96SK8cDA)=?q?H7nnUl}$a#Di~zpNnq zlz!z77B~@Pxa6eCewd4b&2|tv)g^Vqd%;G!m0ML~SJMre6Qp^sp>2hNaI*MRQG1jnrk6S-Uq*kCTlNK z?O&+c)LoB@i4ham7O!)JILHc?OJi_^Q#TJs7VM4K4(W_uE5FLAl>C;k$!@DOiFmwO zRjsY<7vXJa5ezUi(A9GLqG?+|l?>kS2Z;7^8TaS8_{Fn&lOEmZRL)KHZiqJ^Q}*af z$l^2x{BY{^9G7i3zd^$hOt*`DM#R!O31A|T(wql9^tadEe)baRU3jxHTgZ<|&!g@&^qEtwcBaw>~IO)Fqb?@jofTd2t!8%JJNG8&f zLo%B72cz!--}I838GR6)Hvv3kR}<@){q#-bN_H2YMP!dM@l#VA!LU|F9MUfYh*)3K z2%#duy8BO%6gu%C$d^o1JqtlsH=0OY?jM~q`pDD2w_jMLRT zOi13+R1{$2{szDk)sMQ#nV>l~@E-<3y?1!ri5+(|#KtY_xBpeiydEKmlG|{_-13@< z>%}ZlM%Z;WL_Au1-3Cr~w|DB@iuO}Bu-VwC#29j__4%xYj?q15Wo+L}w9Lep@s7sm zc2D(HYuK;1u6$b9p|75wsXpZs) z>NN_oR)ni-7m>yoULI0{&ioSw; zfs0OA)-!!-AG~~WEQH8N;2bM^oYgHx6H(}v*p=#ZsoglpifrCsn6Sk6NatW z=kmq(p>_GH%Z2n9V^N{Z73)pnYN7e=fu>Up;uRsk{LDb0>dA7p0K~K@uzpyAg+|E= zR?f@Q&TfEvapr2-%unp3Y z@$xq;pT!jUg|NYrj{@!nkHPInbmph==#N%$s*e$1vy=L8*&f3Dl;CojvlEAfNa2~5 zsV)xDLq&lI8q;HBtyyFzc5IYkZBK!qx}<4Xjs-sd?B5_X~Jbm^K!GHN?cIT2ZQtAtEM=D zQ3q?AEDNVAB|!;bn^*p_Y=-i4$!2c4EPPHYqW1xcSa3VIkyur#h>jfTY?Xu=vp1UxeMifxS=)<VGrYg-|dSVmh4YmxiE$%dl8yNDd zU~?KWiY_f`pyO`Bx`TbebnX$M+F5*f))g z_urG+|KPCy2diHlrC!T*2E@)Y>Og4)rH)M-IRIgZP%>MHZC=2yJ|dKh$|6H)AuM{f zqOZ-ki$YuZrmYTkyThohn~68LrB7oWX zPZ}kLML5*Fq|=$44m47|f=%t)N(A1#>d? zm*6;t7e33_GX9(k0Tac6z~3%ZaRr5vOr{%9;Jrfk{DQ=Uox@&`#XcAXHsg^Sm##UZ zYa1atBi)+w!G4M}l%f}OEW9?-m6a8Rzknzyp*J4MV=;h>S)yqoWyPsU((J7X5o#(Y zVO;?tpc{dQx_3)$w@v1Xs6&|YrPHUXAj|}ZbpdR{=t(_g-9C448t@jwg!gkinqwT@ zy0;W&ra^OWI8fbs!}D9*ibz>hU&VrLa;>_JyT!)hO>Ka4707#-xEXo_MB&bjc(3xl z(82c6AyaBg-)neZ=3s|PCqKMAFsvLEKGkhHHl$|Y4zw6~h!>-;`|IcHk654XOa85h z3`rCs%xM}+9W8d2Z>2d8gW z!hdt2@_mWp@(=OxdyC`S2l{`wQ1MZfwOivt>^M_DpFr1cYlplBK@o-N02zq5LL&vx z@JSa6RPADjetDTmu0_o?fPf6|`i@VIGnq)fO*W79%SCEjHHqF_e=75^`TmR#IN3Xk zCf)~}>j z^AO>!rJIcYFeI`G74X6`WwUo8B>WWI)>!Vj%f0|3w|241BuSuTGch&XTUTlZY z4;X2I_cOco$ASDYzDpE3z(KS)w0=jCNUBO20d?%5}OAiXO)p zvVJHN25>NLld<~C;7f5bVplsKcr?cHmiznEP>9o6Tq$=MoLgGcaGHJw@B^F~m=82P zV5n73tV1_~C;#2`c7sjbqk7o32FmU)0Qd+Udv-*q1C$$HtfYYR8FMvy5!a;v=~%Iu z^;h#3P?*89kYax!dq`?BAH%&&M2_pQM`D_FT~IP(f}TFoR>fiXS9Spi4zQV00lTc7 zI2~JewlkuJIK%x2iw3%Rg`y^iN$XBV)-6bd7VcC5|5w)-SUzl4OBhZzy{nvtA&&je z9wWr1t(S8U>5YWNQ=5be9GjGL#Ugx(Hav*zj%!~wCZ5W2GvxBQ6O{+wA3+OHAD9Aa zEzIEfqWcb~)=onnUwg9z+=g`z6Se_40AIppnW)LsB#L+shoAQc z-f}oqlB`}%fOYftsP;zRK2};N<$#Q$?@>88dsN)o%%}5y4wI?z1#4RdYyFy<5rCg- z!nP^We9E~Y`^QgIc~9J0qv`nJuzf-3J_!7>Y5=I9s=M3pOT5UdZX~5B{gV8@>YAF4 z$znj?dIA5pUO@iew*X@gTLT+AW5fSMELhO|-eeX-^F`17Vu;|GhB98Qb6W!_V8+s4 zV96%*s4YVN7Ql)lcEAe%I|&P@5&iU<;{4lALhf3RK>eduHk)(lYF(^rs*h*K3YE^3 zt-ipwdJCI>ckW6S$u&y@&_`3FL^@8@q@d;ztb?>S=0t`}eIF08Tify0X+Z)9I*^#m zV=9emEQS$mNF?ZJc}U^m=ys;+XvSxSd-c@9xPmpg7OI_G5h?oKrT^E6);+ae!6shg zMjBk;ZX#4^TYF0ACwoFV56%cx9Y9Uae`mk>YN9&0)a|$M4Y{ zQt}%kzJsHQ`_U<7UJ^*mjtEo4m@0yR65hyx-jL11Y2!-JlOc3LT&6RsLGb`H2NfgV zNKB(R8IfzeLWs0(SRyb*>W$YKt1xMLyw*=+B2U?gL!4C#?vDoM&!dk`ElWVd*rIK} z|E3v#!0I#8gfq060{a@x?;b^W#`(T2CJJz%Ody7m83{AgpzR_SmBwNW?>T_F#Eq0q zW*YX6(w`+y7U+0s=0r&jumPk;FEZ!o1&GOh0!Wqfr~VS5Lee<+?rDxyF zUL`N>Xgz0tB2vGZTU=m4k-K^BYeZ_zLrf7 za-I~A_MkB(f50MHXn#6FX|y1BPv7h*!6k1{AT;U=OeV4evcZTU5t;@7d>$!Iq$o&# zA|c6q7KIo6GfduM+=Qrx>cNW|sW`2|2xBWa?@rp56Qxa|{|>VhL&1S$4>HCLQuFv1 zPCMegd2M3YS$WWDAgTiTU5g5q%K;o%9Jm;eEL@I+47NH0CPo((2s1TD4DNb(YHLW= zV=z)#Z2+&~c>s1j>&NDW1*$=LEsrn_b8h!VeB3BpFj)c3s{EpXdIB%~nFSG_;9vCA z`tKMad;&xv;S49ZpDz!MPgGGn9Y_(Z)>0qH(|Z52+}hngvl(RdsrM0gKdYI z?m%>YeEOPLeR_0pglj;cRipEJHc4R%{?Z1H$d_U?;XAX_HD}ps#PsvTr9I+=O`J*4 z7>)u(#Kw|1AH!8f`RkvOpu(_xo=<*Kd9wUdMiJw^MQ>_f>g^zj_7@1M0NpJM~0-rA6JfkvhK+ z8WeL|9YKd|iHI^ahBrh_2dL$^c&F2wr)2IDCt2PsUS2x(8r=STJX?5PoLtAKlz)^~ z6g1b9T{_&r-M7PJ-wLe0)4H-nnzJj2c!X@&W!%HGA;wCbJvEw%yHI~29&zS+p{wA4 z$%JEeJ(8i%oqg%DdMiKgGZ#;jz%tJK^vvQZnGooe$}yGzrj&K!v%M`tdr~q1S6?)Q zNS%@nGH9ip&F0?Z@z9rf5qcP4`xV5ws_r}_GsKMXY={c6{z~1te!ET! zbe$R95p+00ru~DR2=Nv#cDrr?I~(W2-r11{r*6mI`E}#?IkEGl(c{!fTUlW?X+gx= z^f{N=o!6-P2Cp!4z@W!rWU1GN^Jh$FO0}MD59}B6PdHb`3UZHqwc_#YH1~5I5u)c~ zSG}YaKkK`~S{OHQZC$-YeE1X7K~;gKqZ8AxoTl|@z(v%i@=T_x8xg#czZg97tEnHv zUu8a!=1V>JeCIV|rJut^CZ+Os9u5ggkxbcP!hoy?Xh7N z-61VWCI-Gq_s_9ZFFMI^NH1AAf9#m90}dliUs~$V>oN6yZ#Z{1>;9pPf6O=qZ}E}T z!r>r@U2my%kngov&+m~Tp|O?(?p38z9F)?mq)@N|DY}7Yob+*}W;|Uu_@OI{nM_K^ z5%AadqBk;q8CK1)W~0ZSAfDg}1B2xFwdmI?uEy=@ttl}uV+WtH#S-SWj)w!T^{1|E z1r^xrWhKg$O1+A8iv{ax2`cQaraG+Q>v4RoQ;INh6Ip@Tc~ZVjQhS6dQfl*gIa4g| z{35R%tU6pq9>Ep!m8MH~e6y_eH9f2i6TJ<@?-i?R-O(+8-8BJ^FF02P_>i#ug?kiucB{C-kTq+{!02u zIj#sO)9YCBSZ04SFmxZ{nBRv`Acs@dwIrT3v_9`YwD7Wy}C;8 zBa)0+EsN4svUL18Nx>S$>xKx`a{@V3FSFZSkqfVzNokouX5*>`^mo2HZi5_PiePd& ztWm?1ktNR1duAYOON_BsR#yi#$r}bg8J)RmRwJpcT3m}+_(pYD7@Ib5(Fd-^D8U6O zqIC9azk3O*+CzN+PWD?Bi${+Imv0uY*$}XT*C(CNj%Hlj{*(9lFOCO|GS`1)zuETM z?|jyO;}l_H#PpwF%OZ6hD`IvOpQY;Q4CI=UbEQ{Awg47ci)CLL7ky2s+JiK!7_2Ck zB$mgv?ye(Cu|0?QIvCa;=a?tHnO>7|N+^z^2WzV5yGe*kdQU%_)%cc=brzBoxv2{h zu+cYYhTqzomDVJDE2Lge-FIsj1@pm=8kro=M(Gq{aaxN}UQrSgZsxoSgyxGnoMhKT z!EsJMO^R3YVLWMHRFOIk798OBSn z1McD~(i#%nZn!*rK5q>lFCX_DUA)}fx_?z|Zux2PzS}crjD6nReqKJk4UMd2tP~+O z*H+iNV3=B3S2ILJr?)jf+@y>EIkMnvqWgN|!rVJ*OY&`Qe`{ z1Ej_@pQ;y@yEX|=oE2K4dXTURTvlgx!{DLTB>`eSuh_JC;W7XVD|U5i9v!bTy>B8s zDipPrUBG)Gzn_-OeI1&>XGb_=8=uQU;VIR;vE0OA^8+j-^+ zrD?maT+jeO#10xNd(7L)LDesBO@K&l0<+Uqu13Viac(m}vmYO7_VtURMN-LP6KbawigZ#74ET``mVQ04RqQY3*ZoKyJFHX|btJ)6m~EG`5a z$KE$mGl=2lE|}Tp=E_RuZ>(B?-Uf_})@CSqySwRyd~9@GTRxsYtyI>GHGk8vN68Q} zml57n&P<`<0mc}I5cC{qs4Q^S1Ew;J1OYpCAb-C1M~5|>+`p^<824N1KBs}_8}Ga5 zh!(H?tD?QA3(QRJq1t+DnYC3lyH+Eop^lQocCw5Sc1mNd{C)~QjV_9#04#&g>sRxZ znV^%ex;kY5;MrVvrQE9}y*B^hlU>Y#hyA>pqe}MHw#8dxW6znUleheknVr)x>YhC> z^%hDF+-bhqSoVYG$53Q|=KfH`y?gH@>s$R3!w8n>npfmmKV_qX*J-CR0(rw>0c=(5 zeCqK2$yKxvPl6DZayK8zEu$6zh#h7D0$<48P6~uaZ+{vdNr^thbsc|Vg$O-q_nT|& zj*!>^91exm5eRarhjJiU5--?!9U{M>-yV-FXSyY$_ow>+9xGc4xnKNzrYZ^--%}Qa z6;%$&+PeMGY1mM-)7uVki#0Gq$`ia^cfj0cC1C(*WyweUPS}?F7AHWQYozLl&VHOo z;Yt!N2c;(&>ck%U;c!i&O8{zO5QZtw7B5(CpKc#Qc-)N#D;QNA!V#@RP83G}SgJ`p zv#M5EY<^T7g0GhsOl9EUmp&f^6WNP69u!Wp;-mefJ=C(oRBtUnWjow$Xg;Ck%a`(L zvJht5ka1a&YablOZU{maT#b4wtq=*$`QUoK10Nmo3#V;eKmGND7< z-jFwD$J4txC(1_h+gK+C-I!|6YS=Ab#JqaIi7U713ML*pPq@Dm@?TX|(nYJ|vcfa* ztr^dczLVQVc1(fW-|8HH7a}OP#~F$zfCtC{^xEj;Hb5~u(;vIF{HP#KbRKj!2Z#yv zf08ogGlLk*bMFudws;7pwO~<_)B2oFaH+T*7;Fx8b=sg^Iu>wRrQ{+58T{hP5@z^4 zbT3C&WZ3Mf>8DC0`y7QMr(o)$QsTZQS`4HzSqxN|JfHa?Oxj?NC32KEIy>cTaU(%9 zQ$xbFGtAoKu-6S3`~Ci9b225=dCK*>Nkx$T`$lE`ACVvfS~hmr>`0%JdV;z5Brvms z4!Y^gr$EsaY+@PxHMl<{V-w1Bq*li)TbuW_6#YJUnfJRS9p^TEIEX`wbR12e$KR$E zNzAx1rhXmB1#!Gp`}KR!(L1T97l!LtI-W#kVw-`h$uXPgV2y1>N$pc7mGQ{k70Ec2 zj0J^1eD$fWS1C~wc2FuOjRb{JR{%&psV6`hysw)Q-9n05Gr~kI7H_=)pFM{No^*E3 z9yx=nVs%r(n%aQgzktL9ZRL@qwk0K#BDE1U64L}T0swN^?T>MQvSzRy$yjPm8B%_! zOL|dxV#Yi`OVh=Bb`%@xryzF-7+(_gQTUvA%>1BOcu}-ry>H=%C6a+|wGT-@C|3)E zMwYOJ_jlWD^scN0m{C^4$$jX-!~ycniZ>zN40|aj{hD z@W4o11Ky29S2GDt9?0&4kw9izl*ur#Dr|?Kpd?rMwb?>Z=Jv>xon7H%LZ1MM*t#Y8 zgb-S>v!I=%rB2!;At)65cBnzwmyQJti-7Jz20_}AA)YlbPl8txcO!|g_HcO6gX6GR zGz)`N*74ZIC^Y1XhKZJ1P@rV|NW=Rf)n~cY7-4o~URkF9xQQyq?jpg1)ZfwvI+Z>B z6evcSo&;ETz%6vX^?^4vlj^}u_`#$X`8gJC=OzAVdeBg*jh9X-X4a6BR=~TlqdzClmorM39#H#d@uDbrW$4md z#t2F=6f(hRf0cM?9dguC!jR>>b1V?6!U3xUtpPwr#VqoyN9p+cq29Nn_hqgT_wUod2`d+IzpN@7Yh*bw1o*?`!@B z=D-{vLOD!|Vs-CKM#)E$Q&QzzT?8SEFNzGNyKZNT8PB9 z3d~ueaclK&&EyfXFq@s3NuIuq26Ge~jgYy`}?_qTbJ&YDbV8 zE)91H3dj|-Y|$s2WKTsVK;Uxz!yI<5_Bq=Z+9L%UjFZb^vKvj3A;iM^hSV#Br2nP< zcw%#6G4cU{?AsHV`)L48@9;cKu82NaXamZj&y%;N5ZD<{4i$v!(r3ZMwVe1NIKh~K z741+3LfvjJ%6V2&NV)HC*-PWhzwv^*l&Ro?Vfi`wCLFW87uPj1vf%i-J$-qftzYaN zo^KEC?>x9LA62#B%(YFGR_vL(T64Bua65M;Anob4u^@f;#II~|LcA#s*rpxA7SB4geJw0OrxHIZLGn7MiP25qvX_N!}eo2{RXzQ>;@4al)s-e zv8wd>)2kijD~Aw>v3WAh1*v!pp6M&K?x~ zAP7aB_eur5nx!|}%%X82rHrexN-m5oLXwg~1L+FA29r$d(#HENV5kIMD)U>#&l8@< zU5CQvb97y*m$246rlLF?#O*UV_&a+%;6tZrVWKC?Ae3vzw{Hm8cJ`%zDN8@^oCK0#a`=X{^<_3IqGQ3a;@_8yOSk ziDUk@$8}U>fI_SNUT&-F(9Lzj-KkG4EKgo1epuh1qe@vzgcVIHYhZ&@#b=5_Un(?;m28{`M~v!adY(ymN9-ybmbj$62JR_MXPsQVjHU8vTull={4B?RQ|hr_;b!yqx%Q z52k`<(Z;XgXBPtP_ADCK-n*~eEg{KW21wR<=|ya*g>2?{7V_6ONnL@#IR$Gz*-#5oZ8=nPiV&qL zy-K2h68c{KvP?`9C*DSOBvt^)S}}HEIaTrXo zlUh&@>@213$rJU8Y{nSL!!4v@b3$WOKgSeDfrW?i4r*Z-hK=HvZ&lNBbMr=mX=HWH zn!;j}fD0wtfzDj(QbJd8wWh_v{K=m_$>Z2$Zi1OyV&7W$(hJdW@zOS}SG30wSlBEEGc zu+UyU4cvY^ABMp1LM(Zd&t(7uJkM)uiyU?dP#I2|w{>E}O|3h7UCb_3b>0WO=+CEiQgBX%#su! zO}L)i`2#9xx*oHETEM;_F?kv|#9%mUm#A;-3qaRHZvGH5FUc>e{ISE40XJBt;r9M{ zvjUmY$BBx+)-_E4c&PA3j^{aeAMAwu0PlthkEl8lnJg1=@Qc|zf@+R#S4VgDW_PAe zXO8aMPz~SH+7M?Kofj8l@YJgk%-flD6|?Q zjpYbKg=Z&l?Q4;g^rx^58HkHYrV@s-eS$Fh5LdKW-}1o^!q)iQoZYxM`BMD8t5k97 zorb)e7*r88CrD#$ZB@d$+^kvUSz?$(M|V~4${Emw`bw+QrX!qw8p(dT!u4QL#gV%{ zSi!I$$SfH8QpWyPJJkS!1WLX`IMNUT|G-;n^i-qUy6#jRA4-57Z#nEUnKRJmRH%q? zqo*Sel|zY0=k`O|Y7 zW(cfrBv9e8uJ>xSH3z71$F1s$AI_{nnK=b^7Ruh;3n$p$KnH zr=LS(8~7&X`cTGf=4{o zP6g|=nAx%pCrj5bR4xvR;`XyGgLaRv-eR9}Kp%=KZ6>@UP(2TpbqQiIhCcLpp+cbj zbUVGx)8WG$e^~e0F%|BH9_TmFI)pR(#>as*ZcsXf-HJK=MwUZve|#GMll|L3p9lBo zk-pkei{->%O9Ma4b);;(oJP_MA8S||TNFGno-->(Px3n0XD zIL={<*QmKh9IhYUCQ${$1&oYZj_4mByE0>1MraTgesnz9!3ox|uhkA4A+XB}le<&(g%wNQ~>f>rqFCOF-^D_Z_qHm^^Sz$(kCb`Pep^Ud6Z_tU zAK%|A{6ra8IqxPB7Z9kyl{G}J;0+MJO zMnQYQO}QwMquh_o(n3O!ZE^nQno{){iq&;V_Y>FgZ7oZ6-0zd_D8})p2C=cg() zqE-SbN!U$IC?ZpE5gS5zE|lTdbZrD}s!sn1X!%g7<#4ot15~|Wr(vgY;9Z%Js8!lj z%=&FLTz(VhT3LMW2Q!T%Ls-K?s#0Iq7uTJTJ_(!}EUwBHYC&EDJ7i;0<8M`U>6+IC zs&>{)F*LdzqNX=n%-;L@N(wom1BPu{X{9;(ZaDR+30?Zo#`)WuNSM+3+-pg|&)P&m z94lg%8&xO`F49P0)s-&FY8qw2BFfq_gnF9mSO%GTs}KXh%<4EplV}B;jPvVq1TPSp zpI(T3WmwHY6aFLfcPNB+2kLSUz~Tp8&MZo(UUy-*itt8PaGul;H@!+QSP<0ep?x>4 zU;>^qJ9~Qp1E?K>v64MuGb+|Ph__>KZv~%O#Io>R-&i~dANksdZ5DP7S!QuMVL<;s!kQ7SLf~q9Dw{E zBQrd}a0;-nznPKO35d;8p}92smA5an!5;$0XQU9M_DyY7+%TAE9supDe6O0Qc3Dz} zX3%;=RiJ6fm@>INbhtl$yIkBWcBA>-9BqU%5BXDmM_}03?aG1Alf4IT;_&ADlTwn+}EWkO5mFpo_&vg{{fU)bvy!>r}u@uK)Vqq0DyoU5| z@USBWw)1Hs5rV%LnR?>21|<-uMQ|0Bn2(BHd+u0WG^slp4ULOged zgT+EEG|&Uf;uIZ*^rn9cyocn5%-`XqIi3qd;7M{nxFk$ixpeAA@kDd~6b!J~3_7pl`k^%IY_;-S=8p91RAQlIm4zKcvSaBGTp6Q>R*eH@a$c0{za?|y zKOY~pG%%wTDmC?S-dgqMI#vz#tT8g>JMk zmX`F;a6BHHUmD%;;BL5u5@A)Bth^Uk=nQ4HmUsrzNF}P5c~!HZu4Ws)*FF)yAC@!Z zGY6a!vxxmVS}XGBLh~qe{G2T}E2D%cmCzf##dZ;6HBmtIix9WG%FS;=$gJ&2h*bXs zDEMjC1E;G5A(#N)J!@~PrusC2` z&qCQk>XcTFl4^cI7mhZ4+vbq$M0j^Va(G=|_9A74M6^=P8TtMHfcFSPl<=|EY2 z@0)`o;E*0VL#xl+^OY+tG7-l;QG>U2ZcOO%!+WdKPrshkJi@k1r>~Ar=-+u>@Hi9Q z&Enib-y^@tmRB$jTasbjQ(Gja!l_v)KUVDtZjS@E7hWOm)0e76s@)#mAKakbWq`dQ zJK{~d!H0AVFnB9{bt2MgU?)pv6R&t*v@m%S{d^~iCbA}ZjrGDOvATr%DS!M$na3d> zW8nOiVeg`)xn)~o`;uO@9D~Bi-{v$O=qFX*>3H;lU*y8CAnKnkiyX4y*=bSe{wa|S z#)iQV@3P$Ix6k4)&*ZugYncjBEF*Pv);{y*IG+RuEx1Bxt3wNr!oN2-sgw&52Yoj{ z!!n(N5;nGd!$4UF!hlvD8fApF&4?pXrO4gyRMJ`D$ccp@CeM=7VAj-?-`1i3_M3IA zx1lxED=%@AiTE*`qYK%SDpL~3^n%r__# zTe|JvtUWnBIFV}egL-$KyuPk&tr-sM9gk{iy&UM{%a=7!KTi)c5!9YCdzs7mWiak| zcGqKClJ`1dj;-?6(XEk8XWD1K=Apc%O*hRqI;YeRd|`qWLO6bstM;;Vwrd|gp?!t% z_$=gFL5FxIZ0TS(kfpER!AsqY$Zk%xtUN~qDSdQ-Y-}S+%|Mx{&$!dCkO&#&w zmNz&lLJ@?H$GZP2>PZA9mn9`aXo07(e;cCqqMNyyM5cuo$IJYC5`Q|% zRGhf+D^V%`9Le*YX^$~^w9G76wcHT_xMBYn6@8)jcr(SlTmjDl?#BGnV5Ew7O#XDY znXdkjBONbAYSCap^@Fl9*=mX_nBXP_R)5+fiBDbi6lEZOn^0dd&c~VyNj?kW&|N7y zF_cKz+FU8b*6zoK(by&!pTg~sHI!Y*7 zjSBn~l@}6TqP@h4HBo~9Z6+-NK}#uf`g41?7^ts)y=ES3(-suuxcclK#CJYZH4K5& zR?YcNh_*hfwr0QLQ*iYPx}BnsCtJuPne$o90l9&YhrWn@^ty1+4XTZO-ox|Y3MF;| zIv_`ICNr7y_S!SPjHx+=MP4?y(EG|XTXhY458u$Cwdxxke7Ee&cEL7&C~Ya2#10ky zGy?3}r6i+)BER#dB|v*Y;QgI1&#${70GBHL9uZ*+b zxh1Kp*g6^CcDo#&`u@5Z+uv3u9m~SHcm;;_h#B!znhMtx85ebtm^=9a2k*M(!c^@q5lv{uTHeqKNMcX*FBpbp>3Hk#HN>3&+J% zVOYAX7;uG9V(;O>Mq)d9+N zu`MCXAGER1iKZ2rcf-WYWM7UF)>WI0n|1Rn8`Iw<$q*?7RtDUFaP;i<#Ns#+w% zusWvbSU?+*bsDepHpWV6DKZ;!3i9)dT9lvHGb4cOl4uIK#z7I8R?Kf?hz5X7;8Y(j zU9QcG?LcU*3>eQWB5&OY1V1iO-cM^Q-|*rEK8_3*QVTfWkw4tUkcmf|JfT~)$lAi; zthD>x#9-ILjrV`rX3sAB%~quyygpgk9`$<_0SALt(qf&W>rt{Eu?@TCya=$aqvt7p zF9W!@Xf+4=Q?e*@&@rQ#>s+ET5K zj%Sn50DgA6Wdc3cFQ*i_8p{9jeTA`o#ayQMYm~%he>zruQT)$ctDy!eL=*a9 z;uZPH2%2sUGZwexyadv-zYv+7&-@T;)?ykJ((9Ee@1qFU>LD3DkR5fZb%Hy1(+&Te zXuexp2&HQ~MK)?INZQKt$LK2BCtcnc0J-=;fPjeqo(KP!-uXd$Np#FC+yDWr$UAQ^ zh97VomvpjUr5K2|e>|5DC5059;^Lz-+p(*936rw5a5xsdxTX7^PNgbm5%gIoXhpsgHT3|xMynAY~)wgbd+ zzkqJ>#dD!1(Q=aZZG3xY;Y3>1C+QqU@cTd7X%ZH@0)hchwFv$QFnJ?w3Kd8g>qQ{ zv2%m^F|>(F##(GM-1P|vDa3>-L(}Ba{tOYRt5!xE!*|lwnOd^Y(}^99(t;A_vD0@@ zeY`q6#2y{yx>249!XD#!d|Q2r6Vumg2y;;BlzIgG-+KvgeJ?;F0aytESpTN6`hT$g zHI?i`HHepn9$wKMgi zXLS;-2f^V3A1?L6IurvZr&_2=GY3Z8ZL1)GYBII7@AeC0vv<&jc>OWlc0{n%xdkSF4(;kTc96#gMmVH2kuR<-1ke>Uc zkpiK5Z@8uOs)(}SC2;d&fDUV!GbUwxnRO?sAEL~mNQ_J~1*ul<&^}lc|SbK7fYZ-FlR<%8{7)ocbUkGjFlaNOVh@VM$+Md@BLREH}tz9VMu z{qtC?{gEBG9Z?O!xi1)1Pdx?ndk#5&)J{}nn8}3#Xt4@tCj3u0|Btr%k4TXnBM1$` z2rK%?DM~j(qm8cZ4$A(G+Z^IkN%R70O${+LD8%Pql_YO(?_ss0m~Klswxdt5wXR^H zgLay*Zb=Fm%IMorff)%`F{=5yN!4pTO2eAMdupvf-3P7mTmR7KDT%R#DFD*n0Hl8`rvNbKS#+XCXAN3Pa)AJx<`<9cwRIX>>lEc5s*$ zkjuzc8xv~y#p&_(pCa|M?Q1OZZ^6aGVxIW{;UcAu-?(C4?2l!GeOvSl!;~v|=kf1z z-yYd^IN$!*vRi(L8j%H{7CAsi{-(R=FKYc)htWsm%AVm%rjM(|4ep}p z8Yx4eH48Y&Z`rP0t0ra0PK>#I&Cg;63?n{9s-8#QX+v`=U+nV1jiclEjcXkP-pe_F zx#55c2h}$<5XERPDs_{nzGRLs{!_WZ_5Is1gy~Ieg9E~Pq4+J`%2J>;CFU2b_8W%{ zESd!}*0{lnW9wO=E8G!XJ5HKGXv%OwI9C2Phz2aLKdjuZ=HwkKfSB9^Iw}9A^7Sw9 zfa=!|aVL%21Tr9sK6(ZF66hL-XGmEm>8MQ-tz$A1HtvMKNhJGt!@&hz*`*O&FKl=o z#YAzMV!^}2jZv6udX560Cn%{; zUp;!3^RwF}{CMR>X;LcD8GWTd!~0>8nB&)~t1E*RLOEg& z=;TV!n3{vnBbslhOM2DV7Z4;bi-F(YL&hZ!GV0-2k&(rPk9;G_cOlGh36a!)Qpd^B z=WAT$K$U}o&Yf)NGDTmo^da z109N;y}5ZM2^!kb=}CFYNy(;qML%kVFJEfMWG7{%XsMb(>igd2I+Navbd{&O2Da+0qXbB$*n%TsIudD|FXovuDL=OVIeUgbip>+ z9-!;)g!#XL#oK%nh5G2=oGS!ZQtrEljY^{O16!oB#}${?gD)`SkZ}1-#>gaLUH7c( zsHG@Pm?DefUIv9NE4q>6;UUr8m(HG@^$c^hAAPEu+=H%}+L104?AOwhzZ?8UHD$B} zvcx#&dOFEZ;XLn`Z|Ef^80(=-LVmv{zHxPfao6IDku7N%vhDksr*%~<5g7bOk0Vo; zw~s%74A}r>{D=(J4*vlI!|4Cw;3RGR-alD*4+ruu77EzY{RcdBl)6>CNp#*7vKzQd zxV^nmOq6E2z1=w?0^U$1uq4@P-b+{jp)<4wq9)t2BAAjN1L1;-6u(*tm=T+y`h&;J z?Ys4VncigpxDR~kzgEY7gf2N&8h(HQQsgl!047b=Rf1RMC*o*?HI*-Vmv^FSautb% zjw*B5E-;cwx@;RmuQP3$cU%idrHc8jai&&Lz$zLL(w638nG#HYR462oLy{ROhVK1% z3rjh3uFW(h-&UY8CGCO(2oBL@>YHo8eA0K7nvtn)|f`1u;0Z#|82}Cn=pA_g>q44hhH|?NV5a zDBWsDJF$I#I1DwX18GHobB6<*V#2?Vbxc1zdUD+4Uy)AmWljKiKad2cs~escc(lqOk6st)!Uj zrhn!z3NUKJIQe`Tlpe^2_sPd&&ZhP3$6Lr_%>(nfCf?{h#J_vVVTeyBe=YNr{#f_S z4Xi(s&VOB=lQI*B|IoY}r#tZf_sjF(t+R9CAYtN-r*&2WsD*X}OR~PUm4%(Xu8fJj z&QAm;o(0*6@DF{{7H2tX1ptx(03!K&e*Syz^M48Rb)sc$0A8-xBWE!Ej7Cy@`1gRX zt}66jg1`Hhk>qgU`Pd-1kOq79JS0L>vlS3E<3E($UhB5MbYl{WM`5g zM)ZemaRXhkY9miMTwU$5ti;DL;h9UOw?a@Zm80xKYLes8Rxua?-!2inW1_f$7Kh(6lu)K+++0-T~gfK0LuWL$e9) z1j}G-j&Gs*S{)HU3A2Jd7ZiT7rk0I=e~z$<^sK{>!BLCB(9NuD(xvUV z``xfMr+VV73Mbn7s@CKp{Q8FsD7;v}{e^ZAfc66!uy%5Gu>44@lmCBUmr4jKO&UDj zy<7biH7$Ke`#S%^D*BwXkGu0gNg;ea3QM ztU)BI*U+&DuKZ+XA(F6b{TNQGk2tj=LuIL{Q_*>!gnaM<4@7fNYzn`&&Hcg-Vh|$~ zt^CPJx%$}XmZZ35S>AS+pL;az%w$U4UaU#RY-0^O8|Z0dl^CH)qNMKl`CD!;m9>pf z;RXk-8e0qSxV6NZz7SY($;+wvQo87^0gW_IdfL(nM>uK4Vz57`eWFUSu!*!?L*9X& z?y`N8xN*aBW&QE_RgCZTWQhl?YtsO-d74q?&k|}Wt8v23oO}sZ@X@gKlUt$f!Y?eu zb%IZQqVT)p;_8Fwlj>Gr`15PN#qe3{&E`G-em6$-GA`u;C|d>)qYuoQgYn0_n&zxCl3TqBZ4BiUekip-RtWA)ObcZ+IbvUv+1YMc>0-x}N|s zzXq}7Mg5-TKh7wGSe~PQD=Q@2VljfcJi*6MJp&h%&PwF|(b$UhoptACe-Zbg>vyQv z9~C%H1i2|^0GQK&yzmFc-|=IF39>PN8UMdrvmjX)3C=;BJffWn3JA3~vyzrv5{qP~ zl5hrWpS!KsD63-Q0LxTQ+V2dodzE_ZjEaQGo?mGqY_xQ*iVvxH~{A)DPI@SlcPvaRL0 zMP;wdESHH6-gf!q!{5zqmX)Rm7hokDi)$3+T+%v@F3G$x{fx#;_VQAYpiD$sIbLEU zhIDMmxZVKm2e3Kc9V#6Cz7V9b;4V07_hmNuxAF9J{;1K%NB1Ay@QUvJbN7M(8#D;O z+#YoC07ph}h^>KY-1cuEQkUlfEmkw$zj+DZczk+`*LcY5?C5E#5v7Jeh}5cKJbHD6|ja@XlpZX zGU2KmVNyYq7(}Zr&1X)4ffxe2d0QMM+#9vJR(JE(SdJ6 z3wikB_I_Oco|E(?t@IVIM`{l%&)Wq{cfVw{5ADV6eb@e_baCM^KKQk44gW5pE&fyEPN!E*0 zqkK%)K&ny0jAB$Rmn1B-{#5>D=rHRO@0YQdML~X|b3e3?g*szWR=%nAPD8-J{($GG zM(h}4`I2Zu3xcM-?B{UR?Jix#Nu<7L0OWu;dh5B8v8JV6Q-sJVnjn)BCFhfhs2E^D zd|#pgF(3pqVo~LIWZc=4rB!+or)_H+_0{HN=oe!_D$OBI9k$Vh(o{fkoF=D=VcOj- zq6-=yR8n_mAYQN?>Sy|?t$|fYHw)Qb2(NkEkyQFu-f6G{$RSn*jmQjFMCzbeI#nJ< z-9f2{%QS7B!5zmI^1I-G&8qN9mN>yBz=hoJOZoNAxT?3Pfxdhq)n9>XF#C$)loW7W0tV_Sw?{3ZVi1t<4K}TMhQ9F<$&TzsEe?H87`!_7ESZIDMM@ zzuU#c%U&L10Qz0P_kmq>{22JZVg_0{+J7copk3eq373CmTKoW1rxIKRz@jQg+iPf`Et{0}hyuL&blza*W@o}KC1u&^Ao`}q6sM{1-e>{zBAk>dRyAE*x9?>< z9i_y!SifqxGRSYhVZZRQ{HCp5AI*$qUAi}oZ0lEo=pFGJsOKLM{L&ryxDS996o5#{jRVm zN2BQB!QHZnUA~>Md8B+#*l>G{r;x44wz1nE$RO(cy#G?Px&J^=T}?hh_FsZZC%^xX z;2av>74-i-IM4b%wm>A@S~mjJg2tdT!e-DE5()E%l*%Hb3|0Zo4*Ri`y4V?gbpGUj zKDY-zI!3e!Z~}!>)9;K`Q#!p#1`)n3xV;uEttm(DR)_ez>y2O&UhSN$+{sD%$)caT zuB!n@5J!4}PEr67*2>1ixt>TV?%jbQ6%!VYd`dq%=3I?&zP<2y3z8%BjN@;LxB2OP zqi2!*2pCIwIki?7iP5}E-qD*KtI8;>n6ihgMi33E66db&dtNDgmc3|2dop~iMnrvc zwBK~lUy_uL_R9DN9nilS#Z&=Wmr((&%YWl$Ix_z!b>7LE){fXMNWWwB{44M!CTH%n z)+DvH%8ACy$!wA`6nXJtLqWIvNvb>z{J3Y&Yc3l)das{|hOcD|Zy;&VUvJJEJn>@h zS$J~UtZ#+N)wb`%v(;WIEWEN?H1U?aNUaRL8Cg;H6z%4Fi z@{gUDAh20Rbu`gClO@Gx=fkPhz<>ASHS}&TZF6UFJUW+8OW*Qa`PiDCxMNy&d4@sd z)7nckutaYyRO@DVR?6;HkFrzMEUPGFZXB&CbIKt^Q$l+Ye;tz2(bw3o2xfaJ_(nCZ z`Xx5zSKnb(4mUm)E>=eXjWv@Qsu{Ni^-MAoa@ei&Qn)lboI@}p)Qx(;dxmrMvdI9q zqb4|E-1yk6=)6B&Z?UPVX~twff9Uc477~NuPwcYU4)E$y(wD`l+ufaFjnJ}=?uN{U zK8f)L{d^zdX}M%BTaiK)xSDIU3pgvjzz+3`H8jzex)5*InA!VtX1-l4wPZuahe$%T zSoOK*S4nwr7kJ(^i{1x~zK)a)bn7yyb(%Ar0@dJgZKFsycn*3*CA2rU*h^yg0N6D! zYN_?pt><EljJp4wm0Ddb1d9fY2ivN?5SFZtlCrDeZ)J(Z@?x z%q#;MbW@jC$btDql`f zqb7i{`Elmb=TbE8p@pS(hzMt$$tQJg@*-~yC<7m$Q9Er1VXzh~%KeT$!bq*H-tTw7 zQ$~?b*8b`dlFL3~(j9ner#}>`kTeu~&X%wtpz&i(F=&4YxI#1HZTPXojPfD=Wh|cd zrGv6Q)mhN3WWBr%rtbMSr)zt>;sWyeCOp8W)5iS5mm4Yqe^1cncXd(G0Y5}v~ zlCDyb&`?PKnu?nuRy)sB&qHC3L|Znr%U2#9B{1-m_>O0sSxJ+XDk}*t!J*K!G{+V* zD4;_Ic)`{$XmLSGVAs&=d60D_p(UO9v?S&xP$Cw+5gaD`-h1GBb;QF&W>g5FfewHQ zd}nq%H1iOa*@60@_$?DB;Hi!NprgCsxtD0rtL#o#*^7z>lrfdGtCO&aN25n((YJl; zl_<5$#+%hg-~A?TP5s=ZEedp-VckXfLV!g(EMl7Yob|gM zd%mSMnJoFm!Ae6GcO`lRrP0ySs%ONFc}55~$ihRkCLXLMVy*`af;q#Qs7ywzfS8mr zXu2`)Z|_g4ceq(?gN3nFt|uS#?5Yr;@kiG7-7GG+{UYZkE8KYIs`6VPiRO+$9fT|bpB-4n!#E%9nr|_hLUc#-C>-lrL1dg5CCPf=A{O zKC%xkcXX?;wKSl1nw5C%QdqgF8@f&FaOn8OB_NUpO!8f&hH^!`h$oSnL85+NRVGz8 z8F@0YjI%+{BqCo<;4?M-X9^t#G#RqUqG6VkVZCFpESIzirB#A zRNE=r?JdOY&d%cZZMm(1`MZ<17AQX>s2_VZ^X9sy^W|QsDpx;IB*E!nc%d;)_I@uq z6iE!`TrJsHuY5kmr=nt>OKl;hjfe^|u~G5o^j(XinMaStNg{!4*7!QE(ja+42^+-8 zC53jABu!6Gf}xkE43>K0dTxsNyjEbc3E`!$07f0Q3-xu`E1= zFDTFlcO>*Ugcd|nDxq%yYTjqb5E974dtmA?7e;`RknZQ4L~~ejMb*!KL}&~to)el6 z=p<=juF$cpm!s0>y%|6j4XX^ol0<|MB@pc}rlmL5%z!_1oXUs!rKoXg47pR@VW78@ zxzj@gD>p|xi3C&tK{CWs9kYoz?@=*|9&NTC-s-dvD>vo57tOAKp|aXUv!nMV21_`| zd%pC((Zq9yQG{TT81+k{aL=~C?(XfE1-@j{&Z}g+3Ck-({xF2uc-inL#TsVPHYLfb zT|Z9`HaR2E+m+EahMWwp@_7V8;s;4#YgG4(($OY}V9xh4J;Hr$`- zX`{&D-f4ZOJ*P8MU+`wHS9~xfGmyGo9jD$Z@G-pkGvYekDq>!d%%dHe(7z3iOlEt^ zATy!B9^n!&f>`f_lqMwwQqnGaCr2>eS2pNo3~psC<`ct#s`!Iwtl)w`>ZZska%Z-l zYy{L;;a1jU6m85DgbHW+knSUly6sO_4({BgJ!iDHAhX3#dkPa-plh|aI5jaKoWM9& z&j1D%Oq^JQQO9|VItQUSyBs(X9|0K`?~8#D!={v&m(5+{LyoU@@|5%MdNL`9cxfm-5R27aIx-^P|>-kyr-D-Q_A5M z8llaCl7%_z2FXN@vi&Y|H|ZjFu2e8xPObF%+HdZ3^$@!9?8tbK2D6}u$m++?jr&Q0 z2V>}Lhm;Lm!J!ka-r&bTxyOq#0kV(@swZBS8xP({Pf2qCrejG_3PZlM-oGW5YOZ<^lvnOL0QfKCpV35XEpZ0&v#Pysie;aprUR@P1EH)oAPbr2QzUPg4 z7SgLRhi@aW?P>FVb@8zObvD^?))Mz@Y5S$$5hFu&{Mm4aaWq3vm3x3Q){F|}Fj&~X z99pTmn(gZ*#8mY}wVk_qs4Q&uwoXn@%y8|SmH6+%CRe@LmucL9rps-eP3?!+@!_>E zf$FSug@_m1r~bEK5A&4V9pL%RbqRZ*l9GhCJ6(&>B!QAS=S&4>9>!pD^eI3?8u1Wx z@evpLxIIp%x-3PHRI;$+t#w852q2l-fJ78Ubr?dB07dCTf5gex98KP#Ef`&3Yx4fX zb_G|unu&;Oiw1VAK)m9%vq|}pJ&=k!i?E{r^E~dDR9tuuLP4(+;hveFg(J=7P=$9d zzy+hQb8fCpOQ;nMie9$l4fu7l%Zr%KBI)ghQ^R)zQz%YW!mYTnm#kp=9@)21euSzv zi(PzN7e|3UKQGp078XTH;H!_rf3XcI-rZS;`b8c}(-zQeN_S`K;#5b;pUX*!R3Y{- zDmy1y87fByU=#6Ud?UQif|4TfowW~`+)}Kut(@wq3(K?;?V9JHLhm5sQK4_!%x{f6 zzLnNDgh6~f7-xQc#P9ZWNJ*LR-$G{^+4bn?@S$lE^xn95P0zu(V3F_Ue_g1H64h=b z7grCB=?u(`HiEuUm&`e!tAcJwhfYj2z5896MwDbw5Qo>Eby$u&gDeY}Y1YD4POu?ey}q5X3yOr`wU%~61A*rwAv(> zmljZf5sOwI35FuS9+c!)JB07hk~nqqi-7E4e#e?+Y2%+o+rHAL4M*%Gi&NFgFs4`o zzNc;X-)tLru-35BIdWE5uq;bPPJSI(Z7nSEOYZz5!_f9 zU7&p+|K=cYRQfsDK%BPej3Dyo4_T`fnXHRP6@1smXM3W__vYx`oQEEk`dwdTp?n8i zaeLncbd-ZQb~{FX$7psRVvFPx%w0ES0`%mhu_R!mnvTbK<#KE~y=Pu4fow~UcbQ3c zz*;qzg-fc};0|s?sNUK&UI~96pu2GLOBuOl2d-tR&))VdE&|>hciv`z<>AYe^Z1TM zreW${9#Z*fLwvH2^r~FWZv9KqeDII_>)ZG=kix%Nf%P&QSoAUkEyNnQE=4_d_zF=v z9t*rPA&if?Kfm3Fh&tC6NY1?Fw3~8Ma>cQ4FlzX&HQ3f~6`czv^0xk3{r#|yzMo3@ z{>N*J^wA090^p@(6y|R$avi@KIXjp;x$Bt%o|Zl?7fDvtv0djt^g6ClKNy7CDQk{I zZ^Kc{`(f%QOy7$6Ll_CHv9Z}LS<=7gyrk#13FT<~qPFKTfu9I1pxcSLV}hw2A&Y(M zd+2Mlt1K>Ea{J45=~kDFq#f7`0=BU6>nW=f$jKQq`bHgPCQ609dVaRR7EA| ziFaL{*J%{JFsq-GiS$jk%R`iJ292qKEwIMZ3V+|n!=<*ms}6~Ku0Iv;9_*#~+gUH( ztlY^HsNsw(+VD7cW^C$1r-jZ2<-HteodTtUb@>8!o|WWlGO+${f2#(ay3`GnQ{0st zOFuYNWrHP4bRKwt<*DHj$8waWzv$jG9^DP7(cZUC-i!II}Z3*3IVy$Rq>hVl#`-!%Xwi%FUUco<5fMO+`VEydB)z=GU8;qBk)uTVjP)L?x31mHGKh z5%wdSEt!CuEdPo#Sy`=5;X*@PH4f`iJf&>p9;{JU7*K^#s=kpl`aD{HH_^yK&}*ve zFYjaOz^CvgYipSWm8>EQ433$}Ceh+H&ize$(g*p6uo1M)xMle^7*(=EoRYONzsNU1 z_=9y7Y^Z^IHl@?9YhaNY4nX6Qn(IL_vq*1^B||2Zc4^AF-Ir5}e&<&K{M2#mMd$b> zM*jvCM&pF@pDgQmObO>FBBjD)N*Qm@ZpO%~2Ub7Gcw?PuhJ*`c*z26OypHFgD&T7{ z8XkO^#^<&hG>%+VX7a@=qC5-%{&jT!Ow9+4UanYR1kPD~u?<=5N<%&qrw1 zJ*=K1tf-4+U$%V@5a3cmz3}^t8vzyIwP%YIhu3O=%k~N^oM3p`UOH5JTK`sA;YzCx z-DAb8LbhB@laHh<=`9yMHm6JM0oHzfGXvfx6l!eK;A3pYPrwbdqXNe-klSJtv7kgp z(D3t9E>Jl84oK6;`9NIq9n3t8B2-4#RgpwM>FU@?BRiruzmR8H%!riDJv)ShOVpfS zP2VvshuVmV#H_pxfAV>JkefX8d7@22(+?ohe%__MJhpjMqsW4YPH>|}cFcCkQ76w= z`$kuaXV$XJHDEemR*2wFVZIXUUj$uuN0xydfJ}a`6J_>vjvTGNJNC!_p#XN`?Ds;0NzL z)(yPo85zg^@UW$gKL-Wi*@=t(A$g0^EHII(@XBc_B3ZJIevbPs#ZM}XFL=`XK6Y$=Pd|aj*NJ}IT5Dl|80Z&ch>Sh=^yrJ z{BQijc@&O-G0C4sI*Dkm*ap_t4O1JpC~L*>qJsiX#t8${P_^b?>dRaPp-b8|UEt*i zn#b)KH+u*nvwmr7bCPM%hP&7GlJ)jvwM2l9iFWBV+H(?0|E*M2^YKN~_ROdb5o{AG zZ&V6}BBwHE6J&ixW8}nkU@T|k2t9h=cI*1LZV7ptdRyH(wlRNSUg%b02ct=7qRk+! zR(T@nSA|l)GAxf~y~NCu$^!(^=o7>X*3=S8Vf;0WFq2`2=#>aI1&C*Q7*e#3v!9|{C=qM-pt;~`YW#z99wBterY zQ<*LpL&n|jCiFt^7>YJ;Jxa4>W=@0+8EZXfU4~D+4@45xQc5jjJ;~^1VEE6@&R~eE znq30^@RGD#*LY#5k;yC^ACMt!kIFH$Wc6G#M?K>A~2J6EA7HZn*bpK^Ra01wi( z$pQ7CK#-1_L8aKk(x|7N_#Ii;BoEPzd>4UuGtNXgtzlvk7$ zN%g>6F~~-akkCCv8uMzsMdv0-3^DAMjP`YJ5P&5ZdZ^$?VAkrR*#^JuqKney&zwe} zKn6tANxd8%xikf67-0m72i|LM1~VN`f2K9|Hro-#-QU>$L0D-aEx-%qX$yW5zb|Q+ z#H1SgaGMYVjtoq^Y)l6Hq+6x>jE%Nrb4RfNS*g5~1r!>EekI}T3Lw$}aZ??@utTAY zQ;FS>X{N$Qay^K`;l_nTuWrO=sF9oY7#gM99lWYhQKh1{Sl1%js2b5jH5T07~(@&I;d&7XkyJAIwO|CPfT2n6pr~I;dWlPlyeX~U>l;$qj_os z`@t`DG+GpuFOo^+Ymf3=OW)46GdP$SXNxMKxXh(?c_4SAnyg8Tp-l6vnebSHedQjK z%eHZ$r4PoIv7Iz+G1C|UgYt8_0fvc(YVX?z9wN>u_v#-fq!DI^8)y#`ES@pBj0bSy zi$?O#C2gJYT1Ea7fN-og-G4WKhxFS9`sl_^L5r8~3SPubq{ZdZ$_mWPOb|z$cNhP8 zY@zmH~T3_G?KLF_mU zhkr$y|KD=|9s<5eheq~|Ithsu$oMQc!%y`<*2wB?j41@SBsY~ ztC^oPqgr_syrtPbVzS)ydz3t^NLZR$cD^IHtK(VPcth4Iu@0xRX!~5GZwls|KJ|rnla9UfirLy^kzgNN_vOQz*J{DNXe)H&LKlYm;-tNnMpS`N19BO?vb;44LPm5vJK8ite~n zM8lWTfxZ=5Ev^tVUMkFzJqM8$1Kp3Esj9?BA+;nZ;4r) z4U4yXIumWWrnAwt0-LpkU4uAby{17_<+3ekF89rxT^FzW&dEnPVcSokI~)>f!Jikc&^`aYU>h#re(Khpz5Mj-bNjShw^s80^W-78?Zlb< z>&?!`{%`$ZP8K%y)+YZm`c#XC&2PDe^siP=@Hr|wsEF-i7u^WC-=Z^W`&RcY6D_Qe zLPfKXn62@gl#+Y5@aH=hQ;A+KyEUo{dY;8Ii_k3B;}p|0!``!{R<&c?=gLgd@fF=S z(Sx`a%T9I!O*c!-ofvI2YdONI)zgE}`6xwaQs|p?%8u#=M9sK5wM8Sd&}9d&o+MXRihafK9N7Sd==8tY&p zsbp$IH#0AC%G=e(QkmlwQ%9m-CK>hgr3LzMndBo7SdXo3FbciQ5=E?xqxU&@mx^IU z&C{SqOQ#CFehu5)E_CpU%=bTPQIst@-hP`ipq*Q1_p?^GS#+!AHa3Cil7IDOn+x9|Jm%M|^?hK zu9E+{8$QWINMfpC`{FpknI?Uh^AWtXB8}=1Ae6Wr6Ue@Kh243gUd6N&X-Vt7*)?78 zqFqdXVH+4=@++9WB@cFf7^pwd@nC;gvlPo{l-Pi5Muf+W3=$C)w*32`GU{R6iH~Ih z;e~&D4g-S;!LeeFRDQm z%GfY45wyQ9t%=rW#XR2#-r*9Qnq6I--fZ0-PM!$x2EUD&0ZWJxoA(^?Jts6)zR!m0 zy+>`a!XF<@J}kDvJWF#ANY$TYu(14+)lk}Elhx7W95*oA29W>=t`aQIQ}ls~swHuZ zb&?5rzi5^5um&iMJv3muh>|7(YCzGj+S(SyG4ZPs_slpJhYt*9aCEDF)>hG&IKMX; z0QhG1ray?9J&B^~=Jr`&FixHyhO_MTzQ0q0 zQZECk>b1rO;-l%sR&Ckbu_;bwL5)+TMVZfQ#!rX(Cn5FH0Kh|vg66G{P!pD}vA#TR zkcO{7+5 zylAMV0enU^|DLuy@X<1e(lVGAT}<1SS{M{4EBJK_4A*Cz68+!NJI>kTgHjy5Uz1u_ z_eh9CPB{J~(mf?}iDIoJZ?9m8F;OVGKA>Ox0;y(EVJrd;STf<} z1Oy{v2tS_KtOq!*Ac;z0A^?a2qXUd8)dPx`dbxSL++)*YQ7MtHvMv%a4-`yf9qi1i z)uPLm6f1ZqfuMffKjq=Na+VLNt7BMK$_0R1rfP{A5dx6HF#)MJb`gD^8;t-$e&Xc;j6yCrC+x75KTIZ-Z*;uU|n)l!GBJb+(v zealCYsH}F%0g?B7PpH>5$&4blXzmiX2b|jD&K~tNQVW#&qF0tO?^$+Mk5+v0hy2Q( z5Npeu-jUb0*MJ?3S$CjzZ#60;XD1N|Oy1K$A z9uGoQxi0}_q?8LeaJ`1QEo4Kna=u1?QK^fzY5>$!?Y7a%#5{v|cOaHTIEdFHj)O=T zOTz+_1O(AMq%(@tBg8Q;y)@M2G-kKetDJJ)Tw+xyx~F^Zz)zLHXSL2fq@{O}^d&}j%Y3d1#mA9R<@G7seWFBo zF=A<8a4+2jqa8*|gaHGEIx>YTgwVPCccoFsjsL3Nf)*9M{w*SP0eyh?(2XWs`Reomjumd{*nECBR)^R9VnttpAYx&PG&D| z_{Xo|!B=fUmx@7d(P3PcD*+wS-j$1UBfm(KDX`&c)dDYq{V@-?DJMp3kDsVr{Uoij?ftt+!)@f@c z#@y%xhNEj0ke1`m`g=>03!>GToK!Q0*%ZG@tIF(EF!Yo7#+iRdVLG3c) zS5p&fn1};zB32+F>IW3(?3Xoh{f5!;9>^r#+d^@v0*>Avrkp|#^;io61fGL2!0PSk zuGhW!9>A8eb(#<`Qh{JYNOQtQX~Csee&1NKkjCRoyXG&D0a2oX7FSF%pvKvsyufMK zMrwws7-P)of@X21KV+$BLziO<%kK%L_}G;{$Jdac4-*6c?GMUohiDu70&&G5e273b zSr0QbQ}}$!y^#`Smr}viVKDS?eK1$k_97vfD zLmx0h*A@^h$Q2>35D&euGDG!eT1IF?K;kf4NQ29tkWM7mOr&_53V3j> zQI6R*g>BvyvnhLxu%D~q=&7N5F+y&r$rHjfr5O0aZEZ(1H1?xgDN!kD7#A`o%cboB zEC!}p`8K)D7XMkDO*kOO;WOP#6#x7mYQ3haPP5EkwO+=54;uN^)&5V~Z7rHxvA>`Z zpEGrPd>~1G8n5hOh;A!-vR;xPWBLbo0}x4u(&gk~88>CyM)BM44_;N#S5fBJo~&yC z$ZE7Mjq0fiWzib752|ix^@f3@>me^l<$8%FYyn7?rfxZwX?JDbPr<>2A*vJ-`0N0)o$^&!3&`TeU3yvrXS zankjCKisH7uM@Y6PWxpK3~5PWy2jTOKv#=I_A3#x2|(@j|JHTb8&;(oEyX$-my0@D zu}Kc?yO{3KDT@H9tjDrNLwL8_0_UwY$~qk)|0#^ zq9o5FuVD9$zTx&q#N~Txc!_Hfnmbsd2Oz=hj_=plRhAgzgWhc_dAfU3dJzn?_0oJh zAYtC=D=S%&9@c66^;UEFipv-d$MdUD25U%_tl$ix8f>$<6lKC*1uBMKPy+_i2I(!7 z0U%5Exyxq&XAB;XO0;+e{qy|HI&g?m-SP0+rGbA>)g;4 zS=)GTMW=5SOiFu+0^<**4c7A}uH)%SL8U2BZw5;V6%RvGb38~XLFuKK(j=>pyA2!j z#_+mHONjYpLcCTm9jV%0?f+@JEb1uT(z}OdU7fMEUVQbALZEhpa1^Sny7|?w+$(lj}{uF zT6ba%r0vNU4Fe$=y$CF5@^*v}end+NpVwdg1c>PC%`*s`Mf?J{ayd8?>Bfp-lyMaa zr3(s>JA-lRM^Ot3eea&+w$f2VgN^EOl97FgB~Oh;2T{O2P2lzX8}WoeQKVZ_+j(gW zOr(fv3aK|&re7+I4V$NWzN`` zCKxGF@*~&zKdm?AEQrC)0QoB5O4Y@a-kOFQl-9Jlu#$Coe}A6Gux8#E@dl4R-$r;s zG0D#16nZ*;Mm9Fi^n6`j&ku^|d%N?1j2gc5K#K`_K8Qznl7+t$kCF{5n}ctW3?29z zDLMx1MiS%TcaCHsxgP)8$m5t;sh!mrs%bM1@f{G`LJ&-P=*!1gpU8^w86Ag(gGHfG zKfd13T=02)z8cMPvAACC@?Y%z&6)lD+5Cr%Jr<%qH^6R%5;(Gx11#<%Cme{pZXaAG zOt)NyO5wnO#y5c|HG75SGow^Bjqb%cRM}N!yHY~*Vr)NBGx^EQLqrTL4>98f(X;z zMFJNhzE7|@&2!3(0ycygE~;ZiCEGGH5pQ`3Ox4=*X8)gBGC7i!)ZtKxlhu{P_#TEs z!skqy)0AltA74T#SVU8V^{!GwjbXBo2r+yGBXO5}61X_;>WCwMsDN^T%^`YxpEljb zS@Ah6WNzRF|8Kf)Ce}=XqLvhvJ4(GqOT!uz$+oO{UoXPB3shYV!r4VLeJ1`t1NZ7y zdd^Q9S0YT6bQbX>HKvVd@km<$h<9q&ksc`;1sr4xf`kqPW*;uZ(=p-VVL;GDxD}>2 zHh-j8qu)(b+XG<`HJf>tOGO=g3ZEh{u)dKOAu^x@d=EbJ~ zB&+RvK^zKN)@XBF#PL7}%75Q#F|ra5ux&lX60_fKy98zB#}Rb}m0JhO!mEf-1=N>7 zE6${tnnXcSY8T%w2z1lK4=Ynu+8LDMaXY+wRagCPQL_=XDHcA5FE%|%{pUOl1GLGbdCvG)L=AOxY>b~MRst)lYn zdpSh~8yW8_DaN9V+z77hQ1d9Saup)>#J;a?20=lgzQbL3lqMO(j2Z2t#KL}Yi)szn zNOoyP;Pi@s4EaFR!=Y4(s3>NMyz{p*ZK%)%8AvgV6Ej_Ks2XLO++ZcixhWff>942u_0jhg zJMEo=8M4tcE2_235kWER3GqUfeL}`xY>)?BQTc^bS@R8bSpX5O7g?=#7Kr$_5|H8; zJ7TfCtMva=eWH+ONag~_Cyrt?eAwMNNPbdUbJ-0FW(bZ}T>wTU_Pzlf1Zlu~RP3(1KFhJG+~YPB)Gp)v?z zonDzwS|KKy8n>|%o0yja*%BL_gr)@-6Y2wc=RDG`sm#{k1{mUXO?*UiMmLdXEevDH z@_`^U$!N{pShh*wV!$%)Y7Efac3k+h8-Sl@ER~mGRkZFqSYc7%&x>@(YAqJ?Y)b7@ zjgYkI*u)>2SWHIj2nwW|R@PL^4`Jhpm#l*&th1I@S_?CBWgOCfau_RQF{DkP1Vn)% zixWYnbhJ`6J9`l-gG8o0=SEi!>NHidlw>wDLeZ>F*uTdDf|%DljZ8>@#2~K=V}N0Y z(9*VVYkaose2A5=M^rzz!n9h=T^L2|Kt2eQ79q}vq;DXK_v&0gku5ZG5B$SPHs%$8 zGfL0KEa{&|NG|`UW@>X!w!~X}n+TsiK2u0u5G~1qPeKS#>TI(}I%HzGf$vY9LkZTE zk532+xzbs%zmxk`ZcSu>L;i|aOT73MG4~N*nXYgS;%_@AkjHaLZo`y%CLrX%EITj* zZFs!&i&ZPWCc)rm;+X-IJo90Jx#o*@G!8*rgfuxm+-Ym=K(hRHDLa1^9|dvIf?W(P zc;Ly2f!y0L+~;6IOpRu_(^*l2B3gBXpNB>=mq}VfwwC6CZUzm+W{TMa8mDmjcFp4E z8@`L~l;&vU6RC!GN;E}N9dnitbgjkv2Q80Dtuy~=E!H-j%< zD2p9fU}X;E2l@#trk~TFU|_|pd4ry@TF}u-F`^4_tG(T8$Ce`&Udy*J{*Y$>r7JH* zC`+QgYJ2%Ziy3V{+@v~#!@i+F0)dHQjLc9Zpo*#!iFnvRB}H*(?J*)edKPPml{X*X zfe1QIY}$Drf8Jp09P5{G!m zhjX9jj9=kntHhsgcoI5_A&TscxmI=Eu%HR1fDL?{sfB=O>Y5+yPKdVv|Ii;2MZ;6UOoW?u;;Cefm#e2AeP~{45&@lTtKz zck(Xbspf>p(7T!xcOM79_{`dBR?6gmk{ecE5OcZwXc)h|`5l`VK@e?Gv`8S8|Ez@i zH%>zVNHM&$mf%+$zc0DR53Dirj2~g%+4z^`rDy-mTbl#@;22*rp>~Ptya4PKX%*Pip4yD^7GiKsxY~G>;O3QRa%G#vk)a|?izA&&0bSQ-e^ws{r zAQMLlonCg^TNO2e#;aEtARP3*5YU%ZcAfIA?xBqMoFm{NbgBKU{g#)IL;(U$n+~+hbDql`YneQd2;psSTevZqvC6 z$C>d;uP5vg8n3H553xA8WKR-cdz-g3jP8TD|1KdQgMv6(u zH7q83`d_)&;DQ}L)*JXs`nqA^!d#5UvgIF-O|Cn!1Uf5=maO-a(>AD4%E%jGkEsia z^q1HK6jXE5EukTCxx~xVF8#%pKbxo*XVi3({wWgW`!n$6xk;{6qmhJTmgqBSWS&L# z8w2e486U6NK~q=#CA)F-{R*Kv$<^dCf6&w(5}0C6WldQ2`j?E|-)uJagAZqo+c8cd ztanU)N=U-pt^{tOW;F3GP!&9hYdb;S#&~|R?@%1cmTO8=#7gdyPc!^{j)oH~yQO?e z$Nu6JgoI5|;p!Y+r@yqM5xqxCyT${%y?eXUiOpH#FND_UlG>elvS=BxrWi#m&jPADqjQS~gtP zdvynPyWD;#$n2TWOrQMky|a+pDCnUzed$ z!Y$YN|8pvO0XnOe|7&ZN`ZXM*{b!hxte~8vn5eP}orSHc3M2sV|0Pt)C0-gXP~bP? zA~%pVg4v8=axsRzUuksI!$dRO$qO7c8G+sB>-%3RrANJR*b1w}{5%lZ0VlP`cr0Ik z31{|fjMP_3Y}+b!^Fa0TT4^rLWt#~mg|N_QSa1|d6Dd99>S?-`IF`_KiRygTOFCl= z$arQDe2($b09@Rm0RTZ>e%}trf5lkj zXxKPyiXr@)t!vNlH9uq=bj30Y?KSMDjA@8CA)6upjE8F{x>)uaQ1tW3*-A6Z+cKh}iR(Do`AsmWXyoIDmE&6DLG@u%S97_1GaGOlupoD~m~>e%3rhZ} zVR}`Z{zLIBCN@}4&AFy={Z*;7F!PSYrnY)ox_3oeqSN7lP|+-II^~xW&_rGGX-#F* zY-rap%h9&7RV&dU^FdS6qL+Gtg?X-R&@|ou*V%Cm6B5}XL+kE{{jame%fqc*-*t`3 zgSB2xu3K66d2UX2?#kU|SkY*iFXu$HT0@linO|BpKApLq1%_D@5TPl}*#ee@q3a0k zRLkZ#hQ+Tp*mHogD4sw5JinErejb%ZbdcFg8_ZNYm)b`5M}UAP*&-JLO|!(}Xp+ba zvDdSM{ACw%4aEbW&14jkT}{lod~y5FKxl*`qZcDyE^A3Ck>-Pp7>|LQT;1C%h7OCa z2XdOG$?L^b91ekbH;9M|MA4zYuzKX!bs#xURB))UI=XBapM(u=cZNmB0e2h!bZA&$ z)?@Y#lnUlUTv?lJf>wfkC9ieNyg(PvPVC^fcB7Ta36KW?D!p`X$QtcJn-s0vhy{&| zVY2#W-~)|m5Cj5)J#7%eN={EYOeyUe3e@8VDz;7-cboJ3qra7rWwTh$7;PCH^2485 zLbjujy&AOc>=uxzOUj%_S7~cR9xA5XO0|Kil*PC>sXFusPj5>m6z#T1{u57H>)d={ zO`C?eH}2?j<01&W+1n-;A2&AbPWR20h< zHXC@=HWE)Sxj&EyTFQiptA+gubp5QQz%U3YoaxZ0%2y9 z4wuvPonsi`onXJ^SBcZNj;ZP)Bm%l3)|1Iayek>r4|hC8b0s<_CsO9OjFCkR+Ot?gHd?1) zAEu}kVw0WEsu4Xo%VK(Emt2LT;XFR1yM(vMKW?RyGXl-_1|>2h9okVltKt6;$5c>B zT(RlVM2dB(P9AEB4~8|vyQ{fli*e%1&@9S{#p{CRU`!`wN{%vvR!3&n11xcTZj_}% zJHkl}!KxtQnk<*p*B{;br@r8`x{_=Lpia7r{!t)>Dgi5K>L5@FkeD3L*f%^eMyUbS z7r7XBwZ6qezSzf+0Tb5&T}tHx`q4!xBE=#*6SqPLIH|NLdUgC@aHsCBdq=gy8IdFHvhxGL~_%n8|5BI zDpixQO$t>{1zd5%N_D7^aF)`Q+5z%Bae&Ris=98@Q1-0LVwPownnsb}nT>hE0>Prx zARvA+DN!$fFrYGsMK}YgU28oUwgd5IkKcT4b8#n+_y=nB?_~ne1E*A;VNx;6ji7%x z-0{%?nY1w&Mfo@)+!rU1jgWxh{1tY#GL8@0U!4^|i+lnp?d@}~xx1i^;|LsyH%0xB zadi!8N-#4^pbxFHKiS zkTv+e%H>Fr!(?s3V#+F^IDFp1zC7_Y(zL@ZmPnAX<>6R`1qyQe82$KDE&6~p(alT- zUi|7xr(%t)2tB+PNCLOv@X_k9UsHhk*g+q>MWTICY-pv;w5u^j5Y={{$y?iwf zFbZ=#4}q{_Fw?rIJPdye1OHvvH*)94fEzP#|FQkL@MYrSGTdQuq_GL>ax2tNT1a~y zdboG>USn8lMTFuoz%G5gfx(u14XftIrR3F-b)s#wnVd`&z5foA!mtj3$E6oxDz zH`;q{r$=aL%*u?VZx-%PWty!_i(@A@vPel}0-%M<5Yf+cq3a*7z<%V2{Wi%U7$DZa zV>z!}d7SZWFK~WUP1ts_@m^SjBLWvD(jIvRmMh)kCM8mq|33fSs&r|F#D?o%h^yS$ z>vn!otTl?)@IANqVKFND*#ONbUr-sz#WsTy85LZ`Me0srjE2>H#p?@N z@`m3m@&ktl@Q`?iCfz)}u~h5^U&r0G4H=t-Q|At($eiwinI!wv)4u+^NJ;TlG0h2kmK9g&S9RWF~e{=Nb^{c$!gF?_XSZJPur$S!MHU z9m$^kANug-k^H+a7yy7xqW|CiZ0})h@tbmP^xqvuYW{frSD?gCULKGOmk|}GQt>fK z4EZz+Ei+wNYDuEw;qZ%nKX<%{GtYc~4oT;Q+npVH|3kU>b!ltK5--~4VDCf7xQk-s zlT8Oxh(=cRWD0W1(uh)dmePjMDUnXav(le68y~`Zr-ntj{W5LJhSbjRJyFZZ#Y~>E zk>`bd!w^b;MKsL5MoS~>3xAUeC=-(k*#*?2#QTfI3U9(w4YK0;$4)VsDigwD>HyAO z5CbD-lYlA~d53SM>fdjU@+4cEif~Cuj}g*=f@#G)x#TSO1hu16$pbln*^2aZ@j_*~ zN(qIEG}@I~n^O9XBdG-CrciTEvO7e==_1<^5<)m8 zgV$A6-Ah=C6fDb|*5^hBHQ3JEN_|}y&xklmS6mr?v(*_O>yt*~Bn`Xbm01*xt^n8o zO4a0`8pmps6LzAMb=8As@DZzTv8;Ey{s@<^7GXVby*JP1ThjUXQ+AJ*i|}R5 zK)?B<;fa)KY7IQW8!h&TJ2;6y9SMJcoKvcdbL}otRHY3Od5km2V6Qj$Y0p6l*Qu}X z&AD)UF=R(3Hmaf~Hm2N$cgo|PHnV^zhBnzCbgeMhB)gW?D-pPcrfT0x^Q{}BwaZLk zp^P(JgKLmyc7s}b6-1w2@&(ZgStcs8RBR+>z_flYcmVWZzYyTtcK1w){uU6cld>2m ztf!-FHL<8Aaf&jo+HMSZjoxn?eH)`BM3KezgK~F>KkUP?c&r!wSVV~ieeD;!c{^u+ zZC~+oO4;vS9c|v>F175*i}e)!W9%!d6Xy;5LR*8p{jBlf0l8cA#OTBu*e5G`YMK~H zzL^HSV^r-BObXach2kSUZdNb=A1P!7W?ki&)5cR7 ztH)6;7{1vq@kCXn`iS9`=zwraN_)fa=6qV{mKRiU3U=F_RD{>XRrp9XdLUe%rmJ-9 zibKQd&MbNl>}q!mXp9bO%mAFx;G{)TAMdaVaytn_<1#W_hPAK?Ezw>A8!R{jxz@5` zBIzZ;!$^&SJX|!B+IV1{k<~HXWjB@e=(af7CBB0u*b$q?DsljwawexuJ?jX8yWTHL zM>$AOrSMo0OT+>m7|PnACK1$9=jtvd*ICzy3sw;HdT3Jog5E(DKF{yYf%6I)H zg55`mTKDW&g7Z!rfa^?ISa*zCGLW41_t-wfUIuJGxN%hjg1`pwn>2fS8Eauf8Jx&I zC`MqlSRnM6E(rZrNw41lqoI8#bSzmfQ%T8nhqe+DF@Cy@q25+URRyk#@9pCen-C?$ zF^{(A(7%TF;>A;b-@hv^$hU3_8F#^!9-(aNbxq=YI`1!4V;FZ8TH+$9Eg=$5Mk<~_ zU}hC?_0=1kVPT53XE7QD5-RvsCZd+|2ANs$Gk+q``t5i_5>Y%v>PomS-u=_L%nHCqf}9? z!0D*iJ@iBX$m@YplOi7TqpH(^b^vKQGmo+4S_p9SE@nNpCZq>N}4Psap zK{UGmmmeE3o*6S7u*%+k#Sy>w9gdQs!pR~VF3JI;P6DMp>%m(Vgz<`Jj}1jtO&0@n zWeWrdmYKgC?0myi*1)x!?^d-n)=5~|e&x(40_r-RR7bgUID(5LpquHlVt+>zoiA+wiG<{ z$*Q82?oo8UocK9kKF*JEU0ThnvFaYJIuNS5)h)dRyMdNhVvtu?^21St_N;>Y&KXob zh>_uOri0)vs>ZX}Kg3{uB%73|0|-*6aE`i!YN3s-LkKzjvfDp-lHBElmKCO*z9Ut`2xgDIKFb}LGUdncL&cq93cE>YED z5{R8JFle_$I_f+jIP6_9?%8nHiBd2bh1)XwGAu4cCu->XVD3CapGC`J@)#nwF@c@^ z5P-9ixRoi}pKkL;e^Xge(~kJ_LPKGqo!BxjVv}O^Uxy!J5fM9U%$El9^zExP2X0<{ zl`K&h_o~KQ?m_Va^-LS#%!C?BRg)IW{rK&0e-BYe6i#ep3@&g(d%|_Juc)ba0+K4k zCIg5ac+H^mB{)8g*$%x~(@z2@iRESF(XKs&%{#c*rfpwJ4t$fF^NmYyR?p|-{@d%8 z!I`>mP@ntzZBPj6U@WHZKVL1X zo=6W1${TAZeQ}9LyMF(gFS}>LXph3?B9*7$ z^Ys;44u5hUt3*9fgi`OS$lVxniK5g|fDcP+3Gp2Lig^<=y$WI6s2Tb6p7_$V#g%VW z!g5p)>SZsd4X7A`0ONY6?T}fPWvYk_9-3r+uTYuiRzQ6NNa%=WK^rA;vUvC;6#O(w zY^16JQPgB(_SEsc=|V%dOJEb7=s?0R&;;KO*tUMODTKZpw=>aJnaS2=8(kMh(ak7i zm$xgKia$scn{7|x=-Q#HD~!+{6pOFI*0If+u%iFdy32C?dOys-^DlpM`+2*3I(593jjk+-LP9&5-KafpMV)Eejo$m zVf?}RaBM9Z0L=QHm^yIA3guwrUp2S%xa9lrb?RgFeX`Y7_tcO{{QE~?`26J1=4+*G z_9&AoFU8?2-x-|@B$IE&LAc8SMZsjQ6>{DHc|X>;0CbVU%-IhYZNuqAsfwMCzqf8* zK8BqiD=R-GjNA~^_SVkC%KXZ^df!9G!Crm>w&!COJqK@X zN3mw1J8`30VeKz;ub$m48jBSy*Ll`ZGiMcBc&jyvvBm&SIbTKh_X8yPFS+>?ws50> zgVS#&G!*GIa(a>_%yd8}gwCQ^1Yco=s>(;K^c^-QPZK!3g&)_C8#gaj503mO%Ax)= z{oS#(NsMaeq3`97?c27&{=k~|^`I-d*}Q?;nMvWqA7#iF3~Hb;NVR^&x0Sq56Y$%I zYm2?GG7hz}gSUSy3~OmpUmU(pTvEeX>~hiT11HcbO zBaSdUTl-uc9A3VuJYL_27vBpVRK}oDFJsq0 z-{0^vMm2pf8sqqLWMxB9;FqVG>L|+_`H`fsxc+xPsWX>BfeZEZZ5I4%}Rbt0_B^p&ya-AItmH=JT;}#Cj>#E6;?rJ>J?e$T9PT| z=wX();%)7&V{I1rW4^G`k>}0hwqwl6-rk;oj7`I#-XoEGVs4^=gYPC*b%&Reix2V8 z{!Tu6a^D)SN#FYYStL85??GuIi)-43k~)QUS!RksT~KYdR;IRx#bn;Ll8I-qw@>9~ z@L5;hxN>K5--YM{kWZ8DMk%d;xPt+|YO-^GFP|VI(?r!g1MG@I@QPBexK7treoq|t zEpE5WnT;kHo-7HttJ-khQDHrO3Vmn*GAGuBO}%77h48LW%i1U>?;t^zKAU#vbgTzz z1Bd^EaUDg@lYW4^%F&|~O4W3eaG;A>>VBae0r7h@BW1JX$DMhd-nN>`qbh*u5--;7 z;#>_qk|FG_1Ilu3?Z%_?>W}V*erb80%tokVnYy*3#f>#MmTt6Tzm*WJYQ~f=*g3CK z8VAYtNtB}BbOo)y=eJa@UP)JTtw%?iXS+)2`jFSSBt6vkh>X2=k!5TrYkRvv5SZ#! zYGuK4DB_t*MQUn*-5!g_ZH<-;bk}ZssU|1hphiasoGPFxZ$T1*+y3G-5GS`yyK%ZF zZBe@Ak_L+7Ve{lr_QAMI2ssDK#0YYr6em^5q?0gd+t>D^7qY8mPILH;F)De;M6lhC zK>}ff9~%PAC|l{VKddNIX?UH_yfi&j6TmW}aa9$9kGQUSrwlM#%v7n~qm;a1LE?_s zsb>ka)`hmAn2F-lxh&^a^@XHkW_IJ->j?cs!|;%X$s^ad*8E18B#rPewZfz^^dN!4 zwW!6K_8nFbVOi;!TWPeOv8}hgDN+UboPqh8Y{nLA!T1a3w>%5nZ#c>pzo}|^g!1e@ zuf283a1kPq^Js0VLepSh2i@;*Ysw#6Ej1a&jqUhVU z|G?F8ln4iv6$Er++>9AkbHOtKS)SBQK0g0rgRQS zNSlQhx6TPzejf=#`;T4y2rGO?0jgBi{JfFy#~5ZQGCAb=>)&|$>=CQTAfhC9KmvgP;d1*py%0do<>G_TmJ>1t{s;VJq% zS9{U9Il|eiSmY#1Tp)o^SV!URX$h*am4^-i&;>SP`TH94;M&buI_9E2yx>8f*5qvI zMl}S1PFjczhxE|SfVDU~a3nr4()#!A0uT5TZ-W*-CKiB%6PSqhbN4aXVG2DLFRL?u zm_l46S4-e}_%*42u+=}mC8N${pDe%AafQ4KE1ApQt0EPxH*1sMx*NI~_v6HlI_z4( zUlBYK{-BJeQ7w-62Jq6OB6gK?XWfkQWX6zPQzG z*IeG|dEntEoH>}?Vbd&~Zfy_L5J>mFa#-9Upwu@M$Y&PNFQTMiKTEsmMyKuYyoT07 zafu&D65+c}Hk7>Gs>a#x|E#}iZPvdGYUQbPyB^1Dup5=tu{&HTt)y7!jA#4ZH6pHR zDJKpZkE&zQRa;7FWrbWUT*A1kWyMMyw$>6@T^ae@^RA4*Myvz5Tn@H&rS#EB8^#6g zJFSA;?*SI0*MIS*NBn;U=KnQM6Y0!UWBMJ8asOT@|GCXLnK--H(;1nX{dWVkC^=dA zxVdTRskjp=Ntrl#s+nm?#d_8M#o0SHX4-Ap!m(}JHY!fVxMSP4ZQC|0ww+2+vCWE& ziv4Eq-Ea5lKF{gr^tpb(T-S#+*O(Y{4j?B*6lui(Yfz@LHaf;0s%E-IE>;0Vx-T#K z&oK9vmh?yQ3+5)iNbdiopZBlx?LGc^{Qm*uO4Pn4I606$F7+I4d&oj)%SJnZSHH=` zq==>4U?1vIz5XacD}so z0{2<~9Z!I8a^a#dSoM{J7Vce;HNM~v$NJ*W4Q{o?DB<*bAV&jT5_B5%1q@C&8)I!0 zly__z_)sIk1kwEWsiP?|%I}#pFx8@k6xx7S)9Nv!I41RRO=;x`d>S<@-m+W7Tn$a) zn-ZE|#lkjJ8ySi1MB5Y1JU`LMSA_W-kJOVe6g8UK>q7v=rwARJELNcaI(J-wTvF8G zEPVIUQ3NAqTrdK$k|w2>k`Y;o5;tJ7S(}%g5#V6xiQQu#E*P`$ZMohw^<(; zD&*JsJ5Z6<9XWx%+TYH>tFP;H2#H`Bg;)cl)_4$YH~_oh$0GX|ge?+BoDwDkN`uHI zi+$x5V}p34O|o#iOZ(Ga_z|=^rgtX)b*3B=)(w(+L{KzO8EAb?D!lT`~z#1Y7R;O?a`C3vUQoSFRx@+hJ9lxMWup?Nb1{% z=@9faWhHi3zvbw8#*k-rmZthgshrW?tlo6>og4bt5&AfPJ$l~JL<8rKnrDwV0h6u< zq;D^RRwamcM&xU2kpu1-g__(ZgL;0Co}H7`C-->n1$FLgi_dS!QAkN{{NkhJk{+Gt z#Az_To}gxFpih@wvs0r;WS5W*ctgwZX@_}M0i1{v8|VmExFBtJx=KIVVD5P11PpA( zQkLChQZVKQrUiIXh7W)7njdd+RHN*iFM8A-;qfF@?H6Y{KR^Ot^4T$#5w%xs4mXT7 zm0OUHU&&ul#Sy)Mp>^-o&me{k3o^DYpJB+=ZjG`l+IY`K0t7O_ayeT3E-j}1G~VKe zE*`f;jm^R?3T)-wCPoAnOC`RTJldO7Y)Erh!o$38mATFC!=|PQ8&X2^*zd83Hld>Th-9)jmZ zp`iMPRkc*1ISrZsnAft*CKe?#&u;-Td!562dKaaK6;-gb2zb*R%CwXjt{BtLq&_@Ow6-U?^n_}lR;4bP2gbE?i>$qYtW6ZaEl^D(s#gDS0Sc(c9L zHmx46?U+OLY3LPGoN$Pb=Gct@Y-{h2u&f%6LKiiW$>A1}o zLTydcZ*6d6Re<#3fzM4sGyvTfj(6qSQifhjpe#MZMaKnhg;AchF^{uU;K;$t#u*OH zt4pQC2cl$A0pz%AP(W3~?R?RxUzcEIQu;m5h&0b=M`MIYeRb7IULYoEx}J@IR;Poa z0dzH%cY9Z*nu-+w>DYK)*z-)18_F6p`gRkg=~zq3XF2lh5d*#y%lB&hO8B|8eIV;} z%5cbZA=Rj#ikH83KAns#Com9fq5#tS4uvRuXgLo~L|Z3~DIM^9AmtI9CQ?9ix=7qF zhSN)3Q~R5}D>y^d??O#}Fi20ByY^QuD!0rG+Ur*I{-`tkvxQ_dh%b2x`44Q? z_KKSwsFfB7u8udHs>hegnQcGxmoNr$y)JutRUV$J_kV7^|7!(!!0yo|&B6_X_i73YK3)fyD?aQ1^&(T=h34SxPq)&i~`5_0% z?tT(vSm|}JGvx+zM7d~U>PQ%ZL+&*L`UlNhw_oFRDw|jRplk7KRA5E)>fNzI{s&tlnXnS93l(Vq%!C5KnH&0e9BSZ0@>)u{hR z4XALd`Gj7^0t*HX1L*^qgiS9guVL$Mzxxxwx0l zae6_+BV0JRqMzID-!j#e$|F-&Pyhic@_%+?=uc26iU&Ls`%)uZ>XH5Nv-fxLBhlmE z@O>M+Gl1Sz95q!WXt?UCWUz;nQW@ta6Ax;cMn=1k;v1r&e@2o>>FP(T0;ehTs2fQh z(!mEX-{kg1M&OOs2{Q#G+V^8*CEHVRcG1y9Qly4YY4q%ti?C)!QPb>U&v4QqDOXEo ziU~FJ({oKwe^;E-4w=gv1`} zXOp+}_D|)vzjhSdzq;K9F3xsNMh4a<{{xrFiQWYSez}bFm&^PY<*$Ev-+vhWpE{3s zByx9?Vyf-q%0GMu9@U~tta8}N^dodT|EW%aC^=l$lUk-ww!Lf6#wZHC&D;?Fp ziUDp7>sb8IX9-7e{)D0214U@lS!ii}2WCsOY!d9Q7nQevbzj=j_X7O}rlu7mVy?}H zCrju;Hq?C~ji4@5d*_(V`CiHKoKPLJ?^!}dgPbA-eYWB5$j5~oe6qXzk5k{7 zbi~o3KIW$g36v12bEwwB^QE;V-#0q;mcQKQea8!p!=l0gOLy+g=2QaRa=SB@&gD|Z zVY$YR?gE2go|NW%hUamlc!*(J=!(R%$GrfP+lTk=o~8*WZ6(9Xw^Rt_uv)<8a8x|} ziCx`0I_XyO?lXoG6IB{PrCnZ&f;${$slgk=i9d|F@|t%_pLU^PgQ^c<6y_e}07U%; zL_m~z+W%8iTKCIs$io-EVd7O5rXOUrQ|0PMCW-za!lW%gsvINp2!Yl`wjDaKju7WS z=LV-(kcNOQfFT4#gApyz*kPi_!K77b_|!{?X$eyUu16l{B?*5f?OJLJ&n?C_CR zHCs<}7utIRuxeTb&GFPw@VGTkhKN=}ii2qgRcJ>V5ye|BPn9w?MyPo zBs!qqi?K2is#b~OQ)zFf(8O=ZnW|0but)|JhgVa}Lb2Bz$Wdwv%s51DEOn2y?=Qrp zr%;fncO9tGc68EXCg89QlEa~ggbKJuQ-6UfNCk*ki`K&eqqUG)m(C*Wp7yCS3`Kq% zEw$;iNCa&`&z$I~^P&kKb71g`1gj;+Oe!U>j}Uq+HQJd_=M?ZKmJbs}82E2a zNP-~yz+czleFR_q7=it)_yzbLmn1-xD)ML|C>P!?A;kWmrtx4gRzk3z1 zWGF?v*>Zwe@xGZa)A3UI!5{X3ZqcPD!4jCwn}HBh7EUb;N=U#uAvXaU*_wv^!zlGSK}N|IF)!sRoW3Mf zg*e918f+;yWguElf>eiKyQ3=DUpSQMh<<{VhGFo4)8GMe)6iShpxU^_>M9uW7^B#Gjj;XDhBVMG(lCht#eJUIZY#`H9gNI~ z;LM->V*jKhO1UA7^Mshs;KM=WH!QfW$aOFwQ+8fQ;LME0keq!`4d^W#7aap34_+^< zp59mL(1BGrnFy!cq`_da3S!H~G%fQ1&t5Du!Muhcbz0eyz)2*_{T3^MONrYh zD-97wKUETe%MHv$|CuHDy0XME%@^@1tbko-#*R^GVQdcUlp1!T3ma3eszOl>%!b{n z{@QCsH)@f#_Ot74^~X@o${{=WpF%}uhGBTK!$Jk^T6UZ5)FEnmWQJ>7h6pmbg$B}| zme55XEdX12Y7)Jx39t;JCg#j|%1gDByGT`q|3swTgS;L{KH6kl~k4cKqYf+F$l#@#DJj?;eRtX&oqx1)|Tn7}!tC46!HHt)g zDx~X*qa&bAKDx<8L_*0s6o^G`H{&l(F8tDu@I!4j2>Z5 z84^eKGyHLB3z&^y>kr1vuHX5-HhBeGEF9P97|i|j#y6ZU!}Ivi_1vshDp?gEJ;bPB zH5+oqAMDL3@>J?dw#prUDxB+JYr~NF_LzD@XWpBNdqQJs#bFtI?q5s}q2(L9+TR9$WR7>$bEM zUMjU%3{3M0yG4rz$rP&vC5n_QlYOg@$`le;=kOwYAEdFjIVRo?ko|(HIy`fCs3314 ztW{i8XsvKC9_KrqQf)-POHg1>mzA>8rXK4Wa#z(zVYLF_TIeLbtX2P_2Ejh@QT5f+UK_o ze`&uUn$q`Bk_5b)j4Um+mU0n&-}N4NRus*!8AC>y>CU@(P19`Vx$r=-dD=YX=(gP! z{(;W-hYQfbZk1^}orgme;G2(r7THULQQx@_EnxP*?ZBpdiy?Ia>Da9FB)>kn+zJv z54Mmw>`VCP%=^iSLb>rck7Un#Q#W3>O6KWl3*N=aO5q}+h|7C7zWNqhSZ0cxd>-@Lj z_rI)ae$|G*O2YsCQT|nM^1oR9zl^{Cqxk!e1CV1))p@@9n`B%-K#YHLz(2zuCkr!M z6JtFWM{B3QiGMD%ESw70Fy6O%1OF^u0Je)si6WgpbTB|j#>j#KTNAfG z`jBNNW;WYf_0-Cx@K(%n3~HOXkIwfXC+p}(K)sc2d57-#-2qScph!BMxVCo{w5-c4 zz_Z}H%ZO4$)Gd%mxIqVS%$!=}6H5IfWXt&SpIe&VXeLw7BY!>MLE(j5omv*Poe zsqD*6P4!n63bH(}QoEgqgk(C-I{DsI1LhS!eu0X+y($bcKJc~03Z#gtJ$KopRLKCT z-9S8qR30a7rrn_Xb+sH7l_Tt!F1JL2(oes&VfZeRti>**>>O(kDw9;s7f7jhW5b3N6hMUF=*=%rj13mHL;8!W(3F|ep?*}d?>~nk0D!RCU}oB!O|$| z{Gn!Q3$d@mkH(DIox?JBVwu4MtDqro;5)lpsy=Yy-FUg zB*T!bsJ}^462``I9=)8yAut`%n6HYh89r0krR%d<<0a(*)QPaQxF+2ezZgxdTUDP) zC)34$*{q~59a9vHBK?4#+XsuaMR;xO0vzcf(3`n9C?uCreAdlW(hBTvo2JmN&71~x*T^XN_!N~?8jiC6H5Z6f#`ffZu`N6dcG zc3SO(j24x&Zg@gmx&@OyZQ^&l1_2a)V=UH(drF4rPj@0Ok`Rp7ilL6Njb6jAN5{1sJg1 zfW^ussk010c4_x0y|O+ev}~z;?W$>o+Mzv~2sEp4f7i)oLyZ>B{C4mA1}8~HrZ5Zi z#B+FNwlEz8cBI^;N^N>@Rgj36^oAE_*%}f$+5~#0Kk6C=k>OjF$0qS%(gP^A5ZX0I z42>J!g2WA#a`sYTFWgKQKnr@+c+M+sRZ|UAsO*tM_(LRR&g1co6K(ATR@Kl+qUB@G z*Gxdlohe5Lgdwrhh=EZQ;`QX;TZp*A_F>JDzu!tw*r1E|{G!4>X8`z<%|;7v{%UcS zg&|#!QXb0tm}1I3rWQl%W6`twoSKQWdX6nN?(vnvH7(6G^P4n1013}$VuKcOKsFp@ zvZm2>wd7t~W2rT3@l^a^)Nximpith3Q;dM=uCS*!Wto# z_c|K%G#2IEZR!OmK#(oWaHmmF&Wk0I6bc6Y(l-$iv9Z|wflX0nG%;m1;v8@U7`0I3 z-3A$+4X*SD%9=~y^|$fR$E+vPXFVE(XO5)J(-v$Y6L4+A;yHd%P$sWKA644D_DE0= z?h;LQNSWy*5-7fB9(i%hph99tB)6B|+@RfC>(gJ}!^8b^Ehs&`hoExTR7%cytRhFt z5CYSCs#;t(2n{v82vQ>O=)t8vhG!!IfuWM{7((343rfzJsgy9jZuDKVg>68#B{+~- zuly|kmjOR>BVsi#Xfxw5oHTg>AZ(zna|s|s2{uhQs$n= z$8t0G_xM!w9L0nbz135X$*f)@PMksSR1JA}i#q~0k7{Y9Le<9A=ZJPEDRT z#m2EOaSzOh(Mwr$eJw}1C-}`FzRft)QZ?4z&KX#!1si3^+W@X&9FSQXA@<+7zf*vX zQZjS&DJo&jWPe#@($j@&%!b`0CW4KBZz z3VD4yhf5Bt-m$bRLDvpc=Q^co*PmHngXUm}Oam+Rys<~N2VaS8i7tgHYDc-8utnj) zLB!(llR}rCjM1tINTSjrvurH@ELeOgkUm%R#EgowMn5 zGiz}^tzRgz#M{w~C-cN$2R7D4j<% zrLR6jBf4e_y}8hIS4j~zo;qWpxHL`9JoYX171oa%&*JSdBf_3O}wa|{H z&X2hA`?viY2g`neIL%n+ys}}VB6S-Cvmkzz@KQV*-h2i15E{O@6Wj*Q2#S;{DHpY! z-JV6P9kXonWD#|B;SFomWZFf37UQ+Y9Q#P30}k@n@C~aCxB4I3fCIwYt(gIXFQ$t)hv zmazenL`T|bk=M8$-Oh2oq1RBd8MS8@M^gWp+%CudOOm!g>Bnw@J2a-5g`wA%sDjeYuldPK)=M^U)G!xyi9{mypr zwz;@~K$W~^BLT(EHHLRGe1uYG=fRlsJOaiIuH#z(uc}L7tFrnEIzhyt1IF1gN`GUKekma={I7g{B;4M1%Y0*xY2rvlbb`G zw5twpQXT%1k&<$&KN8vQk+-O#P}cgr*7}ITysoR>u*4Lee>M$e^f$=1dZYfNA(wQ> zg5}Jbu*lvCTTlS&RZJOYs6nyymoG;sH;V6aq&z4QCWAqr>wf}!i22xyD8-C0ff`&> zfWqp2&sdhaF0rQ9V8%6cF5zdpx6X`Ii?aCd+oG_SQQ>Uz1YF;_XZ5VD@ zW`F>Ka@RLQ{>on2fNfcyta-#M5O2&K0g^toalnG#<-qj?dMh{i8Tc0LSLS>|)6~8t zTtj&_UMbG`LC`xj5vm7zLqPYDqMzW!q5dYGNi+8*I309g*8#?V;;QIv*732KhG)8z zuy~zpmZ7MPbye(i@A#C%o>Z-W)c3Vtsolrznu^PhgEs7?; zYmu~ld@dslG-w3s(|h+(PwiaJrf&qF+cuWnm$&WZ& zcC@NzGD#!wj2R$WcGzoLl6FQhF-3tUc9Bqk|-wdHWi-mtqIniAW|& z)fYJ*6*6!Qm2Rbzl!pIW+C^mzaEXzJjmhMpS57!oKlYWhVTCiCe0i$}$za-EC1R2M zmON~`qaH0Lv3JL7*ox|~_Z?+&j)(DqSVA9L{3_^&0Qk7Hy*0NzW-;fi4NQ1=H`6=N zEL_($=o@;l669NNT74ebFSH>fy42ggkyWDX3A-|(Cxn_=mT1Lv$S5MwH8vIouZBLk zaNFM_Nw|2P%l+9X7{!Em>EIUO6HMO?5mR-uA(GbHF$e!}=!!+ek(5f{L;R)_llvGz zRKn!j!+W}N@D~}4;mpGg+oBgk9&C=2>N;!Ro;AcuYR9C*OZSAm!v7ILEbMKq-@id6 zRM?5TJwehP=9k14F09BOF~Bxl5EL+Uf_c_Sbc27rOH_b2v1|TYK9mX592Fa<3GO{M zjKy0MDtP9J<0dY!w1-oiLyA2U@wf75%ufs*SbA-hDxEj0;=aRKuN`z3po#Xmao#zTjVYUr;i>BaSj2WWcNtVtpS z$CEI9%=d(b*D%sN(+s+wH6Iuz6Etg!g()*X_=q=mKG2#>5YNT6UM$)La`N-+nP+8K zy}TQKevE4rK|7TMWO!2nYZ5p7Ux^L zfc5??yA5+X>2j?hfQa%5xT?0|Rjh*c;eC-gVMQIl3CvPOOqZ;cOwMA5uY}b?>@sxg zs4%Xo*Nf{dTktXVY~HHn%IxgoGGta+VR{hyi1?_@@8Ige{q;n5Ur49gUY;rA$9NA? zaZbp)rPZY)gFgmR%hiPeO9N5qB;;nIhTg?^(i4X?@S4nSnXt?hV)|j(k`RO!q9rMd+lUj>CT6N!HEo(ia8$tB$8I7g zC^F2cu+{~R;$vOM$hLI?ZAAb0raI%TpkcyOp|_48<|ppkXzI__az0m{k`|M<&xa+v zxUk)sQubNIuKDhAcJ_nDXMkNFAen!CI_L%eTt?FZ)p_RT*zZ4=r&##Lt609ciPFel zC91zFPklZ8|9q_N4V;|Z>>Q2%=DnU&TaW!BEBe&*8tRkQ1y@Mn5HZ3&30YY+3i||A zO>&sS2w|?75=GTWTnzhdW1kw1KiHz&m4wz?C7Ml5WPkA)frbtJsGdwK`y^!CKQABM zcB0bJVNB_wJcG7?Qma-RsM~)9NUt;m4neMJ4M3X~8No_FPS718I@Km2JBko>1h<2hwfM)Zubr7%{?J-0>Q9!t8E^ zEbcP6J9}exq0_w&QQGYS@~^7wH(GaQVJ|Tefr$p(4*Sj2Hv(8c;r!fhBfu~7q_=gd z<1Ky%acTkhFwMH3-Cvk)4her_&>uwsm2OSI#&f8fr?F=o)r8v~zU+D7eQVuH2D=TM z#@O)a$IJMQ5P`4$&37k;urwBN9MZ2{y&dZ(*pH*Fw<&SC%ZVIF4$on3dx76T5vq?< z&K3CI5Ys~tTN%W5HvpTF81qW5P)X3 zO`)wySh5L(tn(e;KgGhpn@pt)*epe}*00VGRE9-fsn>=j6`2lgF|iNjRWXHnZQM>5 zN(}lC*WE43nm9ieB-7~%G2b_PtF zU$eBF1Uo&HYGNuH)M3ZT!H7;W^O$pnet1!+e@)#xSr*B5Jv$KVPI{H+}y7+TBBLP%oy1G?D=-CSg!DL{SyO#sJVp|<1^%TXzHZFF%SA|7s z3-Cq&Xb%uYxjNv?PBqjO*wwUjT;AC_tEj9wey^0JK5;EB+K~cbG@WrBeg3H-W>H)y zkN;rp+;jnCPCrq!N!^EVFN2kPAivmydo3!Q#{4OGt0y^u=gc8k-Rn_YMryk-msNJ_ zin|CE>Yxm_F8W{-Cr3qhp}aDKHNVwhN4-5WYGXKCyKs-Pj53d((9pJl_0H&#wT$~r zZl}GQk7(D&6*FxK$`YsiVk{0chJ;4W+5x?YEyQVn%~zjf?*46V{!QJw?rR^&gKubZIJ7GD}_$RcEewr9>;l+R6}a8^HbN5 zvcGt|B`kFz85ry|%U1yEJ=B>GUTPXvmYTwT0DDRYYf1G3Nr zv1X`a0EaoS!2=k#lql*?nJ~#zV^}C?Y7&I=$~3UzpV|VUD5HF#^3=%W#KgjMg@z#+ z2<+n7l+-ll$K-R~R+^LbnH2KT1~8Ih?oemM?lO0ZXgFjv^hx0g){c>LBoqX}4XzSN zDvF{8K%}vTE5Zr{6N{lK;Yr+|qDk7i6LTW;HsPrEkU|Rn8U+@DH-f>~)5YF^5U>!7 zF)HXAPHsDH9KzV-#->bvsdkyWkrMR_QxU^S*bD`Iwnjst-Hh~4{o>VuOQ!^u5@p_o zjf7vj!A?qq6C@D($WuQ#*&w(=uqxg7gbUV-uIFLD6ZTP(eTQkB593OTS9mvRZ z8N~Vm;^$dJi@VWF*~62AGPeS0$A#_ApwX>hXYX_n!{1Lni5CNk<^G+KXhq*zSXDol z$E2;`?6Kx#w*3x*E*;amzTgU+R5+9$n1{*Xt?z6T4v&z6^TU|eCE_Q#P+XV7cA3QG z3_du0qzs3>z&t3hY;8_|ADFW(6)Yixj^VC0!Ujs}+%Y_!o%vu2t1~|{F5&S06)vZp zemc_3MYB~E*wsucru!Z;>s5?+fLt60`5|Q1YU|cQbN@{@wbjhy$oDLC5_bs)Ix0A z^zFki)ciOTjRPdG1@y6JV^$-LHsRyT1Z%cQEEgnEqkCr@V_L{YLs28I35(rKX=d7p z1qu8aV#+;PT(7X?pWFt7P%0z47%8RZI2)jpv(03_Mf`z?Z0jTh=*WQQT956KkX@TTv-Cwu&Zrt7 zXPfY|VvlVpo$Vz0Aa&c}%|e?NmNx)RYa&*MP7mIMY_6Nxqp@WV(rw*X?hxN6@`mO! zFYF2fup7Eel3#+c4ExwoFkHVOBdk|T}j zZC&)1ZpQ3*@yaT^vMfi_Sdjq^DHw}>x!FrwMp&=1v#nkdYR%uI8t_rdlz5HKN_UR& zBGOdbnSC(rjPGOc_X1lfSXX}e;T#cWBs@#(V+ywY-fY3w^fIu;u6U>0DXX#0h#u01 zhgikDVj>6V(yj3ljFlsij#)orxMlg*uIpb*?+b6=NbC9ixDV9API~A7M>`UNE>(jU zXO0GgLs>5EL_Ilrwd10XCJqSB%mj$eYH}wwyWA!mk9M~DeYuS7_Sfuf(#zYex-3%J z<9WgJJdsZBY3=~bkG#nTihymWh(khjJ8GQUKXn#gTNSi40E-&cVpai*yX?9Pryu{U za{WhL%5)%!6920%^$YQDQXMCgFL1FnGWi>Hnf#i|+T(!xLKmSnLWZHHCfO_hYC!?6 z2(SCpm4FM~Wga}3kyIm_*aJyN$M?q~G)jlH(nm%8EM%g1Nq#cI#6)kf@49?Y9MkB| zNk+$cxj6$&m@)&w0`kD^ii<;|k^|tfXQHC>1emu;Qwm2=rOI^a$_`x%-;frXwh^~t;j z%p}5NNuEN|=(HEiM&KzkD%_%8wAR*DMtf8x4!~6+yfC=PRq10M)q?Urz@*t=gi4Cb zm7+lvIAFlF>V~5rnD!;$wWKG$k4WkFE2W20&Bkvf(Tx&Fo{=+rIR)i&vIRe|}sfe0{ zc-w)NwO}tErZd2RnT8>Hegyk*Nl2Rq=|yd}8lP(?lnMR~@$wgTfJsq6^ZB=@rTfsmtc0`fFknqjUoa!LT`?%! z=V24j!*2zlSzP_sIB5@k_*MfO$FY4WxE|qfYEagL3kW1`?yN1K>04k*0?tg~$iiMV zZs_@xX1neRhp4hRtEjKtu@pdr z_n<|Gv9e@P&R!Sktx$G*c3AB_Wu$~Jx@bFz`i4SH%{?yP!~F5v3J!N@t|>)WgSGQx z?DS{moR+vUf4>XoN9t||Pch^t7{wvWD#aYy-0MPq%PH;)2!W!sW@{V%pe{dloR1m7 z#P1tvprdE!t5fMu!zY!s2;D03FwNfPF8jj<^YIV)wuS#u;6JIF2eTW*r6`(*e% ze}rFNSTFHh!5fS_6G8y2dv;?@w6>CAYqt5@3*d_MqOfc58RBcqSOTun_c>mTvTCSPYB~J&55KV*Ff0 zglvDlUk}262tne;1S~z#9AvgF$A8qa09v*fNEPNkTY$n=Q_@X@t~7oiAma>bLL#Q% z*r9j+ej~im5JJ`=pdfIBtxGb@qKK8uF;=_6iRtp_?I<6IT=S|(f((YaL9QlWL zdEKT##_sAUr+!s>Fvd`KSaKj2`}WFG5^Rlzu-}e-my%od!?FuX+sqlKn@3T*Q2Rs32Iemsc|i%{hoAGlShKU?BFT_p055C!H)lOcS+v zZKdaN<8iS0b=?=|?7bfDQPh2;(1b;)z~rLA37x(w5HNrhupn2FtBEBfq5 zd4mWu0fPpaGaLjvpbgH1%LGq?;Erg`7$RR?HqomF{Fl%0p1MaOXu|WDG$x6RyD$nR z3leiB87uHdj?h$NqZ!z9Af!xaA8M8R&?Wf<0JT&!+x!Gs3SC4u{$&P~3S5W?lO;_U zXjq83zm)l$NT<@ez?-|El4)4AQ1f?1zr#I3JPb^yWQs6fz^v5CjINJ#_zPhGU%N;ALxdIC-p9#qdxjN&m;Y-H&FdvXt!L|Zh)CZ~p0|F_ zw5l%^jt}QfOqIj8+tMpI2*K0)3@@K~2YgXnY# z==W=IcTWh2Y#qu$V#HcV5He>!9(z?79j*(6nO}?gj7iJ%R%T`+IOQSw7E}qo<*Z~> zDE(k>*V-WkOeSVf^zHe@`VrdUyRDH?dOr|AxLc|s#OC5u3QungHoZjjxmR&cBUO3b zNi@r}*eh3NH7zXtDB2}u?}9%|QDOod?qWN`i{va0%BVte9VGwd4^ZuFbl2^(vb5iw?snTU>s7cUpX6>2P1bmI(k&>rMEU z-wb_P)?X{qCz(vzub7>;2j``tTVPX6)GS&ShgsyC5bgtNwZl7cdp3;n8|e`Z$3#;2 zEZa5nD+jjZH`FrJMMe{2ljiJ3_OU=1S=pg z$Jl@o7o4SZ1bSn)4541}VAs2!fJEigqo8lbC*38%|}2{2B?MN~ghAie1sf z^fyTBj%iB&6{Z?nw+YIPOtF}mExWI~i5Z~Iz5fFu3ttDo%z-&hR~k_`DoQNMO__c) zu|RLemV%tX9Vov-d?~*3y=Ur7lmn2A<7A`d5Of);+@CnMB!WH|@&J-57O>lsa8&sk zZ`(JkeI)p`IU6a0eK3)T<91q>%c*1!*@AB@oz}sqk&puUXN5#?itr;%G*5jeAZV`OU-otNM=Vn2HFL!oD7+?^7ecARAHl?6_P zz>fq@h52u9loL6K)YQ?8e}TfjW(fL~m^WelL{{XGy+Hi{?!2R7u%F)k_)tRxJz13{`mze>w4JT-E%A<{KsP&nk%7HD`gJ z9Ho*Mx4{_KhEis=Do~`vnCy{5@>an;*@bi(Vgr%a@b!4~Oi;(EQc>c_t~#Wip9Nw; zZDN+Isx=&6)K}iYa`^sW;?~$djYQRD$ZV)7WuZ0q0Y39kTdJwk8hdFP)Epw}6-%ujJX)^1=d&z!era!IS>Z%u&EV+F9^_YBTQIc$&va! zNs!JABhEa-qo`@EzASEcr>jAm{r1OYyg#8dx%{oG`#Po%936*|HP1avd>Sp+46XN9 za}T7q0`U$H{ZcvsMzdlCoKhbnXE85fCx#-{L;`i@_%qJ=pdYMJziaPaZjP?1>f4j7 zlZ;#wRA+@@(yBy^S;A>*zuEE=9(%{S;(0$ZOxBY;3Vvv6kFvVQe#sl*+vXU^^-#1F zA9hyzZm|)*3+j5S)%g`%J1Xtz#y>Xx8L7lnwsvYeSu{7;+)a`QFn<`6jBZ9a z?cnBT7+zo%ZR#3SeR89jeOiIa@>}t`)%)G{9OnH1E5i9FdQ7d02U#irIJXvb)k0rP z7q9nO>SsYtoma_`Mu-`t{gUma^W`h)H-B90o=6{$^^@p{2(wzF`GCVHzm{dBvdN-R z_W?xyJl{icadAI9U;evTfm?yDpuN2yF(XIMClBvkPK3;)o8Qy|+_`LWu{h$CBVoCI zjvd&ZNlyB1OSICFQQ&Dp1<$9qguNdDc`Mo&S|CX|hCTMK84(dkF>i3hR(OX`n|El= zmv3}&c>L3Ghs#Y~XZqCrNAc5uU|k+)8bVFeqog_>5-LC>dqehReg|Hs}t1?e7a*}i4FYL#uyDejF8!cdd!;(W>OaY?t>M*;9Cajp~nMX~}ohEbJu zJ3qZR5d}8!{RQmflxafUC7IlAhrA_jzP7@nd3<6S+FZ$Fv19vA#jO(!SzG1fq;YY2h6jeqm>jU?k2TR zg;PJwRsq0zkdA}Tj#G|)1eXygD|M|T1E^6hJI;`dh{AM z&}5_^j|4MF=Q4wQFq0hP0uc%ZKR(%)Bf9^6^6@+)^t}WP0B}SG06_kK+~5Bup3-wO zvS;`&N{`Fm=Qe9&NZ;Ar`Znc6WG#WdW;)`px;+x65c2eHjpZ}?bUOnp13F{`Dqqh` z4XiJI1k5eRl4|WIr94kl6I1(aG1||_Wt{VA7SThQ7bky(X;F;qjqJUor^+mp#zs@` z&D?x^P`Hn9TpGI{!^o3GKr+O%ju@W9FkY+VYG;hE1|GQQ>%vas7#vo0MMY_wIGF4W zPZ&ir31){EaO$vk;qsm$rPIy<1DJLe8f83$$;Iacmc#V6yH(}kHT?R$L3Q!rWVuwdO!?-g%FJ7>tcB;z?60s3Z zlWlx)@r`74(~y}qKtThW1Jt*Hx#X9z)7XPK2D;PetgtnXX3`kpHz%$97 zI!ri+*~EDrsg+0Vvu8Kk4X{Ypj*>(WSaNfLkHaAQfm~PMz_Wn0=?Uk_XLjS+iggEk zgT4&F#DRR~Q;lfPcJ25BAbV3et5=wM+QIajl2;f}RMq~HUrAmn)9omfEnuD0wmYW~nK)HJ#*bn^sETWznm(G> z@NY+xEFfT2cGndNQC6N2E@NKc zr%qL{B7dyFE~8Kz<1n8mpvk?4sCO4K1@gD1RlY?Xpt|{^QqhuL>ejTlz{?eOTm zrsn$kx+bO}S`Sj+b}rKqD#K1wnhcq26(L0P1f;*;OYhn}Vo8a=OVYXTcc z48uqRT9yp(MPQ_KSoga9iqVU0wb&fttz9Y zAC=%x4+8_|jk=CYTCZ?5HEXG8O>^^A9?U}?NW2n|&PS7sQsH$ts?0xI($p2@ zcCYWh?AtG9LrxxKfH@8gI)5Y^_KnYnJ3G2H>V|`z`@OlXWOyTF{d>E;^0=t65uMU9 zYA=Ge0uifJ04fqxC}ZGchjiet{d=>|{7ee+ZyRp9{_3ws3s=@rYE4$xbl%gHkBRqd zG+f9+aKe9`)*6CZn@-+EMK%MT-QEN zy58EHTzxDVOB5X~-Da{?@M2;Prd?u4HZy;3-FC|GwPa|kJX^axw!SslOH6(?>}oqG zcU)@EjOMLX+(obcb#?bVtU5>J@ws?xAKdDx(o&7arTyDdQy}$x9+v+LOfKdq<@VXy zZ_4!g<>1DU{LcdTWI~w>*OfAD$sd!Xga)1U?yiijO|NGixpbGJv=7@UIzFs|Jdf80 z#){4k7_2{(W2dK286&8h{-~`zzX5P280wkia=Cd`%Z8oDH8nxJ!2&QN>Y#O(@#Lo$ zTyigA{QLvE%PiJ1#XUSc1|F?u!;Uj1m{KB?!Jy16dY>Q_ATLvAIDRpsWQ18$Vv;+; z63PS6j2d9i@eN^tb2yVV0{;bkY-0jkIho0{dV=``S_dKQk7&k~FNgu+7WY#(KVXP? zOU327$6nRG(QOBDzaj>3X-8}lCMW@pph+H7!qXbyFAZ$J{xlLlpfmOK z1g}uT{a#s(a>i4#iEL}p^{+pX8%SuB9xzc7ScqEEO~$MX46JQ2i8H)qqDQv#=jBrG z@OGA&V%+k0OO+w-cq7sFRh9pv60ti%lC+!D*I!l~_Lh>cbciar^hHR3X^MsBzt(Le z5yxq|%hgVd)0D?JzDAt&`h+@;dORxpLSHaFT)iJNq?C#>tAul7mn2(eeWwAh9d3{6 zy)FFf6l^~(84-GJU6ndN+h%vjo!8gwPmetrkO?h}fpyRvB6nUbeM$(Sk2q#{DTADg z%QqgWdfMM1r+D0Ni$MjDxTR3pbg`mUPAFccVF9kxu zcXSUkWrK?J-W?wT21VL{DJL9)2_OGaJAWgQp((U(C5$Bby%^X6a1^Skj~qYPOi&>D z(*EH~aJp7xJdn!@1DLc+E`s6-o1EFYh$4F5Ot1ZK;aVTu{@fJDX`nN0_UG5qh3?$g zJzuXX$wOhN2*GQONEsJ6XJo1jGeo8uJ^lB77P-tO(g#3URbK3iTyP`^INTfd?1~2J zpnFpI0}n1K4xDopTWYD5@Es87{qTokFjnr1fjrJM01ew8LaML@1uBw7#T#_>r!k3) zb(5XmbY&^U;#Pgf8MK-q%Hm@Cbr-k>5(LYaM28@fM5nXN?2he|9D%b}*z3I$3C4TO zQnK>Gey@4JFZ$}fHN#s}yl{0LyF^9OhhIl)ue#JlrNR`V?Sls}ssVk4sl|aW5p9&a zU&7!1Wu&v-V{xQqb>yo`zI3aIV|Is%m21Y;h$h`iInqFg=`*`+DJHjb zac#J8ecKo#0w3`4g*6DfSXUd!O4G$v)Co^&%Z%!qh{G>=gaDbBW=$PYMt(ple9bcJ zx^%8XflO0@?M&RCeF;4ePtz)-i})feO411qf5Y#-MtQOIBMQ~2Q{cvfkj(x_*g$?w zPy+d07@#2iA{G?P*JsW$>1Zu*jlIh)7CJ`5QE3Fhu^|;4RRHr982qL57K~1~>hQ=( zY=p_Fj}Z*?O;Mg2kw%R4s_gKouGi1VVWSY|5I;yI{p3hi0qj;EVs-^nK|{_g_j5HI z?))}2Y3|>Zhe|&v=xEHj*&>~IfYqXa&5Wa?)tUqNAd(&2cjK*X7LsAFysB(J`n?QN z5PBfaNz4%8GN0{$F_CrwX#plV+=mgpcfk`|@zXxx@2l1Ql%n2KeWFc4O$M2GLwW zyF)a;wDDI_OB6!g(V0CDJJ6xpj?jQU?E6OW{OdP$5(8g4kw`Nnd)Rxp(aQs{zD=hX z+|t4CH!_MnRH)T8o=feaQY&E(Fj!$s&MDt^#da0T#wNuIiPa`XY#O!WUXKi#*DJ@< zNkx4FWXOJ`3=oN(v>NIc zOHtC|OOHb0gx*N-@8lm}mXSnWB7Yi+ckE__g z3)d$1!KRXO3SGAoJYEs{EEKeB-*xWF~# zI!y5rBZguN4MXm;MY3m4Y8Q_l2`YNZQVZ=tK6K?SHQu?3K*%Gk?gUoj(Ti|8CvD_+QlxSf4+_ zsVA4OaE0<;w&(Hs@tuFgGlh}n@=1t+>N{0X2!51P6?GcZ?1|c!-#`4Q7X#y3$W!O% zw|)^7&FPNkBd^K#`ukulkqqq%I<-vZ4br;pCM)}M2N#hoF1r)<%u4oct;}rNLj$Vt zl#(X4LNpNJTm~d1(S4J%CUYCz{p=x&90oq9u<4QTA;&JF@fAppN9y`e3X)m>Rw^+yfvJh;cRkP!Z@ZCw-_LTs!m#t!=>Yfrt%58KEm% zZo6IZ#5k}&3O#&jh3sT9cClhvzApqb1;p!wnmIN3fdvTMPuhk0fU#J_xHo{xjJ`d8 z-cKpS@sn^yNMR26b!h6NxqbY;6$v&-)4+K(H3sfiHSs~gCy?Q=VA!-U2}wsBFb4w_ zFy@^@p7Ip76AROw4X3 z!hp8~P{d|!g%K&~2Yi8yZaxjG;qBD9aG>Y+fz9X8*fXU@`!cox_VL(nW66Q5!S`>a zZtz83Uz2+2?qQ0@fE8pgWG-}H#uU@AV=Re(leq#2Cct;@%44s108k7a)Wc^!(1v(P zpboTRK7qBMwq0#LZ4t2<*n{5~0J5S($@=&n z+ANxrk&!h~Y|z?D(SOgZ&aQ|CJTFbv>~JmrozW0mP64Vus+lDdM|#TymVKJ|WSM+M z+OD9{w|#dTC`cx|>2!9p>DoDIJU7bwVSlB=1Xt zXx(D@^_mr*mpgiuf;_)buWzDN zOC1Z~oTFZ<{nWZZV_a=J7K;yTY9m#Uo(xX!D!dwJmmD3wl?mPyuG4 zz+{%>P-u2XxkxaeFW!#fqw!=<3<`t-gLr|2ah3s*+53Ek8d{mf7~UMzw2yXfJh5qB)ry6T>OFFeYF5Z;2L8}ARM)lZ z#?FA*lBCdiWA2jmnNtnYD6~l_Vu|YrOOKtiQ%@YHPnac@scVJ?{!75;8gby8nW0)= zwLx-(GqA)734jze1YOLX=tddwkb88N?~G*5^Oi&g6Nz&)D9G%1Us`EPL5{n$<;P4- zJkH2%q4L%oC{*)KuNF^V0RdIXRE5@KmXhZQ%tDLWU6!%wFvlco|KCO=P z*!T6f>+(xs35X)p4IQ>{A%yLq@Yt|$ffcQW@^JL)ojn`YURmZdw`^cCK@;CD-pg60 zWdt7lwVH|InBVE1@kZLpsd%_Yvec;ro77At8!YyJ8k9romESyC1PM~fbR6$9RWE%j z-KO3J^&U~E3qg&)2+g369P9=an|ey|2>$4BAMGJdyS&s>d=k2T!2tw@|+iN@$>rc(LLLA*|XmghN8@v z0U*aNXo^Sy(iljsnMH!viyFau=Yq#$4uvDj1wid|uFzAK{ zYoVz1`DkCv*$t(DY;>gQPpq)ORWY8AXJLJyV25gHW@kn8KljUb*#nq;v!Y3jbw&<4 zO^z4&`(EM7cO+ETzo#1ToJP0$a!$kB`5(`L>1wD0J4Apev^I_qhGM&>8j$WX1jZWQ z>&qFHQSXgBNI$aof9gX8o-hR$ki$5i=(7(AgD}Xb{7k7Jh217BfT_m!<$`xtG077C zEZ#xEGnt0@8n>u(#1H6Qm(`ZWil{YZ?uB#xeGI_I8VheNC^d(cyFh*N-W$vn{8iPWlo~# zQbKK$->ugaT6)SPEf*l(?qLCF2y51qfI4MN z9CLeJ`}zz8@X{@I6KBV8BW_Uj?!X3fY#0W<<4^>s*Y35W=&Ee$Blo15i$JDa?$nyg zAa4&mptCp^Tnp!>jImZ`)Wp6`hxrTGlnzNju_jLt9;&auQaYTUyzkbaH6*&Ft3IIW zh&J!^AWedw9Bn417aKoEv7p?DWWJ$sl=ml@zaGcUT#wL{Su?}5AQ9(Rh&Z-XGd`!xDP2huw)mh(RW(h-8>GZ5{dyBx_OtJ9?6leN*avr2CEp zwxmUabtfD4)K&1ZQxhka0EXxiqN<60aMV2vB6?2Nip!jf3c)OS8b|xqc)t`*fy`J0 zt1Z$Y#81cs)sQ18&LFkS96^uTltK-qydo;DLy9M5d&q)jc-D28HmaF5zvtS_N!SR~ z$NAK1CCqSnLovs|j@+9iOVHi^FB?u<_cls^V~J=cCo*Pm;{0J$*&s)BxRHYRWvo^+ zJQnu43UIP!$k(^tu+J)3*@fMaF!bn5^@Bq6=$!=lLy1G%0bX;?e6#9~`GyE7lZvct zD`$6ohRc=4!Dzibi4yCJ)f=P zXiQfXb>;r~$#n`gn-Y2(F&3MWi4L6kjy3+q)}xRuiqd&S&$`XN?kW7!>O?kuOv|7{ zCM$8JeA$cpdFOKE_k0IuyD#9g0cKEl5CHg_jOd#L*G}oZkx$~159;`;FK_HQm2&%3 zvn5((3n&lA{P=g3Dt&WxgDu@bq=99HAa zoJ(m=)rqC?9=}RK*(R-5QBNB=Q8_HKG!8n7@+&{5bs3^)M|(kU+MuRdhXPYeH0|Z# zzNtMabG%tHXwBjvH`$T99EnH0wFO@aU z_}b+-=eb+Qf}B7m6P(m&#ptn@Z?pnhcYD&nso-Fe!W=a@a0jX>3u#ru>+miPOfu`e zd+Rh%JkWswTt$R}CRAYx42lG0gI~TI+@3BXw?Z9JK*-ed zQp$WN>h0ZwAX%AGn=rq5`Q+_#QdzWGA|J89F0*Y4e8*r`-=kI9>KF&{q%N!H-TP2o zXylD}{a*J_-MD0Gy5ym<@R%Mq%S2Yt_U4HMea-YdwcTq(l{luwfHv+@(L~i9zc^b_ zdR24&=eI)$+1~u@o0HE<4xH%qGw+@xWRm|;?U+sG$g-Vv$CKv& zSA^8RN%f2j82?LI;s4Tz|9gNr{(m%t`^R$s?fLS*Am;yqnE(G7V*a1*@P9%NUGy9c zO#eTg9jgEA?EJfx|8s$B1H#C3#34tCO8c5h7*m^XVK{BAGfu zJ3*#iVo?#r)VMw8^vbQB1@jd2^mSa!dYcYNk;O&4A&+%L;{-G^0Xf9|8bkx6w_dwO z33tWeN#XX@>(eJ94Z9*m2Vv4KbO9}psrLokn4$=c-gdOr0$T4Kw+=AKj@5(sITNT^ zg~(wF{q}f3OWu3hocIRv5qC7+4iM^#I=9)x<2ev09+EWQvK9b;cRB!k3?o!C5NbDr z6;xRN)%b*wE*Ef$y#n=KTM1O&x8Qe9JSm>Y^4%Hhy7a6ccM!3u` z_B0T61&}<<*Wj-+H#E>d7qBIH4}4>jsMdD<@k@@xaq{j&-C({bwdMs7iZnemFggOe zE+afkZ%+Gkm@a}me_gNl>5soXu{SLleGR}j-*$8r54G#>6(WcQu01@dvEuk0fvf%0 zb}EK0eE75%xQzV=I84i_M{-nGShJQeF!9O6cTb@Kq8v9LKn9Vftw{2z;INRsKyMgE z#QrcfzrX8zqzU#V0$^=#@SysemTdYnyMDYKD}KUknO#8?xNuJF`0dH>YxuXqN|{{k z>Nu=Tb}VfS5aOY=8CNOM5@D<^5}OCTZc(uBHf(rOWPZ%3FmF4zYWQ3=yr-#}x3`;L z9DtL%BlyQ}LDm434Dm`sx^KGvwrBx%*D&d8VcyYz^HD7<_T^YY{G=rSm2py5mM5JR z4p!2%HA5){qX3a7zz_0x+0lB%y@R@@Vb1ov zp}M_MTW!UE7R3Mjozcl-F<=CGtwWigo<-_un`c5kiwPefU!|Q3QjdM| zyR0wF>J+6PpiDZ#p3^@9@9LW<)+amz&W~5GH_awXLP^oH5_hCCtN_`{#uy?|jV7^8 z^(%gSX+%avCOzQGPzPM+sCxQ}%P&8x$#Spr3@`B@%$&BG z4_@0W0bDdp6z1VOOjRIvBA@+yn10T~`@nh$5Fs;fEbu9x3#B!m?S3YRg-KjVSB`6n z>M!tN*)fs9i9GDZD{aBN6cqxT6nyMKl}B1$pgR#;huvh6D9vlO8Po@lYE5BciXNM5 z>Es{SZ`m=LgvK#B=gYN4GXR6$Iqh?aAJsi^=#Wf9DCte{oM~My+OPm_S~KHp6sm)! z?5$o?&8X^ZCU$&JbU!LsI9vG9^KgrYw0Ys#yJ2nmONWeY?atOPY+X|w8)GV_II&5` zcXIe^E0Gy)&StQqD=;OsB~mT$m^z-_8ga6^EA_0kTI|fG^;cT=2!S=!xD?Ns#;GeN zII+wd@pNzPcRgqDA!oUBCWgcF`noX!#4+SoT)CWObe09jReGSQb|}eZoa@CpUJju| z2_iZjVjyz-Xrh`33BrA4grf=4`&|KOt(2488BAFp|57DD4v0b+H%TC01KHWfg0JZB z!3Fj#!8Z#-V_MCq29l}Vc#6e&U@oJzF7`i+@i&G^)A*m}SCWVf74phYBFk~)oQA74 zy|3knM5JOxq8G#J4F^Wh4BA)rk5~?skNC?@yQCzp>z+e|Lk4jPK4b6c@}sNzoi9Ka zDu$kb;nQRV7DKLOxfjdFfT;r#tQh$ylM>6QON0+pS9C5xJ*;f zXs64JKe5T#)oj=5niU`mT+EJKN%q)S87m@maSD?5tFk&H4%E2L!-x@pC5s!ea_wS) z|7D}poOs0Kp~JBw>RL35lkc5UD6x}>3cU%Oz683AD>YSL%~ho^p-Ale1BZ$dYuqwA z^5f!&6^VL}LFVtWxRdg>c5{`how*()jKrGpoE*dla<`kMa$$|4Qpt3C^d3FlE)mLld*8A!a(>GeCUA z{ec3Fr}`tTl;a`OHAlWA;3ELPkkU&Y-o8kxefL@@hAoJ@x+r|Q-ZO5d2|rMAzr+K( zU!wz4tiUps6W`p9#v5GbT{1&&2steA>>})pAFY_w*=CCH-k&_-w6ObhcBhTy)8S@( zGSV7N>@{=(hG6Qjw; z7?aM26=OZVjF^>v_}58rR1_N1bXZQLhGrmR8fg0F+#Q;;Ia_3FY5ssFc1x;GbK6@} z-@{g}sj8;&8ScfvV{(tq^P)0tX)Su)a9kJ16>_QsC|dWtpOM$WoNMvUa5p5H1YV>p zj2{F(@4$o=wUFQ-B4oH|%c~Yak^_&v5^6T##qlwHPbynk3xVO6+eZ@j%Jl~8oDA|k zb*y;TvAil;_IIp69NlkzxFQ!NvkwCXvRsv$q=@~aEh#6ujHyALN|8#Z8Pg8StHpc{ zb9RxaR;X&lyFa*n`-oXjes=EDH-^?o?MZ(+~V1*c90%j@6)pd_OHw6q5-Ru=iT&g@Aat|jW5F}iR>7)(beIMS8(mtI3lCWq5NBtsB|N_?PZs| zv5E#TXV%s5McXDuyj~tae>mo}Uc<%Q?rQR060bao^C@)OJUffB)d1scKxAEy>W_zP zF>SXX_E0WBT&c46s75}8!|Hs3)bEp`KV~1Mb~~=oFI&{W4I+^o&m^dW!qc=4rliW} zk-tAj^;eME(uU^<(;{<=e?DdyzjpMw4vCk;?^W4ptxkWS%c*3^Rj(T&Q6%P5A6Lnea_Ko;UkOVomIoMXY+3OJLvdtJSE0|_SQDC{GTu?|9zk5Kg6E>qtEk?CI3~V{;Np+zqUx-`ut4N{vlo< zApijUryZGp?DGFvb$a*%zSt8*_{rTuw<`x&wA;Gsr%>c&-@#S|B-+<<7aaM00ou%> zbEbV4XjZ z(3bda)#sto=nN$j6<@^e9gkWag3q_|$&^Y5JuX0;7%?Ihj zIJTE!$U*}jjcXXk!$Rgt!Tdt%;@TkFbL`3rYp?qTx?!)gdO&S}_Ith$9yf5Avmkld zXj-59x_r=peuGSrf53dvuoq0<1hAcH?0m7qZH$8a05LjtU3c1A&!XA`{${hiL9m^0 z+zI`Iwj{hV!$#2ZLt7$f&BJnhe>vH31*S|z^Zv0|LW_mMv_M?+tUwCZc(B<5p>Km4 z`0Npe;R?od%v$-zZRBx5Sw%)#?0ENqgQP$FleTn49|2&kWw^Hivw?&MzP2@N~|& zb-thc4yvgouQE+}KhB1uQL^29XtSMlD9h=9dc2-NT*1MyqEyB(8mruKZBr=+8>U^j zLw*-}zRx@8NO1|RQisE$DS!^77>0fF^MS$AP_Whxm?fPd5Kp$fBeTCOiInb?E$+}t zWDhDcXWQtcZi3LEbQ`HG3nN7w?G-fd- z3#z7bU$(}*Aq`9)mXH0m8)p|V6_`2&*6_x4G}9S7OSIz0tuW=1DfADoBugQEo!ak6 zh2e%#*BKv46g_ns_S{n_;1L`ho4(~AwLv9~Z0mR&A!pbs7hiJiWj9NV#|k`e@6Rcu zAFqBlAqHLLdKjJ2Yp*SGpQS!lp_HDtcZFG)xa}*2FVU2K7OnFQtFjX~=lH@Z-PbY2 zvddk#<81pr@-e6%MUr1;ao$SxY2Hb?#vx4Mm)#ui_54becNbVzIW<*6pC-HGZx3gl z&D%t<@|JKa3~( z$2;YJ_U}EZ{%y0zj{2R|-M4HGl7wG0Vcv{WMrLHn8I>#u>Ub>LiO3iRh$n%)Dywl- z)BSt_63_oPt8;zTU=6Nr`@MtC`|E|Hq>yf|>R>#Hv}z~vp?woROXoRX>Z%P+^?)e} zvM9Z>QI;<|hwid~=-{ry&b`!QkyHnD#x&zhI&M_e47GywHlYX5jx9VCQB9a}@PG+g z51kY@HL^`ni#U*~UO#@a#qcM#P-BG@dK4WI8mY2WtDbe%f5uQsk@OdNXj?J&?r6L` zKU5D<;vyA*qJAEmG9v;!q>3a+CJ-T{NbDf>ta8C1XiO?0O{adY596ql>{Mi4>}==_ z$PGG0@ksD_(g-tH9tk``5v-sjrgD(SYB5*+bmU{9kCXKg&^C3AWT12o_}|zJIw96AWlGvqF5C;RwkvcOCzim z6Oq^=ZWIKDso)oscy`?@9jYm|z->_IOp&-H{idI|V02wsj4@&fzQFNbT3U~#2C-nk z^b(p@oMZRQYvOMO2>q$Mk2-`{{Ov}RG-1BPVDp_?^s?9?QyzUyiFvLVY0;K>x~mnU zuR!idvFiy;Ha+&gaJbuZJ^)E4a=fh) zG<`Th4DF$NXx(gwmAb##ZHEtb?LO$!U3Ugx`QblzFVU&pp0{kze2vhFwrXww_!GEX z=;^^W#X|L^dEIux-pzJ2)FBG^Y(L*w+w874r%xtRBl*282R{66L43P5+>B9nv|mo; zK4%elu497d8MvDXYHuA&J_mDLt@ffnA&+*Px!Y|Fg7MLEnHcGFIUcCtj9c3g@cpM_ zkZrDKqK@GJV3lCcjOE=0nBrk^4OGze z5^1toVl)g0@>Y%op+i7<$X3kawMwDXOzX+@IQQ=L0a^_v9SabJdx%uBDAb9o`kDh` zyvkQrheZQ=$PA&8doJ`@OTxivR+^wwiC>Qwu3%omT1KGlhy3ne{4GU9S>+#E-wln? zh3mM{*h_r%^J@c%G%yub6v5yrAO;Ip`E&o0V8t=Bd?cM`DO$&9B^)h=Ja59MbXHsq zzDFF%;wNEqn>usiJoL_3lDur-rEZ^t0H`6FXC-xLF&>DBV_f{<_W=_nz3SXn=EAUj zO$UL4nDPZP<}Q{h{;i(?5TMO>Nb9q1&=*dxo*zm=Q#LiSEzx2~i-;9}r)vcnDaD*w zo8=1#?x4uy)Sa{(xZAEYT0j$a$|+Z#G$cIZHUNRGc#$@0g@ND-71L!wbxsaJn6H6U zY3y;OR8@qc!6{`rfuBz*yviLpOn52Uk8^a#egw#WQdyc_wc5PM{G@zuI@MRA|F&Lt zAc0C$a{kK^(8Gal(4y#Dj9~xENqInht^FkX6D`fR;b_q^Q4o%}Kl~yt&}zpQmT{Ej ziLkT|nJEU16n%rU6`b?BkJJrZEX5k5C{M&sV8w## zEg-~rvSXE4)mt@pVnVs%;Ll&f*9&!H4;DNRA1n3FZRm<;N%dy^IK9$c^hQwTfJtF1 z3K=OR$U4?xsF^~I>pRivI{C#U`zRXWWL=4^Iq|x=QevqK!@@NO9&g{(lxUe-r?%K~ zMWdav3H8qWNv*7-8&iK&40sq*bC!gaJmWYk4iO+_RS@%wE|!Io_!PQ%T44tTQu($Z zA>{lBt)~d3F%FmLRTYbv!ryz0yOx5z z!>g?reI4NFEmX19dQpFc{x^wf(P;UL^>6g6%N9*jiA{t6mjJOc%XG7!*isd4hl+{C zltxgC+tdDB=h4g{0mZdBxIt;e^oly1d2%y5BK#tU(-s}$`%Ei6M? zitmdeLoD|uUVn*sT7%Dd(>n-DCRhtRMAY~uH;kv+Uu{nNJ5&rrgnAYJ&Z<%NH6pYt zFCP~z*V|?TvrnM1YL{U}#0hw-aCp_@LHr!3@2wR$v~)2Y<^|4ZU_7{tNHx`um?zY4 zym6PpB6bnDq^D|#3e!Y9a`@XQ#EXT*DK`*J?{9P8K_DVX6Xv7W2%49rv=Fu3>6Bf$ zNis&hU3RT!+O7gI>tL~kSP^-Wk==`H8 z`|O4VbK~6aYqSV<;Vi$_&|JR(|M7nLpU88p?EeQ619+J9g6yBEtC|1+0Ofz!3Hn*? zpU)v9EBin8Mh><%)(%GhRmR9AmYKtk+vTs1NKHwxr3s=S5T1PMZy5W^z<#O&_R*!p zia1R9{D9MBeWq@2ofcvu+JMUyL$W-ry2Z~c@05}AZFwSexl$(VAk%HmAO0alu1g@2 zjsJv#aEO3z!C6n9%fPn_A4I@649`WY8`X_DSNG*dFfHUG_E2Vz7@QDM&xPDftUe#H zGaM{A0XaIpMc`@dfjm0?&(EpyU=RX)1JTznLVVedYr*N*bv8}3cAi~72sCk5`0nUPs9i; z>AUn0a}4}>?fDUjs>jRN@w;c-C<}adGlw=B2@t|3Nm{p~foxQX=$G*GdFe@k#^HKg z{g2V~gFJ%*;DJ&i4!~MAHdJ@k;g+LR1_;st{N8)n2vH(Bd*KCRIg*2NMqGbDwy0Qo z^7(Q|?}%9Bew}urOk;<-1g-(VAq;S(x|}t3yr_GELY-G_ZrW_jh0ilZv!A(aO{8m8 z&pU2xA=OUz`5s1uzVv4}_sD121@6bl?lmsy9EDyT%#AwmZAjXRl}oAECqIeF^^20gRsRJIrb-7Y+U>pS-`uT?Uh)? zTdn4cu#RpTLAQpV=~$i_c%H9+HrW4b}Xc@12t19w2lWrWbz z-8_afjB8;I9YVNs=>d`NO_|9$KEmjd!Xw&-1ihQ`6->Qx42rKpe5sQ9zqz@YWu8C}?VsHN{dSCC z4r4pCb8QIT3;yE=JiQ`^8UCd~N^6WDf{Ux9FNQj50RR2# z$3_)kkkNubemcI64BEb!`c3FE8M9X)=w-AzVoW#gK~pFTB=H}Yc6bC>N#P+2!EzRy zm<2`11!N@^$|8FajyJ6&w4QoqUv}7Kgw&w4emIFLlRJd-j$g5QVEL3a3i)fj{B=AM zmAyIT_39b^jsCm8oXoch@ihKOkP8LZ<4~#{8XAxh!D2LEydFihr|EU4*fEi|FK|v8 zp{|WcK98{g`Ru@;scY$SK@45fAJX6d-B_kSWF30IKC@9diizO2BEN*SwFk?+X`%Y6 z9-BWDN#w+#)yPiAafc|Pq~i(zWp`rHQqh?zahCulwA(pcfOCh$^w{bSUx@cHnY2Zu_W2Rf!~EYA-SqXtDTHxTkV_^0uF+=eWq}I^}2q zm3_3PQPb^MC*r!yLZW_i+??ASI#dLHi}Dh%qsm zy1-2`&3!`7-w@PEZH%>o)DGk{YK*GZu<&e1C_9mXAoVFp+QmLFA&SaNvHl4GqrZ@$y_k@lY41sm16!+2nYEJ;uUI{%&@`NVA&G__l!bJ28ChNwQQic1KKQ!e5aBh4O5-9cc}yN%*%7|1u*nSfKLG za-bAH{9p#DE3&_R6Us&2Z*C%qI(v2RT2x>Gk9sfM571Y|viwYd zSLy+xbl(Pih}}oHnEl3dnPa{HGe`_$BpZ z=!ea#U`_iXpTZnN$DLoaG{D+6$RO-5Y!VGf@?+O+y2qr32&)88JQLWlt_xnJ_&bPZ>+3yPaC9 zR-^cMWiWQ%Qm$6M5E(V>RTww4YPvTmt8O?`QhjZdQI9gvWsXUX)zAf_@zx}aN%ARD zQhg>UX_&2-RX!+{)H4rvI8sunLJeMLk1a($XrX*pploYrw^S=XG7#}dky&Z+7Jjx$ zVCf!;i;+mFL_;0EN@P@GpHLVGf0jfcPD)2k4KB^~-QeIb99jwU=2*sv`9sYEW~V)h zd0khiLY;q?I8V+5+s-5jVU@w`JpC?tEkt)UwKdohyaiFPgd{P@yLh8K;7HLvtO6>D zI#okjYR}3jUaEFU-;fy`A~EA%q`ioyk|p8n2*7(5>Tbtm}njfH|0MaOLsPm!btEuipq3@%b& zVHyJ;CYM8(p2K@`6cX`lf)xz1FJd(xnp(i2$;k!ySbP2_~4 zbqSpJK`Rs0O;h-()!y8`F_m?b-n-=buDHQ#{w94QORb_PD>=vT!AUJCSvL_vlGjfX z&&b7}7N0XXJ(q)#8PyXEvxX(v6+ar>&>*>6*53epQIBWuzC=lR(3-mvZ}lkU>F$jG z2!C0xOpOa)xerKzgNN2V{BRLH7;+K^sodBcXI=mK04JS&%}h@O{)-@A2JFN*ayQia zirXaz(Ir;sv*&1e*>y)L8Q>1y=)}=|*N6|absMkO*}j1Ed)9$&kMEgcJP+={RH^5{ zUsi|ys`tH~ZzCoxx|lyP5b@FPbV^xq23E)08Vt_PDp_0#nZ)5_GQ*XKh_Rc77}X(k zBPwi=4w}=uW8h9!oVz&(@EFfB2+>&}i6E5+M&Qc>4Cqo9EPXsSW^nEcZ=$G1O>ac|(I z#D$ZY&mU1mPW7ZH&(C}R4}0$v-TStzi^g^`VmldeGGg1dZQHiZF=CI{wr$(Cc{A5M zx2?N&n`_T?cRLT~;XLHExBTl@RlUCIUwy^8JSs&iwvWsP_mPT8Zp`9g8pZ_gP!qoZ z*p?Y6i!Sq+tQE2BkqWUp|JjVf-z7m)YE>@YdTSmpZWX?}`z7baa)djk45lzCL_C(vMG0k?lzYE<7%u`Pr?X%G)Avt^E@p)q0If zDwP}}o7zEmEaMqL=k0EIXpQ3Aa()@_rAED7UVWhcQTpx|8$GjmEuZe)b_n?K*7t&_6BY= zkLQ_I3KiN)p?p4S7skCK#0X64U&nR_gqE|=j(t;cyvckh1;coB`klM|47HuY{uQlB<> z)KZwMIx?78aZE;W1RUHE-AHNa=2~pr-pAZhqU16;Hh_86FvCHIG424vb-S716O95& zu&9dKDGz-6bT)sy{8%Y{uCoQ9je3}lWi$3&#s}D88%Db>U#RzKPA9K;s;(P=&+cU= zgL>X*H^u6Zq{55TFt83vqST6_v~f(QN(_8sjp(XM50%6yK?@NF;|Z`5IH9Qk>d1|~_J z7FBj37Xraet^W`%F_O8st!?SE5h|x;FMg-W^ddw)<3q{~w$%byC1c$ZbJ`J82PVzm z$D%NM=xnHB5kC^Q=b`KD0J=?8wvdmkv-coMUOvi2QeW_aj~+OUYP8I^K#V3FzMk2% zE9va*?kwbPFiZRAwb6Ku`gP8%q+)${&HLId4V-bsEm0=HI@@d(ja@(GmSk7fgXq>0 z_WR09-?p*#5hoKvi^4*1r0kdh5y z8Z&<{7hQq0T;8S+898Gow1Q;;zrYc@1uvtsT7P!b+3|0{JQzzO#$HR=C?SKX%B$%q zVlni3Jn?d6YN3RiyPe9*Tx&BqQ6upqU+n`kN9EQ4Ebr`yXH9X34c@;7s3f6IFR{o9 zQRNc5?J#pUBV%TCm6si#YS4pyd7r9xL47<8X*08Cu7mXJ(6OfKs@+kNEz!%C{un}y zj(+C|rA@y4hgRo5ne1{NGTo71x|*iI007i~)l>O@6P1p$k-eF*n~sB-iM5`ill|ZA z$Iw#Hipixz@yhD^M>0-Mt`?&_(f?JGrR=zscGZSt<_!D0IsXW<^}0hgjhAq>p6J*tV@a@XxVDVyZ= zdwu6iq1f`fIF|#KXarSO$AAJOX=lkI8oj9cdM?cj?KWiC!-JBNh?mQ3=)Cu3R}^8! z9A%bRn3F!y^`V=ZXQND5X>FUHp$}O!cl2t};KXBU7;o0g0u|H>DhkoFQ?Xe|Z9~X7 z@68k;q~$|aZut7q5%zm2Vz=ug8EO22yLABD8283_$Q8j>&I`5e^Ks;LPvidJ4o+wr z$p>D?>V|zyPEEXi`V5`L&fxH>N5w$eAwYL3eWT;wJYE9(tB8syB;-&OWu=mA^6K#- zaz-td%y5JR(U1M*$OqdVe!UjEB|q(?2=pvetihl;SLe_dC{31g0R3Mv`j;f0KmS=BGix(P9V2UJGkY6rDy62wX?T6b=xis-3zPUxR&DvxIf8Lgn=R*lb@k&u&07rEyjsQh@l z6E0-a{4uUe#XRl~tVCLTEMFcJWpXBOmNf0OlHD6NyCpVwHU(pAz@Wr~n}wI@Vw;1p zMukhna21+uf|~=qlxIRD27oMRKa_XPvz1?d9K;1;KZTSqLSOP5z$fHwpZw_qC%K<^ zo(d)goz6mQbdN+`(!( zBYTLkp6R5)>7HSn4iD$!S(r7)=KJj%sm0g0Ei-NFX&y)_E*3Hd^`HCwQ>(@8E%Zzr z4P3FLiY7+3F}hv5{ekCo-S_3N)Z(3>AX%btI`+=@h$^8t6MkL(>Q(wZdNIkE)swWG zvV`{7tCnkL3Z;!@ayzxTi;nyQpK=3`A0yhFW!27ameWL!_d;^rhzQHp?1 zKTd2I0WbI^_!N^ZSCtj`Fp{F6JJ}#T`goClMsB|2 zR;eVbR9c(~$~jq-p1a-bpD{a~#)rK$!tdJdGeZSTihA8!(4^(eqsOdAG;LlAG{z zaDKN9dBhes^YIR3`M<1dp4Qr8r2lftu`gu#mw5GmIi-!2m5ue^Aj=@d?&#UC8=z7C zvl}TSzc{=U)Sb5`n~Hx_58gC_6(7ZISIol7vgpCE_1$BjdbSXn3T-qkt&PRzfJD4; zf#5WfA0(!>MBGn$cs2k9qL^u{u-5Fn%=i)2DyP~3zP}@HG4Ti0{6X#Uo<=zJl5J}P zSofBHc@&Rtt3{4{eW_sC%MJkZRUmzHs?=dvk>Z>Rj5wO8)cbdjj1*$3|%etgz0XLe*g>S!L#x>`-7d8^uC&e;?)$tmO;1>Ve)93TRY%7J-)pdId|3=lMF7X#ZD>UrGmRETOU zLy^_Vtcq8PDd}z0vXov}O@e=-ZO2Y!57jf^q=i>*kU*+)Chc+7pL?A=E5R73BE@_U z&-?zGGq8S7g5HIh1*5{UI81gwGL)%H#2JZ34idpq0g9g!3504M9e{FEo+_o#_e%((n{|-0&FITiT z(sMM@G5w=BadkAZx7M@#8^5%Q?>1kfLkOO{R)F!SZiFy^f-J}uNC%k_fFtqv&PV>> zN~ABg5tnGV*+Eo!ZfPcyjv@T{_QEs}ok>V+qTh!A&3HKMD6w`NI;~3OeuJ9nbypu1 zrhO`~gEhdBign3N7(d%v*{uE3n!I-hYCVJClB%WYjz_=kk%0S5^=yD6#Hxb?zyv4B zlsw#dil8f+D{qpA>1MQQ>LC0e(oGFedUmo-N_fajRb6VF&q8IZet zYKxrZPQmALvxc(i8j#B1+=|O1_w&e#AS7Ej!o%p1=JTualdc1w(n|X zs8u)kx^(Y?I@&*UDrkb{@xt9w-vt?lDhbnBv)A6X+)knO+;;k`h;)B6fcRf&DGc^z zv^1kkk<82vaS9*V5q}O)v18xamHq&#bKuTw8L@Cl?rFg|-Ry3`W(4S=b4QasA1*sV)P?5|32uiAJ)H=()>I z9Q7%&cgjngAE2n;E>K;hNO}7F0Qgr&@WQGbIRg0ND1PN7ihp1H8vN4{{)4txL0tDA z<%-}W^Ap|!DjSFwFE5#Z7+e7g3|C-Pam8Awx;$a|XQy_*wY+K0RG#rhnK8-f1OZay z22z@!l369csM7M_*47W%*yw>L0@_`{r7b<<%)?uO&EJyJv z>FV_ktK4a9G#Azr2n5y`SDc1kUlrCnDraE>`dX{F!W(InW zW?y`VzX`9X_p&fc!oBY3ba6bf!1$_&9D~a>ZJQyvxDS$QQA@Lb8Lc?N0l&({d%MKvz4S*&!GCamjM4z8 zFLPtxso)MNFSf)VR^mq*-?^FTrn1)lG^|9dkTa%glU*_MQyL6kY#_e-{Ys?* zGsdToo5oi5Sc3cbEU8$NGN~g>Lp+*2jDdCUngvA6Q~bp5Ws$iHQDxEj8~r%?`Q4>wRni25EG8q+^0gTCIC`^kXa&WZIujLVbqVJ+cxUzf zURj#wo6*z2pcf$G>_bO;84`E@KD96|Nk1H<6^ZcY$Jd%Q7u(8Q(M3rJj$v1|1o8ZB2|mPYd^If;m@o+HJ;wf;r@23K-Rp!^#1C|9_f%F8xbMdf5o zG|!49@T0&fHcB0!sbp>6D_Rg#6_=gowiPfDcSfyU8uN=G*@XxZU=WCG$&D0+hcWADrsyI)OYaY>*w0SXVrQe-yHzqOY;UYv&ON|1_<; zv|#lM03I}#T8x}@1C}U8(^<2!qSg_*zn9-~Jf2hE1@GDx29;#V8P|tuB$Qt!wwRU` zDBWS=AUxp5BvcN09N+57(rM$o1>FTZkZ z#n%_b`}c{#%ErLr@8aJoX52b}4k2)oZwE(N(--0zAj$oNEv2}zvD2 z2G(3refVb5^ljV{uNOuuc4{JiXK7`NOAW(6L{FO^g_Y~~hhs|Iwj}HM9_J_{2f^)z z&bvVc-p!dTO3+1-?jR4My$~Q;s24@m5A$pZpa2sB$E##j-LN`cV&zCpNWyk9d5m`8 zh=^!`GQ1ZAleQn1J>dw!eXcnVMS(OTT&W~r5@W_hN|FkjU5L9u_pH%br{}F-Y~g^P zYMWOOzbk(Xz$!Q@cY?)YATIf`thI z=c@d&BiNT6|8j}YpGAwklQp%Wjlq9#PjN+$L;b^=uzNWEXy3#)x&pw1{1wH{bBbcq z$pNK^!$LZ|5}q8EgVJnFL}-39!_hOAkEsIst$|j6la&Pk_?x1Cjl(cwfzBDbnc64z z=tbJ2ZadgsC)quQNBPYZ&9QEnmoDz*g?!xNy9j|y0CjJ*CU$(t8y8xwDzSc=UWZ{6|2wR?iw z@z7I0HA8eaU|XrZr-nyFSUNBdSCU1DbSA4;?3u?fOf7w!JoS7V zpcYbWpO7nyZRm6PSuMV|XTe#Fnva824Ej=c@+k}tWmbyBt}a<^ESGk=&G(r%A6w>i zZt8WX*-N>4l&KF5Ggkpa3I;5ltLt`oD$1+fElHA&$8E*U41Ch)WJ;OdA48qpWD9Xc zs@Ev_$fB$Fs7ZXEI4%sRI4V;)F5|7CE=l2lJZxLZ?>hACE>xLTSLl@BK?}HeOxB&{ zZ5Aire|*03-}xLT=1__~yT0Lb!LJzHb=h|r_Emz*2pzCv_HN5JoyXQqArYxEH#(}&r9 z<)Y-TKZ1Xs)Bm%e+ua`-qaLEAlq0X8I@sUYmscjImR2GkAElxYqo$z;s*#tEKmnqZ zV7R$OiM$&_LAsSkkw=cOXHbZ^b8Tjr<;A=ok&g&xyuICFyq&rEc4qmFDbjK5N4aY= zYff5)>@oFnn!D9Ej#dTccWjXtd=Rvuj~wDP~s>mxVA@`1bG9>wH3@Y8) zR+$OSCG`4>OqIamkXbG1XErq9`ZT2-K;Iq_LC-!Aj{e9IR1)sb*`2@3v zDHQ0s@vIhbbV7aY5~>;R)Xb7GQ38+{+g0afB)8@AFgwMLfK$~oJ>$>EgQ#LWA6A+& zwoFkc81vzqqzsFFsVMC#ezC(Yc2cxv=4k3bMJ;1IL+y8OB@ES7qefE)0xH+~Us}t? z#Tt3!Xx0~Qnf*yHs*)0}`Aq6UjJAG0_f}(-G$LqOJTNxzG+5@KIb=nWW|3;_zuaK5 z)mgFiQ@}V`GiJ7RIww&5%rxdpzsbmRP+I{n%Ir8|3nV? z=|YK5K^2w|4uvTM#_$54{>-CZ+7&~XAK=}4!G3v&cN%DCxBE+6AO!rUj8QOKpx$xx zeb|Y^N z@KlDgG#p6A!8Kg$?yfK_Vl2_qxTp2>R&wE@lgi?JM0rj6ZheroQCkpyicrl%8)B^rV?cUXC&2CjF-N~_JqzAG3_)YQD@Z0B2(bNr36aDXi#u> z0>7VGGk{rEBozp6j#!+78dD&fwrQ*-Lmw>&hI0F1sVwZ;+ki9-X(~!JOy^;DHFZFh zl53$pWeKOG2$*NvcfB&bo!ZkieF*1G?~CwQnhr-zOncQ2G$R}UYG#IkXa~fw9#G?D z8gPsV>TBphuHZ{S=D^zfcBo)2W>8N;*^RD#6Q1XD=CYbgU%S81@rxS0qiDMcg<2V? z!D-FHm6!4`eZ`~SM8ar?30O?h&8QDxCa6bAW~N3OMBdgdi~<=gTCw%JT-LfUg{^)i zD=O-ITl4U7e<}LZAH0D6d|U{s17+Nphby&w{dl3ffd3+XRkE+0RRU=Lkoq=I=cHid zAxF~6-*SB7_U>%7*6;`L zHzvTI0BC2l&&MzM^GcVS`(4wg=NF?S?yzTLGkJwUmqdxj56JS#D)A_7E?c1NE zu6A$Gf81DqJFJ%64QL0_8(?0t@!^dwfcXhR{zhCTK%Vk z?@WqLsC>O1_QC)Fkp7!+G0^|V%b|{`-dAVb;qS2fFJ&v6yf5XAFI`A+?j_|C+QnDY z`J_-Onj=d~J^SO5(kY;bc9+zG*3stU%F8i-90yjVIdb`)i!sK@HcsG0x`g>!q}5-3 zI@TMTT_O~Xpx~X7WC5q$Kgya{pv!}(_)M4V`V&l+D3viY*@Eycnq^{V5M}jDG z3-l)mgQuaBipyF15y*%84K17@h>osdr|y_$0?w)z`V$ZuGL%2IF^$RgX`*QM4xO>S z=vb*|{jP>hv{UvaM~vychJx!kGqmEhvq}_&?B~m?D`wpDBNqwaKNG?V0R!P7 z64QV?Ga|Kr8MU%k`h~)Oi9HnffssSUE05JSfR~R> zM^?-lU=Wo@DO!w zwrJ2;Q(k>k{7tv9VIGZFn5vQn4YuzEzz#0Q-o(rVWmDXB#vlHv_!F}MCh&;s{hMlg zl|Y2t8lZBaF>nXpgvk!ce0JHkf7tlQAI z9VWH#F_kdaC&$SGy65NT@%illDLVwW_4Cql(qh4*R&v}FB~9VpE)!Lpv9Zu{msKRlcpHS0C(Ns2dTT9Y`5o%*iOndau;^2Ul-$3qceA0b7uQtg zG`xrq?N0d(0CqK?>h%`pBs;MC7(6j@9+rL8yLSFT-pvj-HelP_+;v^W@xUcT}>)Y90*TR^}Y&?X=EtDoX!Q1Fx zNjNfHG~dDgINTI|AF{D4s2>m!NY3wN5iw_rc@-kU&`l^rC`osz$e@k5n8E$C%K2y7 z*1kp&srXV%G86bWgsZQ*$ zN~gqVDJnWzeDz1eWIOSPr6Tb0$?c7*b3~IQgs*X4Hf@ZZQ`Yt+?BhAM0&|mx+$PV? z#?B5PIV2k3&0IStmL4Z{I8AKg_qcZ~F<+%G#V#RR_m4?Y2qpDVp&AY-bhxWt=OP$! zBbO-mf0BGZsBFK>j8RRdqxII+$b{k*O-@4_(DR(+%-#bg3LIbH6(#;{d02qm1$+P* z_ml&Uq89t~96XqoLNKU=Zr2UAoDTdkt+|a28z$G)9vVbHwJX`F-%6)J<~ATtQ^Rx@ zHv-}NMh?#lmz@ikYl}=tBkD_qA8Sd&3!muJ$1H|WaYakE-|)-;)$iWtFUtum-=&*o za>c&|oi!K=`GQSGfy@~mD-Lm!Fi{W*67aLZbOi=N%mMLnoqc%;)2~|D4HMWvT9prI zfGE?)mZz5tqC=bzg+kof)S4$O0Ko~FY_AWL*L8FcnXEs*ahDi4t$^w%&gyHMv3MDm ztp0G)yDLCq*G6WJTDW~EOfCV4iDg1!s`8KIxH?x3z1xh4{?=>DoEZA5m5m>ay> zT-JAYi1N=`*Q7PRasY83M+LoUJ@#bK-(7Z$CT&Vx*Caq>BHUU-!02kr6lFKnU4;km z@4P%nkk7mwenKkFDF9ptadB_Y%gfuaO80~UzWSU&&Ve88`xNDVL=IJnBPnr#_i=lH z%o~c)b#nl02FfLg5r9xr%|Iw1YrkIJ4}I^^3u>QukKx4PV+c2>P^K0>&_rsL5C~}x z;9avMiaz#pLviHKS{YU4U6!96G7zClCv`H)a*?Sh z1-91@Rf1HaoUW~#ht5`t(rg82@8%2IG`{%Ts@$$uzfzLtH73sX$Hj=$V5w*y+SCNx8;QQj~?1IuUz zTC*_4yvU8rwE*>F99|JFyq76RYF!9*Ya8XHRpfg<)ix#d*P!Df^<3(U&@3h+->V5j zex{V7{mJgeM(A4dD9Y6+Y#(H!`0*0q8 zq9Qe^1HGgA>0#}iXF;~0F+Xhz$=tu>t3)%apKU!plTn? zjC2(kwGVxhj=4lAa#vSZRY6csrQY3wkSW_!=-Yo^6>J=-3Co2(Z#0FsB6wPdKgkPP zNGux&I=Ea|xP~pC4P(Vw(AYiMze;8)Y@P?eN`e4WRHmLMr!;f}YR)I!1v^52M1A)L zStQ&aDn5~n%yAY>b-or+<0MP$TQtUaP@ss+w{|8_%E&d^+kT zC8`Vo-B4l~yFK-kraE|Pq4zun8u5!uBOm8&A^*#)$D?P7V~U?$buhZ&)Jm{m&u~5^ z?tJ+jO1yKm@cfBa_ci_iaHr+-qHRbh)ceA~+G>&H%OU&o70K@RXLLo}km@lA^+9d!?P17W)DCR$s-PMAH5TI-z zxdNd&%tM{PNfRfOG=~&iAJi1x?fEyQ?cxrSC^c_@!MAeXIFc*?&}YkE+-1neRUFv_&v3i}%CWqMh}%K)yz+`ds>nW5ePKOr zFfOtX?Z=rDmGFe26uK0k`h0VDznCVbYhJqAMV&4ZtcLx}_>l!@F%K zE-`9|i^U_r8^3bN^*F$s257kr#7*4DO;`cA%MiO);RR|$F4iL8(N0{x?(OJi$U2HE zmLHT5_+pKNND7G-p7*w=oplp#9GjXhDy!zv=wZg%E+~C5H35jygL+T{(NY9(0NP3! z4o3|drjpMpkM>EtLW2{AT9f2@)1W;Uk!OysR|$jAsfCrlSG`qwQ;m=12f1v>k2!Zj z7D#T7Y2sl3oe$Ve=A+{CjQoP|8|I#DL{y#=@;1&Q9wlKm`m!4w+xLt9kY1(Y(HxgV zW^bnzN<0)nY(~aZy*V1+VB0o)^trHkw;d7T@?EP&&I;XmXDUbw_}dYF$PR*vp&2hF zL#bG=utyx`xaok7@4%jBYWn?mtMerFFQ0gSWn_JYpMwNWX~uhY%FQ9(lctFu5iQBdS05j{_HN9 zDGpKGN{SOSBUp|0II(r2D7Kzmh`e8-ENCqQ8x{jOMUr#dwfdYIUWKdzi2XJv;sTF@ zplZuQa%hSzkPxeVXjX({D{Kf$6+a0-zgYo0#~YY!ueZRCIgc4SC$|_z_YCftw&~Ji zvaF0Q0Ohj(EZgIlS*5qcZ`-1p;JO~}y1b$cSjpF*Z2Kf9;CEq{4I*z)XYZY4U+i)p z+TSL%pe`U1b~&83mpN7OlVe3HtUI4PHZe4$k;H_!-BR3~zs>W5$dSU{pC$T={A1e~ ze0cx9_fl-5658u-_8SPD>^sJWf)F$gFAaYGTbcn^XbMLdo)J6>=M?Z{9y@(pulpA3 z+MpS^F{{i-SG*QXP|@vdwXusKwuIiGD)t9Pt^No}vO(lFom2{D`}+nj1^dd#Jx{Hg zQf2A8xqz$VgH@1E`M}}yMSbf882Bi&W9F-c87aBYOAy*F2t#eBa(v=}wYiFwgVM|V zLqA{|bDu(A^sid3lg=EobgJo=%{p&P4)gkyHmg*4f+%Qx78`_39k1Mv?K+(*Q9jZQu+hQf1{vd%uk$aW zN4N!wNGPmxTu=JFRDAPK4>VM2K=2Qi9A;7q?=MRk@pn^TjYGp+SVnXft}4Y57}dYE zZ{J|v&G0%-`A|wn$7F>YxP>y%1>(({;^8(Z0R|hJt>fPV0$rflEGcy^nxTdW$8Xqv zZqH6n$6=>IXiX4eqaCPSu3Mg@icBA{13i!+Q|VD`Xhb(DPUD6*U>o33r>1)^b>jn;&drQ@`i#n^BpodAzHm zmBs6~oECqI9~-i3oPq~eRyXE0cnI*&38k!^2(~mQ)h+`wkJ&Iil9oiSx5lbN8|+74 zB(&0|v`Rf$^L;0T!E<0zeM>~nB%NLbdCwJ|i*F;$BuOpl&uT_H1_ESImgthB)gBwtr--Y_PXkws%=U|VfJv6gu=FwU`G}S?Y(Ln_9Kyn~I%B;%xEuSsjU>^m9 zkbO5400;A$4#JRsf^|v8FE^_UNHXnf%~C%~NinQVBC$gw?ok4wH)7W2n7V)@reX?y zJcSJtAB)ID?Omo`E}Z*Bh7UeM6E*EG2eIDGobLlZ*go#3Sw0F9qha;HWyP3pL8_8c z25U@h&2u%j@9Xt`%Hl{Zk$|G(&|#}9cVmSz6~#$E42} z&;WoS_h0+~kUu&tNy^Lic|%BEhuE_p~xBhcIsNd>eCYg@ozS_D5Z2^AoNbtHga z1F9`xyD>$FrGPU{;HFiMsTkq`+rseAQ0+@2m>dbsq?gAf_ylsqrsiKAOYw;D1{~FmC*Ir5eW+fZ0KGZWP@Tqoo|&*)9_OYg4=B zam89Gj&%}YJ@2htnT&+|`x>&WpFI>bh*=}ll*D$hU&w)<$WG*2D2OQn*!#4rcXOrJ z^Mj7!ERc|Nlzw_1(AuUzY=lTWHYj~Q0*T&MA1FaFf|>N>Hf zhI|hIx)$@wkp)o;uGL*F?Lq?}dR`I6r8M05KtTXh&BPSLzH2C;QgMAgf~W*MA;87L zAyFg}p^L?kLDHV&KY{tc)#I70`_jNdhI8&h3J-5G3ugY| zR|qNf)sYaUWoln;Ppp&LS)ksj%t=Q^+KOC_pT_H9nZtAy8jq zRgaC#Cy`xGCi3lC&C5T=TW-88hYewyf-32Lc*zNR^TK0laS+Q}%L)R@$QwY6y*OM` zHS!AEa|Q|goFXn>gdhk!ne`~*W$}J~09f$QgB`#I0fzg8hH5!SUU~}#lbs@+r@x9G za;V8zicd`DTe$Jz88${(b6SK83U(p`jB<~LPU_tn4%-D*J(bD8b>NGN5nXT8(gIzs zPJ@+3pD8`JTA}F@Z7+KA6T=Mo;rlMW0}jP=a5pv5j)7xiXSxf`UkHoZ_jc&p88#PP z4aOY$?rhw+3OF3nVS&h-zzr$pf#aE5Ufv-^P9k>XN?j?(QLU~biX08mg_*)Up5;=s zR}V4cGVfD*7p0WdaNi&n+e*$&23QA>iAD?v(U5f1sk;+>KM;i63llXAuI`<(B!d>6 zAJN%vjSj}1#eV<D_1{6N}ow zE(dBUQJyrdsKij|?|ZuA`#ot8+|{z|Hstp78hdCre7Cy!r*53d560gh4&N&z6_QI! z^fQT5rk`pfTP4K0lIo9Sde4m!-2p3Y7JCzjDl0nLO6x%@`UG0#5UXu$7E^}KWhPhr zl_MipgVKg%1iv7?%gvFl{Y-aq?;2p$FY=)D?0P<1CAaTJEB7zDXs&jRY4(w|8zie6 zfvak5u)&$N3j&wn!}&XuT776ziIh@(S66UL<^qZ4=PR6}Io2U?c@%jOw*3kp*_Ezs z_1DX2bwbL+jp|BqS(Ms!?92tl@{dQt?CVyJ){Szs@u9E>4b!tv72b~fTE{{MLfl`Y z)zUxAN>VP|J)JGInW(iL7S=(!CerK6=FGn{APAv6kIqDB^h}h3kn}|uq6HNN%R0?? zvKUoV*y^{rzqsI$5zf?o3-p z2S(zEc)ph16rOmmHkH#QPG@R*5C*0k!0fOvC?jqw2-U4{V~ai zrbU^t404+)qgEbE)5zqyw8;3vEse!vjpJJHVZAQatbX6GbH^X(Bf6dFO+svITTmPxa1~r;a`q`BXA^J7<_1g83I*XyeVjPt z3Q|gjQHO?L#3vEp4&%2EfsO&!wK4*OS&xg*A;ybrr1h84d~{JXC=PwQ@1i@^U}W{> zeahg_69a`kH?C^i;ycT~yYO@Y&pfnCDD8WQC_8XroK_tYK@Y>~em>rci z$!%4B5b%8yb3Y+9xn1eJ_tiS@+v*3?EM^zR1&$2#$kRwJgTLG|)mR>04GwB+)N>_3 z8Y-egjc)r$^2vjc!j79m*fjuH1V!ng-yL?7vRZMW(Qhw{{6i%~BscvGW4$OfVKGq5 zGEns1*$&oi6FHJG5A)$PLu*{HVaaVNwv6M%Ot4M`32r?T1+zaz})$uhHOP=d)f~efpc(x%uSaFFmA&Ar|CLfgKO$+T9 zG$`NoNViFBXVu79(dYf-HsdGj>djs{cEYg z{>>)-;ERu3{PhC!m!tGwrN%#ag!)bf7Dj)^MXpy_wfit3r3_;-XYRsi^d~ zgv#8zNXm|9l3@mmVye?4*@jttg=y?%mSW8^ruiU_`~A+o1ST2j#!R_v8bWZxz}e=V zdxrh-zWHoGdRC#~JQGDp!3(j9_9+zjqx?Bic>-si;H)xbvuAD;Qd->ilm&6|kwp}n zpcT;+Nohil+;&AwsOOnb;(!=pUzA?PV%qNcwF!vF}&Z<>~d#~^x9 z7sV)l)Q>CkZ>A}+OAlhIi^a*u1Io-?hr)#|9|UdVM~dqU+Yz7?fjWRD=JBSV#g9M% zD^>$HOoaw`Ihld`FTSa)Y(4ZdNZl6-Z~hd*J>`sd4z8%u93R0HoOfTrPCsPYv`9jK-XP6Sn6}1932dJd$K?3fH1U1&(b7 z8ehuaGNS>cZ@m@7v7Yi1M1)lXyY1`71R$a5!-)EDxR{^Dd}%(D&!k^^{n zqOWTAe!OB%4x9z?;!BY~KuDtdi##MeZ`h%=9{$~n;%p5sHcY_;LZ403z%Y{uKnlgQ zAp;`n^hpIZ4Gr<<2f|W>eQIxAvEYneNI&8h7fH_jw40I3r2xseU2)PLjG0#}Q>j0Q zs12ZT)ygm;*Q{z1ETVTz@O*Cy4VWtIs#1cS(t(%p{IxqY#U5wCrg)I*q~j)KNV_AJ z{4LYG)~r&=%s=QAEMp0^T;$_AyK^9DyzDe@g2+kII$+BP^~QGf_@`aY*V!1Y*t2Kd zv4BGPrELV1`E)kou=b_@MEvDxIlX2~9E^1w zdO2??vsB=5MGJ zb2HE+ppGyJ@>?;iQJd03Dt1csq{cYu*<#JLPk)(Ysb1ZzyS^09$E$DYrIwm*wCGu% z`p&F^DAL{N)a~%#Nx>n@E4jToY{aqP-oSQ8S}u4mm;u1PARaRW-3X#%@F)!NkiLF@ zKE4F>Hmgv!vj1A1cDW}9?`87O&Kc)Vj#c>fj(Cc+%|F0NCeYK0{oTzb?n2m-lh}&C z(ilMeXl7@oN#5N%jxN;vZV(hD0Gln7I)w+`-%y4{MK`2}*f^P&eNE4i6UXbb6(rgo zKYR6M%J5UhLLK!j9oxmh&3jncLDC|)JBAHM`9j<}dAKW&)^#aai*Qp6O0b~yoycSy-IPa_N#?{ zyuQPhf_k3WXzA1Ml;4{AhTY{#eYhRg%UZ_N*GLWg!JU_eF207y!!)_3*Wq^Cy>j@( zpB9^1oBHYf(WgQi(I$%Y)vVsw3OIGYf_G%@+}>*Y!=NknxO%_bc3 zhnR+L_y+2+#iB^kK(?O)3#9nU3g_;eoQ*}9b8pp)_HUYKy#Gtm45|FMsUpYc znyDW|)is1*jTkWrHpsEti4+hm8s~f~b0|%xWGRwIftTH%(&XoqG zf+I>Jyw9%>1o?doI^i`>41oyChBlDUjik}kib;J}H&c9&u~xEOI0L5W!j9?7-+Hav zpkW(vqLJl|HOUu!V+w!M3VQu%^XEmd?<6~^gZ9Noe#SafwlpR^@eCPRZ0~v{7)Kkw0uwOC0=Rh9PYDXw4 z@^c=%v6_2`3~KxJ zOUNH$)xa*XZ640DB@1%JHbfc+>mdAROdu5hT(nL{Qf<{z>kth%?o zY)*wx&qm@z2s#X_LR-LzT zL)oKY`y*`=b>-T{aY4hDiiS0HTS8gO<>Eb$3ky)trT%|Q)YVA$!bmiB+|^|r`fd?M&-bf&pa5SnC|rc=*`OH z=-Gnh9`^Cz!j`=Udo1xv@7)IzD=_hrNiwt>izPO?Ek6Vs^E)92)mx7doXbM8lQX0j z+X>q!sf)_Aga9i*8gT1IjsT6er-=rT9tC{3=YrB=Wy!bo!pHFbGm>>q7s)1WD_dSj z)J(yH%DiGSB`6hTq?(7YqcQ+_;+fWH!|NnW57LwAaK#He6`*GCyC;zG=wDLAK9|~{ zYqSvS-=nfG@{=kG%q_seq~Rh`qEwFUp&5ri$}D!CpdPouiiQdD5zCr~(}pyZMq>B+ zSC0Lb#@C2BDMfyt6)vCA2-GG_*rI)HBsVGyf$kGe{qee6iui?ZX+`#mT-cZRiTi+-~% zN$Hs@7X@VcWN4)f1k*rBR=1c z;>X*I{F*M{8~K!Mxf*eJBbY-3Iq(vT#cWt8A-l?A_>Xd*qUh-|Lg&;iVWl}gzDpk| zPOL;~NytaV&Pcy1m0t_B0BO)2ESBbBJi%EzHuM~Ve;;oUTb4#Eq9A8;(sz3LhT7WX zVa;(f|Icf?_w*P4!O7oGA*ff{=uiIO59$!&pq{6zSU8--k^4dM6RN_=j*m6Q4k6&I zX43>WB=EA4>JYM1XbzV8?I;-i>9Y;iw#c>mzBu%Z3J(z`jz1|F$+RKK1QJ-I$0KW( z)w47rI*z1)vB~9djbu+gq$;f^{D=k;H3AnJQUm(5?Lj@}um4OZKR77p3g1?%s_dQ< z?12?0{LL`St^n{P#M(>Wr|Wn9WG)0jq%$k~DM}R*K@hBqDDoJ^CH8Gj5M05PZdSNC zbC@Nq*3hpvcsPgK$~A@fMRCly5NW&5tK^b^xPVi64nb2NaS#&3(cyx5Se#MV`65*7 zR_kx)CfHM3;;#?90@)Hm%G7u9R3_xgGkIA`MQmf!-Jc|$1J_ek$9yvVF|q5Sl5Oi1 zEFD$Ot>IkI)ZcybC4rReFsP=$Jh}X+6khjTO8oR9(NmvT`k9w;{BIqob;^)|fOLcM z&Y>dzrFk@vC`emZZ;s$*Z}bL~l&15PHpamL}4}D@n8= zrNx;C%qzaXs7A5`!Xj)@-d8yi0&pA;=U5d^ewsg6hDzfWnS}y`^Kib+Z&R@c;?8p- znsBr&4qK;Ls!E+cXLRB9=c1jkDq-3{O}i?eJY;!6%DWOa>uCSs?4n^Rq@=PlM&(j^ zWW89?_4z8=FOw;EKB)!D684`ylQ%C&J#!f-YIN`fvTwU^YLYkT!57;3{iH@_T^-)1 zSHcH~+oSl?yGERNqou3h^HY#W;EQxe{j2&V0xR{(u&s{=0RkR6Uqn9+7AS!--=Mpo z+@DX_E0qzRtON-8mV%uZq%@r+=?XLCZ>;EGio4oT+pZ0LAUw;P57mZR zod-KuC+O%WRmo4y4>tDZ&C?sp`47puI%-!F>{aY3=(q!&UYd3wi}d7$0RaXB*5peJ zzM2s|-!e6nFm03oLn$Pl)uQ9Kc$u2FI`7Z;(q108d-Y>R5^JY&>vJx$KirHi1PgN2 zLbQ@Qgtb89j?=0+{BjF|7d2mS4l#Lqcm8V!J{v@Z9Sel?ed3}yO^b3HfR1m zc6F3&MS$A3D}iGt$;%H=xeB8kUU+qOZZ;<^fkJFFnBBN;W}RW>9^My)dAbeppXu)C zKGmE&^0r_9tLC%(FnB2|*ZEDA1-6et;Sln4*?E<1rzzb?K%qO_ZDG3v{e>~{f-Zq* z(ByS4VIPrSs?xc~U7Y_`xJUR>MZ%{IAPjiDYQR$JK9scS_*gz^tAA_S^n0MS zdNljD|3jbeFb9n**Sit17)K39kCG^hGrJjxD`A_PZZkNhzk(ILYjb#YSv0;(%W?mB z*z@1_?s}&J;~&4si21KI`hP-3{%i04E9-D@xBHLp<$r-_r+$%<4YvOvBldQPgOpY$ zuv=ijZ@C;2#FMtMhbBV|7>&Xk7Dn=5k+?(7{o z)kDj)N%VBAIZSPBEgd=jxw(LC<}}7)^E~HmPX3&fiRRdchxH8f$@D`sM9GylIuuaL zqBjNBv59=PU77UwVCr}+`?pC$s{_5kBPMw&L`>2MG_Cuy^y7Hr^bzTF@oDi%3JnI% z6Y2Q?wdUeXNl;+MC7Q8#D;XFmL?R}&E`T^B<}QPHU_4|+31LWJ;vpicWS#{n?3f0U zqX;aIK7CzyHnu0`3yNc!A`)Qyk}Je2q%MSDHgU5w;gBLoxkNO^eSitHSh5MDgCPVf z2XO1W*wFAMaY;mS=7*Gx!+V2tGK;xF4tZL+zP`p;4!L83{j@q^z7xUteFPL5$o&@1 z8^cD714{OJ3xp&G^_z-dH9QlLI+NhZ-BisAwLd%;56Cf%YmB#PV$xz|YM5=raYNx_ z($1HpA%AMi+>Y7m>S>tr5FIfi+v2=@=Q44?$i zxp=2rD+_P?iuBy@`MA0zb^Z|hzV>xE-jrU0sI6o?o(97Y*o*ynohJAc8}M`h!#`8C zwF^D}vOeq(04UCr)$v_?(7N;pw6XO%f!$u9r2!v^_0ZXY7Qlw9nt)9^0Sl?VdEWx3 z$PJ#-pF{nhD$2RQ_s-U}STTY|-Mupy6O4vy#M@|Kf^N2QOaODzTUItm) zAKB`&5QJe}R-$s>KN)?M+hw}iTA%anoo3qKTDI%N^uyPsvjb-(aZMYS;BG{%?lI?C zJGA0&ittE=b|Vc>92-WH2YFYc_SsNq74dq&uBObHRy1%jqDWIohcs2LRG-I#?l5|{ z&D}d+8mvy0G=EAjWH4LrrI^1jNG+6AyX771jQ2eD!m(i?38zX8kEIZdz z9OvNMN)x8Yu|Q>;cKMrX8ehN>CO@nyy+5HTXqvfZ1p$Z9eJ=3XYh-$F|c`S>8;4r-c@KZADD` z#vX`}SAEI_FKzwZQ@s&1CVTQ#r=EOrUGVyJYH5%goooW51JY)$7j)P)zRu}yu*{wt zC}JNhi5D#>jBi^!WVJZ`ZIOIh{2=S9+ zp(13>`Z%@YN`*XribsD6)tX?x$7a$VOmqab`x%jdjnzn^%Y*Xh_Q43nbk=g@6C>3# z@UMHJ^wee69iS_bu4SobCnm#J2OOb)o}H7P-o3TH_ua1HNqiI~=XW%`gO|rG z-$H_<4GaaEVV!e^Rshq4b}jip7C*Q=dGo@ivWgGQH{dtJ6<&{jcs!ja6UEmqmaeoW z?_0cEytG~d7P?DeebN)^U4;HUmEJMbrs#zuTUn-6%Y1YHpv<*(d{iy9*)k5d@Bgv3 z6=VyQ$u4)_beprN?2T}}sQg3PJT^{x1P)y1UV#TmyAGOZ&X)j-mZt}{RT&H8WgaOR z(brUV7;V%Dp#LI2X9Qq}L=C`31C1Cw6s;8@ z^*U^Av?2?;uEE^RktNQ2h%ho@vOt~y^oSbNJRG}U;klKE+-ZO*Q~)IyfSc9)Lkd<~ zRfTFV02FuqhYcYYd|ZQ6)9wz*m#Hiho5o7AvSP3i!~+|H1_+-G#0FzhKx($6kCzAo z1Eo}rpl6XYNxq+e99{YXIB!-f{$nBCw&53-N%lVhFeIy!f*VfY`RRkfS4Hp|vql*^ zmNl~pX){_S5Y%TEh+ zfWXi112S#hOptoO3XNm&Tp?e$ySnstohJ{ZbWe@)#oBzd9d&nw%9*J+{L*{2`Q#u5 zU_6U9>509gJ&X1s^u>*`*=6|rWH^&8`D4yQ82AyomkVDJC~y1f#*DEQ*W$;CH8W~w z$Dazgwx(7e74|z&B5~cxdED+Bx!&kQfMrYlQXp*!T!WxU&?ML-h2I*pn)SqSA^$C) zy=TaN76`*8;KhovA@42o-NBza@5Nje`yqb?2r9m5AILLW^#-9OJG90pIqSxK7OKw` zwOz&CR2`Fk>Glkrmm0;`B}cMi_zx=c;p*K6vDk6j!j2vFm%`1KCtZ5i5cM&A66x^e zccvuKp3sLPZO4tdL;N37CW;iZ7w1Wc%WSt29{_}(U*@BXhyCjs;XkB|HfT*wR#r|< zj`Uk{TyGu0`);a$mmp4V&rLD!HHTC-7o?ksH6z(*zz2_yD35R?|HpFj(2UlgtWN!N zXVXkFUM3WYf^K+w+${m3Scom$GBTqQeD=bYn=>qUuZfPm(CcOaH5)w7#pPrz;a}s| z$v>d^BjbyAhA~ho?v5^^yD?+Q^BfPyUIM|!r?Cjhwc`2wdbn+H5!D2$V}_x(^U3-) zWV{*RWN9G|xtxYZD-Ad~W=$IZ-291=E2%UNQCF9dPy!p789YeW0wymR#kzx#LV1-T zH`>3WX@Cp|K0zoxIb12+x}{kTPwNMt&Y8NJD{UMR|=F;8mT1lc|CS zatMm9|3sl)imrm0U{A^eLH%nYNv_ck4p_ns!3h$F>_F+sTmj4N|Hoy{vfvvo9UOJIx4f*X4 zZ=~qbbQ~fyV>pK0tzQhd-+Ua9W0AObd2INDF~l&g8KfUR2%D1G3^1*74TA4UF`nJZ z=Gs(?-J%_4K}0(p3%)|)up=z98tjnNl2#yYZXdv^uSHj^i8a{hdInCDZ&HrG=rIBg zOrOQ1)r>um!af7#BLY!kkVJO;&%R@xT4ag?8nb zvh=4Qp=zPpWwWkh#ZrRCuhk>T%Z}LUXjg%ay>Vja=S~WG)}#AwV}?4KmN+CETrw__ zId`TX8$wQthJ=!!@1w_F$gOsvL7pY*%KTBcm_XKVAGSR|$;?a2G|x-=kkv$x)l{6dUBM@Ey+&TDND&qJYe({sw7yVY z)WNn{K)y-+x?M6P$Kyb~!ozP5=bjSymYS=Rf}aKvg?1B4^?HhTEPj;fTzxLl5CC>* z$(R&Bc+(*O%v}e^y$K^V;)?ptjfDi+mlYfK)l(Uy#`7|{6Z0snms%ka9EPt0#noeo zIr0NQm7)!C^j-j(Iu9@%a02`4QlmK?oaywcMo?0`7r?CwgxwF;hVmOV7O9Z9rd(iptrA1qfu5DSVxGB4+g*Tx0@+!^Dit?Wq&}c)5uV$qM(MbNN_PvgmB=H*IViy+QD=&a^vj*cqtny=M1ybz9mGF+&7ozPR+0cODjLEiuct#5w`ZG=!>rzCU#LQ1iQJ<*! z4|&T9D-@j0WP|(RIYZNvbD=X>cfaBJ4MqjUq4Nwv(G~{84NwAzW`1Gbz5F7&K1(g5 zQ-gtroK*mF2mQWec3SoO+`?n9IyDRSGmqwVJjxHKP#k5>UQ{;2>gr9qs8j`$RE;s| zR!v8*GLkmb(9QaVUhg+Fgd`|%54*T9^5(Nq!7Te6rrPHmH+v&mhw8+J9e3tz4R%>< z%}})}j0$Hgm519#&+SR##%KbgYRFjiO5=Pob$foMAaRK!3jDwuWCM7oioxqV`~(cE zaXJTMZ@pQGkSvu&ni5ogYl;j~T(>ci7Io=e$?EpjMq8;K^WiTqm?W$4eSn^9flXid z@l?PaqXSR`7I7giHn2fQBh@6PeIr;+>BEZHO+sl5Bl3s3j_2qT&F~~m`;wGf4x>vI zRsm|p=erfijAe2+M!S1x|IZIU+HMH=omRQn`Op1bh$|wQv@}f-oD7R{TD;0hmx`d` zM53IQ@dcX)H66N=9_I!|1-9LZxgV$>73XF_(*C%~qo6^)XyI!x^@%6E?0;?9*;YeF7ms{*~))F(Yr`27C z56^tX-7f;$L8T7a^dq?jwSudhD|)F&ODb2mOz|%?#tra0eA{aReQP|@R?MvCZTv&9 z`|=+=);4IJNlBVA(^%HT*8xPXq7VB^Zluv7^Rma?u1cH?t_1LAyppB%t}-2QdZjHG zsA^hxsr&;l>@ZXYzWi-$!c5&jJeDS_RJ} zNH3CN92_k#BjF>SFDwi1=s=CWe%{7F0-3ll>F^UtjSjHs!aD7g_ z?Rczn?0Qu?n$hAarlqDNw*@mlO>y&e@K`NKkEM9O!ym6FfFiCoj7!x$M^ZO;Fp6}a znLBP?Xf)7$H|mJ_H3!6 zj>1<}44oGunj)x^Of!T}Cg#%aMio8CqrhYl0QE2!R#dzk?iE8$I&nO%z1KqZ7?Ta> zKwDN)4JLc!JR^BcDhsHEFwCn07y2Gd;D8|btd8%?c0}G0OZb?xb$za^&^x6*Tara` zsn$)zOYPU8+AgDdyDZX^m4BP-;ib^$sB#a`A4g_EHsc(sn1J5h)9(=;OzRWx=iVtEq}448{Ts z`kYl2B`oyCemnJXharwwCJG|;2C!Ha$%ca@Key@DWvT?6Fd9|$F2tg}5EeRPLKf`Q zrrxiH);34B#t4T#XfnxN$mMPWZu8}Gwo0URB_`rT#Gv~8X?<`x4Szu!qk>RCPEle3WF^vVKcs1|?srj^np@1=d~ z#ia=P(9Z?bMzoqGjrAHDf`40lNZQ?tR&Zo!gpv8m_nL zFR3WoAk}y}B(A?G^z=Pn)HFtzHL`G+6Y!nQ)krY`C9W5Yhp?}MkRQ~AI8zma9= z|I)lV#Jc}&sSIdVy^l61l%_mVMJ)B{2XrtaDVnt28O z{UOSf>+L_B2-6N8Lm9n<>gCJ$kVlHXA@dnLcAfJ=183Mrnq9E-KowB5!_z%9K;lqqWz2rvy9(2OZ~10bN3O&CPtHze6lr&J2jm&y zl<)grf*l*_z(|8R(w;p?Pri}w8(&l);zI#LF@lyzgCm5Z#btS2Ovl&WQLolzX`NRLvg^5J8@2nbe$5o8v@CjzQu z$i!g<6NKeIX;fNFmNr^fd$S18xlebsic&mS@<|kBT2>-ykxT^+4PJM>#(n6)uZxD4 zxbsbHCKJs`9>N9J%S@Wg2ugIxopdm)F@a50feInRO#$^j!wpTR8}}vkpcsbu{GX4q zF?pJHvqWK^-w~M#gN*?7(C3|TCpFCBa7H@X>PYXp5#k&TBMNVMt6 zlUuKGri2x7rEiiGeM&~nJ;k<^88Remwt19yJwQ!7Wg3MQJ%c4zU%`r8O`>P;K-Rp20c=fKr46pWJRVzx}MTNDcd zt~_eIL`QxgxWpL=t57Eh>cbZcryG|YqKrkGYFJJOfpAt=_(ytaDTYfiX{7_E`m;kk zKKjcEM_4>lCs^;sml$`aeG@LRp8fO!EfGftpsjR1ThS$tI{=_^^V5=)W^7F|r%jin+?6YJVBoC`sB1+4-Y zN4TZS*Bn%$rSJ}#Rl)c4Nv6Ix93v~Sz%EWbOhtu2)Cy)Pv_0Q{1kC!y{gLBX1BU+z zM>B#{i@|jRaJl>FcA;BzTE09$brC?aZ5^NfDq!_d;f!lC+-<53&q?mIhFUQkww2LZ z7<}rac10dLG5W#krA3j87q7o0w@=@5^BYu$fT-kf_v*LpI!@qLXD(<(Tse~X!vK(J z1IR%+@P-;Nz63Vj2feWcRCEUKxk$S0b-}1H%*9H3i2aPd~Zm-Is7s1pA8gVzZ`>rKw$e;&xJR z%OX{!1I?^uNTRHu2<|&^3$8(NQKMvoPXl3ynCNIG3XINqSEjWqsr^VQdAXZzG>>HZ zS?|@D(n`0DWNpk>jW46i|66j8CDxTHo!()2OHnlRXf%>vRBckAnW%Qn6Isnw;5|L#hQ_+a3$`rWGVGyD&F{Qmj# zFq|`wzTM2?a^FYdp0T6tx!}yr&~3fa(p{9ZLM_fbY*|jBcdJl;YicYiR!)9Cyup|d z81NCfiZ?##tSGZHEy!73^4t!=R|)97*||>PCfQiyW&0#br<_(rAcP}1Oe(eUP_NsolAX2w zB)>?E=xCrJ?HmTc%bRXqLWY!QH#)53p~oDeRG)xtc`d?S05QGh)SOvOv*DK_V}HHC_n{2EHDR}#zYc;?8m1xN5oF8o9;VKJiRGQ3o-D@ z!Dhar_8=U|W%ehTlK~?yP#JBW3!p;y(v6R+@YwXBO{?NEZba#k+gGIx=t zVb%9B3$5hQ&l&3ps}}vTHKL2-W}7rO%im#?Yl_MNvv+_=!CGFsv!S!zZ{C&;*)Vy{ z#F5?PVjnA}NmeBbuz^cg(#hBvt5!;tL91Xxqry;obIFpny~}coZG$YJHRvp4snwz{ z!QGIJ3ykLgH;w-QFl2qD$wQmFa*Hq$&6^2P{7P{vYKrJzq{mb{<(%%xjgbc zEv?CtGYP}v(fXpTes{9awQv7LRmww&eiRSQzlLx+M02o)4}jgL%qQQ0wAsw z-^DjCjxyOzsLUYI#B!3@(avtHFSGUZSRKC*qHe@V|D7%Z*aPRe(iSjn)544kIHdW*CqQY0Eq)IND`><&TJ1; z$n*ZIP4_qD!*!)8>1u*P0~XXYX&Q_+Y|t(LnsPq#w$MhRVRis(J)oguufqzXdd#$-1{@w;8;$;^~f>EdMfF!c`%X_5aR_< z|8HHnuAAK67u4u5m@9*}R{Ub~NAn5#ed+d}^KAUkfCK<-4H@qgwcb&VD0aXlB?k%f zWBO2rKC;n2yLN^rN{Rb)uZ zONAZRjRGz`0rdu={D4Mw*ObU~L4ZH8GPeBlHRa8@kF5q||22Cfa?!*@of>$%%UZsO+Mf8QL)7slbk( znh43fm~Sq>EBZ#W+nU<6fgAUUhk(&Hxnd~wMn29s)$E-t*2iffvzSqHy)E?@3o#hz zoIl4Q6!fp@x2PUniuipUU4E|o?Mw2nk!@}CwFTnfbn!oC-^hsoxuh8!_F)`e0{f#f z}El~M~t7nzho$ZerO`SX67I7lw)dYRp|C(0H(Zw!H92h(-{8Ipt%f9d0y z$%s%>?DESndMaz)Ft*~`bXl{VlfCMGB}d7RC?9(lj<9EtYfbV*!HE)daa-wz-}VtS zN_Rd$88hKV+*GquJlXYqzV6xgzg#v-1q(+SH0jE;EtD8=Z&B|{kS4S1FfpM=xa9`} zQS)_bRc=SmXWBMI3KUIWLBUD1EmrKW%it-mg(`-%{AmCj_peHsq83 z@J4K&10nRiFPWUQC_+*ofHVclh`!9%Q+a8JqCSrN~kd*t4F4dznc)w`olC=9_)WYMHdv z-*t(j-AZ1i`wk|KSe9hM9rH_o640L{;zmM@L_PJ(>g`ZXgB$JmAp4+wP#ty6 z#pqD~uz5|c$;DXne_7c>st>)#s>bl1d`clD?~uSwaHAOqekPQ@0^9}0+yA85ij4mJ zUxlw068DY)zuB*o-%I-c$%p-47xdO9`v1!&W=Z82dlW_Z#U7i<6c&GAb&JyKqJg+?U)R0y3m+FdM9TmWuN2YFKtCGCFzi zdN;?I$|q4#weI%Q$XAE^fa%!;DG42sB2+qeBYA0|lP$AX6VWmgq$qXj492YIpixZFYUm_5Cd3YV51csivu3H)Es%rggstSPFox9A{J`blz|vO3j?sW;2jNx~2;l~n zivLLoTYTSBwOY5025GD%LOj9Awd%=R;7~XsJZ!oVWmBui?u${#fxN0R=5VO69T!)L zSAof8Ri8g^N!Y9-5~`c4AGW5d>Vym5mvj4`)Uk07^CYF^C5~l@yV5)er>-taD-S}T zSrO*AqpHzHYFXG-x1~l{$p+g*FGdiCf!f}WAQKSoT+-C{-^yWGAW#gepe+@pAg)M6 zTx&y_9-NLrYTXNZK$zro#c}6XBO7crosC6ao5mpa5!rp7LQf4JzUrQu&fp7l_3->= z0)WiXV%fb@-RN9+eL385+`q}&155ZL&Gx0cAZq6-+v@=zHfMXKxAfd%iX2a*rTRNh zBc%=L@BO6=0k|J`LE_lGGC2TwdBETsWKq!CS|&+I?7hr&ftyerZRjo2;?I z_E~pf6u_-Ez5Pw;O{`zqkvy8c$^1|%OEyIM5A!N`QW|?%D!}R4c9jd@&8{nkJM#_$ z{`4nB@4Tpw(@g5YahF(vk0)I~e&8D@P5j?@ljN%Sg)zmm`J zRqe9D0Jk~d7JXqd@Y8Q0<8kM?wuM!H%)r5R8to6(gY^d8t_3utLkoq`T!=K44Yo?9 zyS-iMn@mNQ4EZ!e1OO8*HzG#7^DY;|1fgQ5@4^{^4^npi$kV~HU`fep(eVgP1;A zs-sHOAze?QRe1|rmiIAtZ{fm2alklNLwD}Hk#qO4rzI}~`+8{qjEF-AUM3oS$o#7& zZFVabWZxL+68qt?Rh<31*P=>QQ^8|&F+eJ{VsOCKG0{%)`gvY8iq0GR;zi4)LE5b? zZRo0XTK0nIxndm+G#~%iW<&(i9`+zt`Nl%X@BeK_gO&Qj&9W!F1J0&(|1P~1a!weG zc>UM6v9li-2Qc$eH4mxC9+JmK9Pk^iIUfxueWKe6YJ*aVZaSHOF>U>F< zs4~N+`_l6yoiz~&)8oQcO$c5)6!ZR^YAfIBk!5~aXumZ#5~=Mm-qMV zdNQCvH8P^tP!I-_K}4AtYbz{AjjOk(G0LGtC4~%yDjkF}!1=Ez#SBr&oFq8cN%(s( z!BCzwD?Vk?oT?B=h-*U@X*Fs)8J$a6T^EUBQosK(=fVo>#1p;Hg24WPPF1 z9z|48UR@UCwMwVYP&q39j~ikzP;%akLOH>GK~!BpBB%h$uzp@J zC`z}GRq-zPhDH$vK+Nw&&`I$c1JqKXgg|LD0Tfh7+Hk-=s4mGQq;IsMxCDTl7^HsB zox5>Cp1!nVh>DOLT*@3@M&VuYUjWO*F_4JhJtze~QzsuvuPJ6JJ_%wrN{QcdYYv&f zXm^<-@*p2Yr~bPr#>oAnm;!8uqb3K@f^u#oDeK+tpd89;I6%B#OUrSZ5qF{}TM)!B zOOohW^N2gyQw`3!Gu=mv%L-x3-<7RtbejS5SmPIl=5nWt=Nc|{Zy=Esn-^XAllaSL z>i11+_VUBe@AivFkJ!bt_>Fd(b~uwQmoE1OyAAp?P>zi(KGv<|jnGw}ky4tVHkp{A zrc9eJD4~zw4Wr)8?+PmI_AfS)1yYa=Cu`Rao3CPX*R%Ix&#f~~hJ3&|>lW{|V6u{< z#t&ArVH{3fJph3&>W(!Qs4Hx@xgvBpnmS#UEG?SNQg-VW<)VG9kULw3S{d0-!0?o~?lG!=`Ih63tHL|?TtvAJR($hXWE#&K}i zL`WJK^9mS_8WZ@B zB?`YDROog~=Jmo}F!7)=IOy_ib#KlVQR?k?gLd&1REr1_^ytJgnxs#NBj^Ro=$Ae6 zBy_wLH?^0<(##ZzAXx&@Quuch4bdbF{p1LE_JF-;#C!Sl%y)HcmYI9L zd0w_uE-!{j{yo^A&+ID3Ct{i1T`6b>PXl{iqR_vM~ zzDSbwH0PY}B|Ts_{gJ)#yUE7cUpu~Xs zcg_J*L(Y~cwW$C!h!%J%9p98M89+G!i<9@`K;rv}Lhol?H6X3?@X z1NFu)pm}Y`pmAJ$4`Ci$8(&rdPU-Ug?7$=IT2v%x)ZU1;M z`Rs5zLmfioJk_nQDdW*ue#;$)uD$UM_Sg`L;qGrP4K6^EoW=b z<2}ipc{(Y@(P{W|ykzJnTZ&?s?4trG@kN9su}{UL5$>V6guY#>xe<9* zGPMpNw6I*u<@bD9ajeBA z-{Abe$In3|W&v71&%56b+V1~_$NRsR+kc*SdxQT~p*gK_{&T;fef=rpNKq77Ux@zG zxdjjt*l?CnK^IOu2MZxoIFK+Da?qHmi|t(F*Vdm`Kf6B=ec~2YB1&1a4b0aj3}uD=UH{ym17(U%9EP5F^nTXPwPzH1zM zW~FVG)8p~$p$Ki6LYpl(8e+s`70pl}de;ngwE~A$C(;&`5@jlom{&43bd`^ql5c{b z%`_8=&sMH`Xbh>7kiJXuk`gV=Coo%&Lwb?miwANaY4C(3AU%GYnFSdVzDr3IJOEA* z!aSf(FxP7w_CSFa#qCC!vx}3?ab3lL3c*)S2FNQki?}8Hp3reZ;EDngt0zDi;yi|; zND;uPDO`X8POb-LQ~zPtG@FNj%)b6%=r4JtiCtPKWKjAKyGFCNKHr@X0rH1kbIe3c z8n2Kqak(Jqg?FAd$VaYW~LSdXu-Tg?XgSWc-WE8llT&1AFem z(uvg_JZ<$4g_=wyyz_zqu=KdPa~V%`h$9hmI&OwIFWW(B1~|?+fA17Jm0fok+KI6Q ze~g8D@a{wU+tTSWL%V~2&ya1~_vPlnf+rz{q5)ycuB@|fuQiWhdKk_051X&Ga{Qch z&Y@snnZT`m|6thJfrxl|@H(->W^~KIS>#xk&i7oamTgLEG1B$kofVYaTn^E8{s3_GO^-ERY}z(geWFp8h^6AxTyj zyoHllAkWBurd{oLzOT-l!1ntL#&JV@Tf#G;N(yLa2^>Uiaug*4%2n~zQ zXR2u0a5{TId1*q%4jV)R)py(X``mBm`}p~}_iN?%pFay-k#!=YTV=0WS+$LH&%<{BOEv1q@Tz~%zZV1w-`uGJ77$n709k@tgWraHtGF^TpzdM9AUrhK!@xc_A z1{kU+BdzOcP$)#WGwj=aEXmKIeYPjRM3wr6c@E~58Np3LTM3(ijYhL&2R(aVnz zG>C13Ts+Z{DX6tb3rjtBHZjhL9>8$3ORpbq$5ba^^-rVM0}?1;2v#ja9HSA&jJk$o z)h$$IRO85D+p;GozUQbGoGNk?T~%(8w-4@=a}H_qHU+X#K@8Rd0d**d7J@a};>@rD z$Rg+oIq^rf14BD0ZgRt@*1@r^d<}ls5Ra{2U42k`i@Q~Q%j?W8r8eA}Xe+Xsb0D~> zNN9VTOoD98P76yC-O_=Q4w-K34-890RVP){=7hVw)h7e)bfoV?e_c5|}`Rs|g6Nu;aIAnj|+hH@eO~ScB z@P+M=`bw}i#L$IM$-n%<+NDK~te)4!=n>#ho~{+DttxI%&TxI|h&e1KDJm2<-f8V(|4Uf;cNvOqzQ|%pbS>az_q}^_@^bzC@|2flhb{To zC#y=Be6`dX2KQv{3V+PeFKz-JXXvXl1I4f5+dp%*&t7EtJEojyo__D(WlA<6?76Rf5nj5FhQ9@5WigNV ziw9_EF%a@RGW#s`C*{7`(AR&3g}cL8p@rXD)hzCAp7t3+yX5Vkn z2f3eHxotECZHz=vjLnitNo(>k&2!s31#d$Pk7tJ(2B~>mMy~zCH}lpqSbaHpkblKu ziCZ=MkW-}^Ef{f9zmJn3N_aKYTs)l6_s!b>n%N|Sq0(cn4i8t(DKViq+GDipLU4e?Pq?o@?KV{P)Zhj1c_vDun_npM z@#Qg&WeEOBxOF9=I73!?)(sjpCw}7thlfOpo?Gbk3*`O-zoHiO1s4aDN}G`HKC)4T zq$z&pn;(r+tZ_2JCe3}U&c-X}dnD@jMK#Ee;+Q~Ypc#E4TvgU)x-_PiZv;)9sEac_G@481u#GoTcu zgSQ{*jO~bnw}1BR`d>Qe#lGFsV@;U}WJ`(K{Mw+!+_9toT;cr*x&?QL$k`v)*#v#= z{~^jj;-B^f&PVTFi7!@Z^T5#uaVu#sd^gkE2EJm9?Sb_P-!f#+1XYc8{!`)Mk;5lz zL`-i3d*Hy>k#Sf!@ny+{w`bapn5TCK%w4#;yN!aw;S&z4=*0Go-`v?68xW2xZ!xr8 zN1gxsya9G#Mn#^J&!A~Dk^?;4x(?a>;ved*@?gM3K2}e<^^P3n1Ie1H+QJX|TTUOk zs4vFVmLYfbUQBobo~$yzzlD9isIhZ_9DoUpwzUN{&30y5QbC%T?XvjHzEr*0{ebhOG9fHdAcNXa7mLA2dwwTH+U&xEC=^?c`O`80C}h@g z^)MwMl}sd(VOKA#cu1j9?y?~SG-)S5bR^Z7H%hLF^-UR>cLA@HoQ@7-U@xgPc#>o> zFLm4&f2;Kn5+?(jZ=tcmPOUDf}P@Xk`qYGCGtlm^r>lSQp{qOTbVoqhehX z)YZ&bSCS%uMKV`FmQWzlO7z#&&*mhn?3ahr=F+7k+zfM40)h8qU`{9-m_(V-EoziX zPmJrR?6)QMK+Y}Devcbq6UUaCWWN1ic~OI@=X->(`>!E9Tt}`SFm+M_-FHufeZbfs zA0kBJ6439WCZwoaTVw=w3;TYzgndZchQ<3k_0L>n zPnEQ==kjd-s9S}$hBqpwnKj)zI2#QN6fi+!`;#nOMAy)ZYT2SS*i)>*xoOoJ&`Z#! zA!efA@pj2j%%TKaVrTn&ub$~G6gcnXJb|s?N-QUP`wGvwQi8*u%)aAUv6G0+z4DG< z{k6DGnwE+6vaWbAPqNKPdn-&4q~aPVW(S!BSEtUS!e?w5+!wZF&g{R=km}r7G#w$l z5Ir_N5RPi-Jd|lvN^xaS$daZA2f&53>=~<&a2G~;1D$!*aDxnPWDFd`TCJ%_O)d*D z8MQzLSEM460-jgy>(}``VlEI!MH+x)T zcJU6EEozccDD4>qbg|3qS5=EpXz3E*U!YkrnSRi7o^N(B>+;30?VM*pf3dn*|IC5pGkQ!@@J!9;5sH6{Y?*42&>A=BCQ8#}-; zO74f>Q^xB)BR3w5>?f9c5@SkD=L1{dqPOy>xk34+rtkLhyN^9h2dHGoFaJxOMCF6- z31-d;sEHdiDa1IGu4P#z9E3+3^y5&-ow^mdqsAb&R!>n?q*l8ko{sMKLYvT-o{ulj zs7&Ak-U9Tthg{Jr9pM_Lzn?e@PLZHhj!yUds*YXdOM1_HfMp~&o zuURLvTLcW7XSQ09#gy z6Hi*I0Bq6vr!}l3DdW#2GN;Hk`_qZMzf~S&ig03z)9bRy<{l8Osw5mh6VCWipP5DT zy?<*drCDdqy|RFsw;r{+>}uB1%_+*DqJ45cO8J^M;2v5Snv7NR?<$`x>qp-GqKx#E z3SyjaOTmqanya}+V#SmlA z9!;DMg)H2;ivd)|znwo@a=lu#H%{oJtW;l+wN%s|e}4|BFuxO$$n^&@3Mi{x-!g)=U4 z9+%qF2EU|&K>*e;G4iLws5GngFTom)AD&O5@EqVfipH-xvev`;8t zb&0X!W$ImfztZuJKkA%$j8We4cwI={SJO4bzTExtsDBr8r;)lRib;Syx(}o~-a!4y`JI^$uE#7B?W745e;4!ZLfIE#i$1UaNL_tvqtPy_(Erua zU>J$C)9!pi`NX(VXi-dxGc*(8Vck>kGsH)oMHeR#4knT#CopDvp zVhn+UDVYz=T7XfG9noMlt@9ohbQYnfynOd?zbRkqvgqnNEvh`OJtYn^t(^USz7q0p z;m7H|ULJH*Z%52ZB|Q?8Z0`1I8~-wDZER_-5_!d-B#9-}tH(XztuA3N7S(N#?@L~K zxB!zqpI@bo9}67n{d6f?Uh6y7)FqQD?wqY1ow4N0oDOp1wW>s4mlAI;?`-P;RAJbAX)LhSm))N+P~h z_M+Hqj1hm2_4$~b5NeTX_;a}y%d_nCZ2kt@& zOHW+`Y{rINkXl(&hXO9Ev0DV=e>OElQCO10kG`M0Va3MX7aSIyW=kC; zo(Yn-h7*bCEbZi{TD0ztHy68@oX4kHb+1ARN&9J|2e8NY>4}0!o}MqEk)%n9x4{gO zXex83nNyQ7o4$#LJQPCx{c*&Igq{zC!o$WLju6K zTtx&ou*l(8P%yyG;WJ`HLU}}u;x?8PPH1;$)g6JHjwtiWyBwXdCD#U&2}|4dpFHp~ zUI(OaC|T=u9)cjCD+tL9=rkUxDWKo9SjW8BB{qgkcKd1E{l)XHmi;WLKYcSgbO)C1 z4gH{wJ@i;OtqWUb8_td##5CG-H3@(0hV@kUTc8J5N3X5;`*XUS9}l?rUpP#yEgiUz zJ9isjZ&x>lPA{QZ-JK@~VOmgCKS-ZkC{_t5+esYarmCOw7aE?IS&%f!n*Lpo^lx~w zRX|1;SRb$BBDO5pacf#PVtYI87R;y*!5)x+9(=7{9656cWag7!?B@EHT|Xe^esX(vf)u^GY8$)QR3wzauec1BE&<8^{>nzL7vzNZ z#fc9S=1;=S{l7m}AbN3Jdh44Dl^8UVw^PYeppxP^7 znf~Jq2S4Fi(Ox&mIfaF~E}kuJ4h|k3o^*eQQ&}+T$<%(7578lY$*CX-PtTEG+J#IA zR3ZcSVVtTws;|gtuXejKL_IXQ=3iMl#*xFr$ne+Z^`0;zX;a{XhR(YF%FY>u`xLo< z8(w&&)0G5#1b36EeHoDTonNrW3n=LVIUkw(*@AMt4^d&>IS0Rh%Ht{su*p6dkolB$ zrJt~UX#5rV_6OwLtwyBnPu}B8*IkxZWdu?k#fCwXDAU$KFPiB_muJl`v{=~|0){Cu zxG$mmXxUdn1UJpriVE|Z;7IJd9s!FXk8Wyb@)LJ<0d z_H!3GvVt}u(gA+xe|qZ`=4Tm8h>`k$j z3=eTB>c8In5o9^Uv&nlzQ>@Hi4+9W^F;5t$nQMK;=s4acK@*)s)?hLa`_NOy{t~s( zuK|_u7&+xL33c3KP6AMryo&@0Yh*g*2#~#bCPXZ>wHfU@n0Es?0$7kqg6lb$ej&KS z0ZVq``AarC!w+Dv=hyeIJLhb|@Obl#ri?)ons>Z>mm92R!r4E`DE>*W^!w#uS+Z*6 zD`G;Cu7(b7xH|!Zs%4n8=@;zbsj_Fbg&pboSDCJ2_ze|0angYE@RuGyA5P1kvtGU+Q)c4`R7NYI8!g(bLPs9BY6q=NultI z;LN#8k3$;Azdi%V7A7=aK|XRW&-6rczP-R5=?pq^L{NeiSZ-$AlyeL*b^jr_*4y3j zDk%Mzkmnm$xQRI(6-Y$zMp%bCpU_<&*I@1CqIk=t^-a7s>paFz++-Kqe;Tz zu(KK`#uCrRXvK+;Q&hr%EQ`|*fHD!vA>~KYHUQo1et|uDC8FxM?(acHV8h}{*i9t? zF=2U0tiL9MixJSUZy3ACL>b>R57Y`Of}W~gQgUqFxDfk$(;}!i!UDnplCete3i6fR z!^8Uyl*w19hxgv{KplN~{7*Drs&{wWB3MPr2gW3$V%hiP{yq};%hEwXOT&za%S*jZ z`f`q%1}8r0VMj<80U#;OI``yEfmT@pSc2*#x_flN40BU4x|Ai}$~%`)cSd0hoJc}J zx~|EFUfKT4Qc+nJc)3-fU*TWKPGUKsaE{s`c$){fl<~6m zw&+WzQ8|YVlG=s=gJF`9Rxe2Cr_K?5Gc~2r0-nf16UlF88Uos)LdbEr3*hp2U?9%O zzq(gI(HIoQmShP*fr4IpQftpzYf=GFaJRD-j%SH8mCDE=4)SXMtZ}l#qiu;o3j_LK z%+=@G@)=dr1q-Pg`DCuk9lS~vTCi;x4Y?xz(MkK{*tUZ*WLGQ0SG*JEZzBowky(&= zX$!7H)zkHX&>c*oIE-5DRinX>Gz~1(0??GFh!zD-h_H#cNnScW7P@Av5#{`*2@cl1 z5j%n6LlG%nMO%5#1PdWOsEx~!m=p{a)8&LybF>@{_na*5#Ze}3$FCATBqrngVo{&` zD|l$QnEdIF0)epeiK(>$Nl%cnP}U6mAH_aO!IMc5iiq#$9c4h7IC1<9Cf3CnX|M>*2%8u)2Memje;|^ z`mH8%DdE$_kz@i;ayTkuG@__HgwmRJ zn`Jii-xX(pHq>JjR9g2WDf6K=`OW4rl1U2nLUh4Ds`2%rtbQB)d$vOh?XZPYKFw~+ zgyK}gycV-`o74xV_Tl(JFmix`Cmk{nvJBTsBOdD`#a5c!#2-yp+u@w;@KInZdQ z0-#HROVS4N`o}HL+1xQ~TTksVrnoIh+z7uA)$5E1f8jD6>*5ffCx;4>S-_$gBhz9D z;`oHFJdV8vhPsL2yXa~vu{1y#!)pDe)##3vj<6!EYlu*yUvIj0TO;}U5&L>|H~ur< z%iD?l$eO?D;>T;fNEVDfOa#VksEs}K3YFb+WxZ50n4&7ta&l$8SM1-6(+pH4JL(V& zhHoymfD8h1gs1pB5Yjk3()CsRJc9qY^U&+YZg17Z+Rf6uWl-yd9z; zRfF%Uv`%)z0C1)t0HsmtTVDuV!o-s|_jTnulNaOPy)D@J{_64z8g(=JV~Srsv#}Mw z3Rbw8w|V&mg;F+bC;uF7vifvqB({_D`vMa00`v4dshFtRe{0GUTlb8mW(f5V~X`UAEvM!%dqo4c35RNi?|# zKBxO@dbjjCQ8)_Rg7?CA<#V>PNmi^QxqyN@D1jHcwm@{lue2|_qaIltqEFa^40vy3uk-u}2$Pa8g&LY1lNA*wTL#gkp2b+#3w)kmfRSzbjj_%ZFR~WE|f@*JJz@78)M=Wg#b@ z?c7ejis6@NnIWucP~HeOGnEZH8F2MH*bS??m4aoSucGCnBr?m(go0raeod5h&-U{G zh9>FQ8qU?Eq>9kYL%KzGl|?;z@o&no^#M;^pH=J}{VNx3WI#z4?L01gWdyD!wu|6hiO>??l?66_?31kz!?@VpqtcnHrbdWW(c+^cF)zWSzg&{n6%Yj5rjo&KhrPUm z(8909NE~a@=^5b1UJ9r$zPo)OJ3dee{WWz;E;taJ7j+;&z*ITScRTa;q$84-x zb|kjP>N%3PW13vk+>Z<_#B*k-uMg4bdq2*~ikUR{rN72Z7HpL!XG+r6a_eP%%9b{f zhT^tLUo4nAvb_0eNYZ4c0T(K-g(aZBOv-+zl)R|QlhT*B7tE@c__k)b@NL7$-~H}h zfll>SnC0VFr3GF=tLi24F5g)oF|%XdZn#Nk`5<`96rAN`e^T-OQ2UF5b8AnoROa-e zT-JoZ?*x3`pA-4Z;QqZ^VFzbKM0H_|@@DA@VYTmI=OKia?zkPq4tQaVf2WG}@~bxX zM39qz9ee(na!}{1k^Sx5vxr7ecR6euf0JPbcOLAB?{b^^V-4Ca<vy%&&Zk5 zHwPC2?C}E3Rnj6~=pl_qC=V!B47AcSVr%e-GmZ^}r6Zw^MyD@1bAzjvZJck61a|v# zy*#fjq)j}{{(QK@%2tpKqi7q+nloJ)o`2bi>)8BuTpqXN{VaJi77SJO#97x-r83+A zwfwHbHy5sns=BaE-zhr))@5=AQeGjEL6{#=G|b(d>RV*%xz7@X*Qqi;iNv} z9!84jG>972t};iMNm2_T%3(w&@Y8v=&YJoaA6YuSkb;x>KBY=^7YvP%f$wn|7#5MW z8?KCKKdd#LE&TnNaN4+28sL2eCKC*ZYrH_!h9aPmO+TPY7!anwH9r8YfOiKZR|^uL z4MYb5!hxasOQB8#bhxb%UkxMuI1$myk_5^OaVktyAtE>qGu+!P0cpneez4{gD3rLoH5kL{D zkhEbVN=T2u=1ePv#U(F>PVeWa5A|WgsIU5W7#Gy|g}HGZv-|EF){Psjr}sM! zvkBvgZsdCHkb0%lebg62C)-8iyR6Elu2&ZD~KJ}b;WD(9j_j&<*}AHyShvE%NbjXUG%$ z2RGk}V}`pIH~I~C1shImEKq%Lk{tph1Hy+666T@n#EN5V_l`#Z`NNm@?9VOd`SZ*# zEJv&k^sEgpulM(NFCzuIOk*P&xgd(izZ4P0-23I8rvo{C_bhNktLkhn>&Vc_@FOKK zMHiFNseBB!XL)>&v%;CM3|ZW)m|JZY;MmI24d= zQ(W+k&#Mn_&Ay#BZG!ISU<19Wg7a~Hokop0jat1(v=&U{?X+{$8DKgFsZFLZm)ZIRBK9@y;`e&r} zH`h2uA)Wz7c^qaHvZUcKuS{5Lf$MP7i_hpBQFE5#@R?u8$|Pbyx;lg=PS0%Q&@}fd zQvhlF+Ziny9R$Hw+yEgR(7*Bm$S}NjB!nT_P<36(`GL(kCD;@$cYrtTl7naqXFhiBwxJkp4Q>g{!1;czgphF5|;x19vOe_Ht{T(^^VEf1xH ziq}j`BVbr0l$x9CU#E0_Ax_p5HjW|39GI zP|EO4C{4I}fyf>tTwYDIqs|CvL*OoDNN^H{$bcsd(&~PZ2|1TIHEf=O5!PHr-?3;+LV(ol<~byRC;x zKyLb4ibOF^tVgkdBv4jR1hxa_n0O@f{lWMGFuppY;IJJmKz z4y#UwDdAqw;1(8Y>ks0AVwWWYKfBfqD|i2heJXK8O9s+ZlW(kuI#B&}RYvm^Z8yij ziwofrP#Bd&c6<_}#6oa;CCbVHu)V*+8(-Q`Q1UetDnb#nA~=3W2~t9C{l7FXWvu}@ zyw=S&sNh5vnk~y;ujaIR47tn!<04b>QyFhhHwZMd0ca<{I7hqf3*R(P+09Zq)4Og8 zJ^Ec5FEwnT6_U`rNin@zq9a(T38ZT4H%QZHFaYa5#7GKOhO^K=Y>trItal^Pk-8U5 zjW+b*LRS)a;I*T;dnu^0zRgs`!p0o?yTAe4p$UJ&jwz-nPOd|Uv}?o62L=L9Hl7NH zfrPXb=m$Z)#1O-=p+o3v?rwxSXQohr)3RulJY1_i5{DS_W|#l*-{g})r<3c6C3yYbieakrV$rm3C1gaPk}`5LAWwPlKKK)!n~&=iD}v8}zpIJbUENdlmaeyle;Zap^%jm{ zCyjMT4+j@jVbsp$ph~xDO?`U@|C4MJ>o%-M6$s*y6?F7MmicSAHU~Mdcg=hLyur!>RbL&Yt+l4|o%u(1XSg8BoH81A0K0nE z$Vlch&0X)6)`r`|c8VD`_jaBI_ikN)!SvSkQ3m2JBFG-KM_}6P^PMSR*o~T&A3>BlSm*N+ytB%nlEkE;D#k z2WU`~uqBL9jW3l-3e;p(sgb(M52^wRDt9%S6eZ%Mmo*;0gW)_lh{qMMO&q~6$gT^4 zP{t^gJsKl`7lN3FLp&3%4dHx%ny&^SP$o>NeP+NcDcCXamEy$c4T*+85}Ei%@ODrj zzH9;fTbRx?kTFdqQo#m03XQ=7F%hC|FId^D(#KS@aOU4B>Cr`HxJIz*GK`0Ez)HFy zA)qi6E>t7=S~wrOQRQ@ep(vSYiB+2I$iV-pSq*QK(E19j%33d)D1R)18>yACN8!%AoQXn*T)r8)a zNZ5On(vSx^_AQI7mq}&xNN5Csfn*x9I+N0z8x&4_S)~II{7uLRWckZRU(4yOE&}M~ zPb?d0I@KWq1p^CH7 zgv1WZi_eI7#%>Hmzl659Fq%NWy`92D0+x``9oIML5Z+#($oCt(%+I04R7!RzvJzAm z8QdBUBU^;Cd>I?~+7%Z-;-gF%X5?h)HX((Tzh>M=}9s(i~f*%UNpgfHmN=7tz#zK&}Of}NnuZOg!JbO)C{7kXNCKHsy^q3KbfN(V9U-4Ydq!= z`mqiROU$hU=SlKi33LnE{LD5boYoCS^=!sw{Gn(RNNZ(s3Dh^prWMoeCP4?!Mca&% z^iOk#Heu>(AAJqmH_(~CATw^Urc0cslgqIqiic}*77H6jXcq_&2_9noF(k$C)FYb^ zo?W@hQDUWdYUB)>zt@)HatQ8R0)^>#vLynwV}6IGdbW_W`ZxUxv&@1@V z);+e3EgKD$-%UN@3SmVYS13qi7CPpj$MBfg zYN2n|Q2Blugp94NCYRI2K2Do&`3?KbY^l^LWKB%PXB3M&rb9O?NZ2wE9h(&Z-eEvBp;ii*g8 z!MLwY98obY(D#c{ePtCDa2hW&%deY$>yKh*00Y~3x7v`{jGhDAXvToj(<8Cl--v|x z_a3Q+HiF$L^vbOlEwK#mlm86`+3U@?z={0V#!&AM$sd!X&VsyQG{0GNJj1i_4g|yJ ziHF*IxB!}#9m1;V{{vCp6Sh#&-)o{4tmSo2KmX%D za0sWsZhMON!;#?v_D9X@d;`_8r6BQvi9*nz{yK0 z+eL>a0+r6}LLHIfD{K&&a9E!Y#dfjYle$|ZdPFp?@+u}<99DrUrmh~fp7S8z=t1EE zrqtMOk+uIh1W32dV%4Rov<0ZM`39s1ebr_Ih#nj>DNBQayIWGgv+Wvf(=@*~paZQ0 zG(QK|>p0s*=IeI2nGaav-tKSdpWgW5CVG%*FPqqVS%PQU=!sfJLJ?yH$X_6>v{`bZ zI`^j?W;xsDjK}Lu#R7gl9xOI(i*ZxwV4!e<3DYFWsJfym;0``*Tq zf~!63YpJzMO-*d5@(c=u^=|e;tAG%Y$M+%}1)r-ixJbWghq>^E&m|~zzqkmwy_W&pjx0x%%Kc)IPq(foW<@Cr#l6=YXl?QJ>V&w9*mg3frx!l*+5- zzgNmMLG8>pW;sHyQK=fM=w$I=sB%~vpa^!l0DG7={hMrU5+9;594AF*`DGXg3M)Kn zw7&TU59fGM#fhBVcImMhQ6V=02`(|A_xEPl2Us(N=7}13_ieRIBl5q~8iX&GFnO$R zA<6`!Zl{`$43@II_o?u!V?Zxn;P;%xaWxJ`DA+aj?NR}Np!L^wyd!1Nq_Dr~c@uk( zbg)#D&>QkKj@Ye!zePC5KsiXeq<_WTwN^t-v}KhH!o>!BRU2e&kt5ALTxGXx(<5li zh;B21gCxS&x`M^CEl*9IngY93*lEwlPk+Q;nYV$~6nqkJMZ{bYfMxnmKKd>=_AT8Dq+$O8Gmfdb!z44)^eKIC@j$~H0^a|Zh;Y1<& zOq16~l(Qn5au>4r(@gl%`A&On^wF@h)J#ISOuq%XJOq z{dZt{uOG|COpr|g)TkJ7-141lb9jq>{{opgx&Sxt62A|PZ@0U+sGS;?UA7VT&PJrY zvMl#YZ6GLr*%xTB`$mt!M;=*8R2}k5dx!L78dH8#)3l%6(O@qVyD*hbR~X}l@&L+N zUMK|lQTSezvSjf{1Fvca)xPya<*K!5;w^;=^A{~y(cNBGOM+Es`vNh+~z22xUA~ix3}xjl^|J3 z**OS_qd;YPZ6@i<)tddgkLWZz;&zGKQMx!J3v+(daRz?Pug+u+x32gK4d=6vBjbS% zErC{ldn)u|tzl+Q|5&)JlQ2T$VJ{mQw+yTeI4hj)?m;LlbhynC);Zy_3O zaW4s3!IHbo0SvQXX859AZf#svS?1!oX%m4TXmOtVF~C2_S30LD_wY2D4Ct&1dBL96 zOPLR%fB5$M23P!oBV z&=0=Ov8@4}NPjGU57I%Ddrjtu2Cp9f^KpsDN~Y&;yuA@T_=8v;qB1v~C7d)dER%xt z5hUx6=hPa~CC}5xDT*y*k=VlrW22}DQAvsXAoTsa<^#AE@`^pGINbocnuav{%Z}Lj z`x(ul-Yw(DPOH-MR#d z@3DHcMvWl}J?)Hk2~XQD%L-C>^rEe*s(BswjUmrPbimQhHM?$`*lG!cgJp@0zgXmoiQYhp^`+<1vZR%Gj2A+zl>bP3`ZFU@nI$J6Wq*&wLPCP}NLVC`j7KVHK>16Ims(*YdS~EO=p(>GrX+$R zhKM5F>GzsQN+};YU=GV55$iIaxS6%fwU*(AM30b2>BlAPp>m3yW?n0!icl2_XfhI_ z480?8=$9ab7uh6xDT3Xt8Mx9Th7DLI(!v*&yiGgXcYYGZ9;}lXkbL`7a%fyAcg3HH z_92x-xNrZo&xC~xNP&EDogx$`Ddf3R)=@$moGq?LS$9^5g&I=568abywFt%Unr3@7 z<15LfyvL0|DUZZcuKdXjiVc&lSt9sIP6F}M;b^F%g3J1{4HTm-9n>Vhf`m;My}pw!Ifrr*$nDn$0xaWbeiw z@k?j<0)oxtKW2?RK={gn3YcPa&6!Il78>zR)ci82Dj*o&BCqsB`!RS_<&^K2GF3um~ft^=yEk_RZYR`4cOCt+%^h$mye>KGzaoNtC z669s~W?l(Hh(4EL$vl_Mk%61?j|&k&@%%VL83>E>31da2vfwxBwHw!#Z7cJ>w%C(o z@ZBJva6lmf%fHqGVoOX`f2s?sdATN4lY){I`+B(|+H)GvN}qn^?X*@O6q|vESkm^X zpQ(3N`xIheRL9@=VHj~-RBSS+kObeS@;nzV!DwIzF~<`=1*D?B z4fRRFlr`&GFOgK-{qhv+Kz0qPwZHHFH_m0AiQO>j( zC*y^33HJPhzd3emQT3$it@ z4_q{fNgI0g$~Xu)`{hE#q`7@^D#xC{rk;{07#x1K2-+ui#-aWxr|yy1SyvW z3aRDlqE}c8Dd}^y7`$^X&+3Ku{tl+`K*0=yv+#=bWs?RG3JSi@KhMm=jEk^8+Fqe+?IzwMaeDF2u86705?_pm2v1{B1g|85={k1d zjn8dvqVS}c>s=GYOMD`-?&)ptt8<~YEO^Q#PsTfL7?zU7JZ{jPseK*IAE>jrcsWd< zX7Re@cI^j$gq7e~XzOlFkP0ZOU()n-^Isy^k zy(wDB8mOyFmeQGOHzx0!uiU>wiL#R?p6W?yMFa1$AiWjCM8iqenA$^vPP-F9|30OU z6Lnp+Mtscvvdaa%yM-Y)UzPOZVl?o5S;ly$*2`ahVH{hz&#a=G{^`d--qxcbC^boZ zl$JVlvw;)^1(&wv>9*;~b-&bG<4M{X1CL&$+6+&&q~9@)dv+nyf_q|$dwbjNVQpE# zLY*WD91S*A*E6^K7v}v~nB#4Wi}9DKo=FAo8O=Ib#yh^~(T)22nfHst;^sW&N|pE=54wD!zpEQ~ALtFXXX64ck91O5`BZ-4mTZ`jA~(HiPs zR=I%)1;bZYE-&siEbrus3`B*JfjAujW3QSnRp1HnEyt+TqK#?sxDHeHYvwdnD7Xx)shpd-M<_7!u&@QXd_xKC_Lz^T zQmBA%@dxw;nRk5y^p-Y4ye8&MkqDvr^6;FlD?a*>Fy+>hDb6;BRs@c`d zn~hqGQuUke+@r-Bd%&+Zm;lt{&UDgZj zSxYaaT_fkWGA^qXMOU}V-C9>hiDI$YGh!!j=MLBLY4701ClEWDcI8b{SAG~2JBBIe zrJ}Gt7s|APNuB-dc2rYbpi?QvJ))F_Iieu!hy$X#W$uYkUsUWhRV3)SOA2N1+LYri z5A;O6DQ4+-UWXo07J}FsL@uSR8=?vwuh^h&Ai2nh;>G$YwJ9i&@OgoXvb&bkO|HOi4no;qVgrXC5u6ij}JunYFm{M__roC*i@GYxDu^-tu| z!omsXd$BNlqIrmT$YwvFc`;>7-PklYm6Qb2Db6rYtF)}bmdpPUQIJ^IKvQLO050je1{*^n#lu}n7Km-Iu`FA_;zcSIy3 z7wZhL+(Pn-QD3&A#Y^fyr>7z(CMRzte=)*1oRJ%VoXT?Es$PG2It53dCkzp*oCP|S zoY?0Jre%$GpiDFZlba|l4!)Qq4h5R%D*{UQo?u#71=H(eG8w3I=FOU|?$aln_|1Ijz&Qv2# zVhM#})XEpSL(~{q+^82nFb;5tEFwpu=ZIR;O-4!>$c+!>ylRDFdRF6VD6X!Jli%OT zvn0${M#uG3UFi|JShq%`APxoQB3~hGbgPJ2AzWB;DaHWoRn-(VCEIt;9PKaib2lJQ zVXYlJr?HtqO<$3*dtEWpjBHJ_1D)>oA;hPTr3TvM0PpUnc9_B`oe%CVykgmbXj9A9 z%Qf8ZLibzty|@k$MwfX~VOciCweOWQxB4p6jpKz8}MKdDpTz^il{0 zo;CWI7heZj-Qk!1PqS-Njma{%C|QM;OSu|8u}1OPcU-$fF;7^XW_wR)P|Jjy1Ny<8 z8?!By!7?fDGLl;|Lk{~tiQ`;IiQI@B+hw_SXF9=6qy^TC*!mWG@JH0cO&A+C>cub5 zxj)*OX0Qlyy9&B0kDlw;$&K+L@MJ5a6-hweOvJ_AB&_8@HL7mFFcqHP&)34lukJ;E zNjto(&H3R9evffEE4Brk#*5N}k12Xy`eX?xBsX`&lGq&9LX3Qh^XWraU#)c{6>Auf zm5wb52Nwk`O)k$M`Y93>RTz7USvnXlvsX3`7!<__U%^-Omd!y$*L5}h5a?(&hKBv_ z<`w*VX$edrNg6g+(6GbDU41WABea=NxD{kL-M)>)CM~P>sQFTU)7D5cjrXIcinR{v z(MXrF)}mJMQXgtCj*Ky=w2}6V>}yoWsE@}L401CnAH9N>$E<42ioc)G_XU$VrY`gf zjKQ2(lwKCoy@|(exH^l|mk~2n?)OC>FjhLvpcvOk(P3+KQrwfk8-bmYN7}c-Y*FnL zlzBqe?x1|l)A+)8UDs1|;TKg4DoUM!Jps7(Hj3Wzb2urwmj5JXFR*?nT$jl7ZI0+ib##*kyOzSHe#aoUsg=^Fzd+^q)DbbZ^ z{rSCau{$deOsfuS zoGrzI=3(fD*$Xpw=_By&l+t%*?88s!y-Zk7uOYYFUoqga-&0yx;aaDAkLED#T%u#! zR5M@UODh<&E1X?ahwZPzZ&59^)rxtW_W(9GCzaOSX{NS!Gh#|*LzHtRL?t{#XDC-GQQI;p}4<{J7*uTwDqNs^7(3Y^cz2pymB8j9h7s^{(BBJ zuB=Pf%$MFC^q=cB$wjP?X)x0rj?xwI%eci^p~K2g({lNaGz`)kguZ6wtoFsZrp3$` zi*wjlF7YITf2hUu6~|qeC|VlEU5Z_d(}5Okrwr`wzQph$`_g`3u*sMHY|m|BwL$?{ z?Nw8;P7+HqqerZWIhl+uiU2y*KxqZ&sfK)TRBWrV;mOg zly1r1A0m3(8Lw@elrPA_M&wX%DLymWxQGisz?bsoZYGqjqj}y@eeF!J#EYff+dCQf zp;l0}XsI5}%riE+Q6IvUEM4&5n<7kycNAZ4B+&`gqjs$Dagi|&F&i_h_uN>2Q$-kN zU*o~EyC}0`{{EhtC?n6BQ--eFjikI6(1O9@v}@FexOV5G7?>MBel6uvyKc}@tzYo< zCW?3W@O^BUxL689@SJCXAi!D1x(0?9K8{GQFGtwV{c&QaXw~`q5*Tdc8ucQiu{N7t zl3{bKEII2EETX>L;XyGbNoM1A6qrR#BYa_>FLg;R>6=~&;*l0I4Cu(v_V($ot?M`R zjUy+=g7?eJ4v2)$96A>$Ay$bDeWQxPGNT`L?yXQ#hrUQ-4Xd?I;l&JR)K=kJ^ zZ5z+sU_0YOGQ~tJEEL%Ez@J3V4tec>;-DvOT6gk@)?vI#Y3I}L;CYu|MsBJ**`MMy-cr+@a2Qz4shYw3SN$GB4Ch?mkzL$b z9b(XX8S;E+%LMgxHTw=xDL6H`?uD_DcOUVbNA*TG0P!O6t*Tv9 zmg?$iT*i$Ukw`=tL4K2j^aKRz8BZO#b;kL{*LcspBUU* zE)-YHLX1_oRK)PWK4&a$G_$-`gXbmYe50uQbjwC|_ZwYwDs_jayJTfBwTnENv%Yi! z?(Dcwcl8@s+lz`+EpkN-=Mkr8$Zw?8V`oS@&5ghQ6mreuAOi<8T_ zghwhaT^gcuv64CTDqKmTDPGhYc-p{gsdn8`?<0l<{BPc6+bY5@mPMr?DQOD`zjv)2 zE!!zmx3lQND~l3Z-^=>Q+UY)(vUL2JB-jP%xM-{}IMws%p=H#D;cwl5OjI!U_ zdaL|`MPK1*sRlnylQ>_lYTEA0s@Gq%7aIex=D98paK;DAs@TVD#Z2wDBkM^%Vfum_ zc;uEho$Y`1aHrH=Y5wKaLkYBP`GYVct;69$3)H1F&t>A?Nb{UohrT(Il-BG4_TV;G zp8EyEoEF2bmA&&6u{jY%HoHc-9S7{?pQBLZ3vC{m_gZIdB26um`dL#py!EHCXc;yb z_ryyRLq|5SoKthS;*moAf5o-h4@YX!(h5Tb)t6^59c4?xsta+cQp^j*@tl93eqy64_$4KG|ZqcdX8H zRNmSh)P8(Va&B0y6gfJ*4IAzbud(|JT4b}7==WLs1JX+HVOTg56aHJ>SC+k|FC@TSWO z>r|9oGhbkADsa=0JhV>F((uL?uW+M@Ip0?2jYZTU zoS4LDW3gMk`IEFwi{YZu+A9a6Z;366lGPph+=+%wvSJZ;&M6UU7}+^NgdYhTBWejmPIe;qe6zTY1EI&DNj zv=@FwEUW)*+W4yA=x!RyWEBHhMyW}|&LYnfj`BP%6VzMjwTr354M(o&v52zQ3-@^fO-BN}O4$p6REEpD3$7SdlPoZF9Cyj%9G#aUCTP zmx}w3zY^qSa^6FQBbeEnqiX0}na?OzYfx6#&nwmiAM5~6PLgJ`BTcz zNA2^$fmC?~H;1&Sc>;q*%G_}r1oAK*DTK3)2i?}NHW&6c6#F9l##{Z9u%H-gU%-*3 zw6+RqY*efH1LQDe#qRZq`-5zsAoR@orjOKxCE&64L zT=m%4eF2h+Z5MFLZW^(Y8G0qM>9J^hM5duW*dpu4d?(u`c%g@<>sqj)!GXj6?&4+0 zIV1Jwt{?&6){cl!Q1r*n8G&nmt4|$_9KPp0q27+L>irtYD7%Vv%n3w*ob=1YGi zI$0N!Df&+l!lqe%uvB_aJ%q_vXXsiBS1$~DHg?F7daYg&KBtrBu$D(;ZAyts5@l1B z1h+@zu|MBu!!gf0J>Wpy#W+|o{Q2YV$CCW`qKIVwDHOL_B8?>S=UCB=>a*L{C9Ed) zeX(J0~ej*mnaRWKuOvqR#h-fOP-tG62JnyZ(5)xncS^+MHFcPnD zN?0PWQouM0T3aDErqc~hj9yfGRU@C>ioa+tj1GU938pcPBjxac(~T0#>vg`wS%Wx4 zHB&)S$&?ifHHvmnFI-f-xYO;=z28mc(hd=d<}+mQPBN5+a0{kF%a$Jpz{0$CgZ!WZ{wkp^+4;RD-N62IV6Hi&2w zQs%-%b0_iS3O_v09T^MFzG*?EJliD>r9TqfY6FA6KZRwUJCyD+Yh670VdoB=+@*HR zReANRZ27Jo2NONS)5+J;3l=nCTs18kO!B6Mw?;6d8!$Di`|VjvsmeQI@g<_Dv&ui= zGifxx@6NyIK9uxy=0h5t=LdwHb)up)W#TI+(L^Y6v$e)NFR*Iwyx&5QZ)TY+VN&VQ zFc-m}X)T7%Smcq=Hh)^**hr%hyNy}*Fg8}f_sax3P2Q~0xKHhr+eUkK0ACcFIbpPu zJKPvHmc2_$kxy;EB->5cOZeMURZmuPk{&@3y&h@woC|wekB-!a!6m1v-~?4)Kh6F{ z!&Dj>*O~7P{&wtPpidD?KHm+lE(h3pcM}D?E6`O{mji4!OisT3N}2{^IcqqRZ1q#n(eQm^a1!9_57I!}h5fdJvbp z4r3$4{&?|RBO&I-+tyj0tYH`S;s-4ponhv$oMAIii)$BBULAQ{s5O(8#?^lEj^TVd z{qwEslZmr5ljhOdDtmk_h=k;5cl1AMYh7tyb6X)?_}sau?>WZ0VbVrCHPv|5k&}z@ zUq?>gO9Rbcb|Q~|BQ6kk%3KafrTTni>F|I?m+7FOD2}C4Ao&}WTI#tO*|YwILx(lB zYzUrU-W@F_O@Wv6s;hjN*(8|(-=T_nq1st4n@|v|*?T;PGF3s^Z$WCLQINf^aGQQY zY@S>ErIfV&VsfGl^UbY=g={W^anrJhpx6CZm6t@BJ7bEuyd&@AH!%CY-nvawf{`j| z6fiWaoN8w<=vR(GrY6JEQf{A3{{=ty^@Rh338}HBh}x zi5K2Tl-+w6O~l1u-$JHH9ih%x301_7F0IUA_M9!C3`JcTIqCdRaq>uZwb3Gsi`SD{ zaz{p7Da=m8Z6*_qX9^^1SO^`TLqCS})|7ykY#?Gr#=4&;7e*S52-UntX4VswO zC!U$OSBH4FIn!OT`9Yw*t^f||qY~qThfIt@IsPcpH=*h-ZVPG8#6p>^Otp3Fp^P9E zG%r;e>mwhQ3)s~l33SCV2aF<@#q*3AXgtkQa3@m@dN95kP|LE#q_L3bKwM2c%B@@P|FR*ShV* z9dpa{e3g~)g;w(d7Glz_c%J%a&UG1AlBrB1_}e%MkhmCid749JU&SG*pD#8Ia2(oq z{p7|Qm>1nuJMoBa=;L&W1C6)&s!0%wQne(M*&(s1074r#QY%SAxfszB>Ot}2=n-gi zCg|}dKE4&xq^4~JnynXYSRv(mo5&0CPorwPr9S!6;PkH$_U7p3dDH3@iNG&FwY<5r ze94B-F1|{=c+3u7u+4grXXxB^QxxGF@xjTtMgD*!-wwYD-N$(Wez}o@2wKnGYogrp zbHx-Fp6JwQzbQkr;J(1H<~TNwjN_Zl!1}t1dRg7eg>4_PosoGk!Ng8#2j6hZ=7TlO zmojf|BVNVVVOG`6;twBcSj$gmrCuvFZ+Xusepe;vR`~#SJ6=I!dxUU%f$!zrJ^tmI zVhNtbY^l94tIys|ljNNRR%2{A^$%7F%zK5dWp6UN_J(TPmCccdaw&Z@44)8`J>0;X zcW|@2!&8%QCC~~V2-Gob3D^t<>b0yTgl!Ne2~(@T|0=vQUnJZ0}4#!o1oP) z;QH+@wTLvki7<e0t;@5`mg9!7U2yriG_ z%@w7i38RboCPC#6^5Jev(F#g`*1Gu=o6Z; z3tOUe`KV7r$4TCI2%aZJ5>m8UEJF=Gh~IuWz3M7XXK_h+ZnC`l~##67sm-P4Bn)Jfr2@7kStsC9E`(#>|os zO*MB(vr8(Da3sxhchowQbGTa@j8>{6I;1wMH>Na)cN0?PX~SZ&^UiUT&*di$zPhAC z?KPfwFtqvH+@M2xNj^tJfX6c_)NSbL`qlNkdRD<;o6te|3^z4TPvXJaZ2pVajBf5u zAkZt_y?^6F%96H0!8{?(T0oED`xld|F1um6COYSQOLs`$yhL7`8SA+;6l**427`2e zVMJDlq`L9hXR<=|0@bh}^y=#e`?q<5{iaCDspl8?APk0_+{7yvuM_9 z%`^XY#O3$xji1%kKI?S4%Diw|x#h!fjU?5av6t+o&ZF{&Ssjl#qHTDvVeYsaz6_IS zw45ecW4hNkJ-3fh`^4}q!McvSw3kR_)s8up-T`|Z>k;n*D!&=;fyw!<0LM_T>)O{w zo8mrj?TUDd)#Xxa#G<=oc(roWh?Bla7r0f!H$5~V+$_~1(BAsi)GZ`}i9A?9{7^tz zQS;Ju_|%+hoH+i6@zfSt%!j+ojkA5uW(_{h$QPbW%}Otb4fhCKeb0F)7M0~Na3?0$ zX-1G-{UMS>bNZsAyJ?29K;Tymava~@0LsdBRbROqf+AHewH2Jd zo&4sE!hZ1zd_uMsd}r{=DQYYo7aMyEV@n$s9V1sG11Cq&!2YG|T2MtwRWyOl8ql39 z%ztvNNS9*h^!4pq5PEY#6pqWXwu|n7X@R<{?JfsC1z8bd{*aw5!mNv2o&J&&+exk>G#Wnl_|P@FWhd#onT8ys7CkwRpfpp?O^U4yiRYe zQ6+j2v5oF7i>O5S=R&qfGRl%6q!-tvU78HU3JfAF4TjEs(=a8~T9BuboZ)3IPk72gw^+)LuKe3{nPocYd z9=z1&1_uu2JgZKpx16R7qr=lIzJivf1I+-GH_fmK=a@O%n@Ml&Y}Vat!$+)&(cMEI z5?*&k(3Q0nk#d$FeAv+-d|%dL7|Su+Udm5%-1Gh-gKgp)qOvHSC;C062rI)3Pai) z#-4<&3lj9D2h{BV0yFwxsUD{x^{*#h^ zbZ0DVbJPmXMZ@4No(cmf7$=m@d|RWbwnSpO`C8#V58v(UR*wjzvN=zfdy#B&6njy_Eb~EXsnk?n}SK&sA3$ZYU5-;y}O zeZ+irw<*sx6JOU>#O*UKYNx88^S)UvQHPOiP6m9a zrfZ_Kq}orACvjCS_VZo&G&cFlxsUwvDjPTWn78;|LO|fxx#$kWy4TKqKWgL zBl;iE4LnqA!nyfWIV8^CnUt`OBC*C;%&c@WUjcif|FGx0q>2RLg&3>CmmLqLiiC=} znw}P4@VUCEiJ6}F^*+7*HWM3p|BBfLZ?hVSyS(VPT0 zh`RlQD~+ojr#u>PhTY+OoD_Q`aNb7tIrh-2J8METZ)b?NQQ7NsdA2@eB1j0ol=E7{ z6Hca&^1UK&ozp4j*(_kN(swyaEzwG6`(skLE!*QSrXBKl3}LUdn#Xg9vh+sU>g4RG zGE&(*eben}>u;z|?%!;x2#6+(#!CK(M5ySarqtRy5GJ;s;#u(I|0`nHxPlc>3civnv2Z>MmvSStCZvIwNq84QwF9eI`G-6`r2 z4=*QH1HC41nYPjqk)nrIW4-2;@$3(t_A_!Wz4Z@+pNF=u5CqiV)+g3mhos}g-_qF~ zop}}D*}j;E^cl@yLp?ODdGF|a?_wd2>n&IDC1T$Fu2ytj!Oz=yT+DaO^46%cY#^I; zo-9SU58!+y9Uc@E?s19aYoq>`R8g;P1*V+nhrPuKJ$Io42ZVF>74m6vBC1|~8jyar zt@;R?EfA*RzHd1ZEvDdMyPO2A&G`G&B26oUJE$WVEs`;BJO{ zv#+k?1(~K4d)~>WkZARRbVRZ|tSIyYo>}&B*hHv6u9EDVq@;;b&-8Kn?vX}^^Ts?* zp@@_auAxrJbb}HNXf=VwBa=9Ayk;Poy^dQJfb_77N33pk5jQJ?=zNT5lm0?h34u3f z%R^3)ju=@kSV#pqHNSs;52ofHcEY*~^ahv8*~JH1e=bfsa=1U_hYT z#dFj}5lBQb%7HM#D7zf05+O^xv!te2bU61vJ9X!XTV?SxshCq?8m$aWjcyozVkMVR zO<2bzQVvDJlu6y*zW zGmQ;3vH3D`C8=pV?1HQ=)C0owJ=j4OdI9f|>rfk({wsQ6_=9+zwvyqR*rnVtIYBtZPL*8Cwel zX7WdU-qcB=9a(to_TH|}N(E80eE;o_FR#guG~wT7+6?i~^=O8}tm!juEG9}a2zllO zj~ttT=E0EF#`4r-keDOG1htzcr`-!2Ry|VsxEul(Bua(BN+IjhnGyTxA*^Q#Jcr_HsYUGV+i|b1Ux_vI z{gC{`oU?_+{94uGK*H?jmQZSel{I{=huHUmZLcUNGuZ@%j(O1go_jyeP9L39I4m6Y zNc(_7)7;kk5m)Z*cf!_7hbzv(a`G640Tbo-CZx9+UfXb|J4@*uQN824qV1bqfzO}R zfy1%AXT0~aUS*SLLSFeO^u@sTwI|`LQ_Hy=pM9+5OrxPSPeA>a;HE=@T5-AlmQFzirwm>H}6%S{=8AqxU|MJ#1*BW^>Cja@5Gn`@GHa?qdw&CC}@d zoU=A$+7JEbP=%tMthq8i49N>-I}&YA@Zno<6(pKNqm&MjvTNGB9)|LaOKHQr(DwS7 zFk-{4H~tncD8GdDV9(TW%ECq`e`psF^~hv}k~ct?ep?=~M}_lt&Y3~90{e+;oRx`w zFwH8?=p2q(i*x+jxvXhs!Iz>zOxnAl2L22O#ptkG$yRqJuCWu1o$G42bu-GQ3~kFN zufFNCe;@4Yr94lDeU%Q2Oh1@MHmkKCWLXy1t({CbkQe;YY}!-Ar3*z;LYUc=bN|NJbc%CVpP0p}6Y(J5ANaDJkUYMzBUfs0iCWt#rlFGH`6797T`z zESmD}RPT8iUI|n4iu3Q*)UQXc6_mcmB`U?#d-1SEK;c&YzKdymQysaPY@n_{sy$iq zFvZi%mtU;2O|FKdu_fJ#99UnjZHRq<6)vWI8GKf%SRDDuO&lQ#tvz_M$#+iW6NqkO zcTjyDInxnzZ5G$XTq3G)5I2WaF011#gc>tmH&V({gm&NG7|YTsi<)l<&O_Smwo>z( zd%dONH@{21Z{}!%CYzN-MP|+LI6x<1=*zPTeCA@6M_zI<8N&$M^tG-1&^gVz#AA|)G&^gNE`=&Tx+&OJqBf?}IU*P<%;%Dus@on_ zfV6R$DCwxV#${6?-`zCTH9MtG?vrhPR92T>YSN^xrkQi!>#H;gUH%F=9`1YgJhLWh z?rNXbk8TORVVD(s>GE78r{~5BQtYtYJ)gY!^K(w>fxF%8j_xl;RPutu5&x+Jkom@jmQkko471Nb8!pvfy}yYxry28 zx9>b0zr}v(xC9ECEz7fPvd3Jra%2Fl-52n=x?@kueHZv`fNrbj-~c*1!@nfx2dY1B zJSdE{b5x>i8$lmjD}5uLz(i2?x^N86E6OxDner)l?9kC&mx_fa)tB?pTglya_+55B z(oQu=Si8iI8RCC_4rM)E?+Nb};+CV=?l4lb^18`e%}+t96Vul_M2&x{IafbtG5`F# zAcqJG2{u)$>hqfC7+gx;#0taXSx`$UT0@nRUSlg8#ydw>GtyT$XPSCWS&G`5j69ny zA6eTxH_(pBz9_wzJ?wDOXnv3gH#4T3P&Ck$aGc5&hc~+Neh|a0Kwkw$4ykqCHg5DB z&Q38he4KMEDhjY#4BbBZmFeK>_~2!(=T`5yFVUBiV!u`M751(ksZKW~o27Eo7qBrp zAGE}ifeJf&i4s*+4%JVsX3dWJAMN*WD?F_N%{4INzdAO+MWFQ6INu&)O*VE zpg9G67at+ksxr3hc)W>yQsSHHy)?5~7prHO^f0;wZyUH=xMdPJUvxfPdzVGW?;eiV zynmXm0CdL@tf%YY*twlPu^Hz001mXKWTxEI;PO<#q@woJBj~l4gK7ROTgKV}C9|&; zuIQ%Lf-`OeA;h-JukR?+CtSK)raR{+Ko0H~5UF!tTMkuKrn%QfuihQc496;NI=w*y z%~r!cie_cl&SNV$2jvxq*6V1F@BrHwqp3bPl6yzl1ufVtBsYw!T=#u<3L2IyltTqt zNFoozOk?R!&E<}8Sd?7{TDl|pGX zUf1NOC+8LX`uv10;upuQv8I>4lfej;3SG-v%+98jFnK7Q!Rf4#H`u#iU!FnTqJvvS zai?VqOUG?Eoc!$)ExseKWY|?SvZt3nzR^%MYiuhwb8f{VdF=$1X8lB(9i5#DbL`6E z*u|U`>p0|k5ds5?!yLlCteY6BBonq)vrwhonlQ_;1??jPc!Thwd_lWQwB8|ld3u*I zmo9c$gb|g*VOPqP<_yBupsGLKKDkE??*Yy>Mm zbkKG8vr=uE#r*(yK4kolm%6LwK1aREj>v>_g(GGuQu2sh+ON9et%WiiTOIMhe4yw^ zAqjRb2_1CcDeZU07_uE(af88&jN6H@>%;Gc+ja_ zSZ>aR#yMJYt@IQ7J!`e?gY#1}{@aVvelsZ#?Mm1Wsjvy?3GS}-U0I(|sw>oe-iI_4 zcY!R74wJ1%)Qa^pZHCIj9S_?b{q0mxfZTQ_kPxVaaqjs zC8cX=zsiGUkU$@4ro{#|XcY!1i(|g_kS3|VyGTUud@CqzH8NOaCzdli&toNkQ&jOe z(gAvlEkZpl{l_R`v3ifI$@uvX*hCI0pU9XbjO+B5uRhrw_3@mEZwX?pHEY~nWVjmj zM$XKy;p(XVt}VJbb?geT&$k%(ys7Db8qq|YCeZ6J$m8x zb6>;;?G@sw$Y(N$rTtudXIGnw-j>-c!#y~HR*;51M+|d3y(5DT>H7~l)atdbf6}*c zv|(ZcCuv#1&g=Q_mo|U9u=!Kq*L~Mvzl;qj@L#|F!v3E?aJdsI+e-{mAhU^?qnU}d zjlIz~(a5#szuyJ@DSH2xEnPH-XPtr+t#4-i{dsx8CDUV{_v=+t1gNji>wFN=nS=rb zHOveJh4Y{1UG<^?kNqD5d=;wapl4`hX8q>}{wW(RODpgeI8SyFG@t|!vN6zpldbP$ zZ3sS+WM-vj@)wKz1Sl4wg2D!;dYZw4LOy&8#%}-y#%Fr?ENj%%bs#$)8GKe2;^Ap= zewS@`TAK`GXX94^dsJM6^p909BZA+>x_m7LPAgi#%O{=-0Sf8?5OfLRiOtD>6K!Ov ze*&j`@ol znprzI>RB5Y=@>a1SvxwM);@Y)I*H4HQzrv82q7@Sxqid2FtfA-7puPmofg69y;yDz zDio9|?XOxS@cf2gspn)3taMfkCe_KUyFdeFa0G)0LW@|5-!aS_9RCmD)Q0MKk$l$) zKum!Jxd;JaF7rEtg9H0%?Gw6b%{~ZXiWkHy{3h7eP0z|q$HB$- zVN}KW3ZL;uO3C`~&>_n)J~BCmjmvYl0RJhLjF+^{KL6F35UMK5aB1f;^&Z z2$`9(y&{g03UN#dlmYQ4!Oer&-PkN`~9)?oE%NR^>{$|iG{#V z;rb{-q(~r)kb$3*{Kw+iUMOY93)eB#`Ax<-DQv}uZr3Zz4FzTD^D9t7&Bh;dH(YW2{*&0<1`UXXvUpYK9LLlgpEvXK!Qe zcxwNEh2Cul07TCNiY96Z>ro;85#S$uqaacYLt!UV_7@vS(+JsqmVP;z2&c?WCP_LHb~l35Kh9u z|3^Ga8xxZs?-)Ft8FfUfkm3Q<7J#CJfNB%`Bb1H4xsk#5jmgs?jp5Oc-3KTs;9ZH3 zNKZxnjAU;Hlsbd?n$GpFdjp+lLH30-Un<~||F$YW8R}aQJAif{yr}W7P{;GdkB;R# zUwF!$0Oeq$XK!G7YQm7~@B!7SotacmtJ|x(N z(x1T$Z48{OKw$mVc0Z#^XtXM<4;1TUSASh5L#@AofrjHO9^t7o({=+iu30|6LPc5r z8R|bLd%1RUGfaNNvu9*(Fc^P0@^KR2;Ejv{sLv=WN%=k z=lDH?{XO~rRL>V7g?0?wnYf(*+5BdEm1iD>H96R% z8C?eSCkyBglFM<2PeyXHva|xX1i_&4uP%4|+WZ)xedltWSHD9Ux>@U4*?fChAXxfS zG*rvUEdEtCP(4z8?jwOceb-^=6U`E<7R90cV9c6a()aHzf(@Q z;O{U-25hHJ1wLBQ4Xt23@W5=~$FSpjmp*xdJs{@4F8J{#`hoG?L_(4$U^qG2I2h%;K<{rV>4A#F`E1>5k2D{Y6M&*({FOY_jQ<8_0tWE5e_gGgaiib` zbMqbW2oK;9kg74-{|)QMKFb;4j=EHr>Oo~-2{zQ2AQtYn^WWgU@9uzXWNh~RrTnK$ zMNRYp%Q~QqeGESI3fUs1-u)Y-t&OFb!C3|ms~8n`>3}&=03a=de%gM21G6`>bNX>~ z_;jiT&*CP%1!zYg`$3L5R=_)sf9;#UvJn`I|1Z?O61kz1rP243P^ZIV7#gB<1|-W4 zcDErbBCgQC;W_G^WzkTU?!K1-Hax=o)h&^uk4MtCv@x(SG;%gG0Qt$*_H08H&vHtt zue;tgfW{yzqw2&H;J)px|3d^kX}&;M32gWVjHV!=7*kJzasw1>Wbb^ssnqdI{3E6O zkBPI7hx#`Bb@^IWo|=l24tZLN02Pbq{OW^=e+T&oF3vI3p`8Nchz;hkkm3BO?ypMe zS?bxJrRWO?P#bdx0p<%_5wc-_+xb^8Fh$g}wJ|%>h-F~-X#@oT`~o}#60ocL?|^1t zp!4-jD`z0&NR`IQcc25M0f3N%tla-sKua)oHnKE2;}|Is!H^u8Gx%3X zDc4L6IsZZWKg6(=BANIA@Bkjbfsozd zh``Bem>TI>I+}vCb*2re30Ske`+(XFffXUW4MFT=tRKYwb>eoGHU#O)MU1Z-0a-vh zNX;lRSy!HQMO6hU}!v9WtodHCeuKCskH z05S(o@IV$~cZ2^7bmlR9=loTU9I)_XU=H%5CH}tnHPOdISs1z5>Y1G-ZaQFI5YB+O zkpQ(hWG_&abUeoQe#*ht$iU3l%;0pz<*ypt4@vMy;zuu__ z1(A`xF*w|EhPA@tOpC|`P7(CmP#HGwcrb8|05k$%f9tGa1UIu#zzd)@0fR#hk`Nyq z4`pv;0&1=^dY|qC&bfWCTy-FxAkCLrbP~+h9>B=(M`}8Q`E0D4#P0!W7~=f3Z~I(+ zJlMa|-2US?uy(*J_JE`OXgR)n`dHHmNY;9`4yHC|*@yJz*5W4wwi5v}i^~vAc4o&3 zAhtG+U(0c0BO^lxCw&J4d$Y41Ur>RPAGi*DViPPFWPH(dpA7nAvCiJxdwss1{Sx>g z73l3D9b^Ff^>Mwu15nZy6vsxUpe(nsKg-xcXho*}1+XX~a1wk7i^7f^4|cY89?sfB zRt}UYhh)EQ)R9jdFW=e55^PVO!N-jSF`|}%=M96|Ib`1Gc=IQSGn08W9TV48U@HNj z&yN#F-(CE|>hVCgZLI&U36OTo`Ao}Zu6i5_4A8|8o;0U4d| zPmuf%#r^d~@69d%LI@}T(mrat$3On5c4AIrAD5$J0Icj!ZJ&wkrMF+3VRR7ZkoIXi zLGnM?C$Dn%9;jrYU_dT|3~WRg*xv&iJifk-jUzbUrDv=2b(nmn_C8tKKVTEg(Cnpu z&1Ri&$AkUT5yP%ciNg+Hv>pHj+5KL^KOV-&%Knty@6VD$ypUEE)JsrofD$1qHg3}6 zQGU>`&RMGffzp08UjWld^s5uoQyvcmCLw2;yGOd$IzI+s$ix3@wGP8{48W<++Z`V! zKdw!EpZ>w|{f_YUX344J#4wr;rwPPKH=x1)F88XJ%1NUCAyClsx0(-tQ@sNj7qV)$ z0q-6DH3NQAK;Oy0!ssle#6Vf&*k#~SodD;*yP&IHEutr2{8NA=zK9{I1OdMUs^$NZ z;;NUN#0fZ{1vo?c3$sn?X$Q-3And-@c(u_ZL`3)GMWwt}R10*&J( z|AXf;8;CiQBR>dyAgP5Ftz5Bc2B<#R?e;q0Pmqb>* z4vrW656f-*cptMB%*S9Df4#)icKg_e*E6s@!vd6s-$MffLoHPv^7{{q~VofxvoSoQjlcmm2Pj`sd659ltw zTBQfM0!0XN8M3{EvW}I1Himt)p;62Uju>6D{Ploiec7?%zY?F-*JCU?PPUd{7<%d( zLqAn=k@pRc1?edoq>lggWvgDA1IJ=Iff>OWTubZStIGR8KF=@mv%N${ewTmR8&5x< zpFw|C<`Hl#B(T*3*_08DA0yw<(bhrd48C=mgh4g?JCK>4)v0S8Cg0@M_kcjKLEeL; zKZg?}{{w@xBp~UN1G7y<{hBbU7mk1YGg#^8a)cTtd8vTqb_YuWIosmgJ01pju$7*@ z8`x1lgI5(8TgJ zyq9|{5G?u)kTw2W7G`@Lohc5y#fv#@6H%W!q2xs^*;4+6BjK-l;(5k!&w@aJ}x=O!dDt`8L%& z3u(aznOZ6WMY6%*{=X?b+pB=_M3k?$WY6Hj_^+Mou0lgW;eu#^Y&DgyoQwfZ@92Ej z>I_gGH;5F5K;mr{{MBe5xK4yR<#_CPf`r^TCXzf3#lh6+3{^Gp8d}-cud08I7f7DB z`GfR-2pF1*03}5LQVKMK++V#RbFv!Wwj914#QS?q`qP`4q=^MG52IFz|W@{pC0eckaLLU zC&$3JoW3tTc@BYAeZ1s<@MNJanbk2cahh`e)k^Q59uM+Qery7FW9}9pgSRf3;l)bX3*W9(wN}C{jcqv;;!vML=30 zgpx=XhLB8>A(>>DNg#Ph84#6X=pah-5g`kaQi2V7T*kOpAZ)ah&9AlCR~ zxr?rU)*BkfLf6)Mv<#T;D0-W-ye)AIEl~fa80I0@ zeEw`_k6VwC7_m(-<=I8FhB> zP9C}NB*Z!qrvbG0rd?ES-PWW}z%A?0;E*^ZO2p#7gZna5EEPQ?v%yP9?%5-9o5GX0amZalym*Q@o zc#OBC>1)O?ZxmZpuIp^8J>&BDwOO0zAn$MKH;{lN8e(la_H{d};dh zpSAVrk6_vr6vnAonylnO$Q_(}T0E_a4@C;JmlBq9e%;cB*e@Bj0FLfE<{l8jIgLjXgzx!jx%cW(9*A>jr-mZ9qu14s z9=~}HTEP0`XqaqvOuV`enqY`5Yc7VO<%1wkFEjas7|5q0#_rH99gl&gpWp)M-PW5I z#ZbN!G6J5i4c-QS)(goSb)6M1i;-dz(P0*)tCiF(sLkE@5C-m3LIU1!7J#85U< zr0azQwR^#QZKK^AnSXf0*n@N{T4uSeTh>A-=nwEHTjuZ=_Cd#HmH)iva%AuwaBxLS zK2QJTe!N`w#;U0guNQ2LB<6{y&%gVayS#MwM)U3OmO&BM@Dv3T>c4t4DB|Vw!F>gR zT{>ee3i^7p+&Z=9tIyIJ11c*W?L2wR+v%AA{ReG_u5Z~EgeIUo zpk41Y@Qjd0KM=zpM9Or2Cal=7`c=q%C+MNMSm**Sh}C9_Hm69Car0Q4T~{EEHRuAQ z8}vONaS_}NLs5xi6MSqL+S&3-$F-nqBl66=Eko&!;_JAmx>|<#zMk$Wi*Deo@0zzI zOfTJHUu69CHro^Th3IKwG(fPj58ggBsNT}ns9lVU_Eh1QIzLeGV27p=AWmWuExVR6zO1`(}=546baO{Zap6KD+ZTgcCLTTg3y*i>pd3$|#Wo9`1Zww5cUJsk0 z4dR1rf>&k)??=tCRZGu75pVvYNt87vmplnUmAYqmqWnKq6PJ)qp z2W{LIi<)iE5uVl?igK>Ns#q@;roqeH2pCvp^Zvu|5H~=OOYN0XQ@OT69Oo8Vf|&b% z*1A*+08h{uOI=Z5M^jz#W z55uacH=58-0Fx*=*>=M1lq@qGY8MpF^4C0o$+HPnEI_UPY z&L}=WZrwzFD^NIS@FqVv?}LyGl}b(mA4;M!$9Lea9h&06$J;nCN<~J0Ejw>p5#9Np zQd`~!r#-#&!-6G1*Ctp?-hi6z_$pb25UB$xbIT#Gv2n(Op<{G=_VOeZ487A=-FnvV z+3gS6ef)daH&fGJg`0C;e#|8mWy>kyV%lvoU-|-@v!@u0PYIeiQw2x9FgN61CpvV1 zCYM53rRM;P7ia@TnW80#iVVDYqB1OpO&Op@+UgTEti*)GXp6-n(;JXEe%P^5fLRQ( zsCs8E(*vcyKcM!~Jvj~_v9nOB5o6NTkdjAQ%-UNwzCWN_knpuU9R04KUEh#{17SQb z0gT4XPX#Oc-~~ElWwi@OUQ-PZSW&OpZv+4HCAJ`L`^@^_@h!}}GT^SH2h8)me3%!- zOzi`3J;JD;#eqrCFN|3m-mQBUA0LK5Mujm}u(A)r7=Lm5@?p@|9i2S6wE77D^Cg6# zhfBsY9GKDixzr~A&ZK?tBn9B_DVIJMtn34q-o1D3=fMDB&+Ji^J$#=3`7g$~YwMC* zWw+VtS7n3fc0eH1)sMr&Xdx~_OV#w}<76-wbsd<2PX(k+x8N4_kx9tF%BM#`*g$FwRO z7)(;Z%^@R(erf8L9&`V&ArC{pxgT6scsA}38>)Lns0)IXeK7Y$H-1>;kCba9P86sl zW;Nk|zE9>cvtIo4RKUE1%#NBz<<r_P)2f z^I@cC+m~HX?!+mus!$B{Glnh5P^g*LSjWeTijB19zIdZ461Tux zoGlqt4>;vv`IIKd20o7SUKqyG$ z=M_HN6wcnjesmzSQEmAslNHbUqN3t z$i5-n{d@H8)<@6uF$QA^YQ3Z=t9<%-YC`h&DvS&7(?6_p*Klou+&o-9>gUQxA`R?E zJ;8+wDo9C*R%1LGA>*~N++L^X(0zV*J;!hYI!?n($BP83GsY*xnDF3VfX%M!2~^L1 zKKfQ~$|bToo^#WmM8>a7=*;$v&#`Bq-@Ko*swE=M6o8GE-fJSun_qiY1a{a0=eTtB zAFq0Ub0}8Ve%4%(ttUt5$!Y-ZpXG^MPL9}}Z;NTfUjE{)P^DCwCyEz0x}Gi7FM#B> z<_F18&Pn)DBfp>v2bmY%*$#+Jvgm%wes7WEk4D1!8sPBTb@hb#m~_Y1>q5T0crL=2 z1UOB-bz8-NRdv2$$Dz$}VUHe%j=L{JW5Q$OKXAoLAD$k;HT_#1FA47%p~GyHU3dFX(c3P79V5z=oTX zxL)_sKW%$rBd1+>9{XDJ7Z(}0Q*{(kI{zKO)JQMl-8~;Y z#2j2hFy!nKs$v@X3kVUFcaqsFr22H#Z5sz`yw`pYpaQWup|(+_h&r6|D5EUilsP}u zSPfguM0lsA1RBo%zpPtKIHz_yp9QLcM#)P zY%(~jkX^3n8uig|xXK}@9oHlt%h6qkV@ArH@*=J++Oerma!Lt9zJd?H@q)}^`-bGp zUmZ?>{qf8`T&MSn|8Lz(h=RsJjSR%?9QyaQ(SrM~P*rlA5kA@>Brz1`thnGB6ifplytA^f`JN+Nw>wlYa&v zUjrm{d-MC~fMlLs2F|wj!*$9 zeB()myc5AkPoDtXGr-X;RZ5sT9QcSQab>X9{9bYT5@4~`66$YPg{xw@Uc;0Dd-~Oa z%Xm`S@BpEN`rDP5rK+Nt@VPQLAMe@QaX5%NgCE6-8=!$>m%#}rd*{hIV3Y^*C{CS$ z>NxRILTb8e`~(MZmV$Q_r_msF9E(kco}lelzGQlS7C6*J6&tLI!#0&NKwa7&y)pto z=kTKhWorOgr3AF$X4XgPz+m&`Xqjrw5LJR=q+mWM`{)MSVupChCu68842C1huG%=q)|pyBCI;RaH~0RCZKRhHN2-u3&$u26`ND7*-~&W%PYoSidKipZRC@oYu??x z?V2PcaXa;n|?owW?Z?iFK=W_6s=QaES+ z>vwDdaOy!PDBt2nspFvJB0*2#jg!VR`F!uix2|#OIAfz_a6-#hIJh1dCB67oC|(tZ z9XUzR^HOGH8k==k5qn$e7D`!!IQAf!EhoDVA9C$RAUg#724!@b@>hIE43qnSobn`v zl$gr4u?1GgRyrEv1E@Oma;V^;>?c>e?R3%JdOr5NE6}1&^Y25`#5m4j|54KQflceS z$9xP~E%vHI?7%k&@^AZSX`+VCU;3|;@ZSXyXeskHt`y=x?_`;GqtJ-~E1=^&VY99Y zZ1_*q*)OglIH8?-_vqF&{H*|Uf}TPpt3KpCsB_)3R@OegnygKZ=7eQ!cO78!!4-F( zpwxT57Tyme#gq6AWMOM6P8XAGk$S$ERb@&xkG)U@)vOy>-1P{F^2J+rl7*GWC|(%3 zuoRGKKeM+l5%4S zZ{^$uY2hdHt4HN>*{oEL+8!mJ%xi4<1MSZU4;MPwCol2 zE$72u)S`5ky_U|r+Iuoc2muK+Y22m`pqm)Cvgdpre&8~^z%aN`T8sMRlqxaK`$Q70 z34H#!E*mR?zxopg(tWm|!9WFp#UdENi$5-(&Hf_Y(dwK!f)5-lmli#!EiAe=ZZ^{) zV2d@toGi4PV-ieJXsP^%y7<~(<4`DZpEhx(bxs&k?s+I3({y2mU}Yy6Y=DAHLt4}J z$JRH6TM0q$Ki%rJy3HrR31f}0+c7{dfn^U*b$)&Z`SmqF&-0M|CDce!EHon3WDC?}oSQIP_^6so5cPM#vC-3) z$CdS=HPuApi^-K7DPnq5mxG3avu+9HO$q2}Sa(0c(z=DiC%}cG5KzB?0L6W->=P{w zNpdoSOH4&Jt*6;l)-0TpY;y#7F9`8@EfNtSbNiS2T{|BDfhwGj6@Yod?U}%9Q+gL`ilg zh!^)kGEG)Ld~Y5^!H!U<4>+%aV@@z98BGae%s75$onsP$`m56Lp|1n58vtEd98-mK z$M+FJu0_PhqkU5@kDnhltb8+onBn25vk8k70%6crL_)Gv8tl7amxBTU=K0iWy5m14 zF<7F-BK1u0mw)c+|8Fph6=tcD+hWBiHWTiG7-N~lQZUywAwp&h%J`}qj$Vg&N`W}q zo-uBe9!N{G8q+%*1c_0m zd3R*gCN~LI_JKiHPX?bDh_IA};v6lb&Y#Ht{74+wApYTPQuuGXjQtP4SFw905`1%K z&j24Xm5;?dj1jj8ofoR*=d;+KcdJnnN&b(JK6N<57xDoU(F<-fMn*(pbe8)@l#W)D zHkA_RUW2pA0DZ0`2}X)s;v1yrUWx`aa57PPbZdwGT`C}pG?cI7DX$W%RB$AAm+lxe z;(Di}InWI=Qe~&o%5$^O4dWQRLKda;h3gN1({q}%iU(|mCZ19D*$EX`J;Mv# zoE+8jzI`&J_8)MQx{a)}Dj=5V2+hFY1{r2PP<8@NMCN#HpO^it27=kac%O=m!<7J= zQai>m1Yi?^PtADz?^?mnyT#f<6%jh_T&x3u2eEzmd zS|Ea4f@083A{%YYMRA4@2?wZ$Tki$(z$C^CiMjQoXlWm~)jVH9TqUU1%kT-5TU(n6 z2>ActmVY}Qn8ZnZ-h&aUTknXL_JLbzrgSJFtU*04KNXCOfl4V^U(A zDd8V6#=krB@H7XgD4MHYB@FH&SlS1MjcohciXvdv_4=Nrp?`}|oD7p5aMFBkrK9H% zt#=@hwjh!z+*?3S9v~x*dE=t-TpsEv=n|96xmvI6SNJYGPJK+%ah;Jezn1Oj^A-o2 z;rgfpv?e)wC}(x&I2GL|6Z1P+k{W_-PZ6uB`Ckz)?qr; Date: Fri, 19 Dec 2014 18:21:15 -0800 Subject: [PATCH 084/227] [SPARK-2261] Make event logger use a single file. Currently the event logger uses a directory and several files to describe an app's event log, all but one of which are empty. This is not very HDFS-friendly, since creating lots of nodes in HDFS (especially when they don't contain any data) is frowned upon due to the node metadata being kept in the NameNode's memory. Instead, add a header section to the event log file that contains metadata needed to read the events. This metadata includes things like the Spark version (for future code that may need it for backwards compatibility) and the compression codec used for the event data. With the new approach, aside from reducing the load on the NN, there's also a lot less remote calls needed when reading the log directory. Author: Marcelo Vanzin Closes #1222 from vanzin/hist-server-single-log and squashes the following commits: cc8f5de [Marcelo Vanzin] Store header in plain text. c7e6123 [Marcelo Vanzin] Update comment. 59c561c [Marcelo Vanzin] Review feedback. 216c5a3 [Marcelo Vanzin] Review comments. dce28e9 [Marcelo Vanzin] Fix log overwrite test. f91c13e [Marcelo Vanzin] Handle "spark.eventLog.overwrite", and add unit test. 346f0b4 [Marcelo Vanzin] Review feedback. ed0023e [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log 3f4500f [Marcelo Vanzin] Unit test for SPARK-3697. 45c7a1f [Marcelo Vanzin] Version of SPARK-3697 for this branch. b3ee30b [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log a6d5c50 [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log 16fd491 [Marcelo Vanzin] Use unique log directory for each codec. 0ef3f70 [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log d93c44a [Marcelo Vanzin] Add a newline to make the header more readable. 9e928ba [Marcelo Vanzin] Add types. bd6ba8c [Marcelo Vanzin] Review feedback. a624a89 [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log 04364dc [Marcelo Vanzin] Merge branch 'master' into hist-server-single-log bb7c2d3 [Marcelo Vanzin] Fix scalastyle warning. 16661a3 [Marcelo Vanzin] Simplify some internal code. cc6bce4 [Marcelo Vanzin] Some review feedback. a722184 [Marcelo Vanzin] Do not encode metadata in log file name. 3700586 [Marcelo Vanzin] Restore log flushing. f677930 [Marcelo Vanzin] Fix botched rebase. ae571fa [Marcelo Vanzin] Fix end-to-end event logger test. 9db0efd [Marcelo Vanzin] Show prettier name in UI. 8f42274 [Marcelo Vanzin] Make history server parse old-style log directories. 6251dd7 [Marcelo Vanzin] Make event logger use a single file. --- .../spark/deploy/ApplicationDescription.scala | 2 +- .../deploy/history/FsHistoryProvider.scala | 194 +++++++---- .../apache/spark/deploy/master/Master.scala | 46 +-- .../scheduler/EventLoggingListener.scala | 291 ++++++++++------ .../spark/scheduler/ReplayListenerBus.scala | 77 ++--- .../org/apache/spark/util/FileLogger.scala | 237 ------------- .../history/FsHistoryProviderSuite.scala | 185 ++++++++++ .../scheduler/EventLoggingListenerSuite.scala | 321 +++++------------- .../spark/scheduler/ReplayListenerSuite.scala | 82 +++-- .../apache/spark/util/FileLoggerSuite.scala | 169 --------- 10 files changed, 675 insertions(+), 929 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/FileLogger.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 65a1a8fd7e929..b10b7b8d32523 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -23,7 +23,7 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, var appUiUrl: String, - val eventLogDir: Option[String] = None) + val eventLogFile: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 82a54dbfb5330..792d15b99ea0d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,14 +17,16 @@ package org.apache.spark.deploy.history -import java.io.FileNotFoundException +import java.io.{BufferedInputStream, FileNotFoundException, InputStream} import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.permission.AccessControlException import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils @@ -64,6 +66,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] = new mutable.LinkedHashMap() + // Constants used to parse Spark 1.0.0 log directories. + private[history] val LOG_PREFIX = "EVENT_LOG_" + private[history] val SPARK_VERSION_PREFIX = "SPARK_VERSION_" + private[history] val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + private[history] val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + /** * A background thread that periodically checks for event log updates on disk. * @@ -90,7 +98,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis initialize() - private def initialize() { + private def initialize(): Unit = { // Validate the log directory. val path = new Path(logDir) if (!fs.exists(path)) { @@ -106,8 +114,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } checkForLogs() - logCheckingThread.setDaemon(true) - logCheckingThread.start() + + // Disable the background thread during tests. + if (!conf.contains("spark.testing")) { + logCheckingThread.setDaemon(true) + logCheckingThread.start() + } } override def getListing() = applications.values @@ -115,8 +127,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis override def getAppUI(appId: String): Option[SparkUI] = { try { applications.get(appId).map { info => - val (replayBus, appListener) = createReplayBus(fs.getFileStatus( - new Path(logDir, info.logDir))) + val replayBus = new ReplayListenerBus() val ui = { val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) @@ -125,15 +136,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // Do not call ui.bind() to avoid creating a new server for each application } - replayBus.replay() + val appListener = new ApplicationEventListener() + replayBus.addListener(appListener) + val appInfo = replay(fs.getFileStatus(new Path(logDir, info.logPath)), replayBus) - ui.setAppName(s"${appListener.appName.getOrElse(NOT_STARTED)} ($appId)") + ui.setAppName(s"${appInfo.name} ($appId)") val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) ui.getSecurityManager.setAcls(uiAclsEnabled) // make sure to set admin acls before view acls so they are properly picked up ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) - ui.getSecurityManager.setViewAcls(appListener.sparkUser.getOrElse(NOT_STARTED), + ui.getSecurityManager.setViewAcls(appInfo.sparkUser, appListener.viewAcls.getOrElse("")) ui } @@ -149,41 +162,45 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis * Tries to reuse as much of the data already in memory as possible, by not reading * applications that haven't been updated since last time the logs were checked. */ - private def checkForLogs() = { + private[history] def checkForLogs(): Unit = { lastLogCheckTimeMs = getMonotonicTimeMs() logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) - try { - val logStatus = fs.listStatus(new Path(logDir)) - val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - // Load all new logs from the log directory. Only directories that have a modification time - // later than the last known log directory will be loaded. + try { var newLastModifiedTime = lastModifiedTime - val logInfos = logDirs - .filter { dir => - if (fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))) { - val modTime = getModificationTime(dir) - newLastModifiedTime = math.max(newLastModifiedTime, modTime) - modTime > lastModifiedTime - } else { - false + val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) + .getOrElse(Seq[FileStatus]()) + val logInfos = statusList + .filter { entry => + try { + val isFinishedApplication = + if (isLegacyLogDirectory(entry)) { + fs.exists(new Path(entry.getPath(), APPLICATION_COMPLETE)) + } else { + !entry.getPath().getName().endsWith(EventLoggingListener.IN_PROGRESS) + } + + if (isFinishedApplication) { + val modTime = getModificationTime(entry) + newLastModifiedTime = math.max(newLastModifiedTime, modTime) + modTime >= lastModifiedTime + } else { + false + } + } catch { + case e: AccessControlException => + // Do not use "logInfo" since these messages can get pretty noisy if printed on + // every poll. + logDebug(s"No permission to read $entry, ignoring.") + false } } - .flatMap { dir => + .flatMap { entry => try { - val (replayBus, appListener) = createReplayBus(dir) - replayBus.replay() - Some(new FsApplicationHistoryInfo( - dir.getPath().getName(), - appListener.appId.getOrElse(dir.getPath().getName()), - appListener.appName.getOrElse(NOT_STARTED), - appListener.startTime.getOrElse(-1L), - appListener.endTime.getOrElse(-1L), - getModificationTime(dir), - appListener.sparkUser.getOrElse(NOT_STARTED))) + Some(replay(entry, new ReplayListenerBus())) } catch { case e: Exception => - logInfo(s"Failed to load application log data from $dir.", e) + logError(s"Failed to load application log data from $entry.", e) None } } @@ -217,37 +234,100 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis applications = newApps } } catch { - case t: Throwable => logError("Exception in checking for event log updates", t) + case e: Exception => logError("Exception in checking for event log updates", e) } } - private def createReplayBus(logDir: FileStatus): (ReplayListenerBus, ApplicationEventListener) = { - val path = logDir.getPath() - val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs) - val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec) - val appListener = new ApplicationEventListener - replayBus.addListener(appListener) - (replayBus, appListener) + /** + * Replays the events in the specified log file and returns information about the associated + * application. + */ + private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationHistoryInfo = { + val logPath = eventLog.getPath() + val (logInput, sparkVersion) = + if (isLegacyLogDirectory(eventLog)) { + openLegacyEventLog(logPath) + } else { + EventLoggingListener.openEventLog(logPath, fs) + } + try { + val appListener = new ApplicationEventListener + bus.addListener(appListener) + bus.replay(logInput, sparkVersion) + new FsApplicationHistoryInfo( + logPath.getName(), + appListener.appId.getOrElse(logPath.getName()), + appListener.appName.getOrElse(NOT_STARTED), + appListener.startTime.getOrElse(-1L), + appListener.endTime.getOrElse(-1L), + getModificationTime(eventLog), + appListener.sparkUser.getOrElse(NOT_STARTED)) + } finally { + logInput.close() + } } - /** Return when this directory was last modified. */ - private def getModificationTime(dir: FileStatus): Long = { - try { - val logFiles = fs.listStatus(dir.getPath) - if (logFiles != null && !logFiles.isEmpty) { - logFiles.map(_.getModificationTime).max - } else { - dir.getModificationTime + /** + * Loads a legacy log directory. This assumes that the log directory contains a single event + * log file (along with other metadata files), which is the case for directories generated by + * the code in previous releases. + * + * @return 2-tuple of (input stream of the events, version of Spark which wrote the log) + */ + private[history] def openLegacyEventLog(dir: Path): (InputStream, String) = { + val children = fs.listStatus(dir) + var eventLogPath: Path = null + var codecName: Option[String] = None + var sparkVersion: String = null + + children.foreach { child => + child.getPath().getName() match { + case name if name.startsWith(LOG_PREFIX) => + eventLogPath = child.getPath() + + case codec if codec.startsWith(COMPRESSION_CODEC_PREFIX) => + codecName = Some(codec.substring(COMPRESSION_CODEC_PREFIX.length())) + + case version if version.startsWith(SPARK_VERSION_PREFIX) => + sparkVersion = version.substring(SPARK_VERSION_PREFIX.length()) + + case _ => } - } catch { - case t: Throwable => - logError("Exception in accessing modification time of %s".format(dir.getPath), t) - -1L + } + + if (eventLogPath == null || sparkVersion == null) { + throw new IllegalArgumentException(s"$dir is not a Spark application log directory.") + } + + val codec = try { + codecName.map { c => CompressionCodec.createCodec(conf, c) } + } catch { + case e: Exception => + throw new IllegalArgumentException(s"Unknown compression codec $codecName.") + } + + val in = new BufferedInputStream(fs.open(eventLogPath)) + (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion) + } + + /** + * Return whether the specified event log path contains a old directory-based event log. + * Previously, the event log of an application comprises of multiple files in a directory. + * As of Spark 1.3, these files are consolidated into a single one that replaces the directory. + * See SPARK-2261 for more detail. + */ + private def isLegacyLogDirectory(entry: FileStatus): Boolean = entry.isDir() + + private def getModificationTime(fsEntry: FileStatus): Long = { + if (fsEntry.isDir) { + fs.listStatus(fsEntry.getPath).map(_.getModificationTime()).max + } else { + fsEntry.getModificationTime() } } /** Returns the system's mononotically increasing time. */ - private def getMonotonicTimeMs() = System.nanoTime() / (1000 * 1000) + private def getMonotonicTimeMs(): Long = System.nanoTime() / (1000 * 1000) } @@ -256,7 +336,7 @@ private object FsHistoryProvider { } private class FsApplicationHistoryInfo( - val logDir: String, + val logPath: String, id: String, name: String, startTime: Long, 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 1f9f35d32059d..ed5eab9f473ce 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 @@ -17,6 +17,7 @@ package org.apache.spark.deploy.master +import java.io.FileNotFoundException import java.net.URLEncoder import java.text.SimpleDateFormat import java.util.Date @@ -32,6 +33,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.Serialization import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.Path import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, @@ -56,6 +58,7 @@ private[spark] class Master( import context.dispatcher // to use Akka's scheduler.schedule() val conf = new SparkConf + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 @@ -514,7 +517,7 @@ private[spark] class Master( val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) val numWorkersAlive = 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 @@ -711,41 +714,38 @@ private[spark] class Master( def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogDir = app.desc.eventLogDir.getOrElse { + val eventLogFile = app.desc.eventLogFile.getOrElse { // Event logging is not enabled for this application app.desc.appUiUrl = notFoundBasePath return false } - val appEventLogDir = EventLoggingListener.getLogDirPath(eventLogDir, app.id) - val fileSystem = Utils.getHadoopFileSystem(appEventLogDir, - SparkHadoopUtil.get.newConfiguration(conf)) - val eventLogInfo = EventLoggingListener.parseLoggingInfo(appEventLogDir, fileSystem) - val eventLogPaths = eventLogInfo.logPaths - val compressionCodec = eventLogInfo.compressionCodec - - if (eventLogPaths.isEmpty) { - // Event logging is enabled for this application, but no event logs are found - val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in $appEventLogDir." - logWarning(msg) - msg += " Did you specify the correct logging directory?" - msg = URLEncoder.encode(msg, "UTF-8") - app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" - return false - } - try { - val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) + val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) + val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}") - replayBus.replay() + try { + replayBus.replay(logInput, sparkVersion) + } finally { + logInput.close() + } appIdToUI(app.id) = ui webUi.attachSparkUI(ui) // Application UI is successfully rebuilt, so link the Master UI to it - app.desc.appUiUrl = ui.getBasePath + app.desc.appUiUrl = ui.basePath true } catch { + case fnf: FileNotFoundException => + // Event logging is enabled for this application, but no event logs are found + val title = s"Application history not found (${app.id})" + var msg = s"No event logs found for application $appName in $eventLogFile." + logWarning(msg) + msg += " Did you specify the correct logging directory?" + msg = URLEncoder.encode(msg, "UTF-8") + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" + false case e: Exception => // Relay exception message to application UI page val title = s"Application history load error (${app.id})" 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 597dbc884913c..27bf4f1599076 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,20 +17,23 @@ package org.apache.spark.scheduler +import java.io._ +import java.net.URI + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import com.google.common.base.Charsets import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SPARK_VERSION} 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} +import org.apache.spark.util.{JsonProtocol, Utils} /** * A SparkListener that logs events to persistent storage. @@ -58,36 +61,78 @@ private[spark] class EventLoggingListener( private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - val logDir = EventLoggingListener.getLogDirPath(logBaseDir, appId) - val logDirName: String = logDir.split("/").last - protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, - shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) + private val fileSystem = Utils.getHadoopFileSystem(new URI(logBaseDir), hadoopConf) + + // Only defined if the file system scheme is not local + private var hadoopDataStream: Option[FSDataOutputStream] = None + + // The Hadoop APIs have changed over time, so we use reflection to figure out + // the correct method to use to flush a hadoop data stream. See SPARK-1518 + // for details. + private val hadoopFlushMethod = { + val cls = classOf[FSDataOutputStream] + scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync")) + } + + private var writer: Option[PrintWriter] = None // For testing. Keep track of all JSON serialized events that have been logged. private[scheduler] val loggedEvents = new ArrayBuffer[JValue] + // Visible for tests only. + private[scheduler] val logPath = getLogPath(logBaseDir, appId) + /** - * Begin logging events. - * If compression is used, log a file that indicates which compression library is used. + * Creates the log file in the configured log directory. */ def start() { - logger.start() - logInfo("Logging events to %s".format(logDir)) - if (shouldCompress) { - val codec = - sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) - logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + if (!fileSystem.isDirectory(new Path(logBaseDir))) { + throw new IllegalArgumentException(s"Log directory $logBaseDir does not exist.") + } + + val workingPath = logPath + IN_PROGRESS + val uri = new URI(workingPath) + val path = new Path(workingPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme + val isDefaultLocal = defaultFs == null || defaultFs == "file" + + if (shouldOverwrite && fileSystem.exists(path)) { + logWarning(s"Event log $path already exists. Overwriting...") + fileSystem.delete(path, true) } - logger.newFile(SPARK_VERSION_PREFIX + SPARK_VERSION) - logger.newFile(LOG_PREFIX + logger.fileIndex) + + /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. */ + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { + new FileOutputStream(uri.getPath) + } else { + hadoopDataStream = Some(fileSystem.create(path)) + hadoopDataStream.get + } + + val compressionCodec = + if (shouldCompress) { + Some(CompressionCodec.createCodec(sparkConf)) + } else { + None + } + + fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) + val logStream = initEventLog(new BufferedOutputStream(dstream, outputBufferSize), + compressionCodec) + writer = Some(new PrintWriter(logStream)) + + logInfo("Logging events to %s".format(logPath)) } /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = JsonProtocol.sparkEventToJson(event) - logger.logLine(compact(render(eventJson))) + writer.foreach(_.println(compact(render(eventJson)))) if (flushLogger) { - logger.flush() + writer.foreach(_.flush()) + hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) } if (testing) { loggedEvents += eventJson @@ -123,130 +168,164 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) override def onApplicationEnd(event: SparkListenerApplicationEnd) = logEvent(event, flushLogger = true) + // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } /** - * Stop logging events. - * In addition, create an empty special file to indicate application completion. + * Stop logging events. The event log file will be renamed so that it loses the + * ".inprogress" suffix. */ def stop() = { - logger.newFile(APPLICATION_COMPLETE) - logger.stop() + writer.foreach(_.close()) + + val target = new Path(logPath) + if (fileSystem.exists(target)) { + if (shouldOverwrite) { + logWarning(s"Event log $target already exists. Overwriting...") + fileSystem.delete(target, true) + } else { + throw new IOException("Target log file already exists (%s)".format(logPath)) + } + } + fileSystem.rename(new Path(logPath + IN_PROGRESS), target) } + } private[spark] object EventLoggingListener extends Logging { + // Suffix applied to the names of files still being written by applications. + val IN_PROGRESS = ".inprogress" val DEFAULT_LOG_DIR = "/tmp/spark-events" - val LOG_PREFIX = "EVENT_LOG_" - val SPARK_VERSION_PREFIX = "SPARK_VERSION_" - val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" - val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" - val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) - // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = new mutable.HashMap[String, CompressionCodec] + private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) - def isEventLogFile(fileName: String): Boolean = { - fileName.startsWith(LOG_PREFIX) - } + // Marker for the end of header data in a log file. After this marker, log data, potentially + // compressed, will be found. + private val HEADER_END_MARKER = "=== LOG_HEADER_END ===" - def isSparkVersionFile(fileName: String): Boolean = { - fileName.startsWith(SPARK_VERSION_PREFIX) - } + // To avoid corrupted files causing the heap to fill up. Value is arbitrary. + private val MAX_HEADER_LINE_LENGTH = 4096 - def isCompressionCodecFile(fileName: String): Boolean = { - fileName.startsWith(COMPRESSION_CODEC_PREFIX) - } + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] - def isApplicationCompleteFile(fileName: String): Boolean = { - fileName == APPLICATION_COMPLETE - } + /** + * Write metadata about the event log to the given stream. + * + * The header is a serialized version of a map, except it does not use Java serialization to + * avoid incompatibilities between different JDKs. It writes one map entry per line, in + * "key=value" format. + * + * The very last entry in the header is the `HEADER_END_MARKER` marker, so that the parsing code + * can know when to stop. + * + * The format needs to be kept in sync with the openEventLog() method below. Also, it cannot + * change in new Spark versions without some other way of detecting the change (like some + * metadata encoded in the file name). + * + * @param logStream Raw output stream to the even log file. + * @param compressionCodec Optional compression codec to use. + * @return A stream where to write event log data. This may be a wrapper around the original + * stream (for example, when compression is enabled). + */ + def initEventLog( + logStream: OutputStream, + compressionCodec: Option[CompressionCodec]): OutputStream = { + val meta = mutable.HashMap(("version" -> SPARK_VERSION)) + compressionCodec.foreach { codec => + meta += ("compressionCodec" -> codec.getClass().getName()) + } - def parseSparkVersion(fileName: String): String = { - if (isSparkVersionFile(fileName)) { - fileName.replaceAll(SPARK_VERSION_PREFIX, "") - } else "" - } + def write(entry: String) = { + val bytes = entry.getBytes(Charsets.UTF_8) + if (bytes.length > MAX_HEADER_LINE_LENGTH) { + throw new IOException(s"Header entry too long: ${entry}") + } + logStream.write(bytes, 0, bytes.length) + } - def parseCompressionCodec(fileName: String): String = { - if (isCompressionCodecFile(fileName)) { - fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") - } else "" + meta.foreach { case (k, v) => write(s"$k=$v\n") } + write(s"$HEADER_END_MARKER\n") + compressionCodec.map(_.compressedOutputStream(logStream)).getOrElse(logStream) } /** - * Return a file-system-safe path to the log directory for the given application. + * Return a file-system-safe path to the log file for the given application. * - * @param logBaseDir A base directory for the path to the log directory for given application. + * @param logBaseDir Directory where the log file will be written. * @param appId A unique app ID. * @return A path which consists of file-system-safe characters. */ - def getLogDirPath(logBaseDir: String, appId: String): String = { + def getLogPath(logBaseDir: String, appId: String): String = { val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") } /** - * Parse the event logging information associated with the logs in the given directory. + * Opens an event log file and returns an input stream to the event data. * - * Specifically, this looks for event log files, the Spark version file, the compression - * codec file (if event logs are compressed), and the application completion file (if the - * application has run to completion). + * @return 2-tuple (event input stream, Spark version of event data) */ - def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + def openEventLog(log: Path, fs: FileSystem): (InputStream, String) = { + // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain + // IOException when a file does not exist, so try our best to throw a proper exception. + if (!fs.exists(log)) { + throw new FileNotFoundException(s"File $log does not exist.") + } + + val in = new BufferedInputStream(fs.open(log)) + // Read a single line from the input stream without buffering. + // We cannot use BufferedReader because we must avoid reading + // beyond the end of the header, after which the content of the + // file may be compressed. + def readLine(): String = { + val bytes = new ByteArrayOutputStream() + var next = in.read() + var count = 0 + while (next != '\n') { + if (next == -1) { + throw new IOException("Unexpected end of file.") + } + bytes.write(next) + count = count + 1 + if (count > MAX_HEADER_LINE_LENGTH) { + throw new IOException("Maximum header line length exceeded.") + } + next = in.read() + } + new String(bytes.toByteArray(), Charsets.UTF_8) + } + + // Parse the header metadata in the form of k=v pairs + // This assumes that every line before the header end marker follows this format try { - val fileStatuses = fileSystem.listStatus(logDir) - val filePaths = - if (fileStatuses != null) { - fileStatuses.filter(!_.isDir).map(_.getPath).toSeq - } else { - Seq[Path]() + val meta = new mutable.HashMap[String, String]() + var foundEndMarker = false + while (!foundEndMarker) { + readLine() match { + case HEADER_END_MARKER => + foundEndMarker = true + case entry => + val prop = entry.split("=", 2) + if (prop.length != 2) { + throw new IllegalArgumentException("Invalid metadata in log file.") + } + meta += (prop(0) -> prop(1)) } - if (filePaths.isEmpty) { - logWarning("No files found in logging directory %s".format(logDir)) } - EventLoggingInfo( - logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, - sparkVersion = filePaths - .find { path => isSparkVersionFile(path.getName) } - .map { path => parseSparkVersion(path.getName) } - .getOrElse(""), - compressionCodec = filePaths - .find { path => isCompressionCodecFile(path.getName) } - .map { path => - val codec = EventLoggingListener.parseCompressionCodec(path.getName) - val conf = new SparkConf - conf.set("spark.io.compression.codec", codec) - codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) - }, - applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } - ) + + val sparkVersion = meta.get("version").getOrElse( + throw new IllegalArgumentException("Missing Spark version in log metadata.")) + val codec = meta.get("compressionCodec").map { codecName => + codecMap.getOrElseUpdate(codecName, CompressionCodec.createCodec(new SparkConf, codecName)) + } + (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion) } catch { case e: Exception => - logError("Exception in parsing logging info from directory %s".format(logDir), e) - EventLoggingInfo.empty + in.close() + throw e } } - /** - * Parse the event logging information associated with the logs in the given directory. - */ - def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { - parseLoggingInfo(new Path(logDir), fileSystem) - } -} - - -/** - * Information needed to process the event logs associated with an application. - */ -private[spark] case class EventLoggingInfo( - logPaths: Seq[Path], - sparkVersion: String, - compressionCodec: Option[CompressionCodec], - applicationComplete: Boolean = false) - -private[spark] object EventLoggingInfo { - def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index f89724d4ea196..584f4e7789d1a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -17,74 +17,45 @@ package org.apache.spark.scheduler -import java.io.{BufferedInputStream, InputStream} +import java.io.{InputStream, IOException} import scala.io.Source -import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ import org.apache.spark.Logging -import org.apache.spark.io.CompressionCodec import org.apache.spark.util.JsonProtocol /** - * A SparkListenerBus that replays logged events from persisted storage. - * - * This assumes the given paths are valid log files, where each line can be deserialized into - * exactly one SparkListenerEvent. + * A SparkListenerBus that can be used to replay events from serialized event data. */ -private[spark] class ReplayListenerBus( - logPaths: Seq[Path], - fileSystem: FileSystem, - compressionCodec: Option[CompressionCodec]) - extends SparkListenerBus with Logging { - - private var replayed = false - - if (logPaths.length == 0) { - logWarning("Log path provided contains no log files.") - } +private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { /** - * Replay each event in the order maintained in the given logs. - * This should only be called exactly once. + * Replay each event in the order maintained in the given stream. The stream is expected to + * contain one JSON-encoded SparkListenerEvent per line. + * + * This method can be called multiple times, but the listener behavior is undefined after any + * error is thrown by this method. + * + * @param logData Stream containing event log data. + * @param version Spark version that generated the events. */ - def replay() { - assert(!replayed, "ReplayListenerBus cannot replay events more than once") - logPaths.foreach { path => - // Keep track of input streams at all levels to close them later - // This is necessary because an exception can occur in between stream initializations - var fileStream: Option[InputStream] = None - var bufferedStream: Option[InputStream] = None - var compressStream: Option[InputStream] = None - var currentLine = "" - try { - fileStream = Some(fileSystem.open(path)) - bufferedStream = Some(new BufferedInputStream(fileStream.get)) - compressStream = Some(wrapForCompression(bufferedStream.get)) - - // Parse each line as an event and post the event to all attached listeners - val lines = Source.fromInputStream(compressStream.get).getLines() - lines.foreach { line => - currentLine = line - postToAll(JsonProtocol.sparkEventFromJson(parse(line))) - } - } catch { - case e: Exception => - logError("Exception in parsing Spark event log %s".format(path), e) - logError("Malformed line: %s\n".format(currentLine)) - } finally { - fileStream.foreach(_.close()) - bufferedStream.foreach(_.close()) - compressStream.foreach(_.close()) + def replay(logData: InputStream, version: String) { + var currentLine: String = null + try { + val lines = Source.fromInputStream(logData).getLines() + lines.foreach { line => + currentLine = line + postToAll(JsonProtocol.sparkEventFromJson(parse(line))) } + } catch { + case ioe: IOException => + throw ioe + case e: Exception => + logError("Exception in parsing Spark event log.", e) + logError("Malformed line: %s\n".format(currentLine)) } - replayed = true } - /** If a compression codec is specified, wrap the given stream in a compression stream. */ - private def wrapForCompression(stream: InputStream): InputStream = { - compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) - } } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala deleted file mode 100644 index fdc73f08261a6..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.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.util - -import java.io.{BufferedOutputStream, FileOutputStream, IOException, PrintWriter} -import java.net.URI -import java.text.SimpleDateFormat -import java.util.Date - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} -import org.apache.hadoop.fs.permission.FsPermission - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec - -/** - * A generic class for logging information to file. - * - * @param logDir Path to the directory in which files are logged - * @param outputBufferSize The buffer size to use when writing to an output stream in bytes - * @param compress Whether to compress output - * @param overwrite Whether to overwrite existing files - */ -private[spark] class FileLogger( - logDir: String, - sparkConf: SparkConf, - hadoopConf: Configuration, - outputBufferSize: Int = 8 * 1024, // 8 KB - compress: Boolean = false, - overwrite: Boolean = true, - dirPermissions: Option[FsPermission] = None) - extends Logging { - - def this( - logDir: String, - sparkConf: SparkConf, - compress: Boolean, - overwrite: Boolean) = { - this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = compress, - overwrite = overwrite) - } - - def this( - logDir: String, - sparkConf: SparkConf, - compress: Boolean) = { - this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = compress, - overwrite = true) - } - - def this( - logDir: String, - sparkConf: SparkConf) = { - this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = false, - overwrite = true) - } - - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - } - - /** - * To avoid effects of FileSystem#close or FileSystem.closeAll called from other modules, - * create unique FileSystem instance only for FileLogger - */ - private val fileSystem = { - val conf = SparkHadoopUtil.get.newConfiguration(sparkConf) - val logUri = new URI(logDir) - val scheme = logUri.getScheme - if (scheme == "hdfs") { - conf.setBoolean("fs.hdfs.impl.disable.cache", true) - } - FileSystem.get(logUri, conf) - } - - var fileIndex = 0 - - // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(sparkConf) - - // Only defined if the file system scheme is not local - private var hadoopDataStream: Option[FSDataOutputStream] = None - - // The Hadoop APIs have changed over time, so we use reflection to figure out - // the correct method to use to flush a hadoop data stream. See SPARK-1518 - // for details. - private val hadoopFlushMethod = { - val cls = classOf[FSDataOutputStream] - scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync")) - } - - private var writer: Option[PrintWriter] = None - - /** - * Start this logger by creating the logging directory. - */ - def start() { - createLogDir() - } - - /** - * Create a logging directory with the given path. - */ - private def createLogDir() { - val path = new Path(logDir) - if (fileSystem.exists(path)) { - if (overwrite) { - logWarning("Log directory %s already exists. Overwriting...".format(logDir)) - // Second parameter is whether to delete recursively - fileSystem.delete(path, true) - } else { - throw new IOException("Log directory %s already exists!".format(logDir)) - } - } - if (!fileSystem.mkdirs(path)) { - throw new IOException("Error in creating log directory: %s".format(logDir)) - } - if (dirPermissions.isDefined) { - val fsStatus = fileSystem.getFileStatus(path) - if (fsStatus.getPermission.toShort != dirPermissions.get.toShort) { - fileSystem.setPermission(path, dirPermissions.get) - } - } - } - - /** - * Create a new writer for the file identified by the given path. - * If the permissions are not passed in, it will default to use the permissions - * (dirPermissions) used when class was instantiated. - */ - private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { - val logPath = logDir + "/" + fileName - val uri = new URI(logPath) - val path = new Path(logPath) - val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme - val isDefaultLocal = defaultFs == null || defaultFs == "file" - - /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). - * Therefore, for local files, use FileOutputStream instead. */ - val dstream = - if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { - // Second parameter is whether to append - new FileOutputStream(uri.getPath, !overwrite) - } else { - hadoopDataStream = Some(fileSystem.create(path, overwrite)) - hadoopDataStream.get - } - - perms.orElse(dirPermissions).foreach { p => fileSystem.setPermission(path, p) } - val bstream = new BufferedOutputStream(dstream, outputBufferSize) - val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream - new PrintWriter(cstream) - } - - /** - * Log the message to the given writer. - * @param msg The message to be logged - * @param withTime Whether to prepend message with a timestamp - */ - def log(msg: String, withTime: Boolean = false) { - val writeInfo = if (!withTime) { - msg - } else { - val date = new Date(System.currentTimeMillis) - dateFormat.get.format(date) + ": " + msg - } - writer.foreach(_.print(writeInfo)) - } - - /** - * Log the message to the given writer as a new line. - * @param msg The message to be logged - * @param withTime Whether to prepend message with a timestamp - */ - def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) - - /** - * Flush the writer to disk manually. - * - * When using a Hadoop filesystem, we need to invoke the hflush or sync - * method. In HDFS, hflush guarantees that the data gets to all the - * DataNodes. - */ - def flush() { - writer.foreach(_.flush()) - hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) - } - - /** - * Close the writer. Any subsequent calls to log or flush will have no effect. - */ - def close() { - writer.foreach(_.close()) - writer = None - } - - /** - * Start a writer for a new file, closing the existing one if it exists. - * @param fileName Name of the new file, defaulting to the file index if not provided. - * @param perms Permissions to put on the new file. - */ - def newFile(fileName: String = "", perms: Option[FsPermission] = None) { - fileIndex += 1 - writer.foreach(_.close()) - val name = fileName match { - case "" => fileIndex.toString - case _ => fileName - } - writer = Some(createWriter(name, perms)) - } - - /** - * Close all open writers, streams, and file systems. Any subsequent uses of this FileLogger - * instance will throw exceptions. - */ - def stop() { - hadoopDataStream.foreach(_.close()) - writer.foreach(_.close()) - } -} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala new file mode 100644 index 0000000000000..d719e9301f4fd --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -0,0 +1,185 @@ +/* + * 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.deploy.history + +import java.io.{File, FileOutputStream, OutputStreamWriter} + +import scala.io.Source + +import com.google.common.io.Files +import org.apache.hadoop.fs.Path +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.Matchers + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.io._ +import org.apache.spark.scheduler._ +import org.apache.spark.util.{JsonProtocol, Utils} + +class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { + + private var testDir: File = null + + private var provider: FsHistoryProvider = null + + before { + testDir = Utils.createTempDir() + provider = new FsHistoryProvider(new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0")) + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Parse new and old application logs") { + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0") + val provider = new FsHistoryProvider(conf) + + // Write a new-style application log. + val logFile1 = new File(testDir, "new1") + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + + // Write an unfinished app, new-style. + writeFile(new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS), true, None, + SparkListenerApplicationStart("app2-2", None, 1L, "test") + ) + + // Write an old-style application log. + val oldLog = new File(testDir, "old1") + oldLog.mkdir() + createEmptyFile(new File(oldLog, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldLog, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("app3", None, 2L, "test"), + SparkListenerApplicationEnd(3L) + ) + createEmptyFile(new File(oldLog, provider.APPLICATION_COMPLETE)) + + // Write an unfinished app, old-style. + val oldLog2 = new File(testDir, "old2") + oldLog2.mkdir() + createEmptyFile(new File(oldLog2, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldLog2, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("app4", None, 2L, "test") + ) + + // Force a reload of data from the log directory, and check that both logs are loaded. + // Take the opportunity to check that the offset checks work as expected. + provider.checkForLogs() + + val list = provider.getListing().toSeq + list should not be (null) + list.size should be (2) + + list(0) should be (ApplicationHistoryInfo(oldLog.getName(), "app3", 2L, 3L, + oldLog.lastModified(), "test")) + list(1) should be (ApplicationHistoryInfo(logFile1.getName(), "app1-1", 1L, 2L, + logFile1.lastModified(), "test")) + + // Make sure the UI can be rendered. + list.foreach { case info => + val appUi = provider.getAppUI(info.id) + appUi should not be null + } + } + + test("Parse legacy logs with compression codec set") { + val testCodecs = List((classOf[LZFCompressionCodec].getName(), true), + (classOf[SnappyCompressionCodec].getName(), true), + ("invalid.codec", false)) + + testCodecs.foreach { case (codecName, valid) => + val codec = if (valid) CompressionCodec.createCodec(new SparkConf(), codecName) else null + val logDir = new File(testDir, codecName) + logDir.mkdir() + createEmptyFile(new File(logDir, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(logDir, provider.LOG_PREFIX + "1"), false, Option(codec), + SparkListenerApplicationStart("app2", None, 2L, "test"), + SparkListenerApplicationEnd(3L) + ) + createEmptyFile(new File(logDir, provider.COMPRESSION_CODEC_PREFIX + codecName)) + + val logPath = new Path(logDir.getAbsolutePath()) + try { + val (logInput, sparkVersion) = provider.openLegacyEventLog(logPath) + try { + Source.fromInputStream(logInput).getLines().toSeq.size should be (2) + } finally { + logInput.close() + } + } catch { + case e: IllegalArgumentException => + valid should be (false) + } + } + } + + test("SPARK-3697: ignore directories that cannot be read.") { + val logFile1 = new File(testDir, "new1") + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + val logFile2 = new File(testDir, "new2") + writeFile(logFile2, true, None, + SparkListenerApplicationStart("app1-2", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + logFile2.setReadable(false, false) + + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0") + val provider = new FsHistoryProvider(conf) + provider.checkForLogs() + + val list = provider.getListing().toSeq + list should not be (null) + list.size should be (1) + } + + private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], + events: SparkListenerEvent*) = { + val out = + if (isNewFormat) { + EventLoggingListener.initEventLog(new FileOutputStream(file), codec) + } else { + val fileStream = new FileOutputStream(file) + codec.map(_.compressedOutputStream(fileStream)).getOrElse(fileStream) + } + val writer = new OutputStreamWriter(out, "UTF-8") + try { + events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) + } finally { + writer.close() + } + } + + private def createEmptyFile(file: File) = { + new FileOutputStream(file).close() + } + +} 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 abc300fcffaf9..5909811c2034f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -17,69 +17,59 @@ package org.apache.spark.scheduler +import java.io.{File, FileOutputStream, InputStream, IOException} + import scala.collection.mutable import scala.io.Source -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{SparkConf, SparkContext} +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.io._ import org.apache.spark.util.{JsonProtocol, Utils} -import java.io.File - /** * Test whether EventLoggingListener logs events properly. * - * This tests whether EventLoggingListener actually creates special files while logging events, - * whether the parsing of these special files is correct, and whether the logged events can be - * read and deserialized into actual SparkListenerEvents. + * This tests whether EventLoggingListener actually log files with expected name patterns while + * logging events, whether the parsing of the file names is correct, and whether the logged events + * can be read and deserialized into actual SparkListenerEvents. */ -class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { +class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Logging { + import EventLoggingListenerSuite._ + private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = Seq[String]( - "org.apache.spark.io.LZFCompressionCodec", - "org.apache.spark.io.SnappyCompressionCodec" - ) private var testDir: File = _ - private var logDirPath: Path = _ + private var testDirPath: Path = _ before { testDir = Utils.createTempDir() - logDirPath = Utils.getFilePath(testDir, "spark-events") + testDir.deleteOnExit() + testDirPath = new Path(testDir.getAbsolutePath()) } after { Utils.deleteRecursively(testDir) } - test("Parse names of special files") { - testParsingFileName() - } - - test("Verify special files exist") { - testSpecialFilesExist() - } - - test("Verify special files exist with compression") { - allCompressionCodecs.foreach { codec => - testSpecialFilesExist(compressionCodec = Some(codec)) - } - } + test("Verify log file exist") { + // Verify logging directory exists + val conf = getLoggingConf(testDirPath) + val eventLogger = new EventLoggingListener("test", testDirPath.toUri().toString(), conf) + eventLogger.start() - test("Parse event logging info") { - testParsingLogInfo() - } + val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) + assert(fileSystem.exists(logPath)) + val logStatus = fileSystem.getFileStatus(logPath) + assert(logStatus.isFile) - test("Parse event logging info with compression") { - allCompressionCodecs.foreach { codec => - testParsingLogInfo(compressionCodec = Some(codec)) - } + // Verify log is renamed after stop() + eventLogger.stop() + assert(fileSystem.getFileStatus(new Path(eventLogger.logPath)).isFile()) } test("Basic event logging") { @@ -87,7 +77,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } test("Basic event logging with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testEventLogging(compressionCodec = Some(codec)) } } @@ -97,11 +87,25 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } test("End-to-end event logging with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testApplicationEventLogging(compressionCodec = Some(codec)) } } + test("Log overwriting") { + val log = new FileOutputStream(new File(testDir, "test")) + log.close() + try { + testEventLogging() + assert(false) + } catch { + case e: IOException => + // Expected, since we haven't enabled log overwrite. + } + + // Try again, but enable overwriting. + testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true")) + } /* ----------------- * * Actual test logic * @@ -109,130 +113,19 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { import EventLoggingListenerSuite._ - /** - * Test whether names of special files are correctly identified and parsed. - */ - private def testParsingFileName() { - val logPrefix = EventLoggingListener.LOG_PREFIX - val sparkVersionPrefix = EventLoggingListener.SPARK_VERSION_PREFIX - val compressionCodecPrefix = EventLoggingListener.COMPRESSION_CODEC_PREFIX - val applicationComplete = EventLoggingListener.APPLICATION_COMPLETE - assert(EventLoggingListener.isEventLogFile(logPrefix + "0")) - assert(EventLoggingListener.isEventLogFile(logPrefix + "100")) - assert(EventLoggingListener.isEventLogFile(logPrefix + "ANYTHING")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "0.9.1")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "1.0.0")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "ANYTHING")) - assert(EventLoggingListener.isApplicationCompleteFile(applicationComplete)) - allCompressionCodecs.foreach { codec => - assert(EventLoggingListener.isCompressionCodecFile(compressionCodecPrefix + codec)) - } - - // Negatives - assert(!EventLoggingListener.isEventLogFile("The greatest man of all mankind")) - assert(!EventLoggingListener.isSparkVersionFile("Will never falter in the face of death!")) - assert(!EventLoggingListener.isCompressionCodecFile("Unless he chooses to leave behind")) - assert(!EventLoggingListener.isApplicationCompleteFile("The very treasure he calls Macbeth")) - - // Verify that parsing is correct - assert(EventLoggingListener.parseSparkVersion(sparkVersionPrefix + "1.0.0") === "1.0.0") - allCompressionCodecs.foreach { codec => - assert(EventLoggingListener.parseCompressionCodec(compressionCodecPrefix + codec) === codec) - } - } - - /** - * Test whether the special files produced by EventLoggingListener exist. - * - * There should be exactly one event log and one spark version file throughout the entire - * execution. If a compression codec is specified, then the compression codec file should - * also exist. Only after the application has completed does the test expect the application - * completed file to be present. - */ - private def testSpecialFilesExist(compressionCodec: Option[String] = None) { - - def assertFilesExist(logFiles: Array[FileStatus], loggerStopped: Boolean) { - val numCompressionCodecFiles = if (compressionCodec.isDefined) 1 else 0 - val numApplicationCompleteFiles = if (loggerStopped) 1 else 0 - assert(logFiles.size === 2 + numCompressionCodecFiles + numApplicationCompleteFiles) - assert(eventLogsExist(logFiles)) - assert(sparkVersionExists(logFiles)) - assert(compressionCodecExists(logFiles) === compressionCodec.isDefined) - assert(applicationCompleteExists(logFiles) === loggerStopped) - assertSparkVersionIsValid(logFiles) - compressionCodec.foreach { codec => - assertCompressionCodecIsValid(logFiles, codec) - } - } - - // Verify logging directory exists - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) - eventLogger.start() - val logPath = new Path(eventLogger.logDir) - assert(fileSystem.exists(logPath)) - val logDir = fileSystem.getFileStatus(logPath) - assert(logDir.isDir) - - // Verify special files are as expected before stop() - var logFiles = fileSystem.listStatus(logPath) - assert(logFiles != null) - assertFilesExist(logFiles, loggerStopped = false) - - // Verify special files are as expected after stop() - eventLogger.stop() - logFiles = fileSystem.listStatus(logPath) - assertFilesExist(logFiles, loggerStopped = true) - } - - /** - * Test whether EventLoggingListener correctly parses the correct information from the logs. - * - * This includes whether it returns the correct Spark version, compression codec (if any), - * and the application's completion status. - */ - private def testParsingLogInfo(compressionCodec: Option[String] = None) { - - def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { - assert(info.logPaths.size > 0) - assert(info.sparkVersion === SPARK_VERSION) - assert(info.compressionCodec.isDefined === compressionCodec.isDefined) - info.compressionCodec.foreach { codec => - assert(compressionCodec.isDefined) - val expectedCodec = compressionCodec.get.split('.').last - assert(codec.getClass.getSimpleName === expectedCodec) - } - assert(info.applicationComplete === loggerStopped) - } - - // Verify that all information is correctly parsed before stop() - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) - eventLogger.start() - var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assertInfoCorrect(eventLoggingInfo, loggerStopped = false) - - // Verify that all information is correctly parsed after stop() - eventLogger.stop() - eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assertInfoCorrect(eventLoggingInfo, loggerStopped = true) - } - /** * Test basic event logging functionality. * * This creates two simple events, posts them to the EventLoggingListener, and verifies that * exactly these two events are logged in the expected file. */ - private def testEventLogging(compressionCodec: Option[String] = None) { - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) + private def testEventLogging( + compressionCodec: Option[String] = None, + extraConf: Map[String, String] = Map()) { + val conf = getLoggingConf(testDirPath, compressionCodec) + extraConf.foreach { case (k, v) => conf.set(k, v) } + val logName = compressionCodec.map("test-" + _).getOrElse("test") + val eventLogger = new EventLoggingListener(logName, testDirPath.toUri().toString(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey") @@ -244,17 +137,21 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { listenerBus.addListener(eventLogger) listenerBus.postToAll(applicationStart) listenerBus.postToAll(applicationEnd) + eventLogger.stop() // Verify file contains exactly the two events logged - val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assert(eventLoggingInfo.logPaths.size > 0) - val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) - assert(lines.size === 2) - assert(lines(0).contains("SparkListenerApplicationStart")) - assert(lines(1).contains("SparkListenerApplicationEnd")) - assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) - assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) - eventLogger.stop() + val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), + fileSystem) + try { + val lines = readLines(logData) + assert(lines.size === 2) + assert(lines(0).contains("SparkListenerApplicationStart")) + assert(lines(1).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + } finally { + logData.close() + } } /** @@ -262,12 +159,12 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { * This runs a simple Spark job and asserts that the expected events are logged when expected. */ private def testApplicationEventLogging(compressionCodec: Option[String] = None) { - val conf = getLoggingConf(logDirPath, compressionCodec) + val conf = getLoggingConf(testDirPath, compressionCodec) val sc = new SparkContext("local", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get - val expectedLogDir = logDirPath.toString - assert(eventLogger.logDir.contains(expectedLogDir)) + val expectedLogDir = testDir.toURI().toString() + assert(eventLogger.logPath.startsWith(expectedLogDir + "/")) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) @@ -279,16 +176,22 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Ensure all asserts have actually been triggered eventExistenceListener.assertAllCallbacksInvoked() - } - /** - * Assert that all of the specified events are logged by the given EventLoggingListener. - */ - private def assertEventsExist(eventLogger: EventLoggingListener, events: Seq[String]) { - val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assert(eventLoggingInfo.logPaths.size > 0) - val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) - val eventSet = mutable.Set(events: _*) + // Make sure expected events exist in the log file. + val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), + fileSystem) + val lines = readLines(logData) + val eventSet = mutable.Set( + SparkListenerApplicationStart, + SparkListenerBlockManagerAdded, + SparkListenerEnvironmentUpdate, + SparkListenerJobStart, + SparkListenerJobEnd, + SparkListenerStageSubmitted, + SparkListenerStageCompleted, + SparkListenerTaskStart, + SparkListenerTaskEnd, + SparkListenerApplicationEnd).map(Utils.getFormattedClassName) lines.foreach { line => eventSet.foreach { event => if (line.contains(event)) { @@ -303,19 +206,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) } - /** - * Read all lines from the file specified by the given path. - * If a compression codec is specified, use it to read the file. - */ - private def readFileLines( - filePath: Path, - compressionCodec: Option[CompressionCodec]): Seq[String] = { - val fstream = fileSystem.open(filePath) - val cstream = - compressionCodec.map { codec => - codec.compressedInputStream(fstream) - }.getOrElse(fstream) - Source.fromInputStream(cstream).getLines().toSeq + private def readLines(in: InputStream): Seq[String] = { + Source.fromInputStream(in).getLines().toSeq } /** @@ -328,30 +220,14 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { var appEnded = false override def onJobStart(jobStart: SparkListenerJobStart) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerApplicationStart), - Utils.getFormattedClassName(SparkListenerBlockManagerAdded), - Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) - )) jobStarted = true } override def onJobEnd(jobEnd: SparkListenerJobEnd) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerJobStart), - Utils.getFormattedClassName(SparkListenerJobEnd), - Utils.getFormattedClassName(SparkListenerStageSubmitted), - Utils.getFormattedClassName(SparkListenerStageCompleted), - Utils.getFormattedClassName(SparkListenerTaskStart), - Utils.getFormattedClassName(SparkListenerTaskEnd) - )) jobEnded = true } override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerApplicationEnd) - )) appEnded = true } @@ -362,39 +238,6 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } } - - /* -------------------------------------------------------- * - * Helper methods for validating state of the special files * - * -------------------------------------------------------- */ - - private def eventLogsExist(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isEventLogFile) - } - - private def sparkVersionExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isSparkVersionFile) - } - - private def compressionCodecExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isCompressionCodecFile) - } - - private def applicationCompleteExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isApplicationCompleteFile) - } - - private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { - val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) - assert(file.isDefined) - assert(EventLoggingListener.parseSparkVersion(file.get) === SPARK_VERSION) - } - - private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { - val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isCompressionCodecFile) - assert(file.isDefined) - assert(EventLoggingListener.parseCompressionCodec(file.get) === compressionCodec) - } - } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 90bdfe07f61c9..7e635cb061441 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -22,6 +22,7 @@ import java.io.{File, PrintWriter} import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.{SparkConf, SparkContext, SPARK_VERSION} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec @@ -33,7 +34,6 @@ import org.apache.spark.util.{JsonProtocol, Utils} class ReplayListenerSuite extends FunSuite with BeforeAndAfter { private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = CompressionCodec.ALL_COMPRESSION_CODECS private var testDir: File = _ before { @@ -45,13 +45,29 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { } test("Simple replay") { - testSimpleReplay() - } + val logFilePath = Utils.getFilePath(testDir, "events.txt") + val fstream = fileSystem.create(logFilePath) + val writer = new PrintWriter(fstream) + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, + 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + writer.close() - test("Simple replay with compression") { - allCompressionCodecs.foreach { codec => - testSimpleReplay(Some(codec)) + val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val logData = fileSystem.open(logFilePath) + val eventMonster = new EventMonster(conf) + try { + val replayer = new ReplayListenerBus() + replayer.addListener(eventMonster) + replayer.replay(logData, SPARK_VERSION) + } finally { + logData.close() } + assert(eventMonster.loggedEvents.size === 2) + assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) + assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) } // This assumes the correctness of EventLoggingListener @@ -61,7 +77,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { // This assumes the correctness of EventLoggingListener test("End-to-end replay with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testApplicationReplay(Some(codec)) } } @@ -71,31 +87,6 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * Actual test logic * * ----------------- */ - /** - * Test simple replaying of events. - */ - private def testSimpleReplay(codecName: Option[String] = None) { - val logFilePath = Utils.getFilePath(testDir, "events.txt") - val codec = codecName.map(getCompressionCodec) - val fstream = fileSystem.create(logFilePath) - val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) - val writer = new PrintWriter(cstream) - val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, - 125L, "Mickey") - val applicationEnd = SparkListenerApplicationEnd(1000L) - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) - writer.close() - val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath, codecName) - val eventMonster = new EventMonster(conf) - replayer.addListener(eventMonster) - replayer.replay() - assert(eventMonster.loggedEvents.size === 2) - assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) - assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) - } - /** * Test end-to-end replaying of events. * @@ -105,6 +96,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { */ private def testApplicationReplay(codecName: Option[String] = None) { val logDirPath = Utils.getFilePath(testDir, "test-replay") + fileSystem.mkdirs(logDirPath) + val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) val sc = new SparkContext("local-cluster[2,1,512]", "Test replay", conf) @@ -116,22 +109,21 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { sc.stop() // Prepare information needed for replay - val codec = codecName.map(getCompressionCodec) val applications = fileSystem.listStatus(logDirPath) assert(applications != null && applications.size > 0) - val eventLogDir = applications.sortBy(_.getAccessTime).last - assert(eventLogDir.isDir) - val logFiles = fileSystem.listStatus(eventLogDir.getPath) - assert(logFiles != null && logFiles.size > 0) - val logFile = logFiles.find(_.getPath.getName.startsWith("EVENT_LOG_")) - assert(logFile.isDefined) - val logFilePath = logFile.get.getPath + val eventLog = applications.sortBy(_.getModificationTime).last + assert(eventLog.isFile) // Replay events - val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) val eventMonster = new EventMonster(conf) - replayer.addListener(eventMonster) - replayer.replay() + try { + val replayer = new ReplayListenerBus() + replayer.addListener(eventMonster) + replayer.replay(logData, version) + } finally { + logData.close() + } // Verify the same events are replayed in the same order assert(sc.eventLogger.isDefined) @@ -154,7 +146,9 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { */ private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", "testdir", conf) { - logger.close() + + override def start() { } + } private def getCompressionCodec(codecName: String) = { diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala deleted file mode 100644 index 72466a3aa1130..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ /dev/null @@ -1,169 +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.util - -import java.io.{File, IOException} - -import scala.io.Source - -import org.apache.hadoop.fs.Path -import org.scalatest.{BeforeAndAfter, FunSuite} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec - -/** - * Test writing files through the FileLogger. - */ -class FileLoggerSuite extends FunSuite with BeforeAndAfter { - private val fileSystem = Utils.getHadoopFileSystem("/", - SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = Seq[String]( - "org.apache.spark.io.LZFCompressionCodec", - "org.apache.spark.io.SnappyCompressionCodec" - ) - private var testDir: File = _ - private var logDirPath: Path = _ - private var logDirPathString: String = _ - - before { - testDir = Utils.createTempDir() - logDirPath = Utils.getFilePath(testDir, "test-file-logger") - logDirPathString = logDirPath.toString - } - - after { - Utils.deleteRecursively(testDir) - } - - test("Simple logging") { - testSingleFile() - } - - test ("Simple logging with compression") { - allCompressionCodecs.foreach { codec => - testSingleFile(Some(codec)) - } - } - - test("Logging multiple files") { - testMultipleFiles() - } - - test("Logging multiple files with compression") { - allCompressionCodecs.foreach { codec => - testMultipleFiles(Some(codec)) - } - } - - test("Logging when directory already exists") { - // Create the logging directory multiple times - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - - // If overwrite is not enabled, an exception should be thrown - intercept[IOException] { - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = false).start() - } - } - - - /* ----------------- * - * Actual test logic * - * ----------------- */ - - /** - * Test logging to a single file. - */ - private def testSingleFile(codecName: Option[String] = None) { - val conf = getLoggingConf(codecName) - val codec = codecName.map { c => CompressionCodec.createCodec(conf) } - val logger = - if (codecName.isDefined) { - new FileLogger(logDirPathString, conf, compress = true) - } else { - new FileLogger(logDirPathString, conf) - } - logger.start() - assert(fileSystem.exists(logDirPath)) - assert(fileSystem.getFileStatus(logDirPath).isDir) - assert(fileSystem.listStatus(logDirPath).size === 0) - - logger.newFile() - val files = fileSystem.listStatus(logDirPath) - assert(files.size === 1) - val firstFile = files.head - val firstFilePath = firstFile.getPath - - logger.log("hello") - logger.flush() - assert(readFileContent(firstFilePath, codec) === "hello") - - logger.log(" world") - logger.close() - assert(readFileContent(firstFilePath, codec) === "hello world") - } - - /** - * Test logging to multiple files. - */ - private def testMultipleFiles(codecName: Option[String] = None) { - val conf = getLoggingConf(codecName) - val codec = codecName.map { c => CompressionCodec.createCodec(conf) } - val logger = - if (codecName.isDefined) { - new FileLogger(logDirPathString, conf, compress = true) - } else { - new FileLogger(logDirPathString, conf) - } - logger.start() - logger.newFile("Jean_Valjean") - logger.logLine("Who am I?") - logger.logLine("Destiny?") - logger.newFile("John_Valjohn") - logger.logLine("One") - logger.logLine("Two three...") - logger.newFile("Wolverine") - logger.logLine("There was a time") - logger.logLine("A time when our enemies knew honor.") - logger.close() - assert(readFileContent(new Path(logDirPath, "Jean_Valjean"), codec) === "Who am I?\nDestiny?") - assert(readFileContent(new Path(logDirPath, "John_Valjohn"), codec) === "One\nTwo three...") - assert(readFileContent(new Path(logDirPath, "Wolverine"), codec) === - "There was a time\nA time when our enemies knew honor.") - } - - /** - * Read the content of the file specified by the given path. - * If a compression codec is specified, use it to read the file. - */ - private def readFileContent(logPath: Path, codec: Option[CompressionCodec] = None): String = { - val fstream = fileSystem.open(logPath) - val cstream = codec.map(_.compressedInputStream(fstream)).getOrElse(fstream) - Source.fromInputStream(cstream).getLines().mkString("\n") - } - - private def getLoggingConf(codecName: Option[String]) = { - val conf = new SparkConf - codecName.foreach { c => conf.set("spark.io.compression.codec", c) } - conf - } - -} From c25c669d95293ce7f72f2758d6737fabd0d4390a Mon Sep 17 00:00:00 2001 From: Eran Medan Date: Fri, 19 Dec 2014 18:29:36 -0800 Subject: [PATCH 085/227] change signature of example to match released code the signature of registerKryoClasses is actually of Array[Class[_]] not Seq Author: Eran Medan Closes #3747 from eranation/patch-1 and squashes the following commits: ee9885d [Eran Medan] change signature of example to match released code --- docs/tuning.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tuning.md b/docs/tuning.md index e2fdcfe6a37d9..efaac9d3d405f 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -51,7 +51,7 @@ To register your own custom classes with Kryo, use the `registerKryoClasses` met {% highlight scala %} val conf = new SparkConf().setMaster(...).setAppName(...) -conf.registerKryoClasses(Seq(classOf[MyClass1], classOf[MyClass2])) +conf.registerKryoClasses(Array(classOf[MyClass1], classOf[MyClass2])) val sc = new SparkContext(conf) {% endhighlight %} From 8d932475e6759e869c16ce6cac203a2e56558716 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Fri, 19 Dec 2014 19:19:53 -0800 Subject: [PATCH 086/227] [SPARK-3060] spark-shell.cmd doesn't accept application options in Windows OS Added equivalent module as utils.sh and modified spark-shell2.cmd to use it to parse options. Now we can use application options. ex) `bin\spark-shell.cmd --master spark://master:7077 -i path\to\script.txt` Author: Masayoshi TSUZUKI Closes #3350 from tsudukim/feature/SPARK-3060 and squashes the following commits: 4551e56 [Masayoshi TSUZUKI] Modified too long line which defines the submission options to pass findstr command. 3a11361 [Masayoshi TSUZUKI] [SPARK-3060] spark-shell.cmd doesn't accept application options in Windows OS --- bin/spark-shell2.cmd | 21 ++++++++++++++- bin/windows-utils.cmd | 59 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 bin/windows-utils.cmd diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index 2ee60b4e2a2b3..1d1a40da315eb 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -19,4 +19,23 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %* spark-shell +echo "%*" | findstr " --help -h" >nul +if %ERRORLEVEL% equ 0 ( + call :usage + exit /b 0 +) + +call %SPARK_HOME%\bin\windows-utils.cmd %* +if %ERRORLEVEL% equ 1 ( + call :usage + exit /b 1 +) + +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %SUBMISSION_OPTS% spark-shell %APPLICATION_OPTS% + +exit /b 0 + +:usage +echo "Usage: .\bin\spark-shell.cmd [options]" >&2 +%SPARK_HOME%\bin\spark-submit --help 2>&1 | findstr /V "Usage" 1>&2 +exit /b 0 diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd new file mode 100644 index 0000000000000..1082a952dac99 --- /dev/null +++ b/bin/windows-utils.cmd @@ -0,0 +1,59 @@ +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem Gather all spark-submit options into SUBMISSION_OPTS + +set SUBMISSION_OPTS= +set APPLICATION_OPTS= + +rem NOTE: If you add or remove spark-sumbmit options, +rem modify NOT ONLY this script but also SparkSubmitArgument.scala + +:OptsLoop +if "x%1"=="x" ( + goto :OptsLoopEnd +) + +SET opts="\<--master\> \<--deploy-mode\> \<--class\> \<--name\> \<--jars\> \<--py-files\> \<--files\>" +SET opts="%opts:~1,-1% \<--conf\> \<--properties-file\> \<--driver-memory\> \<--driver-java-options\>" +SET opts="%opts:~1,-1% \<--driver-library-path\> \<--driver-class-path\> \<--executor-memory\>" +SET opts="%opts:~1,-1% \<--driver-cores\> \<--total-executor-cores\> \<--executor-cores\> \<--queue\>" +SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\>" + +echo %1 | findstr %opts% >nul +if %ERRORLEVEL% equ 0 ( + if "x%2"=="x" ( + echo "%1" requires an argument. >&2 + exit /b 1 + ) + set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 %2 + shift + shift + goto :OptsLoop +) +echo %1 | findstr "\<--verbose\> \<-v\> \<--supervise\>" >nul +if %ERRORLEVEL% equ 0 ( + set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 + shift + goto :OptsLoop +) +set APPLICATION_OPTS=%APPLICATION_OPTS% %1 +shift +goto :OptsLoop + +:OptsLoopEnd +exit /b 0 From 1d648123a77bbcd9b7a34cc0d66c14fa85edfecd Mon Sep 17 00:00:00 2001 From: Kanwaljit Singh Date: Fri, 19 Dec 2014 19:25:39 -0800 Subject: [PATCH 087/227] SPARK-2641: Passing num executors to spark arguments from properties file Since we can set spark executor memory and executor cores using property file, we must also be allowed to set the executor instances. Author: Kanwaljit Singh Closes #1657 from kjsingh/branch-1.0 and squashes the following commits: d8a5a12 [Kanwaljit Singh] SPARK-2641: Fixing how spark arguments are loaded from properties file for num executors Conflicts: core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f0e9ee67f6a67..958bd03617b3c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -120,6 +120,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull + numExecutors = Option(numExecutors) + .getOrElse(defaultProperties.get("spark.executor.instances").orNull) // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { From 7cb3f54793124c527d62906c565aba2c3544e422 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Fri, 19 Dec 2014 19:32:39 -0800 Subject: [PATCH 088/227] [SPARK-4831] Do not include SPARK_CLASSPATH if empty My guess for fixing https://issues.apache.org/jira/browse/SPARK-4831. Author: Daniel Darabos Closes #3678 from darabos/patch-1 and squashes the following commits: 36e1243 [Daniel Darabos] Do not include SPARK_CLASSPATH if empty. --- bin/compute-classpath.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 685051eeed9f1..a31ea73d3ce19 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -25,7 +25,11 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)" . "$FWDIR"/bin/load-spark-env.sh -CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" +if [ -n "$SPARK_CLASSPATH" ]; then + CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" +else + CLASSPATH="$SPARK_SUBMIT_CLASSPATH" +fi # Build up classpath if [ -n "$SPARK_CONF_DIR" ]; then From 15c03e1e0efac29855f32984da7c6b0321f0e37a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 19 Dec 2014 19:36:20 -0800 Subject: [PATCH 089/227] [SPARK-4140] Document dynamic allocation Once the external shuffle service is also documented, the dynamic allocation section will link to it. Let me know if the whole dynamic allocation should be moved to its separate page; I personally think the organization might be cleaner that way. This patch builds on top of oza's work in #3689. aarondav pwendell Author: Andrew Or Author: Tsuyoshi Ozawa Closes #3731 from andrewor14/document-dynamic-allocation and squashes the following commits: 1281447 [Andrew Or] Address a few comments b9843f2 [Andrew Or] Document the configs as well 246fb44 [Andrew Or] Merge branch 'SPARK-4839' of github.com:oza/spark into document-dynamic-allocation 8c64004 [Andrew Or] Add documentation for dynamic allocation (without configs) 6827b56 [Tsuyoshi Ozawa] Fixing a documentation of spark.dynamicAllocation.enabled. 53cff58 [Tsuyoshi Ozawa] Adding a documentation about dynamic resource allocation. --- docs/configuration.md | 61 +++++++++++++++++++++++ docs/job-scheduling.md | 108 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 169 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 64aa94f622afa..2c8dea869b092 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1008,6 +1008,67 @@ Apart from these, the following properties are also available, and may be useful +#### Dynamic allocation + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.dynamicAllocation.enabledfalse + Whether to use dynamic resource allocation, which scales the number of executors registered + with this application up and down based on the workload. Note that this is currently only + available on YARN mode. For more detail, see the description + here. +

+ This requires the following configurations to be set: + spark.dynamicAllocation.minExecutors, + spark.dynamicAllocation.maxExecutors, and + spark.shuffle.service.enabled +
spark.dynamicAllocation.minExecutors(none) + Lower bound for the number of executors if dynamic allocation is enabled (required). +
spark.dynamicAllocation.maxExecutors(none) + Upper bound for the number of executors if dynamic allocation is enabled (required). +
spark.dynamicAllocation.schedulerBacklogTimeout60 + If dynamic allocation is enabled and there have been pending tasks backlogged for more than + this duration (in seconds), new executors will be requested. For more detail, see this + description. +
spark.dynamicAllocation.sustainedSchedulerBacklogTimeoutschedulerBacklogTimeout + Same as spark.dynamicAllocation.schedulerBacklogTimeout, but used only for + subsequent executor requests. For more detail, see this + description. +
spark.dynamicAllocation.executorIdleTimeout600 + If dynamic allocation is enabled and an executor has been idle for more than this duration + (in seconds), the executor will be removed. For more detail, see this + description. +
+ #### Security diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 94604f301dd46..dfbb871f01d2c 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -56,6 +56,114 @@ the same RDDs. For example, the [Shark](http://shark.cs.berkeley.edu) JDBC serve queries. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will provide another approach to share RDDs. +## Dynamic Resource Allocation + +Spark 1.2 introduces the ability to dynamically scale the set of cluster resources allocated to +your application up and down based on the workload. This means that your application may give +resources back to the cluster if they are no longer used and request them again later when there +is demand. This feature is particularly useful if multiple applications share resources in your +Spark cluster. If a subset of the resources allocated to an application becomes idle, it can be +returned to the cluster's pool of resources and acquired by other applications. In Spark, dynamic +resource allocation is performed on the granularity of the executor and can be enabled through +`spark.dynamicAllocation.enabled`. + +This feature is currently disabled by default and available only on [YARN](running-on-yarn.html). +A future release will extend this to [standalone mode](spark-standalone.html) and +[Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes). Note that although Spark on +Mesos already has a similar notion of dynamic resource sharing in fine-grained mode, enabling +dynamic allocation allows your Mesos application to take advantage of coarse-grained low-latency +scheduling while sharing cluster resources efficiently. + +### Configuration and Setup + +All configurations used by this feature live under the `spark.dynamicAllocation.*` namespace. +To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true` and +provide lower and upper bounds for the number of executors through +`spark.dynamicAllocation.minExecutors` and `spark.dynamicAllocation.maxExecutors`. Other relevant +configurations are described on the [configurations page](configuration.html#dynamic-allocation) +and in the subsequent sections in detail. + +Additionally, your application must use an external shuffle service. The purpose of the service is +to preserve the shuffle files written by executors so the executors can be safely removed (more +detail described [below](job-scheduling.html#graceful-decommission-of-executors)). To enable +this service, set `spark.shuffle.service.enabled` to `true`. In YARN, this external shuffle service +is implemented in `org.apache.spark.yarn.network.YarnShuffleService` that runs in each `NodeManager` +in your cluster. To start this service, follow these steps: + +1. Build Spark with the [YARN profile](building-spark.html). Skip this step if you are using a +pre-packaged distribution. +2. Locate the `spark--yarn-shuffle.jar`. This should be under +`$SPARK_HOME/network/yarn/target/scala-` if you are building Spark yourself, and under +`lib` if you are using a distribution. +2. Add this jar to the classpath of all `NodeManager`s in your cluster. +3. In the `yarn-site.xml` on each node, add `spark_shuffle` to `yarn.nodemanager.aux-services`, +then set `yarn.nodemanager.aux-services.spark_shuffle.class` to +`org.apache.spark.yarn.network.YarnShuffleService`. Additionally, set all relevant +`spark.shuffle.service.*` [configurations](configuration.html). +4. Restart all `NodeManager`s in your cluster. + +### Resource Allocation Policy + +At a high level, Spark should relinquish executors when they are no longer used and acquire +executors when they are needed. Since there is no definitive way to predict whether an executor +that is about to be removed will run a task in the near future, or whether a new executor that is +about to be added will actually be idle, we need a set of heuristics to determine when to remove +and request executors. + +#### Request Policy + +A Spark application with dynamic allocation enabled requests additional executors when it has +pending tasks waiting to be scheduled. This condition necessarily implies that the existing set +of executors is insufficient to simultaneously saturate all tasks that have been submitted but +not yet finished. + +Spark requests executors in rounds. The actual request is triggered when there have been pending +tasks for `spark.dynamicAllocation.schedulerBacklogTimeout` seconds, and then triggered again +every `spark.dynamicAllocation.sustainedSchedulerBacklogTimeout` seconds thereafter if the queue +of pending tasks persists. Additionally, the number of executors requested in each round increases +exponentially from the previous round. For instance, an application will add 1 executor in the +first round, and then 2, 4, 8 and so on executors in the subsequent rounds. + +The motivation for an exponential increase policy is twofold. First, an application should request +executors cautiously in the beginning in case it turns out that only a few additional executors is +sufficient. This echoes the justification for TCP slow start. Second, the application should be +able to ramp up its resource usage in a timely manner in case it turns out that many executors are +actually needed. + +#### Remove Policy + +The policy for removing executors is much simpler. A Spark application removes an executor when +it has been idle for more than `spark.dynamicAllocation.executorIdleTimeout` seconds. Note that, +under most circumstances, this condition is mutually exclusive with the request condition, in that +an executor should not be idle if there are still pending tasks to be scheduled. + +### Graceful Decommission of Executors + +Before dynamic allocation, a Spark executor exits either on failure or when the associated +application has also exited. In both scenarios, all state associated with the executor is no +longer needed and can be safely discarded. With dynamic allocation, however, the application +is still running when an executor is explicitly removed. If the application attempts to access +state stored in or written by the executor, it will have to perform a recompute the state. Thus, +Spark needs a mechanism to decommission an executor gracefully by preserving its state before +removing it. + +This requirement is especially important for shuffles. During a shuffle, the Spark executor first +writes its own map outputs locally to disk, and then acts as the server for those files when other +executors attempt to fetch them. In the event of stragglers, which are tasks that run for much +longer than their peers, dynamic allocation may remove an executor before the shuffle completes, +in which case the shuffle files written by that executor must be recomputed unnecessarily. + +The solution for preserving shuffle files is to use an external shuffle service, also introduced +in Spark 1.2. This service refers to a long-running process that runs on each node of your cluster +independently of your Spark applications and their executors. If the service is enabled, Spark +executors will fetch shuffle files from the service instead of from each other. This means any +shuffle state written by an executor may continue to be served beyond the executor's lifetime. + +In addition to writing shuffle files, executors also cache data either on disk or in memory. +When an executor is removed, however, all cached data will no longer be accessible. There is +currently not yet a solution for this in Spark 1.2. In future releases, the cached data may be +preserved through an off-heap storage similar in spirit to how shuffle files are preserved through +the external shuffle service. # Scheduling Within an Application From a764960b3b6d842eef7fa4777c8fa99d3f60fa1e Mon Sep 17 00:00:00 2001 From: huangzhaowei Date: Fri, 19 Dec 2014 23:32:56 -0800 Subject: [PATCH 090/227] [Minor] Build Failed: value defaultProperties not found Mvn Build Failed: value defaultProperties not found .Maybe related to this pr: https://github.com/apache/spark/commit/1d648123a77bbcd9b7a34cc0d66c14fa85edfecd andrewor14 can you look at this problem? Author: huangzhaowei Closes #3749 from SaintBacchus/Mvn-Build-Fail and squashes the following commits: 8e2917c [huangzhaowei] Build Failed: value defaultProperties not found --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 958bd03617b3c..607b4df73db99 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -121,7 +121,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull numExecutors = Option(numExecutors) - .getOrElse(defaultProperties.get("spark.executor.instances").orNull) + .getOrElse(sparkProperties.get("spark.executor.instances").orNull) // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { From c6a3c0d5052e5bf6f981e5f91e05cba38b707237 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 21 Dec 2014 13:16:57 -0800 Subject: [PATCH 091/227] SPARK-4910 [CORE] build failed (use of FileStatus.isFile in Hadoop 1.x) Fix small Hadoop 1 compile error from SPARK-2261. In Hadoop 1.x, all we have is FileStatus.isDir, so these "is file" assertions are changed to "is not a dir". This is how similar checks are done so far in the code base. Author: Sean Owen Closes #3754 from srowen/SPARK-4910 and squashes the following commits: 52c5e4e [Sean Owen] Fix small Hadoop 1 compile error from SPARK-2261 --- .../apache/spark/scheduler/EventLoggingListenerSuite.scala | 4 ++-- .../org/apache/spark/scheduler/ReplayListenerSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 5909811c2034f..1de7e130039a5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -65,11 +65,11 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) assert(fileSystem.exists(logPath)) val logStatus = fileSystem.getFileStatus(logPath) - assert(logStatus.isFile) + assert(!logStatus.isDir) // Verify log is renamed after stop() eventLogger.stop() - assert(fileSystem.getFileStatus(new Path(eventLogger.logPath)).isFile()) + assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDir) } test("Basic event logging") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 7e635cb061441..7e360cc6082ec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -112,7 +112,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { val applications = fileSystem.listStatus(logDirPath) assert(applications != null && applications.size > 0) val eventLog = applications.sortBy(_.getModificationTime).last - assert(eventLog.isFile) + assert(!eventLog.isDir) // Replay events val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) From 6ee6aa70b7d52408cc66bd1434cbeae3212e3f01 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 21 Dec 2014 22:10:19 -0800 Subject: [PATCH 092/227] [SPARK-2075][Core] Make the compiler generate same bytes code for Hadoop 1.+ and Hadoop 2.+ `NullWritable` is a `Comparable` rather than `Comparable[NullWritable]` in Hadoop 1.+, so the compiler cannot find an implicit Ordering for it. It will generate different anonymous classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+. Therefore, here we provide an Ordering for NullWritable so that the compiler will generate same codes. I used the following commands to confirm the generated byte codes are some. ``` mvn -Dhadoop.version=1.2.1 -DskipTests clean package -pl core -am javap -private -c -classpath core/target/scala-2.10/classes org.apache.spark.rdd.RDD > ~/hadoop1.txt mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package -pl core -am javap -private -c -classpath core/target/scala-2.10/classes org.apache.spark.rdd.RDD > ~/hadoop2.txt diff ~/hadoop1.txt ~/hadoop2.txt ``` However, the compiler will generate different codes for the classes which call methods of `JobContext/TaskAttemptContext`. `JobContext/TaskAttemptContext` is a class in Hadoop 1.+, and calling its method will use `invokevirtual`, while it's an interface in Hadoop 2.+, and will use `invokeinterface`. To fix it, we can use reflection to call `JobContext/TaskAttemptContext.getConfiguration`. Author: zsxwing Closes #3740 from zsxwing/SPARK-2075 and squashes the following commits: 39d9df2 [zsxwing] Fix the code style e4ad8b5 [zsxwing] Use null for the implicit Ordering 734bac9 [zsxwing] Explicitly set the implicit parameters ca03559 [zsxwing] Use reflection to access JobContext/TaskAttemptContext.getConfiguration fa40db0 [zsxwing] Add an Ordering for NullWritable to make the compiler generate same byte codes for RDD --- .../apache/spark/deploy/SparkHadoopUtil.scala | 12 +++++++++++ .../input/FixedLengthBinaryInputFormat.scala | 3 ++- .../input/FixedLengthBinaryRecordReader.scala | 3 ++- .../spark/input/PortableDataStream.scala | 4 +++- .../input/WholeTextFileRecordReader.scala | 4 +++- .../main/scala/org/apache/spark/rdd/RDD.scala | 21 +++++++++++++++++-- 6 files changed, 41 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 60ee115e393ce..57f9faf5ddd1d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation @@ -183,6 +184,17 @@ class SparkHadoopUtil extends Logging { Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") statisticsDataClass.getDeclaredMethod(methodName) } + + /** + * Using reflection to get the Configuration from JobContext/TaskAttemptContext. If we directly + * call `JobContext/TaskAttemptContext.getConfiguration`, it will generate different byte codes + * for Hadoop 1.+ and Hadoop 2.+ because JobContext/TaskAttemptContext is class in Hadoop 1.+ + * while it's interface in Hadoop 2.+. + */ + def getConfigurationFromJobContext(context: JobContext): Configuration = { + val method = context.getClass.getMethod("getConfiguration") + method.invoke(context).asInstanceOf[Configuration] + } } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 89b29af2000c8..c219d21fbefa9 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} +import org.apache.spark.deploy.SparkHadoopUtil /** * Custom Input Format for reading and splitting flat binary files that contain records, @@ -33,7 +34,7 @@ private[spark] object FixedLengthBinaryInputFormat { /** Retrieves the record length property from a Hadoop configuration */ def getRecordLength(context: JobContext): Int = { - context.getConfiguration.get(RECORD_LENGTH_PROPERTY).toInt + SparkHadoopUtil.get.getConfigurationFromJobContext(context).get(RECORD_LENGTH_PROPERTY).toInt } } diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index 36a1e5d475f46..67a96925da019 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.spark.deploy.SparkHadoopUtil /** * FixedLengthBinaryRecordReader is returned by FixedLengthBinaryInputFormat. @@ -82,7 +83,7 @@ private[spark] class FixedLengthBinaryRecordReader // the actual file we will be reading from val file = fileSplit.getPath // job configuration - val job = context.getConfiguration + val job = SparkHadoopUtil.get.getConfigurationFromJobContext(context) // check compression val codec = new CompressionCodecFactory(job).getCodec(file) if (codec != null) { diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 457472547fcbb..593a62b3e3b32 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAt import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat, CombineFileRecordReader, CombineFileSplit} import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil /** * A general format for reading whole files in as streams, byte arrays, @@ -145,7 +146,8 @@ class PortableDataStream( private val confBytes = { val baos = new ByteArrayOutputStream() - context.getConfiguration.write(new DataOutputStream(baos)) + SparkHadoopUtil.get.getConfigurationFromJobContext(context). + write(new DataOutputStream(baos)) baos.toByteArray } diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala index 1b1131b9b8831..31bde8a78f3c6 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, CombineFileRecordReader} import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.spark.deploy.SparkHadoopUtil /** @@ -51,7 +52,8 @@ private[spark] class WholeTextFileRecordReader( extends RecordReader[String, String] with Configurable { private[this] val path = split.getPath(index) - private[this] val fs = path.getFileSystem(context.getConfiguration) + private[this] val fs = path.getFileSystem( + SparkHadoopUtil.get.getConfigurationFromJobContext(context)) // True means the current file has been processed, then skip it. private[this] var processed = false 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 214f22bc5b603..a94206963b52f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1174,7 +1174,20 @@ abstract class RDD[T: ClassTag]( * Save this RDD as a text file, using string representations of elements. */ def saveAsTextFile(path: String) { - this.map(x => (NullWritable.get(), new Text(x.toString))) + // https://issues.apache.org/jira/browse/SPARK-2075 + // + // NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit + // Ordering for it and will use the default `null`. However, it's a `Comparable[NullWritable]` + // in Hadoop 2.+, so the compiler will call the implicit `Ordering.ordered` method to create an + // Ordering for `NullWritable`. That's why the compiler will generate different anonymous + // classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+. + // + // Therefore, here we provide an explicit Ordering `null` to make sure the compiler generate + // same bytecodes for `saveAsTextFile`. + val nullWritableClassTag = implicitly[ClassTag[NullWritable]] + val textClassTag = implicitly[ClassTag[Text]] + val r = this.map(x => (NullWritable.get(), new Text(x.toString))) + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) } @@ -1182,7 +1195,11 @@ abstract class RDD[T: ClassTag]( * Save this RDD as a compressed text file, using string representations of elements. */ def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { - this.map(x => (NullWritable.get(), new Text(x.toString))) + // https://issues.apache.org/jira/browse/SPARK-2075 + val nullWritableClassTag = implicitly[ClassTag[NullWritable]] + val textClassTag = implicitly[ClassTag[Text]] + val r = this.map(x => (NullWritable.get(), new Text(x.toString))) + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) } From 93b2f3a8826e189f6398c9b30bc00de205a3c64a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 22 Dec 2014 11:20:00 -0800 Subject: [PATCH 093/227] [SPARK-4918][Core] Reuse Text in saveAsTextFile Reuse Text in saveAsTextFile to reduce GC. /cc rxin Author: zsxwing Closes #3762 from zsxwing/SPARK-4918 and squashes the following commits: 59f03eb [zsxwing] Reuse Text in saveAsTextFile --- .../main/scala/org/apache/spark/rdd/RDD.scala | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 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 a94206963b52f..f47c2d1fcdcc7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1186,7 +1186,13 @@ abstract class RDD[T: ClassTag]( // same bytecodes for `saveAsTextFile`. val nullWritableClassTag = implicitly[ClassTag[NullWritable]] val textClassTag = implicitly[ClassTag[Text]] - val r = this.map(x => (NullWritable.get(), new Text(x.toString))) + val r = this.mapPartitions { iter => + val text = new Text() + iter.map { x => + text.set(x.toString) + (NullWritable.get(), text) + } + } RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) } @@ -1198,7 +1204,13 @@ abstract class RDD[T: ClassTag]( // https://issues.apache.org/jira/browse/SPARK-2075 val nullWritableClassTag = implicitly[ClassTag[NullWritable]] val textClassTag = implicitly[ClassTag[Text]] - val r = this.map(x => (NullWritable.get(), new Text(x.toString))) + val r = this.mapPartitions { iter => + val text = new Text() + iter.map { x => + text.set(x.toString) + (NullWritable.get(), text) + } + } RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) } From 96606f69b7cd88edea5d4b6427f598a91ccb4c33 Mon Sep 17 00:00:00 2001 From: Tsuyoshi Ozawa Date: Mon, 22 Dec 2014 11:28:05 -0800 Subject: [PATCH 094/227] [SPARK-4915][YARN] Fix classname to be specified for external shuffle service. Author: Tsuyoshi Ozawa Closes #3757 from oza/SPARK-4915 and squashes the following commits: 3b0d6d6 [Tsuyoshi Ozawa] Fix classname to be specified for external shuffle service. --- docs/job-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index dfbb871f01d2c..a5425eb3557b2 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -98,7 +98,7 @@ pre-packaged distribution. 2. Add this jar to the classpath of all `NodeManager`s in your cluster. 3. In the `yarn-site.xml` on each node, add `spark_shuffle` to `yarn.nodemanager.aux-services`, then set `yarn.nodemanager.aux-services.spark_shuffle.class` to -`org.apache.spark.yarn.network.YarnShuffleService`. Additionally, set all relevant +`org.apache.spark.network.yarn.YarnShuffleService`. Additionally, set all relevant `spark.shuffle.service.*` [configurations](configuration.html). 4. Restart all `NodeManager`s in your cluster. From 39272c8cdb3dc8ffbce44f6ee8055217d041e165 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 22 Dec 2014 11:36:49 -0800 Subject: [PATCH 095/227] [SPARK-4870] Add spark version to driver log Author: Zhang, Liye Closes #3717 from liyezhang556520/version2Log and squashes the following commits: ccd30d7 [Zhang, Liye] delete log in sparkConf 330f70c [Zhang, Liye] move the log from SaprkConf to SparkContext 96dc115 [Zhang, Liye] remove curly brace e833330 [Zhang, Liye] add spark version to driver log --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b1c6a90d182f6..57bc3d4e4ae36 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -172,6 +172,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = this(master, appName, sparkHome, jars, Map(), Map()) + // log out Spark Version in Spark driver log + logInfo(s"Running Spark version $SPARK_VERSION") + private[spark] val conf = config.clone() conf.validateSettings() From 8773705fd4ab167bb06bee7f274b337560e3c293 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 22 Dec 2014 12:11:36 -0800 Subject: [PATCH 096/227] [SPARK-4883][Shuffle] Add a name to the directoryCleaner thread Author: zsxwing Closes #3734 from zsxwing/SPARK-4883 and squashes the following commits: e6f2b61 [zsxwing] Fix the name cc74727 [zsxwing] Add a name to the directoryCleaner thread --- .../network/shuffle/ExternalShuffleBlockManager.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java index dfe0ba0595090..93e6fdd7161fa 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java @@ -37,6 +37,7 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; /** @@ -49,7 +50,7 @@ * the Executor's memory, unlike the IndexShuffleBlockManager. */ public class ExternalShuffleBlockManager { - private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class); + private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class); // Map containing all registered executors' metadata. private final ConcurrentMap executors; @@ -60,8 +61,9 @@ public class ExternalShuffleBlockManager { private final TransportConf conf; public ExternalShuffleBlockManager(TransportConf conf) { - // TODO: Give this thread a name. - this(conf, Executors.newSingleThreadExecutor()); + this(conf, Executors.newSingleThreadExecutor( + // Add `spark` prefix because it will run in NM in Yarn mode. + NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); } // Allows tests to have more control over when directories are cleaned up. From 1d9788e42ec96407bd63da8f2a2abebc7253c671 Mon Sep 17 00:00:00 2001 From: carlmartin Date: Mon, 22 Dec 2014 12:13:53 -0800 Subject: [PATCH 097/227] [Minor] Improve some code in BroadcastTest for short Using val arr1 = (0 until num).toArray instead of val arr1 = new Array[Int](num) for (i <- 0 until arr1.length) { arr1(i) = i } for short. Author: carlmartin Closes #3750 from SaintBacchus/BroadcastTest and squashes the following commits: 43adb70 [carlmartin] Improve some code in BroadcastTest for short --- .../main/scala/org/apache/spark/examples/BroadcastTest.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 973049b95a7bd..adecd934358c4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -38,10 +38,7 @@ object BroadcastTest { val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 - val arr1 = new Array[Int](num) - for (i <- 0 until arr1.length) { - arr1(i) = i - } + val arr1 = (0 until num).toArray for (i <- 0 until 3) { println("Iteration " + i) From fb8e85e80e50904e1e93daf30dcadef62c3b7ca1 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 22 Dec 2014 12:19:23 -0800 Subject: [PATCH 098/227] [SPARK-4733] Add missing prameter comments in ShuffleDependency Add missing Javadoc comments in ShuffleDependency. Author: Takeshi Yamamuro Closes #3594 from maropu/DependencyJavadocFix and squashes the following commits: 32129b4 [Takeshi Yamamuro] Fix comments in @aggregator and @mapSideCombine 303c75d [Takeshi Yamamuro] [SPARK-4733] Add missing prameter comments in ShuffleDependency --- core/src/main/scala/org/apache/spark/Dependency.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index ab2594cfc02eb..9a7cd4523e5ab 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -60,6 +60,9 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will * be used. + * @param keyOrdering key ordering for RDD's shuffles + * @param aggregator map/reduce-side aggregator for RDD's shuffle + * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine) */ @DeveloperApi class ShuffleDependency[K, V, C]( From d62da642ace17b37283eab64149545723c8474a7 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 22 Dec 2014 12:23:43 -0800 Subject: [PATCH 099/227] SPARK-4447. Remove layers of abstraction in YARN code no longer needed after dropping yarn-alpha Author: Sandy Ryza Closes #3652 from sryza/sandy-spark-4447 and squashes the following commits: 2791158 [Sandy Ryza] Review feedback c23507b [Sandy Ryza] Strip margin from client arguments help string 18be7ba [Sandy Ryza] SPARK-4447 --- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../spark/deploy/yarn/ClientArguments.scala | 2 +- .../deploy/yarn/YarnAllocationHandler.scala | 213 ------------------ .../spark/deploy/yarn/YarnAllocator.scala | 201 ++++++++++++++--- .../spark/deploy/yarn/YarnRMClient.scala | 78 ++++++- .../spark/deploy/yarn/YarnRMClientImpl.scala | 110 --------- 6 files changed, 234 insertions(+), 372 deletions(-) delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index dc7a078446324..b2e45435c4abb 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -511,7 +511,7 @@ object ApplicationMaster extends Logging { SignalLogger.register(log) val amArgs = new ApplicationMasterArguments(args) SparkHadoopUtil.get.runAsSparkUser { () => - master = new ApplicationMaster(amArgs, new YarnRMClientImpl(amArgs)) + master = new ApplicationMaster(amArgs, new YarnRMClient(amArgs)) System.exit(master.run()) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index c439969510fc9..7305249f80e83 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -197,6 +197,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | to work with. | --files files Comma separated list of files to be distributed with the job. | --archives archives Comma separated list of archives to be distributed with the job. - """ + """.stripMargin } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala deleted file mode 100644 index 2bbf5d7db8668..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ /dev/null @@ -1,213 +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.deploy.yarn - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap} - -import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.util.Records - -/** - * Acquires resources for executors from a ResourceManager and launches executors in new containers. - */ -private[yarn] class YarnAllocationHandler( - conf: Configuration, - sparkConf: SparkConf, - amClient: AMRMClient[ContainerRequest], - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]], - securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { - - override protected def releaseContainer(container: Container) = { - amClient.releaseAssignedContainer(container.getId()) - } - - // pending isn't used on stable as the AMRMClient handles incremental asks - override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { - addResourceRequests(count) - - // We have already set the container request. Poll the ResourceManager for a response. - // This doubles as a heartbeat if there are no pending container requests. - val progressIndicator = 0.1f - new StableAllocateResponse(amClient.allocate(progressIndicator)) - } - - private def createRackResourceRequests( - hostContainers: ArrayBuffer[ContainerRequest] - ): ArrayBuffer[ContainerRequest] = { - // Generate modified racks and new set of hosts under it before issuing requests. - val rackToCounts = new HashMap[String, Int]() - - for (container <- hostContainers) { - val candidateHost = container.getNodes.last - assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - var count = rackToCounts.getOrElse(rack, 0) - count += 1 - rackToCounts.put(rack, count) - } - } - - val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) - for ((rack, count) <- rackToCounts) { - requestedContainers ++= createResourceRequests( - AllocationType.RACK, - rack, - count, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - - requestedContainers - } - - private def addResourceRequests(numExecutors: Int) { - val containerRequests: List[ContainerRequest] = - if (numExecutors <= 0) { - logDebug("numExecutors: " + numExecutors) - List() - } else if (preferredHostToCount.isEmpty) { - logDebug("host preferences is empty") - createResourceRequests( - AllocationType.ANY, - resource = null, - numExecutors, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY).toList - } else { - // Request for all hosts in preferred nodes and for numExecutors - - // candidates.size, request by default allocation policy. - val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) - for ((candidateHost, candidateCount) <- preferredHostToCount) { - val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) - - if (requiredCount > 0) { - hostContainerRequests ++= createResourceRequests( - AllocationType.HOST, - candidateHost, - requiredCount, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - } - val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( - hostContainerRequests).toList - - val anyContainerRequests = createResourceRequests( - AllocationType.ANY, - resource = null, - numExecutors, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - - val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( - hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) - - containerRequestBuffer ++= hostContainerRequests - containerRequestBuffer ++= rackContainerRequests - containerRequestBuffer ++= anyContainerRequests - containerRequestBuffer.toList - } - - for (request <- containerRequests) { - amClient.addContainerRequest(request) - } - - for (request <- containerRequests) { - val nodes = request.getNodes - var hostStr = if (nodes == null || nodes.isEmpty) { - "Any" - } else { - nodes.last - } - logInfo("Container request (host: %s, priority: %s, capability: %s".format( - hostStr, - request.getPriority().getPriority, - request.getCapability)) - } - } - - private def createResourceRequests( - requestType: AllocationType.AllocationType, - resource: String, - numExecutors: Int, - priority: Int - ): ArrayBuffer[ContainerRequest] = { - - // If hostname is specified, then we need at least two requests - node local and rack local. - // There must be a third request, which is ANY. That will be specially handled. - requestType match { - case AllocationType.HOST => { - assert(YarnSparkHadoopUtil.ANY_HOST != resource) - val hostname = resource - val nodeLocal = constructContainerRequests( - Array(hostname), - racks = null, - numExecutors, - priority) - - // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. - YarnSparkHadoopUtil.populateRackInfo(conf, hostname) - nodeLocal - } - case AllocationType.RACK => { - val rack = resource - constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) - } - case AllocationType.ANY => constructContainerRequests( - hosts = null, racks = null, numExecutors, priority) - case _ => throw new IllegalArgumentException( - "Unexpected/unsupported request type: " + requestType) - } - } - - private def constructContainerRequests( - hosts: Array[String], - racks: Array[String], - numExecutors: Int, - priority: Int - ): ArrayBuffer[ContainerRequest] = { - - val memoryRequest = executorMemory + memoryOverhead - val resource = Resource.newInstance(memoryRequest, executorCores) - - val prioritySetting = Records.newRecord(classOf[Priority]) - prioritySetting.setPriority(priority) - - val requests = new ArrayBuffer[ContainerRequest]() - for (i <- 0 until numExecutors) { - requests += new ContainerRequest(resource, hosts, racks, prioritySetting) - } - requests - } - - private class StableAllocateResponse(response: AllocateResponse) extends YarnAllocateResponse { - override def getAllocatedContainers() = response.getAllocatedContainers() - override def getAvailableResources() = response.getAvailableResources() - override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index b32e15738f28b..de65ef23ad1ce 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.util.{List => JList} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern @@ -25,17 +24,20 @@ import java.util.regex.Pattern import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import com.google.common.util.concurrent.ThreadFactoryBuilder + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ - object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value @@ -52,12 +54,12 @@ object AllocationType extends Enumeration { // more info on how we are requesting for containers. /** - * Common code for the Yarn container allocator. Contains all the version-agnostic code to - * manage container allocation for a running Spark application. + * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ -private[yarn] abstract class YarnAllocator( +private[yarn] class YarnAllocator( conf: Configuration, sparkConf: SparkConf, + amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], @@ -67,7 +69,7 @@ private[yarn] abstract class YarnAllocator( import YarnAllocator._ // These three are locked on allocatedHostToContainersMap. Complementary data structures - // allocatedHostToContainersMap : containers which are running : host, Set + // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]() @@ -161,8 +163,6 @@ private[yarn] abstract class YarnAllocator( def allocateResources(): Unit = synchronized { val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() - // this is needed by alpha, do it here since we add numPending right after this - val executorsPending = numPendingAllocate.get() if (missing > 0) { val totalExecutorMemory = executorMemory + memoryOverhead numPendingAllocate.addAndGet(missing) @@ -172,7 +172,7 @@ private[yarn] abstract class YarnAllocator( logDebug("Empty allocation request ...") } - val allocateResponse = allocateContainers(missing, executorsPending) + val allocateResponse = allocateContainers(missing) val allocatedContainers = allocateResponse.getAllocatedContainers() if (allocatedContainers.size > 0) { @@ -368,7 +368,7 @@ private[yarn] abstract class YarnAllocator( val containerId = completedContainer.getContainerId if (releasedContainers.containsKey(containerId)) { - // YarnAllocationHandler already marked the container for release, so remove it from + // Already marked the container for release, so remove it from // `releasedContainers`. releasedContainers.remove(containerId) } else { @@ -441,20 +441,16 @@ private[yarn] abstract class YarnAllocator( } } - protected def allocatedContainersOnHost(host: String): Int = { - var retval = 0 + private def allocatedContainersOnHost(host: String): Int = { allocatedHostToContainersMap.synchronized { - retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + allocatedHostToContainersMap.getOrElse(host, Set()).size } - retval } - protected def allocatedContainersOnRack(rack: String): Int = { - var retval = 0 + private def allocatedContainersOnRack(rack: String): Int = { allocatedHostToContainersMap.synchronized { - retval = allocatedRackCount.getOrElse(rack, 0) + allocatedRackCount.getOrElse(rack, 0) } - retval } private def isResourceConstraintSatisfied(container: Container): Boolean = { @@ -464,9 +460,8 @@ private[yarn] abstract class YarnAllocator( // A simple method to copy the split info map. private def generateNodeToWeight( conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]] - ): (Map[String, Int], Map[String, Int]) = { - + input: collection.Map[String, collection.Set[SplitInfo]]) + : (Map[String, Int], Map[String, Int]) = { if (input == null) { return (Map[String, Int](), Map[String, Int]()) } @@ -488,9 +483,9 @@ private[yarn] abstract class YarnAllocator( (hostToCount.toMap, rackToCount.toMap) } - private def internalReleaseContainer(container: Container) = { + private def internalReleaseContainer(container: Container): Unit = { releasedContainers.put(container.getId(), true) - releaseContainer(container) + amClient.releaseAssignedContainer(container.getId()) } /** @@ -498,26 +493,158 @@ private[yarn] abstract class YarnAllocator( * * @param count Number of containers to allocate. * If zero, should still contact RM (as a heartbeat). - * @param pending Number of containers pending allocate. Only used on alpha. * @return Response to the allocation request. */ - protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse + private def allocateContainers(count: Int): AllocateResponse = { + addResourceRequests(count) + + // We have already set the container request. Poll the ResourceManager for a response. + // This doubles as a heartbeat if there are no pending container requests. + val progressIndicator = 0.1f + amClient.allocate(progressIndicator) + } - /** Called to release a previously allocated container. */ - protected def releaseContainer(container: Container): Unit + private def createRackResourceRequests(hostContainers: ArrayBuffer[ContainerRequest]) + : ArrayBuffer[ContainerRequest] = { + // Generate modified racks and new set of hosts under it before issuing requests. + val rackToCounts = new HashMap[String, Int]() - /** - * Defines the interface for an allocate response from the RM. This is needed since the alpha - * and stable interfaces differ here in ways that cannot be fixed using other routes. - */ - protected trait YarnAllocateResponse { + for (container <- hostContainers) { + val candidateHost = container.getNodes.last + assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += 1 + rackToCounts.put(rack, count) + } + } + + val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts) { + requestedContainers ++= createResourceRequests( + AllocationType.RACK, + rack, + count, + RM_REQUEST_PRIORITY) + } + + requestedContainers + } + + private def addResourceRequests(numExecutors: Int): Unit = { + val containerRequests: List[ContainerRequest] = + if (numExecutors <= 0) { + logDebug("numExecutors: " + numExecutors) + List() + } else if (preferredHostToCount.isEmpty) { + logDebug("host preferences is empty") + createResourceRequests( + AllocationType.ANY, + resource = null, + numExecutors, + RM_REQUEST_PRIORITY).toList + } else { + // Request for all hosts in preferred nodes and for numExecutors - + // candidates.size, request by default allocation policy. + val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) + for ((candidateHost, candidateCount) <- preferredHostToCount) { + val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) + + if (requiredCount > 0) { + hostContainerRequests ++= createResourceRequests( + AllocationType.HOST, + candidateHost, + requiredCount, + RM_REQUEST_PRIORITY) + } + } + val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( + hostContainerRequests).toList + + val anyContainerRequests = createResourceRequests( + AllocationType.ANY, + resource = null, + numExecutors, + RM_REQUEST_PRIORITY) + + val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( + hostContainerRequests.size + rackContainerRequests.size + anyContainerRequests.size) + + containerRequestBuffer ++= hostContainerRequests + containerRequestBuffer ++= rackContainerRequests + containerRequestBuffer ++= anyContainerRequests + containerRequestBuffer.toList + } - def getAllocatedContainers(): JList[Container] + for (request <- containerRequests) { + amClient.addContainerRequest(request) + } - def getAvailableResources(): Resource + for (request <- containerRequests) { + val nodes = request.getNodes + val hostStr = if (nodes == null || nodes.isEmpty) { + "Any" + } else { + nodes.last + } + logInfo("Container request (host: %s, priority: %s, capability: %s".format( + hostStr, + request.getPriority().getPriority, + request.getCapability)) + } + } - def getCompletedContainersStatuses(): JList[ContainerStatus] + private def createResourceRequests( + requestType: AllocationType.AllocationType, + resource: String, + numExecutors: Int, + priority: Int): ArrayBuffer[ContainerRequest] = { + // If hostname is specified, then we need at least two requests - node local and rack local. + // There must be a third request, which is ANY. That will be specially handled. + requestType match { + case AllocationType.HOST => { + assert(YarnSparkHadoopUtil.ANY_HOST != resource) + val hostname = resource + val nodeLocal = constructContainerRequests( + Array(hostname), + racks = null, + numExecutors, + priority) + + // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. + YarnSparkHadoopUtil.populateRackInfo(conf, hostname) + nodeLocal + } + case AllocationType.RACK => { + val rack = resource + constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) + } + case AllocationType.ANY => constructContainerRequests( + hosts = null, racks = null, numExecutors, priority) + case _ => throw new IllegalArgumentException( + "Unexpected/unsupported request type: " + requestType) + } + } + private def constructContainerRequests( + hosts: Array[String], + racks: Array[String], + numExecutors: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { + val memoryRequest = executorMemory + memoryOverhead + val resource = Resource.newInstance(memoryRequest, executorCores) + + val prioritySetting = Records.newRecord(classOf[Priority]) + prioritySetting.setPriority(priority) + + val requests = new ArrayBuffer[ContainerRequest]() + for (i <- 0 until numExecutors) { + requests += new ContainerRequest(resource, hosts, racks, prioritySetting) + } + requests } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 2510b9c9cef68..bf4e15908bb46 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -17,19 +17,33 @@ package org.apache.spark.deploy.yarn +import java.util.{List => JList} + +import scala.collection.JavaConversions._ import scala.collection.{Map, Set} +import scala.util.Try -import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils +import org.apache.hadoop.yarn.webapp.util.WebAppUtils -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.util.Utils /** - * Interface that defines a Yarn RM client. Abstracts away Yarn version-specific functionality that - * is used by Spark's AM. + * Handles registering and unregistering the application with the YARN ResourceManager. */ -trait YarnRMClient { +private[spark] class YarnRMClient(args: ApplicationMasterArguments) extends Logging { + + private var amClient: AMRMClient[ContainerRequest] = _ + private var uiHistoryAddress: String = _ + private var registered: Boolean = false /** * Registers the application master with the RM. @@ -46,7 +60,21 @@ trait YarnRMClient { preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, uiHistoryAddress: String, - securityMgr: SecurityManager): YarnAllocator + securityMgr: SecurityManager + ): YarnAllocator = { + amClient = AMRMClient.createAMRMClient() + amClient.init(conf) + amClient.start() + this.uiHistoryAddress = uiHistoryAddress + + logInfo("Registering the ApplicationMaster") + synchronized { + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + registered = true + } + new YarnAllocator(conf, sparkConf, amClient, getAttemptId(), args, + preferredNodeLocations, securityMgr) + } /** * Unregister the AM. Guaranteed to only be called once. @@ -54,15 +82,45 @@ trait YarnRMClient { * @param status The final status of the AM. * @param diagnostics Diagnostics message to include in the final status. */ - def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit + def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit = synchronized { + if (registered) { + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + } + } /** Returns the attempt ID. */ - def getAttemptId(): ApplicationAttemptId + def getAttemptId(): ApplicationAttemptId = { + val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + containerId.getApplicationAttemptId() + } /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ - def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] + def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = { + // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, + // so not all stable releases have it. + val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) + .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") + + // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. + try { + val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", + classOf[Configuration]) + val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] + val hosts = proxies.map { proxy => proxy.split(":")(0) } + val uriBases = proxies.map { proxy => prefix + proxy + proxyBase } + Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) + } catch { + case e: NoSuchMethodException => + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val uriBase = prefix + proxy + proxyBase + Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) + } + } /** Returns the maximum number of attempts to register the AM. */ - def getMaxRegAttempts(conf: YarnConfiguration): Int + def getMaxRegAttempts(conf: YarnConfiguration): Int = + conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala deleted file mode 100644 index 8d4b96ed79933..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ /dev/null @@ -1,110 +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.deploy.yarn - -import java.util.{List => JList} - -import scala.collection.{Map, Set} -import scala.collection.JavaConversions._ -import scala.util._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.ConverterUtils -import org.apache.hadoop.yarn.webapp.util.WebAppUtils - -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.util.Utils - - -/** - * YarnRMClient implementation for the Yarn stable API. - */ -private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMClient with Logging { - - private var amClient: AMRMClient[ContainerRequest] = _ - private var uiHistoryAddress: String = _ - private var registered: Boolean = false - - override def register( - conf: YarnConfiguration, - sparkConf: SparkConf, - preferredNodeLocations: Map[String, Set[SplitInfo]], - uiAddress: String, - uiHistoryAddress: String, - securityMgr: SecurityManager) = { - amClient = AMRMClient.createAMRMClient() - amClient.init(conf) - amClient.start() - this.uiHistoryAddress = uiHistoryAddress - - logInfo("Registering the ApplicationMaster") - synchronized { - amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) - registered = true - } - new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, - preferredNodeLocations, securityMgr) - } - - override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { - if (registered) { - amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) - } - } - - override def getAttemptId() = { - val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - appAttemptId - } - - override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { - // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, - // so not all stable releases have it. - val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) - .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") - - // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. - try { - val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", - classOf[Configuration]) - val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] - val hosts = proxies.map { proxy => proxy.split(":")(0) } - val uriBases = proxies.map { proxy => prefix + proxy + proxyBase } - Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) - } catch { - case e: NoSuchMethodException => - val proxy = WebAppUtils.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val uriBase = prefix + proxy + proxyBase - Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) - } - } - - override def getMaxRegAttempts(conf: YarnConfiguration) = - conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) - -} From 7c0ed13d298d9cf66842c667602e2dccb8f5605b Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Mon, 22 Dec 2014 13:07:01 -0800 Subject: [PATCH 100/227] [SPARK-4079] [CORE] Consolidates Errors if a CompressionCodec is not available This commit consolidates some of the exceptions thrown if compression codecs are not available. If a bad configuration string was passed in, a ClassNotFoundException was through. Also, if Snappy was not available, it would throw an InvocationTargetException when the codec was being used (not when it was being initialized). Now, an IllegalArgumentException is thrown when a codec is not available at creation time - either because the class does not exist or the codec itself is not available in the system. This will allow us to have a better message and fail faster. Author: Kostas Sakellis Closes #3119 from ksakellis/kostas-spark-4079 and squashes the following commits: 9709c7c [Kostas Sakellis] Removed unnecessary Logging class 63bfdd0 [Kostas Sakellis] Removed isAvailable to preserve binary compatibility 1d0ef2f [Kostas Sakellis] [SPARK-4079] [CORE] Added more information to exception 64f3d27 [Kostas Sakellis] [SPARK-4079] [CORE] Code review feedback 52dfa8f [Kostas Sakellis] [SPARK-4079] [CORE] Default to LZF if Snappy not available --- .../apache/spark/io/CompressionCodec.scala | 27 ++++++++++++++----- .../spark/io/CompressionCodecSuite.scala | 6 +++++ 2 files changed, 27 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 1ac7f4e448eb1..f856890d279f4 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -21,11 +21,12 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} -import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} +import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils +import org.apache.spark.Logging /** * :: DeveloperApi :: @@ -44,25 +45,33 @@ trait CompressionCodec { def compressedInputStream(s: InputStream): InputStream } - private[spark] object CompressionCodec { + private val configKey = "spark.io.compression.codec" private val shortCompressionCodecNames = Map( "lz4" -> classOf[LZ4CompressionCodec].getName, "lzf" -> classOf[LZFCompressionCodec].getName, "snappy" -> classOf[SnappyCompressionCodec].getName) def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) + createCodec(conf, conf.get(configKey, DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName) - val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) - .getConstructor(classOf[SparkConf]) - ctor.newInstance(conf).asInstanceOf[CompressionCodec] + val codec = try { + val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) + .getConstructor(classOf[SparkConf]) + Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) + } catch { + case e: ClassNotFoundException => None + case e: IllegalArgumentException => None + } + codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. " + + s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC")) } + val FALLBACK_COMPRESSION_CODEC = "lzf" val DEFAULT_COMPRESSION_CODEC = "snappy" val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq } @@ -120,6 +129,12 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { @DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { + try { + Snappy.getNativeLibraryVersion + } catch { + case e: Error => throw new IllegalArgumentException + } + override def compressedOutputStream(s: OutputStream): OutputStream = { val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768) new SnappyOutputStream(s, blockSize) diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 25be7f25c21bb..8c6035fb367fe 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -85,4 +85,10 @@ class CompressionCodecSuite extends FunSuite { assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } + + test("bad compression codec") { + intercept[IllegalArgumentException] { + CompressionCodec.createCodec(conf, "foobar") + } + } } From fbca6b6ce293b1997b40abeb9ab77b8a969a5fc9 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 22 Dec 2014 13:09:22 -0800 Subject: [PATCH 101/227] [SPARK-4864] Add documentation to Netty-based configs Author: Aaron Davidson Closes #3713 from aarondav/netty-configs and squashes the following commits: 8a8b373 [Aaron Davidson] Address Patrick's comments 3b1f84e [Aaron Davidson] [SPARK-4864] Add documentation to Netty-based configs --- docs/configuration.md | 35 +++++++++++++++++++ .../spark/network/util/TransportConf.java | 2 +- 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 2c8dea869b092..2cc013c47fdbb 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -852,6 +852,41 @@ Apart from these, the following properties are also available, and may be useful between nodes leading to flooding the network with those. + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.shuffle.io.preferDirectBufstrue + (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache + block transfer. For environments where off-heap memory is tightly limited, users may wish to + turn this off to force all allocations from Netty to be on-heap. +
spark.shuffle.io.numConnectionsPerPeer1 + (Netty only) Connections between hosts are reused in order to reduce connection buildup for + large clusters. For clusters with many hard disks and few hosts, this may result in insufficient + concurrency to saturate all disks, and so users may consider increasing this value. +
spark.shuffle.io.maxRetries3 + (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is + set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC + pauses or transient network connectivity issues. +
spark.shuffle.io.retryWait5 + (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying + is simply maxRetries * retryWait, by default 15 seconds. +
#### Scheduling diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 13b37f96f8ce2..7c9adf52af0f0 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -40,7 +40,7 @@ public int connectionTimeoutMs() { return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; } - /** Number of concurrent connections between two nodes for fetching data. **/ + /** Number of concurrent connections between two nodes for fetching data. */ public int numConnectionsPerPeer() { return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 1); } From a61aa669af474b4b82976541cf3a6d2787014a0b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 22 Dec 2014 14:13:31 -0800 Subject: [PATCH 102/227] [Minor] Fix scala doc Minor fix for an obvious scala doc error. Author: Liang-Chi Hsieh Closes #3751 from viirya/fix_scaladoc and squashes the following commits: 03fddaa [Liang-Chi Hsieh] Fix scala doc. --- .../main/scala/org/apache/spark/sql/hive/HiveInspectors.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index e1147478130dd..a156d6f7e285d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -105,9 +105,9 @@ import scala.collection.JavaConversions._ {{{ public interface PrimitiveObjectInspector { // Java Primitives (java.lang.Integer, java.lang.String etc.) - Object getPrimitiveWritableObject(Object o); + Object getPrimitiveJavaObject(Object o); // Writables (hadoop.io.IntWritable, hadoop.io.Text etc.) - Object getPrimitiveJavaObject(Object o); + Object getPrimitiveWritableObject(Object o); // ObjectInspector only inspect the `writable` always return true, we need to check it // before invoking the methods above. boolean preferWritable(); From de9d7d2b5b6d80963505571700e83779fd98f850 Mon Sep 17 00:00:00 2001 From: "genmao.ygm" Date: Mon, 22 Dec 2014 14:14:39 -0800 Subject: [PATCH 103/227] [SPARK-4920][UI]:current spark version in UI is not striking. It is not convenient to see the Spark version. We can keep the same style with Spark website. ![spark_version](https://cloud.githubusercontent.com/assets/7402327/5527025/1c8c721c-8a35-11e4-8d6a-2734f3c6bdf8.jpg) Author: genmao.ygm Closes #3763 from uncleGen/master-clean-141222 and squashes the following commits: 0dcb9a9 [genmao.ygm] [SPARK-4920][UI]:current spark version in UI is not striking. --- .../org/apache/spark/ui/static/webui.css | 10 ++++++++++ .../scala/org/apache/spark/ui/UIUtils.scala | 19 ++++++------------- 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 68c52ac09b55a..5751964b792ce 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -169,6 +169,16 @@ span.additional-metric-title { display: inline-block; } +.version { + line-height: 30px; + vertical-align: bottom; + font-size: 12px; + padding: 0; + margin: 0; + font-weight: bold; + color: #777; +} + /* Hide all additional metrics by default. This is done here rather than using JavaScript to * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ .scheduler_delay, .deserialization_time, .serialization_time, .getting_result_time { diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index d970fa30c1c35..7486cb6b1bbc0 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -194,9 +194,12 @@ private[spark] object UIUtils extends Logging {

- } @@ -245,11 +243,6 @@ private[spark] object UIUtils extends Logging { {content} - } From c233ab3d8d75a33495298964fe73dbf7dd8fe305 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 22 Dec 2014 14:26:28 -0800 Subject: [PATCH 104/227] [SPARK-4818][Core] Add 'iterator' to reduce memory consumed by join In Scala, `map` and `flatMap` of `Iterable` will copy the contents of `Iterable` to a new `Seq`. Such as, ```Scala val iterable = Seq(1, 2, 3).map(v => { println(v) v }) println("Iterable map done") val iterator = Seq(1, 2, 3).iterator.map(v => { println(v) v }) println("Iterator map done") ``` outputed ``` 1 2 3 Iterable map done Iterator map done ``` So we should use 'iterator' to reduce memory consumed by join. Found by Johannes Simon in http://mail-archives.apache.org/mod_mbox/spark-user/201412.mbox/%3C5BE70814-9D03-4F61-AE2C-0D63F2DE4446%40mail.de%3E Author: zsxwing Closes #3671 from zsxwing/SPARK-4824 and squashes the following commits: 48ee7b9 [zsxwing] Remove the explicit types 95d59d6 [zsxwing] Add 'iterator' to reduce memory consumed by join --- .../org/apache/spark/rdd/PairRDDFunctions.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index fe3129b62f71e..4469c89e6bb1c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -483,7 +483,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues( pair => - for (v <- pair._1; w <- pair._2) yield (v, w) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, w) ) } @@ -496,9 +496,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._2.isEmpty) { - pair._1.map(v => (v, None)) + pair._1.iterator.map(v => (v, None)) } else { - for (v <- pair._1; w <- pair._2) yield (v, Some(w)) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, Some(w)) } } } @@ -513,9 +513,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._1.isEmpty) { - pair._2.map(w => (None, w)) + pair._2.iterator.map(w => (None, w)) } else { - for (v <- pair._1; w <- pair._2) yield (Some(v), w) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (Some(v), w) } } } @@ -531,9 +531,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { - case (vs, Seq()) => vs.map(v => (Some(v), None)) - case (Seq(), ws) => ws.map(w => (None, Some(w))) - case (vs, ws) => for (v <- vs; w <- ws) yield (Some(v), Some(w)) + case (vs, Seq()) => vs.iterator.map(v => (Some(v), None)) + case (Seq(), ws) => ws.iterator.map(w => (None, Some(w))) + case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), Some(w)) } } From a96b72781ae40bb303613990b8d8b4721b84e1c3 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 22 Dec 2014 16:42:55 -0800 Subject: [PATCH 105/227] [SPARK-4907][MLlib] Inconsistent loss and gradient in LeastSquaresGradient compared with R In most of the academic paper and algorithm implementations, people use L = 1/2n ||A weights-y||^2 instead of L = 1/n ||A weights-y||^2 for least-squared loss. See Eq. (1) in http://web.stanford.edu/~hastie/Papers/glmnet.pdf Since MLlib uses different convention, this will result different residuals and all the stats properties will be different from GLMNET package in R. The model coefficients will be still the same under this change. Author: DB Tsai Closes #3746 from dbtsai/lir and squashes the following commits: 19c2e85 [DB Tsai] make stepsize twice to converge to the same solution 0b2c29c [DB Tsai] first commit --- .../org/apache/spark/mllib/optimization/Gradient.scala | 10 +++++----- .../regression/StreamingLinearRegressionSuite.scala | 6 +++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 45dbf6044fcc5..5a419d1640292 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -94,16 +94,16 @@ class LogisticGradient extends Gradient { * :: DeveloperApi :: * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) - * L = 1/n ||A weights-y||^2 + * L = 1/2n ||A weights-y||^2 * See also the documentation for the precise formulation. */ @DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val diff = dot(data, weights) - label - val loss = diff * diff + val loss = diff * diff / 2.0 val gradient = data.copy - scal(2.0 * diff, gradient) + scal(diff, gradient) (gradient, loss) } @@ -113,8 +113,8 @@ class LeastSquaresGradient extends Gradient { weights: Vector, cumGradient: Vector): Double = { val diff = dot(data, weights) - label - axpy(2.0 * diff, data, cumGradient) - diff * diff + axpy(diff, data, cumGradient) + diff * diff / 2.0 } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 03b71301e9ab1..70b43ddb7daf5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -52,7 +52,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) - .setStepSize(0.1) + .setStepSize(0.2) .setNumIterations(25) // generate sequence of simulated data @@ -84,7 +84,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) - .setStepSize(0.1) + .setStepSize(0.2) .setNumIterations(25) // generate sequence of simulated data @@ -118,7 +118,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // create model initialized with true weights val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(10.0, 10.0)) - .setStepSize(0.1) + .setStepSize(0.2) .setNumIterations(25) // generate sequence of simulated data for testing From 0e532ccb2b282ea5f7b818e67d521dc44d94c951 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Mon, 22 Dec 2014 22:54:32 -0800 Subject: [PATCH 106/227] [Docs] Minor typo fixes Author: Nicholas Chammas Closes #3772 from nchammas/patch-1 and squashes the following commits: b7d9083 [Nicholas Chammas] [Docs] Minor typo fixes --- docs/submitting-applications.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 2581c9f69fa37..3bd1deaccfafe 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -10,7 +10,7 @@ through a uniform interface so you don't have to configure your application spec # Bundling Your Application's Dependencies If your code depends on other projects, you will need to package them alongside your application in order to distribute the code to a Spark cluster. To do this, -to create an assembly jar (or "uber" jar) containing your code and its dependencies. Both +create an assembly jar (or "uber" jar) containing your code and its dependencies. Both [sbt](https://github.com/sbt/sbt-assembly) and [Maven](http://maven.apache.org/plugins/maven-shade-plugin/) have assembly plugins. When creating assembly jars, list Spark and Hadoop @@ -59,7 +59,7 @@ for applications that involve the REPL (e.g. Spark shell). Alternatively, if your application is submitted from a machine far from the worker machines (e.g. locally on your laptop), it is common to use `cluster` mode to minimize network latency between the drivers and the executors. Note that `cluster` mode is currently not supported for standalone -clusters, Mesos clusters, or python applications. +clusters, Mesos clusters, or Python applications. For Python applications, simply pass a `.py` file in the place of `` instead of a JAR, and add Python `.zip`, `.egg` or `.py` files to the search path with `--py-files`. @@ -174,7 +174,7 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. -For python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries +For Python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries to executors. # More Information From 2823c7f021efd775a14049d6cd0a1f1789a31b6c Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Tue, 23 Dec 2014 11:12:16 -0800 Subject: [PATCH 107/227] [SPARK-4890] Ignore downloaded EC2 libs PR #3737 changed `spark-ec2` to automatically download boto from PyPI. This PR tell git to ignore those downloaded library files. Author: Nicholas Chammas Closes #3770 from nchammas/ignore-ec2-lib and squashes the following commits: 5c440d3 [Nicholas Chammas] gitignore downloaded EC2 libs --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 30b1e12bf1b03..20095dd97343e 100644 --- a/.gitignore +++ b/.gitignore @@ -55,6 +55,7 @@ dev/create-release/*final spark-*-bin-*.tgz unit-tests.log /lib/ +ec2/lib/ rat-results.txt scalastyle.txt scalastyle-output.xml From 2d215aebaad4b4596354f1543d997d06ef340bd5 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 23 Dec 2014 11:18:06 -0800 Subject: [PATCH 108/227] [SPARK-4931][Yarn][Docs] Fix the format of running-on-yarn.md Currently, the format about log4j in running-on-yarn.md is a bit messy. ![running-on-yarn](https://cloud.githubusercontent.com/assets/1000778/5535248/204c4b64-8ab4-11e4-83c3-b4722ea0ad9d.png) Author: zsxwing Closes #3774 from zsxwing/SPARK-4931 and squashes the following commits: 4a5f853 [zsxwing] Fix the format of running-on-yarn.md --- docs/running-on-yarn.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 86276b1aa9cea..da1c8e8aa8667 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -211,18 +211,18 @@ settings and a restart of all node managers. Thus, this is not applicable to hos To use a custom log4j configuration for the application master or executors, there are two options: -- upload a custom log4j.properties using spark-submit, by adding it to the "--files" list of files +- upload a custom `log4j.properties` using `spark-submit`, by adding it to the `--files` list of files to be uploaded with the application. -- add "-Dlog4j.configuration=" to "spark.driver.extraJavaOptions" - (for the driver) or "spark.executor.extraJavaOptions" (for executors). Note that if using a file, - the "file:" protocol should be explicitly provided, and the file needs to exist locally on all +- add `-Dlog4j.configuration=` to `spark.driver.extraJavaOptions` + (for the driver) or `spark.executor.extraJavaOptions` (for executors). Note that if using a file, + the `file:` protocol should be explicitly provided, and the file needs to exist locally on all the nodes. Note that for the first option, both executors and the application master will share the same log4j configuration, which may cause issues when they run on the same node (e.g. trying to write to the same log file). -If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use "${spark.yarn.app.container.log.dir}" in your log4j.properties. For example, log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log. For streaming application, configuring RollingFileAppender and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. +If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use `spark.yarn.app.container.log.dir` in your log4j.properties. For example, `log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log`. For streaming application, configuring `RollingFileAppender` and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. # Important notes From dd155369a04d7dfbf6a5745cbb243e22218367dc Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 23 Dec 2014 12:02:08 -0800 Subject: [PATCH 109/227] [SPARK-4834] [standalone] Clean up application files after app finishes. Commit 7aacb7bfa added support for sharing downloaded files among multiple executors of the same app. That works great in Yarn, since the app's directory is cleaned up after the app is done. But Spark standalone mode didn't do that, so the lock/cache files created by that change were left around and could eventually fill up the disk hosting /tmp. To solve that, create app-specific directories under the local dirs when launching executors. Multiple executors launched by the same Worker will use the same app directories, so they should be able to share the downloaded files. When the application finishes, a new message is sent to all workers telling them the application has finished; once that message has been received, and all executors registered for the application shut down, then those directories will be cleaned up by the Worker. Note: Unit testing this is hard (if even possible), since local-cluster mode doesn't seem to leave the Master/Worker daemons running long enough after `sc.stop()` is called for the clean up protocol to take effect. Author: Marcelo Vanzin Closes #3705 from vanzin/SPARK-4834 and squashes the following commits: b430534 [Marcelo Vanzin] Remove seemingly unnecessary synchronization. 50eb4b9 [Marcelo Vanzin] Review feedback. c0e5ea5 [Marcelo Vanzin] [SPARK-4834] [standalone] Clean up application files after app finishes. --- .../apache/spark/deploy/DeployMessage.scala | 3 ++ .../apache/spark/deploy/master/Master.scala | 5 +++ .../spark/deploy/worker/ExecutorRunner.scala | 4 ++- .../apache/spark/deploy/worker/Worker.scala | 36 +++++++++++++++++-- .../scala/org/apache/spark/util/Utils.scala | 16 +++++++-- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../deploy/worker/ExecutorRunnerTest.scala | 3 +- 7 files changed, 61 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index c46f84de8444a..243d8edb72ed3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -88,6 +88,8 @@ private[deploy] object DeployMessages { case class KillDriver(driverId: String) extends DeployMessage + case class ApplicationFinished(id: String) + // Worker internal case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders @@ -175,4 +177,5 @@ private[deploy] object DeployMessages { // Liveness checks in various places case object SendHeartbeat + } 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 ed5eab9f473ce..f8137bf93c124 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 @@ -704,6 +704,11 @@ private[spark] class Master( } persistenceEngine.removeApplication(app) schedule() + + // Tell all workers that the application has finished, so they can clean up any app state. + workers.foreach { w => + w.actor ! ApplicationFinished(app.id) + } } } 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 f4fedc6327ab9..acbdf0d8bd7bc 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 @@ -47,6 +47,7 @@ private[spark] class ExecutorRunner( val executorDir: File, val workerUrl: String, val conf: SparkConf, + val appLocalDirs: Seq[String], var state: ExecutorState.Value) extends Logging { @@ -77,7 +78,7 @@ private[spark] class ExecutorRunner( /** * Kill executor process, wait for exit and notify worker to update resource status. * - * @param message the exception message which caused the executor's death + * @param message the exception message which caused the executor's death */ private def killProcess(message: Option[String]) { var exitCode: Option[Int] = None @@ -129,6 +130,7 @@ private[spark] class ExecutorRunner( logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) builder.directory(executorDir) + builder.environment.put("SPARK_LOCAL_DIRS", appLocalDirs.mkString(",")) // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0") 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 6863b625514c6..86a87ec22235e 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 @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import java.util.{UUID, Date} import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random @@ -109,6 +109,8 @@ private[spark] class Worker( val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] val finishedDrivers = new HashMap[String, DriverRunner] + val appDirectories = new HashMap[String, Seq[String]] + val finishedApps = new HashSet[String] // The shuffle service is not actually started unless configured. val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) @@ -294,7 +296,7 @@ private[spark] class Worker( val isAppStillRunning = executors.values.map(_.appId).contains(appIdFromDir) dir.isDirectory && !isAppStillRunning && !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECS) - }.foreach { dir => + }.foreach { dir => logInfo(s"Removing directory: ${dir.getPath}") Utils.deleteRecursively(dir) } @@ -339,8 +341,19 @@ private[spark] class Worker( throw new IOException("Failed to create directory " + executorDir) } + // Create local dirs for the executor. These are passed to the executor via the + // SPARK_LOCAL_DIRS environment variable, and deleted by the Worker when the + // application finishes. + val appLocalDirs = appDirectories.get(appId).getOrElse { + Utils.getOrCreateLocalRootDirs(conf).map { dir => + Utils.createDirectory(dir).getAbsolutePath() + }.toSeq + } + appDirectories(appId) = appLocalDirs + val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, executorDir, akkaUrl, conf, ExecutorState.LOADING) + self, workerId, host, sparkHome, executorDir, akkaUrl, conf, appLocalDirs, + ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -377,6 +390,7 @@ private[spark] class Worker( message.map(" message " + _).getOrElse("") + exitStatus.map(" exitStatus " + _).getOrElse("")) } + maybeCleanupApplication(appId) } case KillExecutor(masterUrl, appId, execId) => @@ -446,6 +460,9 @@ private[spark] class Worker( case ReregisterWithMaster => reregisterWithMaster() + case ApplicationFinished(id) => + finishedApps += id + maybeCleanupApplication(id) } private def masterDisconnected() { @@ -454,6 +471,19 @@ private[spark] class Worker( registerWithMaster() } + private def maybeCleanupApplication(id: String): Unit = { + val shouldCleanup = finishedApps.contains(id) && !executors.values.exists(_.appId == id) + if (shouldCleanup) { + finishedApps -= id + appDirectories.remove(id).foreach { dirList => + logInfo(s"Cleaning up local directories for application $id") + dirList.foreach { dir => + Utils.deleteRecursively(new File(dir)) + } + } + } + } + def generateWorkerId(): String = { "worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port) } 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 5e1cb0c7a7712..8c00f2c36aed2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -246,8 +246,11 @@ private[spark] object Utils extends Logging { retval } - /** Create a temporary directory inside the given parent directory */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + /** + * Create a directory inside the given parent directory. The directory is guaranteed to be + * newly created, and is not marked for automatic deletion. + */ + def createDirectory(root: String): File = { var attempts = 0 val maxAttempts = 10 var dir: File = null @@ -265,6 +268,15 @@ private[spark] object Utils extends Logging { } catch { case e: SecurityException => dir = null; } } + dir + } + + /** + * Create a temporary directory inside the given parent directory. The directory will be + * automatically deleted when the VM shuts down. + */ + def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + val dir = createDirectory(root) registerShutdownDeleteDir(dir) dir } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 3f1cd0752e766..aa65f7e8915e6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -119,7 +119,7 @@ class JsonProtocolSuite extends FunSuite { def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", - new SparkConf, ExecutorState.RUNNING) + new SparkConf, Seq("localDir"), ExecutorState.RUNNING) } def createDriverRunner(): DriverRunner = { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 196217062991e..6f233d7cf97aa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -33,7 +33,8 @@ class ExecutorRunnerTest extends FunSuite { val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", - new File(sparkHome), new File("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) + new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), + ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) } From 9c251c555f5ee527143d0cdb9e6c3cb7530fc8f8 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 23 Dec 2014 12:39:41 -0800 Subject: [PATCH 110/227] [SPARK-4932] Add help comments in Analytics Trivial modifications for usability. Author: Takeshi Yamamuro Closes #3775 from maropu/AddHelpCommentInAnalytics and squashes the following commits: fbea8f5 [Takeshi Yamamuro] Add help comments in Analytics --- .../scala/org/apache/spark/examples/graphx/Analytics.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 9cc47327f9a48..409721b01c8fd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -33,6 +33,10 @@ object Analytics extends Logging { if (args.length < 2) { System.err.println( "Usage: Analytics --numEPart= [other options]") + System.err.println("Supported 'taskType' as follows:") + System.err.println(" pagerank Compute PageRank") + System.err.println(" cc Compute the connected components of vertices") + System.err.println(" triangles Count the number of triangles") System.exit(1) } From 395b771feed3fc90e5053defbe86dbd673fde582 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 23 Dec 2014 12:54:20 -0800 Subject: [PATCH 111/227] [SPARK-4914][Build] Cleans lib_managed before compiling with Hive 0.13.1 This PR tries to fix the Hive tests failure encountered in PR #3157 by cleaning `lib_managed` before building assembly jar against Hive 0.13.1 in `dev/run-tests`. Otherwise two sets of datanucleus jars would be left in `lib_managed` and may mess up class paths while executing Hive test suites. Please refer to [this thread] [1] for details. A clean build would be even safer, but we only clean `lib_managed` here to save build time. This PR also takes the chance to clean up some minor typos and formatting issues in the comments. [1]: https://github.com/apache/spark/pull/3157#issuecomment-67656488 [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3756) Author: Cheng Lian Closes #3756 from liancheng/clean-lib-managed and squashes the following commits: e2bd21d [Cheng Lian] Adds lib_managed to clean set c9f2f3e [Cheng Lian] Cleans lib_managed before compiling with Hive 0.13.1 --- dev/run-tests | 26 ++++++++++++++------------ pom.xml | 3 +++ 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 328a73bd8b26d..9192cb7e169f3 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -141,20 +141,22 @@ CURRENT_BLOCK=$BLOCK_BUILD { # NOTE: 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. + # (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. # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - #+ single argument! + # single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - # First build with 0.12 to ensure patches do not break the hive 12 build + # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" - echo "[info] Compile with hive 0.12" + echo "[info] Compile with Hive 0.12.0" echo -e "q\n" \ | sbt/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - # Then build with default version(0.13.1) because tests are based on this version + # Then build with default Hive version (0.13.1) because tests are based on this version + echo "[info] Compile with Hive 0.13.1" + rm -rf lib_managed echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ " -Phive -Phive-thriftserver" echo -e "q\n" \ @@ -178,7 +180,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string - #+ will be interpreted as a single test, which doesn't work. + # will be interpreted as a single test, which doesn't work. SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test") else SBT_MAVEN_TEST_ARGS=("test") @@ -187,11 +189,11 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" # NOTE: 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. + # (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. # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a - #+ single argument! - #+ "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. + # single argument! + # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ @@ -210,7 +212,7 @@ CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS echo "" echo "=========================================================================" -echo "Detecting binary incompatibilites with MiMa" +echo "Detecting binary incompatibilities with MiMa" echo "=========================================================================" CURRENT_BLOCK=$BLOCK_MIMA diff --git a/pom.xml b/pom.xml index 4be8c220223f8..e4db1393ba9cf 100644 --- a/pom.xml +++ b/pom.xml @@ -1010,6 +1010,9 @@ checkpoint + + lib_managed + From 27c5399f4dd542e36ea579956b8cb0613de25c6d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 23 Dec 2014 14:28:36 -0800 Subject: [PATCH 112/227] [SPARK-4730][YARN] Warn against deprecated YARN settings See https://issues.apache.org/jira/browse/SPARK-4730. Author: Andrew Or Closes #3590 from andrewor14/yarn-settings and squashes the following commits: 36e0753 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-settings dcd1316 [Andrew Or] Warn against deprecated YARN settings --- .../spark/deploy/SparkSubmitArguments.scala | 5 ++++- .../cluster/YarnClientSchedulerBackend.scala | 17 ++++++++++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 607b4df73db99..f174bc1af59b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -214,7 +214,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St """.stripMargin } - /** Fill in values by parsing user options. */ + /** + * Fill in values by parsing user options. + * NOTE: Any changes here must be reflected in YarnClientSchedulerBackend. + */ private def parseOpts(opts: Seq[String]): Unit = { val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 2923e6729cd6b..09597bd0e6ab9 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -65,7 +65,8 @@ private[spark] class YarnClientSchedulerBackend( */ private def getExtraClientArguments: Seq[String] = { val extraArgs = new ArrayBuffer[String] - val optionTuples = // List of (target Client argument, environment variable, Spark property) + // List of (target Client argument, environment variable, Spark property) + val optionTuples = List( ("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), @@ -78,11 +79,25 @@ private[spark] class YarnClientSchedulerBackend( ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") ) + // Warn against the following deprecated environment variables: env var -> suggestion + val deprecatedEnvVars = Map( + "SPARK_MASTER_MEMORY" -> "SPARK_DRIVER_MEMORY or --driver-memory through spark-submit", + "SPARK_WORKER_INSTANCES" -> "SPARK_WORKER_INSTANCES or --num-executors through spark-submit", + "SPARK_WORKER_MEMORY" -> "SPARK_EXECUTOR_MEMORY or --executor-memory through spark-submit", + "SPARK_WORKER_CORES" -> "SPARK_EXECUTOR_CORES or --executor-cores through spark-submit") + // Do the same for deprecated properties: property -> suggestion + val deprecatedProps = Map("spark.master.memory" -> "--driver-memory through spark-submit") optionTuples.foreach { case (optionName, envVar, sparkProp) => if (System.getenv(envVar) != null) { extraArgs += (optionName, System.getenv(envVar)) + if (deprecatedEnvVars.contains(envVar)) { + logWarning(s"NOTE: $envVar is deprecated. Use ${deprecatedEnvVars(envVar)} instead.") + } } else if (sc.getConf.contains(sparkProp)) { extraArgs += (optionName, sc.getConf.get(sparkProp)) + if (deprecatedProps.contains(sparkProp)) { + logWarning(s"NOTE: $sparkProp is deprecated. Use ${deprecatedProps(sparkProp)} instead.") + } } } extraArgs From 96281cd0c3ffb4c0fbdb64b8f7b705f863d46b33 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 23 Dec 2014 14:58:33 -0800 Subject: [PATCH 113/227] [SPARK-4913] Fix incorrect event log path SPARK-2261 uses a single file to log events for an app. `eventLogDir` in `ApplicationDescription` is replaced with `eventLogFile`. However, `ApplicationDescription` in `SparkDeploySchedulerBackend` is initialized with `SparkContext`'s `eventLogDir`. It is just the log directory, not the actual log file path. `Master.rebuildSparkUI` can not correctly rebuild a new SparkUI for the app. Because the `ApplicationDescription` is remotely registered with `Master` and the app's id is then generated in `Master`, we can not get the app id in advance before registration. So the received description needs to be modified with correct `eventLogFile` value. Author: Liang-Chi Hsieh Closes #3755 from viirya/fix_app_logdir and squashes the following commits: 5e0ea35 [Liang-Chi Hsieh] Revision for comment. b5730a1 [Liang-Chi Hsieh] Fix incorrect event log path. Closes #3777 (a duplicate PR for the same JIRA) --- .../spark/deploy/ApplicationDescription.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 20 +++++++++++++++---- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index b10b7b8d32523..65a1a8fd7e929 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -23,7 +23,7 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, var appUiUrl: String, - val eventLogFile: Option[String] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") 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 f8137bf93c124..81fa0770bbaf9 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 @@ -719,14 +719,26 @@ private[spark] class Master( def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogFile = app.desc.eventLogFile.getOrElse { - // Event logging is not enabled for this application - app.desc.appUiUrl = notFoundBasePath + val eventLogFile = app.desc.eventLogDir + .map { dir => EventLoggingListener.getLogPath(dir, app.id) } + .getOrElse { + // Event logging is not enabled for this application + app.desc.appUiUrl = notFoundBasePath + return false + } + val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) + + if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) { + // Event logging is enabled for this application, but the application is still in progress + val title = s"Application history not found (${app.id})" + var msg = s"Application $appName is still in progress." + logWarning(msg) + msg = URLEncoder.encode(msg, "UTF-8") + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" return false } try { - val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), From 10d69e9cbfdabe95d0e513176d5347d7b59da0ee Mon Sep 17 00:00:00 2001 From: Ilayaperumal Gopinathan Date: Tue, 23 Dec 2014 15:14:54 -0800 Subject: [PATCH 114/227] [SPARK-4802] [streaming] Remove receiverInfo once receiver is de-registered Once the streaming receiver is de-registered at executor, the `ReceiverTrackerActor` needs to remove the corresponding reveiverInfo from the `receiverInfo` map at `ReceiverTracker`. Author: Ilayaperumal Gopinathan Closes #3647 from ilayaperumalg/receiverInfo-RTracker and squashes the following commits: 6eb97d5 [Ilayaperumal Gopinathan] Polishing based on the review 3640c86 [Ilayaperumal Gopinathan] Remove receiverInfo once receiver is de-registered --- .../apache/spark/streaming/scheduler/ReceiverTracker.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 32e481dabc8ca..1f0e442a12283 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -150,8 +150,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false logWarning("No prior receiver info") ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) } - receiverInfo(streamId) = newReceiverInfo - listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId))) + receiverInfo -= streamId + listenerBus.post(StreamingListenerReceiverStopped(newReceiverInfo)) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { From 3f5f4cc4e7b3bc458e0579d247a0652dca365853 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 23 Dec 2014 15:45:53 -0800 Subject: [PATCH 115/227] [SPARK-4671][Streaming]Do not replicate streaming block when WAL is enabled Currently streaming block will be replicated when specific storage level is set, since WAL is already fault tolerant, so replication is needless and will hurt the throughput of streaming application. Hi tdas , as per discussed about this issue, I fixed with this implementation, I'm not is this the way you want, would you mind taking a look at it? Thanks a lot. Author: jerryshao Closes #3534 from jerryshao/SPARK-4671 and squashes the following commits: 500b456 [jerryshao] Do not replicate streaming block when WAL is enabled --- .../receiver/ReceivedBlockHandler.scala | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index fdf995320beb4..c0670e22a7aee 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -121,6 +121,24 @@ private[streaming] class WriteAheadLogBasedBlockHandler( private val maxFailures = conf.getInt( "spark.streaming.receiver.writeAheadLog.maxFailures", 3) + private val effectiveStorageLevel = { + if (storageLevel.deserialized) { + logWarning(s"Storage level serialization ${storageLevel.deserialized} is not supported when" + + s" write ahead log is enabled, change to serialization false") + } + if (storageLevel.replication > 1) { + logWarning(s"Storage level replication ${storageLevel.replication} is unnecessary when " + + s"write ahead log is enabled, change to replication 1") + } + + StorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.useOffHeap, false, 1) + } + + if (storageLevel != effectiveStorageLevel) { + logWarning(s"User defined storage level $storageLevel is changed to effective storage level " + + s"$effectiveStorageLevel when write ahead log is enabled") + } + // Manages rolling log files private val logManager = new WriteAheadLogManager( checkpointDirToLogDir(checkpointDir, streamId), @@ -156,7 +174,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler( // Store the block in block manager val storeInBlockManagerFuture = Future { val putResult = - blockManager.putBytes(blockId, serializedBlock, storageLevel, tellMaster = true) + blockManager.putBytes(blockId, serializedBlock, effectiveStorageLevel, tellMaster = true) if (!putResult.map { _._1 }.contains(blockId)) { throw new SparkException( s"Could not store $blockId to block manager with storage level $storageLevel") From 7e2deb71c4239564631b19c748e95c3d1aa1c77d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 23 Dec 2014 16:02:59 -0800 Subject: [PATCH 116/227] [SPARK-4606] Send EOF to child JVM when there's no more data to read. Author: Marcelo Vanzin Closes #3460 from vanzin/SPARK-4606 and squashes the following commits: 031207d [Marcelo Vanzin] [SPARK-4606] Send EOF to child JVM when there's no more data to read. --- .../SparkSubmitDriverBootstrapper.scala | 3 ++- .../scala/org/apache/spark/util/Utils.scala | 24 +++++++++++++------ 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index d2687faad62b1..2eab9981845e8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -151,7 +151,8 @@ private[spark] object SparkSubmitDriverBootstrapper { val isWindows = Utils.isWindows val isSubprocess = sys.env.contains("IS_SUBPROCESS") if (!isWindows) { - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin", + propagateEof = true) stdinThread.start() // Spark submit (JVM) may run as a subprocess, and so this JVM should terminate on // broken pipe, signaling that the parent process has exited. This is the case if the 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 8c00f2c36aed2..0d771baaa6abc 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1847,19 +1847,29 @@ private[spark] object Utils extends Logging { /** * A utility class to redirect the child process's stdout or stderr. */ -private[spark] class RedirectThread(in: InputStream, out: OutputStream, name: String) +private[spark] class RedirectThread( + in: InputStream, + out: OutputStream, + name: String, + propagateEof: Boolean = false) extends Thread(name) { setDaemon(true) override def run() { scala.util.control.Exception.ignoring(classOf[IOException]) { // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - out.write(buf, 0, len) - out.flush() - len = in.read(buf) + try { + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } finally { + if (propagateEof) { + out.close() + } } } } From fd41eb9574280b5cfee9b94b4f92e4c44363fb14 Mon Sep 17 00:00:00 2001 From: jbencook Date: Tue, 23 Dec 2014 17:46:24 -0800 Subject: [PATCH 117/227] [SPARK-4860][pyspark][sql] speeding up `sample()` and `takeSample()` This PR modifies the python `SchemaRDD` to use `sample()` and `takeSample()` from Scala instead of the slower python implementations from `rdd.py`. This is worthwhile because the `Row`'s are already serialized as Java objects. In order to use the faster `takeSample()`, a `takeSampleToPython()` method was implemented in `SchemaRDD.scala` following the pattern of `collectToPython()`. Author: jbencook Author: J. Benjamin Cook Closes #3764 from jbencook/master and squashes the following commits: 6fbc769 [J. Benjamin Cook] [SPARK-4860][pyspark][sql] fixing sloppy indentation for takeSampleToPython() arguments 5170da2 [J. Benjamin Cook] [SPARK-4860][pyspark][sql] fixing typo: from RDD to SchemaRDD de22f70 [jbencook] [SPARK-4860][pyspark][sql] using sample() method from JavaSchemaRDD b916442 [jbencook] [SPARK-4860][pyspark][sql] adding sample() to JavaSchemaRDD 020cbdf [jbencook] [SPARK-4860][pyspark][sql] using Scala implementations of `sample()` and `takeSample()` --- python/pyspark/sql.py | 28 +++++++++++++++++++ .../org/apache/spark/sql/SchemaRDD.scala | 15 ++++++++++ .../spark/sql/api/java/JavaSchemaRDD.scala | 6 ++++ 3 files changed, 49 insertions(+) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 469f82473af97..9807a84a66f11 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -2085,6 +2085,34 @@ def subtract(self, other, numPartitions=None): else: raise ValueError("Can only subtract another SchemaRDD") + def sample(self, withReplacement, fraction, seed=None): + """ + Return a sampled subset of this SchemaRDD. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.sample(False, 0.5, 97).count() + 2L + """ + assert fraction >= 0.0, "Negative fraction value: %s" % fraction + seed = seed if seed is not None else random.randint(0, sys.maxint) + rdd = self._jschema_rdd.sample(withReplacement, fraction, long(seed)) + return SchemaRDD(rdd, self.sql_ctx) + + def takeSample(self, withReplacement, num, seed=None): + """Return a fixed-size sampled subset of this SchemaRDD. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.takeSample(False, 2, 97) + [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')] + """ + seed = seed if seed is not None else random.randint(0, sys.maxint) + with SCCallSiteSync(self.context) as css: + bytesInJava = self._jschema_rdd.baseSchemaRDD() \ + .takeSampleToPython(withReplacement, num, long(seed)) \ + .iterator() + cls = _create_cls(self.schema()) + return map(cls, self._collect_iterator_through_file(bytesInJava)) + def _test(): import doctest 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 7baf8ffcef787..856b10f1a8fd8 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 @@ -437,6 +437,21 @@ class SchemaRDD( }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable) } + /** + * Serializes the Array[Row] returned by SchemaRDD's takeSample(), using the same + * format as javaToPython and collectToPython. It is used by pyspark. + */ + private[sql] def takeSampleToPython( + withReplacement: Boolean, + num: Int, + seed: Long): JList[Array[Byte]] = { + val fieldTypes = schema.fields.map(_.dataType) + val pickle = new Pickler + new java.util.ArrayList(this.takeSample(withReplacement, num, seed).map { row => + EvaluatePython.rowToArray(row, fieldTypes) + }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable) + } + /** * Creates SchemaRDD by applying own schema to derived RDD. Typically used to wrap return value * of base RDD functions that do not change schema. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index ac4844f9b9290..5b9c612487ace 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -218,4 +218,10 @@ class JavaSchemaRDD( */ def subtract(other: JavaSchemaRDD, p: Partitioner): JavaSchemaRDD = this.baseSchemaRDD.subtract(other.baseSchemaRDD, p).toJavaSchemaRDD + + /** + * Return a SchemaRDD with a sampled version of the underlying dataset. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaSchemaRDD = + this.baseSchemaRDD.sample(withReplacement, fraction, seed).toJavaSchemaRDD } From 199e59aacd540e17b31f38e0e32a3618870e9055 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 23 Dec 2014 19:14:34 -0800 Subject: [PATCH 118/227] [SPARK-4881][Minor] Use SparkConf#getBoolean instead of get().toBoolean It's really a minor issue. In ApplicationMaster, there is code like as follows. val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean I think, the code can be simplified like as follows. val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false) Author: Kousuke Saruta Closes #3733 from sarutak/SPARK-4881 and squashes the following commits: 1771430 [Kousuke Saruta] Modified the code like sparkConf.get(...).toBoolean to sparkConf.getBoolean(...) c63daa0 [Kousuke Saruta] Simplified code --- core/src/main/scala/org/apache/spark/SecurityManager.scala | 4 ++-- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 49dae5231a92c..ec82d09cd079b 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -151,8 +151,8 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with private val authOn = sparkConf.getBoolean("spark.authenticate", false) // keep spark.ui.acls.enable for backwards compatibility with 1.0 - private var aclsOn = sparkConf.getOption("spark.acls.enable").getOrElse( - sparkConf.get("spark.ui.acls.enable", "false")).toBoolean + private var aclsOn = + sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false)) // admin acls should be set before view or modify acls private var adminAcls: Set[String] = diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index a157e36e2286e..0001c2329c83a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -131,7 +131,7 @@ class HadoopRDD[K, V]( // used to build JobTracker ID private val createTime = new Date() - private val shouldCloneJobConf = sc.conf.get("spark.hadoop.cloneConf", "false").toBoolean + private val shouldCloneJobConf = sc.conf.getBoolean("spark.hadoop.cloneConf", false) // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index b2e45435c4abb..9c77dff48dc8b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -311,7 +311,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def cleanupStagingDir(fs: FileSystem) { var stagingDirPath: Path = null try { - val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false) if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 5f0c67f05c9dd..eb97a7b3c59a4 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -693,7 +693,7 @@ private[spark] object ClientBase extends Logging { addClasspathEntry(Environment.PWD.$(), env) // Normally the users app.jar is last in case conflicts with spark jars - if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { + if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { addUserClasspath(args, sparkConf, env) addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) populateHadoopClasspath(conf, env) From 29fabb1b528e60b2f65132a9ab64f2fd95b729ba Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 24 Dec 2014 13:32:51 -0800 Subject: [PATCH 119/227] SPARK-4297 [BUILD] Build warning fixes omnibus There are a number of warnings generated in a normal, successful build right now. They're mostly Java unchecked cast warnings, which can be suppressed. But there's a grab bag of other Scala language warnings and so on that can all be easily fixed. The forthcoming PR fixes about 90% of the build warnings I see now. Author: Sean Owen Closes #3157 from srowen/SPARK-4297 and squashes the following commits: 8c9e469 [Sean Owen] Suppress unchecked cast warnings, and several other build warning fixes --- core/pom.xml | 4 ++-- .../spark/scheduler/TaskResultGetter.scala | 1 + .../java/org/apache/spark/JavaAPISuite.java | 4 +++- .../metrics/InputOutputMetricsSuite.scala | 4 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/mllib/feature/JavaTfIdfSuite.java | 2 ++ .../spark/sql/api/java/UserDefinedType.java | 1 + .../spark/sql/parquet/ParquetTypes.scala | 4 ++-- .../spark/sql/api/java/JavaRowSuite.java | 1 + .../org/apache/spark/sql/DslQuerySuite.scala | 2 ++ .../spark/sql/parquet/ParquetQuerySuite.scala | 14 ++++++++++--- .../sql/hive/parquet/FakeParquetSerDe.scala | 2 +- .../sql/hive/execution/UDFListListInt.java | 20 ++++++++----------- .../apache/spark/streaming/JavaAPISuite.java | 16 +++++++-------- 14 files changed, 45 insertions(+), 32 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 1feb00b3a7fb8..c5c41b2b5de42 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -352,9 +352,9 @@ - + - + diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 819b51e12ad8c..4896ec845bbc9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import scala.language.existentials import scala.util.control.NonFatal import org.apache.spark._ diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e5bdad6bda2fa..5ce299d05824b 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -184,6 +184,7 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @SuppressWarnings("unchecked") @Test public void repartitionAndSortWithinPartitions() { List> pairs = new ArrayList>(); @@ -491,6 +492,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(33, sum); } + @SuppressWarnings("unchecked") @Test public void aggregateByKey() { JavaPairRDD pairs = sc.parallelizePairs( @@ -1556,7 +1558,7 @@ static class Class2 {} @Test public void testRegisterKryoClasses() { SparkConf conf = new SparkConf(); - conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); + conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); Assert.assertEquals( Class1.class.getName() + "," + Class2.class.getName(), conf.get("spark.kryo.classesToRegister")); diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index ca226fd4e694f..f8bcde12a371a 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -24,14 +24,14 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener} import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} import scala.collection.mutable.ArrayBuffer -class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with ShouldMatchers { +class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with Matchers { test("input metrics when reading text file with single split") { val file = new File(getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(file)) 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 436eea4f1fdcf..d6ec9e129cceb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -739,7 +739,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F test("accumulator not calculated for resubmitted result stage") { //just for register - val accum = new Accumulator[Int](0, SparkContext.IntAccumulatorParam) + val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index 064263e02cd11..fbc26167ce66f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -49,6 +49,7 @@ public void tearDown() { public void tfIdf() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); + @SuppressWarnings("unchecked") JavaRDD> documents = sc.parallelize(Lists.newArrayList( Lists.newArrayList("this is a sentence".split(" ")), Lists.newArrayList("this is another sentence".split(" ")), @@ -68,6 +69,7 @@ public void tfIdf() { public void tfIdfMinimumDocumentFrequency() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); + @SuppressWarnings("unchecked") JavaRDD> documents = sc.parallelize(Lists.newArrayList( Lists.newArrayList("this is a sentence".split(" ")), Lists.newArrayList("this is another sentence".split(" ")), diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java index b751847b464fd..f0d079d25b5d4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java @@ -35,6 +35,7 @@ protected UserDefinedType() { } public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; + @SuppressWarnings("unchecked") UserDefinedType that = (UserDefinedType) o; return this.sqlType().equals(that.sqlType()); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 0e6fb57d57bca..97447871a11ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -24,8 +24,8 @@ import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter +import parquet.format.converter.ParquetMetadataConverter import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} import parquet.hadoop.util.ContextUtil @@ -458,7 +458,7 @@ private[parquet] object ParquetTypesConverter extends Logging { // ... and fallback to "_metadata" if no such file exists (which implies the Parquet file is // empty, thus normally the "_metadata" file is expected to be fairly small). .orElse(children.find(_.getPath.getName == ParquetFileWriter.PARQUET_METADATA_FILE)) - .map(ParquetFileReader.readFooter(conf, _)) + .map(ParquetFileReader.readFooter(conf, _, ParquetMetadataConverter.NO_FILTER)) .getOrElse( throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path")) } diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java index bc5cd66482add..2b5812159d07d 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java @@ -141,6 +141,7 @@ public void constructComplexRow() { doubleValue, stringValue, timestampValue, null); // Complex array + @SuppressWarnings("unchecked") List> arrayOfMaps = Arrays.asList(simpleMap); List arrayOfRows = Arrays.asList(simpleStruct); 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 e40d034ce4dc0..691c4b38287bf 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 @@ -24,6 +24,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl._ import org.apache.spark.sql.test.TestSQLContext._ +import scala.language.postfixOps + class DslQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ 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 074855389d746..a5fe2e8da2840 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,6 +17,8 @@ package org.apache.spark.sql.parquet +import scala.reflect.ClassTag + import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} @@ -459,11 +461,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("make RecordFilter for simple predicates") { - def checkFilter[T <: FilterPredicate](predicate: Expression, defined: Boolean = true): Unit = { + def checkFilter[T <: FilterPredicate : ClassTag]( + predicate: Expression, + defined: Boolean = true): Unit = { val filter = ParquetFilters.createFilter(predicate) if (defined) { assert(filter.isDefined) - assert(filter.get.isInstanceOf[T]) + val tClass = implicitly[ClassTag[T]].runtimeClass + val filterGet = filter.get + assert( + tClass.isInstance(filterGet), + s"$filterGet of type ${filterGet.getClass} is not an instance of $tClass") } else { assert(filter.isEmpty) } @@ -484,7 +492,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA checkFilter[Operators.And]('a.int === 1 && 'a.int < 4) checkFilter[Operators.Or]('a.int === 1 || 'a.int < 4) - checkFilter[Operators.Not](!('a.int === 1)) + checkFilter[Operators.NotEq[Integer]](!('a.int === 1)) checkFilter('a.int > 'b.int, defined = false) checkFilter(('a.int > 'b.int) && ('a.int > 'b.int), defined = false) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala index abed299cd957f..2a16c9d1a27c9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.io.Writable * when "spark.sql.hive.convertMetastoreParquet" is set to true. */ @deprecated("No code should depend on FakeParquetHiveSerDe as it is only intended as a " + - "placeholder in the Hive MetaStore") + "placeholder in the Hive MetaStore", "1.2.0") class FakeParquetSerDe extends SerDe { override def getObjectInspector: ObjectInspector = new ObjectInspector { override def getCategory: Category = Category.PRIMITIVE diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java index d2d39a8c4dc28..808e2986d3b77 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java @@ -23,25 +23,21 @@ public class UDFListListInt extends UDF { /** - * * @param obj - * SQL schema: array> - * Java Type: List> - * @return + * SQL schema: array<struct<x: int, y: int, z: int>> + * Java Type: List<List<Integer>> */ + @SuppressWarnings("unchecked") public long evaluate(Object obj) { if (obj == null) { - return 0l; + return 0L; } - List listList = (List) obj; + List> listList = (List>) obj; long retVal = 0; - for (List aList : listList) { - @SuppressWarnings("unchecked") - List list = (List) aList; - @SuppressWarnings("unchecked") - Integer someInt = (Integer) list.get(1); + for (List aList : listList) { + Number someInt = (Number) aList.get(1); try { - retVal += (long) (someInt.intValue()); + retVal += someInt.longValue(); } catch (NullPointerException e) { System.out.println(e); } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index ce645fccba1d0..12cc0de7509d6 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -57,7 +57,7 @@ public void equalIterable(Iterable a, Iterable b) { @Test public void testInitialization() { - Assert.assertNotNull(ssc.sc()); + Assert.assertNotNull(ssc.sparkContext()); } @SuppressWarnings("unchecked") @@ -662,7 +662,7 @@ public void testStreamingContextTransform(){ listOfDStreams1, new Function2>, Time, JavaRDD>() { public JavaRDD call(List> listOfRDDs, Time time) { - assert(listOfRDDs.size() == 2); + Assert.assertEquals(2, listOfRDDs.size()); return null; } } @@ -675,7 +675,7 @@ public JavaRDD call(List> listOfRDDs, Time time) { listOfDStreams2, new Function2>, Time, JavaPairRDD>>() { public JavaPairRDD> call(List> listOfRDDs, Time time) { - assert(listOfRDDs.size() == 3); + Assert.assertEquals(3, listOfRDDs.size()); JavaRDD rdd1 = (JavaRDD)listOfRDDs.get(0); JavaRDD rdd2 = (JavaRDD)listOfRDDs.get(1); JavaRDD> rdd3 = (JavaRDD>)listOfRDDs.get(2); @@ -969,7 +969,7 @@ public Integer call(Tuple2 in) throws Exception { }); JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } @@ -1012,7 +1012,7 @@ public Iterable> call(Tuple2 in) throws } }); JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } @@ -1163,9 +1163,9 @@ public void testGroupByKeyAndWindow() { JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); - assert(result.size() == expected.size()); + Assert.assertEquals(expected.size(), result.size()); for (int i = 0; i < result.size(); i++) { - assert(convert(result.get(i)).equals(convert(expected.get(i)))); + Assert.assertEquals(convert(expected.get(i)), convert(result.get(i))); } } @@ -1383,7 +1383,7 @@ public JavaPairRDD call(JavaPairRDD in) thro }); JavaTestUtils.attachTestOutputStream(sorted); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } From b4d0db80a0bfba7f1e045d4edb9357b4b2c0a557 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 24 Dec 2014 19:49:41 -0800 Subject: [PATCH 120/227] [SPARK-4873][Streaming] Use `Future.zip` instead of `Future.flatMap`(for-loop) in WriteAheadLogBasedBlockHandler Use `Future.zip` instead of `Future.flatMap`(for-loop). `zip` implies these two Futures will run concurrently, while `flatMap` usually means one Future depends on the other one. Author: zsxwing Closes #3721 from zsxwing/SPARK-4873 and squashes the following commits: 46a2cd9 [zsxwing] Use Future.zip instead of Future.flatMap(for-loop) --- .../spark/streaming/receiver/ReceivedBlockHandler.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index c0670e22a7aee..8b97db8dd36f1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -187,10 +187,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler( } // Combine the futures, wait for both to complete, and return the write ahead log segment - val combinedFuture = for { - _ <- storeInBlockManagerFuture - fileSegment <- storeInWriteAheadLogFuture - } yield fileSegment + val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2) val segment = Await.result(combinedFuture, blockStoreTimeout) WriteAheadLogBasedStoreResult(blockId, segment) } From 11dd99317b1b0bcd18a0a0cc3c4eb79d3aaa4a2b Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 25 Dec 2014 07:05:43 -0800 Subject: [PATCH 121/227] [SPARK-4953][Doc] Fix the description of building Spark with YARN At the section "Specifying the Hadoop Version" In building-spark.md, there is description about building with YARN with Hadoop 0.23. Spark 1.3.0 will not support Hadoop 0.23 so we should fix the description. Author: Kousuke Saruta Closes #3787 from sarutak/SPARK-4953 and squashes the following commits: ee9c355 [Kousuke Saruta] Removed description related to a specific vendor 9ab0c24 [Kousuke Saruta] Fix the description about building SPARK with YARN --- docs/building-spark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 70165eabca435..dab3d2aef497e 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -60,7 +60,7 @@ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package {% endhighlight %} -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". As of Spark 1.3, Spark only supports YARN versions 2.2.0 and later. +You can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". Spark only supports YARN versions 2.2.0 and later. Examples: From 08b18c7eb790c65670778eab8a6e32486c5f76e9 Mon Sep 17 00:00:00 2001 From: Denny Lee Date: Thu, 25 Dec 2014 14:05:55 -0800 Subject: [PATCH 122/227] Fix "Building Spark With Maven" link in README.md Corrected link to the Building Spark with Maven page from its original (http://spark.apache.org/docs/latest/building-with-maven.html) to the current page (http://spark.apache.org/docs/latest/building-spark.html) Author: Denny Lee Closes #3802 from dennyglee/patch-1 and squashes the following commits: 15f601a [Denny Lee] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8d57d50da96c9..16628bd406775 100644 --- a/README.md +++ b/README.md @@ -26,7 +26,7 @@ To build Spark and its example programs, run: (You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at -["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-with-maven.html). +["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-spark.html). ## Interactive Scala Shell From b6b6393b477baa7246e2c7c4942b30c14cb05fa4 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 25 Dec 2014 14:13:12 -0800 Subject: [PATCH 123/227] [EC2] Update default Spark version to 1.2.0 Now that 1.2.0 is out, let's update the default Spark version. Author: Nicholas Chammas Closes #3793 from nchammas/patch-1 and squashes the following commits: 3255832 [Nicholas Chammas] add 1.2.0 version to Spark-Shark map ec0e904 [Nicholas Chammas] [EC2] Update default Spark version to 1.2.0 --- ec2/spark_ec2.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 556d99d1027b8..c6679f7a0b7e9 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -39,7 +39,7 @@ from optparse import OptionParser from sys import stderr -DEFAULT_SPARK_VERSION = "1.1.0" +DEFAULT_SPARK_VERSION = "1.2.0" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) MESOS_SPARK_EC2_BRANCH = "v4" @@ -251,10 +251,13 @@ def get_spark_shark_version(opts): "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1", + # These are dummy versions (no Shark versions after this) "1.0.0": "1.0.0", "1.0.1": "1.0.1", "1.0.2": "1.0.2", "1.1.0": "1.1.0", + "1.1.1": "1.1.1", + "1.2.0": "1.2.0", } version = opts.spark_version.replace("v", "") if version not in spark_shark_map: From ac8278593ea68dd3be8cddf8cd5ce739f163ab84 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 25 Dec 2014 14:16:50 -0800 Subject: [PATCH 124/227] [EC2] Update mesos/spark-ec2 branch to branch-1.3 Going forward, we'll use matching branch names across the mesos/spark-ec2 and apache/spark repositories, per [the discussion here](https://github.com/mesos/spark-ec2/pull/85#issuecomment-68069589). Author: Nicholas Chammas Closes #3804 from nchammas/patch-2 and squashes the following commits: cd2c0d4 [Nicholas Chammas] [EC2] Update mesos/spark-ec2 branch to branch-1.3 --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index c6679f7a0b7e9..485eea4f5e683 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -42,7 +42,7 @@ DEFAULT_SPARK_VERSION = "1.2.0" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) -MESOS_SPARK_EC2_BRANCH = "v4" +MESOS_SPARK_EC2_BRANCH = "branch-1.3" # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH) From f205fe477c33a541053c198cd43a5811d6cf9fe2 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 25 Dec 2014 19:39:49 -0800 Subject: [PATCH 125/227] [SPARK-4537][Streaming] Expand StreamingSource to add more metrics Add `processingDelay`, `schedulingDelay` and `totalDelay` for the last completed batch. Add `lastReceivedBatchRecords` and `totalReceivedBatchRecords` to the received records counting. Author: jerryshao Closes #3466 from jerryshao/SPARK-4537 and squashes the following commits: 00f5f7f [jerryshao] Change the code style and add totalProcessedRecords 44721a6 [jerryshao] Further address the comments c097ddc [jerryshao] Address the comments 02dd44f [jerryshao] Fix the addressed comments c7a9376 [jerryshao] Expand StreamingSource to add more metrics --- .../spark/streaming/StreamingSource.scala | 53 ++++++++++++++----- .../ui/StreamingJobProgressListener.scala | 19 ++++++- 2 files changed, 57 insertions(+), 15 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index e35a568ddf115..9697437dd2fe5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -29,9 +29,17 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { private val streamingListener = ssc.progressListener private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, - defaultValue: T) { + defaultValue: T): Unit = { + registerGaugeWithOption[T](name, + (l: StreamingJobProgressListener) => Option(f(streamingListener)), defaultValue) + } + + private def registerGaugeWithOption[T]( + name: String, + f: StreamingJobProgressListener => Option[T], + defaultValue: T): Unit = { metricRegistry.register(MetricRegistry.name("streaming", name), new Gauge[T] { - override def getValue: T = Option(f(streamingListener)).getOrElse(defaultValue) + override def getValue: T = f(streamingListener).getOrElse(defaultValue) }) } @@ -41,6 +49,12 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { // Gauge for number of total completed batches registerGauge("totalCompletedBatches", _.numTotalCompletedBatches, 0L) + // Gauge for number of total received records + registerGauge("totalReceivedRecords", _.numTotalReceivedRecords, 0L) + + // Gauge for number of total processed records + registerGauge("totalProcessedRecords", _.numTotalProcessedRecords, 0L) + // Gauge for number of unprocessed batches registerGauge("unprocessedBatches", _.numUnprocessedBatches, 0L) @@ -55,19 +69,30 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { // Gauge for last completed batch, useful for monitoring the streaming job's running status, // displayed data -1 for any abnormal condition. - registerGauge("lastCompletedBatch_submissionTime", - _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) - registerGauge("lastCompletedBatch_processStartTime", - _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) - registerGauge("lastCompletedBatch_processEndTime", - _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) + registerGaugeWithOption("lastCompletedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime), -1L) + registerGaugeWithOption("lastCompletedBatch_processingStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime), -1L) + registerGaugeWithOption("lastCompletedBatch_processingEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime), -1L) + + // Gauge for last completed batch's delay information. + registerGaugeWithOption("lastCompletedBatch_processingDelay", + _.lastCompletedBatch.flatMap(_.processingDelay), -1L) + registerGaugeWithOption("lastCompletedBatch_schedulingDelay", + _.lastCompletedBatch.flatMap(_.schedulingDelay), -1L) + registerGaugeWithOption("lastCompletedBatch_totalDelay", + _.lastCompletedBatch.flatMap(_.totalDelay), -1L) // Gauge for last received batch, useful for monitoring the streaming job's running status, // displayed data -1 for any abnormal condition. - registerGauge("lastReceivedBatch_submissionTime", - _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) - registerGauge("lastReceivedBatch_processStartTime", - _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) - registerGauge("lastReceivedBatch_processEndTime", - _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) + registerGaugeWithOption("lastReceivedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime), -1L) + registerGaugeWithOption("lastReceivedBatch_processingStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime), -1L) + registerGaugeWithOption("lastReceivedBatch_processingEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime), -1L) + + // Gauge for last received batch records. + registerGauge("lastReceivedBatch_records", _.lastReceivedBatchRecords.values.sum, 0L) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index f61069b56db5e..5ee53a5c5f561 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -25,7 +25,6 @@ import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted import org.apache.spark.streaming.scheduler.BatchInfo import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution -import org.apache.spark.Logging private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) @@ -36,6 +35,8 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) private val completedaBatchInfos = new Queue[BatchInfo] private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) private var totalCompletedBatches = 0L + private var totalReceivedRecords = 0L + private var totalProcessedRecords = 0L private val receiverInfos = new HashMap[Int, ReceiverInfo] val batchDuration = ssc.graph.batchDuration.milliseconds @@ -65,6 +66,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) + + batchStarted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => + totalReceivedRecords += infos.map(_.numRecords).sum + } } override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { @@ -73,6 +78,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) completedaBatchInfos.enqueue(batchCompleted.batchInfo) if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() totalCompletedBatches += 1L + + batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => + totalProcessedRecords += infos.map(_.numRecords).sum + } } def numReceivers = synchronized { @@ -83,6 +92,14 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) totalCompletedBatches } + def numTotalReceivedRecords: Long = synchronized { + totalReceivedRecords + } + + def numTotalProcessedRecords: Long = synchronized { + totalProcessedRecords + } + def numUnprocessedBatches: Long = synchronized { waitingBatchInfos.size + runningBatchInfos.size } From f9ed2b6641b9df39cee4b98a33cd5a3ddda2d146 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 25 Dec 2014 19:46:05 -0800 Subject: [PATCH 126/227] [SPARK-4608][Streaming] Reorganize StreamingContext implicit to improve API convenience There is only one implicit function `toPairDStreamFunctions` in `StreamingContext`. This PR did similar reorganization like [SPARK-4397](https://issues.apache.org/jira/browse/SPARK-4397). Compiled the following codes with Spark Streaming 1.1.0 and ran it with this PR. Everything is fine. ```Scala import org.apache.spark._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ object StreamingApp { def main(args: Array[String]) { val conf = new SparkConf().setMaster("local[2]").setAppName("FileWordCount") val ssc = new StreamingContext(conf, Seconds(10)) val lines = ssc.textFileStream("/some/path") val words = lines.flatMap(_.split(" ")) val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) wordCounts.print() ssc.start() ssc.awaitTermination() } } ``` Author: zsxwing Closes #3464 from zsxwing/SPARK-4608 and squashes the following commits: aa6d44a [zsxwing] Fix a copy-paste error f74c190 [zsxwing] Merge branch 'master' into SPARK-4608 e6f9cc9 [zsxwing] Update the docs 27833bb [zsxwing] Remove `import StreamingContext._` c15162c [zsxwing] Reorganize StreamingContext implicit to improve API convenience --- docs/streaming-programming-guide.md | 4 +-- .../examples/streaming/KafkaWordCount.scala | 1 - .../examples/streaming/CustomReceiver.scala | 1 - .../examples/streaming/HdfsWordCount.scala | 1 - .../examples/streaming/MQTTWordCount.scala | 1 - .../examples/streaming/NetworkWordCount.scala | 1 - .../examples/streaming/QueueStream.scala | 1 - .../RecoverableNetworkWordCount.scala | 1 - .../streaming/StatefulNetworkWordCount.scala | 1 - .../streaming/TwitterAlgebirdCMS.scala | 1 - .../streaming/TwitterPopularTags.scala | 1 - .../examples/streaming/ZeroMQWordCount.scala | 1 - .../clickstream/PageViewStream.scala | 1 - .../mllib/clustering/StreamingKMeans.scala | 1 - .../regression/StreamingLinearAlgorithm.scala | 1 - .../spark/streaming/StreamingContext.scala | 7 ++-- .../streaming/api/java/JavaPairDStream.scala | 3 +- .../spark/streaming/dstream/DStream.scala | 22 ++++++++---- .../dstream/PairDStreamFunctions.scala | 4 +-- .../dstream/ReducedWindowedDStream.scala | 2 -- .../org/apache/spark/streaming/package.scala | 2 +- .../streaming/BasicOperationsSuite.scala | 1 - .../spark/streaming/CheckpointSuite.scala | 1 - .../spark/streaming/MasterFailureTest.scala | 1 - .../streaming/WindowOperationsSuite.scala | 1 - .../spark/streamingtest/ImplicitSuite.scala | 35 +++++++++++++++++++ 26 files changed, 60 insertions(+), 37 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 1ac5b9e863ad4..01450efe35e55 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -75,7 +75,7 @@ main entry point for all streaming functionality. We create a local StreamingCon {% highlight scala %} import org.apache.spark._ import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+ // Create a local StreamingContext with two working thread and batch interval of 1 second. // The master requires 2 cores to prevent from a starvation scenario. @@ -107,7 +107,7 @@ each line will be split into multiple words and the stream of words is represent `words` DStream. Next, we want to count these words. {% highlight scala %} -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+ // Count each word in each batch val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index c9e1511278ede..2adc63f7ff30e 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -22,7 +22,6 @@ import java.util.Properties import kafka.producer._ import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.kafka._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 6bb659fbd8be8..30269a7ccae97 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -23,7 +23,6 @@ import java.net.Socket import org.apache.spark.{SparkConf, Logging} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.receiver.Receiver /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 6c24bc3ad09e0..4b4667fec44e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ /** * Counts words in new text files created in the given directory diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index e4283e04a1b11..6ff0c47793a25 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -22,7 +22,6 @@ import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.mqtt._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index ae0a08c6cdb1a..2cd8073dada14 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.storage.StorageLevel /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index 4caa90659111a..13ba9a43ec3c9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.SynchronizedQueue import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ object QueueStream { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 19427e629f76d..c3a05c89d817e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -25,7 +25,6 @@ import com.google.common.io.Files import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.util.IntParam /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index ed186ea5650c4..345d0bc441351 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -20,7 +20,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.HashPartitioner import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ /** * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index 683752ac96241..62f49530edb12 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -23,7 +23,6 @@ import org.apache.spark.SparkConf import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.twitter._ // scalastyle:off diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index f55d23ab3924b..f253d75b279f7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -18,7 +18,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} -import StreamingContext._ import org.apache.spark.SparkContext._ import org.apache.spark.streaming.twitter._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 79905af381a12..6510c70bd1866 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -24,7 +24,6 @@ import akka.zeromq.Subscribe import akka.util.ByteString import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.zeromq._ import scala.language.implicitConversions diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index 55226c0a6df60..fbacaee98690f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming.clickstream import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.examples.streaming.StreamingExamples // scalastyle:off /** Analyses a streaming dataset of web page views. This class demonstrates several types of diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index 6189dce9b27da..7752c1988fdd1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -24,7 +24,6 @@ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index 8db0442a7a569..b549b7c475fc3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -22,7 +22,6 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream /** 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 ecab5510a8e7b..8ef0787137845 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.Map import scala.collection.mutable.Queue -import scala.language.implicitConversions import scala.reflect.ClassTag import akka.actor.{Props, SupervisorStrategy} @@ -523,9 +522,11 @@ object StreamingContext extends Logging { private[streaming] val DEFAULT_CLEANER_TTL = 3600 - implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) + @deprecated("Replaced by implicit functions in the DStream companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { - new PairDStreamFunctions[K, V](stream) + DStream.toPairDStreamFunctions(stream)(kt, vt, ord) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index bb44b906d7386..de124cf40eff1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -36,7 +36,6 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream /** @@ -815,6 +814,6 @@ object JavaPairDStream { def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long]) : JavaPairDStream[K, JLong] = { - StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) + DStream.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index dbf1ebbaf653a..7f8651e719d84 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -20,8 +20,8 @@ package org.apache.spark.streaming.dstream import java.io.{IOException, ObjectInputStream, ObjectOutputStream} -import scala.deprecated import scala.collection.mutable.HashMap +import scala.language.implicitConversions import scala.reflect.ClassTag import scala.util.matching.Regex @@ -29,7 +29,7 @@ import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext.rddToFileName import org.apache.spark.streaming.scheduler.Job import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} @@ -48,8 +48,7 @@ import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} * `window`. In addition, [[org.apache.spark.streaming.dstream.PairDStreamFunctions]] contains * operations available only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and * `join`. These operations are automatically available on any DStream of pairs - * (e.g., DStream[(Int, Int)] through implicit conversions when - * `org.apache.spark.streaming.StreamingContext._` is imported. + * (e.g., DStream[(Int, Int)] through implicit conversions. * * DStreams internally is characterized by a few basic properties: * - A list of other DStreams that the DStream depends on @@ -802,10 +801,21 @@ abstract class DStream[T: ClassTag] ( } } -private[streaming] object DStream { +object DStream { + + // `toPairDStreamFunctions` was in SparkContext before 1.3 and users had to + // `import StreamingContext._` to enable it. Now we move it here to make the compiler find + // it automatically. However, we still keep the old function in StreamingContext for backward + // compatibility and forward to the following function directly. + + implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null): + PairDStreamFunctions[K, V] = { + new PairDStreamFunctions[K, V](stream) + } /** Get the creation site of a DStream from the stack trace of when the DStream is created. */ - def getCreationSite(): CallSite = { + private[streaming] def getCreationSite(): CallSite = { val SPARK_CLASS_REGEX = """^org\.apache\.spark""".r val SPARK_STREAMING_TESTCLASS_REGEX = """^org\.apache\.spark\.streaming\.test""".r val SPARK_EXAMPLES_CLASS_REGEX = """^org\.apache\.spark\.examples""".r diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 98539e06b4e29..8a58571632447 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -27,12 +27,10 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner, SerializableWritable} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext.rddToFileName /** * Extra functions available on DStream of (key, value) pairs through an implicit conversion. - * Import `org.apache.spark.streaming.StreamingContext._` at the top of your program to use - * these functions. */ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index 1a47089e513c4..c0a5af0b65cc3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -17,8 +17,6 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.StreamingContext._ - import org.apache.spark.rdd.RDD import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD} import org.apache.spark.Partitioner diff --git a/streaming/src/main/scala/org/apache/spark/streaming/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/package.scala index 4dd985cf5a178..2153ae0d34184 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/package.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/package.scala @@ -26,7 +26,7 @@ package org.apache.spark * available only on DStreams * of key-value pairs, such as `groupByKey` and `reduceByKey`. These operations are automatically * available on any DStream of the right type (e.g. DStream[(Int, Int)] through implicit - * conversions when you `import org.apache.spark.streaming.StreamingContext._`. + * conversions. * * For the Java API of Spark Streaming, take a look at the * [[org.apache.spark.streaming.api.java.JavaStreamingContext]] which serves as the entry point, and diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 86b96785d7b87..199f5e7161124 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, WindowedDStream} import org.apache.spark.HashPartitioner diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index c97998add8ffa..72d055eb2ea31 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -30,7 +30,6 @@ import org.apache.hadoop.io.{IntWritable, Text} import org.apache.hadoop.mapred.TextOutputFormat import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 5dbb7232009eb..e0f14fd954280 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import org.apache.spark.Logging import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -import org.apache.spark.streaming.StreamingContext._ import scala.util.Random import scala.collection.mutable.ArrayBuffer diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index 471c99fab4682..a5d2bb2fde16c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream import org.apache.spark.storage.StorageLevel diff --git a/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala new file mode 100644 index 0000000000000..d0bf328f2b74d --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala @@ -0,0 +1,35 @@ +/* + * 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.streamingtest + +/** + * A test suite to make sure all `implicit` functions work correctly. + * + * As `implicit` is a compiler feature, we don't need to run this class. + * What we need to do is making the compiler happy. + */ +class ImplicitSuite { + + // We only want to test if `implict` works well with the compiler, so we don't need a real DStream. + def mockDStream[T]: org.apache.spark.streaming.dstream.DStream[T] = null + + def testToPairDStreamFunctions(): Unit = { + val dstream: org.apache.spark.streaming.dstream.DStream[(Int, Int)] = mockDStream + dstream.groupByKey() + } +} From fda4331d58aae454a2b0f4c757d105f8bc228a3d Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 26 Dec 2014 12:03:22 -0800 Subject: [PATCH 127/227] SPARK-4971: Fix typo in BlockGenerator comment Author: CodingCat Closes #3807 from CodingCat/new_branch and squashes the following commits: 5167f01 [CodingCat] fix typo in the comment --- .../org/apache/spark/streaming/receiver/BlockGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 55765dc90698b..79263a7183977 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -116,7 +116,7 @@ private[streaming] class BlockGenerator( /** * Push a single data item into the buffer. After buffering the data, the - * `BlockGeneratorListnere.onAddData` callback will be called. All received data items + * `BlockGeneratorListener.onAddData` callback will be called. All received data items * will be periodically pushed into BlockManager. */ def addDataWithCallback(data: Any, metadata: Any) = synchronized { From 534f24b2d094eaac8554f3e4c8c8addd40c62441 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 26 Dec 2014 22:39:56 -0800 Subject: [PATCH 128/227] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #3456 (close requested by 'pwendell') Closes #1602 (close requested by 'tdas') Closes #2633 (close requested by 'tdas') Closes #2059 (close requested by 'JoshRosen') Closes #2348 (close requested by 'tdas') Closes #3662 (close requested by 'tdas') Closes #2031 (close requested by 'andrewor14') Closes #265 (close requested by 'JoshRosen') From de95c57ac6fcb2a62cf5083feb3cc79ab18a836f Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 26 Dec 2014 22:52:04 -0800 Subject: [PATCH 129/227] [SPARK-3787][BUILD] Assembly jar name is wrong when we build with sbt omitting -Dhadoop.version This PR is another solution for When we build with sbt with profile for hadoop and without property for hadoop version like: sbt/sbt -Phadoop-2.2 assembly jar name is always used default version (1.0.4). When we build with maven with same condition for sbt, default version for each profile is used. For instance, if we build like: mvn -Phadoop-2.2 package jar name is used hadoop2.2.0 as a default version of hadoop-2.2. Author: Kousuke Saruta Closes #3046 from sarutak/fix-assembly-jarname-2 and squashes the following commits: 41ef90e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-assembly-jarname-2 50c8676 [Kousuke Saruta] Merge branch 'fix-assembly-jarname-2' of github.com:sarutak/spark into fix-assembly-jarname-2 52a1cd2 [Kousuke Saruta] Fixed comflicts dd30768 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-assembly-jarname2 f1c90bb [Kousuke Saruta] Fixed SparkBuild.scala in order to read `hadoop.version` property from pom.xml af6b100 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-assembly-jarname c81806b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-assembly-jarname ad1f96e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-assembly-jarname b2318eb [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-assembly-jarname 5fc1259 [Kousuke Saruta] Fixed typo. eebbb7d [Kousuke Saruta] Fixed wrong jar name --- project/SparkBuild.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f52074282e1f7..4d103f9dc0e2c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -15,6 +15,8 @@ * limitations under the License. */ +import java.io.File + import scala.util.Properties import scala.collection.JavaConversions._ @@ -23,7 +25,7 @@ import sbt.Classpaths.publishTask import sbt.Keys._ import sbtunidoc.Plugin.genjavadocSettings import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion -import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} +import com.typesafe.sbt.pom.{loadEffectivePom, PomBuild, SbtPomKeys} import net.virtualvoid.sbt.graph.Plugin.graphSettings object BuildCommons { @@ -112,6 +114,15 @@ object SparkBuild extends PomBuild { override val userPropertiesMap = System.getProperties.toMap + val pom = loadEffectivePom(new File("pom.xml"), + profiles = profiles, + userProps = userPropertiesMap) + + if (System.getProperty("hadoop.version") == null) { + System.setProperty("hadoop.version", + pom.getProperties.get("hadoop.version").asInstanceOf[String]) + } + lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") @@ -297,8 +308,7 @@ object Assembly { // This must match the same name used in maven (see network/yarn/pom.xml) "spark-" + v + "-yarn-shuffle.jar" } else { - mName + "-" + v + "-hadoop" + - Option(System.getProperty("hadoop.version")).getOrElse("1.0.4") + ".jar" + mName + "-" + v + "-hadoop" + System.getProperty("hadoop.version") + ".jar" } }, mergeStrategy in assembly := { From 82bf4bee15bb449a418290c0c07652e9a8ec8fdd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 26 Dec 2014 22:55:04 -0800 Subject: [PATCH 130/227] HOTFIX: Slight tweak on previous commit. Meant to merge this in when committing SPARK-3787. --- project/SparkBuild.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4d103f9dc0e2c..c512b62f6137e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -114,10 +114,12 @@ object SparkBuild extends PomBuild { override val userPropertiesMap = System.getProperties.toMap + // Handle case where hadoop.version is set via profile. + // Needed only because we read back this property in sbt + // when we create the assembly jar. val pom = loadEffectivePom(new File("pom.xml"), profiles = profiles, userProps = userPropertiesMap) - if (System.getProperty("hadoop.version") == null) { System.setProperty("hadoop.version", pom.getProperties.get("hadoop.version").asInstanceOf[String]) From 2483c1efb6429a7d8a20c96d18ce2fec93a1aff9 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Fri, 26 Dec 2014 22:59:34 -0800 Subject: [PATCH 131/227] [SPARK-3955] Different versions between jackson-mapper-asl and jackson-c... ...ore-asl - set the same version to jackson-mapper-asl and jackson-core-asl - It's related with #2818 - coded a same patch from a latest master Author: Jongyoul Lee Closes #3716 from jongyoul/SPARK-3955 and squashes the following commits: efa29aa [Jongyoul Lee] [SPARK-3955] Different versions between jackson-mapper-asl and jackson-core-asl - set the same version to jackson-mapper-asl and jackson-core-asl --- pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index e4db1393ba9cf..a843af2b22d6c 100644 --- a/pom.xml +++ b/pom.xml @@ -149,6 +149,7 @@ 2.10 ${scala.version} org.scala-lang + 1.8.8 @@ -819,10 +820,15 @@ - + org.codehaus.jackson jackson-mapper-asl - 1.8.8 + ${jackson.version} + + + org.codehaus.jackson + jackson-mapper-asl + ${jackson.version} From 786808abfd6ca8c8d3a2331d1be49c1466006a46 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 26 Dec 2014 23:23:13 -0800 Subject: [PATCH 132/227] [SPARK-4954][Core] add spark version infomation in log for standalone mode The master and worker spark version may be not the same with Driver spark version. That is because spark Jar file might be replaced for new application without restarting the spark cluster. So there shall log out the spark-version in both Mater and Worker log. Author: Zhang, Liye Closes #3790 from liyezhang556520/version4Standalone and squashes the following commits: e05e1e3 [Zhang, Liye] add spark version infomation in log for standalone mode --- core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 1 + core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 1 + 2 files changed, 2 insertions(+) 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 81fa0770bbaf9..e8a5cfc746fed 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 @@ -123,6 +123,7 @@ private[spark] class Master( override def preStart() { logInfo("Starting Spark master at " + masterUrl) + logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.bind() 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 86a87ec22235e..f0f3da5eec4df 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 @@ -155,6 +155,7 @@ private[spark] class Worker( assert(!registered) logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format( host, port, cores, Utils.megabytesToString(memory))) + logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") logInfo("Spark home: " + sparkHome) createWorkDir() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) From 080ceb771a1e6b9f844cfd4f1baa01133c106888 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Fri, 26 Dec 2014 23:31:29 -0800 Subject: [PATCH 133/227] [SPARK-4952][Core]Handle ConcurrentModificationExceptions in SparkEnv.environmentDetails Author: GuoQiang Li Closes #3788 from witgo/SPARK-4952 and squashes the following commits: d903529 [GuoQiang Li] Handle ConcurrentModificationExceptions in SparkEnv.environmentDetails --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6656df44d3599..43436a1697000 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -395,7 +395,7 @@ object SparkEnv extends Logging { val sparkProperties = (conf.getAll ++ schedulerMode).sorted // System properties that are not java classpaths - val systemProperties = System.getProperties.iterator.toSeq + val systemProperties = Utils.getSystemProperties.toSeq val otherProperties = systemProperties.filter { case (k, _) => k != "java.class.path" && !k.startsWith("spark.") }.sorted From a3e51cc990812c8099dcaf1f3bd6d5bae45cf8e6 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Sat, 27 Dec 2014 13:25:18 -0800 Subject: [PATCH 134/227] [SPARK-4501][Core] - Create build/mvn to automatically download maven/zinc/scalac Creates a top level directory script (as `build/mvn`) to automatically download zinc and the specific version of scala used to easily build spark. This will also download and install maven if the user doesn't already have it and all packages are hosted under the `build/` directory. Tested on both Linux and OSX OS's and both work. All commands pass through to the maven binary so it acts exactly as a traditional maven call would. Author: Brennon York Closes #3707 from brennonyork/SPARK-4501 and squashes the following commits: 0e5a0e4 [Brennon York] minor incorrect doc verbage (with -> this) 9b79e38 [Brennon York] fixed merge conflicts with dev/run-tests, properly quoted args in sbt/sbt, fixed bug where relative paths would fail if passed in from build/mvn d2d41b6 [Brennon York] added blurb about leverging zinc with build/mvn b979c58 [Brennon York] updated the merge conflict c5634de [Brennon York] updated documentation to overview build/mvn, updated all points where sbt/sbt was referenced with build/sbt b8437ba [Brennon York] set progress bars for curl and wget when not run on jenkins, no progress bar when run on jenkins, moved sbt script to build/sbt, wrote stub and warning under sbt/sbt which calls build/sbt, modified build/sbt to use the correct directory, fixed bug in build/sbt-launch-lib.bash to correctly pull the sbt version be11317 [Brennon York] added switch to silence download progress only if AMPLAB_JENKINS is set 28d0a99 [Brennon York] updated to remove the python dependency, uses grep instead 7e785a6 [Brennon York] added silent and quiet flags to curl and wget respectively, added single echo output to denote start of a download if download is needed 14a5da0 [Brennon York] removed unnecessary zinc output on startup 1af4a94 [Brennon York] fixed bug with uppercase vs lowercase variable 3e8b9b3 [Brennon York] updated to properly only restart zinc if it was freshly installed a680d12 [Brennon York] Added comments to functions and tested various mvn calls bb8cc9d [Brennon York] removed package files ef017e6 [Brennon York] removed OS complexities, setup generic install_app call, removed extra file complexities, removed help, removed forced install (defaults now), removed double-dash from cli 07bf018 [Brennon York] Updated to specifically handle pulling down the correct scala version f914dea [Brennon York] Beginning final portions of localized scala home 69c4e44 [Brennon York] working linux and osx installers for purely local mvn build 4a1609c [Brennon York] finalizing working linux install for maven to local ./build/apache-maven folder cbfcc68 [Brennon York] Changed the default sbt/sbt to build/sbt and added a build/mvn which will automatically download, install, and execute maven with zinc for easier build capability --- .gitignore | 7 +- build/mvn | 132 +++++++++++++++++++++++ build/sbt | 111 +++++++++++++++++++ {sbt => build}/sbt-launch-lib.bash | 6 +- dev/create-release/create-release.sh | 10 +- dev/mima | 8 +- dev/run-tests | 24 ++--- dev/scalastyle | 4 +- docs/README.md | 6 +- docs/_plugins/copy_api_dirs.rb | 4 +- docs/building-spark.md | 45 +++++--- docs/hadoop-third-party-distributions.md | 10 +- extras/java8-tests/README.md | 6 +- python/pyspark/sql.py | 2 +- sbt/sbt | 117 ++------------------ sql/README.md | 4 +- 16 files changed, 330 insertions(+), 166 deletions(-) create mode 100755 build/mvn create mode 100755 build/sbt rename {sbt => build}/sbt-launch-lib.bash (96%) diff --git a/.gitignore b/.gitignore index 20095dd97343e..9757054a50f9e 100644 --- a/.gitignore +++ b/.gitignore @@ -8,16 +8,19 @@ *.pyc .idea/ .idea_modules/ -sbt/*.jar +build/*.jar .settings .cache +cache .generated-mima* -/build/ work/ out/ .DS_Store third_party/libmesos.so third_party/libmesos.dylib +build/apache-maven* +build/zinc* +build/scala* conf/java-opts conf/*.sh conf/*.cmd diff --git a/build/mvn b/build/mvn new file mode 100755 index 0000000000000..dde486a8ac605 --- /dev/null +++ b/build/mvn @@ -0,0 +1,132 @@ +#!/usr/bin/env bash + +# Determine the current working directory +_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +# Preserve the calling directory +_CALLING_DIR="$(pwd)" + +# Installs any application tarball given a URL, the expected tarball name, +# and, optionally, a checkable binary path to determine if the binary has +# already been installed +## Arg1 - URL +## Arg2 - Tarball Name +## Arg3 - Checkable Binary +install_app() { + local remote_tarball="$1/$2" + local local_tarball="${_DIR}/$2" + local binary="${_DIR}/$3" + + # setup `curl` and `wget` silent options if we're running on Jenkins + local curl_opts="" + local wget_opts="" + if [ -n "$AMPLAB_JENKINS" ]; then + curl_opts="-s" + wget_opts="--quiet" + else + curl_opts="--progress-bar" + wget_opts="--progress=bar:force" + fi + + if [ -z "$3" -o ! -f "$binary" ]; then + # check if we already have the tarball + # check if we have curl installed + # download application + [ ! -f "${local_tarball}" ] && [ -n "`which curl 2>/dev/null`" ] && \ + echo "exec: curl ${curl_opts} ${remote_tarball}" && \ + curl ${curl_opts} "${remote_tarball}" > "${local_tarball}" + # if the file still doesn't exist, lets try `wget` and cross our fingers + [ ! -f "${local_tarball}" ] && [ -n "`which wget 2>/dev/null`" ] && \ + echo "exec: wget ${wget_opts} ${remote_tarball}" && \ + wget ${wget_opts} -O "${local_tarball}" "${remote_tarball}" + # if both were unsuccessful, exit + [ ! -f "${local_tarball}" ] && \ + echo -n "ERROR: Cannot download $2 with cURL or wget; " && \ + echo "please install manually and try again." && \ + exit 2 + cd "${_DIR}" && tar -xzf "$2" + rm -rf "$local_tarball" + fi +} + +# Install maven under the build/ folder +install_mvn() { + install_app \ + "http://apache.claz.org/maven/maven-3/3.2.3/binaries" \ + "apache-maven-3.2.3-bin.tar.gz" \ + "apache-maven-3.2.3/bin/mvn" + MVN_BIN="${_DIR}/apache-maven-3.2.3/bin/mvn" +} + +# Install zinc under the build/ folder +install_zinc() { + local zinc_path="zinc-0.3.5.3/bin/zinc" + [ ! -f "${zinc_path}" ] && ZINC_INSTALL_FLAG=1 + install_app \ + "http://downloads.typesafe.com/zinc/0.3.5.3" \ + "zinc-0.3.5.3.tgz" \ + "${zinc_path}" + ZINC_BIN="${_DIR}/${zinc_path}" +} + +# Determine the Scala version from the root pom.xml file, set the Scala URL, +# and, with that, download the specific version of Scala necessary under +# the build/ folder +install_scala() { + # determine the Scala version used in Spark + local scala_version=`grep "scala.version" "${_DIR}/../pom.xml" | \ + head -1 | cut -f2 -d'>' | cut -f1 -d'<'` + local scala_bin="${_DIR}/scala-${scala_version}/bin/scala" + + install_app \ + "http://downloads.typesafe.com/scala/${scala_version}" \ + "scala-${scala_version}.tgz" \ + "scala-${scala_version}/bin/scala" + + SCALA_COMPILER="$(cd "$(dirname ${scala_bin})/../lib" && pwd)/scala-compiler.jar" + SCALA_LIBRARY="$(cd "$(dirname ${scala_bin})/../lib" && pwd)/scala-library.jar" +} + +# Determines if a given application is already installed. If not, will attempt +# to install +## Arg1 - application name +## Arg2 - Alternate path to local install under build/ dir +check_and_install_app() { + # create the local environment variable in uppercase + local app_bin="`echo $1 | awk '{print toupper(\$0)}'`_BIN" + # some black magic to set the generated app variable (i.e. MVN_BIN) into the + # environment + eval "${app_bin}=`which $1 2>/dev/null`" + + if [ -z "`which $1 2>/dev/null`" ]; then + install_$1 + fi +} + +# Setup healthy defaults for the Zinc port if none were provided from +# the environment +ZINC_PORT=${ZINC_PORT:-"3030"} + +# Check and install all applications necessary to build Spark +check_and_install_app "mvn" + +# Install the proper version of Scala and Zinc for the build +install_zinc +install_scala + +# Reset the current working directory +cd "${_CALLING_DIR}" + +# Now that zinc is ensured to be installed, check its status and, if its +# not running or just installed, start it +if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`${ZINC_BIN} -status`" ]; then + ${ZINC_BIN} -shutdown + ${ZINC_BIN} -start -port ${ZINC_PORT} \ + -scala-compiler "${SCALA_COMPILER}" \ + -scala-library "${SCALA_LIBRARY}" &>/dev/null +fi + +# Set any `mvn` options if not already present +export MAVEN_OPTS=${MAVEN_OPTS:-"-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"} + +# Last, call the `mvn` command as usual +${MVN_BIN} "$@" diff --git a/build/sbt b/build/sbt new file mode 100755 index 0000000000000..0a251d97db95c --- /dev/null +++ b/build/sbt @@ -0,0 +1,111 @@ +#!/usr/bin/env bash + +# 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" +done +export HADOOP_CLASSPATH + +realpath () { +( + TARGET_FILE="$1" + + 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)" + COUNT=$(($COUNT + 1)) + done + + echo "$(pwd -P)/"$TARGET_FILE"" +) +} + +. "$(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" +declare -r sbt_opts_file=".sbtopts" +declare -r etc_sbt_opts_file="/etc/sbt/sbtopts" + +usage() { + cat < path to global settings/plugins directory (default: ~/.sbt) + -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series) + -ivy path to local Ivy repository (default: ~/.ivy2) + -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem)) + -no-share use all local caches; no sharing + -no-global uses global caches, but does not use global ~/.sbt directory. + -jvm-debug Turn on JVM debugging, open at the given port. + -batch Disable interactive mode + + # sbt version (default: from project/build.properties if present, else latest release) + -sbt-version use the specified version of sbt + -sbt-jar use the specified jar as the sbt launcher + -sbt-rc use an RC version of sbt + -sbt-snapshot use a snapshot version of sbt + + # java version (default: java from PATH, currently $(java -version 2>&1 | grep version)) + -java-home alternate JAVA_HOME + + # jvm options and output control + JAVA_OPTS environment variable, if unset uses "$java_opts" + SBT_OPTS environment variable, if unset uses "$default_sbt_opts" + .sbtopts if this file exists in the current directory, it is + prepended to the runner args + /etc/sbt/sbtopts if this file exists, it is prepended to the runner args + -Dkey=val pass -Dkey=val directly to the java runtime + -J-X pass option -X directly to the java runtime + (-J is stripped) + -S-X add -X to sbt's scalacOptions (-S is stripped) + -PmavenProfiles Enable a maven profile for the build. + +In the case of duplicated or conflicting options, the order above +shows precedence: JAVA_OPTS lowest, command line options highest. +EOM +} + +process_my_args () { + while [[ $# -gt 0 ]]; do + case "$1" in + -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; + -no-share) addJava "$noshare_opts" && shift ;; + -no-global) addJava "-Dsbt.global.base=$(pwd)/project/.sbtboot" && shift ;; + -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; + -sbt-dir) require_arg path "$1" "$2" && addJava "-Dsbt.global.base=$2" && shift 2 ;; + -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; + -batch) exec &1 \ | grep -e "^java version" --max-count=1 \ | sed "s/java version \"\(.*\)\.\(.*\)\.\(.*\)\"/\1\2/" ) - + if [ "$JAVA_VERSION" -lt 18 ]; then echo "[warn] Java 8 tests will not run because JDK version is < 1.8." fi @@ -79,7 +79,7 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" # Partial solution for SPARK-1455. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - + sql_diffs=$( git diff --name-only master \ | grep -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh" @@ -93,7 +93,7 @@ if [ -n "$AMPLAB_JENKINS" ]; then if [ -n "$sql_diffs" ]; then echo "[info] Detected changes in SQL. Will run Hive test suite." _RUN_SQL_TESTS=true - + if [ -z "$non_sql_diffs" ]; then echo "[info] Detected no changes except in SQL. Will only run SQL tests." _SQL_TESTS_ONLY=true @@ -151,7 +151,7 @@ CURRENT_BLOCK=$BLOCK_BUILD HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" echo "[info] Compile with Hive 0.12.0" echo -e "q\n" \ - | sbt/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ + | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" # Then build with default Hive version (0.13.1) because tests are based on this version @@ -160,7 +160,7 @@ CURRENT_BLOCK=$BLOCK_BUILD echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ " -Phive -Phive-thriftserver" echo -e "q\n" \ - | sbt/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } @@ -177,7 +177,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_RUN_SQL_TESTS" ]; then SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" fi - + if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string # will be interpreted as a single test, which doesn't work. @@ -185,19 +185,19 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS else SBT_MAVEN_TEST_ARGS=("test") fi - + echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" - + # NOTE: 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. - # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a # single argument! # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ - | sbt/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } diff --git a/dev/scalastyle b/dev/scalastyle index 3a4df6e4bf1bc..86919227ed1ab 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,9 +17,9 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | build/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') diff --git a/docs/README.md b/docs/README.md index 119484038083f..8a54724c4beae 100644 --- a/docs/README.md +++ b/docs/README.md @@ -21,7 +21,7 @@ read those text files directly if you want. Start with index.md. The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). `Jekyll` and a few dependencies must be installed for this to work. We recommend -installing via the Ruby Gem dependency manager. Since the exact HTML output +installing via the Ruby Gem dependency manager. Since the exact HTML output varies between versions of Jekyll and its dependencies, we list specific versions here in some cases: @@ -60,7 +60,7 @@ We use Sphinx to generate Python API docs, so you will need to install it by run ## API Docs (Scaladoc and Sphinx) -You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. +You can build just the Spark scaladoc by running `build/sbt doc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark docs by running `make html` from the SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as @@ -68,7 +68,7 @@ public in `__init__.py`. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a -jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it +jekyll plugin to run `build/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs [Sphinx](http://sphinx-doc.org/). diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 4566a2fff562b..3c626a0b7f54b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -25,8 +25,8 @@ curr_dir = pwd cd("..") - puts "Running 'sbt/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt -Pkinesis-asl compile unidoc` + puts "Running 'build/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `build/sbt -Pkinesis-asl compile unidoc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/building-spark.md b/docs/building-spark.md index dab3d2aef497e..c1bcd91b5b853 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -9,6 +9,15 @@ redirect_from: "building-with-maven.html" Building Spark using Maven requires Maven 3.0.4 or newer and Java 6+. +# Building with `build/mvn` + +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: + +{% highlight bash %} +build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package +{% endhighlight %} + +Other build examples can be found below. # Setting up Maven's Memory Usage @@ -28,7 +37,9 @@ If you don't run this, you may see errors like the following: You can fix this by setting the `MAVEN_OPTS` variable as discussed before. -**Note:** *For Java 8 and above this step is not required.* +**Note:** +* *For Java 8 and above this step is not required.* +* *If using `build/mvn` and `MAVEN_OPTS` were not already set, the script will automate this for you.* # Specifying the Hadoop Version @@ -84,7 +95,7 @@ mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests # Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. -By default Spark will build with Hive 0.13.1 bindings. You can also build for +By default Spark will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using the `-Phive-0.12.0` profile. {% highlight bash %} # Apache Hadoop 2.4.X with Hive 13 support @@ -106,7 +117,7 @@ supported in Scala 2.11 builds. # Spark Tests in Maven -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: @@ -124,7 +135,7 @@ We use the scala-maven-plugin which supports incremental and continuous compilat mvn scala:cc -should run continuous compilation (i.e. wait for changes). However, this has not been tested +should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: * it only scans the paths `src/main` and `src/test` (see [docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work @@ -157,9 +168,9 @@ The debian package can then be found under assembly/target. We added the short c Running only Java 8 tests and nothing else. mvn install -DskipTests -Pjava8-tests - -Java 8 tests are run when `-Pjava8-tests` profile is enabled, they will run in spite of `-DskipTests`. -For these tests to run your system must have a JDK 8 installation. + +Java 8 tests are run when `-Pjava8-tests` profile is enabled, they will run in spite of `-DskipTests`. +For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. # Building for PySpark on YARN @@ -171,7 +182,7 @@ then ship it over to the cluster. We are investigating the exact cause for this. # Packaging without Hadoop Dependencies for YARN -The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. +The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. # Building with SBT @@ -182,22 +193,22 @@ compilation. More advanced developers may wish to use SBT. The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables can be set to control the SBT build. For example: - sbt/sbt -Pyarn -Phadoop-2.3 assembly + build/sbt -Pyarn -Phadoop-2.3 assembly # Testing with SBT -Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: +Some of the tests require Spark to be packaged first, so always run `build/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" To run test suites of a specific sub project as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test # Speeding up Compilation with Zinc @@ -206,3 +217,9 @@ compiler. When run locally as a background process, it speeds up builds of Scala like Spark. Developers who regularly recompile Spark with Maven will be the most interested in Zinc. The project site gives instructions for building and running `zinc`; OS X users can install it using `brew install zinc`. + +If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all +builds. This process will auto-start after the first time `build/mvn` is called and bind to port +3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be +shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically +restart whenever `build/mvn` is called. diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index dd73e9dc54440..87dcc58feb494 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -18,7 +18,7 @@ see the guide on [building with maven](building-spark.html#specifying-the-hadoop The table below lists the corresponding `hadoop.version` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark -distribution may "just work" without you needing to compile. That said, we recommend compiling with +distribution may "just work" without you needing to compile. That said, we recommend compiling with the _exact_ Hadoop version you are running to avoid any compatibility errors. @@ -50,7 +50,7 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors. In SBT, the equivalent can be achieved by setting the the `hadoop.version` property: - sbt/sbt -Dhadoop.version=1.0.4 assembly + build/sbt -Dhadoop.version=1.0.4 assembly # Linking Applications to the Hadoop Version @@ -98,11 +98,11 @@ Spark can run in a variety of deployment modes: * Using dedicated set of Spark nodes in your cluster. These nodes should be co-located with your Hadoop installation. -* Running on the same nodes as an existing Hadoop installation, with a fixed amount memory and +* Running on the same nodes as an existing Hadoop installation, with a fixed amount memory and cores dedicated to Spark on each node. * Run Spark alongside Hadoop using a cluster resource manager, such as YARN or Mesos. -These options are identical for those using CDH and HDP. +These options are identical for those using CDH and HDP. # Inheriting Cluster Configuration @@ -116,5 +116,5 @@ The location of these configuration files varies across CDH and HDP versions, bu a common location is inside of `/etc/hadoop/conf`. Some tools, such as Cloudera Manager, create configurations on-the-fly, but offer a mechanisms to download copies of them. -To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` +To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` to a location containing the configuration files. diff --git a/extras/java8-tests/README.md b/extras/java8-tests/README.md index e95b73ac7702a..dc9e87f2eeb92 100644 --- a/extras/java8-tests/README.md +++ b/extras/java8-tests/README.md @@ -8,7 +8,7 @@ to your Java location. The set-up depends a bit on the build system: `-java-home` to the sbt launch script. If a Java 8 JDK is detected sbt will automatically include the Java 8 test project. - `$ JAVA_HOME=/opt/jdk1.8.0/ sbt/sbt clean "test-only org.apache.spark.Java8APISuite"` + `$ JAVA_HOME=/opt/jdk1.8.0/ build/sbt clean "test-only org.apache.spark.Java8APISuite"` * For Maven users, @@ -19,6 +19,6 @@ to your Java location. The set-up depends a bit on the build system: `$ JAVA_HOME=/opt/jdk1.8.0/ mvn clean install -DskipTests` `$ JAVA_HOME=/opt/jdk1.8.0/ mvn test -Pjava8-tests -DwildcardSuites=org.apache.spark.Java8APISuite` - Note that the above command can only be run from project root directory since this module - depends on core and the test-jars of core and streaming. This means an install step is + Note that the above command can only be run from project root directory since this module + depends on core and the test-jars of core and streaming. This means an install step is required to make the test dependencies visible to the Java 8 sub-project. diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 9807a84a66f11..0e8b398fc6b97 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1671,7 +1671,7 @@ def _ssql_ctx(self): except Py4JError as e: raise Exception("You must build Spark with Hive. " "Export 'SPARK_HIVE=true' and run " - "sbt/sbt assembly", e) + "build/sbt assembly", e) def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) diff --git a/sbt/sbt b/sbt/sbt index 0a251d97db95c..6f3e5e08ed27a 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,111 +1,12 @@ -#!/usr/bin/env bash +#!/bin/bash -# 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" -done -export HADOOP_CLASSPATH +# Determine the current working directory +_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" -realpath () { -( - TARGET_FILE="$1" +echo "NOTE: The sbt/sbt script has been relocated to build/sbt." >&2 +echo " Please update references to point to the new location." >&2 +echo "" >&2 +echo " Invoking 'build/sbt $@' now ..." >&2 +echo "" >&2 - 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)" - COUNT=$(($COUNT + 1)) - done - - echo "$(pwd -P)/"$TARGET_FILE"" -) -} - -. "$(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" -declare -r sbt_opts_file=".sbtopts" -declare -r etc_sbt_opts_file="/etc/sbt/sbtopts" - -usage() { - cat < path to global settings/plugins directory (default: ~/.sbt) - -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series) - -ivy path to local Ivy repository (default: ~/.ivy2) - -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem)) - -no-share use all local caches; no sharing - -no-global uses global caches, but does not use global ~/.sbt directory. - -jvm-debug Turn on JVM debugging, open at the given port. - -batch Disable interactive mode - - # sbt version (default: from project/build.properties if present, else latest release) - -sbt-version use the specified version of sbt - -sbt-jar use the specified jar as the sbt launcher - -sbt-rc use an RC version of sbt - -sbt-snapshot use a snapshot version of sbt - - # java version (default: java from PATH, currently $(java -version 2>&1 | grep version)) - -java-home alternate JAVA_HOME - - # jvm options and output control - JAVA_OPTS environment variable, if unset uses "$java_opts" - SBT_OPTS environment variable, if unset uses "$default_sbt_opts" - .sbtopts if this file exists in the current directory, it is - prepended to the runner args - /etc/sbt/sbtopts if this file exists, it is prepended to the runner args - -Dkey=val pass -Dkey=val directly to the java runtime - -J-X pass option -X directly to the java runtime - (-J is stripped) - -S-X add -X to sbt's scalacOptions (-S is stripped) - -PmavenProfiles Enable a maven profile for the build. - -In the case of duplicated or conflicting options, the order above -shows precedence: JAVA_OPTS lowest, command line options highest. -EOM -} - -process_my_args () { - while [[ $# -gt 0 ]]; do - case "$1" in - -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; - -no-share) addJava "$noshare_opts" && shift ;; - -no-global) addJava "-Dsbt.global.base=$(pwd)/project/.sbtboot" && shift ;; - -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; - -sbt-dir) require_arg path "$1" "$2" && addJava "-Dsbt.global.base=$2" && shift 2 ;; - -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; - -batch) exec Date: Mon, 29 Dec 2014 08:20:30 -0600 Subject: [PATCH 135/227] [SPARK-4966][YARN]The MemoryOverhead value is setted not correctly Author: meiyoula <1039320815@qq.com> Closes #3797 from XuTingjun/MemoryOverhead and squashes the following commits: 5a780fc [meiyoula] Update ClientArguments.scala --- .../scala/org/apache/spark/deploy/yarn/ClientArguments.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 7305249f80e83..39f1021c9d942 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -39,6 +39,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var appName: String = "Spark" var priority = 0 + parseArgs(args.toList) + // Additional memory to allocate to containers // For now, use driver's memory overhead as our AM container's memory overhead val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", @@ -50,7 +52,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) private val isDynamicAllocationEnabled = sparkConf.getBoolean("spark.dynamicAllocation.enabled", false) - parseArgs(args.toList) loadEnvironmentArgs() validateArgs() From 6645e52580747990321e22340ae742f26d2f2504 Mon Sep 17 00:00:00 2001 From: wangxiaojing Date: Mon, 29 Dec 2014 10:45:14 -0800 Subject: [PATCH 136/227] [SPARK-4982][DOC] `spark.ui.retainedJobs` description is wrong in Spark UI configuration guide Author: wangxiaojing Closes #3818 from wangxiaojing/SPARK-4982 and squashes the following commits: fe2ad5f [wangxiaojing] change stages to jobs --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 2cc013c47fdbb..fa9d311f85068 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -452,7 +452,7 @@ Apart from these, the following properties are also available, and may be useful From 4cef05e1c1d420af89164d6f4fabbad090542f1b Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 29 Dec 2014 10:48:53 -0800 Subject: [PATCH 137/227] Adde LICENSE Header to build/mvn, build/sbt and sbt/sbt Recently, build/mvn and build/sbt are added, and sbt/sbt is changed but there are no license headers. Should we add license headers to the scripts right? If it's not right, please let me correct. This PR doesn't affect behavior of Spark, I don't file in JIRA. Author: Kousuke Saruta Closes #3817 from sarutak/add-license-header and squashes the following commits: 1abc972 [Kousuke Saruta] Added LICENSE Header --- build/mvn | 17 +++++++++++++++++ build/sbt | 17 +++++++++++++++++ sbt/sbt | 19 ++++++++++++++++++- 3 files changed, 52 insertions(+), 1 deletion(-) diff --git a/build/mvn b/build/mvn index dde486a8ac605..43471f83e904c 100755 --- a/build/mvn +++ b/build/mvn @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Determine the current working directory _DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" # Preserve the calling directory diff --git a/build/sbt b/build/sbt index 0a251d97db95c..28ebb64f7197c 100755 --- a/build/sbt +++ b/build/sbt @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # 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. diff --git a/sbt/sbt b/sbt/sbt index 6f3e5e08ed27a..41438251f681e 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,4 +1,21 @@ -#!/bin/bash +#!/usr/bin/env bash + +# +# 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. +# # Determine the current working directory _DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" From 815de54002f9c1cfedc398e95896fa207b4a5305 Mon Sep 17 00:00:00 2001 From: YanTangZhai Date: Mon, 29 Dec 2014 11:30:54 -0800 Subject: [PATCH 138/227] [SPARK-4946] [CORE] Using AkkaUtils.askWithReply in MapOutputTracker.askTracker to reduce the chance of the communicating problem Using AkkaUtils.askWithReply in MapOutputTracker.askTracker to reduce the chance of the communicating problem Author: YanTangZhai Author: yantangzhai Closes #3785 from YanTangZhai/SPARK-4946 and squashes the following commits: 9ca6541 [yantangzhai] [SPARK-4946] [CORE] Using AkkaUtils.askWithReply in MapOutputTracker.askTracker to reduce the chance of the communicating problem e4c2c0a [YanTangZhai] Merge pull request #15 from apache/master 718afeb [YanTangZhai] Merge pull request #12 from apache/master 6e643f8 [YanTangZhai] Merge pull request #11 from apache/master e249846 [YanTangZhai] Merge pull request #10 from apache/master d26d982 [YanTangZhai] Merge pull request #9 from apache/master 76d4027 [YanTangZhai] Merge pull request #8 from apache/master 03b62b0 [YanTangZhai] Merge pull request #7 from apache/master 8a00106 [YanTangZhai] Merge pull request #6 from apache/master cbcba66 [YanTangZhai] Merge pull request #3 from apache/master cdef539 [YanTangZhai] Merge pull request #1 from apache/master --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a074ab8ece1b7..6e4edc7c80d7a 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -76,6 +76,8 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster */ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) + private val retryAttempts = AkkaUtils.numRetries(conf) + private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) /** Set to the MapOutputTrackerActor living on the driver. */ var trackerActor: ActorRef = _ @@ -108,8 +110,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging */ protected def askTracker(message: Any): Any = { try { - val future = trackerActor.ask(message)(timeout) - Await.result(future, timeout) + AkkaUtils.askWithReply(message, trackerActor, retryAttempts, retryIntervalMs, timeout) } catch { case e: Exception => logError("Error communicating with MapOutputTracker", e) From 8d72341ab75a7fb138b056cfb4e21db42aca55fb Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 29 Dec 2014 12:05:08 -0800 Subject: [PATCH 139/227] [Minor] Fix a typo of type parameter in JavaUtils.scala In JavaUtils.scala, thare is a typo of type parameter. In addition, the type information is removed at the time of compile by erasure. This issue is really minor so I don't file in JIRA. Author: Kousuke Saruta Closes #3789 from sarutak/fix-typo-in-javautils and squashes the following commits: e20193d [Kousuke Saruta] Fixed a typo of type parameter 82bc5d9 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fix-typo-in-javautils 99f6f63 [Kousuke Saruta] Fixed a typo of type parameter in JavaUtils.scala --- core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index 86e94931300f8..71b26737b8c02 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -80,7 +80,7 @@ private[spark] object JavaUtils { prev match { case Some(k) => underlying match { - case mm: mutable.Map[a, _] => + case mm: mutable.Map[A, _] => mm remove k prev = None case _ => From 02b55de3dce9a1fef806be13e5cefa0f39ea2fcc Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 29 Dec 2014 13:24:26 -0800 Subject: [PATCH 140/227] [SPARK-4409][MLlib] Additional Linear Algebra Utils Addition of a very limited number of local matrix manipulation and generation methods that would be helpful in the further development for algorithms on top of BlockMatrix (SPARK-3974), such as Randomized SVD, and Multi Model Training (SPARK-1486). The proposed methods for addition are: For `Matrix` - map: maps the values in the matrix with a given function. Produces a new matrix. - update: the values in the matrix are updated with a given function. Occurs in place. Factory methods for `DenseMatrix`: - *zeros: Generate a matrix consisting of zeros - *ones: Generate a matrix consisting of ones - *eye: Generate an identity matrix - *rand: Generate a matrix consisting of i.i.d. uniform random numbers - *randn: Generate a matrix consisting of i.i.d. gaussian random numbers - *diag: Generate a diagonal matrix from a supplied vector *These methods already exist in the factory methods for `Matrices`, however for cases where we require a `DenseMatrix`, you constantly have to add `.asInstanceOf[DenseMatrix]` everywhere, which makes the code "dirtier". I propose moving these functions to factory methods for `DenseMatrix` where the putput will be a `DenseMatrix` and the factory methods for `Matrices` will call these functions directly and output a generic `Matrix`. Factory methods for `SparseMatrix`: - speye: Identity matrix in sparse format. Saves a ton of memory when dimensions are large, especially in Multi Model Training, where each row requires being multiplied by a scalar. - sprand: Generate a sparse matrix with a given density consisting of i.i.d. uniform random numbers. - sprandn: Generate a sparse matrix with a given density consisting of i.i.d. gaussian random numbers. - diag: Generate a diagonal matrix from a supplied vector, but is memory efficient, because it just stores the diagonal. Again, very helpful in Multi Model Training. Factory methods for `Matrices`: - Include all the factory methods given above, but return a generic `Matrix` rather than `SparseMatrix` or `DenseMatrix`. - horzCat: Horizontally concatenate matrices to form one larger matrix. Very useful in both Multi Model Training, and for the repartitioning of BlockMatrix. - vertCat: Vertically concatenate matrices to form one larger matrix. Very useful for the repartitioning of BlockMatrix. The names for these methods were selected from MATLAB Author: Burak Yavuz Author: Xiangrui Meng Closes #3319 from brkyvz/SPARK-4409 and squashes the following commits: b0354f6 [Burak Yavuz] [SPARK-4409] Incorporated mengxr's code 04c4829 [Burak Yavuz] Merge pull request #1 from mengxr/SPARK-4409 80cfa29 [Xiangrui Meng] minor changes ecc937a [Xiangrui Meng] update sprand 4e95e24 [Xiangrui Meng] simplify fromCOO implementation 10a63a6 [Burak Yavuz] [SPARK-4409] Fourth pass of code review f62d6c7 [Burak Yavuz] [SPARK-4409] Modified genRandMatrix 3971c93 [Burak Yavuz] [SPARK-4409] Third pass of code review 75239f8 [Burak Yavuz] [SPARK-4409] Second pass of code review e4bd0c0 [Burak Yavuz] [SPARK-4409] Modified horzcat and vertcat 65c562e [Burak Yavuz] [SPARK-4409] Hopefully fixed Java Test d8be7bc [Burak Yavuz] [SPARK-4409] Organized imports 065b531 [Burak Yavuz] [SPARK-4409] First pass after code review a8120d2 [Burak Yavuz] [SPARK-4409] Finished updates to API according to SPARK-4614 f798c82 [Burak Yavuz] [SPARK-4409] Updated API according to SPARK-4614 c75f3cd [Burak Yavuz] [SPARK-4409] Added JavaAPI Tests, and fixed a couple of bugs d662f9d [Burak Yavuz] [SPARK-4409] Modified according to remote repo 83dfe37 [Burak Yavuz] [SPARK-4409] Scalastyle error fixed a14c0da [Burak Yavuz] [SPARK-4409] Initial commit to add methods --- .../apache/spark/mllib/linalg/Matrices.scala | 570 ++++++++++++++++-- .../spark/mllib/linalg/JavaMatricesSuite.java | 163 +++++ .../spark/mllib/linalg/MatricesSuite.scala | 172 +++++- .../spark/mllib/util/TestingUtils.scala | 6 +- 4 files changed, 868 insertions(+), 43 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 327366a1a3a82..5a7281ec6dc3c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -17,9 +17,11 @@ package org.apache.spark.mllib.linalg -import java.util.{Random, Arrays} +import java.util.{Arrays, Random} -import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} +import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHashSet, ArrayBuffer} + +import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} /** * Trait for a local matrix. @@ -80,6 +82,16 @@ sealed trait Matrix extends Serializable { /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() + + /** Map the values of this matrix using a function. Generates a new matrix. Performs the + * function on only the backing array. For example, an operation such as addition or + * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */ + private[mllib] def map(f: Double => Double): Matrix + + /** Update all the values of this matrix using the function f. Performed in-place on the + * backing array. For example, an operation such as addition or subtraction will only be + * performed on the non-zero values in a `SparseMatrix`. */ + private[mllib] def update(f: Double => Double): Matrix } /** @@ -123,6 +135,122 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) } override def copy = new DenseMatrix(numRows, numCols, values.clone()) + + private[mllib] def map(f: Double => Double) = new DenseMatrix(numRows, numCols, values.map(f)) + + private[mllib] def update(f: Double => Double): DenseMatrix = { + val len = values.length + var i = 0 + while (i < len) { + values(i) = f(values(i)) + i += 1 + } + this + } + + /** Generate a `SparseMatrix` from the given `DenseMatrix`. */ + def toSparse(): SparseMatrix = { + val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble + val colPtrs: Array[Int] = new Array[Int](numCols + 1) + val rowIndices: MArrayBuilder[Int] = new MArrayBuilder.ofInt + var nnz = 0 + var j = 0 + while (j < numCols) { + var i = 0 + val indStart = j * numRows + while (i < numRows) { + val v = values(indStart + i) + if (v != 0.0) { + rowIndices += i + spVals += v + nnz += 1 + } + i += 1 + } + j += 1 + colPtrs(j) = nnz + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), spVals.result()) + } +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.DenseMatrix]]. + */ +object DenseMatrix { + + /** + * Generate a `DenseMatrix` consisting of zeros. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros + */ + def zeros(numRows: Int, numCols: Int): DenseMatrix = + new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + + /** + * Generate a `DenseMatrix` consisting of ones. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones + */ + def ones(numRows: Int, numCols: Int): DenseMatrix = + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + + /** + * Generate an Identity Matrix in `DenseMatrix` format. + * @param n number of rows and columns of the matrix + * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal + */ + def eye(n: Int): DenseMatrix = { + val identity = DenseMatrix.zeros(n, n) + var i = 0 + while (i < n) { + identity.update(i, i, 1.0) + i += 1 + } + identity + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param rng a random number generator + * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param rng a random number generator + * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) + } + + /** + * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. + * @param vector a `Vector` that will form the values on the diagonal of the matrix + * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` + * on the diagonal + */ + def diag(vector: Vector): DenseMatrix = { + val n = vector.size + val matrix = DenseMatrix.zeros(n, n) + val values = vector.toArray + var i = 0 + while (i < n) { + matrix.update(i, i, values(i)) + i += 1 + } + matrix + } } /** @@ -156,6 +284,8 @@ class SparseMatrix( require(colPtrs.length == numCols + 1, "The length of the column indices should be the " + s"number of columns + 1. Currently, colPointers.length: ${colPtrs.length}, " + s"numCols: $numCols") + require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " + + s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}") override def toArray: Array[Double] = { val arr = new Array[Double](numRows * numCols) @@ -188,7 +318,7 @@ class SparseMatrix( private[mllib] def update(i: Int, j: Int, v: Double): Unit = { val ind = index(i, j) - if (ind == -1){ + if (ind == -1) { throw new NoSuchElementException("The given row and column indices correspond to a zero " + "value. Only non-zero elements in Sparse Matrices can be updated.") } else { @@ -197,6 +327,192 @@ class SparseMatrix( } override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) + + private[mllib] def map(f: Double => Double) = + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.map(f)) + + private[mllib] def update(f: Double => Double): SparseMatrix = { + val len = values.length + var i = 0 + while (i < len) { + values(i) = f(values(i)) + i += 1 + } + this + } + + /** Generate a `DenseMatrix` from the given `SparseMatrix`. */ + def toDense(): DenseMatrix = { + new DenseMatrix(numRows, numCols, toArray) + } +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.SparseMatrix]]. + */ +object SparseMatrix { + + /** + * Generate a `SparseMatrix` from Coordinate List (COO) format. Input must be an array of + * (i, j, value) tuples. Entries that have duplicate values of i and j are + * added together. Tuples where value is equal to zero will be omitted. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param entries Array of (i, j, value) tuples + * @return The corresponding `SparseMatrix` + */ + def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = { + val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1)) + val numEntries = sortedEntries.size + if (sortedEntries.nonEmpty) { + // Since the entries are sorted by column index, we only need to check the first and the last. + for (col <- Seq(sortedEntries.head._2, sortedEntries.last._2)) { + require(col >= 0 && col < numCols, s"Column index out of range [0, $numCols): $col.") + } + } + val colPtrs = new Array[Int](numCols + 1) + val rowIndices = MArrayBuilder.make[Int] + rowIndices.sizeHint(numEntries) + val values = MArrayBuilder.make[Double] + values.sizeHint(numEntries) + var nnz = 0 + var prevCol = 0 + var prevRow = -1 + var prevVal = 0.0 + // Append a dummy entry to include the last one at the end of the loop. + (sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) => + if (v != 0) { + if (i == prevRow && j == prevCol) { + prevVal += v + } else { + if (prevVal != 0) { + require(prevRow >= 0 && prevRow < numRows, + s"Row index out of range [0, $numRows): $prevRow.") + nnz += 1 + rowIndices += prevRow + values += prevVal + } + prevRow = i + prevVal = v + while (prevCol < j) { + colPtrs(prevCol + 1) = nnz + prevCol += 1 + } + } + } + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), values.result()) + } + + /** + * Generate an Identity Matrix in `SparseMatrix` format. + * @param n number of rows and columns of the matrix + * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal + */ + def speye(n: Int): SparseMatrix = { + new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0)) + } + + /** + * Generates the skeleton of a random `SparseMatrix` with a given random number generator. + * The values of the matrix returned are undefined. + */ + private def genRandMatrix( + numRows: Int, + numCols: Int, + density: Double, + rng: Random): SparseMatrix = { + require(numRows > 0, s"numRows must be greater than 0 but got $numRows") + require(numCols > 0, s"numCols must be greater than 0 but got $numCols") + require(density >= 0.0 && density <= 1.0, + s"density must be a double in the range 0.0 <= d <= 1.0. Currently, density: $density") + val size = numRows.toLong * numCols + val expected = size * density + assert(expected < Int.MaxValue, + "The expected number of nonzeros cannot be greater than Int.MaxValue.") + val nnz = math.ceil(expected).toInt + if (density == 0.0) { + new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array[Int](), Array[Double]()) + } else if (density == 1.0) { + val colPtrs = Array.tabulate(numCols + 1)(j => j * numRows) + val rowIndices = Array.tabulate(size.toInt)(idx => idx % numRows) + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](numRows * numCols)) + } else if (density < 0.34) { + // draw-by-draw, expected number of iterations is less than 1.5 * nnz + val entries = MHashSet[(Int, Int)]() + while (entries.size < nnz) { + entries += ((rng.nextInt(numRows), rng.nextInt(numCols))) + } + SparseMatrix.fromCOO(numRows, numCols, entries.map(v => (v._1, v._2, 1.0))) + } else { + // selection-rejection method + var idx = 0L + var numSelected = 0 + var j = 0 + val colPtrs = new Array[Int](numCols + 1) + val rowIndices = new Array[Int](nnz) + while (j < numCols && numSelected < nnz) { + var i = 0 + while (i < numRows && numSelected < nnz) { + if (rng.nextDouble() < 1.0 * (nnz - numSelected) / (size - idx)) { + rowIndices(numSelected) = i + numSelected += 1 + } + i += 1 + idx += 1 + } + colPtrs(j + 1) = numSelected + j += 1 + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](nnz)) + } + } + + /** + * Generate a `SparseMatrix` consisting of i.i.d. uniform random numbers. The number of non-zero + * elements equal the ceiling of `numRows` x `numCols` x `density` + * + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { + val mat = genRandMatrix(numRows, numCols, density, rng) + mat.update(i => rng.nextDouble()) + } + + /** + * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { + val mat = genRandMatrix(numRows, numCols, density, rng) + mat.update(i => rng.nextGaussian()) + } + + /** + * Generate a diagonal matrix in `SparseMatrix` format from the supplied values. + * @param vector a `Vector` that will form the values on the diagonal of the matrix + * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero + * `values` on the diagonal + */ + def diag(vector: Vector): SparseMatrix = { + val n = vector.size + vector match { + case sVec: SparseVector => + SparseMatrix.fromCOO(n, n, sVec.indices.zip(sVec.values).map(v => (v._1, v._1, v._2))) + case dVec: DenseVector => + val entries = dVec.values.zipWithIndex + val nnzVals = entries.filter(v => v._1 != 0.0) + SparseMatrix.fromCOO(n, n, nnzVals.map(v => (v._2, v._2, v._1))) + } + } } /** @@ -256,72 +572,250 @@ object Matrices { * Generate a `DenseMatrix` consisting of zeros. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix - * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros + * @return `Matrix` with size `numRows` x `numCols` and values of zeros */ - def zeros(numRows: Int, numCols: Int): Matrix = - new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols) /** * Generate a `DenseMatrix` consisting of ones. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix - * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones + * @return `Matrix` with size `numRows` x `numCols` and values of ones */ - def ones(numRows: Int, numCols: Int): Matrix = - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols) /** - * Generate an Identity Matrix in `DenseMatrix` format. + * Generate a dense Identity Matrix in `Matrix` format. * @param n number of rows and columns of the matrix - * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal + * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ - def eye(n: Int): Matrix = { - val identity = Matrices.zeros(n, n) - var i = 0 - while (i < n){ - identity.update(i, i, 1.0) - i += 1 - } - identity - } + def eye(n: Int): Matrix = DenseMatrix.eye(n) + + /** + * Generate a sparse Identity Matrix in `Matrix` format. + * @param n number of rows and columns of the matrix + * @return `Matrix` with size `n` x `n` and values of ones on the diagonal + */ + def speye(n: Int): Matrix = SparseMatrix.speye(n) /** * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param rng a random number generator - * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ - def rand(numRows: Int, numCols: Int, rng: Random): Matrix = { - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) - } + def rand(numRows: Int, numCols: Int, rng: Random): Matrix = + DenseMatrix.rand(numRows, numCols, rng) + + /** + * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = + SparseMatrix.sprand(numRows, numCols, density, rng) /** * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param rng a random number generator - * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ - def randn(numRows: Int, numCols: Int, rng: Random): Matrix = { - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) - } + def randn(numRows: Int, numCols: Int, rng: Random): Matrix = + DenseMatrix.randn(numRows, numCols, rng) + + /** + * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = + SparseMatrix.sprandn(numRows, numCols, density, rng) /** * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. * @param vector a `Vector` tat will form the values on the diagonal of the matrix - * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` + * @return Square `Matrix` with size `values.length` x `values.length` and `values` * on the diagonal */ - def diag(vector: Vector): Matrix = { - val n = vector.size - val matrix = Matrices.eye(n) - val values = vector.toArray - var i = 0 - while (i < n) { - matrix.update(i, i, values(i)) - i += 1 + def diag(vector: Vector): Matrix = DenseMatrix.diag(vector) + + /** + * Horizontally concatenate a sequence of matrices. The returned matrix will be in the format + * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in + * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned. + * @param matrices array of matrices + * @return a single `Matrix` composed of the matrices that were horizontally concatenated + */ + def horzcat(matrices: Array[Matrix]): Matrix = { + if (matrices.isEmpty) { + return new DenseMatrix(0, 0, Array[Double]()) + } else if (matrices.size == 1) { + return matrices(0) + } + val numRows = matrices(0).numRows + var hasSparse = false + var numCols = 0 + matrices.foreach { mat => + require(numRows == mat.numRows, "The number of rows of the matrices in this sequence, " + + "don't match!") + mat match { + case sparse: SparseMatrix => hasSparse = true + case dense: DenseMatrix => // empty on purpose + case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " + + s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}") + } + numCols += mat.numCols + } + if (!hasSparse) { + new DenseMatrix(numRows, numCols, matrices.flatMap(_.toArray)) + } else { + var startCol = 0 + val entries: Array[(Int, Int, Double)] = matrices.flatMap { + case spMat: SparseMatrix => + var j = 0 + val colPtrs = spMat.colPtrs + val rowIndices = spMat.rowIndices + val values = spMat.values + val data = new Array[(Int, Int, Double)](values.length) + val nCols = spMat.numCols + while (j < nCols) { + var idx = colPtrs(j) + while (idx < colPtrs(j + 1)) { + val i = rowIndices(idx) + val v = values(idx) + data(idx) = (i, j + startCol, v) + idx += 1 + } + j += 1 + } + startCol += nCols + data + case dnMat: DenseMatrix => + val data = new ArrayBuffer[(Int, Int, Double)]() + var j = 0 + val nCols = dnMat.numCols + val nRows = dnMat.numRows + val values = dnMat.values + while (j < nCols) { + var i = 0 + val indStart = j * nRows + while (i < nRows) { + val v = values(indStart + i) + if (v != 0.0) { + data.append((i, j + startCol, v)) + } + i += 1 + } + j += 1 + } + startCol += nCols + data + } + SparseMatrix.fromCOO(numRows, numCols, entries) + } + } + + /** + * Vertically concatenate a sequence of matrices. The returned matrix will be in the format + * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in + * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned. + * @param matrices array of matrices + * @return a single `Matrix` composed of the matrices that were vertically concatenated + */ + def vertcat(matrices: Array[Matrix]): Matrix = { + if (matrices.isEmpty) { + return new DenseMatrix(0, 0, Array[Double]()) + } else if (matrices.size == 1) { + return matrices(0) + } + val numCols = matrices(0).numCols + var hasSparse = false + var numRows = 0 + matrices.foreach { mat => + require(numCols == mat.numCols, "The number of rows of the matrices in this sequence, " + + "don't match!") + mat match { + case sparse: SparseMatrix => + hasSparse = true + case dense: DenseMatrix => + case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " + + s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}") + } + numRows += mat.numRows + + } + if (!hasSparse) { + val allValues = new Array[Double](numRows * numCols) + var startRow = 0 + matrices.foreach { mat => + var j = 0 + val nRows = mat.numRows + val values = mat.toArray + while (j < numCols) { + var i = 0 + val indStart = j * numRows + startRow + val subMatStart = j * nRows + while (i < nRows) { + allValues(indStart + i) = values(subMatStart + i) + i += 1 + } + j += 1 + } + startRow += nRows + } + new DenseMatrix(numRows, numCols, allValues) + } else { + var startRow = 0 + val entries: Array[(Int, Int, Double)] = matrices.flatMap { + case spMat: SparseMatrix => + var j = 0 + val colPtrs = spMat.colPtrs + val rowIndices = spMat.rowIndices + val values = spMat.values + val data = new Array[(Int, Int, Double)](values.length) + while (j < numCols) { + var idx = colPtrs(j) + while (idx < colPtrs(j + 1)) { + val i = rowIndices(idx) + val v = values(idx) + data(idx) = (i + startRow, j, v) + idx += 1 + } + j += 1 + } + startRow += spMat.numRows + data + case dnMat: DenseMatrix => + val data = new ArrayBuffer[(Int, Int, Double)]() + var j = 0 + val nCols = dnMat.numCols + val nRows = dnMat.numRows + val values = dnMat.values + while (j < nCols) { + var i = 0 + val indStart = j * nRows + while (i < nRows) { + val v = values(indStart + i) + if (v != 0.0) { + data.append((i + startRow, j, v)) + } + i += 1 + } + j += 1 + } + startRow += nRows + data + } + SparseMatrix.fromCOO(numRows, numCols, entries) } - matrix } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java new file mode 100644 index 0000000000000..704d484d0b585 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java @@ -0,0 +1,163 @@ +/* + * 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.linalg; + +import static org.junit.Assert.*; +import org.junit.Test; + +import java.io.Serializable; +import java.util.Random; + +public class JavaMatricesSuite implements Serializable { + + @Test + public void randMatrixConstruction() { + Random rng = new Random(24); + Matrix r = Matrices.rand(3, 4, rng); + rng.setSeed(24); + DenseMatrix dr = DenseMatrix.rand(3, 4, rng); + assertArrayEquals(r.toArray(), dr.toArray(), 0.0); + + rng.setSeed(24); + Matrix rn = Matrices.randn(3, 4, rng); + rng.setSeed(24); + DenseMatrix drn = DenseMatrix.randn(3, 4, rng); + assertArrayEquals(rn.toArray(), drn.toArray(), 0.0); + + rng.setSeed(24); + Matrix s = Matrices.sprand(3, 4, 0.5, rng); + rng.setSeed(24); + SparseMatrix sr = SparseMatrix.sprand(3, 4, 0.5, rng); + assertArrayEquals(s.toArray(), sr.toArray(), 0.0); + + rng.setSeed(24); + Matrix sn = Matrices.sprandn(3, 4, 0.5, rng); + rng.setSeed(24); + SparseMatrix srn = SparseMatrix.sprandn(3, 4, 0.5, rng); + assertArrayEquals(sn.toArray(), srn.toArray(), 0.0); + } + + @Test + public void identityMatrixConstruction() { + Matrix r = Matrices.eye(2); + DenseMatrix dr = DenseMatrix.eye(2); + SparseMatrix sr = SparseMatrix.speye(2); + assertArrayEquals(r.toArray(), dr.toArray(), 0.0); + assertArrayEquals(sr.toArray(), dr.toArray(), 0.0); + assertArrayEquals(r.toArray(), new double[]{1.0, 0.0, 0.0, 1.0}, 0.0); + } + + @Test + public void diagonalMatrixConstruction() { + Vector v = Vectors.dense(1.0, 0.0, 2.0); + Vector sv = Vectors.sparse(3, new int[]{0, 2}, new double[]{1.0, 2.0}); + + Matrix m = Matrices.diag(v); + Matrix sm = Matrices.diag(sv); + DenseMatrix d = DenseMatrix.diag(v); + DenseMatrix sd = DenseMatrix.diag(sv); + SparseMatrix s = SparseMatrix.diag(v); + SparseMatrix ss = SparseMatrix.diag(sv); + + assertArrayEquals(m.toArray(), sm.toArray(), 0.0); + assertArrayEquals(d.toArray(), sm.toArray(), 0.0); + assertArrayEquals(d.toArray(), sd.toArray(), 0.0); + assertArrayEquals(sd.toArray(), s.toArray(), 0.0); + assertArrayEquals(s.toArray(), ss.toArray(), 0.0); + assertArrayEquals(s.values(), ss.values(), 0.0); + assert(s.values().length == 2); + assert(ss.values().length == 2); + assert(s.colPtrs().length == 4); + assert(ss.colPtrs().length == 4); + } + + @Test + public void zerosMatrixConstruction() { + Matrix z = Matrices.zeros(2, 2); + Matrix one = Matrices.ones(2, 2); + DenseMatrix dz = DenseMatrix.zeros(2, 2); + DenseMatrix done = DenseMatrix.ones(2, 2); + + assertArrayEquals(z.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); + assertArrayEquals(dz.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); + assertArrayEquals(one.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); + assertArrayEquals(done.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); + } + + @Test + public void sparseDenseConversion() { + int m = 3; + int n = 2; + double[] values = new double[]{1.0, 2.0, 4.0, 5.0}; + double[] allValues = new double[]{1.0, 2.0, 0.0, 0.0, 4.0, 5.0}; + int[] colPtrs = new int[]{0, 2, 4}; + int[] rowIndices = new int[]{0, 1, 1, 2}; + + SparseMatrix spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values); + DenseMatrix deMat1 = new DenseMatrix(m, n, allValues); + + SparseMatrix spMat2 = deMat1.toSparse(); + DenseMatrix deMat2 = spMat1.toDense(); + + assertArrayEquals(spMat1.toArray(), spMat2.toArray(), 0.0); + assertArrayEquals(deMat1.toArray(), deMat2.toArray(), 0.0); + } + + @Test + public void concatenateMatrices() { + int m = 3; + int n = 2; + + Random rng = new Random(42); + SparseMatrix spMat1 = SparseMatrix.sprand(m, n, 0.5, rng); + rng.setSeed(42); + DenseMatrix deMat1 = DenseMatrix.rand(m, n, rng); + Matrix deMat2 = Matrices.eye(3); + Matrix spMat2 = Matrices.speye(3); + Matrix deMat3 = Matrices.eye(2); + Matrix spMat3 = Matrices.speye(2); + + Matrix spHorz = Matrices.horzcat(new Matrix[]{spMat1, spMat2}); + Matrix deHorz1 = Matrices.horzcat(new Matrix[]{deMat1, deMat2}); + Matrix deHorz2 = Matrices.horzcat(new Matrix[]{spMat1, deMat2}); + Matrix deHorz3 = Matrices.horzcat(new Matrix[]{deMat1, spMat2}); + + assert(deHorz1.numRows() == 3); + assert(deHorz2.numRows() == 3); + assert(deHorz3.numRows() == 3); + assert(spHorz.numRows() == 3); + assert(deHorz1.numCols() == 5); + assert(deHorz2.numCols() == 5); + assert(deHorz3.numCols() == 5); + assert(spHorz.numCols() == 5); + + Matrix spVert = Matrices.vertcat(new Matrix[]{spMat1, spMat3}); + Matrix deVert1 = Matrices.vertcat(new Matrix[]{deMat1, deMat3}); + Matrix deVert2 = Matrices.vertcat(new Matrix[]{spMat1, deMat3}); + Matrix deVert3 = Matrices.vertcat(new Matrix[]{deMat1, spMat3}); + + assert(deVert1.numRows() == 5); + assert(deVert2.numRows() == 5); + assert(deVert3.numRows() == 5); + assert(spVert.numRows() == 5); + assert(deVert1.numCols() == 2); + assert(deVert2.numCols() == 2); + assert(deVert3.numCols() == 2); + assert(spVert.numCols() == 2); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 322a0e9242918..a35d0fe389fdd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -43,9 +43,9 @@ class MatricesSuite extends FunSuite { test("sparse matrix construction") { val m = 3 - val n = 2 + val n = 4 val values = Array(1.0, 2.0, 4.0, 5.0) - val colPtrs = Array(0, 2, 4) + val colPtrs = Array(0, 2, 2, 4, 4) val rowIndices = Array(1, 2, 1, 2) val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix] assert(mat.numRows === m) @@ -53,6 +53,13 @@ class MatricesSuite extends FunSuite { assert(mat.values.eq(values), "should not copy data") assert(mat.colPtrs.eq(colPtrs), "should not copy data") assert(mat.rowIndices.eq(rowIndices), "should not copy data") + + val entries: Array[(Int, Int, Double)] = Array((2, 2, 3.0), (1, 0, 1.0), (2, 0, 2.0), + (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0)) + + val mat2 = SparseMatrix.fromCOO(m, n, entries) + assert(mat.toBreeze === mat2.toBreeze) + assert(mat2.values.length == 4) } test("sparse matrix construction with wrong number of elements") { @@ -117,6 +124,142 @@ class MatricesSuite extends FunSuite { assert(sparseMat.values(2) === 10.0) } + test("toSparse, toDense") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + + val spMat2 = deMat1.toSparse() + val deMat2 = spMat1.toDense() + + assert(spMat1.toBreeze === spMat2.toBreeze) + assert(deMat1.toBreeze === deMat2.toBreeze) + } + + test("map, update") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + val deMat2 = deMat1.map(_ * 2) + val spMat2 = spMat1.map(_ * 2) + deMat1.update(_ * 2) + spMat1.update(_ * 2) + + assert(spMat1.toArray === spMat2.toArray) + assert(deMat1.toArray === deMat2.toArray) + } + + test("horzcat, vertcat, eye, speye") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + val deMat2 = Matrices.eye(3) + val spMat2 = Matrices.speye(3) + val deMat3 = Matrices.eye(2) + val spMat3 = Matrices.speye(2) + + val spHorz = Matrices.horzcat(Array(spMat1, spMat2)) + val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) + val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) + val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) + + val deHorz2 = Matrices.horzcat(Array[Matrix]()) + + assert(deHorz1.numRows === 3) + assert(spHorz2.numRows === 3) + assert(spHorz3.numRows === 3) + assert(spHorz.numRows === 3) + assert(deHorz1.numCols === 5) + assert(spHorz2.numCols === 5) + assert(spHorz3.numCols === 5) + assert(spHorz.numCols === 5) + assert(deHorz2.numRows === 0) + assert(deHorz2.numCols === 0) + assert(deHorz2.toArray.length === 0) + + assert(deHorz1.toBreeze.toDenseMatrix === spHorz2.toBreeze.toDenseMatrix) + assert(spHorz2.toBreeze === spHorz3.toBreeze) + assert(spHorz(0, 0) === 1.0) + assert(spHorz(2, 1) === 5.0) + assert(spHorz(0, 2) === 1.0) + assert(spHorz(1, 2) === 0.0) + assert(spHorz(1, 3) === 1.0) + assert(spHorz(2, 4) === 1.0) + assert(spHorz(1, 4) === 0.0) + assert(deHorz1(0, 0) === 1.0) + assert(deHorz1(2, 1) === 5.0) + assert(deHorz1(0, 2) === 1.0) + assert(deHorz1(1, 2) == 0.0) + assert(deHorz1(1, 3) === 1.0) + assert(deHorz1(2, 4) === 1.0) + assert(deHorz1(1, 4) === 0.0) + + intercept[IllegalArgumentException] { + Matrices.horzcat(Array(spMat1, spMat3)) + } + + intercept[IllegalArgumentException] { + Matrices.horzcat(Array(deMat1, spMat3)) + } + + val spVert = Matrices.vertcat(Array(spMat1, spMat3)) + val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) + val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) + val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) + val deVert2 = Matrices.vertcat(Array[Matrix]()) + + assert(deVert1.numRows === 5) + assert(spVert2.numRows === 5) + assert(spVert3.numRows === 5) + assert(spVert.numRows === 5) + assert(deVert1.numCols === 2) + assert(spVert2.numCols === 2) + assert(spVert3.numCols === 2) + assert(spVert.numCols === 2) + assert(deVert2.numRows === 0) + assert(deVert2.numCols === 0) + assert(deVert2.toArray.length === 0) + + assert(deVert1.toBreeze.toDenseMatrix === spVert2.toBreeze.toDenseMatrix) + assert(spVert2.toBreeze === spVert3.toBreeze) + assert(spVert(0, 0) === 1.0) + assert(spVert(2, 1) === 5.0) + assert(spVert(3, 0) === 1.0) + assert(spVert(3, 1) === 0.0) + assert(spVert(4, 1) === 1.0) + assert(deVert1(0, 0) === 1.0) + assert(deVert1(2, 1) === 5.0) + assert(deVert1(3, 0) === 1.0) + assert(deVert1(3, 1) === 0.0) + assert(deVert1(4, 1) === 1.0) + + intercept[IllegalArgumentException] { + Matrices.vertcat(Array(spMat1, spMat2)) + } + + intercept[IllegalArgumentException] { + Matrices.vertcat(Array(deMat1, spMat2)) + } + } + test("zeros") { val mat = Matrices.zeros(2, 3).asInstanceOf[DenseMatrix] assert(mat.numRows === 2) @@ -162,4 +305,29 @@ class MatricesSuite extends FunSuite { assert(mat.numCols === 2) assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 2.0)) } + + test("sprand") { + val rng = mock[Random] + when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0) + when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0) + val mat = SparseMatrix.sprand(4, 4, 0.25, rng) + assert(mat.numRows === 4) + assert(mat.numCols === 4) + assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1)) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + val mat2 = SparseMatrix.sprand(2, 3, 1.0, rng) + assert(mat2.rowIndices.toSeq === Seq(0, 1, 0, 1, 0, 1)) + assert(mat2.colPtrs.toSeq === Seq(0, 2, 4, 6)) + } + + test("sprandn") { + val rng = mock[Random] + when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0) + when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = SparseMatrix.sprandn(4, 4, 0.25, rng) + assert(mat.numRows === 4) + assert(mat.numCols === 4) + assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1)) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index 30b906aaa3ba4..e957fa5d25f4c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -178,17 +178,17 @@ object TestingUtils { implicit class MatrixWithAlmostEquals(val x: Matrix) { /** - * When the difference of two vectors are within eps, returns true; otherwise, returns false. + * When the difference of two matrices are within eps, returns true; otherwise, returns false. */ def ~=(r: CompareMatrixRightSide): Boolean = r.fun(x, r.y, r.eps) /** - * When the difference of two vectors are within eps, returns false; otherwise, returns true. + * When the difference of two matrices are within eps, returns false; otherwise, returns true. */ def !~=(r: CompareMatrixRightSide): Boolean = !r.fun(x, r.y, r.eps) /** - * Throws exception when the difference of two vectors are NOT within eps; + * Throws exception when the difference of two matrices are NOT within eps; * otherwise, returns true. */ def ~==(r: CompareMatrixRightSide): Boolean = { From 9bc0df6804f241aff24520d9c6ec54d9b11f5785 Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Mon, 29 Dec 2014 13:49:45 -0800 Subject: [PATCH 141/227] SPARK-4968: takeOrdered to skip reduce step in case mappers return no partitions takeOrdered should skip reduce step in case mapped RDDs have no partitions. This prevents the mentioned exception : 4. run query SELECT * FROM testTable WHERE market = 'market2' ORDER BY End_Time DESC LIMIT 100; Error trace java.lang.UnsupportedOperationException: empty collection at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:863) at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:863) at scala.Option.getOrElse(Option.scala:120) at org.apache.spark.rdd.RDD.reduce(RDD.scala:863) at org.apache.spark.rdd.RDD.takeOrdered(RDD.scala:1136) Author: Yash Datta Closes #3830 from saucam/fix_takeorder and squashes the following commits: 5974d10 [Yash Datta] SPARK-4968: takeOrdered to skip reduce step in case mappers return no partitions --- .../src/main/scala/org/apache/spark/rdd/RDD.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 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 f47c2d1fcdcc7..5118e2b911120 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1146,15 +1146,20 @@ abstract class RDD[T: ClassTag]( if (num == 0) { Array.empty } else { - mapPartitions { items => + val mapRDDs = 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 (mapRDDs.partitions.size == 0) { + Array.empty + } else { + mapRDDs.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord) + } } } From 6cf6fdf3ff5d1cf33c2dc28f039adc4d7c0f0464 Mon Sep 17 00:00:00 2001 From: Travis Galoppo Date: Mon, 29 Dec 2014 15:29:15 -0800 Subject: [PATCH 142/227] SPARK-4156 [MLLIB] EM algorithm for GMMs Implementation of Expectation-Maximization for Gaussian Mixture Models. This is my maiden contribution to Apache Spark, so I apologize now if I have done anything incorrectly; having said that, this work is my own, and I offer it to the project under the project's open source license. Author: Travis Galoppo Author: Travis Galoppo Author: tgaloppo Author: FlytxtRnD Closes #3022 from tgaloppo/master and squashes the following commits: aaa8f25 [Travis Galoppo] MLUtils: changed privacy of EPSILON from [util] to [mllib] 709e4bf [Travis Galoppo] fixed usage line to include optional maxIterations parameter acf1fba [Travis Galoppo] Fixed parameter comment in GaussianMixtureModel Made maximum iterations an optional parameter to DenseGmmEM 9b2fc2a [Travis Galoppo] Style improvements Changed ExpectationSum to a private class b97fe00 [Travis Galoppo] Minor fixes and tweaks. 1de73f3 [Travis Galoppo] Removed redundant array from array creation 578c2d1 [Travis Galoppo] Removed unused import 227ad66 [Travis Galoppo] Moved prediction methods into model class. 308c8ad [Travis Galoppo] Numerous changes to improve code cff73e0 [Travis Galoppo] Replaced accumulators with RDD.aggregate 20ebca1 [Travis Galoppo] Removed unusued code 42b2142 [Travis Galoppo] Added functionality to allow setting of GMM starting point. Added two cluster test to testing suite. 8b633f3 [Travis Galoppo] Style issue 9be2534 [Travis Galoppo] Style issue d695034 [Travis Galoppo] Fixed style issues c3b8ce0 [Travis Galoppo] Merge branch 'master' of https://github.com/tgaloppo/spark Adds predict() method 2df336b [Travis Galoppo] Fixed style issue b99ecc4 [tgaloppo] Merge pull request #1 from FlytxtRnD/predictBranch f407b4c [FlytxtRnD] Added predict() to return the cluster labels and membership values 97044cf [Travis Galoppo] Fixed style issues dc9c742 [Travis Galoppo] Moved MultivariateGaussian utility class e7d413b [Travis Galoppo] Moved multivariate Gaussian utility class to mllib/stat/impl Improved comments 9770261 [Travis Galoppo] Corrected a variety of style and naming issues. 8aaa17d [Travis Galoppo] Added additional train() method to companion object for cluster count and tolerance parameters. 676e523 [Travis Galoppo] Fixed to no longer ignore delta value provided on command line e6ea805 [Travis Galoppo] Merged with master branch; update test suite with latest context changes. Improved cluster initialization strategy. 86fb382 [Travis Galoppo] Merge remote-tracking branch 'upstream/master' 719d8cc [Travis Galoppo] Added scala test suite with basic test c1a8e16 [Travis Galoppo] Made GaussianMixtureModel class serializable Modified sum function for better performance 5c96c57 [Travis Galoppo] Merge remote-tracking branch 'upstream/master' c15405c [Travis Galoppo] SPARK-4156 --- .../spark/examples/mllib/DenseGmmEM.scala | 67 +++++ .../mllib/clustering/GaussianMixtureEM.scala | 241 ++++++++++++++++++ .../clustering/GaussianMixtureModel.scala | 91 +++++++ .../stat/impl/MultivariateGaussian.scala | 39 +++ .../org/apache/spark/mllib/util/MLUtils.scala | 2 +- .../GMMExpectationMaximizationSuite.scala | 78 ++++++ 6 files changed, 517 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala new file mode 100644 index 0000000000000..948c350953e27 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala @@ -0,0 +1,67 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.clustering.GaussianMixtureEM +import org.apache.spark.mllib.linalg.Vectors + +/** + * An example Gaussian Mixture Model EM app. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.DenseGmmEM + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DenseGmmEM { + def main(args: Array[String]): Unit = { + if (args.length < 3) { + println("usage: DenseGmmEM [maxIterations]") + } else { + val maxIterations = if (args.length > 3) args(3).toInt else 100 + run(args(0), args(1).toInt, args(2).toDouble, maxIterations) + } + } + + private def run(inputFile: String, k: Int, convergenceTol: Double, maxIterations: Int) { + val conf = new SparkConf().setAppName("Gaussian Mixture Model EM example") + val ctx = new SparkContext(conf) + + val data = ctx.textFile(inputFile).map { line => + Vectors.dense(line.trim.split(' ').map(_.toDouble)) + }.cache() + + val clusters = new GaussianMixtureEM() + .setK(k) + .setConvergenceTol(convergenceTol) + .setMaxIterations(maxIterations) + .run(data) + + for (i <- 0 until clusters.k) { + println("weight=%f\nmu=%s\nsigma=\n%s\n" format + (clusters.weight(i), clusters.mu(i), clusters.sigma(i))) + } + + println("Cluster labels (first <= 100):") + val clusterLabels = clusters.predict(data) + clusterLabels.take(100).foreach { x => + print(" " + x) + } + println() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala new file mode 100644 index 0000000000000..bdf984aee4dae --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala @@ -0,0 +1,241 @@ +/* + * 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.clustering + +import scala.collection.mutable.IndexedSeq + +import breeze.linalg.{DenseVector => BreezeVector, DenseMatrix => BreezeMatrix, diag, Transpose} +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors} +import org.apache.spark.mllib.stat.impl.MultivariateGaussian +import org.apache.spark.mllib.util.MLUtils + +/** + * This class performs expectation maximization for multivariate Gaussian + * Mixture Models (GMMs). A GMM represents a composite distribution of + * independent Gaussian distributions with associated "mixing" weights + * specifying each's contribution to the composite. + * + * Given a set of sample points, this class will maximize the log-likelihood + * for a mixture of k Gaussians, iterating until the log-likelihood changes by + * less than convergenceTol, or until it has reached the max number of iterations. + * While this process is generally guaranteed to converge, it is not guaranteed + * to find a global optimum. + * + * @param k The number of independent Gaussians in the mixture model + * @param convergenceTol The maximum change in log-likelihood at which convergence + * is considered to have occurred. + * @param maxIterations The maximum number of iterations to perform + */ +class GaussianMixtureEM private ( + private var k: Int, + private var convergenceTol: Double, + private var maxIterations: Int) extends Serializable { + + /** A default instance, 2 Gaussians, 100 iterations, 0.01 log-likelihood threshold */ + def this() = this(2, 0.01, 100) + + // number of samples per cluster to use when initializing Gaussians + private val nSamples = 5 + + // an initializing GMM can be provided rather than using the + // default random starting point + private var initialModel: Option[GaussianMixtureModel] = None + + /** Set the initial GMM starting point, bypassing the random initialization. + * You must call setK() prior to calling this method, and the condition + * (model.k == this.k) must be met; failure will result in an IllegalArgumentException + */ + def setInitialModel(model: GaussianMixtureModel): this.type = { + if (model.k == k) { + initialModel = Some(model) + } else { + throw new IllegalArgumentException("mismatched cluster count (model.k != k)") + } + this + } + + /** Return the user supplied initial GMM, if supplied */ + def getInitialModel: Option[GaussianMixtureModel] = initialModel + + /** Set the number of Gaussians in the mixture model. Default: 2 */ + def setK(k: Int): this.type = { + this.k = k + this + } + + /** Return the number of Gaussians in the mixture model */ + def getK: Int = k + + /** Set the maximum number of iterations to run. Default: 100 */ + def setMaxIterations(maxIterations: Int): this.type = { + this.maxIterations = maxIterations + this + } + + /** Return the maximum number of iterations to run */ + def getMaxIterations: Int = maxIterations + + /** + * Set the largest change in log-likelihood at which convergence is + * considered to have occurred. + */ + def setConvergenceTol(convergenceTol: Double): this.type = { + this.convergenceTol = convergenceTol + this + } + + /** Return the largest change in log-likelihood at which convergence is + * considered to have occurred. + */ + def getConvergenceTol: Double = convergenceTol + + /** Perform expectation maximization */ + def run(data: RDD[Vector]): GaussianMixtureModel = { + val sc = data.sparkContext + + // we will operate on the data as breeze data + val breezeData = data.map(u => u.toBreeze.toDenseVector).cache() + + // Get length of the input vectors + val d = breezeData.first.length + + // Determine initial weights and corresponding Gaussians. + // If the user supplied an initial GMM, we use those values, otherwise + // we start with uniform weights, a random mean from the data, and + // diagonal covariance matrices using component variances + // derived from the samples + val (weights, gaussians) = initialModel match { + case Some(gmm) => (gmm.weight, gmm.mu.zip(gmm.sigma).map { case(mu, sigma) => + new MultivariateGaussian(mu.toBreeze.toDenseVector, sigma.toBreeze.toDenseMatrix) + }) + + case None => { + val samples = breezeData.takeSample(true, k * nSamples, scala.util.Random.nextInt) + (Array.fill(k)(1.0 / k), Array.tabulate(k) { i => + val slice = samples.view(i * nSamples, (i + 1) * nSamples) + new MultivariateGaussian(vectorMean(slice), initCovariance(slice)) + }) + } + } + + var llh = Double.MinValue // current log-likelihood + var llhp = 0.0 // previous log-likelihood + + var iter = 0 + while(iter < maxIterations && Math.abs(llh-llhp) > convergenceTol) { + // create and broadcast curried cluster contribution function + val compute = sc.broadcast(ExpectationSum.add(weights, gaussians)_) + + // aggregate the cluster contribution for all sample points + val sums = breezeData.aggregate(ExpectationSum.zero(k, d))(compute.value, _ += _) + + // Create new distributions based on the partial assignments + // (often referred to as the "M" step in literature) + val sumWeights = sums.weights.sum + var i = 0 + while (i < k) { + val mu = sums.means(i) / sums.weights(i) + val sigma = sums.sigmas(i) / sums.weights(i) - mu * new Transpose(mu) // TODO: Use BLAS.dsyr + weights(i) = sums.weights(i) / sumWeights + gaussians(i) = new MultivariateGaussian(mu, sigma) + i = i + 1 + } + + llhp = llh // current becomes previous + llh = sums.logLikelihood // this is the freshly computed log-likelihood + iter += 1 + } + + // Need to convert the breeze matrices to MLlib matrices + val means = Array.tabulate(k) { i => Vectors.fromBreeze(gaussians(i).mu) } + val sigmas = Array.tabulate(k) { i => Matrices.fromBreeze(gaussians(i).sigma) } + new GaussianMixtureModel(weights, means, sigmas) + } + + /** Average of dense breeze vectors */ + private def vectorMean(x: IndexedSeq[BreezeVector[Double]]): BreezeVector[Double] = { + val v = BreezeVector.zeros[Double](x(0).length) + x.foreach(xi => v += xi) + v / x.length.toDouble + } + + /** + * Construct matrix where diagonal entries are element-wise + * variance of input vectors (computes biased variance) + */ + private def initCovariance(x: IndexedSeq[BreezeVector[Double]]): BreezeMatrix[Double] = { + val mu = vectorMean(x) + val ss = BreezeVector.zeros[Double](x(0).length) + x.map(xi => (xi - mu) :^ 2.0).foreach(u => ss += u) + diag(ss / x.length.toDouble) + } +} + +// companion class to provide zero constructor for ExpectationSum +private object ExpectationSum { + def zero(k: Int, d: Int): ExpectationSum = { + new ExpectationSum(0.0, Array.fill(k)(0.0), + Array.fill(k)(BreezeVector.zeros(d)), Array.fill(k)(BreezeMatrix.zeros(d,d))) + } + + // compute cluster contributions for each input point + // (U, T) => U for aggregation + def add( + weights: Array[Double], + dists: Array[MultivariateGaussian]) + (sums: ExpectationSum, x: BreezeVector[Double]): ExpectationSum = { + val p = weights.zip(dists).map { + case (weight, dist) => MLUtils.EPSILON + weight * dist.pdf(x) + } + val pSum = p.sum + sums.logLikelihood += math.log(pSum) + val xxt = x * new Transpose(x) + var i = 0 + while (i < sums.k) { + p(i) /= pSum + sums.weights(i) += p(i) + sums.means(i) += x * p(i) + sums.sigmas(i) += xxt * p(i) // TODO: use BLAS.dsyr + i = i + 1 + } + sums + } +} + +// Aggregation class for partial expectation results +private class ExpectationSum( + var logLikelihood: Double, + val weights: Array[Double], + val means: Array[BreezeVector[Double]], + val sigmas: Array[BreezeMatrix[Double]]) extends Serializable { + + val k = weights.length + + def +=(x: ExpectationSum): ExpectationSum = { + var i = 0 + while (i < k) { + weights(i) += x.weights(i) + means(i) += x.means(i) + sigmas(i) += x.sigmas(i) + i = i + 1 + } + logLikelihood += x.logLikelihood + this + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala new file mode 100644 index 0000000000000..11a110db1f7ca --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -0,0 +1,91 @@ +/* + * 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.clustering + +import breeze.linalg.{DenseVector => BreezeVector} + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.stat.impl.MultivariateGaussian +import org.apache.spark.mllib.util.MLUtils + +/** + * Multivariate Gaussian Mixture Model (GMM) consisting of k Gaussians, where points + * are drawn from each Gaussian i=1..k with probability w(i); mu(i) and sigma(i) are + * the respective mean and covariance for each Gaussian distribution i=1..k. + * + * @param weight Weights for each Gaussian distribution in the mixture, where weight(i) is + * the weight for Gaussian i, and weight.sum == 1 + * @param mu Means for each Gaussian in the mixture, where mu(i) is the mean for Gaussian i + * @param sigma Covariance maxtrix for each Gaussian in the mixture, where sigma(i) is the + * covariance matrix for Gaussian i + */ +class GaussianMixtureModel( + val weight: Array[Double], + val mu: Array[Vector], + val sigma: Array[Matrix]) extends Serializable { + + /** Number of gaussians in mixture */ + def k: Int = weight.length + + /** Maps given points to their cluster indices. */ + def predict(points: RDD[Vector]): RDD[Int] = { + val responsibilityMatrix = predictMembership(points, mu, sigma, weight, k) + responsibilityMatrix.map(r => r.indexOf(r.max)) + } + + /** + * Given the input vectors, return the membership value of each vector + * to all mixture components. + */ + def predictMembership( + points: RDD[Vector], + mu: Array[Vector], + sigma: Array[Matrix], + weight: Array[Double], + k: Int): RDD[Array[Double]] = { + val sc = points.sparkContext + val dists = sc.broadcast { + (0 until k).map { i => + new MultivariateGaussian(mu(i).toBreeze.toDenseVector, sigma(i).toBreeze.toDenseMatrix) + }.toArray + } + val weights = sc.broadcast(weight) + points.map { x => + computeSoftAssignments(x.toBreeze.toDenseVector, dists.value, weights.value, k) + } + } + + /** + * Compute the partial assignments for each vector + */ + private def computeSoftAssignments( + pt: BreezeVector[Double], + dists: Array[MultivariateGaussian], + weights: Array[Double], + k: Int): Array[Double] = { + val p = weights.zip(dists).map { + case (weight, dist) => MLUtils.EPSILON + weight * dist.pdf(pt) + } + val pSum = p.sum + for (i <- 0 until k) { + p(i) /= pSum + } + p + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala new file mode 100644 index 0000000000000..2eab5d277827d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala @@ -0,0 +1,39 @@ +/* + * 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.stat.impl + +import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, Transpose, det, pinv} + +/** + * Utility class to implement the density function for multivariate Gaussian distribution. + * Breeze provides this functionality, but it requires the Apache Commons Math library, + * so this class is here so-as to not introduce a new dependency in Spark. + */ +private[mllib] class MultivariateGaussian( + val mu: DBV[Double], + val sigma: DBM[Double]) extends Serializable { + private val sigmaInv2 = pinv(sigma) * -0.5 + private val U = math.pow(2.0 * math.Pi, -mu.length / 2.0) * math.pow(det(sigma), -0.5) + + /** Returns density of this multivariate Gaussian at given point, x */ + def pdf(x: DBV[Double]): Double = { + val delta = x - mu + val deltaTranspose = new Transpose(delta) + U * math.exp(deltaTranspose * sigmaInv2 * delta) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index b0d05ae33e1b5..1d07b5dab8268 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -39,7 +39,7 @@ import org.apache.spark.streaming.dstream.DStream */ object MLUtils { - private[util] lazy val EPSILON = { + private[mllib] lazy val EPSILON = { var eps = 1.0 while ((1.0 + (eps / 2.0)) != 1.0) { eps /= 2.0 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala new file mode 100644 index 0000000000000..23feb82874b70 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala @@ -0,0 +1,78 @@ +/* + * 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.clustering + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{Vectors, Matrices} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class GMMExpectationMaximizationSuite extends FunSuite with MLlibTestSparkContext { + test("single cluster") { + val data = sc.parallelize(Array( + Vectors.dense(6.0, 9.0), + Vectors.dense(5.0, 10.0), + Vectors.dense(4.0, 11.0) + )) + + // expectations + val Ew = 1.0 + val Emu = Vectors.dense(5.0, 10.0) + val Esigma = Matrices.dense(2, 2, Array(2.0 / 3.0, -2.0 / 3.0, -2.0 / 3.0, 2.0 / 3.0)) + + val gmm = new GaussianMixtureEM().setK(1).run(data) + + assert(gmm.weight(0) ~== Ew absTol 1E-5) + assert(gmm.mu(0) ~== Emu absTol 1E-5) + assert(gmm.sigma(0) ~== Esigma absTol 1E-5) + } + + test("two clusters") { + val data = sc.parallelize(Array( + Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), + Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), + Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), + Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), + Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) + )) + + // we set an initial gaussian to induce expected results + val initialGmm = new GaussianMixtureModel( + Array(0.5, 0.5), + Array(Vectors.dense(-1.0), Vectors.dense(1.0)), + Array(Matrices.dense(1, 1, Array(1.0)), Matrices.dense(1, 1, Array(1.0))) + ) + + val Ew = Array(1.0 / 3.0, 2.0 / 3.0) + val Emu = Array(Vectors.dense(-4.3673), Vectors.dense(5.1604)) + val Esigma = Array(Matrices.dense(1, 1, Array(1.1098)), Matrices.dense(1, 1, Array(0.86644))) + + val gmm = new GaussianMixtureEM() + .setK(2) + .setInitialModel(initialGmm) + .run(data) + + assert(gmm.weight(0) ~== Ew(0) absTol 1E-3) + assert(gmm.weight(1) ~== Ew(1) absTol 1E-3) + assert(gmm.mu(0) ~== Emu(0) absTol 1E-3) + assert(gmm.mu(1) ~== Emu(1) absTol 1E-3) + assert(gmm.sigma(0) ~== Esigma(0) absTol 1E-3) + assert(gmm.sigma(1) ~== Esigma(1) absTol 1E-3) + } +} From 343db392b58fb33a3e4bc6fda1da69aaf686b5a9 Mon Sep 17 00:00:00 2001 From: ganonp Date: Mon, 29 Dec 2014 15:31:19 -0800 Subject: [PATCH 143/227] Added setMinCount to Word2Vec.scala Wanted to customize the private minCount variable in the Word2Vec class. Added a method to do so. Author: ganonp Closes #3693 from ganonp/my-custom-spark and squashes the following commits: ad534f2 [ganonp] made norm method public 5110a6f [ganonp] Reorganized 854958b [ganonp] Fixed Indentation for setMinCount 12ed8f9 [ganonp] Update Word2Vec.scala 76bdf5a [ganonp] Update Word2Vec.scala ffb88bb [ganonp] Update Word2Vec.scala 5eb9100 [ganonp] Added setMinCount to Word2Vec.scala --- .../org/apache/spark/mllib/feature/Word2Vec.scala | 15 +++++++++++---- .../org/apache/spark/mllib/linalg/Vectors.scala | 2 +- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 7960f3cab576f..d25a7cd5b439d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -71,7 +71,8 @@ class Word2Vec extends Serializable with Logging { private var numPartitions = 1 private var numIterations = 1 private var seed = Utils.random.nextLong() - + private var minCount = 5 + /** * Sets vector size (default: 100). */ @@ -114,6 +115,15 @@ class Word2Vec extends Serializable with Logging { this } + /** + * Sets minCount, the minimum number of times a token must appear to be included in the word2vec + * model's vocabulary (default: 5). + */ + def setMinCount(minCount: Int): this.type = { + this.minCount = minCount + this + } + private val EXP_TABLE_SIZE = 1000 private val MAX_EXP = 6 private val MAX_CODE_LENGTH = 40 @@ -122,9 +132,6 @@ class Word2Vec extends Serializable with Logging { /** context words from [-window, window] */ private val window = 5 - /** minimum frequency to consider a vocabulary word */ - private val minCount = 5 - private var trainWordsCount = 0 private var vocabSize = 0 private var vocab: Array[VocabWord] = null diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 47d1a76fa361d..01f3f90577142 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -268,7 +268,7 @@ object Vectors { * @param p norm. * @return norm in L^p^ space. */ - private[spark] def norm(vector: Vector, p: Double): Double = { + def norm(vector: Vector, p: Double): Double = { require(p >= 1.0) val values = vector match { case dv: DenseVector => dv.values From 040d6f2d13b132b3ef2a1e4f12f9f0e781c5a0b8 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 29 Dec 2014 17:17:12 -0800 Subject: [PATCH 144/227] [SPARK-4972][MLlib] Updated the scala doc for lasso and ridge regression for the change of LeastSquaresGradient In #SPARK-4907, we added factor of 2 into the LeastSquaresGradient. We updated the scala doc for lasso and ridge regression here. Author: DB Tsai Closes #3808 from dbtsai/doc and squashes the following commits: ec3c989 [DB Tsai] first commit --- .../main/scala/org/apache/spark/mllib/regression/Lasso.scala | 2 +- .../org/apache/spark/mllib/regression/RidgeRegression.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index f9791c6571782..8ecd5c6ad93c0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -45,7 +45,7 @@ class LassoModel ( /** * Train a regression model with L1-regularization using Stochastic Gradient Descent. * This solves the l1-regularized least squares regression formulation - * f(weights) = 1/n ||A weights-y||^2 + regParam ||weights||_1 + * f(weights) = 1/2n ||A weights-y||^2 + regParam ||weights||_1 * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with * its corresponding right hand side label y. * See also the documentation for the precise formulation. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index c8cad773f5efb..076ba35051c9d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -45,7 +45,7 @@ class RidgeRegressionModel ( /** * Train a regression model with L2-regularization using Stochastic Gradient Descent. * This solves the l1-regularized least squares regression formulation - * f(weights) = 1/n ||A weights-y||^2 + regParam/2 ||weights||^2 + * f(weights) = 1/2n ||A weights-y||^2 + regParam/2 ||weights||^2 * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with * its corresponding right hand side label y. * See also the documentation for the precise formulation. From 9077e721cd36adfecd50cbd1fd7735d28e5be8b5 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 30 Dec 2014 09:19:47 -0800 Subject: [PATCH 145/227] [SPARK-4920][UI] add version on master and worker page for standalone mode Author: Zhang, Liye Closes #3769 from liyezhang556520/spark-4920_WebVersion and squashes the following commits: 3bb7e0d [Zhang, Liye] add version on master and worker page --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 7486cb6b1bbc0..b5022fe853c49 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -234,8 +234,9 @@ private[spark] object UIUtils extends Logging {

- + + {org.apache.spark.SPARK_VERSION} {title}

From efa80a531ecd485f6cf0cdc24ffa42ba17eea46d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 30 Dec 2014 09:29:52 -0800 Subject: [PATCH 146/227] [SPARK-4882] Register PythonBroadcast with Kryo so that PySpark works with KryoSerializer This PR fixes an issue where PySpark broadcast variables caused NullPointerExceptions if KryoSerializer was used. The fix is to register PythonBroadcast with Kryo so that it's deserialized with a KryoJavaSerializer. Author: Josh Rosen Closes #3831 from JoshRosen/SPARK-4882 and squashes the following commits: 0466c7a [Josh Rosen] Register PythonBroadcast with Kryo. d5b409f [Josh Rosen] Enable registrationRequired, which would have caught this bug. 069d8a7 [Josh Rosen] Add failing test for SPARK-4882 --- .../spark/serializer/KryoSerializer.scala | 2 + .../api/python/PythonBroadcastSuite.scala | 60 +++++++++++++++++++ 2 files changed, 62 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala 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 621a951c27d07..d2947dcea4f7c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -26,6 +26,7 @@ import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializ import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.spark._ +import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} import org.apache.spark.scheduler.MapStatus @@ -90,6 +91,7 @@ class KryoSerializer(conf: SparkConf) // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) + kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) try { // Use the default classloader when calling the user registrator. diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala new file mode 100644 index 0000000000000..8959a843dbd7d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -0,0 +1,60 @@ +/* + * 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.api.python + +import scala.io.Source + +import java.io.{PrintWriter, File} + +import org.scalatest.{Matchers, FunSuite} + +import org.apache.spark.{SharedSparkContext, SparkConf} +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.util.Utils + +// This test suite uses SharedSparkContext because we need a SparkEnv in order to deserialize +// a PythonBroadcast: +class PythonBroadcastSuite extends FunSuite with Matchers with SharedSparkContext { + test("PythonBroadcast can be serialized with Kryo (SPARK-4882)") { + val tempDir = Utils.createTempDir() + val broadcastedString = "Hello, world!" + def assertBroadcastIsValid(broadcast: PythonBroadcast): Unit = { + val source = Source.fromFile(broadcast.path) + val contents = source.mkString + source.close() + contents should be (broadcastedString) + } + try { + val broadcastDataFile: File = { + val file = new File(tempDir, "broadcastData") + val printWriter = new PrintWriter(file) + printWriter.write(broadcastedString) + printWriter.close() + file + } + val broadcast = new PythonBroadcast(broadcastDataFile.getAbsolutePath) + assertBroadcastIsValid(broadcast) + val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val deserializedBroadcast = + Utils.clone[PythonBroadcast](broadcast, new KryoSerializer(conf).newInstance()) + assertBroadcastIsValid(deserializedBroadcast) + } finally { + Utils.deleteRecursively(tempDir) + } + } +} From 480bd1d2edd1de06af607b0cf3ff3c0b16089add Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 30 Dec 2014 11:24:46 -0800 Subject: [PATCH 147/227] [SPARK-4908][SQL] Prevent multiple concurrent hive native commands This is just a quick fix that locks when calling `runHive`. If we can find a way to avoid the error without a global lock that would be better. Author: Michael Armbrust Closes #3834 from marmbrus/hiveConcurrency and squashes the following commits: bf25300 [Michael Armbrust] prevent multiple concurrent hive native commands --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 2 +- .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 56fe27a77b838..982e0593fcfd1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -284,7 +284,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * Execute the command using Hive and return the results as a sequence. Each element * in the sequence is one row. */ - protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { + protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = synchronized { try { val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") 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 4d81acc753a27..fb6da33e88ef6 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 @@ -56,6 +56,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { Locale.setDefault(originalLocale) } + test("SPARK-4908: concurent hive native commands") { + (1 to 100).par.map { _ => + sql("USE default") + sql("SHOW TABLES") + } + } + createQueryTest("constant object inspector for generic udf", """SELECT named_struct( lower("AA"), "10", From 94d60b7021960dc10d98039dbc6ad7193e8557f5 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 30 Dec 2014 11:29:13 -0800 Subject: [PATCH 148/227] [SQL] enable view test This is a follow up of #3396 , just add a test to white list. Author: Daoyuan Wang Closes #3826 from adrian-wang/viewtest and squashes the following commits: f105f68 [Daoyuan Wang] enable view test --- .../execution/HiveCompatibilitySuite.scala | 3 +- ...anslate-0-dc7fc9ce5109ef459ee84ccfbb12d2c0 | 0 ...anslate-1-3896ae0e680a5fdc01833533b11c07bb | 0 ...nslate-10-7016e1e3a4248564f3d08cddad7ae116 | 0 ...nslate-11-e27c6a59a833dcbc2e5cdb7ff7972828 | 0 ...anslate-2-6b4caec6d7e3a91e61720bbd6b7697f0 | 0 ...anslate-3-30dc3e80e3873af5115e4f5e39078a13 | 27 ++++++++++++++++ ...anslate-4-cefb7530126f9e60cb4a29441d578f23 | 0 ...anslate-5-856ea995681b18a543dc0e53b8b43a8e | 32 +++++++++++++++++++ ...anslate-6-a14cfe3eff322066e61023ec06c7735d | 0 ...anslate-7-e947bf2dacc907825df154a4131a3fcc | 0 ...anslate-8-b1a99b0beffb0b298aec9233ecc0707f | 0 ...anslate-9-fc0dc39c4796d917685e0797bc4a9786 | 0 13 files changed, 61 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-0-dc7fc9ce5109ef459ee84ccfbb12d2c0 create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-1-3896ae0e680a5fdc01833533b11c07bb create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-10-7016e1e3a4248564f3d08cddad7ae116 create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-11-e27c6a59a833dcbc2e5cdb7ff7972828 create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-2-6b4caec6d7e3a91e61720bbd6b7697f0 create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-3-30dc3e80e3873af5115e4f5e39078a13 create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-4-cefb7530126f9e60cb4a29441d578f23 create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-5-856ea995681b18a543dc0e53b8b43a8e create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-6-a14cfe3eff322066e61023ec06c7735d create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-7-e947bf2dacc907825df154a4131a3fcc create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-8-b1a99b0beffb0b298aec9233ecc0707f create mode 100644 sql/hive/src/test/resources/golden/create_view_translate-9-fc0dc39c4796d917685e0797bc4a9786 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 1e44dd239458a..23283fd3fe6b1 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -101,6 +101,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "describe_comment_nonascii", "create_merge_compressed", + "create_view", "create_view_partitioned", "database_location", "database_properties", @@ -110,7 +111,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Weird DDL differences result in failures on jenkins. "create_like2", - "create_view_translate", "partitions_json", // This test is totally fine except that it includes wrong queries and expects errors, but error @@ -349,6 +349,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_nested_type", "create_skewed_table1", "create_struct_table", + "create_view_translate", "cross_join", "cross_product_check_1", "cross_product_check_2", diff --git a/sql/hive/src/test/resources/golden/create_view_translate-0-dc7fc9ce5109ef459ee84ccfbb12d2c0 b/sql/hive/src/test/resources/golden/create_view_translate-0-dc7fc9ce5109ef459ee84ccfbb12d2c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-1-3896ae0e680a5fdc01833533b11c07bb b/sql/hive/src/test/resources/golden/create_view_translate-1-3896ae0e680a5fdc01833533b11c07bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-10-7016e1e3a4248564f3d08cddad7ae116 b/sql/hive/src/test/resources/golden/create_view_translate-10-7016e1e3a4248564f3d08cddad7ae116 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-11-e27c6a59a833dcbc2e5cdb7ff7972828 b/sql/hive/src/test/resources/golden/create_view_translate-11-e27c6a59a833dcbc2e5cdb7ff7972828 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-2-6b4caec6d7e3a91e61720bbd6b7697f0 b/sql/hive/src/test/resources/golden/create_view_translate-2-6b4caec6d7e3a91e61720bbd6b7697f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-3-30dc3e80e3873af5115e4f5e39078a13 b/sql/hive/src/test/resources/golden/create_view_translate-3-30dc3e80e3873af5115e4f5e39078a13 new file mode 100644 index 0000000000000..cec5f77033aa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_view_translate-3-30dc3e80e3873af5115e4f5e39078a13 @@ -0,0 +1,27 @@ +# col_name data_type comment + +key string + +# Detailed Table Information +Database: default +Owner: animal +CreateTime: Mon Dec 29 00:57:55 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Table Type: VIRTUAL_VIEW +Table Parameters: + transient_lastDdlTime 1419843475 + +# Storage Information +SerDe Library: null +InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] + +# View Information +View Original Text: select cast(key as string) from src +View Expanded Text: select cast(`src`.`key` as string) from `default`.`src` diff --git a/sql/hive/src/test/resources/golden/create_view_translate-4-cefb7530126f9e60cb4a29441d578f23 b/sql/hive/src/test/resources/golden/create_view_translate-4-cefb7530126f9e60cb4a29441d578f23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-5-856ea995681b18a543dc0e53b8b43a8e b/sql/hive/src/test/resources/golden/create_view_translate-5-856ea995681b18a543dc0e53b8b43a8e new file mode 100644 index 0000000000000..bf582fc0964a3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_view_translate-5-856ea995681b18a543dc0e53b8b43a8e @@ -0,0 +1,32 @@ +# col_name data_type comment + +key int +value string + +# Detailed Table Information +Database: default +Owner: animal +CreateTime: Mon Dec 29 00:57:55 PST 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Table Type: VIRTUAL_VIEW +Table Parameters: + transient_lastDdlTime 1419843475 + +# Storage Information +SerDe Library: null +InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] + +# View Information +View Original Text: select key, value from ( + select key, value from src +) a +View Expanded Text: select key, value from ( + select `src`.`key`, `src`.`value` from `default`.`src` +) `a` diff --git a/sql/hive/src/test/resources/golden/create_view_translate-6-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/create_view_translate-6-a14cfe3eff322066e61023ec06c7735d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-7-e947bf2dacc907825df154a4131a3fcc b/sql/hive/src/test/resources/golden/create_view_translate-7-e947bf2dacc907825df154a4131a3fcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-8-b1a99b0beffb0b298aec9233ecc0707f b/sql/hive/src/test/resources/golden/create_view_translate-8-b1a99b0beffb0b298aec9233ecc0707f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_translate-9-fc0dc39c4796d917685e0797bc4a9786 b/sql/hive/src/test/resources/golden/create_view_translate-9-fc0dc39c4796d917685e0797bc4a9786 new file mode 100644 index 0000000000000..e69de29bb2d1d From 65357f11c25a7c91577df5da31ebf349d7845eef Mon Sep 17 00:00:00 2001 From: scwf Date: Tue, 30 Dec 2014 11:30:47 -0800 Subject: [PATCH 149/227] [SPARK-4975][SQL] Fix HiveInspectorSuite test failure MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit HiveInspectorSuite test failure: [info] - wrap / unwrap null, constant null and writables *** FAILED *** (21 milliseconds) [info] 1 did not equal 0 (HiveInspectorSuite.scala:136) this is because the origin date(is 3914-10-23) not equals the date returned by ```unwrap```(is 3914-10-22). Setting TimeZone and Locale fix this. Another minor change here is rename ```def checkValues(v1: Any, v2: Any): Unit``` to ```def checkValue(v1: Any, v2: Any): Unit ``` to make the code more clear Author: scwf Author: Fei Wang Closes #3814 from scwf/fix-inspectorsuite and squashes the following commits: d8531ef [Fei Wang] Delete test.log 72b19a9 [scwf] fix HiveInspectorSuite test error --- .../spark/sql/hive/HiveInspectorSuite.scala | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index bfe608a51a30b..f90d3607915ae 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.sql.Date import java.util +import java.util.{Locale, TimeZone} import org.apache.hadoop.hive.serde2.io.DoubleWritable import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory @@ -63,6 +64,11 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { .get()) } + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + val data = Literal(true) :: Literal(0.asInstanceOf[Byte]) :: @@ -121,11 +127,11 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { def checkValues(row1: Seq[Any], row2: Seq[Any]): Unit = { row1.zip(row2).map { - case (r1, r2) => checkValues(r1, r2) + case (r1, r2) => checkValue(r1, r2) } } - def checkValues(v1: Any, v2: Any): Unit = { + def checkValue(v1: Any, v2: Any): Unit = { (v1, v2) match { case (r1: Decimal, r2: Decimal) => // Ignore the Decimal precision @@ -195,26 +201,26 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { }) checkValues(row, unwrap(wrap(row, toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) - checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) } test("wrap / unwrap Array Type") { val dt = ArrayType(dataTypes(0)) val d = row(0) :: row(0) :: Nil - checkValues(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) - checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValues(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValues(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) + checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) } test("wrap / unwrap Map Type") { val dt = MapType(dataTypes(0), dataTypes(1)) val d = Map(row(0) -> row(1)) - checkValues(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) - checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValues(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValues(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) + checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) } } From 5595eaa74f139fdb6fd8a7bb0ca6ed421ef00ac8 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 30 Dec 2014 11:33:47 -0800 Subject: [PATCH 150/227] [SPARK-4959] [SQL] Attributes are case sensitive when using a select query from a projection Author: Cheng Hao Closes #3796 from chenghao-intel/spark_4959 and squashes the following commits: 3ec08f8 [Cheng Hao] Replace the attribute in comparing its exprId other than itself --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 8 ++++---- .../sql/hive/execution/HiveTableScanSuite.scala | 14 +++++++++++++- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 806c1394eb151..0f2eae6400d21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -142,16 +142,16 @@ object ColumnPruning extends Rule[LogicalPlan] { case Project(projectList1, Project(projectList2, child)) => // Create a map of Aliases to their values from the child projection. // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). - val aliasMap = projectList2.collect { - case a @ Alias(e, _) => (a.toAttribute: Expression, a) - }.toMap + val aliasMap = AttributeMap(projectList2.collect { + case a @ Alias(e, _) => (a.toAttribute, a) + }) // Substitute any attributes that are produced by the child projection, so that we safely // eliminate it. // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' // TODO: Fix TransformBase to avoid the cast below. val substitutedProjection = projectList1.map(_.transform { - case a if aliasMap.contains(a) => aliasMap(a) + case a: Attribute if aliasMap.contains(a) => aliasMap(a) }).asInstanceOf[Seq[NamedExpression]] Project(substitutedProjection, child) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index a0ace91060a28..16f77a438e1ae 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.{Row, SchemaRDD} +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.Row import org.apache.spark.util.Utils @@ -76,4 +77,15 @@ class HiveTableScanSuite extends HiveComparisonTest { === Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")),Row(null))) TestHive.sql("DROP TABLE timestamp_query_null") } + + test("Spark-4959 Attributes are case sensitive when using a select query from a projection") { + sql("create table spark_4959 (col1 string)") + sql("""insert into table spark_4959 select "hi" from src limit 1""") + table("spark_4959").select( + 'col1.as('CaseSensitiveColName), + 'col1.as('CaseSensitiveColName2)).registerTempTable("spark_4959_2") + + assert(sql("select CaseSensitiveColName from spark_4959_2").first() === Row("hi")) + assert(sql("select casesensitivecolname from spark_4959_2").first() === Row("hi")) + } } From 63b84b7d6785a687dd7f4c0e2bb1e348800d30d8 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 30 Dec 2014 11:47:08 -0800 Subject: [PATCH 151/227] [SPARK-4904] [SQL] Remove the unnecessary code change in Generic UDF Since #3429 has been merged, the bug of wrapping to Writable for HiveGenericUDF is resolved, we can safely remove the foldable checking in `HiveGenericUdf.eval`, which discussed in #2802. Author: Cheng Hao Closes #3745 from chenghao-intel/generic_udf and squashes the following commits: 622ad03 [Cheng Hao] Remove the unnecessary code change in Generic UDF --- .../src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 93b6ef9fbc59b..7d863f9d89dae 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -158,11 +158,6 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr override def foldable = isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] - @transient - protected def constantReturnValue = unwrap( - returnInspector.asInstanceOf[ConstantObjectInspector].getWritableConstantValue(), - returnInspector) - @transient protected lazy val deferedObjects = argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] @@ -171,7 +166,6 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr override def eval(input: Row): Any = { returnInspector // Make sure initialized. - if(foldable) return constantReturnValue var i = 0 while (i < children.length) { From daac221302e0cf71a7b7bda31625134cf7b9dce1 Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 30 Dec 2014 12:07:24 -0800 Subject: [PATCH 152/227] [SPARK-5002][SQL] Using ascending by default when not specify order in order by spark sql does not support ```SELECT a, b FROM testData2 ORDER BY a desc, b```. Author: wangfei Closes #3838 from scwf/orderby and squashes the following commits: 114b64a [wangfei] remove nouse methods 48145d3 [wangfei] fix order, using asc by default --- .../scala/org/apache/spark/sql/catalyst/SqlParser.scala | 8 ++------ .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 7 +++++++ 2 files changed, 9 insertions(+), 6 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 d4fc9bbfd3118..66860a4c0923a 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 @@ -209,15 +209,11 @@ class SqlParser extends AbstractSparkSQLParser { ) protected lazy val ordering: Parser[Seq[SortOrder]] = - ( rep1sep(singleOrder, ",") - | rep1sep(expression, ",") ~ direction.? ^^ { - case exps ~ d => exps.map(SortOrder(_, d.getOrElse(Ascending))) + ( rep1sep(expression ~ direction.? , ",") ^^ { + case exps => exps.map(pair => SortOrder(pair._1, pair._2.getOrElse(Ascending))) } ) - protected lazy val singleOrder: Parser[SortOrder] = - expression ~ direction ^^ { case e ~ o => SortOrder(e, o) } - protected lazy val direction: Parser[SortDirection] = ( ASC ^^^ Ascending | DESC ^^^ Descending 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 ddf4776ecf7ae..add4e218a22ee 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 @@ -987,6 +987,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { ) } + test("oder by asc by default when not specify ascending and descending") { + checkAnswer( + sql("SELECT a, b FROM testData2 ORDER BY a desc, b"), + Seq((3, 1), (3, 2), (2, 1), (2,2), (1, 1), (1, 2)) + ) + } + test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) From 53f0a00b6051fb6cb52a90f91ae01bcd77e332c5 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 30 Dec 2014 12:11:44 -0800 Subject: [PATCH 153/227] [Spark-4512] [SQL] Unresolved Attribute Exception in Sort By It will cause exception while do query like: SELECT key+key FROM src sort by value; Author: Cheng Hao Closes #3386 from chenghao-intel/sort and squashes the following commits: 38c78cc [Cheng Hao] revert the SortPartition in SparkStrategies 7e9dd15 [Cheng Hao] update the typo fcd1d64 [Cheng Hao] rebase the latest master and update the SortBy unit test --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 ++-- .../sql/catalyst/analysis/Analyzer.scala | 13 +++++++------ .../spark/sql/catalyst/dsl/package.scala | 4 ++-- .../plans/logical/basicOperators.scala | 11 ++++++++++- .../org/apache/spark/sql/SchemaRDD.scala | 5 ++--- .../spark/sql/execution/SparkStrategies.scala | 11 +++++------ .../org/apache/spark/sql/DslQuerySuite.scala | 19 ++++++++++++++----- .../scala/org/apache/spark/sql/TestData.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 8 ++++---- .../hive/execution/HiveComparisonTest.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 7 +++++++ 11 files changed, 55 insertions(+), 31 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 66860a4c0923a..f79d4ff444dc0 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 @@ -204,8 +204,8 @@ class SqlParser extends AbstractSparkSQLParser { ) protected lazy val sortType: Parser[LogicalPlan => LogicalPlan] = - ( ORDER ~ BY ~> ordering ^^ { case o => l: LogicalPlan => Sort(o, l) } - | SORT ~ BY ~> ordering ^^ { case o => l: LogicalPlan => SortPartitions(o, l) } + ( ORDER ~ BY ~> ordering ^^ { case o => l: LogicalPlan => Sort(o, true, l) } + | SORT ~ BY ~> ordering ^^ { case o => l: LogicalPlan => Sort(o, false, l) } ) protected lazy val ordering: Parser[Seq[SortOrder]] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1c4088b8438e1..72680f37a0b4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -246,7 +246,7 @@ class Analyzer(catalog: Catalog, case p: LogicalPlan if !p.childrenResolved => p // If the projection list contains Stars, expand it. - case p@Project(projectList, child) if containsStar(projectList) => + case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { case s: Star => s.expand(child.output, resolver) @@ -310,7 +310,8 @@ class Analyzer(catalog: Catalog, */ object ResolveSortReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => + case s @ Sort(ordering, global, p @ Project(projectList, child)) + if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) val resolved = unresolved.flatMap(child.resolve(_, resolver)) val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) @@ -319,13 +320,14 @@ class Analyzer(catalog: Catalog, if (missingInProject.nonEmpty) { // Add missing attributes and then project them away after the sort. Project(projectList.map(_.toAttribute), - Sort(ordering, + Sort(ordering, global, Project(projectList ++ missingInProject, child))) } else { logDebug(s"Failed to find $missingInProject in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } - case s @ Sort(ordering, a @ Aggregate(grouping, aggs, child)) if !s.resolved && a.resolved => + case s @ Sort(ordering, global, a @ Aggregate(grouping, aggs, child)) + if !s.resolved && a.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) // A small hack to create an object that will allow us to resolve any references that // refer to named expressions that are present in the grouping expressions. @@ -340,8 +342,7 @@ class Analyzer(catalog: Catalog, if (missingInAggs.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. Project(a.output, - Sort(ordering, - Aggregate(grouping, aggs ++ missingInAggs, child))) + Sort(ordering, global, Aggregate(grouping, aggs ++ missingInAggs, child))) } else { s // Nothing we can do here. Return original plan. } 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 index fb252cdf51534..a14e5b9ef14d0 100755 --- 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 @@ -244,9 +244,9 @@ package object dsl { condition: Option[Expression] = None) = Join(logicalPlan, otherPlan, joinType, condition) - def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, logicalPlan) + def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, true, logicalPlan) - def sortBy(sortExprs: SortOrder*) = SortPartitions(sortExprs, logicalPlan) + def sortBy(sortExprs: SortOrder*) = Sort(sortExprs, false, logicalPlan) def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = { val aliasedExprs = aggregateExprs.map { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index a9282b98adfab..0b9f01cbae9ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -130,7 +130,16 @@ case class WriteToFile( override def output = child.output } -case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { +/** + * @param order The ordering expressions + * @param global True means global sorting apply for entire data set, + * False means sorting only apply within the partition. + * @param child Child logical plan + */ +case class Sort( + order: Seq[SortOrder], + global: Boolean, + child: LogicalPlan) extends UnaryNode { override def output = child.output } 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 856b10f1a8fd8..80787b61ce1bf 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 @@ -214,7 +214,7 @@ class SchemaRDD( * @group Query */ def orderBy(sortExprs: SortOrder*): SchemaRDD = - new SchemaRDD(sqlContext, Sort(sortExprs, logicalPlan)) + new SchemaRDD(sqlContext, Sort(sortExprs, true, logicalPlan)) /** * Sorts the results by the given expressions within partition. @@ -227,7 +227,7 @@ class SchemaRDD( * @group Query */ def sortBy(sortExprs: SortOrder*): SchemaRDD = - new SchemaRDD(sqlContext, SortPartitions(sortExprs, logicalPlan)) + new SchemaRDD(sqlContext, Sort(sortExprs, false, logicalPlan)) @deprecated("use limit with integer argument", "1.1.0") def limit(limitExpr: Expression): SchemaRDD = @@ -238,7 +238,6 @@ class SchemaRDD( * {{{ * schemaRDD.limit(10) * }}} - * * @group Query */ def limit(limitNum: Int): SchemaRDD = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 2954d4ce7d2d8..9151da69ed44c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -190,7 +190,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object TakeOrdered extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) => + case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => execution.TakeOrdered(limit, order, planLater(child)) :: Nil case _ => Nil } @@ -257,15 +257,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Distinct(partial = false, execution.Distinct(partial = true, planLater(child))) :: Nil - case logical.Sort(sortExprs, child) if sqlContext.externalSortEnabled => - execution.ExternalSort(sortExprs, global = true, planLater(child)):: Nil - case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, global = true, planLater(child)):: Nil - case logical.SortPartitions(sortExprs, child) => // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. execution.Sort(sortExprs, global = false, planLater(child)) :: Nil + case logical.Sort(sortExprs, global, child) if sqlContext.externalSortEnabled => + execution.ExternalSort(sortExprs, global, planLater(child)):: Nil + case logical.Sort(sortExprs, global, child) => + execution.Sort(sortExprs, global, planLater(child)):: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => 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 691c4b38287bf..c0b9cf5163120 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 @@ -88,7 +88,7 @@ class DslQuerySuite extends QueryTest { Seq(Seq(6))) } - test("sorting") { + test("global sorting") { checkAnswer( testData2.orderBy('a.asc, 'b.asc), Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) @@ -122,22 +122,31 @@ class DslQuerySuite extends QueryTest { mapData.collect().sortBy(_.data(1)).reverse.toSeq) } - test("sorting #2") { + test("partition wide sorting") { + // 2 partitions totally, and + // Partition #1 with values: + // (1, 1) + // (1, 2) + // (2, 1) + // Partition #2 with values: + // (2, 2) + // (3, 1) + // (3, 2) checkAnswer( testData2.sortBy('a.asc, 'b.asc), Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) checkAnswer( testData2.sortBy('a.asc, 'b.desc), - Seq((1,2), (1,1), (2,2), (2,1), (3,2), (3,1))) + Seq((1,2), (1,1), (2,1), (2,2), (3,2), (3,1))) checkAnswer( testData2.sortBy('a.desc, 'b.desc), - Seq((3,2), (3,1), (2,2), (2,1), (1,2), (1,1))) + Seq((2,1), (1,2), (1,1), (3,2), (3,1), (2,2))) checkAnswer( testData2.sortBy('a.desc, 'b.asc), - Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + Seq((2,1), (1,1), (1,2), (3,1), (3,2), (2,2))) } test("limit") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index bb553a0a1e50c..497897c3c0d4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -55,7 +55,7 @@ object TestData { TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: - TestData2(3, 2) :: Nil).toSchemaRDD + TestData2(3, 2) :: Nil, 2).toSchemaRDD testData2.registerTempTable("testData2") case class DecimalData(a: BigDecimal, b: BigDecimal) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 3f3d9e7cd4fbe..8a9613cf96e54 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -680,16 +680,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), withHaving) + Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withHaving) case (None, Some(perPartitionOrdering), None, None) => - SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withHaving) + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withHaving) case (None, None, Some(partitionExprs), None) => Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving) case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => - SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving)) case (None, None, None, Some(clusterExprs)) => - SortPartitions(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), + Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false, Repartition(clusterExprs.getChildren.map(nodeToExpr), withHaving)) case (None, None, None, None) => withHaving case _ => sys.error("Unsupported set of ordering / distribution clauses.") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 8011f9b8773b3..4104df8f8e022 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -132,7 +132,7 @@ abstract class HiveComparisonTest def isSorted(plan: LogicalPlan): Boolean = plan match { case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false - case PhysicalOperation(_, _, Sort(_, _)) => true + case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) } 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 f57f31af15566..5d0fb7237011f 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 @@ -32,6 +32,13 @@ case class Nested3(f3: Int) * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") { + checkAnswer( + sql("SELECT * FROM (SELECT key + key AS a FROM src SORT BY value) t ORDER BY t.a"), + sql("SELECT key + key as a FROM src ORDER BY a").collect().toSeq + ) + } + test("CTAS with serde") { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect sql( From 19a8802e703e6b075a148ba73dc9dd80748d6322 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 30 Dec 2014 12:16:45 -0800 Subject: [PATCH 154/227] [SPARK-4493][SQL] Tests for IsNull / IsNotNull in the ParquetFilterSuite This is a follow-up of #3367 and #3644. At the time #3644 was written, #3367 hadn't been merged yet, thus `IsNull` and `IsNotNull` filters are not covered in the first version of `ParquetFilterSuite`. This PR adds corresponding test cases. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3748) Author: Cheng Lian Closes #3748 from liancheng/test-null-filters and squashes the following commits: 1ab943f [Cheng Lian] IsNull and IsNotNull Parquet filter test case for boolean type bcd616b [Cheng Lian] Adds Parquet filter pushedown tests for IsNull and IsNotNull --- .../sql/parquet/ParquetFilterSuite.scala | 60 +++++++++++++++---- 1 file changed, 50 insertions(+), 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index b17300475b6f6..4c3a04506ce42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -28,11 +28,14 @@ import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD} /** * A test suite that tests Parquet filter2 API based filter pushdown optimization. * - * Notice that `!(a cmp b)` are always transformed to its negated form `a cmp' b` by the - * `BooleanSimplification` optimization rule whenever possible. As a result, predicate `!(a < 1)` - * results a `GtEq` filter predicate rather than a `Not`. + * NOTE: * - * @todo Add test cases for `IsNull` and `IsNotNull` after merging PR #3367 + * 1. `!(a cmp b)` is always transformed to its negated form `a cmp' b` by the + * `BooleanSimplification` optimization rule whenever possible. As a result, predicate `!(a < 1)` + * results in a `GtEq` filter predicate rather than a `Not`. + * + * 2. `Tuple1(Option(x))` is used together with `AnyVal` types like `Int` to ensure the inferred + * data type is nullable. */ class ParquetFilterSuite extends QueryTest with ParquetTest { val sqlContext = TestSQLContext @@ -85,14 +88,26 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - boolean") { - withParquetRDD((true :: false :: Nil).map(Tuple1.apply)) { rdd => + withParquetRDD((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Boolean]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Boolean]]) { + Seq(Row(true), Row(false)) + } + checkFilterPushdown(rdd, '_1)('_1 === true, classOf[Eq[java.lang.Boolean]])(true) - checkFilterPushdown(rdd, '_1)('_1 !== true, classOf[Operators.NotEq[java.lang.Boolean]])(false) + checkFilterPushdown(rdd, '_1)('_1 !== true, classOf[Operators.NotEq[java.lang.Boolean]]) { + false + } } } test("filter pushdown - integer") { - withParquetRDD((1 to 4).map(Tuple1.apply)) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[Integer]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[Integer]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[Integer]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[Integer]]) { (2 to 4).map(Row.apply(_)) @@ -118,7 +133,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - long") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toLong))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toLong)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Long]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Long]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Long]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Long]]) { (2 to 4).map(Row.apply(_)) @@ -144,7 +164,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - float") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toFloat))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Float]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Float]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Float]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Float]]) { (2 to 4).map(Row.apply(_)) @@ -170,7 +195,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - double") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toDouble))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Double]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Double]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Double]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Double]]) { (2 to 4).map(Row.apply(_)) @@ -197,6 +227,11 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { test("filter pushdown - string") { withParquetRDD((1 to 4).map(i => Tuple1(i.toString))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.String]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.String]]) { + (1 to 4).map(i => Row.apply(i.toString)) + } + checkFilterPushdown(rdd, '_1)('_1 === "1", classOf[Eq[String]])("1") checkFilterPushdown(rdd, '_1)('_1 !== "1", classOf[Operators.NotEq[String]]) { (2 to 4).map(i => Row.apply(i.toString)) @@ -227,6 +262,11 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { rdd => + checkBinaryFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.String]])(Seq.empty[Row]) + checkBinaryFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.String]]) { + (1 to 4).map(i => Row.apply(i.b)).toSeq + } + checkBinaryFilterPushdown(rdd, '_1)('_1 === 1.b, classOf[Eq[Array[Byte]]])(1.b) checkBinaryFilterPushdown(rdd, '_1)('_1 !== 1.b, classOf[Operators.NotEq[Array[Byte]]]) { (2 to 4).map(i => Row.apply(i.b)).toSeq From f7a41a0e79561a722e41800257dca886732ccaad Mon Sep 17 00:00:00 2001 From: luogankun Date: Tue, 30 Dec 2014 12:17:49 -0800 Subject: [PATCH 155/227] [SPARK-4916][SQL][DOCS]Update SQL programming guide about cache section `SchemeRDD.cache()` now uses in-memory columnar storage. Author: luogankun Closes #3759 from luogankun/SPARK-4916 and squashes the following commits: 7b39864 [luogankun] [SPARK-4916]Update SQL programming guide 6018122 [luogankun] Merge branch 'master' of https://github.com/apache/spark into SPARK-4916 0b93785 [luogankun] [SPARK-4916]Update SQL programming guide 99b2336 [luogankun] [SPARK-4916]Update SQL programming guide --- docs/sql-programming-guide.md | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2aea8a8aedafc..1b5fde991e405 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -831,13 +831,10 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")`. +Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `schemaRDD.cache()`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Note that if you call `schemaRDD.cache()` rather than `sqlContext.cacheTable(...)`, tables will _not_ be cached using -the in-memory columnar format, and therefore `sqlContext.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. From 2deac748b4e1245c2cb9bd43ad87c80d6d130a83 Mon Sep 17 00:00:00 2001 From: luogankun Date: Tue, 30 Dec 2014 12:18:55 -0800 Subject: [PATCH 156/227] [SPARK-4930][SQL][DOCS]Update SQL programming guide, CACHE TABLE is eager `CACHE TABLE tbl` is now __eager__ by default not __lazy__ Author: luogankun Closes #3773 from luogankun/SPARK-4930 and squashes the following commits: cc17b7d [luogankun] [SPARK-4930][SQL][DOCS]Update SQL programming guide, add CACHE [LAZY] TABLE [AS SELECT] ... bffe0e8 [luogankun] [SPARK-4930][SQL][DOCS]Update SQL programming guide, CACHE TABLE tbl is eager --- docs/sql-programming-guide.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 1b5fde991e405..729045b81a8c0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1007,12 +1007,11 @@ let user control table caching explicitly: CACHE TABLE logs_last_month; UNCACHE TABLE logs_last_month; -**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`: +**NOTE:** `CACHE TABLE tbl` is now __eager__ by default not __lazy__. Don’t need to trigger cache materialization manually anymore. - CACHE TABLE logs_last_month; - SELECT COUNT(1) FROM logs_last_month; +Spark SQL newly introduced a statement to let user control table caching whether or not lazy since Spark 1.2.0: + + CACHE [LAZY] TABLE [AS SELECT] ... Several caching related features are not supported yet: From a75dd83b72586695768c89ed32b240aa8f48f32c Mon Sep 17 00:00:00 2001 From: guowei2 Date: Tue, 30 Dec 2014 12:21:00 -0800 Subject: [PATCH 157/227] [SPARK-4928][SQL] Fix: Operator '>,<,>=,<=' with decimal between different precision report error case operator with decimal between different precision, we need change them to unlimited Author: guowei2 Closes #3767 from guowei2/SPARK-4928 and squashes the following commits: c6a6e3e [guowei2] fix code style 3214e0a [guowei2] add test case b4985a2 [guowei2] fix code style 27adf42 [guowei2] Fix: Operation '>,<,>=,<=' with Decimal report error --- .../catalyst/analysis/HiveTypeCoercion.scala | 16 ++++++++++++++++ .../analysis/DecimalPrecisionSuite.scala | 17 +++++++++++++++++ 2 files changed, 33 insertions(+) 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 e38114ab3cf25..242f28f670298 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 @@ -361,6 +361,22 @@ trait HiveTypeCoercion { DecimalType(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) ) + case LessThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case LessThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b: BinaryExpression if b.left.dataType != b.right.dataType => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index d5b7d2789a103..3677a6e72e23a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -49,6 +49,15 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { assert(analyzer(plan).schema.fields(0).dataType === expectedType) } + private def checkComparison(expression: Expression, expectedType: DataType): Unit = { + val plan = Project(Alias(expression, "c")() :: Nil, relation) + val comparison = analyzer(plan).collect { + case Project(Alias(e: BinaryComparison, _) :: Nil, _) => e + }.head + assert(comparison.left.dataType === expectedType) + assert(comparison.right.dataType === expectedType) + } + test("basic operations") { checkType(Add(d1, d2), DecimalType(6, 2)) checkType(Subtract(d1, d2), DecimalType(6, 2)) @@ -65,6 +74,14 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { checkType(Add(Add(d1, d2), Add(d1, d2)), DecimalType(7, 2)) } + test("Comparison operations") { + checkComparison(LessThan(i, d1), DecimalType.Unlimited) + checkComparison(LessThanOrEqual(d1, d2), DecimalType.Unlimited) + checkComparison(GreaterThan(d2, u), DecimalType.Unlimited) + checkComparison(GreaterThanOrEqual(d1, f), DoubleType) + checkComparison(GreaterThan(d2, d2), DecimalType(5, 2)) + } + test("bringing in primitive types") { checkType(Add(d1, i), DecimalType(12, 1)) checkType(Add(d1, f), DoubleType) From 61a99f6a11d85e931e7d60f9ab4370b3b40a52ef Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 30 Dec 2014 13:38:27 -0800 Subject: [PATCH 158/227] [SPARK-4937][SQL] Normalizes conjunctions and disjunctions to eliminate common predicates This PR is a simplified version of several filter optimization rules introduced in #3778 authored by scwf. Newly introduced optimizations include: 1. `a && a` => `a` 2. `a || a` => `a` 3. `(a || b || c || ...) && (a || b || d || ...)` => `a && b && (c || d || ...)` The 3rd rule is particularly useful for optimizing the following query, which is planned into a cartesian product ```sql SELECT * FROM t1, t2 WHERE (t1.key = t2.key AND t1.value > 10) OR (t1.key = t2.key AND t2.value < 20) ``` to the following one, which is planned into an equi-join: ```sql SELECT * FROM t1, t2 WHERE t1.key = t2.key AND (t1.value > 10 OR t2.value < 20) ``` The example above is quite artificial, but common predicates are likely to appear in real life complex queries (like the one mentioned in #3778). A difference between this PR and #3778 is that these optimizations are not limited to `Filter`, but are generalized to all logical plan nodes. Thanks to scwf for bringing up these optimizations, and chenghao-intel for the generalization suggestion. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3784) Author: Cheng Lian Closes #3784 from liancheng/normalize-filters and squashes the following commits: caca560 [Cheng Lian] Moves filter normalization into BooleanSimplification rule 4ab3a58 [Cheng Lian] Fixes test failure, adds more tests 5d54349 [Cheng Lian] Fixes typo in comment 2abbf8e [Cheng Lian] Forgot our sacred Apache licence header... cf95639 [Cheng Lian] Adds an optimization rule for filter normalization --- .../sql/catalyst/expressions/predicates.scala | 9 ++- .../sql/catalyst/optimizer/Optimizer.scala | 27 +++++-- .../optimizer/NormalizeFiltersSuite.scala | 72 +++++++++++++++++++ .../columnar/PartitionBatchPruningSuite.scala | 10 ++- 4 files changed, 110 insertions(+), 8 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFiltersSuite.scala 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 94b6fb084d38a..cb5ff67959868 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 @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import scala.collection.immutable.HashSet import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.BooleanType @@ -48,6 +47,14 @@ trait PredicateHelper { } } + protected def splitDisjunctivePredicates(condition: Expression): Seq[Expression] = { + condition match { + case Or(cond1, cond2) => + splitDisjunctivePredicates(cond1) ++ splitDisjunctivePredicates(cond2) + case other => other :: Nil + } + } + /** * Returns true if `expr` can be evaluated using only the output of `plan`. This method * can be used to determine when is is acceptable to move expression evaluation within a query diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 0f2eae6400d21..cd3137980ca43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -294,11 +294,16 @@ object OptimizeIn extends Rule[LogicalPlan] { } /** - * Simplifies boolean expressions where the answer can be determined without evaluating both sides. + * Simplifies boolean expressions: + * + * 1. Simplifies expressions whose answer can be determined without evaluating both sides. + * 2. Eliminates / extracts common factors. + * 3. Removes `Not` operator. + * * Note that this rule can eliminate expressions that might otherwise have been evaluated and thus * is only safe when evaluations of expressions does not result in side effects. */ -object BooleanSimplification extends Rule[LogicalPlan] { +object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case and @ And(left, right) => @@ -307,7 +312,9 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (l, Literal(true, BooleanType)) => l case (Literal(false, BooleanType), _) => Literal(false) case (_, Literal(false, BooleanType)) => Literal(false) - case (_, _) => and + // a && a && a ... => a + case _ if splitConjunctivePredicates(and).distinct.size == 1 => left + case _ => and } case or @ Or(left, right) => @@ -316,7 +323,19 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (_, Literal(true, BooleanType)) => Literal(true) case (Literal(false, BooleanType), r) => r case (l, Literal(false, BooleanType)) => l - case (_, _) => or + // a || a || a ... => a + case _ if splitDisjunctivePredicates(or).distinct.size == 1 => left + // (a && b && c && ...) || (a && b && d && ...) => a && b && (c || d || ...) + case _ => + val lhsSet = splitConjunctivePredicates(left).toSet + val rhsSet = splitConjunctivePredicates(right).toSet + val common = lhsSet.intersect(rhsSet) + + (lhsSet.diff(common).reduceOption(And) ++ rhsSet.diff(common).reduceOption(And)) + .reduceOption(Or) + .map(_ :: common.toList) + .getOrElse(common.toList) + .reduce(And) } case not @ Not(exp) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFiltersSuite.scala new file mode 100644 index 0000000000000..906300d8336cb --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFiltersSuite.scala @@ -0,0 +1,72 @@ +/* + * 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.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.expressions.{And, Expression, Or} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +// For implicit conversions +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ + +class NormalizeFiltersSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Seq( + Batch("AnalysisNodes", Once, + EliminateAnalysisOperators), + Batch("NormalizeFilters", FixedPoint(100), + BooleanSimplification, + SimplifyFilters)) + } + + val relation = LocalRelation('a.int, 'b.int, 'c.string) + + def checkExpression(original: Expression, expected: Expression): Unit = { + val actual = Optimize(relation.where(original)).collect { case f: Filter => f.condition }.head + val result = (actual, expected) match { + case (And(l1, r1), And(l2, r2)) => (l1 == l2 && r1 == r2) || (l1 == r2 && l2 == r1) + case (Or (l1, r1), Or (l2, r2)) => (l1 == l2 && r1 == r2) || (l1 == r2 && l2 == r1) + case (lhs, rhs) => lhs fastEquals rhs + } + + assert(result, s"$actual isn't equivalent to $expected") + } + + test("a && a => a") { + checkExpression('a === 1 && 'a === 1, 'a === 1) + checkExpression('a === 1 && 'a === 1 && 'a === 1, 'a === 1) + } + + test("a || a => a") { + checkExpression('a === 1 || 'a === 1, 'a === 1) + checkExpression('a === 1 || 'a === 1 || 'a === 1, 'a === 1) + } + + test("(a && b) || (a && c) => a && (b || c)") { + checkExpression( + ('a === 1 && 'a < 10) || ('a > 2 && 'a === 1), + ('a === 1) && ('a < 10 || 'a > 2)) + + checkExpression( + ('a < 1 && 'b > 2 && 'c.isNull) || ('a < 1 && 'c === "hello" && 'b > 2), + ('c.isNull || 'c === "hello") && 'a < 1 && 'b > 2) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 82afa31a99a7e..1915c25392f1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -105,7 +105,9 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be test(query) { val schemaRdd = sql(query) - assertResult(expectedQueryResult.toArray, "Wrong query result") { + val queryExecution = schemaRdd.queryExecution + + assertResult(expectedQueryResult.toArray, s"Wrong query result: $queryExecution") { schemaRdd.collect().map(_.head).toArray } @@ -113,8 +115,10 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) }.head - assert(readBatches === expectedReadBatches, "Wrong number of read batches") - assert(readPartitions === expectedReadPartitions, "Wrong number of read partitions") + assert(readBatches === expectedReadBatches, s"Wrong number of read batches: $queryExecution") + assert( + readPartitions === expectedReadPartitions, + s"Wrong number of read partitions: $queryExecution") } } } From 7425bec320227bf8818dc2844c12d5373d166364 Mon Sep 17 00:00:00 2001 From: Michael Davies Date: Tue, 30 Dec 2014 13:40:51 -0800 Subject: [PATCH 159/227] [SPARK-4386] Improve performance when writing Parquet files Convert type of RowWriteSupport.attributes to Array. Analysis of performance for writing very wide tables shows that time is spent predominantly in apply method on attributes var. Type of attributes previously was LinearSeqOptimized and apply is O(N) which made write O(N squared). Measurements on 575 column table showed this change made a 6x improvement in write times. Author: Michael Davies Closes #3843 from MickDavies/SPARK-4386 and squashes the following commits: 892519d [Michael Davies] [SPARK-4386] Improve performance when writing Parquet files --- .../org/apache/spark/sql/parquet/ParquetTableSupport.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index ef3687e692964..9049eb5932b79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -130,7 +130,7 @@ private[parquet] object RowReadSupport { private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] var writer: RecordConsumer = null - private[parquet] var attributes: Seq[Attribute] = null + private[parquet] var attributes: Array[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) @@ -138,7 +138,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) if (attributes == null) { - attributes = ParquetTypesConverter.convertFromString(origAttributesStr) + attributes = ParquetTypesConverter.convertFromString(origAttributesStr).toArray } log.debug(s"write support initialized for requested schema $attributes") From 8f29b7cafc2b6e802e4eb21f681d6369da2f30fa Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 30 Dec 2014 13:44:30 -0800 Subject: [PATCH 160/227] [SPARK-4935][SQL] When hive.cli.print.header configured, spark-sql aborted if passed in a invalid sql If we passed in a wrong sql like ```abdcdfsfs```, the spark-sql script aborted. Author: wangfei Author: Fei Wang Closes #3761 from scwf/patch-10 and squashes the following commits: 46dc344 [Fei Wang] revert console.printError(rc.getErrorMessage()) 0330e07 [wangfei] avoid to print error message repeatedly 1614a11 [wangfei] spark-sql abort when passed in a wrong sql --- .../spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala index 6ed8fd2768f95..7a3d76c61c3a1 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala @@ -60,7 +60,7 @@ private[hive] abstract class AbstractSparkSQLDriver( } catch { case cause: Throwable => logError(s"Failed in [$command]", cause) - new CommandProcessorResponse(0, ExceptionUtils.getFullStackTrace(cause), null) + new CommandProcessorResponse(1, ExceptionUtils.getFullStackTrace(cause), null) } } From 07fa1910d9c4092d670381c447403105f01c584e Mon Sep 17 00:00:00 2001 From: wangxiaojing Date: Tue, 30 Dec 2014 13:54:12 -0800 Subject: [PATCH 161/227] [SPARK-4570][SQL]add BroadcastLeftSemiJoinHash JIRA issue: [SPARK-4570](https://issues.apache.org/jira/browse/SPARK-4570) We are planning to create a `BroadcastLeftSemiJoinHash` to implement the broadcast join for `left semijoin` In left semijoin : If the size of data from right side is smaller than the user-settable threshold `AUTO_BROADCASTJOIN_THRESHOLD`, the planner would mark it as the `broadcast` relation and mark the other relation as the stream side. The broadcast table will be broadcasted to all of the executors involved in the join, as a `org.apache.spark.broadcast.Broadcast` object. It will use `joins.BroadcastLeftSemiJoinHash`.,else it will use `joins.LeftSemiJoinHash`. The benchmark suggests these made the optimized version 4x faster when `left semijoin`

Original:
left semi join : 9288 ms
Optimized:
left semi join : 1963 ms
The micro benchmark load `data1/kv3.txt` into a normal Hive table. Benchmark code:

 def benchmark(f: => Unit) = {
    val begin = System.currentTimeMillis()
    f
    val end = System.currentTimeMillis()
    end - begin
  }
  val sc = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))
  val hiveContext = new HiveContext(sc)
  import hiveContext._
  sql("drop table if exists left_table")
  sql("drop table if exists right_table")
  sql( """create table left_table (key int, value string)
       """.stripMargin)
  sql( s"""load data local inpath "/data1/kv3.txt" into table left_table""")
  sql( """create table right_table (key int, value string)
       """.stripMargin)
  sql(
    """
      |from left_table
      |insert overwrite table right_table
      |select left_table.key, left_table.value
    """.stripMargin)

  val leftSimeJoin = sql(
    """select a.key from left_table a
      |left semi join right_table b on a.key = b.key""".stripMargin)
  val leftSemiJoinDuration = benchmark(leftSimeJoin.count())
  println(s"left semi join : $leftSemiJoinDuration ms ")
Author: wangxiaojing Closes #3442 from wangxiaojing/SPARK-4570 and squashes the following commits: a4a43c9 [wangxiaojing] rebase f103983 [wangxiaojing] change style fbe4887 [wangxiaojing] change style ff2e618 [wangxiaojing] add testsuite 1a8da2a [wangxiaojing] add BroadcastLeftSemiJoinHash --- .../spark/sql/execution/SparkStrategies.scala | 6 ++ .../joins/BroadcastLeftSemiJoinHash.scala | 67 +++++++++++++++++++ .../org/apache/spark/sql/JoinSuite.scala | 38 +++++++++++ .../spark/sql/hive/StatisticsSuite.scala | 50 +++++++++++++- 4 files changed, 160 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 9151da69ed44c..ce878c137e627 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -33,6 +33,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object LeftSemiJoin extends Strategy with PredicateHelper { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) + if sqlContext.autoBroadcastJoinThreshold > 0 && + right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => + val semiJoin = joins.BroadcastLeftSemiJoinHash( + leftKeys, rightKeys, planLater(left), planLater(right)) + condition.map(Filter(_, semiJoin)).getOrElse(semiJoin) :: Nil // Find left semi joins where at least some predicates can be evaluated by matching join keys case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) => val semiJoin = joins.LeftSemiJoinHash( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala new file mode 100644 index 0000000000000..2ab064fd0151e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -0,0 +1,67 @@ +/* + * 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.sql.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.{Expression, Row} +import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} + +/** + * :: DeveloperApi :: + * Build the right table's join keys into a HashSet, and iteratively go through the left + * table, to find the if join keys are in the Hash set. + */ +@DeveloperApi +case class BroadcastLeftSemiJoinHash( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode with HashJoin { + + override val buildSide = BuildRight + + override def output = left.output + + override def execute() = { + val buildIter= buildPlan.execute().map(_.copy()).collect().toIterator + val hashSet = new java.util.HashSet[Row]() + var currentRow: Row = null + + // Create a Hash set of buildKeys + while (buildIter.hasNext) { + currentRow = buildIter.next() + val rowKey = buildSideKeyGenerator(currentRow) + if (!rowKey.anyNull) { + val keyExists = hashSet.contains(rowKey) + if (!keyExists) { + hashSet.add(rowKey) + } + } + } + + val broadcastedRelation = sparkContext.broadcast(hashSet) + + streamedPlan.execute().mapPartitions { streamIter => + val joinKeys = streamSideKeyGenerator() + streamIter.filter(current => { + !joinKeys(current).anyNull && broadcastedRelation.value.contains(joinKeys.currentValue) + }) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 0378fd7e367f0..1a4232dab86e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -48,6 +48,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: LeftSemiJoinBNL => j case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j + case j: BroadcastLeftSemiJoinHash => j } assert(operators.size === 1) @@ -382,4 +383,41 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { """.stripMargin), (null, 10) :: Nil) } + + test("broadcasted left semi join operator selection") { + clearCache() + sql("CACHE TABLE testData") + val tmp = autoBroadcastJoinThreshold + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=1000000000") + Seq( + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", + classOf[BroadcastLeftSemiJoinHash]) + ).foreach { + case (query, joinClass) => assertJoin(query, joinClass) + } + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1") + + Seq( + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]) + ).foreach { + case (query, joinClass) => assertJoin(query, joinClass) + } + + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, tmp.toString) + sql("UNCACHE TABLE testData") + } + + test("left semi join") { + val rdd = sql("SELECT * FROM testData2 LEFT SEMI JOIN testData ON key = a") + checkAnswer(rdd, + (1, 1) :: + (1, 2) :: + (2, 1) :: + (2, 2) :: + (3, 1) :: + (3, 2) :: Nil) + + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index ff4071d8e2f10..4b6a9308b9811 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.BeforeAndAfterAll import scala.reflect.ClassTag import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.execution._ @@ -193,4 +193,52 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { ) } + test("auto converts to broadcast left semi join, by size estimate of a relation") { + val leftSemiJoinQuery = + """SELECT * FROM src a + |left semi JOIN src b ON a.key=86 and a.key = b.key""".stripMargin + val answer = (86, "val_86") :: Nil + + var rdd = sql(leftSemiJoinQuery) + + // Assert src has a size smaller than the threshold. + val sizes = rdd.queryExecution.analyzed.collect { + case r if implicitly[ClassTag[MetastoreRelation]].runtimeClass + .isAssignableFrom(r.getClass) => + r.statistics.sizeInBytes + } + assert(sizes.size === 2 && sizes(1) <= autoBroadcastJoinThreshold + && sizes(0) <= autoBroadcastJoinThreshold, + s"query should contain two relations, each of which has size smaller than autoConvertSize") + + // Using `sparkPlan` because for relevant patterns in HashJoin to be + // matched, other strategies need to be applied. + var bhj = rdd.queryExecution.sparkPlan.collect { + case j: BroadcastLeftSemiJoinHash => j + } + assert(bhj.size === 1, + s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") + + checkAnswer(rdd, answer) // check correctness of output + + TestHive.settings.synchronized { + val tmp = autoBroadcastJoinThreshold + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1") + rdd = sql(leftSemiJoinQuery) + bhj = rdd.queryExecution.sparkPlan.collect { + case j: BroadcastLeftSemiJoinHash => j + } + assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") + + val shj = rdd.queryExecution.sparkPlan.collect { + case j: LeftSemiJoinHash => j + } + assert(shj.size === 1, + "LeftSemiJoinHash should be planned when BroadcastHashJoin is turned off") + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp") + } + + } } From b239ea1c31aeaa752d5dc8f45423df1f8c0924ca Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 30 Dec 2014 14:00:57 -0800 Subject: [PATCH 162/227] SPARK-3955 part 2 [CORE] [HOTFIX] Different versions between jackson-mapper-asl and jackson-core-asl pwendell https://github.com/apache/spark/commit/2483c1efb6429a7d8a20c96d18ce2fec93a1aff9 didn't actually add a reference to `jackson-core-asl` as intended, but a second redundant reference to `jackson-mapper-asl`, as markhamstra picked up on (https://github.com/apache/spark/pull/3716#issuecomment-68180192) This just rectifies the typo. I missed it as well; the original PR https://github.com/apache/spark/pull/2818 had it correct and I also didn't see the problem. Author: Sean Owen Closes #3829 from srowen/SPARK-3955 and squashes the following commits: 6cfdc4e [Sean Owen] Actually refer to jackson-core-asl --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a843af2b22d6c..05f59a9b4140b 100644 --- a/pom.xml +++ b/pom.xml @@ -827,7 +827,7 @@ org.codehaus.jackson - jackson-mapper-asl + jackson-core-asl ${jackson.version} From 0f31992c61f6662e5347745f6a1ac272a5fd63c9 Mon Sep 17 00:00:00 2001 From: Jakub Dubovsky Date: Tue, 30 Dec 2014 14:19:07 -0800 Subject: [PATCH 163/227] [Spark-4995] Replace Vector.toBreeze.activeIterator with foreachActive New foreachActive method of vector was introduced by SPARK-4431 as more efficient alternative to vector.toBreeze.activeIterator. There are some parts of codebase where it was not yet replaced. dbtsai Author: Jakub Dubovsky Closes #3846 from james64/SPARK-4995-foreachActive and squashes the following commits: 3eb7e37 [Jakub Dubovsky] Scalastyle fix 32fe6c6 [Jakub Dubovsky] activeIterator removed - IndexedRowMatrix.toBreeze 47a4777 [Jakub Dubovsky] activeIterator removed in RowMatrix.toBreeze 90a7d98 [Jakub Dubovsky] activeIterator removed in MLUtils.saveAsLibSVMFile --- .../spark/mllib/linalg/distributed/IndexedRowMatrix.scala | 2 +- .../apache/spark/mllib/linalg/distributed/RowMatrix.scala | 4 ++-- .../main/scala/org/apache/spark/mllib/util/MLUtils.scala | 8 +++++--- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index 5c1acca0ec532..36d8cadd2bdd7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -142,7 +142,7 @@ class IndexedRowMatrix( val mat = BDM.zeros[Double](m, n) rows.collect().foreach { case IndexedRow(rowIndex, vector) => val i = rowIndex.toInt - vector.toBreeze.activeIterator.foreach { case (j, v) => + vector.foreachActive { case (j, v) => mat(i, j) = v } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 10a515af88802..a3fca53929ab7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -588,8 +588,8 @@ class RowMatrix( val n = numCols().toInt val mat = BDM.zeros[Double](m, n) var i = 0 - rows.collect().foreach { v => - v.toBreeze.activeIterator.foreach { case (j, v) => + rows.collect().foreach { vector => + vector.foreachActive { case (j, v) => mat(i, j) = v } i += 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 1d07b5dab8268..da0da0a168c1d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -154,10 +154,12 @@ object MLUtils { def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { // TODO: allow to specify label precision and feature precision. val dataStr = data.map { case LabeledPoint(label, features) => - val featureStrings = features.toBreeze.activeIterator.map { case (i, v) => - s"${i + 1}:$v" + val sb = new StringBuilder(label.toString) + features.foreachActive { case (i, v) => + sb += ' ' + sb ++= s"${i + 1}:$v" } - (Iterator(label) ++ featureStrings).mkString(" ") + sb.mkString } dataStr.saveAsTextFile(dir) } From 6a897829444e2ef273586511f93a40d36e64fb0b Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 30 Dec 2014 14:39:13 -0800 Subject: [PATCH 164/227] [SPARK-4813][Streaming] Fix the issue that ContextWaiter didn't handle 'spurious wakeup' Used `Condition` to rewrite `ContextWaiter` because it provides a convenient API `awaitNanos` for timeout. Author: zsxwing Closes #3661 from zsxwing/SPARK-4813 and squashes the following commits: 52247f5 [zsxwing] Add explicit unit type be42bcf [zsxwing] Update as per review suggestion e06bd4f [zsxwing] Fix the issue that ContextWaiter didn't handle 'spurious wakeup' --- .../spark/streaming/ContextWaiter.scala | 63 ++++++++++++++----- 1 file changed, 48 insertions(+), 15 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala index a0aeacbc733bd..fdbbe2aa6ef08 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala @@ -17,30 +17,63 @@ package org.apache.spark.streaming +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock + private[streaming] class ContextWaiter { + + private val lock = new ReentrantLock() + private val condition = lock.newCondition() + + // Guarded by "lock" private var error: Throwable = null - private var stopped: Boolean = false - def notifyError(e: Throwable) = synchronized { - error = e - notifyAll() - } + // Guarded by "lock" + private var stopped: Boolean = false - def notifyStop() = synchronized { - stopped = true - notifyAll() + def notifyError(e: Throwable): Unit = { + lock.lock() + try { + error = e + condition.signalAll() + } finally { + lock.unlock() + } } - def waitForStopOrError(timeout: Long = -1) = synchronized { - // If already had error, then throw it - if (error != null) { - throw error + def notifyStop(): Unit = { + lock.lock() + try { + stopped = true + condition.signalAll() + } finally { + lock.unlock() } + } - // If not already stopped, then wait - if (!stopped) { - if (timeout < 0) wait() else wait(timeout) + /** + * Return `true` if it's stopped; or throw the reported error if `notifyError` has been called; or + * `false` if the waiting time detectably elapsed before return from the method. + */ + def waitForStopOrError(timeout: Long = -1): Boolean = { + lock.lock() + try { + if (timeout < 0) { + while (!stopped && error == null) { + condition.await() + } + } else { + var nanos = TimeUnit.MILLISECONDS.toNanos(timeout) + while (!stopped && error == null && nanos > 0) { + nanos = condition.awaitNanos(nanos) + } + } + // If already had error, then throw it if (error != null) throw error + // already stopped or timeout + stopped + } finally { + lock.unlock() } } } From 035bac88c732247c79a1bbad4f9191090cbbdc9a Mon Sep 17 00:00:00 2001 From: Liu Jiongzhou Date: Tue, 30 Dec 2014 15:55:56 -0800 Subject: [PATCH 165/227] [SPARK-4998][MLlib]delete the "train" function To make the functions with the same in "object" effective, specially when using java reflection. As the "train" function defined in "class DecisionTree" will hide the functions with the same name in "object DecisionTree". JIRA[SPARK-4998] Author: Liu Jiongzhou Closes #3836 from ljzzju/master and squashes the following commits: 4e13133 [Liu Jiongzhou] [MLlib]delete the "train" function --- .../scala/org/apache/spark/mllib/tree/DecisionTree.scala | 7 ------- 1 file changed, 7 deletions(-) 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 73e7e32c6db31..b3e8ed9af8c51 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 @@ -64,13 +64,6 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val rfModel = rf.run(input) rfModel.trees(0) } - - /** - * Trains a decision tree model over an RDD. This is deprecated because it hides the static - * methods with the same name in Java. - */ - @deprecated("Please use DecisionTree.run instead.", "1.2.0") - def train(input: RDD[LabeledPoint]): DecisionTreeModel = run(input) } object DecisionTree extends Serializable with Logging { From 352ed6bbe3c3b67e52e298e7c535ae414d96beca Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 30 Dec 2014 18:12:20 -0800 Subject: [PATCH 166/227] [SPARK-1010] Clean up uses of System.setProperty in unit tests Several of our tests call System.setProperty (or test code which implicitly sets system properties) and don't always reset/clear the modified properties, which can create ordering dependencies between tests and cause hard-to-diagnose failures. This patch removes most uses of System.setProperty from our tests, since in most cases we can use SparkConf to set these configurations (there are a few exceptions, including the tests of SparkConf itself). For the cases where we continue to use System.setProperty, this patch introduces a `ResetSystemProperties` ScalaTest mixin class which snapshots the system properties before individual tests and to automatically restores them on test completion / failure. See the block comment at the top of the ResetSystemProperties class for more details. Author: Josh Rosen Closes #3739 from JoshRosen/cleanup-system-properties-in-tests and squashes the following commits: 0236d66 [Josh Rosen] Replace setProperty uses in two example programs / tools 3888fe3 [Josh Rosen] Remove setProperty use in LocalJavaStreamingContext 4f4031d [Josh Rosen] Add note on why SparkSubmitSuite needs ResetSystemProperties 4742a5b [Josh Rosen] Clarify ResetSystemProperties trait inheritance ordering. 0eaf0b6 [Josh Rosen] Remove setProperty call in TaskResultGetterSuite. 7a3d224 [Josh Rosen] Fix trait ordering 3fdb554 [Josh Rosen] Remove setProperty call in TaskSchedulerImplSuite bee20df [Josh Rosen] Remove setProperty calls in SparkContextSchedulerCreationSuite 655587c [Josh Rosen] Remove setProperty calls in JobCancellationSuite 3f2f955 [Josh Rosen] Remove System.setProperty calls in DistributedSuite cfe9cce [Josh Rosen] Remove use of system properties in SparkContextSuite 8783ab0 [Josh Rosen] Remove TestUtils.setSystemProperty, since it is subsumed by the ResetSystemProperties trait. 633a84a [Josh Rosen] Remove use of system properties in FileServerSuite 25bfce2 [Josh Rosen] Use ResetSystemProperties in UtilsSuite 1d1aa5a [Josh Rosen] Use ResetSystemProperties in SizeEstimatorSuite dd9492b [Josh Rosen] Use ResetSystemProperties in AkkaUtilsSuite b0daff2 [Josh Rosen] Use ResetSystemProperties in BlockManagerSuite e9ded62 [Josh Rosen] Use ResetSystemProperties in TaskSchedulerImplSuite 5b3cb54 [Josh Rosen] Use ResetSystemProperties in SparkListenerSuite 0995c4b [Josh Rosen] Use ResetSystemProperties in SparkContextSchedulerCreationSuite c83ded8 [Josh Rosen] Use ResetSystemProperties in SparkConfSuite 51aa870 [Josh Rosen] Use withSystemProperty in ShuffleSuite 60a63a1 [Josh Rosen] Use ResetSystemProperties in JobCancellationSuite 14a92e4 [Josh Rosen] Use withSystemProperty in FileServerSuite 628f46c [Josh Rosen] Use ResetSystemProperties in DistributedSuite 9e3e0dd [Josh Rosen] Add ResetSystemProperties test fixture mixin; use it in SparkSubmitSuite. 4dcea38 [Josh Rosen] Move withSystemProperty to TestUtils class. --- .../org/apache/spark/DistributedSuite.scala | 21 ++----- .../org/apache/spark/FileServerSuite.scala | 16 ++--- .../apache/spark/JobCancellationSuite.scala | 21 +++---- .../scala/org/apache/spark/ShuffleSuite.scala | 22 +++---- .../org/apache/spark/SparkConfSuite.scala | 51 ++++++--------- .../SparkContextSchedulerCreationSuite.scala | 31 ++++------ .../org/apache/spark/SparkContextSuite.scala | 62 +++++++------------ .../spark/deploy/SparkSubmitSuite.scala | 6 +- .../spark/scheduler/SparkListenerSuite.scala | 9 +-- .../scheduler/TaskResultGetterSuite.scala | 23 +++---- .../scheduler/TaskSchedulerImplSuite.scala | 6 +- .../spark/storage/BlockManagerSuite.scala | 23 +++---- .../apache/spark/util/AkkaUtilsSuite.scala | 2 +- .../spark/util/ResetSystemProperties.scala | 57 +++++++++++++++++ .../spark/util/SizeEstimatorSuite.scala | 38 +++--------- .../org/apache/spark/util/UtilsSuite.scala | 2 +- .../apache/spark/examples/BroadcastTest.scala | 6 +- .../streaming/LocalJavaStreamingContext.java | 8 ++- .../streaming/LocalJavaStreamingContext.java | 8 ++- .../streaming/LocalJavaStreamingContext.java | 8 ++- .../streaming/LocalJavaStreamingContext.java | 8 ++- .../streaming/LocalJavaStreamingContext.java | 8 ++- .../spark/tools/StoragePerfTester.scala | 12 ++-- 23 files changed, 216 insertions(+), 232 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 998f3008ec0ea..97ea3578aa8ba 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -import org.scalatest.BeforeAndAfter import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers @@ -29,16 +28,10 @@ class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} -class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter - with LocalSparkContext { +class DistributedSuite extends FunSuite with Matchers with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" - after { - System.clearProperty("spark.reducer.maxMbInFlight") - System.clearProperty("spark.storage.memoryFraction") - } - test("task throws not serializable exception") { // Ensures that executors do not crash when an exn is not serializable. If executors crash, // this test will hang. Correct behavior is that executors don't crash but fail tasks @@ -84,15 +77,14 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("groupByKey where map output sizes exceed maxMbInFlight") { - System.setProperty("spark.reducer.maxMbInFlight", "1") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1") + sc = new SparkContext(clusterUrl, "test", conf) // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output // file should be about 2.5 MB val pairs = sc.parallelize(1 to 2000, 4).map(x => (x % 16, new Array[Byte](10000))) val groups = pairs.groupByKey(2).map(x => (x._1, x._2.size)).collect() assert(groups.length === 16) assert(groups.map(_._2).sum === 2000) - // Note that spark.reducer.maxMbInFlight will be cleared in the test suite's after{} block } test("accumulators") { @@ -210,7 +202,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("compute without caching when no partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.0001") sc = new SparkContext(clusterUrl, "test") // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 50 KB (0.0001 of 512 MB), so no partitions should fit in memory @@ -218,12 +209,11 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("compute when only some partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.01") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.storage.memoryFraction", "0.01") + sc = new SparkContext(clusterUrl, "test", conf) // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions // to make sure that *some* of them do fit though @@ -231,7 +221,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("passing environment variables to cluster") { diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 49426545c767e..0f49ce4754fbb 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -31,10 +31,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ + def newConf: SparkConf = new SparkConf(loadDefaults = false).set("spark.authenticate", "false") + override def beforeEach() { super.beforeEach() resetSparkContext() - System.setProperty("spark.authenticate", "false") } override def beforeAll() { @@ -52,7 +53,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) - System.setProperty("spark.authenticate", "false") val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) @@ -74,7 +74,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -108,7 +108,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test("Distributing files locally using URL as input") { // addFile("file:///....") - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(new File(tmpFile.toString).toURI.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -122,7 +122,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1, 1)) sc.parallelize(testData).foreach { x => @@ -133,7 +133,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -147,7 +147,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => @@ -158,7 +158,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster using local: URL") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl.replace("file", "local")) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 41ed2bce55ce1..7584ae79fc920 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -40,12 +40,11 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter override def afterEach() { super.afterEach() resetSparkContext() - System.clearProperty("spark.scheduler.mode") } test("local mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local[2]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -53,10 +52,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("local mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local[2]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -64,8 +63,8 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local-cluster[2,1,512]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -73,10 +72,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local-cluster[2,1,512]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 58a96245a9b53..f57921b768310 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -35,19 +35,15 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex conf.set("spark.test.noStageRetry", "true") test("groupByKey without compression") { - try { - System.setProperty("spark.shuffle.compress", "false") - sc = new SparkContext("local", "test", conf) - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) - val groups = pairs.groupByKey(4).collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } finally { - System.setProperty("spark.shuffle.compress", "true") - } + val myConf = conf.clone().set("spark.shuffle.compress", "false") + sc = new SparkContext("local", "test", myConf) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) + val groups = pairs.groupByKey(4).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) } test("shuffle non-zero block size") { diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 5d018ea9868a7..790976a5ac308 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -19,27 +19,20 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} +import org.apache.spark.util.ResetSystemProperties import com.esotericsoftware.kryo.Kryo -class SparkConfSuite extends FunSuite with LocalSparkContext { +class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("loading from system properties") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf() - assert(conf.get("spark.test.testProperty") === "2") - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf() + assert(conf.get("spark.test.testProperty") === "2") } test("initializing without loading defaults") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf(false) - assert(!conf.contains("spark.test.testProperty")) - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf(false) + assert(!conf.contains("spark.test.testProperty")) } test("named set methods") { @@ -117,23 +110,17 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { test("nested property names") { // This wasn't supported by some external conf parsing libraries - try { - System.setProperty("spark.test.a", "a") - System.setProperty("spark.test.a.b", "a.b") - System.setProperty("spark.test.a.b.c", "a.b.c") - val conf = new SparkConf() - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "a.b") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - conf.set("spark.test.a.b", "A.B") - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "A.B") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - } finally { - System.clearProperty("spark.test.a") - System.clearProperty("spark.test.a.b") - System.clearProperty("spark.test.a.b.c") - } + System.setProperty("spark.test.a", "a") + System.setProperty("spark.test.a.b", "a.b") + System.setProperty("spark.test.a.b.c", "a.b.c") + val conf = new SparkConf() + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "a.b") + assert(conf.get("spark.test.a.b.c") === "a.b.c") + conf.set("spark.test.a.b", "A.B") + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "A.B") + assert(conf.get("spark.test.a.b.c") === "a.b.c") } test("register kryo classes through registerKryoClasses") { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 0390a2e4f1dbb..8ae4f243ec1ae 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -27,10 +27,13 @@ import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite extends FunSuite with LocalSparkContext with PrivateMethodTester with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = { + def createTaskScheduler(master: String): TaskSchedulerImpl = + createTaskScheduler(master, new SparkConf()) + + def createTaskScheduler(master: String, conf: SparkConf): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) @@ -102,19 +105,13 @@ class SparkContextSchedulerCreationSuite } test("local-default-parallelism") { - val defaultParallelism = System.getProperty("spark.default.parallelism") - System.setProperty("spark.default.parallelism", "16") - val sched = createTaskScheduler("local") + val conf = new SparkConf().set("spark.default.parallelism", "16") + val sched = createTaskScheduler("local", conf) sched.backend match { case s: LocalBackend => assert(s.defaultParallelism() === 16) case _ => fail() } - - Option(defaultParallelism) match { - case Some(v) => System.setProperty("spark.default.parallelism", v) - case _ => System.clearProperty("spark.default.parallelism") - } } test("simr") { @@ -155,9 +152,10 @@ class SparkContextSchedulerCreationSuite testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") } - def testMesos(master: String, expectedClass: Class[_]) { + def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) { + val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString) try { - val sched = createTaskScheduler(master) + val sched = createTaskScheduler(master, conf) assert(sched.backend.getClass === expectedClass) } catch { case e: UnsatisfiedLinkError => @@ -168,17 +166,14 @@ class SparkContextSchedulerCreationSuite } test("mesos fine-grained") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) } test("mesos coarse-grained") { - System.setProperty("spark.mesos.coarse", "true") - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) } test("mesos with zookeeper") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend]) + testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 1362022104195..8b3c6871a7b39 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -23,55 +23,37 @@ import org.apache.hadoop.io.BytesWritable class SparkContextSuite extends FunSuite with LocalSparkContext { - /** Allows system properties to be changed in tests */ - private def withSystemProperty[T](property: String, value: String)(block: => T): T = { - val originalValue = System.getProperty(property) - try { - System.setProperty(property, value) - block - } finally { - if (originalValue == null) { - System.clearProperty(property) - } else { - System.setProperty(property, originalValue) - } - } - } - test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - // A SparkContext is already running, so we shouldn't be able to create a second one - intercept[SparkException] { new SparkContext(conf) } - // After stopping the running context, we should be able to create a new one - resetSparkContext() - sc = new SparkContext(conf) - } + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "false") + sc = new SparkContext(conf) + // A SparkContext is already running, so we shouldn't be able to create a second one + intercept[SparkException] { new SparkContext(conf) } + // After stopping the running context, we should be able to create a new one + resetSparkContext() + sc = new SparkContext(conf) } test("Can still construct a new SparkContext after failing to construct a previous one") { - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - // This is an invalid configuration (no app name or master URL) - intercept[SparkException] { - new SparkContext(new SparkConf()) - } - // Even though those earlier calls failed, we should still be able to create a new context - sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) + val conf = new SparkConf().set("spark.driver.allowMultipleContexts", "false") + // This is an invalid configuration (no app name or master URL) + intercept[SparkException] { + new SparkContext(conf) } + // Even though those earlier calls failed, we should still be able to create a new context + sc = new SparkContext(conf.setMaster("local").setAppName("test")) } test("Check for multiple SparkContexts can be disabled via undocumented debug option") { - withSystemProperty("spark.driver.allowMultipleContexts", "true") { - var secondSparkContext: SparkContext = null - try { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - secondSparkContext = new SparkContext(conf) - } finally { - Option(secondSparkContext).foreach(_.stop()) - } + var secondSparkContext: SparkContext = null + try { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true") + sc = new SparkContext(conf) + secondSparkContext = new SparkContext(conf) + } finally { + Option(secondSparkContext).foreach(_.stop()) } } 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 eb7bd7ab3986e..5eda2d41f0e6d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -23,11 +23,13 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.deploy.SparkSubmit._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ResetSystemProperties, Utils} import org.scalatest.FunSuite import org.scalatest.Matchers -class SparkSubmitSuite extends FunSuite with Matchers { +// Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch +// of properties that neeed to be cleared after tests. +class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties { def beforeAll() { System.setProperty("spark.testing", "true") } 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 b276343cb412c..24f41bf8cccda 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -26,9 +26,10 @@ import org.scalatest.Matchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.ResetSystemProperties -class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers - with BeforeAndAfter with BeforeAndAfterAll { +class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers with BeforeAndAfter + with BeforeAndAfterAll with ResetSystemProperties { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 @@ -37,10 +38,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll() { - System.clearProperty("spark.akka.frameSize") - } - test("basic creation and shutdown of LiveListenerBus") { val counter = new BasicJobCounter val bus = new LiveListenerBus diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 5768a3a733f00..3aab5a156ee77 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv} import org.apache.spark.storage.TaskResultBlockId /** @@ -55,27 +55,20 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule /** * Tests related to handling task results (both direct and indirect). */ -class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll - with LocalSparkContext { +class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - override def beforeAll { - // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small - // as we can make it) so the tests don't take too long. - System.setProperty("spark.akka.frameSize", "1") - } - - override def afterAll { - System.clearProperty("spark.akka.frameSize") - } + // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small + // as we can make it) so the tests don't take too long. + def conf: SparkConf = new SparkConf().set("spark.akka.frameSize", "1") test("handling results smaller than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) } test("handling results larger than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) @@ -89,7 +82,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("task retried if result missing from block manager") { // Set the maximum number of task failures to > 0, so that the task set isn't aborted // after the result is missing. - sc = new SparkContext("local[1,2]", "test") + sc = new SparkContext("local[1,2]", "test", conf) // If this test hangs, it's probably because no resource offers were made after the task // failed. val scheduler: TaskSchedulerImpl = sc.taskScheduler match { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 7532da88c6065..40aaf9dd1f1e9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -162,12 +162,12 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } test("Fair Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + val conf = new SparkConf().set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) val taskScheduler = new TaskSchedulerImpl(sc) val taskSet = FakeTask.createTaskSet(1) - val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) schedulableBuilder.buildPools() 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 5554efbcbadf8..ffe6f039145ea 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,7 +33,7 @@ import akka.util.Timeout import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ @@ -44,18 +44,17 @@ 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 org.apache.spark.util._ -class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter - with PrivateMethodTester { +class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach + with PrivateMethodTester with ResetSystemProperties { private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null var master: BlockManagerMaster = null - var oldArch: String = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -79,13 +78,13 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter manager } - before { + override def beforeEach(): Unit = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem( "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") + System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.driver.port", boundPort.toString) @@ -100,7 +99,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter SizeEstimator invokePrivate initialize() } - after { + override def afterEach(): Unit = { if (store != null) { store.stop() store = null @@ -113,14 +112,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter actorSystem.awaitTermination() actorSystem = null master = null - - if (oldArch != null) { - conf.set("os.arch", oldArch) - } else { - System.clearProperty("os.arch") - } - - System.clearProperty("spark.test.useCompressedOops") } test("StorageLevel object caching") { diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 7bca1711ae226..6bbf72e929dcb 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.BlockManagerId /** * Test the AkkaUtils with various security settings. */ -class AkkaUtilsSuite extends FunSuite with LocalSparkContext { +class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("remote fetch security bad password") { val conf = new SparkConf diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala new file mode 100644 index 0000000000000..d4b92f33dd9e6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala @@ -0,0 +1,57 @@ +/* + * 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.util + +import java.util.Properties + +import org.scalatest.{BeforeAndAfterEach, Suite} + +/** + * Mixin for automatically resetting system properties that are modified in ScalaTest tests. + * This resets the properties after each individual test. + * + * The order in which fixtures are mixed in affects the order in which they are invoked by tests. + * If we have a suite `MySuite extends FunSuite with Foo with Bar`, then + * Bar's `super` is Foo, so Bar's beforeEach() will and afterEach() methods will be invoked first + * by the rest runner. + * + * This means that ResetSystemProperties should appear as the last trait in test suites that it's + * mixed into in order to ensure that the system properties snapshot occurs as early as possible. + * ResetSystemProperties calls super.afterEach() before performing its own cleanup, ensuring that + * the old properties are restored as late as possible. + * + * See the "Composing fixtures by stacking traits" section at + * http://www.scalatest.org/user_guide/sharing_fixtures for more details about this pattern. + */ +private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Suite => + var oldProperties: Properties = null + + override def beforeEach(): Unit = { + oldProperties = new Properties(System.getProperties) + super.beforeEach() + } + + override def afterEach(): Unit = { + try { + super.afterEach() + } finally { + System.setProperties(oldProperties) + oldProperties = null + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 0ea2d13a83505..7424c2e91d4f2 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.util -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite, PrivateMethodTester} class DummyClass1 {} @@ -46,20 +44,12 @@ class DummyString(val arr: Array[Char]) { } class SizeEstimatorSuite - extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { + extends FunSuite with BeforeAndAfterEach with PrivateMethodTester with ResetSystemProperties { - var oldArch: String = _ - var oldOops: String = _ - - override def beforeAll() { + override def beforeEach() { // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - oldOops = System.setProperty("spark.test.useCompressedOops", "true") - } - - override def afterAll() { - resetOrClear("os.arch", oldArch) - resetOrClear("spark.test.useCompressedOops", oldOops) + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "true") } test("simple classes") { @@ -122,7 +112,7 @@ class SizeEstimatorSuite } test("32-bit arch") { - val arch = System.setProperty("os.arch", "x86") + System.setProperty("os.arch", "x86") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -131,14 +121,13 @@ class SizeEstimatorSuite assertResult(48)(SizeEstimator.estimate(DummyString("a"))) assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) - resetOrClear("os.arch", arch) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("64-bit arch with no compressed oops") { - val arch = System.setProperty("os.arch", "amd64") - val oops = System.setProperty("spark.test.useCompressedOops", "false") + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -146,16 +135,5 @@ class SizeEstimatorSuite assertResult(64)(SizeEstimator.estimate(DummyString("a"))) assertResult(64)(SizeEstimator.estimate(DummyString("ab"))) assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) - - resetOrClear("os.arch", arch) - resetOrClear("spark.test.useCompressedOops", oops) - } - - def resetOrClear(prop: String, oldValue: String) { - if (oldValue != null) { - System.setProperty(prop, oldValue) - } else { - System.clearProperty(prop) - } } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index f9d4bea823f7c..4544382094f96 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf -class UtilsSuite extends FunSuite { +class UtilsSuite extends FunSuite with ResetSystemProperties { test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index adecd934358c4..1b53f3edbe92e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -28,11 +28,9 @@ object BroadcastTest { val bcName = if (args.length > 2) args(2) else "Http" val blockSize = if (args.length > 3) args(3) else "4096" - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast." + bcName + - "BroadcastFactory") - System.setProperty("spark.broadcast.blockSize", blockSize) val sparkConf = new SparkConf().setAppName("Broadcast Test") - + .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroaddcastFactory") + .set("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(sparkConf) val slices = if (args.length > 0) args(0).toInt else 2 diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071b..1e24da7f5f60c 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index db58eb642b56d..15ee95070a3d3 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -21,7 +21,7 @@ import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.util.Utils @@ -49,13 +49,13 @@ object StoragePerfTester { val writeData = "1" * recordLength val executor = Executors.newFixedThreadPool(numMaps) - System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.shuffle.sync", "true") - System.setProperty("spark.shuffle.manager", - "org.apache.spark.shuffle.hash.HashShuffleManager") + val conf = new SparkConf() + .set("spark.shuffle.compress", "false") + .set("spark.shuffle.sync", "true") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") // This is only used to instantiate a BlockManager. All thread scheduling is done manually. - val sc = new SparkContext("local[4]", "Write Tester") + val sc = new SparkContext("local[4]", "Write Tester", conf) val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] def writeOutputBytes(mapId: Int, total: AtomicLong) = { From 06a9aa589c518a40a3c7cc201e89d75af77ab93e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 31 Dec 2014 11:50:53 -0800 Subject: [PATCH 167/227] [SPARK-4797] Replace breezeSquaredDistance This PR replaces slow breezeSquaredDistance. Author: Liang-Chi Hsieh Closes #3643 from viirya/faster_squareddistance and squashes the following commits: f28b275 [Liang-Chi Hsieh] Move the implementation to linalg.Vectors and rename as sqdist. 0bc48ee [Liang-Chi Hsieh] Merge branch 'master' into faster_squareddistance ba34422 [Liang-Chi Hsieh] Fix bug. 91849d0 [Liang-Chi Hsieh] Modified for comment. 44a65ad [Liang-Chi Hsieh] Modified for comments. 35db395 [Liang-Chi Hsieh] Fix bug and some modifications for comments. f4f5ebb [Liang-Chi Hsieh] Follow BLAS.dot pattern to replace intersect, diff with while-loop. a36e09f [Liang-Chi Hsieh] Use while-loop to replace foreach for better performance. d3e0628 [Liang-Chi Hsieh] Make the methods private. dd415bc [Liang-Chi Hsieh] Consider different cases of SparseVector and DenseVector. 13669db [Liang-Chi Hsieh] Replace breezeSquaredDistance. --- .../apache/spark/mllib/linalg/Vectors.scala | 80 +++++++++++++++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 13 ++- .../spark/mllib/util/MLUtilsSuite.scala | 15 ++++ 3 files changed, 100 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 01f3f90577142..6a782b079aac3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -312,6 +312,86 @@ object Vectors { math.pow(sum, 1.0 / p) } } + + /** + * Returns the squared distance between two Vectors. + * @param v1 first Vector. + * @param v2 second Vector. + * @return squared distance between two Vectors. + */ + def sqdist(v1: Vector, v2: Vector): Double = { + var squaredDistance = 0.0 + (v1, v2) match { + case (v1: SparseVector, v2: SparseVector) => + val v1Values = v1.values + val v1Indices = v1.indices + val v2Values = v2.values + val v2Indices = v2.indices + val nnzv1 = v1Indices.size + val nnzv2 = v2Indices.size + + var kv1 = 0 + var kv2 = 0 + while (kv1 < nnzv1 || kv2 < nnzv2) { + var score = 0.0 + + if (kv2 >= nnzv2 || (kv1 < nnzv1 && v1Indices(kv1) < v2Indices(kv2))) { + score = v1Values(kv1) + kv1 += 1 + } else if (kv1 >= nnzv1 || (kv2 < nnzv2 && v2Indices(kv2) < v1Indices(kv1))) { + score = v2Values(kv2) + kv2 += 1 + } else { + score = v1Values(kv1) - v2Values(kv2) + kv1 += 1 + kv2 += 1 + } + squaredDistance += score * score + } + + case (v1: SparseVector, v2: DenseVector) if v1.indices.length / v1.size < 0.5 => + squaredDistance = sqdist(v1, v2) + + case (v1: DenseVector, v2: SparseVector) if v2.indices.length / v2.size < 0.5 => + squaredDistance = sqdist(v2, v1) + + // When a SparseVector is approximately dense, we treat it as a DenseVector + case (v1, v2) => + squaredDistance = v1.toArray.zip(v2.toArray).foldLeft(0.0){ (distance, elems) => + val score = elems._1 - elems._2 + distance + score * score + } + } + squaredDistance + } + + /** + * Returns the squared distance between DenseVector and SparseVector. + */ + private[mllib] def sqdist(v1: SparseVector, v2: DenseVector): Double = { + var kv1 = 0 + var kv2 = 0 + val indices = v1.indices + var squaredDistance = 0.0 + var iv1 = indices(kv1) + val nnzv2 = v2.size + + while (kv2 < nnzv2) { + var score = 0.0 + if (kv2 != iv1) { + score = v2(kv2) + } else { + score = v1.values(kv1) - v2(kv2) + if (kv1 < indices.length - 1) { + kv1 += 1 + iv1 = indices(kv1) + } + } + squaredDistance += score * score + kv2 += 1 + } + squaredDistance + } } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index da0da0a168c1d..c7843464a7505 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -19,8 +19,7 @@ package org.apache.spark.mllib.util import scala.reflect.ClassTag -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, - squaredDistance => breezeSquaredDistance} +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext @@ -28,7 +27,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD import org.apache.spark.util.random.BernoulliCellSampler import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext @@ -266,7 +265,7 @@ object MLUtils { } Vectors.fromBreeze(vector1) } - + /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: @@ -316,12 +315,10 @@ object MLUtils { val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dotValue)) / (sqDist + EPSILON) if (precisionBound2 > precision) { - // TODO: breezeSquaredDistance is slow, - // so we should replace it with our own implementation. - sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) + sqDist = Vectors.sqdist(v1, v2) } } else { - sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) + sqDist = Vectors.sqdist(v1, v2) } sqDist } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index df07987093fbf..7778847f8b72a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -52,12 +52,27 @@ class MLUtilsSuite extends FunSuite with MLlibTestSparkContext { val values = indices.map(i => a(i)) val v2 = Vectors.sparse(n, indices, values) val norm2 = Vectors.norm(v2, 2.0) + val v3 = Vectors.sparse(n, indices, indices.map(i => a(i) + 0.5)) + val norm3 = Vectors.norm(v3, 2.0) val squaredDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision) assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m") val fastSquaredDist2 = fastSquaredDistance(v1, norm1, Vectors.dense(v2.toArray), norm2, precision) assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") + val squaredDist2 = breezeSquaredDistance(v2.toBreeze, v3.toBreeze) + val fastSquaredDist3 = + fastSquaredDistance(v2, norm2, v3, norm3, precision) + assert((fastSquaredDist3 - squaredDist2) <= precision * squaredDist2, s"failed with m = $m") + if (m > 10) { + val v4 = Vectors.sparse(n, indices.slice(0, m - 10), + indices.map(i => a(i) + 0.5).slice(0, m - 10)) + val norm4 = Vectors.norm(v4, 2.0) + val squaredDist = breezeSquaredDistance(v2.toBreeze, v4.toBreeze) + val fastSquaredDist = + fastSquaredDistance(v2, norm2, v4, norm4, precision) + assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m") + } } } From 8e14c5eb551ab06c94859c7f6d8c6b62b4d00d59 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 31 Dec 2014 11:54:10 -0800 Subject: [PATCH 168/227] [SPARK-4298][Core] - The spark-submit cannot read Main-Class from Manifest. Resolves a bug where the `Main-Class` from a .jar file wasn't being read in properly. This was caused by the fact that the `primaryResource` object was a URI and needed to be normalized through a call to `.getPath` before it could be passed into the `JarFile` object. Author: Brennon York Closes #3561 from brennonyork/SPARK-4298 and squashes the following commits: 5e0fce1 [Brennon York] Use string interpolation for error messages, moved comment line from original code to above its necessary code segment 14daa20 [Brennon York] pushed mainClass assignment into match statement, removed spurious spaces, removed { } from case statements, removed return values c6dad68 [Brennon York] Set case statement to support multiple jar URI's and enabled the 'file' URI to load the main-class 8d20936 [Brennon York] updated to reset the error message back to the default a043039 [Brennon York] updated to split the uri and jar vals 8da7cbf [Brennon York] fixes SPARK-4298 --- .../spark/deploy/SparkSubmitArguments.scala | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f174bc1af59b4..1faabe91f49a8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy +import java.net.URI import java.util.jar.JarFile import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -125,14 +126,23 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { - try { - val jar = new JarFile(primaryResource) - // Note that this might still return null if no main-class is set; we catch that later - mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") - } catch { - case e: Exception => - SparkSubmit.printErrorAndExit("Cannot load main class from JAR: " + primaryResource) - return + val uri = new URI(primaryResource) + val uriScheme = uri.getScheme() + + uriScheme match { + case "file" => + try { + val jar = new JarFile(uri.getPath) + // Note that this might still return null if no main-class is set; we catch that later + mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } catch { + case e: Exception => + SparkSubmit.printErrorAndExit(s"Cannot load main class from JAR $primaryResource") + } + case _ => + SparkSubmit.printErrorAndExit( + s"Cannot load main class from JAR $primaryResource with URI $uriScheme. " + + "Please specify a class through --class.") } } From 3d194cc75761fceba77b2c91291b36479b8b556c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 31 Dec 2014 13:37:04 -0800 Subject: [PATCH 169/227] SPARK-4547 [MLLIB] OOM when making bins in BinaryClassificationMetrics Now that I've implemented the basics here, I'm less convinced there is a need for this change, somehow. Callers can downsample before or after. Really the OOM is not in the ROC curve code, but in code that might `collect()` it for local analysis. Still, might be useful to down-sample since the ROC curve probably never needs millions of points. This is a first pass. Since the `(score,label)` are already grouped and sorted, I think it's sufficient to just take every Nth such pair, in order to downsample by a factor of N? this is just like retaining every Nth point on the curve, which I think is the goal. All of the data is still used to build the curve of course. What do you think about the API, and usefulness? Author: Sean Owen Closes #3702 from srowen/SPARK-4547 and squashes the following commits: 1d34d05 [Sean Owen] Indent and reorganize numBins scaladoc 692d825 [Sean Owen] Change handling of large numBins, make 2nd consturctor instead of optional param, style change a03610e [Sean Owen] Add downsamplingFactor to BinaryClassificationMetrics --- .../BinaryClassificationMetrics.scala | 59 ++++++++++++++++++- .../BinaryClassificationMetricsSuite.scala | 36 +++++++++++ 2 files changed, 92 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 1af40de2c7fcf..ced042e2f96ca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -28,9 +28,30 @@ import org.apache.spark.rdd.{RDD, UnionRDD} * Evaluator for binary classification. * * @param scoreAndLabels an RDD of (score, label) pairs. + * @param numBins if greater than 0, then the curves (ROC curve, PR curve) computed internally + * will be down-sampled to this many "bins". If 0, no down-sampling will occur. + * This is useful because the curve contains a point for each distinct score + * in the input, and this could be as large as the input itself -- millions of + * points or more, when thousands may be entirely sufficient to summarize + * the curve. After down-sampling, the curves will instead be made of approximately + * `numBins` points instead. Points are made from bins of equal numbers of + * consecutive points. The size of each bin is + * `floor(scoreAndLabels.count() / numBins)`, which means the resulting number + * of bins may not exactly equal numBins. The last bin in each partition may + * be smaller as a result, meaning there may be an extra sample at + * partition boundaries. */ @Experimental -class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends Logging { +class BinaryClassificationMetrics( + val scoreAndLabels: RDD[(Double, Double)], + val numBins: Int) extends Logging { + + require(numBins >= 0, "numBins must be nonnegative") + + /** + * Defaults `numBins` to 0. + */ + def this(scoreAndLabels: RDD[(Double, Double)]) = this(scoreAndLabels, 0) /** Unpersist intermediate RDDs used in the computation. */ def unpersist() { @@ -103,7 +124,39 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends mergeValue = (c: BinaryLabelCounter, label: Double) => c += label, mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2 ).sortByKey(ascending = false) - val agg = counts.values.mapPartitions { iter => + + val binnedCounts = + // Only down-sample if bins is > 0 + if (numBins == 0) { + // Use original directly + counts + } else { + val countsSize = counts.count() + // Group the iterator into chunks of about countsSize / numBins points, + // so that the resulting number of bins is about numBins + var grouping = countsSize / numBins + if (grouping < 2) { + // numBins was more than half of the size; no real point in down-sampling to bins + logInfo(s"Curve is too small ($countsSize) for $numBins bins to be useful") + counts + } else { + if (grouping >= Int.MaxValue) { + logWarning( + s"Curve too large ($countsSize) for $numBins bins; capping at ${Int.MaxValue}") + grouping = Int.MaxValue + } + counts.mapPartitions(_.grouped(grouping.toInt).map { pairs => + // The score of the combined point will be just the first one's score + val firstScore = pairs.head._1 + // The point will contain all counts in this chunk + val agg = new BinaryLabelCounter() + pairs.foreach(pair => agg += pair._2) + (firstScore, agg) + }) + } + } + + val agg = binnedCounts.values.mapPartitions { iter => val agg = new BinaryLabelCounter() iter.foreach(agg += _) Iterator(agg) @@ -113,7 +166,7 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c) val totalCount = partitionwiseCumulativeCounts.last logInfo(s"Total counts: $totalCount") - val cumulativeCounts = counts.mapPartitionsWithIndex( + val cumulativeCounts = binnedCounts.mapPartitionsWithIndex( (index: Int, iter: Iterator[(Double, BinaryLabelCounter)]) => { val cumCount = partitionwiseCumulativeCounts(index) iter.map { case (score, c) => diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 8a18e2971cab6..e0224f960cc43 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -124,4 +124,40 @@ class BinaryClassificationMetricsSuite extends FunSuite with MLlibTestSparkConte validateMetrics(metrics, thresholds, rocCurve, prCurve, f1, f2, precisions, recalls) } + + test("binary evaluation metrics with downsampling") { + val scoreAndLabels = Seq( + (0.1, 0.0), (0.2, 0.0), (0.3, 1.0), (0.4, 0.0), (0.5, 0.0), + (0.6, 1.0), (0.7, 1.0), (0.8, 0.0), (0.9, 1.0)) + + val scoreAndLabelsRDD = sc.parallelize(scoreAndLabels, 1) + + val original = new BinaryClassificationMetrics(scoreAndLabelsRDD) + val originalROC = original.roc().collect().sorted.toList + // Add 2 for (0,0) and (1,1) appended at either end + assert(2 + scoreAndLabels.size == originalROC.size) + assert( + List( + (0.0, 0.0), (0.0, 0.25), (0.2, 0.25), (0.2, 0.5), (0.2, 0.75), + (0.4, 0.75), (0.6, 0.75), (0.6, 1.0), (0.8, 1.0), (1.0, 1.0), + (1.0, 1.0) + ) == + originalROC) + + val numBins = 4 + + val downsampled = new BinaryClassificationMetrics(scoreAndLabelsRDD, numBins) + val downsampledROC = downsampled.roc().collect().sorted.toList + assert( + // May have to add 1 if the sample factor didn't divide evenly + 2 + (numBins + (if (scoreAndLabels.size % numBins == 0) 0 else 1)) == + downsampledROC.size) + assert( + List( + (0.0, 0.0), (0.2, 0.25), (0.2, 0.75), (0.6, 0.75), (0.8, 1.0), + (1.0, 1.0), (1.0, 1.0) + ) == + downsampledROC) + } + } From e24d3a9a29962023cc722896a14c7bfe06e8e601 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Dec 2014 12:38:37 -0800 Subject: [PATCH 170/227] [HOTFIX] Disable Spark UI in SparkSubmitSuite tests This should fix a major cause of build breaks when running many parallel tests. --- .../test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 ++ 1 file changed, 2 insertions(+) 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 5eda2d41f0e6d..065b7534cece6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -290,6 +290,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local", + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -304,6 +305,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } From c88a3d7fca20d36ee566d48e0cb91fe33a7a6d99 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 31 Dec 2014 14:25:03 -0800 Subject: [PATCH 171/227] [SPARK-5038][SQL] Add explicit return type for implicit functions in Spark SQL As we learned in https://github.com/apache/spark/pull/3580, not explicitly typing implicit functions can lead to compiler bugs and potentially unexpected runtime behavior. Author: Reynold Xin Closes #3859 from rxin/sql-implicits and squashes the following commits: 30c2c24 [Reynold Xin] [SPARK-5038] Add explicit return type for implicit functions in Spark SQL. --- .../spark/sql/catalyst/dsl/package.scala | 80 +++++++++---------- .../org/apache/spark/sql/SQLContext.scala | 2 +- 2 files changed, 41 insertions(+), 41 deletions(-) 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 index a14e5b9ef14d0..8e39f79d2ca51 100755 --- 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 @@ -19,8 +19,6 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} @@ -29,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal /** * A collection of implicit conversions that create a DSL for constructing catalyst data structures. @@ -119,21 +118,22 @@ package object dsl { def expr = e } - implicit def booleanToLiteral(b: Boolean) = Literal(b) - implicit def byteToLiteral(b: Byte) = Literal(b) - implicit def shortToLiteral(s: Short) = Literal(s) - implicit def intToLiteral(i: Int) = Literal(i) - implicit def longToLiteral(l: Long) = Literal(l) - implicit def floatToLiteral(f: Float) = Literal(f) - implicit def doubleToLiteral(d: Double) = Literal(d) - implicit def stringToLiteral(s: String) = Literal(s) - implicit def dateToLiteral(d: Date) = Literal(d) - implicit def bigDecimalToLiteral(d: BigDecimal) = Literal(d) - implicit def decimalToLiteral(d: Decimal) = Literal(d) - implicit def timestampToLiteral(t: Timestamp) = Literal(t) - implicit def binaryToLiteral(a: Array[Byte]) = Literal(a) - - implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name) + implicit def booleanToLiteral(b: Boolean): Literal = Literal(b) + implicit def byteToLiteral(b: Byte): Literal = Literal(b) + implicit def shortToLiteral(s: Short): Literal = Literal(s) + implicit def intToLiteral(i: Int): Literal = Literal(i) + implicit def longToLiteral(l: Long): Literal = Literal(l) + implicit def floatToLiteral(f: Float): Literal = Literal(f) + implicit def doubleToLiteral(d: Double): Literal = Literal(d) + implicit def stringToLiteral(s: String): Literal = Literal(s) + implicit def dateToLiteral(d: Date): Literal = Literal(d) + implicit def bigDecimalToLiteral(d: BigDecimal): Literal = Literal(d) + implicit def decimalToLiteral(d: Decimal): Literal = Literal(d) + implicit def timestampToLiteral(t: Timestamp): Literal = Literal(t) + implicit def binaryToLiteral(a: Array[Byte]): Literal = Literal(a) + + implicit def symbolToUnresolvedAttribute(s: Symbol): analysis.UnresolvedAttribute = + analysis.UnresolvedAttribute(s.name) def sum(e: Expression) = Sum(e) def sumDistinct(e: Expression) = SumDistinct(e) @@ -301,52 +301,52 @@ package object dsl { (1 to 22).map { x => val argTypes = Seq.fill(x)("_").mkString(", ") - s"implicit def functionToUdfBuilder[T: TypeTag](func: Function$x[$argTypes, T]) = ScalaUdfBuilder(func)" + s"implicit def functionToUdfBuilder[T: TypeTag](func: Function$x[$argTypes, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func)" } */ - implicit def functionToUdfBuilder[T: TypeTag](func: Function1[_, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function1[_, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function2[_, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function2[_, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function3[_, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function3[_, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function4[_, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function4[_, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function5[_, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function5[_, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function6[_, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function6[_, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function7[_, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function7[_, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function8[_, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function8[_, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function9[_, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function9[_, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function10[_, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function10[_, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) // scalastyle:on } 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 7a13302229012..6a1a4d995bf61 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 @@ -106,7 +106,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = { + implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]): SchemaRDD = { SparkPlan.currentContext.set(self) val attributeSeq = ScalaReflection.attributesFor[A] val schema = StructType.fromAttributes(attributeSeq) From 3610d3c615112faef98d94f04efaea602cc4aa8f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 31 Dec 2014 14:35:07 -0800 Subject: [PATCH 172/227] [SPARK-4790][STREAMING] Fix ReceivedBlockTrackerSuite waits for old file... ...s to get deleted before continuing. Since the deletes are happening asynchronously, the getFileStatus call might throw an exception in older HDFS versions, if the delete happens between the time listFiles is called on the directory and getFileStatus is called on the file in the getFileStatus method. This PR addresses this by adding an option to delete the files synchronously and then waiting for the deletion to complete before proceeding. Author: Hari Shreedharan Closes #3726 from harishreedharan/spark-4790 and squashes the following commits: bbbacd1 [Hari Shreedharan] Call cleanUpOldLogs only once in the tests. 3255f17 [Hari Shreedharan] Add test for async deletion. Remove method from ReceiverTracker that does not take waitForCompletion. e4c83ec [Hari Shreedharan] Making waitForCompletion a mandatory param. Remove eventually from WALSuite since the cleanup method returns only after all files are deleted. af00fd1 [Hari Shreedharan] [SPARK-4790][STREAMING] Fix ReceivedBlockTrackerSuite waits for old files to get deleted before continuing. --- .../receiver/ReceivedBlockHandler.scala | 8 ++++---- .../scheduler/ReceivedBlockTracker.scala | 9 ++++++--- .../streaming/scheduler/ReceiverTracker.scala | 2 +- .../streaming/util/WriteAheadLogManager.scala | 17 +++++++++++++---- .../streaming/ReceivedBlockHandlerSuite.scala | 2 +- .../streaming/ReceivedBlockTrackerSuite.scala | 2 +- .../streaming/util/WriteAheadLogSuite.scala | 18 ++++++++++++++++-- 7 files changed, 42 insertions(+), 16 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index 8b97db8dd36f1..f7a8ebee8a544 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -42,7 +42,7 @@ private[streaming] trait ReceivedBlockHandler { def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock): ReceivedBlockStoreResult /** Cleanup old blocks older than the given threshold time */ - def cleanupOldBlock(threshTime: Long) + def cleanupOldBlocks(threshTime: Long) } @@ -82,7 +82,7 @@ private[streaming] class BlockManagerBasedBlockHandler( BlockManagerBasedStoreResult(blockId) } - def cleanupOldBlock(threshTime: Long) { + def cleanupOldBlocks(threshTime: Long) { // this is not used as blocks inserted into the BlockManager are cleared by DStream's clearing // of BlockRDDs. } @@ -192,8 +192,8 @@ private[streaming] class WriteAheadLogBasedBlockHandler( WriteAheadLogBasedStoreResult(blockId, segment) } - def cleanupOldBlock(threshTime: Long) { - logManager.cleanupOldLogs(threshTime) + def cleanupOldBlocks(threshTime: Long) { + logManager.cleanupOldLogs(threshTime, waitForCompletion = false) } def stop() { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 02758e0bca6c5..2ce458cddec1a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -139,14 +139,17 @@ private[streaming] class ReceivedBlockTracker( getReceivedBlockQueue(streamId).toSeq } - /** Clean up block information of old batches. */ - def cleanupOldBatches(cleanupThreshTime: Time): Unit = synchronized { + /** + * Clean up block information of old batches. If waitForCompletion is true, this method + * returns only after the files are cleaned up. + */ + def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { assert(cleanupThreshTime.milliseconds < clock.currentTime()) val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq logInfo("Deleting batches " + timesToCleanup) writeToLog(BatchCleanupEvent(timesToCleanup)) timeToAllocatedBlocks --= timesToCleanup - logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds)) + logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds, waitForCompletion)) log } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 1f0e442a12283..8dbb42a86e3bd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -121,7 +121,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Clean up metadata older than the given threshold time */ def cleanupOldMetadata(cleanupThreshTime: Time) { - receivedBlockTracker.cleanupOldBatches(cleanupThreshTime) + receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false) } /** Register a receiver */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala index 70d234320be7c..166661b7496df 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala @@ -19,11 +19,11 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, ExecutionContext, Future} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.Logging import org.apache.spark.util.Utils import WriteAheadLogManager._ @@ -124,8 +124,12 @@ private[streaming] class WriteAheadLogManager( * files, which is usually based on the local system time. So if there is coordination necessary * between the node calculating the threshTime (say, driver node), and the local system time * (say, worker node), the caller has to take account of possible time skew. + * + * If waitForCompletion is set to true, this method will return only after old logs have been + * deleted. This should be set to true only for testing. Else the files will be deleted + * asynchronously. */ - def cleanupOldLogs(threshTime: Long): Unit = { + def cleanupOldLogs(threshTime: Long, waitForCompletion: Boolean): Unit = { val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") @@ -146,10 +150,15 @@ private[streaming] class WriteAheadLogManager( logInfo(s"Cleared log files in $logDirectory older than $threshTime") } if (!executionContext.isShutdown) { - Future { deleteFiles() } + val f = Future { deleteFiles() } + if (waitForCompletion) { + import scala.concurrent.duration._ + Await.ready(f, 1 second) + } } } + /** Stop the manager, close any open log writer */ def stop(): Unit = synchronized { if (currentLogWriter != null) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 3661e16a9ef2f..132ff2443fc0f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -168,7 +168,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche manualClock.currentTime() shouldEqual 5000L val cleanupThreshTime = 3000L - handler.cleanupOldBlock(cleanupThreshTime) + handler.cleanupOldBlocks(cleanupThreshTime) eventually(timeout(10000 millis), interval(10 millis)) { getWriteAheadLogFiles().size should be < preCleanupLogFiles.size } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 01a09b67b99dc..de7e9d624bf6b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -166,7 +166,7 @@ class ReceivedBlockTrackerSuite // Cleanup first batch but not second batch val oldestLogFile = getWriteAheadLogFiles().head incrementTime() - tracker3.cleanupOldBatches(batchTime2) + tracker3.cleanupOldBatches(batchTime2, waitForCompletion = true) // Verify that the batch allocations have been cleaned, and the act has been written to log tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual Seq.empty diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 8f69bcb64279d..7ce9499dc614d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -182,15 +182,29 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { } test("WriteAheadLogManager - cleanup old logs") { + logCleanUpTest(waitForCompletion = false) + } + + test("WriteAheadLogManager - cleanup old logs synchronously") { + logCleanUpTest(waitForCompletion = true) + } + + private def logCleanUpTest(waitForCompletion: Boolean): Unit = { // Write data with manager, recover with new manager and verify val manualClock = new ManualClock val dataToWrite = generateRandomData() manager = writeDataUsingManager(testDir, dataToWrite, manualClock, stopManager = false) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - manager.cleanupOldLogs(manualClock.currentTime() / 2) - eventually(timeout(1 second), interval(10 milliseconds)) { + + manager.cleanupOldLogs(manualClock.currentTime() / 2, waitForCompletion) + + if (waitForCompletion) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } else { + eventually(timeout(1 second), interval(10 milliseconds)) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } } } From fdc2aa4918fd4c510f04812b782cc0bfef9a2107 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 31 Dec 2014 14:45:31 -0800 Subject: [PATCH 173/227] [SPARK-5028][Streaming]Add total received and processed records metrics to Streaming UI This is a follow-up work of [SPARK-4537](https://issues.apache.org/jira/browse/SPARK-4537). Adding total received records and processed records metrics back to UI. ![screenshot](https://dl.dropboxusercontent.com/u/19230832/screenshot.png) Author: jerryshao Closes #3852 from jerryshao/SPARK-5028 and squashes the following commits: c8c4877 [jerryshao] Add total received and processed metrics to Streaming UI --- .../scala/org/apache/spark/streaming/ui/StreamingPage.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 1353e487c72cf..98e9a2e639e25 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -67,6 +67,12 @@ private[ui] class StreamingPage(parent: StreamingTab)
  • Waiting batches: {listener.numUnprocessedBatches}
  • +
  • + Received records: {listener.numTotalReceivedRecords} +
  • +
  • + Processed records: {listener.numTotalProcessedRecords} +
  • } From c4f0b4f334f7f3565375921fcac184ad5b1fb207 Mon Sep 17 00:00:00 2001 From: Travis Galoppo Date: Wed, 31 Dec 2014 15:39:58 -0800 Subject: [PATCH 174/227] SPARK-5020 [MLlib] GaussianMixtureModel.predictMembership() should take an RDD only Removed unnecessary parameters to predictMembership() CC: jkbradley Author: Travis Galoppo Closes #3854 from tgaloppo/spark-5020 and squashes the following commits: 1bf4669 [Travis Galoppo] renamed predictMembership() to predictSoft() 0f1d96e [Travis Galoppo] SPARK-5020 - Removed superfluous parameters from predictMembership() --- .../spark/mllib/clustering/GaussianMixtureModel.scala | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 11a110db1f7ca..b461ea4f0f06e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -45,7 +45,7 @@ class GaussianMixtureModel( /** Maps given points to their cluster indices. */ def predict(points: RDD[Vector]): RDD[Int] = { - val responsibilityMatrix = predictMembership(points, mu, sigma, weight, k) + val responsibilityMatrix = predictSoft(points) responsibilityMatrix.map(r => r.indexOf(r.max)) } @@ -53,12 +53,7 @@ class GaussianMixtureModel( * Given the input vectors, return the membership value of each vector * to all mixture components. */ - def predictMembership( - points: RDD[Vector], - mu: Array[Vector], - sigma: Array[Matrix], - weight: Array[Double], - k: Int): RDD[Array[Double]] = { + def predictSoft(points: RDD[Vector]): RDD[Array[Double]] = { val sc = points.sparkContext val dists = sc.broadcast { (0 until k).map { i => From fe6efacc0b865e9e827a1565877077000e63976e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 31 Dec 2014 16:02:47 -0800 Subject: [PATCH 175/227] [SPARK-5035] [Streaming] ReceiverMessage trait should extend Serializable Spark Streaming's ReceiverMessage trait should extend Serializable in order to fix a subtle bug that only occurs when running on a real cluster: If you attempt to send a fire-and-forget message to a remote Akka actor and that message cannot be serialized, then this seems to lead to more-or-less silent failures. As an optimization, Akka skips message serialization for messages sent within the same JVM. As a result, Spark's unit tests will never fail due to non-serializable Akka messages, but these will cause mostly-silent failures when running on a real cluster. Before this patch, here was the code for ReceiverMessage: ``` /** Messages sent to the NetworkReceiver. */ private[streaming] sealed trait ReceiverMessage private[streaming] object StopReceiver extends ReceiverMessage ``` Since ReceiverMessage does not extend Serializable and StopReceiver is a regular `object`, not a `case object`, StopReceiver will throw serialization errors. As a result, graceful receiver shutdown is broken on real clusters (and local-cluster mode) but works in local modes. If you want to reproduce this, try running the word count example from the Streaming Programming Guide in the Spark shell: ``` import org.apache.spark._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ val ssc = new StreamingContext(sc, Seconds(10)) // Create a DStream that will connect to hostname:port, like localhost:9999 val lines = ssc.socketTextStream("localhost", 9999) // Split each line into words val words = lines.flatMap(_.split(" ")) import org.apache.spark.streaming.StreamingContext._ // Count each word in each batch val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) // Print the first ten elements of each RDD generated in this DStream to the console wordCounts.print() ssc.start() Thread.sleep(10000) ssc.stop(true, true) ``` Prior to this patch, this would work correctly in local mode but fail when running against a real cluster (it would report that some receivers were not shut down). Author: Josh Rosen Closes #3857 from JoshRosen/SPARK-5035 and squashes the following commits: 71d0eae [Josh Rosen] [SPARK-5035] ReceiverMessage trait should extend Serializable. --- .../org/apache/spark/streaming/receiver/ReceiverMessage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala index bf39d1e891cae..ab9fa192191aa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala @@ -18,6 +18,6 @@ package org.apache.spark.streaming.receiver /** Messages sent to the NetworkReceiver. */ -private[streaming] sealed trait ReceiverMessage +private[streaming] sealed trait ReceiverMessage extends Serializable private[streaming] object StopReceiver extends ReceiverMessage From 4bb12488d56ea651c56d9688996b464b99095582 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 31 Dec 2014 16:59:17 -0800 Subject: [PATCH 176/227] SPARK-2757 [BUILD] [STREAMING] Add Mima test for Spark Sink after 1.10 is released Re-enable MiMa for Streaming Flume Sink module, now that 1.1.0 is released, per the JIRA TO-DO. That's pretty much all there is to this. Author: Sean Owen Closes #3842 from srowen/SPARK-2757 and squashes the following commits: 50ff80e [Sean Owen] Exclude apparent false positive turned up by re-enabling MiMa checks for Streaming Flume Sink 0e5ba5c [Sean Owen] Re-enable MiMa for Streaming Flume Sink module --- project/MimaExcludes.scala | 5 +++++ project/SparkBuild.scala | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 230239aa40500..c377e5cffa7d2 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -53,6 +53,11 @@ object MimaExcludes { "org.apache.spark.mllib.linalg.Matrices.randn"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrices.rand") + ) ++ Seq( + // SPARK-2757 + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." + + "removeAndGetProcessor") ) case v if v.startsWith("1.2") => diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c512b62f6137e..46a54c6818409 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -166,7 +166,7 @@ object SparkBuild extends PomBuild { // TODO: Add Sql to mima checks allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - streamingFlumeSink, networkCommon, networkShuffle, networkYarn).contains(x)).foreach { + networkCommon, networkShuffle, networkYarn).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) } From 7749dd6c36a182478b20f4636734c8db0b7ddb00 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 31 Dec 2014 17:07:47 -0800 Subject: [PATCH 177/227] [SPARK-5038] Add explicit return type for implicit functions. As we learned in #3580, not explicitly typing implicit functions can lead to compiler bugs and potentially unexpected runtime behavior. This is a follow up PR for rest of Spark (outside Spark SQL). The original PR for Spark SQL can be found at https://github.com/apache/spark/pull/3859 Author: Reynold Xin Closes #3860 from rxin/implicit and squashes the following commits: 73702f9 [Reynold Xin] [SPARK-5038] Add explicit return type for implicit functions. --- .../scala/org/apache/spark/SparkContext.scala | 14 ++--- .../scala/org/apache/spark/util/Vector.scala | 38 +++++------ .../graphx/impl/EdgePartitionBuilder.scala | 63 ++++++++++--------- .../impl/ShippableVertexPartition.scala | 4 +- .../spark/graphx/impl/VertexPartition.scala | 4 +- .../graphx/impl/VertexPartitionBaseOps.scala | 4 +- 6 files changed, 64 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 57bc3d4e4ae36..df1cb3cda2dba 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1708,19 +1708,19 @@ object SparkContext extends Logging { // Implicit conversions to common Writable types, for saveAsSequenceFile - implicit def intToIntWritable(i: Int) = new IntWritable(i) + implicit def intToIntWritable(i: Int): IntWritable = new IntWritable(i) - implicit def longToLongWritable(l: Long) = new LongWritable(l) + implicit def longToLongWritable(l: Long): LongWritable = new LongWritable(l) - implicit def floatToFloatWritable(f: Float) = new FloatWritable(f) + implicit def floatToFloatWritable(f: Float): FloatWritable = new FloatWritable(f) - implicit def doubleToDoubleWritable(d: Double) = new DoubleWritable(d) + implicit def doubleToDoubleWritable(d: Double): DoubleWritable = new DoubleWritable(d) - implicit def boolToBoolWritable (b: Boolean) = new BooleanWritable(b) + implicit def boolToBoolWritable (b: Boolean): BooleanWritable = new BooleanWritable(b) - implicit def bytesToBytesWritable (aob: Array[Byte]) = new BytesWritable(aob) + implicit def bytesToBytesWritable (aob: Array[Byte]): BytesWritable = new BytesWritable(aob) - implicit def stringToText(s: String) = new Text(s) + implicit def stringToText(s: String): Text = new Text(s) private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]) : ArrayWritable = { diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index c6cab82c3e546..2ed827eab46df 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -24,9 +24,9 @@ import org.apache.spark.util.random.XORShiftRandom @deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { - def length = elements.length + def length: Int = elements.length - def apply(index: Int) = elements(index) + def apply(index: Int): Double = elements(index) def + (other: Vector): Vector = { if (length != other.length) { @@ -35,7 +35,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) + other(i)) } - def add(other: Vector) = this + other + def add(other: Vector): Vector = this + other def - (other: Vector): Vector = { if (length != other.length) { @@ -44,7 +44,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) - other(i)) } - def subtract(other: Vector) = this - other + def subtract(other: Vector): Vector = this - other def dot(other: Vector): Double = { if (length != other.length) { @@ -93,19 +93,19 @@ class Vector(val elements: Array[Double]) extends Serializable { this } - def addInPlace(other: Vector) = this +=other + def addInPlace(other: Vector): Vector = this +=other def * (scale: Double): Vector = Vector(length, i => this(i) * scale) - def multiply (d: Double) = this * d + def multiply (d: Double): Vector = this * d def / (d: Double): Vector = this * (1 / d) - def divide (d: Double) = this / d + def divide (d: Double): Vector = this / d - def unary_- = this * -1 + def unary_- : Vector = this * -1 - def sum = elements.reduceLeft(_ + _) + def sum: Double = elements.reduceLeft(_ + _) def squaredDist(other: Vector): Double = { var ans = 0.0 @@ -119,40 +119,40 @@ class Vector(val elements: Array[Double]) extends Serializable { def dist(other: Vector): Double = math.sqrt(squaredDist(other)) - override def toString = elements.mkString("(", ", ", ")") + override def toString: String = elements.mkString("(", ", ", ")") } object Vector { - def apply(elements: Array[Double]) = new Vector(elements) + def apply(elements: Array[Double]): Vector = new Vector(elements) - def apply(elements: Double*) = new Vector(elements.toArray) + def apply(elements: Double*): Vector = new Vector(elements.toArray) def apply(length: Int, initializer: Int => Double): Vector = { val elements: Array[Double] = Array.tabulate(length)(initializer) new Vector(elements) } - def zeros(length: Int) = new Vector(new Array[Double](length)) + def zeros(length: Int): Vector = new Vector(new Array[Double](length)) - def ones(length: Int) = Vector(length, _ => 1) + def ones(length: Int): Vector = Vector(length, _ => 1) /** * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ - def random(length: Int, random: Random = new XORShiftRandom()) = + def random(length: Int, random: Random = new XORShiftRandom()): Vector = Vector(length, _ => random.nextDouble()) class Multiplier(num: Double) { - def * (vec: Vector) = vec * num + def * (vec: Vector): Vector = vec * num } - implicit def doubleToMultiplier(num: Double) = new Multiplier(num) + implicit def doubleToMultiplier(num: Double): Multiplier = new Multiplier(num) implicit object VectorAccumParam extends org.apache.spark.AccumulatorParam[Vector] { - def addInPlace(t1: Vector, t2: Vector) = t1 + t2 + def addInPlace(t1: Vector, t2: Vector): Vector = t1 + t2 - def zero(initialValue: Vector) = Vector.zeros(initialValue.length) + def zero(initialValue: Vector): Vector = Vector.zeros(initialValue.length) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 409cf60977f6f..906d42328fcb9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -129,44 +129,45 @@ private[impl] case class EdgeWithLocalIds[@specialized ED]( srcId: VertexId, dstId: VertexId, localSrcId: Int, localDstId: Int, attr: ED) private[impl] object EdgeWithLocalIds { - implicit def lexicographicOrdering[ED] = new Ordering[EdgeWithLocalIds[ED]] { - override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = { - if (a.srcId == b.srcId) { - if (a.dstId == b.dstId) 0 - else if (a.dstId < b.dstId) -1 + implicit def lexicographicOrdering[ED]: Ordering[EdgeWithLocalIds[ED]] = + new Ordering[EdgeWithLocalIds[ED]] { + override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = { + if (a.srcId == b.srcId) { + if (a.dstId == b.dstId) 0 + else if (a.dstId < b.dstId) -1 + else 1 + } else if (a.srcId < b.srcId) -1 else 1 - } else if (a.srcId < b.srcId) -1 - else 1 + } } - } - private[graphx] def edgeArraySortDataFormat[ED] - = new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { - override def getKey( - data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = { - data(pos) - } + private[graphx] def edgeArraySortDataFormat[ED] = { + new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { + override def getKey(data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = { + data(pos) + } - override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = { - val tmp = data(pos0) - data(pos0) = data(pos1) - data(pos1) = tmp - } + override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } - override def copyElement( - src: Array[EdgeWithLocalIds[ED]], srcPos: Int, - dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) { - dst(dstPos) = src(srcPos) - } + override def copyElement( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) { + dst(dstPos) = src(srcPos) + } - override def copyRange( - src: Array[EdgeWithLocalIds[ED]], srcPos: Int, - dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) { - System.arraycopy(src, srcPos, dst, dstPos, length) - } + override def copyRange( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } - override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = { - new Array[EdgeWithLocalIds[ED]](length) + override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = { + new Array[EdgeWithLocalIds[ED]](length) + } } } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index 5412d720475dc..aa320088f2088 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -74,8 +74,8 @@ object ShippableVertexPartition { * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a * `ShippableVertexPartition`. */ - implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) = - new ShippableVertexPartitionOps(partition) + implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) + : ShippableVertexPartitionOps[VD] = new ShippableVertexPartitionOps(partition) /** * Implicit evidence that `ShippableVertexPartition` is a member of the diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 55c7a19d1bdab..fbe53acfc32aa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -38,8 +38,8 @@ private[graphx] object VertexPartition { * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a * `VertexPartition`. */ - implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) = - new VertexPartitionOps(partition) + implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) + : VertexPartitionOps[VD] = new VertexPartitionOps(partition) /** * Implicit evidence that `VertexPartition` is a member of the `VertexPartitionBaseOpsConstructor` diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index b40aa1b417a0f..4fd2548b7faf6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -238,8 +238,8 @@ private[graphx] abstract class VertexPartitionBaseOps * because these methods return a `Self` and this implicit conversion re-wraps that in a * `VertexPartitionBaseOps`. This relies on the context bound on `Self`. */ - private implicit def toOps[VD2: ClassTag]( - partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = { + private implicit def toOps[VD2: ClassTag](partition: Self[VD2]) + : VertexPartitionBaseOps[VD2, Self] = { implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition) } } From 012839807c3dc6e7c8c41ac6e956d52a550bb031 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 1 Jan 2015 15:03:54 -0800 Subject: [PATCH 178/227] [HOTFIX] Bind web UI to ephemeral port in DriverSuite The job launched by DriverSuite should bind the web UI to an ephemeral port, since it looks like port contention in this test has caused a large number of Jenkins failures when many builds are started simultaneously. Our tests already disable the web UI, but this doesn't affect subprocesses launched by our tests. In this case, I've opted to bind to an ephemeral port instead of disabling the UI because disabling features in this test may mask its ability to catch certain bugs. See also: e24d3a9 Author: Josh Rosen Closes #3873 from JoshRosen/driversuite-webui-port and squashes the following commits: 48cd05c [Josh Rosen] [HOTFIX] Bind web UI to ephemeral port in DriverSuite. --- core/src/test/scala/org/apache/spark/DriverSuite.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 5265ba904032f..541d8eac80556 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -50,7 +50,10 @@ class DriverSuite extends FunSuite with Timeouts { object DriverWithoutCleanup { def main(args: Array[String]) { Utils.configTestLog4j("INFO") - val sc = new SparkContext(args(0), "DriverWithoutCleanup") + // Bind the web UI to an ephemeral port in order to avoid conflicts with other tests running on + // the same machine (we shouldn't just disable the UI here, since that might mask bugs): + val conf = new SparkConf().set("spark.ui.port", "0") + val sc = new SparkContext(args(0), "DriverWithoutCleanup", conf) sc.parallelize(1 to 100, 4).count() } } From bd88b7185358ae60efc83dc6cbb3fb1d2bff6074 Mon Sep 17 00:00:00 2001 From: Yadong Qi Date: Fri, 2 Jan 2015 15:09:41 -0800 Subject: [PATCH 179/227] [SPARK-3325][Streaming] Add a parameter to the method print in class DStream This PR is a fixed version of the original PR #3237 by watermen and scwf. This adds the ability to specify how many elements to print in `DStream.print`. Author: Yadong Qi Author: q00251598 Author: Tathagata Das Author: wangfei Closes #3865 from tdas/print-num and squashes the following commits: cd34e9e [Tathagata Das] Fix bug 7c09f16 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into HEAD bb35d1a [Yadong Qi] Update MimaExcludes.scala f8098ca [Yadong Qi] Update MimaExcludes.scala f6ac3cb [Yadong Qi] Update MimaExcludes.scala e4ed897 [Yadong Qi] Update MimaExcludes.scala 3b9d5cf [wangfei] fix conflicts ec8a3af [q00251598] move to Spark 1.3 26a70c0 [q00251598] extend the Python DStream's print b589a4b [q00251598] add another print function --- project/MimaExcludes.scala | 3 +++ python/pyspark/streaming/dstream.py | 12 +++++++----- .../spark/streaming/api/java/JavaDStreamLike.scala | 10 +++++++++- .../apache/spark/streaming/dstream/DStream.scala | 14 +++++++++++--- 4 files changed, 30 insertions(+), 9 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c377e5cffa7d2..31d4c317ae569 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -54,6 +54,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrices.rand") ) ++ Seq( + // SPARK-3325 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.streaming.api.java.JavaDStreamLike.print"), // SPARK-2757 ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." + diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 0826ddc56e844..2fe39392ff081 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -157,18 +157,20 @@ def foreachRDD(self, func): api = self._ssc._jvm.PythonDStream api.callForeachRDD(self._jdstream, jfunc) - def pprint(self): + def pprint(self, num=10): """ - Print the first ten elements of each RDD generated in this DStream. + Print the first num elements of each RDD generated in this DStream. + + @param num: the number of elements from the first will be printed. """ def takeAndPrint(time, rdd): - taken = rdd.take(11) + taken = rdd.take(num + 1) print "-------------------------------------------" print "Time: %s" % time print "-------------------------------------------" - for record in taken[:10]: + for record in taken[:num]: print record - if len(taken) > 10: + if len(taken) > num: print "..." print diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 2a7004e56ef53..e0542eda1383f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -51,7 +51,15 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * operator, so this DStream will be registered as an output stream and there materialized. */ def print(): Unit = { - dstream.print() + print(10) + } + + /** + * Print the first num elements of each RDD generated in this DStream. This is an output + * operator, so this DStream will be registered as an output stream and there materialized. + */ + def print(num: Int): Unit = { + dstream.print(num) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 7f8651e719d84..28fc00cf3944f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -605,13 +605,21 @@ abstract class DStream[T: ClassTag] ( * operator, so this DStream will be registered as an output stream and there materialized. */ def print() { + print(10) + } + + /** + * Print the first num elements of each RDD generated in this DStream. This is an output + * operator, so this DStream will be registered as an output stream and there materialized. + */ + def print(num: Int) { def foreachFunc = (rdd: RDD[T], time: Time) => { - val first11 = rdd.take(11) + val firstNum = rdd.take(num + 1) println ("-------------------------------------------") println ("Time: " + time) println ("-------------------------------------------") - first11.take(10).foreach(println) - if (first11.size > 10) println("...") + firstNum.take(num).foreach(println) + if (firstNum.size > num) println("...") println() } new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register() From cdccc263b20c1bb27b864411c82cfad7daca1f47 Mon Sep 17 00:00:00 2001 From: Akhil Das Date: Fri, 2 Jan 2015 15:12:27 -0800 Subject: [PATCH 180/227] Fixed typos in streaming-kafka-integration.md Changed projrect to project :) Author: Akhil Das Closes #3876 from akhld/patch-1 and squashes the following commits: e0cf9ef [Akhil Das] Fixed typos in streaming-kafka-integration.md --- docs/streaming-kafka-integration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 1c956fcb40da8..4378521dcac70 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -4,7 +4,7 @@ title: Spark Streaming + Kafka Integration Guide --- [Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. -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-kafka_{{site.SCALA_BINARY_VERSION}} From 342612b65f3d77c660383a332f0346872f076647 Mon Sep 17 00:00:00 2001 From: sigmoidanalytics Date: Sat, 3 Jan 2015 19:46:08 -0800 Subject: [PATCH 181/227] [SPARK-5058] Updated broken links Updated the broken link pointing to the KafkaWordCount example to the correct one. Author: sigmoidanalytics Closes #3877 from sigmoidanalytics/patch-1 and squashes the following commits: 3e19b31 [sigmoidanalytics] Updated broken links --- docs/streaming-kafka-integration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 4378521dcac70..0e38fe2144e9f 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -20,7 +20,7 @@ title: Spark Streaming + Kafka Integration Guide streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]) See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
    import org.apache.spark.streaming.kafka.*; From b96008d5529bac5fd57b76554fd01760139cffff Mon Sep 17 00:00:00 2001 From: Brennon York Date: Sun, 4 Jan 2015 12:40:39 -0800 Subject: [PATCH 182/227] [SPARK-794][Core] Remove sleep() in ClusterScheduler.stop Removed `sleep()` from the `stop()` method of the `TaskSchedulerImpl` class which, from the JIRA ticket, is believed to be a legacy artifact slowing down testing originally introduced in the `ClusterScheduler` class. Author: Brennon York Closes #3851 from brennonyork/SPARK-794 and squashes the following commits: 04c3e64 [Brennon York] Removed sleep() from the stop() method --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index cd3c015321e85..a41f3eef195d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -394,9 +394,6 @@ private[spark] class TaskSchedulerImpl( taskResultGetter.stop() } starvationTimer.cancel() - - // sleeping for an arbitrary 1 seconds to ensure that messages are sent out. - Thread.sleep(1000L) } override def defaultParallelism() = backend.defaultParallelism() From 3fddc9468fa50e7683caa973fec6c52e1132268d Mon Sep 17 00:00:00 2001 From: Dale Date: Sun, 4 Jan 2015 13:28:37 -0800 Subject: [PATCH 183/227] [SPARK-4787] Stop SparkContext if a DAGScheduler init error occurs Author: Dale Closes #3809 from tigerquoll/SPARK-4787 and squashes the following commits: 5661e01 [Dale] [SPARK-4787] Ensure that call to stop() doesn't lose the exception by using a finally block. 2172578 [Dale] [SPARK-4787] Stop context properly if an exception occurs during DAGScheduler initialization. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 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 df1cb3cda2dba..4c25d5d6c0ceb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -329,8 +329,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli try { dagScheduler = new DAGScheduler(this) } catch { - case e: Exception => throw - new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage)) + case e: Exception => { + try { + stop() + } finally { + throw new SparkException("Error while constructing DAGScheduler", e) + } + } } // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's From e767d7ddac5c2330af553f2a74b8575dfc7afb67 Mon Sep 17 00:00:00 2001 From: bilna Date: Sun, 4 Jan 2015 19:37:48 -0800 Subject: [PATCH 184/227] [SPARK-4631] unit test for MQTT Please review the unit test for MQTT Author: bilna Author: Bilna P Closes #3844 from Bilna/master and squashes the following commits: acea3a3 [bilna] Adding dependency with scope test 28681fa [bilna] Merge remote-tracking branch 'upstream/master' fac3904 [bilna] Correction in Indentation and coding style ed9db4c [bilna] Merge remote-tracking branch 'upstream/master' 4b34ee7 [Bilna P] Update MQTTStreamSuite.scala 04503cf [bilna] Added embedded broker service for mqtt test 89d804e [bilna] Merge remote-tracking branch 'upstream/master' fc8eb28 [bilna] Merge remote-tracking branch 'upstream/master' 4b58094 [Bilna P] Update MQTTStreamSuite.scala b1ac4ad [bilna] Added BeforeAndAfter 5f6bfd2 [bilna] Added BeforeAndAfter e8b6623 [Bilna P] Update MQTTStreamSuite.scala 5ca6691 [Bilna P] Update MQTTStreamSuite.scala 8616495 [bilna] [SPARK-4631] unit test for MQTT --- external/mqtt/pom.xml | 6 + .../streaming/mqtt/MQTTStreamSuite.scala | 110 +++++++++++++++--- 2 files changed, 101 insertions(+), 15 deletions(-) diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9025915f4447e..d478267b605ba 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -66,6 +66,12 @@ junit-interface test + + org.apache.activemq + activemq-core + 5.7.0 + test + target/scala-${scala.binary.version}/classes diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 84595acf45ccb..98fe6cb301f52 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -17,31 +17,111 @@ package org.apache.spark.streaming.mqtt -import org.scalatest.FunSuite +import java.net.{URI, ServerSocket} -import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.activemq.broker.{TransportConnector, BrokerService} +import org.apache.spark.util.Utils +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually +import scala.concurrent.duration._ +import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.eclipse.paho.client.mqttv3._ +import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence -class MQTTStreamSuite extends FunSuite { - - val batchDuration = Seconds(1) +class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { + private val batchDuration = Milliseconds(500) private val master: String = "local[2]" - private val framework: String = this.getClass.getSimpleName + private val freePort = findFreePort() + private val brokerUri = "//localhost:" + freePort + private val topic = "def" + private var ssc: StreamingContext = _ + private val persistenceDir = Utils.createTempDir() + private var broker: BrokerService = _ + private var connector: TransportConnector = _ - test("mqtt input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val brokerUrl = "abc" - val topic = "def" + before { + ssc = new StreamingContext(master, framework, batchDuration) + setupMQTT() + } - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, brokerUrl, topic) - val test2: ReceiverInputDStream[String] = - MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + Utils.deleteRecursively(persistenceDir) + tearDownMQTT() + } - // TODO: Actually test receiving data + test("mqtt input stream") { + val sendMessage = "MQTT demo for spark streaming" + val receiveStream: ReceiverInputDStream[String] = + MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY) + var receiveMessage: List[String] = List() + receiveStream.foreachRDD { rdd => + if (rdd.collect.length > 0) { + receiveMessage = receiveMessage ::: List(rdd.first) + receiveMessage + } + } + ssc.start() + publishData(sendMessage) + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + assert(sendMessage.equals(receiveMessage(0))) + } ssc.stop() } + + private def setupMQTT() { + broker = new BrokerService() + connector = new TransportConnector() + connector.setName("mqtt") + connector.setUri(new URI("mqtt:" + brokerUri)) + broker.addConnector(connector) + broker.start() + } + + private def tearDownMQTT() { + if (broker != null) { + broker.stop() + broker = null + } + if (connector != null) { + connector.stop() + connector = null + } + } + + private def findFreePort(): Int = { + Utils.startServiceOnPort(23456, (trialPort: Int) => { + val socket = new ServerSocket(trialPort) + socket.close() + (null, trialPort) + })._2 + } + + def publishData(data: String): Unit = { + var client: MqttClient = null + try { + val persistence: MqttClientPersistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath) + client = new MqttClient("tcp:" + brokerUri, MqttClient.generateClientId(), persistence) + client.connect() + if (client.isConnected) { + val msgTopic: MqttTopic = client.getTopic(topic) + val message: MqttMessage = new MqttMessage(data.getBytes("utf-8")) + message.setQos(1) + message.setRetained(true) + for (i <- 0 to 100) + msgTopic.publish(message) + } + } finally { + client.disconnect() + client.close() + client = null + } + } } From 939ba1f8f6e32fef9026cc43fce55b36e4b9bfd1 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 4 Jan 2015 20:26:18 -0800 Subject: [PATCH 185/227] [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs This patch disables output spec. validation for jobs launched through Spark Streaming, since this interferes with checkpoint recovery. Hadoop OutputFormats have a `checkOutputSpecs` method which performs certain checks prior to writing output, such as checking whether the output directory already exists. SPARK-1100 added checks for FileOutputFormat, SPARK-1677 (#947) added a SparkConf configuration to disable these checks, and SPARK-2309 (#1088) extended these checks to run for all OutputFormats, not just FileOutputFormat. In Spark Streaming, we might have to re-process a batch during checkpoint recovery, so `save` actions may be called multiple times. In addition to `DStream`'s own save actions, users might use `transform` or `foreachRDD` and call the `RDD` and `PairRDD` save actions. When output spec. validation is enabled, the second calls to these actions will fail due to existing output. This patch automatically disables output spec. validation for jobs submitted by the Spark Streaming scheduler. This is done by using Scala's `DynamicVariable` to propagate the bypass setting without having to mutate SparkConf or introduce a global variable. Author: Josh Rosen Closes #3832 from JoshRosen/SPARK-4835 and squashes the following commits: 36eaf35 [Josh Rosen] Add comment explaining use of transform() in test. 6485cf8 [Josh Rosen] Add test case in Streaming; fix bug for transform() 7b3e06a [Josh Rosen] Remove Streaming-specific setting to undo this change; update conf. guide bf9094d [Josh Rosen] Revise disableOutputSpecValidation() comment to not refer to Spark Streaming. e581d17 [Josh Rosen] Deduplicate isOutputSpecValidationEnabled logic. 762e473 [Josh Rosen] [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs. --- .../apache/spark/rdd/PairRDDFunctions.scala | 19 ++++++++- docs/configuration.md | 4 +- .../spark/streaming/dstream/DStream.scala | 10 ++++- .../dstream/TransformedDStream.scala | 2 +- .../streaming/scheduler/JobScheduler.scala | 8 +++- .../spark/streaming/CheckpointSuite.scala | 39 +++++++++++++++++++ 6 files changed, 75 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 4469c89e6bb1c..f8df5b2a08866 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -25,6 +25,7 @@ import scala.collection.{Map, mutable} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag +import scala.util.DynamicVariable import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} @@ -964,7 +965,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val outfmt = job.getOutputFormatClass val jobFormat = outfmt.newInstance - if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { + if (isOutputSpecValidationEnabled) { // FileOutputFormat ignores the filesystem parameter jobFormat.checkOutputSpecs(job) } @@ -1042,7 +1043,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") - if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { + if (isOutputSpecValidationEnabled) { // FileOutputFormat ignores the filesystem parameter val ignoredFs = FileSystem.get(hadoopConf) hadoopConf.getOutputFormat.checkOutputSpecs(ignoredFs, hadoopConf) @@ -1117,8 +1118,22 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) private[spark] def valueClass: Class[_] = vt.runtimeClass private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) + + // Note: this needs to be a function instead of a 'val' so that the disableOutputSpecValidation + // setting can take effect: + private def isOutputSpecValidationEnabled: Boolean = { + val validationDisabled = PairRDDFunctions.disableOutputSpecValidation.value + val enabledInConf = self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true) + enabledInConf && !validationDisabled + } } private[spark] object PairRDDFunctions { val RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES = 256 + + /** + * Allows for the `spark.hadoop.validateOutputSpecs` checks to be disabled on a case-by-case + * basis; see SPARK-4835 for more details. + */ + val disableOutputSpecValidation: DynamicVariable[Boolean] = new DynamicVariable[Boolean](false) } diff --git a/docs/configuration.md b/docs/configuration.md index fa9d311f85068..9bb6499993735 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -709,7 +709,9 @@ Apart from these, the following properties are also available, and may be useful
    + previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since + data may need to be rewritten to pre-existing output directories during checkpoint recovery. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 28fc00cf3944f..b874f561c12eb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -26,7 +26,7 @@ import scala.reflect.ClassTag import scala.util.matching.Regex import org.apache.spark.{Logging, SparkException} -import org.apache.spark.rdd.{BlockRDD, RDD} +import org.apache.spark.rdd.{BlockRDD, PairRDDFunctions, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext.rddToFileName @@ -292,7 +292,13 @@ abstract class DStream[T: ClassTag] ( // set this DStream's creation site, generate RDDs and then restore the previous call site. val prevCallSite = ssc.sparkContext.getCallSite() ssc.sparkContext.setCallSite(creationSite) - val rddOption = compute(time) + // Disable checks for existing output directories in jobs launched by the streaming + // scheduler, since we may need to write output to an existing directory during checkpoint + // recovery; see SPARK-4835 for more details. We need to have this call here because + // compute() might cause Spark jobs to be launched. + val rddOption = PairRDDFunctions.disableOutputSpecValidation.withValue(true) { + compute(time) + } ssc.sparkContext.setCallSite(prevCallSite) rddOption.foreach { case newRDD => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 7cd4554282ca1..71b61856e23c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{PairRDDFunctions, RDD} import org.apache.spark.streaming.{Duration, Time} import scala.reflect.ClassTag diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index cfa3cd8925c80..0e0f5bd3b9db4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -22,6 +22,7 @@ import scala.collection.JavaConversions._ import java.util.concurrent.{TimeUnit, ConcurrentHashMap, Executors} import akka.actor.{ActorRef, Actor, Props} import org.apache.spark.{SparkException, Logging, SparkEnv} +import org.apache.spark.rdd.PairRDDFunctions import org.apache.spark.streaming._ @@ -168,7 +169,12 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private class JobHandler(job: Job) extends Runnable { def run() { eventActor ! JobStarted(job) - job.run() + // Disable checks for existing output directories in jobs launched by the streaming scheduler, + // since we may need to write output to an existing directory during checkpoint recovery; + // see SPARK-4835 for more details. + PairRDDFunctions.disableOutputSpecValidation.withValue(true) { + job.run() + } eventActor ! JobCompleted(job) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 72d055eb2ea31..5d232c6ade7a9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -255,6 +255,45 @@ class CheckpointSuite extends TestSuiteBase { } } + test("recovery with saveAsHadoopFile inside transform operation") { + // Regression test for SPARK-4835. + // + // In that issue, the problem was that `saveAsHadoopFile(s)` would fail when the last batch + // was restarted from a checkpoint since the output directory would already exist. However, + // the other saveAsHadoopFile* tests couldn't catch this because they only tested whether the + // output matched correctly and not whether the post-restart batch had successfully finished + // without throwing any errors. The following test reproduces the same bug with a test that + // actually fails because the error in saveAsHadoopFile causes transform() to fail, which + // prevents the expected output from being written to the output stream. + // + // This is not actually a valid use of transform, but it's being used here so that we can test + // the fix for SPARK-4835 independently of additional test cleanup. + // + // After SPARK-5079 is addressed, should be able to remove this test since a strengthened + // version of the other saveAsHadoopFile* tests would prevent regressions for this issue. + val tempDir = Files.createTempDir() + try { + testCheckpointedOperation( + Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), + (s: DStream[String]) => { + s.transform { (rdd, time) => + val output = rdd.map(x => (x, 1)).reduceByKey(_ + _) + output.saveAsHadoopFile( + new File(tempDir, "result-" + time.milliseconds).getAbsolutePath, + classOf[Text], + classOf[IntWritable], + classOf[TextOutputFormat[Text, IntWritable]]) + output + } + }, + Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + 3 + ) + } finally { + Utils.deleteRecursively(tempDir) + } + } + // This tests whether the StateDStream's RDD checkpoints works correctly such // that the system can recover from a master failure. This assumes as reliable, // replayable input source - TestInputDStream. From 72396522bcf5303f761956658510672e4feb2845 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 4 Jan 2015 21:03:17 -0800 Subject: [PATCH 186/227] [SPARK-5067][Core] Use '===' to compare well-defined case class A simple fix would be adding `assert(e1.appId == e2.appId)` for `SparkListenerApplicationStart`. But actually we can use `===` for well-defined case class directly. Therefore, instead of fixing this issue, I use `===` to compare those well-defined case classes (all fields have implemented a correct `equals` method, such as primitive types) Author: zsxwing Closes #3886 from zsxwing/SPARK-5067 and squashes the following commits: 0a51711 [zsxwing] Use '===' to compare well-defined case class --- .../apache/spark/util/JsonProtocolSuite.scala | 32 +++---------------- 1 file changed, 4 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 593d6dd8c3794..63c2559c5c5f5 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -280,7 +280,7 @@ class JsonProtocolSuite extends FunSuite { private def testBlockManagerId(id: BlockManagerId) { val newId = JsonProtocol.blockManagerIdFromJson(JsonProtocol.blockManagerIdToJson(id)) - assertEquals(id, newId) + assert(id === newId) } private def testTaskInfo(info: TaskInfo) { @@ -335,22 +335,8 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.jobResult, e2.jobResult) case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => assertEquals(e1.environmentDetails, e2.environmentDetails) - case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => - assert(e1.maxMem === e2.maxMem) - assert(e1.time === e2.time) - assertEquals(e1.blockManagerId, e2.blockManagerId) - case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => - assert(e1.time === e2.time) - assertEquals(e1.blockManagerId, e2.blockManagerId) - case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId == e2.rddId) - case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => - assert(e1.appName == e2.appName) - assert(e1.time == e2.time) - assert(e1.sparkUser == e2.sparkUser) - case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => - assert(e1.time == e2.time) - case (SparkListenerShutdown, SparkListenerShutdown) => + case (e1, e2) => + assert(e1 === e2) case _ => fail("Events don't match in types!") } } @@ -435,16 +421,6 @@ class JsonProtocolSuite extends FunSuite { assert(metrics1.bytesRead === metrics2.bytesRead) } - private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { - if (bm1 == null || bm2 == null) { - assert(bm1 === bm2) - } else { - assert(bm1.executorId === bm2.executorId) - assert(bm1.host === bm2.host) - assert(bm1.port === bm2.port) - } - } - private def assertEquals(result1: JobResult, result2: JobResult) { (result1, result2) match { case (JobSucceeded, JobSucceeded) => @@ -462,7 +438,7 @@ class JsonProtocolSuite extends FunSuite { assert(r1.shuffleId === r2.shuffleId) assert(r1.mapId === r2.mapId) assert(r1.reduceId === r2.reduceId) - assertEquals(r1.bmAddress, r2.bmAddress) + assert(r1.bmAddress === r2.bmAddress) assert(r1.message === r2.message) case (r1: ExceptionFailure, r2: ExceptionFailure) => assert(r1.className === r2.className) From 6c726a3fbd9cd3aa5f3a1992b2132b25eabb76a0 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 4 Jan 2015 21:06:04 -0800 Subject: [PATCH 187/227] [SPARK-5069][Core] Fix the race condition of TaskSchedulerImpl.dagScheduler It's not necessary to set `TaskSchedulerImpl.dagScheduler` in preStart. It's safe to set it after `initializeEventProcessActor()`. Author: zsxwing Closes #3887 from zsxwing/SPARK-5069 and squashes the following commits: d95894f [zsxwing] Fix the race condition of TaskSchedulerImpl.dagScheduler --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 +------ .../apache/spark/scheduler/TaskSchedulerImplSuite.scala | 1 - 2 files changed, 1 insertion(+), 7 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 cb8ccfbdbdcbb..259621d263d7c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -138,6 +138,7 @@ class DAGScheduler( } initializeEventProcessActor() + taskScheduler.setDAGScheduler(this) // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { @@ -1375,12 +1376,6 @@ private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGScheduler) extends Actor with Logging { - override def preStart() { - // set DAGScheduler for taskScheduler to ensure eventProcessActor is always - // valid when the messages arrive - dagScheduler.taskScheduler.setDAGScheduler(dagScheduler) - } - /** * The main event loop of the DAG scheduler. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 40aaf9dd1f1e9..00812e6018d1f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -305,7 +305,6 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} override def executorAdded(execId: String, host: String) {} } - taskScheduler.setDAGScheduler(dagScheduler) // Give zero core offers. Should not generate any tasks val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) From 27e7f5a7237d9d64a3b2c8a030ba3e3a9a96b26c Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 4 Jan 2015 21:09:21 -0800 Subject: [PATCH 188/227] [SPARK-5083][Core] Fix a flaky test in TaskResultGetterSuite Because `sparkEnv.blockManager.master.removeBlock` is asynchronous, we need to make sure the block has already been removed before calling `super.enqueueSuccessfulTask`. Author: zsxwing Closes #3894 from zsxwing/SPARK-5083 and squashes the following commits: d97c03d [zsxwing] Fix a flaky test in TaskResultGetterSuite --- .../scheduler/TaskResultGetterSuite.scala | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 3aab5a156ee77..e3a3803e6483a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -19,7 +19,12 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.control.NonFatal + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually._ import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv} import org.apache.spark.storage.TaskResultBlockId @@ -34,6 +39,8 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule extends TaskResultGetter(sparkEnv, scheduler) { var removedResult = false + @volatile var removeBlockSuccessfully = false + override def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { if (!removedResult) { @@ -42,6 +49,15 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule serializer.get().deserialize[TaskResult[_]](serializedData) match { case IndirectTaskResult(blockId, size) => sparkEnv.blockManager.master.removeBlock(blockId) + // removeBlock is asynchronous. Need to wait it's removed successfully + try { + eventually(timeout(3 seconds), interval(200 milliseconds)) { + assert(!sparkEnv.blockManager.master.contains(blockId)) + } + removeBlockSuccessfully = true + } catch { + case NonFatal(e) => removeBlockSuccessfully = false + } case directResult: DirectTaskResult[_] => taskSetManager.abort("Internal error: expect only indirect results") } @@ -92,10 +108,12 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSpark assert(false, "Expect local cluster to use TaskSchedulerImpl") throw new ClassCastException } - scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) + val resultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) + scheduler.taskResultGetter = resultGetter val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + assert(resultGetter.removeBlockSuccessfully) assert(result === 1.to(akkaFrameSize).toArray) // Make sure two tasks were run (one failed one, and a second retried one). From 5c506cecb933b156b2f06a688ee08c4347bf0d47 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 4 Jan 2015 21:18:33 -0800 Subject: [PATCH 189/227] [SPARK-5074][Core] Fix a non-deterministic test failure Add `assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))` to make sure `sparkListener` receive the message. Author: zsxwing Closes #3889 from zsxwing/SPARK-5074 and squashes the following commits: e61c198 [zsxwing] Fix a non-deterministic test failure --- .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 1 + 1 file changed, 1 insertion(+) 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 d6ec9e129cceb..d30eb10bbe947 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -247,6 +247,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F 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(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.stageByOrderOfExecution.length === 2) assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) } From d3f07fd23cc26a70f44c52e24445974d4885d58a Mon Sep 17 00:00:00 2001 From: Varun Saxena Date: Mon, 5 Jan 2015 10:32:37 -0800 Subject: [PATCH 190/227] [SPARK-4688] Have a single shared network timeout in Spark [SPARK-4688] Have a single shared network timeout in Spark Author: Varun Saxena Author: varunsaxena Closes #3562 from varunsaxena/SPARK-4688 and squashes the following commits: 6e97f72 [Varun Saxena] [SPARK-4688] Single shared network timeout cd783a2 [Varun Saxena] SPARK-4688 d6f8c29 [Varun Saxena] SCALA-4688 9562b15 [Varun Saxena] SPARK-4688 a75f014 [varunsaxena] SPARK-4688 594226c [varunsaxena] SPARK-4688 --- .../apache/spark/network/nio/ConnectionManager.scala | 3 ++- .../apache/spark/storage/BlockManagerMasterActor.scala | 7 +++++-- .../main/scala/org/apache/spark/util/AkkaUtils.scala | 2 +- docs/configuration.md | 10 ++++++++++ .../org/apache/spark/network/util/TransportConf.java | 4 +++- 5 files changed, 21 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 243b71c980864..98455c0968263 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -81,7 +81,8 @@ private[nio] class ConnectionManager( private val ackTimeoutMonitor = new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) - private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) + private val ackTimeout = + conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 100)) // Get the thread counts from the Spark Configuration. // diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 9cbda41223a8b..9d77cf27882eb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -52,8 +52,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", - math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000)) + val slaveTimeout = { + val defaultMs = math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000) + val networkTimeout = conf.getInt("spark.network.timeout", defaultMs / 1000) + conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", networkTimeout * 1000) + } val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8c2457f56bffe..64e3a5416c6b5 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -65,7 +65,7 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", 100) + val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 100)) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" diff --git a/docs/configuration.md b/docs/configuration.md index 9bb6499993735..7ada67fc303c6 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -818,6 +818,16 @@ Apart from these, the following properties are also available, and may be useful Communication timeout between Spark nodes, in seconds. + + + + + diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 7c9adf52af0f0..e34382da22a50 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -37,7 +37,9 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; + int timeout = + conf.getInt("spark.shuffle.io.connectionTimeout", conf.getInt("spark.network.timeout", 100)); + return timeout * 1000; } /** Number of concurrent connections between two nodes for fetching data. */ From ce39b34404868de4ca51be06832169187b1aef7d Mon Sep 17 00:00:00 2001 From: WangTao Date: Mon, 5 Jan 2015 11:59:38 -0800 Subject: [PATCH 191/227] [SPARK-5057] Log message in failed askWithReply attempts https://issues.apache.org/jira/browse/SPARK-5057 Author: WangTao Author: WangTaoTheTonic Closes #3875 from WangTaoTheTonic/SPARK-5057 and squashes the following commits: 1503487 [WangTao] use string interpolation 706c8a7 [WangTaoTheTonic] log more messages --- .../scala/org/apache/spark/util/AkkaUtils.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 64e3a5416c6b5..8d86fd3e11ad7 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -89,7 +89,7 @@ private[spark] object AkkaUtils extends Logging { } val requireCookie = if (isAuthOn) "on" else "off" val secureCookie = if (isAuthOn) secretKey else "" - logDebug("In createActorSystem, requireCookie is: " + requireCookie) + logDebug(s"In createActorSystem, requireCookie is: $requireCookie") val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( ConfigFactory.parseString( @@ -140,8 +140,8 @@ private[spark] object AkkaUtils extends Logging { def maxFrameSizeBytes(conf: SparkConf): Int = { val frameSizeInMB = conf.getInt("spark.akka.frameSize", 10) if (frameSizeInMB > AKKA_MAX_FRAME_SIZE_IN_MB) { - throw new IllegalArgumentException("spark.akka.frameSize should not be greater than " - + AKKA_MAX_FRAME_SIZE_IN_MB + "MB") + throw new IllegalArgumentException( + s"spark.akka.frameSize should not be greater than $AKKA_MAX_FRAME_SIZE_IN_MB MB") } frameSizeInMB * 1024 * 1024 } @@ -182,8 +182,8 @@ private[spark] object AkkaUtils extends Logging { timeout: FiniteDuration): T = { // TODO: Consider removing multiple attempts if (actor == null) { - throw new SparkException("Error sending message as actor is null " + - "[message = " + message + "]") + throw new SparkException(s"Error sending message [message = $message]" + + " as actor is null ") } var attempts = 0 var lastException: Exception = null @@ -200,13 +200,13 @@ private[spark] object AkkaUtils extends Logging { case ie: InterruptedException => throw ie case e: Exception => lastException = e - logWarning("Error sending message in " + attempts + " attempts", e) + logWarning(s"Error sending message [message = $message] in $attempts attempts", e) } Thread.sleep(retryInterval) } throw new SparkException( - "Error sending message [message = " + message + "]", lastException) + s"Error sending message [message = $message]", lastException) } def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { From 1c0e7ce056c79e1db96f85b8c56a479b8b043970 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Mon, 5 Jan 2015 12:05:09 -0800 Subject: [PATCH 192/227] [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environme... ...nt at all. - fixed a scope of runAsSparkUser from MesosExecutorDriver.run to MesosExecutorBackend.launchTask - See the Jira Issue for more details. Author: Jongyoul Lee Closes #3741 from jongyoul/SPARK-4465 and squashes the following commits: 46ad71e [Jongyoul Lee] [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environment at all. - Removed unused import 3d6631f [Jongyoul Lee] [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environment at all. - Removed comments and adjusted indentations 2343f13 [Jongyoul Lee] [SPARK-4465] runAsSparkUser doesn't affect TaskRunner in Mesos environment at all. - fixed a scope of runAsSparkUser from MesosExecutorDriver.run to MesosExecutorBackend.launchTask --- .../spark/executor/MesosExecutorBackend.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index a098d07bd8659..2e23ae0a4f831 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.JavaConversions._ import org.apache.mesos.protobuf.ByteString -import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} +import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv} @@ -80,7 +80,9 @@ private[spark] class MesosExecutorBackend if (executor == null) { logError("Received launchTask but executor was null") } else { - executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer) + SparkHadoopUtil.get.runAsSparkUser { () => + executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer) + } } } @@ -112,11 +114,8 @@ private[spark] class MesosExecutorBackend private[spark] object MesosExecutorBackend extends Logging { def main(args: Array[String]) { SignalLogger.register(log) - SparkHadoopUtil.get.runAsSparkUser { () => - MesosNativeLibrary.load() - // Create a new Executor and start it running - val runner = new MesosExecutorBackend() - new MesosExecutorDriver(runner).run() - } + // Create a new Executor and start it running + val runner = new MesosExecutorBackend() + new MesosExecutorDriver(runner).run() } } From 6c6f32574023b8e43a24f2081ff17e6e446de2f3 Mon Sep 17 00:00:00 2001 From: freeman Date: Mon, 5 Jan 2015 13:10:59 -0800 Subject: [PATCH 193/227] [SPARK-5089][PYSPARK][MLLIB] Fix vector convert This is a small change addressing a potentially significant bug in how PySpark + MLlib handles non-float64 numpy arrays. The automatic conversion to `DenseVector` that occurs when passing RDDs to MLlib algorithms in PySpark should automatically upcast to float64s, but currently this wasn't actually happening. As a result, non-float64 would be silently parsed inappropriately during SerDe, yielding erroneous results when running, for example, KMeans. The PR includes the fix, as well as a new test for the correct conversion behavior. davies Author: freeman Closes #3902 from freeman-lab/fix-vector-convert and squashes the following commits: 764db47 [freeman] Add a test for proper conversion behavior 704f97e [freeman] Return array after changing type --- python/pyspark/mllib/linalg.py | 2 +- python/pyspark/mllib/tests.py | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index f7aa2b0cb04b3..4f8491f43e457 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -178,7 +178,7 @@ def __init__(self, ar): elif not isinstance(ar, np.ndarray): ar = np.array(ar, dtype=np.float64) if ar.dtype != np.float64: - ar.astype(np.float64) + ar = ar.astype(np.float64) self.array = ar def __reduce__(self): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 5034f229e824a..1f48bc1219dba 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -110,6 +110,16 @@ def test_squared_distance(self): self.assertEquals(0.0, _squared_distance(dv, dv)) self.assertEquals(0.0, _squared_distance(lst, lst)) + def test_conversion(self): + # numpy arrays should be automatically upcast to float64 + # tests for fix of [SPARK-5089] + v = array([1, 2, 3, 4], dtype='float64') + dv = DenseVector(v) + self.assertTrue(dv.array.dtype == 'float64') + v = array([1, 2, 3, 4], dtype='float32') + dv = DenseVector(v) + self.assertTrue(dv.array.dtype == 'float64') + class ListTests(PySparkTestCase): From bbcba3a9430365640c0188e7ca6e0677d3227dd8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 5 Jan 2015 15:19:53 -0800 Subject: [PATCH 194/227] [SPARK-5093] Set spark.network.timeout to 120s consistently. Author: Reynold Xin Closes #3903 from rxin/timeout-120 and squashes the following commits: 7c2138e [Reynold Xin] [SPARK-5093] Set spark.network.timeout to 120s consistently. --- .../org/apache/spark/network/nio/ConnectionManager.scala | 2 +- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 6 +----- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 2 +- docs/configuration.md | 6 +++--- .../java/org/apache/spark/network/util/TransportConf.java | 5 ++--- 5 files changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 98455c0968263..3340fca08014e 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,7 +82,7 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 100)) + conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 120)) // Get the thread counts from the Spark Configuration. // diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 9d77cf27882eb..64133464d8daa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -52,11 +52,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = { - val defaultMs = math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000) - val networkTimeout = conf.getInt("spark.network.timeout", defaultMs / 1000) - conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", networkTimeout * 1000) - } + val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120 * 1000) val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8d86fd3e11ad7..db2531dc171f8 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -65,7 +65,7 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 100)) + val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 120)) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" diff --git a/docs/configuration.md b/docs/configuration.md index 7ada67fc303c6..2add48569bece 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -820,12 +820,12 @@ Apart from these, the following properties are also available, and may be useful - + diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index e34382da22a50..6c9178688693f 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -37,9 +37,8 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - int timeout = - conf.getInt("spark.shuffle.io.connectionTimeout", conf.getInt("spark.network.timeout", 100)); - return timeout * 1000; + int defaultTimeout = conf.getInt("spark.network.timeout", 120); + return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000; } /** Number of concurrent connections between two nodes for fetching data. */ From 04d55d8e8e4890d110ce5561b5c1ae608c34a7c9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 5 Jan 2015 15:34:22 -0800 Subject: [PATCH 195/227] [SPARK-5040][SQL] Support expressing unresolved attributes using $"attribute name" notation in SQL DSL. Author: Reynold Xin Closes #3862 from rxin/stringcontext-attr and squashes the following commits: 9b10f57 [Reynold Xin] Rename StrongToAttributeConversionHelper 72121af [Reynold Xin] [SPARK-5040][SQL] Support expressing unresolved attributes using $"attribute name" notation in SQL DSL. --- .../org/apache/spark/sql/catalyst/dsl/package.scala | 9 +++++++++ .../scala/org/apache/spark/sql/DslQuerySuite.scala | 12 ++++++++++++ 2 files changed, 21 insertions(+) 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 index 8e39f79d2ca51..9608e15c0f302 100755 --- 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 @@ -135,6 +135,15 @@ package object dsl { implicit def symbolToUnresolvedAttribute(s: Symbol): analysis.UnresolvedAttribute = analysis.UnresolvedAttribute(s.name) + /** Converts $"col name" into an [[analysis.UnresolvedAttribute]]. */ + implicit class StringToAttributeConversionHelper(val sc: StringContext) { + // Note that if we make ExpressionConversions an object rather than a trait, we can + // then make this a value class to avoid the small penalty of runtime instantiation. + def $(args: Any*): analysis.UnresolvedAttribute = { + analysis.UnresolvedAttribute(sc.s(args :_*)) + } + } + def sum(e: Expression) = Sum(e) def sumDistinct(e: Expression) = SumDistinct(e) def count(e: Expression) = Count(e) 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 c0b9cf5163120..ab88f3ad10d66 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 @@ -56,6 +56,18 @@ class DslQuerySuite extends QueryTest { ) } + test("convert $\"attribute name\" into unresolved attribute") { + checkAnswer( + testData.where($"key" === 1).select($"value"), + Seq(Seq("1"))) + } + + test("convert Scala Symbol 'attrname into unresolved attribute") { + checkAnswer( + testData.where('key === 1).select('value), + Seq(Seq("1"))) + } + test("select *") { checkAnswer( testData.select(Star(None)), From 451546aa6d2e61e43b0c0f0669f18cfb7489e584 Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Mon, 5 Jan 2015 23:26:33 -0800 Subject: [PATCH 196/227] SPARK-4843 [YARN] Squash ExecutorRunnableUtil and ExecutorRunnable ExecutorRunnableUtil is a parent of ExecutorRunnable because of the yarn-alpha and yarn-stable split. Now that yarn-alpha is gone, this commit squashes the unnecessary hierarchy. The methods from ExecutorRunnableUtil are added as private. Author: Kostas Sakellis Closes #3696 from ksakellis/kostas-spark-4843 and squashes the following commits: 486716f [Kostas Sakellis] Moved prepareEnvironment call to after yarnConf declaration 470e22e [Kostas Sakellis] Fixed indentation and renamed sparkConf variable 9b1b1c9 [Kostas Sakellis] SPARK-4843 [YARN] Squash ExecutorRunnableUtil and ExecutorRunnable --- .../spark/deploy/yarn/ExecutorRunnable.scala | 182 +++++++++++++++- .../deploy/yarn/ExecutorRunnableUtil.scala | 203 ------------------ 2 files changed, 172 insertions(+), 213 deletions(-) delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index fdd3c2300fa78..6d9198c122e97 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -17,32 +17,33 @@ package org.apache.spark.deploy.yarn +import java.net.URI import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.spark.util.Utils import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils -import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.client.api.NMClient import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SecurityManager, SparkConf, Logging} import org.apache.spark.network.util.JavaUtils - class ExecutorRunnable( container: Container, conf: Configuration, - spConf: SparkConf, + sparkConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, @@ -50,13 +51,13 @@ class ExecutorRunnable( executorCores: Int, appId: String, securityMgr: SecurityManager) - extends Runnable with ExecutorRunnableUtil with Logging { + extends Runnable with Logging { var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ - val sparkConf = spConf val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - + lazy val env = prepareEnvironment + def run = { logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() @@ -110,4 +111,165 @@ class ExecutorRunnable( nmClient.startContainer(container, ctx) } + private def prepareCommand( + masterAddress: String, + slaveId: String, + hostname: String, + executorMemory: Int, + executorCores: Int, + appId: String, + localResources: HashMap[String, LocalResource]): List[String] = { + // Extra options for the JVM + val javaOpts = ListBuffer[String]() + + // Set the environment variable through a command prefix + // to append to the existing value of the variable + var prefixEnv: Option[String] = None + + // Set the JVM memory + val executorMemoryString = executorMemory + "m" + javaOpts += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " + + // Set extra Java options for the executor, if defined + sys.props.get("spark.executor.extraJavaOptions").foreach { opts => + javaOpts += opts + } + sys.env.get("SPARK_JAVA_OPTS").foreach { opts => + javaOpts += opts + } + sys.props.get("spark.executor.extraLibraryPath").foreach { p => + prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(p))) + } + + javaOpts += "-Djava.io.tmpdir=" + + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + + // Certain configs need to be passed here because they are needed before the Executor + // registers with the Scheduler and transfers the spark configs. Since the Executor backend + // uses Akka to connect to the scheduler, the akka settings are needed as well as the + // authentication settings. + sparkConf.getAll. + filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + + sparkConf.getAkkaConf. + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence + // if there are multiple containers in same node, spark gc effects all other containers + // performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. + /* + else { + // If no java_opts specified, default to using -XX:+CMSIncrementalMode + // It might be possible that other modes/config is being done in + // spark.executor.extraJavaOptions, so we dont want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tennent machines + // The options are based on + // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use + // %20the%20Concurrent%20Low%20Pause%20Collector|outline + javaOpts += " -XX:+UseConcMarkSweepGC " + javaOpts += " -XX:+CMSIncrementalMode " + javaOpts += " -XX:+CMSIncrementalPacing " + javaOpts += " -XX:CMSIncrementalDutyCycleMin=0 " + javaOpts += " -XX:CMSIncrementalDutyCycle=10 " + } + */ + + // For log4j configuration to reference + javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) + + val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", + "-server", + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. + // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in + // an inconsistent state. + // TODO: If the OOM is not recoverable by rescheduling it on different node, then do + // 'something' to fail job ... akin to blacklisting trackers in mapred ? + "-XX:OnOutOfMemoryError='kill %p'") ++ + javaOpts ++ + Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", + masterAddress.toString, + slaveId.toString, + hostname.toString, + executorCores.toString, + appId, + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + // TODO: it would be nicer to just make sure there are no null commands here + commands.map(s => if (s == null) "null" else s).toList + } + + private def setupDistributedCache( + file: String, + rtype: LocalResourceType, + localResources: HashMap[String, LocalResource], + timestamp: String, + size: String, + vis: String): Unit = { + val uri = new URI(file) + val amJarRsrc = Records.newRecord(classOf[LocalResource]) + amJarRsrc.setType(rtype) + amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) + amJarRsrc.setTimestamp(timestamp.toLong) + amJarRsrc.setSize(size.toLong) + localResources(uri.getFragment()) = amJarRsrc + } + + private def prepareLocalResources: HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + val localResources = HashMap[String, LocalResource]() + + if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') + val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') + for( i <- 0 to distFiles.length - 1) { + setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), + fileSizes(i), visibilities(i)) + } + } + + if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') + val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') + for( i <- 0 to distArchives.length - 1) { + setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, + timeStamps(i), fileSizes(i), visibilities(i)) + } + } + + logInfo("Prepared Local resources " + localResources) + localResources + } + + private def prepareEnvironment: HashMap[String, String] = { + val env = new HashMap[String, String]() + val extraCp = sparkConf.getOption("spark.executor.extraClassPath") + ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) + + sparkConf.getExecutorEnv.foreach { case (key, value) => + // This assumes each executor environment variable set here is a path + // This is kept for backward compatibility and consistency with hadoop + YarnSparkHadoopUtil.addPathToEnvironment(env, key, value) + } + + // Keep this for backwards compatibility but users should move to the config + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) + } + + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } + env + } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala deleted file mode 100644 index 22d73ecf6d010..0000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ /dev/null @@ -1,203 +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.deploy.yarn - -import java.net.URI - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, ListBuffer} - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils - -trait ExecutorRunnableUtil extends Logging { - - val yarnConf: YarnConfiguration - val sparkConf: SparkConf - lazy val env = prepareEnvironment - - def prepareCommand( - masterAddress: String, - slaveId: String, - hostname: String, - executorMemory: Int, - executorCores: Int, - appId: String, - localResources: HashMap[String, LocalResource]): List[String] = { - // Extra options for the JVM - val javaOpts = ListBuffer[String]() - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - var prefixEnv: Option[String] = None - - // Set the JVM memory - val executorMemoryString = executorMemory + "m" - javaOpts += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - - // Set extra Java options for the executor, if defined - sys.props.get("spark.executor.extraJavaOptions").foreach { opts => - javaOpts += opts - } - sys.env.get("SPARK_JAVA_OPTS").foreach { opts => - javaOpts += opts - } - sys.props.get("spark.executor.extraLibraryPath").foreach { p => - prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(p))) - } - - javaOpts += "-Djava.io.tmpdir=" + - new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - - // Certain configs need to be passed here because they are needed before the Executor - // registers with the Scheduler and transfers the spark configs. Since the Executor backend - // uses Akka to connect to the scheduler, the akka settings are needed as well as the - // authentication settings. - sparkConf.getAll. - filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. - foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - - sparkConf.getAkkaConf. - foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - - // Commenting it out for now - so that people can refer to the properties if required. Remove - // it once cpuset version is pushed out. - // The context is, default gc for server class machines end up using all cores to do gc - hence - // if there are multiple containers in same node, spark gc effects all other containers - // performance (which can also be other spark containers) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in - // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset - // of cores on a node. - /* - else { - // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in - // spark.executor.extraJavaOptions, so we dont want to mess with it. - // In our expts, using (default) throughput collector has severe perf ramnifications in - // multi-tennent machines - // The options are based on - // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use - // %20the%20Concurrent%20Low%20Pause%20Collector|outline - javaOpts += " -XX:+UseConcMarkSweepGC " - javaOpts += " -XX:+CMSIncrementalMode " - javaOpts += " -XX:+CMSIncrementalPacing " - javaOpts += " -XX:CMSIncrementalDutyCycleMin=0 " - javaOpts += " -XX:CMSIncrementalDutyCycle=10 " - } - */ - - // For log4j configuration to reference - javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) - - val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", - "-server", - // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in - // an inconsistent state. - // TODO: If the OOM is not recoverable by rescheduling it on different node, then do - // 'something' to fail job ... akin to blacklisting trackers in mapred ? - "-XX:OnOutOfMemoryError='kill %p'") ++ - javaOpts ++ - Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", - masterAddress.toString, - slaveId.toString, - hostname.toString, - executorCores.toString, - appId, - "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", - "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - // TODO: it would be nicer to just make sure there are no null commands here - commands.map(s => if (s == null) "null" else s).toList - } - - private def setupDistributedCache( - file: String, - rtype: LocalResourceType, - localResources: HashMap[String, LocalResource], - timestamp: String, - size: String, - vis: String): Unit = { - val uri = new URI(file) - val amJarRsrc = Records.newRecord(classOf[LocalResource]) - amJarRsrc.setType(rtype) - amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) - amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) - amJarRsrc.setTimestamp(timestamp.toLong) - amJarRsrc.setSize(size.toLong) - localResources(uri.getFragment()) = amJarRsrc - } - - def prepareLocalResources: HashMap[String, LocalResource] = { - logInfo("Preparing Local resources") - val localResources = HashMap[String, LocalResource]() - - if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') - val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') - for( i <- 0 to distFiles.length - 1) { - setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), - fileSizes(i), visibilities(i)) - } - } - - if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') - val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') - for( i <- 0 to distArchives.length - 1) { - setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, - timeStamps(i), fileSizes(i), visibilities(i)) - } - } - - logInfo("Prepared Local resources " + localResources) - localResources - } - - def prepareEnvironment: HashMap[String, String] = { - val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) - - sparkConf.getExecutorEnv.foreach { case (key, value) => - // This assumes each executor environment variable set here is a path - // This is kept for backward compatibility and consistency with hadoop - YarnSparkHadoopUtil.addPathToEnvironment(env, key, value) - } - - // Keep this for backwards compatibility but users should move to the config - sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => - YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) - } - - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } - env - } - -} From a6394bc2c094c6c662237236c2effa2dabe67910 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 6 Jan 2015 00:31:19 -0800 Subject: [PATCH 197/227] [SPARK-1600] Refactor FileInputStream tests to remove Thread.sleep() calls and SystemClock usage This patch refactors Spark Streaming's FileInputStream tests to remove uses of Thread.sleep() and SystemClock, which should hopefully resolve some longstanding flakiness in these tests (see SPARK-1600). Key changes: - Modify FileInputDStream to use the scheduler's Clock instead of System.currentTimeMillis(); this allows it to be tested using ManualClock. - Fix a synchronization issue in ManualClock's `currentTime` method. - Add a StreamingTestWaiter class which allows callers to block until a certain number of batches have finished. - Change the FileInputStream tests so that files' modification times are manually set based off of ManualClock; this eliminates many Thread.sleep calls. - Update these tests to use the withStreamingContext fixture. Author: Josh Rosen Closes #3801 from JoshRosen/SPARK-1600 and squashes the following commits: e4494f4 [Josh Rosen] Address a potential race when setting file modification times 8340bd0 [Josh Rosen] Use set comparisons for output. 0b9c252 [Josh Rosen] Fix some ManualClock usage problems. 1cc689f [Josh Rosen] ConcurrentHashMap -> SynchronizedMap db26c3a [Josh Rosen] Use standard timeout in ScalaTest `eventually` blocks. 3939432 [Josh Rosen] Rename StreamingTestWaiter to BatchCounter 0b9c3a1 [Josh Rosen] Wait for checkpoint to complete 863d71a [Josh Rosen] Remove Thread.sleep that was used to make task run slowly b4442c3 [Josh Rosen] batchTimeToSelectedFiles should be thread-safe 15b48ee [Josh Rosen] Replace several TestWaiter methods w/ ScalaTest eventually. fffc51c [Josh Rosen] Revert "Remove last remaining sleep() call" dbb8247 [Josh Rosen] Remove last remaining sleep() call 566a63f [Josh Rosen] Fix log message and comment typos da32f3f [Josh Rosen] Fix log message and comment typos 3689214 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-1600 c8f06b1 [Josh Rosen] Remove Thread.sleep calls in FileInputStream CheckpointSuite test. d4f2d87 [Josh Rosen] Refactor file input stream tests to not rely on SystemClock. dda1403 [Josh Rosen] Add StreamingTestWaiter class. 3c3efc3 [Josh Rosen] Synchronize `currentTime` in ManualClock a95ddc4 [Josh Rosen] Modify FileInputDStream to use Clock class. --- .../streaming/dstream/FileInputDStream.scala | 16 +- .../apache/spark/streaming/util/Clock.scala | 6 +- .../streaming/BasicOperationsSuite.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 248 +++++++++++------- .../spark/streaming/InputStreamsSuite.scala | 69 +++-- .../spark/streaming/TestSuiteBase.scala | 46 +++- 6 files changed, 251 insertions(+), 136 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 5f13fdc5579ed..e7c5639a63499 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.dstream import java.io.{IOException, ObjectInputStream} +import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable import scala.reflect.ClassTag @@ -74,12 +75,15 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas newFilesOnly: Boolean = true) extends InputDStream[(K, V)](ssc_) { + // This is a def so that it works during checkpoint recovery: + private def clock = ssc.scheduler.clock + // Data to be saved as part of the streaming checkpoints protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData // Initial ignore threshold based on which old, existing files in the directory (at the time of // starting the streaming application) will be ignored or considered - private val initialModTimeIgnoreThreshold = if (newFilesOnly) System.currentTimeMillis() else 0L + private val initialModTimeIgnoreThreshold = if (newFilesOnly) clock.currentTime() else 0L /* * Make sure that the information of files selected in the last few batches are remembered. @@ -91,8 +95,9 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas remember(durationToRemember) // Map of batch-time to selected file info for the remembered batches + // This is a concurrent map because it's also accessed in unit tests @transient private[streaming] var batchTimeToSelectedFiles = - new mutable.HashMap[Time, Array[String]] + new mutable.HashMap[Time, Array[String]] with mutable.SynchronizedMap[Time, Array[String]] // Set of files that were selected in the remembered batches @transient private var recentlySelectedFiles = new mutable.HashSet[String]() @@ -151,7 +156,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas */ private def findNewFiles(currentTime: Long): Array[String] = { try { - lastNewFileFindingTime = System.currentTimeMillis + lastNewFileFindingTime = clock.currentTime() // Calculate ignore threshold val modTimeIgnoreThreshold = math.max( @@ -164,7 +169,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold) } val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) - val timeTaken = System.currentTimeMillis - lastNewFileFindingTime + val timeTaken = clock.currentTime() - lastNewFileFindingTime logInfo("Finding new files took " + timeTaken + " ms") logDebug("# cached file times = " + fileToModTime.size) if (timeTaken > slideDuration.milliseconds) { @@ -267,7 +272,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() generatedRDDs = new mutable.HashMap[Time, RDD[(K,V)]] () - batchTimeToSelectedFiles = new mutable.HashMap[Time, Array[String]]() + batchTimeToSelectedFiles = + new mutable.HashMap[Time, Array[String]] with mutable.SynchronizedMap[Time, Array[String]] recentlySelectedFiles = new mutable.HashSet[String]() fileToModTime = new TimeStampedHashMap[String, Long](true) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index 7cd867ce34b87..d6d96d7ba00fd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -59,9 +59,11 @@ class SystemClock() extends Clock { private[streaming] class ManualClock() extends Clock { - var time = 0L + private var time = 0L - def currentTime() = time + def currentTime() = this.synchronized { + time + } def setTime(timeToSet: Long) = { this.synchronized { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 199f5e7161124..e8f4a7779ec21 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -638,7 +638,7 @@ class BasicOperationsSuite extends TestSuiteBase { if (rememberDuration != null) ssc.remember(rememberDuration) val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput) val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - assert(clock.time === Seconds(10).milliseconds) + assert(clock.currentTime() === Seconds(10).milliseconds) assert(output.size === numExpectedOutput) operatedStream } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 5d232c6ade7a9..8f8bc61437ba5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -18,17 +18,18 @@ package org.apache.spark.streaming import java.io.File -import java.nio.charset.Charset -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag +import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{IntWritable, Text} import org.apache.hadoop.mapred.TextOutputFormat import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.scalatest.concurrent.Eventually._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} import org.apache.spark.streaming.util.ManualClock @@ -45,8 +46,6 @@ class CheckpointSuite extends TestSuiteBase { override def batchDuration = Milliseconds(500) - override def actuallyWait = true // to allow checkpoints to be written - override def beforeFunction() { super.beforeFunction() Utils.deleteRecursively(new File(checkpointDir)) @@ -143,7 +142,6 @@ class CheckpointSuite extends TestSuiteBase { ssc.start() advanceTimeWithRealDelay(ssc, 4) ssc.stop() - System.clearProperty("spark.streaming.manualClock.jump") ssc = null } @@ -312,109 +310,161 @@ class CheckpointSuite extends TestSuiteBase { testCheckpointedOperation(input, operation, output, 7) } - // This tests whether file input stream remembers what files were seen before // the master failure and uses them again to process a large window operation. // It also tests whether batches, whose processing was incomplete due to the // failure, are re-processed or not. test("recovery with file input stream") { // Set up the streaming context and input streams + val batchDuration = Seconds(2) // Due to 1-second resolution of setLastModified() on some OS's. val testDir = Utils.createTempDir() - var ssc = new StreamingContext(master, framework, Seconds(1)) - ssc.checkpoint(checkpointDir) - val fileStream = ssc.textFileStream(testDir.toString) - // Making value 3 take large time to process, to ensure that the master - // shuts down in the middle of processing the 3rd batch - val mappedStream = fileStream.map(s => { - val i = s.toInt - if (i == 3) Thread.sleep(2000) - i - }) - - // Reducing over a large window to ensure that recovery from master failure - // requires reprocessing of all the files seen before the failure - val reducedStream = mappedStream.reduceByWindow(_ + _, Seconds(30), Seconds(1)) - val outputBuffer = new ArrayBuffer[Seq[Int]] - var outputStream = new TestOutputStream(reducedStream, outputBuffer) - outputStream.register() - ssc.start() - - // Create files and advance manual clock to process them - // var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - Thread.sleep(1000) - for (i <- Seq(1, 2, 3)) { - Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) - // wait to make sure that the file is written such that it gets shown in the file listings - Thread.sleep(1000) + val outputBuffer = new ArrayBuffer[Seq[Int]] with SynchronizedBuffer[Seq[Int]] + + /** + * Writes a file named `i` (which contains the number `i`) to the test directory and sets its + * modification time to `clock`'s current time. + */ + def writeFile(i: Int, clock: ManualClock): Unit = { + val file = new File(testDir, i.toString) + Files.write(i + "\n", file, Charsets.UTF_8) + assert(file.setLastModified(clock.currentTime())) + // Check that the file's modification date is actually the value we wrote, since rounding or + // truncation will break the test: + assert(file.lastModified() === clock.currentTime()) } - logInfo("Output = " + outputStream.output.mkString(",")) - assert(outputStream.output.size > 0, "No files processed before restart") - ssc.stop() - // Verify whether files created have been recorded correctly or not - var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - def recordedFiles = fileInputDStream.batchTimeToSelectedFiles.values.flatten - assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) - - // Create files while the master is down - for (i <- Seq(4, 5, 6)) { - Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) - Thread.sleep(1000) + /** + * Returns ids that identify which files which have been recorded by the file input stream. + */ + def recordedFiles(ssc: StreamingContext): Seq[Int] = { + val fileInputDStream = + ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] + val filenames = fileInputDStream.batchTimeToSelectedFiles.values.flatten + filenames.map(_.split(File.separator).last.toInt).toSeq.sorted } - // Recover context from checkpoint file and verify whether the files that were - // recorded before failure were saved and successfully recovered - logInfo("*********** RESTARTING ************") - ssc = new StreamingContext(checkpointDir) - fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) + try { + // This is a var because it's re-assigned when we restart from a checkpoint + var clock: ManualClock = null + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + ssc.checkpoint(checkpointDir) + clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val batchCounter = new BatchCounter(ssc) + val fileStream = ssc.textFileStream(testDir.toString) + // Make value 3 take a large time to process, to ensure that the driver + // shuts down in the middle of processing the 3rd batch + CheckpointSuite.batchThreeShouldBlockIndefinitely = true + val mappedStream = fileStream.map(s => { + val i = s.toInt + if (i == 3) { + while (CheckpointSuite.batchThreeShouldBlockIndefinitely) { + Thread.sleep(Long.MaxValue) + } + } + i + }) + + // Reducing over a large window to ensure that recovery from driver failure + // requires reprocessing of all the files seen before the failure + val reducedStream = mappedStream.reduceByWindow(_ + _, batchDuration * 30, batchDuration) + val outputStream = new TestOutputStream(reducedStream, outputBuffer) + outputStream.register() + ssc.start() + + // Advance half a batch so that the first file is created after the StreamingContext starts + clock.addToTime(batchDuration.milliseconds / 2) + // Create files and advance manual clock to process them + for (i <- Seq(1, 2, 3)) { + writeFile(i, clock) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + if (i != 3) { + // Since we want to shut down while the 3rd batch is processing + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === i) + } + } + } + clock.addToTime(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + // Wait until all files have been recorded and all batches have started + assert(recordedFiles(ssc) === Seq(1, 2, 3) && batchCounter.getNumStartedBatches === 3) + } + // Wait for a checkpoint to be written + val fs = new Path(checkpointDir).getFileSystem(ssc.sc.hadoopConfiguration) + eventually(eventuallyTimeout) { + assert(Checkpoint.getCheckpointFiles(checkpointDir, fs).size === 6) + } + ssc.stop() + // Check that we shut down while the third batch was being processed + assert(batchCounter.getNumCompletedBatches === 2) + assert(outputStream.output.flatten === Seq(1, 3)) + } - // Restart stream computation - ssc.start() - for (i <- Seq(7, 8, 9)) { - Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) - Thread.sleep(1000) - } - Thread.sleep(1000) - logInfo("Output = " + outputStream.output.mkString("[", ", ", "]")) - assert(outputStream.output.size > 0, "No files processed after restart") - ssc.stop() + // The original StreamingContext has now been stopped. + CheckpointSuite.batchThreeShouldBlockIndefinitely = false - // Verify whether files created while the driver was down have been recorded or not - assert(!recordedFiles.filter(_.endsWith("4")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("5")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("6")).isEmpty) - - // Verify whether new files created after recover have been recorded or not - assert(!recordedFiles.filter(_.endsWith("7")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("8")).isEmpty) - assert(!recordedFiles.filter(_.endsWith("9")).isEmpty) - - // Append the new output to the old buffer - outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] - outputBuffer ++= outputStream.output - - val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45) - logInfo("--------------------------------") - logInfo("output, size = " + outputBuffer.size) - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output, size = " + expectedOutput.size) - expectedOutput.foreach(x => logInfo("[" + x + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - val output = outputBuffer.flatMap(x => x) - assert(output.contains(6)) // To ensure that the 3rd input (i.e., 3) was processed - output.foreach(o => // To ensure all the inputs are correctly added cumulatively - assert(expectedOutput.contains(o), "Expected value " + o + " not found") - ) - // To ensure that all the inputs were received correctly - assert(expectedOutput.last === output.last) - Utils.deleteRecursively(testDir) + // Create files while the streaming driver is down + for (i <- Seq(4, 5, 6)) { + writeFile(i, clock) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + } + + // Recover context from checkpoint file and verify whether the files that were + // recorded before failure were saved and successfully recovered + logInfo("*********** RESTARTING ************") + withStreamingContext(new StreamingContext(checkpointDir)) { ssc => + // So that the restarted StreamingContext's clock has gone forward in time since failure + ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration * 3).milliseconds.toString) + val oldClockTime = clock.currentTime() + clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val batchCounter = new BatchCounter(ssc) + val outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] + // Check that we remember files that were recorded before the restart + assert(recordedFiles(ssc) === Seq(1, 2, 3)) + + // Restart stream computation + ssc.start() + // Verify that the clock has traveled forward to the expected time + eventually(eventuallyTimeout) { + clock.currentTime() === oldClockTime + } + // Wait for pre-failure batch to be recomputed (3 while SSC was down plus last batch) + val numBatchesAfterRestart = 4 + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === numBatchesAfterRestart) + } + for ((i, index) <- Seq(7, 8, 9).zipWithIndex) { + writeFile(i, clock) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === index + numBatchesAfterRestart + 1) + } + } + clock.addToTime(batchDuration.milliseconds) + logInfo("Output after restart = " + outputStream.output.mkString("[", ", ", "]")) + assert(outputStream.output.size > 0, "No files processed after restart") + ssc.stop() + + // Verify whether files created while the driver was down (4, 5, 6) and files created after + // recovery (7, 8, 9) have been recorded + assert(recordedFiles(ssc) === (1 to 9)) + + // Append the new output to the old buffer + outputBuffer ++= outputStream.output + + // Verify whether all the elements received are as expected + val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45) + assert(outputBuffer.flatten.toSet === expectedOutput.toSet) + } + } finally { + Utils.deleteRecursively(testDir) + } } @@ -471,12 +521,12 @@ class CheckpointSuite extends TestSuiteBase { */ def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - logInfo("Manual clock before advancing = " + clock.time) + logInfo("Manual clock before advancing = " + clock.currentTime()) for (i <- 1 to numBatches.toInt) { clock.addToTime(batchDuration.milliseconds) Thread.sleep(batchDuration.milliseconds) } - logInfo("Manual clock after advancing = " + clock.time) + logInfo("Manual clock after advancing = " + clock.currentTime()) Thread.sleep(batchDuration.milliseconds) val outputStream = ssc.graph.getOutputStreams.filter { dstream => @@ -485,3 +535,7 @@ class CheckpointSuite extends TestSuiteBase { outputStream.output.map(_.flatten) } } + +private object CheckpointSuite extends Serializable { + var batchThreeShouldBlockIndefinitely: Boolean = true +} \ No newline at end of file diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 307052a4a9cbb..bddf51e130422 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -28,7 +28,6 @@ import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer, SynchronizedQueue} -import scala.concurrent.duration._ import scala.language.postfixOps import com.google.common.io.Files @@ -234,45 +233,57 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } def testFileStream(newFilesOnly: Boolean) { - var ssc: StreamingContext = null val testDir: File = null try { + val batchDuration = Seconds(2) val testDir = Utils.createTempDir() + // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") Files.write("0\n", existingFile, Charset.forName("UTF-8")) + assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) - Thread.sleep(1000) // Set up the streaming context and input streams - val newConf = conf.clone.set( - "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") - ssc = new StreamingContext(newConf, batchDuration) - val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( - testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(fileStream, outputBuffer) - outputStream.register() - ssc.start() - - // Create files in the directory - val input = Seq(1, 2, 3, 4, 5) - input.foreach { i => - Thread.sleep(batchDuration.milliseconds) - val file = new File(testDir, i.toString) - Files.write(i + "\n", file, Charset.forName("UTF-8")) - logInfo("Created file " + file) - } + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + // This `setTime` call ensures that the clock is past the creation time of `existingFile` + clock.setTime(existingFile.lastModified + batchDuration.milliseconds) + val batchCounter = new BatchCounter(ssc) + val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( + testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputBuffer) + outputStream.register() + ssc.start() + + // Advance the clock so that the files are created after StreamingContext starts, but + // not enough to trigger a batch + clock.addToTime(batchDuration.milliseconds / 2) + + // Over time, create files in the directory + val input = Seq(1, 2, 3, 4, 5) + input.foreach { i => + val file = new File(testDir, i.toString) + Files.write(i + "\n", file, Charset.forName("UTF-8")) + assert(file.setLastModified(clock.currentTime())) + assert(file.lastModified === clock.currentTime) + logInfo("Created file " + file) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === i) + } + } - // Verify that all the files have been read - val expectedOutput = if (newFilesOnly) { - input.map(_.toString).toSet - } else { - (Seq(0) ++ input).map(_.toString).toSet - } - eventually(timeout(maxWaitTimeMillis milliseconds), interval(100 milliseconds)) { + // Verify that all the files have been read + val expectedOutput = if (newFilesOnly) { + input.map(_.toString).toSet + } else { + (Seq(0) ++ input).map(_.toString).toSet + } assert(outputBuffer.flatten.toSet === expectedOutput) } } finally { - if (ssc != null) ssc.stop() if (testDir != null) Utils.deleteRecursively(testDir) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 52972f63c6c5c..7d82c3e4aadcf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -21,11 +21,16 @@ import java.io.{ObjectInputStream, IOException} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.SynchronizedBuffer +import scala.language.implicitConversions import scala.reflect.ClassTag import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.time.{Span, Seconds => ScalaTestSeconds} +import org.scalatest.concurrent.Eventually.timeout +import org.scalatest.concurrent.PatienceConfiguration import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} +import org.apache.spark.streaming.scheduler.{StreamingListenerBatchStarted, StreamingListenerBatchCompleted, StreamingListener} import org.apache.spark.streaming.util.ManualClock import org.apache.spark.{SparkConf, Logging} import org.apache.spark.rdd.RDD @@ -103,6 +108,40 @@ class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], def toTestOutputStream = new TestOutputStream[T](this.parent, this.output.map(_.flatten)) } +/** + * An object that counts the number of started / completed batches. This is implemented using a + * StreamingListener. Constructing a new instance automatically registers a StreamingListener on + * the given StreamingContext. + */ +class BatchCounter(ssc: StreamingContext) { + + // All access to this state should be guarded by `BatchCounter.this.synchronized` + private var numCompletedBatches = 0 + private var numStartedBatches = 0 + + private val listener = new StreamingListener { + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = + BatchCounter.this.synchronized { + numStartedBatches += 1 + BatchCounter.this.notifyAll() + } + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = + BatchCounter.this.synchronized { + numCompletedBatches += 1 + BatchCounter.this.notifyAll() + } + } + ssc.addStreamingListener(listener) + + def getNumCompletedBatches: Int = this.synchronized { + numCompletedBatches + } + + def getNumStartedBatches: Int = this.synchronized { + numStartedBatches + } +} + /** * This is the base trait for Spark Streaming testsuites. This provides basic functionality * to run user-defined set of input on user-defined stream operations, and verify the output. @@ -142,6 +181,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { .setMaster(master) .setAppName(framework) + // Timeout for use in ScalaTest `eventually` blocks + val eventuallyTimeout: PatienceConfiguration.Timeout = timeout(Span(10, ScalaTestSeconds)) + // Default before function for any streaming test suite. Override this // if you want to add your stuff to "before" (i.e., don't call before { } ) def beforeFunction() { @@ -291,7 +333,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Advance manual clock val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - logInfo("Manual clock before advancing = " + clock.time) + logInfo("Manual clock before advancing = " + clock.currentTime()) if (actuallyWait) { for (i <- 1 to numBatches) { logInfo("Actually waiting for " + batchDuration) @@ -301,7 +343,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { } else { clock.addToTime(numBatches * batchDuration.milliseconds) } - logInfo("Manual clock after advancing = " + clock.time) + logInfo("Manual clock after advancing = " + clock.currentTime()) // Wait until expected number of output items have been generated val startTime = System.currentTimeMillis() From 5e3ec1110495899a298313c4aa9c6c151c1f54da Mon Sep 17 00:00:00 2001 From: kj-ki Date: Tue, 6 Jan 2015 09:49:37 -0800 Subject: [PATCH 198/227] [Minor] Fix comments for GraphX 2D partitioning strategy The sum of vertices on matrix (v0 to v11) is 12. And, I think one same block overlaps in this strategy. This is minor PR, so I didn't file in JIRA. Author: kj-ki Closes #3904 from kj-ki/fix-partitionstrategy-comments and squashes the following commits: 79829d9 [kj-ki] Fix comments for 2D partitioning. --- .../scala/org/apache/spark/graphx/PartitionStrategy.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 13033fee0e6b5..7372dfbd9fe98 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -32,9 +32,9 @@ trait PartitionStrategy extends Serializable { object PartitionStrategy { /** * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix, - * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. + * guaranteeing a `2 * sqrt(numParts) - 1` bound on vertex replication. * - * Suppose we have a graph with 11 vertices that we want to partition + * Suppose we have a graph with 12 vertices that we want to partition * over 9 machines. We can use the following sparse matrix representation: * *
    @@ -61,7 +61,7 @@ object PartitionStrategy {
        * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3,
        * P6)` or the last
        * row of blocks `(P6, P7, P8)`.  As a consequence we can guarantee that `v11` will need to be
    -   * replicated to at most `2 * sqrt(numParts)` machines.
    +   * replicated to at most `2 * sqrt(numParts) - 1` machines.
        *
        * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work
        * balance.  To improve balance we first multiply each vertex id by a large prime to shuffle the
    
    From 4cba6eb42031b1a4cc3308833116ca5d9ccb1a89 Mon Sep 17 00:00:00 2001
    From: Sean Owen 
    Date: Tue, 6 Jan 2015 12:02:08 -0800
    Subject: [PATCH 199/227] SPARK-4159 [CORE] Maven build doesn't run JUnit test
     suites
    
    This PR:
    
    - Reenables `surefire`, and copies config from `scalatest` (which is itself an old fork of `surefire`, so similar)
    - Tells `surefire` to test only Java tests
    - Enables `surefire` and `scalatest` for all children, and in turn eliminates some duplication.
    
    For me this causes the Scala and Java tests to be run once each, it seems, as desired. It doesn't affect the SBT build but works for Maven. I still need to verify that all of the Scala tests and Java tests are being run.
    
    Author: Sean Owen 
    
    Closes #3651 from srowen/SPARK-4159 and squashes the following commits:
    
    2e8a0af [Sean Owen] Remove specialized SPARK_HOME setting for REPL, YARN tests as it appears to be obsolete
    12e4558 [Sean Owen] Append to unit-test.log instead of overwriting, so that both surefire and scalatest output is preserved. Also standardize/correct comments a bit.
    e6f8601 [Sean Owen] Reenable Java tests by reenabling surefire with config cloned from scalatest; centralize test config in the parent
    ---
     bagel/pom.xml                                 | 11 -----
     bagel/src/test/resources/log4j.properties     |  4 +-
     core/pom.xml                                  | 18 --------
     core/src/test/resources/log4j.properties      |  4 +-
     examples/pom.xml                              |  5 ---
     external/flume-sink/pom.xml                   |  9 ----
     .../src/test/resources/log4j.properties       |  3 +-
     external/flume/pom.xml                        | 11 -----
     .../flume/src/test/resources/log4j.properties |  5 +--
     external/kafka/pom.xml                        | 11 -----
     .../kafka/src/test/resources/log4j.properties |  5 +--
     external/mqtt/pom.xml                         | 11 -----
     .../mqtt/src/test/resources/log4j.properties  |  5 +--
     external/twitter/pom.xml                      | 11 -----
     .../src/test/resources/log4j.properties       |  5 +--
     external/zeromq/pom.xml                       | 11 -----
     .../src/test/resources/log4j.properties       |  5 +--
     extras/java8-tests/pom.xml                    | 15 -------
     .../src/test/resources/log4j.properties       |  2 +-
     extras/kinesis-asl/pom.xml                    | 11 -----
     .../src/test/resources/log4j.properties       |  5 ++-
     graphx/pom.xml                                | 11 -----
     graphx/src/test/resources/log4j.properties    |  4 +-
     mllib/pom.xml                                 | 11 -----
     mllib/src/test/resources/log4j.properties     |  4 +-
     network/common/pom.xml                        |  5 ---
     network/shuffle/pom.xml                       |  5 ---
     pom.xml                                       | 45 +++++++++++++++++--
     repl/pom.xml                                  | 14 ------
     repl/src/test/resources/log4j.properties      |  4 +-
     sql/catalyst/pom.xml                          | 10 -----
     sql/core/pom.xml                              | 11 -----
     sql/hive-thriftserver/pom.xml                 |  9 ----
     sql/hive/pom.xml                              |  5 ---
     streaming/pom.xml                             | 10 -----
     streaming/src/test/resources/log4j.properties |  5 +--
     tools/pom.xml                                 |  9 ----
     yarn/pom.xml                                  | 14 ------
     yarn/src/test/resources/log4j.properties      |  4 +-
     39 files changed, 70 insertions(+), 277 deletions(-)
    
    diff --git a/bagel/pom.xml b/bagel/pom.xml
    index 0327ffa402671..3bcd38fa3245c 100644
    --- a/bagel/pom.xml
    +++ b/bagel/pom.xml
    @@ -44,11 +44,6 @@
           org.eclipse.jetty
           jetty-server
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -58,11 +53,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties
    index 789869f72e3b0..853ef0ed2986f 100644
    --- a/bagel/src/test/resources/log4j.properties
    +++ b/bagel/src/test/resources/log4j.properties
    @@ -15,10 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file bagel/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/core/pom.xml b/core/pom.xml
    index c5c41b2b5de42..d9a49c9e08afc 100644
    --- a/core/pom.xml
    +++ b/core/pom.xml
    @@ -276,11 +276,6 @@
           selenium-java
           test
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.mockito
           mockito-all
    @@ -326,19 +321,6 @@
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
         
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -        
    -          
    -            test
    -            
    -              test
    -            
    -          
    -        
    -      
    -
           
           
             org.apache.maven.plugins
    diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties
    index 9dd05f17f012b..287c8e3563503 100644
    --- a/core/src/test/resources/log4j.properties
    +++ b/core/src/test/resources/log4j.properties
    @@ -15,10 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file core/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/examples/pom.xml b/examples/pom.xml
    index 8713230e1e8ed..bdc5d0562f3e1 100644
    --- a/examples/pom.xml
    +++ b/examples/pom.xml
    @@ -244,11 +244,6 @@
           algebird-core_${scala.binary.version}
           0.8.1
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml
    index 72618b6515f83..71f595d0a6800 100644
    --- a/external/flume-sink/pom.xml
    +++ b/external/flume-sink/pom.xml
    @@ -65,11 +65,6 @@
             
           
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scala-lang
           scala-library
    @@ -91,10 +86,6 @@
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
         
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
           
             org.apache.avro
             avro-maven-plugin
    diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties
    index 4411d6e20c52a..2a58e99817224 100644
    --- a/external/flume-sink/src/test/resources/log4j.properties
    +++ b/external/flume-sink/src/test/resources/log4j.properties
    @@ -17,9 +17,8 @@
     
     # Set everything to be logged to the file streaming/target/unit-tests.log
     log4j.rootCategory=INFO, file
    -# log4j.appender.file=org.apache.log4j.FileAppender
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/external/flume/pom.xml b/external/flume/pom.xml
    index a682f0e8471d8..0374262212e08 100644
    --- a/external/flume/pom.xml
    +++ b/external/flume/pom.xml
    @@ -61,11 +61,6 @@
             
           
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -85,11 +80,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties
    index 4411d6e20c52a..9697237bfa1a3 100644
    --- a/external/flume/src/test/resources/log4j.properties
    +++ b/external/flume/src/test/resources/log4j.properties
    @@ -15,11 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file streaming/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
    -# log4j.appender.file=org.apache.log4j.FileAppender
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
    index b3f44471cd326..b29b0509656ba 100644
    --- a/external/kafka/pom.xml
    +++ b/external/kafka/pom.xml
    @@ -74,11 +74,6 @@
           3.2
           test
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -98,11 +93,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties
    index 4411d6e20c52a..9697237bfa1a3 100644
    --- a/external/kafka/src/test/resources/log4j.properties
    +++ b/external/kafka/src/test/resources/log4j.properties
    @@ -15,11 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file streaming/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
    -# log4j.appender.file=org.apache.log4j.FileAppender
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
    index d478267b605ba..560c8b9d18276 100644
    --- a/external/mqtt/pom.xml
    +++ b/external/mqtt/pom.xml
    @@ -46,11 +46,6 @@
           org.eclipse.paho.client.mqttv3
           1.0.1
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -76,11 +71,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties
    index 4411d6e20c52a..9697237bfa1a3 100644
    --- a/external/mqtt/src/test/resources/log4j.properties
    +++ b/external/mqtt/src/test/resources/log4j.properties
    @@ -15,11 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file streaming/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
    -# log4j.appender.file=org.apache.log4j.FileAppender
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml
    index 000ace1446e5e..da6ffe7662f63 100644
    --- a/external/twitter/pom.xml
    +++ b/external/twitter/pom.xml
    @@ -46,11 +46,6 @@
           twitter4j-stream
           3.0.3
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -70,11 +65,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties
    index 4411d6e20c52a..64bfc5745088f 100644
    --- a/external/twitter/src/test/resources/log4j.properties
    +++ b/external/twitter/src/test/resources/log4j.properties
    @@ -15,11 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file streaming/target/unit-tests.log
    +# Set everything to be logged to the filetarget/unit-tests.log
     log4j.rootCategory=INFO, file
    -# log4j.appender.file=org.apache.log4j.FileAppender
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
    index 29c452093502e..2fb5f0ed2f57c 100644
    --- a/external/zeromq/pom.xml
    +++ b/external/zeromq/pom.xml
    @@ -46,11 +46,6 @@
           akka-zeromq_${scala.binary.version}
           ${akka.version}
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -70,11 +65,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties
    index 4411d6e20c52a..9697237bfa1a3 100644
    --- a/external/zeromq/src/test/resources/log4j.properties
    +++ b/external/zeromq/src/test/resources/log4j.properties
    @@ -15,11 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file streaming/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
    -# log4j.appender.file=org.apache.log4j.FileAppender
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml
    index c8477a6566311..0fb431808bacd 100644
    --- a/extras/java8-tests/pom.xml
    +++ b/extras/java8-tests/pom.xml
    @@ -60,11 +60,6 @@
           junit-interface
           test
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
       
     
       
    @@ -159,16 +154,6 @@
               
             
           
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -        
    -          
    -            test
    -            none
    -          
    -        
    -      
         
       
     
    diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties
    index bb0ab319a0080..287c8e3563503 100644
    --- a/extras/java8-tests/src/test/resources/log4j.properties
    +++ b/extras/java8-tests/src/test/resources/log4j.properties
    @@ -18,7 +18,7 @@
     # Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml
    index c0d3a61119113..c815eda52bda7 100644
    --- a/extras/kinesis-asl/pom.xml
    +++ b/extras/kinesis-asl/pom.xml
    @@ -57,11 +57,6 @@
           aws-java-sdk
           ${aws.java.sdk.version}
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.mockito
           mockito-all
    @@ -86,11 +81,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties
    index d9d08f68687d3..853ef0ed2986f 100644
    --- a/extras/kinesis-asl/src/test/resources/log4j.properties
    +++ b/extras/kinesis-asl/src/test/resources/log4j.properties
    @@ -14,10 +14,11 @@
     # See the License for the specific language governing permissions and
     # limitations under the License.
     #
    +
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
    -# log4j.appender.file=org.apache.log4j.FileAppender
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/graphx/pom.xml b/graphx/pom.xml
    index 9982b36f9b62f..91db799d244ad 100644
    --- a/graphx/pom.xml
    +++ b/graphx/pom.xml
    @@ -49,11 +49,6 @@
           org.eclipse.jetty
           jetty-server
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -63,11 +58,5 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
       
     
    diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties
    index 9dd05f17f012b..287c8e3563503 100644
    --- a/graphx/src/test/resources/log4j.properties
    +++ b/graphx/src/test/resources/log4j.properties
    @@ -15,10 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file core/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/mllib/pom.xml b/mllib/pom.xml
    index 0a6dda0ab8c80..2198757481684 100644
    --- a/mllib/pom.xml
    +++ b/mllib/pom.xml
    @@ -80,11 +80,6 @@
           org.apache.commons
           commons-math3
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -129,12 +124,6 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
         
           
             ../python
    diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties
    index a469badf603c6..9697237bfa1a3 100644
    --- a/mllib/src/test/resources/log4j.properties
    +++ b/mllib/src/test/resources/log4j.properties
    @@ -15,10 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file core/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/network/common/pom.xml b/network/common/pom.xml
    index baca859fa5011..245a96b8c4038 100644
    --- a/network/common/pom.xml
    +++ b/network/common/pom.xml
    @@ -75,11 +75,6 @@
           mockito-all
           test
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
       
     
       
    diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml
    index 12468567c3aed..5bfa1ac9c373e 100644
    --- a/network/shuffle/pom.xml
    +++ b/network/shuffle/pom.xml
    @@ -83,11 +83,6 @@
           mockito-all
           test
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
       
     
       
    diff --git a/pom.xml b/pom.xml
    index 05f59a9b4140b..46ff211f91160 100644
    --- a/pom.xml
    +++ b/pom.xml
    @@ -256,7 +256,7 @@
           1.0.0
         
         
    @@ -266,6 +266,15 @@
           2.3.7
           provided
         
    +    
    +    
    +      org.scalatest
    +      scalatest_${scala.binary.version}
    +      test
    +    
       
       
         
    @@ -935,19 +944,38 @@
                 true
               
             
    +        
             
               org.apache.maven.plugins
               maven-surefire-plugin
    -          2.17
    +          2.18
    +          
               
    -            
    -            true
    +            
    +              **/Test*.java
    +              **/*Test.java
    +              **/*TestCase.java
    +              **/*Suite.java
    +            
    +            ${project.build.directory}/surefire-reports
    +            -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
    +            
    +              true
    +              ${session.executionRootDirectory}
    +              1
    +              false
    +              false
    +              ${test_classpath}
    +              true
    +            
               
             
    +        
             
               org.scalatest
               scalatest-maven-plugin
               1.0
    +          
               
                 ${project.build.directory}/surefire-reports
                 .
    @@ -1159,6 +1187,15 @@
               
             
           
    +      
    +      
    +        org.apache.maven.plugins
    +        maven-surefire-plugin
    +      
    +      
    +        org.scalatest
    +        scalatest-maven-plugin
    +      
         
       
     
    diff --git a/repl/pom.xml b/repl/pom.xml
    index 9b2290429fee5..97165e024926e 100644
    --- a/repl/pom.xml
    +++ b/repl/pom.xml
    @@ -86,11 +86,6 @@
           org.slf4j
           jul-to-slf4j
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -115,15 +110,6 @@
               true
             
           
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -        
    -          
    -            ${basedir}/..
    -          
    -        
    -      
           
           
             org.codehaus.mojo
    diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties
    index 52098993f5c3c..e7e4a4113174a 100644
    --- a/repl/src/test/resources/log4j.properties
    +++ b/repl/src/test/resources/log4j.properties
    @@ -15,10 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the repl/target/unit-tests.log
    +# Set everything to be logged to the target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     log4j.appender.file.file=target/unit-tests.log
     log4j.appender.file.layout=org.apache.log4j.PatternLayout
     log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
    diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
    index 1caa297e24e37..a1947fb022e54 100644
    --- a/sql/catalyst/pom.xml
    +++ b/sql/catalyst/pom.xml
    @@ -50,11 +50,6 @@
           spark-core_${scala.binary.version}
           ${project.version}
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -65,11 +60,6 @@
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
         
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -
           
    +    
    +      hadoop-provided
    +      
    +        provided
    +      
    +    
    +    
    +      hive-provided
    +      
    +        provided
    +      
    +    
    +    
    +      parquet-provided
    +      
    +        provided
    +      
    +    
       
     
    diff --git a/bagel/pom.xml b/bagel/pom.xml
    index 3bcd38fa3245c..510e92640eff8 100644
    --- a/bagel/pom.xml
    +++ b/bagel/pom.xml
    @@ -40,10 +40,6 @@
           spark-core_${scala.binary.version}
           ${project.version}
         
    -    
    -      org.eclipse.jetty
    -      jetty-server
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
    index a4c099fb45b14..088f993954d9e 100644
    --- a/bin/compute-classpath.cmd
    +++ b/bin/compute-classpath.cmd
    @@ -109,6 +109,13 @@ if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir
       set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%
     :no_yarn_conf_dir
     
    +rem To allow for distributions to append needed libraries to the classpath (e.g. when
    +rem using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and
    +rem append it to tbe final classpath.
    +if not "x%$SPARK_DIST_CLASSPATH%"=="x" (
    +  set CLASSPATH=%CLASSPATH%;%SPARK_DIST_CLASSPATH%
    +)
    +
     rem A bit of a hack to allow calling this script within run2.cmd without seeing output
     if "%DONT_PRINT_CLASSPATH%"=="1" goto exit
     
    diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
    index a31ea73d3ce19..8f3b396ffd086 100755
    --- a/bin/compute-classpath.sh
    +++ b/bin/compute-classpath.sh
    @@ -146,4 +146,11 @@ if [ -n "$YARN_CONF_DIR" ]; then
       CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
     fi
     
    +# To allow for distributions to append needed libraries to the classpath (e.g. when
    +# using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and
    +# append it to tbe final classpath.
    +if [ -n "$SPARK_DIST_CLASSPATH" ]; then
    +  CLASSPATH="$CLASSPATH:$SPARK_DIST_CLASSPATH"
    +fi
    +
     echo "$CLASSPATH"
    diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
    index 8c7de75600b5f..7eb87a564d6f5 100644
    --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
    +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
    @@ -55,19 +55,26 @@ private[spark] class SparkDeploySchedulerBackend(
           "{{WORKER_URL}}")
         val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions")
           .map(Utils.splitCommandString).getOrElse(Seq.empty)
    -    val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp =>
    -      cp.split(java.io.File.pathSeparator)
    -    }
    -    val libraryPathEntries =
    -      sc.conf.getOption("spark.executor.extraLibraryPath").toSeq.flatMap { cp =>
    -        cp.split(java.io.File.pathSeparator)
    +    val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath")
    +      .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil)
    +    val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath")
    +      .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil)
    +
    +    // When testing, expose the parent class path to the child. This is processed by
    +    // compute-classpath.{cmd,sh} and makes all needed jars available to child processes
    +    // when the assembly is built with the "*-provided" profiles enabled.
    +    val testingClassPath =
    +      if (sys.props.contains("spark.testing")) {
    +        sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq
    +      } else {
    +        Nil
           }
     
         // Start executors with a few necessary configs for registering with the scheduler
         val sparkJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf)
         val javaOpts = sparkJavaOpts ++ extraJavaOpts
         val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend",
    -      args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts)
    +      args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts)
         val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("")
         val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command,
           appUIAddress, sc.eventLogDir)
    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 9d6b6161ce4da..c4f1898a2db15 100644
    --- a/core/src/main/scala/org/apache/spark/util/Utils.scala
    +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
    @@ -990,11 +990,12 @@ private[spark] object Utils extends Logging {
         for ((key, value) <- extraEnvironment) {
           environment.put(key, value)
         }
    +
         val process = builder.start()
         new Thread("read stderr for " + command(0)) {
           override def run() {
             for (line <- Source.fromInputStream(process.getErrorStream).getLines()) {
    -          System.err.println(line)
    +          logInfo(line)
             }
           }
         }.start()
    @@ -1089,7 +1090,7 @@ private[spark] object Utils extends Logging {
         var firstUserLine = 0
         var insideSpark = true
         var callStack = new ArrayBuffer[String]() :+ ""
    - 
    +
         Thread.currentThread.getStackTrace().foreach { ste: StackTraceElement =>
           // When running under some profilers, the current stack trace might contain some bogus
           // frames. This is intended to ensure that we don't crash in these situations by
    diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
    index 541d8eac80556..8a54360e81795 100644
    --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
    +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
    @@ -35,7 +35,7 @@ class DriverSuite extends FunSuite with Timeouts {
         forAll(masters) { (master: String) =>
           failAfter(60 seconds) {
             Utils.executeAndGetOutput(
    -          Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
    +          Seq(s"$sparkHome/bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
               new File(sparkHome),
               Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
           }
    diff --git a/examples/pom.xml b/examples/pom.xml
    index bdc5d0562f3e1..002d4458c4b3e 100644
    --- a/examples/pom.xml
    +++ b/examples/pom.xml
    @@ -98,143 +98,145 @@
           ${project.version}
         
         
    -      org.eclipse.jetty
    -      jetty-server
    +      org.apache.hbase
    +      hbase-testing-util
    +      ${hbase.version}
    +      ${hbase.deps.scope}
    +      
    +        
    +          
    +          org.apache.hbase
    +          hbase-annotations
    +        
    +        
    +          org.jruby
    +          jruby-complete
    +        
    +      
    +    
    +    
    +      org.apache.hbase
    +      hbase-protocol
    +      ${hbase.version}
    +      ${hbase.deps.scope}
    +    
    +    
    +      org.apache.hbase
    +      hbase-common
    +      ${hbase.version}
    +      ${hbase.deps.scope}
    +      
    +        
    +          
    +          org.apache.hbase
    +          hbase-annotations
    +        
    +      
    +    
    +    
    +      org.apache.hbase
    +      hbase-client
    +      ${hbase.version}
    +      ${hbase.deps.scope}
    +      
    +        
    +          
    +          org.apache.hbase
    +          hbase-annotations
    +        
    +       
    +        io.netty
    +        netty
    +       
    +     
    +    
    +    
    +      org.apache.hbase
    +      hbase-server
    +      ${hbase.version}
    +      ${hbase.deps.scope}
    +      
    +        
    +          
    +          org.apache.hbase
    +          hbase-annotations
    +        
    +        
    +          org.apache.hadoop
    +          hadoop-core
    +        
    +        
    +          org.apache.hadoop
    +          hadoop-client
    +        
    +        
    +          org.apache.hadoop
    +          hadoop-mapreduce-client-jobclient
    +        
    +        
    +          org.apache.hadoop
    +          hadoop-mapreduce-client-core
    +        
    +        
    +          org.apache.hadoop
    +          hadoop-auth
    +        
    +        
    +          org.apache.hadoop
    +          hadoop-annotations
    +        
    +        
    +          org.apache.hadoop
    +          hadoop-hdfs
    +        
    +        
    +          org.apache.hbase
    +          hbase-hadoop1-compat
    +        
    +        
    +          org.apache.commons
    +          commons-math
    +        
    +        
    +          com.sun.jersey
    +          jersey-core
    +        
    +        
    +          org.slf4j
    +          slf4j-api
    +        
    +        
    +          com.sun.jersey
    +          jersey-server
    +        
    +        
    +          com.sun.jersey
    +          jersey-core
    +        
    +        
    +          com.sun.jersey
    +          jersey-json
    +        
    +        
    +          
    +          commons-io
    +          commons-io
    +        
    +      
    +    
    +    
    +      org.apache.hbase
    +      hbase-hadoop-compat
    +      ${hbase.version}
    +      ${hbase.deps.scope}
    +    
    +    
    +      org.apache.hbase
    +      hbase-hadoop-compat
    +      ${hbase.version}
    +      test-jar
    +      test
         
    -      
    -        org.apache.hbase
    -        hbase-testing-util
    -        ${hbase.version}
    -        
    -          
    -            
    -            org.apache.hbase
    -            hbase-annotations
    -          
    -          
    -            org.jruby
    -            jruby-complete
    -          
    -        
    -      
    -      
    -        org.apache.hbase
    -        hbase-protocol
    -        ${hbase.version}
    -      
    -      
    -        org.apache.hbase
    -        hbase-common
    -        ${hbase.version}
    -        
    -          
    -            
    -            org.apache.hbase
    -            hbase-annotations
    -          
    -        
    -      
    -      
    -        org.apache.hbase
    -        hbase-client
    -        ${hbase.version}
    -        
    -          
    -            
    -            org.apache.hbase
    -            hbase-annotations
    -          
    -         
    -          io.netty
    -          netty
    -         
    -       
    -      
    -      
    -        org.apache.hbase
    -        hbase-server
    -        ${hbase.version}
    -        
    -          
    -            org.apache.hadoop
    -            hadoop-core
    -          
    -          
    -            org.apache.hadoop
    -            hadoop-client
    -          
    -          
    -            org.apache.hadoop
    -            hadoop-mapreduce-client-jobclient
    -          
    -          
    -            org.apache.hadoop
    -            hadoop-mapreduce-client-core
    -          
    -          
    -            org.apache.hadoop
    -            hadoop-auth
    -          
    -          
    -            
    -            org.apache.hbase
    -            hbase-annotations
    -          
    -          
    -            org.apache.hadoop
    -            hadoop-annotations
    -          
    -          
    -            org.apache.hadoop
    -            hadoop-hdfs
    -          
    -          
    -            org.apache.hbase
    -            hbase-hadoop1-compat
    -          
    -          
    -            org.apache.commons
    -            commons-math
    -          
    -          
    -            com.sun.jersey
    -            jersey-core
    -          
    -          
    -            org.slf4j
    -            slf4j-api
    -          
    -          
    -            com.sun.jersey
    -            jersey-server
    -          
    -          
    -            com.sun.jersey
    -            jersey-core
    -          
    -          
    -            com.sun.jersey
    -            jersey-json
    -          
    -          
    -            
    -            commons-io
    -            commons-io
    -          
    -        
    -      
    -      
    -        org.apache.hbase
    -        hbase-hadoop-compat
    -        ${hbase.version}
    -      
    -      
    -        org.apache.hbase
    -        hbase-hadoop-compat
    -        ${hbase.version}
    -        test-jar
    -        test
    -      
         
           org.apache.commons
           commons-math3
    @@ -308,31 +310,6 @@
           
             org.apache.maven.plugins
             maven-shade-plugin
    -        
    -          false
    -          ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar
    -          
    -            
    -              *:*
    -            
    -          
    -          
    -            
    -              com.google.guava:guava
    -              
    -                com/google/common/base/Optional*
    -              
    -            
    -            
    -              *:*
    -              
    -                META-INF/*.SF
    -                META-INF/*.DSA
    -                META-INF/*.RSA
    -              
    -            
    -          
    -        
             
               
                 package
    @@ -340,6 +317,34 @@
                   shade
                 
                 
    +            false
    +            ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar
    +            
    +              
    +                *:*
    +              
    +            
    +            
    +              
    +                com.google.guava:guava
    +                
    +                  
    +                  **
    +                
    +              
    +              
    +                *:*
    +                
    +                  META-INF/*.SF
    +                  META-INF/*.DSA
    +                  META-INF/*.RSA
    +                
    +              
    +            
                   
                     
                       com.google
    @@ -411,7 +416,7 @@
           
         
         
    -      
           scala-2.10
           
    @@ -449,5 +454,37 @@
             
           
         
    +
    +    
    +    
    +      flume-provided
    +      
    +        provided
    +      
    +    
    +    
    +      hadoop-provided
    +      
    +        provided
    +      
    +    
    +    
    +      hbase-provided
    +      
    +        provided
    +      
    +    
    +    
    +      hive-provided
    +      
    +        provided
    +      
    +    
    +    
    +      parquet-provided
    +      
    +        provided
    +      
    +    
       
     
    diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml
    index 71f595d0a6800..0706f1ebf66e2 100644
    --- a/external/flume-sink/pom.xml
    +++ b/external/flume-sink/pom.xml
    @@ -38,32 +38,10 @@
         
           org.apache.flume
           flume-ng-sdk
    -      ${flume.version}
    -      
    -        
    -          io.netty
    -          netty
    -        
    -        
    -          org.apache.thrift
    -          libthrift
    -        
    -      
         
         
           org.apache.flume
           flume-ng-core
    -      ${flume.version}
    -      
    -        
    -          io.netty
    -          netty
    -        
    -        
    -          org.apache.thrift
    -          libthrift
    -        
    -      
         
         
           org.scala-lang
    diff --git a/external/flume/pom.xml b/external/flume/pom.xml
    index 0374262212e08..1f2681394c583 100644
    --- a/external/flume/pom.xml
    +++ b/external/flume/pom.xml
    @@ -46,20 +46,13 @@
           spark-streaming-flume-sink_${scala.binary.version}
           ${project.version}
         
    +    
    +      org.apache.flume
    +      flume-ng-core
    +    
         
           org.apache.flume
           flume-ng-sdk
    -      ${flume.version}
    -      
    -        
    -          io.netty
    -          netty
    -        
    -        
    -          org.apache.thrift
    -          libthrift
    -        
    -      
         
         
           org.scalacheck
    diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
    index 2fb5f0ed2f57c..e919c2c9b19ea 100644
    --- a/external/zeromq/pom.xml
    +++ b/external/zeromq/pom.xml
    @@ -44,7 +44,6 @@
         
           ${akka.group}
           akka-zeromq_${scala.binary.version}
    -      ${akka.version}
         
         
           org.scalacheck
    diff --git a/graphx/pom.xml b/graphx/pom.xml
    index 91db799d244ad..72374aae6da9b 100644
    --- a/graphx/pom.xml
    +++ b/graphx/pom.xml
    @@ -45,10 +45,6 @@
           jblas
           ${jblas.version}
         
    -    
    -      org.eclipse.jetty
    -      jetty-server
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    diff --git a/mllib/pom.xml b/mllib/pom.xml
    index 2198757481684..a0bda89ccaa71 100644
    --- a/mllib/pom.xml
    +++ b/mllib/pom.xml
    @@ -29,7 +29,7 @@
       spark-mllib_2.10
       
         mllib
    -    
    +  
       jar
       Spark Project ML Library
       http://spark.apache.org/
    @@ -50,10 +50,6 @@
           spark-sql_${scala.binary.version}
           ${project.version}
         
    -    
    -      org.eclipse.jetty
    -      jetty-server
    -    
         
           org.jblas
           jblas
    diff --git a/pom.xml b/pom.xml
    index 46ff211f91160..703e5c47bf59b 100644
    --- a/pom.xml
    +++ b/pom.xml
    @@ -123,8 +123,10 @@
         2.4.1
         ${hadoop.version}
         0.94.6
    +    hbase
         1.4.0
         3.4.5
    +    org.spark-project.hive
         
         0.13.1a
         
    @@ -143,13 +145,36 @@
         4.2.6
         3.1.1
         ${project.build.directory}/spark-test-classpath.txt
    -    64m
    -    512m
         2.10.4
         2.10
         ${scala.version}
         org.scala-lang
    -    1.8.8
    +    1.8.8
    +    1.1.1.6
    +
    +    
    +    compile
    +    compile
    +    compile
    +    compile
    +    compile
    +
    +    
    +    ${session.executionRootDirectory}
    +
    +    64m
    +    512m
    +    512m
       
     
       
    @@ -244,21 +269,20 @@
           
         
       
    -
       
    -  
    +    
         
           org.spark-project.spark
           unused
           1.0.0
         
         
         
           org.codehaus.groovy
    @@ -369,11 +393,13 @@
             org.slf4j
             slf4j-api
             ${slf4j.version}
    +        ${hadoop.deps.scope}
           
           
             org.slf4j
             slf4j-log4j12
             ${slf4j.version}
    +        ${hadoop.deps.scope}
           
           
             org.slf4j
    @@ -390,6 +416,7 @@
             log4j
             log4j
             ${log4j.version}
    +        ${hadoop.deps.scope}
           
           
             com.ning
    @@ -399,7 +426,8 @@
           
             org.xerial.snappy
             snappy-java
    -        1.1.1.6
    +        ${snappy.version}
    +        ${hadoop.deps.scope}
           
           
             net.jpountz.lz4
    @@ -427,6 +455,7 @@
             com.google.protobuf
             protobuf-java
             ${protobuf.version}
    +        ${hadoop.deps.scope}
           
           
             ${akka.group}
    @@ -448,6 +477,17 @@
             akka-testkit_${scala.binary.version}
             ${akka.version}
           
    +      
    +        ${akka.group}
    +        akka-zeromq_${scala.binary.version}
    +        ${akka.version}
    +        
    +          
    +            ${akka.group}
    +            akka-actor_${scala.binary.version}
    +          
    +        
    +      
           
             org.apache.mesos
             mesos
    @@ -577,6 +617,7 @@
             org.apache.curator
             curator-recipes
             2.4.0
    +        ${hadoop.deps.scope}
             
               
                 org.jboss.netty
    @@ -588,6 +629,7 @@
             org.apache.hadoop
             hadoop-client
             ${hadoop.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -623,11 +665,13 @@
             org.apache.avro
             avro
             ${avro.version}
    +        ${hadoop.deps.scope}
           
           
             org.apache.avro
             avro-ipc
             ${avro.version}
    +        ${hadoop.deps.scope}
             
               
                 io.netty
    @@ -656,6 +700,7 @@
             avro-mapred
             ${avro.version}
             ${avro.mapred.classifier}
    +        ${hive.deps.scope}
             
               
                 io.netty
    @@ -684,6 +729,7 @@
             net.java.dev.jets3t
             jets3t
             ${jets3t.version}
    +        ${hadoop.deps.scope}
             
               
                 commons-logging
    @@ -695,6 +741,7 @@
             org.apache.hadoop
             hadoop-yarn-api
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 javax.servlet
    @@ -722,6 +769,7 @@
             org.apache.hadoop
             hadoop-yarn-common
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -778,6 +826,7 @@
             org.apache.hadoop
             hadoop-yarn-server-web-proxy
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -805,6 +854,7 @@
             org.apache.hadoop
             hadoop-yarn-client
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -829,15 +879,126 @@
             
           
           
    -        
    -        org.codehaus.jackson
    -        jackson-mapper-asl
    -        ${jackson.version}
    +        org.apache.zookeeper
    +        zookeeper
    +        ${zookeeper.version}
    +        ${hadoop.deps.scope}
           
           
             org.codehaus.jackson
             jackson-core-asl
    -        ${jackson.version}
    +        ${codehaus.jackson.version}
    +        ${hadoop.deps.scope}
    +      
    +      
    +        org.codehaus.jackson
    +        jackson-mapper-asl
    +        ${codehaus.jackson.version}
    +        ${hadoop.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-beeline
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-cli
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-exec
    +        ${hive.version}
    +        ${hive.deps.scope}
    +        
    +          
    +            commons-logging
    +            commons-logging
    +          
    +          
    +            com.esotericsoftware.kryo
    +            kryo
    +          
    +        
    +      
    +      
    +        ${hive.group}
    +        hive-jdbc
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-metastore
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-serde
    +        ${hive.version}
    +        ${hive.deps.scope}
    +        
    +          
    +            commons-logging
    +            commons-logging
    +          
    +          
    +            commons-logging
    +            commons-logging-api
    +          
    +        
    +      
    +      
    +        com.twitter
    +        parquet-column
    +        ${parquet.version}
    +        ${parquet.deps.scope}
    +      
    +      
    +        com.twitter
    +        parquet-hadoop
    +        ${parquet.version}
    +        ${parquet.deps.scope}
    +      
    +      
    +        org.apache.flume
    +        flume-ng-core
    +        ${flume.version}
    +        ${flume.deps.scope}
    +        
    +          
    +            io.netty
    +            netty
    +          
    +          
    +            org.apache.thrift
    +            libthrift
    +          
    +          
    +            org.mortbay.jetty
    +            servlet-api
    +          
    +        
    +      
    +      
    +        org.apache.flume
    +        flume-ng-sdk
    +        ${flume.version}
    +        ${flume.deps.scope}
    +        
    +          
    +            io.netty
    +            netty
    +          
    +          
    +            org.apache.thrift
    +            libthrift
    +          
    +        
           
         
       
    @@ -914,6 +1075,7 @@
                   -Xmx1024m
                   -XX:PermSize=${PermGen}
                   -XX:MaxPermSize=${MaxPermGen}
    +              -XX:ReservedCodeCacheSize=${CodeCacheSize}
                 
                 
                   -source
    @@ -980,15 +1142,21 @@
                 ${project.build.directory}/surefire-reports
                 .
                 SparkTestSuite.txt
    -            -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
    +            -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize}
                 
    +            
    +              
    +              ${test_classpath}
    +            
                 
                   true
    -              ${session.executionRootDirectory}
    +              ${spark.test.home}
                   1
                   false
                   false
    -              ${test_classpath}
                   true
                 
               
    @@ -1011,11 +1179,6 @@
               maven-antrun-plugin
               1.7
             
    -        
    -          org.apache.maven.plugins
    -          maven-shade-plugin
    -          2.2
    -        
             
               org.apache.maven.plugins
               maven-source-plugin
    @@ -1104,6 +1267,7 @@
           
             org.apache.maven.plugins
             maven-shade-plugin
    +        2.2
             
               false
               
    @@ -1373,53 +1537,6 @@
           
         
     
    -    
    -    
    -      hadoop-provided
    -      
    -        
    -          org.apache.hadoop
    -          hadoop-client
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-api
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-common
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-server-web-proxy
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-client
    -          provided
    -        
    -        
    -          org.apache.avro
    -          avro
    -          provided
    -        
    -        
    -          org.apache.avro
    -          avro-ipc
    -          provided
    -        
    -        
    -          org.apache.zookeeper
    -          zookeeper
    -          ${zookeeper.version}
    -          provided
    -        
    -      
    -    
         
           hive-thriftserver
           
    @@ -1472,5 +1589,25 @@
           
         
     
    +    
    +    
    +      flume-provided
    +    
    +    
    +      hadoop-provided
    +    
    +    
    +      hbase-provided
    +    
    +    
    +      hive-provided
    +    
    +    
    +      parquet-provided
    +    
       
     
    diff --git a/repl/pom.xml b/repl/pom.xml
    index 97165e024926e..0bc8bccf90a6d 100644
    --- a/repl/pom.xml
    +++ b/repl/pom.xml
    @@ -68,10 +68,6 @@
           ${project.version}
           test
         
    -    
    -      org.eclipse.jetty
    -      jetty-server
    -    
         
           org.scala-lang
           scala-compiler
    @@ -103,13 +99,6 @@
               true
             
           
    -      
    -        org.apache.maven.plugins
    -        maven-install-plugin
    -        
    -          true
    -        
    -      
           
           
             org.codehaus.mojo
    diff --git a/sql/core/pom.xml b/sql/core/pom.xml
    index 023ce2041bb86..3e9ef07df9db6 100644
    --- a/sql/core/pom.xml
    +++ b/sql/core/pom.xml
    @@ -56,12 +56,10 @@
         
           com.twitter
           parquet-column
    -      ${parquet.version}
         
         
           com.twitter
           parquet-hadoop
    -      ${parquet.version}
         
         
           com.fasterxml.jackson.core
    diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml
    index d3a517375cf25..259eef0b80d03 100644
    --- a/sql/hive-thriftserver/pom.xml
    +++ b/sql/hive-thriftserver/pom.xml
    @@ -42,19 +42,16 @@
           ${project.version}
         
         
    -      org.spark-project.hive
    +      ${hive.group}
           hive-cli
    -      ${hive.version}
         
         
    -      org.spark-project.hive
    +      ${hive.group}
           hive-jdbc
    -      ${hive.version}
         
         
    -      org.spark-project.hive
    +      ${hive.group}
           hive-beeline
    -      ${hive.version}
         
       
       
    diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
    index e8ffbc5b954d4..60953576d0e37 100644
    --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
    +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
    @@ -48,6 +48,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
              |  --master local
              |  --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl
              |  --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath
    +         |  --driver-class-path ${sys.props("java.class.path")}
            """.stripMargin.split("\\s+").toSeq ++ extraArgs
         }
     
    @@ -70,7 +71,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
         }
     
         // Searching expected output line from both stdout and stderr of the CLI process
    -    val process = (Process(command) #< queryStream).run(
    +    val process = (Process(command, None) #< queryStream).run(
           ProcessLogger(captureOutput("stdout"), captureOutput("stderr")))
     
         try {
    diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala
    index 94d5ed4f1d15e..7814aa38f4146 100644
    --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala
    +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala
    @@ -142,6 +142,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
                  |  --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost
                  |  --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=http
                  |  --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT}=$port
    +             |  --driver-class-path ${sys.props("java.class.path")}
                """.stripMargin.split("\\s+").toSeq
           } else {
               s"""$startScript
    @@ -151,6 +152,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
                  |  --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath
                  |  --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost
                  |  --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$port
    +             |  --driver-class-path ${sys.props("java.class.path")}
                """.stripMargin.split("\\s+").toSeq
           }
     
    @@ -179,8 +181,9 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
           }
         }
     
    -    // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths
    -    val env = Seq("SPARK_TESTING" -> "0")
    +    val env = Seq(
    +      // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths
    +      "SPARK_TESTING" -> "0")
     
         Process(command, None, env: _*).run(ProcessLogger(
           captureThriftServerOutput("stdout"),
    @@ -214,7 +217,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
         } finally {
           warehousePath.delete()
           metastorePath.delete()
    -      Process(stopScript).run().exitValue()
    +      Process(stopScript, None, env: _*).run().exitValue()
           // The `spark-daemon.sh' script uses kill, which is not synchronous, have to wait for a while.
           Thread.sleep(3.seconds.toMillis)
           Option(logTailingProcess).map(_.destroy())
    diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
    index 46aacad01113f..58b0722464be8 100644
    --- a/sql/hive/pom.xml
    +++ b/sql/hive/pom.xml
    @@ -47,9 +47,8 @@
           ${project.version}
         
         
    -      org.spark-project.hive
    +      ${hive.group}
           hive-metastore
    -      ${hive.version}
         
         
           commons-httpclient
    @@ -57,51 +56,27 @@
           3.1
         
         
    -      org.spark-project.hive
    +      ${hive.group}
           hive-exec
    -      ${hive.version}
    -      
    -        
    -          commons-logging
    -          commons-logging
    -        
    -        
    -          com.esotericsoftware.kryo
    -          kryo
    -        
    -      
         
         
           org.codehaus.jackson
           jackson-mapper-asl
         
         
    -      org.spark-project.hive
    +      ${hive.group}
           hive-serde
    -      ${hive.version}
    -      
    -        
    -          commons-logging
    -          commons-logging
    -        
    -        
    -          commons-logging
    -          commons-logging-api
    -        
    -      
         
         
         
           org.apache.avro
           avro
    -      ${avro.version}
         
         
         
           org.apache.avro
           avro-mapred
    -      ${avro.version}
           ${avro.mapred.classifier}
         
         
    diff --git a/streaming/pom.xml b/streaming/pom.xml
    index 2023210d9b9be..d3c6d0347a622 100644
    --- a/streaming/pom.xml
    +++ b/streaming/pom.xml
    @@ -68,13 +68,13 @@
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
         
    -      
           
    diff --git a/yarn/pom.xml b/yarn/pom.xml
    index bcb77b3e3c70e..b86857db7bde6 100644
    --- a/yarn/pom.xml
    +++ b/yarn/pom.xml
    @@ -131,13 +131,6 @@
               true
             
           
    -      
    -        org.apache.maven.plugins
    -        maven-install-plugin
    -        
    -          true
    -        
    -      
         
     
         target/scala-${scala.binary.version}/classes
    diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
    index 8d0543771309b..c363d755c1752 100644
    --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
    +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
    @@ -367,6 +367,10 @@ private[spark] class Client(
           }
         }
     
    +    sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp =>
    +      env(ENV_DIST_CLASSPATH) = dcp
    +    }
    +
         env
       }
     
    @@ -652,6 +656,9 @@ object Client extends Logging {
       val APP_FILE_PERMISSION: FsPermission =
         FsPermission.createImmutable(Integer.parseInt("644", 8).toShort)
     
    +  // Distribution-defined classpath to add to processes
    +  val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH"
    +
       /**
        * Find the user-defined Spark jar if configured, or return the jar containing this
        * class if not.
    
    From 167a5ab0bd1d37f3ac23bec49e484a238610cf75 Mon Sep 17 00:00:00 2001
    From: Nicholas Chammas 
    Date: Thu, 8 Jan 2015 17:42:08 -0800
    Subject: [PATCH 221/227] [SPARK-5122] Remove Shark from spark-ec2
    
    I moved the Spark-Shark version map [to the wiki](https://cwiki.apache.org/confluence/display/SPARK/Spark-Shark+version+mapping).
    
    This PR has a [matching PR in mesos/spark-ec2](https://github.com/mesos/spark-ec2/pull/89).
    
    Author: Nicholas Chammas 
    
    Closes #3939 from nchammas/remove-shark and squashes the following commits:
    
    66e0841 [Nicholas Chammas] fix style
    ceeab85 [Nicholas Chammas] show default Spark GitHub repo
    7270126 [Nicholas Chammas] validate Spark hashes
    db4935d [Nicholas Chammas] validate spark version upfront
    fc0d5b9 [Nicholas Chammas] remove Shark
    ---
     ec2/spark_ec2.py | 78 +++++++++++++++++++++++++++---------------------
     1 file changed, 44 insertions(+), 34 deletions(-)
    
    diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
    index 485eea4f5e683..abab209a05ba0 100755
    --- a/ec2/spark_ec2.py
    +++ b/ec2/spark_ec2.py
    @@ -39,10 +39,26 @@
     from optparse import OptionParser
     from sys import stderr
     
    +VALID_SPARK_VERSIONS = set([
    +    "0.7.3",
    +    "0.8.0",
    +    "0.8.1",
    +    "0.9.0",
    +    "0.9.1",
    +    "0.9.2",
    +    "1.0.0",
    +    "1.0.1",
    +    "1.0.2",
    +    "1.1.0",
    +    "1.1.1",
    +    "1.2.0",
    +])
    +
     DEFAULT_SPARK_VERSION = "1.2.0"
    +DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark"
     SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__))
    -
     MESOS_SPARK_EC2_BRANCH = "branch-1.3"
    +
     # A URL prefix from which to fetch AMI information
     AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH)
     
    @@ -126,8 +142,8 @@ def parse_args():
             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")
    +        default=DEFAULT_SPARK_GITHUB_REPO,
    +        help="Github repo from which to checkout supplied commit hash (default: %default)")
         parser.add_option(
             "--hadoop-major-version", default="1",
             help="Major version of Hadoop (default: %default)")
    @@ -236,6 +252,26 @@ def get_or_make_group(conn, name, vpc_id):
             return conn.create_security_group(name, "Spark EC2 group", vpc_id)
     
     
    +def get_validate_spark_version(version, repo):
    +    if "." in version:
    +        version = version.replace("v", "")
    +        if version not in VALID_SPARK_VERSIONS:
    +            print >> stderr, "Don't know about Spark version: {v}".format(v=version)
    +            sys.exit(1)
    +        return version
    +    else:
    +        github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version)
    +        request = urllib2.Request(github_commit_url)
    +        request.get_method = lambda: 'HEAD'
    +        try:
    +            response = urllib2.urlopen(request)
    +        except urllib2.HTTPError, e:
    +            print >> stderr, "Couldn't validate Spark commit: {url}".format(url=github_commit_url)
    +            print >> stderr, "Received HTTP response code of {code}.".format(code=e.code)
    +            sys.exit(1)
    +        return version
    +
    +
     # Check whether a given EC2 instance object is in a state we consider active,
     # i.e. not terminating or terminated. We count both stopping and stopped as
     # active since we can restart stopped clusters.
    @@ -243,29 +279,6 @@ def is_active(instance):
         return (instance.state in ['pending', 'running', 'stopping', 'stopped'])
     
     
    -# Return correct versions of Spark and Shark, given the supplied Spark version
    -def get_spark_shark_version(opts):
    -    spark_shark_map = {
    -        "0.7.3": "0.7.1",
    -        "0.8.0": "0.8.0",
    -        "0.8.1": "0.8.1",
    -        "0.9.0": "0.9.0",
    -        "0.9.1": "0.9.1",
    -        # These are dummy versions (no Shark versions after this)
    -        "1.0.0": "1.0.0",
    -        "1.0.1": "1.0.1",
    -        "1.0.2": "1.0.2",
    -        "1.1.0": "1.1.0",
    -        "1.1.1": "1.1.1",
    -        "1.2.0": "1.2.0",
    -    }
    -    version = opts.spark_version.replace("v", "")
    -    if version not in spark_shark_map:
    -        print >> stderr, "Don't know about Spark version: %s" % version
    -        sys.exit(1)
    -    return (version, spark_shark_map[version])
    -
    -
     # Attempt to resolve an appropriate AMI given the architecture and region of the request.
     # Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/
     # Last Updated: 2014-06-20
    @@ -619,7 +632,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
                 print slave.public_dns_name
                 ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar)
     
    -    modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs',
    +    modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs',
                    'mapreduce', 'spark-standalone', 'tachyon']
     
         if opts.hadoop_major_version == "1":
    @@ -706,9 +719,7 @@ def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state):
         sys.stdout.flush()
     
         start_time = datetime.now()
    -
         num_attempts = 0
    -    conn = ec2.connect_to_region(opts.region)
     
         while True:
             time.sleep(5 * num_attempts)  # seconds
    @@ -815,13 +826,11 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
         cluster_url = "%s:7077" % active_master
     
         if "." in opts.spark_version:
    -        # Pre-built spark & shark deploy
    -        (spark_v, shark_v) = get_spark_shark_version(opts)
    +        # Pre-built Spark deploy
    +        spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo)
         else:
             # Spark-only custom deploy
             spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version)
    -        shark_v = ""
    -        modules = filter(lambda x: x != "shark", modules)
     
         template_vars = {
             "master_list": '\n'.join([i.public_dns_name for i in master_nodes]),
    @@ -834,7 +843,6 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
             "swap": str(opts.swap),
             "modules": '\n'.join(modules),
             "spark_version": spark_v,
    -        "shark_version": shark_v,
             "hadoop_major_version": opts.hadoop_major_version,
             "spark_worker_instances": "%d" % opts.worker_instances,
             "spark_master_opts": opts.master_opts
    @@ -983,6 +991,8 @@ def real_main():
         (opts, action, cluster_name) = parse_args()
     
         # Input parameter validation
    +    get_validate_spark_version(opts.spark_version, opts.spark_git_repo)
    +
         if opts.wait is not None:
             # NOTE: DeprecationWarnings are silent in 2.7+ by default.
             #       To show them, run Python with the -Wdefault switch.
    
    From f3da4bd7289d493014ad3c5176ada60794dfcfe0 Mon Sep 17 00:00:00 2001
    From: WangTaoTheTonic 
    Date: Fri, 9 Jan 2015 08:10:09 -0600
    Subject: [PATCH 222/227] [SPARK-5169][YARN]fetch the correct max attempts
    
    Soryy for fetching the wrong max attempts in this commit https://github.com/apache/spark/commit/8fdd48959c93b9cf809f03549e2ae6c4687d1fcd.
    We need to fix it now.
    
    tgravescs
    
    If we set an spark.yarn.maxAppAttempts which is larger than `yarn.resourcemanager.am.max-attempts` in yarn side, it will be overrided as described here:
    >The maximum number of application attempts. It's a global setting for all application masters. Each application master can specify its individual maximum number of application attempts via the API, but the individual number cannot be more than the global upper bound. If it is, the resourcemanager will override it. The default number is set to 2, to allow at least one retry for AM.
    
    http://hadoop.apache.org/docs/r2.6.0/hadoop-yarn/hadoop-yarn-common/yarn-default.xml
    
    Author: WangTaoTheTonic 
    
    Closes #3942 from WangTaoTheTonic/HOTFIX and squashes the following commits:
    
    9ac16ce [WangTaoTheTonic] fetch the correct max attempts
    ---
     .../org/apache/spark/deploy/yarn/YarnRMClient.scala  | 12 +++++++++---
     1 file changed, 9 insertions(+), 3 deletions(-)
    
    diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
    index e183efccbb6f7..b45e599588ad3 100644
    --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
    +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
    @@ -121,9 +121,15 @@ private[spark] class YarnRMClient(args: ApplicationMasterArguments) extends Logg
     
       /** Returns the maximum number of attempts to register the AM. */
       def getMaxRegAttempts(sparkConf: SparkConf, yarnConf: YarnConfiguration): Int = {
    -    sparkConf.getOption("spark.yarn.maxAppAttempts").map(_.toInt).getOrElse(
    -      yarnConf.getInt(
    -        YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS))
    +    val sparkMaxAttempts = sparkConf.getOption("spark.yarn.maxAppAttempts").map(_.toInt)
    +    val yarnMaxAttempts = yarnConf.getInt(
    +      YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
    +    val retval: Int = sparkMaxAttempts match {
    +      case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
    +      case None => yarnMaxAttempts
    +    }
    +
    +    retval
       }
     
     }
    
    From b4034c3f889bf24f60eb806802866b48e4cbe55c Mon Sep 17 00:00:00 2001
    From: Aaron Davidson 
    Date: Fri, 9 Jan 2015 09:20:16 -0800
    Subject: [PATCH 223/227] [Minor] Fix test RetryingBlockFetcherSuite after
     changed config name
    
    Flakey due to the default retry interval being the same as our test's wait timeout.
    
    Author: Aaron Davidson 
    
    Closes #3972 from aarondav/fix-test and squashes the following commits:
    
    db77cab [Aaron Davidson] [Minor] Fix test after changed config name
    ---
     .../spark/network/shuffle/RetryingBlockFetcherSuite.java      | 4 ++--
     1 file changed, 2 insertions(+), 2 deletions(-)
    
    diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
    index 0191fe529e1be..1ad0d72ae5ec5 100644
    --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
    +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
    @@ -54,13 +54,13 @@ public class RetryingBlockFetcherSuite {
       @Before
       public void beforeEach() {
         System.setProperty("spark.shuffle.io.maxRetries", "2");
    -    System.setProperty("spark.shuffle.io.retryWaitMs", "0");
    +    System.setProperty("spark.shuffle.io.retryWait", "0");
       }
     
       @After
       public void afterEach() {
         System.clearProperty("spark.shuffle.io.maxRetries");
    -    System.clearProperty("spark.shuffle.io.retryWaitMs");
    +    System.clearProperty("spark.shuffle.io.retryWait");
       }
     
       @Test
    
    From 547df97715580f99ae573a49a86da12bf20cbc3d Mon Sep 17 00:00:00 2001
    From: Sean Owen 
    Date: Fri, 9 Jan 2015 09:35:46 -0800
    Subject: [PATCH 224/227] SPARK-5136 [DOCS] Improve documentation around
     setting up Spark IntelliJ project
    
    This PR simply points to the IntelliJ wiki page instead of also including IntelliJ notes in the docs. The intent however is to also update the wiki page with updated tips. This is the text I propose for the IntelliJ section on the wiki. I realize it omits some of the existing instructions on the wiki, about enabling Hive, but I think those are actually optional.
    
    ------
    
    IntelliJ supports both Maven- and SBT-based projects. It is recommended, however, to import Spark as a Maven project. Choose "Import Project..." from the File menu, and select the `pom.xml` file in the Spark root directory.
    
    It is fine to leave all settings at their default values in the Maven import wizard, with two caveats. First, it is usually useful to enable "Import Maven projects automatically", sincchanges to the project structure will automatically update the IntelliJ project.
    
    Second, note the step that prompts you to choose active Maven build profiles. As documented above, some build configuration require specific profiles to be enabled. The same profiles that are enabled with `-P[profile name]` above may be enabled on this screen. For example, if developing for Hadoop 2.4 with YARN support, enable profiles `yarn` and `hadoop-2.4`.
    
    These selections can be changed later by accessing the "Maven Projects" tool window from the View menu, and expanding the Profiles section.
    
    "Rebuild Project" can fail the first time the project is compiled, because generate source files are not automatically generated. Try clicking the  "Generate Sources and Update Folders For All Projects" button in the "Maven Projects" tool window to manually generate these sources.
    
    Compilation may fail with an error like "scalac: bad option: -P:/home/jakub/.m2/repository/org/scalamacros/paradise_2.10.4/2.0.1/paradise_2.10.4-2.0.1.jar". If so, go to Preferences > Build, Execution, Deployment > Scala Compiler and clear the "Additional compiler options" field. It will work then although the option will come back when the project reimports.
    
    Author: Sean Owen 
    
    Closes #3952 from srowen/SPARK-5136 and squashes the following commits:
    
    f3baa66 [Sean Owen] Point to new IJ / Eclipse wiki link
    016b7df [Sean Owen] Point to IntelliJ wiki page instead of also including IntelliJ notes in the docs
    ---
     docs/building-spark.md | 5 +++--
     1 file changed, 3 insertions(+), 2 deletions(-)
    
    diff --git a/docs/building-spark.md b/docs/building-spark.md
    index c1bcd91b5b853..fb93017861ed0 100644
    --- a/docs/building-spark.md
    +++ b/docs/building-spark.md
    @@ -151,9 +151,10 @@ Thus, the full flow for running continuous-compilation of the `core` submodule m
      $ mvn scala:cc
     ```
     
    -# Using With IntelliJ IDEA
    +# Building Spark with IntelliJ IDEA or Eclipse
     
    -This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the pom.xml file in the project root folder, you only need to activate either the hadoop1 or hadoop2 profile in the "Maven Properties" popout. We have not tried Eclipse/Scala IDE with this.
    +For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the
    +[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-IDESetup).
     
     # Building Spark Debian Packages
     
    
    From 1790b38695b46400a24b0b7e278e8e8388748211 Mon Sep 17 00:00:00 2001
    From: Patrick Wendell 
    Date: Fri, 9 Jan 2015 09:40:18 -0800
    Subject: [PATCH 225/227] HOTFIX: Minor improvements to make-distribution.sh
    
    1. Renames $FWDIR to $SPARK_HOME (vast majority of diff).
    2. Use Spark-provided Maven.
    3. Logs build flags in the RELEASE file.
    
    Author: Patrick Wendell 
    
    Closes #3973 from pwendell/master and squashes the following commits:
    
    340a2fa [Patrick Wendell] HOTFIX: Minor improvements to make-distribution.sh
    ---
     make-distribution.sh | 61 ++++++++++++++++++++++++--------------------
     1 file changed, 34 insertions(+), 27 deletions(-)
    
    diff --git a/make-distribution.sh b/make-distribution.sh
    index 45c99e42e5a5b..4e2f400be3053 100755
    --- a/make-distribution.sh
    +++ b/make-distribution.sh
    @@ -28,18 +28,20 @@ set -o pipefail
     set -e
     
     # Figure out where the Spark framework is installed
    -FWDIR="$(cd "`dirname "$0"`"; pwd)"
    -DISTDIR="$FWDIR/dist"
    +SPARK_HOME="$(cd "`dirname "$0"`"; pwd)"
    +DISTDIR="$SPARK_HOME/dist"
     
     SPARK_TACHYON=false
     MAKE_TGZ=false
     NAME=none
    +MVN="$SPARK_HOME/build/mvn"
     
     function exit_with_usage {
       echo "make-distribution.sh - tool for making binary distributions of Spark"
       echo ""
       echo "usage:"
    -  echo "./make-distribution.sh [--name] [--tgz] [--with-tachyon] "
    +  cl_options="[--name] [--tgz] [--mvn ] [--with-tachyon]"
    +  echo "./make-distribution.sh $cl_options "
       echo "See Spark's \"Building Spark\" doc for correct Maven options."
       echo ""
       exit 1
    @@ -71,6 +73,10 @@ while (( "$#" )); do
         --tgz)
           MAKE_TGZ=true
           ;;
    +    --mvn)
    +      MVN="$2"
    +      shift
    +      ;;
         --name)
           NAME="$2"
           shift
    @@ -109,9 +115,9 @@ if which git &>/dev/null; then
         unset GITREV
     fi
     
    -if ! which mvn &>/dev/null; then
    -    echo -e "You need Maven installed to build Spark."
    -    echo -e "Download Maven from https://maven.apache.org/"
    +if ! which $MVN &>/dev/null; then
    +    echo -e "Could not locate Maven command: '$MVN'."
    +    echo -e "Specify the Maven command with the --mvn flag"
         exit -1;
     fi
     
    @@ -119,7 +125,7 @@ VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "
     SPARK_HADOOP_VERSION=$(mvn help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\
         | grep -v "INFO"\
         | tail -n 1)
    -SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\
    +SPARK_HIVE=$($MVN help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\
         | grep -v "INFO"\
         | fgrep --count "hive";\
         # Reset exit status to 0, otherwise the script stops here if the last grep finds nothing\
    @@ -161,11 +167,11 @@ else
     fi
     
     # Build uber fat JAR
    -cd "$FWDIR"
    +cd "$SPARK_HOME"
     
     export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"
     
    -BUILD_COMMAND="mvn clean package -DskipTests $@"
    +BUILD_COMMAND="$MVN clean package -DskipTests $@"
     
     # Actually build the jar
     echo -e "\nBuilding with..."
    @@ -177,41 +183,42 @@ ${BUILD_COMMAND}
     rm -rf "$DISTDIR"
     mkdir -p "$DISTDIR/lib"
     echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE"
    +echo "Build flags: $@" >> "$DISTDIR/RELEASE"
     
     # Copy jars
    -cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
    -cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
    +cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
    +cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
     # This will fail if the -Pyarn profile is not provided
     # In this case, silence the error and ignore the return code of this command
    -cp "$FWDIR"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
    +cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
     
     # Copy example sources (needed for python and SQL)
     mkdir -p "$DISTDIR/examples/src/main"
    -cp -r "$FWDIR"/examples/src/main "$DISTDIR/examples/src/"
    +cp -r "$SPARK_HOME"/examples/src/main "$DISTDIR/examples/src/"
     
     if [ "$SPARK_HIVE" == "1" ]; then
    -  cp "$FWDIR"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/"
    +  cp "$SPARK_HOME"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/"
     fi
     
     # Copy license and ASF files
    -cp "$FWDIR/LICENSE" "$DISTDIR"
    -cp "$FWDIR/NOTICE" "$DISTDIR"
    +cp "$SPARK_HOME/LICENSE" "$DISTDIR"
    +cp "$SPARK_HOME/NOTICE" "$DISTDIR"
     
    -if [ -e "$FWDIR"/CHANGES.txt ]; then
    -  cp "$FWDIR/CHANGES.txt" "$DISTDIR"
    +if [ -e "$SPARK_HOME"/CHANGES.txt ]; then
    +  cp "$SPARK_HOME/CHANGES.txt" "$DISTDIR"
     fi
     
     # Copy data files
    -cp -r "$FWDIR/data" "$DISTDIR"
    +cp -r "$SPARK_HOME/data" "$DISTDIR"
     
     # Copy other things
     mkdir "$DISTDIR"/conf
    -cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
    -cp "$FWDIR/README.md" "$DISTDIR"
    -cp -r "$FWDIR/bin" "$DISTDIR"
    -cp -r "$FWDIR/python" "$DISTDIR"
    -cp -r "$FWDIR/sbin" "$DISTDIR"
    -cp -r "$FWDIR/ec2" "$DISTDIR"
    +cp "$SPARK_HOME"/conf/*.template "$DISTDIR"/conf
    +cp "$SPARK_HOME/README.md" "$DISTDIR"
    +cp -r "$SPARK_HOME/bin" "$DISTDIR"
    +cp -r "$SPARK_HOME/python" "$DISTDIR"
    +cp -r "$SPARK_HOME/sbin" "$DISTDIR"
    +cp -r "$SPARK_HOME/ec2" "$DISTDIR"
     
     # Download and copy in tachyon, if requested
     if [ "$SPARK_TACHYON" == "true" ]; then
    @@ -243,9 +250,9 @@ fi
     
     if [ "$MAKE_TGZ" == "true" ]; then
       TARDIR_NAME=spark-$VERSION-bin-$NAME
    -  TARDIR="$FWDIR/$TARDIR_NAME"
    +  TARDIR="$SPARK_HOME/$TARDIR_NAME"
       rm -rf "$TARDIR"
       cp -r "$DISTDIR" "$TARDIR"
    -  tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME"
    +  tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$SPARK_HOME" "$TARDIR_NAME"
       rm -rf "$TARDIR"
     fi
    
    From b6aa557300275b835cce7baa7bc8a80eb5425cbb Mon Sep 17 00:00:00 2001
    From: Kay Ousterhout 
    Date: Fri, 9 Jan 2015 09:47:06 -0800
    Subject: [PATCH 226/227] [SPARK-1143] Separate pool tests into their own
     suite.
    
    The current TaskSchedulerImplSuite includes some tests that are
    actually for the TaskSchedulerImpl, but the remainder of the tests avoid using
    the TaskSchedulerImpl entirely, and actually test the pool and scheduling
    algorithm mechanisms. This commit separates the pool/scheduling algorithm
    tests into their own suite, and also simplifies those tests.
    
    The pull request replaces #339.
    
    Author: Kay Ousterhout 
    
    Closes #3967 from kayousterhout/SPARK-1143 and squashes the following commits:
    
    8a898c4 [Kay Ousterhout] [SPARK-1143] Separate pool tests into their own suite.
    ---
     .../apache/spark/scheduler/PoolSuite.scala    | 183 ++++++++++++++
     .../scheduler/TaskSchedulerImplSuite.scala    | 230 ------------------
     2 files changed, 183 insertions(+), 230 deletions(-)
     create mode 100644 core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala
    
    diff --git a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala
    new file mode 100644
    index 0000000000000..e8f461e2f56c9
    --- /dev/null
    +++ b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.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.scheduler
    +
    +import java.util.Properties
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext}
    +
    +/**
    + * Tests that pools and the associated scheduling algorithms for FIFO and fair scheduling work
    + * correctly.
    + */
    +class PoolSuite extends FunSuite with LocalSparkContext {
    +
    +  def createTaskSetManager(stageId: Int, numTasks: Int, taskScheduler: TaskSchedulerImpl)
    +    : TaskSetManager = {
    +    val tasks = Array.tabulate[Task[_]](numTasks) { i =>
    +      new FakeTask(i, Nil)
    +    }
    +    new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null), 0)
    +  }
    +
    +  def scheduleTaskAndVerifyId(taskId: Int, rootPool: Pool, expectedStageId: Int) {
    +    val taskSetQueue = rootPool.getSortedTaskSetQueue
    +    val nextTaskSetToSchedule =
    +      taskSetQueue.find(t => (t.runningTasks + t.tasksSuccessful) < t.numTasks)
    +    assert(nextTaskSetToSchedule.isDefined)
    +    nextTaskSetToSchedule.get.addRunningTask(taskId)
    +    assert(nextTaskSetToSchedule.get.stageId === expectedStageId)
    +  }
    +
    +  test("FIFO Scheduler Test") {
    +    sc = new SparkContext("local", "TaskSchedulerImplSuite")
    +    val taskScheduler = new TaskSchedulerImpl(sc)
    +
    +    val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0)
    +    val schedulableBuilder = new FIFOSchedulableBuilder(rootPool)
    +    schedulableBuilder.buildPools()
    +
    +    val taskSetManager0 = createTaskSetManager(0, 2, taskScheduler)
    +    val taskSetManager1 = createTaskSetManager(1, 2, taskScheduler)
    +    val taskSetManager2 = createTaskSetManager(2, 2, taskScheduler)
    +    schedulableBuilder.addTaskSetManager(taskSetManager0, null)
    +    schedulableBuilder.addTaskSetManager(taskSetManager1, null)
    +    schedulableBuilder.addTaskSetManager(taskSetManager2, null)
    +
    +    scheduleTaskAndVerifyId(0, rootPool, 0)
    +    scheduleTaskAndVerifyId(1, rootPool, 0)
    +    scheduleTaskAndVerifyId(2, rootPool, 1)
    +    scheduleTaskAndVerifyId(3, rootPool, 1)
    +    scheduleTaskAndVerifyId(4, rootPool, 2)
    +    scheduleTaskAndVerifyId(5, rootPool, 2)
    +  }
    +
    +  /**
    +   * This test creates three scheduling pools, and creates task set managers in the first
    +   * two scheduling pools. The test verifies that as tasks are scheduled, the fair scheduling
    +   * algorithm properly orders the two scheduling pools.
    +   */
    +  test("Fair Scheduler Test") {
    +    val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
    +    val conf = new SparkConf().set("spark.scheduler.allocation.file", xmlPath)
    +    sc = new SparkContext("local", "TaskSchedulerImplSuite", conf)
    +    val taskScheduler = new TaskSchedulerImpl(sc)
    +
    +    val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
    +    val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf)
    +    schedulableBuilder.buildPools()
    +
    +    // Ensure that the XML file was read in correctly.
    +    assert(rootPool.getSchedulableByName("default") != null)
    +    assert(rootPool.getSchedulableByName("1") != null)
    +    assert(rootPool.getSchedulableByName("2") != null)
    +    assert(rootPool.getSchedulableByName("3") != null)
    +    assert(rootPool.getSchedulableByName("1").minShare === 2)
    +    assert(rootPool.getSchedulableByName("1").weight === 1)
    +    assert(rootPool.getSchedulableByName("2").minShare === 3)
    +    assert(rootPool.getSchedulableByName("2").weight === 1)
    +    assert(rootPool.getSchedulableByName("3").minShare === 0)
    +    assert(rootPool.getSchedulableByName("3").weight === 1)
    +
    +    val properties1 = new Properties()
    +    properties1.setProperty("spark.scheduler.pool","1")
    +    val properties2 = new Properties()
    +    properties2.setProperty("spark.scheduler.pool","2")
    +
    +    val taskSetManager10 = createTaskSetManager(0, 1, taskScheduler)
    +    val taskSetManager11 = createTaskSetManager(1, 1, taskScheduler)
    +    val taskSetManager12 = createTaskSetManager(2, 2, taskScheduler)
    +    schedulableBuilder.addTaskSetManager(taskSetManager10, properties1)
    +    schedulableBuilder.addTaskSetManager(taskSetManager11, properties1)
    +    schedulableBuilder.addTaskSetManager(taskSetManager12, properties1)
    +
    +    val taskSetManager23 = createTaskSetManager(3, 2, taskScheduler)
    +    val taskSetManager24 = createTaskSetManager(4, 2, taskScheduler)
    +    schedulableBuilder.addTaskSetManager(taskSetManager23, properties2)
    +    schedulableBuilder.addTaskSetManager(taskSetManager24, properties2)
    +
    +    // Pool 1 share ratio: 0. Pool 2 share ratio: 0. 1 gets scheduled based on ordering of names.
    +    scheduleTaskAndVerifyId(0, rootPool, 0)
    +    // Pool 1 share ratio: 1/2. Pool 2 share ratio: 0. 2 gets scheduled because ratio is lower.
    +    scheduleTaskAndVerifyId(1, rootPool, 3)
    +    // Pool 1 share ratio: 1/2. Pool 2 share ratio: 1/3. 2 gets scheduled because ratio is lower.
    +    scheduleTaskAndVerifyId(2, rootPool, 3)
    +    // Pool 1 share ratio: 1/2. Pool 2 share ratio: 2/3. 1 gets scheduled because ratio is lower.
    +    scheduleTaskAndVerifyId(3, rootPool, 1)
    +    // Pool 1 share ratio: 1. Pool 2 share ratio: 2/3. 2 gets scheduled because ratio is lower.
    +    scheduleTaskAndVerifyId(4, rootPool, 4)
    +    // Neither pool is needy so ordering is based on number of running tasks.
    +    // Pool 1 running tasks: 2, Pool 2 running tasks: 3. 1 gets scheduled because fewer running
    +    // tasks.
    +    scheduleTaskAndVerifyId(5, rootPool, 2)
    +    // Pool 1 running tasks: 3, Pool 2 running tasks: 3. 1 gets scheduled because of naming
    +    // ordering.
    +    scheduleTaskAndVerifyId(6, rootPool, 2)
    +    // Pool 1 running tasks: 4, Pool 2 running tasks: 3. 2 gets scheduled because fewer running
    +    // tasks.
    +    scheduleTaskAndVerifyId(7, rootPool, 4)
    +  }
    +
    +  test("Nested Pool Test") {
    +    sc = new SparkContext("local", "TaskSchedulerImplSuite")
    +    val taskScheduler = new TaskSchedulerImpl(sc)
    +
    +    val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
    +    val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1)
    +    val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1)
    +    rootPool.addSchedulable(pool0)
    +    rootPool.addSchedulable(pool1)
    +
    +    val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2)
    +    val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1)
    +    pool0.addSchedulable(pool00)
    +    pool0.addSchedulable(pool01)
    +
    +    val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2)
    +    val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1)
    +    pool1.addSchedulable(pool10)
    +    pool1.addSchedulable(pool11)
    +
    +    val taskSetManager000 = createTaskSetManager(0, 5, taskScheduler)
    +    val taskSetManager001 = createTaskSetManager(1, 5, taskScheduler)
    +    pool00.addSchedulable(taskSetManager000)
    +    pool00.addSchedulable(taskSetManager001)
    +
    +    val taskSetManager010 = createTaskSetManager(2, 5, taskScheduler)
    +    val taskSetManager011 = createTaskSetManager(3, 5, taskScheduler)
    +    pool01.addSchedulable(taskSetManager010)
    +    pool01.addSchedulable(taskSetManager011)
    +
    +    val taskSetManager100 = createTaskSetManager(4, 5, taskScheduler)
    +    val taskSetManager101 = createTaskSetManager(5, 5, taskScheduler)
    +    pool10.addSchedulable(taskSetManager100)
    +    pool10.addSchedulable(taskSetManager101)
    +
    +    val taskSetManager110 = createTaskSetManager(6, 5, taskScheduler)
    +    val taskSetManager111 = createTaskSetManager(7, 5, taskScheduler)
    +    pool11.addSchedulable(taskSetManager110)
    +    pool11.addSchedulable(taskSetManager111)
    +
    +    scheduleTaskAndVerifyId(0, rootPool, 0)
    +    scheduleTaskAndVerifyId(1, rootPool, 4)
    +    scheduleTaskAndVerifyId(2, rootPool, 6)
    +    scheduleTaskAndVerifyId(3, rootPool, 2)
    +  }
    +}
    diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
    index 00812e6018d1f..8874cf00e9993 100644
    --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
    +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
    @@ -30,238 +30,8 @@ class FakeSchedulerBackend extends SchedulerBackend {
       def defaultParallelism() = 1
     }
     
    -class FakeTaskSetManager(
    -    initPriority: Int,
    -    initStageId: Int,
    -    initNumTasks: Int,
    -    taskScheduler: TaskSchedulerImpl,
    -    taskSet: TaskSet)
    -  extends TaskSetManager(taskScheduler, taskSet, 0) {
    -
    -  parent = null
    -  weight = 1
    -  minShare = 2
    -  priority = initPriority
    -  stageId = initStageId
    -  name = "TaskSet_"+stageId
    -  override val numTasks = initNumTasks
    -  tasksSuccessful = 0
    -
    -  var numRunningTasks = 0
    -  override def runningTasks = numRunningTasks
    -
    -  def increaseRunningTasks(taskNum: Int) {
    -    numRunningTasks += taskNum
    -    if (parent != null) {
    -      parent.increaseRunningTasks(taskNum)
    -    }
    -  }
    -
    -  def decreaseRunningTasks(taskNum: Int) {
    -    numRunningTasks -= taskNum
    -    if (parent != null) {
    -      parent.decreaseRunningTasks(taskNum)
    -    }
    -  }
    -
    -  override def addSchedulable(schedulable: Schedulable) {
    -  }
    -
    -  override def removeSchedulable(schedulable: Schedulable) {
    -  }
    -
    -  override def getSchedulableByName(name: String): Schedulable = {
    -    null
    -  }
    -
    -  override def executorLost(executorId: String, host: String): Unit = {
    -  }
    -
    -  override def resourceOffer(
    -      execId: String,
    -      host: String,
    -      maxLocality: TaskLocality.TaskLocality)
    -    : Option[TaskDescription] =
    -  {
    -    if (tasksSuccessful + numRunningTasks < numTasks) {
    -      increaseRunningTasks(1)
    -      Some(new TaskDescription(0, execId, "task 0:0", 0, null))
    -    } else {
    -      None
    -    }
    -  }
    -
    -  override def checkSpeculatableTasks(): Boolean = {
    -    true
    -  }
    -
    -  def taskFinished() {
    -    decreaseRunningTasks(1)
    -    tasksSuccessful +=1
    -    if (tasksSuccessful == numTasks) {
    -      parent.removeSchedulable(this)
    -    }
    -  }
    -
    -  def abort() {
    -    decreaseRunningTasks(numRunningTasks)
    -    parent.removeSchedulable(this)
    -  }
    -}
    -
     class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging {
     
    -  def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl,
    -      taskSet: TaskSet): FakeTaskSetManager = {
    -    new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet)
    -  }
    -
    -  def resourceOffer(rootPool: Pool): Int = {
    -    val taskSetQueue = rootPool.getSortedTaskSetQueue
    -    /* Just for Test*/
    -    for (manager <- taskSetQueue) {
    -       logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(
    -         manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks))
    -    }
    -    for (taskSet <- taskSetQueue) {
    -      taskSet.resourceOffer("execId_1", "hostname_1", TaskLocality.ANY) match {
    -        case Some(task) =>
    -          return taskSet.stageId
    -        case None => {}
    -      }
    -    }
    -    -1
    -  }
    -
    -  def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) {
    -    assert(resourceOffer(rootPool) === expectedTaskSetId)
    -  }
    -
    -  test("FIFO Scheduler Test") {
    -    sc = new SparkContext("local", "TaskSchedulerImplSuite")
    -    val taskScheduler = new TaskSchedulerImpl(sc)
    -    val taskSet = FakeTask.createTaskSet(1)
    -
    -    val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0)
    -    val schedulableBuilder = new FIFOSchedulableBuilder(rootPool)
    -    schedulableBuilder.buildPools()
    -
    -    val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet)
    -    val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet)
    -    val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet)
    -    schedulableBuilder.addTaskSetManager(taskSetManager0, null)
    -    schedulableBuilder.addTaskSetManager(taskSetManager1, null)
    -    schedulableBuilder.addTaskSetManager(taskSetManager2, null)
    -
    -    checkTaskSetId(rootPool, 0)
    -    resourceOffer(rootPool)
    -    checkTaskSetId(rootPool, 1)
    -    resourceOffer(rootPool)
    -    taskSetManager1.abort()
    -    checkTaskSetId(rootPool, 2)
    -  }
    -
    -  test("Fair Scheduler Test") {
    -    val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
    -    val conf = new SparkConf().set("spark.scheduler.allocation.file", xmlPath)
    -    sc = new SparkContext("local", "TaskSchedulerImplSuite", conf)
    -    val taskScheduler = new TaskSchedulerImpl(sc)
    -    val taskSet = FakeTask.createTaskSet(1)
    -
    -    val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
    -    val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf)
    -    schedulableBuilder.buildPools()
    -
    -    assert(rootPool.getSchedulableByName("default") != null)
    -    assert(rootPool.getSchedulableByName("1") != null)
    -    assert(rootPool.getSchedulableByName("2") != null)
    -    assert(rootPool.getSchedulableByName("3") != null)
    -    assert(rootPool.getSchedulableByName("1").minShare === 2)
    -    assert(rootPool.getSchedulableByName("1").weight === 1)
    -    assert(rootPool.getSchedulableByName("2").minShare === 3)
    -    assert(rootPool.getSchedulableByName("2").weight === 1)
    -    assert(rootPool.getSchedulableByName("3").minShare === 0)
    -    assert(rootPool.getSchedulableByName("3").weight === 1)
    -
    -    val properties1 = new Properties()
    -    properties1.setProperty("spark.scheduler.pool","1")
    -    val properties2 = new Properties()
    -    properties2.setProperty("spark.scheduler.pool","2")
    -
    -    val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet)
    -    val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet)
    -    val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet)
    -    schedulableBuilder.addTaskSetManager(taskSetManager10, properties1)
    -    schedulableBuilder.addTaskSetManager(taskSetManager11, properties1)
    -    schedulableBuilder.addTaskSetManager(taskSetManager12, properties1)
    -
    -    val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet)
    -    val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet)
    -    schedulableBuilder.addTaskSetManager(taskSetManager23, properties2)
    -    schedulableBuilder.addTaskSetManager(taskSetManager24, properties2)
    -
    -    checkTaskSetId(rootPool, 0)
    -    checkTaskSetId(rootPool, 3)
    -    checkTaskSetId(rootPool, 3)
    -    checkTaskSetId(rootPool, 1)
    -    checkTaskSetId(rootPool, 4)
    -    checkTaskSetId(rootPool, 2)
    -    checkTaskSetId(rootPool, 2)
    -    checkTaskSetId(rootPool, 4)
    -
    -    taskSetManager12.taskFinished()
    -    assert(rootPool.getSchedulableByName("1").runningTasks === 3)
    -    taskSetManager24.abort()
    -    assert(rootPool.getSchedulableByName("2").runningTasks === 2)
    -  }
    -
    -  test("Nested Pool Test") {
    -    sc = new SparkContext("local", "TaskSchedulerImplSuite")
    -    val taskScheduler = new TaskSchedulerImpl(sc)
    -    val taskSet = FakeTask.createTaskSet(1)
    -
    -    val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
    -    val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1)
    -    val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1)
    -    rootPool.addSchedulable(pool0)
    -    rootPool.addSchedulable(pool1)
    -
    -    val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2)
    -    val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1)
    -    pool0.addSchedulable(pool00)
    -    pool0.addSchedulable(pool01)
    -
    -    val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2)
    -    val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1)
    -    pool1.addSchedulable(pool10)
    -    pool1.addSchedulable(pool11)
    -
    -    val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet)
    -    val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet)
    -    pool00.addSchedulable(taskSetManager000)
    -    pool00.addSchedulable(taskSetManager001)
    -
    -    val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet)
    -    val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet)
    -    pool01.addSchedulable(taskSetManager010)
    -    pool01.addSchedulable(taskSetManager011)
    -
    -    val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet)
    -    val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet)
    -    pool10.addSchedulable(taskSetManager100)
    -    pool10.addSchedulable(taskSetManager101)
    -
    -    val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet)
    -    val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet)
    -    pool11.addSchedulable(taskSetManager110)
    -    pool11.addSchedulable(taskSetManager111)
    -
    -    checkTaskSetId(rootPool, 0)
    -    checkTaskSetId(rootPool, 4)
    -    checkTaskSetId(rootPool, 6)
    -    checkTaskSetId(rootPool, 2)
    -  }
    -
       test("Scheduler does not always schedule tasks on the same workers") {
         sc = new SparkContext("local", "TaskSchedulerImplSuite")
         val taskScheduler = new TaskSchedulerImpl(sc)
    
    From e9ca16ec943b9553056482d0c085eacb6046821e Mon Sep 17 00:00:00 2001
    From: Liang-Chi Hsieh 
    Date: Fri, 9 Jan 2015 10:27:33 -0800
    Subject: [PATCH 227/227] [SPARK-5145][Mllib] Add BLAS.dsyr and use it in
     GaussianMixtureEM
    
    This pr uses BLAS.dsyr to replace few implementations in GaussianMixtureEM.
    
    Author: Liang-Chi Hsieh 
    
    Closes #3949 from viirya/blas_dsyr and squashes the following commits:
    
    4e4d6cf [Liang-Chi Hsieh] Add unit test. Rename function name, modify doc and style.
    3f57fd2 [Liang-Chi Hsieh] Add BLAS.dsyr and use it in GaussianMixtureEM.
    ---
     .../mllib/clustering/GaussianMixtureEM.scala  | 10 +++--
     .../org/apache/spark/mllib/linalg/BLAS.scala  | 26 ++++++++++++
     .../apache/spark/mllib/linalg/BLASSuite.scala | 41 +++++++++++++++++++
     3 files changed, 73 insertions(+), 4 deletions(-)
    
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala
    index bdf984aee4dae..3a6c0e681e3fa 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala
    @@ -21,7 +21,7 @@ import scala.collection.mutable.IndexedSeq
     
     import breeze.linalg.{DenseVector => BreezeVector, DenseMatrix => BreezeMatrix, diag, Transpose}
     import org.apache.spark.rdd.RDD
    -import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors}
    +import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors, DenseVector, DenseMatrix, BLAS}
     import org.apache.spark.mllib.stat.impl.MultivariateGaussian
     import org.apache.spark.mllib.util.MLUtils
     
    @@ -151,9 +151,10 @@ class GaussianMixtureEM private (
           var i = 0
           while (i < k) {
             val mu = sums.means(i) / sums.weights(i)
    -        val sigma = sums.sigmas(i) / sums.weights(i) - mu * new Transpose(mu) // TODO: Use BLAS.dsyr
    +        BLAS.syr(-sums.weights(i), Vectors.fromBreeze(mu).asInstanceOf[DenseVector],
    +          Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix])
             weights(i) = sums.weights(i) / sumWeights
    -        gaussians(i) = new MultivariateGaussian(mu, sigma)
    +        gaussians(i) = new MultivariateGaussian(mu, sums.sigmas(i) / sums.weights(i))
             i = i + 1
           }
        
    @@ -211,7 +212,8 @@ private object ExpectationSum {
           p(i) /= pSum
           sums.weights(i) += p(i)
           sums.means(i) += x * p(i)
    -      sums.sigmas(i) += xxt * p(i) // TODO: use BLAS.dsyr
    +      BLAS.syr(p(i), Vectors.fromBreeze(x).asInstanceOf[DenseVector],
    +        Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix])
           i = i + 1
         }
         sums
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
    index 9fed513becddc..3414daccd7ca4 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
    @@ -228,6 +228,32 @@ private[spark] object BLAS extends Serializable with Logging {
         }
         _nativeBLAS
       }
    + 
    +  /**
    +   * A := alpha * x * x^T^ + A
    +   * @param alpha a real scalar that will be multiplied to x * x^T^.
    +   * @param x the vector x that contains the n elements.
    +   * @param A the symmetric matrix A. Size of n x n.
    +   */
    +  def syr(alpha: Double, x: DenseVector, A: DenseMatrix) {
    +    val mA = A.numRows
    +    val nA = A.numCols
    +    require(mA == nA, s"A is not a symmetric matrix. A: $mA x $nA")
    +    require(mA == x.size, s"The size of x doesn't match the rank of A. A: $mA x $nA, x: ${x.size}")
    +
    +    nativeBLAS.dsyr("U", x.size, alpha, x.values, 1, A.values, nA)
    +
    +    // Fill lower triangular part of A
    +    var i = 0
    +    while (i < mA) {
    +      var j = i + 1
    +      while (j < nA) {
    +        A(j, i) = A(i, j)
    +        j += 1
    +      }
    +      i += 1
    +    }    
    +  }
     
       /**
        * C := alpha * A * B + beta * C
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala
    index 5d70c914f14b0..771878e925ea7 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala
    @@ -127,6 +127,47 @@ class BLASSuite extends FunSuite {
         }
       }
     
    +  test("syr") {
    +    val dA = new DenseMatrix(4, 4,
    +      Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0, 3.1, 4.6, 3.0, 0.8))
    +    val x = new DenseVector(Array(0.0, 2.7, 3.5, 2.1))
    +    val alpha = 0.15
    +
    +    val expected = new DenseMatrix(4, 4,
    +      Array(0.0, 1.2, 2.2, 3.1, 1.2, 4.2935, 6.7175, 5.4505, 2.2, 6.7175, 3.6375, 4.1025, 3.1,
    +        5.4505, 4.1025, 1.4615))
    +
    +    syr(alpha, x, dA)
    +
    +    assert(dA ~== expected absTol 1e-15)
    + 
    +    val dB =
    +      new DenseMatrix(3, 4, Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0))
    +
    +    withClue("Matrix A must be a symmetric Matrix") {
    +      intercept[Exception] {
    +        syr(alpha, x, dB)
    +      }
    +    }
    + 
    +    val dC =
    +      new DenseMatrix(3, 3, Array(0.0, 1.2, 2.2, 1.2, 3.2, 5.3, 2.2, 5.3, 1.8))
    +
    +    withClue("Size of vector must match the rank of matrix") {
    +      intercept[Exception] {
    +        syr(alpha, x, dC)
    +      }
    +    }
    + 
    +    val y = new DenseVector(Array(0.0, 2.7, 3.5, 2.1, 1.5))
    +
    +    withClue("Size of vector must match the rank of matrix") {
    +      intercept[Exception] {
    +        syr(alpha, y, dA)
    +      }
    +    }
    +  }
    +
       test("gemm") {
     
         val dA =
    
    spark.ui.retainedJobs 1000 - How many stages the Spark UI and status APIs remember before garbage + How many jobs the Spark UI and status APIs remember before garbage collecting.
    If set to true, validates the output specification (e.g. checking if the output directory already exists) used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing output directories. We recommend that users do not disable this except if trying to achieve compatibility with - previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand.
    spark.hadoop.cloneConf
    spark.network.timeout100 + Default timeout for all network interactions, in seconds. This config will be used in + place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, + if they are not configured. +
    spark.akka.heartbeat.pauses 6000
    spark.network.timeout100120 Default timeout for all network interactions, in seconds. This config will be used in place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, - spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, - if they are not configured. + spark.storage.blockManagerSlaveTimeoutMs or + spark.shuffle.io.connectionTimeout, if they are not configured.