From 85ee0cabe87a27b6947c2d3e8525f04c77f80f6f Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 14:29:07 -0700 Subject: [PATCH 001/144] [SPARK-6130] [SQL] support if not exists for insert overwrite into partition in hiveQl MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Standard syntax: INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 FROM from_statement; INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 FROM from_statement;   Hive extension (multiple inserts): FROM from_statement INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 [INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] [INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] ...; FROM from_statement INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 [INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] [INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] ...;   Hive extension (dynamic partition inserts): INSERT OVERWRITE TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement; INSERT INTO TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement; Author: Daoyuan Wang Closes #4865 from adrian-wang/insertoverwrite and squashes the following commits: 2fce94f [Daoyuan Wang] add assert 10ea6f3 [Daoyuan Wang] add name for boolean parameter 0bbe9b9 [Daoyuan Wang] fix failure 4391154 [Daoyuan Wang] support if not exists for insert overwrite into partition in hiveQl --- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../plans/logical/basicOperators.scala | 4 ++- .../org/apache/spark/sql/DataFrame.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../sql/sources/DataSourceStrategy.scala | 2 +- .../org/apache/spark/sql/sources/rules.scala | 7 +++-- .../execution/HiveCompatibilitySuite.scala | 1 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 19 ++++++------ .../org/apache/spark/sql/hive/HiveQl.scala | 22 ++++++++++++- .../spark/sql/hive/HiveStrategies.scala | 10 +++--- .../hive/execution/CreateTableAsSelect.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 31 +++++++++++++------ ...titions-0-d5edc0daa94b33915df794df3b710774 | 0 ...titions-1-9eb9372f4855928fae16f5fa554b3a62 | 0 ...itions-10-ec2cef3d37146c450c60202a572f5cab | 0 ...itions-11-8854d6001200fc11529b2e2da755e5a2 | 0 ...itions-12-71ff68fda0aa7a36cb50d8fab0d70d25 | 0 ...titions-13-7e4e7d7003fc6ef17bc19c3461ad899 | 0 ...itions-14-ec2cef3d37146c450c60202a572f5cab | 0 ...itions-15-a3b2e230efde74e970ae8a3b55f383fc | 0 ...titions-2-8396c17a66e3d9a374d4361873b9bfe3 | 0 ...titions-3-3876bb356dd8af7e78d061093d555457 | 0 ...rtitions-4-528e23afb272c2e69004c86ddaa70ee | 0 ...titions-5-de5d56456c28d63775554e56355911d2 | 0 ...titions-6-3efdc331b3b4bdac3e60c757600fff53 | 5 +++ ...titions-7-92f6af82704504968de078c133f222f8 | 0 ...titions-8-316cad7c63ddd4fb043be2affa5b0a67 | 0 ...titions-9-3efdc331b3b4bdac3e60c757600fff53 | 5 +++ 30 files changed, 84 insertions(+), 35 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 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 ee04cb579deb6..bc8d3751f6616 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -155,7 +155,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected lazy val insert: Parser[LogicalPlan] = INSERT ~> (OVERWRITE ^^^ true | INTO ^^^ false) ~ (TABLE ~> relation) ~ select ^^ { - case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o) + case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o, false) } protected lazy val cte: Parser[LogicalPlan] = 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 50702ac6832ec..8b68b0df35f48 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 @@ -193,7 +193,7 @@ class Analyzer( } realPlan transform { - case i@InsertIntoTable(u: UnresolvedRelation, _, _, _) => + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => i.copy( table = EliminateSubQueries(getTable(u, cteRelations))) case u: UnresolvedRelation => 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 145f062dd6817..21c15ad14fd19 100644 --- 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 @@ -293,7 +293,7 @@ package object dsl { def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan = InsertIntoTable( - analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite) + analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite, false) def analyze: LogicalPlan = EliminateSubQueries(analysis.SimpleAnalyzer(logicalPlan)) } 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 5d31a6eecfce2..17522976dc2c9 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 @@ -125,12 +125,14 @@ case class InsertIntoTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = child.output + assert(overwrite || !ifNotExists) override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { case (childAttr, tableAttr) => DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 9b9adf855077a..94ae2d65fd0e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1209,7 +1209,7 @@ class DataFrame private[sql]( @Experimental def insertInto(tableName: String, overwrite: Boolean): Unit = { sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite)).toRdd + Map.empty, logicalPlan, overwrite, ifNotExists = false)).toRdd } /** 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 23f7e5609414b..5268b7334051a 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 @@ -211,7 +211,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) // Note: overwrite=false because otherwise the metadata we just created will be deleted InsertIntoParquetTable(relation, planLater(child), overwrite = false) :: Nil - case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: ParquetRelation, partition, child, overwrite, ifNotExists) => InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index e13759b7feb7b..34d048e426d10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -56,7 +56,7 @@ private[sql] object DataSourceStrategy extends Strategy { execution.PhysicalRDD(l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), part, query, overwrite) if part.isEmpty => + l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 5a78001117d1b..6ed68d179edc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -37,7 +37,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // We are inserting into an InsertableRelation. case i @ InsertIntoTable( - l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite) => { + l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite, ifNotExists) => { // First, make sure the data to be inserted have the same number of fields with the // schema of the relation. if (l.output.size != child.output.size) { @@ -84,7 +84,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => def apply(plan: LogicalPlan): Unit = { plan.foreach { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite) => + l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { failAnalysis(s"Insert into a partition is not allowed because $l is not partitioned.") @@ -102,7 +102,8 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } case i @ logical.InsertIntoTable( - l: LogicalRelation, partition, query, overwrite) if !l.isInstanceOf[InsertableRelation] => + l: LogicalRelation, partition, query, overwrite, ifNotExists) + if !l.isInstanceOf[InsertableRelation] => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") 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 2ae9d018e1b1b..81ee48ef4152f 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 @@ -532,6 +532,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inputddl7", "inputddl8", "insert1", + "insert1_overwrite_partitions", "insert2_overwrite_partitions", "insert_compressed", "join0", 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 3ed5c5b031736..f1c0bd92aa23d 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 @@ -527,7 +527,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Collects all `MetastoreRelation`s which should be replaced val toBeReplaced = plan.collect { // Write path - case InsertIntoTable(relation: MetastoreRelation, _, _, _) + case InsertIntoTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -538,7 +538,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with (relation, parquetRelation, attributedRewrites) // Write path - case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _) + case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -569,15 +569,15 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val alias = r.alias.getOrElse(r.tableName) Subquery(alias, parquetRelation) - case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite) + case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) - case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite) + case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) case other => other.transformExpressions { case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) @@ -698,7 +698,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _, _) => castChildOutput(p, table, child) } @@ -715,7 +715,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with .forall { case (left, right) => left.sameType(right) }) { // If both types ignoring nullability of ArrayType, MapType, StructType are the same, // use InsertIntoHiveTable instead of InsertIntoTable. - InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite) + InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite, p.ifNotExists) } else { // Only do the casting when child output data types differ from table output data types. val castedChildOutput = child.output.zip(table.output).map { @@ -753,7 +753,8 @@ private[hive] case class InsertIntoHiveTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil 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 b2ae74efeb097..53a204b8c2932 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 @@ -1002,7 +1002,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite) + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, false) + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: + Token("TOK_IFNOTEXISTS", + ifNotExists) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.getChildren.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") 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 5f7e897295117..1ccb0c279c60e 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 @@ -184,12 +184,14 @@ private[hive] trait HiveStrategies { object DataSinks extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil - case hive.InsertIntoHiveTable(table: MetastoreRelation, partition, child, overwrite) => + table, partition, planLater(child), overwrite, ifNotExists) :: Nil + case hive.InsertIntoHiveTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil + table, partition, planLater(child), overwrite, ifNotExists) :: 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 fade9e5852eaa..76a1965f3cb25 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 @@ -67,7 +67,7 @@ case class CreateTableAsSelect( new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName") } } else { - hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd + hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd } Seq.empty[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 6c96747439683..89995a91b1a92 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 @@ -45,7 +45,8 @@ case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], child: SparkPlan, - overwrite: Boolean) extends UnaryNode with HiveInspectors { + overwrite: Boolean, + ifNotExists: Boolean) extends UnaryNode with HiveInspectors { @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @@ -219,15 +220,25 @@ case class InsertIntoHiveTable( isSkewedStoreAsSubdir) } } else { - catalog.synchronized { - catalog.client.loadPartition( - outputPath, - qualifiedTableName, - orderedPartitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + // scalastyle:off + // ifNotExists is only valid with static partition, refer to + // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries + // scalastyle:on + val oldPart = catalog.synchronized { + catalog.client.getPartition( + catalog.client.getTable(qualifiedTableName), partitionSpec, false) + } + if (oldPart == null || !ifNotExists) { + catalog.synchronized { + catalog.client.loadPartition( + outputPath, + qualifiedTableName, + orderedPartitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } } } } else { diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 From 3a205bbd9e352668a020c3146391e1e4441467af Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Mon, 13 Apr 2015 14:43:07 -0700 Subject: [PATCH 002/144] [SQL][SPARK-6742]: Don't push down predicates which reference partition column(s) cc liancheng Author: Yash Datta Closes #5390 from saucam/fpush and squashes the following commits: 3f026d6 [Yash Datta] SPARK-6742: Fix scalastyle ce3d702 [Yash Datta] SPARK-6742: Add test case, fix scalastyle 8592acc [Yash Datta] SPARK-6742: Don't push down predicates which reference partition column(s) --- .../spark/sql/execution/SparkStrategies.scala | 11 ++++++++- .../sql/parquet/ParquetFilterSuite.scala | 24 ++++++++++++++++++- 2 files changed, 33 insertions(+), 2 deletions(-) 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 5268b7334051a..f0d92ffffcda3 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 @@ -215,6 +215,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { table: ParquetRelation, partition, child, overwrite, ifNotExists) => InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => + val partitionColNames = relation.partitioningAttributes.map(_.name).toSet + val filtersToPush = filters.filter { pred => + val referencedColNames = pred.references.map(_.name).toSet + referencedColNames.intersect(partitionColNames).isEmpty + } val prunePushedDownFilters = if (sqlContext.conf.parquetFilterPushDown) { (predicates: Seq[Expression]) => { @@ -226,6 +231,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // "A AND B" in the higher-level filter, not just "B". predicates.map(p => p -> ParquetFilters.createFilter(p)).collect { case (predicate, None) => predicate + // Filter needs to be applied above when it contains partitioning + // columns + case (predicate, _) if(!predicate.references.map(_.name).toSet + .intersect (partitionColNames).isEmpty) => predicate } } } else { @@ -238,7 +247,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetTableScan( _, relation, - if (sqlContext.conf.parquetFilterPushDown) filters else Nil)) :: Nil + if (sqlContext.conf.parquetFilterPushDown) filtersToPush else Nil)) :: Nil case _ => Nil } 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 6a2c2a7c4080a..10d0ede4dc0dc 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 @@ -22,7 +22,7 @@ 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.{Attribute, Cast, Literal, Predicate, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.test.TestSQLContext @@ -350,4 +350,26 @@ class ParquetDataSourceOffFilterSuite extends ParquetFilterSuiteBase with Before override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6742: don't push down predicates which reference partition columns") { + import sqlContext.implicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part=1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").saveAsParquetFile(path) + + // If the "part = 1" filter gets pushed down, this query will throw an exception since + // "part" is not a valid column in the actual Parquet file + val df = DataFrame(sqlContext, org.apache.spark.sql.parquet.ParquetRelation( + path, + Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext, + Seq(AttributeReference("part", IntegerType, false)()) )) + + checkAnswer( + df.filter("a = 1 or part = 1"), + (1 to 3).map(i => Row(1, i, i.toString))) + } + } + } } From 2a55cb41bf7da1786be2c76b8af398da8fedb44b Mon Sep 17 00:00:00 2001 From: MechCoder Date: Mon, 13 Apr 2015 15:36:33 -0700 Subject: [PATCH 003/144] [SPARK-5972] [MLlib] Cache residuals and gradient in GBT during training and validation The previous PR https://github.com/apache/spark/pull/4906 helped to extract the learning curve giving the error for each iteration. This continues the work refactoring some code and extending the same logic during training and validation. Author: MechCoder Closes #5330 from MechCoder/spark-5972 and squashes the following commits: 0b5d659 [MechCoder] minor 32d409d [MechCoder] EvaluateeachIteration and training cache should follow different paths d542bb0 [MechCoder] Remove unused imports and docs 58f4932 [MechCoder] Remove unpersist 70d3b4c [MechCoder] Broadcast for each tree 5869533 [MechCoder] Access broadcasted values locally and other minor changes 923dbf6 [MechCoder] [SPARK-5972] Cache residuals and gradient in GBT during training and validation --- .../mllib/tree/GradientBoostedTrees.scala | 42 ++++++---- .../spark/mllib/tree/loss/AbsoluteError.scala | 10 +-- .../spark/mllib/tree/loss/LogLoss.scala | 11 +-- .../apache/spark/mllib/tree/loss/Loss.scala | 8 +- .../spark/mllib/tree/loss/SquaredError.scala | 10 +-- .../mllib/tree/model/treeEnsembleModels.scala | 77 +++++++++++++++---- 6 files changed, 105 insertions(+), 53 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index a9c93e181e3ce..c02c79f094b66 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -157,7 +157,6 @@ object GradientBoostedTrees extends Logging { validationInput: RDD[LabeledPoint], boostingStrategy: BoostingStrategy, validate: Boolean): GradientBoostedTreesModel = { - val timer = new TimeTracker() timer.start("total") timer.start("init") @@ -192,20 +191,29 @@ object GradientBoostedTrees extends Logging { // Initialize tree timer.start("building tree 0") val firstTreeModel = new DecisionTree(treeStrategy).run(data) + val firstTreeWeight = 1.0 baseLearners(0) = firstTreeModel - baseLearnerWeights(0) = 1.0 - val startingModel = new GradientBoostedTreesModel(Regression, Array(firstTreeModel), Array(1.0)) - logDebug("error of gbt = " + loss.computeError(startingModel, input)) + baseLearnerWeights(0) = firstTreeWeight + val startingModel = new GradientBoostedTreesModel( + Regression, Array(firstTreeModel), baseLearnerWeights.slice(0, 1)) + + var predError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) + logDebug("error of gbt = " + predError.values.mean()) // Note: A model of type regression is used since we require raw prediction timer.stop("building tree 0") - var bestValidateError = if (validate) loss.computeError(startingModel, validationInput) else 0.0 + var validatePredError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(validationInput, firstTreeWeight, firstTreeModel, loss) + var bestValidateError = if (validate) validatePredError.values.mean() else 0.0 var bestM = 1 - // psuedo-residual for second iteration - data = input.map(point => LabeledPoint(loss.gradient(startingModel, point), - point.features)) + // pseudo-residual for second iteration + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } + var m = 1 while (m < numIterations) { timer.start(s"building tree $m") @@ -222,15 +230,22 @@ object GradientBoostedTrees extends Logging { baseLearnerWeights(m) = learningRate // Note: A model of type regression is used since we require raw prediction val partialModel = new GradientBoostedTreesModel( - Regression, baseLearners.slice(0, m + 1), baseLearnerWeights.slice(0, m + 1)) - logDebug("error of gbt = " + loss.computeError(partialModel, input)) + Regression, baseLearners.slice(0, m + 1), + baseLearnerWeights.slice(0, m + 1)) + + predError = GradientBoostedTreesModel.updatePredictionError( + input, predError, baseLearnerWeights(m), baseLearners(m), loss) + logDebug("error of gbt = " + predError.values.mean()) if (validate) { // Stop training early if // 1. Reduction in error is less than the validationTol or // 2. If the error increases, that is if the model is overfit. // We want the model returned corresponding to the best validation error. - val currentValidateError = loss.computeError(partialModel, validationInput) + + validatePredError = GradientBoostedTreesModel.updatePredictionError( + validationInput, validatePredError, baseLearnerWeights(m), baseLearners(m), loss) + val currentValidateError = validatePredError.values.mean() if (bestValidateError - currentValidateError < validationTol) { return new GradientBoostedTreesModel( boostingStrategy.treeStrategy.algo, @@ -242,8 +257,9 @@ object GradientBoostedTrees extends Logging { } } // Update data with pseudo-residuals - data = input.map(point => LabeledPoint(-loss.gradient(partialModel, point), - point.features)) + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } m += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index 793dd664c5d5a..6f570b4e09c79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -37,14 +37,12 @@ object AbsoluteError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * absolute error calculation. * The gradient with respect to F(x) is: sign(F(x) - y) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - if ((point.label - model.predict(point.features)) < 0) 1.0 else -1.0 + override def gradient(prediction: Double, label: Double): Double = { + if (label - prediction < 0) 1.0 else -1.0 } override def computeError(prediction: Double, label: Double): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 51b1aed167b66..24ee9f3d51293 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -39,15 +39,12 @@ object LogLoss extends Loss { * Method to calculate the loss gradients for the gradient boosting calculation for binary * classification * The gradient with respect to F(x) is: - 4 y / (1 + exp(2 y F(x))) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - val prediction = model.predict(point.features) - - 4.0 * point.label / (1.0 + math.exp(2.0 * point.label * prediction)) + override def gradient(prediction: Double, label: Double): Double = { + - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } override def computeError(prediction: Double, label: Double): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index 357869ff6b333..d3b82b752fa0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -31,13 +31,11 @@ trait Loss extends Serializable { /** * Method to calculate the gradients for the gradient boosting calculation. - * @param model Model of the weak learner. - * @param point Instance of the training dataset. + * @param prediction Predicted feature + * @param label true label. * @return Loss gradient. */ - def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double + def gradient(prediction: Double, label: Double): Double /** * Method to calculate error of the base learner for the gradient boosting calculation. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index b990707ca4525..58857ae15e93e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -37,14 +37,12 @@ object SquaredError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * squares error calculation. * The gradient with respect to F(x) is: - 2 (y - F(x)) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - 2.0 * (model.predict(point.features) - point.label) + override def gradient(prediction: Double, label: Double): Double = { + 2.0 * (prediction - label) } override def computeError(prediction: Double, label: Double): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index 1950254b2aa6d..fef3d2acb202a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -130,35 +130,28 @@ class GradientBoostedTreesModel( val numIterations = trees.length val evaluationArray = Array.fill(numIterations)(0.0) + val localTreeWeights = treeWeights + + var predictionAndError = GradientBoostedTreesModel.computeInitialPredictionAndError( + remappedData, localTreeWeights(0), trees(0), loss) - var predictionAndError: RDD[(Double, Double)] = remappedData.map { i => - val pred = treeWeights(0) * trees(0).predict(i.features) - val error = loss.computeError(pred, i.label) - (pred, error) - } evaluationArray(0) = predictionAndError.values.mean() - // Avoid the model being copied across numIterations. val broadcastTrees = sc.broadcast(trees) - val broadcastWeights = sc.broadcast(treeWeights) - (1 until numIterations).map { nTree => predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => val currentTree = broadcastTrees.value(nTree) - val currentTreeWeight = broadcastWeights.value(nTree) - iter.map { - case (point, (pred, error)) => { - val newPred = pred + currentTree.predict(point.features) * currentTreeWeight - val newError = loss.computeError(newPred, point.label) - (newPred, newError) - } + val currentTreeWeight = localTreeWeights(nTree) + iter.map { case (point, (pred, error)) => + val newPred = pred + currentTree.predict(point.features) * currentTreeWeight + val newError = loss.computeError(newPred, point.label) + (newPred, newError) } } evaluationArray(nTree) = predictionAndError.values.mean() } broadcastTrees.unpersist() - broadcastWeights.unpersist() evaluationArray } @@ -166,6 +159,58 @@ class GradientBoostedTreesModel( object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { + /** + * Compute the initial predictions and errors for a dataset for the first + * iteration of gradient boosting. + * @param data: training data. + * @param initTreeWeight: learning rate assigned to the first tree. + * @param initTree: first DecisionTreeModel. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to every sample. + */ + def computeInitialPredictionAndError( + data: RDD[LabeledPoint], + initTreeWeight: Double, + initTree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + data.map { lp => + val pred = initTreeWeight * initTree.predict(lp.features) + val error = loss.computeError(pred, lp.label) + (pred, error) + } + } + + /** + * Update a zipped predictionError RDD + * (as obtained with computeInitialPredictionAndError) + * @param data: training data. + * @param predictionAndError: predictionError RDD + * @param treeWeight: Learning rate. + * @param tree: Tree using which the prediction and error should be updated. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to each sample. + */ + def updatePredictionError( + data: RDD[LabeledPoint], + predictionAndError: RDD[(Double, Double)], + treeWeight: Double, + tree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + + val newPredError = data.zip(predictionAndError).mapPartitions { iter => + iter.map { + case (lp, (pred, error)) => { + val newPred = pred + tree.predict(lp.features) * treeWeight + val newError = loss.computeError(newPred, lp.label) + (newPred, newError) + } + } + } + newPredError + } + override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName From e63a86abe2794332cdad71d87b72a7c56327a43d Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 16:00:58 -0700 Subject: [PATCH 004/144] [SPARK-6872] [SQL] add copy in external sort We need add copy before call externalsort. Author: Daoyuan Wang Closes #5481 from adrian-wang/extsort and squashes the following commits: 9611586 [Daoyuan Wang] fix bug in external sort --- .../scala/org/apache/spark/sql/execution/basicOperators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 6eec520abff53..f8221f41bc6c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -194,7 +194,7 @@ case class ExternalSort( child.execute().mapPartitions( { iterator => val ordering = newOrdering(sortOrder, child.output) val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering)) - sorter.insertAll(iterator.map(r => (r, null))) + sorter.insertAll(iterator.map(r => (r.copy, null))) val baseIterator = sorter.iterator.map(_._1) // TODO(marmbrus): The complex type signature below thwarts inference for no reason. CompletionIterator[Row, Iterator[Row]](baseIterator, sorter.stop()) From c5602bdc310cc8f82dc304500bebe40217cba785 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 13 Apr 2015 16:02:18 -0700 Subject: [PATCH 005/144] [SPARK-5941] [SQL] Unit Test loads the table `src` twice for leftsemijoin.q In `leftsemijoin.q`, there is a data loading command for table `sales` already, but in `TestHive`, it also created the table `sales`, which causes duplicated records inserted into the `sales`. Author: Cheng Hao Closes #4506 from chenghao-intel/df_table and squashes the following commits: 0be05f7 [Cheng Hao] Remove the table `sales` creating from TestHive --- .../sql/columnar/InMemoryColumnarQuerySuite.scala | 2 +- .../org/apache/spark/sql/hive/test/TestHive.scala | 6 ------ ...ftsemijoin-10-89737a8857b5b61cc909e0c797f86aea | 2 -- .../leftsemijoin-8-73cad58a10a1483ccb15e94a857013 | 2 -- .../spark/sql/hive/execution/HiveSerDeSuite.scala | 15 +++++++++++---- 5 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 27dfabca90217..479210d1c9c43 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { .toDF().registerTempTable("sizeTst") cacheTable("sizeTst") assert( - table("sizeTst").queryExecution.logical.statistics.sizeInBytes > + table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > conf.autoBroadcastJoinThreshold) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index a3497eadd67f6..6570fa1043900 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -262,12 +262,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { |WITH SERDEPROPERTIES ('field.delim'='\\t') """.stripMargin.cmd, "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), - TestTable("sales", - s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) - |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' - |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales".cmd), TestTable("episodes", s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea +++ b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 +++ b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index d05e11fcf281b..5586a793618bd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -25,18 +25,25 @@ import org.apache.spark.sql.hive.test.TestHive * A set of tests that validates support for Hive SerDe. */ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { - override def beforeAll(): Unit = { + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + super.beforeAll() TestHive.cacheTables = false - super.beforeAll() + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales") } + // table sales is not a cache table, and will be clear after reset + createQueryTest("Read with RegexSerDe", "SELECT * FROM sales", false) + createQueryTest( "Read and write with LazySimpleSerDe (tab separated)", "SELECT * from serdeins") - createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") - createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") createQueryTest("Read Partitioned with AvroSerDe", "SELECT * FROM episodes_part") From c4ab255e94366ba9b9023d5431f9d2412e0d6dc7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 16:28:07 -0700 Subject: [PATCH 006/144] [SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin Author: Ilya Ganelin Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time --- .../spark/ExecutorAllocationManager.scala | 34 ++++---- .../org/apache/spark/HeartbeatReceiver.scala | 15 ++-- .../scala/org/apache/spark/SparkConf.scala | 36 ++++++++ .../org/apache/spark/executor/Executor.scala | 6 +- .../spark/network/nio/ConnectionManager.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 10 +-- .../spark/scheduler/TaskSetManager.scala | 21 +++-- .../CoarseGrainedSchedulerBackend.scala | 13 +-- .../org/apache/spark/util/AkkaUtils.scala | 14 +-- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 26 +++++- .../ExecutorAllocationManagerSuite.scala | 7 +- .../network/nio/ConnectionManagerSuite.scala | 8 +- .../spark/scheduler/TaskSetManagerSuite.scala | 25 +++--- .../BlockManagerReplicationSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 44 ++++++++++ docs/configuration.md | 86 ++++++++++--------- docs/running-on-yarn.md | 4 +- .../apache/spark/network/util/JavaUtils.java | 66 ++++++++++++++ .../spark/network/util/TransportConf.java | 15 +++- .../streaming/receiver/BlockGenerator.scala | 8 +- .../streaming/scheduler/JobGenerator.scala | 12 ++- .../spark/streaming/ReceiverSuite.scala | 14 +-- .../streaming/StreamingContextSuite.scala | 22 ++--- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +- 25 files changed, 345 insertions(+), 157 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 9385f557c4614..4e7bf51fc0622 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -80,16 +80,16 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.schedulerBacklogTimeout", 5) + private val schedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.schedulerBacklogTimeout", "5s") - // Same as above, but used only after `schedulerBacklogTimeout` is exceeded - private val sustainedSchedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) + // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded + private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s") // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeout = conf.getLong( - "spark.dynamicAllocation.executorIdleTimeout", 600) + private val executorIdleTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.executorIdleTimeout", "600s") // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) @@ -150,14 +150,14 @@ private[spark] class ExecutorAllocationManager( throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") } - if (schedulerBacklogTimeout <= 0) { + if (schedulerBacklogTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") } - if (sustainedSchedulerBacklogTimeout <= 0) { + if (sustainedSchedulerBacklogTimeoutS <= 0) { throw new SparkException( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") } - if (executorIdleTimeout <= 0) { + if (executorIdleTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!") } // Require external shuffle service for dynamic allocation @@ -262,8 +262,8 @@ private[spark] class ExecutorAllocationManager( } else if (addTime != NOT_SET && now >= addTime) { val delta = addExecutors(maxNeeded) logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeout seconds)") - addTime += sustainedSchedulerBacklogTimeout * 1000 + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") + addTime += sustainedSchedulerBacklogTimeoutS * 1000 delta } else { 0 @@ -351,7 +351,7 @@ private[spark] class ExecutorAllocationManager( 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})") + s"$executorIdleTimeoutS seconds (new desired total will be ${numExistingExecutors - 1})") executorsPendingToRemove.add(executorId) true } else { @@ -407,8 +407,8 @@ private[spark] class ExecutorAllocationManager( private def onSchedulerBacklogged(): Unit = synchronized { if (addTime == NOT_SET) { logDebug(s"Starting timer to add executors because pending tasks " + - s"are building up (to expire in $schedulerBacklogTimeout seconds)") - addTime = clock.getTimeMillis + schedulerBacklogTimeout * 1000 + s"are building up (to expire in $schedulerBacklogTimeoutS seconds)") + addTime = clock.getTimeMillis + schedulerBacklogTimeoutS * 1000 } } @@ -431,8 +431,8 @@ private[spark] class ExecutorAllocationManager( if (executorIds.contains(executorId)) { if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") - removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + s"scheduled to run on the executor (to expire in $executorIdleTimeoutS seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeoutS * 1000 } } else { logWarning(s"Attempted to mark unknown executor $executorId idle") diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 5871b8c869f03..e3bd16f1cbf24 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -62,14 +62,17 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val executorTimeoutMs = sc.conf.getOption("spark.network.timeout").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120000)) - + private val slaveTimeoutMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", "120s") + private val executorTimeoutMs = + sc.conf.getTimeAsSeconds("spark.network.timeout", s"${slaveTimeoutMs}ms") * 1000 + // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val checkTimeoutIntervalMs = - sc.conf.getOption("spark.network.timeoutInterval").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)) + private val timeoutIntervalMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") + private val checkTimeoutIntervalMs = + sc.conf.getTimeAsSeconds("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0c123c96b8d7b..390e631647bd6 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -174,6 +174,42 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsSeconds(key: String): Long = { + Utils.timeStringAsSeconds(get(key)) + } + + /** + * Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + * + */ + def getTimeAsSeconds(key: String, defaultValue: String): Long = { + Utils.timeStringAsSeconds(get(key, defaultValue)) + } + + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsMs(key: String): Long = { + Utils.timeStringAsMs(get(key)) + } + + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + */ + def getTimeAsMs(key: String, defaultValue: String): Long = { + Utils.timeStringAsMs(get(key, defaultValue)) + } + + /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { Option(settings.get(key)) 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 14f99a464b6e9..516f619529c48 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -436,14 +436,14 @@ private[spark] class Executor( * This thread stops running when the executor is stopped. */ private def startDriverHeartbeater(): Unit = { - val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") val thread = new Thread() { override def run() { // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) + Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) while (!isStopped) { reportHeartBeat() - Thread.sleep(interval) + Thread.sleep(intervalMs) } } } 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 741fe3e1ea750..8e3c30fc3d781 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,8 @@ 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", 120)) + conf.getTimeAsSeconds("spark.core.connection.ack.wait.timeout", + conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. // 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 076b36e86c0ce..2362cc7240039 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -62,10 +62,10 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getLong("spark.speculation.interval", 100) + val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000) + val STARVATION_TIMEOUT_MS = conf.getTimeAsMs("spark.starvation.timeout", "15s") // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) @@ -143,8 +143,8 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher - sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, - SPECULATION_INTERVAL milliseconds) { + sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, + SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } } } @@ -173,7 +173,7 @@ private[spark] class TaskSchedulerImpl( this.cancel() } } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + }, STARVATION_TIMEOUT_MS, STARVATION_TIMEOUT_MS) } hasReceivedTask = true } 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 d509881c74fef..7dc325283d961 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -848,15 +848,18 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get("spark.locality.wait", "3000") - level match { - case TaskLocality.PROCESS_LOCAL => - conf.get("spark.locality.wait.process", defaultWait).toLong - case TaskLocality.NODE_LOCAL => - conf.get("spark.locality.wait.node", defaultWait).toLong - case TaskLocality.RACK_LOCAL => - conf.get("spark.locality.wait.rack", defaultWait).toLong - case _ => 0L + val defaultWait = conf.get("spark.locality.wait", "3s") + val localityWaitKey = level match { + case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" + case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" + case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" + case _ => null + } + + if (localityWaitKey != null) { + conf.getTimeAsMs(localityWaitKey, defaultWait) + } else { + 0L } } 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 4c49da87af9dc..63987dfb32695 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 @@ -52,8 +52,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - val maxRegisteredWaitingTime = - conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) + val maxRegisteredWaitingTimeMs = + conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] @@ -77,12 +77,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def onStart() { // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000) + val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "1s") + reviveThread.scheduleAtFixedRate(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { Option(self).foreach(_.send(ReviveOffers)) } - }, 0, reviveInterval, TimeUnit.MILLISECONDS) + }, 0, reviveIntervalMs, TimeUnit.MILLISECONDS) } override def receive: PartialFunction[Any, Unit] = { @@ -301,9 +302,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp s"reached minRegisteredResourcesRatio: $minRegisteredRatio") return true } - if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { + if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTimeMs) { logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)") + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeMs(ms)") return true } false 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 6c2c5261306e7..8e8cc7cc6389e 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await import scala.concurrent.duration.{Duration, FiniteDuration} -import scala.util.Try import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -66,7 +65,8 @@ 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", 120)) + val akkaTimeoutS = conf.getTimeAsSeconds("spark.akka.timeout", + conf.get("spark.network.timeout", "120s")) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -78,8 +78,8 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) - val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) + val akkaHeartBeatPausesS = conf.getTimeAsSeconds("spark.akka.heartbeat.pauses", "6000s") + val akkaHeartBeatIntervalS = conf.getTimeAsSeconds("spark.akka.heartbeat.interval", "1000s") val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() @@ -102,14 +102,14 @@ private[spark] object AkkaUtils extends Logging { |akka.jvm-exit-on-fatal-error = off |akka.remote.require-cookie = "$requireCookie" |akka.remote.secure-cookie = "$secureCookie" - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatIntervalS s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPausesS s |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on - |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s + |akka.remote.netty.tcp.connection-timeout = $akkaTimeoutS s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 375ed430bde45..2bbfc988a99a8 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - conf.getInt("spark.cleaner.ttl", -1) + conf.getTimeAsSeconds("spark.cleaner.ttl", "-1").toInt } def getDelaySeconds( 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 a541d660cd5c6..1029b0f9fce1e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} -import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ @@ -47,6 +47,7 @@ import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -612,9 +613,10 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 - uc.setConnectTimeout(timeout) - uc.setReadTimeout(timeout) + val timeoutMs = + conf.getTimeAsSeconds("spark.files.fetchTimeout", "60s").toInt * 1000 + uc.setConnectTimeout(timeoutMs) + uc.setReadTimeout(timeoutMs) uc.connect() val in = uc.getInputStream() downloadFile(url, in, targetFile, fileOverwrite) @@ -1018,6 +1020,22 @@ private[spark] object Utils extends Logging { ) } + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + def timeStringAsMs(str: String): Long = { + JavaUtils.timeStringAsMs(str) + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + def timeStringAsSeconds(str: String): Long = { + JavaUtils.timeStringAsSec(str) + } + /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. */ diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 3ded1e4af8742..6b3049b28cd5e 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -684,10 +684,11 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext wit .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) - .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.schedulerBacklogTimeout", + s"${schedulerBacklogTimeout.toString}s") .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", - sustainedSchedulerBacklogTimeout.toString) - .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) + s"${sustainedSchedulerBacklogTimeout.toString}s") + .set("spark.dynamicAllocation.executorIdleTimeout", s"${executorIdleTimeout.toString}s") .set("spark.dynamicAllocation.testing", "true") val sc = new SparkContext(conf) contexts += sc diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index 716f875d30b8a..02424c59d6831 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -260,8 +260,8 @@ class ConnectionManagerSuite extends FunSuite { test("sendMessageReliably timeout") { val clientConf = new SparkConf clientConf.set("spark.authenticate", "false") - val ackTimeout = 30 - clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + val ackTimeoutS = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeoutS}s") val clientSecurityManager = new SecurityManager(clientConf) val manager = new ConnectionManager(0, clientConf, clientSecurityManager) @@ -272,7 +272,7 @@ class ConnectionManagerSuite extends FunSuite { val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { // sleep 60 sec > ack timeout for simulating server slow down or hang up - Thread.sleep(ackTimeout * 3 * 1000) + Thread.sleep(ackTimeoutS * 3 * 1000) None }) @@ -287,7 +287,7 @@ class ConnectionManagerSuite extends FunSuite { // Otherwise TimeoutExcepton is thrown from Await.result. // We expect TimeoutException is not thrown. intercept[IOException] { - Await.result(future, (ackTimeout * 2) second) + Await.result(future, (ackTimeoutS * 2) second) } manager.stop() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 716d12c0762cf..6198cea46ddf8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} import java.util.Random import scala.collection.mutable.ArrayBuffer @@ -27,7 +26,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, Utils} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -152,7 +151,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) + val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") val MAX_TASK_FAILURES = 4 override def beforeEach() { @@ -240,7 +239,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 2) should // get chosen before the noPref task assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) @@ -251,7 +250,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host2, exec3 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) } @@ -292,7 +291,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host1 again: nothing should get chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -306,7 +305,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Now that we've launched a local task, we should no longer launch the task for host3 assert(manager.resourceOffer("exec2", "host2", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) @@ -338,7 +337,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // nothing should be chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT * 2) + clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -528,7 +527,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) @@ -622,12 +621,12 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) manager.speculatableTasks += 1 - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) // schedule the speculative task assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } @@ -716,13 +715,13 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) assert(manager.resourceOffer("execA", "host1", ANY) !== None) - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1") manager.executorLost("execB.2", "host2") - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index b4de90b65d545..ffa5162a31841 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -76,7 +76,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd conf.set("spark.storage.unrollMemoryThreshold", "512") // to make a replication attempt to inactive store fail fast - conf.set("spark.core.connection.ack.wait.timeout", "1") + conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") 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 449fb87f111c4..fb97e650ff95c 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -23,6 +23,7 @@ import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStr import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.text.DecimalFormatSymbols +import java.util.concurrent.TimeUnit import java.util.Locale import com.google.common.base.Charsets.UTF_8 @@ -35,7 +36,50 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { + + test("timeConversion") { + // Test -1 + assert(Utils.timeStringAsSeconds("-1") === -1) + + // Test zero + assert(Utils.timeStringAsSeconds("0") === 0) + + assert(Utils.timeStringAsSeconds("1") === 1) + assert(Utils.timeStringAsSeconds("1s") === 1) + assert(Utils.timeStringAsSeconds("1000ms") === 1) + assert(Utils.timeStringAsSeconds("1000000us") === 1) + assert(Utils.timeStringAsSeconds("1m") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1min") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1h") === TimeUnit.HOURS.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1d") === TimeUnit.DAYS.toSeconds(1)) + + assert(Utils.timeStringAsMs("1") === 1) + assert(Utils.timeStringAsMs("1ms") === 1) + assert(Utils.timeStringAsMs("1000us") === 1) + assert(Utils.timeStringAsMs("1s") === TimeUnit.SECONDS.toMillis(1)) + assert(Utils.timeStringAsMs("1m") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) + assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) + + // Test invalid strings + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600s") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600ds") + } + intercept[NumberFormatException] { + Utils.timeStringAsMs("600s This breaks") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This 123s breaks") + } + } + test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") assert(Utils.bytesToString(1500) === "1500.0 B") diff --git a/docs/configuration.md b/docs/configuration.md index 7fe11475212b3..7169ec295ef7f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,9 +35,19 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually -require one to prevent any sort of starvation issues. +Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may +actually require one to prevent any sort of starvation issues. +Properties that specify some time duration should be configured with a unit of time. +The following format is accepted: + + 25ms (milliseconds) + 5s (seconds) + 10m or 10min (minutes) + 3h (hours) + 5d (days) + 1y (years) + ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For instance, if you'd like to run the same application with different masters or different @@ -429,10 +439,10 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait - 5 + 5s - (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying + is 15 seconds by default, calculated as maxRetries * retryWait. @@ -732,17 +742,17 @@ Apart from these, the following properties are also available, and may be useful spark.executor.heartbeatInterval - 10000 - Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + 10s + Interval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks. spark.files.fetchTimeout - 60 + 60s Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. + the driver. @@ -853,11 +863,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval - 1000 + 1000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + interval value reduces network overhead and a smaller value ( ~ 1 s) might be more informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need to. A likely positive use case for using failure detector would be: a sensistive failure detector can help evict rogue executors quickly. However this is usually not the case @@ -868,11 +878,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.pauses - 6000 + 6000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart - beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune + beat pause for Akka. This can be used to control sensitivity to GC pauses. Tune this along with `spark.akka.heartbeat.interval` if you need to. @@ -886,9 +896,9 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout - 100 + 100s - Communication timeout between Spark nodes, in seconds. + Communication timeout between Spark nodes. @@ -938,10 +948,10 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout - 120 + 120s - 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, + Default timeout for all network interactions. 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. @@ -989,9 +999,9 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000 + 3s - Number of milliseconds to wait to launch a data-local task before giving up and launching it + How long to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels (process-local, node-local, rack-local and then any). It is also possible to customize the waiting time for each level by setting spark.locality.wait.node, etc. @@ -1024,10 +1034,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000 + 30s - Maximum amount of time to wait for resources to register before scheduling begins - (in milliseconds). + Maximum amount of time to wait for resources to register before scheduling begins. @@ -1054,10 +1063,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000 + 1s - The interval length for the scheduler to revive the worker resource offers to run tasks - (in milliseconds). + The interval length for the scheduler to revive the worker resource offers to run tasks. @@ -1070,9 +1078,9 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval - 100 + 100ms - How often Spark will check for tasks to speculate, in milliseconds. + How often Spark will check for tasks to speculate. @@ -1127,10 +1135,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout - 600 + 600s - 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 + If dynamic allocation is enabled and an executor has been idle for more than this duration, + the executor will be removed. For more detail, see this description. @@ -1157,10 +1165,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 5 + 5s 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 + this duration, new executors will be requested. For more detail, see this description. @@ -1215,18 +1223,18 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout - 60 + 60s - Number of seconds for the connection to wait for ack to occur before timing + How long for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. spark.core.connection.auth.wait.timeout - 30 + 30s - Number of seconds for the connection to wait for authentication to occur before timing + How long for the connection to wait for authentication to occur before timing out and giving up. @@ -1347,9 +1355,9 @@ Apart from these, the following properties are also available, and may be useful Property NameDefaultMeaning spark.streaming.blockInterval - 200 + 200ms - Interval (milliseconds) at which data received by Spark Streaming receivers is chunked + Interval at which data received by Spark Streaming receivers is chunked into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the performance tuning section in the Spark Streaming programing guide for more details. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index ed5bb263a5809..853c9f26b0ec9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,9 +48,9 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.am.waitTime - 100000 + 100s - In yarn-cluster mode, time in milliseconds for the application master to wait for the + In yarn-cluster mode, time 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. diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 73da9b7346f4d..b6fbace509a0e 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -21,9 +21,13 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,4 +125,66 @@ private static boolean isSymlink(File file) throws IOException { } return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); } + + private static ImmutableMap timeSuffixes = + ImmutableMap.builder() + .put("us", TimeUnit.MICROSECONDS) + .put("ms", TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("m", TimeUnit.MINUTES) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .build(); + + /** + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count for + * internal use. If no suffix is provided a direct conversion is attempted. + */ + private static long parseTimeString(String str, TimeUnit unit) { + String lower = str.toLowerCase().trim(); + + try { + String suffix; + long val; + Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); + if (m.matches()) { + val = Long.parseLong(m.group(1)); + suffix = m.group(2); + } else { + throw new NumberFormatException("Failed to parse time string: " + str); + } + + // Check for invalid suffixes + if (suffix != null && !timeSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit); + } catch (NumberFormatException e) { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (m or min) hour (h), or day (d). " + + "E.g. 50s, 100ms, or 250us."; + + throw new NumberFormatException(timeError + "\n" + e.getMessage()); + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + public static long timeStringAsMs(String str) { + return parseTimeString(str, TimeUnit.MILLISECONDS); + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + public static long timeStringAsSec(String str) { + return parseTimeString(str, TimeUnit.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 2eaf3b71d9a49..0aef7f1987315 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,8 +37,11 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - int defaultTimeout = conf.getInt("spark.network.timeout", 120); - return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000; + long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( + conf.get("spark.network.timeout", "120s")); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; + return (int) defaultTimeoutMs; } /** Number of concurrent connections between two nodes for fetching data. */ @@ -68,7 +71,9 @@ public int numConnectionsPerPeer() { public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeoutMs() { return conf.getInt("spark.shuffle.sasl.timeout", 30) * 1000; } + public int saslRTTimeoutMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; + } /** * Max number of times we will try IO exceptions (such as connection timeouts) per request. @@ -80,7 +85,9 @@ public int numConnectionsPerPeer() { * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. * Only relevant if maxIORetries > 0. */ - public int ioRetryWaitTimeMs() { return conf.getInt("spark.shuffle.io.retryWait", 5) * 1000; } + public int ioRetryWaitTimeMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; + } /** * Minimum size of a block that we should start using memory map rather than reading in through 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 42514d8b47dcf..f4963a78e1d18 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 @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.SystemClock +import org.apache.spark.util.{SystemClock, Utils} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { @@ -79,9 +79,9 @@ private[streaming] class BlockGenerator( private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) private val clock = new SystemClock() - private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalMs = conf.getTimeAsMs("spark.streaming.blockInterval", "200ms") private val blockIntervalTimer = - new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } @@ -132,7 +132,7 @@ private[streaming] class BlockGenerator( val newBlockBuffer = currentBuffer currentBuffer = new ArrayBuffer[Any] if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(receiverId, time - blockInterval) + val blockId = StreamBlockId(receiverId, time - blockIntervalMs) val newBlock = new Block(blockId, newBlockBuffer) listener.onGenerateBlock(blockId) blocksForPushing.put(newBlock) // put is blocking when queue is full diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 4946806d2ee95..58e56638a2dca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -24,7 +24,7 @@ import akka.actor.{ActorRef, Props, Actor} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Clock, ManualClock} +import org.apache.spark.util.{Clock, ManualClock, Utils} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -104,17 +104,15 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = conf.getLong( - "spark.streaming.gracefulStopTimeout", - 10 * ssc.graph.batchDuration.milliseconds - ) + val stopTimeoutMs = conf.getTimeAsMs( + "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms") val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut: Boolean = { - val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeout + val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeoutMs if (timedOut) { - logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeoutMs + ")") } timedOut } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 10c35cba8dc53..91261a9db7360 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -131,11 +131,11 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 200 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val blockIntervalMs = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms") val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 5 - val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val waitTime = expectedBlocks * blockIntervalMs + (blockIntervalMs / 2) val generatedData = new ArrayBuffer[Int] // Generate blocks @@ -157,15 +157,15 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 100 + val blockIntervalMs = 100 val maxRate = 100 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms"). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 20 - val waitTime = expectedBlocks * blockInterval + val waitTime = expectedBlocks * blockIntervalMs val expectedMessages = maxRate * waitTime / 1000 - val expectedMessagesPerBlock = maxRate * blockInterval / 1000 + val expectedMessagesPerBlock = maxRate * blockIntervalMs / 1000 val generatedData = new ArrayBuffer[Int] // Generate blocks diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index d1bbf39dc7897..58353a5f97c8a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -73,9 +73,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from existing SparkContext") { @@ -85,24 +85,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") + assert( + Utils.timeStringAsSeconds(cp.sparkConfPairs + .toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getInt("spark.cleaner.ttl", -1) === 10) + assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("start and stop state check") { @@ -176,7 +178,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600") + conf.set("spark.cleaner.ttl", "3600s") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") @@ -207,7 +209,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.streaming.gracefulStopTimeout", "20000") + conf.set("spark.streaming.gracefulStopTimeout", "20000s") sc = new SparkContext(conf) logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) 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 26259cee77151..c357b7ae9d4da 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 @@ -296,7 +296,7 @@ private[spark] class ApplicationMaster( // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) + sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "5s") // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) @@ -379,7 +379,8 @@ private[spark] class ApplicationMaster( logWarning( "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") } - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", waitTries.getOrElse(100000L)) + val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", + s"${waitTries.getOrElse(100000L)}ms") val deadline = System.currentTimeMillis() + totalWaitTime while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { @@ -404,8 +405,8 @@ private[spark] class ApplicationMaster( // 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. - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", 100000L) - val deadline = System.currentTimeMillis + totalWaitTime + val totalWaitTimeMs = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") + val deadline = System.currentTimeMillis + totalWaitTimeMs while (!driverUp && !finished && System.currentTimeMillis < deadline) { try { From d7f2c198678d1f5ffc1c6da3a6fb189b0a4ef070 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 13 Apr 2015 16:53:50 -0700 Subject: [PATCH 007/144] [SPARK-6881][SparkR] Changes the checkpoint directory name. Author: hlin09 Closes #5493 from hlin09/fixCheckpointDir and squashes the following commits: e67fc40 [hlin09] Change to temp dir. 1f7ed9e [hlin09] Change the checkpoint dir name. --- R/pkg/R/RDD.R | 2 +- R/pkg/R/context.R | 2 +- R/pkg/inst/tests/test_rdd.R | 5 +++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 604ad03c407b9..d6a75007a6ad7 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -279,7 +279,7 @@ setMethod("unpersist", #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' setCheckpointDir(sc, "checkpoints") +#' setCheckpointDir(sc, "checkpoint") #' rdd <- parallelize(sc, 1:10, 2L) #' checkpoint(rdd) #'} diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 2fc0bb294bcce..ebbb8fba1052d 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -216,7 +216,7 @@ broadcast <- function(sc, object) { #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' setCheckpointDir(sc, "~/checkpoints") +#' setCheckpointDir(sc, "~/checkpoint") #' rdd <- parallelize(sc, 1:2, 2L) #' checkpoint(rdd) #'} diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index f75e0817b9406..b76e4db03e715 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -141,7 +141,8 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp unpersist(rdd2) expect_false(rdd2@env$isCached) - setCheckpointDir(sc, "checkpoints") + tempDir <- tempfile(pattern = "checkpoint") + setCheckpointDir(sc, tempDir) checkpoint(rdd2) expect_true(rdd2@env$isCheckpointed) @@ -152,7 +153,7 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp # make sure the data is collectable collect(rdd2) - unlink("checkpoints") + unlink(tempDir) }) test_that("reduce on RDD", { From 5b8b324f33e857b95de65031334846a7ca26fa60 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 13 Apr 2015 18:15:29 -0700 Subject: [PATCH 008/144] [SPARK-6303][SQL] Remove unnecessary Average in GeneratedAggregate Because `Average` is a `PartialAggregate`, we never get a `Average` node when reaching `HashAggregation` to prepare `GeneratedAggregate`. That is why in SQLQuerySuite there is already a test for `avg` with codegen. And it works. But we can find a case in `GeneratedAggregate` to deal with `Average`. Based on the above, we actually never execute this case. So we can remove this case from `GeneratedAggregate`. Author: Liang-Chi Hsieh Closes #4996 from viirya/add_average_codegened and squashes the following commits: 621c12f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened 368cfbc [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened 74926d1 [Liang-Chi Hsieh] Add Average in canBeCodeGened lists. --- .../sql/execution/GeneratedAggregate.scala | 45 ------------------- 1 file changed, 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 95176e425132d..b510cf033c4a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -153,51 +153,6 @@ case class GeneratedAggregate( AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) - case a @ Average(expr) => - val calcType = - expr.dataType match { - case DecimalType.Fixed(_, _) => - DecimalType.Unlimited - case _ => - expr.dataType - } - - val currentCount = AttributeReference("currentCount", LongType, nullable = false)() - val currentSum = AttributeReference("currentSum", calcType, nullable = false)() - val initialCount = Literal(0L) - val initialSum = Cast(Literal(0L), calcType) - - // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its - // UnscaledValue will be null if and only if x is null; helps with Average on decimals - val toCount = expr match { - case UnscaledValue(e) => e - case _ => expr - } - - val updateCount = If(IsNotNull(toCount), Add(currentCount, Literal(1L)), currentCount) - val updateSum = Coalesce(Add(Cast(expr, calcType), currentSum) :: currentSum :: Nil) - - val result = - expr.dataType match { - case DecimalType.Fixed(_, _) => - If(EqualTo(currentCount, Literal(0L)), - Literal.create(null, a.dataType), - Cast(Divide( - Cast(currentSum, DecimalType.Unlimited), - Cast(currentCount, DecimalType.Unlimited)), a.dataType)) - case _ => - If(EqualTo(currentCount, Literal(0L)), - Literal.create(null, a.dataType), - Divide(Cast(currentSum, a.dataType), Cast(currentCount, a.dataType))) - } - - AggregateEvaluation( - currentCount :: currentSum :: Nil, - initialCount :: initialSum :: Nil, - updateCount :: updateSum :: Nil, - result - ) - case m @ Max(expr) => val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() val initialValue = Literal.create(null, expr.dataType) From 4898dfa464be55772e3f9db10c48adcb3cfc9a3d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 13 Apr 2015 18:16:33 -0700 Subject: [PATCH 009/144] [SPARK-6877][SQL] Add code generation support for Min Currently `min` is not supported in code generation. This pr adds the support for it. Author: Liang-Chi Hsieh Closes #5487 from viirya/add_min_codegen and squashes the following commits: 0ddec23 [Liang-Chi Hsieh] Add code generation support for Min. --- .../sql/catalyst/expressions/arithmetic.scala | 45 +++++++++++++++++++ .../expressions/codegen/CodeGenerator.scala | 24 ++++++++++ .../ExpressionEvaluationSuite.scala | 10 +++++ .../sql/execution/GeneratedAggregate.scala | 13 ++++++ .../spark/sql/execution/SparkStrategies.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 14 ++++-- 6 files changed, 104 insertions(+), 4 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 1f6526ef66c56..566b34f7c3a6a 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 @@ -369,6 +369,51 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def toString: String = s"MaxOf($left, $right)" } +case class MinOf(left: Expression, right: Expression) extends Expression { + type EvaluatedType = Any + + override def foldable: Boolean = left.foldable && right.foldable + + override def nullable: Boolean = left.nullable && right.nullable + + override def children: Seq[Expression] = left :: right :: Nil + + override lazy val resolved = + left.resolved && right.resolved && + left.dataType == right.dataType + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, + s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + } + left.dataType + } + + lazy val ordering = left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + val evalE2 = right.eval(input) + if (evalE1 == null) { + evalE2 + } else if (evalE2 == null) { + evalE1 + } else { + if (ordering.compare(evalE1, evalE2) < 0) { + evalE1 + } else { + evalE2 + } + } + } + + override def toString: String = s"MinOf($left, $right)" +} + /** * A function that get the absolute value of the numeric value. */ 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 aac56e1568332..d141354a0f427 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 @@ -524,6 +524,30 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } """.children + case MinOf(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = ${defaultPrimitive(e1.dataType)} + + if (${eval1.nullTerm}) { + $nullTerm = ${eval2.nullTerm} + $primitiveTerm = ${eval2.primitiveTerm} + } else if (${eval2.nullTerm}) { + $nullTerm = ${eval1.nullTerm} + $primitiveTerm = ${eval1.primitiveTerm} + } else { + if (${eval1.primitiveTerm} < ${eval2.primitiveTerm}) { + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $primitiveTerm = ${eval2.primitiveTerm} + } + } + """.children + case UnscaledValue(child) => val childEval = expressionEvaluator(child) 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 d2b1090a0cdd5..d4362a91d992c 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 @@ -233,6 +233,16 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2) } + test("MinOf") { + checkEvaluation(MinOf(1, 2), 1) + checkEvaluation(MinOf(2, 1), 1) + checkEvaluation(MinOf(1L, 2L), 1L) + checkEvaluation(MinOf(2L, 1L), 1L) + + checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1) + checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal.create(null, StringType).like("a"), null) checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index b510cf033c4a4..b1ef6556de1e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -164,6 +164,17 @@ case class GeneratedAggregate( updateMax :: Nil, currentMax) + case m @ Min(expr) => + val currentMin = AttributeReference("currentMin", expr.dataType, nullable = true)() + val initialValue = Literal.create(null, expr.dataType) + val updateMin = MinOf(currentMin, expr) + + AggregateEvaluation( + currentMin :: Nil, + initialValue :: Nil, + updateMin :: Nil, + currentMin) + case CollectHashSet(Seq(expr)) => val set = AttributeReference("hashSet", new OpenHashSetUDT(expr.dataType), nullable = false)() @@ -188,6 +199,8 @@ case class GeneratedAggregate( initialValue :: Nil, collectSets :: Nil, CountSet(set)) + + case o => sys.error(s"$o can't be codegened.") } val computationSchema = computeFunctions.flatMap(_.schema) 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 f0d92ffffcda3..5b99e40c2f491 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 @@ -155,7 +155,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]): Boolean = !aggs.exists { - case _: CombineSum | _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + case _: CombineSum | _: Sum | _: Count | _: Max | _: Min | _: CombineSetsAndCount => false // The generated set implementation is pretty limited ATM. case CollectHashSet(exprs) if exprs.size == 1 && Seq(IntegerType, LongType).contains(exprs.head.dataType) => false 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 5e453e05e2ac7..73fb791c3ead7 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 @@ -172,6 +172,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { testCodeGen( "SELECT max(key) FROM testData3x", Row(100) :: Nil) + // MIN + testCodeGen( + "SELECT value, min(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT min(key) FROM testData3x", + Row(1) :: Nil) // Some combinations. testCodeGen( """ @@ -179,16 +186,17 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { | value, | sum(key), | max(key), + | min(key), | avg(key), | count(key), | count(distinct key) |FROM testData3x |GROUP BY value """.stripMargin, - (1 to 100).map(i => Row(i.toString, i*3, i, i, 3, 1))) + (1 to 100).map(i => Row(i.toString, i*3, i, i, i, 3, 1))) testCodeGen( - "SELECT max(key), avg(key), count(key), count(distinct key) FROM testData3x", - Row(100, 50.5, 300, 100) :: Nil) + "SELECT max(key), min(key), avg(key), count(key), count(distinct key) FROM testData3x", + Row(100, 1, 50.5, 300, 100) :: Nil) // Aggregate with Code generation handling all null values testCodeGen( "SELECT sum('a'), avg('a'), count(null) FROM testData", From 435b8779df01a7477addecb1023605957bca4e9b Mon Sep 17 00:00:00 2001 From: Nathan Kronenfeld Date: Mon, 13 Apr 2015 18:21:16 -0700 Subject: [PATCH 010/144] [Spark-4848] Allow different Worker configurations in standalone cluster This refixes #3699 with the latest code. This fixes SPARK-4848 I've changed the stand-alone cluster scripts to allow different workers to have different numbers of instances, with both port and web-ui port following allong appropriately. I did this by moving the loop over instances from start-slaves and stop-slaves (on the master) to start-slave and stop-slave (on the worker). Wile I was at it, I changed SPARK_WORKER_PORT to work the same way as SPARK_WORKER_WEBUI_PORT, since the new methods work fine for both. Author: Nathan Kronenfeld Closes #5140 from nkronenfeld/feature/spark-4848 and squashes the following commits: cf5f47e [Nathan Kronenfeld] Merge remote branch 'upstream/master' into feature/spark-4848 044ca6f [Nathan Kronenfeld] Documentation and formatting as requested by by andrewor14 d739640 [Nathan Kronenfeld] Move looping through instances from the master to the workers, so that each worker respects its own number of instances and web-ui port --- sbin/start-slave.sh | 59 +++++++++++++++++++++++++++++++++++++++++--- sbin/start-slaves.sh | 11 +-------- sbin/stop-slave.sh | 43 ++++++++++++++++++++++++++++++++ sbin/stop-slaves.sh | 12 +++------ 4 files changed, 103 insertions(+), 22 deletions(-) create mode 100755 sbin/stop-slave.sh diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 5a6de11afdd3d..4c919ff76a8f5 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -18,15 +18,68 @@ # # Starts a slave on the machine this script is executed on. +# +# Environment Variables +# +# SPARK_WORKER_INSTANCES The number of worker instances to run on this +# slave. Default is 1. +# SPARK_WORKER_PORT The base port number for the first worker. If set, +# subsequent workers will increment this number. If +# unset, Spark will find a valid port number, but +# with no guarantee of a predictable pattern. +# SPARK_WORKER_WEBUI_PORT The base port for the web interface of the first +# worker. Subsequent workers will increment this +# number. Default is 8081. -usage="Usage: start-slave.sh where is like spark://localhost:7077" +usage="Usage: start-slave.sh where is like spark://localhost:7077" -if [ $# -lt 2 ]; then +if [ $# -lt 1 ]; then echo $usage + echo Called as start-slave.sh $* exit 1 fi sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" -"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +# First argument should be the master; we need to store it aside because we may +# need to insert arguments between it and the other arguments +MASTER=$1 +shift + +# Determine desired worker port +if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then + SPARK_WORKER_WEBUI_PORT=8081 +fi + +# Start up the appropriate number of workers on this machine. +# quick local function to start a worker +function start_instance { + WORKER_NUM=$1 + shift + + if [ "$SPARK_WORKER_PORT" = "" ]; then + PORT_FLAG= + PORT_NUM= + else + PORT_FLAG="--port" + PORT_NUM=$(( $SPARK_WORKER_PORT + $WORKER_NUM - 1 )) + fi + WEBUI_PORT=$(( $SPARK_WORKER_WEBUI_PORT + $WORKER_NUM - 1 )) + + "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker $WORKER_NUM \ + --webui-port "$WEBUI_PORT" $PORT_FLAG $PORT_NUM $MASTER "$@" +} + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + start_instance 1 "$@" +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + start_instance $(( 1 + $i )) "$@" + done +fi + diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 4356c03657109..24d6268815ed3 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -59,13 +59,4 @@ if [ "$START_TACHYON" == "true" ]; then fi # Launch the slaves -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" -else - if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then - SPARK_WORKER_WEBUI_PORT=8081 - fi - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" diff --git a/sbin/stop-slave.sh b/sbin/stop-slave.sh new file mode 100755 index 0000000000000..3d1da5b254f2a --- /dev/null +++ b/sbin/stop-slave.sh @@ -0,0 +1,43 @@ +#!/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. +# + +# A shell script to stop all workers on a single slave +# +# Environment variables +# +# SPARK_WORKER_INSTANCES The number of worker instances that should be +# running on this slave. Default is 1. + +# Usage: stop-slave.sh +# Stops all slaves on this worker machine + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) + done +fi diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index 7c2201100ef97..54c9bd46803a9 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" @@ -29,10 +29,4 @@ if [ -e "$sbin"/../tachyon/bin/tachyon ]; then "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon killAll tachyon.worker.Worker fi -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1 -else - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/stop-slave.sh From 3782e1f2bec07b5ffbc8503e40591e96fce67256 Mon Sep 17 00:00:00 2001 From: Fei Wang Date: Mon, 13 Apr 2015 18:23:35 -0700 Subject: [PATCH 011/144] [SQL] [Minor] Fix for SqlApp.scala SqlApp.scala is out of date. Author: Fei Wang Closes #5485 from scwf/patch-1 and squashes the following commits: 6f731c2 [Fei Wang] SqlApp.scala compile error --- dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala index d888de929fdda..cc86ef45858c9 100644 --- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -36,8 +36,10 @@ object SparkSqlExample { val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ import sqlContext._ - val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) + + val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)).toDF() people.registerTempTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() From b45059d0d7809a986ba07a447deb71f11ec6afe4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 18:26:00 -0700 Subject: [PATCH 012/144] [SPARK-5794] [SQL] fix add jar Author: Daoyuan Wang Closes #4586 from adrian-wang/addjar and squashes the following commits: efdd602 [Daoyuan Wang] move jar to another place 6c707e8 [Daoyuan Wang] restrict hive version for test 32c4fb8 [Daoyuan Wang] fix style and add a test 9957d87 [Daoyuan Wang] use sessionstate classloader in makeRDDforTable 0810e71 [Daoyuan Wang] remove variable substitution 1898309 [Daoyuan Wang] fix classnotfound 95a40da [Daoyuan Wang] support env argus in add jar, and set add jar ret to 0 --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 5 +++-- .../apache/spark/sql/hive/TableReader.scala | 5 ++++- .../spark/sql/hive/execution/commands.scala | 2 +- .../resources/hive-hcatalog-core-0.13.1.jar | Bin 0 -> 468533 bytes .../sql/hive/execution/HiveQuerySuite.scala | 15 +++++++++++++++ 5 files changed, 23 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 6272cdedb3e48..62c061bef690a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{SetProcessor, CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket @@ -264,7 +264,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf) if (proc != null) { - if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { + if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || + proc.isInstanceOf[AddResourceProcessor]) { val driver = new SparkSQLDriver driver.init() 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 d35291543c9f9..e556c74ffb015 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 @@ -35,6 +35,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DateUtils +import org.apache.spark.util.Utils /** * A trait for subclasses that handle table scans. @@ -76,7 +77,9 @@ class HadoopTableReader( override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] = makeRDDForTable( hiveTable, - relation.tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + Class.forName( + relation.tableDesc.getSerdeClassName, true, sc.sessionState.getConf.getClassLoader) + .asInstanceOf[Class[Deserializer]], filterOpt = None) /** 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 99dc58646ddd6..902a12785e3e9 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 @@ -80,7 +80,7 @@ case class AddJar(path: String) extends RunnableCommand { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) - Seq.empty[Row] + Seq(Row(0)) } } diff --git a/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar b/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar new file mode 100644 index 0000000000000000000000000000000000000000..37af9aafad8a41d3007b63a610b1af37c324544a GIT binary patch literal 468533 zcmb5V1DIt`l0IDSvTfV8ZQHkO+qP{RUG74cZQHKuGP>AR|DN5M{l1yq>EG^s&U5kH zc;iGy<{KFqc~VgZ6bu^ZZ!ex2Q?dW}^7l8W-=A_~szP*9^5P7Ne;0!U;{GkhtKaH6 z|NHN+z(7E#|9>$#A$cirF%?yMIq}ehDXRfS#E@%{?*rWdA##PlTgW0((1C)&bw-KH z4HdLlTJrO4^%oCcjHT;Xnpg-r#q6!mlR1?yY;T~YwH#VwIvOyd{O_jcY)|(_c59+S zB4adtsWsgyaDwRcN1a=-Yh;r1CUjifha%kXnU)z$9KEJ+-r|~TYll<54EB*rKrgrB zE~AmV?7^xmE-DJYx}BsKlY z6aa~cRYKg?+8-nwDIXKxiT-T&gr8c&_=gXGfRd{I*;Qb_ePHio{$F1D?-DS7moRcL z0$7;-K@jnu1uZPy{v?g{KS)~uj9iRt?Ehp9-k)Fq8_Pe~NB>t)6;oRyI~PlUke!i@ zr?aK=A9UgWNtem*R{Q}=@t0T^CnGy&GgGHOD53k`D{*!)a`{tF(EjgLS=u_-{J|)) zztrUXdkES7$vFgnQekiJW%;L;5dH}au(!3fxBG+jlz&1yyVyG!nVVV|*_qhH<8^oV^Nc9KQ z|AB)48bJOLJN|-w#Q)~*{{j175bs|{$v@rpzwm#>xqpoT|Ff3Apc~Eii^Q1Y+S!#|+^ zoraMAF(Vs|gi$DB00F@;0|8P0i+}$%js6!G|Hdpy5hE878%uhCjghl+iw=~J>e78~?K5^-~K6QO3SY3m=rTomEnenjvI};c;P7LA` zaCK?%*vT5t7Q_aWz1V>@DRa`n40n5g)S9YCC%hG$uJsYPF7b$3v{7y>wydsg2_BE& zJ~@8qJK^N)qb=fi^<<9mw|(R7L0gEnw+dO0tw|l(tAr;qa`*8!Wlx*bGh1?dju+-+ z_Y9wQshMotr1o6$@FP5?HTG{4hjz){vT?5!VeVL68xqwaPBGqaac_ja{U#G5zFiXh zxCEYDRQXh%C34#|!q>^|G_u z4vQ%XSfAMmE{_o=4oJxTRRh{4MlSoWMnpdeFt~qPUOwiI<8HpD12o+`K1V zU?`vc1wZ&Y)wG4bBkDR?qNgwac8;xa7yF}z_lo1$nZJV`l#8_Z$9GL-lHxBW&GUy} zhVt>5UyKM4Zj6Gz_&(rwuQ#0^tC9N4`iauMR*Z>2_-{)P95P_OXT-xt9TB8IVSqoA zCHK(M@*CZ0__$#BFCCIV2w2{!@PA)|WFyyKAjt8DKijkNi`^iui~-)amktqWpd+H7-q9+cSwfLxo|@Py zqv*hhcc4V;7gyWUY_`(#rBow`HfGVVr^GhT(r<0C)UdFxphgEFu%-HPu{e4Kn`ItQ zFa8x3Zm38jOnFYUYcJ~S76mAIH4}bY?yDwcW^;q1@Ahz^B=KDOngB_4#5XRqSTdQB zGA)t|YqO%rs;OI(j(Uc@k_$-@-kgYldfma7&_ya_J6J9$9t`JY$T7CLjIAWeTB(wQ zMBSJqq?qAl-ETC9E+UPU;H-lsNf}j`Di#hPd~IuEq!ulu!c;ntVJNqC2ev8YRUR5k z+%e}&h~Zxk91v`8L2x~?IW8?Mv^G}fdre4L7si<0Wmhey3?bL}=MqV(xH6IP3b6OK zC6mnxU1&GGVniHcL&5pqpm}nP62~r(W5c}~Hn+@qz8lt8Y@7kIfMP@3Il^C?@1?N{ z%3bbyBu_qZ5FdvGO3z%1VwM(c{DA<1w%EH$7EzN9$8t zaJMN<{)1mwp`WRj44IspQ!KToO#tI=!j&PEBU@jzca6P_?wAuXD&F40f?LXF?P#20 zHanct@#L3sAhomrH}RQK4X&8esSR>cT7j5$A%!T8o!uNod~1QYbs2ag(Kx0R8t7bZ z(lt2^L^X`k+C|`y-bp-}AQZFpG7&NpkDTOO0bi4D`mH5X z9yikHVC3NQpe&mk1wm(0*g}1SG$J~v;L-Vpm~IO8i&(l=#V}>(gSG6apZy@6wk8bv zaBKa-j<-@3WGuHjuDZCL$(0G9-*3I>x|3yFC|K1L2K%JHe%oEN^k@;Uonx0g<}s{2 z2>Hn&mQ0n)(i|Ce;UPMTE~xETbUP%IR+LrbQ$%QmCVo&kHKxcR+tGEQN@YAKS$n2x zzztsHR-q9dEgkdvc6*<3xsjeN$1z@pC(l|sl+%HMB>)0$CrTb+MJ`=BoP|i%CJbouQ3Z{Yeoc*(o4*D_vp_)2JLqRzomWuTN?DwY=hvZ2X6FF1OiiT z+Za_y39V#@ejqTq4}>22ctFICAvIDn>TPtGu?&^kAnX0%tvO|S>E?Lwkg^<|DJ6;( zl4a1EEZ#+!&#hs4@+ls$lz3Z;MrPVwrLl~b6$qt^r2*ZTUJk5ygX+N6mOOFnH_f7* zzPb<;*~its(dnub)hffr8F-1#A~(^rMSH6XTYE_Iq4Do-xLTE7tPV-`4h^N`RZ_Aj zZ_MD%pP`ZnmNx=ghk42J>wPix;B2}mGo!p+0hYs3OA!y7pfBDRn1|VQcnp*Th7f~Z zuFZXyi>ME>07PvZ1#@m9R+6;4>PlJXktH{6-eB`0--bn2HRCz^(>8PkfxsnusH#*lc?umV_ z7^^Fk1^W4)P0#df14Zd)Xwh-V#Kyx?ItI^90NfXyH(|xri=-E^K4zJz6xv}JCt;3q z4m0X%g%SYLcZlT(k#Z83L1pDDB*+4j3avEgswOzL4P|Flstyd9DYbiAHUlFkli+vt zQRSnm7|XlcLbJ+H5Q?wrfJ$h!$Ub%vx<=JhZJSEx#r8Dr_;_ubsD}>}ZzQH?Er;-a zy@*NrhDBw%&Kc#2)32q|8W@^Z?NzO+vAxl)WElIIDMdchtC>t2O{6)b$O={mH|eG7 z6)o7(s?5G+g>Y)tyPjV13NmnJ))691nGfvaLDV1?gW^gS7#b zEiRbS+9*NW#C^VVstU9Ct$a>v+@TOj)o?A_$~dYh3!b0hez1>Cc$b=tWWO|9N@SN} zP_bp#DFqS_Z`nu|#3P#NUP5U&AXkXGR!&#trTDvbDZYdDN%L@e59KvqUKnukJeL`o zV(<^Cy$M(13)Vr~!Ku4}mH{9GE6h0# zY;+pCY^qfn>i46f#xK958MFWeD9hL;74;#>v{br1n?Tq)9=&WJBQH@&x2IUT0d(Cp`{rB+UPx+^@ltA9t8CNO>kwJewXDKBaK%_O@kCo) zgW4;*IC2+X$~pE{Nl?6vYVs6$GpNwGIyI?FTA7xD=Wm_M2~!2XJKRZKsv4Ea2%dux z^AXV|2W7~wc{a__=Q<<|1-}i--I0sP+hOC$r>`xX9v;kS05DQx*I4c^R0x8@SroQC z9ajq<%`W-mjSY?|IqiLNiKi#$gQYbfW|Q*H{|zl z`m((}7FW2~VB~!@*lK&JS=V5OJqa;3H!r#cBI^X@H*ULrWO!Vi6;US2=Q?y@X4X>D z_)xbu-&humy5VjahzTvIEmJZU^EeL#+^M%3cDb{r9`w+iR$A$1$0LGcB*(Fci)TzOMSp35wvg?F#*#=zm7cq5F1{ap zme(ro#87t_>lmhkm04anIb_j_2g|K!4ZaseJYrnR#ZBuM)#&J> z93?WA-KDjHW4I%A9jRGN40lpe3-$@gD4-q{x>1hmBO`p*VPg8X4sVLi+OZ0Hx; z&?$#|kbUPaw5hmk@ju0U9qEq4$$?cr@rt{}qn`xe4 z+X*$ANc3Ed?76FL9>>ld4PzjbLmtl;8|4kixQad6F*ailpn_D$o`*qH>6h4wXwu~S z?_qOn8#G^ee4%Z{nzT9Ouerzg-1R}%mXi|*ICznA28U>l^Qd*f4g4Ub#|g9D5s7V+ z`Zq$}pd13GIQf>sSk$X2G4n?5)BEbiR0CR^zT)Z|u>Z7`I9?>hz~5(=Y#Hd`W-Y$o z;B6c)!2<&I;ayQs95~&*l%~@oVP%}36Pp~_3Sm0|=?o%Pu84`7JlVC;9bVafStvrnBcq%qrzi>!L4g$g z=ivbBkNA>@XMNhHGQ#w?*&yo&yOzjC&ve6y++EaClteSKv(Sqy%M6nzG1*E ziGue(BBVRdut88>0;0PbmCr3MpI%bYdr>(#t0eQP;_yKD!LA3{n$xk=|`Brvb*z&2lr!XEToWii~3a*wG7e8p=L| z1zm}YO^GJ;=BO*$yHbBl?p3Qm-LUt&R^t0WzIKgPteUHdGN@c(3v2jtV82&Y zAzZ-%h_qaDVpb*IjQqN&-lDjp4_9+0e876kqR`WPX++KCgKgC8wIxUA*dF2KiJA*s z+rowikKRQ3!iFct7Hr)17*t2#)0~`7!1l<>LhU>KE8>pymlwB=*z1yw7yXW4f+C(5 zrjF?RlFk>N2PnTZ{wao8zIW%2bbpDb`-ulGzf|~JCTvA|?iyNbWtG!=YC(g*=J?9Y zhi_G-3{?<5^~DU8j}N9xG71%Z4<#K*dCK->^=Gr)2+G+W0`dmplJ+6|_~jImff6vq z^$AeY_e`(US_MSKBpE`z3&{_t%Tr1lf@eQ8H!6?k79QYA;yF2A$bDh5+!nzsj&urM z5e4LS3e88SHayvV)4L_`H7(HjWOgV(kp*M~fbyW)^UF>c?}XQR&OBp$5+220iNuE< zNG?`Il2@=F0Nmo%l2{KY!TN;kr{#4kmeE2C*bgl4?lIk>W@b)cX0tL7zcvdDAYU0M zCf^WplQTJ`7ap3x6eg-zPBl8=UR|;7j((?wYqXvSC60T({7K3uBq3niy0?(FrreVE zjH|;BV(Z(=`Y(`r&DI z3ABeVuOz6Em(9{r@gJi85G?*<#|N+<27wIHROGHHJ`ley*|Z@+7#3)-x;?2zF$GsO zn)?Z?=%21}3mjyQ028`m6PsjL$Sb)nTBeb2n^p?F2l22;z6oaL!S%w%Gc1yN$lg3G z4J(@pFn485yHP^t{E~Toq#;7*iMe&I&YulD(_IKUKLUblDo`ANz!y4=t_WfE;Z^Db z6LE+J++LyjPr%R1@|9xri@+S_zowp=r-ZP3~Gk;XN3=;R^(TSU_37~sc zMomL_#}yVbiVJ~Z<9lLqn_$+w9fo(r2w>!bV&X>H6O{1D1s^UJQId0{ys6y$C~{`1 za|74?VC4=DW(7q2?1z0D7dz8{o%NGNm`w$B+2!Buk~+xY6V-4PKCtJP)i^o$E_M@k zimm8T{Q$jsY70bnpW*>OKN#R0P0JRVIYb5V{#0v%G8oSQtI_-w}%c(dk|GpT9X%7RI8x>V6 zN4)x=DP;HSZKB6@25Gb6#3b^Jtw~D81YwSIw|fVN$HZkbB*R;n(AKXqv4O7$$QkB)NdGnSeeSqwCbK_uM~~ zF?3RPBVEW8%Bnq~^+x@(xHUJgAlR-_JV{X0 z$sIm-H7)iw2;_Um_z3!PahEfEt5VFl=7-e&yzuNjL%8LQGer~fqv!e={Au^vT)uI) ziQ58Ds2oP%ybY zZoEBj`e13X&)Nc^zLd3IVy7LL&jiIcNoUr#g;X7WFXAeRY zmNpmnEU_7{Mq=hiza^pB@m-fqCsSffv8vt8dlXhQ#!Lp>H60F@!t`vo7%m1@d9TRp zE|*tcjktBY;Cs}>Tp>~}aFCd8;NWTLr4Q(@>4Q*C@=sf}U4P(z?3Dj!UH-m@WwZ<) z2uOtp2uS3w>hk|>?@GkU^!LuIs*$mcsieKV^?w%ZYqelJaF<*^;#=0&Mphe-FsyrN z#am6a=GP%`x|kufk`ir*pk$a=fpc!JG_kunn=Ynbm&h$uBysMnkS&;O@rUG=j_20l zN73y9?!M~0Le8>+A!8#awOUGBKw9De96otS*0epzCi3t1a^JY0a-3&gX1JcBPPV-- z9D!&|%ptQK31b9?(pld#!vcXmc)D)v;2cw@~MPqg4hM83gNfIM#? zvhQBA@?r?1egS#*Uit{u_Mr{%R}J=uKFIdacl0C*!ML3P_K_Lrv3Pa^VxVf>nY(d< zWa=u~>m;Hbc@p+O`>fh~Bhn29wYte#e*$%#CaBu`aehzo_~Hop8RoxoLtu4-Qu^dy zwXMMRt^pbAK^nqw(*(4wdg&j2>)eGl%y5$u);3@<#K86*Z}q8h`63{8Uri{#pNlBJ zw?();aE9mwu8RgWnv#$bsgUd>_a0OLuu;gw6JJMFZQU`wU^b>AS-|+==A>R$FSKg_ zFr_uGEvWGtGYhOAlTT_c+8WQm3w>~7St-V>m~!ivv}Jcn!$wee&GDmQ_*S?z{Wug; zlX%Jj__a*$ha3hdx#-p;HXZ7iYEE>l5d{oOUQX$}XsS%J;>Jr<+g_KT7urm_q(3G8 z09_J$k14S==WBH@H`XFa=K$2=qBU%pAEQ>2^BiP&=Y-ueTU=b7?MmUK$z~}NF4_)( zJyrLc^WV+Jw6x&>I2Zm)`c^2$yR%&*px_@IgmGA?pi!$AAa3;s7UG-=NTRYc4)X zPx7X(6iYrLZT6_&!`mtB(j*>rQO;#0J$0VOSc9Y(J$s6B2Uv_WXti}?Z;{(EJ`0jc zxK`6cjAy5uFt_QYGs5>})AmNqLaC}EDNU4IQ|aW&W}UXBbfz5PIQ5>-mAG%w{3XIR z6&v)2$j4qXM1bXtM&=7$xE?C_sve4vV+U#$`E`s)`|F zPwBIGv05ZrJJ0pX#*R8BHI(amZG+C8*<^8QPR2A5er>z4i&!Hs9Z72uYO;M4s&<@C zFnZdtK2`n7Eq3B&DW%ttLQ^TBhp{BKYLfh27O9;;t~6R-G_71bSEhJ8c=OtPsHqV< zdiUhOcZnQh?%Ql|Y^>`&uRK4^JOa-RF9j@vxRo{5L-`u97IojtR+jge8Q^KQbC8@sJw|P z8Gg!qDx_sV+T&a)20F3AeCYKQy%SLm<4!&$(<42VwYWG(6$j7OF{{B(*ud6GIcGyW zeQ%YN-349k+{0brq%7^&0`P=66e+Uc^g4wray+?aW1ljCRQcF382Vy2=b0qs?A$p{ z(NUB5gQCQ(Nubic#koYEhJ%sf+^Luq+AK%TuvWY~-%>5zbsR>nQ8wMoa?bQS_I3$> z?YNw6lwq);S)_2qgt*uWJ;#svLPHELjY_;1&Gfwo7MvgRRC+bM18814JsPI2?PBaq zra$KC__7}2BB<#_J8aZ%8GUJ!T-((wdN)dx(&_qXU-;Nh;0bMGi!!f>jstIWH}n_b9n za_@Z9SlX^coiEWxu5ra0@fEo9M8wQQ6Rq9|3pLMCrB5SCO=SnjPqPB}-W7M*PFT9{ zjx`TZ#!%<83sYliG9R{ESMooDoPLgJ>|@nDTs637(Qc=!rz>yhk?w6ertwx+Yk*&W zCA+AQRp;REx^!8e{uV%9x5(DFC)U*1d6c%D7FU+`XnRSwvCiG$<&C)~mYn;Y@5*LK zEAUEfU1g+cs^&aNqE7X-17X3{JJT#e6~Cw|<@}^u+t*WuPn5nA9|NBV9ISp4YFT!h zM@L2yM$^3CLjEyno? zO+}jKZ7@S^U-LGZA{2Yd=rOfW_j{(yif)^PCm&DRN70R@0;#Uw-CD=yO}OLL&yYPR zI3-r;i`k5+&X%`dfpQIvS$os}0JKhYeqUY0r))8GeSQ%Ed zem?(PNwI^YkF{!30$rxD6A@d&1v^&xlGu>QG9UDVyj-?YvKD6~gq}Ag9#ynz(VWGZ zq=kDUcV)e`-Bbr)uV8&r)><3OXPb++q0NoG($dCyf6SL>FU?X$u9m#EJ2b|Xg>RVTPLH$20O?95UG9v`pVk@p{VWzfhJooQJNS4Mph1 z&)zKMU=qn&8wtIF`OP3^5veABDj;y<;6& z5{USN7ao+z+;LHo7Ld+h`4XO&eZW2p_{y{-g(MkoqNJQYvyH{yL=zE+gcZX{_J}4? zE=I(cI#q8->7#!j`68xkOcM;EArO^i;EK3RAn9tg8zO2{4pFzJ!i0ZGZ^9cS*0^5? zhe)^yfir5jXf(BC)%kh`zxouCNB+ED>pf1pQn0SRq-F%n5v5NOTe zyG>3Y2&2v(iu!|op5Qo#Y(7HUgBV8|&q*-9>Wv{Dmm6{$eN(uD@8m)#84jmIDQ(%@ z3J|!3zC~tfL8?TLIHt?E6U;OxpFjbMeW0mIkOr-!XL`j2!KWP*EA`Rcs|5iJ5*<{t zhqi}%vw8PNYN+1^!f7gx=^M>#*N^_qQ=Wwqc9^T=`pIBeytlog@ zY=<}Du=zcpRoHw^S&Ja8p6TFvYWD_m5aD_T{F^D{I-{NYsJu*Hx?DGr{4ku^HjDSR z-De$Ebt{1y+_(9jQSXQ=p@U*F83JCtBI{kc##YQ>byk91{^!U@HXTF)aHyNl@ibEeIJ zj`2PQ>`dDh;u~q162y&?)p0p0nkE6@+EkSCL;9oCN^ zKP-P!@@wOg3YD{XZ{*|MtAvKdp_uOr3;{oK638Jv^lOrh&VJ_qn+h8ffZq2s~ES5{8{h zo4`b_1R6mk2u)=dLCDm=j+ZvZY_zsE2_I4-XX{&nz_z}uS=_p;VJX*aoC@ZxeaT>} z@L7Vd^VaU3K6N+RXu?Gi-o&!=viBHeTcALLy^3tq_XVBl)IrDZ(Q@51u2i^l$CY&`5e~R-zsr@-Nvm` z-Ij{ZgaYG=9$`{C;|}1mDw(i*%x;}PTw#1&$+@oUKg66j?hG5sYAT3UEUaZ^*5LW1 zQXR8W*0ncYC=f65qO_$fjg6@Yz5<{YvK`u&C#>y8i{_*yPVWPaU0olUf)seJG6eN& z99fJU?vSQ!QkiDdz^-V)V9VbwY(a3G-R6I2)tzxMBl|9=^wyzGvDz2alv1r{Nmy;( zO(GrVHxPn4m0B-&d~3)^oIsF719?HFuh!*`C%zow5*whw_N$fNs>Y@_ttLC*UQv=r zNvE$9l0@rh5(X6>R`o=$>TM4j(J-ic)$^W@(_yM^p)`s-+7@=t`0-SbD=F`xjFkM` zSH$==j~ivXi#^W?Boe1X4n&#>gBE^W9HS6!0#Kld^DG_)9-~G-eyX4xE!fSh()+Wz zu`hwD!~9JFo!8{!e{is1S8OMtkeD$cr2?g9ckp=px1;R_86PpS~O1HW_lPD0se z9w#KDzcs~)vsyljju`P0JIq+okfWR*14QUwsp9$Yn!Zur^iG+TEutjHTi-EJI2hVy zIiUBZzArN6-08A(i0a}ugScZ@$g3xp>LvDyh~eMX^ulb2ySaxhW13m>kCZ9TCxf1G z*dsMIzj_+&Rd)81nO2r`MVMT1pMTw*X7h7Kp@8(za_th%VLSxmVc*Njw3_n?HeE8Y zq^9~R$L~R~d#3M}yXcJqHE|h}O$CCwI={fP=-YKL*Ow;}|A7Ak^MVhRYa8!ub!TNO z^@LT=4-aHRJp!m2kK_A}k)31NOTNC~$0zR%|+LDcMCNfVsI`uM!^1=+5>)@+e*5|TKlGSbH#}I2TD5-na zH#5XHR6a?!ND}YTI%qq0pPtxmMtD~fbK2>1>ovzwt|pCI#8gWh=%%{jFR*9jBiO&ID&bTt6Ip);~et*Le_x zy9O9yed->0~HP7J9^13(o{=(F)FO6pi(D`Uau?K zL+GO3LD=X@3+Md{h&NYpsdI-q&gwtdViQt6dGDWLmI>g%>xzEfV;NWUG4Tr7-0{W0 zo#PFN7bUTC3VMU=2(fifBvP`^S$I1HEl2DTim#omaRzakDlam}9Y`uh2P2Vk9iU3+ z#d(aoQD96Lw|1!H%^6&BXZ z3=F)Bb&Gh`-orMUSN+ybj33 z-lm`nj3d9qHu(I^LM#L0nEvGcW=e1qe{>VR1eTMqQalT96K8l@-wPOkBlcddit4CS z-RyNm8)SjfmMRk}9Av8L8(pj%;F~XBux)gfPhywMVRwJCU+Uf)ck%&TXnpPh^HThQ zw-rW#DW9_B>sQSSN^kXFZ%lsm+|i`<^QYP zw`l6P<1C?l#L;=$cp_5}xe;3kg{F+IIO0MQQb6dqohu%6G$3Qshc>tn%h66nWGW$N zqt8287h*^jJ}E;Zcv=vgPT7rXmn!f+P&Aw#@vt-^52e{Ks92v(&hfup{JONicEjY{B>>3>aX|0ci$WftFCspW>6Q~q6f^=j?d4E)mLEW2;*BilcA&^pPqKI0$Ku5a zjtrGINmO}N&g9&Q^MM7|96t@kL|0l=jOq(3&T6#EQ;Ee9LFj^it{Aj9DC*zrD0i@gcl6167Urtv}7ce<;-$SviCs?td*NNp+*7M# zTy?#{*K>_vYq9s)$1JTc?x&rWiXh^KB+9R(2RSjX0K0Q1^)%m!2CfFSHo>BwAy^g9 z>T?+ly!oJWHht8`V6DMLOLZv`M=1emaK&htJ4{B+)?UqvNjetr<@CR^fxQwK5WYvU zMpb>9WRbQIUmsthxVh;%&z(tnezUY-%f45weJdl&#SGH$T+@+z>(HbZTfjPV28G%Ara&ijfw9A*W+|&!wuJe)*t*uFv zBM?UfsxF384Q`olEb$j?b}8n;opR z-`&Hb_s&zPDWCQCcjA$dF>v~&R)ywn{-_ z&#LpypV4)D$`jO51Ma1)%r>Vt-BZG9K0Dad?lRn5fMnVxTIW66AA`r8N3l4g5Ev|> zJO{MW^bC1lz0$aS z5y)}KMPAHaXPmva7!SI><#40~rPhBldrWh&pz)Hep@`gbxL?W6=V4S- zdPdhUC#lR*aw^RRzm&WdW&I*EynaatWIfrwEQW9H39+8W&&QURiEQBpvo;yiLA-QD z0OB@Ky0{y%6ZvDi3G-NHQO_qI<^4|87-uj23o}{xD$BDfV6Iy&LZERWh@r_ACgkZ3 z^t2EJo&*c?T#TTFQ>#k?aG9ca*%OBzxMsbY9pw`;+U`5&VS;8$fKVgIgMNl%p(pev zFn7x__z^Wd+F;?{%yY+|hkY6s23pzbu>t+?W0ma28_XX-bebk`-|qkQn&DKp?acy`3Ac~IqMEb{EV%_5j3NM)JVZCltNRA+*V$d^$S=DuYV_hFjnl zX2By&Gj9ZucDHbQl4Xv7TvYhj2_qeTVSUAZ^?2zy-jFYFkA|awt2XBeT#;vh$MvDi zO>hoUpU4tz2)^<+P3b zfIp;R*%JGb-3Jh@`eyfKjIYS36I0`m4JD>t+SQ*Y^v;nK;QrJ*n)im~4Ol?UIX2Ye`p08Zg(C36_q9Yi?1)ob4-9odM=llJ)v-g0+s^o;*Y1y?rz2S zksXYil2vRdQQlbBZYC5%mBp`x6B_*@Xb18|i97Qzh4iSbkD$lnHQyPL{HZ=N+=MSn z;7CzjGWb8Y<+{JRRIEGn)`w3CQjXe9A%qi~^%Lb11c>~6{`tlWYF03_fE$rPGZchU z{nJFVN73J((xP6opS9K75<=(Ibzx;{zm;oDn4Sccna4NrJ8#?mTq~kk3Gsy?XIx{e zQdc!IY#qVuFMOuqs;UfC!og=is&Al(@WwdX`$596P!!xb;X_h7N#)2(ljYq z)pz0H9jtq{=j@k~)V#H?rCC_X^%dAHKE4nm>bBE+W>EPffX zLoL2#XJQCoSRr5)yO}_WVm8lOY%xg`!V$!}P423~#UuG%*!OHK#w>xS(@c_SAUm#a zlxNBp43J{KijbiQ$uy&&#Z(E+P3#LeyGGmBh`08PVN(mFKDK92Bn53vrXT>M?TbM< zPBR|^{m@KniOUF4Jv@&CCA$~LgrR1FGRV)YARf*YwGHB8Vqd5kiMt;T4gCUEN3up6 zco=)hR9R>SBE%<$yQ?dLA6lnu5h*N#)}dW!3QebB5h}d0I<(UQ?JG&dpivIZbWcq$ zhNeBmRx#BsUf6QcA2%DYx3cmj1ZDHAs(h$l8m{xJ&Vbx%r4rl{b+ddicNIlVo~Ea| z`6)k3nvdA(@>e__RI6DB7Oz#3YY8pf)mDCO1sq5XGcD)aZJc-(ehXqd6tk8vv2RCi zo{T1G5)l1+2}b8z(Q8tRlA5fAQ4)pZvF!7%A(_QZhaG`}25oi@UR$_}2oBGOLgQ1j z`h2H!D;1gSkrL73wTt9o-TK9D57Fvh_NpS$m9SrDXWC6ijK#ynbDJ{@i|62)#R_rQ z)}=RC*X4OwJEVBs=ILYIlH-NSlrUbf!m(Xda7g428n4Yly;#pn19mry=k(738CHu7 zUO}kFy;i7YzA@Udutex{qb0zxKks!d65tt=*tjLbF{iP8GwhU3)no07;~87oI&%4| z=0;B8`|qAo!&p01VDB9}WQse_@I(0v=e`%6B)br-&{#IIUE>QsivBFmc=?6x8->5i zS3Vcq04zWU!CNzj)X3^3`QRIduYQUY%P*6^aC+Y_U{D}V^e(pDjO`m4NA+MDi(fi| z`Whu3zkt%^%7!~g6dQIb727SXa4WxjYH3AMll^WP`PJ`EdigeN60-k+E4BC;48;;1 zxwz5%ST3L6C~kf6z)X_fUc%mto{2n9JYdLE<^!wP0diZ z+EgbW8CvE3@Hsj$>#WJa*vW1NgaFXUg<-ascRw=$NZm!xUO=S_g#SD`CI&8}W^XF^r${Qr{nr zE4|n1d2^bY9lB=~K3~z+N_x-MR%Qeoo#|DO<-4rOjIq1@#4Z$iLnY^QCfVx_ShYJV z%l(O}Hy)is9z|b0>T&JTa6jqsCTepldP`!LfvwZFxZ<{4SAA)5I$9JapLlX>*7PXX zcN4HycZ~yy>E)h{v^tY&5zm=YQ@cxLplc-hgn$WrUqErCr&CE$7`T+)@VX3qPyDJE zX-c%I{hM>-WVGmh*gD+oO?rGE)MZ>6QH;pZLuyz@68VMEA#loS7lL{Diy{>=ofs5e zRQtZ4dHVoAUvC-=u1t7otzKe}IqRTk&0m2vR~gi#*-LodH#0vhEbJ4 zdZOzHY=sIY#*pUf`7%e)j?qVyY{GmSR6S~0*_XA877fYgHdaODCBAH6Oqaq6m!lXv zNQ*GxFvRp?glZ_oC`-{*uFdG}Qof>wKB71EWs=h{Z4*kahhtqv5(slZ%)*-Cc5_v+ z#_SHh8~4)AvXj>g6jkFO5o0}r{(r>1Q*dulx9z)P+qP{xS;0THZQEF}ZQHhOd&Rbs z75ikLeeS9H>Q>$Tc;Dun594LjtkFm7t^KaeVxWKZb~nl5Cc z9K(yavUgXY6~+1DYoT%#u1h$(0T}Grx+#5orq+d}(el%}q*l>7wp;jqjGK)p2SXb% zV|;KLX^JJ#AUk?%;tPH6M6B3i~Zz^(fcf9kWu`zA(I8JWV*%> zku!0+$x^VhK|xNk5_i(F@bDL#^<_=Y;RM>`rqxcInO=HPCxx03m82-))Uc{G$nNfr z>t&r5hXKW+c3fNGacS%hue9WYY7S_(exu7B%j~r_xPU-2CP+s<3z-ARv^MRf`!A&z z2yq9KM<~vU!Zb%{uAow|jm{7CfgN`Hcf6A51lyzsZb=Twbz+OkqHO8japy+w9pbZ} z?}^TxMLeWiRONaLwvsNA&rEoR*zq^-WO&A&X=gyMzNtlS0MFQ>+=K@qNd$nW=t3x| zkLbcEnUC-S1=*)=l&k`&i;C zkK!vErAn9b%t`y=aJHqCnTssyD}!)k?UAzh2GgNx>6x8CNWdcHs-~hlvY1)g!7Gei z*OQVpZnYqgzzU=1-vy?n`NgtQlvP27f_U8#mj@QsMTWZY`Gs@)T~BfN7|wt8OCvfS zuv*A&11XGSUdXbC zYfsTWVz~553HCUqUbWt!w)0kQy*%;k7PQSw?4^L} z)Wvn3WguP28}pCyczxDGL3<|m*vo0FeK1YS1zHVpvt5BUY+uSQhexh$E87hhO@Y>U zWI}z1EP;t3g1WyB~;~36Bwku17 zRoCnFhf-V0E#yGKBhf(Qt!MLjnqhwnbA-w^@IQ3bPMA$xN;UC%@`&c`0U}h7NZCT` z%Fa_#%h@&u8tleJQv4tAV%tdjSG&KZ49Kt0?m|}o^AIN*^ zt&A;Ow3b#ehwfc-xrYWkJl*MPxPt|${&^4N%x*9T4)j(2F8}nF+g&hrB_UJ}=#l|W zC4;ClFvMkf9GAS`*FysKA-}ld#pw?dBbcd&w5bPxsYjqQB9fISJdDuU6L6|oaXQOj zyPdHDSRs(8NAXxOp69)BOK9wq(092i`dZ7O&C0QNO2yoxdEG-*^@D1N3algxmz8C# zUNr`LU915hF7kLbC7$bI7iaA2f^JGATL2kn0hi?{=!S}Lk3H*BWs>BV zvUvM@6!yn5bD{x$CgTmHcc_71La+6!6W1+mp}PBy^aGZzLZLHFeEvP$%G+>7rv z6`u%C{?^RfzQa$Ud4bWIZ2euQ+@J2|um*Wd<38`}%M%YbvD)ls0r7V8$B8Gf4~T)w zUSR0kFGp6~YSRz+Kb4WV!Z5|gANPaE4%}UCXs>$DEF-xyCu+GUemIm;1^5iEuf=Ss#?`Q zs`ytd1^fk#tbPpEY_{}KeZT*nBM8{(&WuB3w=zupFugw5bn3b9GW++ooT>*_2W*F< z9uwtl%y!7i2|f|OQokeks|E<62Nw?8d<2M#n{oyIjMI9TA9voN*T3*3&ELpfa5T_y zU^vv7*LoMpd$Z4hmzp3J%N>^j)*CpN^_D6tDOS$Pomj#)cm7TTE|+EQ4+UA_EY1sX zPC{u<*`5Vnsk66ajX>d^21xWv!{%EAu8+#-%HiP!_m4FK4z3i8TCLQ{!Y^9TqsXCD zFmGCN4VVq@ENhD`?W%bTPt`7j#syDyVdAoBvneZ<7W~7%iDGVbukF);#*Pf)@R0Z@T0tN2|t{XX>Oa%y3&Emw%4LsP4rdv|07`FZbA+5~oi(4##@Mm_!D~ zjok1P9SRGtoCD80c8bhq$MWm>`S_rd(4?B_+< z7_-SuQmiHz^Ksy0rXV5=Rbzs#{AR`mTN2T;P1pcJx7yH%ebV^E}9-k zz(>O@Q6gy3W?-i7Exq-n8?KGIE94vu_FBdr%-9w|j=dZtbxlL_RaaV6+*~#Gh-o7v zTPXsBjHP4s{0#IQ#rrWz3j(lI z{?KfwVQ3s-n=%;QP4gE${1)@KT_JXsFDW5*R-C1KVCWF0y?^EpU^^tWG!ra%Q#_e0 zkQ2!+*-H*LPOCSJbWr60)m zFvXMdxNkaUi~`&WHF+8`T@eKHEeUH@9!7Jg- z5^d=(Y`%jpTjuj9<#jhf>FIk+ex43w8!$ZWw=tbsJnWfh1VkmaIZrHwd`v~rfLj-r^Fk{Y*BKnh<1xXi&Fat0R)%-iIgFc&P)iiKiSEH!P zbSBRlfmp;JC|^?%Oi=hxBlmbKF*{c6Pk&e_jSRrP4EJkaGQ(SM-Bhn7WzbA%hw(V7 zh5%<5TD4k}Ljpxa>@stfJ{Qi)*tPQsRNE%Nd*jgpQi?ox;07gfwYV^}SyJstVX~!v zq~#k$4-+i&l#S#*xCew`Sz3FI34>aML ze9Dk|)uN1~1CY9I^;+da%iN#~xGex19~b`)cf=ijFf-qQ)aZqA@1wr#-?YC8S*02D zEf>0WSqrjVnLPZ@HUMg)Ir(g9@NsW-> z3I=qrA*`fjA5ul?tIt5ykxW5e>SUrVP8Gid2Q(G!S;q5b-Ui!DS;QC}TUCBM0uyFM ziQlEcVYJv_#c4_K`bwHcN$4;y^X<6*N^ylQLyQ->|6-3ql%~50^ke+#Ki#iDJu0#I5ROB`=4zcy)fq zOq&mS=B)O8wo~?!=bNvV86<{GBVw;x)#%5Bcu7oeH|fU+N=pX1n_?xOMWHX1Ulmxg z*yHpGpC`^mFZ?eie8w)aFvDqU9J=|*^_pm%-cpgIuR|~hpiAT`xliJOwF*b z10|!FJ%B=-uQ$|*;wf=5Ko_o0$r&U+0EyM~?uE=-lpTUq0gXju1L-t&#>(Y1F^_nh zk_!M$?GEyAxu?+;a#!PMGB_<&dowwh8ON&klE7sM<>9Cs>;4LLw ztHm4?23$L*BYaSE%9s-3EGdj4My3cB-$##u{4bT) z$jsp)jQ;K}iDNhMJ2GAm+gyL=1ATjhSb?Tvn4LAOi<30Qz8ldgRi&rO=ufL5_(6US z5>qTvJ!U2*K_-^T)#h4&O;jen^8v{!Jj~7Luj}AI1o;VD+#!72K?nz2f)PnY-tgFc zjfZ_M(mP=5g3}YXpH z5zUmi2*F7-dQ@kC2LiD65h-&J((P%$A}*>Ntr6u7S(nydv<+1d6bHo$O^3 z^e9tIapp{Q+99Q;as1;cSce2v4RHWRBrN}s!mmov1XaZ+^iGRPm#GpU2Oc_8A*J*e zaMLe{NVW%Ngj(W?28Ayby+V1VqZ7X1EeAhII4Zc0uZu-<>m_ZHIDIJ1^eqszxiPcK z5a+>OLeq%KFe>(h(EU8Q5h1+Z-yo@ROnG6)@zc60_RpXH7Z&{A5lBSO6`a?f>Id+H zC;Xuk{s$rSKhT8#8=xs|;OX&Sy&y#sBRfaq|1N>L)!zTB1p3}4Wy~Cp6QmTPOtJuL zSfkbOFb}?^gaZ{$EFlMqdYm7>HfAwqZk(AS2dzks_^WDD`8dD+M)|Z?XlXtPj!pMi z{rK6^^T_cHCO}YqvYozlX(XHx@Kwj-cFp(fHNzKs!*{>$m*a)shyGEB4waQMAMm`q z_t^#*fOwhO@;=VtUfK<0<_|&uw5bjFAZ;>;jEIwZE0gC#7}NHTzMRtcJRczQBlT05 zut{N<4&Kbfm_D0g% zuhom$Er;wG819+av1mX?kb1%POOM#C3Mw@jdTrs%iNB@Y@sRxuzqt8vk4(>~?DmYW zZ+_X5?L5F1+KLBo-Z*QzZepQLw{MdEE%ykIx0r~J!Cew<1*O!u0Oc1GRH3t>zsm4r zg+<3zK=HdN$;J$sWTY*}-i4yrPI6Uvvu+cX&zfTXhHI_pB+&WDj7w9KLX4(Bht8nG zlKV1gW^T?#{cU=!o@fheE<{=ESMam|}a^*}cHY?idYWPn-^!V{ia$|3Tcu<=rJLkn-kle@&e1WV8g!M4^u3F3l>0=njyu*;-c2M?`q|`+ty{fl)q7E}9lonZ#2(u;#Yk_$W6TiQfcKCeS)1%Ik2Vkm; z)-q+!s%gwy9BNR!aROBhxHr-a5O39k-4H?P_>0iSUXrS+mKP-CAZq)937d&gl@eoG z6a#qS9?YyAVj381Tb&=hbify{(DtmATu+w_k(u|njcmxZi|lrESZH6YgSF!r6l_{3 z1N-aEg%tj=-_>%$>>uV#<}EVtv)_t7UM)9nQqAT7^{&cIq7W4{8V*x9mSU$i3e58` zkQhbb_eUIxJTUj@;^Wx@pSQXcwr)ks^>kJcKc)BtuR~v+3(`l~K%Dxnq>1&mPw4Hi zD`?$5B0~f)9zn_tJvKBNcaR#jTbUmtU+Wzn)Q-w6pLgy|FQTofI>*I2{o>MMh9v0) z8lVbIYiS@VV*zS%PWk{;OxQn~V9wuc+^+V(JTVIh4YyyRhrePs6jR?kXW3=du`xaB z%H28B)LHzXPmJ3r<*GcxVjaa(>d|_6T$38ftVxXQv$*F7TGyCUgS!&D#!@}Uw%GfZ zuTy14@f$uIZY4TzrDaV;x!49rc}QuA^_EIIS;}=G>=|{bq04Ih=n`;$3)AME-@=SX zrv7VY$&908W)K0@3zIMLUMaM1NTDbj5&nZD)l@iaHPOGUrenufstN-zfzevh56ynN zTdfY6<1$5Rcd*{U8?vqhz~yfIQbY6iY#Gmo8FSRC)r z_6okLjsj5SSbA%Y8h8^v7h$!CB}jMm%PA-HDPa4^L{U=$-06;8d< z!d-?unzLHyuaI7v)sPhyQ9Y_qrlnMN{8z&wt`@U*7U5o>@x`Zxn!uPCOg z4*Kf}97>#6gz5{QHd6cJ+2GO{>w@<@shk`5)jx*T{Sfvh7!ApYVaH*hrxYD}nBHFu zXrT>b(A&kg$7`w>>`bTMf$w}%aP@Ux=B}ptL-iCd*%us9n$*Jr%G9m5%N`YF1=ar; z9<{7jZk2vwSFPq1kz;?qe`~2SNxHE)+#&VW9@iC#$!)+t8pW^WFPLa_EE-tWR4QHL zb!-aO&Wa?;WUGOPYj%D4_YC4WC*pTGj)=91ztorW0z4U+B!S^fl#Ic@p<5ISa=VSy zr83!e-anKk?QS`O5ATXy0Zkh_?F}CXT}lU|#0bj?Kauj@@qwz!mzvS$hpS#RK{i!~ z>tY2oyPBayO?J<}fp@;qUv3K`Mdw+%R z@e(WXvBW{(T zZc30SM#5i$44DgXL*4y~#t1AX!!2s1$0g>nJtgtaug9XriaL@Phz_|v*RguJg;i~; z39lYEI0K`t966(YI-LBr&L%#?Hc&%BpZ~_EEb0w+))Swlv^Z`l57LZXZVPWxgVOSe zMAZ(oeTakOh~z+@dH`pqE_>rRC)*tO607k|w6`ayel#MqzY-EYMxSIc$FX%V2AgJ78~+1PQ1|xH%^2g=S;f3~y~6c`jmN zGf?Vx^Wx)#cpUl#W$(Bya;~|gScCEsBTP7#wR1$7LZ>!bI>#c+WY1>#rAiTb7NKG9^ItRCjf#8q_N&%H|Z~MH`V5MWy=IvAb*n z8#_>;QM`ndVcpVNm1J$myy5opoHLxCWhMyFXZgFF8z75j$m(-Tfof3k1Ht5K5NhDe!X$g;?pO>>)9 z?rOV_nWy8=Y*U1?>^A4NWF#A!W{bVLS^K)cU9py*Uj=>asLGP=9M_ak)KY%Whhe zl1tVdyG}y21BWg9)G5YjZoXe1!)G3gqN#mIKLhgt!@ochelv~||D|~2ryQpozUQ2i zuFcPbo6y)_+I>>+9@-Ozu&E{CdUD)B3akKW)gSlSp~x|f#NH(}l!(Kr^g z)VSB&{*@azvU4szcAtM60zCB$tjC`_@}CCawywD(hXjh_x-P)C{MARsq!z` z3UZ&*6Lb*xJeAe*eB+z-_Hn^TCylF+eH&K6+(@6U4D>x_Kqn3_$lEeW-ADu6v>aT2 z*E>gw)dqLm4QE`7&aY9SPrX)@ZWO&N0D;l?4EVknOhRFDCk%-16$ZaGE8IMS)Sdge z^gq%xn=@|O1;MNxTlzAXbJEl-kx#?g@%Pij^^vlbOgi&`? z%wBQfz<2YeO>cV*`2uR;ICgBHj$1Qnjp6Rd7doS+=e&TQ(FC ziv?AJl3N0*1pN(x)>^bWoEc2qv?$T+JMcE>HnKZ9>D<5={%qDxOj}74#$(JcRRmF> zK>gbd3@80(P8`mgAwkUa_1*a(twfdoXZ-2CXJ;z?JKgDcYW!hBU`H325hC|EADhj! zP2Q^X(4b1GJ}N(Fc(Ng@m(g=}D9Xkn@`-Vjk7Y{CYh znVj@c4{X+U&dy$-iE_zeO<_-=1n@-8{G$dy04M-a#gQg_fUM$7vVOz5hcb5=G0nWc zSlfsIah^}j*hL9f&SreN^>Q8_BOg))y6WL*cr%`M7O~MzHZz~-l$yx1yC?e;l1xu$ zpyFj3M7;czzqXisYY9|*Yc;bD*fu(ztayMT&#|bYMkcI_nBsCFMyG-I9h9ZYZ39vc zs4Dke-4F~8t95j-gL+ZgZc)P0sxFN_MWd$0x{(({D89LkA3`{ZrdC!=)IF90-F6k3 z(UcoJ_1kZMWWChCrS1Eg1%qge1Ksm8Q>;Jq1`-W2c|=*O`A6r)bSrZ%sKemsZDZxy zNQdZwBPtT4zlqInv^63=UdZ!p@eyR)AD$hH=4$hmhJhfNb!-06Qx5ttE!=f*wJ_4y z?N$)myQUi&u614E)zsU^+&p+`lP<0XtsplImj-K-4Y4KylRznfw4v(ojkpF}6OK_w zoUWDbxquvJefHGNNG13zQ?9{(K$G+-pR^F_@C~?zTvLugN0gIHDF?I!4c2CB{z||{ z4{bVYqgVSUm)P2Q^tyTUT>dj_z=-zlC{TPFC@YFod}Q**GYYOmkV)HSLTCg~OK>(> z8rEx84UyAq*7fbz^{Z298>PcuugBOE6Iwp8B_NtsPa|%s($BgsaI$bKkQcRz3GRQuHb~-YD0iDnW$fQ3vn3O+SHY=pNz|Prp_L!dVyTa z!!%d9+#fxKt=vF~Z~A(&TUM8jT)4ecvUQ5EJzgW;*c(&xBdNRJv=VmSbYAz|Xf?DL zzf8qez(%*9=hvu`YzqQRGpr4;OBhfrgi}lcmhqt5Eez2GmLeI?;q;5l=;iMSjp#Cj z2|yX9a8_RDJJz}++EdIRZyf^#M1y+9Ds>rA~PPk?e zv-rfkk2^j{4{h?xTC@F!GUY)or$58dp0;N2PxKf#k%9?h8|&b33KKwodo%9K%f|V8 zpcJZ_T$X5x?GEc_q4J-FO8;x2D`WhH4~=7@t5TK?8G4f$mi8#FokK^{(D7L&j5OZB z_PN9KDI;m$oc6IpjVU9R53sB{BVO{p2t(;X=YNWt-uGiCgVCCf2-ryErL_uG6tPAi z?1^^$vTqZ03*sXVS%d=Nv-T+;xIF_ahafNb0&#jHvoFCP*uH}|ha9&QU&Os+w8FtE zt3&QWpOJ=psKtjS1;Vt1ZZNr!2^WD)Qt8Zr-X)6V z%JB?}=gHt5v$)C&lj;)i4q!-kiDjs->Jm$0jcU>-x}si|He?`spuEc)1Sti z6-%6$;l`XbO01Z-;?Dl+BeSO;eugurQ=3>OS!7E?rMHYX$(67$XQ!SONUQyzau#LM zZ4tnSxhuZ6Xjx}V;>8Biz9|_cf>NpxsfoVc{8X7gH&gbJG{Uc45-?Jw`mB) z{FJCV(;}=NX3YrL>9Zxi8G3@<##VOuvedcn^#1d(@Zea7RfG8T>*{Bu%a3*Xf6&MN zU)osOyW5f|>Zsjn%UTNER?8m*??1p#LK^)X)xj-EEp18S0FUsDk~4O)%p}rI#oK5* z@4|wBfHFdA==1&+u8BMGZ^~FV@->Q&>q7u?MXxdc_;uC>so7nhRxr*MqGtC*>sl zQ9eFP_+-M7*!F&!AWXx)&(eoNu(_? zjF$bfB7P?lEUuCaMSzjl+eel~DiLfoFpns5%D@sRu}VKx=g;QLI?#$Ea#BlAZNdd_ zWSWZx?ayvWre92AWO{4+b#w81P&Izf7eD0L1trg8sO$8Nu21&bDae$Gxj;o8bB~Wl z^iLcqK;>YJjnyEHokBwZvl+ljD;GGas75TbdJYmKX#;oKlAti9QAW4Yprw@qf3r?K zhB!7N=#h|VAu>B9Y>V7zYs(GY<&+3r*M3%!u(?@;wP1Zn-h6RfQrXYLfd|FT0W+`+ z**ZTqUk&mM_s1x$Or3?gi{QNm637~Ii5vivoDL>@b0-Z6L)`_XgAz0sJU-68Lp`}x zks?)zE)!0~RtRvN85+;Xf{Tv%yMQNuA`MH0I|`g@aF5nT;O=n8@99}lm#T}ja0k-3 zYh8yA0&VYGXLOTZwOa(0ae)GDC_410epQhC*g$j-VkFFdGd z(q7|4B|GAToY{e#F(N4LJ}cMHEF&0nnI31KXyBl5hY&XcYczpHfwNYXUus=7sx07& zC|R+CZV{x!7}HChn;uIp*J9GvHntS_q5XSGrC6ww=bX@N=bS4ib8+r9i%<(fOtmqA zeRSSQrBJMvaV$LN-3XLo1rC|*iP&h6i#!0S@C$7G!f9f9{;=Km9dr!KOMRL; zh0esGd}}gws!{q59sF~*OW?M>&!pJ~KBWSS*|DU`T5&w4$o3d++0wD~7r^z<8OM%b z?!KaUzoi(k0;lDu*Yf!%w#cRpN!8@RLvRHxvFBES7U-=eUWU0{0Z+}uGBW3W%0W`K z!_lasU~(iEg@j8w!x+23;j)7tIhr=3euLNuxTI}vl_+UUx}e>wtzW@{CRfuS z0Jn`1t}O#}FX{HSf3)`4|3LdI6Fkul_CR-S`}3(gNY6s6k3VdR1$X0r3>k{C?GLGj z?YaG`FLqf*OLGf%H~>2T9qAN#0B+Nq7YJ-q9l(w&q*)nHd7Ri!!B*v9g5TeRscWTB zf@G+cw@*E!UhhiXveZ70Qu#P!sl(Jeq}eu_j5)%;Z;DH{UL$xUEoxO3criZ=$Dh?! ziujDWk1vaR9sG`&_0EJF^EWRtv4~RzwG{eWy#Sv;VPO&A={f$Z0AepcPSi(Yi6Gn@ zPh<%m&mVGL=aPLBvcV%O(zOqu5CAyg(7N<{|;GR9{5LR>$OW&^b?e z7&(2d0cN+N5s*4V{wkxe$KRy1SfjB*26`f%NyPG_kq*{o1wg$zp#VNp zd8;Xr)r)vyNt(5Sm1Wd60<>+zQBWad!D zR96BL6cZ7T&|Csi0W|0jq@>^gatbISqJspvfj%>e8M(FPmfKZ<3lPnxUb6gWJBcRGp8yy= zgnfGLcPlRiIDUZ)?YAapPZ2PBXi+nJl>0aJDqoQ-dY*oyCZJ5 zv9{hAJ5GbhABlb2ZhH{)AC14h4R=Gqzn0-fcg4ZKrr`>26+i@B4|eVd28JlWea}OP zKc?V}ZgUqt+HrdO{7m*!U+h4>4+g$F3~+tPZuC0SpE$sMF9x~?_S_sd%yxY_vLyO2 zgXoE`Wg|k#Aqo>0Ww4Cf#>jfpfgL8N2c5_l#WbV?-Y}D{LfT)-A{}2+-#RGOeBay4 zdOwb#nt^D+vCZcQ&#seG`)#p>#oNrwGGmGg509M|8L{bGk(jl6>R1(DxHN4?jR#Vd z`-BS=Y^jTgE%4~zh8=MZN|3o#b7Gtp)wy+ZM%pcSW$YKBSQT_B+RKy3Cl^sUvL%#$ zBBYrm^UfT}3hkNH<{^l=7IWjas}hZ{70!dE88&mc99d$GI;?6TSrif~%>J+Rn)-9X z9NK*s6MbUB9hx~msLV)~^hq*&O{vm0^K?kd_S`Y)~JNZBy+dMXU=nnJhFsE!GqWm3mdG!op{I>KAP{)wS0{Vz?c?X4vT2aGk z{Fk7=ofaTySWOCqnY=$kqg)r$fw?9@o)_2X-*SXb+;@ez)j~LD^X~a>8FlpMg7MAs zBVjvML8_g@S#>SvOz>A_QYRLT%rWbZq9>@ZB?fX_;qO zPBjsD#aF_sUXMHxx&%2en}$JpOR9-Y!M9KSQ%@;~t$f?Z#Wpr?seMi6rt#ggZR0np z={u&yT5+61A+`&6j$BhWTN;zr)X!|+%0sX7bH}fA1F!G<-kp2P&OsbEh*}}QT9ibe z;)D>qcI_U*HZ>1hCi7G=nz0_rGF=yqY(2Ab7*8bwpD-VsKm{3T2KNM;cn@QTftf(r z+`h&k+nq?}LlV@>d^gt<+T%k)1f7`!O4*SuZDG}3aln|&t~I|s{Un`uF{#sWqK2`E z8o5^ypGGqafr36{r+wky{Q4s`b)^5;#@4DaeN$@@XBmI4JlD8P>smWzXY25vEWBr;N1IG0kkIy zeD>t`mGGi8z*FH`Auw+u{nMM@MFH@9pDytdD3KJ}dWtix`XWwC4775x8f9Sf5{c5J zh?%q>>oB^F<&x;Q>86s;Q%k?GaWpoG%dl{)Kx8u5XN7-@a(1|N)fMemia>%aung0R zk>CX7PT2c^?ggm2xdPf*uXaMxLbzqjoR0<19gq9EGVC3ute}jr^72#u`WV z_h?Ss=AxGK$1H;iJ@u2(7-ASn`BY`}3+-cMAO}75C27$LAzYc!{P-lR#lPU!K_a4w8GKEB7!fN_bWF?ta)@CrF_@?( z@45H=+g%fOC?Qnauyqr@fsi?)=;>+U!d}jcaW#sZ%R51AaZXXt)cF36&uk?^%pD#h z&N5#mpq6_B`MVl0p>U=Rq)Hrh=nF$!pGSE&WnNV8g{?yswf|*0_q%1T0i_n^!@)y; zM9~=i$Yn#_kOK-N`fUB=^?6auhF;a#7DT6G( zAvPeZG!T>{Cp^w#G+l`@i@HIp5gG$w{kK0Yr-@IFyet%ZCte@BS>}S3wDv7osF(dz z?5e`g=x5SFxAhzic=tp*N9mR}9p-~ppl&1U_&HB+5AVRiXa?~f_uoXYd3}0|DwMRz|XB|`01R!qIdIxgf?Q9Dz$U_f2`NGaYQlM~NLD3e8Sowk^omKS7Q7pvOA3P+usST7rgDfUj^y^<;{AF5xP_eIM z_3Sf$mg56nCG*d=lW~JT&>?2t^p(LCqM0ij0<+K>=Uo&%QV`CWbNyjg`l|quBn92Q zmgxA@@Cl*A2JpK&?SeJb5~q(B*0#Y3Xd+JBt7XCJ$TYMnl3 z(6}k2C0BBpv(LcJx;FYfkchOiZ~I(2%FITf6{D3wuw1;=%r#RJ&UIaE4B-@zcCpE3 zmc*aOTzS~t++@n(*2L;NxfgJ&;ih1v_Le9#Mq`j_q@4{zpx)OO2mJPEYTBq#is@*| z8N#jSaL7MnwhzlW99+=vhe`PPtjlkv8rzMT&z>Jr^t+<<)swZSiUj6(c#+;+qCi=} zzv8z|G@5Jo9?X||B~!M8YWHwdqvy%H#r>F1eG>We1aKZO68VdI8=mW+#^VZK6ZZJ` zu`D{E@@7SY;pumz-!frY%XFR=Q92=53g#u2@?vG)cs>KwPU9l90{NNFl-{O9MVU`? z$$ez?><0`MdQSm~UoziJdpxV7TnWihuR$Wdi)^COz3C5nhVd8#%GI+Ob1-ZY1^2*< zE8131KR6Bc6`LcoISz6kfLHKeH%M{P^Qi`zyFa?!6ZT3@S;Hu|92rfqPb>w%O8JKU zyhhZ;?n1hZ$tW<$G&}WDM<3~r?AXs|6RQ|8Lq3o3$M)HFASW9@disT!x_PoAev8Uj z>iYYlSwZ&F43$Pfd!ExiG&8(j#kJuZqU|PXlNcwj7Ppv6SPw`SNsdHGxSZWJkW25G zB6m;I)amvm<(r-X+z;;q+HOrwHM|q7DkZuSPoOxU3&Dp88Dy0jQIUlQ_EbuQ~K4bf&wmp zdsJY#{wn*N>7Zcj3Zu~2hm9b& zIjKBacU}R^HlZ9mVNgNLwyr!}F(ltbIeJ%6Rv%#B#T%`3&_030mabocvLssLc4(4N zE`>JQiE(5SkuRIHQ(nR?DRUq!p<9w!A)V|bNy>l*DNd2yR!ff@W^ob0#=$1Gx;;T% zS;C-sjDj%=mgJn)uafjJbEr6>OhIGXp*a1UGe1?fVTu4IsrF7B#dbm2X$UU%tfcd5 zugELhd;r<6Da9+2Z(5A<+=KxuE_v!7QM_las}$7;hg^~7FuumL8Vn~nDN^-V&X6=^ zMRs)13fQ^ntC1DwB;&I1S&u|hndUS*Q_OTpg+gTe1nV+#bCOcq@Se-wSV&z;$x*MA zPr>(PgRH`&XqnGpi?%Jzp=*UTzj!9dzb-}F(s3I!VFQmM4QbN+u^;ym$gtSoh;uzH@c1Wml)!HSNh!+}#f3Yp`FqFPCp$C+S z^4H6VL-w;;&v(n%$;wia#EA3%^g8X+%$WHbKYqzaIn{H8N@s)EVK11CIo1>58hdWY zwtB+x1hWQleP;QTvC?!d$qF<1Vm9XDKyefit`Te|LMZ7HDbqGP)iKe-g1s zXDyfJcS%y(@xxA}u({6fu`FwZaBnE@B!0Zd9nq3`VlSK!(sC*%jKle}I?sx!JC`|= zK`Mh<21b9$V^5smk(=;P7Uz5O4}1*fYv<%#fGrQk1zCi4-sx8Wf=-JaJEe7O%V?xsiaR2#T$FtD9E%9b z2wcmc6IFW4eT#g}D=NwU>5`-gsIFw3bNlNs7}Q?gK9fkh1isABSyqxO}%#La&xNR;9aaM%hU_pu?w2=hNwNARVz>jN&GH_aGUfOX$isQe~s0X<5#RX{u zT_zoh?;Am&hKu3JJ0qUeNu)fBkDb>&hsSNWWHvo}G+bRGE2EGv8Xc`4k>~`(!t=yV z`JFgXWi>B>sd8{Q-w)eB1psc$R3Y4-d{De}GpGZ}8wX0?yB16kGzmiE?M#&RyBI*?)_$xDa)|bdL=J;+Z&@3R0oi$MmM`Z2AYC!Wg=!;>ScH&QCx#B?~_pS(!8&eQip!um0gp~IeTt&#F&Kg zF$bO38%)nyiPOgh2_tVlS`d(E<6A5s*%%9%)L!mnR@_`Q(L#)COGy|w#snzP zOstBdRZ%9UWUZV3F%q`q%~GH9%_{YAb8(b|2oQ~D9@J*sn74NC2l6D9#Y$ghTf|td z%Ftu?`t1>o>t0bI(Zk99LY9Y)zgp{zYSv5tyLOg;sw#`A)lg;m!GlG;VgEKwMwSSA z!P7}@;x*Pb%YD6&BjgL6yBFU;aCDLEin2zZ)0SXk3@U4Ws@dH{gcBJq>D_1xk5Yp^ z_rlmoSUBJ-E>eE8vTx}txrKAu>BJ_Eph?qFz(4ddOmGzI7aS@s#uC)e_}+y>6m05`f9T=RMupQRLh|6nbhsZRMV$f`2^iUEUX~5Y=T(%X+=sRk6X8_;<#JM(14iW8gL!YFtOc~W>Q)%4_Pv%_93 zhoCFjW*G?JIr>Jsvi)#QUlq$x+Prd&Z_#q$xSg=wZ8S++gW#KzK4ZVn>r|0wjR0}M ziR;*H#GL3`6a9d|@j8Z}@ff@Fj=h1s;%f+y=I}P^D zcrA=y@n*-bZ!F;0Kn?D)yZ|G9fnhPK$M6IH8V3>$@u|#x{`ZJ?xj6fd zbf^~d26PCUKoati1421U@oIa!Av%J>>b6JUC+rP+Dzr69#%K$o4hygDpZ;O2nRfac z;hQ)kZ8%Lj7&O$2@Cr2f!W@LcpVuVH7N`(T8~XQ&Y0IC^B`)q2l3pxbGnh23e7il{ zG0hg3H(ubijo0amsc8%Up6a;CUv?oQXGVp}HlfmXziA#SkrzI@X9hxvI^q7|dX@og zM;X4G?5bi~HpA_6>+RTq&)VKHC-=)q0UT&x*FeORxS_jaR59Nmamxu}6zO)Y{0(CJ z?1{Jn6hf`8kLdKaHT8?YtwS#rjW}={r>rgm0w_Gt3)9^5Q}BgpKG66>3j6bm-LZcz zdVGLJAn<5FC9p>wsTbVAJ;OSqLY6Hum_uzc3h^BX>R8 zFt`5z(youBUlm^&Pi>U5wa|{}9m_)~4`24r^-&`JRV< zdf0&2CvF`8m5uP?_Enw+K06S?15@0;pnJYjfj~Gt14Y$}_Yp)#+3#b{HP>gfF-Am> z5fQ!PgiC0Tht@C-!P;5Utb>T%i}EB?L{~9-VxZpl^TRgc4AUBe(XN4wH1|5F4$(2X zQsDYgm|TWBOgN5Q2SUk1+h~?KQDgxqVDGqdLwqYC^~~!kAL-LteZF#=2XHOb+lur~ zPTA8co8Y>p+0*g`t^Ji*e{*nl>r^gT&3F zQka1w20#li{eb0g#dqgqU+O~&bVFgDmoi9+hSKaCM)U*BV*sp90GT^&%@g8k=tx}- zXCwJ9j9En72}I2=%iT=~Q*eCyT~56r!?(5;<(hMM@W4v0nv-BH5cyRE5~U1UZ@u}a z5OOY(iTT8k?nFc8#1e_?;6&te_9?xVvm!aoa&$^#`-EFuR*Z3J4;qyuvr(M^4nz9P`_9dB^fU&QlN0(l zimN6@Hza{3!6v|-OQT?P@e+6>)KhPQQbFyib*XM`2Xi#2UqgnTTiB@6?0);isa~~cjn7oP8Uo3YA>YT#1@nK6&Je{$n zWoM5j#Tr-k((au;6WdbL9br?)+hI*2$O)`EVDnWVGvpRXoN1z}S{n)GW)q9{nsZIq zTEv6~0JA(x8{W}ko$N94aQvx@$<>{j29v$BB1{%Fd6y#%M5b4y2br98i0U(Q===kO z#fNX$$ScXrp6)5cwGjRrJ66qXeiH7rWTlBEAitO1gnfE1US@QtjdSM79>4dRaHf|U zr729S3YzUq;PUYXj4#S`1~@W9hIB`~rBMi2h_Sg!qk_=icrxSHQwulrE)rs;Ud z97*zSf2o4Zb(H{P4B85wIp9fH0Hgv>YaNUCiZq2|Y)&sSHAgT4(mJ3w&C3Lr>|zx87qi-ZQ%nuaOM84x*o$Xm4Fl%T6b6s8D_*Y@ zP-|2T%Zm}Occ{7(q<)KMy6=as&DloZ)B)QsH16gFz~{WU0j zs}S*k5#?R+_*MSQ_l7vKGeHf+a>c8AZkO0b;l$n%x4_&vXvF}8OW~9N`b$%z^==;5 z%x!3g;a-G}{!FmX%<7v*V0*(p`&AGeZG#0+5-tN7}p8#4@Cc<;X>BrVep$!{B@H{ z`ivIYqiHU%cfdT}1<3W@)%iPR5W6l=+%2$9cbOkp=nVC_v32;vxzpMW*S&yzQ(6q) zw!%>u;N4?eipFN;f z7h}|>=nZmvVx6DK+WdZIk|fyE`im?gWmquYSIdo@xd`ADi5^U&9%;=8IB$yF7=UMp zETXuY{K;{@fc!+N>C6??SM+W&-({f1_!6hKN5y@%8M@(s6$TV^;>AvXLV+=316;lP zBlRbcJA(XB@@_F?XF!R=?%k|JEwZ|UKg*f4#;+2mSlXiBupgTRu}@LWiANL@W8;aS zeDm@ZrqM!7J57_3AmF8S=ri1i^##qtCxU61J@A{$kg(?@G1w!0i(PTX1_m{STuVFs zQZq8}-+BGYxmQqkjf49*9ITbj4#zI)s@7Khw3UO&xU^&t_9NfLOAHCeG#ld`8|Twc zRU!si{Q#1RYhSsiP41fgs6VjN&A`0e{?lCshrThGSi0Ip+Vp9b?G8PqM0UOmQA>$q zhi@yhCqII{$Q@dH<)Rnu8!3r%mrC7slvW|v*Si}ChfgAH9~KgdS4&xN&HZe5 ze~sI8I|?i$yl4#sb$(AdYWIkAT+O9an4NTq@SGmLt??)|jGog%IQe+(r%ghSw_-!^ zqM+}t)nd*B9J%9H_QU*!X9#4}17_-kW#31u#i|K}xrMl}^99`*vEIe*M7J9`-y_;g zND%8xm?YLS#9)$yIYVRbhjT@rL#Xcw0fYZcJp&Nx$6Q>Xu94Uc$~Ir@#Fa;DZ#PpY z4mow?EfQf}qdQdU7!#ydwinKyX_#Q}L1+%OX?qMyw}@17K8t%Op)lVu+iY$a{L~v} z2Vk7kn)ueMKv!RM!tz9Sm!UP&(M?k=(dkLVucX_ww}ILH^szObnRvao#W34_`fx)% z*|vk@DqJMe{J>KJ+O%9TfUf|j(Ne2a->b&swZu5t>quByAwHc^?#X7^=~&Hb5y9Vv zL%AtYhn9BpbvwT&Dg)3AhY`0@t(H6|-^Dv}xtM!AW~%GXZ={R`ezbPtf_n8dI!H zq%{`q20`a8yo%cyP6N>S!{%Q98cHY`x~#`|Da}6#Wf0BEr$)?Mio+6OluTgUF{bVv zr6^(5+{-k$doSl+>QL-N&jJfHEzHT=6r@!8fC_v@EQ}vuP6w5rYrI~SA!k!Jx=8VL zq4=t8fVoB^-w14z`9El#5W&p0QLwZl7}`y5WzR=ox!?6t@Hfq-!5AphT~{{BNg|^P|P8k zy`;OT45^RD$vFr;Y3nPS*w@#Bt1wB!XgyMAo)iw9L>aBYy2}^zHO`Yo-Ia#**7UE@ z=tb(-TPl=(qk|zO`V3jm((`iUNxtFk(i^(rh~uI*1MWeA@0?CbCUG;o%GB)=>V5(b zN#UTB(Q1`hufODgcJi~Y>SypsYB5P_d8D+xf%tS8x46Es@MIw5!u~9rz2k$Q?o2-< zh?l+89d`Sv`-$I=&MBYsIrw{$*BHj69BTN9lw^C7@5Z-mBgds|b8AhU;%R^gG+1 zc7M_`dt4Z~G%W5FDf2&9=dYfK)_1Q-GpoJ?Yd0>D_zm}e)@$aAw&g~&lX8gOSd|`p zqZZmv?BRn*Mkqg!qZj5){~AZR4{di){UEh>Jl-(-ewY{bcl@tj>reWVrR%^!*^BGR3&#Zx6j4ZM*9QomzTI`Go7tZBHc!GGtFSbv4i zE57ew`VvSCSTINbE^}s0USRr`N+`{MK2xD47Kko(i9#zvwkkG0~{FERs)~Pi2knE;<_*GaVN>qE0m{<;^GpS1!7nvq_eBTExsK z8CFg%|NUB?fL;pLD7mEEWL1dm0yoeytK*9$}qZBlTw+DH>Bkz zF;VX5hd0d%IF=Q$6JaOkl7U<{=_&P55RCJzOKFE8()RG4-cLy>Q6}H^0qFL3=Mqp| zJS!NKZz7Di-E7+p{Bt{e>4hUf8{2SS*pQK{5FHS|zjTWisauqINnCqfVTP1%c^Frz zwmC5d^EEn%CJ#$r;HL1{i!FX%1>HdF5U=2$G9>T~1D;yNER!RYr`Cq(6>lMQWKMHeI^mHb6EC~V1KQg9H&}d=VY?|6} z2rSVOoFonNnRD!L{nMz^vUzbYO8fn~V>QO6?O+Cy0xy+dpKWNcBd*I?VFrXCT6_0l zf-WCA%NQwEcmQBi*#ovAwdi^9uepfBV1_*we_V`CdFk8rC>iQtGOI~e23vo5l{mkd zU@sN-jOVW(|Jq9RUYIwb4E*CqH|YOGS>b=TQvIK9XL%zlQ)B0!c20orAA3h}!+*Rz z{-24NDwQ4i1qIa4EB0ku^z>hlzao@;%fBad@)1Wjg$KY#$@y=bE$x?$E_K`O#`O;y zGBAx{{eJi*-EL0XNC~_97xJBW+s?dp=Qv&SeSF-5_A=`oG6wKynlPmoomK}vvu>&z zNToxJ^DZ)08K&$-E54nIHESQqiiE-iakO1R4;;lL$qqbn_gjR|B29$y@*=w)f>n`o zVPW$yHG8&>lFY^JF^TN=8jW_=`!L+(vxOeM_w$RCQ{we8s@wwCZG8eL`?&=k&c*;@ z>zg|#c_lYHVNkc>hV_nvJev<5vy3gKRkj_b@-K2!u%<5whPCJpZ+uj-PNxJ|j%;i2 zwu(|z#FM_3>GS4hf)#)S)pKfu{RncY^U5gzkd%5 zDu;&Oqn&9e<i_wzJ2!GT;=8YyA~{?MuiL}y<@j}A=3&4&?9H~w!`gT6w<8-U>yvX;_j9v` z`D$l-2DZxxOe+C{npW4^7GH(cQq>1C-DZR;NwbNIajM~?Qqn8cdMej?39Ue}T1rG* zA7OPO0kLxgshHp^#OhehCI@MEIn66x+2XW$VY5*hIp=H`cAqnW8Kf1{dm!p<^e#I% zW2oh1Lawqo=qsAdF54^V-C$~rrn*DW1+i&#Ji)MRy;GM(5k|7-9OPEPXy*a|X6Xw~ z6~Cwj%1C`l36p~7yY)^IEzdhtF5iQ;I`-?|bi9dpyKcw7XD_S2i7fd3gC_ld+qVDg z2&_`mcG?g{{akjX8&aJ&nD^SEmi@8zlNvHr&j#3XzOh&?W0(vP6Q6y4SW%HQmNYD> zryu2uaFTI)c3N;JZI6Qq)|_la)@Gn_cGBeeV$JDwy2V`m^Pr}r2RLm=ohxe32aKTf zg)t13UkSh$hSJIw%%R914$ZH|m^y&SuXtOQb#esI9tzCDK-(NMM3r`It=P>8q!wx3 zcb8)wVKiLCzt^>ECLUq4mcvZ;S{ZT)oeZ>x2~}nrL7}P2mPO*-&!o4!W82xk;zq!2 zyP{;7Yer$-tb7$3IFz-sg>l+gE9%=va{CCiT(H^J-kr;+eedcynTulo}&1aBT=>^09j$gze}mQbmC#Wv}Q%BfetD=|m# zl#W*ww$Z&KGY&0g5UIHki^LfQIk{bKGMU%DaO1uR4LPjG@Cf42>56#Adk=@Sqz`D3 z9O>(gCV55axuY@TL2l5wh!w9g94WgCdRnAAYawsO{FK)NCxchfH8EKHGw}FJdx!qs z?BrpFpoTALXeKN{`%e@QD(c%?%quL1UVr1fJ#i+X`m=TbSqd)H$(?{+koCOgfMu3p z>tAApB^R-vj(eekCX&w6=^V0*_yaFkGR=tXLqkfNt=osul`=IxGm#zWcf1cC{6BgV zE+eL^GnjvBq*$J9@K>RH_y82Cxr4#TU%EdlJaZt?b)cvg0R(Jt6O@!1b2mTOLL0j9 z{5piXLYxY;}NhAbt~`QB~D)TbUeT-(;pEy5>WNSX2GDIA&o1te)$ zsO|DgdD$8F35p*rwrqDOXai|;_LAt9{j@aY@ZLY_8+oqVA3!mEFklWI#BhS%rAbI| za{1lNEy*mhi#4faWeNO0P;s&f6nzl_XhwxcJQ5xTVKD$q@CCcb&<1tR>4m6k#PFFH zXIotk^SWeQAyD(O`*hf*e@Z|}MehGBUSs-(E{gN}_)mU5IAvLo@Tk846@<*sDEJ19 z-;j_Oj=CkjqOxfqS-I-N0J5EC?7gRRu)rkvs$VaN_-A-xrW%70jfj{L)7Z6RDmr>dI(mi z@68W18XUgKK$cOr{4H>1|fA9ze)Ieve@{cCI+Q9hz?`*sjtm_L4q z{{MFp{}tQ*t9w|((ivq1?Q=MV?aqC5Qf=->AS5+TP{bNC08D9y3uLhJeBvy5U}i=PYih0T4edD01?)s_p5Q(PZw3pwwXXSq39^Vd_O36*ty z^=D<59vhurJ6$^;+dlT*j{>QBU<$F%sFkAXWZFz%6LWMV^xRMg3D8zIP>N})l0>@U zs-u*g5ynOrt(2Z7S*87at?GTES*TTSCYv=1pFndS3IQ+(W9}NmRA4M~M|~vuzKcO4 z!TBS6?!pfQ>Wm<{s1pKI%f3>>CA=?5o+caObUvy0a%u6pw%MMXTd4A6nq)Fk@^CIz z5={o=wu6BijVdGXbU*vX%}B6U=CPDh0XZN*yN zmI9`JNzg&tI?K=`j|flXAU8#YZlYu?XthRaPdNU1O~#TLZov$j8X53pwk6#kbR=pU z3BUbyOj??LIz8g^K#LpTSLD@qddk$L9b1CuwIAb9QGGS7uT_(D$U&ypwYZV}pCF z-6&?=uW3Z;An6zQveqa9X^ovfJjF%c4#Ei*&uf;`6ts7APe%a{XCq+2qLl{X*$h%t z)ShVOY2?}=bv0$Jfw`p?gIZA5BSX{iAgUC^OVoSKiu>&wtnJaPJBL7@RC=kmp%EOn zFiMf=pgREBGmM_>YX)43GB(OWnDS6-MHMspU?XJ(K>X z8Q)D^1~?q+eEqJzpaynvFk)c?x5}a*1fuTlmMZ;qYMV|Od*rX&dp5;Le^1>!vg17| z877AKuG44FbJ&7+2NU+*Eb>Vl_D;ez*wIh4ZQi_z>rOfvRg1J@w0smFVUAe!SeMqN z+>E;EYz7a^!K#bgp$Q)!I1x)@nuXXhDVW%6%w4gUrcfCLm#MV!%S)plpSs>aA*p>L5exaP+3*5~Y^emPKnER1RDU?PO&4zf z89sILD=T4NlBaPY?W5>S4!#`IA=ySQjN2>`fmEk`?!y{4CmC96yYA7J7yWGT z(S`z$Iox#aNcjp)U-4A!(3{lCbR_Nk)DBZ!$I-3Oxz=5NY(F_+CnO9^0^Q5FJiATRn5y|#u8ZRORmICDw^imu!W_eWxCMTsI%cKr3x!@zA97-H z--$_LU#c(lLb2^PBa_w_*0>haA%QUSC-GkIm(_EsH+f7U_Ps)E@{e&QX>a@5BcvVr zUA*YaZ+byrVe9=gf1K0wqJC^hz#YAfp>z|4Ax(2#wJ1un z6F8dWwPVa=o!fLlW%IY7^e!}=CyITRgjiE12s6Qqm`XbUKjQQ`-)c;{*0=qKbd7uQ z(pBksNPFT?@KQC9jLmI3y-`9izl@%)V)s1XRyLkH5WjuM3fq$|T}wW~f=Ef7(GM0aaK?rr1cLIXC!-X`FxAJfOZKTYF=z9J)1fT> zNgr9x`cjPUkI3nH`qYpSa2YcXo^r@LQefWLICkVsINBEEMLe4(szL0Qw`h z`s)1Vt`|(Cjgg$#d)u)Ot}zgX!^$IopT9U(ShXR&30BxzVta}f)&h68{^e8qTU}ZZ5Ip^v zm;&s$>svwo3SQM{4u2y zsJs~!N?(EIxU*xiY&y!Kwem4D(ekW#c#EZV8eKY#%x@JO7GMxb;WPBSZHl~gXh_rZ zt=t7K!V#MybxjryK^4R)7TkZDj*9ETcK~pSQXT zU~2H%tPbZPu%sic`)_7Uq&WFPAwfEqWkc$08j?a{Z5K!Nz_FSvvn7y3K2E6uIS$WT zmUk!t`-jheBLTz=y&gk^`SC*s?Z*$=|0^^S{gzHt@c{gnz;lrXjCZCg+Sd+4EW<|v ze~TRqdP|=_y{R(-)~u`X)OwxOy6^^E%NU8Y2##c8Lh(hOoG7Am#Hd#%`|+kbD0xk# zu}0|cT~W|QBIr?R(4##NG@$e^r^MnhHJA5`>xqxglkedvzUM5j=?}YSZ|5&Y|1~L1 zPN2-9*O+iR@@{LRif~(UJ0^#U@C}(wJ7Uv)coZDjO-rMW@ay!}iNP)Om=V7DP3yjp zH(FF8+rAh)bX?5(-9b4R@~aku0eTp8e8j!$j=kYpRs`s_4LF2QSzePp#&^Mtuq}5` z0p|k=ca@A0nfKLbtBX2?z+4T=p#${s8)cXy_kLk8U$gz6DDs+V;ZyXybl17k_b?c~ zRM%bTR<}{%`swa!ecR2Xhzbh6A^Rb>tAfnmGy!np5CiP(B2;OgPfvnQ#+h;O5pSt3 zY%s3%6-8AQuZ73(M08*hC;Lw^^hBFUFpVwQq^(JKk-(SH?Z%yf4P`#z0>4$=f4aE3 zG@5|Mk{sDvT3=|8tAMD|>TH&}%p9mlPi zyScJ8=g>xpgK1kAgl?s;rO+;>3=R&a$VIC!3?9eCJi%}9e=1fIAxDyOMOS(LRjMP| z##5t&6E)gv?(0t86IBP8CS^qk@}fr3!;upUo2NgS*qaFEaU;q+n-TJ`!|)JT`^lYH7X-#_zsZN3f5NP8$woq$wESbM1wYCPAv|AB6Nwu`{^oUq-kXVXHQK4;nHd|ww?PMjswD&mh!}`eO zDDXOSLa}_NQA6zS1k1ms9YPLpyt_Zz_Tkpd+Fki8CQd}ieFwC}YfRnJ{bZRp(f_AVV<3Oy1CA52u;(w}X-)J#KJwO7a!&xf8=BE5pK>#;wB1 z2Ke@Zg~uov5Ha%R6^HXJc++TS7f}foH4{t52x27$`ZNN_nYsm)&Fy*PagoMe{f{wP zrSiyo6hJ!;Za-zZek=douUjX@(CSEw3h5GB_nj1yV!0F{uohJ2L_JFcom}Mr*r`}0 zskGs4p`F3pd!L>^69zW$VfIoZ==Itcc_?P27J6uAs26&uX6*dD&w$|ACr9v&UEy;F z@r$b)LrOiqHVQu`;B!Z@Pj}x4a=W3hLq_T$nO@0{Rc8y0`uwRcx9e(hlWiaX}B z{G^(CgAR)VFEVCKIi$`|59|LJlguK@YW>Q&MG(*5s+Z-JHaZzpA1eM8S?qoT#GiDm z4}C=yznXNB^SDcLQLt3W!FjhU<=b~PUZ~hNJV$0XcnxMb-jGj`HTe`K2$K$OpM6|6l4 zjIP)62mD-ZwVIn{{^$i3l~BD_&pw)UJv08TJe(ZMDFqX!svj4uHHTmLGWs)7a%__9-H=2vQh@ySsAAb6TpdQw$-xB`2;PGHC32}F- zHgD_U%yX7_f2u6$b~u+^*l~ACR+xG`SCa5)gWnbPaL4Njb~bY~R}#<97t0^?knZ6e zw}MPILa|}zXhozNmHXO_r{HE8U+aBleXPhCSw%tOkNnHi5R6HMLtASp&q5O=pvx2%2W?aC4%^)pY}AUA zP-yvWBm@^I1v-VILT%Cgn^}kgni``~P$A^^a8i7JmuuJX1XHvvOPQFCJq&{}&~~Ag z;@A&0CO{%qD+yNZ8SJVIxD`Bz>d22NWB&}yG%;8i8W2Zj5Jzf|x~u?5<{w+IyRRK4D%SAY_&dcpE#3t5~Ccu>5)LdZ!Mayv2@hf6o)S5^) z?VHH2I0-PsezBuVaXl1AN~1eYj(CUHx^2)(&lE@Sqb>6Ee%>?5kpw%wZP&M39Pq zG~c__ruTd_kVq!|yxiGgK&@Q~Bg1N=+x)?i*TpEQWcsLRQ-zx0uARBJoVL7?eZa5~ z)F$3!Gh`seV+xP6bGj-|`uE7WobkFGZk~er#p59$yv3a&MrGBXUPyxKLfl2Stc5ge zqO*vHNw-hCSl`IVqsT`DefB2!mWs%DrPO_R7pyCl6@!j-f3HRK&5Gs& z{w5g9M@rYy62`)u5xGb3D|G^+)Vv#i!;xAhBm`{LRa=?ifnh`U!L-nksPt81)?Ei< zrX0rhn?eWj9QNswp@b|i#+;|t)_OKTnJ4blV^zS~G0I|}k~tcTyx@v<1dIdZKRAP~o@dC@bury<+)bzs z*&IfE@kP`sQJ-(*7gh;>hKfYt*;KQI{Ho)fIhJEl5%Ohj5uLGSh(rqPQP!M=cp}%C zoAq-VJ{;F`>RuwUhSU#fyP3^zler>m#x=ds=%zK?T&CMA+E5Brsz(kHFwNYaYTii} z(^^9j_)JSlw!;qiN_{F{?NCac^`|@7$QZlv{;zoS`&X$;mEd$pY*|u$Tz4 zH^l!42dNs`Pu1GHN1AlWviHc^f3ES86#BN`@X|cPP!2>^Rx;d*dVxvos^*;4e1FA2 zd}TU6X}?E#O4wIY;o1WN>azyG~Zg8#IpF;T)a4!6@e$yI?b;AAvID% z*2^#j=&mH0$Gy|3<#Ea%kBH)5naI<=6o9=CEzKG|8XPU@T*1?^2oq-qT7tnC9TQeb z(Ji#5uz=36)r~)IoQTnYRx*j5f4#V!Z(yAd`YuY2eHSGq{;%1Iv!kJ%lbNX_gOjtN zvnhk9shOdRjkEGU{wO%wyIPw3w{(k&w(^1?>SuF{AY@4X(J>riQ;`(vwZCd`UzzeO zlBxJk^H>7-Mq$Gm^C!?(ggn1{Iu%~}EyPO!uDmd)#I2CWs9U}H%+ZF^^i|8Xem8L1 zfC!$Z%o%6w86tf!KyiQ!?@S?ETu1_mC9l2kfVujizqb-xnS!fmuRPQmHvIQct;S=K z@f~{0pZ>}Y9#C}k&OM%D*?E-&ufCKxv)`@A$Kt=_i+T>$h3Oeh&~;~2O+!ucTF2tF zo^1OV6AD{g4)5HvorRNdRU!{xn=8gQrBDeHq9V8B%w`?(< zacMTB+>hiq8G+*+=@;z0*5 zIFPUwp?L%oYfKz}7Mg&XKzRu7!UPtYgPt1jEB^$>j~k69t#Bd*?6yE2!AsPQH97YtX5k?t{3mgN`0#eH)Mtq(PR9Yd&_DIpD#RZ}IqzgjT zb&}0njB>%1Gt__Gyh6 z8-L!n;BL3mogks~zXAoZ%LMRBCgGS0f#MAE@C}?Tepk3+>A#}L{r;`^zOCtV8UMY% ztG^$?|8$@Ke{68o7#{e3K|~QgB7)z-S-{xh`G-Fw%LK=vLZ@nfqMFW`jqOo#{s$mJ2Ex{2Z+ettY2-eM_jV-dw=pYw1F6R*n0ueqG$G=67m>+&|_q zafi4-$#Ka#AQ`uOJ`F@5qmplQ+MZoR$){l`JjQBP=hHxZ5E|yc|#Ef zaeg(Uqc7VZIB`fzk`9~RTA<_iiiB(OKpf*>nr-Kv!1Q7(M+=Ks%b=Q#0Aj_Tmc#q^ zQEd}i;rM^1s=>8X)K>!zLc!A(BzlB0V9VDvLh=lU-3TVICz@I3@QQwOln&9_evCDuI*I95w$M3P{sNK=n*#W+`z z2WpJOdT^8lDNzEhs_Y~l2-RYZj61ZDFm0b**;4Ap>Czs>QI}m5#TNI}h*jg`>)XMm z%^_O;$a?O_mr#SJlm|foXCvTHKbx7SnCc!|)jc^WcM=~kjw8@R7ot0J#W~RBFTR*5 zePL`Iwno`gGekwaGF8+~z^JRc&;p1$ni!)J8?~mV`=J8?cR|}=U@5&_Q`Y!5cF-}u_ft=*7Kt2G-t6eUvNE{rfH9=ITST)hC%!MQj=}K=K(Jd*>sMt%TXa!fxH%$(G+wj$0FJyCuiCYi z-~B_LUSG!OgAL9#BUmE6-jM2~Y|~##%IDIiwn%-W;wPfGEu{F{Dnl;v0l<3s;+;4kRMV=6wVo21}cdnjz2zS^M?aBYMZ&|57# ze@@IA>)IL99SIGLAwY=C`MNB-lYirBsOgbqIP$41i$dk(=L2Z?t+i>2(5cx7uki=v znyd!?bpP;8sz^{`S;fAEnUI0<&-U2LS_Pzgl0DSixbwN0K~@yqK2H0hX2KFzH$i21 z2kunB)<=DV`%%3B3 zwFH5u;G@-Vje;S5={52~xdav<2%?)jKD3u#6a)u|$Tf0+G#y}xmqV0?9`ZNKbEp*{ z+#=@`3Vcfl%bD3xmS1^$C6c^*%^*QKEgKTgv*B-&2SA~%Jd21UjN(5guv$WnV)i(f zjz&AqDJb0M0at}=-VyRqdpxoO>3Qa-jIN0yB=U2wgAv#VU8oSKE67Te3+Y90@FvT6 zqqCB^Y^im>$X*U+Er*{mig-J>(cO@Mk3B%X zgFhQNn)o@wZA_YV>@lajVe$BS-j+D9z7F+=$cQkcf;Mt2 zV6o?4o#3hMaINt-3U>QOa{qx76ti=-boLN7bowtioUAA-4Z?)zW4Y42y!6fA{)5kb z7GI|kGH@s;pU@YNx=X&rA+z30nw{>Yffow@9q5zNp-9(^gTi^$- zVMPS-^0)?raY2-9zL74JM{FnJ9jep<&OD*XFL%nypw7nicf8}7xPKTBj4`eZ*zlDg8dr655Wr#Z^95|DtflOtf8gu(sX&@lKs$wAl(fUW8s z>#=op`g8faR{-smA_`_QBZ{77$vwz@iuAFvvDYEf z7DvP6_=K3&kZYmR!~v~HW&&ZVgR184-`Xa~OuQ-jzt_z3n*{YgST`k8Lz925-9Lj@ zk&3N6iV&i2N{XJgyay5z8d9O8ev&m6p@yi3D(Ia5-dnToc>00VRki46GAS#Y@69*R zj$-K&E_e{3vSfa{xZ-8uVUfRn&du%lA!AGv!Ecg(I^ql(G%9Y9Y05fa5jsa(pegPY zb(|lp$DCol4xFiO<7k7<*13EmK2ZMy^g(t*ZMvLHSG^5e4YV;i!V)c6#0{eOh{}A% zNv4Bzw&z1NM{*#+b?j!w5jan5fxU|?K#;eN0a1qzBUx~!*|H*gTv3lp-(*`!h90A^ zi2*P1IeOr?n?*bD+GV{m^I;t=F7?4IFOhOwPUARPfn>!V*Ssr2lJNz#rcMjr0#DJT z^zHeAQS`dwyBeLJqQw<>eCcY!9sB3mq9E^Fx%$P6CbuZZ#M3gr>Ni=N#;wf63uWvW zjj*2eE1ria+&t8Z*OC3Jm3s0oy4~%?!c=Gxu0oJdsYH(4Tk?GN+QYKaGS5oh~sfUkDj@$frsw zCDn(#CI9sNScA_Xs-3+BrD|G06fRvwlpeIE78CfCFy~WOK~D||bllS5mIe86F6dFB zDQn@0Fq3BzC#dwj;V6rgkfD_MRVYX?&^R$Kl}qbPVSUO?G8-~tQqUMUDimcL9CyXD z2PJ;aPg*MOue@G&4Fex1lTD%^7PoRmE!=6P8tf6Rf+Sn)yQ_xg$ubAYT91-sflS*4 zq_oWl#Sn^`vk;qz@D~bu42!SG{IA zOe^7lj(G)8!snPz*7p6bCu?iLW45p>1NVsT1i8qTSt6X)s+aJIofrmN~%Z^!(xW6L1Q*F#8ukfP}n*ybV6!_iu^4xYS5a=r^YG{4SgQ2gl-{8NT6v zC!ycXhU&@^sGsDZ@IPb370cE$F(USeODvkI@-&J{6-8lFB2xbVn1cfVa-NO}WCBG3 zzbj_=H85POs`;PWdWsR6c4UY%q}1D5>zTb~CUbmVpSD>}Vte%8!FOSOC5;8Di#c_q z3n0J-bU6m#Ae1eg^WyqsWy}FEELfZ3bFdBYS0=mr+7M2mzUH9U+-YtAc|s(rfCgnl zl_wL`ztZE{^KkIpQKn?HUSvuPF(wMyD)H(n^{iNf+zhvp};3C#xi1y$Q2Y!GZa>;0d-Y5)kDl`+m%@6`L%gv5!}p6lB%YK zQFZ|gjvzcPZ&N3Q++<*7@E;ti>P^39z!CYz^d&Ms4}4ibadfBW;kZg#4a}m^a8R8 z=%n;@%TpVny_Z&0lH3Fvwi2Uj=YOXHueg<_0_aYmD{8Kjt2-kTUlxaea9KA4BSlVK z96%+&{x&O_iEf6(CV(6nA&va%ieygFv7m*)T_x?JSDRe!_A9u~4>ZN7fCLq9*}Aud zwx^?|5kTks$hFdvOuxrjp@k`*(tYaP5Igrrm^|bes{qMpeX3TXnvG{!+NgpZZHU;r zw)X;SO|pmi2XIp}0G>{$l)b;K%Xk4$fbej=NKN3(X5che?&CzP2y&!zX_hr;a$AWX zm2IkdPQ7q<^WcKfUmZqU!4+ozZ06_p?+tTR-6g5^ zjxv_))a!F_gv#2zApJY))88h)E|h_V>kH4s`2ZVUlI8k4_du`11uHvE5PN4A)qEsl zf7I|tZI*+qh8?1a3Wv=0mIl{Ua`7IL_3A+k6Ymal7M^zuyfFtC=R0h+d{rsawupm4 z#sOU+qlmd(&z_*`Kmcv@gO-zKqCCXJ5)*DRAR2S zN(8zFKj|R?$!)@T0(`eKpbyCYkO}2k66UGK!f?|Od3}$s-xrhtRs}h8YhoWMv=(X;rDdJLZgn{86_y680ZZXZ z2*V=yfM}kExuhD8Y^q^9xwjk5ri;#N3=h7hW0PEvmkWK!XKeJ(C3K)5MW~P!XR@Hj z@n>DlM|Z^5#yY+8(Uhr<&N1m3H9~sB84j_eW)e@f`Mhs z$r<6Mop!ppTdZc7CVB7J4g6>ArlJ)yhQqswH~1Mody zShO}*jG!7|C%*i$ij%SMWA&H@<^{l?_o?S3&64b#>_027_Ach)Ff|N+h-hGzz!VJ9 zSDmRB@=B2&pZbgo`dN-=V~ft`3uLNMLzK+}t*R>_r$uB7M)Szd|KT#g<~r@4+1J5+ zPwGTz#4ZZ4mrUx$4atmu1p-SZ5$!%fCutO4K+5*sb6*`qyddhs&;F4aZio>ZMiyou z583Phe4zx}5k=EjVrc)@unh8>l%@i)(i8ZC+6>C&@)W)4h}+UhQJ6!rBm9-~ggGHT z_7&s|fz0*Cf&F@yZ|B8%7*=jVZ{L?Z{P9n z=Rde#`Ui0R_xq(7S$H5J)Zp`1>j81vtbk~_LAc;SseY)Obo1gO7kMod2UZve0Oi~T z?Vk!MvgF0{4|m|Syqa0nQB=?1NnN>)^$d4%){#WEyq5daxQTHMBG{Jxkl+wEMLP%2 zLmHH0{-g}+X_~p@Hu6Ksere&&~aB^awsgtxoo(5TD)h4L<_&jW3orA;Nj>P7uOAx}{z6F*}0YNEoZP z#`4uoZx9H+OkegqZqhM5oZG}Rz9s*DpTupQ_hw`)%o)X<1LX7n;_MxxD{Ysp;qKVB zZQHhO+jcrmR&3k0osMmEY}@LjlQ(DY{hhO)=lgLm*7~!?J;wTRkE*(=<~3(c$p!Mt zXYU(Hp1fUUEWF8q(pyY;shc=JsA!OP5hslj-q?uwZCPmdjO9t{CNLgNFXF0?$_r0Q ztno`1i~j6Q8@T*V8;d?lAbIg?0*gKxqW;*By6E_F@RTyspGO{^XnTq4%=VWQ@aft_ir&Jb zXRJ2j#Fc7<#_*aoX`_N+!SpB_h$l!>;P_I8!v<5&A^U5!I5JwtaNbb{>aLNtpnkvv{oAUg{vG)Lhv@>#0*P-{neoot)Bi0gH-5hk5OP?vBomKgDw zm(Xk8wjqt=WH~L-!Ki0)qOhR6HtB`~Ot@Q6$4Rs~Wo(}LZTI2mQKLo6c_My`=5Y6# zcM2+o9oD4+Wozpjb<5^}O96_}W>3kdd!;f{(Sepzd9pb-09SV(E@8;wDa8)d_QBQe zlZ>5u8ox#5E__J8q_>3ZIO{&tw4j71r!X8SORzmU?ov25`SbheSOz>ZJhcbXh!NH3 z);?Bg8O_*8w~6Hn;c_w)?Lw&nA3zQ0MQs7S;eC(ih-q!o381&Gl$H{~c?#saHdI;X zqN1Y}JzCS;V*3QRBuTvq@$v@FB7Lh+Z0#&5i%pcibA+>n6NkzKnxQ&O0>HC2onUtnk`QdjTssWT^bOi(+H-k_`>CGB zEjD=Wf$4{!=C^nYrMGlT{;oNqf1@7KNEUYzys+)^j$3Q_5*p*Tr;qhfz0d!W8^eAR z2tei%mOG^qHa(+~HeG;qNe)^@k&}+c#?&UJB1D4+#f3$B$e4FYuDmM)+)+$o@>ZWw z>BnpWpjnI0D+FZc9eM2(G_uUNZfcg7Ojt#WWS3O)xn$KB7F{~pYRt-rbf;8?piECY zm<9w#=vbxIN&78ZxOCN>i`dmOJgH*CZGr^h?`PH*re=5~iM!54-+$<4=T9##0Rh#b zomR-u{!(qP*va0v>KQCqlg4PGp}@(ba#bx(p^%Z%OAXdwX-d&RhkbY<)kVd;s^{iP zxswk){IN2p+N?!2#iUDH3SQB;2o>U5s&_z1Bb^w25qqu4we=RibasPMRA=%dHBe}X zqb7jRmr;w(ES@_)Y52!4(>1#kDr}{-ni_YgU5c3tG%PbsucT5fB079O+{vy< z?HCR==1TCfIV}j`4ZQnoz!A zGNpLaSDXnNsb#vzg{DY@%n|AFcv`}ecRI7!Dm?Y7!wg*Foz$?yp`nB#B` zoa{We^yLoo2?_~Y`1%~Ipj;e8Gr>b$c z$m}cKZ400Gz%q)B4rZ~(i+)jle(5R32dg*LEMiUYTb*JD?K?8U^`&Xd?Po?O9-*OE~w-*HZ0|_o_rI;KVJ{;S@gsx^SEnk z<3ncIzFW2$Ry*?-n;VOc%647x()z~W^UF!=r$KJ!UdPDTMlFIT^L*Gwy>W4D5j6R& zGRQGYvr`mRlE~iu+$hk{*Pjc^JN_OP+K07FpuA$l#+^8 z8X87gtZP~5`&{0EI*5q@uy`s#q93QWo+9Tr*R)NUuH@?XSdcm(PL1&Biz(-^ouLwp z%=TV z5YXggafVX}GW>z0r|&Yh6%`lahD(T9ZYT>HijT%#U4AIcMU>+r{AN!1ZSbY+xWbZe z#A&*${tLMnEF1usyUg>O%&k~YRe0qQqqLJ@nq%ytER4lU#`H_7>6#&-weqYKExA!E zi(*!uRLQ)~a4~-hk?;XKAwO8cp7^?LApy|RXpn9&)V7s+3;!40h>4Z;%Lh7lg)Ulv z5o3)IcnflS2t=O%IcsP9XluVX7eA#L@jZyBnFRyuh5!8QQd$0+>j{?UHHj#id+Ng&b#EKG1VxWYU?80a zV4sC}b0jTx*>}_0A#nVOE+};_h+mE`nub){n^;G&RItli*KrEF((6gU(%*!@KBy%*#gN^u}RUp)B(4GZm?>V5j#DlBwtkr>FkJ8>p?M|Q_WvAu*>+OA0J&5AhQ)mpL+a<2Ec9OSHdoZQX`x@R_6`(=k_ zF%X8*6Oh&;7+q(cdyo<$_Z2AgtR+eExLhyeX9?_1c{`bEStXD*%B$>)pOK8GM~m!6 zniicVZBvY&^5Ycgl+0300bCKNOzlpKD9&kj4dJf9J~BPp$Y1>FHTXrAOs(nx*`l_X z3|;vM?6ffZqSDQamjNgx8;OI6lp>vaUc=kVwqIjyt4+6^r-_&`&1RAS3xSfL8E=*@ z_IQ#9!Pngm4l1c3h1+Z*s@RU3!L_^Ow zLw9!|411uo77~LWkpJ19ch=VJ;JaGSJd6D z@U1$!$nectj%_Bt9+!*}dS*m>tAlvvj_5>G%TQ=o3|Bfd@mO_7);n&E>lU6Q_?#LW zSg^{c;;nu8{OicH+TU2e@ueg0{E~^P{vU<}NxT1?5&k(wlKd0zHMVy$k#zZ&;cv65 zwlj(-%7+(yx(QP|#%N%k31ZR|9mDtA3uv$p}RhI zKmX4F@G2!GQq`SC`40kV-p$T2J0YkRDW5dH($1N~w`tDqw>SNtd|(cE)NI4I!wADS zCDE-nEqYt^H#H%e*uZIx4vP3w+8#6gZ$TP$j8W2TtEsd)Ep-;V!eArP2NgGJptX== z3RPkaS9fXNu)Ay?v=K(FW5$$L;p93@Q6|;4+!G8?@W|PIg%-k`5XEu50+!9oDvL;+5GpS$pHr@xD8BD|bu+BJ?%<_kyx&mZ4D-eU~ zEAS%@i`oefShI5s3$Q$SG>+$&-=+=4Sl`hGB-p)06?e0wFrYNyJSs^~e(xU}s3(M0 zV+(coGQ+_nN@0*&F(%03su^>)Qv)N)NRB72N1xhtMW1(mXnykJ&K5Uebv4smQ2<;F ze>?a^S03YjwHZ;sgl&G{KN_1@=X1N#KYf0o4h2)ub?hZ`(bTIP^piX%&?%Y(Lw9 znby)U(|C=|g&KI-Dw)&v?XdPv<2xUkefuTcLUJ|FLcZ8sg)Tyy5DLu;>{LxyxPG*G z5{^{~ttwZ9l7Ge_`-D|Xc^c|!+@2UJZbs>;XHie z6XqC-WM>=*2N}6pJUeZ1M96}Pg!8)LvvBC9O6JV0eI5SAM44{C;lLnS)H;mQ(E)YG zYX-&-VfN>f3+*5ddA4b|X&w0Ka=a`yC23s?Iy0_Wy0})$X<``tT!$8hyTK-Lb6f`Y z?M#|=S{B3+^stR~Igr$%o(VL3qlZ4cV|uTwMqG~5X^^aub1}W%2{IO5&>#M(2A_X3 z8b(hyS0ni{8s_=J9fbdv<>W6gl;jr*@vm!CRZV9URSaJD7CAF4AVQFSB0`ZYk~p=n z2m~R9NEkUJAvCtu%=GCZ!+G8(AV@p|Yq7^hbf1p!hKOm)soW04I z6GNCND=V)r-!|W}uYasxZhyQUs(&lJ5k-1qDBrb0LX=@EKvl*nNeoLxXQrIo%h1x3 z-vzU)IIKq58hEpM5f})?;*%gMkj40OAJVtlt-3&Wsxw4EtTD+3A1$uCxb5(kr)k#Y1guJ=C=X=}~2ANIA|$ zw#aILYIy)&M&anR>*POg0~Jm&vFl?&Gud*K8ZMw|H{%;k7aOJn{YVnxV;9KNn65tt zjj=B%bc0FAY=sJ-_HjCT75QnZ{oyabvdJp_NJuUZC965XHWD<%xsaXWEGm`JC>PTs z@2H0!~1juOLj8gMPOp2@B|phbfkU0E`jN$2@;l#rVrOylNN zqVF6W=RJRJT(r_$Lt3w+Me5}H=hk*m<~uvuWX0)`%kuF1gFS*BHFdOGU0cMAfH??@ z77mnhGT8%KVzFnXl0xl` zZj1j_gPqmJbL}yn5@h*vPaBT)Bt7gHm7}0DQ6Wwp=#yOpb(|3t`O$>rqC+T1X;$x4 zdSMDx?=M)@%}N88uBiR0BP|)ajv5`c$HIMxHBheB1Tox^+4B4{V?-Y=8xAZfEISHYsaydmIPT}&%s&jSNwpOUH4JRmx8_S$iC z=c3d+S7wTKpVO~1Q1?s7o9*fb#iSDjfXl zuAEA!K&YoRqJ{)l)g30LcCR@CHOR@8Z0irR5x?3gnb0n_E6P49uxXrfQnr{rDUuIr zToI6_U{LS#dEE38At|*4Q0+qCih!`Tys-l^7Lfg(jHqWGpOPyhy|`5{S0R1aqP9#W z(~lI8iQAWe#_`2GqWOD>XkOJ^wYaFfE-A>UY>fcA8`!6N&?OwNzsnKc zp~qNZcnwS8mJkd_BlZ+}lYYXWYtB2T$Gcfh1AQY70@x#q;Z#WfE8w2AO_ zt`cK*;HctM$YmLRYdBaHiDtYK6LL>0`dP0XW1SE)&u}nfK zDA-1acR;X)?^bd|G8Dy#$@8I2Py4k_#I#ecOwW-hRv3MAQfB@L|Cukn^fo&XZ6TU) zdlPiO%Hllto^F3$`FwdU^#gW*xW}3n&ASl+QN(D#wibok$C^aL=?#t}XO$U?L=}Xd zp12YM@kBAHH+3Xh2`YflkLndIN5vcO=h=*qZllq!9bG0jeeD3)TfCS-_k@%Zt%TTH zduRCu&cqU#6J!b97+Z7G5C07QdvO33b=RE0v^4hMK zWR7DOCul0IC{Z2J$`fWdZj1#hvN$Q58I5DOXBib3py)bxWicaLc4|BUf|<)_H>_7v z++_?qv_>O4)4&oj@6SkDRIyfD(ag@E9twHbP@P)C?mB|?PGIav;Z4_( ztUM8MQ6kJ0(J={Qf(7iu$xgJy(_`eGA9L*>pqOab>6Z?X&#j&tITOF zv^#ySt(Bmm*palA(8x=<>b<-Kq`FlinNKEKXAk92pC#f|_vtiE z9fDtNWqXCSun?P<$WLYW7wekvi2RN6u8;1>%xvZE@fXa>?$Sj$q216Gtq(O&%kr)@ zVt=Cz`*Y#eyQ^?tx2s^67x#J}`1T=|>m!)|`~_N1eMPZw8g*|OlZ+=+=F^GU{m;MOifgm|N4wkZ;?>RRcWXjz74)r#d{$7Z2fXk5q7 zK=4CP6G>{+HYf*d1i3EP9QWq-}1 z;()BOvl*aJ_!+ab{2o5+l|uXMfM{#EPaZ+{Cf`w5(P9c(Q}k&6z@T=wBwl{ElFFv; z*&4!*f#L#JdpoG4kkt%PG77UEhJna0 zTsGcv=6%gYbFE7#D{oiU^vfvQL1C?o%WA@r>PF^>zL~){m2Dot3PrS;T7_zXIx|n< z=rRSQy;HqZ2OZKe`-{7PI=x7zX}-T4Mf<$s)LnNZSu?|SJ9i4sYz7tvE>{}9D0E?j z`IPyO57g#z?MAZHiBZKgL3f0YW+se$`!uOvM?8cb+zM5GR)w;MhNvM+0NrS(%_ldcgQCAF_eB=oplPJ4N%ONicaPpc3LcSba!R{49FV7nQ)umytW~jIXJ&tnc;Pc z@%N*5n%BQwtJ3?@z!OH&zsxwd6(gCV?}PX(ou2!8dB*wX3_|jRy^Z-G2}>ltq2%-e z^FfV-g-&1pd3U5S;JYSQ4|DP!^Ni36efjg<Gd2`F%VM2iP zcbka*BU7wT;pD?d<%1V>W_2chwF_SeZijKQPAGNd(b)I#R}P@CzQnBQWxxm98wg`7W8< zOC277bU4cO6%4V@Vt$UMJ`d@oNJ>naPxw!Q$662t{8-vrKF_}g*UBPTO^YjENhav)Rmwq ze~7-bZ1OBw<%iI%7D+3SiicgWtun5OpJESWJBJ~chNp&m9OCRbZd4%R{}K3rZOUTlR8a&}MEkrlTg8?Ql>Q-* zuHitSl5qUe%h?*umZ=N0Lw(Bv!nr~Gov*M@*QG1AEx1f&HlLaHoaXkPzPRS+_x+wW zWQ8Qj!(?cr<=MidX=;{~&dOlk-`f=c3LJ}B#XABUya8LZpNt7`#((|Zs(Y1u$y4S$q$38*L@Y0CR*RZ?p6aE!1-is$O>Fx-U3%iy8IUv7ZuL z6j947KMI@r){}=m(OyBb-ZdPFYN(zC%p{8E(5lrU-Lw{Pkn@wgsGHd1v>lzJZ`Wjy z$nuDsyzC;ZLHQA5t0ZHow6mYwB)CN-+dN5OhYy!Zk;>-Q$0>mdFMAOJ$_@ipVa{C# zN!I+B$nde(@lsarTq#+R_#v~a)@ydu6vOhO&+xJh9y(fEJ%eA>*(BP3OtE#z9>tWlGfJ{wxPwS%&LCYaT|(p3|ML~}D? zVbL<#ARj*^+`&Wi$;*l*?<3fpyU(qWco=BbNpsV0Lc@#XRTX2uE-pa;%`cVd! zbo;6EeUfoH{+I$KLf;Ptpq_0SQj1^G;Z!bte9fCJ!IsSYYh;=gTdum+ISK`hk0k@n zMC+HWI}&5k=x?#*JuXnE+t?uAg%V~^im2sjM-8or_(GxA_k7xRF?)A!Hc7)Ldlybb z2l(3GaeyKipIaEW=Vg0xu&5i4PBRle^zaoLHGBC2rsCL_ZlGqF=ePj| z^q15DWyR5J__%J#f21~pnWNR#3X6o*-kv3y9DuVOW8H@>pLGe|4Wb&nAx+cw-YeCt zj3tH7V3Tl8h*R)5rRFuBJX59<#(F5Joj+hrtw`bZp%;w~d&)FQ#SH4(x480%6Cl$y zTC$hOjuPUehq+fvg_Du(J3Vn6-_vQnF2nJ;yV9-wyrQz?4#=^7v#xQm9f+lb9}it~ zFG$OoyR}PqXUEvccQ0=xob1tP_izpJSP|mwI)i7crvnW9q=|w81Dkj!SKZC!#QUL8hm9kd6Eb6$C79*F5F}ld?C1ncXq|?> z!&GxOxg0c%@*^UBAE-+ciax2oQPiUoMUN&Vi0Hivr6-28hN|CL^u-kNhp~=x;z+fl z3pV16Yt3zm9^{eV-+<#|0e%>YuV6^~5{wG`&jaEgi@E+KfB);6Qni0d-O-@(EfGaT zQRpG`@(2{|sH#Nfv{8_g-11iC%s8#Tv16-s?Qm4 zlw_GnG&Q@zA7>ag$#D)Qyk205Z4b3>4d@(SrfG1Y;o@p#+F(8GGKmV4WuA%Ca@nXF zcGlTuX_9S}h@J**&8ljV>n78&J7$Mb&n#ZGvSgZZ-EH%h1pKNZ)SGg+{k3xrn5bL8 zN0IlMT4pj0vYq08354g6)&zI*aKxG%I#Vj%RRDlWI-n zPO$HCvK-IV+L*SY?q|KUxoaYo^`Ap!wsuyGoAdqcT@Q2zNL;L-X)3S68pZS%cOJqwa79is$@DQ%r|qENHW8I}kYx^d#VLKbzqzA?`JmFO;H=gy zN)%_x?4@x0eF_DE*UD|5qPp88Y&Z1%;dBc4q(mJ;)Fh8hMJzzd_-AkU7(M)leKJ^c z1Tbz9xc;av>OTr8%!OD#e0X8%YIRt;3`Fjfj5t-I>^Tc1eawkG6B315tkps}{|1pn zX`S1T7oj}X^PY6=AYc&O&!2ysqfwU*z?akj@67q; z!|ZHEc09AK|MUB#F?Rq-4q04(BL{#1o7_ExHj_59nzQ9-WxO%j{DvnI6fKe_CsTu| z-o{Au8_+N9W!0?~Vryhw?L@o%$h(V~ zoTnx0<>e1|DkI_rV0v7&p7EuO!v*UqU2>Go){=Ml`9ft~_wD2pIpB1g+e)4Uw}s`H zKYDmbRB~n6cQrhDmGIdg?nT;-&p+flSpzjdATt)k821Y+6cqxFnrqgf+3{H@?`&g4 z^$f~e34p@8y%s9V*fTj3I#zcg*?Rt8f-9Az{-q9ma$(Uj9r z(tdi8l0gAY=-#a*1=^IktQX1hv!Z#Q2KGu_FBBf=wSfj@w+xp7v{9dD+()?H?Fkc8 ze=+(U>KO#$MWMVfBkqRJoihk+0a$9cVgy8v?Uzws09sbUKQd$_ka$+2PtHbkW&g|@n8lp$mZw9r9vXXwFUto zSG`0I!xqu#Rnyjb6yBoo&GVyo91ff|MM_LB1X?BLD z4rkrb2{S<)Lsy8ST_;Mh6*6ez){EAhqG@dGXA72Y6Q&c85JIwZ{FXSFFM&D=QE`_R z`X0!0%&7YQT|IzvlNs-N3axxR$jhta=g0KvwUgOnZ7ts$xWD#oehjvOHAg-%7pA{F{ZQV4o4tKcn-DA^+4@rplEixxh$-W~`{;nkM zx#7faX|{(HUvA0(Ggr=#Ggr!%RTEh z-Erwf1wT<-R=sr0=`xSBf%U?Z#f=_Ywvw;cRj}%f_^^k`QrYFoN16Z+w}S>j{RHjZ z%^Br-{nIkq5zRH8bIoM(WRf<+M5ydc%Jr^Me{Sp8K?>zE_2E>6W?Jjr{S#4W;Ay65 zYZCok`nD}DyS3?Ojc$qGC{>$Yd|UWM(I7J0C)ul3v;2bLa+Ez9-iEmv$~Su1z?|kC zIwyKNba|)~Z#lH82>VVs)d6~tUuKyDc{PehGI^!(59Q;edw-@%*TlrVB2sC%wB%c= z(PTy(*QwAkDej7PJ=!NFOu@C7os}o{Q11|HrMhx1OwL;h8ZD&4+;q|ez?7b88r#$x zDx}y0@-L6vnZLlHVzjpKRXE{gvqZb`l)~LqAY6FwmKhCO3l}EcdwS3CPF5$~WxGs- zrG==OF3oIsDXBIjH*1qxAjVVeoYT%68VJjZT~oZfP0a1aY$V&pM?-kEgp6IYF5wjQBYb+_1b<)G2FX(q5ZAph2k zPo*~AxSv#RkcYA9i+1wU7@!8z8>)rb04M?zJ7J7igS8mP2Iv>+57@)JqkZw1>~`E= zx&izc9>4=&TZ7tQTf^K{^uT_Dn81GhomgAU4hY^Fd+KE*z}NdUwOL7t@Gz5mVE}&Y z7wC_Q-K>D0k~@wv7#IV%K(C*QV4w}#EHRS{sIt&k{ekBc`w}?xLkYT^oVu+UmY2K% zE`?0>^;1TQ5qgy9k#w_G%G`x}=AU z9q|e4+iF{Dh+|DsD6V3VSp|8OmM3m`)jQ4hC34&mM)E;}%aoa7rt9|E56L3TzL4VG zELPeg&7GdnxoC|_^;rdHOO;?fieQ-+!l>;Tf1p4&3K)PWeIMc4RJqG)K2UEktSgJ+ zel^jIYjAxSmdn2PHKzN0OV)Ga&((fK8(^zC%2MiMoJfWobUWx8jJStIog1F}9! zm2r{}7ulTNmoQ)t>O`w*Z$%zlvjeXeOvrYZSsDSdBi#-2VaM!1G8%!fE6es6iyt|w zR`7yoY7ff}mBcgPCnV^SU79$_&&?m>0|1iwPMByBx@}c;@AdcsYRFaO6)^!YH;O!a z$DEz}^|PyONFRJipYyY;AVZXWSH-UAQa6IJ`_%69cz4`w)$|VDh&r8Bw8Ja!EX9Ta ziEYG!;{HsR;TuJlc-6C)OIL<+2(pmuGbAA?I|*J7aIT_8Gj?NF&TEKGkf@#+)(H1e ziwyUq)(fR5KuUa6E0{Yv?q$rKT#nh*_e^d^F#?z|s}1AWM&kSQvC08C7ZLD|?6!;r8;0 z_}$b|O57IbN^8^-W3Uc5>8dQ%X>LI$@3JP@ImEsrinU{ukwa3qtNtLX?g%Tkv($79f(9~Dbe>|A)Ya?5 z`mepEbo5{!*4Kn5`gIil_ZYadv4yFv;XiapVwR>hCd&W#P4Xp6GByA5S^C$Ztg5KI zpn>v1Cz;6tV){MaZBAF$L0hGw=A=Q?raEtvb}wOEW~5Xak~FQvCWDD<|N2(b!E94g ze8k)@X`eIo#Gi{#VIkeQC(--rwa3{mp~w646Z@O`jXD%pjb>ZSHf$93%RoN^l+BS+ zn?H6J&&&x~AyFYxp|;I9nHbGot87%dMCK@l27<(-btH{ZS%dPRk(zZ__Q_i)%ETbZqt73U6-ovr2V6#a4$Ue5eg`L zeq5@YE{`_(3YsK)Z9K?Hwl__KRu!Ly1869e*O|n@i8qcmb-9P2 zsOPw=H}VKn>}Nq~f_CxdT#2}|B&Qy8oFaRXlvWD*WOUtT1@EdGER7NtaX{&6;bU*- zPvK9mA{7)b1VV-#k%n6fNxPAI=3^{hp5mYIFqVWTEk|A|Jn}9n3s(SG_c`O5_!*_r z^qJw<_vx7RwxCWhDj;^Rf@ViBzcr3fecPyAq|8J%rYNSU`LB=Iib76r{XjJIB%bHs zgBbg}TkocsGdNgBSIZ$AQO4*}VG>#z=LC(DRxP$NkQ6?@Ao_7 zdifE)0bhoJ`+VL~i5>F>$>tW(S92A-(U}7y^nNzvwfTzTm z)Ue9LDVB;is2=?O6;J7HmT2WCylo(Gm7+39X*tCW{A%2yD?>0u;mN#&TiMhI_$Yb& z2x)O@Xvw2Cn#^zJjEL9V`GePr?m#_!u9EJ-TXYPCzPuVBWl(++6@y9xigKU{57~IJ z{;UqU{%o(nV6I0JN_Rbh3tz4v#R4!q!*6`q#*qMw9jYmbNb@^v(F=15zfw>1zwLnkE%pAHb*e6wHqIonh7NzxI9IFw^8yNh zmP(h*9pu3OS3kt7)= zR%@Fof+W+FBOo4%A|#%*eTJ6YxX6;NOJGJLZQpk;7-3pgQv3G;O#VtQl(5INWTNsE zq-7*}h>ld-1dnNa5$$f;hSqM4$~CFs%8XR3px^YpJk-1tbShzo#d?2sW7{$0T4wdH z(&e6}t}S7ayL!Ij6d2?B04<`88=BKs7}{{zo}p+UL=4D;Sv_@Am82CV6KHXl(rGy! z`jBM0dU@o~kB36ayB-%C`2pVXOYl>3_3`tgkqWrVGm)&gUt))&jX!-}DAl})Kj7^j zij*B>8R=DjbH|jyRa%M61^%Ka;J_C}&cIxQw`Mv2+U$TTHSmW5Zzmrht|;U0A5G<> z%De8Rci==tG+tRPxPmzRBimx^TIMiAnw2Nk`iX4{3&zycEpHviTOb!}FdulyWq@MX zFe#IF2t>jO#5Z7Mmx4?)KxCW98q~C~kHBhc?9Ce98FCO!k91lmM13vCGprV_+Vfl; zTU+QT93@~}LhV%h2~52XFU|!uK#BymEn=c+#Db1XAxWoj=}}3A{p7T7O38%Tq!yv+ zg#Z*~n=?!Te$akw!QE6uD_o*lr2ylPef~2^U5Bv&SwOM;cjZ-cl1peCyM3T3+opL& zp*6&cd5GbGc>cehV?(BA+_hiHB=D6?GXK|P61KOo`SK04w72_9PBkan{)cVg$Y(3e zc!wN@e1!aq{|ah8RM@yb2s#=}WRC#j%T-NvVUe}uI-=~4UGaGlQPNxBm;9(>H&<+B zWK9Oy>-KK=pKtunIo`kjj4a!KJ5o@H>r=skvKya8c`%R?#On~EUBKMvkBZU+{lj3g zk>VIC=ySSK(`HpQk(6Dea^YBV5)FfbJLgZEUgb)k zvYunPY1u{Q#^zA~aZlo~O4E7Vn$DcN?s^o?wzr16&IH_OzIcu~xM*r_0Wdd1|y9NF@Q;EDTWI!%kcVhYjnTQ^~eLQ2uG|uz);1yNIcbB zbzjmn_Ngb%Pw+P1)okgVtw!j;%70E;QRE3`i3El~G$k-n3sBY~DX9~Mxdh+m@`RY+ z8=R9ZTvYa(FNaCyP=UubeVnwYJ`k~Lo_hAy#A|veGJ2n93SkGe0co{~?+<2o?(B07 zCAh^4Y47@AD7nqewaDv~@`CHHrJhY#uF; zGkZ|BFonlhHRN3AiLeT>vpbh{z=|E%OV$_aOS8sPF&nl=-&?@1K3$kjB3h=1hs{FtM~CCXh@-(S*>2{-an? zgkLUOGC~?EH>P(9GHH{8>5>{770oSFXsI2GbS=&Pw)LPD0aCiw#Y?|B(6yJZmO88} ztCK&DyV<4tgGuvQx7po)e)7I?(#QHb&G>vHqc$;Sv6~UY?AhD`IPQneU&oo1f_L5!!yoO^aXPo=xp}T!OdqY`!TUyen;6&1 z`0e#~GquYzT&EVoOWfx{!?)im2Y9bU=eFO5VZUv>w1e;8K(phy?lZFcp7z)BPz>(d z(Ik8<+o`lG(mKeuIi}EDCXFTH@r0CadWhxt{)#@{c!~Zpkn`9I+TL@Y?;?2UrYJ<$ z^%x6D)%6hV)2$haI&v@x3RJ^{!i>Mm!;8jJMAuVwShTfZM*)aLO9;UK!j88fz|m6U zlozB`!O*~i6E*gtP^$&!h9R%w7Zqd&jWRH9-Wu-YjMRRV+eBX!p6XfCAVILoDc-pY zU0;h*h@u~yL^q>_6VLGR>b$aN{6QNa$L7mh0zNR+BlgrovApiG4TQw4UuQk_3PKp` zkmD)lJ-7o}7!UJ`3^R3!ZKXpbG9BZ_qEQJ|io==VDoSC!u#S$2O~V*daSAa6WGm)F zfMo%JWty+RQ8Gd?SJoef!bsYV2!a?#%5g`QIIf_qVQ9u~^)T$vwn-GxVZ}oBc3|R9 z?jubc&>|4U?fRq}2q%?bnr{kTaJbv@lvl4u*l4gvMODJHA&~)24;LJD7SjSotW=t4 zp=hxpuRUvtn6(}mqap3DBXgeED>>V{mUgADUEI3vq1gzWFHp5eQj+sDZ!<`PIZc_i zLa&S|P*gOO`iM$9BofRvvFIG0GQ7fjXR@&czLKS3l)JhAr%=UMxUcW}QJ4!nDExvK z_56=mr)|maT|FKuxOsk$0mh>Q&C@f1{M6Yi-s$@Vvn_8N2_;L9{hc;tv~Q<^t*yBW8+pC61&y zV%mR)|ZETqgw#Ho$lrie;r(YM36SHo7KNHGku&BglO&stH9tEKOajk3Qys4P#$M zjFELHi_+-K@(x>T*@k7U>!CZaeG34v-=+Yl9g_jAAsOyBHGt>3U{9few=c;DS5;~^ zDU|Y?QFQ-~7_2!o^k*FUmTH8>$>a=i@N7vAs6ITFJ;eZDJSd`MRa0 zSfRSd8@cS*`Yhso1WgxHv4nZ_1yx{Ae@Qf|bW zUM$^~J@q~rHZ6zBTxtChyU`7|Y~2RQe(zjbN<3daSuC)kZtV*R%BE#{mgPHy9)jru zXS-Pz1x~W9ctn8Cr&z{8wsqD2Jp^=Qyf57;^_Pm8>Mzw_>S4du)}s;lFQw*HwY!J) zLju1RE}7N~#Ff~Q5|HIuaq+SylBqo_O^XD{Dr@^h4opq`P*DOL9Ep5y?4AGB5!jQ; z_vdYItcproYS}g-c%4Rx?m}Md^7A>Nv8trRhV--`Nn|?Nhi;<5>f91 zShq-aP-WxJ1=Skspb$p+hohH%a_+lP6F*1MI}4}+nryWw*kFI;zB~<7#@0;U#dtC?u0y^OySfUUarK~dBWxT@twTH0EE7s=aiER8 z{pq9$xwi&V?hrH(8#E#h87xnB! zGQ85J5F6ba^8G`ssLNW>t_K5#h&Z)q$~%b8q&#RVmors}??wz$K1d?{VU+J#%=WQA z0L?AaXEIT+v{g1yrX=5m(~7N@pG+wF6C&JW2sg3Pe_D{ww#@F6X7(Gb;S}j#GPbOwO8=zEAhf4m9^ORM zV#!cDo4OrvwBi!OigViB)K)YiaoXr}!@brUqx7T70dQNA*s+oKj7WU4w|j>FywExB znk&2yq>QLUM~9GPp(I&IX$U4`qcYDVj*l3MQH*wjhB$Q%(069Q;?ub}4dj-LFdcJ` zxD_eQg3(Ty@KwskcYY9_zm1nzgt$@(Dq^r@phZ5hYw?wCUp05EIUO&~q32VbpK5nA zOg_}?Bwz;H{3Si)J+KvW(n4_z>5^HdI`zd-nW5eAH*zKsk6 z>5qn_XCiR0LtouM^CS+0hJ(dH&dxDbmy~jY^VO64Kp&9fhuuthqe|Qmf;qtilK(u)nIqf;vk1oS#z8O)w`hvR9cqd^s&?vooV;QpTzm!!2mDlC2dEf-DwW z5~tjyTycWm{feP;Qf+s_Td^bQpq8Rs<7q84DMwPNdwf8gNh6-ahdybADMg8%qQkE{jfkVHWG@2BBXlu~bgY7tzT( zrmu>eS_9zUnBMV)Upo`6)4JipwoY99V+zsc48HFNWIHxqQpvB9imb-xH-txHEs>G0 zjBnbmLgN_EdfvK-JLL#dHZRN2YFv{&>*6Jy_=KAByI?XeteiR{zgX) zOmrP!`;4glA({{vq$0#L+bTNT-dW3+7lq}ZU8>K=^Z{#ESiZQQ_K)hbdmf|FOn&Qm zjvo}$#`=Tbmv6*RL+t|IzaNx5L_hEKjQzw2lq{ z|2s_g!N3z(2#PV15L40sm_0#6Hi*K=f+2wNH%9F$WY55;DW*FnD$k*6eU{d(*ONkV zq`PqoEHbN__U-ECXIpD{@9pYp^WOB9?-kuh1MY>q*~`q=nZN%J=TqtHFMk+;R*uO~ zva4l>c3AAD{xNWJx2^`fZj#bWiD0LS+}ekI0vk%|&; zzQ^jL2`6^)&=`(@ZEldC<)u3lJGn!}$#XHJ%S)Q^llY+Gn{cxo1^tR=+s{8y5Zmp?njlKZ|R}?Q23YMy%2)C`3`oYKQUB)@>|{Mxt=tCPa)OVkeP&mnD4+D zbNBFW0KZhxK|S8Cf_=zV*XOj~upVda=;dpeH8@_WDL%;Thd?}K!rJk3Hf``VD#oxd(jo0Q?O!s28so;NTiw-(BZ*7hJ<(%skdWQUtz3;QG_-l7M{wmSLQ<9wVM(Ub#|*w1EL!<>$BhmOCcvK3 zK@j!lmyH!*Z7F09Y+fLNz|sQ!*la=v0`tcuYpCrOx-gR?^r++O(*r8ybPB({o4;%r zKUA07vCgmv@(*sG85H#V>7k$Kh)voM;WJ{tXMh8%K%t>C2ML-tQOQcvQ}WhPnjC8P zR&5c+8A4gUWcPa(?}L=gvfJt(K9Yn}g~Z4?=~>p%jg?ZZh#ddAoP05!T`bC=Wb_8^ zCxrP!nE?90*ziU~1M6r}V87>qx&`K)lRoTXeA7obBs*;li$7^j8-&=v8)2MTvPK!B zXLC#-%JX{KK+8glny8dfNCT})=YKE!Npz5x_L)6>V;R3oYw#Jlw}s0V3~nkd$`+CUR{Kayc6}UftFVJ>Yp6|o zKlBxuS85HPSS*u!^?Sq|g(-IFk%D6@oy=0v3~Dk*(QQ1Arj`|a-(49sbr3D`2R7HbN!zIx!JL1Tp-<4#zgUpW*>C#|GmoSoY@x?8`E^LhJxI-UYzJAvQd0s8k1k0Pkz zWD{~fXtP_ZZ^2~lT_ZS>bI#R`cIKg_rl>n}B!SbHBtLS8$H>pOITWCrDIaS5owx<$ z08$eDXSmq+>V1Kb$z=?ueIJC^X#QG`o`tm)K0aY+qpeEh8wOyIlHuLnfrQJ&RJ z9pp{ac57F#5N(rFG3SniLM-Kd(F22CpwN&sYq6mgMfFNmvz+?={Y9!dIqoqF!!6`B%K$2db@v z_l)O7l%2%*Nz4z)DrJOAF$t}Ww6wgoJ|yH4X_-g%YSTkGwYv`bXzd`FG9RnF1Yyee zKsn^Cimjw+uLo#3{uX}n3L}ztFQ`1b^B>=VcL^@90|Dped}?wX$-njo$_cPQA;pkCm6$B z;s%IpL{N|UJCPoWCfaiKsW%>h;$&VKMu@)>1CuuE>Q2lcx|LzM3rUp5}X zkWCvD6hSnL$N%7k86*0`0*te?l*KNV$YgaTt@6Idl({8yGi7u|FPQJ3l_W3ZUJzcG zcaUYQjf!J`S9kUpfo*u(3_GDuY_O0 z_qxE<52Oh+#y)1|boTOqs6Xl*t6T{kP&~=Gzb|Fbg(Yq^;bPv15{yN2rtk-0Tjxd@ z(8HW^NHQj)mEVdIQeq?;sey?SXV{|Ro1x2W5br>pCe!(& z0dkkkf!Aoa^(o=qDTb{R$K0sV35;0g06sb4C^_=+`T;mL;-GXUS~?<_Ix)=-(3JE5 z)ebmr2TnUc>ZBs*_@K_iqY>U&?7 ztH&n(kfap7vZfLA(PST6@Wozk1$JWg*gK$EXC*?BQ(Wikos&A!uC%5gdc zChM?_F?B==Onkm%&6teyP^K%i#ts0oJ-WUZhyIphyBI`afv3_j!e+@z{7qS^r@BnC z1;lGMkW_-CgAcw2l}l{eCl;$K3#R@~H(id5f<1%}z;rFmn(4m(ttb{$Y6ojd%P4eM zqeL`L){|mDYuA{9W`n5oA3a)B0&lYrHSVI!dMU;p~ zwZn!&X}j_-)mXk2W(-!{`7jO@XB8*3h#zbM$WX@3*c`xTd8F&IC7-(9ak=7k*`iy$ z*m1e(rOA0khI&21;SqS{m#7No_+WT3M$8)|_CSx{oYxWRJTubgwI)}7OIcN9J%t6#1}~sD9D*P z%AqGMVFH|-nkg48cFr*LevjTt*vNE*`PH?7jvHzP1BiHq?cfgO742anJ{p92Tr-AjkxT^CPs{Xn8}J-5&+ddJ^OT zrTl;NxEPN831jQ{jO_e9nHq!jSyky&3F$UM+SGdenLg+bA5#qcL>fP?xhEmtl*1JM z3x+BEWv>Zhv$p zUwdU#_E@nHT>py5yBj(pdt)E3R#Mbe=iHXFe#DqtUKl-<4Sh^n3HO_O`G+f}R?3LX zqX|dA z5^xNZ{A|QwO;mkEs6INg9y+UA{;(%{edSwL&;%v1<;DGs7?bJdD!wf>I)ycQh?Xt& z$)B`l&G>S7{%!v#klHPl!^Sv^`owu`i5=?rQPZ%_i70a%s5jTa-(M#kfy= z;K@!k&B4hSC$BWF*^WCGkuDsPOBb3+OG{>)V)2P7tHFyOV!Vb(Dl;7QW0}a-QfXOf z=QPR;rF99RoZ0AaTGttbIV`ls##{@gB=&3FhgX#v_Tm#Aa=RO|E@@`?NT{RNBZ5pC541B0DPVjLbt3?d{nvVCwK9v3!K=Y*OBE z?l_m|uFj*Ee%Bqd)C%6%?zk&mL6?`K)JAg1q)U!0&0*4bTRS;a_1VsuNK$1I@lB}<3IB?3 zS?yM%TCn32B~R_amm;=Xcau<@2w3%vwsTjHN{qDV=CZrmuoc7jV!f~HY~KxYuuBlz zmaIw6ZR_(cT$x20hULhMN3E1cCQBOky98;}6@0Zfou)$;%$~C;lYy6wG~0@$HLE;= zZ(7Q`56x0KJ7=0=OssX}OGgBEvc&rcny>j@;fmad47C$0GA!J(6)mNsC^l-vW7Axw zS!ya2k2Xm$_saUKKdwFcL;Br<+%mztraR8zR%14`iVWd*O+fIMl%<&}5n&V~3vVqt zz^Bl4SlRl*S{RcpEC5pQ)r~owBE# zbDmFzym>57>jvvszkw;?B^%zr|G5L{FR9`B(3#G3@U%jGgGom>!{{TdNIvqbbwhSm zDfeOU`QbA3*~C1NVuOu0K?3c{04#qw^5-8@WyBbG>!fT1}Ks zb9mo%N*KEumWQPr!WVjj1d-j-*mA5&BOdQM8EY+{rxh8k208_0~U(HnS(Du!koC*!9+r@xPaJ5W# z*W(_36|>>k(E#FW#=sAnjq$mo4Lq*e_2k``Q_ z;{Grki!t=2^gHZWaJC{L9e$g(i8uIGk9E_vs!jbYc7G~=6PBWdu`jJZ(RC33B@VCp zSD2DNM&4|gp@;n7S#Kt-{~3dnoM0;GFjS6^n1gC_|2KMt>Nm2A2S=|MTs{B|CtHir7AjTM4+DKk6YjvOaWa?h zk)=wEVK|!BCOzt8g}`RxNHI=l(>v+rqap4DM#gMoWL0Pt1NkDhBTQr^u|jRJB}`;T zEQ+fUj<>{;UUGd-bNy#6;7euwUN z4Nn?jPmssWfEh|>XfKZUb|ZSt&cuh!8{I;Pyl8n;ZB*1NFeZaG;FI2fjX;{Z#1UhWi^3d zt^h7oiGg;=~EGN@q$zs<-5t;RsFjE zyKV-Z(z52boKZjDnYjoJ_1lciI+^mvL`tUaOgApFAmd!S^GiLR)0IlQ6rWP`Xef*g+XOR zF&UyGB|1Oy9|RzO^mV3x*mZb@YJZi^#Ut@1IXLh9s@q09T; za~y1U3X2}jp@|W_z|9NYbI*BSzvbP%!@AzLEn;5Laiss$*_-kxeFMc* zn1~Jy?rtXJNs|a~uz+I6fVj&fid>G4AJ_Z^7&FoErVv&LDj*r8o)wPli{ouAO*w6E zXm5Mc|DS@85Gol`i#B1L7hx7s3?x5JJxiRuCXs@jAtIxF)qL&k9F_G$U}j%%R-0GtlG)j+PBXx=brP}d!F|k@9RY-W?m0O0rF4FU=-rUyT&V)a`KKEPu{ZglR>d@;^BL*-60Ex)-KEuP9_I&SRV6P*ti&L^j zxlsu1p7|Xpx;XY%!eBhwmxjQ`+3P3Qb}z{4{&@-E{@v)2*d*%Gk|i6lI;5?#wC`9F1b@p2Wo;%Dc{chaT}XoJka3^my#d4dIUdHoFsq zADk? %*<3BEL!W$81o>)Ext(+AfISk120w15cUO>miBe>WfO5ixwCfweWW~jk2N7Rt|=>XxHcYKdL!(Z0^hS$HXfmcCl{-(0Vw*IyjT>a3zG`9_C1ztsH zbc<*eT!qktZNxCLjc6rY4b#LmyJoNs*Zjn-_rf5@;}~)XIe@Yj*W`xjAE-m_lH>n~}sy$Ptp z<;!jB3uqt1eL|T3tSfp^OW!tMyIzg^lKs7;w)_4VoYnb!iAl&lhJq(gD)#`OnW)i2 z;s;%q3#`3LA#sO9VPx%ukF&hMI%P9~vV_mhfm zxqZ|+1o*kCwx~WVfJ(e%J`_-bmMzO*vzYc%LdtlKrv#V5Wjc+g443iPOcIyjF`g)< zQ5;2)lcMtubOuIKO+4%TjN29aQ>h?Dpl%8!{akTIOi^?IPk9P~K!0FNv}s-Bk4Vpa zb1VF=1)Maj>I)vgYj_F__5f3Ua6RlKh>5i+(-zRn-;-%rEChsogjG?Zt=RX1mFnpP zWd&c;tgi3_lR|!yNYxc_sGMd9O+3Rc?-1$Mh!Xnp9#`d(-3FlQR*1K|bf)3Ww{TFd z1-2@;pkU;X_2M0pxwcSwWCo?f{u97Q)TGVol5U8ZS1if8ofB~t4E#iudKu6yGH&-M z(SIol|I!!t=9?<;4Hw|mm+>}a#OVo3KWO#L>PTWr>J%$UKg3@3VqU&l5^V?u|3g{^}Z%IS6kknNeOvM0hslG z+mm*&;XJY}(lAx-#eIU%x|Qf^b6<2fjWmRcgO&ac^ ztm%)Cie%St?hrntie$$)btoUnCaI&FG;AKxCh3bVs!M#ci_{k(KPQN^CarT`4DhnF zNC@-JWihH|u}OwVPY&BAhu97v;m+msC^hVFexhVrSX8kh z)JmYD6`5*~gjJ-`Zb1TfXvZL0ZPkhxw*D>HW=k^OkqZVsFA`*NOCo6Un(#ZyTN0^} z4Mn6p7cB7)kzn1cH7221yeGXa?OL;vO=-_g*tN^ec1JwSr<+uc7**8kQsBRy-rYd) zJ0Lnk^1M+xp6Hz?pv_?aV$te8{YC)^w1AR88hIt@l!PF2Udn{V+Ee{puBu(Z zuxiyxtx~-glqga4QQ4{65pUOB>)P7fvF-NN(7t{V6n*iZ?PboCfdCnL5qG-f+~Yjs z-h1!+dF>$gc^x*-2W61gdl?+vGM*2}_qxv)eWU+s$K-!JIC~4d_=?BuSIO1)ytn@4 z7wiuAwF91t{a|O^xbM#04jzWaPyBmye_kBZx zga;NE@kt_ouR6Jx3Y3Qd{06BztHh&H_DmNEpfDVl5G{{SSgqQKQmXo# zR@_ilv92XBtx&4x!U$8TmQN-Ws8^)|5!UBug`7BLc}+8$^dZW6XuD#dO`NOb)1)v* z9m{Bz5U_?VV-$*FRF7%q{FaVVAUCfbQxd+2YLHB$q`Vj#chWpKDjp=KHcjd>BwBxA9Ow~K@DOh&w*piGi@9KuMO zIIF?2f!hvNsyQ5;>&Jq33D-(;l&E2TokW;-`RRo=_kkRpf_0GRpq-9>0n@(aCSO6_ zv;sp~AR`j1>!>9w6;As3X))>`T-b8TKpvxy<~ zdTTlg5*_BCQq#3PFyXLr58Hf4sI5XIU(>X8(`T6FSUdg;||?NTy%@bH&F}y4UN-i z!l3TkF5>A!S8vhe)xvX20ZVjXaU#{s(@hh0v~{Z?R+$qoX=W!-o3dh-X=Af{t!UPa z)ba&m)pnRnygwLBu2mdN*@pAHBYf+8;wz0sZumF6TZWg#*+zKR6wWqEEJp6OkN0S&X?ua>! zV@kWjXk6{FA)!{x5msw1sRr;-FddqR?^>4r4k2hZI$JN=nt_iWAa4XDOc#z%+11+` zQkjYnPc0URydxs4cRV?2JhdRrMw1-IdV|nMHe6#yYQ0~oo#A(s*nI^mbB9em5|oI^ zjO!UKSh=h(58dcV>G<6(Cat24lY3RVW!fhlHt(?==yq#3k7p>Zj-hRz=fh)8`;3@E znZeDgly)`SaGfIw((s*E9U*ZG*T;EAHaJAGo0Z)0zV?7W5$?U zsAE{ zSxa=O8ABzGoM1Xbsj`M=tZVm5m79D+3Uj8trlkx{la($H!HsemYhr*@s%n`{Vg=z& z*~kc6BfO-rm&|*H!n`A$a4W0-a-B+YK5~3@!=Ck| z+J>05Z^pdKOyyWm&hok9McJE zPT*xpD&cyH{A4-&dkkfMYBu;vV4DMt!B~-RV2e8hik303ZC;gtwLMeA42;X>L{gnUX@bR}tIu2te$>{Y@r(ig_`39)7_S>GIQT+M~a4Q;mmZmFHaZ zi@TIkuT?yk2@wz}Zl~FQ^Y!r@q&gR6h$JSjx%g=-hCZ!dhkf$gN65VMx9fx~vk%cH9wt*yX@&$XHs#=pKX9 z?52BkG5Q>ftL9K%Hi~hN%I>%osK-AOcBHk;9T2>OeEtT#uifCyOD}G^9PP; zF1`riEXd-jfkh%iY%$Y)k^$6Y+W=elK&07KRLQ&v)^j%F2JZEug(_8mC-lfm%dGD5 z{3M9kc}EgYP13S#QW?K`&PAKLNR1u!3(8k!VPdlub&jxH1>a?o0z&duHOZPNUe|Nn zW8KW%eTxwvMpK5Y)vYIXhW7F@>oPKWzcBc2JoU!#@>xH?Oc{`aa?x@Gi8H@mHMmyDOQ>vZ7h3iG zJMG_(aB|=Dgm$|v)${_%6kEMrJ5plXO5XpYx;jbE>u33@i9Q!+j3T%@bw5YOF?TBW zA<^FfAZZZ*FNg@ksJl}EG_JgcUGjYKqV{b!^+$hjk58v^OZ~vgV2H)lcJP~Hwi973 z`P=d3w3*xoQ{^gpSL9C;Z^yqWmu|9l6rsK zTAr|;!yhS+r$S9ol^+Ju@VK~YeJh@EMis!w0eiuyb@05D^*N1(*-+C=RQ`!#OmcYy+%dt?n1HapTs zy$s3wE#=oM6k8bpF~1yy>||6@Fhe>V+D$8Mj<0QMghBh%m8hv1;>-eQSPbb_3vjF( zu`cH$;5iU+9kJO8VbkZ4(v|4b50FFdX6kGX%=d5P>!m=GFPdWRz}NCH(y5ka6gQz%ta{Z|UXDtD zb8;R-;(DVeVE%|86WtjzXNt9sDC3z~MxN~>_$1<*+4+&*II+hkmz_dV>knMvQ|v@9 z%6c<~SMik={c`?PT#bJ)Vw?I@?1WUEQ@TLJ;QC-uq9 z-mO!Y++dJwGY+So$St8XgI5iPf+q74wlzRqcBC3z%j- zc&S;mC-vI_7efhCY9ydpS!)Es2i)Ddnr*+jBOIl3h;ABbwcIX3LM5&_@Rkv8Lx`JJ zM^|!V)&ij~3S^_yLBCY68`6B$qe^dZ@+%aa6dd8&kEV)>5^JQ0=Sq0yH&(M1OfX&P z=pF9&5HL;M{Rb~8xZmyX*~u|k@TSN8Xuml++6b-%p_r3LKB7lItsmf+v0#xcc8@A? ziqqnZ@Uccb`2y_jh&%GgQ}M`q^QQ^=vh^uPBK5uDf#sNqV@LLx_QagN*|c1q9nh47etOYb_tOcehM<=$#3@|1w_v~|I+Zjz)Fu~sCVSz6>X7ye>-(Mi8|aroP2la#*bwsmU+=v8f4W4`bGI9> zxCceOYbPUoMOCYL`3rp6#{p_#s!3?P;m)yX4|O|Sw(>Vq1nkOD5s%HXMw9^dBMVk>>K0k4q zpB(!;`29UK*Pr_Hg8sou4@l`dZLuNuKRNxz+aJ6OuLQ|kpVROD27mnX8Smm_%>0Xp z|7Lo;!}i7heIwnwMFGD(0OA`7XTpFq4XZG4RhUiTsYH(#yGz}6cIBocz31Hkq2w?$+P^yp`XrTymUKprj>phVmiBeZ@`@!SP-E0{jbGV9Hq?5 zfnSS#_`wFW{5MdIZq8C8J~>$QzcUWNd!aS*k?R9_BY4KjN4oBw_HV?+eGQgeD%u3c zbizGFa-;?csYMcHhe#PQ6|+O^^bjqRcD8zDJKS{_j|)tkN=n~K?4oFnyFZr{X7z(Z zDFOw*4%K_}OLGDm#4!7|6GckhL;hfm?s-+c0x~I!ROpX=Oc_G9ReG#>l5(4=lV-vo z;K<2Q=9zeLcc`DcOel4VSiXsK*~DSonbQNix87ZyZi#pR?z(n|TSu8Uw(?&mzzfSm z6TbmQQwpcDbp?>8aXL&S|EjXz5J&lPS~vzLK4|F0&y`$~Jni$(umAaY6gVwUh=KNA!V)(zFeEze&_MfSN|3GND>aafAC$2xgG#^%uHp${nL?kIj=l3ZZWkDQ+Y(OOO zP=k|OI1rPN1lmlrBWbWTnIvIY6jp1!(YoT?Y-mecC~as-^|Uwa0_Fwu2eb<2l?qIE zV08C();~&0ew=Sp6Y!XG-p@XHXWeICbG`c>fA)&U^nMP7fz-xl{O|_ufd~eRH+?j- z>#^RdF!djYgX?iUrV;7mtRFkr>f2{U#{ZuMDqyVaKWxskolxp)Ij3M(@7uvtp zL(N^DYJPd&+49#7y1RVj{QCP=7kKtpBLeelJsuPE&(JHJ&t5$9oe+@qeNv$RW3ghq z#WFF@Jt*kU&6rUtjoO6KS9svB4`ek-4N}7%n|*YCEtSQhB#f`|0soIiAyR6UCJ=tM zJ5*5qXJhm@?|bHJ|GF9ap10DM{)b}tpOZ2FqZOcU$RF$dwTpRsmj`Y9zjD0ay@C7{ zOZ^L{tw7%qYoHv2^I~w0;Pb@jn;r1?3}#A|3rAC40EUq%hsgLq)PA-UFf2Ak&oC8i zrI~TbSUEXn6NF&$R<2BI3H)FR!Q5qBrg+DccP{ArCX>K83>=Sq!e}f8PF-BPzigvY zv-4re5Mh$D%VEi4ViL29Vad(dc%I#CIW%m?c!Azqt92_un|ZwpjTY8SSvWS=GiE52 zgKmN=A1+obxU#10;>n#I1x>_>QYLFqR9R^@<>wzRF^gGwB2;~x?Aw@gHnA_HM_rm- z{8`%sGbAoIP&*riBNYak@?6a8IQWq-hyd+y6l0r@;tdNX-gO+~JugoloM>}56O;{) zFuFIU9CxwiPQyV?*|y|J7i!Xln`H^Qt-g|xds<)bqraM3l0ow9_%~DVa3jnhYd5Y> z1~iG2Ftb#Tzj36?#a;fPu$46g)w@oxa~e#!IA}S4$u@a)Xcg-vMwNNW=qHWEP$A#@ zm}&|ebEZsDCst(_kS{vL5S55Cu}X={Q)^xninidM^~pKx!-9-(YzCiOVm; zi!zlk!$vK(*toUz*?pkq;NxbHhnp9$;p)@lUi@G5>M9KLI|r?U(ASozB(FT zatLRQlO1CR`VW79ze%tO+BiDpnsp`bOHZQuNYho@J3qX@@X7O}&fppJ58lk&s8Ho; z=Euavjdt2Me4Wg;wfs@ljqJ%riZ)wSo z$Ffw3f6y>$eDl zC#AG$I>pSAu~c=I^iSC%p0EUrjh-y5CLPBk?ZjXTx>FwI*<~7asqFmQDqP`3wTf>m zC8mUk_;s1fcqo^a@ua9zW8A7$^K8oVVr4PBPcu76BO@2r2vTLfE}7k6ex_5TjS1AA zf@b!F2%?39Y|~z8&`J3;ahVPougLLFxQNUKNw&E)cdVey2GK1NOFoLfg~}eKhU3MR z;pEApHWWU3S38>E#8q}4(WO(Qi3~sSpl#*bH}a_$&uW2wHAHD9gmT*?n=RB7Cp_r6 z4%KRc1Z#}h`!$YP(p3WW)1n(r(^Sfi6KV5$3hyFm1zmVZ}_!( zO5q5~HHqZ}KIIQmF_H|FwQcN#!6q4sqPY}8Y?gOHJQ|YZXQSz}Hv=ByNj+Z2_$z7- z<65hWtR$1kMf<;Ty^YJMGUAzfl(L5{F)6@R%(=}*-nFN$Vw|PNBd+HzOa!u-Nvi(N zOlodLDDjkQlxnZmCQ@=&_)_K`hroalLHL>OV=vcC1HBJ2_Pu-p0=aUnXSXQqx_YPJ z0@E}yIlJnW*yeVpCav0C-Hy%$>twansyn2}W(pd!(z{!oK-KWNzU zR7DR#jnTRi|Js%(oSKKYUvk3WJ6ep6yVog3x}7FU{amu$Yv<|~1fR*&;=1?Rn~@$& zqbwEqjZVdC0E3)QTw^VLJ8T8O&0^5(#G>02_`}9_nY45|t1K*QQEGY`L34RsWwpwj zmbJT2Q}XiS6-iMxmo)3OXca__#X6<+`Hj+ga zmFCqJjRR?GnwoCS9hPh^lhLYnqJO)1xHAQEx1eyYz_2yTS!q3*3#_Of9p0_Rw?M%y z+@G{~*-5&11v!g!aq#Y$OUGaZ0ko4cH##CIdTNnUigLY^f5s(v00OM+GYOsKk$5-d z2SpFtIek|%Ww79)x=RyL*GqO~U{ujR_S@7(ZCWQa?!ObGHioh7GYg(Io=H?|cZKX5 zwr`EPJZfQ@)PXf($^p=5t?9mQH2C&K66nvS7o?^ojA-~fczg1;?vodj#ELv!N$#$i zIXs6a1MiOKvS|M1v)PRZs|xQ=;+e%KWY`o7a1>qmnYwb^I{Rd>3WP|_0@oH`S3z5Z z!0ZR|=*@GZO~uAqT%u3ik>0%B@f~g~K5b!uv~NX#V(upg|D4k?wW&qVo%IcguZ`*&_W{N*Fgj)cFE>-B~| zlrNZ#{FC2CFY-}t}k{M`fiW>Ni;oE3yA={ogY>}Me9rCpXD9)}b<;4YY7p^R~5%__U(`$+n z;;?b!$e&ki^7D@Y>`K@Os#*jgRIsq4iZt8UEMyH7hqV@u;J$)Kb|FH$cO-Ia<}miv z1993uPIHm!Lc2k!w-mGM*iW(rgopJOuwbBqdpl5JUu}R)IxR4Z`{aat`la!7PHGe= z@HZy&a1hU)+#vv39x9_SS*R=_b37?p7^MzQC5)j?0uDguCW`a*ZdS9Fy5xKDed4!qG^5AOTB+fNJY9rvm{E4-^WRI^l7e9HgV zCK>bT=kpEV8|{zhKThwtjdy|@QaIbNJUsltusZ{gvs=C|-Wz{OyS798ySVbgIxEb@ zxWdd_x}lu6bfU@S@UA-tPpfvc16%LF@-Gsz=PC8i>LH#~?i)Z|pa6MH2eo19j(hM^(lss?-I3ZK+4;0z@&-N7s8A32 zSsXCz;JnIb&Iv$lTL+3bv5SS&2asVDeIokjm+nQenls{5DHs|myMhdk&KY|KPa@Iw zW9frr$BD2j_D!tar^C#LQaW4C(s;M&{oE0QlcOoE6c^J*mmsZ*%|1L+@3B0!>4S3U z)$6r9uJZd6P+qO%aaE=EElsYnc=MD`oK4?6hRWeie&c+r#~*WH^QBsLq^;Gwh17?on}Br zJLcl5*Y5MM2QbdmVdu#)N9{zL=9;+%^gkjvlNq?72e_sqbj3mbfDT?_6g+c$mO)gN zK^C84e5FAasYB-T9~2!7XY|0? zB?!+;M!!q$4!0YZzmNWc*-ho&<1Ws+?F4e)F*y(&eD$gobssOx4f3v%472BXYQ@VN z;gEc;_g*WEg!*u;62-UW5rlYSb(;V6B$rnXbS43G|Q9IWg0$MuLgY;^LK&I8`4JJkdhu}xnZSPxn zCI7FvF7!-rMi=-u)BWNH>Hkl;&d$!$!h~MRz|}zTe}V@8GuEZ3?kjDIA$-HZFbPqj zAZaZmvrz(Kv-hSpOJXgs{7v5&UR!d^C(F6VIbfI=OQfdEhzB3{D(06doq7WNP{_F% z&)|?biax~eYV+#)yuS7!|M}eA{Q=Mz!RO)=)*6b8MJ!>K^v)WF#nrUOh4e1_+Lc#0 zl7;0zdqZqXK5c#2Qmo2j*;;0dVa0vDYYP43GEGoBK!=5AIl}mRyIzaCV}>DH;)aRj zajU>2lVYQ;mfH$ntcMN`=V3aLOPZ~_zd7N1C=)#D+8={AAPCg4ROtg@&3FvtBAY=@f^N zIzt#KJbU@%zsyFzKqU_yxe7f?yRJAE5bDw>ds|^QF00+bzkEwwRS)AG>k&F)V97ky zhwqAVhl_~EOcDpNM|YaNuf!5qeKt^zwSHpHP>p7_8a~rqLHZ9`kR10izotsAU{QmrNS2&4 zBPw_Py+CO4bUmWm0sO#7oP%$d!feK~WzC$oFAYmV7C1 z+Aazp@MaY|7~)_}&?W;3vQC3)K_lGti&h3w6D6uydJ0|uZ&+^`M;*pklivwS@BkwT zK|BM$Q4CiXL<9$`ko+Qc?`JZ$bME8vc4Gmzf{YHJ&>qKr{|nD0maf9+tR983nEm>wrfi&33jFlqNzVL%EbVrKk**=;+mD?M>TRmv zGGbhd&)szzQV5B|gvv>eVsCCE34`m`cU|H@*$G|33PFd(yZyVMx5%Wj0UKOLg!#1j zhev@IxDBRSqdeboU>Fmcb%}QCFK+0FpL%P3Wm-+aj6#{VgYRx6b38Ir;OLgRk9hgM z0_e;*+Eo{feto9kUdZo8H36Lw#Z0xeQ#>~uv$9O}%gSN_>6F3Il-9wN_7{x6zk;e> z)e^K~@pu+nW8?Pk-Rd=Eb8XwvL1KMn&Q)!m&YSF}>{E zcqGv_L=2@tfxs_y&8rv^Fs?Tc*N>f__*s(woPNx>m5<<&sX(*PiP;iin(T?V3Eg>Wi5h3 zMgS6N_X8L-Hc&$b5`sVkgGd4pQ^=4d8QU`Fzi^$_1v6B0Uk^U0;S`dAW(&HSd%QvZBWC{@ z7nKiZcfL2%{Srm@bD5_+P%V4I{`uY(1wFvm`7Dcq8tHT2_q#lh&Am^JDvkU(J7|wG z4R7mkAEn$G(U8O-3!y8CGL@bt#bu&aH0^$6V2FlZ-kXF`CK^EmcB|=AbkCf!0-#{ntqDYI7q^#py>b@+R;z)JNN2@NE z#_TQw8J9b5)8y=-6-$%Sl;aqyJYym;neCXX+^M@nFzF!%`l`-lf=aJ)o%nbRl%@lr zH6be3fuY(lOqq59Ft_M*jOmC8UpJo6;>d|^b~1PA7y{jr^5ubs?xItDQ1~aU%4I!& zl9ldZqJ{1v{?d8*g+K>H&ZK-q|-9NdG zO;Po6=xincF<^pq>L5DIM4#^S5IM}5lGpL@sMvAL&gchmesnLLC&rGnC?B1xQ=lUH zgLic9W701#%Gu5-N_w-o-O;Anx#JYMcVF~vsDwmtV}&$zL4^|KH1*Sk?FV2T?|3h| zlLMu)F9i8dhNd&(&hAh@{!Hq+6uDd%Ty%V=iA&w5_U-{u;D^Jh0&JIeNL}B=KK_vr z*-oCbWcDVSeQvVvv*bKeGGDQ@wzG#O9*-L3or%6S#^Sci2e{C$qAkZ9x?AV@#Rt0Y zcI~v+_A^b7aDKkB$=R+m4-a}jZ^5+Jjx$jYen0PdN__i?p{Z-|j`kT+{6u)}Y6^nN6kEEukj}I_y4xrhH!mSppvTW|uWuK?zIqH=xA!k^AhZfYJU(Bl(>=?) zf8>{tAzakGgnN1l^q)JAC^bmEI{44xYWw9(9$TRtm%_Ljl>Q{~pIP19Uf+dm^56pI z0S#P2T0aN>`?6u>D&+MAIOn_Xea<~gE$8=#l^-Iw1-%r;eJ;>nYNOv+Yw6&}cmnTQ z4HoXH;@N@r9*`3}Hj%XQYBtTbJWjOVFUL@D~{*#wx}zFD91{ zIwm_D-W>D`z2JtcG}Pyhp578ZRG~cdy({u! z5F|AHmVOcC0v&B^k?_$%nl&R!{_ioa6vOhkBIA5Dkk(c2s_;`nh!|%!_B1cVb0BSd+~~iPXf0KYB~Z5ut;8&L36Crhg?j;1lXj zOfc{TdhU+Y0~0+4_mu>r{sMrJfqM(er23+)>u!O18i)BLd;{1|;=`Q`8}sh&qOU%? zP#UGhRX^pt)x=UDR4SWmbpgQ)o@?@>a^ z#whBo0hiBpd#W%n4T6Xy=pj4XP@x!UUGS1ja;#0(k7BcCf#jskj8&4-!%(Cz1=^$z zOu1`oYc&;?R(CM(ip7%?=d*bXD)u$z8v)Xk(evplERWlOQ$s8=O;yw2Vyo+Kd2C@x zC&?2qk`G+Xmyik1BOig(rVg}wiuA(J4t$S^dAB^Iou@CswJo2=Rwp6ETi0oEq?}*^ z2cvRvXY%%3QBnXNchco89k05)!)A**WXp0{KZ<*?i3U9v0OV=udV`3Ed5S+Th{f}R z$Ol3-MA52hboZsnyM?LSFk8USxJ|vR6CZRv=LS#Yvglhx8>ey^6O=-ovYv#kV4sJw zHl&2drKkvz2!Q#OEVk8iMJj<Cwenu@@>~oB4C{*U_V{7o@`W{=2(fOTZ08fPH}? zb8`&r?Qu}uNaS(!z+%Kn(&Z05Q=hy3pmpkA zS;63Vvm$c&V{*fUKOW0nOggo~kTZa9fW;kPf5Rb>jtt&zkT<)D|6KDR&OXo*Fhf*+ z=fR~txHVD@uve8SXgq@J*VE}yYj13pEbw=8H$u<1s2Xp{h%Ebkc zZ?nrGU&Fi$NhwRo7>+UT4l5StL^p6LG&vfpS5Ab1{|rF%LKR#l@2JVQ+7b3%JAb2Z zz_)DsN*YmM-Cg~?S${v1jafv0q0wNVD)Pm(E>cpciO9POaPXDcaokoiylf;f3DT;AJZvR(sj8$5gs|nQHcz!{0GHmq&MjiY;J7cytpHE7BpfGg-Wcs)6Ed_?DOj91B1IyftXO(l>c zl}Z@1+fsXoSh1y9&dPt7oZ-K3iioXY_U2@kWm39EWsexz2qLGC^pLGP@+%I@F=n9n ztDpcbMDF>Deq5sT>#@2Q(e5I0n2&y-#io-68ibzDu!VZ{G`BOR zN@b)OCNBbR5ZE_$MtI@@e}ni?}+Z`K+yiLY74g-)l0-hT@8%-hRA zF5>I~YrcEg(cdaih z@if)>e%c-#CXb4OJ0DDc@p#AIrQ+HW_BpHv%25EJFQoQh9qeDk&fIYu?0dF1UT>Xb zE|vK_sOq^t?ll|bu8uOA-R zz-8!H%$E%gIq89=u$R})>I=~y4c-AP^dz*my2 zE0GT0I7}7c5801xxUwjx@H4#bH?k*hY80CAxAeERf@W!lzVOlnyT@!B%sak6LEPkO zy1gB7tjyFq82z0~=BpHPXy&R15pO+4sCQV$SM65?T^uHs#o!!C-E&|mW~T6@P;<`s zC#?2{GwtS_^K`^C*2{CiWA6-7U1vOnCY-32;z2~}QwR}NrWu?=x`~c*0m6?F(oQuCbJZ=S=@D-_GzJ@ae4{@?t7uI1Q7|?QtspAgijhKdSFCtz z=QD}sqyUgR@F+#nhQy-V$ zqVI$B$sU$%$|Z~6n}vtw?E)vV8HHOq1Uc=!o(XORl|V`XXbX|=EgGqhsdrK-xr zQ?<*hdOMlTX+Por+K%_9av4bgVgP4)!4Nd=mSd)WG=;y~f))~03O2~h&&S^G8rE=w zr^5&hTr4CfR8ULr#bB>--mQeKPLqr`>{X@^=0(J~)i*88H`npUIm^rfx-R-e5X5B8(-|kxfYiA zy^q4&7~h%zJS13j6Pug({5pCuv$1LK53=KgBtwCf6%8vIFP7)<7v4@h z%dKE#l$Yk{mg*fvqGG|?8$VSM`3=9$p&+yRg>_F@5v2Q(bT8SPd8?NWU7e?P6eC^^O#RALaWz%s~S6!BE zAh)5JwX#%+4Hnu-fLStEN@pL=H%crTCRpH$t+n3uaGaml zN;O{{tAjgq$aMjMzfp}2#RUnu=7%`7%J9lezdSuxxXq3P1QH{I zRnhn;;K7bO7T(ND8Z&DMX5-$0c5x=JPMq8i>G*a;os0_?OLzlUKR_G!h8Q3cVPbJx z=V__&V+pPA;K{esO(a`ntJ{5B^Jbh)!gR0Mi%gZdQxaz6Z)1`yhxYN}9zOC;FG99* z{cBCg)JRS5ly_IJe03+9mNLT;WTq}r5id|oJvmn0zhkc^P#GELQ42PZ^{wKF=D;+{ z6#Q^`vpSwO^2LrfJ%4-byMS5WG==u`Jy7O$PyCvpE6kmx$EId&&Aeh>{Kb@)&i&DA z-x&fE2%0Izs6t#Uv|yzmpJ6ts!&;0TXNS!^*U9)qD`BQ(uz8eoMxchD@eX8fF?c*l zo#xiQTxGSc-qhS(Xf`o=oQa94!M@zMScRm*&(Nx{qLiNF$Ry>kLfL@VriHcCR(?n= zN_bi#>1JEDv9+?cvbNaiC@9!AE{^z?nE8-^$DpNFv8L^p&0ak5+#+Jnz60WF3QJrj z=N#$A-=kL+dsIfQn+xd4wYLZ9SkIJdYJHJ~g-nbm1C#yvhWk)hT?(Gg)Xu~Pp{26f zzKrwaj2kiYtLPB!-t7Yu@^~L?M3nyKz~>v>LSQe*wj%M7Ki*W$ed*EDC}|G{w2O2Q z^_o}FcaPqJYL~@zF0afh1)Sp{Ww>dKGj=}r2j5<6X|7eaX|mOCjy$odXHe;RBz^}G(%bP|Vg zPifGO%b^JxENca7qC8TR{5c@eqb`X02L*CSmS|Qgc)o>r0;^reRn~qX z2&!j0*Ve{}5#bp~6NEhxT9b0FYmB9wgL*C;3}jjIPyWyO*KS7!Zp)rOqTjk*m| zM+L7z9n=TX=y1W@vIE=1+=QI*q86+ws#^9B22lo{WgbT}CO(AP|trODC0lh%ZxbC_lEAmj^B9#TBK5$ufd0E9us3*!y zK}(OlxSoAh+d9g)Tt_xtX%1F&SKDyoALhDdQ{V@9pu3QDK^r=;ir)dkKLs;o@e8Fs z7?$&|X*$UQ?wChN7%`chEFxk?sG_zwn`QQs{6%Z&A_GPYKjJx5Wd?Zu9&En-qBC)d z*ROBH!Oi88yo^SQsGCb-iAI~b&80*Wj0|MeJRDXKGrf{0NNl+G02=&C@?9y)$&s2K zJry3s!l%grwqx?2WT6&NHGH@*4J*?_6~R&>&kex{rh%VG{WXL7Wzm@xjkM@cn&u`; zOJuUi^h1WNrPrrZInnr%BzmJFG&|F)qL`8q*fatyT6I{%J*>;tNba~%px4V9810>t4@Dsh=KeS%Nm=K_aXuV>V@^W^JGJPZpoKqRh0mII^2 zHS7&OKyT%>>?1TX8~4ujLvWy}(J0ku@G?;0_b&|u?ja;FfY-tMBZ>7#=26tL8KJ0C zOAJIyw%N@{)Y4(f0XjE3q<8iG){_*)B%D zF+(13$!-qCD{MB`rtg(o;^{e4dWw030xQ*QMh#o5@KQE_LOk_bd>G!6+8b;3$*hfj zi85YFn(^}Ę-3`U8T#PHoKbSe&DAueEQ65xwm-~^T6@j{ZIiI3QrEu3s|Wz4Of z8m8gxi6943M9sRH(5|+bAWs!DtoyH`B8;86N%RbrD^3YlOvsnXiCI$#CM%qcg_*#$cAE)ZnCPy%R;SGS5>1J{YrzoL+ggY^R)9Z0c$hAh}FS& zzZM6OC4?{ZN&&^h!D3~DKChdxHZHb|n}L5OUR6ETESD7aHEnFdrcN7Yuk9qnV$^dP zMcGb`x7V@gqAw105AinndkcE#1_i{U4 zq0LBbk#j5wa{Y@Tx}6_^G>#{^+7(74p)Q9kBwihIbyiU7A|VE=y^;E0gf2LF6K*to zzr!YU*);&mqgXaO{lKyZ?!`Me2Sdmm2HB8!dJJ+0R@oJkMy4mDtPTvaFKYD|>=V61 z5{2@CD=mmQoGb!1p)DMap^4NM3ztBa2=j)s1EVYwt;RZN)eUOa@H)bb$r2q~QK~Q? zqxkf6>FPQ33CxB++6zFV#vV<uA<(5A4Yl)!-;>PyNR!LPi_wE)Ux z$P8c+1Y#}AAsrQ>Zu4DAR7oRbOvx28>u3chfVO5kgH;T-CTrK#70(4e#x9_XO$}C& zHRPX*@t#EuUv||39G(|OY@KEU(GYotv<)?{StyPx62dY0&yXE~5|Ef40T4&jp>N9i zTvR%H9~5mU$e)qqg9hVO>wY3xa06DiS3p!R0OZV*I@F(9UmzHts3Yrd1hl`e-Jm3= zs7|?#PHhOPzg+%+z9cqH6KSHo>q?YrD~%YSR^OTtE_aLq_KAH0HX3VqAHw&?o!dU* zuXb+zpb#X1qLEUSKfqdk@J8_zhO` zA{sgwzv$~hXal>*hpIJ~Tf%FTItmvRxVew4iUXRZubku3UKUL}2s91F%;8M^hC?U~ zuV8Y?pV;~R@QKe#Wb5Y7zzQq1n0jXDa&~ABd3pG~p%xJFdbr+Vdgp$1QadQwjaqg| z+t6`e&`Bh>MQ%HEz(fxN{VVZ;4}jQZ{h$kY^oH}EQ$;(B`;>6Kz=7kE+8{u$yiULd z7>Bhfwtce&5`wyxuaZD_-aismZvV_uOZ9@2deT9iYs+c_Fp*eDZRh&{hZ>5+@(;m# zLTO`us|K5zuDySZTyrYl<%evpXSJ5}?HjxT_#8Ul<|{TV0oW5QA;+;HTd5MQ@PRrv zZxycvqlvgkh(RUz{_gk%;|CNF&n9m`dXoCzTE|}wM0+&%2KR`Qdj#Ela!D=@vil6s zpNay0HSIHr5mf-Z+<)DKX83T5;)c&MV^BfyLxOy22uSf)ZLo{uT8j2P1u5(hKp5ZM zolK%_y(9nrK8$oUUqGt-3P&U^XkH`3P$K)ipx4|K@s{-gMv=T-6WOxyJr!skCBINx z`b~6KBgXfb87r3vvSlOq`N~V+W@Hoi*~=(FoPZ&`t~k3tU@64%FSPKDi^ugX?*L;O zeYbdVWHH<}ZzSW%<^pwg2&PX!J?eJe<-nx>TR3~1zi@MyIx_Im|06~!p05<;P%eDp zS<3tdg*`mZs86R;a*Kiw&~Ume{H^#6CC{}Px9o`=UC{Vbp61nb;~+J9@D~Z zkxJKF3q8*xO^ZSsz3uFv$ux*+Q}XO!#R1LA7TC%*42xoAOwRyS4H17ZkXvSjMLl*b zQ>e2E$=d*oV&({yJ5&SsVt2&T$|Lz$N15(;n|0li=TqYq;vi2r@Sye>dh1m(*ZdKs z-kWYld5NDtyX+(o^i>UpartS#H$UNmDW|{FuxD?l$T<&$M|!IB0r57>?vzKYDYB{N zCd-0o^_uV`vwII6a{(OW9tN73eU)a)aU#kIMNK2wDolk|EDXNqlPD_9HX?22;F%!( z6<{m9rqv#FJqn>O!2TcF;4^n6lApidk2u6?-kJA3*fpwOY1FXZIn+J7^#lEsZxSB5 z-6Hec1bT7)t+ZsFZpRDAoZCr<`@-p=cEig0@L6hxkd9oTfRq&*>pf zm+}^DfQ~qH%3>O*0|n}1Yum;49cminzw_&Xsk`4 z0?Rz)yfF9UNT(DHtNsgT(iOPQK;MdCt|+|xbfimb(iPj3Hskag(Ws7}9tMV!T6QCP zr>|-M^pM`jZGP#qIHwy>xxGNQvwX_iH@ED+1^2R_g6f+o2TPw94hs?^Tf#uG(3GE||s;Ih3U{Y+Ds<6uTw}hUM zYa;6RBxZ)?$|kYyfUL~DL*Mq zGVV~nngup#IJj-XQu(=m;1-^oewu6(j3m9gJR+~_hq-(b%!@HzTw=4sXWC(BP+n(E z#4tW=V!N^PPGRO%yif0lp?vs7x5H;4!Ya1#J`NxPdeQT4Mo$X`Ze{Jzz2HxAghJGB zy_(=~kn3%xV^HI0^~I=2>tapEoC<^2&5}3yk4GNO^11Tr<2lY{Pjd$Ag0tgto|+$L zvg@RcCT;1IZc$3>)Kn%4G)V>{3nTioWS)XNkRvX0jvq0og!ILIzvyme?5U@5aAZPs8br%k4BT%)Q&K@j%e9k z_#~M;sKg_{Ov1Xr@vh97c>Mb59hmfn)0A~)q^TVylP}3-6m!?wMI67nMvKYh@iIbS z^K`H|N0VM)?>`9iS}GeBqHSe(1^71h=wBRS^t&>lU3=u^I7bS6gez=--w-VCSeX?3 zQc1@xt=skh{24DYU z5%{(05zva2zZ;3J8m6={62GLKzXFU$u03XR9QYaTmd_{|5eU`uiJz&%FS6~(lN5)} zf>r1#g8WeaQ=7J*yPVUQa9h}R>};pz7T+T8qrAyWBF2lK4IgrH{D%Lpg}BMHz^=`| zWLNXw@;>(eH-%Rgt|r3P7ACgN!Ujg>CM3-NtItw-Qx2OEftSppl_m?xyd6bql5{R! zipItr_?eW7N@+u?`FNXgW7?WRwdy&ktHj!{`eEgxYRGgLba*P#tw!yvu%b+T>r! zNfU8m=D)0V?^KUfJaCN|1Vh?E!n6e6Tn=fcMyX8Xb?gbRU8c^uK4n$gZ*Jpw56MkG4@z5)L0 zM7YH>;}rb$UfUA$jYThd(`+e*maQE@-MpY8>A}acq-@*pKbIaBkv< z1kvI4>jR1DjQ8b0#dXL2`Xi6(U<^B;6W%xyf(t|4sM*IsceJmDAai7deA@Tv@DhZ9 zAK%3XhUrrD_h5#9U;b{kc`{>0@7e>&bu>sE`a}%l08b4hKcu)oOE-7-lE;K|MYSb$W{T)ag|AO*$T} zKq6bJHj-k84&_v+}D_US>wuNGP90`|V`EEGLvgQRcUo6ITmk1v7^bbjhEma*TP*^@bLmjYuEV%^HS>9i|2a6n zw@5`}=sN5#GN!_!HkvNB$d+tiwW1L8blMbMDwE~FS<~#8T(ov|yClS;Or5XFLd4Qn zQ%S;wbz)$frkw zx|fSH&FBf8RW_5C6_k$KVVy>wYtb*RQ-uBiRykQXTDwmEiw}AB{ft=QOoUbjqM=P> z;-bY%sQV~=a{_bNGqhvF=A-2=FqxRQ1h*O6L~CK{cBv*FXx-VB9(#P3={vztv<`7m zldwa_yF@6fUzJhZYC$OMJIaQu7KezY!Kk_U@ZkMt#}e#X{^#5%U~-$d+`@P9DhjD2 z)>VwsK~b3pPlH~Ulty9ar7;YWl3V__BsmtCszg2FQ_!m;yU%Dt5Ni_>DVAt{SF@V zr+B#6A9LjrD1gG=-D!BPrvDGOi1{FBFSF^_O4DS8K&E1HeRB?Vrn%|~(K&(~Z~m(k z9%y8u8M2+Bq+|r4dB^(DRkpo#E^Mj50jz}#=#SDJm7bsJcY4~Q))d21JpKLHN*?UW zd*M^`vEYD}_m*bp8tZIdQAZ2ykqL~Z%A4=&dVA-awO4Ar*awi&&KB|{;USM4?7vQW zqV416-FkDwz$I@!UK~2DUhfWp3@@B3hd*;14^RLb}DCAz#iyvuur8qy}7NG`qv7qF;r}4}}wyI>S)?nwQYSJM8LRn2_&J&!IA} z9!36>r1Df;^@^h8bczk2*Hl{brH0h46>G6ks@=c3rMI|9_eZn;{Wh&VnE- zpHOisJB&o_#EaKp59P7?7tiMK7eM`K6Mjj}_fOG>%a-Z#Okj7=V3C zh1BJ@lHTpx$UJF`5|Q-`6@}dCLoEk=qDZS-(bTFlohb*tCKjIGQ{<*8z? zp^$S*C(cdW$KNz2nV7ncl=e3wP{dc9W@G%O;VyoBvhn30g*Q#(Ea$w`>ke`X9K++y zO|mTB51oyYgm*9YSLR&?&(-W7E`B|Da~Kh69ZAon*krBJktM&V`HP2$GIk~`B>X0E z5P55Dw$aR8d^>M!*+(fz&eq!$#`?SH)&k6y`W%GRSoNFOn;s^&$?}rvli2>d%m4sl z?v0>h(#^e#B<}?9|$j3rbpH6K4mhd|s_^a#G0$ z@r4-yDzXhDo07AOnfP@tuAb-8w;+kdBkhdh8jbE9UDC_#=_I?xnKgV#N#+>)-vhRP z?a>4=W^dQ7w!uTg3yr}u^?n=(XL;T;ywv~p*}A;2hBi26DnNl0qsV-WFr@y_5|v~y z6UB$VhC7LMEqYFigEDtZ*n0q9Jgw8g; zLm`)E9Xb(CUz%sn(9-(L5!@q!U13>417+&zgA$xACO~)tuMGuFu}958WA>SquRw z_`0@YmP*z`AX8b8nt3&(Z4t?}gCdmv3Q}PMjd*Yqd(IEl7k2k=YdnZ!vfv)N@6-2+9FN=s`)~eLUdgb47j^5HKDB-scpg!sG0Pi z2$)>%@-(-^H1Chct~=X@+d-vn@LSStDdR(|t0ud4hSGxeZpHFf2h!sl(9z2=^*dis ze0NmPOB9`~=^}Wt;T{4sxZ3h|6QClWTi42`Sc99ZNL`AxU|pC+N$JQ1wIJ* zD09+*=o!n=f25PK1|C-H1ZpT8cpUy$7ceuRnB?WHDXT`JK+&(tWvCW=g2x!eG7-rh zg;xZq%u(Y;kapTYZM^EtgkvK1j6;8B&e>j<(`OvQw{~S+atKx=OEW*u5MGN=;PpxI zNT!6#M~BRZiDkIrnC5x@irfd&+C*G~=|X~EMT%;nEl!8sBT8jYgFGp2O8k5AiB&ch z?@SGhqkL-*l&5$jIWx=igq@(I!G4{RgOyvTAWYFxGIhe9WRYJ;A;q6o;x7>{zo4Py z4OJa&;L_#V9Y71c$7RiN%Rc=H%ukbYu;w%{d=_e)FCpMPIvO48svS5!{@YJ6(nn z+UbWu40#&PW*6p~KSR-BoHAp!FFJ}G(;H4pB;ZyO{$}cBN*#SN=J8E&;CsnzUaIYg z7&oA4#P}ycDp)E8Pa@<+yI_NdK0UZ0sN)WDE?j;o!yoD*9?uCvuNU$`tN=&4mS;;` zT^hE^r0D6fKK+u?rr&eD3w3Bii(gHnGu)Bjgj)E6HgXcUmW`d&^K$m}-{?Q+j4xSD z|IvH)+a>z{C!DhWXDX>t^^;!|!0@%jUP`$jBftU`&9SAHc&8*no{vZfHw+}apDn`C zmRu#a5xR-Fn>VdEHsX&+7$9A2vkO zOjb3zm}4w7P#Lw@T2LHLkANW0A2;H)L?_(5~;$MVBA3f?b5^ket~Z9j0AYwxdDm*VL}ewe(3$*9~yf$2aUMV`%yf!*&r!lu*oR4<<1cDdoc=rfrr zx5w;fNYUtr5zQu)N;6<+q>uvjBopdIlj;jGbyuV!H|38=AX`a$v2H--T7YJmVt86a z+4N4+rhBzqx%k^!KjZ=1hS_K(H+^WZSZ2SAGm4{`S#X$$G;908C#$bYNyF+<($tBb z{J^^YWOKi!c)5#@%~!fKcSGyZN)0?(g($3RYnje!5AFmuQ@H9M&>g_HWpY-|1N>Ny z(H;gm;=9{V3NtS#k&O=X{>H=%lhZx}1OLPSgGsh2`Bs|DEv^Imi((2$Tx?KT>st&f zxQ$z5t4dLT7nYX|uJT<*=PB-Mi;(WulXotgmF+B!3{J#?P2NEhrgY`mM2l^;ocst; z8K1P#1`#xH&6YZHexOqSW6B`?!?x+@fji{Pex{P+89#BVx&yqlNSNP6Dr<-TG42)H zdlDZ>!|7y8r--_M?1wi4d!)b1apFcOE9qY!DDU`uk0{T4k% z{RpfCepVRkBl1ev?8ex7Yhw*e+dBC~6-)gNKFqy!&@xu2pnC!+H)63N>M~+ptK>yH zs3%T%VYg>`L|rIC=s#!CsSw2h>@tbRJHZ`(CI4Tu`H}}Ui0!YBu?^|}(k=bJvbm_E zqn+dbBXCJkTld3ON99AN?`%c&GPO37=W1yhYeXg|v{*E?K=Ht-nn#m#r7=_dyP%P) zlBrmdfSXpLkgDVgm0WX`+%hL(kvX@3v?4SK{3*AO=zfU5H_EAQr0FUm`DNe7yvuvK zb(`yTlcM)N`$!Mi|L4sEn`H=mV?XGQ5FJJLZr_yyGH&q>G4q`Q7Qbj85{qw!;O`#t zwYhI%@BiTJD}yTQwj?PEx5C}s-Q67uxVXE!JB3}`-QC@t!rk57t#B=1=y}}}@pVk} zOuS#`od0L-l{<5-Oo2hd-ko`4{!#tRUbB(Ew1LnNIV5*?aPmygplN!>jH5d;#=dDn z@x1_J-{4-gR+Gn%6{^lJ*qOrP~P4B;YJOfEU9 zx^mha-X(KYqDQDQ)hW#UfSCbAsZzX`bH~HHkQa1glAo5(Ik5R_6zq>UtECCL_hGvdjWe}9jbc1S`3(fu8O4pg^1WQ`CG)Z ztk5JbJ`kgk%Z9TgZFRR*hjoJXF}V^ArakS(>M&mS4z(EUgi)FaudDGCm$|wsq6mlE zN2_a{yX0`pQ|gyaw*PvBb+v6Fjd`kPYi7<;1hfR~PpmYg9scP_E$r4Rv4h;CB}WuZ zmz*ik$>L-Qb3Wr#ppWP;e~57~Q26BR9(Av!p3UaLsLHa{L*gTi{TG7nEv*S2u_-GY zlUN}N5 z$B!AC%xNUqb^MmG!LJFxnNjDZBUeRL4qnOMLSh_$QGJ2O&f{sDj6Z5xccZS$ab#;Z z6moY#d3S)g2jC~i*)TQ(Gmb+dts-QBZN*LT)rqx;Koh0sj19!^ov2UQwC981MWrcY zTw1dX#V&jWW{I7yvm+^Y;iKznb*6~{s*Ow~CdsIDDGf&tkvPbSyU}OiWSh+vO#QTp zHn?V$hPn+emiq_I!7Zgq)F&23Lp@~7$+}9i5$J5v%{b&EG8)+v)ax83JYv32q8Nv{ z>G{q>dlh0BP+vvdOaV+y z%al(b(_)!-Unrp(;8LVZAUZChGkEN@wL*=mHgkXc6_&xuOevr?wf|A2w76VaBL>|; z8}nt#MD$iLZkf0wyM{WZOG^P==LO+(b)yw z9dMy`y&SkV!HQW;4*+u;>-jX0P`QGLlY{X`Ex=F6u2D zH%}XE`1ESafEK*=S`x71WofX|rr)*|NOYF}*h&Isjvyp3*y^RH=9sy44<^pJZUDj* zT~njfCt_Rk-eyIPdHQHJ)V*+AQi@%v9M3)~%HR3fxDK=X4+Im4Zw_kmqt=>t#X6cDc?Bq=KC0Ddpn4)L-k33~Uc%K`_P3beSOx zQn{ye8gNS^;a|pu4cdcv1@HOeY95HZLMVv!`xWuDrVF&ssrC$lRi}^P+mXHXq0{+{ zA3iPhrkr^0ScLXdmd}fqQjQPexw)2#!dNcIr!POP zC_8mph!umJ-#hWdS$+sDIETJ3QsMnnOKE9AJ0)}R>9LnxE*sC~##@}sZKu8HaU}|V zbHC(c3(ig#{wbF?tSE`j_Oqhk;nKbM&o4LbD#l1jvyD9;^tEi=Suj-{lt#`7=xciS zMj`vRF@$tGOzmH898jh-YQC)owX2LPP0Y`;5WR5tfltbTYF!f@5UPWEzm5twW=YxJ zMO^g)7u9kTSac?p<^@dRSdh&Q(92Hs968K9Ip^1@+kA;?b|*BaDxIXN^~TP*8vI#s z(hgpeI>T5pxEjO;YW2q5?4@@XvWSP?#jp021xlL3F`uMW?T80961`$RdAU!ZjfLu*l&N9^kV_uw1DF8b}wXgTlH_oOUfLmBr{Q30l zW+rz}nb!dQ=mW-!t~}ns0+C{(wqMc@r!%OMD-^QPhji~m0KYsQipl5$$Le$$l6;wA z%fV|0?^xM>CN_#!Vf4AC;@H zayLk^#{>pbUZBs_N~8w*^cmT=T$AD=X7~QB$fSAKbtf2p`o#Drbi37|5kCCV*J}Sw zeeM6aXpDckXrS`V7dlu)z7oVoMGozIV^~WM z27JkDh|9K!WRn4!w_W=MPSZXU57T7D1T7B3!#W#xvpqgPe(>@d zn_F8HY_f|~ZE_iwKGo+Ls-IXD@S~p!u8J^8LTOx?DUnL_o!xrDE$LK#aU)=POk83zG!rByb6YUz(Cid-ENsVy&22${{nns>? zBPL{CB=WsH2Wq8c^GUz!_ujhr=$)7Ecv!nIGI==1P$6|HhU!Vb*~IKJgx>=zYn8g@ zO#qW&bGnI4tnn@45j3F-|bi9w(q&TLb+VtAVmZNUePi`YwpXBrA#D}gEjI49z ziRO#U!EiDd#gjWxW8<5j1GQ@iC7TTGjoJ8PCE^TnsY~M}C=@?CfU6I7`g*5~fi%WA zXsr>ytnI~nRA}RCQc@TZeEAR_e=<1*tWllmk)v+q0IsJ|(cPqW*MjdUzH%J0v%59w zRBJU~&RZ4ptc)?rN$;Oje5T$Z{`u$BNuBe!f`Ndf!hnEC{u|1y#r#!f=G8E7)S!iSsa*8sr`mHeD;Jo zq!l#A4-nH@MNfCa<}Yg7?`~~w(ob*cG2L{r7bMuK+Xz;DfZI}vrBa~4+!SAT#Hmtc zihfpoXk#hEHlO8?<*CxLjMrl5o@|t0T4~J3&Jx{4Z=V;z{l^k~ZOJsysd7I_dtQBh z87zn%vCCpLnS&~q+)HEzFb*0Q*&$O3B z>$){esa1SwvNGc11Q49HS$AeJDNtoOo!ImtUGXf_cNbr!LXMIitZPDutvePPmEw9U z)K}$_W`mAEP9B+QKUX%@x+%ZExcclb{cY~)wUjWbT=Np1uD1}y9Bj(Ds>*v;fn(a6 zA<0)r%l&pvy>i=c>#iK=F-^&v&QdPKZb(+3YI($vDXR!Q;!$#iK4B5z_S3oc_e2kj zcpdHTcW@ea%3Tt0f_)V55_`?Q5*YseW8{+=Y8aTQoyq_wOvP(t-DE)%_J{57BVbcB zj6!uC3lPg-rEGY5Ra%E^^NaPqJ;muiVZtfvYEi9s;(ocNzFH$@v;Sg_2nWSk29I_D zYVNfDHs(s_-=^l3X0G91?HCZ>q_H??HnHMLwvnJh8h_*CsU;m!p8+oj551IwJgSdy zyN$Sm)v#n;aBs{Ws|XfbWL}B@O(I!jvVb!7>=5)KSK9R49{kQ|NmijWe@p5VEu?{n zc>sHmv!mkZ@pp%3VZoM1E;+s{yGTF2tO(0z%^`!o#@T7{DgLR-aMOnUz}6eVYIhBZ z0n#LE<>fv8jD8hg*)NUa`|c0%RN(ftV9?ga!lHTdj`&f`!ua2~HLNHfCAm4=V8#<{Ej~*9EZJQ1qnpI4HG8(1n|0O@)VCyEg2M=>30vX~{Hho?f-CZb+DoBwO5yDv0w4{jHl^GYSPA^A1W>cA~+kU0jB6A`?qf;N z-UhC2QI9$C4F$wy^U>f{vwM(n(Q_eimqRlZ8Cq~f(O8neGTXw#JL?vPeYq{QgEH5s z9m|N}xTcsaGA-)wm>cj_CL4;pQluWLKsjAIz|A1%JSk2yVu@M3`pCn`QCXG{!^O3a zybAxUa2n}cFH0KGR6%@xkW`bNd`Ht}X^{7d`71x`Al+WK`FPclH!i`Wuihi>(bFKA zpWUtr1N*Kx(u*E_r$c=}32U5Qb;h+WT!T9?k7G!o?(jAIyk{HEXcQ9eBbIgHd`Nr? zgFZ+$-qS0Nyl~0N(#=w~HO{0f#^CTN}UdHHbbj@N-Mds~H z!1OqjU4}#qk*=PzICZ;Vg%IFan_Z+~4cYj?z?p8;+Y~!y>($XV$!PRLlV9nK(&Cml z!nk~^x1i#1hR!G{oX>d<=3ageck%$$!lu4z*kaP}yVbkku1s`UEG`+`HU6RHka$17xqsCqL}Ln>r!Hy~c8W|Fh}#iY@kTgM)xPz=D7%{Tog9e-A|&7+RbB zNB5O8aW()r+d2N{z|?=WUmF058mccn^eQ5{hr~HfZ@zFTtysBgBi#_lEQlp&chg>d z8!2pt&W1I}Z1YmuN1$)n`#0a42o@D)sa#KivAC zGrx5!*vA}e;9uQEPgQUYv7s81#HRirQH$Z7fI_b=?NL%dETuy$(n39vp*tCGSG|>$ zVZmLeY^!eMlY7bxurMe2NNnUwOSq~POwiV0)=9&2$r9G)xK$F|K!VB`Y&}FZQ6=2V zaZhD}?Q|2%WpOyyydAn+m31U$%4kiJk}XaEBs7eUu7(*?C&|v26$*-M)J0_5da<;t z#DD9q6Fdp9$k?I)t;le6^TJLirzj2_H>NkxsceJ;oEd{JW$BnS;3)y8RNHqTHgsw)VXCL{u?8i>qD|iyE$c%@hh>! z*>bDVWn7ztGvwpMu-#z9u+Ga~UvS5By2B(htxN-NrO^>sB|0ZR%63pX8aCSuUz?+& z=74^zti6|qAp_dwRgJIep1ViMts**Gw1?S_8rur>bMWm%Rk5!=9E}zhH^=#6pj_h~ z>XLcWDVms5&N`a=s&#>mfnZw3NO6!FU+JowPMezF)NJ~$cIeqbH);o$Y=FL3;aYOg z$eYTZdbd3q=dd;1m3}N!0#?*0&VrN)Wu8Fp$oe(?Qtdh^yyixpcrXR{ung zx?nH9xnga{ul!V|c*qhBrWr9|v}0;ayyRrl2<6sGA!a)~o!x)#2UY5VTgEt7AG~8< zBdsmJa^M0bj@-JviT|Vc#YO79H&@fpP~`@>np1r|TH)3vKtNww(#^$98SQ>>Q771Cgh@yTTna+_x$Sd*>2uuy^G%6(iXg$Sbx0$h zbEpAdp=(!tyrvq&aBy9rT_Rks!1n%0e}UWWGaKh~H3hv_b%N&;BEd}2s?dB5=R6VI zej=oPs@V`gBf+L%$@Y(T$c~(?m-&NlHNcS?Y;VzXLaC{l^mM4Im6`_DyR!C`7^p7s z{zE?c22@yNqIIOHnaOEY zq9AIhVMcB7nK9dic${4fv|os@gD=#FYxRzZ7}R4P+;YGfOe+iTsN#%iBx!J6Fui9O z7>`+5I2LWlG7-|;PzTWg%CkXCY|*)pKZK#4ui66?-MU7rGbQ}ulW5>MH)ImjDc$EG zV4R3r65}{If)5=8lmxm3Nuq4BB2B9}P~oiDVh~k^*l&Zfi&Y59Sg?YP@dPQ*i1YCEolyIEVRPJ!4Wu3ox(nWp|4zY=j6V+7?Pk z8LZSdi9<+<7F4%G3CF_E=?N9Wo_fS+A^0N3&r&e*yaPWBsrOfaztn4P4;+(q?0eQ} z_M|Dpw&~M_@9XOs+b{l&@O-%9Xi87#`~06GzoWq*oJH*w<7BYuty=SUsYS|i>YQx& z;JT#WeU+%-<8Ep%*FK>71!)^yhIvIu zsW8Ar#Kv3)CoEls+?~98iKJAdH8;|Zv#YXU`$7Z8Kp>39T!K@a-RUVL=Lp!{wFvxU zq@S`!#@MbH?+26$Exz_Yc?^d~VE&>xn-`hXI<{Il)hL-tw42B!q?`qECfgtZkc<;2Ol(M1%3z;TEq z-R`|~ytHr8t@IW9)`z$mw<8V*(3@DZNO`NkZ`ue^@Npo$Q|Wr;YMNMXJ+5X>^%xlugI|VAG#cn( z!VJeL>cja$k8mO3Gd-B`5VnR3J)Gp`&A{xRa7i69Lo**KY(|)GpZQD!B8~j;jXCbe z%-P=YIjg3Ugsg`~fO3-vqwwUSV1el@6_o?^;Vt@N@ttHc$F}Az{56C9&syN4lV%pm z$MZbeI!Xje_5^zDWzZQQ?+S?EVh?beC`em!O>E9UP}L~&Oll`ugQbXnT8%KXQaBw{@@_G}2)8&dE=Vbd>9 zb<(eR>ts;MNlMjS7zI=}Y!|Sy$iE8(ao#yE-_*OT$g6{&Ay9otl0tz_*qUsO7Mwqd zcl$8L8u>=J#Tjq4A%g*CxH{ij!P*r?wE4gpL_SIi;qQGiPIx0u-2Pc-GNBX)(O)$N zME`#zL;u(C@>e#rh@GvIvw^L%)4ysgDb+0#Wf*OEd(O5Eig6f=GcXDs^ZSt3kh}w& zLnlSm6?&mg9gPkr&!{Qc%1)_;7x(B%9z%@A+z zN8~4HeDnD;nUTg}w>7cF_t|x4OWY3pV38%cKO*uRXY5MaYK~u;TMs$GyLg#Oa>nV3 zc<;>E^;%i?T3vqwp1c!hp`Ip6mZa@G620~yZW7I$Byv}&i*r{Tp~_5Zo@QF7gGL=h zuMA(_v~N~MW%#b!EImRuQeU$1bZ_35TzW1eolXck@mbQ(LGJb{BP5~HRRv`?371jM zwDafbQbWu=04&(<#I86u-+7LJv$wriMveR$_!>rg^oSc@ebXzZ(gJN!z&y)L8A4Ef^aC$fo zdX6FsfL0Z{1yjysgr>rN<^|Sn>@OzHb_hKoK-b?TsTZ1s@!ewM7$eTxpR)$_P@i0N zQ~?|tp`--UgZ47TFq`#WNvM!vSSpqswXIZrFiacucMYM9hny*~lDA53)c1E(I2D-+ za4rl;mV(ssJmbm$Yt%*k-6o|iW>~JKP@xSRw=psL$kS!?+1i;wfYBZ!b9-{saBq;< zdv3sm5(QhEIx}i(m?nz|isg7>36ZYbogoizrukN;=m45VLp{(@vap5%C74*VNOm-o zc@O92R)j~^`e?djifW6kRN;C^a4Fv;$NJ|3sNZ@%XUcAG)5@XGV!NnFY%~C?h$)JYF#4HM2-Y zeU^8rYNYv>|n4awyXWHX?ydg+HjeIAOO_;RB9 zJBdI0eUWACd))1?!zhgQQ<*REjF?Xttw@x0#DhT8FfJ?J!UyftNT*?XrT9-8AQov4 zoz#|SdZq1w32U25E{(>T2A7BMsKz_HBQ}rekkVbSHu7MO(R@ym(7`($D? zIV4Uo>QR~O6JDw(h&nsFp zU|QVwuWzl7be3~>CG|cnqPb{qo5knbgfnwl9mU|9$&cj-(WjZ~&W*MO7mX3FZCAB4 zVQO{f^RDYAxX>vtPPP)cH<`;{4it19%*T<_ZNoQ!I-icK#UUEqf?c-ZlDTFAJw5JH z=e-YUzB9L#`}WK51eZYFS2!|N62QH*XOf`Woz9BNv{e+xh3a})bjDO&^*kJ3>7?rN z^q1qy<&RkcrF)52K=_cka&Fi0qVAv9us~ED(E^eC6x8>ukGOolIrH$ji(@@Fr-@VW zKr!5N6wgal+_I}=GIS^gd+N4HjUUi%s(i4S$~Wl^PbLRm5?RFy4pvmuN>aHMA2lOS zQR)l?4%FR($l8*5uvC$++7ZcG31d1fR~>R?J4>?L zR_koZj@AeM%~z``B~ijGguUNEm;3+XapK`AuU~40Jk9%C9ho@`B4gP~4n!7}1Ke<` zr?H{Do~H`L-?L)KEL&ECmi{!KMK3hsq>kDX!jQtko9UG3Ub4WEhR*7TOhLmHTP!DJ z*YTkr1q-Ioqe?3_J0o5LqxxNt5=eBpq(|`_rbMaK-Li@qR00+q&_` z3-#;WIJ8m?>#^Kb?c>be`>T)g7mc%J1}D**iKgpQOstZHv&I#huBd+=>dH4R_0O9& zq*-I>BB&KW3X|2`XzntXv%!IE=h^DcY$K%Rbm(LPd4Ys4ga z$I@?U(@Ib-+!Ki^KFdV@s!w{q2v2_BLM84oaH+R#H9_Fm>=1wd^ijjuAdmO9;LJjB zZDX}ki`}@3UpHSYGKYP{5k=eg>>jE$#;G-K;GOH8BVcr)BVeBbPOb8tS7O9;U-mVF zC{>2jx?XxpPG;QOHqvO?kdgFR&PfSPXKvDOb6e@5N1C!DY~6X2ZQ1zdC$E33u_bmQ)nAMC?y+aK*n`N=PH zo`yuPU)oPwU-^{cDMvyN&d7a4*u1^C7k4!#i`zRX+LzZ9D0*{hk6bY|;&=M*M|}l8 zxf2)-yI*e%oN@#*96cjnqenA<>kZwgUb9EH1Tq}ELq5Yqy9Ky8bmDlE4SV&dIneqK zb9G~PrtDkwEIWGpU4Q5FC43gb)r0h999|B5a_A)Zix_POB9OR`)5|=Jvg3?83`b#h z9((AH%sW^I!{2iwWKVyaaD0HRe%$(%L3u-6+k0bKJ9>l7NH6NYZ6W``cx@1E*2Cf8 z&GNj9D**aeGMetXfU~*RB${T2nhSb5z$()h;lj}y%sX+PyS3K=<-%bDlwiLV{>uI$ zumhI>QXqXFWyd3$;CpxQ{&0ZZ;j_ceu+v|#YZNYD=)bJE{z8Wnx5%jf!2q;37a(yJ&{PSq!tkDDEH<00~p?gWlhx4!loS>{OG)%EQJ-SF29bFYi;!U2nsBzuysd zXl2o0Ob6qYZ(xpuNsEMA11eyS&GtXd4;ehHx^)lQwH+#)_b{TFKRE{AjRq?uw)MypuC6AHQEcd#M6~#_*2F#97 zzr)agf^%!6?Y)s2Z1quNKm&|t@ibNvtP06Gr?FpKd?r1s?B{T=uu?|}8&aDL*lZ!p zB_IhQVKjH(zsDb#Ym)E#nr${wHkenkiR~7sBAN7HfoXoD@ov@)n(?UV4%>Av`HD4n z@s+|MX#KZ-1s=AZMo6EjiDSB)iUbkIeeiay6Su{Yg-O#(pGfU`QJ~Sd`$l+P4N+Be z{p1lJo&6DR=pJ%Y=vl%*{)TK8T(>W>iUftDjEJmXs%ct`nAxE;Vb;;@IFoT6PH5OM z(0~*923~Jc3%$*#x=Qx zyCkw!D??;J#Zi4tPtP(mu>Jv=<&4*?vBP9Xt5a*Dq;cJ;$|74*z%tBf5v>l><#mFI zZi$s>-j=b)pbad%HABr}ZkLiy;=a5A$T3olp&`wJ=9~DM8<3ZH!Wr@A2Fumo-Nz+C5EKjjP32cp#4SP z-P}@3mFFdD1OE|LF(_uLV@&6`;SsR4s7NUX=7uyzo-L@eDYhNDB86*0tT&Bq>2h~ zf;Q2cKja6}F z)RwAd&dlU+5ac6L5JiHZ`D}ycVOyGc5{I>AC{p~k^SH+AgRk#%_S-S!Tc(%hlylx| z!Vg^1&#jcq>Rp6e@@PEL&)F1jaZR0%|Hk(4-zbf9e~AZDUu(2X|DVUy*Dci9#NGM7 zv;ix%FKuAch&Ulgh%yw7fUSv&R0yl7NHdRe5NZ~YRJx+mif^kXCRHu#jayrY`&b+A%6@KT*dD@b~Gx@fQm9v4= zqbROsGwmlJX^_iscEU`DxLk%yrkKh_X<{aeOm~Gxu@OL5ej@US-WUcMqS|^+Sw^M7 zyMTE2OWD`hyiR2q7hHELwmEoroVA~}5{sa51mPY;6i;XauPQG~HceRwI8Jx9plp(2 z$>Ne&uEd&T1NYn%U0E6fu4q9{rvl~H@5M+Ij~qCYs{M}}CedkYrN%bv_QSMQN=Ozb)gBjK#YOU#5n+PimRb`*L6`6u z8q>Lu(KK^*ILnsuf7E3bZ}+HmL?-n^mKz#;O@biSV6uz#igbUG2PPlPQ7hJv0l37= zlW|Lou_}Zxn!@ws>0rWHBM{AU=-4`$`hrE`3^vQZlIsJn(xl0SUJ?sa5*&3pveIY zH%XB-Xx@Q#=UZ2Ow$STKOVD1!ul=O^JkV(v@uFDUGO6y09!9@s=!Q-GZI6XKaYvOM zdS}G6u&k|Jgg7i5lL0?7(ay-S9Vw^VAf~q}v6Q(wP)TNLj_??!Wu-Zjwg2?LCKX4M zvpbGxyqofo3|1pe(1i6L$+bRi6rsy66!>-j|1-SxHM}mZIu~$E4uL@N9;3MzE;;un#UIL%$)wa` z)Xr8|u@N%m=WX;cnZ;}2DQ1EbPKG)-gP42|vxT8c>V7$}AA}K>vOXZ}Qhsjd-gz;N z>O%(hq@4MOx3sj9D3*&0%Y9%TZXZeW^lkcyBs=_NF#d##XBLIDgd^CMtwl%b-4=OD zz)H8(XRimYx^WT^{cbWQfOiLX{S3b@n!?NLaDPE?1^VsNI^*!bx)E-AD$_L$;O(Zg zOE+O}K;IG$DLN`9jM!&F9rV1yDZcvP+J?oKAj?09k0ylOY@Pf!ya?^4$uIatyVHIp zhl~7A2eH$aI%{BNVs2n-Y;EGmAnIac<01J49)(<-%@rK&TrG@E9RGDR532Smp$H*f zN`cZqAtNd(qM|X0njngap~3{BYf_2TwYodhV{wSsxb_>P4^qE_`Sl0l-y0`E3iNzl zlXR*-%CZv){*o(_M(9Yp%zo1Sy)`=V@%pw;0V3PyOfj(A4$eWFTE9c?|Jc!oVwo(+ z%hoJR7{jZe5gu_d~p?2iGiBCowjV#PNE(A(B;TYIaeYZ0u4O2&tg;3@G zq-l6tx5)Td%ET`C(V-6gsjvdAEv;#4`VVCTsn>Jj=|i}0RV4T z3q!bzC_yNa#X)~86xv|0WyJb1Lz?{~#Z{zEJ`hj5ua}O<@MyH`BLtCYmdIO6z|tSc z{9k}Y8NQWmjT4uGFik=jObQ{QL+#?E<2>enIGC5=DqJ@wo8zaNv@JqR?~B|+&Ubvy zOi;V2`xIu$sFs4q@%ZPF)jMRQr*5ZTGzm$k&r$uo=;Dx1d;(G2`AWcBpuJZgOiTGz^{xaK?G2S9sA8{36R zkxh`>*P^44(FXOnJF%fWZw|;UAz44#s^aaKa5|}v?{hLWeV}il#f>QwBtDq_v^4M6L}(t(8uul%`A|67f@kL) z8g`l0nSr=$eQUlK`eoOoHEwI^$*>j_F~G{|bcYJVT& zz;y-e=jb!ED<*?A62As#{d>ZFPVj%>lf)r4YPg=MJ>0BgyWee5qv5+%-AME3C-v!R>@+*-e zFJa%_P=F#$XN(g%Q2YWl6CrD27tK!eWgdA!=Y@C0KI(w{`_=ijD+heK%7c9vsj$OFK(fmj`vsVNB2c>)M_doq_CcK6D->W%!Ds~LO2^8wS z5uip8C|w)OPQ&06C|-vKNDLlHm}H{74};AbG4ogNpcsC9?UhnLCr19PkOZR@DA;MB z<`0>UcxRw|&R=;%_c!6?w9XPoZ&KEJ@U7-m~$Q;G=Q#S zqOe1a!K@}bdKe7lFj67Cz;@cO*rddH!2;yenXVqdPg8Qwzeet!dBJl49>!GcR;ZF_ zr&%@wsQqok0&~VRu9nnA_MjlAL12VCxzMU z&Yrju`lBz?oVFB3xCpdH?<2#HTgf@--g9?7lg+qXS6<3sp1RVA2&DLTXbc9S%r9?= zy`VF;X$hgj3BTNzty>{@(`}9E?UKP&#D<=rlU)g#f+Fr28{xZ#S`~iED~b1=O4uzdh6)xklp8LLm8p((J(nt3gR8cj8P>=DmT2wMH%#N zHVrdMq*%yy9}$=CR}<-*t4_+9)y#e-`!$~21}0iWhTHb^$X~sy>JbxOL|ENUup2|Sfhed_LN($#L69-K_*m> zt(mPeKnC8*j8!FREbCw7-|_bQTbo|YX@MyF zX+9*7>34DsXdQq5sPRT-JqK#DcdG51y_Uuy0cAs@z?JPNaLN7 z#%poZC8k0r@zX4wFgTU2vR3lMH*g?j=7=jqC!#*#0n{QR+xJ-^eYJ>fx~%q?R$N() zxiKA~yDfkEFZ>xzfBcYsa*ghR)uzd*{g_K zqb$QyhdB!Yt(#qzoFUA7wy^}xwUNd5w#MDL>X|;c$Fv3SV z;SFFOkV(PAr%M&X210vOQH(L&Fkf^vf9$hVJ85pXD#PNuNN*3N@cuz^P^nYlJenMI z4Q;?Pzj<2N@<4xEI-Yg(Z$Q65wYu=vLh7|?#n~gi{b`7?b>*}X?RBQ@{AgHH^~BZ4 zyRMJ;fRP|KU<1Fm+1euC8Br3PYyJm_c=KL}bcee{+Par+KY`U)THJyST`Q2+`crh2BN|S z`oc7q=HA!dcBQg*r3d=vHgJ^h3**wum=AM=^0lK7)}+HJdAd9pm&jm!m*|;Dw2kO`TSK zo4Fo4#c+EA7a4wDV0~FLIBDtJkTNTNn1q$PjCAy-CsKxzw6+JDjzbY}Jv#_JX2=#} zp9ej5VzA)uM5<|ceIKr2%jQh;0os`%m!ZWRe@weewaVYW)f7K4Wv?D&Rl*y!oWKV= z127A2ZJX&wGYy-*lI5?XHrx#Mh!Wqd+b)O`dE0i2EMTkUqZMt)wk{HTTH(^TM0r&* z4c7_lA)EV)Sk~7bp<=apk&5L1%SDHcbt?wa_ix<9?5E>bKE{Q2i;`|gc zemj{}8G>cYuJFx<0@ElOx`QZs*d600Bw~QB)rY8TOvz(1qRvp|7pZE|ZRU1@S%Q&ywqHvNksfOvoM2W8-66Yo?v$z>4lm;IWuY2Sj)MkIuq(D!Gib8LMmw)~o!*mP=)BI{Oefb8(Xb#*%R%Z zq-l?I1ifskDx||3?ubL!!*qIh=FFN|7G*nRGd`2qai^L4nL;RrY;4p!+g1}i^@yFLdC~|RA)_3x+B8gs-N$3nciHcnTN56>_*A|a7 z9$H$R9nl&>?abSeW60?&+OhF>IiNOrEvznbW+}_U!cTB$)KC#v8X;^ zf`F>o2%1BaD1a_H23tbBQW*~`g&3xR#Y6uh59UgXvI(g!fcEAfI0z^O)O~NF&OWr| z%csV!{W++6p;(s@;bsr$?V#T3r$e)G7aFwbtl2}bg)yhOM?oXl5own|BiJt-sIobM7q7c%O@+;+c5ZAc~2Od6s8N?k~(JBxM{F<%v<;3F7^xFhA zKzKekXs<@3bd+a_x%RcWGXgR8S0FS0LjdPV6`VpFEKwhqC>>izJpQCasV}$P0aiJ9 ziO5=iD}!Rpv2?+2Le7X8!vzurD1&cJ&d`=+F|WAq!mN&ozKP=Ae^m(tWS@6)j(cS~ zdkeUFcFB5narKk`P^Oz_&MAL8i65L9{)BXf&3?+O`La&{tD32Ce$@&Z|5K(BH|8$7 z<-tnvC-pb6<1`OHN+}_6kJR;3A?PKSD0VWU#dTQb^+`mySfnO2`S{48MY&ahqN*dp z;Q{CZjCbIdR)swg01a_9#Zs1!eL}8Pq8NqmIM~2tZJAPAHfh#C7bEveAcj992P(Cz zAnh`awYOt}dWA89jlb$5mjdGsQ2e_;-FzmeGvlWd!&oV=L^iu%l~Cm{xOFe$=Xe2B zt7suTN`gVmE3i^0hr5N-Qgn(a-3b9nyw-Yih~|FiQ*0hp5T)heo9h z+~phmPbgMk!4CyCb(C`G>HPExuF4FeIEz`&SDP8LT>JH>kW+&Clpc`=cjmJn(+bLU zf$qnj_|hj5LA=oT`ujV$mZm+f?21lzaMD-P=MJgzpqK-`aEF*B6*99_6qJ^9`8nbY zxJtuCOQg0t)V6WI>)Gj6_b05Ja2PyqCadHb0)tMw=EV?@NH>;8s60mu*@&`;ChVhu zEl#Wph~A;P;AUv|RH5)xC!Zmk0hZv$CsRiyh`CI0=d4y*r`%HvVapoU8n`tbs@OGP zK9z*`ODZDsS1G+=?y_kVHrPQju@ zOPA(3XWO=I+qP}nwr$(CZQHhO+r6u9b#(pR5f#<-x}MjZGjrrOawv(q>2$&jT*p#c zIxnxO1@i3w)QZ=gb8Eh>;TWwogoPSC@)w=UC#`Ltot5NVpJlJT;C#QK^iqD062$0C z+E)1fvqsT9h(^0sZ#B`Y=1ngT`anKLtJK0qG|H#qG#vdZ@}xox=)w%o^b-u>kqfQ_BDhm12QC+Fr0p zRd1G2eQ&De8b+}Uk45AzR!+PETzVeTO`Od4`_LWC=D3c}sw8CVqN?8s+?mX8BAN%- z05dYw<7;7a9*wiyT#a!fPy?W1KlazN3c$Ve&{Co{OJK1gs0hrvR*e}CCdGB5X;LPV z7-Ojbi!mIIyEWDbby7DYC+>gb*v{KFaOmggMv^O-j{ZjYeHO7${liQ7CCpC`3 z3&VO<6Aj_kR%J| zxo<~J1JC-cuWiQ3WsiyC_*9()5XbJJq0E)kk846E zab8A}cIm^IRDjI1;ssgb0>5J^S{3LeSTu~(o(~YlykL?(5eEp`|%z~I)LAgS~ ztqb4zB;yj<&IA3;fcACSy{2bN62R_=y@$gb1Lza|K@2Upi{^Jk{_6q$6cvmD`K>?S zVE&Kl&;R)V|3`7|Qd#>Q2BQ9K!J>i&=*r^l3l|~7kF_qY6N?(fF{AhW@0btN@g(|;>Y8o8ezeacIAx)7Mb$V@jlH{04KTE4trKCZtWzO#1z zU=Fb0HR=^Y!)eCe@`Ea&mJOML8lb|=?FxY*;4SLcW7Fu92THh-1x~c~XQWuvh19f- zXQ&XCesW;>2S);G@lyw8rN-$7&iY%x8w=3JrRdW6@=2>DUqhyrUz}45IDp)+X6tEO z=kAp+6ld${rl6X;Lr&bfqm{FF5Tp77I18JYN@65)Yfj2sOtb1t!b)$Xn@g8Q_x+>* z2W&_3&C3*7k(87IDP_zhp&bA-rJt7-XDMJ(S){hpv(pOZJecb#&LA>*6`mr6Z^vF8 zD$$6oYi?oGPU&yoE71#btb{5uG;7H+Zj6%yETXpM7uf5>lj$+!=(0ApQ9(Mb$5Vsz zh3Dx_Dm1ClS05@FNWTa>O*zviMDN)-P^U1E)F5N-MbTHcH+W^2oIo4~ADua9*H`QZ zFU^nT#tQST0IEW6EUYYIlVVlhN+8S0UgX%oE4IpSgDsEb%FN9Os%CRbQ z3j#=Vv?YCC0xf0enNko?T#$RlCj-|Xih8^eX}+#@N;un(uyUZ53;h}AFkKDQ&#rXT z;-9~Pjm|7V7c0fD2py6T#iAzQqrGn*BYNY!RE_?uZmP_tp``N|2_StG^hj`2Ct8IE zcl<+!ZF0=mg&Un1cmXf!v6ULvpjdM{`-t;{XEKf_wtXjN%?*=qev94WUJxqFJ$gEV z_(NM7JGAejP_BRljj8V+Fg|CHf(3<-;#v^$8y=4v$WrhWhX&c#X7?TuZe~oPH(+SEU>=^ zlNrf2DdDO#Y~h-vN^$_zDYlq!hrFG92W_)GxelZl3gvrhnq zGw}+TLqFK!yie$?3}#OBhNZsTNlp+DaGdPq>2zeUC%zg4Tx^~ zVjd}<(dlftJD5JNTrh)HE#hRW!dkDxS`8z2R(c=%HfC`FTSdZqlxu@R z_c+rAgDX&3zufTrNRKxfu)MShZ*N)Rm?dVTKHKggqmq^}oA=Zoo~ZnQJ{$Ry7vjJYZJZP42A_qsy0p6xdYN5xn?@W9{T zgcUL$+xkgMU;2a%?H{@Pb*aZrI%pR&(qFu#th~o%;{fuf+n1*tB zalC^0$^Y>*4`2Gq;87-zrYk@eOAGKLjQuZWVY4u#bAEvOnvU;?U1+zHtN-xVl*(Be za_-G_4qR){aPy6<4~mjP1n4(5lu8R#Zm>ipJbcR>iYvTcCT>DDe7wtB$6OF8t~(tj z@!FO?3WNfSG_tat5hYX@wr}s0spa2!uElJgPT%&u>(v@iI?7ybxJbYb6xOe;n zDuubw#?WOPp;PGak#b2Z9HIQ2PISVG5Ytb#`#s8N0;%q-5i-byQ&3`B@t7&Qy(^=2 zy5F3-J~{g!OfsRwm*_d^cBFxP(5&22%K$!m>G+EdJ0O_@w0W;gMcf1{v1>B~M7lf@b6hQx5eWYr zaW^AJ|2gmYBYytvE7yTlm5I0Ji{TwG+t(CRP1o=38CabNlC1&ywjrrM724YQudmD) zQOf4jZ<%QGOTzK~|5rK0tbfNS|EUuH^T~^CQZS z6EXi~K{+36)5S=6wxX)R0@9eTT**4l!U3d#@|$<6*&q>gEY%1yzX-*z=5IogYYcv5^BSwl?_6%B+iT$ z3r+U-Zkosnuab#a8n=2QyPS=W{&g4Z1$~F14eg0|=LIu}iq3Yv4d$1L{J939xhD_v zfds5-dq@V7JeaSZ-IZ@3>+bfXie-C>z?}M`UGc=At1)4<+u&Hp9V2Pqhv1Vp<&Y2h zKlT3#!*`q&uNAwA$n2-ux>mww26 z-~fY%+p3@FEFOybQ znj<)py5#HB$+*>7WSQ=mOhI+P=eLioC(23BTB!hFBs~K{h^_aSmoDB$@CUXWjw1HN zdBg1$H?byRgDqytkjqSqF5ko%@xkGAoG(E7i3`P!f~@HzP2QQX?UL`CZJYP_w>FEc zl^F+1Gir>r0~c$WH42=t5rgB?3oPWVC#K;{B`zs z^rgIN`1{SV7slP){kv)E9|W0ZNl9Jkl;X~Dl&?V~YIg*q*2pb3SBdfUgYDUel!n+0 zmVy7C2Alh%=60+;D5a;n$!xc*mHg_WMI{eRa$Y5p7fz^~_E^#3hJ zr_{ha6^GD%Wa6hbEU`iNLiutdK&aV5@k0sWAjM*JMH#_nclmUjb>c%hK9;#;FUy2H zHkPxMJ=Ox0t&l7nL@8DQG(1|Go1SZ{x-=|q0`$*(j<2kpS!i1GMruCm86J+k9=vuw zdn}IcPI+)VL3_#ThC-@Z8{mJZsa~i3AchZoIMM^&INX6S6C)jjV_Wu)F_|BsFgQb9 z_jlY-AdH(jVKIAuak!)9pbUP^O0B_QILCUM5g=Zyp)#hghCtZ;csS*Ds?hy1sh|g+km~m4CNve@EV&^j!u8ScIwk0S!v0o@g4@5NGmqRLtw-xalD}-+=6n9XZ z#40`t^gqeLHWI)w68<-${v?c^Du~P}?-lQqtqS8OA0}9-YyCn5* zWtrbTD!Qqs40nGWQ#E8&q>m>Hc*`U-%nS znyt+R2J<|$Z4qJ_rQ!MgWf4xKUlauvShgj%9!(-CvPwH7sO@A0+uXADsAXU`gs3DR zCX6O)X0>SbgboW(;b`!pV-Q_fMzV>zLjUHU$>Va6qMY@K2c{yVbj%c-kH#1{HYPv^ z&#?1rqOMq)t^s2ao-E9h-JT5V?hGN1{<4oP2;2q}VVHq_h;{}JSsx?$|z zQUm_S@$;pdsx1#h^h9)^?5J8?ur-WbKxZi!f|1o`TrBQK)Yq8ln0-cJ8-JYNXA4*u zuiRQUAujrZ42$fO<#<24T?$v{pMrjONOY_{JeJ9C4?pKACSmQ;0VkaXmZn4!c4}34 z7o4U<1ok#Rg?xBPp5c)Ga1@88Ml~rj;$e`M=xOYSPo>}&k6ll!gM>of;)MP^oL;(j zgC1QU2=@xUq1%tXE7l$L!3WuYD0*i-eX&WJcPbt`fc$ zd7@V*TNDRink#LedQBFa!F^Kt5pv9{WzBxUVbS4Z`}9yTX+SR|3A0WC9A7%y=^x8f zDaF7pncv3G3+r_6CCj%?VjedK5=?pqWm>BFCbTa_(K~zl+`wY~0c!SA9hAyxC~omV z4r?{tY#=NVJ3p-xDr(|Qemg0WQTpv^zNRUAJgVUrO+RMFCE!3?EomcSLwv^Ql4fvy zpMZU)36PenP!O?adnqD&mMQz`gY8N65TzfK(5N*_NTed`8F@AljYKQGB~*3dg_Tg- zeEtlb+J*jaOk1v=x)>h^d|kn~`&xNH7+ZTPMR{TRgCm^|B`m8+G4#x10*=hQBAFH? z;!-=S(&DLp9IF&c#1Om|C6dy=rJ(ChI-?)~`;;&%xwF9U!mF8NS08q_kA|dVicbOO z%^YrwcM(j4l@9`C|56~4&j90l^@Ygyf^++WmJni;kC#&Rs}>NX#=HrqJI=k7DSE$k zimyYbd?UMu^h6;O@Mg1yj zYS;_LxxxtJIeazk(ugPh0BY*^aiDNk+X{m)YV{%Ev05pVt9${J48n^DqXxC?l5jAs zmdCwve~3^JcNDlhOViOcVl{yMB zq>cdCpAgGH|EyKuS1t;&+afsXI@o0w>9o*^JIdQyDdSp(-#}-j@s*Di3CGzzx5ETw zG=5Ctq8J0vTvoF+Hi3y#7*SXk9o6hH!df=-awNgj53Wa`ik#G)E?3XAX#~WyerKf3Qwtv-rmmM*4pCw>R|8S_~6j! zV0UqDgjoEI1?4ABY`q5@86-Qj7fHfn(7RVhX|86;@i0+aEDz3d@ZJt`upS;~Q<)Zr zcQ&%gdiaHXsN(;!xiO3{DxXwqB#kh6ZBR*&dPx9Kq|T(U&pJ8VkeFU|P+MJHk@#|L zb-Z>~6~v5q5+E}a;9-l0*K58b^3p{BWRbXENv)^R0HRC*x}p%7h6BCz*Q=(?g8)&3 zN_w$44R5J}p%^FuXyOv`aQkc4JMy`6Gj$BCV4k6UVZdBL7Q@0st@Ox9e7)PVhGe~PJ6&g$s zbdIGYI|LTmM=cEdV{|PRri(a4oTsY^(f>@ql95+qN;7bWwHA)Bff;RW+Q2{1E4bwl zNsAetI|(l>G0+MvgBwlE235uWkmDT!Wn2|`nj6xrdYF&kLfgvvPpl?aj3J#GRd4{F z3h;b@Ymc@RC{k%>XbPC%4Rx(~VL2}DxZCCSf4Xu1AXlMIKw$KXqU6MiNV*P>s3 zeZVH&-4ruwKD2!tW%-L5!`>=94C|y-Axh+*W{}`5An^NoA z)0=FEl7QBK<{bcZ1*nlIU_PNej?O#p4oE$DH|_?^RV-@@?dMD49JItkQWp$LmP<4K zhr3kmlsH=6MOD=tA}($!-f9jIOe^V45Y50DVGNZVyv0_kRb>l?5CxjQfYwntyGr0T z++0HE3|QC5)qhASr>(Bhq>BjAagN=08BmJdic=DucMMCX;fPuyW!)`qMGXc;M&2Ew zv6f*_QDjClD;s42TO!%UM>9xAxfnY-H%P$)TjZoVp?$6)M*(ec$C`}TKFA%vlh{xf zPKJ?iDGx15G!f1ZNNpz<{;kyi$(RMO4_%S;<$%O#ZFaKc+yJb_I)^Yp0_(!`q2Oy5?@t z6rvNMl!i~MJ%{&q-TUQ3_}2)IO=+Gog~j+e#_(J#PDIC9^-@dUhArw+2qx6D73x5_ z07$DD6wIDprobOU*{;-?XQ<4sqyiQ_wrzj+&ZNE{=gb(FT%Z<)k4?XngRa6>1eXRL zUk}<~k1>{=tSN+FrsN)BzHOeHfT8xia+L@VJpsHEKY?GSaov86b{mQ z)y16JF|}}QQ7_WR&F0hZ!5GCHSWV-RleLX~bkZTBP=>==VP2Ab+;p;KC_+2GL z*>WdU(}T9u+yvZ@r~~Stk=bj?m=4{vLa%%Pel^B(;37SrMT6ZG+-|t>V%1xaLkw>!j)+>7637Te1gE zv6XkFXHf;F8$O?N>vD}d!4s=*SRRt~=4ru8^DjM z^b(Hqfd#Ab6amliVgl@Ouc!q_8L1ICv?SbN%AcW0KO>auGIhw`iA)zPfEQ-yS7^dB zH>eRkCW7C7cCRG*w<_uHH?Qo``*xoJn9*bfIx?y)v^CveHpDl|=~q;YJ;SS-C6hYS zCq`#2HAZmu+t+*=?^IH`n19V5l{=-{1``r#wE81#5D3rU9ngMmWThJu)%Kj{-RiCP zXLnMIdI%(ZK+cLX77IREmF5*?yA^p-Y;tdo21ZtiUlj!fpUk)H}KUvcUU@BDA7s>(yx^`bvk*Y`g-gqeGPbg^JdBUi)---zA2)9PWcH>kDY@Xazs^O)LfeLH-oaFSmIze zro|Y}jFtxSmWof0D!_N9}+?5R=$v7yro6%uSIn%>7*y1JTsPn zv<%M+Lp-Ra&{AiCU`m_l$WKfOqC|^~>kGfKL}Scu!P@BPjSV#^%#e7%Z@I_MI#DBw zFq2@9IFo~SuD-ONNuIXdXBQYKodgf|Lfe2-8K;2g&pLnrnjfT}-w(RjY+7p1x4gY) zb~Dd%3^$s2;W)9kP-}%8E<6BrY3!9C;v-JU2J6!%(zA;}F{JM5kEk9^ri$z49Ph&2 zLF;SizuE}mkmOc6gcSK*c8)*iTxkV=aX>ysEN@lU$dj$MU%{!hD@`qBInI=nJ|i@` ze*|x|Cb!^^CyA80hpi@P!Ub-VjzZCS+E9O~uHUa55q+{CPR0r_LOW>EKjHTh-bN&J)O8nvO)$z#ll@BTD{sqv;SK$&119|O_Vs89l) zesW>ks1}3WG@~toHR|Eo0KA-)dbQ$~sd^GJoSH+%>7P2u5mG%}B_eLbLcu7)=kW^v zc7SlbPaiYq;ACQqQ&b=~htR|2@T$etv9;IZb>P zq@-Xtm>qo#{oRL4gyaVR!!fwkC``W0#@04Ak1P9j#-ps()toFVt}XM2%Jgt3-!}C2 zplN@zsJs!iYZW19-9qERFc+Nj#=m%+69>vGb-DF|^t7%kSHN4PuaKfJr#czf{j~Lc zMY5BAUhHM$uaB0@3>_wjz%M`s$DdU%X6;J^qVH7q?!@T%42!4E*i6tFT^ur-K!`7n z`m*#D#5iWn82=fh`5mtO8La&oJ@6W^`5nf$eg^4#M=q_@?+R5iMr8{ufb?QA)-^8} z{{RC!$L3bXKI`RM!uWj!fbMt3ysMT8p}Pv%S31dp6q%p%^(<9tqGs|4-_e`>rdqBs2!ZM4r(Gv{-Zj7~6m*VdxIOSp5 z9i0M#o!5drzT3RsMnyuw{luZ8$6nogvpt}SRExaar;Du~g2sWk%iT}mbW6Z~DdM<8 zla7DL_=WJNB4GM_Zj7HcEtq>b5Rt()SctA)eYc)@Q(x=V|7rxL5`(7!JBM?D=JS}@ zEdzDfE&m4Dde_N3mR%Fq3ITo6bVKL0GaC`*OT5!B!JFz-f$xV;n)d_bzp7zfx_*gq zzZ*wEe@C?cOTOa&QNyI1EFI1MXDN*EcZagFo~4t~f3))=1u2DJOORKB)F?;_ zkdWBF`fyjd!FpLza#D~|*IP!*;{M(GjP=R<%ugzBMYmi0? zJ(G5wm(FW%hL3}jmYYAuwj3dd#0{eQ;xsD@weziU!&1NpZL3D8nrr(UoA#|54azLd zfmPS^hAW0sN94CaMO7(aVpp~#pk3f@Tn_U|=Gl7rvu>RTzdU+8gm%*OOO2=sCs;E} zV9zuQW*)|^<+WlfO%qOlSQ8lrWLo`dm0-}6(Qjg_{{RLQU)%>pwoZd<)<#DEdLl>T zs;1pYh1y2vb%ePt#D(21X&ZW}doD7UPe#QZG0s%1O-+Y}&9XK-_7mor5dM?}8bn;x z$xRuU(y~7cO86F`3{rUBEZTU4$a1VQ$5Vew_-CA|<(V2xSzf8TsI{vf{;{=Q@Mb|u zb1B7HTNI=5=nGH{wfLT**oY^`N&QBlq-cs>sYLAFn&zAI31|}as%OfIovfzzQ)hdz zk2r$)lT(L<_U?wh+qopqGnu5ATmfzMUap;>uD z=lXmqbwTb^##fFrt?-X2_@jDkZ*F$XCFR#B*x~36*KUTMHjZ@cmdq%{I@J1ygifz` z!iQ*L@w46~oT0AoGG9KhM--t)0HsF+!X=h8JU22{!7CYVV}AYmJP^WYi&0c#y?qsB zLVbR-G=ej*$w4u8Y>9(>(Tu34Wz&2C-!nI*jWC$qnL7HBbq1aH+meD!PsRS3f3Izk40oM@I-{n2G;J$A z@wXOeCVDx6JEE^t0X+JHAZW^#%<%$xOnJh=O}Dk^Op$`AH-Q^xc`!u@aE1J4?+*Tmo;$S?AVFmsDxg%T*bxm zkVeqZe)z%r*j}X7>1Tz*vxnO@YDapTq3NRLbG9a&am*~JV{l8B)0M7O|*4FO-p;&(^P;+O{aESmPB}YWpU#m4fQe5Wm}>0hT<$2 zc2yKy^tnVHOMa=t1vY|VkhzW8T>k~ z=uVqyg>0AOl-Ne9^fGJ4NG(W`gi143kx)`0w9+7!dHMl<_+!D#;{q#s;B}##b7(1& zqkL6XKIdQ|U?*E)2r}oxU5pFNEr9c4>k|)L_`f{~R8#;Kpk-82wnY7LkFg&sgT#kbP z;DK{bb>n5h>4BQ?!1TraW921GVzoK0yC`$>)Rx&&AlmxcdHEWQz3FSq2f}S5>IP1I zV(^9$MS>2}V*KU(^Y4H_p7~mShloHF!q2d3Pm^dceihV;EvF(ybG#YmeYP^uGROsSnj216SMqz@rlKi5f-)WQTExCB z_e78~i+5zvqI}@unKAGxp~ebCW%)DBAJ*tY2ijF_MRty?G`(N8Bj19@>huO za$St+lb9Z|(?Mb&jn@K<3{pJAPb?Zb%E@d214$gy!x4j^F1ZrVqDr}2n9l_A{wySS&;1tKkQw|=T)Kw#cWA7gv z&W>b~#zcj+JtbZq8a5xoL1eB{`VEN88A;@p&@0NpDTvC`O8(W)W9wED#7kVAzDUd| zN}`qWdPe6v(eDORt(gM&{nciBa>zqL3@t}nI>q{soUt>o`PbM!aW&H zwK+bww2Q64dDH=$G9v}ir@{AKq;FnVon&Xh6J5ZjwK?u-ZRhsrEUurJ%t`H&#m%gy zgjICjZ6hdQE4Avd`%!k#Ou{Qc^2up1SPsH}_Sd%Y-H|=a|Ql8ZsH8PzeY+ zeVQwd*9fzZ$4Z46imQhoC74oxVGbJ55$Qz*x7+tu0`J@VTh}R79K5OMLPo$-Ri&q? zkk|m84o~Rijx`tA?6NlZ5_(XwrS;{{*#|<+$jK50Jr|^2bgt<1jT&xsp;UEqm~`gG za!kCe8cWRhan5SjUz`VW8K}d}?(n!DV3kT*;BRWye9crT;`4~%)ibUAgAGzs7iu%O zEbE7BY_mrOJyMgDjKrvn2AwUEX(hyUOphIMD`2M|I8DFB7}KDRK3A34`MNI~LgakJ z3tu!(v^(G=Vg}6~5?REx<-W=KV@mr+u>Ugm$jNr9RR(V6TGWx+AQrZ8lHgsvvE`az zC@4*L8@wk6FDa%thmDCi-HliITE-!l%8C1n*HrU?sd7l|G-wDO(~ij+32qhLwW?uR zOO0AJdtA9m#%HVZ#M8da`W0n@D4!)2Ce8vSf;f)3s^*xWOp;i6z^ezQ>vW{=lgx|$ zbxJsWtp=;|1B0U6AV_|GKw<{Fetxh1nA_G8)7Jwpgnv{QPZLw3L-d_WR}B7g+=1zt@sm(>0X zsXe6&(h4L5Ys0mm7%2obaI=_a&yqW+DN&uocQjohExhl=L;Sd9J+w>Ygf6PkLu?SH z>lmYPmSE_R3ex8y>7WKjmfq#(PfkjS(XWP;r3)X|sPz|Al+va1iPGQ3D&_P)$KvDB ziNRwHM@aufJ_bF$K^^eF4=PYaq%0gEMK&RQHgKbsgCZdxC)#JOysP)4(4qG;|AuFt zwATH$qqK^DHsDl!MO5~%qawUVAdwx?@t4?xJ!$iQu>GtTP@S$tv3~=Pj=|!iQbREZ zH}ut~@TCxua{+yXTOkBF63C6I*04nnK~DwM0ZT!m91B(e1AZ-^3vuut;v4dR#>v8U zykqT`l9x^Am6Amt67v|NqG04310N)n-i~Qwwuh{sQe0lI6e_tK$b7$N+AI@bZTfSv z$a1@mF=F;~^w->`L}T@f^ore|!6lJ}DKhV^4uG@3bE*4{H6R^gX1t;Y#nYh@ z7Bbotv3fEpkKS4YBx!HUdN{ zm{;M&?hGNM)UVm^8r%~h@c4T}Pzqah$)+hS3%AxjG0%sb6x*Y6Oa2r@uOuF;RRk!d zOtw^C^)z0&Od(bwG(_mw0a-UIe7=dNu0&Hp5!6SH!MsOLHY2)BD$` z!#5GFWT1L+Jskt)q=YP|tVO3)z!DnmO~7p(VXuu)WlbmJu89Dd(<{a7Kq2RVB=jk4 z?X>eFgD}GYEf8KGj2PAzNYR~Ez5~8xKKJnV9K}Sg$2DRj*#xT@&{8Z-O^@cw8tJj+ z3+KOf($dPM7oopgN$W3H!uEeAF8&YpOx?{>amDF7nsGkCI9m-6RM3*Zk~bib(Q=qs z7n2gtgBR4_5EC-&3zcgybgpx6D#0p~**a^hs^R8iKDilBI*bZ0ZJ2m!e^9KUMXa63 zEVEc_VR0*CndxoXqhK!5I97Pk2l4B;yCc)#dhMF+WE)3@=jRpS53_hP_Q#znDhO?P z7oF9#8AJ7yAZpBkXKEME@?J^Piy)xPK0g*W!f?)RgPtdJ4$Ey2ZFWCO(_6e>$$)8Z z3ep}$`)3LJ*Ai`aaLvUn2An6tZ>aIcD}&an!`40roF~f8LQoIr0nMEX@VENhkFZI1 zO)AgqaR~T(3$VB0E*bz2G(q?r`)8K75B%`XLCN}u8Y`+E3?1P zaL?(k4%BCc_s>?Z59y%K`EF0*k&f_eMp7TFPXiU`BA{;e|9fo0JQ7 z>mCat>DB7q$dZNTjkNlDH4ci>>ou2(+`JJx5QDYFx+#G=Y!lmA=N7nmYW~ipkUDzh z{Z=fF?9LFW{`<;A22_k@`f4uP_*#P&C-v>W9DA(*AeI)r&MHZ^y#0HFp%?3BHZfk} zwauZGOuMXUs`Fu6d&FE9rsLO+!3@b2Tt}oKuCwA%tNus~1@L-$(0buid>2JWsJkt^ zfbgoei{tA-|G?D8FVRVHl|e-h-@6ZW*YAZj@7?R^E3c3&43O7cY=7L;X6&l?KA0)E zV&#`>XaWu`b`8myEj^TRc}U*FVU->LoR^&wl*!$C;MHBo_yN&tS%%xebHln$v6)nx zSMK)DsITYF3Qyf1Jc27dDpx~OB!`~W(fYMiI2*W7iz^>>D`y-M&ibs17|2V{B3&ID zxa}D%!I&rY_bOpdFxYwJau$=s{xxkRoE5eRz{Y474;P0uKj-2uofp7^(CU|v(Y#Gs3BPiXwHNf+l1E;0up5$mG8|*MWO)SqX)gSe>tnC3 z7fp1v$Xml?iGnaTgwLmBpQLNYhZ^D0Y%!;+iSQmcTTu?YxwasZ;x&mb<|ec+5#Pfa zK_if^I#-vKB|c%Jf-P0#A2K{_LZzPyFcR&+@Xzta=Lw#ro||^g7O49pJlXJ z&fRh%a#(u3a=yP7ji{GISkclTmTMWARsD3L^BFCn@h`EWJ1)FtIBKj=oj?_=X)_LG zUqGf|ltH#}L@eN(YTwYgnwZt@89|mcKjhzZuRakNWl($>Z>-RMq^w}AaJFBcapFde z7eu6yRzRW3{-wD^A20c~W@^+Tq=RZYP2xe#jDj3>Xd3f~TECL0y=F4*&8)a2A&J?a zMXVj1f??k~sBx>@xAaCOR^ujqGF4sUDDW3R=TS)1!}!d8=Yz& zGV2*~+WuI{Q9@R;*x+9IAtPmHHm=8(U*BaZA2rlBo|rPB)!=R+mPmJGU6*(%rvOi{ z3krsnz&DJh5Pgt!s8*ywyTwl|umD>SpPhtITxhr&+X#(`fV0x@M%=kJIN(~KyI_R* zM}WRKbMY0eijjD#aGHKB<(igAAv0{=NrXO|OLjMTPTT-RLBQ-YXACh&iJ)&m`P@05 zUWd4{a^(h*_H8~qVjAt<{EfWMz}g5fi?@NTqWEwZln7E$u^CeomQ)#V>`Gbb-$>xF zlxm2Ogu~Uk{n^VrvpDO(fb@C_tE_b)uJvTM0C(6lfq*v_ms`{ZqXn44HF0T>qDr5U ztrgoqvRr2I3(??|Qsq`PLne#p%;Yp&?xZreOj5u3&R6mUk<)?eJL*O8s${+NRiSD) zFTRz-ADCuB0LnXhW|rlWf}n&Z$v!A=xBjtAa{mh#XPM-;jDm$-lN6;S_bnr&oyac3 zwq5F79yhz@(ti&mV=u$tF@~MD(DkVpvXKDKymOC`E}Na2M3j@dFZ&;nV^UA_sSq4g zMl()$$A*Y$D4K!Lt7s!niaaedmx&Ht8$gB=BQf$~{uCm1>aH7*;T4BQy5b!T4Qg*p zN(X*ASVx$qQe^ozvmloSWWe~@qEY&Qk%dN8W2a6L@cTaAg@iC%D+dt$84%eD9Y|R$5Wli*G98>(c zUHOAa6*f=7oKx<8Q2rmrcU9Cm_vf4>rwze04H|0^mG`af_$!8z+1?U}f!9r%$H(ux zl^-yF<8pI4%tGe9`OU+U-IdNFwZ{j>9PxiM;?bOvO-|~k%cqebT`}oDO{ba~EO*Hr z@kd0&r1J-h<8b%B?d3mrLd(A|$m)HH3I-?gQy3LUFIGr8S4Y@Fhn=JYgkBB>0Ov5m z?G0t1L!M_ka?EdnOu2W3o>D*-yoA1 zY@L)P`i4zABO%J!?~F}I2qusS#73FgW!$^yi4bhOabCOC6*(LU_{@B&Ue z8TogScNGcz+<~9_xV=2|bVj{j{N5>`ls!I(Q19-0FK!dhJpruQIx_bxfe79_M?N*X z>$LNj9GGA1Jxia=P1yoU6_NSs8L}1|q~_`?QTfl5C>3#f9hgm)FSvo!iv}pic_0zB zC=;_037TfmpnXEiCM{zolAqc2ia;@53_JyuK$6k6*VvmG24@(hd)SB_R+z#z78V{C z2Bm5)&`t%VGcf&C7sRl66d4As`j`*Yy50BRsO35Qg-u?N5RRia{7J{0=+Hp7C$*-C z5ha1@0&I;ekMJP-`pw35t)!=(Eo?T^sptu##;@QT!BajyB={pE;GfkWWW>R0d znk15T6q+l*)`C?Imd+}C!??CrzFUG2eX9boPQt_9T;u~vn&@rgv$}GUiI835-G;2@s%UowUug!$v_#hIbt(uw8rE?1 z{35lU&GWx^8344dEN+Mqi@7{?ClImM-7^f1!8!$9_oIi*i!3UFj#PZ$Q><8M(+TV~*293oM3N*=u8 z)PLZ=E`JFl`n(?hnuCg0ybTl7Z*PB_y^~jF@4f@)d6!l9d5hkTTWfn-L61(58>1Ua z<~=W0o~kGA;C-0gfW}#=%rDr7z}N(t`TZNBM*+vrnB9Uw*VMl+2CrB<1ld%zFUX56 znueVf%(orD-ZQ;spc&@0-Mkd88ajRz7*GNtArauZpuRX9aYw>b?9?yRt1mo!S?+6TWDm!?8jnM};1f7Osyn=2oY5{jq*4%kF4F|m*oBUX}6q*QEVd_Y&&pOY;yGrcoa z=&%+P5a$PbsmSxTqF~Y+Qq*^6WGp3wWQqr`$YUx?Evu4B3rlR1X8}fPIB!H|q#`ij zl4xo*rZ!N;q?D}@bmWRFX^{|vX3QvNPo)j|=hU&L*p=P4R4$_U_aE7%DAaSgaLPPJ zL0r8vSLNU`MKJMpLx51b(W()8mUr@$v)9-x`_Lsx1EyEt=x)r9P?>q zZDihBbbC5Bd@zKTlu0VGpfhx-0HnhVu9%0ll0Mn`LerSK?ufqnY*JpbiGuQahdP#e z&KWfihtDBm>@qwBgwu&pyweFggY5nU(K^QIv|PF}DV2 zF}rf%Drh`?GGs7DyQLF_eeI#fvY->MXxIf9ca3H`m;sDmm_56%MUf+SfsrIa4~W^u zr>rfNyRO2+HMbUZXAPJ$W3N!&78Y8}nP}8rQhpP(SBjHHpU^WCu77F3DYW7LV3B}i zB3eWYEtbMtFz1)?g?H6Kj@o*EWBXa9ujSZD`&vix@^Brrqn#=(x$=eyE{3w6PtbeE zs2G0oF_Ldn9mKpMjN};oUyQv|aAyIx?c1?!chIqI+qRQ`)Uj>bwr$(CZQJPR=BsmW z?Q{0Sx%++9dRn#Ss4>U@tz8B{fud@e|*^$Ch<3qkV zG2cPXh%D#+)|nHB1&;^|0o6F2Xw;FlG3*(^Vz+x z5H*Zz#*wHXKNA+MSQZ?u`6@Fd8Qotx`Q4f`RS6)yz-<)^{c1-%G>Uw?+ z33qE=QI(7(-;N2fRuhbs(0*gBo7v1#{uk{eHkhi3jZTe9qjNYt#V#I5Px!{ z8qP!?6ZGH%w1zB(Wk{0JY3HDc+NOIPV_fza>ys&aY*Y_oFS0hQ?=`4G{ai!bHf}2_ z?K$0m{drGXv0e}a;}mWs7I+0#P=P7tbbu`lJLpS#5MEPAN^7Rn=FzqN3QcQs^QO+u zDF^5bIs2H~MEHE{2foGpZ4|$A?@eCNP}Js(*e&P!!o^E+QXxZC-Kd@;>QVvu)g1<0 zL8ZdIl(!A`3Cq6^UOt+fZiqEqx0W1&{z%cX<6qj|>*e_&#%|ar?{D-U#HnY(FS2YK z9z@MKur7IN>q+EA1Gs(>TVcq~7{v`83Io@DlFoFTs|t<{?{#TBzmo8T{CPqNpAnsF zL+s!@Ks(k7R>kfiR*lREb+{BFK~3_|QUv-Dfm;0^-)H@%TwzrG3NpUdZ8( z38ltzdC5Fc?$5BCYYm5OC)78Uhd4fzBVQJ$Zzw7~o<#I-M+bAtrg|E_#pqFlwMs%^~ z7J^so)p35tRdNkkh1=L(Ys1fv1Zi%+8PwBl=49ffIAo#NV><7`3mv zuLzRt+fO$8Hu&HR=^{-zL;5UGTxS~ELdxd9i`q| z170*`sfOQ6Pso3*r%kEcj6hVLK-4V~g~Fh%ia3YwD7YB#49eAYfZi#DWYkV6J@WmZ ziY(1)pVIFqwwCcjwqyODDYE|@aw`9)hFHSd#O8l#uPW8|{}iBnSvQOopZ!pa{q9x8 ztQL}QDWSnYwPFRO`UIOq0)}yPBtB;Kd09|uWksuL2I#3K-9x|`Tj{M1cBJQO2R`N z-8SOFkL}t1A_WJAG?EWNlNl2!lDbDMe1b>l!`q{O~$7C4J3Nq>O` z4^D|OtRzj=+CBaVEWs6{7@D}rg{$fa6yD64Yc@WtmaGT-)$uYLDp9sTpI5?Wh4=-) zts&TA2&FdIYnVCH2=+M(oVpCR zl+&25a5WdM1HBpSfAy)RAuFk$xnVnlWWZS`2m_PAr}wPTWBs;HO=!m6xxn3PV=^b? z2rDE6khm5Lw!z|1uKQGv)i5(m;+BYKwUuRJWg1RssF)61#axnQ&kxtSI_r5Mz*r}n z!wWHvYnQD541|r6V;nBGcPt$Kjp>j>cPeeQznnTylwI$4rZyQ-iGJaVBgctz+LTD$ zvm8d4Ze4ry@62#kzit+CRu3V716E1X&CKT_WW8JFx--6CwGS(3cE6zU_Gw=Rd9mL;3!Ybmc!D!|1k<^yGO|=Rbb6mZ~*B4p4rU$#n$w^F$#l zP9>kf|6sMGtL>rFdvo;r{B(`vrc@Xwu>YemXAc7KBB_~MbHzK-5oqtP*ylXHB$aOy z=hT4x4uQB)|DM_}v}TUQc^m`hx0^$kbdTBP9EAQn=i}E1Xv^xWNTJ0+6?vo+XmcC>eT3@0i-9&?*PVZxUNjV{3lmu+Bw9D{N$D4?>7o{Z;879jJjh^gL8MU zfM=Lct8YSUs8Z`7nckkMXVlJJl;pQPn}NpurlHu{gd=TV%_Rkmbm=8%ghGW{Xisgb z$Q8saQ`?$|2Y&RUU2lw+ zlLKB1(|h!j-S!Bk3n^{SIpgx=a&R{=me=UW_iPn+Gzj`=5>i|A6=lieHlRW!#R2Gz z7~NnplsSX~i-=Yk{PC~h*&s{5>rUa&?eE${c)8MWxBLILMV>&sC0PObnPxwIIlTX~ zq4vM~3ID0hF)+9O|BQtua#IO!ZMO#i2Xj+)W_%5 ztX_3})&c&)a50H<7^zJnps6o}Txq<0-2Al1dE;gH!lON38FXy{fP=pH+ z<2r@j0bqP6Y8Nw-opJe$(<}v5Y(Fv@t9GCv)Mw+_0`UuWOnFOV)JgDV&~S)yE)p(Q zMg98KvVRLw`-$+Iv`+u(%ODuAN|ruFuMqm0z!OGSmgLuhdlil(p%3K2Lzp?Y8&u&9 zHx`Ac_*U2bgH$(wb(O@lj})cgjxz3YiKp_6e%rPb37xl$k(=tV^K2*)`=vPivgnr* zPyW}=HSiaG9En)ec8VsDLdx%CrAkWQDSK(eb^P+d$6~&xTW#ebsk*Vo8*9d*sQuR0 z`=3>v!$*zK_xax_b?|qYdh=^zeklldOTjnIwl5oP_pO=jg}`xg!-h!QdtkaRXIoIp zh-B*6D&>uBbUp}tJ_PP@_zZww6|D8`NDsRtwM~7=0OLgcp-;fAOE;0jxGFk8*M^LX z_hjuDg{yUFSrvQALyAA2<^bN*F@0c+9{=@m@!(eR5bb$uz)Kg;x_5r)@PlqNlLQE! zF^Am+rIcCX8rUM;1nqB_r!D;dse>>m!qcOEG|&gl|4(Vae?|EJgZTa56a4@Ev(bR@ z)CMGd=Qz%DWo>rPV^Dx8of6N;?(k!86iY}h6ibNDi%Ax3PHvL4WK9OTHG+E1QikV_ z4nUwHAvHmkqS8y?wN)J30Y0G+b6xfG)l}Tue93P{B zxCMdZd(4CUJ{{oO%UPQ2LEY24v*G=K{I1>`(fufdY`7!f{Xngu=>J~4Z4dswH{kUy z-h-q2+6Y1Z&>r}Ht%dx)82Wy@h56`7|0q)2Q?q#d`{cS1(sJvJydMK%}pHNgU?WI#Kk%q8Q zsEwpts!*g>z9_|_x5&ym5l(bB!7b7?PgT)buNhZ$H?AqO{8p}3^ocME-qfs`;8Jmf(^Wr4m4W7#rPgewcoRSYcRp7-rYcj7CSBFkuj*Nln_tO+ zeD#C_BAPq%_=43ZZrrB;dw=a@P!FqTU&QEf}fS4tH z+2AQS%9Tj+5BNjl3s+T;{m!H=)3!A@(Om97eNnJ?hrb1#^X7x_OG5`H-@aFf4<^BV ztuTyjR5Kzm^!=puhS+R?-~%}Hq>e>^f!ySd$>2LGAa@2DY$<{thJ77MRo0Qu{|th7 zOJ*-yT#VZy$U}vifEga=D;DYciOYdxK3>>==`FC~VC-rdyU3CnKP(*{#D^D_6V;2k z%{^Gt>M?yI(w4q((>ZB^)=HKw=;PC56EZ%>2ZxNa9z9;gU^22CH9M|ZQW1cl9z zzkgAi`UZ^JJJ+OHfW*F$n@5u~kzeKp>vZ;clriW=nA6m%&Z6pZQMV>n7qK-c{1r2} zrDLs~9S%@plT-q!O!MFRfF{WV6Th0fV_y!$7NWgWQZBHWPyB}C7qco25*S>2!3LvV zSi}sfksT-y_AZe95Vf&g&^=cTS}rCOkIPvlM^sAK=BBn#mCE(BzL;bmvYq+lP2!d4-ek&D`@#fwK4)T@8E?!U%d2bsxK>C6lCc=keu@3s$rFe2I!J z)6?`uBxT61=%cefZz}788$TEJ=wZlQ^p!f?=e4n&SWHNbNcaYtIlOv9&?$`j=8-`p z8++9Wh^0$Qr*bS5LhCa5RjWW`(gL?+5Qwl=w0kx!h9hyF*wAwSc~MP8>4t<+3T!Z* zO!Z4$BW1362F!x1u>p*_m*ZR)gH3bN&2mk60GQM;xvGaj*qXRa6U;G#?0Wrk(J2_$ zW@eI*ZZLg6j?po21uDL3Ce|^7{RP`FdQ&D=`x2v2AG}9YImR+_-zMy}3@m5B;Ky;( zANkI6^oFOr)d7i+s)?^dLv$A*C$ssnXEodR!m0R@%DQ2PC)^41$;g{9;e_4u!Iktd z)0b<;O*9du~1NX%U$(CG|CDj}d9H$HZEoIAWDm&9Ymxt8O7xk#@L<^nkmNXHu4 z-_|O-Bw>;$>nFTYlo>?k0F*`C=q@oGoXnoJ_z!Y`yvTjFUXAfvicWP0Rmb0y7A!m+ zcljuEv+@|+sZGRs>8R_TS7*Cs^X=zt2?go63wFZ1rDGxLm5yyXypa1!#bSZ(IrlAsh*kgjCY!JsYa$~#IucZ-v56Z;77i@&yy7hd?^`KV`1jBQWjUBpyz0mG#CIvpkT!;DT5s+M%1tq)o~3AJ<;eV5lge^v@k2r;d+tc1$+P^bQZe7#JO_zNrE#WiHnDiZl){cs zRb?F~ETx#+;6O)**#eewcHszzQFwF-puPqWmW>*_@q`YhElJ9M2P6i$Uq(B+>6V?Z z8Ne$-=$RLmEP@kSy!&^Gdrrxo<#OULM}|~`lJ^K*9ysojt4FC?Ic+jK z^v}Y>l0xqGC$Dx#?>Wnqh^U@Zm6E7PTM6{ki>K+V^S9^B41ZgLGueA-c&8}fY;oALc%H+b_VCeu)mNIJ*=0x`EUcL@6`JG+YfwlJe!|H)~Cno*1!5y@& zYU9uJ++WYwa=gte3s5L`Ku~MJ{M=mHthaC32>)oft7L`Y3))=56Hr_JS)%G`XT-s71#M#Pp)#Dltb}}JzGd9`YvgRp}f|Zs})iOSE(KEJDZc>r>otl7VTb4nblGqH=y>$sS zr?jZvmNpU5e;ikAOKlwpQtVZVH9^&L)d4P4S5S^3+*jDGc67EuBT5akrx7@6?QuST zL-bOZ_Ck`-hEH8*_=A)y*nuvb@s51cVPG09q@vm(TVD%UZ%C=NW!U2V2wO=<5QLvc`Tuanfl?zpjsZ)%(UJgT<6{K9}}%6VV| z_b>LmN**~1V9V}KeLg|ba2|uhA%svh19?LQ8Fz8g0Z&o3g z*vpQjkt0RbUS54&K#zJ9ol>7{V;W})aJ8R7VXl8s9V8dm@r2g&4#BkT63@yu%&~N^ z9(8+)>tZbNy}Q}RbSv8*mtA7P(oWA`3`*fEvA2GtggJo_j7xR%4p{j^&4iDQs+|Bq z)dELeoMH^D)sEH3XXW*C=-iEpAPf$Um(wC!>>d+Qu7V+!_ps81A#lHSO# zlx0`@raEU+5`tuD=Ohax^gG{jUL(u$x7TH2r5P~{YM#MDp*{N&TxqjK_Cas*axzF1 z*9Kn^ zI_F+|k(!_tKkT$tG>z`e>g34Wq@7V0gcqg5+=B!gZ7!uW`FuoPb%t+){8Ev;v zJ*az}j!E%FE5XGR-3@$3X!8u~% zVW11-g0pH=uM4^e&63b;2z}S^fhYA{q`s((fy5rgBnhN6P1^ZnjQ+C5WVJ*cIqjUR zLGLgFh_WD5p3g5^DBSg5h|>MJx-8+E6(W@p{2_8h1y; zt}FDCe?7y6h7i$Hq#bzS_rj^ik#`WQ3w!^R)HePAc#&@wHXBBoPedmV9k_Nh>0?y& zBApWpP%7$RGHM| zCZ5vlj3Lgcw_O*q+5LR|)m4kRGi1dsnUi<`z#V`XGhopj3>lpeS|htN@C@|c4j!R+ zDtJ85<@V*W!|-}`*G#{6pxD8EBRGf@`nH?z$C=FQu~hkLPTl^giP z<}KS(!sdnR49X3jkq?DV2d*3p7|Z35bt2dwzMM_edUc{aiJ7aMaF&Fz>e6=WtQ;=r z&+9CzFLy^p(@}9Qojr`qIzLCMWcT~SlC4mANqGlrw3Jq9mq$cS!95`I5M981HgMF8 z@RxghKyjZVh(!t0p2F|1+`K$?c`dm2y9)|jske)K2g|DYH>Y_6TEko9LMD_Vt^kA5 zE;4pWc(9(tLQP)5@*G3PX*y>+0OEwpeDUHeFu-xZfOD5zreR*mu{CK+N`MYASUh`!+cC|T=AMVm$YSH>Xp$+8NMAj=%h4s#CaBK zgDXae*%J^FO!oSnuw72Q1DV~gmSAhQnGHs%LKI`DqXys{W9IPV(BNPQd>UvyKVwL> zM_B4&15u(}2v;K|R;6f@6qy4?WmgEK&~@#UJo1=kh*NCHeyj|x0i0bb`K(I}&wVxq z4%^5fD^>Ht&q#dfIzY1sc64{lM+dx@zjYXZHnC(=MlW6tBgdFRX5dcNnVUu$I;=32 z;tVGuqx+&v+x$*2;e}GA26V?$wZj!EP0?+JTQG%v_?6wl+}@$>K2?G@ zb7t>+!pUxVHm2XhkfiD&)4pKas>!~%ClmqZ6{zSn>2QHXHl`J4HZf-Sy^K7%#746Cdzk^J3WgV3aDqLT=b?z#| zv;|NlecaH;udk97Z$3eu1y85&@Nk zQ%Q6N68Bzv_7d|-(9t^Bm+dsmQ6&`3RX86P=s-yWEOlR56K-L+c(=TC(}FMA75m?@E?amqM7%GkON6({pJLs zw!;x(w#yL?u8R>5uF{s;h(tdT8_-sa0(lbQ!Fo#JkDHoLT8D70F)e~G8xUlp~4enjF{t+@#>OI>Kvk+ zEkGMC>LJvCMDSoe5xc!K#ISsbcym9GxRE}w0MmqAs{B|W$-sdSrjbB_OHfEK@~{ZX zsF9A>gx>Mc406scX!BH*8TH_2<6nr%!kk)Y+wCVy=Gkh9Y zuCB?QmK$Oa41VRdv!4l@>|$Xzb#BOhGGp`?YLs41)>$hVTS}crbW_o^(pKOt7UVYa z2S%+h0V!l2gG3l_@vo_sGd4V`{Ft6#$-Jm~N4}M}Uq)tfAtKV-%KDX?xu6tEifP|5 zR`IX46r*E@!BZ~huu^*xoDyz03TIzfAv>9|8C^0;@VY{_asv$$bBA$w9$`wfRG5fl z@n+iTX1bUxlc1p*<~f5Qly$g*X)cu~%$_Xmq63kBRR@S(o=fBCZM4Epg=ID-omSrV=pv;5g59x40mflTsx!cl4Xq zt6XEnuEE*;bxwiwefRL+V4GZqW;DU-!41_hREH5^=5>h})#WI;iiJfC zE$(XhBF06$#F&V>+Tc%KZ?$B3$KR;QiZNt=_qZ(&HM@bpX;wsPfS6atErqLR9XwVxv z$S9_iB}`5ii9AjbYUACEpPGs!4uE11efozdbC+ik_Xt)t>6|;6Rmfaar~H?Eg{;%O zOEMPA=LAl2E&niY_&;V~tJvu&^XEhSwNIStzu6IHs3n2>z!Kv(ZOGGv)^Nv92&eM> zovAJaMXiJ4NYd z{t^rHO){hsFbjzJA`s+o&cp72`oIV7{K0n8OYln9ze}{kP(Bm-S;s@*6(%0@2F}Vt z+qUq9`p(@@ABG`V;v1P!pxgNtzn)Nn@8Z6Kh) zWustzgfnu{3mD`;v1*;z67c-RsBH#?QCIVaEqXB}2V2f#5HbhB0*YQ(htkCB1&2ij zkTES!0+fwSlFIqq*h(8zR+&@Wfq8`{p$-(ufkJjn9Bx#Uai}qBh^MA^~a!z)U0? z!xp#~gv=xK~N7fL8`V%!Mac%#{Wo*lCDi_UmRUdClYo`uje?`FJYh#;dEV#pAjPLzg69<%2j0e(nPLvq%!~=+5%#|5cv; zmF}$}<0IrlMe9>CyAyHdgZRs)^fBg9j`EY~Ga~g}E$FK%-DI{WV#0^~F=zkDCv3t8 zCiVMn{=1wx<4Z12&+Y76C0)*j^6_sH`uqiDm&i#)+Gz@OgJeXj^pX8@r~+7Vn_`iH zs`9hF3R+Zc`MgE7s>T6!kZCD*f++>JRuO46S_&r+yVzbEWz#+P9lhQj7-iL=dk{wW zJ3zNv`-sXlu9X^n)NAPjURN<_E(w~gbWyg(!;Rv)6P9zYLiyRDASGjP(%#Oz`WW51 z>i8F(f*V-P<2_}P+5B5u9%K`h=CPwOU(1jUVu)j()}$;ur`H1bUVM%pI~4* z^KS8zUha8=UZ^~YAIA!v#E62Iu$B+(h^Lje8y?(?*RxAmP(wo?R&b#og+qd2E3(S! z2O?9Sn|$xsu8r;;MTNi@Ch?gJAETn+F_-bZep`+X3Gj5vwMgG(rarC?kj~a5hCZd@ zB`Sm|*4AJ%J4x%g*3%OdQ#SwN=GU22iw~{*wA?nBj%0thRdBfkAL#=@NHl;16On-Q~%tE0p6JU%pVDZ^D7<7!2uGGoQ01w`2F8ilAE(e;~>M3oV z$M$q@Ere+OphI8^@jZ}O=Zm>qq=qRo(0t{m3?ZZ~{D$7cgOpK$@WUEMwsvYWtahtk z1M0Ke9JZs7>sxWIuq~xY2Rm4hLah+{sRTSPv#k+MA%>9HRWW`QH5qp{t%9@Uo(NSv>+!&)cV2eQ3 z;9$dvX11{!v>VTfY!kxS#9T4_provt_XXvFxLi#@KCeSzO)erv~H*| zi_uXhh6?>a)JEGf+T@B5%Cc*o`2h>VWDbjU^&9RMw22wom-fI53P>i1sdU4+M@}ka zNao+R=B5r|`M?_0job8-`?gKZ>ecSZQplD|kf^lXoKlBUxf1|sC)_OqE3dlBrxNr( zsY*)AkucMe4tLrw$lL9RS(xDkr_9nkmfPl+j`zL>RzMXlI+Lby`P1ooi}Rg_r*K{ld$vQ!9w zsoG3JPt$$Fb%y{w!9U@Z)D_K#b&vhvRe$qfCDNske!8wA)XYd5PTdc}kd+CsD_Rqz z-@&k4Fq}SspZ?8L%V0Zl+Fn%-5bz z@8V>dts;jlT|=X?d2HHj(Lv+Qf(9V)Y>Ux~kokw;1p|2$$_mw2LE~}gCQV#~jZuI! zwP%w1sWo3)h4RgyS7F&THvwYKm?O_;dki2!{^o`HB-=>I&T;s^b^hq#9y9FF;!Xc1 zu4gDY(Nk%M;8))`d(H?yLG>s|#T&C3fvJbR;h}Z}xmhwaSJ{h`@oY!y4m$JO$(|11 zPITq+-y1Aezlw?05Ga^HmCU%>?r#n+!X7$>e--M640grd8HYgYmO0qa7?~D50D`x` zHT<2CB%^sCFbqhQ2|_UE?l%Y$V~Ms^KENa}E}v)+Sa^kJWo`x?PL5}#!k&+)x=U@v zHPLDwvtMxBsKtj0f`h5Qiz88C!am*p31~9VTu1jdn~a_?Kk|}Qy$RD95&?n+_Y1Io zmetgssiWG^0T*0vt(OKX(x$|DcU)Lz+eLRBZ#>vLe}rA>dgKboP8Xni^bR`S@T2&8 z2^L>3qOuhZLI(eZ zyrHH~o3l7$^Y!N#LMN_vn}3w5tM;%lB8TCJy&CgKrfcz2eWVV@JfL9vr1KTbwVZK5 z`=}nw+$CU(Kt8#7Hy7C8GYt?bcv!S2(7>5^Vf$p~%)c`G2<*{GRuJubib;E{rtp5m z%FF%z__Oy*Z~62OZ~5&{Rc-n3%q^CNbH7mm66j6?Hs8q1-w&2APVzs+T3{n~Q6q0| zqluTK#xo1pSTG@45CzMn@cn?6N5V=bp!gftqJC4B>|Kc{ySGEYJy`R1%SaR)=ugB!>J<=0@R5zHMC zMq2}HhUlRTof%-*G1_Di{9>7sU@Lb|b@W-ZB30(79x{KQSiPzHOQPxDXkXpKe7g=< zWvgq-3{})f8UvDzv2A35^r|Ks$$(2>99Xsdsxk3@FhaqB8(N!Gi!UN`tJ{*7(?l-L}HV zJJ1X&0>RaT(Iu=xZeamDc?JsUxRe^7LP$r_gOs)_WV4}Vv{anuqHQcW%nR(SD*Qs3 z{6x@?^5jxd5;Bb#x-W*$Jspn8dCdXQ_CyDvoA zV24iFe9#VNyc!~24(P>VCNR%w>nxVP5!Fujkud$mUO}toALH2paJn!YX zEMr)3dXrASd2G=ys#!E*Aoe$0j&Bi1Up;H$!i)Z30nTSE$csx5q|s2)Ah~}G87bJu&w_OcCMuKt(aa+S_|{ zxz-D;q^2EZ><&oLX}v)8@$1<|^7zw_cqYJ{vDWi@jSRTqu;I>Y=o(w`-LB+JCOgu7 zpHcA>%=R!dtbZkoCnNw?gy~vRS@zwnt><0PPRUBlBVP=*LMi44J$NwE_4YS@#Hv;EjA3H^SK9*^jbt0ZVY5~{$$)6fyZggR37V7gU}IqQ!%dH`ei z0i}1B=%Oq>rvR&HI*725Bc=--US5GRE#fV{BE6iWMff;;X3_ojQNp1?dLeXicd|se zF{cPGENXGe*v7GW^$T^)i?pH&R#Za!_v712`f!@I5r z%3mK|kp={PFB)Da_-u}ezbBHHxyzo($5}uQ_!NuC^s0I#jNM~8)xpF zJK%H9;sz}OBz#kaMvi2v(|G{L%xx$;n-WfUj4lWVp2ft}fL%w7t`za;ZLyn>{LaT; zTQV>FK?J>*+?!cR1cQFMe*2&i!$ES$`+(B^0Ed(@CRV%pAk55iCRmOlbdp;WOu&ywH6=_Ms` zJLAxV=ApqUsnEO5t-FoqGSro}UA&{*ZLOt=EcKs$0oON06wp#{3}4Ys3NxOUe&kQyNN<~dP$K}{oj z<6?T9A?FlE3pQ9xCCKYx3nQ(>;&6p@`h$eXd< z@`;q>&2RO-gZ(3N<%E=!TS(ErC2So{IitE=h1~W*BURyd%V2K6VE$WZR}sii7$HAC z3ID-F@hS%7BgmPzU!zI?!d+QI=_m@GBREq$DI2KIiSTWFVzmwi+I515k=vDuQmwYq zh-;!9GV>2=onAsMMah}S^VhhSeEHA`bzY5t0*TmxqpG_!p3HBLq@vKSUxrGLQJ{5V zwX%g{(Z}eB<`3LrR!WQTdZ(pbCYjYiFuH8AI1J@mff2WPQwNJ25?bS>62%i$*3}k8 zKG}h@2z|2%#0oj_n^ksZvutM~V>)8t9q{BY017FHF8`z>^^WB3AnGGKyUK97KghUp zK%(+%>5`Y?WGIVEE7~4B#;dHdTkjcvTa12;`RB!F&VcUqEc;)1&QS~3cI|l@>Uo<< zfdm6jcIiRBov`@Lu)3(e5LUh?j{T;AmL{{+fclcWI3G78^7sGU5fir7_&blG-j9h976e2Fgvb1^8K9Mp07^sHTi z9&&LQbzN%UM)S$<`kkEX8T3bAuoL~+VWCdU3_|V5hWOeRwu#JV&j0ZK>1@i0t1mO+ zzLGt9gw6lem+i|SGf2bn$RIoWmHuMaE&-p77%au)+%Pas<6otfRR_W*3-Q-xK-l5@ z?3uYnsnF<&U+;}Grh&hZnxsj1+yb3At=``%iH1llXn4#l8^#5XI>1=ysPWqxA&2)H zMGef{KLx3?5*R!S!2!>=ma`CApxWaU0qKwr?7=6Htg6i?TRuNScHB9{V zE#VQumyF{dg(|&CPA*?Ei%4>8I3$SFVJ8e>r1VpUh zc77Z5C&dJCJ!J{{H&*1WX1Sjx5lyhLg4h6xF|VA}%s6tqb2)tFn?u_@of+y&&~G+> zX_< zUmY$a>!z#4ru$|p!;Q~dR~Klte}dAJyCT>31kg!tq|>`V23+H@4l5{0jy-NGUbig4 zUOYI;zA0xFzs_SdY^(*G1j3RUD-(s9dc3jZ?IDyszZa*F94%H~F#?5U+7uvr5q+o{ z5<%XyB)k^QKRc@7+CaDK;xws3_p&JLEj}_bkIO0Qmy{iCwn$cx$%pXIxpW%T869*n zF(s5b>QTQsLDXKLyn-xQz0Wr5No4=wF#Utubn9h{b0v7mud&0C^m`G>j5WP&r}spQ z>DyZLU2nXnYk|S&wOi`BAhI#6flZ!gwk+&0+x}C#=MH>3Yd+DURKFmYND_GC+am=y zs#6S|oH^a=8*L_ahG_)kxDjjO(oKCorUX)D7y|cg@RUI{;0Cepn9WshjVgEef>`vo zezwf^9XP^T#Z3L%4r8na)ANYs3Z`jX(T=;^s%9T$jVz6^BgfD#@OBDyA5>EPt|}=# zF~u}kC&v_awu$=pf1?1$54fbUK>YfZi}pW09{OKjgrtq(e;p4sscP7ws-W^@0Z64e zNzwuXLD96+q+WTgSAN z7)Lt+4W|=Cl+nnTM%013I?KOUYolRuBQ`B^S!*V{iNe}$3|{u%b48zk3>a0Y>NX9} zPLjEzhrUdh8$0Y^`IVU$Zs_aFoSBr^NCpVYZ zg*;@ZG2j6$k>-wuLu*#0wx6Pb@NbKh%_)*;eKSl?)g!$*ZDu9?Vzb1c#u%cbE8 zQrTQnACt<68Do}k0Yshqa}$f5YS+-VGbs{vJMEDIF;by00IJ3XNiCf+*wr26aVj%UUxao|Qk#vGM~8?E<4e$ejPwUE)K$@v8@(IIRa|IKdeZhfFznM-1BIsC2QjxQ&au zkX0sRqatP$FG_dHHjI4~HG_XBBK?Dc6~8zCsas2cMP1)g+djjIfhGv*| zg{i*eBRxG@^1@iz)cY)9EoS8dc&zgn1owMESr=OS`Oq;(m{{ddm`F84w9KnhTvBUm z{;~d3poM_;A!tpE9MCsfS#a*VgX6SVbj?DtK zXMcEmKO5bBGdl>j*nOq+V{yp91`1;$Hta(5QxUaU7>uf&42QKZDq2ZFSKK!x9)K3J zMhuyS>sZ1K?t0*`tEQR4X05Y&prtou%f)Jm=@~?NVYgC0(5FfU{leKwz|{0yjCz+K z)+BxxNFwTh*PVc24d?)k0%_K*-F-Eftx4hQec)*(99|qR9Q~$4_w!1fjn2c1fjXqs zE596fjrBllxtm_McfKOxIRI1{JsQ73{+*{vH#B^~GW@ViN*33UA{jjmN8c`6&NpCt zo+alhm2sY(fQB$n9CM)x+yLygB=*1P+x+e*yZ?( zYm?r$#gBd_enS)4rH!hQJUGD-c?ke6OHl9X*j@Acu_^+j>KcGgPe$25UBi3jLTCe# z%^F&JKJIT2#;4@h*9bSTfxUY_XNfEhER*UWvy(lm{ffKDDx56Kj`f)L+Glr1YzwKAW5nM`eM@!G2PVUuYAlXf-)|Sz4duBn6eHqtU>MpSM$B z=p6ZvJ0agjC=dVpzezLRLaGj|A6XU#`|B6m|7`d5pDSJ&16xIa@qcZ$-ibAW03C1S8<86!mQ5r4wkztf#NRKd~+{2 zPtb+v<2H>Ti>BiB*^?IF>Xu5;0-s_r%9$NFP0`()Vzk;lt!08U!>ySrkEJUr?>OmH z16Aj86qOF)YtjK#!xzcqEa8G1;tnUgH8-L2)dgxRnA7MqeR%oxIh%I43Kd-D3$}wZ z@B;_J5X5w8CS6hGE5Pq(+EjjpjhBiAd~sOuDG( z7=uyLcDBmpjeLXFeJZoyvux#Q6U_U8hE!mZocFcLn+6x5Om-43&anyJQK+g7$1}`T zULB{9(n5-f|A(<}2of#ql6+;`wr$(C?W$L{ZQHhO+qPfXwyU}p6VuT>F%kdf?!L{9 z69;)R^Ds)g6L?a(vl)0Ack$Y2Vf{_J*9n5|?UAU!xp|dotRgBA|7WExVy~9kPQR_u zmjd^uRAUHg%QO`eq1_)7!Hp~VSrftIRlSRa5?fchtA(<&=ZV6%TOR?XE`M7H850mt z5P>)WN$KDdV_=h)iKDmg*D545x3MvO6tW!jqZHJq<|$xfxQ5bgYU%#hk(3T1 za*vpzbC2+ly8A5k{d~i8J0#_0hc1PHQbz%ELhSuJZZO7~#S(o$Qmx$@O6&ZHFYb+& z5%O~6VneY7W52Y<2;1#nwn~g*DC^k@g-3JLn0#%PK@9KO4F5NUe#3^mXG#w9ImQ!G zxtYH29KN4xS+*0I*&S_j1y~#oK&@gLk9?MODEE`amW7;hgPmnwsB0gwlpDWY!zJ9v36$#E1A6t$h-WwE*aDdtt@1VRXx~NZ5@W$NVvhun z*)6g^`PHelc9}KYOne_sC%il^7SXsvEiMxXMB^Hs*e_OR5{o^LSI|=9eHov0@aKQg z$3#osXsUj*jrZ>XkpIx6SKh(a%)!LbQPITC*5UtJHWjOGDq@KtUurK^Pj+ z!lELI_Kd{zqYL)!85%){u*WDhqiP8XXj)gR(!K_M3ikN5Dk~#U4FKAWync zdQvHH4@xotshKpI?ze~k-Yf?+Z8>~>j-j*jEI&MAOtCa+y_`yE)}U-pFF5jeHYKq% zTdBH#?mAHYJc^igDe3sDjeqTEy76*V;7Lzu_oY`4TZ=DiOn7tnCE>3%{y>YXfd)RC z#A@xoQru<4SB|W=`7h(Cqc8qdk-`#Xg@%r9=ShEUXCf8k8ryK9*C4Ukt5SXf1U%QH zILj<`a@MN#@U2Cxnx?WRdE3XT9a6UaEtp?v{h4D=Eo>`Y`+da2$OJMaakA=N_un{0 zdVB+H7+x-#`OLa>#XuGJwRAOFrYrVlCWZP@!*nrRz*?hhF+1k(Y?;7@|uR zAy1ndvPj-Vom%I$zuuLYA(>+0K^5b1SH>*I)R@uSl7bH7xz~cKY~2+w(QNQ~+=C>F z8h(argYNWVj1ejA*cXKLX6F`3CHVpHPyqjnghl!8Vfs?V@0DMftpOfYFpFiHLkW8p zcS+7Cehr*PE{)Bn2a-~&eGnwV1p#|ScL!tHym=VdIJY}rPD(lcZ7HLZ4l5h*Egmrh z+!1qK8kMdpp48OfW%oy?Xo)3s&O7=L)*&NvvCzKh2R#`EZBbunp)`iO;=ZwTDbbpA z#yiH4+FQyH3GTzQlt7~fjAM`|JX#q4B?}7W+Gvm4KF{S9qmda`sfLAftmiEXHEocl z_REbNlE?~Gozi){G!lBJknQ6ye7J0#rXvGLGI!*;SvYwrx39ikywnk*B-{9Wl^;8FSb z&l9u}Xn}1&*ggJY^R!KPX?$LeDraF!Sg0b2vO)34d*+a4(CEqb_%geR$9PYy|2kudssBy#aSVOhDcV`)m+$WG9}@&5>jg>0>@Eu4Py;{VbRS1VatF7dFHR~gM?!w)Qg*Ubb-=t}4GaA-`d*R+PQ(sCRr&?UKsECRO89e4kvnO{zh&EwIC8~a)M{y?*Qz0JA ze%PDE=n^**^SjgIC3FwzF}(f=1Ew(<2Sk2 zNrpA4Ckuga-vGi1H`YU*_*W{s8l^H`_pF(857;R~Ne%%+<(3$KO;$inB65>cDTNas z7K9KUGn@3%A{muYw6!8aIaN5xZZ0lW&&+x_l`wGqo@IGYA@Q?X(((i6ez%-?(=7J&BaqdIz(6c6YQlk*zv$)OkQ3kzQuZ4ycl;Z${*H z51M{-TJzrkqebpN?J5I^Z@K>&LH%)59TvaQ^Yz=ZGyG=}ByZs0^xv@Aw5pUNmI(a! z%F1BED&5{%{dnf?m?f7KE@=q5P@NI+L1-9usF^;s%%O9&%kbZ1%H1&p8*Gkyusq0M zNHCv2X5SEOu*f`(R9c$RN-c7y^o!ezdUCxb&A#6sA9#Ok4C$hwR}|6vKMXa{CPUO& zhZ{&_p)9=ozyfHVjj-TAnu*H90^a52@gUBP>b(+%N+rBfhKyhaov$^f{_W{*J7kBx zp}=fpg!r4RmVZp$xu$LJ+5_{+49Z8l_Z%hK)HjV*N)JXmuU4j2jfAniRFyU*rTvyY z!xRg)3&q(f8YVu!tz~-M7oAuYw-^qSOwUQy%2hUvWXrMNrnq#fDP4&nvd4@AMG>z# zOLI`qxtsD~(`mL1MvnxqC~cxk2cMhL%_chWrmMz)x~+ImtHqtXH1pjmRhgpWS=n|)&*>VMuQLw=;PX^)0QIlYJ>ZR+n!~Mo3m9#^}(%O zVV|*alLi_?np!jFh|c1b%5#6g>5uxlvn9Z_)Mk}Ku7q+7@NmpK1)&lddqQ8L7+ZHC zLCa-Ids)cQ?X;+Ks^`7-BP`17KZB*Yq0oV}UNZ&NlNd)ee@a|Z`?^LUmMspGzru~q zZBHuIM;m-U5AWEG!E$QD(!n3Wr_}Aq*}TKd>BLIIg`gAU>B><__5+8>N=MY$R^?!m zVQu7$TC~m3&K`5$1+y*Mr#Wn2LAreC9O)ifvYJX{iP2q{@!i3f<&w+EuPY=(IJBUH zSF*}3m+`F9YE{m%EZM@Fp@CP=k&xbx=2B?eA!_@TDQ>(#)h8Z0?D>OzC!D186 zXKg`~oHctDG1Mi_UH~aWH8F^&PF+!s5=qW>@Yh{)K7?hF0Ttlq0eQOpDlklBX;{c_ z2D~jUFYe=|ph<+2ERpxQEAO0cpWPY?QM{6oR-`a=&de3aibFyO@gc+l0V|n&%k+7f z$lY>h&>5b=p`T^vIJ=DAe8&8&bFmUp`_S`7j}~)*_6bVQ;WJ^M6JK6!df+dSd$P^( zA1LE5VbY1&Y9u@&7tP>uC`Bq{>HZiA(}*u@R}veMBeru<>7*V}?W7*j;p#{XMuh)_ zx6;I!<9~N61aHvZGDT7AC*qcI(!KhRaTxGZ`eV>vf~KAfVi(SkNNus`s__G!=)y4p zGfHWe$U~eQjs7532_vYudHnDAFlVUJU8KDk5sWk$MK*IM+Ob8#r`-vN<6#n#K{%;N z;Ozcd26O%E$k`wi?J^DSU9y$pK@y#F;aI%|CuZgN+H&H1L}!Zn#JC;8a+z7zprr-7 zqms8+l6JnxQ^~t*$-DXFYpMY^l>I;ob~+HcLt2n_)R5nz5W0&0wy;2aD}j5XT?cY# z=4dB!;lDV&)=VuZzGCQiq@axaR|KF)2GAVV-~}0-Z$+;Wo@i4BwKjCHV>(^~2ZsB` zAjhw%qK67aeZ2DHRXoVI$0dcMKZ~SQW&a-i{IALf1n`#g@0VXY=KmjSNB-L=tzcIwwM6MOb^% z)HYuceJY~qjwZ{Lg<>^56vT=bPwTTdSCz{BM_5bx0eT3d+3u1pyioLHVi%ozL{BWO z<%eeT#2TY>$FZEQ7v4CrrgmvKZAmV-Sks0#!BbtZZgz14x^i%_PJ>hXTs7U*FkkL$ z{&Gx?d~x&R1gQQqI>34(4d_abTbG{JI<;qg;vAjm6%(2%o^WUV_V(#X*5gl{eDmSI zp)DF!&HkAKb?zFQ@D=Vw(|8*OenYIVD#qnoc`IgBACC+RNmV#=pwtm4&OQBmv{ z*+}C8zs%Y zj~qj5xaZZJv!U8CI_j;v?^gDV(&dpw;ueS{v;L&`mh#}10m`kvGkbI7+%XHxos-Nt zp<8>6{48evW~4buW^POA{d>^6uT}nNxgWT*_G0R*3qxua{{Xk9i6PNOYVDM6({4+ekSAXAo z{<*09&Gzj|wPSnqQ`ay1El2e;HNjVEqo@9ss`^bw`JFlPJv#AYdGynGUt9HQsd)o{ z^%H_e{3o>PQ!Arphm@bba#xk0tZZ*#cudnr5!(zB!VWA# zMN&u`Cf<@_z#x&8t`9HYSEoeDJB+j#gDZfSpolnCkX}YS5AfKdP>_UEgf%4PG2NRO zwtq~M$-GUVjVayPQTpK!{>hB{-UmLnBjk~7fFxTGof^hLcrX_r(;D^~aPwEm67-%7 z;~{xK+KCKas{ggS3(X>+gh-{#R1d-;L)8*#rcHs5DHD^0mpZGD>R8>C4cDwJD@>Oy zOgU0jkn4r%FmH-LS^8HLnXFnbiB@U7?^G{}W{FY6rc7IDSyQI_Pg|*_DnVdo;i)FN zMzOZCvb%}*^u?)pc3AePr-pxS8AW^V{<*Ebr^!XZdn&IgLWKor*LXRVj?R)?58I+4 zg2_%_Lr2F%M#RzENJZsI5)=?%K8x4a)1q&a)zkY-QO~n;GuAQ2W_#i2{IO}N{Zl}0 z?%+aD858JQ71W9)ue6L{Lb3x<1D$mm<;~WS+y)*%A1DvydWf6hS_*8g;T9A zP@qK9<@Y6x&hh!ko6-f(l}?R*EZ%A-Yz)4$FjI>nLo1U~qH%$(ZaoV~TfIN-1FbD!+|!1+MoH}~%w2vxmK z^^LM)GN&h8XGre-Xk#VfLBY z^)u>uPyep_anoVR>%>nIWz1h0>Q`=U?O&%TMlqgio;-kIY5T)K$6)zLER{pYpI^d; zq>5@W7!?J=2cov~vRIm?Zf;}dHE2E|?D@Y>VcLOtcEn#pMrb>wRHw-K-Z20}VzQCu5+CIzpsgitG3S<3S4?qhQB za+D`gfn=Dx?XNMKnmi>S!fh)(daC|&3qM7<=w)JX7`1DwXuJu%iFr$ukr_|hOeHe4 zt&jtDq8lz^(K`*m7TLz#6T^c#U_=!O5n(K?osy=V-k1Vf^=aV+p#Nb@XP~4xCHJ=p z?GLE(1#}6?1|jlxi{enh^y0sRR7^$27||=}5>ht6Hxc{VLuN~?8pZj6<3#nch<_!j za$VQXogLC6S$<u{7w= z6`TZa$Njy3K`=%S)V0*)%yjB5l-c4*OM|@#edVN_NJo6Uj0cjjixX}sK;0OG#2ccpd)JT^4AGcMOP6>&?pO7rK&Y?|Js}GUQj$;6QrxYUmBiJ{MN^=*8z{8rpyzWAmS^>zJ2&J#zB(NoyMo}lqvyGX%Nb8T~zBI2Vb=c=J*nT;6zfOPfz zM>+J_R5avam!+0;b@JD7=#f3-#ljd(JM8%WV;(Hi&4baq`Fp zJ_k9_XjhtYBl??hzi&6G$cvVR5!FMwWH3NB36CbVSn=z4^d*ftQ-Vk;o*$JQXDrGUKt|Km zjTIvucy7A|4zRs$^uRjVz*(?e0yUY~f`NNrooAaIW8yl5C&n@~skltASbKr7YP_2A z#Ut(|W{H^Cs6d^^js>M?VX>pT2Gun*HVsOr zcg1h~R+R&=Q{rPE4tNp;I64aE49GVUySBAJHnliO^(WA1tvL(RGWgyf=7oD$Mgx`$9SpEtnyO= zPL#E0K32duB%y}ZwXVtDouydz%Z3iKFPiarC0GpYV!qLwg{cogN`lr~D^~4q-n)&f zlrTw}&EHi}*9$|nJRu8v9TJL-xe`2Dqy^SgWoW|;3?vu)kNqI`?3!sH8dg@VZeF!Y z450B-qP2W_!EThxKFWjwDNXPre`<8d`J$)WBmrFaCM_1l99W1k)+o`5x9zJnq8tan zLKl=yswRwIk<~P=1MkKKI`P@w>g&0uCT9{&gp8Sq%S+XR2O!Smw%L>0;e24SXewNK zRD7L@sZss$G)*)l>PV+EyAp23iVsTP5zMSkvJpnhV zY$k}Eq2>feXyeMx)11*>e6W^+Se75PjBv8MSXd%0>*+vcq^yq9*+6NeO}1WXBnfm3 zp-Rny?%o`qY&}@4xH}2nJ@G_HzG`bijY1Vq#qjd-{J_XQhfM21Ubf-QA|0LXc!aX3W|-0v_a>kcw%LTd^s-_TBcbA8ld0w=R~bYE#oP48;x?gL1_j_eyKmL{dTDA=wwx)-bL?t`tR9!q~%DGOkg2 zRGF4y?Q7a+_pj77gK`CK^yDcrSu(7JU%CldA+28Qg9=%paXHN!hGAJ0WDZT2g(gf2 z(A#0p=+AA~Fbq}imv)$&#%1D<;@LdCN>*ADRWskRGHVNMXbNq`F!$@aqVLPvhtpm& z4s8qPMPjh(jm3F&@8br_m=#0Eh&f?f(;64AoW+3tpoU7z6+&})bl^CZ>_2qYp&TeH zukuo7`^>akmR{`Z3T=^Hvkd|53p6!{yr#dl_s5!AVoZg>8*^%SB@0+WFs(kyruquy zl?Jy#(J*ML8yCWaxQTOH)jil@vCrvdP0uh4$=-q6_%ynAiBowj!FLLaqj`s{O_wsV zx91OD@sFI{HgguZJBO0(nmZFAj-6bqhE`TCBb?qoxoP3(^yC?3Z_e-EFL=Pziy@q& zg{)E08qj-$-TwKCJO1tMLfoXeW4PgU@>UydI3*G_51kgD#F)b&>0WJF0e- ztPV?sxM_41KsB{B#uB$yMykQwQQcVzeg2B@!d8baLSC!g2H|agq5S<9^rIx!%op^P zrkeB~eHlvxAo?M^A+3)^A+ z{BGtUw*5R>Sd4wG`C3c(RucrYfw{Ccjn%e%2>Zr-iwu!fMQ9l`v;toj#)dpolQ@>R zY8V=oMhs0%N|0&zT6{h8vyha0lY)NMCTg0@7@&he8tuYs^M?;ki)K`@Ik8>}SZLX$ zd!l{_73*n)Y5(lZcnd(#k1%g5a}MH#XVCPzzpic@*z7RL;!5l|r9@-R;F*wxtDLM7 z>DKm7z+14Xg_}1*x8_b#A&1fnj6JmDNeIs6mhB-arAI_VkeA1H?q4=)7pS35+CZ>! zX#1Yh^@Yu)n->J$d-U7#`wONlvLJDtRemLDvIHV&()`_ z_<41fm-wN$og0NbVQsL*%w>PgU97A6|LNNjQe++oj$%8GgaMMi|78Y$bNa&{oSYwx z`Kq`bVA$F>1vybo1r~Aramu#HH!D z&u(HaW}dX*n99{zqC>y#47}hRMoh;cToz2n(k?41t(em|E7D~DMGD$FfpvYZ#9Zmj zp1Ol+y2dmKvD(kSk-H$Bp-ja3>h$bBYx%&XNwhH|BEXurcWAy`HUF5xMSrYnV%Q;! z+Due%4^RSmi|q4&2pYIV8#PPnCM(A5fB2Iey0k+COY z7?&u|l$f={CX?f|RET-PiJkt~#LgiHS&le3HGc}aDN?jgvLZ>}d991{s~1$UHqH$B z0^fi`(?AR;EO{B-HZV6zf+y!<@WF_d+QNIznvpflBmNP9{Dp_tf^S<(d!kzkJVRYV z^QTTAIGNf-rDDIZg+Q$59{2t0*4&n6W`1*`J4vkY#T<7L(rTfU9<32c?UG`=-#7sf z2}NTOd7OSzsyRY4%%O!0_CjdFK?xYQc~RHl(-fwAP%y8sHjacSWTA;UcRU>@3pSfi zTN^Zu+@QEVqxvn*P3l?3>|954V;oTe?>#!S$ot&HktDhwNsmQ|C3~GYFq#9pwB&vV zyk=9sS`sL`_c-?1kcc!4j!!WohVf9Tz#J{!F+Hq`q$0NJcJutvQ`3q|r^*C=2u;+Q z@nId)#34xF`1m0)Xxr{M>0)A`iS`I3fnZ>x<)FK6IBXO;a_k{?O254XVGVk|WCg1e z>pVs{{cy&i?K{pmhAgDY_G!eG_3U2ouL4utP4>DMhM-pQ+;Y)t@wSe`g}_;FWwOo{rQGC9{5rVNsk#C)Cjhq zRg)~rv@O&k=wS;)gvwjX7f5Zb{D+&48XB}SwdKpPzF&DHFx+a{XX&?KJjZ+T|?ME=_xs4uaY7$PZZKx zvLtoUDe!H->X)o5)eWu{jWXF3S_$pWnF88+J&?T;x1wnAB5lwc#A^0oZ`IgHSk?n^ z#Ot~Y*(#H7Ae{WY8s(~z8sko)3>#%YWaC=o`W(1S5GC>RW;^ zEjbR7VJ+~*D?&oNOk=eUzTGDfL<)0fswa9+#71z-xzfLFxF}==jVVdX;kteZVZedP z!$}MA+?4tJB65%Aw!@sxrOCV*;HgBqQ5rBqDi+Ft6o9A>IPhv_4W#b!E}1rhBo;Yf zvB|C+fRMop2>${VpvoC-myU-l_#Wsv-jF*0B~alEOoH}8$Taz;WUBXQHOV9vUp*HG zM8u9f`mA;J#%R#n=nFJ`s;(1kLLypZdG0quA;86@n+KR>&j2_IB|Rk~;IQSuihNq9 za#Gyqo!J>U?j#O@Okk9!$#r_^DQ@>Z^P$5bBAiIg zekA-w5-dzgO$2gl8`}@V2h1qgpN*@ zb6jF*;sR1q0HP8Q!|h0DXuj}?RWTQV ztfHmJeVmyd=;pf!(($e07JQY%Yz~U1Lsq!z$pau5K%o$4c~$K3R<%1$ z3V`P6*R3d6p1`l`(@eKsE|(yce{?luf!R20=$XeOB!?!s$XW6}5{WEvTFUxXBjJE( zjQrNH?&0kDP<%;{%u1jJ4vEo1M3?6&txrj`*&gV!F;fhhC3|y5l~`^@O^(DZW<#yI zU|M7}_+QT5JXVxf^`sP}UI{j&87daZA(ugv)YC^I2P;;(_f>KeKGDo9@db~xBU+)8 zx5^6w`hi__={oIR?pOE6Ce2ionUxb2sj8~_cHGP`V1Q{{pT+M2+o88h^Hj-Q4hv(; z3PX~@1(kO9FXCO_>{mpyQedZ9K_BoEOBsy%i5`^=jhPO@E+TsoBOS%dM;e1=z zD)QN{!>__X1)wBK9-KZM3umHOF{qOvB_w}Kpbnk?z8kmBpp{uy%Iob>SYSq1Pa^7O zH#d6@iorpTPz?0 zV<=&ZB;|Ks{2iV?pQJ1h zbq2mAxja+=OxzQuL~v2sp%G@1hH|OgVy6oE5O-uuU%36xPu0*E(;~1#mc2r|vT+nbpU`Ml+l8?S#~g(0?kbKVOfdQwab zlLWKPjS;AKpqG`T?9XIB%JiSHbaagZk&Q#HP)$;5AFS{|5&UJ!wAa}0Eu6m(FSA4% zZf5|@hz`;>)Ai@n} zzW^tlMnk?>@Zh@6b((;yu|n(>>kv37$)?+EQ;gzwqZkdLJ7n7#9}2Rye}Zp6y*eTf z)Dc_L;!t{}~IZx1!`iMW^b+4E?C%r78LT6J-$0@N}e37qOzj@Jtu({zFwIXI88}lV3|= zoU3uq7RqZC_q2ZLcF8A}BG=c8WOPnuz(N zR;0+_40ftXXTG+ss@?>)V5esSMg)A;NnnI#d>ve+65n#kJ64`=ziD79A!`GE?;_*_ zE@~YbUNL*>O+P{%tUpB)r&@Le=t-vzWe@o=7*?#W@|p#7_Ua@ASvB;f0aK_C6BWps z27E=2!%dI3B1B}>mw{FTg0}^1)^823A8DtYuSugdRS_Pdt~nqJLMkP-&R8VRHD$t5 zg?dskxPp=Q6o?cwBfGfnki85vXvnvRXuYp)47Mko&0=1KurtqcjUj%5QYA5u!Ep47 z$6rIJEouNhLZ~qstncMyfYFBWCzC&PIu6K8DoCu{03pZ`VJ{gxBMzz0VgFbQCH+R) zv1k5o4|di?*=ZJ1f#b1wKvt+@4xEJ@z>KTGR97LovCXZhU0;$_TWke6Kf_@OmX>hFc~b;8bW{(DB>&S zXE)~2ZbzEMsAvgJB|9N^HIK~(Q^ML@IMB&v%;XS_U#4hJ{&bbd^k^<1R8c`6Dg{IX zD(X`|v8H~s=1UoRKq_*mz5`vAqE70;Y!VNBaz8G6ioa5$rDvWWfhL)v1!iDr-kilc zlt|?zD#eJm-yg|KR4>_^U}>`n-L(3NoCPqZ+{-K9lqLY@+~DnEeY{n$#%zke4Z*KQ zI{=BN&IH|j|1Oho0yw2t{X zYStBGvbunvJ8f2*;HPW}1E^1fol1$P0Xv_@L*sL(6u!Uw-czCtV7)lkgx^_SN}--W z^K9n!TgS0=~pclj>A2`c16{87;b^!SJvC+}h>? zI<-icWs1EVFh0J{WG++r`4H<1^F05dURY0ALhosrchwu)CKhW9JG)!eOmmC?7gz^c zNSZ$`8hd6Vo`+rz^^VP!y2z&+lLm4@7?AF7>5llBDAFEs{A@5x+#y2?UGENQ>n09a zmlCY2YyGLtcM|9Dug0IzD`9tFQ??0(++Q&qu3{lkP!;JBaZ}k$Mo&7VJ`Wt0g}SF^ zbXkTKS1Ue;r;e|}Tt<_Um-_RG;Cf@Necx$5{T=0X$E!RLDi4R(k8ump)K@wO;46gt z&Q)>wWryJfy4rs-Hmpb3`VGeMfd2^yE73p$Ml~>&h$(|#izW$su*MeajrTSj03Ty^ zVkCi|WpN33h|LJ*Nwi^6JI;Y4Pv4GC1!l2BZrhaZk#6_7ZqtX?cWY#Z(!tPOSp`^S^6dbnrZ zgvA<6Ckt?;kX;95HAKH=XFb}Cx=8aeeAYTx4Sdxm6b*aca=L89PB+-bT!P6Olk_-m zhNNQCW_J3sljpI`=^79&tW5XU&00L$%I2a_oqtK4mP;TTaJKr2&|EQi{3T1gB)gtz3|%P~uo*`qaT+y}bmbwd27)7_1&2?tYYNZg zP-m1R;*B2@7oVdg=_ahvW%+XyJa8stxxQTErZC51=bdN|G4>4^=P!%;I~5vU-iTtWoqp&%o| z3%nN}@JXSLit8hrSvP$`GaMB5(r+PnhJSpHNQ#Xy zTfD~L!t`3nB%%n3ss9Z>V zC#>TN&ZUiGO=f*j=9hh8G5Rd%fmYeVIuB>3+W8v=Q?Gx8tpPLyD=T)82509D)fg2! zI8^bzBBi zb$n;-+y^4B!@}t<@a0F`wlpCHhUYfood%qWNV~b%Y>MY3vg1?37?z!k+8x-@b*202 z#;o$^C3x))oO{T~GA9iP&Pa0DHp#{q{uWIIvmQj2f#8HZwB6&MKNIV*d6?c&m@W)f z{MFd{d?D4BjBS7ytAt1hQq2ZGF!bSHQD4|>#R%JNp=*O}OUvR5*<+}jRl!JcDzwGZ zgLy<%e6@+2W5#C)Te!4RvHPr$8XLuHfKmjkNGg2RkvP;l{BN!Ls~UKNagG%;E<|U& z1FQ8Z4sajA5)T|<;0|!w)G#56Wdv>79;*{7xq8U+QmFSo#kv{7BuaF}$dtC3%5n@A zxG*>*79@C-@y{+^5^MjC$yqcBVXh5JO;sa@ZQ1~_%*BP3$Q_+%{)S)BA9nzdfT197 z0yK}kMPI54pR%3`#1JFWjj&9KI0`4E0xC)Kg+Lz+BRJ?%9|evhjw7U!B(V7tn8qgk zk&*O6ICFMgGRFy^SO_FZ9dT$H@AbU9`TgH#lU>%B2j|_lNQY}wsRTzgkp0V^yO^LX zB&25%QqmJT7fTTo+U+q}1J=qNx+%|nFaehfmct5$Q3(ZZa2{elnRqR7 z=`FpHUbVGK((Bv|`~LD_lFmI;l*fOPA55{_;rn&OpG|N!L_MXet%aTXJ@`LfKolv< zudSji|L*LJJCoR6xjPi;2-SLk6YcNMF4f3ydqYSrS=!s)*uRXM=z0Zlq$jyq8m5#K zPQFEHca_H*wNrG3deFdI1G-~~n#MpW(YwKgx-W)ZoBUPz-}C~`H6(jrQn>=ZQe^Z; z@d18eYDz}+f1?T*ZE$E}&Ne1dGu=#8jyI4OTb(l5o(8hE>LU0Zo`hQS-*`?n{jGRI+@Ts&@@@c+OpU^c6L`W?=%4MDsBFh6~!)+CU?=C zKrFmkh`3-WT5+L^JTs&0i(2Tx3w@9cHu*-J8Yn%2KcH%oRBne*+pcDV{w|{>2PF3D zd!w){%j##Gf^6h=faDzMC2ZtUfm=!;Cg7d|d=H^Q-p2Cn62`6QfuK6~(sAsnGWORE z+H2769daBsWvm=>3R}uD??|SEaPDzCFm47JpV1x(`20zq+1NwBaC%0+!*lLV>`*=c zTk7fdM4$26GQZG#275wU>1#NL-qVwTGKX1MW3|*#Je*J*P`aZlbZd%d8!hOV$**>eWSZnb@&>GqJ`ygqe?O7Dw5)pch}AC|p9 zZBLlo*>a~_Z#lfE`22S!PwsiVaD78^r}+91KHdPoS?G|W=#VjMk&)bz$%l84-V_xk zErtEq!lJ!`*)!R~BVQ@Y(Rzaz%mHPMdWAyAMx^9@zj1L&op>>U4D3|(txZmwh2kLG0#{ufZUogKqYV{d0 zPtq6g*0nQKLuLc{XKVYOX^KSNyAukl0wQv&g(cQcqy@HnVzb&# zvO9Q&Yj$9QfgTs241jJ6D0E1XkkopGXK*Iv|7BU>YqTcNx(Ta}&(37v2Be-89hGi_ zyLkTPmHeS1w=NHmtWAJiaO}u#vASF32w?@=wyjJeH}m0S-G*&MwnM^NkbmOnC2R~d ze6w7DlPt4asXil5pB*#c3tk6Oz00io>qgU+wL1806-N;k&EtcG`y%SLm5IP5J_Boj zH81aSA>=}HzFif_Ab#T_zK0Vx%|jnj8w%2elwyZhb-X<{H5q^dUiiELmX$%SAqCDsyzSaf91ca0HU@;OTF5$< z6sf1tc;OnBl&{)lbTJ&UXKCG$72gL~gDb1n(04e=@ISPGCwS_)b+cDrK577=V_!Zs z?y7*1*~1_8_ORoNQHYSMpdX`FA@E$9(@-P==dqbTJADH{W3TlEw<7Fd6cg}QR@wuk z%rq@HQ*?`MNr!znKRVz(v3$N|{i9lrO=&^2?!og(HjwT?A7RVRojDJqS*;Y?g;FKU z<&7xCknV?aGs?(Su5qRRwn2PjP)pa8a@PPTx^LZkMr?zF)u&glfyuJ%Hjv?Z(T;~- z!mr{Ve6k+^Qd|Z?V+X{DVe!OhB6*zi2s-MQiH zn=rZhv3#MGNvs8B!ZfcG2r2qk>0Z9CvF1VKXJ?UUj{5b0n`Wa zfZ2XHoE}1VvA@Bsjym`LDnG71==DW@V!X!jL4CYA-qQ8N0{Hry+@`Jze?wC3)i!8; zb6W2Anqa*$T~hzfLt{IMh>w37`h8w`KN7y67}X)B%zi`%Gs0m>;swcZb`R_rob@co=hptxV5IgLt#7p!mIqfNgj%e$uaaMAK$r;lkSrN zRjfd{{1ht&FR&aa$S@pa=RuNaIx9dNdt2e1!=qkau~OV(LI#mN>j zjIo>cr5nG-6@|?FL=uJm?0NBPqXHIJ2}7ijKEVGTH_P*5q73yQ(4hXDB`lleo98Nw zaQ{p#;QsX%Ojjx`AKOt(-&cnIBNfV$CWV)Liaj8aFg7|2szObD;z7~&rco8!+j{f5S~bn?`ANxZ|gg>r&p zt<&*Zqq&l}+I-5Iqsw4FJhmWsuS(`?thnK;P{1RN^{AF)I(1TLOKd=}3rVI6$Vr_R zdEv8L5owud@YuJp<;0h4DR0L13fNb$(_g8JGb2A$_Ed(A6c%x!5thExbOO?h%k)#$ zTgZY`O~e9A+TTodoZukSgzS10qTg*;4WIb-$(_X__l_APog-M{+sgAXE}*WyUpJOC zLVZ%Z|fcCANhQco8V7*33ETeN;5y4Zv&cV1Dfy%}#&ose@hdLxT^Q=rRKA6w=tH-*Ix zkx`?yShSkE3YE7geJF3j6)U$!3DS7Bnm~V1CXNfvg_AxAF{k6@kAFgOrsd`ZzNwM# z<`%SnVkzH#nbY$9a^z3UDSQ1I-%{-ee%LZ5fBHo}soJmugRDQX+QRXIxaiB)hClWo z9xro{pmh-d{~=?3N~o(>h?^A?24>TV)yVB#do&7M!itJ{f+uriJ<&Pjr#B)%+jktz1O|3 z{2ix0f``DuO^o@vSP|LC%qsLL>=;4S>|o!lN>Vd_KIFV1SudkSWV0}Xz-Gx7q1B`%EZ!;f zXyF>U+2lENwIOTY`kZmP{7rbh%!|Ue_#FB1+roi0*$xSz65zHJMTk)s3Xko>caCrC zs|f*Q0VowSRW~X|xl;sZPG8pPW@FcX%!3$Sx{MjD_Cv3yOn*mT14WOCl2N2OQXd)Q z(MLjErbZHslsD+X+J=z2hZ1X9`NL=rw!ZBvIl*YQgGgE27GuZLjP#8n>c*sj+KRjK zsx|e&<^)`O<)onH$Z&ySI|#=dha@7?+9i&(&s3iRUm@mBAUk6I7h~_(oLShd?S>uO zwr$($*tTtUY}>Zev2EM9W83Jwd1~#dz1MncSFJB|{(x&%U1N?h&f{>J_2xy>B?a|7 zrStrk9Gm^v!TV=I(9bJja}IAp6n+9=x#drL#Wt1}54SY8NOs72m4`WAM)GXA!8b>? zYDgkMdziNk}vR?%q6nHck=5Qp-^Seu$l%Z3cs`f*UaUYw6S zx2!R=em3}S6xb=fMObYgpI85+=#(%fZsiU0EJ(N5nWtqgz@&7l&l_H-WoC53>5QX3 zwam-4Mq^V8)pr{-7G#8J|_#} zWe6|Z6(Jp~BvP8K2jjgFLScHD0xu95OhaM40zcP#zZ~P&kljaJ5y_R-aAh8gNM6c25HWMo9wp_+dctDFBm|t+&l<5^N(J=Q zaE%TuaGv`ypUBf<5QGl+CF#iX0OdxO7Urvxm=bB1hVa0Z(*+ykBQ03JVbCto|HhSz z9Sd<~5&p?qW4d<-C02oVy>ErmI=cDQ6plkYo`yX(yk8Yh?gxIh&%vnG1%EJ1U{L4* za~v+v&wEFdP0SzYbVsieE+B%pf>Xakx)YSL?`tva1+l`hXRqLc%VmoTr(AqISP%a) zP84TE!+2?ZG&=UP#GXls0PrIISTa7Zb~%m5mc(m13OXX35}B!6cq{hGk*}3lX&t!? zOpymDPk->E9I+c)q)d`unI#bbQ#9a2wIhUW&K|ki>-DXSBVqF>XOHWYtpT(+`*LNa zjs#8+A`z68hCOUU*)LE({%3#;pd2cH{Nrh8rUwGz{@(>(|F7uuf5{q}wP1Wyeq;?j zQ>5<4Gi1R)0RU5fAR>9l@Juk$U_udm$Or?$fUJy369ZJ!g3p+ACyznRkN90H_=t0^y)ERj3;s{cwD%9T((#zu_-CPNCbZUDr& zgbJuZl=5I&xyAP;^okyl(v6;#19~aP_pA-fx<^>N$^cDh{d|WCX&BFzs1tv-T$)T~ zgxQ0)Le2bJVFbE;=n$IWR%u%GDozKunTck6`V+B2Oxn z`JV%GGH9)X|2WO94eRH;hk-M1a*RpfR!aNWp%vS+{lef^=)kYVHrIK4n>8%x=h0y+ zq0RS6dh7Kul`Zrb2x}BTj*RC(n3IvR@+KNx1(p26`}1?@ zl0~3CbJ}TS?XL0AK>?5hdPeX^Qh|f5He~{srn@;M=6n(-sIuma(}5(vx`B)#wsR*E zg3<14wC|M{EFp&3>>A-;3Eer&$lyepblf8t4koUwBH#!D>9fGTDs8B4r@9Q$T5gfjBe zQA8d)LmDpCoM;hr-hvhsMmFCNkUo}Ts)-QJ<0j^vZkCsX3rl7}I=wArte;4wU`Mj; zoZrBH!uSVWu+)bScY#Q6IFH!p_d2Y-+vN0q-|ei*?hrH9=-C2FXVzYtZ-a>!zh}=V zcElm>XybZc)9GjH;SCyaV8v|g0U+avuM_sqg59wWsxQ)-9Ce^3MUXqE8_q{#P(%nK z?G;nd*h0fzC=Lh3r((=cTWP+@ucg&>`+E)uPEHV%Ecjob@?UCiE~!Xd?JrArOnt0xYv!V1jPanbj4H zdDS$Zx9zp_n>%lqnUrD2CYTx3-Q0yhr*U8l@au$8Ynrm^|G?YoqyLhgoXQa|hf_Pe zomy^mM5QvynwY_}@C(oI5xl%SMnKcVQ$s0|Xy}&d-zV%MON}Iek(;k-n>u0J$WNrd z^m9L&sAz;Q6^p1`MwGq1lRt$2cxQG$oN~;8(Sua z_yw~87i zaYCSZ4iFQ2c7;pM{dyB5lj-!DAjb{!Y|x`h?L2CJSC{;f88T3M6W)x918-F6Ys3CB ztm;#D3mp(ILo(iA^P>1ZCDOc#ejfcA$&9BIk`_-!WfKm?#w!wvF>;9!r#SPB`8^MV zy)DRLwI$KohyNdWLy`X61*1xg@xFJ=h@<#)!ctq_i&arSRZhR{lY0eFy|4hT_lZuBYvp#k7_(b`fSFc zOGWMcsRa8^qCm-A*_&C2{roG7k6_W}sas5Le$bC!qLs?0bHx2NN7j18rS_-T>Hjv* z@G%}uO+p7Bn6>=$?2!R5Som=427OFgT*X#A>Ws&n1sWpibJ~DCF+vv?Cxjy6`XpFt zWs1t>?V&vB#@LztC=rHUtb7P|9*n*^#?(@zZ=;+#Mk81~qP$m(=pWTtJ~F+x4d}1FO8Mdb<1aVz$^_OIOvfR=+J#rsfAqrIe;LSyGuw!Y zI%P1b@E;%B^l^V}eB}zh`T8jW)LH)l3xr3bEGBicolOGwVBz)+jAet8+#hb*A5R$>lyK(lPEU`NO*Y z_6a#xdV26&Lj1!O9xLStmFNOLD)Mwk3Ar)pw8D9CJ^9`Xg4T)~j+PXJbYW%tywx2~ ziOy6u!?IPr#5hV_8Os4KDig5iaA4tNAs&y%54BZWt7I{J{XhzSSvBipU;9PJBk9;C zm`{bvKM0yi=SDO}cJjRypj&XMTjV!H08=ByW&i7Mp9sjfQgC(?Mq2a%PE3yB8mjCE zG^|+D;I`u{prqLctX5HP=2FkS%*^QGJVxSW;vd*m-@Z-wj8-j? z`z2f=NRMxx=xR%`#{njZg&1P}SJttkj#@wT1F&(;ez(?;<;64oG5fjqJX7sinmGIV zh$A>%+-@~4GV-tw4s4b#FHdpv3mN~2S&QwJb0WT39{_-#Nr1dET*AI*ZVB( z%iGNHrZdaqrGzP33@I}(o%6W-&;Su)mx)W)26Y2v*=@@2EJg(8I-d(H)6_TT{8nD$ zS}yFCXf5h~&1)24CyIA(WVI&h20q@~=vgNBbcAHv9tYX+7*lH(0a@5cS5R`aoy%Pz z8$C>1NLg7r>?5WXv8!hcC z;UkUsO{<{*ps+-9myG$Dj331XtF4XR#U`oW$a6Z1wU4clir@DR@z7OamS{4#vx-DH#ZKH}u8)Dm+J^h3 zJcI;^-8^TF`8tvhXU^%9piSa=quHQ3+hY3l0IM!K^#fCF(zZ+kN4A~ZDs`bNwz)T? zw)Z=%_KwZas!lTXmpP0(G#x&F-?!HpPNCuL6xC4+qwdCcFuC*4V{-2t5tV(y0>Q9i zy>NzkoWp`Y1!s?SiT6s*e=_f3-;uXzTv>vyMHu9tuZuq#Rz*+*?}X-jAR9}Nv;6;X z!hM_eWhv+bFOevncs^3{nQ3ku`1=IH7eZWnk~6k3?rL*M#y(dj(cn&IC*`mZ6kJ!1 z&vd+hk|etambwrWRSfR{XC6;Wum~=OS%~9%J~UXG`3AHa#JB0)QSSr-XM%(4_(SRw z3V^CWQv&aGjgLdf6QCwKNJ~wHp+@`*DxJ%!f7Vw7{i3KT)K7}@li7YmKhuC&xG4^1 z4;=k!T`hb+TfE6Mss5P-%UVbi3d^(WL|Q zZw}E>G70D6=;42NO3H%^?B=;bH}Toyo5gbxFXJ_Ht+Z0S5V7(L52>>ZgVB4UES_|= zxcQa1n)R7T)2vZ{sdE?Qa6Q>L10bC3vmC1inNPjUQ*aR!Tzz{u{)Ag$&balSE;F|< zAgA()Mv3*pAIJPdZoH{{rUq?l+PQM=l|AUPW7wYyd6{39u*guyDvY}KFO!wkM!`6V zunlsN>W<9ObM-$*P`X00LX3(5*&sO%DA~C{EiXf3z?eBk?nbj5Q}JT` zabSXh=#7#d-OR&J8DYu8*?_WzsH+cLOM{S;ugaKn9D5#FDWt_GNhi`o9>=b(ZUz?; zpeo41xRSzi?~YqE8IK%lL8lyAFfEOvg$FxLWpvU+t-M<2odaL7L|w20e2*u6@fTzZ zNniALPdap4E_8c+kSp5#}szX#QgIkbmQ-fKHWP@PB}7GX8Da zyw!1}h^+v^(FSU@r)3i_rs%L{E}LaV(FANYA?-mQeq#+pZvHCo-fl25TOOCNa57z< z>Zr_^!EbdOT(sPkQ%;D@V-_ireJ1s0=m&Ew ziI31KMyE0HB^+2b^h+y6Z9m;rG7SB}p;xLvT07v+@MBi9qfr41_cn!7KHj8W(Tus* z`{w(w;C(SiaZ1=hw~^^Gwu1}`!DD!HIWrLci5U|}gpCV??fT%T zkWOg@h#Fm;pi~(9uOKWONr+9?18?kjBxCR-+Xvc(EAG}$C$-Ymh>lnE-{mAT%htAUt*hE42MC4^If zMPaP|SC3aCq$x(3%C;s6C9F!7XiFyDzYOVzv$wFhXV$4$dL>fW4FmtcDqU}mbRSCs zmnS^`L_JZPNAy*q^uo0FRl^#rTrb#2F?3VV^>X_9k<2iVGB^=sPsAO0bED9Yc#h9; zC2E2&z;(Qj6A14H=)^I0OoKYmj>9OByCGE*Gp`zgBPk+t$bMVU8vS+Emf-ak%gnlH?=HVU1-i~RjF!U zXv!e{+kG)}Z|s~3m3#dDNZ`2X_Ppt=dolBLdl88P>XW6C4*>d94S3nJ-|DLCvwH0W zA?OR+`V+cUP~2LJ@nUNL;MAiqOrp#jdXfmsZuuaRZFYI_2=F|gls{yDb-7?(;0 zU_MoSc56k1gw&J;gCW2Dj*%dz+zX~a(_mn9$|*I%D+9$ess)+g4*(83oPqVmQbe&s zK7r96?GDM&MQFE-%LD*Szs5gA9RL=rQFLMO*n_Wwc8-8uxkIzW=n`n$0d2Pf6D$M; z1NeXbyRwE#rA{p4A&Z#%0rZb;4jqA}oP!}p^woqb4TWKCe5p(e zL50p{hblj56%LY)xKcIZ!&#O5vy9r46FE;k=fk^Y%VZmZ~83yp)Hm60mhZ5%k9gq=$6ff5Lk&|{7; z%3xlS)5TSicBfIH;4^M*2G{) zjiIr!pUAP9K2_n6axkGA@Q#tj0eW65+Vagrm`j7Me$%Y3%$bT*ovBuQ%0tw@)-BB< z`Dt>ZV7G6qi`al;++y!cHTdQ|ns#LNLd*87SY-{pbHvx85!->mGT7i_!!Svp`sCj21L{c_4%zPDQM`9$!}1z(>bMC2<7P`kW5XBiPLVjG{t~v#k#yoWegp0 z;$Y^Oq4gHKmQ%nfVR|e?oaHQLsW!6v$=*LcchUsrd~9;FTMk`tSOhbg?`oOar7cK8 z{{4MP0$H>ep*z6F`eOc7u9Cn+X)HN{KT*binp~m5iB7jAzbau>iqJ-c#{Chox)I_q zXK*(rd5G48r7J{-43|`6`|l5_Ywol(iOMj;y4$%ha3aymamw^=8n*@lx{YB7nQnwp za6VXcIO0d;O-33lSauWSmC|OQZ3muRmJV|cbZ3_x%~w5uR;udGDO9*Q-x^=zQ}r)_vaD^4(k=8UTz_+@W&DzLP``p`6o!C_>>J%Ye6P^a`TnIS z7)XIZN?H|2(c*37w921&2X3}HJ)9>+i-&V%T0s*V>xilMW}NXI7w=%vW`_tARg~^%d9{a~+uoRRWrU=M zA8vgj6{+?7F_iL^?uffIhGlM*sCk(|b0gCLN6i03hx7J_A<;DDSZ9D__%otSx`c;m z?x=&wkzQnC3`kWfQ=m5)bj?LOWSR{s&`Y30U!PJVtqPi0aG{f1Fm-X0=vZjkrj(Ra zDzL83phCuF!`-_wiEAQP_nMji$qrxKdO^2Q@ARpxlo^stK*FOr_AHp}&p<)#gmYB9 zI{#1%u!sB@1H)p=-DB{np3ZV?qPawRvFwF8?FjoyHKbr)ivsot0ZGji+ieT>E-HUZ zt3gFd07bv(XAC8JpY7;d2Z9F?fQ6ByI^kA?$kXHO6bvUOs}S;J3K4O8K#vQ zC0-IUkR5Xh7>VH3NTtf!Jt{a01-hM%#G5KT2>xjef7}6MaS%s0N!?T14alf&O+CL8 zz8JNM4fA*L*m~M>RzEpesXNj{-R1YE?o#s)VYUDUssH^V5T`83&wq)487*wdNN3pD1+)8mPg_dUoex-0{=_5Y8yWhzCLtuOLM*H@kt$5}Arx=n3 zKtmfasC)(KDIdt&o27p2qoV!?^AL9+OV0q#Y}zBt#sk1(<92(wkd59TljY zwB>hmx`EV8rsJcbVyv%MyxN>Mi-E@kv`RJPFsO;lQt_3S*t~Julj;^)=bKFtW z5nqkO8DT|ysM#MZ#fSMWPu)w{e-A}9?tN1*Ad{XQSqqE;B3oe>pFA$lr8bStnQneJ zA2V-uYtH26>Od!$7W;AUI>ltL&X?lOin}$f=|!WG^^NKt73;vP72s#Ko9RhTxhX6) z)OI|RjIZmsRWvDhWHnEuU=nAirHlk5oHz;3Or&jtD>mf$^b$E*N{BPACc(tEoO$cV zw?~^SVP0j}{TZfkw>C@SWasAf@pt$&OsMNOjaTJb=PwCjU4KpF%ayuX)U?Tc##MJ zJ^6tV{=VA*LwmQDUTTy6@?6P765ktAXxt3ZZfblX*hvpvqW=MAm10Kwl zbZ(-U&7mBRbEIhj({IQ;0P4(awleJ}YVh~@r0R_QKt(s-x}Kgeks&S085qkH`+KGP zL3DBsJ2Q2aot;)^S!r`)XKrV1t0{71@_Z9A(Gs*`COhIn<*Pk1#O+o-`h-9d_bmWqdvf3ft;U@Jo zWM*KhBKsI8CKoFK){;h|*zu9{eP^Gfe5qrD!rxW)GgE5)SPiO%@c1RN9^7~$x|YUp zBHuQuLVxU)u5s$~q3R5?8y803xtE00?}Tj0Db%CsEfb=N-2CHK6DDFIM`^lN{;*FA z0zzh6w>bg~afjfs*Ib8n_nz5iu~g;KmM__a$^M{a(nqU^$3tAP4teFVzyOiPiZY{Y zK0&*QU%_%78uv8z1L5XyK$iYlPQQ}<+Rq-V4>1m89mA}^3W2k{Rm#c&FywAWTyX@n z%&>>ixI9y1-g^YJH#Wbxdha;!E?5~S=o{xNoi=5{@cD2VPd$X$28kQVxC9m2_Lg~? z%kyqNY?IudukJAW8|rFSQS;<+dIpjF5ubu88~Dgd=%o7b=ZO`SZhUwyBN)6v3s65G z#IYoZ0*xwxnsf30-25%cY$u<5GXZ*o)7ql{d(ByE9QMyRFo2R6B{M;$Jc=;18C%hcLi|Yx9{q4~czYo4J%Km*p=`1LV+AqJnhQ!lxOiUg&)Cw( zY{RH19N#`gEj-S?VnP6XD|Bzm`?KowDc zKl}Q*;ZKeBAJ!Gw(0^a9nr+pXb`oU_OgqI?ajXAnHIW+pK>)6B`z>)S(_0*hSgW$* zG(%_@Yt98Kg#YpOXJl!m$@=%DQyVw_NwrB#O)T;`SOAg$<_T?RWd)7eOBM;kb|B&K z(7TC8IDoW!cZ@(jXax!64?1#x#qostPlGVmkZ~C0EsLbt+EPVKzSvxE9lZ~toTQ)I z>S5RfuDVRE8C{SeTY46R6V6Yv%EOtjAmLO~1sycvA1=##-#UUQ5(5U3|1ij$g*8() zyg5+#PR1<>h#&e>&LgijkiaNN?l(0OQ#2+QaMr{-Vco`=+q00LvY02V$-DEDkdUjM z6YKsIACp4m!}gnY0b+rc+oZOTSR4GYQynBgk+Rl}Y_(ID2~$mXk3PKr_QLR$9kaoR zBsn>H3*^IgxBi66T@O}#*no8~qCfrOUHx*F#$%xylsx=I4Kpep#8}5@`1LjNE`x^r zHu7&`1XFrb;l>jy>a&m~fo!eYJT8ixAnq4JZCFsm8WoQLuRzPd5-Q`OG`tDp(Pm$x z5sYk{(E6{(A&6!0l@_v<7A#2Zg6!hLwv;v%>UN4^B}Ls+dIt=<5NV|2EGIO`gla&WtTlk!p`=*8|?m8?TG%azlnn(h|7LNm8>UfQsJg# z4`+CWbQ{3IZCwbll;*x^DJi&`N;Jh=5HMrE_B)QvbqQ#z)sS`88(FDhN`(6b1K&9& zx11NN~}Diagiapi&ncEMYUZtgw*Qy5O|bNKYgB znY8npJHo4br>E&xX3A5C4Lo6EuYwNV4a8v85-pX?#XZF`{Jnj|2(s$~X#Ka2m@pmj z9It&)#cf(m+!Ht3!PRW*=#N#e_pcD@L3yb08UdnrHe`RYt}IR{JW1aee+T_KXCTSb z0jAf08g}4~4ok`h`5h`gw+D6#MBqU+aEhwRJ>J*};APyCz8%a#jleHHeW3034ax78 zP~07~s-?$Jn!JLp8z;b}f1y4JOaUk0PRtMxbu<^sJ z?%_Dl`GKqg${d1JfHmm)&}*bQ zuvsOHHVMCj_|M4KsJGz|q+SLF7lP__yF3W>k@mK(OiAz8z-MZD#n0ACTKs{6D3mX z;0n@+2PrFO1AZMKj6m6~n6RilCFXMRbh=~I{Y#)WSbB=64O!r{6P5kJd?GP4P2qj;$p~}5 z|94IDZ++Xg=$6)}4f&a-i(TL+=RdijhoGhhN6gMJu=AgI!5});yU7~$pa`=~62K74 z4^*KWrC66yF;s$Feg~YybdA69R6Ct?u;|f?VhXzxXJk&RcwR?~+ZLnEa+-tDjpFc% zs0P&(_zSWo5x|6nFK@5`rTl{EJhKFsd}HEagaKRRhHyssMMmVikdk_n8;EyjqBW_s z4>mcTjC?kp5%CDhT!NsCY$?Q?fF6HtKP_1--@oYCcak_K#m5*6648=e;|SDP>!5-jVT`B;oXbSbT`9*RP|{=)x^h4k%S{PF!i z`gR8(oF`^K!VDNtARwXt-PYHC_3Zxtkl`hXv%d!f5l4DMgiF-!7|#2HgRsko$n_Kv z5W<9sY{E({?-+@O$*z)fNUMqRtmN28bE#xnLC?2idpoxJR&G%q zyKBz*xiUyFx9>RI`S)vhL4(6KP8o$Sm(_OuA^6O18+)64Y$(J7+$i_^Dv@>WP0@xoHq89{_36f zfS1tq1!{(TSQ?}$ED%XybE)#k{l7O~fi~zEHo<{_$bZ)N#Qt}OC~9hE=xX!d`mg`P zanh{rt&Spr`VChXGf9jJCZ=80Vno=tp;d)GbibqllA1Pb<+x63$Tn`rfs@>GIoU(- z{Po(ZpWngpR_@)cgJRzyKtFV2%}c&=rhf zqGG@rj7c!+W*{(Ps?!(c!>t%4B|Dxz5*;1oAyvAq36=Kx`Zsve=OZ%OtBcqeUfsID zP){7s15Ye7+7oqWuI+#0e zy^&bjF(RjqH(7cNXFlYll({*muxMQ>*BVzq!fV9^rP|2mJFkGz+-ACNCEbWeQX6Zo z%X9H=zG048`KzX_UqQgC6i$9GnLV#KWkgYCHIXIL_`bbYJEA$^RkaGlo$FY?Z z4dB4URRoV2y~$>QP87Fkp;lZFzCzoOju98CoJvHdfqU}VH)YF!*h{89Bh_o3NtfPy zPI^6auwo~+C-=UK&$oCsR*EHIWvg0*dJFRO_G&pmqK@A16&`<4saVRHbK}}nQb~ql zNotj85vrCB%1v&yX#tj28c^zt5^BfgNI&O*NpEihig28q+P-tsAwGdFRQz~h$>5!@ zz)eMXwnG(Kq&-|vEdz6aEu%?}u)748rJb5!Amfn&i2I&T{>C3D)J85-H5&C=IeqohwrydM}hd5?O`NU9%hE4jXUkFARy00(ZrPY2bC+7;uD zTFyD)K<>duKlS6rg5DYShJ|2{3JiR4%)0mp2u48j363N4Dcr;4KZ?p@g0*R8I#g~D zOj~@-j`|QDNQ0i)iw&+r^@*`Z^@+Mc^@$@u{S41*F>xC|Wz2>pGz=<|BhF5ynZNUa zGg|v598;9`9v$DPMPe{0pSRhR9L4GwKv-=Ii)zgy=>ko?{v>tmr^p;Ukx~1OxAtTm ztaXrzy{D{QhF&QW*~+~yDGSeG+yrG&{=NrrZleQ90O4+AN7>;?#f}UQm-eiQ zHUiTbM`J8kJTr%1>{4wpeU_$2R2O+7A*AvChI166+dt%|Qk9i)xc^DjOX?;BTGDY{ zWlDwSbQoLIG8wL=l5|Wyvb$kC^=L33I1t)1eJcG(P>b#dxZAUd4wYu}U1IXVFz9u2#q1H>;yc-}P?hupHtD0V?dn z3~B=op;N(U3sDu3JKLmHxtKHDYAdC**)y1qJ&qN=p&f62@7hrDK0E6Xl|-5N;E(qA zQklB>4S{uTqH2BD$f^;8k150FOe#NsWTfSNM*!r2fBwe2P$ID?4XYPTj;Je=Zh+l> z(F^c2*92oF*lZ7*%2aLs-v;Wi^Ie6;7dLq11g>B|v{CpEXb=jpY6>rc+(Ln|t#ISw z-zA8iphSiK2MOM5{qaHBB3IPOYuM*zBzRnxCf4P7CELG3(RdfIG{p}1SXCEIRzVqO zXNXwiV{vc&DOl}K9N+n5`rw9qK`0_Vk`)?2W(!7Jo70}YQaENjNVc zzLV==E%b(ab`?bo5sq#t;ZnGG-PD6qXjUluNJtBDT$6b^MW3@#W+MN={P){qIA{1k z?xzG01OWo#{omal%BIGyPL?j7q_T!yo|5)9CZ$~0 zdla%26^jQW5*si=h@3lUgXjKxJ<&$9)|-eQ9Ox$~K;6-)$+ro+#qzS{JPkXLsD zG^4;4FQhq`8UQw-S=>-#C?OIeWY4O7wQ2Sh3s`$r-Q737h7D$mSB8d1XuEi+K_Z}; zhru&!4o0EInySTOW;+|-3ngm*GiNBd{FQ4!xwe2Z?XZMaIo(OmM7HVPwh53He@m1F z2jwzkwE4&DeY>5j0Bbh*5M0^m!;GZT`O|*f$wSm!w2tj$HE;S<;M&&{*M1PVq7dOf zBLB;X8LOsYEF3iB@FrjmJ8V&R=))+ZjP@$eLMN z?SGE(-;YU+F~=$z5(sGY=aT>Nya3rdnKKwV7#drcGFVu;{XC2fT?}pP%^7SB9h^)} zT#Zc`{xf0ykL2b5HObbfTe_pHV)>0|i6=sUnxPSi+6awcf=g01P(d+~Q#Qo(!vJTs z8C%jKB$y3NjTyjFX>PD;w>HTY$ZQnXbe6zL(Fj{4)i%jx`Pm3OKW6(~Ay_Eq&*mhU zLebz#ddK>@p3HRe-#qlFd%a9(+k>t}3rN%>wL`+D8#WOV^ebvGUqXWHG>{Y_3Q6)t zAEaxy#@DCq}4|j*M|j65+AcL z&rG~V2ESDVUZuC=FJm3_7HNhCTbv-d>d2wu@@3UGMZ57TdM%R^wf%T(&BMU-yDS9* zd$(n+k&oTZLsxXiVl341xD$`0jLR~V8bK>dqD~eHcCi+1p|ZH;Xp$5jtgK{ZqR$R= zlq!qNre%rCj_K%hB`qWPt|}X~XRW@X1N~04Gi}XAo1dN9kbmuS7+Ki&k7Hy*OeGXE zS+%GZuaq2b!n1ioXfBVDdf~}%FuR~jWvmO&oT@q7k+YGW^F;u&Y3GOg3^mdlK1@r9 zx}QN6Ni=6eK-d7Ax(4$~X=a{Ve29^C4V8!xk3f?iW}nX$saCn^?$98)?_;JAYx!c5Gs5z0a&O%vFfBAFyt7s}O zmS)IDZx}JgHe?&h7yJ=;+uaj0viKc1TUbK7=(43=u!NZvrVVd{rGL`@tkS|NPttAH zFKuab*Z`ZdXkS!cW`M#z=^@q6aVrN7wMBGb3f_4OU1I(e7HALHL{9Y)eE3U;>I0z+ z-4k^~^~&#;zZd4A+8=jo3|5g$6LKuHejVldoq0gqviCF8K}f!8MjfJEML!2D~(@-Ej#d#p7gGsea0#D zG>*lR$rvfmHS8SW^)MzQ9{F2>zm6DnIhqcta$E5v+MAD9PK%?>z$-WlP6_6yK=GV>!;M*Yti zTy~?Y}OSO6b#E^O^3Bv>@BCBGviET^=FNV zjA>MBT!k8gOaT^EP0bm%yPmP-T!-qTI2{i84(mngQ{1`u#+TOMB%1-xl4D-F%Nl+3 z8rcp*EJYF{k|6yA)DghYD3^$EqL;Rn!PLQOQP;Gv4tF+zRH?W(=jE}8G=4qABUP)Un5S=k**#8%gwBjc;DKJZVCdoad%-IXYouW+IstX z>?j9nW%~gBBq>;%%A*^H(8yAArq~gqAMMVnsL{7A75rG{%DTh&_@BP|#qTPp)NE#F z&zJo+_%|z`oKfk2q3dG=?6e(iN^LL>vss}R==@}_R1u3$Cqy|1rSR@fa2;VZ4jAgg zD1B-^!Fh%@gWJi4PTPc^;-ZrB1^m<;^yRFmv^dm8fR(wW zW59Alby#q;Bgm=7bn}~Ity$qW+oDgvMjJ6|xC8JOZ4g{rVNyGvW%B{%jskl5Ls6$g zG^YU79)!BWIK2W`RfE)%s3G9_GZVxyMR%wudiQTc(_hawh+ixH=z%P>JVPPOsFqdE;DaeSxNb+z|^ax>LsWmYlDg5z16Y?opjcRS% zW(!@b8rLcqRhezA)?m@|1*>JNZEb6`nij1W-zxv<@9dWyS+m46M&RC??>onnosXIJ zldZ;GZ$QNN_6v|d)dVBL%!nPw%mBV+&(q$Y(_{R88PP}x#fe)5a#K&rES0yt`@}3> z{QJaGrPwK0q;egEqj?3aV;TE&=|NR&Kp`kE=5Gk-A<4JXq#dD!=tO|USxmxCKQ-Py zaW4eKVgWNg%`~SabTJmgG?*oMrOIi)C^a(k@P80idYQ5%delXO#auciRC>t^DsOJV zX|cshDY5rP&~ix`MXKE5zmBL?Js4w3Wt^+S)N>0K9Pw2Y64K0|f0sb8oO&S4jEOxQxd*T=(^on!3cu&FM(L=~BsrUyyW(rf$^@?{ zA<&7mVn{4FS)yXpAhy@4oS7ih6e@!`G>82r+m?AkmknZ#x$k$F$4<#qu|mHC=PfU5 z#y@TPotY1N;!z$V97vF=R~5C=Nx5G@Ieh4x#rnf}7RNEA$x_*D$GLp)zd- z8M$o-yf=%BR8ync9_2AE^4l?_qYD$pDySuO0M&UK9302*AX{jZOBe-c5aL1yXQE5Q zh=bG+9W~&RMQ*M#j$`sYiNPAKdxM=TdXlu zKwwK*9wF}X3IUy^^RgbYpw%{E4JL^}FE~rW+XDeGzg4b6#ApIi)%b|J&xTp3pFwn( zg1!;&6I+z6$NhGC>NnN7{=SD7FK2Y73Qi@i27bj$%4t_O)1_)m7bk6EmLWxAWf{pD z0{j%p7mF^aJ+D$%(!2AeNbNAXX+tBHZ|a7g&60)vJhHpmzflICR14S)2lo5t(F>>H zF8}m9Bz5n${3@a~RZP->&qB5rz^s9pWe>EWlyW(SC~SasTv)0w*-)izE!Er5w5S{N zww&jfzCJSLoxVdo-*}xVh0S#ox36av=d9KThsh0!g<4+(EfBQ%WhiFog}>g5^dwSb z0Y&7j);1rXjs%IdY)khfUi=$3QDpMdAqZcmPy&Rxg(BiRwe-NjnVk)@hFEe((yW@R z?>;*r7VP9#p!j`muxgHGlENv{cXKHe%3?AUz<(*m#xBO*Z17t?-`Kh~<_zv)O+Z$B zYKTp&3C<*znBPdn`_z-0CZL-gni}!&s60dX`JAG3N@dg(g>r4POC{n> zm6a`EOyWs&8-D6-11QT8*%{&2RJA8U>Hj+^Mj z(kdWWV9cwNTys7*H32qcIKfprPIwA0m+<6oqA`k$f`zRwWI`<_jCXU~66s;FQ7Rqj zvnD?2hoF##o*U1!7qBzM2*>vZa`_RH_a4)mG#+ZOxUY%PlVvg4TBr?x`t_wt z8cPXRNFEfJq#g}+wRv}O!W6l%lFX_vxgOk^V+|a;C{|1*>U!9Z8cb<${Xw-3(ZRR| zvB9Ud9~Y*T3)PqGU~^lna;iXQ zJ~8}r_v4!ebfcZ#91%ESdP8+-u69fJN%RW#{j3=hPgJZKl22Tmq}f`=_D;^{EoRMv zJlZ9m|Mbwb+Aaxis$VQsMBf~4mCPd~4&0sx#En}otKjgQ|BNLl30Hd))dR28RxK7> z_=m$U7mq{%Uu5y;OmR@Mj|u@iNFsj06o)^KA@);FvE)t_JMt`hpwIuV@ z3h3fyYo>`*nx3Nh>c=dJuVLC87>!p@+Ye$2W8A4rnx4Y>zO!8%fq`Qy$!0c25n>xU>)Odd?}fhn0!|_~z5xeWob0cz>KnW~LZNy)0M+X`_ zHq?I?9r<*JQrbIEK2_klEGw1Ic9aiS@6&hN>rOUkiBg(+XHKe%tg5ScjWTqoT0yge zPAwMlbL^f0TS&={GyZ{Ax7D}9psEta^Du3@FEVhe&1->tUH!m~oW?4xID}~!!w>h! zt(1(5+X?z;$X`S0E3ujiGUp28J4U-vF>=Tdw-Zl1-WyI?lmL*(5>Fu*mj&~pjT*`7 znV4*BmS16*JUlxgKN;QaZ^V@t9xNJG_DaLbkd7;tt}~g|YtSF+D2t`TQ1KMD;ywUa zev$61$+RQxOP!+0EN=M@hU<2G@D!bXt#)TG0`2PJaoT@gQFhUxM!4~mwT=Oi-8-%^ z=&)na32O3WH+?LRqoW2ZIG9T?53}=hoke9zti^cf8P`4Jt=x%HTqL(}c&8_HI%>^U z?)vgG7vQPOQ<`%I2KpTZHY*pQ?#r!yFjs;|j|MV3!EWz2u1(2~e>0on9b>q7)Qu|h z)yex-^@UXISWVN(xFZe07wG1XBj=YF@vR-LjgN~ut2(Q)zW+H@GPg;4$!NyINc-6= ze2OmBo$-873}KoSH776F5;BX7EoxjB0Ew#M+w0&+lNSP?MWB zi3|EIBAl9ZrtMuJB7Exyn~r%eaN5e2sh6;(H+5r~gFx@f)827mZTQgWCOB*b_ zhsWu)kHLRr&oJo%Y0dT|^5`TezJ8{nrORy~(9Bw$b|iV2p{2W# zG~;WJkn4}1w74@i@f3LUFGG9s@{REAp9iHc_PzF?ZitRsk0qGEvb3XmUt>)M6Z6Bm zk^I7SsrEE_x&J^;Za@4DX@3T~$-Hp$7V@*&SBY0*eS7{IYp5qNK1-HH)0pck#s`j@ z*Xi#5L$qz^t6e|8>s zd#(V8CI^7YYegH=b!BUnSsoxcTxzwU-eLKEU*0UY`b94 zuE`naqVXne)aG*&Dv2`skLvp^_+%dCmVTA{&=1x&*pXqSWPn)4Rw#;h6hAQ=kAJw2ua!kBm1(Fxm@C7nAz$I3=(9@q4;VE}0)?sh00e z{&o(MhT!qbycW98{+kJ^TY)aov-;D;rH{o4L)REPGq2tCuwL(&COEv_+`P{;9yd_D ze>1O^bk{N&jUjt;a-V!WloN+0oLIYIpBqhBn4@MaAp_=YqK)8g#z4DIGO?l{Gt2D^ zY+-OmN#Qg{e$jEIsp|D?p_ZZ?>IDn#E9>?^-qnlmBaF7Jc*Xb@oy{xtjZmS-SHChUp;W)~{>WEK=P!rpK1^6Zvf+$8q)nS?p;%~)f?xfX z{*^0iqqu-4rOgby*8bG#jfg&9K&r1FL8jL9!@5Naj?NAH&@xK?YK$hu#)4Xl`LTiT zTHx_S)cN29#s|&iDVnhE3`rx@aegQ0+dY1T;my>I(2xh)+t#MB zn~^+Z7SEsQ%Z5D=3L(5=P+ROezX)#b;nP{ebiymKDUXmzSAT;{=AXPsZP*XT_a=i{ z%9uLMN_m#jTj{Hzf3dYDi$%SUq+>>bQnE2HUPlTsqwjkB%mU2313C0a^jQJu`;Unq z^LcSuMTFT{=^DB%*TxFoY2pI}W0S1y`$WWJURS{&{KFyGHarhfQ)g=VHn`hV=j#%u zG{6qIWZTV^f1^hVc6a8fZbrp`_cFMruNiD4@anQ4#fQqGy{@%8&jWtH2xFP9`;vP% zl2O6f$}I=>+<_qlz8KPLVskL5udC*l=sCydu#%IQf28Hf+W}&|Z-uUZERzH2S|+&v?2O;`i)*2yXvedZh~t z`nEvc(!Rb3Q4ftxI=_fjj+ReSzsN!FyPEU&Dsddh9=rLK<^wIt70Q9LgnRY8l^AnQ zrH6&aY&dfV+)sI;RPmk0gV)dhT&K1>X|!nwK+@KxdOw+WPqZp~kQnJ~^_bVc^L(0G zjW^s12c=?&w+_?G8DoHS)t|#<)9b?m;!!7iahDYj?pxvph>vycv?5%4y?`2IxfGGJ9JqCHq9u5(FN3K|PUV;-7Lz=t#asmXJ3Lv_=)e)LcH z3D%OmZR34Vv-*@@XTVEy6`c!nF_Anh5`P`OL&p6;@pCMb2HPw4%1=)mxRIc?8u@1@ za$yaV(M;p9Po|{;`O!WvJOGKD<;nqOIhqH9fJZCj)Ci<4Sb7Pc&Ny@bfEygL86Z~6 z{x)Vz_2J79wr&h*S`ja!coeC$sod1sK4ZEZ0b)~_xsch}YF5sJ)L(JqwR!h@xV;E%TRm8YdPt1|L!WU2XUu{lNi`gPAiX=LE}HTSqFf zk=CxFP+I@ha_L?PhIkXYFGy?cn6@ z_TPA&(T4WIQ^)`M=h`!4wm>Ss$dHnhw&0LNp(`^dPhDcYQ6it*at_}j$FXR(F>}+@ zMGYTCWb{*5OdOmW>}NFO28e@d(Qi$4G)1uyT*YoiG~|k}{|ZlM+a*a0yga@)Iqr17 z?ykP~rI#D{UOE4WwyqDe*s9(a1t^N5-mKpJ5TFYub@5U{AWR&R)PKnUdjW)n-$BFY z4=Z!?QrtE4lj2b9E5CkHa}R^Z*=A5oJROFFjWH2P`|0!sltINGLc=Q_G&A@yV7k2! zM1FV&GNK(sb5Y-s!yf}yLm8(|HIMF6@OrC^IeIA?^BD$Lf$-dZs(ZR1yDLhew^E~X zRu#jZ$9FX;e|>R!;|Z;-`dv|O!%>bTaRTUs-+(p!ifnZ0DgwXrCmjWozk}Oe217I* zA4mbj;f5IdnlJ9i0m0?q0knn&2|xN!UwXm>_u^5$6qBa%!9R=k>tA$P5X9}g#IdRt7&HCE8mzK>XY@orbYYNbHXPEM$u0y)tl**+uw(P)X(suF?Seus5 zpfX_A9l_dgk!5=+F&<2|HBuVHkU0j!`ZgaubRoU(aG6PwF7BVO_9NEUFFAYUr$<>p zR*o%HnYM6ESL!OTRb2-;6d|CCSCNt4E-NokiH507f@rOQx{WAktahTwe*SN9`uwPTHSB?71-H?I zJ#`<0gM?Xk#)h(c6J;+Z3pWlNI=X31g7`UunG^?BT<+6~nvd4_4$qdLa7m+=(*nF( zo;g__D=9Ob@p%6#GwG>gJ!*0qiGP$>I!s6&4y(Dj)`jdd9U=uv#x*r_VOiRqTzatN zM|nM(O)k0-vY7zl1Jvd!VZ$_0Fe9$+vP8i4u;cZ8t--!Dscgj?aoV( zQEmu2>T=8!O?dNwkp&rzXStmGYx7{-8pShDVgr9@|5-AbkS-Y-2e->~1>A%%LhvOD ziw*3m6&NolOMDuyv<&vuHFg9`jhCa$X$PBXwfYdM+sovIPM{{Z0V)?cz;4f5n2ai1 zY0beMJXXtbm@MnL4@Nkg<|7}h$`)NyunTixL$n9EZ=yA+TIoDtwp603;Cm%gkc<0d zu^;|sWB62{0yvqQ%S53P0uK6li7NB{BidBf?`36D@=D{cRjp}VNyv{NigUKiGAHAC zYns$EwpbbHhnyv)K@-R-nP(dCl~59Xi5T{RbDq?BxLk{*Y$p3Zp6-be^OVu#nw+@` z#f#uG3+HfIwTqc*b&GgeQ{sx{%~k~IqPeqWuOZqpm1-eNcLW#;8V3~$u;W9xw-GLH zqWt`Y1t=`b!q8QJDwC?9Si{0*vJ7bdP(L_&&<%r423Kj)BVN-I9%(y>Yp;Z=(TC&Y zc3+`r_#`Tv{xPXrfVf?_}1)P{n^FY$-bYtpYL6b$Sd_3i`7^Ci>b4<9&(o@c zs|lXVv4|mjy=4K%UZ|jJovpBG4OwV|S2=hrQJ%}v?K$sIxgI)u;3$Oy-6>TbZ5eQAH8HOHiO~7%vFuW{jKG(7PyqatB==DSafEbp6k9&lZEcXivmp4WDRFn{z$A}&PnfB{6^0?wdz{<|(&)I; zVJc7`Tak*an%Cq~Ke9wjZaX2gZheYjVbVXgZB5yX&PGAP7I7l{!Rc~46`8ZX=T!7S zDKWExah2qaWzH=BoJ(@-jy$gZQ*rgGBOjWV4+D(75i|`qdBY|;!nA98iL@6qg_}v^zc-^)ZhxN^d=H5c%>m7M_-pz|PyJHSxvugNKVsVy1 zW7hH0-U~Vs_^ixwD&%d6meBa8 z$>Of{9^t=99m;F=u%d>1CR~V$UBDfy3Y%1bnUuq07-@ht8-xCqZoT??zHOCX zoG~fl0;0(v`cSm@dwrEbw|`piv~(jTI4o%5I_dDkt}?N9_N3|;e(gBOO~P+Gj%?KD z1M7c1`1Z1Vs5WeQ-v1ajNAKf?iU+J#2jg(_u|>b!0ddK!Xy?tF626g69%)ZLFM{~W zaTIKzL^%|QUwpKChO`@A=B#twA7lY1ngo#ehtddbR74#BtZTjtjvs9%xPq{a_`Fy$ zB&$GH$vXAXMAhZ^En$$Xx4mH4)+nh=_=wy!@QQT@z3~vn18)LBMvahXGN@ZikB*NV zO>{|uulrZce!+Rf)w0Q?!O74i#9Bge=d)lwO6r^$Dsvf2X2OZ~t+L=AfEX}0eNQ)v zn#n6InIe#a5z-#o#lQ@jqH&!=754R>Mp2~xO0tph7K#GVXrazJ!w*l@&-m`ilJ~SuVCEyKdNZcu$)=GpPc(1d^)*x8VY#juOYnkM~_L`&N+?K z#~75(b-Gl0=XLLmMn9T8`D0iim1f;eQ>J4NYlcs`PDY%coTq<@FwpJg75>rp0wI_% zFV0a%$`dP9#1&d*@a)y)jSKx(@r2$Hy`_W4?sBKmbc*rn*3R(r%~=62dK=u~AK2{7 zKpbg#9R2_cr)^tED4I+$SK8z$3-0`}2fPutY}=rO+j@lhO%Ruc#;cgSpr3>Dl4cFX zh+;$FLuDM!Iv}=1SYZ8Q!5?}00EqJG&;b00hm=b-ATTY#*)?-u1?R*B-bg?kHzGS9 zI)!zLR3}Urw+E9mtb0dSHwX*BVZ6Vf)|2Jtl6wW@xoru#I+HH%QRoC_TeCY9@^tHl zj@xHUd!bLyh#{M%GtV)^y(Mc+i`&6HcLTp{U}GA43ytWITC z%U{dOUC(^f!q;gklQ@O;>qaT;`=G(-Q9Gsf>t2foR)g2=3!Xf}=z*;EhRkqBD>jb4 ziV6vXI8I504FTdx_sxis*=Y_K3u)r=bg=iU*LF`|VXDlHYBT}|BAvd|)^}hGL^(qG z5>&JDp$9nmqr^q@Znl(4`9r|2G}5b%pKMoEZm_dq?kvBzyq#+V<3@`s|71*o{Ej7Z z!aTj+&WQA7hI}@rzd2R%`VG4fJeg{Y;Yu&6H%GCoG^EhFJz7MIDLmuQzo{+_`@PA& z-uBFcrSD))nn6zgTW9Uf-9{(HPHNC^=M zUVES~RKr&r)-o3YL?q@G9v-u^na!7meZM}y8e-I8O&i7%$GTZyC+TRKOJb78hWH4M z8>g9&ql8@AiG!5hcEW4nA)e9-0XM$J!#x!P+j--}pUe=|xG(z1Vk#kU$Je3()42v@ z`j*dRUqQsTxAk3oB}o&2;bVibO3A=S9vfYRHhP;~j0qE(n2+C{!?-uC`p9J352cT0 zEu;O_mUD0U(^S_w!2?>)n?Z#w^!&hH&n2`00>@5~nf-Pt3I&sS9%-EUK(eseHbOVN zo#R^2eY|l~7J35b)mtvfZrliudh&`^!U&_O&1dN!)uctWfe?$!I$2U=GYu9L<{l%O z_${ndGz&!mGILEj<&=XL*Ac3l3F*0ihx+J3ejA83q6 zQP)2R9tHbY!b-J}k9C$2VJ(>Yay~p~Rexu$N5LywJ(t2cF}J=uaP=y{@+3m2?c0br zREOn%bR|=V_)slSDDnj1{6oXYDAkK^RGhC+LDBja+u?v42(og(>WsenNXc2=!Gurk zfsIdXpp8#$5f--u$6eK};BOL!;2k=>zMJssA~Lizq1mZV%pkSFT*RFId$mG`o%OHP z5HVfuHrceTzX*3753wCZ2)1;d1R1Z)kv>i0W~GrST6a23Z!*kmY>;4C(CvW%%o-jz zr|Q5nkVsz8RZUPjW61SU6|)jYlfCQ-@_$d;cUe?yD&NSC`DTYW|1)GuJAAv#9E|P$ z%SZn|i48A5%ihDWBq|Yo6~McROWY03N8;Kr#r0Gk-i^*?owEmCs^7 zy?j|~N9_zgC@X{o_Kw!yXW^?i_MN5@8=dYEeM2pQ=Mp>Z04K7VHl;kR#-qHVgSVbXcJcQe^Dxeyf_?afR<~7db z-PRdt{%<{sBfep{L2jhEYoytI*>3lk8PR-E;E0~QJC62hajG#5yWfaMnM{rP=(^2{ z741E0`<`X(D|4;iG;H#`pd@vB&4Kp%myUJ*_M{%6MZDMAe6(J^;aasp%nx?vzSird zoUmqH3on0XYxv6U-!e&Zgt4ZbZEfPLTJ9I{8!|nOJ*p&kA9_9`);#9mJnl7Ib(WH( zM@$h@N8u5&mOGH`Mi+IYUh53oZJ<57R zKSr5@weMZuhLa6Pa10U^i=}+l#g@mI*2SZ6C2$eQ%ktL@TO1+K`H%H@9leUO^;%A>+Z8h^A&}x)S2MFyNT)S=YYQ5_wBL!^{=gO z-y`gi4(g^e08jHEBG7jbu+rZ{Ap=APC1hs~Du#&5q#COTe4m_;Jv8>K0eL?B35pm7 zObCq6&WJ^Y5HQ|LHujqw*2`f7G6~db0&OJrUGscfbf#Q69A50j#XHDF2UYl!DF^PsqCnD^wPAfQ=b>t9sGAg(95D0A z4uXwBcd_9}*kTIJ%Z>Dq>kmyyFxz={VjNOB?< zt^7&;NIjm{@0kobBeN0t$eRpVNf5ee`<}v(ujWqRXGyXTw-y--1SHEGgU(z*U%dp{ zKQ*k-{mZLU_$sR@S-He_O5(o}T0mG30Jo~oj2dh!58iicSiZcJyLrg6jgC~gAOrnE z+SO&b^~Nr6+oLc!G7sIM*n;y6HFDT7w%mL1*rSHbIzQee1sJ_`&%^ptrYe91je~J; zmg*lb!c_FtZxR%J90#hXSlA&0W)ed%;74ThEb+*T9W@kTTUl~Kci~O{>L<9hyr$*0=S%FeJnMnUEVhxGqC^I7~Hn z&Fnm)h`kTTcaj}lhr0@cuIJ?uW>lpfg_*<2p4@NGc{IajmPVW!mNNt~n6MtE#*BW1Vl$b*9X5@=r8_NAZW1yN z{KAqYfA0}cFPk0ibdgdQMb&<8iI(>Ds6p2g!vH-EE6@hEu@Q1%J#PC8*KqbFVFTgZ zfG<=s={s4G`~8lVY&b0O^zlSngB25Olos*nPlpO!)hf@^jVbFDE;XUKYvvPvSH`r= zGYOj3wCQE;?Pe1o+Ef|Ml#IU|Z#U<>3J%5Tw8<2Wv$)*bCLU1Ya$C%p^BA-|!bS&0 z5PiTeN5LXGE3Fn_iz*-Q8ctc2iv7cjs>-7X+&<8di*|agBoZV<)3ln@(x5NO=6^N( zvudww{j}N>*M>%&hwf38xBq##juHA#YVU}-S9KeYUzL(n%PH%P{x=DV`JPUbUKKsJ zriNCNN>yeK4n4P`eJ&$=Oh{)l?#V#zUW`N8@&6pBy?2*jmj$JanVOZW>6MGAky|rn63_4{OnN>2SkZu%6K9 z2x=^9(suiWv`j{So@76rm*zI*Q5D9Zo7>gPy#K^bb0;UdTEUE?dh?^XhFo(}J?yHX zMACr|HPB(4KsJxU7n^%AgB2!+IvGT(w_Mmr{8Nadx!;jnTkfhQVQ)K{xw2%)Ob?+BJY{+YV}2d?PoVhf9Gkr3&$ex?Mlr9CtFw5 zZI-S%JUo{y)Q(ERIw^5ZW%0Jsm=Vv*)Ve2%Qq$$pHPwFXq;K7H^;G`1G5_Hc*T9RZ z13M;ISr~Lzhf|K>k37zMoM4YCU0)fhO2bf-(;8(7f_kS6=|Md+d6wv@wwBvu+T}#1 z;@J0BQ=4R&*R?s26d&WC`Su1Puj&vrB&DPbxU#-c?q=&SD|$J@CEe(=#uu9SBle?e zlLCiT7^F@zCzQ2WixdC=C);pPh0BM+{bclbx1*|rzHJkxyW-7GM|C;f)@!-6Xr-_x z75e3V^Hj5YnI^=|A0(UflS)RLJ-#cxGhmS-MD{#DCFISFt;&4Bdy37$fEgh>N6)Vf zaFsH?jzC$O{bcX4U~#d&wtL&Gj%LSj5-KiHSYt|tzr4dE9FHlu2c}IkLZ8&Utpz>n zHGU1r{~j|lVYTPbdIo2U%cw?*1S3=6 zs9sOZH)f$NXx;q`&HsTP+FEqg>ZFNg=J0ArXs)19f1m~!jM=!E^8OegVSjT*KOuzF zxY3(g*6HQOE_7Q~9!F-d*sJu*@&jcKX+Y>Iq@aWkv26&yb>V~eZ3e@*L?v=Vw7^@z zr2b&(@rC~4dNd<4C$CJc$mOQ0gh}0sUqDWAUykGhGX>iZR}uhuxT*6sQlF{89jQUt z3T@R_Keanl@V7YRvbe3Q=Z&g@RBShNY40^%9EUp34cy z3-4$aKPg&IRCG{1amWqfd7p|$1F~h4Tq2jp(y>NjA4H$6N|^8qp&d2}GXhUPt_)o{(3JWU@+o;Xu{=M~OO7`g;nr2w@swni>((@(kc@K{%<@! zH^QSM@K(dHjRcJ4&;}D!-srJ!HZ-5?d>!!ZdoyOY3VwLEOBP+o7no%j-M_gg3Na4F zy_hR&$7t_RvWjyqk#XM5yUiH{q^cN8ux+g0G?5fK-Xe3n74LdA&%L+{;rT`D9rmxu zecWliq0X0Wft@wp#e7p+zi_9o;MS z8MH^?+J+T4FNHH1ntOenv&o!4soaIMt|FQip?TxeUBEm#R>A$=q8%B74YgT7MF3j% zdV>SjKw5G|tto_cAM8}Nh&sM}>2z75%CU1UO}@i+tw7vC6l8otHaOc6%fcMlp9d?6 zJ%hhr$9#pV$N|`KCfxeQ5bW73=DM-EhlCU%lDB6J(#zo z&!u$ux2w)I)9A`4I$;TI;I(%@%?WTWZ^(5%SV*XVLu%(*+SRif}u}%e8{(pzrMfxsr+AGK$L#y zkBTBfOf~2J!R(5YJNaiP0;23PD>P?L&XagD-8k>AqS^upsq1% z^`t5(nnlBt+dhTB21U2Bd&haq0x$4oc|Jo6(o{*sxipf zrUb$g`~cNq0BtR?70nyyMLW@mK#6yo7s-1yE3bO>ax5*>%Ik0!~m-060A+and@MWK;y~GZQk8X zvu!9)6ljyv>rB{s5QVA7EQ(QOJq^OOy@A!rAH126-zh{3=2aC9}Q5|r%poY z3rz|$dFp0t!&-GOp$IP|jpWewJd&a8K1+A1YM^BMkX? zIBjv(!+m{yCFx!(&$&{EH;F)*PJdPQcl^ zs9r=psO7KxSDEI0x&IGIY6@7NIc4mjYj(-dEUO9(8?gl?MOrAWWIVHoZahL`^8MDpt9 zT>Ofx4{GAP-Zxq|E&^^ou+(esJI3gLfDmE55vZ?HZZqcSOFz)EedkXkR%NC91)z3V zOFEpp^2Lu<1l4!Y4KG4!9sDt>S~%vrXs%#VJaQ}Gw>J=r+w6~&Ylv0Jp_kYKOW14m zO}}*9vq}|>U=xL)N;2UnwM_2G3)R89VWNuo05-Vgxeyx`I-BPQ`hck|Zpy3}G*0o@ z=?V)#Py}L0|I9zGcemkEn|`<28n0~DtHhfiTtvE#fCvP;0GDv| z-&4TQ9P!nN7?iIQcL;3;oH5LzOGe^Qjh~15Y~92@Ik#T=DCg0YIrmn@2rU465YJ&E zFdQiuS&R=X(AzsXAs&-0gT2LpIigfYOUgzn)QGfpUp;JcK2jELUzs;Kf@{TZ&ctj< zflRC{z3?q2lVw{VER*AJR(VGw5eY1x!{)hEQaLM09=n*?aGdyc3hhm25BW6-sy1h? zd$9+Lbtl6BVWv5h*fqeYf{Xr{4 z(05pm1d#?c$+M(yk5H>R8Ya(Ogi~Y@U7uxM8Uc{rJHcHazfU1`daWW^uePYhLFniW znR<-op3I^Q0cXUeiebT$<+`d(%s%2A+>s-hQb^)k1<1v=5?vKfP0sTEKyR?K}F zXZZ2Gwz5n|377`xQ3=M-d6OgLRJ5skr`QOuMpZmzQ91E&$5&8tp4a4wSuU)>9u(A^ zBP4w@q!^0iB#(dO$j8GH$8*6M@7XDy>y0x>#CLM&Ml#SCJoR`<(XO$!X`fExFw~NjAVW{fAk^F&I%A=XZbEK?im;uhX_QD zTmmu+7;_~ks zGwvL0&_gI<586b+%8~)ttd$FJdgp=ky{2cs?iot=eenhNB*S)C`eh1he9Sfy_anQ% zeQZTDouLO7XWls^I+%0kp)vkrwkKUEgV*I2Iwe`e3m#^R{OyhP1hk}FqRKZa*osjv zF}6BzU9mAlDCdtB8f8TpLv#qsk=2C-HAVR~ zAgSOJxlGXV923x0Ll=cA{P8_sn>~{u1kDPyQpjggGZNf}FHZXWQQK4efY|WPR@?v6 zTtoO!S?v%juT(qQGE4&xZ0QOK0W~G3bstK*eUVQ{k>@O5Ae=ksXQvEVk3Tj#K5d1b zL_x+en2oba#`ZhZ*ZtmC4m&$hIyAhkv*0+9{`r=2zBOokm_hEpB=o z2sxYW6cmOn0nrSmtO+5*;LSQ60sF|>Pzb+Dn*$HvsGXZUD?g|o!OS7}V9g)~59FibULmW5EE zrs8ZrmWR&wmpALC+M&5kdIQQu?${2G@#t+%n#=3c@8Roz-!ISCE|FmnKYkp4-!K2+ zTmApD$^Kt|_kX|DYc(!4(af>F;OXFMf-%L=*u+^!z_-M%Ny4aYOrXf15YEJ*tbCyzl{}dIoZcm6I2vdyZ3%P)CX!o`!g47MIqWM}n9fX~$b4gpO#U;{Cg1Olh*GSyg;litd*;C+$q!2=0UiN7jlw_tJ=A?38!D22?#|z>bjGfUSTQ$@AWX;3KVM}$$0EFNK}Hr z@@6fN+e{PK2-3RAJZApBv_U`Rw1MJ7L#~qNYwxqwmOP)q#TGuD0Y-$0{gyz$`c!^c zW;_xzBcK%>rHGz_kJFBNXqFkY0{%O3KG%mU{xvJ9tY=7#Xf7GvOWMWByd4!zF$MF*=cvz-CWZd%;p4uzYDl&Npt31+t`!lh#({l!jpTFq8{R44v} zj(N^^*dTA0Zykpv=~{pNUQ>yama4jP(F1YQnl=wz*;#ZTZzMfE>Q*cNtF&kw;4{!A z(?|0b%ulAa%P{K!CWU*2@W2jUsLC22$EAzz=jhXR6d+A!Tn^^%Zx{!OGnS>@{gIaV z&hr3#@o&Ftp;_yQ&@&+w3|@U`|4Cz`Ou3r<5~vd*`ARI8=4?P6LGdg_{u(p>mwbR` zl5?Wx`XkXXNApfLJLh;pq0UD+3MqdBE+tS()O3+8Dj*G~iz`h9=4{X*4q0)woyKCY zj>K}%BryjXcVuA@)ei+Cmugn>mv+&rH;eXT6P-_H{AbD5ugO(kouY=H zug)Lx{=nBAyi5tI1t&!FRB}#an+!4Q5@!^f66+*_rI4FK6NqEbMW|gdX?ZIYSqJ56 ztv6L!Ga`jw@D}b^)yzTeOxRNukIw0qKYf8QkEPl^)#K|Lf}k@CnM(%rtCw9~>*ao?LuhHUM(h3e_ttcR9KqWgSqmh~f~6ZRx>O-1EvSvL(YzY3c@LG6o}akWSl3gUa|%t^cI>%6f3QM+jBUD`+Qmhn}ep0&+EmqwI%<%7;`3Pxcb#n{tdgUT;| z#ZBXgTKhkoy;H1cQIu_awr%@t+qP}nwr$(CZQHhO+t%54*Q?6St9z4|%1hSQ`dI5{ zW{x>WAH8+d8Ur&)cFh4&^lgQIwEw-ZL17|Gh6hZ-dLNM|kQ7Mx zC9^;kCAB1pYED!(^^G(%dj`dDh%f%JZqwQ#-ZHq0K3#Nrj!)i+HF@&Al7U- z_PXpjj=bf5&3g3Zot>{V?R^3C8-7T_8Q?kDc^ajV=jq_8!Z;H9|T!{Hoyx;p>HoWOmH82b02KR&1XZ|I?J!9sk ztIvKp3Jr}U*B2i~g5oo!@;CW_Agk^BAYS6R-26b1h8B(xW)nY7Z z2^2n#L;B*|EZMw>62>Q;hH>dss(D^6Ry`x@bQCeY`fO(Y$cvW)yTg&$TxKcXNrB{m}va3NYk=Un-P(__0^`lsj zMzPR1D{*o;r%silyFtsF_Hh#Bt(=6Csa=7N8dDb5WJSnuB}2%fZ^~(S%1J|RvvHTp ztYvh{&9{?#nf%T=`ek%_N^b&Di{8WAT~d}{dDn(ydW@&iJ9j-*jA#>V%4p&OD@yHp zn*}9P;1u$pLJK!ql#vp+Ms5+-zmsxav}&1qzHOP4t!`Apn(=jrO>s3bEUg~LgrcIS zG4W=qhs<3|MTp)bE;cNk-Z1s%JWtQ&A`4GuRba-_7G$g*zdJH8N_bO(m(ZXlW4T#ZE@gi; z2BU*sV0sfX|7|{_uCwHj1cex&a;fNKb9a&!;e~qY+9JDgc+#-eCz2;Hnp8vIrBx9TOp1g4~4TO1L}0#>0J?zKnvpvH3ox4BTriR$cATuc`=zj z9}BWrlB;Nq`-@1mT5evUS7og{y!)c6(COko5TQ;+)Ip+ng|Wp5?|$J#ve^^`;rw(7 zgw25Mj4-!^O6RMTuF#BJxtkZ%5z3#%U#YdG> z(&Cg4d*jnNICc}+3lPS_gy##fqH*N;GL45!fCXi% z?6`a_uh|G_wUceYiOzJkjKTZAA0X-+LkP^w`{XuQgH3CbdQABV9Fr=si++IRB_aVu z8_W`98EM0Y_SqWE?vp={fjMLsrCsAx=#9AqIhn4DCP&ZvKbks|Gb+kTb=h~n>ggPJ z^f;KJ?Rt|KjmuXh^#PIJFi+na1Ny8GSQXB^Q&Lw|)F=`L<7S5pt)3UK$aAa_w=Ts$ zw2dcw$%xqHDS3?Q8IcX09V3&MdNrBq%1H{H=W!H-07DEfZwG}~g z(2|QsTTCWHc@OxF72G?%U+HRYq-a{#AO$v#;W~@2OUL{#8slTx2?WLF+6ElSlR3KTZnV>WabmmjqvTJd|UQhTyp%d!rJ+@*no!}msk6Hp52>Ht2phX(qNtO*67J%Wy>?FA?BxRyP7>}Th;|RzWg3QN zxoBuI%cs=h3L5c}1+M$-wT*Er{X+xQS&i&29p+02a9$!Jbz`Z)<@&_$cp zTH#%ovka*;ZlmaSBKQnm`gw_6`I26Ov;hByL%n*Qi2!kgefek8T$@S+5hy64{ie_+ zIXKi`B){QBLn@C-X7=D{&lgRE<;mIXA+aaC&R6ONAu^>3Vp@0AGfKS*_DST%*OjX( z?N^D28wpBjZ7qSAg=i@V+eYfH;SH?9vLlgmac#e$>del=^C*nixYFl*`c~~@7dzr6 z_D1;47Vq+*=@`EwS}|suiDMIIUrY*n_2cN1Z#R#5kN)n2mRYIS4f{h#W_sCCPG?!Qyly$w{EmV@0+YlpQXL z;3YH;rJQ#4c%cWm_=HyChj3eSv!n)6*6ISK${PdGRZ z$S=BAPWj?))$R6VF`8}ZZ5M?!CHigYW1XcPM*nz8vV5wl)0fP!bb`XYzt&nx-z#5|^> z3dDWF#ZW29YdRzWAqrNvb(e5~#S9IzQq@Etu4SF&lFDEG+~-%!!%uRquHy=M#)vN% z($9#Ul6CS1GIX|Ch_`yvDX!c3-PHSAQdE%1W;K!_H)GO4^fv}GeeIQT_FNVVP`L}O z6vQ6&1=K}6dbo8gr#CuQg)y{aY($Q$)agPu4Ceak*Joqg;WovDCJPQF+h}^6E_^tO z6m>|ecioY?C~c?!F|LOkJhi7%oV#4peT!bu4_V#Epgfe<k=N;&V& z0*6~eG!CvvyS=|21ZMI?1d%`Ak58SOd;V{e8k>U86ir&k5Szi9F8oRS<|kpDg-m~V(&2#&10fGddJ z24dq4O~X2@VhzzpkDObx18CRfKeGjNi^}1+LmS+Xz;H%=-Ef4j z53!x1&%l9&Io-fxhT8cD%CI_t;XpGt%uhpsMf)LO4+M*b_+I^)qQ&dG!`iUdDq>@g z0oj)*d;q3t0bmy7jgezp(^x&i6mS$xMqp|RsX#Kb$Jeq}3bl;NK;y8`#&L#mhH0LK z7nK&=KCtWNzizr=@Dk}pU<(N*ofFyR(IWl*zTMH)Y#G*tS|LKExkjsXIVat!i(>Sz ziF0eYNs)Tm7Z-**x5TDodBZ$&_zj-rkVRzT+Ub__%mJ?-e=E4ujx}bFt|!Rzj)Nzp zcc9x1DYtL44eoZ2zbi8IjwdHhf{XmwO#=16Tz&XegNyPKqMYjp8hYXB5Y@m5sW3!S zoLY=3y2y?5-GiIgm4Q`M3g+l=uq+irO~~d(D0)hS(!B$#nE_3D%awJIBBFI{S2X+w z{$G|B35L$CLFRQZhr6b1&8=`8cFn*jguE~hE%7yHdF1pPJCC@ySJJVF7v$?Ar7PkG zfQ7JN>C#%4TXL>RpzbgKZfdDf%NooY#9cdB_K@-E>U|+<_CQ}^o$B$PJ%-CIh%Z{p z@y#pAi=z{F^424(Q>o1t$zPx0c;PN6WY*FRv0xv{6n^e*_?2FJQ1Q!GrM__gT>pt6 zZFd}Jh9Na)zam6;@$M2c=NO$Wy4?lWJtI*6B=+NsgbrCkf_@R#Ck-dewu~q^xCP9-sIl+mFKzjBv-hH?2Zzt@%YtYF`=YuhQ*Q~EVbw#<>6I$|5s??Y zSZBf`m_~@*c}U`!9!qCc>bh@wV!~DkXJ*1md3<&P4VH7wad*NnQDjvg6Y*?kZN1r#gumZr~4>r(}1cO?kV z(!8F}a}v0FZiP0ThWD?)s~g+3;cKOumr19a2QoARcjy=7)KnPi;pJji{S_KU)Dz0! zPUUwNs9{-+2+GN!Oij_O?fNluhkGd0#nR|gk!GhU<0Qdi zbyme6>`eliP3P?P>34P%R2klb{A48yq=ViWnHm(rlr-B@Khx_jhEM2+BmjqU}Gx_8CFfqYG7>xU1u2E%A5VYmo8v+18*lM z0Bq{o0WIOuxcH3$9^YI!e`wgRQbHz1?bAC8AHY#@T;N#S@I(~_j}6mq6%x z(i?WwRQ=Bl$lng+phQtgP6ssVF`n%|J}JYb#>_Dj%5FLVR!6T~S}SC& zP}TzyG3epN;SQ6)s$4$Mx@?MSh_2{4z7MIZ&Yj6 zp>FAh?I%bhjg{VFhZU7vo5TW~OX(x+xuWPQL~ei4raoqGSg=YsK5*H>us23Mt!duN zpQ63M|GnEpm2OEm|0|>5{$jWPhn$@M=r;Xdou+10Ygepc6d#h5?u4$Ea(eNp$_0*a zDlAGi|E@7+7#9KH7^FIToxDB^<`h=->xktE{GzU=<8ELL$fTScA8T3J-Z{sW$3Dc4@>{sH;fhN1!Zg%4ILwVs;sAl*sj^&YcomYOsPtx&#+kUBXBFG7``t8YLCzVIsoh+ z74-CG9FDw-(-LBTv)Uo8ha2ve4Vc}b0k@3UrEIVmzr)y!U zsngQTweDe^950HHx1nOHFl*`e=xOLl(T&N5tnegnvH0Dqj|)poboJdWqKvjiIE+(h z0^4obYwdyy+L|U#jNsaGTtyi}+lDfm^n~`!yCh&C zU=QC$%H*=5zO82iqw3y1qvFtF%)n`iBkIA}B2Pm}_^XF2M6NU#gMOxGVvtUTkjJV% zq&P{jD{xa5?qfb4hbyn@7|(%A*&a}Vn$9#(|6GV!`-+Uh-4nv_F5Xk3VwDfO*&vTgJ;O%L1w8PpB zv^XF^z5+)M1r4|Df3Mi%;90zhjn>{{`9G0N*7PYj?$?~-$}T|~7!2qY(OXaFntA*#74Eai_O?~xFv@IqfrES<6TFGrarQ;06&d@+ z&3TEUUH!esC1n+DOs8chOG=$uzWiP%n@NSa=E)|Rv=8MbUD;B%R|qDE$72IkcZIZV zZ9Xomv&M3hBiXt=bM5T`IT%Z)=-ys6leeXP&X(6oUIq0Ly;sV1xEwUzG_Cd-fngWzb4}l`Bd#EVAsv%jEL+ zamj?aZi%!E#Q;LU-oNTzyXVE7tKKgQF>Kz24hkvXA)y5KFv~DB4(655Y|u5i(q&7K zh~LG$DRbdPuY3u_|D;8bTz-4aR(Tpl`Aj0{4V95u9p9DR@RLL#uBmdkrckYgADCo| zuyk(4$rD7@7bZsnZN!uY4MHJ?;e2~bk6_OCWrF_@M+X^QhvEuJpR4qRc>mRRcgnZc z1!fy_uW*pya>;ZSZue;aOj8P`)nIuqx9RNyzSjE2dBF zJ{+XLU?C}T6U?Rzk0Ol}Zq^L+uPfplTcs}Fh^^*W!^OUg&7ATqQ9xvTOeQ^@{1T8> zeUKT_g@@5F37e5MlbO-9jBu#D*L;91vow0)+r*|ysZI;T6O#&^eUs+BB0b;0d?ndn zDXMcss2{s#?hhAjvGZI46+@dldW9O)=$x#c(Rs})SLZ6C`L_q zl3PM5q1zOt49I212z5|p9y>idoi5uRAsLv`5_twQhB)V2dO5pc0`jF6gb<@V_X29% zIlqJC&uVn4n5*NC(<~EWj`&MWqC>VwOuTdMZ4L<#C1a0&#He}u#Jm5pR_)k_hz3S2 zZMKRHbmnjuC$TzrF(1CFW@u?Nl*mW}qhR5y&Qu0*d;_?_6mvstqd_r@OGRp2o~Bsb zPWAZCV~#^V*$JazjUme8QWSehJXX_`K#V~4c!qa;!jz%_Hf5w07^FzLg67557`6N> z)p*0@l!^AHd|jo?06i&`$;vb|6$Ew-F#+)f5`^YJX~2sgU)$!~st#;H1<->`(@^{foI<@}P>$?5VgI z3=<+^jw4lSyp3g%t2*BeZMNurKT92A~6Z;!bu93Cs#};ZbpD`#(&NwfnOiI-^*aJ4^RyUB!R9W6bjoYuz#r8^c|2erIuEZqMJac5Ll9 zIOwGfoWF2>vZm|RE8`f~RWeiN(wwf+X>fjqz&Ln@lmYHh zV!EASadGhW4RIdZN)C~TJu^!!CvB-WwciT@y9sWrE$#ZP|Z za%=ORRR%gZKEBT)SRT<6WZ&!{y`dOk({Hv>ijmqiM$6$&~ zpfpHtHZ{2_09I(*!=6qzP9!YT;}(a10X34l?Wg4Om?NUcG3G-rTC92#@592F5|h^t!R_)?Taj(zXc=?5ksIYzG0xP{3Y2Ixl&CTqW& zU3oErSH}e5@@QpN{mjGv({t2I_Xaa32doU4+D&T6s(b1=H;{TM6J%eB-u{I@M?li7 z#a?wh;ZD8E`9=s3CWru(y=r@Z{zo;~lCAmAaR%}yOD%oPUUr1w*Pjm3pxZr~ngPCE z+dMw;WXn*xIEh_8NsC>WAO&yMJeqYRfPm59h(ojCzbps98|>e~*GZ+fo2ae?0Kz-r z$Qno4Vgy6mgKP($GfM(L$kdO_stE1unCLIA_Dp>2OI2Tbc@M94t+igIwb&>*l;EtF zKY22|8jHQ9CAzm-)+#e#9H-)%KwL!l?ZQI;FbE)PV0{D3G%Y_EhXl;JDVR0!ed2

+ zb^G6qZ@8pYXXnlx{byQ#|HCfH&Y^l@CW@4@5V>Z@(QJ!=)jL3vlIezft~qq_&yHXU zy+vtm1y5XP333Ao!Ny6c?wQi*r4l09-^P-kBELakSJc(_4a)4ahwoqu(8hXGr=RuX zO}9Tr$PIw=hnng&R585nsaX=0N!L4k%zH>8`5fo?)^bmn;Te(T(Aeh_Qqdj7^Byzu z4RZ6y+2Iw`d8c3{lWqDn-MmYhWv5@!O;z9?9AI-s0q>+$-@?T(=oxE&5q7{)q-Ew< z!&7VI)t>hd{E%RTPR6ap021zwM#vV3Z8AmP+!tulElUCJUC$GDW|@mi3J&lp+w2`r zP?0%n^)4(#^c_ri(D0{UCc5}baH8y~^gg#16&J@XwleH8&dgufDO)Bne%o9g1I03Vshrhu2{25&99c!!A3pdLTwOZ>j+JJnysy zbP(St-cU^5B@s*zt)+j6NWU$+KjLV)W3SMv>SFFP1!a_cHHJHK%@b$dvSBUzmLG(K=_?wAsVs7gE%eDEhp+VJm zSFB}}e>qgJ3ZaTe=nFBh52RsLcsE4@HfY& zUf@t)U9s@b@ghx!^WeCHhh3fmgNgWD#rx;rQe1-s zG9q=^sbo1Y`|aR4qfnSS$YyEHr64WQ8}CvBV;JQa>1p%}E!mP&80}qBXo?K<2C{JA zRgIA{HD}P9>z&h5{Cb0DRrUj7$T*7(mU~w0WeB>gjkbQ3Cv^YsnVVClDG6A%1@uGxQBERlYPBzPg_hJHT>I$dhe}b9EEnQR^)>-= zq+#WJ6IF?%bwFSfv(nAZTjgU!Q>HE*ZRM%rq9-t;Cryh=40h*D%jz6TOw~;iGS)3w z)qtGbQKo~4l;c)*&}S22sr0-x&2pv7cMTz`WK$v(#8AOAJv}@RmN_Y7LE2NLNBoJ6 zmCZFX9_slq09T9VfY=CYT^6X!Ifo41V`Nit{v00hbMt}Pm^vuvAG?tmC_2P-&xo)W zY{9E(ug)-~&Rzbj&RxN+&Rv6(on2j_uOKb80;x?@>GW0K?^5(!D7P1eWOnM! z`8cl`Gg6_D-k~wOhd}<(IE+qX8#1F!N$0Q@d<@lxf?h)<-c^T4UKRUr96W<~n zEnmEH*gcYoiKek`un1y;CrU!V4ROyvU~*tjmxj7}xN81sFAv7{BnHtmc#ayXAD1L} zM^uP*4VbplX)pI9Uf_dSh#)oUP+!m()Y%R#|A18tsoe3wnx3!XSi(H{=*npfNr7t>ZIDhhq9Fg0iJmx~Fh8Bo;wK&|Fj@I;m zIc;|KV}2yRPMI>v!U#4`h%q-8|5srdOPq-IyEy=cUQa~&3beiZSwHpS%ly!JJq^n| z3SOI~@#V>0yAp~~!T=RKAswrHobDH%mfQ%xb)?SIHG}`K19u#r!gE#DXh~Zl`|x20 z>uyEB9H46qA^V6ttvnciVS55J8x{l8%eF^E>_GI(4K7DH@Lats&&6y5sZ!J#Y8K*e}1KMWDiO2pj1J*Z@1Q_9s4rVcS4GBiGl~ z)Ki7H+yu1#hsaXL3nOujiu#v|DV)72lF9`fOPe5+$3B+9u&%1_*^;$p)Q2I!Rq!ZY z_S#0S=lx6sM#%PIjeF2qr@jYcFmqk!VbM_ zuY)CoG>=#0o!YGHT7A?>xv>zr3RWRCu~oz(?S_OTL?)X3IxDxSmh5D_C;a8GU|2+G zy(m`!7nSDft;d?~y&bm;1)di=sRj6f?pon~>eGn{Hn}EA2qgfUa=m^NVi0|c6NGg@ z$pNbWU}wr5lK)~_-t=)mmF?~&ZR5YD>E57fDaghAAXpyT!^f9yY=IY3FluDso;?TG zoKckAug^v{_O8TV^yFhr(As>C8Jaa~>YTSq{zT8K0s()PkQkR_yiZJzpH5$x8$#j% zu~ek0JaQ==t3uYc5grOpq*b~8rp4ZG&47KpKmWbhZX}dQDgP!@v!ng_Bl|zQAphUR z_W$+ff7`$Qv*tFdzWXALqWsL9ttEGjfT6?Fhu`yC8xqoupkp|$`Rm=q55@Fj2RAp3 zSx#yQYsjA^H%V_6nJy5LpJ$giZXYPEr`||%pcrBFNWF|N1j1P>Fc=6CR zdOV&?fA+lg*pB>s?QXULIPKx_H0iCxstggM`?SHMB?RHg5|D=^0WZQ~*khQ}>!J>d zPS}B{4nK$v-_i5(#K?-h72zqnRY1!gDjK{wVekZ+PTZ+`$w23(-c?1N4o#p19iR>0 z5!*kd8hZ&(>7ooiGU&iRt{Qo1PwmFtOVjs~9TL;~N)9kX&@w7vnlr|;d`7RUZ!?bCE|%az!Ip>Pg6@{#Mpv3}l+!Q0l#^Q%`6~7zuqt!c5Mw0yvrT`~Gbb;M ztsHx4f-&$KT5w@vGbeg*Z7`Y<74kFZhm)LB&g;+eCQk;JJJU@iOeou-x; zO>r3$8Y|Y@5|FsT#*J&)i%T(G>7A<2|LE`g;jT^xLVG-JNF7}(5QfCtmBp5qrxPB% zSPdY@SgQU~O6);hdlBY^Ai>A%JX{ncW6Bfh6O`aE^_f{}NSyCEDV3JeYDyYDF6~hgjY6_3FX5fO)znv__xi%NN6676 z+wuMm8!=?lHHQi2d$@ll-Mx1DQT-(gtW>YttPL5|pjck3kHjVFF4jd8rk%;E=N?!K zIYFJ&pl~U7*yP^Y3dLHJgF{aNO9ceuYqzQ>~E#vtyBx@t9aEbmY*J`%x&XC#puXZ$4k#z8F133No=A*lUJTKQx#=ykWZ$Ro@5L6c#vsy zGUA@p=s4xP7UI;`5vqF`gBuOTHDqc`YE_bF4{@`>40UaWs&w_ zFhD9fq}hy2@l+K5hTJ^6T~_%pOexgh0^zG3{AjLhP~gHl(b^ATjWO&WGO*a;0dB>@>V)Xm=w+oECaCUFjOQFIuZez~?hUeG&P9I*StDk` z3&4$Y+=0X9p&?qF*Kmgk+9^8mL=Cb5!eO3M+y?T=uFc05`aL^GRmFXVE!+qm&D$r~ zfan&%Bm@CKkP>Aj(TV+_1MLiBK*vXNjuDHPzJ^*G5U-agGQfok%Xh)mJViZCJ!lcu+sBfh zWzA2u9r+8V7hv1Z{?8AkRK~#s{@4w>5INinB?e(96}Ww!V1SgM|9l^(hBInMjJ_$DcPlKzXWYB z)r#*T)r85^a%QYc8+T+&DR(H#QA@>+b>PENPt-F&?4Of*zh&*fqnyCaWzN<&nd?gs z_{@HHs0=sCmj@j5ea?F}I1PBXg%6ztM0jp9f(`I68zp&ps4$xSd>DHwsQAQx?gRj> zeKsMd-b(ZWpFA((ZGZO*F6lf|UpV3`+o>4ws>PZNw(uvBB-LqBCgMgVOvbS8fV)s1 zNPfG$lw=`M3g24?m6GneGF_mzMP0YUHao#KJ#e9Jte|f`LT=!UUw;m|Rw3u?9h_O8 z1EgS_6lamV`vt3P3Tf(d2UKGLI%#Ouob}iE8L^$~VQZwczraj)3As)E242MiLl(^E z)Pg`o$*{eJ0(}bl2ZaI)g#ry4YzfGFChjz66#`ND1X3AUG`3WGnlw6%Xxu^}q7?_kCb@+*(Aq6eihz3gD>( zM&>3iT&G#<7@yz$o34LW?3nprCSlyzY^Oo+QoXcHz?;RAaJl9ez^CD#lW<44PRGTD z2zlfvVn?}F4dh4&b~4Jlr}s!H7!QK=+uk+TSZH&08AYNT7qkC@YxNQ`n%^*RU4C_+ z$Q!G;pm)_^f`^#Mniv#fzXtD|d?ztf=SaA93^q5NnlCUCs?$;Jo(88Sw?t*3n~CfP zMtQLw3*(nP&F7aK;id;+ZvtJ*iSU- zcdhTDKxmI$uXXQypbEZ3w$C}m2s_Es{^S!?Si^xKi5l{|Rj6t#^=MJ2gOG)kYR0`MbcYVp| zw2HqJ6F`%KC<=rys$~GV_yXfi0dTTut~;`c^W?eW(ikB0tN! zwxIa0^?OBA@Vo$Z7-q&QOB@M4Er37!X^4%ZV(U4JeWv5g`ZPUjc8UQjYpEgL&1Cy@ zdFH3fZf54D6YTL1=PR)_B*fCsTifl9<1F`U&aIa(k$AY zL3XN}J2@?q0kh7-okKS5(}L3aEkO;lTd>z9v<~^M@`4uCVkJxsYg_-GNZ=~^Q&7L9 zu}yaW<#A{K9!WMWy<1`bG-#_^d4#6(&QbBOfIDNa_$jr{2UDwidv>ctaG(^bYYZ4K1_7oSr1;HpeUa)@5-t_LD=H zE7~0{ESepjXi@k@SF2x(otCL)4@6)h=O=EF*YsX1)Lk7kY~)=H(zy<;(~kZfCyY<3lkMF5whm;VE%XlIbTmWP_$2G2 zCyY<-%x!9OWS7w+^}<%sW3|o(E!4{+SkZgl4Bo>6KA1!1)4;o$0)98)LM6kSOMLP+t4LbO-wOzzXdG(o@T(I#!5EXJ2P z`$0FT?}()6TUmeZ@;gr;@9FXE#WMA)SXWsX9gXr@~4>=K7vk~W_P}e zvyN5{2Uf_VViWORuE;dlYU(d*GyBG3&|(na{#-vF0djOzcg19cc<8uQgiw{6A)DSN z>WdY2v*};-=#dIf!OyCT8Z%v8CIuJ;O{Dxf?Tkzk8Rl7bDKd?2W&={gdNVG4kX4b6 z0{$#YqXTb$_WB| zlIam8W0?|r#Ig422J1NsNnpej_^Sw!j^OdEf(R#7TfSNZoV`nVXJo6R+2I=!FrcxV zG&@|hkXe$9wz=yIs0e=gXCI~0T7o6~xw5!m9Fh$c0iil}T##xZv8&62 zhkEg8b}hsb748Nt^(IeiA_QK#Dx2T{$7G5+GbW`8CUV0Da{XbT@G`L}93bi}0|%D} z2KoscxS<7_c7jbCXAFX^Cd*%Qv6?-J(K2KaS{+Ll`u(7NPTxvNEWa2_FicrJvvXPBPrsnv zvWS=yeaCwT-i>X2DOf%{|5k+ca^hZNfr>*jzps8#Ci)KcGQWfT!y$8`v?QO=vicNM ztJu5AabtpT)T-$DcVo7dd)EBbue}U04&%dV;vC!|s3MobWTURR+p}UD$?;JI9>9Mc z*1chCS3ll+%1aj=%T9jLC*fv{c=}U#$4zRk=`%|91z8jJn<0osC(T=ljsWl3?^}xk z>YJDx{ATPo=%<#;U|%20o9{HvZ{|&Oj!abcN9rHrPq6t-ngvH9lskb1XyVU=FXOxL zKTbt$P9sGXALjqyogF}J>#_!rDT%Y6G}Vi(jkEs%tS{)}cwsajV^vtm*1{7S`Ac~71GHkn$ii}NOPBe?mwdE*gCFRkJ zBbgh()pJEZ5kX$2s9*+gwEY?lsUT^~BpDg95PyUs&&tZB7b2vs8(4;yW{})=j9&no z%3!6dg(!_W{<@|~DkILfMaw0uW#Oi8C|PHeXV*%zHe+l^X+LIeP+4P^fp%WkBWf*L z$HEV_n=(%GxFY1S02?-F%BMivS~xZ-C}v8E;ukpsj8UKM;z>MxR5#t&uefVq94S^> z9+$A_R_bXy{E)@P?QO{8y(207UH;Jb#5}TivxYJ!GRwvN%s!%tQi+k_OXukT*y)vN z_Yt`Il0ZR%d#|MWx$5W^r}WWVw))BZR^CRBC?5d~*Ws&qxxG_;{!_vQ>z54}29QUC zVBVLp?u@WStx{d32q4*4qEbxF)NDaqJxSY0IpI1)`Kqd;tls=MC#*UP_NPIMNMwLt zv6D{|I@4(mRhfF%-7GvXfN~3q9EHy+zJ{8hRx`*luc*a??fRNcDcWx15UY-vd_$@e!I5 zmwJ`RO;)mO9#>ce`J%)(Glj}#j}l4#3uuCF0IEc@FrqFN;&~s@nriw8HC4PEm$?mT-^YjJUfmdI*v2|=YZpCYDZ&r zESr|YYU0xH3YJuzl(-Vc)NnTA7pP2~UH8~VM=*ckp-U>4KODD0W}PYMQMVC7sc$xz zOlZ6YhFQinti?Ra^%A9NOa&a?@-(@h=$&&OLvpXpU9I4J4);M}QJkIJT`i_C=N|Br zn&Pzk)?wMpO~dSnMh#Ke$gq$;1Js$kVy0UABC1T14~=J#!iJX`C6sTJ*L6GVT%oW< zxl7T*%|5;z6m1`u{}%S2It&gkfE+xXeZ0@g7P)g zgkeU_P?xU`$aChUfgQ5_h(M{b7m5<3D->?V1QR=%F7|y@-EKKHtVx23P<;Ep=WPQ> z;x#%1XV>hI&9K`_c_7#aU4inw9MMt9QDKgol#hzg69`fulRkHL4#?kUcj3(%}+|dZHwnWRm@V76s2H~z&b&4 z6tM&;B+eQEA&zf5ucvjeq6ZEoW8!mnEFa0?&F&nLrc(ApQs3b&h? z#?g@O9@K$@M(GpKR8vJ5X_6{E=lD67~y)FN$f%tKM#?VoLi>YZ`p z>~lksGHTV7J6xuL>vHF;icQ#qhZB7xcq!kf@qw+e$e%ZxICZ)R^aNsBZwYg*uJs`?1u&JjActuB;fScFGd=-v;M^aJmX1})Ej3S`vbbr>`M7HTE{DVGK+bwye@ z%q%7h1!|@sc?_haC6oweHt{#RRWDA0BgD+&ls;ocgIVm<^{oVMAqsYPivPhCStF3i z($B<|jnQ#QsyyTTH#1mudPo>$RK5ay?(klF85f^>SutHUk?q0AAjz(VUmyc4k*&MTy? zdWZf)ssX{Y@w6lT#ud8dOm2UYaT5+dU}?zW#W*Ie zEx}KJguSH|;d{*JXL z?Qt)0W`AAg^Gf;j5d4VEFE8KXR_mB^#rC%hqJAZ?`LZ^Jeq0UMS>YX8fRvKt<7R^zM& z{hs6vNfC~O94=Exdb^P^uHej*DLJqJrF=57IBm>dja5xgU^BE%C5CDwowcspT6+rt z7jGz-?-4F+*WeVA;Y{E(cO%MG>we9vF5LCQ4`$hep-A7LU(n@}UVtd>cdp%1`WG}~ zZm4fwu-s>i`Jnrx5uJ@8ehp(N-M%UAuU7m<7{l%0JA#-ZtZ3J?puG;&vf|Y675GOI z&_t{c{$o#)39!FE5h9tP58sHXMdazfL60?@cEkP%CceWDUf$uA1OY%C!Bo^I0L+>J8 zL55bvbMCC4{Sai!HJ(GpF*qQ|55^z4jVmPP2FvD<8Khl+Ji8!mR_%32z)OcgUMem# z7WJ{?XP#qA{9*H<@K+gWZ!_Qm_phs8O<<(WQpKg+f&S=^9{iFp%?03cJDHtwqGlbg zlogv}?`T7Ur-VF8R0U4ax(m{8V~Trg~VK@p$7n!_0s3NKZFH{_YY&tftE05SI{u5SvavjIcK`lV`&N$`sP z9JXZAbxT6+$85JVmzpBiE~Inkk4o%kMzdvO>oOGWKboV@rH*FGJbC|N+>6qmNLf-e z#qEXY^C$rEeF+RAeF=^pxQ3UwMp<5w=8g~_Xpetz5K{`k(ePFjt<*898 zVUJ5M`oQ;yK#A534Z;pXIPp6nV(!Vu)hAa$1jMt={(Mn+V`P5k7_dmL+L$5W+z_Hy z1DW_K52Fc5!F?lWmw=c)K&*1BM z^p2uJG4!7ncj6iggG%r$Dm}jxDx=<^uymNFvK1|;MdGIlwAE zGaT}abnL81Fjp->>1~H8D2e_R;>_eV;FQSk(>*f{e$8X;9QJznOCj5AV8`MQqhubh zuu8VyJnkvdt0_I`OAJYU|5_duO!Z$?Z~x=JLq>hy`HG&z*2UKAbKt7u(G{Riizxci z__`uKUR-1y7%PV`ovmfIr zD_Es>ffsoqRC40wcav)`zf+#Qx7n{Rztf{r6$k!Ltl#K~2!)7uA17cIYCn7^%1s z_~xw=REkMg?FoNa8dI$?x&3B;r=SjRUfBjLk7M0|uy7-B;E`IrNE}MhY}%E^G@Fdz zE{QuE%ub4BHI@QBmvroWFyW!h8%ue*^I&djsg%ur+47*`*SlveY$YwkB+O(isJBKf zUvqqSR!KST(>?M-Ug)guFRqSB@X-`>R7jhx#-MM>=UC9!TP?L_4%^e5sX6M`ZhsXB z;vKm=90@X{HPuI1Qg4JDv&B#%AlBKY%OV#Mo*`!UfOw8e7TxbQi za>#k4c9xcXi}q~U79)!@(38g?t4)O+iFFucPE}k5vjn4k=rLG(k5vlwuK_$#f6~{p zN#>ljxn`_zkV2|m$!~1yo2Y`VU3qR|tkr+j=#gTj*adNo?C+#uV^w>(mUD>H$-a> z6XFo$XR-^%kN!xyFWhFsM2e9+*B2x&kRhq!sOTk)pw+OvWZso;es7OhY;W12%t+bv zb)C1=PNs}N$StDDm0BrJ92HNvzSnGBDB#NT%hc^X`9N;z>=uLC)NjpN9G*bP5Sy%~ z$0*m8Dwl5jl8w|#0@8PrDSRlK=cP1JuBOXxeb*~%*-G?WJY_;->b9gzZW+XCA+w=j z{sZ?99rnX^MJLtvjSmx|-U+DZ961rnSXTOk4dddZ_#9f1M=(hw;>1Pb%s<6vtIK+r zRnoMWb<(()HR>%Bk@b_JU7@GG+r<<0FzC^Tik*m93D4;{t^sr{ghgI}!uE;f7)iO1-3nlWY z%qJJh3`%;b6Zdy>-rMW7-C~EITxm+3)$vteE@RAgez_GIWI&yQV7OuFiYo zKNQlTcbtz8-I?>GPf4CGYvoUDxps(S zoHc42XR@9(ife5yr2@7pWEYPDPngZOH?2na9@=9_w1yvy{YhgG{-C9$&gvrPFnz+7Od}tvsj0G7SjF zn0whM)|t@-bME)?JFU#^JV77APw)LmT+SXxtCBP2yix7W=w0*IWw$cj+VRz5_0)`W zG|J0#%LvNJ;IAa`{i;A`5J`>u|g^3wB$#a+BdyD z#O^40?@Av%?4-(77I%3pn&`qV!yTRJ@a`l`3Q_%@*`5&v$y{X)^u?aD?*-1a#TPRD z%M6?D9lDB+_$*JmnUI(0+`#Idv6(Gu3x#76xr@62xHR|eHF5*Azyy7v;?kOW&rK=w zA}t0JiJWRHai3RUkVf9z+Xd?wci!tUNspEH#CSUN4VmxN8w0zAK_nT$%;YtJPufJX zpVM~`!j2HkEWX9~L#KDd?X}Ke+@u*BS%O`evis%;GjYi^I06Hc+74Zoj}CXHHhXhR zo2|p;*~JxcNi;W)kI(k*WRp!to5LK+99@&O^mVIr#s$C0+Kq`Wv7}Xi8&%bu-{HG+ zs&}idDhoYx5X?_~ebnlV&jDMS2JL?Cp?og3Xqmn^w%s&tU22qIU)=Xa`y$Vq93Fd< z)ci-+hv)8uX)ldvat`feUIyxW+>DqE^5itYO{UAu&uh}5=gx>yXe`D47@MmYUy%8o ztSyNX`t0RK-J3}z`sdqZT)G9B@H@&N_1YnKZDs86xfbTyYWH3BMX5>pl-Ff026{h_ z7fVej9iBw}d2J@6ZtPPj3}t!F2kI;(*Xi9ZTL`kwvCm_XZ*i=~l74rQN2)$?YK#j* z4OBf~x2u%Rljzu=H&fQ6=Hm>Rkmbk_-DH7lyt83*~NSIg#7H2 zg;+ep7dqU~`fHzlzn~VN{MT&M=Tr4Wwk%`F!sjIIUIFNQI0`^2qa;e#a%+qxmzOLO zw&_=W$+8bUH~!I3RX~~1$?U&5VG)Y?@)2XY`-p;roWKYxn4O7hg{wt6LNjv|Lp3s2 za@amE7AD;bga19B8jFzp4v#r)X?<$3b@+2-b+x_NTGLQg(`4c!BXRitjG-@DbeJP& zHPT${&5X+HUfpK*1j#-imjwP4id{rvYwK%!JnhME!}U=pzKdPo6vJ(uV5zaXIb$Ir zfH=L^TvF0d7RqG~8j-62@2AYfESj?_dTtH3y*F7&swsf`QfWOJ!bhfj&_3(N8_dOK zgzdcmDaa_+^QlF1=s?rjaC%iOrUf%~zeyS|QY||$^ZC9ro6;U?u!GuPvwltO4GAah z)L4hdFKf1TDHFydHqR7+&9p~)Yga>!_yOn-sJ4gNq|O_jBs;0Ap}`g2cT&X@yCn{p zRJZC?3)~3Y=>0hQ*SP`WK&y^iixtElQxU`@~ViIiqtFBTH`0}3gEsrb_tMwzn(9OVB%f<$td@NI60kBjx#u| z`6fH9OxCrpQv!c*w73b0j5GL0lM#;>FzM)sa{Z<#ES__odVoPp60`-rL=AS zLrJfEFa(9VT;OPl^#BH0FSSCC!3&Tv_SPgEEKz$1ciiBMpeBnWYJ7sCuj@31(x@D; zPE~`2iIvepFeuNpNgqvrZ_|oF=N(kuhBlkx9(iev*Gdz?#YCpJA4y%?OA?N`6CSQpe5qX{S1i2y_s221ueyKmT*v$uiTS*J4#sdt zRoaF$lBrNFrt3$Lt{t2EB6AMWw(uowh~ zST<;dDqZjv`hw<^vONr|7d#i|wQv*Zf}HO^;45f!Iz3Oa?77s~Mx^)3I!^Md=+he~48G}(!;wpPKo z#wqhL!7IXwmzsew){_d<*BMeOAK+L-bD`l8Y%C1{Seef7hX>>@vaTFPbsaU>oy^N937?sIE62gI7W%icNSu{fWhK=VAJH z*MF$K!Wr}=VyV0dFwtH&SgEcQv4s8zYY<|Lxs=kQt41&H5?#@~XdN97Wt>xD@j(B(R+wPBWc zBOpWM@V=wfj&v2vbk5D+%j*u@5EIMkl&Ansi_ZB&57n=mw`9 zdv}xQy>ijLW;`CKZ2%0SA`KF9``oFXL1Yhc`tuf;1vHd~-(6O7h0-s5<5}Ne)Z2#_ z%RO?}D30L>?FqLGM#686UMu(_;^LG2IS%5GXad4Cy*6zZ8DK5v^xCrsw8-gRZG4k+ ztx}Kam8SGiRvO22zD#tdbe>G;jZy$Vmw_|CO7)D{@|ith(yU%zsiv22BURGn$#)8Y zyYp1EtG5%_^rzRHk;i3<=j!U&!@6nOMe7e#9Dn^YTjttZKF$oE_AFqb-yFKpBB%i^ z2W+1u27kXm%FI68fP0|-wPTLO3KdZJUO>RY`SC;Iztb`Qk3IeWXB+Kp=KLQvzW;2U zM>M`P@K(^jx_3eY&0Z53;%&8U=F3e3Vt$1+r)JIFQw6nK zYH+qHm2~=8D^^#_o8>-*tA3qXwe&Uf{rx8FU!C@RQ#sT66KA-L5FM_!U#@%dJZ}76 zcHUmT_&;!e>%HkJgK$l07%1#cZ@v0#5(g$BmryQe_gp1(cHrF)&arc8Km!Rkr;qx@O9%82GX;2 zN4J$>YpmV$`B_#D&2w(#e~_op&#v6I(~Zby&dSYVGD!hRtm0SDQ=Hfw%Id6Fo4d)x zVt8x(cHXT?jGbINxpNt7tJI2}Yx3**S-<;?KJo?ghp-#KDt)d((52!Kb*_JN;Za!l z)uFN-2AA4Qz0k7k$8J&u0>I-7OD#ro(vf4g#QWIn$$5D;a~Tg+sCdj=J<@kUPpqbP z97<_PDgavgA3-(ZIIVl@GwRPi=B|H;Su#cp5?qz_F~a%+tt{8LW6aY7pW_ErfW`qTxkwIF4Eiecn{%g zv1(&O`SB}N&(7+T-%yR=qiQ1CY|CQl`i9Wxm~}2i_2Y;B20G{U$QhdK%6s!pvGGZm z7tUp(wOgoh%ydznT1B!|*QLu5V7M<=H5#JJFc8 zV35%^{GoOYe{(|Ij5OU$it>ygq=fl-S&{U7(sCDsDyhq2i=r51F>p|9=Zdmf-whG< z6zE{8*{@loWFC;H4W>@b0&d$!od&m2=;K{dt(WYo5ogwSx*4gtPtFGb1((wxMwOVZE;q$N}mu)H^Ybb{TaSWc!p+Yf+Vh zX~WPTHT>Z}wpo48_{?L)VU1s3qTtY>Ue_!F5fsf+Q*6_))hO<2JA}_|KXmmUTmKmT zRfl%0=j$LMV+2^V4cz|Wdp4dDV?Le+L(Kqz{kZS#!D!lEZhGKPbrXET15$iK6Y>3L z(8?}=z##*u2ZlP68=i1p|18^{H>gkT$1Li4{d6n^-*Y5DoYm?(jK_$V*6~Gqib7%S zi6V-Xdx}SK%uE-#{mVP9Vi`4QC>xL8Bu&eeWmdRdJ7P0jCD%qgEn*5*W4{)ZZ=$lX znFqzQe!MC-#nb3xoW5>edq|~ASyqJ#d-&gUk^ySt@cSEZaLXelE~O;#KKR$iOc6c& zweBGghg0_NG_^5>$ipyQ!?DHMj2fJoHO49o84amGJLi255Qn$Ms^Z-wNlkl+ZCT{} z{?zN-f!cy+35ZTazHOIkzx-20({h@{-oCafmrxxyFCMVg8YB=5W_$0x!|^(dK#4S( zl_^`FSi@GrDNc?gw^*7{Yo|ON^Hb&~K+x3l!)AF#RQd9u2df=8%7xkCr?;9P0(@S5 z;XREU$;4wz)+r|Kb=G)e9J)k72 z<0m}c(=@29geKL2SPnY>^onyW{U@~Vm#En*%x4pnFHD}P?=*JbqWTDhBO>dcQSR-u zvKIQP7Ofie%W=~@QRT|E*W{oS;ArWT==wTq^kRmXHX+ff-|}j0v;7Q|X^ZrQ{U6m- z>V{}2Ur|N}9p`JZ)2-s4N7WR}9bX@Btri3!?fo3FzI9FrnTBy{?%>C#$mu{IMWRL4 za2R&NKt)4@&i-%0ooY#cTDdh9bjoz$`%O7lI9=2y3L9NX!D2xEcRVB>b(1_h@}Wc`+iweBQnp4PlXn&ay<|#;o&CiHD3M&ED26 z5gh{Z9KScWpqJDYg_(*qA-Cj$1XrKW_odb6Q21BgRmr|Bs*Aw}_=C<=eL{1bcRjK( z^Q%yL$b-vr)CBK^Qj=!poS4duN+&6@9JdIzu44%R%sn+^T7ha2U`0H!=-fly=mjE zLX1B7T`%Xo{RaNAN57=-L%Ptq*oXUaL?EO!W(${Z3w6D)5;Au%j6qj$6AY`A&u1_(;e8J7)z)jtSA#{TvB%wKz zCirO$_(8YiZg?>R@*3pu+d?I$oqB$d<2zF+z#{ut`d?4i@ck^{J3vks%+jwK8XEQ}Vs)-WZT`re{Azgf+*=025M%&$9 zG5@^XOy^&{ph=^PI#Hr?Bbr@Svn`hz{j3y+-7I?0jQl9=bY^WyZ=J08XZJBQoE*3P zETfD$IYX>(m^iHb>^K^&i8zMbRvR;EtCr?1?VM!C;^Xl;OfB?S21S_WL`vW=iCb@K z^`f~sewpb}4eX+4p<%M@TGA+3pPH3^a;9?{8=L9r#ZZN_a)%&iMYo`Yn>%f?I*aKSygi{oR`Aerlier>%L8IYiJ^|8 zD~6Y1Iu1Jdgptj}-KTV6abK8d2Qm2tSaUnzuuRS&7ouz}zs^ocKM`Bxif0%9x)HBb z0GdiOdXnnBg4Zsi!MXK3P0#!GV(oe$6@(hXJ>5*;rhT;#4>ybVs{P^XMno$|?TMDj zAqHI$M71cjUithDK+|D~UG)_9DY48J$Qi)1pT!A78rr zE45J`A5pDU0ydj3lZ!8P64MfMCN52ec~IHpY@=eeSi5cdx*dc zW3ARY`vIv%MRFAqqY{gUpilH9xD`rm#gpiw{k2}n)~s(e>DSpKLjOWKgZoav1pbKO zk$%?0yO;~?!SSzXH``zz#Q9qF>)~YuO~`N0C?SG7{+JL{qUfX?nK=-lwmI28zOJ%F zO79HH`I#_>#MMND6GvNg{*X&%DM6~o7b|1=k3#!j*t)?24i}DHT&o2C=tl0dWjo_M znuB(bKl&D!f873_BUC4F&iDHI>K#@)t=>;C+QuNCu2xShtN$b7+1DG~?q_Br_F8Jg z;lpX+!DN8I`CBp+2s{(({mU6i?g0ymr#mPQ7s1J}lRNx#nQ@NA#`qvD5`^|crwo4|}0Tba@l7!|)H zKcey>@xO8ab+3B6szZ3B5SW7mP<)TkqjHk(I7b7(=BLUifNM1 zz(1@NUyf&b!XJXa2;M+*@(K=Rc^gCctk2d@g5iByQ z-IXLPB0}!U9(GA@eEo040W|fd`Nsc1`+VmliTrndp8szR|No>6{I5BA(%8)Qwv;eb z95IOkaacdp{RLQHJQxa;lw_f#pd)h!`Ok@IM+;%7npJIsmfk3hnrAT^Cem~=s5I8hjn*;;0%@Hk?voDBYXQ~-~y^fV_pLfZ7}-kw?(|4 zIWUm_^alji?@T_6f&!G4VjjF+ErhPHT1k{v50@SqD`k(_f(7LHz|Is1}Xxzbm zc15k-%JBGyeks1)yns76hYwr<6)H+iyv zhrYt95iz4;#ZRYbMXXo4NV4#PL#JLL3)dY@j~VB@&?Z^}qg^@A9I6}g=UQ<8hmoI1 z|CzH_5P?J6^~JL*1gMZk+fDp6_xiU5Fed)G@&YOpK8|C%^2PhpD6ehu<{X=|5Z8^? zKx+IxWH4lEOk-(t73*wSi^L;m?gqBbVxe)W&Pb%PBTI38|KT#$uLr+#;pF@bn0pZR zdu_=jcI-CNVH zRd?8?Dx-*h8~!u|oQ%OVB#|lcEZT%tOJa}Tw}kiUhHUi-a;96^9Mb2*dItaOE_@ph zq{K6bZ;dk&diFu^()BYkh^rdFgwcrVAy?PS^}Yn{=>h3$(Q>LimRh*5G?hwE9Ub-P`KedCS`U2i zVDgl3LuTFWHq4*f!%m@*A4yswWaRb|D7}_{Y8hX`yhgz4CH0n*8=C#?wEwk5mQan4i#yY7@sbc{-z?K>^j|&3{mm~15%f>4lPGJh(GYdGokd;XfAhB%nlK*o$#mv;1 zq2;GMd!i#rDfL^_Pm_b!Hc0c)5pDfAgR;ea7bb@3s$0+NpoFd5r(*-p!TQO+&tCP# zQV>PDBUz`ZX#riSc!$=k}F6CzJBa|a*VO?%;Xn*SNeTd}R3|8#?LFQh+QL+y_ zzxh>aTMZ$m;94-iwn_NB2tc-(InF?CcEvc?7&g$rn{*r@H5Qs-PEL!J6#5|ix3|7c zC&GZM_E>4v6udIMuSKYUbzMH&H(4tRb997jAml@HY%<18l&qM<#{!P>9N&nweE1hH zAJFrP@%mTg!Sr22hB>c$3XESqbn@D#ds!F&o# zjjCmea22|C(Y$um(MO1|eQ3Iznq`i#h7(BTymu`x-7Gwe-4U5B{3Eu$Uy}lD5Q`eH zSypo*hK)N-;ZK`)S(snD{*c>xl@hw0a=CU9P9<6OOuKLuyX&fHO&Qc4j=M>ma2DJ4 zZ+hWlu&3dAyNW_ORSC^(qmhwo4gw3?dCxQQXt|# zKb1R4vea<&@m)mEz20WL}ATE^4bw~-XFtfXf`Ie`MR>m2;LD5XxtzF~cMyth%i7{5o z;}EQt=UfVgSo=9w&!GGg2dWp!&cs9Q7tfYlSx9w?=H}^e{8H`7s^2B0B#X9&8QYfd zadxEgRLteAqhUt}Wu0&R`cn;-WwZ=f{g7*}Nqm z`DEiLAIrrJ3K?u3(Hr;;O1z0b+1#~a0SjyB?_uI`3moX&wF`%=5`5s1-4DvWBBz~^moksj${Q)VibHgF{xiUt#@J^4>UP4 zsxr#OR(7&Epj@UdR5H-mT?w)YZC9IB%lP!wMd@<2wiXwc8|!)-YFbMg zDvb=TT2@+S?4IOeVTxj7=~BYTbj+RY^{`2t7)mhMw^tn=iQ}k?cF8(1KV~6`QaP3yJ7o^;MB1} zCQpDn={ada19!g5KZHr(IHJC$d=X`1_*7EeTfx8^=I`UH1I3XlXI(A@9UYrDnF^3M z(*X2EQz68W*oy|=BsSlZB2I=5LR3(DU(70;5v!KD4n^+zwhO#V-)Z%hmVDE1fJcTv z!UuXoxGI6v9eg!+)5fkMg_n++7WkGG-7y8!+`K|&t(3saGVUkLNy)|41=8U0`{^Lb zMc^KNDfLZjYKcy(qGYv-O>z{B#1R+9Nhjr1eZ?=YZR3QUcQ}z33Zhn1Pw7au(Q=S4 zZ}AG6P1_9z6XEI)oBt&lSKCB478@lnlY`+|+EN~PiwXr&VUPNNYChLs_<3B#hsQds zHH*-bW}aU9ue1&Eg-jvMz_ZfZ~GkSoG_n$J?S_+JxZ&H^0hdh zl`sg@$({(q2QDlua)hnH@p>=IrBf$b#&u8cB_RLWqzH+$lH#m=2tuTY-u_A! zH!1O0yW{x0adqw%yePq=I-M0zJWYGB%#|?Be6ZRGO_QXW2DD}+i-ZdCS`T&q!TxU}mQVeX`;S@XPZW>2&&S)WXsXEF`=$Icx>f}L z6YU(E{o^tWGeHX1(qFY{8PTD3FT~LShBVZ5o)E23P>r)hS+~W8K5>LoZ(A7leM8Oi zi#3V%Nv>VRv5D+bo1>3fr5aN&If?o-R$awiWILqi`cgfK?$VVq-tp%|OYyz}zpMn7 ziewR|Ube`3L>76-dNdY4$oQzweFC&TH*GgO$5W#v=(cBKjb#qxn>&H1nn$U>8{t1XcnBv6&orebVz@*( z`|&rX5BB#69veYHmdZR|ZS}+2yfRxwzM8IM&k|FR|@+|*Av2#@!;`D+C7`ySC)4G?+x#;;Bd>{6Y4>y zTPunX&^8qJqcL5iIS91s(H*X8J5lp^RS3&ggc3ulCbwSn1`W2S72FNx{Q@1Y5tH1o zeGg_V{LSNsglYHgtb-QlvrLqQgyZ>hR5d!qp^{F^iFL5B))2uk0ZGllD#aa@dhzX# zZ%f|E>}Z*j3v1@-_K1TDm};6h<5PV)SvrrCTwyJ+G!}&pp0x~~=_K3%l!2mTo)nexhIVO(VUxUf#8-92_&={1*F{Ytr|5^blBEOsMQYqBl>bj z?rc2Lx+p_GFx+kMU`kiaT(NU{3!s&1jtfRMP!P(aph?l17M)JUq@}BTTg#NrXl6fL zfbjFyp1n#Pjz1*J#N=3_K7-FFT8Fevq{f3s^vk7&GA#nKaRMScwF zB$)@ltUkxBDHI%;b#xDssc*YN>x$gS#|4ZBOhTqR?aI>4M&1spX$o9o%S!jkeK>W& zs0PS3-JAsJp(zW1szsLtXDSV%Tv{TkEVf&Yh+Tx<8&HGW@I6Wl^L5ToE;S@TH$j#g zv8En?$PCoZoS+(tbk785m`p;2<4Pc143)ZSXAIK%%kY$!*oke zt@~*J^|WK7o5RqwC8DkPYD(a3#n88uL}HsGGb=JTtytjaF*-vp&e$uRvUL)SHOwczJ@3ywY}6)Ex_2;D zc8F&ImM2cZsR~Vpz#V7To;OBWPbW;=+fN61+GknSKWWsU+Knq=M!g#gX&6;MR0Da3 z1evWn8;W=s-QD+KYcKvn`_OcFmb{P9^j7b4!-w>bg5Xrj}49tUWV?Q_Ee9u+J8Vh#XLO~ z12aqcnj6$a1U`3SAeu|Yhp0`y*#A-Q_gf9--}d7Dwpg{I<29eIH!>_buLIu~NiLYQ zmykjI!H2;?T{YHhWKMx$aF)p@AJW$vZ%%Lf!kff@^+4QEAb*W~dmv=de*B>PKS(0~ zp@4`wTe<%)qH&34bb!h+wtp}A6&Dv@Ny+>S9Bd%;uXu8B^b)cD-Tci|X)J{GD^@N$ zP$_xN)8r{MCvK-g=C%&5jO&x%9R-6Y+T=?exLezo^UvXDshI1Z^i=bAv7D;j{=U*9Kb;3hAk}=eheSAwfkwXK`~(Y*SaIJ$^l>(vL35L1s4)DP z`OR}xuR%c+Y^(>-%JXTgyrB#-wPomdYIo@9bK_4I=)P(qU!rcO*Ct;U5%5hb9}5VqQ9PLl@jy}Ba?D;zHb zLji3Mdd-o{03LEeuSa~~p%=3sRaS@rxsQ%>?Sh015m)OQE97swEIn1R?j0+jW|AwN2XEKea>M21?KO69$a z=f*Q~vk&??ewh?vnEIwGgQn~%5~*(zJ9DVSD5Wh0cr8}$dx7sM-Rgl;pm*o*i#>y5 z^!`q*54$R+;+ufUrR+&}g>K`S4zg?Ho=KincP|30SQ@2%5{i*JZuwAy;xhn~fBXV0 z$?D_p4ZXN^#}J;iqw+a>iH=duLQN<9{gv7*cK|4XF!thB89;RF4r&5*p}1K*HQAbz z2^28)I`c%YgVs%(K&Rb`Es-6YIqD%A**1n<;)NDzWz)sV^K>eRbNj@qTUt#{uI7neHeM%1X4{z%D-)R^V}Z2D|F8p$KGH$szz*9}?M83dZCUK# zr*t(V`!2qt5?1d#K7;Lir8Pp+Wt^rEye>+f0j*QJi+2j=5%3yXD+}zXN#r`sXFGGO zf%D7$gcKEB@4t*j=Bnu)e&@(CqU8Wm@h(wl=)ymftB(80K@2(IkoU{t#P85 zftQfR9|StSv`mrNqv@E9vFWMnx9e`@9hBFQU3xZA??!m?9RSY|J!B?e+O~R7iFsRG z8H5)D5*u4gk?Kl{NF)mNwGUENHz#$gYYa-O-0MYL@VWG{wg~f(nVUaXa@Kh8Oh8`6>Dv_lAFclix#@1L9 z6Hluu($Ad98C)6=`X#S&J`as!%_%9oJI+jYANSzYCLOP@9We#elF6K#%D0jsU=-F(q@tsgOM&Xl%@xV!lO77ug&QRWKH?m@%PZxG7d3I#>4?0wZA_f zgvMnE@;IC@PP1O%0Pd+Tum(-OS1M7K%?e&c1Be{aT+-NLg>{b8z~KKWslFsi*VR?j;SA6^ zXkNk>l~MMBQddTmTv-Z@E5Z2_;9OW}> z@9o1^M)9%<^}WP$cdE*^(|M*TbB4zvABaqkJ?5mly-v>7C9V>{hPKN&hpIi z7#b;*e~j*$r-U)iYY)F1$>up-k>Nq@Ms1o8g|O}S6%T+e+MO=1GvD5t5AT|f0}~(| z3pd;!k~HC={ga;>XUxDK7QD?hoI{G$pCoypmF|fU>=~HMU_>Erf|+pSNZ@tQ5Ly1) zqaODwFc@~7#@wi{e5j`o1X2|2CGwk+p$_zKKtPO;=cQx^5*G^mLh;l@yNy^6AF~`W za{3V$Rphx{6}-k2&Y773^n*f@3(?v`eCsO;9q!9G|BLXz8%U?iBe$h(CG%So!Gz{I zOnS#=R2(_P%0vKk82R3l&DwgHeL@IshoFwS%CLKoGjV0uoH)18!M)zxJd*Sks%kde zJfS3Sm4&)WEOA#`A)qbt>T$zcT}694L}A_|N+Z(2=sBQ4bl}J3ElHt1+oZz`8TCrf zdv{Sg&v}+=ymEh@e=tng@6B&Y7o`YI6^|bXGXEBuDb5!>la8+FG%Bf?C(3fgFV8<| zXZy_Div$b4??z>7jvmA`e;)<@YbgZSAbtRc{uZ_W;ZU_^Y|LgbKr3r4n+<+hvdl(tJyxmW?MVVdvI`GEk zgl6_{7ey^QUsChAP?COeGJ%y%V*RKS?%7W1%g!Ue|3%;YT$$XSOZAePJ-lx5RcV;Jx|HOuFxYs;rm@2RiGd9ZXBYdr{*9j zGB35rC?uJ$Bzei0DeXY`$Qg0NGbsu`&cK-I@?o)O0Bum5#BF;F#Vs^;-f$Vot1DPu zLIZZ0_3bZyo!F(<><|{_RbX<=+#xJ;Zv;c++V>;QU(mQ*aoD+ARp+qO0DO1l% zk25HRPT}(i_R7-^!J1J@1<`YicX}EXpH#SSpf=Hf~0@BV*Eks!sP|d2VhNT?By6tuVkc~w`J9rCW@Yij5 zt1-gawNY<^8mzW=2H{U?r&{SVu_zfPU@LFT5I;M20PlDb&(UQX<(G@NOL$?WiLR0} z>#)?g@>7hv^*H&9tA!oDyjEKSGA0_hrLqZ=VBwv6>+vEU?a@srRJ0*us9C$=f?2Vk zLsOmINVltaNSGp0#%x22PgNlc2XdxQz}o!cjvn4)?hTR|M%CwYOJ%2Kwwmofztt`| zm9Pm@VHQQc9hV5h#SAkO(JY+59K2M;vMu%-r&?3dy~JrB)UMRKl7iCz9>(Zz5n{(o z3ztC>AcT(#sZUIG)OLsTGo@6sUz-h9)zWl&%xbU<>$6CFRKe`Zw9DtOhKm$y?U1nP zXSac;J+k8evq?eEtNe>E*vQ-Tz?h*$CsK4XA&-mf;0oq1v@I9-)XqtURbWm4%)S7kRhCV0_Foz6jJo;0f-Cgf?CX`p~eHBw%>gW!-T z*I3=j?<^{+O_a1PDDwzI)s2~2%^eBHU+!UrVR0XZRoXupY-03+B&5HLE2Yt-$??~k z!nCKjx01yWOt}J4P&)UfHYImUp_iwnJ*#9rWT|p_gi)(Hc}n+P*Q@ri*X4N%4X`1c zuBcD7!Rkk6q52~2Dc{)oh!47<`oiy)%j!Y`FxO_q*bOmwYB5F1?Hb3l>?5@jDk$YN zy3EeUDKVjFz5?9%rJcKNwX+d)%J&iAqTh?5^ag-T&DYihmnY4b(qtVkW*xMV2SnM7 z%TB7Y3^k5;O1sjm%wkXRH0Qi#~kK=Cc6wQf>7A188wi+K!q1`GvgnUrml)OVoxh}p>QTSI`%Arz^Ma+!^#Lwz}|TUPa+9LM;)18OJr{(gQ0aaAQUT^A&aklZU8PLTzM#xgcetL(RWNterQ?qGwlpichlWH1d=%h26S zR76cYyz-BdJb!WP{2z+KZZ9Bb2{hGclxWdq!8zEisHt1p`z%o%X8NKMKrU$yNV z9{2-q2WWs88PdwvoE(U-jg8(~y#s1hk@9g#p+{_f#7}FAo9yqjZ^zl8j~Qc_AW{rE zU-?^lk10T^2ANl?7;2HL zi}}~SynS3zv}@pWz1e_MMTa|f3Q9*rH6CHXUQrq@TDy#pF-P2_$s8PD;43?gk^&9- z%SME_3exCz>=h)8i3?_?4T-TLQ%f=lV-meEh}|Ha6LA#Q^;j(nQS-R~`F`w6K{Uea zk_V%P))1eUv;qGGq|R2ilzFnR0ne&G*!6drml4Mk{KUhLKSFZ?I zKD9)DZU=H%e#GcoVC;+bvLDhER#FKo+-HJ6)GVA8+6^fLdjFAgHJ4gh&K|08)ox>J zN@e8G3r)Yv6;a_(t72E6?_DK`PBQC3#u+TtlesOU98Cw0Bhoo}88*b3Rnup5x+at! z!~ki>14?}1rD<{d78Vm~;TlY*keZv-iAE|`lR}F1u!6nTcq?`p&I+_9z$=yepf4k3 z4Ph9Dw8P6Cw=8V41zrx|SK#FZOO`!+;n+?cjefmvTLS(iU5SC(VVP zN)^!0;c0=It=nauHhoJS6NE7$vvcDC3V)Bi3cHp$f z%GU18oEvLA+TX3aba-R~;^#H|S}E&YL6ng8PX z<_#A$;+V~-%Ti{yk1@OPzj>bZy-zyoL6D{6b0o!xG8nep9!yxykmf$H{e zaG*41jDYJWooHkKI5RY63?@yOdg{*Z$HG_{sgL&Mjb#(Clky@;WxSAuqsw^%P7u6GW{E4Nr(#ySqedQMg1 z1`#noRNv6g1Afxz66j)TUIK?exHcnJ*4W~-1809EK0HTx(T@8tGhQAG(DMdCl#|lp zj`6B}o9>=A(iPwG4&vmv{xzwL8|eXx1uW;vw^QEFiS&Fap<+!_#TsMvkQwA@q-$H{ zZ(@#>p1A)iJxKt2!GrN)X8z5WH1_?4pYg`0ALVEEBuznhUFSz*5^N$(}L7we<{FX`gULmfRC6x;@6q z^0qg`#)p9!l{;LxJ}AAYFu_TWEFF_#WL|=jEK$znZ=9v70LD74eAw6#q)z0CT0iCZ z^FP#Zg&#et#9%-`6EHwPRR5>nRm$1kPTAB+#FT{bzZkgmEoWp?jIsQB*)B;90s<*c z3J63_33Y@j2(Tr#QfYQc$t5AmbDQMWji%W(SuG0M!(XY(vFLFOP8TpU6S3z8;)&Ge zCOvLq-$3TaOVH-WQxCrEF6_Z7l!r0^o_YJ%t-FT3Y+1lj{@yld1Lhg=dYJmKM(~ZH z6gP%8$+gM)Xj3-n+!B@`_GYIgkxXT9W9TwonjBiHDZMm$^jMOKWMc{O`550V0diT2 zsm(!(@rB`q>BhWMKxzC5fi(X#VcIZFy`+ckP{2T)#!H3fBzp3P#Ecub@kc}+JyxTU zUj2C*F9vnO_>_xZlUSa+H6uH77iXmP7c4}d_rh;?nNLU3TeU zi6Z4D7^Ej;Sd1Wwsm%ZvCRvyXP9SN6%W0u0jB_(m^ej^lAtc(t|=R(GAgHxG!yVn{c$mC|fwFJ$1{_@7!6Zm{MOZM)Xx z-|PjI8#sbYyRy=gr%b%IA%l4-S#pvb)D8x2!t7G3Op^RhK#W)Gt@^x-Hrqb!FNR+Pt}Ogkpi6VIE}hkJr^t@7PQ|pA@z_C+ zGG%(?3r_e~pKJK%nQaRLn5Ip|)R)@tAD^HL_%iZ zUL`r=8P1P<{!JJEn}&uE=A$EtS!?kN>MVOFI?Z~KXA1ZAi{f;$kBi>nWYatPlXRuQ z#VbB!`ft&6WVW|AmcrvB<`6=Bbp$bMt$LF>%WoQI*-zCO!+dpSai2KG`Gu57ge#Z-Dk2B!BYW82wx77o6WXJ_Hn@31dDFRa&I^gnGoY5wU0TI+oQT|r~1r(OP#$iSVK|V_PC@0|YFz7AiE2)EskG!9%Y^0qegJ+i3fQjDuTQVpv8yt;Bj8j}8`$HdceNZP#DWl>tahvlSHTd(V;lv}Th zJJ-T&*)lXet7qkqbXGdN=Xh3L#_OUdozC;3D&5Y5*&}k-e4$5w2#o}vpqDvUVwR-2 ze+YfEsg|XXYcnBz62v1^rTmp@$~vQn@Z(c)0J}SVp+ACMk?d}4%ftOOssDd)$ZPNH zTL%9^?fF09G3Ea$)c(&ZQ*FAg^OguoSiI4=lMc6|6~$g@sqB?c3tgxXAz{mV5vASI z7@+`2N^A`WRFeq_r|psubV{aP3MSpY|A1`9LC`@K{-}c+=`=uVM&f0TWXX%P_s(-} zW3}%4{dK+{XrG8@Zh%4qb0U~FSa@F6Yd!fO3C7*DgM7FT<~P%J`avMf-o$+^47{la z$*?<<551t37&D8xT9AmNk8sd&kESXq2Bmda*HVpr$@7 zoS8I(8e6SaBuhk7ahe;-A;L~gYLy3_ZZ`QpOe;gp6^quYig-|~u}AtvLXuppR>hHK zwnFx&KOk1uRlR&DYxyW}T)o{*byVR~Ka~?(exi>W6GnwhYB2=M&`5Se%nC$g= zan$a)^C(yxU)gLt-+V$`AL$NBkT`zwlUP1Ey~9)I<=45jCmi8Y(1yu$*97Y3Y6-a8 zmcV}3^@Vaxabeq&=pfFB(|a_j4n7IbaWRSg}fjvY$!Q13`b#wb%+>hAwA*z|liB z=x{~`>)OdH7aq_L3(0d@cMv$s22m#jyzzY-)FqhuGhteWs8T~gfMN)S%$y##t`-(_e*@kTkaBG z9G5impmFA|92Ot79vlELyR{oU8;aD?7K&j>LWu*kYGqOK9dUNHM` zGzl{irugP{;Xpd$rQ!n`;<;xHjyorg<&1lV9F}3pM-FP<{hr^=h>mLr(*p4|G9H?;S zxO`BflZ-hy@y_kPb70>%h~|7pJQ1MJV8S0EeWG}GIEG=&GsZnpu+P|(*GFE=?>o&~ zaf4DJ40y$A@Qn?91LyYw*ZRU4&{9nBQZ%6}`)#q2vv)N-=GJ2XoAP^`-ViUpZu6 z{@0~O-;tB@-jyl!%G zsyEWw{%f$?y^Jqy`2HfBzN-4Y%r9^H+2cPgkNq)oA{)hb8UuAIK)5G>x zcls`y{?GS&Ef2V&_wF~Jb#qVnWIxKE*DF)rdjIjL!|=}o!2jLX*LVK}{+-y@wf{(wFuQg^dKTVLPHGaf-|NDq;*H#I})vXt61NqmZjNmKBo499K? z5sH^7ye_dgDaQxI{Sl2&$n&juQ9OJ3ZvYc@d88ur9ln9mfx8g;Cl*x?#dylf7_X7!M&z4flsmznd z;n~^136`>)Fb8RG>x#L(D%|iH{^kBP{arlhwv>5zG2`AHU&Ib5iAh#v~dO+ z3@fPgt2ZqaoSek_4M?D6#?L4aomTddq>PNFB8;U=^Q;vR(O~{ldr0^m^wjUnMF)$zenPak8gndrcsmGo@U^$C;uG@^>^ULl_Z}5|m`Qtu?O15_d`ydo z)vzMlU$%x>OZ=v!og51YaZh+)F2hi&S&9D)tfJOHi@#TC1I5Vn0y$;j!-5(y6Py(v zn8?&w-;A($_%WbXTHEX3MA9tGwSQrK(N2FQ7OWi!sN2XgR2AK%Q|7Ar5-1@9A?uZ* zJhg+)4=!YiVvZ~zz_F&PAEDaV#lNL)Sz1ChP$1bYbAlEvr>KIbCB>v*57pLcDC#C~ zUSR>-vJp8{pJ$R?V;j~;*tkM3N6g?Dh$?O3hu8lxy)0gx%&U^D%b*<{sNQ1g4`LhQTqbyH=+>t8S7U&&sth{XvzsFb?Pb9A zT|BX3DZ7yEFH|!tG}jwi+UTJ{1sViNbWmacrtF}=G%-~Lh_Xsr7=%AV9jOaCwpH<; z&BQN}u`Ps`TiBb#h}bL2S4Y)WQ=w%#At_-OVY}05iV#10=}UibB65NH^{b`P0qrW} zvzV83SB(E|$wbWb~y?aiOJzT)V{CpCuMjSDsoMcw0g%U?33w5lWvI2!7B z_DYFuW>)(9@rH7UG;gD2I>74iSCIs9D#AR03B)YQbfMrCnhZONPyL#cEVdLXGoQRx zn^7@v)7@)(OL!WG9<+MVXEYSrExxWonxtNFVMg2>)x(1R>!JL>>;^3#TPz->YY^Y4 z9J9^9SGgPNIP2T{(Y9?!JTOpVws1$$FjH*k;;$t+g7eH>__FB(7Qj6W?enMqe4HshCLWN@&szehFs9PLu2QqgnF`sS+%yJajHY^V)lu``c zyyt6%-0BU8=qc}V%TeHQSc5b!EimplKP9InPe07W(uyTszF4f|=n{YKUo(}1LkJejkD$zQ>X+c0$>5F{EDDm$>uusYcP3&(z2W*`EIRm(M8^*4u5 zcD~3Bkx%pALhYUeRaR%i1f_ZqX+b z&CM%vhpi@EU6rEkdVRuZFUGfta&L$agIkw`NRmI3Eq6mS6KDDxvw^5io}*vP+Zkpo zYN@vbR^(6drx3H2bC>zD3YITcR$QDsx2+%;WXtyPAw7;>tjpwW%Ab|7D5j0pV?F1alsns+c;y*^Xn+<`H zXd^>lZ1AaIXGHejZKQjNS1seu5v_A(S;rtO4wkvl<;K0_NgG-_KRQ z(f%gth3CmM9iw%wrog6cleKF7jKs!bc@u!%9kL)?$Mbu)ka3WDRyjFu1I@AVSN}>18YL_4W5f?*&y2*c${xamG(<>& z!A9}7ku4PW;|tx7Lk-2oG92bl)6=|{(yi8;6jOaX%A9;=nrZov)NRM_f}0VcbCg(h-i zclnu!x9&{W0MnjWO25p+1AuXA`VmyK!`!%p(x1A?s|7F^qEU zU>sr*GtzGG*VsbRn64f=NDJt9$15xQs3gn$PNPr?Mux}W?(x%K8KX!=@d5amX;bz* zTN|igBE*|zxfgI)f0jH)InJnO{QYvM-~^cpc6s^jAG$Gb3;D_Xz@>&5Z>;4|5Me!Z zBf-RM(022hP#>_tIKs!pv&*r<0!4eoFbjl^O93DqToYh9wZG{tZ;XK)6S3J=`ZjR% zrU;v}9Q0dRTF@-K-zW?s;o34(MKj>OPj!z?Qv~fdM8PJ#$EJSB8DqJszFnayR!Njw zQqN@EV40#layWvpv75-S}ltCX+% zxWwj|&Zt}S&0!p7GOKdtyQQ|RNQfajHV#KLVd(V|pd5$ysWicArZH~&nq#S=Dc-EO zKqSG5kqN&Qu8X;LHn0j6XNbk#zee#+%-hl~3x4f-7AEU>4ZiC&E%JqH^FD^-or`B~ z8Ih06EW??ABwSut(YnI)aJG73lL-cOv$0gv?)8qRyOi{HY1B1GKPx+10 z%P67uAv-Z5S{5BvlO|BX**4afRy#^lWJ^1!Vb(QD%}j`~WE~yLY5HCqCx)li!UWRn zTYp!6z2o$u(?&n4v49y%NiLw+OS&jUWJ-PO59XEkJK%f8SLNaY#ddCzr6vNRdDSaw^t2Co4!pB`dwn>4jU>w z`%1nX3!PC>>S}|~sI@gqX@sze0sLqZ@m}L-rF;+)xuCMx=gGjTS=R_J>JJL+eHQe@ zNPCnT&(db}dQDb_t38~n(oe~eU0Eihu5J3px|6%S;`1jsYvVPjm$};9=6ctzo{G6b*Kp^0Xw}&Ztrq#(H^;4ch_LI8rT)1W8~qLV$gTZFi^SztpTgNb z32te(tK)V3XtRApMFQhTU$(>C)u zoEKVXR@?Pld-}qYtivtg^k&zUe9tI%JuiG)eYN6O82peJ8x%*l$-ausf+(e&Q>!~K zc5dL#c_4CgNf=kV#aw&IdfS(UVOJgTn$hX=dU4hY`-t__Tw9yt-%JgLda=L9#`MZE z7~)Xj>Y=Zp+dP!6SJI!~vcSKZAL9DCAn#J za4dl4J1QtX?=qA)fq2Wn(qv8ktoA2K^l!{pfDj|o^~ry`EAi`%lWA3mK;ssL_C}z+ z2&qD(qgb>l%RRGJW|6DrysFk~CX1BaygFL_T(jZaBR#o`WLrrnV%psSJ_5qc0&=_d zrd#e}3=ZVp{X@I)(KSll5)X_!Q*=#m)c)Cmx8c9>kTTo%PtL3wI=^~r#KKi?Y&9TW z0OyC>MJ|^?ejdA*J$K#QoM}+?Nj3p5g8N%R^K#|79pX?5{)EIly~;`7c_5qDTHMKyowc-GEXt{YG%ajI0i7z`*yJxEHFZ|X zJED#w`8O| z`I_ToyF#4ErPq%7{69!q|1m>-{&n)p$|v!xim@kj_xHBwpkAm$u;>sg?kp zme`Nrq?6Z@|Ap_;&m!QLZv@`J&|RTCnqRmr5%K)0tQ{e(sq7_5b2bmS?9}uviTzGV z56+iL*A&;}rhu??o6;?Smc@b@_O^hK)YMa$%hCe;~8!xe%jU(P=ULoGWT(hh?XP_s#soyRTp)e}M=&6RH}kBcM1V zBQ&-HsRu$z0r-+`N6r*l#ZK~<1G?<4&`By4;f7Q=eYg=$V`$uUN%3U?v@+Y`;;Z7V zT9rng`wSYFTlP?zdKdfR%BMxjd~a}=Ab_tOs>cCZ)$Q7|IaqGRjo7|xp0_ePuGuxf z_tq}UP(cdebIF`~R_E5)wontBF_95Lf?2lNzrF}cMp=|CJ?zjIm&^l1%Py$GF6?Am zNYw?KwvcIqqG(b_A^BRi#4erFkIh|`Y{_T0 z1c+O2MDn@$tD{fdmw{Qbw*bw@c22xSeFb#<)!ArJw&kJ&>SKJ%>q6wJUQ&v%-Tej!U@EqqZgw&xERHIB@lLrsJ6dp;%O%Fn;Hc?HPxi1w+-f8Ll+4-`CF%GVl{hm zql-F$d~{!Nv)W6d(&#c&HzlT4ljQaRcRz-hpl(v=v_FJi9DMd6wQKXibM=hf=oT2g z67Nq>b+mpcR)KrAPniVsYGF97xdG;Mof@EE&a270+@f$8Cow&0unyP0Ja)ZiY z=Tkjy#~yL=_z&JecNW6M_bo>sGvTp(KmIb5zyX3rs4DDAyM=Xn_sv}r@f~(Ip!>R_ zu~5FV_3p>$peVi~SvoYU%1l3Ia8s_ByDKEQx5qyE8vop%(rFdnKY9-a>$;Q;f%q-} zcTBJyHm^h+&2F#(j9b@|7yxRv@!}@yc{ip197f{dUOc6wFsrfGG9c}}l-OG#bGIAM z;`!ha7W6J%<( zMXqkevlOQKaJGn#-4+PCPn)d!Z1PX0Z_1v-08OLmwJX?O_f0MI^aOqs_u6hA;$Kx0 ztCDADWQXeoE_q>TZ{Y7uQod=pncuhApz!Cr7TSa=`MnWHpy&n!2HpwLu%dWuNbi3z z&TW3d7{IQ4`y4Bw6KU*4IQT*jJqkn2Vk4&HhZ26FNc~{Ve90>;x@%gHgV37~u4cSt z&;BN)iwW-L4085_fK5x>8<-<`ca!r1=9is6WhKf`Xcg6qKV z{z&LUk3R+PL}-5s`Gq2XY`)K!cV_erBJ+)#H*)8UxisQ5c|dGRf=^oP85MIywEMXb@SVPwZM9W6*uo#TCubW<+n{}gleX@9yRf^E-{BjXuee4>Bv{d8V-d*Fpl~v} z0A@TGyl`3)E~J4?e0`+X!(S};Wk4_sbNrv|3M;2E7Ujgy#vHWx)xedhxB(>o_bjSJ ze$;*jy0b42Gm8u1qL-@tIP|JJPCXg2^7Vd6Cbc4K9`dzHb+5xaR*JOe#YS0&*HxDFT zo=182J^9v2P;S{F(=b{3Sh^WGi}Z4&`;pC>?53Q$S{KX7dl*^$`gn|N*7`Fypgbjt-k?_ggV#GUn0>H^l-| z$@1q-r!*^O)LUm+@=uw1_r+FB&INb6nd5j~_P<4(F))-bmRN4RWbf@F%Gc@0N9nZn z%;>*ju^U10j#cN$lCMxpem%=z$WzNuioD~LLD>ag_{UmtKpBn@M7Tj1(S|g{X?gPM zPMj^E`q1ePpDmiT*q_ud%fB&srO}t2y&-vJ(wF{u2bc`$hAU2A=U5{kT_h8#p#$Q> zJvej1uz&f58Uo+afH! zTqvU$d4Klv-3n}3T~5}xR0ESbpfL7!MX;y3F4#mJy|-sPEO5fpUHb2i+073*;pbT& zaLkeO%&!OXx{6z}Z&9ioQZ2|%Cc6uQw>{lCMWmz`4(wds>H=nX1Kv=a=n3|u z1!O|Ee^2~%jPNN*fD`DePrwHPv9SeqjAta!q0uTuJl_rJ*Y^WWdl(5-~4iE!( zZI$>AVR4KSTGQkMecH6Y&0DR?SX@|Y@Ll-a(yQn%g|q@7Gl z6_0I6;bgn1X7Ck!F&f*yAwRvI*PU6GX8-Ea$UQ8_(2+pF5l0b93E%iaER z+7^v0`rFAxQE;=qtb+6kA#*wBVtoJaF5Ol1&rbzX|75P6J>-Ro?WB8IjZttl^-hj6 zPrsrkX-R&8o?7;=F9qgd?zWg72Pn0&O8O40IbYH=GS(#}f{0`S4A z+`}t(07f8RG^tGgAuGRwY<`dzInkbS||2irIQ zu7uVHopnus^o9GA zqAz@b>X-X7;eWuM@YFlW{Z9Lxuy?BarSAB+H!^)U;ozq}cF&*i^;g*pAXjku(_nw*1LqJ3y zt?fsy2?)yoqdz^qwGK$a)r|1|bz_0;KBNVBGV~I)UGpL4RR4w2qAc{rBFby${ zM~(@Mb8&2RXpUm+B?29bag4Uq7K$CS!OSp0|xwI=Tha_z!l;S;s$TQ9GR)&n*u zI!9&aAGt~zH35vFwsj3+5inStFrLAE#^QA8h{dBlc#R7S_lDzSP10vviO{8wEKVB_ zRV0kieEgiz&r>j^1M&0ra9~wBml&5xulUZ_Q;`&GYj`k!)x-Cg0bh@J9 z#H`oSS%#0NakiTqlsa?Dc|4im##UYls#%g^qij=iUlz`zeJTt7SVqc7a-+oOr#*3C{ss4Ejx%$@iM7T^eI8?Cg^i}p^XB-?uKsJh z{q6ECa=L^A?qChQ9Q;gnqX&U>m@2?^jm;n0UQ$?naym$E0 z&^DwFNhfp0mM5+3m-d62<>IqJC<~ z#xV!K?~7y3(3&YB85NUWIL|D-=bXLfJ715-+l_&wJ(7kw>`W3zMbdh*qjr}Zv=b%4 z+)=hy8A=Vq4oVk2&P3a^k8Fj*XoK53Z=i-x;!2f($Dz!HlXu-Sk6b`|g&L52{)^ z`=^EFAN1jo4|#`0o&)fmmd^yIx07P)Zq#@VxK9{T4xok&h!0<|t9arH{Y@rVP1svR zN0m9t*elAq8cfe~DBFfm@laL0s48*R`>s6%tI3Gexv}Bn636Fx4ysWtBr)b+N^F&})|oul=hpj1hrgxKKlgXaGFRiMe3j`(7+D zhL`=*7Va|&0p1?lPRgy@Jhq*lWhbZawDV6pPlT`CRokNq z9XVOGaOwB`c#2q>H*&jFo3qyJGRIo2SS4sRwYLmCzV(ydB{yR^H*;z96&_1?Oc)}j zO~NB`2_*3p-l5#ScwaFi~ zirLhBX+qE)!&;LpOC;}gq2Y?E4uMWlCE$jk4r%Va>oiaO+Jo37p6FS7fMw+mY>HRf z4ymNRp@c!46Gx1!g4*a4HkZ#y(}VE|`QJC}E;yO)`aiu)(?2^Fj{k2|=>OQT{|i5= zRl8Hh7RB&OE9oTWr4W7zU%({_SiMw+nLVy=l6#+K$Bm>7}6*-A9Wdr-~zA!_vks@)lv=(UMg8#(K+lT1fo- zV#KuTL=<*~%3q_+v)@W(p=h$>Xr2s$#Ld^`LYOMF^m-+i< zncuzTMFNcnl*a?2p5aBjyXA@N-9w^ZaSy&S4YuXX=T0lrC3HoJk6=dq3m*1+{nE?$248?EKFS+B~2n( z^%`ja8IyOFS|)#~(T+l=*35aX4i`;o_2>-i)O}%+*+b%Hf9YJnqJN^I57ZI5|02a51`rVc|F3BAU&q`33L2vt)*dM8n119N*EV5<0v(+i28Gae9kn6a(1GAX zv@V^j6Rnn+*K%ygY}{KrkO&knV)W}(wY6(rwW`r*)jBp1pxQNUs6E!TTKN7S=ga+k z-wzkWedcD(-5Zhw+55&xob8Z8zz7A4iffkh{#7)HrW zg@2-~5YR@=P-vXjj4l5Qy&h!*QTZsf;3Yg12)&1koEd`Q@emz# z#E7HHhz*60i~Q|?mxmA~_jyY>&sSi8l#e>eN3!JLZ}R%z*s2ZtC}4}~MOnj*qv9QbWFey`

cF39OH z%W%-r)dl26TCMPc&-I`I@tBy=SJjtKzF%{h7VsUCN6k)aDVZ(K5BEAbEBt9InX%># z6K!1O`3p&o2dRlMTRD~vvec>n7iI4homseTi&kvgwr$(CZQB*w72CG|*tX4zom5=O z$=YkTv+r8>wsY5cm=E)5w%NY%jnPLR=q0>cKv~=D-ea77*I)1H7kF;S1 zxrG;bPcToB8fD9kI2kjSc35S6>sS*hAY}Li*Vno-VB+x=c$`X2wxC>NHk_Y>c&Eu+ zg#>FCs(8RL(5}1TB3q>RUrNP3;5Ip|X5b8j40@XrUj3O+mC$ zYrsxq$W3SCl##Y)9OMI~i|!B)O<)iWN+v92#!Of3&gDaRppa++#{Wp}h0MR?VD&`~ zDW#5tp;1fzqfPk3qws}qC&lr-9zh=~K=BU$lhP>rf_OlLdX`KgJN%-C5sz{S~moHj-A99wb&7ZU&e2q?(3pr8G^;JRz}kq)u)ZSEE`jR{roCn8^utG zOl|BfW0AcN*#Wk-1G~rUpuZU`7Y<89eT(26nt!M`=R`?~3(fuQ1aP6@j}aN5-aQkw z0$lbI30Vfjm1WjE#He#co7E|hXFfFz9EUTuAauV-r7la`MWOBaa$}rCb-$BC;P}XD zaAV;P^ytVokoaz-wx<3`ee88k}#t{xb&{6e6#5$g^3JLYLHaQ(G> z&G0IVE6Zk{k1hlB%OvV$!f=v;U!BkhZPTnh`0rcLr?uzWF$dga3$E6Xwy_@532!mW z(=Rt#v=u=(3Wq`OR#gjRc0-a2X;xzt3&p@k{F5MW7nb&X5Pe(o4u>1^71EV z!Q;cupVA1u#;Ef*BjsRahF`cpY`YIK7%d*U?8|jDWAv}j#Fp`UPk=`xKgzK>y9#|P zJPJ8@ZjpgMp?fU~pMghG_L(bv7(ao@e;KVh)w$kpym#MTX57?q#z!V5Yc51f`FJyV z&3gG3Z|ufW4dALT6S!@#o}ka?OcdfQ&X~O63?z1JLtbY3WV(CV@~#-tPX5Leix@?m z!3t^0?X`hC1y9X_vShiFbbKPIR&adeDtzQ^-Gjb92=xk&Jrp-?$!p((lv=L6oj9@qFb*Erv=tx2uu z{lAERn4k0pUB{TO%@^sjas&#$971Z1SSPa6`h*Ft8~iLP&6;1ePUlLh<4jU~AL^RY z_eFh5Bc67}zDY&FgxTsKO9-)M1E)eiDjX|>Jh0Y3eNOxgQynqu+b^03!mw*im&;qU z)Cu5x=^`~W6)8+PqtBDw{tgJoE{sF{VB3(o*tu!txaf!~=`Q+|2{C z&J2SM@vfHSc-KaP&n&QXxUoo#Ic78)NTW!Y#UecVrhl119;jurNBlxaf;o)#>2QxE zmW@7ASiZuFULfN3L3n=#91vwO*KEa%I{)gxP%x)L88p$FXg=(xHH#xmDVZV@uAawc zaC=8GlR1ViqFNq^>HCqS2#_mBkStFH7UuHy`2Bc2b9P0t#H{Kc5fQU8zsjh>NIml; z7*mNFgUry;Z6VE1zm+P(TXhKwB8wBhfltdF9;HuvOt0q*Z`QFA!@_O>dZfk*nJ$y_ zz=Ut~1k0n-vZ)2yo->OYM{goLq)7lj-+Z(5ey8#Q<*@wz1@o`?h$LH$zW#fg6CLu8 z9~A#5d?fn4(n;0J@xR48OEhd;b=A;5XS%LUkB^sKqg2XTHKmiG5ErZiN$E^CnnlaL zr#+I^S2sq zsh6poLAW78CwJ!?x4P?N#Hotov;NxlB1KT1`}}n+T2R$`1MkLmEl(+pXR2FWoM=ll zVaZNU?o8J{*DvB6y(>N&QEL^)T3Z7$M?!~I?+W)sxio;3g|+hXCFC+#}HB&jGprQVaGv8a(fYYvue z6x`^6i29-;ulZ=wS$`?ePF8Be*(S4X3$^Deg)&GG8t|WaxQ0m+xp7VF`U;VTInZ&gO{_4e*!;`slA_{X2RPvw)Y;TvDft7^>lt*@jnu%X zQ!}a)6Q`bZ&NyQ}*exjIWN(Ld>OvaJZiGtJM4VKEa3jmZvS>%+el_D&hEg9SELT5 z;b|j7)H6K4?6W7s#ICBI<4BVl)3V8 zg$HF$!>C+Th$FpOSB7S+KS~<0{ec!Q#q`6%hj1oFv6oV2TJdWMYcI1Ciaq-svp&-u zwLhInFB`s|l{+L;UB(NXzs;Tpx&G*lR8J(wEd+|g0bT7fDwN?t`~sVcF*aWj6;H$b z`5JCp*p+BzGXHPWRFa~|qo0U2weotYJ;ZFI$I{bGckG{}(|KqgF31&Jp=2C)@Snu} z-Bpg~Uv#NvGKmU(%&Bfph9%kAd)1A>sd}e%RSxNey6ULH=O#5dO4=RJFxAmIdRh<~ zS~inYFd1y?x(hNkWog!KaaL|xce4~5rhd6c~F1#m#6iCRROe6 zcC0yN?#>SgbmdRw$PcrLXz3#t$~+z7Jc*7HQTviDbz+9nu3}OCyn8S*XP)1ZkzK#; z8<9JL@9&Ylz<$HX9LY}@P$l+Cb7>RT9T_Q0F`0EnWs>itWN!?F7vL|FV~qF9(O49P z*JN-AGQ4@#B`Bv7RHH9$2yL(2$~bm5?&IRlk8pE7_p7+br;d@0QH;@*YbQv|xeA5l z=M&!Ww|4_mi~dGBLOGfZ%L@&lm^T+H%?A=n3EF6kqs_ZsKyv7lQb7b_Ye} z%p?uZ?(#V>5%V!+_3p+sMI6h9TU6iLr>Xw{S140n5q$QFsRrJo)t{%439UR{sIM6g z34#%>wm))x<{n^ryv<>de=bGvP@<8t(9`K)w_>e%c|&SKY*V7QGG( z|Bjpa;B4RKSO7d*5KRP)b2-{2vHNdEx%#ZN9FZJDga(T6vmiKQlJK8~6b&rlSV4Y6 zJ-G5ny5m($0X#vokt4grJ%8!0^DoMMS!nM;d4<}-I+fIhK(tPXNen@GP0A4fz#~di zu7s!bKS^Fbh#mSSx1c?ncwk$;hIG8S&7(ohT0MiCVdlNW=CUAq1jYI|z}rY@4C^j~ zOwwD(EpbOMv_G)D!7_yc;G5!&%6#f>y;J zu!b7O_DDsv1ZqPyu)hlcp7J>NOer5$a0kz9;7vZ8-*B;oxkAd=kdGWkjIv){EmtsM zNB!G%QQl9vuHhLIw`8Y{$koBoYex|6kLlafIt^j7Ua+hVp>zW=GQR2#sXyR0Z}D3X z{dA)doN}B^!Ylj#Sd-f;PPZ%34alAIPMmz}vBzyLLvIzRAILAqcP^E*>(q?$SwlWo zvR%DP&mU-67KI8tL6L98#hDt0+ zj8(?prIUn+cqeyF-He(0OCm7(*HsQ;X)2h`DS!scep!3#3~|{B!uieo7Oz*F{D#ot zGYhjP@y-xshRji*8EYZZBhFB-Ya!G#dx+v=LH%C5(lS}Ir=$G3@(gI%iy)?>2Vvd{ zbbGC9!uGow76~8;fOja@va)oyH?|&&1fVC3Xa#)S~_tLx}s!jH$L-EoWTi$mu3GTagfzJ;z4+} z8H#8ZUSnlI{J{5(fr|pErUBW|f+-Aj& z4vs!gfV)Y@@wmk?%$eO{VtFPFKTs{*4ib-TqZ>cX4nCyEW4z>Yj-(yZ9KEXbDCsY9 zv4tPvWoMZt{n|sX-Jt`3wmJT>r&@ZW#uQd5*&Qu*_2D{J*Bj}LzOmX2p>K)evM?S~ zx%)R#Y`xgG>+d(%3kLfCS>gFVH6J5m+y91sW(l*>gMx@7y;jC17rh&m9Cewxy0_X$ z(qgcvpfKJ$>&<~tilrr*=->PR~h++;2x}P>8LTjX(Mq-!`ZQ#LD(cH*Ztr zgqC4>qZezlc~J(Wd!0H7<&_5)MfQELOHe&T$8IX^d|I*^Zx^v@SL>1vQ>d8yuB?B)>vCaUP3t|dF&!xNGRi5qBCsW9+QzxcU3ChCVi1{7SV4m_l8li()hp}mv#t*4By0uRs*717QSPJz3#q!F z4&!5ro`P-bIT{l0NeZ?W_(adJ%_gtoG8Z$s>9QR&>KB%u+~uvh+^0P88KhRD2s|11 zAWY!u-~1V}K}WB&8zu5P@yx8=Ek)hz1I$$CSCzC~8Sslg;0~f_XFC2g^+SLv_!Id; zo~pHk!%R!ip!^U!!ygu`Ta1n1OurD*Z;=K#D} zMU^o)?KC?ct*)U5h_gi?odKq8pra+@KElxwA?ObaM@LldF3 z><6KqIFSQDNes`4)*=`A7j`(?c&9h<&Ip7B#**@dI4Yu!+(THrQ)v67j8P9mKZ)X_ z+fj+oMFSN>#t$;;=|-k+6xyx{%sY@ z)=RQ`*0+UseLp<^7nc72Vd83Wa?k^Uu%Ww4pOzX2RIVn1x8TICC37f4?D9I&WwN9A zGocW-iU%<~O`cF*UIbnNxePxbj<8a()S)BMHI|^GuWA*?WFG2MvN!cn*-K0)G;EGr z(K~NE%>$g0=B0nb5pZKyMM~6S7mVDrb7WfYj%(qUc4}q<2Y*UJU-T8`Ldd1*P^8G9 zp%RBEHw1t;uO|z4=Hh!6j792 z-#ZDc-gSLTDA6s`lly6p8IR+ssT+aa0O0gtV|cxER!4?#Q1T{5#uABC`%<_m%yc%# z-9C`89Ys+3XLnIVz(PPn$4Fl;vo5WSfa;b8&sMbP+u_*9#X3T7DwcBV<1^+64Rg76 z*T|Kh(UptDL}rNw?Z8#XGUZ;>DLNLjg6^)G4D7hDQ}~|Q8Dv;))J1g7>2xh*?t@-D zu^*aMGOWC@-51LEPdp$LAjJI^&2$shwqXJZ}v2GWI7g>O;q|zw{2bQ^6L+ zvMEG|Y}Dc;f18_SbjN3qeLP5t4EHoWU_!^N{^Df=#57pPzb}VX41KI>m>pmn zO&@J^pR~^&tFjcoAP~l5foLI*%z$z?fi-m4zDYPUw?PTLgl^&+d0BG9@UP))o;6mO z@jH~QeDguf|B=D}10Vd;=>KW$C2BevC>p4r@|c*C;z1GRHFnF&uozejnl%Lt=mmrr zLbT{t!-Gc&uwfal%tEhJf5H5Y`x`A+&(Bo|-nIl5`Ir1pKm#oS>diDXCznmLJMWuq zO(*dI{-0=lW-n5>!&mV1C{nH_$|?q@W)mZ>$ad+n`|#$<>BG9v5A9WHU^vlKob47S zQa8^ndui;ccD#u7U5uJ2r+IKiK{S&KG}LXB4@R2qe`0xNh8a{zu-=Ul&?I@)fbDad zDNMAWVeE4|aU___t1HI*5MLWIA&mug3NwF0*c9}Yn4^&#jGSv#Bhi=z+Q?C=8)MOU zNY%k8wV$1+iMo~HRy4-^Vxn;*W4T4=C#A^ss;pK~Y`9AiyjaX^LCIKm>!8}UzD(P) zsvP5tX#|-B#*^_UN^oK;RIN>1S_|l8VxvFwnSd$sWw(fF@!B5=BY=r9-&ymj*`?MP zj5xP5@!uQe3e=I?T(8ND1yNbA-0nR4FyKc%c_ih=@V8RW#MnR@kL795&+3lN`xNtLjgK}!M zbOO9C&MkNH4%z*bt2nzfEnAER&y0frR%@}t-7#KLkL_i>u-cu#e_Dz}H+!)eMG`o! z>ILz){a}yNo9)k7Khq;8XQ&70TV@5qU$=^T$7&>uQ&? zKb1G|D}n*gl+duf<2V&fZXRxWUcRRp;^>cvP#{GBff>yZZi;n|It=Z6vLcktT31k+ zs2>ts;s~JQ6DwNG%vYa1{th`W1>lX*jm)!5W`vsVG>qZ7Y4_oU3xoOj2Ux=^b`%Y? zGHU4UC4~($7MW-9fo2!9y{X$D>DnI+8)3$Or~QE`rf0zS>~(-|4p>2VSabuQ$6tR~ zByjHS(N=)Y+#QQNb`!rV@rBz6%&2gS#Eo```dB-9LKr)qav^W>wi^DlDeLf#Q0Em+ zf@2?B%*f;nV}!+&?xcft10rEPNpgSU!WggM@)-YNa2yM+zYZdgaGfu9s2w~wI}flE zW%J}1ZV5oYKQ(M~Z)8EhmAv6eAeW2}l*x)AdpUI3h`wQvZgVzRV(iVgGlO{An!6c; zyDHQ(#J~f7z;I7QeE|X?RR-?u?=XVPYjk%KCE#XDVP8@>eY+dZ+fcIEELP$1hQHE{ z=6#Lv!PBE;I){@b>VxeW1tvaxp~UMLTZlWrHdA0xSiOscGItV47fTkI5tU7EEjMdQ z^O1W9lbR?Try0dROP1`)XPO*Ik5#ZcvfTpxEKSi~5g-|$6Y-043ch3dk_vW$(Aebu z1p8P2<6UMuVf~$w_|@ zM9~a~G9XsbQ*ZB zXxyKH;xhr(r{Nqh3I9izvymF!-ttrx56!BPWN)`pa%Q(BDk&0(PhL==Zj3knJtYq29?u#jZx0V(ubOdwnF zB7{}Yho(GL3a<7ADGADi%x~FqftpmE#LW|DOVanTo_KC-V4p82M9UA~bG;j-F!jAX z0r)JgKkSNoyD*-0So@t>ZYvYC$;s0uS{w}B3dJW33jGcO@Udpv=Yxn7lrS@@)dE;V&7 zp`0Qy!m)i1Cwd-oL^Zuf>#UBhq|BB4TTH2Iq(0#t*?he*;2c*{Lm;WN(WpDfS5!P1 zS&Qk^TZragpxsXE(9lMxs;33mT-9dDmTa|w=kZ5opO1}d@enP6*XLf*9b6KKqV^IS zr_or$C|g2`tzXS99oK+pl(hvDj~DY8&AiJ*eMvQADk9vZ(-XR^S8j?<;vUMH157#Y zyXGXT=_r$GE!;vL(u%Vk-6{^EN86#_+zAcByR-2cK7i?4OByPRB#A*MLze=)GNSmv zT)r$=yE#Hg4&@|i{Gph)TZH?489<6cS_SSM#|@&(H#eYayzPqK94=L4QKJ`}4%TVm zoZ40uO1Aku=I+$L807=5hWtR zK7S>wxDK$yc=0zg`~PHVT=U2TDPF*UTtPD4){sbV(Uu5tYmU>T{tMD}KBpa-D@>M= zFG?kyG{RU)dC6KbgSgOrL^2KP?<_vcwBOAWC-sH(riOxo2sSYO&0Q%Q^F`^r87x;~+)S&;< zd+|@k#pr(s0d+eS6$w;-c2Ig!3}Pf-D;u#AOnuVfG;26zTHyj<>w2Yn?X)bi4(#jc ztu4-bl{b(Le>#>HShv!v6aVYFv+xZI4oJ{N`R3&D>-FuX6R(-c+@tv3fLF9WFHfdN z!_weE45mk8br~1$mxY08%ne4IQC{=|&7m4j`@=Z=M)cH!dZ*O^4euGNVITDCHWRKe z`)W;3!+o1T$$U`plfU^1sb*)WYJk%*#ncaB?pZE8@P(@C3c?&UW`V0v`}Rbn2=1OX zaz;fDg=4?o>=lYLDtP+>?6J8&d^I=THa&fs$(+>rfr@+xUTn%#FuL{!#$>+=q#4fc z-1h`rtU^~0e+#CVbhFmhR9?y(?mvaDAVR01zRw9Lo2N}=^TbNxm z!AUf?3yYc4Ow?fA-;g+p1+Wn@nidBV3mk+awjNxT2#4pkJ%oy4&n+Z%biQ${RaFMl zVrFaDw84z!t9#p$&N~VBGu40!t`upbgm`46wTrRtT>%r{IWAdg=xlilNsU+=3p%W; zd(Z6v#Oqq*Y|`#>T*&2%QUADzV%WnbTb9}ts*#PcIS9dTMPr*+FCy031hiFsg5m@~pv-4cT{MtR0 zLpk@=tZFFYO?q?LTAO`d%)Z7ZW&W@+8tiCo%r?jAfsoMI=G`J5qrBoB zH}X34C)zvL>bR(aJjB2FISljN*|PTgw+`NiqaYn=u68LEiX(yA>haeJ)0ldRM~r3( z_J$iTw@|%Ihl9NIEp8tm|LV}5)FG!#zXN^Xw^+e{#smFVdic*sPb%zYW&1y#?U$Vc zvJm3v*W_ZqS4%|Hp9<|7=*Ruyr+u(uYa5)FwQ|fD1`JN4Ki&yu^N%`bFpHun0zU~R zIDK?!LXbgrqSrgtGj?Y#WuI1WZUlg;4Yx&spOGU-ip12Tv(aI|(dCBfO%2z7P9xGl zE0oKHIg;xbMcB5T!w+CC^e)VB0$;}%vwgLufc#`_P zkGOzM843glLb*#B6#HM%e>PvFb62-l4VSXCZl(`rp%V~|@erzM(#;JY4 zXp$TV#cckz{ao7EGb55PSp&PqI)x|?E4kpLcuAD~At?c}sj*w0VCYHV7uaIeFJ5)8IYW zU-C5*aHBZmQo6S$tCA0-X}mx`GP010J}A$hO{6usAf<^xwU4~?p#2Er zJfc;JIxk_0*OOB1XIZOMmV*+-KK4MuHcJ6tM>1uD10wan{ZS-_EI^m{XZ2%sg0A9z>W$JC*k^AQwr-KyeT&7TTeUF}9CbxS+?ad{ zUh2i+^}l4f*AdLVzZrp-g*c9;c{s(bpjtWI%PTi&Sc)-!{@bX_8?Of!+&7um@g0r- zGeYklDD~e7J#`%w6bnrM9$ZaHbtO?$1hoX~A8yOqHb_Wt%0>J6xYbWb5{)6t^;y?K z{rFvHemC5UlTRa6GtDe&;oOUNfPjL<$xo=V8owSnN#J6Y>?!|~XRo|xzdQ$y-oFo% zbwA|ym4G`LEB3vCkz`x*K{!}t$q*S7N|!CkBek|<5B#>2i#DZve*?vWV0)bldH4s<|^t^WOb0;*kF9YpuHj|~ib#SXz7ug!2Q zX^Jj=Bxs69eq8ex+3FMo*W%b(Nnf7w*hv^9>Lk}wSKmH3yY(nxhgTJ1`vQeUt^;8@pWy+0|js%sYsD0rFi3}ts*c|l|+6wJrx#rQqQ3?!#QYArMgIfDf{oqt!^eQ z=&CX!AwnlFDvMNA#9VM0L63?A?C0%^j&N1s^KQ7`>Lh+-bXD14iDKeTZD8)~A*hlY z{Rcz7T5#CcNt?e=+PPeopi_Fq0i$T33Nh)f61(R5qgkA?e^>6E=iPocD#yfua8k}S zWhkF>hT!8pD=yWS4~7*L6d$UzbMwATWyJk>cH`|szIi^bBzH7T?pSDu3CHGNyyL>f zGuV>>I%KP3+qE@9j#{8sRhE^jKK5;CT^YQG<9`g$qU!ZvpRi+F9h#C=RU>t*Kjzi@ zmCJGcQY3G9auGK)h(laoRa%QQqT<2OTHF2LJLQ&U~cQio1(-x zt)5~#nu#H`oimSK%z7idMHTex`95~_NCN}W2?Obujdlzo|-{E7UXsJ zDd84{2IZL&80{{tDkz4c!X+Z_pKpFR=0PEGPn1zZTC0)=bVb#w1FH5hadQZR8!^~a zEDrnxNS;hm>{Z%d?D2*@hlQ|?P*-;b&ZNoDg_{2miogkizUEb6A}o(;r8PKbTW!RA zNMzsCQ%>AzA=61bfcT`G7>N~sQDwfXY0_9CX$lbs-P2Mry1Rk_`r6i}`_Th2*0}vI z!L+TKuI{P?Y}R)~1p3feNksZ}&Bj(4BWURjX-$_U0?Imge!4WD4&ZF@5isM`GUJ)Y zV2rt>ACeLW6vzw?r!B3M(GRsJ9f#y1J*KpFddL2tZp5OOyM>^`4*#i>`? zj0#q>j7+9E@=qbO$<^Q!W64 zavmrqY1W@bG?3HjA>2XOtC6aiu84o(TGcL-+ zi^0EwgM%N6U`^k=@yGY`pZVASCph?Tm6ViNCV3DhM6s{Fuuw8&pAb~5{veus!V*C$ z%5sXG{)-edlFi7WWlHCtxYWkP*xKfgCl@jX4sm5SAU`Rlbeie7g`ZR1*C(P8gim*y z>Kdb#SL?zfpJm&1#Bh}F=DVfw7UvF94VB%)1lHq>#V1oloXr(E*j#7{lBNk3?h}6u z_~I2Eo%EGR&Kp?tK;h=B+fp>7IDRqM_9~u&`Ja4O{II8%aCFbQmE1N8T+ZtP{;gDY zOSV-W2=hHmN$}%`$bZD5TiH3ajMyHVD&8o%^UiQ?|DZBhhX{A46ztG?CL8H%BL9>?zl9# zGlm8!zKoz&&&Khscx!-?s2WEwj5;R)kC7j6+^@}3a5jEPZ$9uDjiIy z$P;wvTU2Sp@;*zUc>dy}8#wIdX_V_r(8rtYB;=@jX6lVFjcHb_5s&Fjoy~^XC6lic z&(iNrL(vO#do@bS2vE6DC%J<9b=(_!jvDo%?r0V>xx}CIEz9-7%<{76YdHn!0~5GM zVSpmN9yhqKZYFz)!|x`BWxKUFX7wE59ewAASVg}~{D=^bFmBOZR~1z9vWrfE%q1@? z?lWeOHM0;NU%FN_XpbehY%QTMGP;7edZSUg`%_O7$rDPI4<{`EDG6qzsGo65T|X0_ zJ(fEqa_yv0Ub@{Hc}LT>sn}CUkf^@s)T|tE4r(-YHjA_rwD%HoQJ5KbJ7u<71EU5a zowae+yX~DYDHacObkp?>J0%4sLOMuc>?Rsm-mx)W)XN7B&NJAQd~=Wq{kRO_(Bs|mfEBo_;Q59+BIV7Fa-<>W?2#Mb9|09lcs1uPT7rFjTf z!!>EdNe=30wYy?}a${(Ct7F|o_(GLeE5F{GPc_aD) zA6kpkHo8!$^_L`i)MA2S z?k!EOBk|Q-RpZJ@xMDRYDzHkDjd`q_^ zZ~`m0D4}TUx;a&B$Vg)JpU@U3**{d~soql=U1Bxw#GlFfp*lKy8x)r(^8nK|c9?&d z_L4F%WdvUsmo`UT?O$`q@lq4FyZ_9sKSF?t9!q-xnJjSb2}Z&8AvKzJhXEI`e8=&j zGFs&+H`wd?f%7$g$NgD#7{Dp!a0bm;!N??2s9ZiqZh(sd#WSE%oT&_$PUFok^<N>r~5XO$4^%pbmSJ6tp-dRsaKa#&%8>w^nSD8)UaX7_G|g@yk?Mf{1j6sRajnT z?JBK0vkt2@Uy!kGtvxR90A!XOH)*$LGBj2^5$6h}xQ8)gyZR{RZ?43x~UCHszv`&FnYxSs8FdQ^cu91k5`E+&U)hD zrb~QJ+26kMiX1ay@seDJPn1-{kpOZUYF*QxGM~(gvewO!Z^%ICO}e@hhAJxQmpS1% z$cpwqYO2dnlHT9+grk8|tt1oe>c)Ogqu1d%@=N%jhT`M%}5kY?ZH{7*>e{bfD-}cztjJ~IbU0nHDv83d{%Asp`=w#8k@sZI-Is4Ni0|TnaDK zEOmHl#`MP}uH9IV!wLr$l=!T@V#(G%^ONyk3g_RAMTBkq;R}8bsSe-n5akGD)E_Ek zBw{>91NcIunQNLBTstff&(5x!+q<4EXI;X z`83GM%DC*)hO*7Jl3nuRK(K%A+v9OCHjz*K4xY-|1J51o?Y0(lcb8T?l|&9p)xSo} zd&t6TFR=Dk5@Fl!?DHSdAVSx&5_yXs;nP^()QXpaGhH&Zd8a~SM3MjAt?8a1&H+U_ z%x`!HBxi3G20*m_lB0E95k3n2Hveb5gRI z5Dxdrm>ldV@RZ#t@eDaP;5n6f3YUwHU|~MCQc_P(LcvarShEWHvr>AIk|5Y{ZN&M0 zI=41f0x-@gJ6}C%bP%9^HYtUaXpZjCrosAGE7b45XI2-t0ZkY0jP>MZt`B8QU!E96 zlhlXSKt7g1P4S#3JVNoN0ng%y?twRxuRZijS%<^3I3kifVQ82peFpRLRsy-X~PBY1J#xd)+dNVSu3&cq+ zWT(dC%N`5ABFQ@N1KqTSJ9*17l=A1Ya|WIraSZNrG(fP};1|Z}r=Jd?p5Tl&gO1TK z#$7qjteD{QsY(&#NYz`P<)t)Fk`CpFl8B-n3>hH^1yDG(NB-QE?gJ}x8!AFqTMLUG zXA{M+9VNTGa`H~&ro@-dKOIh43uGteKtR$356;{cJ=#Pf)Js5Jx+(c2Qae9$kQm2P)Vx}D*UTleiym%kXsk}^zYgnpx7w9p}_ zcJX5NW||FbZ53QbU-5*2bh0aGQ{;Zg*$>(%tUN4NY=_f#e>w06PHD+(lk{rASH^KxwL%@$fg~$lO=jSU?_=GpRo7wd4%n*_7S{*Bh(br@ZpQ z@5fFE1|{{FM3uM`6s0y~1Z9U!p(7ya-C*Hb+*0~?Oq?W4a-K3B&r1pfk1Oa}si|#* zSt@xe`6<2e1fQ(1DpuVZB%mouf#Djl3>$gV+yPKj1kL@RChjLx=_ZFuTw4>1-I2?U zM;;)1=gzRd9=Ec9^+&MUaj(r>$iJq=8h`extH+B40=eUw%$%v3Cc{L9D^km4aB$sz z>c*+TGQ8SLk|krs4c6{&b+&ucsZ6-(cIQ?cfODUTQ@~PJ82S;59XSolA$iFYNpgGh zJcC=NwwuG=8w^*9BF`<(e7R9o8OUmOn>?xu#I!x`W#hsc48w8_zuHeQewfeT9h?Il z&+4>$)9|?_jNr**^^lvr9Q%jeNMiMnEZ&CEhMkNHIFegy{d5^H4m53TbSZn<<1N$h z0d@m5)>^;Euim?yz^^_R@-wKxx!7}~m|f6IkMG2T+eT?5ESUyA`E@5O=rPfQlL2#d zLhwOz_=11^<)Z|ZtK77NRn_(HD^t=*r*2lhH&7&h&y_3w-ywV{`~MB_|H&%{kIEbY zZdbtSV8#_I#TxhT(|zRj*}eFpjN67ysLmrPbeUsJyG?jz6E<&gh%_QmVGZCe+Z}(F z5Om~88{E|hsurqGm8={;V|we}5n8vqC0(aLYviOqM1z`I_9OVUj5(Haa(=GMg~^gt zqP*N?5=1Q*w~!sNbK9WAWBUmJj>BazxsrZN&s_3~Fx@>|MD%16%w9T+!)00GYZ6MYXo!K(4is)Jf% zNel~tGI_}YcCWrZ%R$IJ)hqJeX}7}jjo+QQ5h(RRze(xt<#9?6y&JiK|k|5O^pe??Kw6 zF*~B3Qtq@=tQH43WcWb)6K4V}h&FO*mZz&nT416GGfZa)UD+ z?#Ie*sh@EC^V$s=7Wr4NaP`eK3o|O1AJ)6yJ3n=pFLNuzJLOMMx!k>Fg9C9bo0;n< zF9M(NAzT4<Hc3-vP3e>@g&6r zWLGRnR7vhuIMu43U=?|mX(Q3iafzVkV}gX?IGpCZv21aNpoYy>DBXepkwzEhmaH36 zt2)Cl^_(k(sPey)C3%V7A&&!h1c}qUr2+5-kDo_Q|AA(+p9ISw&4KpnydcX7RD z+FzptJDeWtaKFzG(kH|=MYHUNy8sZ%Vwao|GNU`_n_-UZG9NHWRw;7qV)&V^=C#77 zrk}|tYWa_{E zt=GkR-<)mryND8i@}IVj|8JJy|K&se*+J2w@#=x9hW@F~T`XCq2@1x6h`CgpTnai& z7>UIMHMa&TC(H;mEbT$2H>bC(t}*)aA|`ILnw$Hb|9TR4^pKR9OCsQz;2vW(X1C^? zHs_aIQ@YFjThFG)-)p~{p3n1uufM>3_#ZTJI!r4uIh<5WRB-WQ$Mc-nPTJ+82aHVg zGURbt31}0@*206PxHbvVgCiN}*<$*?py?dxYB^%n`6nQPPJUUP$TopC2Grn5?t z)RZ;VsE_Ro3fQi^lf$%FS-2+0GPG9du)&&6wbq?Sn3d%lE)gP4(}Y#puGakH2PWNU z#Kz2YEe1}18!M#pEw<^EX;YiLneXJVHh!@B&bmsYq+0y)U2KaX(js`=qT;df(z7Fh zj_tNzQPxar#k0xR#cb;`a^Y2&32V+`dLSvanCuy_eRN@Wxv3Z|(4#zt3uGKS&c8qd zl~egZj+9009kBaDRz9alP3@|0UI_i%vQj|3|A(jND(^x%kEu+y=yKERuv`l^u`vfq z!)Ubx3)!;Dj>@(&>`B)Nf2Vu7aC@KKfje1ttSM)_HYFP$+ZMl(0&kJQrJl2Hx52j^I5 z7WNeip{SULncEb7M88NVWLxwoakwdmny%O4_UU`B5uP`I1&q5#LBLG4_aD}zlv@#Y z?V!i~eg%Yfl8{2pL(M*st?IY+tsqv8Y&U!MHWmfxJZFcUpk2p;7In&tfO?M6`Cwh- z(@C6tK` z+_QY>jxvP2k@*j|b`R~KbkW$Xjg@tjreSr5a-uZk84HCG0Kn1MaVSK2K?%sOf zef32{{OU#)gA(3ue zY@FGmd*~DLS?$VyY1=8*iQSuiqC+)E8P>#V&}l)v$Yil+#_JSM7^x zO9wX+Ydr$O$84q6#@CM2$Ktnir79HbnjXr!S4C44=Ijc4`-p7*<=V$VGjk9+4Udw;XHxHv4R;u!!dK(#AKg7*26Eng3=mye2kw% z4P2Mr>^z3TEwilo1-XzZxM#M@RFNjE*p?Z{`e$KB^zJAmSZNIQKHpJ|i8H(rWF%^; z_7ZrJs$psy%NYpUAf;yzQ!5>15VlIvCoZe+a7NbN`;FS>xy}N)fGJ{c8O1AVZ%`r+ zst9_0Xc_^rRtK;d3}J^T$S!o!hanI)D;h(n>UT^?!s*Ue0Iv%#b^7D7_+g{MoX**B zetJe0CXDkpE;PSA)if@=p2aiP5KF-<77d?V>6gi8v3&v>Xi4Qhl$@bsOX$m1M%I)L ze|jIBW^>ci$-V)V%V$RS|ug9P9 zgQO|2ah@5SoV{J#Jy){wxB5vQ32}zwXC!#N#4ssVLn4QK-N#SLx6$@+cgc&G<@bqQ|DGh^2$-D+`N*clW`;GVPA%JhfOCqpr^!QWT7m4$fGi2d%zfXO z{DND*(lfR(6oH@IZI+(c`nU9Ty8NxqvTy!D z6RYhbYx>&Or>$-Lmaoq{Fny4zdBKfba1_)@>g;}DYJ!PGA2BXg5zDw)iZMMr^&0wR z)Rl%zYOe1d$k(uRI+rn1$OJ>dA3J+l4p^*f`gN<6$Y7qgZcjEv|b9mn( zcFl9Wao-U`>J@d&a60KUNgIh)+3SM6SG%e82+D(@UEX<&top`4I3mbUNK+93{dNyoN^K-^5gi!gFe5~3kyiVB}W06sQXQgF70b2o1eV3FJrA1U!JP|tm zJtm_eBKTjN13XhLK`lbrrIWHWiJ+;#Am(lATyUvH;1&U@@pc} zPPq_&JE%PKI-*09JH`wiiY4#6Gm64Mm!PidAWKPTFi4!y$}H*yc(+z@+h@ZsEcx8c<6J_|6*Ztf(f@VH zc99(u#@ z#91fR@VS<}94x#u^^4?Qrp-8G%+;F#Na|b18cN0dw3i&mSuj`Q~p0Q|R2r#n5&K-B5A&q(7BtlcTV-I9~pZgp^Sr8D00y6L0?R z!8!h;zu9+L^lU&(65 zdyHm`+Q&Xt0|J>tVZPr27P~E!X=08bv0h#dx4c7-XYZU}H^|-aY0gUa?EvV@XS=0f zPu}%vgTgQf;6!cfhBf3AYa18PfRBc1(7YtpQ#+@1XbC*)Dafd|vd`N25AJT?W#acn+Gq;aut_( z--!sp?0CuSY&iy7olFPi@0AQ4hxW=I?Sl(v9nXV|CtF^4y_KhPi2M9+?*4qla^8xU znkYE`;xGM(?s(taXN~jzK9STvEUx;xg?u-(IG@H^N;NcUJG#ZtU*k91d;UDzQn%Dz z&S$W@*yKzt{k4p=yduL4<6tcj-x#SdhXKrYXxtoVlQsxlVrPY@LlK%TsaN2qj}El% z5K?*AwboA`bzr(^7B;ta9MfjR8!gjFwg(CNFYJGtBaDVuzg~K` z@b$gf4b1-X_2V18y*Gj-DRLUrOLRXz*uQt}x^vFHe7SwtGyT!mVn`IO4{{`3VAsQOEw3VJY zmuF7hX>#~j;1+D6n6DJC!h_a9iSsSeJ6 zrp&IxkU)!PuGcmdM{AUNxWV)|HP(3Ol4Mc1;?&4YYEf)BOr*;+_!J#u2P!%;p4s^| zEwt5CATu5l%HpLw6^5?0R6n|YCrnrq#B%25pY|l7aFG6NPFgu4H`#q8(I(>njEfpM ziI@%{sLTBe_RH46fx)4iq6+i^q7lK%Rrr)79{)THy_z;%ShIR;s7tAhHj|y4&dH{d zzWj#y3*Bk6GEe~X(>(HsELn^O8!tUcc1Soh`ImbH{F*#nDfA>M{J^!XgOBes!O|?_ z(MEUkK*W6V@g`aIaLgvbZ90-t0w@JErd5v$U`AIo&bS%{0CVOx_%OC5DP|~z(GuH3 zQpC6%xwpzVL7(*F!6(O}W+9MySPI-5gVB!Zk7mk3(%L=u`y$L}jrE7yrmh#Q73Ys7 zO)o)o198=jb;{cQKXGMihOY@ZYkzhmd^09&e7-tfB&)P%4e}PsqvovZ}NR zgd4+N#(g%pw9Dq9>Vk=z0d!NH}XI7kqen`-8|1+lD3AbCZfCmSb-j&UoJtOA4Opr z1{H(B^-}}x*lC@0lV-jxX((JMpVtrkQMW6X7kcK{S^THXRE{GbSKqIf7np7&2`CbV zY<_4M`w0HQ6N(iLBg@!IU`?>5q-70*YsT>4Z|Mj(W39z*Xs9y~q*}F>s^?miMWfSL zKfq7jYE4Gk6L=7CvHDrW&m6yj9AeF8p+TVDq}@5kTVi(AS;p?fq-CR&SYOPYKDA>! zzSXP=T(^FGs=%V`tULBmcAGd?GKOV-;yG&VJ)iZyz0&2g^V;+k8K*IJFDf4VQso$B za=`wwAnB0{e75Ra&fV6CeR;<7%YY)D##cc%!RoDvc&mggcQ^2tvdcWs0;j|KB}Fr> ztM`=48E7pel2d-T!6275c`f?GEdR^5KQdTh3U)i(5 z=*p7C?PQgAwS;fy{@tU60Aa#TyUo7N$G(!FH{>VT=6;LMdXEvuaJD7co0PmMe&~0m zpF2I%zMp8BN#b)5KR{ZEh2Td*wpm9&&8Gtmm~$BFDrh11T9Eb3I&%Mja5A_6aQAQd z5HAP@J#s+)G_LdzoVh`S&iyc?7n@Y@gV^U5)GecKos@6!LbPi*V}>1Gyi1&Ih*iX&+j$bBpWHS+T2%VM}Il2`^4CYyh z*X|d;f<3(vy0(!kuJ;et>K-?w<<~+WLmEJ|hEn*$97$qBAd`!4NW=R4Z$J%=@8X#4 zyG^k~{PBbS|8}bX;Rf-)i-3Pt~8rx{J^f z#(uPE(`bs^0^K0S+OtFJGR&lGuNyWni%aH38rLx(yD9N)F;}xw<77? z(>47wLD{a8NSmw#=R{mffmVa23la908LgmSg(VlU5+uS>0F zyycP^2GE77m|`{I7na+V$tlOXI=O{D>=LO=3XM(Ltw)2iCRsIsQrQi(P7E9WX|G_{ zpkxqN_c0;d+I3==sL)R}tE=jg_oC6mVJv}OSySU8RdA_pVqYz!9BTqBur_g}Pe_*G zz(S8)k6TnVKsu2?E!rmuY%S~7H1!?E&E}AETCEn^u57R|eVRu5%QPjMqn;7nDc-Jd z?IN~AqCe2qz39rLj*rEQqytA%+7aE=Y2MRgNt z<;`Y6qA(^pO~(zZ@XZyjj3v#jdJE`;zfIiAdO)ysxlktXz>5Bo9xi@MDYHsP1!!6h z=73kuPsuDrl%$Kvr#*#dtvbw`Pf;s6n=-1aiemo?{LY`pD<4Nm?NVm8<9S^UN^aMm zzZ;)(V1>2nbH4Qv>l*|2`J)QE9`O^T&jEA57H&Pn9^|iJHed_e4RsGohv^#PUA`|H z_8n6YTxUoZ_8rx>rWiJ3ARTP)$9sGFl>Y&zcIF3X?TO|I_^oVVK@ z4j=1Tj1zAl#b3KclEC$#g4}KcwIB}6f>hZK9ZgGgL>RUujyr0z-#=|m>dp_U;Or!L z&Jry=ZbgH6f?*`45~E~6ssU9ZNgU!VZCQvv-imq0ZmHekSP`YRgE-F0jFghaOhCax zSMt>&h;U9yVE?DE-e`MOQ|`)VQsAP8%jyM%zqT`qPM;Q5s^#CtV=8EqQ=}-5(xWO! z3)4MJK3tPlD-P;GxpIRF2}ihFLS3)S_tS~heo|mcpK16uRIcOFE?c_+X3hH1k+vei zRFNS*YV{C4!=_t&g3vL-BS)?+gW~~>%MnPedTva?46Ym|S0JJ%N&`_kf=^kJ&#HpA z)*?!vQ__;gBoB@*On-QA`Vy}gf#05ViZY{5qK|EKIg#LdT%K)H27;IPVj;#Ap*zEl z4EC3^$+PlW9Bo*Z1w?TB2`AFbU6E7NHgYn}3bcP{yAGhFCBQEaE=97S*84v=`)TE= ztMr2jhQr?r$;B)Lm13VCuPGDZpb9)OvHZUm!K$uYF?j%|71UvoDA@HpoQ+ZMfT=a5 z^XA|td|2zVUyuIu;Wug2n-Z04F$IIhXC6k{L%3xMt|FL8~I&6r{X)1s-hv1moDYMpo*sYEo*b& z{iJ`EaYZCyKPN$tjlA#|Xl7+Co^fpy1O8Ag9zrZ^l%pGdWKEG}jR|pu`nf6QORYXJ zUd2$F&7cQG+2onN&0T0LLHil#V=V|HaJ4{FuZRDK;Fo~t5zB_7pycxP@^GtUwrjSn zpH@yP(VS5&hYaE&&e38N!4u*=3O4a%PL=WNl`32!LeCJz8@~_;Ziw%|t_6F?LEb6J zXBT9x56YqQxuO-xnzJ-3@g>|(gg_#p4ob%bBv!f|iz?Ac?N=+mwP@mJ?Qx9lJX~Xk zMe(SzA)6|p>1GB;pnKSzDt>&J+-Aj33QJUe+49M0yNqo*1x@%5L4DbE^vGobASb$9 zQz%WWBWL@>8xM@=z`X|wynGRw@W@_RZ5)`n_$V#<*DL0zoA`!XC6p}C&Rufc7x7@9 zeH#%l-Lu+m2?#t z+n%+6rEh2n`VB4F{%@DCqoJ9xwf=vxa25aSQQ67d%JF|PO&XOfzcWoRd?qo@0D|Co zT0+95S{=%Ah+wgBtm-(}xgf&KH`A<|wGEP*a(`2YMfqY%{`O2+7(_6<|L{$?;Tf=k z*@Zw7Upl(YlH79KtiQWkdeZ(uoMPR}aX)B>b7WXR3%N!vfOeSY=K}-+3=Reh)k3me z=SPK9CDAgOGja+uw={a!LBwSveu!F?2-iC9JXAVt>^K&~)3*@$b46W~jd`n~6FPQp z6sD?G|8+hWk=f|WCS|8f-eFx8Hg|zq&`F_QYVE!l0}DJj3A2|e%7(sSUutdG)}{i} zCbn7~#e9qdt2wFF68eiAgHea^et+==O_kavIiVxp#^NBtT+L(E8P_!+SN);&F=G)a zBKJbiOd(TRm5H$^CEq*BLBZS~+iQf4<|-{El2L3e?ZB-_JB{tVE?*KO$&NTmK4Ku2 z-x8;p9&gBcUH|yg71&-X#3mm25;_IRPPV(jufPvw5EEctp4eHbGK%%odXRvUsdf{v zpWy92+6Su_Vrso+XG(qP4j76lDXdck0Y8XAmQ2xiQ{tnb=pcfwK1bDAQFS|@53=PU zKwH^eEE@hg*`EH^!*od(3rL_sw>mLST|bNQ?X|R-J5sSPI!*SKCK{wAacHVZ9EdTS zI0PBRYSg>;+EvB;l@)NVsu5$&Lqo48LKHW?8yP?ajDLx<%t*Ql!fc>Y&rnX*$c5y= zk%z2}bO?h`Zuha(Y~V}lBEC}l!0G;kN9QGaC#d;Z7s)~u8#!;4P@z=MP&boiO_W}8 z<}LNyc4@zJ_R0o>5A#mme{3!afi)A#dykAT-N+`vE#h0CzRQf3gM@wqi&Yy1&251H zz0z{S=5iBV_ls{B7pqyu%^>pn?dXc+3d%00Y^a=ARc4@kqI{sb;G*1FT}JDH+Q5@A z9r*%6@J&sRD>hey?`aDBH)!Sk z4{Jrl+}O(KzYB)so4{^t`W*!H5BRG5-!VY67mHOWx<6v2y&yIaD{TpCLdT&vaV?HX z)Zd*jjg{J?5+?imxAsg2gd#-tOuvgIV~jaxP-r>MGi%QC(mm_W>7vW~>lv(DnOjU? zC@=^+g_+(+Z}m6+Z3l`1MxXR%mcCQ?40S$H{#r+Z)V(IkRSE`kENd(SJ$Z~rhnfzB z>X>BUL|jCt2$ct=UUJ`bxP<@u@--`H5mjZ$&akt(?p;P>TZHkn1v?c=CS2Qm!XytT z*Og~aSo!TvcI7Q_=|H`PX&t>tI@c8(d=_*=-II(B3)BivZ*S@@d~6 zJ~}0GJH4kC;H9TsN}hs;9Ms#0m34Rsd>X#HQ-+uAcT-jWo6Ntm4@4th7b`0?e0-ik zI`ZE4O_Jk9?h<~xdgMi141G6pfXE|>0y`XX0=u|6CX^RL>)#!sOR8qJ6v*ttmI@cz zd9UB0A7?`|g;g$30{9<({i?@aIhITQ^$n|Q#u$SktK?7S2WYA}GwAdpbg` za7OA}4rL|@30gTH)}->HBg16Q9&m=phw0J`F9U^}6D}*V3%q>A7znRF`8NYbv1hc&{qJk(?7P(X|HD%I=W9t?-|k!ZWpb0?P4(|T>Ip)=PtlB7 zYc%wFWY~i=?a}pj-bY?f8JnN4A0H5YYs16GB8(yQWc_x6y#*0J;*b&uh*~J~k(+<- zyNUKDz+$?RuE*PD3wTstR9q&oKrz7r)fzzZF-T8LVyxRU2kUcMu7Cw4w`$BMQX$_vCBKw3NI=Pg*lBGb&wcR;Okl&80G4Q)pcEKxQ_ zBo@yTsc+0@DNArn=p1nTq5e>pJQ(C`cCM|)2CE)|@!2YwQag6N!5G_7Tr81z>NUDu< zx6QC5Zhxu|tIr;RcOT5)Cjjv8(u?T-|qVW64gasn>V)@L9_%{tW+3bcTLpr^9vTE4LqjA@wnf-~;N=tOHf z(ca0&cV_JmPD>yBkz)4EsF|jf#)~)<%Y#nR)7?S-^HmSQbLRiST=mlmM4+P|@}ido zQp7&X1OU?_wi$4V+}06;{03>IG7~F(neQ0y?Vqqre9H5zj;g(dc&~4We``_~Hck8T zhHcPE7R81!q)E4E7g>#04kNGCYaGU*GL>)A`n&w5t%c&FYF9&hn|lZBPl8xnRsW?w z{R-33G*Ba9j1mHg%5DdScQx^1#JFof1;5e2-e=TmUWj;+Rdj&7u zl;?;n!GF$rsqODroFTuE50rJ%=(y&Yv~GW` zuWWbGZ0$`Nsnv^{H63$Al>{~&(Xv?$A@L5}4m$_wI=LZ+@Q3k)wRmgOQRL~JB9Mh% z93`QGbzh6{@$V-cS*Npjil#V!8{FP|3H)hV!Lz+o6DnA93DxiN#DqN^T?<=ITITP8 z1K@U^b#i5?)yY#c&z7|RB~CljfDyD+C&Dy&uyGnmRC~hm5w2Ke%oO1wnHmLMq7SS9 zhrKXsS)q`-_7$v#j@MgYO+P?xSBapnSRudVT0oR&XH!*52~{kMys1{5rjc+zaJ2cWc3ghg!JXT}>H3L}JBKQ>Z4rWGikiL_4 zf1%pZ?~Wgad;u zY0>uEw~y&N=j#8ez5J&}=bymae`B^zRZlym1&` z-$*3(quQEI*U%KwzHckJ2ZLv)Pw}7ET%STS382r3M*j71DoPr2E zN50L82@d3@HuCk`um8y&dZOm8)am=6t|iFC0H*WKkREbWEMs|W_imz8CrHDgG6c` z+ufS=*SJvIwCAMIWN93zZPX5nZ5B)jSR59+@$NHdlgdmdPQ#lGXca-`(|$|ObD>^w z4ECyR%ax5I3Mm=(n~;tAkx+7?+T%e0Cs?CMZV4-zG|7dTL;S|v)$`6LWl(Qa$lP4l z<|V1Eopy6)AN?;p-OboGJ++u@flwe$&y-to$7m;utrIfmFZYzgQR3#nekzVAl-e_K2 zxk&p(@Y>>`U`8U3eRgqX??SnsJh3L?Y-xDG&0?-ia$0e9W$nC6<6;@js#p<;)?{w# z;~Cy%q)L)&cc+F*T#|QKu;F5xzi|}$OARelByNl)O4^CPINk|}jy9c5XH7OIM(Um# zJ2;0L0+NW$AdHeibR2ZQNojdq+2tknrqsR_r&cz{M>Tye*{EUe^KGkrY!Fec)sMj! zE|5}?UZ*o&GZJ>ZJZttu>I?Xq zT(l{sX;reY_Ni#}H2bqMo|@FX6Iq**r01*f8!hb|TD3|cU&sSz=;G^Pnl31UY=lYtsC@6g|YJxy&#>xyKfY3PgP z$>O7enB0aBs?%akbioX_a4dpB#^ns~MmY>)5G-B9IE^InxG>UOdYAjC;??LjM^I-E zZVv2_h7+v(7>VhZ45@+?&cl@Ppj0Yq=P0(w3LT)Q~ zgd{b%L+w?UXm2Ca>eSGV?4_wC#DWgI3;4u^`BC;vCaSfo_aPE~Y`Ji`abptH-*Hp0hip%yx-F&LPMWAaOh#;Q zalqwP*p?h;`~V?PvnQEVYP^@xE9UONtLkV+uz=d=^+x9`lG@b6sB?~(Djjd>_<_6B z7N=NYkdryC>PVGm33X`XU??|=@Cq^Fz3=)Z%8mx#H4g3ggVTzjalQVTK8>CAZY#PY zluGTbu)%|9FKf8&Hh!2+m3$j{7hQos2#O&v4Qnw z(X2l-EqH5o$d@Al3Woyn%YhS{ANo;sN5vA~>cE{UJY4nhPv$!`tBIZp0F$UAqix?K z&Un|(IF|#XF9~_QFZ*3q#yCxba+V3Ntm#YtgR1$RL|fY~3h)7{LxX~hz3`-REX`gl z={-=5D@rss|Mgo9BEqf2-z3`F)giQk<)+Y#?2s^!F` zJ&R&J`)Kui6oZ)yG%~vyPThSucMy@Rz@K(x1brkL>wN@+IL?vh?2|iY*FGP(fP(0# zycffi=%Lg+*=Yq?M#^l1Qy%Fk(-a(tlD>Mo>x`!U->Tsaq@1%!$6}g!g2h%{S)tNi zoC7BOos#lCAowhL*k&x#gHW!FXk+oFq3B{`FQl{34Y!!O;P!4@@n+C1OU@6A%_LQ2fc_;0DSP7aTO7S__aDKFD|{sPst;YfflpBk_iDYL zI>z>1k*wGRTk=_)gNoI2B15GkP?Z-Bs?iNi+U9Srg%?|elEz(G&NIloJjk-o!9u2> z8#&3Uv4n-|ggAfvaW|Pg6zWCd1A-=yu?d>(8DO-WNa zd<>iU_O|;qnhR{TL;w(=INGRQC{wy`3HkY79qlgT^umoAR|JZN|~%S3GS?iFQ3 zEd~1u{Y3Y6g7R>T_lnNE<(=v>7C8%#x>SH;k>4BnbsEbRdk;}oMYx`5K z^k$y`OB*xU0Q2Vw$&}tt_E2A=*Mg`~*`2wx3ztkD_j9BLwGIO^E@zgcs+1SLe;rkS zy|nR$!~ggZK>42|xPO`n`~$&hz__6-Ie$jG42YEjK>-mlg2N80Rm7lyE`u9_iHi|< zHtLBFqdmtaGK+<)wvS#yN8rXUhvK&|U$r&9*=4lE6xf6>82u~?6<6Sj%nZJ>Ln=0p zFA~RXHcwa8t&P9uV|tsFz^JUD>S|?jylOpizIvHzt>=DAC+CC9k;l8%`yq?%Hu#6A z({~euE$cA{%C~$1o!bp<%6&ST&f}WYs}*JF8Wm*9a!Qql{)Xk^rOn^gYzn5Z1z~Ic z($@T`8Ls2FhsaBG=u>8>t9--Pb{n6&1#)Zd@NM-Yn)3hx{8QZcefrPAvpgmE%@nxK ztsbh+dC%B$vcfx+&#wNRE#5oVr(k+E*0Va4&!s>5Upd~t^M`zr*Sa2Gw$`7b>0b+n zs;eD&6XmP#8kmanw%3{#3cRPN|;+YFd1nc6iN zQ={ND7zDgj&e$zui3R4xDrFPuxLjGmk+nepDr5BkPLqprBnf6nEf!1n(Cd{m0+Mo< z2*MTJ%VK{~&!8TIR}@d8;X(U4d(_0X6f-aT?Lz97-tub+b?unN7Q1%*^@uQ{;9tPq zp9ZYWyDGMMLgm=#OC_hy{78q$n7Lfb01%nmyp7HE?4;X5JGge_Z+1IzwW(=9d(R3rorRh3dG)eOrMgqZPGXZK zp0azTh8|g~$%GZ7M{8XcNvl`gFKSd3nLkX5a@{Ny%EH2UTrRD>oYQIedSK~GLWFC& z@ncdz{EJXgGNI4v6bV-V_0m4;SzO$3IJFBxu+|`V>=psx7?XAutoo897PDQIK(Gn) zh{gb#`e-u6Mag_h?1Gt?$Rew_F=?D$ZGe{kXky(;0*za9w`z`}bO%0uj=w`9rDkGJ zkvdR(X!Ne(=uXLP+iC*YL8CsP!-b>Lgp-CivyspIDxfDi`YjHxad4?1E6Ip zk?fQ{OY8}I!gAoz+LOS$LRFIb%RklPZ*`)ynOOu2&;wB_t=S)16Odp8janf=UuLYP zfLMS~10jn3NAa3b_PW`q^l4kVeS%(G=D_KVouC&920WE>9|aKmX^K~P++X4J9M+Fj zSTm-Q$e?_{Ih(x(z{wbo^(V*_^qCZ95~5=}CB%G^=0cKlkVpZgJ|+Vx(f!hQSYk|4 zHkeeXeSJK&4H1w*y6d65kyvj614jvr$8bLee40Z#4(owSYhOaMxzGkBTMhFtu%sh3 z8uxi~c=fv5Ai@LpSqYpO#tmQnCISEGA!ed#4bY(J&q zW#XF&Mg@l%{Ro@P#}t!7ZcLpInjGsyaOmoZ8?2D1NZwCR53(FkY*R-A`h6dPw=+kw z-5qy}#0erj2iBc3MpUlgi=al!A}$*$e@BN^H9gcMjqXzvkc&||ztu_29~0;NS5$V; zmc$l`4+|`fNAVin>J3h=s-mW%vZ8c7Pepg$>M1wo4p^cOx6|_*bB)D~z ze8sAx7gxuZ_Fcs%WR@_R#8PbPTej@g&BdB~yN6X(hrDQ>nG@}q`QE07Ye9W%4}_I$ z7yvoF4c0pLf*mWAIEaN$eFj+C=7yM(*8W zIlF)VrggyS#E)FOaPct3z-8xo<4!J3m=7Mx&bb=)?65jot9jOgw(LY2$nr-Ff(yp# zEr0S#x2tB8Q`HoRt2O_$kmLm3;F<7O>FvEIuP%*OcrI6t)kEzRLE4gI=0s_loqr7- zX&#f|>Y+bL&JGTIMhPqa#lHrsLQgw7Ec+vh{QkJR2d((nKuWg$<#Cmo6ByP#-*}1E zSJTl?mIv3Lx7~K~QQK4Fow==L<8Qe-mjK3K&)iJ=nSf{nYWourV zb7GY)Ey`q(Y)+Km?hGOftci#q z2QM_|=c@#^SNg14oZ~RJ0- zoV!lqX?vp!YSxsvq`?V+w!a2IcL%^cITqqn@biJ^_#PmB!1uWX+G4w*Z-#dk6UbiY zLjG)e9pjLLV0$DOC7&bgrPv(xEX~qXgr%*e^p9hrIOPIc5jY7c(`+ax)BUco@7IH}O&@}G^z>^oc`F=RdCfUy{=OPGgg$ZKY*FX zW*nk1Bri>bA4)4>0QF*P&oza97PvbfSh#R`Y9OONNQr7%(-B>!-TQI=YDQcN>{arF6D$v$o(lUKHc{2V#dHhCVDo#=}DiDhyNm_`I^E9 z;;Lu~fA3(rGkEV#{{2@HmfKZW+MudFmfKcAkOv>{_BnCu$|Z;n-jKwVdWA$DS*Rp7 zZ`JRt`lp0T^9}EA>5j`I=ZGawckh@b{Oebo>Zxc$XQX6O`8UL*!tAAtpys;C#QxP_ zInJn8QKO!$E8n@rj2{WXGwyb;`?pB*{kq*=>B=e3Y^3a(Slu8F<19z$%~1 zrjQU}SN58oxBs#4{RzX{3rBZg&ke}eAKjjjd|9{XMAS`HmQHwpYPqRxp!=%FT>D*; zZ^jkD!H`{kIR!WY34A*dc{u})hunlOl{haDny25>Wg%g%Pf44@JL7J>gUPS?xwBZn z=t|2IOEbu^?Afkyn>r2H`9ZwnJR`6GnZAZ@4VfoJ0P+DI15lN3=ayhf+l{P~kUOl#+JV#?rg!I^4#lW(_T-&ZQ? zk=pGR>no_szu)&GWj6OHL!(i=wK&p*c!QCYdJ=14Vv(EqxC^ifk~Q;*r+K^<2bnjZ zf;aqNJ?7yVyP|InGd`&}W=uH9l99|hkN9g`#nyYYAOgr_)lb=7^4bl0C#%pAWk)Ax z7LV^|)g^nk1ZT<38mFu*wU7Q z{IPIJ!Wj?gRPa9Bc*m@_XYGs*Ec#2~ z8*i`BH})dh3Zh(o2L-wpgq<)L6`Ql?5VP->QJz}srby6A!19jH9EFT1hl*&d&T-F& zizFwnyhg^zMyirZy4Ut;@$pq?`Am!)kAIR#n`3w^b)eZarBBw`Ozgi5mMcAT2VY|F zcC|l&RNC-5FI|rpq6_t)sy2=9heJNTqYAl{Hqdp^|8-IGLg;!H9U?%d&*Wourg$Kh z=c$xEF~1kjyt%S6Z?M@R&PitfOIeP2#@GU?-YNbv5ZnFxa33nRkZ#>(@2hnoB9C*X zk;ov+;CNeu5FXr(^or z8F%A4IQh+z+@AfpAx7cZuY*7bbP$v< zmgoECZmqA(G;Hw`m2^G_U$CJqf^=oy;l9kVaOB3N-NylJh^oSa3zF2%Zz3DJ4 z+FKjXs89UBF|<)aY}+~wym!(ZQ-qO^!B32jS8Hj(6Iw}}>K9t#5VQw5) zb#7o+ymW8RWWs%2&PSk1kvB@g&1h(ADBiFxa;8GPa<2Ohd(oVqv*r*P9hNxLH}`JO zVhAxEVGb@%{b^aYDm~`3NST64MV?bDE08lMgVifO$Jlhl1S0YX?@cg~)n@GyV02cH zVO?F{w(XZfZ_NiLz#-6cOJySKKc$tY=Mo#g5f!_D%QaT($7F=$I5xznS#!)5PYfGp z1NKm8CK62kQ`iHSr#85lU@d|ll~3WuFqQi1*Rqn#>Ox@9cB=$F9xFBD99Fvlj6tWb zSet`RT$r5eZpWBgK~peU2dJ9f4uenLr!$CS+XtJQdzdv?`yR9rWSS=RyEN zt!~fy7!D{M(o*0+i=+t4d2h9tf|9H$91jN;i%1AhF(C}v-$7gEceKBA*Pi-JDo~2O zlSWpOy_C0(%E5aWsruxn%krvJ1;sOr^YU|HXqnNVd3qc70|#f9ESb3q*--KZ7U7I| z<|6@>y9ijHA$;Z-z1iflwe00AutbGBbTtDB4HTbcJGQ`*J z)vEY50-16OOQgQAMU~CT9dxt*E|%Zm_OA)J7$cL*-$uNxFoT{Xo<%c*^zs{4pMPAMz!(~L zg!5<6nnSblz_t*y19s+>(Io;*8K6Z@RHZWkUI7H*ZLQRUc9m)eGP-u)aL6e~Lo!X} z1Gg`{=U&6bIGs+{C>wBkRAX={Be!HtN%c(ua;m$W4MYHC5`7l^&4{U(1|NkaB6A4w zr8~8uEu!XK3ULk*g{GNaPKD{{i{l;g;Y9KC?u!PjAs??awD$ml`(5~#_?;F}H{6@` zP+p|g_)aX(^;)=!je%DKP}&(Ge&N4i?Wm~PV6=L*VN=@OzfiNk$fFl{66ov(XuW25 zki7ac2ckUFNHbwx&#JJ8dL1W>>P2hjcJ19eG1B*Mk&SAEsG&(2Mw8fr1!NX-GWCEF#>;B5x9NuEz zvO2mmEf0AD4$v^+ct1fbz<%#`Z4T9FDcD#iH>g}qQn%<(;iCD#_T0ap5^yt6&CUy5 z@^JtP4O?>8wep6w3iN1qQ>nl^+JCTL0{tJ(-toD!AW+wxnb@{}v2EM7ZQIGjwr$%^ zCbn(cc5-vh-nZ`AyXx!@w^sFEuvS<1>VDqmMJKRdfd4}wu$3ZAr~>%22V7*QD}_4#G1~=x$&+RFs%WLu-Q+(@pnY2z*y6F zdFBg$e+K*67l=&{)a@}YK;#T#gmY;LK5mZ1fA!3D7x zG}UtQ;nvQY^jr@MhJvOUIPWVfV|J7bh&Q3K@h^j96nLgxpBbuS4%%NH0Iu5so`RM5 z7=9)9yAx(pX+w-0P+Rr=!;QL3GiniZveSPuP z2*@L%1mb=Rf310x)`FRsi@=BfBI%PR6UZlg!HX~(1ZL8^6np#e@aN>p&X6%mm!!Ge zb=WTOW?kr5GR&B8C`vLOpImHqxIS-nz1?4LfBm|F=oaT6+GHAAQK-$=BaJdwLcc0Y z6`$YneyUZPV7M+%HLNaAHLWfiq34y0GxaIW|EHx|_s^wcGbde1<~*#96}6)8i~lLak{nD|wg4F+()le!;23X2u31WMGIqO-9SG zah;w_%#)CjWPO`wwPc~PS}B3PRc}E8w^?;*tzrCD zi}fhsoFOvdvFSwCrG3x8-KlyuT$=7&Pr2$nzec%^Xv2%aG=AX7dSajD>hS6pFjDkH zaJf->7^2hU>agRCdBx9%DlAmvO}VQx#o?0EA+i%p-GAlsQL8)w*S?jyXw%-?y?*PS z=0?r#B+G^SzRaO?n2z_h=?tN|l^{agBiqm+%5y0&_N1A&e(6l602l^3(_joGz0OS2 zg!Lq~3^?qOfhf9kshNLh_L};bNQFi?}||xOYcpRXL%Mo&Pp6GW3T64mCB*j&SEW(wcZpg`%e&=tRF)m>8$Wns_6^ zgQ_AT6!xu=Ino!7*A-Qvxoc#XChYQK1apkM_8kVXq7US7|c zjZCG)JN@4f$qDA_;V}pr^$A1*TVs(8l)$wS_VVEk%i;G0wC3J@Qi9qMIdxFZfV!^P z2^I>o5zo@)mLD7CSOeeWroowuEPE1Ugq*~2*ae1Mf(av6WElENFUb&S)}k@>H(Z<; zv`GN#$^Zr#_#%K7VLoF!e_|M&aZljG-X?#QL7-i_9cY46;r&Z!GZ|EZMwx{5Ln77i z0?)Sm`3Il%Jz>*?Y~yU0o1wH7B+V?B1xPrBAr>nu$<}`p5|z!Sy;LCl`c*~s>le-c z#G(Fw@m@t^2ciF*x?LJjdP<9FUtbeaS9EQ=2;lm1Mv1@Nr5I4*iSZE;0kcH-{P12f z46LL7jBGduKfc~&d4cr&I!^+^SV?msc<=Xk!*6n4XlZ?} z`e|<@7hjXmzNML|AieewTP)l=6;6<(JB{#>m~PD#3d64YwVL`84RWLs z#V^K;YNX_^5%I>1E#jmS<{%if#})z+YbNB7dq^0W*#||3O5WT?)On~9=NAlFf>M5g z#LqLc7l1o6h)zekY)~baGJ&e zbHgGzT7V3u)ymPLKdR&D_aMa7Ci$F+kh!*r69Z>@c{PHCj=hsm!L7q{CdJf~s zH_djXyeLsY8u_Ehq*7)vF2nt0Ux5T<(-=A?&xX5huBF^bp+g-hG(d=?*iPj)*@u)M1UcGx$yD)TN=i~CDsQ0d#Tj*dEnmF{X_n;H%>(i z8UHz%CX)JM!-&~5i)hmK7X~DM;Q}_7V8z|kRh;k}9F0wQnJPw(Nueu>`V0+R8HZ`5 zg-BAYf>ArsiBNE(hFm*!CR}Oik~>NYR`3lNlR(VO1;i*U>(4G65reLDEA=8q>~(k< zA?0m?c13Ra@@fk%fhMdlk%0Uc741#)2le>c0+yIy!&%Fb%g%7FvoofB?L1JP{LRm> zT8(C=aXCvQlN^rZI4ukD8WX6as2I5E%(GC4;$zbh#+biTd2nRHk-rs9iWzoBCib3` zS{*GdTc_ueNm3vLP()Rf;|p~(4KG#Zc@*GI{daCF384u=C$m&8`no)s_Hz4uZ!sbSSliB>1f~`X-l@urgU+PKP3I%`ZiPSAb6}=DN zzua)4+}ck*^Odesh|IK7hJM?%0X-!a+#_MiajWJTAUm#y$wH-8l= z_1s+|I#VGADig>1;ZS+l3Rv$(ck{(kcg-)3vD$qx^&vz{q*ZuR9mE^q@oFJEX(u$_5Ec#MO z(=vM5?j^%h)2=&n2-~JudjOQ8*S-<7>~zt)-|i*fT{teYnzt`n-XTd|`W0A^AFX-| zq}^TFTE#YNRANXA3WBo#`3r>#Mo5YfBQ&Gz-%@KEcWoDGrh-1c0cf+RBKPl(Czw=i zzpvJGc(_vdd215T#5UWcj@Mo!a1A}Vz#2JC2RI!hDGUhUbEII30jD>CfOITvfSGVrmdDG@h`y%t9NILVwv0qIRuT5I@QTpZ0{kRp< zX-%_iNoAeRV!_Vt8rH44-IIG%<5iv$z=XRPgkElyerrayY^aLomKg1xlk|U>O4xlx ze^Q2%@mBc8CwG0M_jXvC z^yFW{mA^%*`Hl_~0NWrE`V*n9B_5TQWIod#@!oZ26yM!RDMyBM?4&UFQbD_g))8$e zDMtt{R(b6wY_=aXM;cd*38FFy@zMjuXN7+9?nW??)flbc@%HBIJzJaN=U>LR_PBZE zl{+Gm((+B~rTKg_I&l>R8((#29d#*%>}}>=tf?5}vPFranZzfPuCvV9P|IU*E>#`} zoxDd1(YR$yR!!##tKexWam$1s>ZM~Ki{+iU96pL}Z6Wu>u_(Sc;Y0k+k7d|pHPfLD zPGuM=oSYzOdpQjAfQr_c9U=wTNG4jgS=)>|rkbMREW((9Fi{)bo~ym!NB>1q*;c@C zezQVJv^~w?5G2uKotg@N^VCT`y#&SG0HyDcX;s>&2%u&w%>B^4+w8v9PL-*`T6>e? zE~P%<`l~FmK-#@JyOjo;fT~!P=@?D|X{q{$*woqX)JJi(Ft;h;(XM*5bo|St(ca0I z&JU3aO3MT~6iPRby9OWqc%00^RKPtl-~;Wv^2|S2aEGgpIa#=NS^ypEtY7pPm|#7$ zNKWZ&eZ;=6V%n%`s#FHNM{c#;V06^p?L)&xx{Wd*pOmOy&b8S5Mpe>8Ve-IG5HmaM zM9Ik+v(87z*%#+)seddwUG>Z6MYgc#E3aTBCn z?Qo|h(30HDP*je{T=4?%D2|W8+N-XdW>@DA@z206xll!8KU%)ln zxglEi3cP$Hdu$ZOUf=D#`b>zs3f|y$~e=XhtgUPc&PX_6Qxxyo@`4jVW zpKUM)kgWxgE5GO2u7&J%rM6-vR9Hxmz!==Y7uR099?Oug!gP&Y|#hgqh>{U$~WTq+-1C4C1EN939=pjlG%(<4`;9hFS#7F zDuPKVib_hIIJU4T!eE^7Uo{)_suZTC$@f_k-yB&YZbi(P4WKj&r7TOSxRG|+Sw9&V z4^`99GWtoa=396p=F~I2L-Iu7EbLox1y1iYY`F4t2d}R5d%%~#?M8NzHePBAsN`Ml~kUtz#RDCSAx#351bt1SGpFwSv>MceGSoWb>QNrQ+8ziXK2xod2?uPZb>q1pbk{J5ivm`Q9`i0@DxpFPA zW(QZ|0K9`uQZ1()2q4cs1=>w&XGD3~*xHm{pS7uS&%jkLy31qc=Yls?<_M#K0VnBSl&w!i{_pn!OaOgFb?s2zg1OJ6E?c0nvrbRP!u~Vtz80Fz5TSE5D>Esl$>fz;4P11e8fR7;aU@H;fx>bNEQmr|@p)CYhc|*sB5&>VtsmAVq&EDaooYf_ zI02~b{OIMOLu!= z;EK(jZYK;wZ^vektgh;UKXUcR;#rG(Z@=>kby^dXt^sw4qC-2JVznTp(%=;4PJ>Uc z7~X%LWJ@=n0cu>Joxc5y-4aNe6tr#`hKZ_(5-nIzGnkD`l9Rdulgp9#9>y}@7$`hb zHyxI+dWsyO(1r~jS{i?&R5IPNII1yj15iW_QEocstt_l;M-3iCPzz9o2gTZ!5zv{j z(FhM?*dwD)(wPgi8c2qj8B5oL)xEwl)!slh4^jq(Yff`}@&dk4P4}p&hSwtlZD-ki zhg%;#^RMBk2DkL5z6k}tIV1gi4Vc~;aZYIm#`!Yc4`HkZ9|%`fW_wu9g#7++O+Ek*1Oovtj=O%R-3b)CSQ%omd;#e4n5(q zlHb?7kFv!Pl8Rz{ur060HYf6*R$SF4F{qd7qe}DFAQ$eEGOf%AtIhNeoEVy9AXSu_ zFlZ}<6hz5!HQKv}CFP-XmhnI4R)+nWhqRyM87Y@X|I~4`$dM~9^o4AAHj?NR5Kf$- zAA7~ycMg%R>L21>te&lJqZ-@Wb7@_1d4({yrMp;jf#BeRBuWv~Mh*EB5`gwSRz9uM zHi!9JrcSHDHN`NWR9>VHeord{S3uG3%F4EobzOcvv9(YIeEPxG&f zih*Fz*>^`4Nsij?_kV-+znq+{Xn#r`TYqo{%Kz!u=RaCr{J zGKd2w(X3f$Hd$y|F0L(UG%cxEYBr(161 z4C+(x)>Vkt!+cX;{=)g`Hu$fi{LB*6rzX6+dW5%>`MY!`2U-nui|)qM&vbO}ZPh>T zUUhJ3pgg`#$x_*nB|)rd30&f+P!VTUjiQ4oc|!e$Ue7YO zQo1b}m6Fh|s*>=YIZR1~E-%!bDqE)IJXsP;LEX5*-vR16-H2n6BJ~Ah;hudxWFxCl z!Dt?tFe7@vqnx5*k)qkI%?hCIY*CrzR2}_TC`Q6THnPLha33~%3hau(@;BY0-TAVe z8ikyah58oc9#?syrn$%Z%`m!3o;ZRSLoY8riOIxp($vu~Bg>B!V=>Z||7M_RQpJ;{ zS8R8KFt|h)=8v+AgP|gF}^MuHsU=vA}Pa6QNDbR z=LJ+mnkr*y1cB!ExSrbiyuN)cYs%QuJfdj#AYHAV?ZZ=$oB@7$%}nW%{z9+evA-M% zduxs>MgL{L2O$nwb)XeZrNg5O$v?D#(-UA28d*9h?C?i z+*SGm=9<0~Rz`mdw}<*I}FZg+;>l~_howe7x8yW{P+5R?@gQl zUd9XXxAKW1(sw^k&{IQ!uJAxnY=#AH?bv8tG|O6qL-_E3dT81n3f`sr--AFYl4cod z>u`dvY;wr?+FI5>=4S!Yaw(ET`x>Bqk_t;}E*wkYDV9y-%b66c#Q6M|0*NJ^#95Mq z<49$9(NvbxXXeGJ3i$#!1|#^2pyTlE6N!0y2-pf2zDVMj(!pgPcw%K?HM{2%>G^ha z#8k$?(5%LRLMJYYB6BPZ5+Z2kYgiV=k#V!d*yZ>QyR3iFIRjT0@WEhSTMi zQW!Nf>NH4dA+S=5r1SLYwty_6kkMV?-;AtTin?P&Z_*S-4hX~TP8{Zg3dKUoTjUH) zp#>&s0ol+WHe}fBk8MT@lJbLN9MsbN>`%Pd&Ln1vq?JaQ!o^nqkiwO& zR3hH7EFZwNSMmd4yy2?-SuL#)j_BbjjvljAT%R(7n!l~uJ(3&3-@2Wiw50h-hSOLM zwM2XdH@Xw2h{Oe(nX#AG8Yz`pQM}Ee8Z=fZ&caE|25{@FWY@8n%GIL0PcPWHdrKSF z`kp2)!f74YVBFd}3~yU4n0~PrJrAy}S0O`CP=em5@hau9*saeF^02$DMI%)9Qg)=f|Q!?O2xZd1ta-XvLZ% ztjU5DT0~YR&KSx$nT)Lm-0_H(&!V2I=~WRoks8Ad2A>#T8aCz_(0hLXLoSQ>=0Ag; zHU%vgVxBOmM1VY8Dd#@Xp)pNsFttLl9f!+{fxspF14_!}}Si`Xy< zarh_d6upu3g&W%Qa8n5C`Q~(HhT1>_+-b4Wu7M0|zmGkE?sNhNw8a!0thWlYrAYetz}c>VPP-cgqR}3?|~G;6Xn9$cFnT4B=`!FYZ}$)okTPA5~Z2ninT7Oo_hjSe?DcEs+mZWdR^1X z-loj6`j9Px%RTV=HU~3n4JIbS7XSLLOwc=v&gCN5*PZka6h(nMFy%eCb1zYS;D}&3 zHv-&j03e$S)Kz~3_KHzyCP*JLi2_-rR4Q+m6MM3qP9QH-fY5aWHso2#cBNxj_cB7n zs~H$UcRH{Rs~|ZfN+z=Uslybs&#$%)BrcS9Er@=t-M$oSoCVnE{ELQ4YWIKx|{RmU|oceH_)&aZ1~=r4Q%>W z`I!OF(e?)B85i|Vr5PMA58=I5%}WjNJCj7vtvhB2k{z4Um}FQZlVEt4iXDPpJ&=SQ zXL4?c9z=4AKLbY!ORi6|ADt-= zMd)~4i0}=5_UjBo$1+WyKfH~@*A4r_&nnRq(^k=LJAbw7Z z1I~;|<09?i#iE_!#V|98;@5vhNhVSL@Fh`S`N>YxX6jjAVppk_+-j6Lw4f~0#i>XV zlnu;HZ3sPj6|`rEePF~cuXli|p`EFdU8m_`q=hY0-zy==6Gu=_;f=h%MGncZR2~o| zays!!#Ok65lY!H|*<{Hni<)(S=5x=SGU4t3~crxyzocNDk%O;lrXG5;dcelY;O}tJRh9+KY}n1 zY#A`qhm+~CljV#0R1>bods;zW7rOHL*1$S1VnR>G{a2UhXS(?7!B@A~2JOfuQgmzD z9f8j9h!?94bUX&M=Y2%`T?PAn28dk;i~!05IjzCSEwTptChRE7LtrgZs9JHs)Vm!STuSo7Y2;7#60S}R{#h|sFqDJG1#|ZU)n%IoXN*Bc| zX_sQp*7ngUA6gx~FIIU&*xiPJTzKcHp{}%lG2SA*peCaSRg3Nu+7FSRzozA!RKxj<> zGW^>%Bhd#wmn}K~2b;4Ax(o@6t@7W771`H(!VtG$u-yw262V3`Y+yES;Zq@*%&w`!suc}T@BX0rOKCyN&*qQK9jBk4{70WUVy! zI1zWu*d84>gVAgQSf&GW8;=+-Nt;!Q`%`@#^+Dky>D7B;Vp>g5p8h}mbLOGws9VmQ z&@a{CoX};01|4JUrfgV?gp{C}7Eu=o)dl+Ve7`lr?~?rh5}3^}7rA|Q+|}jDhK`CZ zL0D@lQz%pNdSIQYXuDWDil>&Pm@9+VpcZs63-a`KU`huQrPi}!?!zp`@N&T>Vo7r9 zm*y}1d_eEl@sio`c#9LvxCzDnJb4H0 z(_s*zZ{8=8g{dm`Ti{7aYi$x=J1ob=&3?O zC?AF)IDy>+GB}a=NO0my?;ota>ev&N|LjnbHj<4$<%as` zv?4bfIMqX5{+)f2?J@H0c7lFoJMI|M>)g z7-bllq$(3@H4!~6!xi8Xsf3q6`OTNpuY@Ophc|n$Z-h5bnrdE&y`Ny3y2;s#9h5y9 zS@mS^Vw>6vV6sSw?Q#w$|3RHve0^`oEN`|F3a1 zNpaogr=E#BqtM;}8*_|0@ejX!22>*|{?2cuGRWxA^hSkwC%a2Tn?>iq>Y6q)G#e5Z3R;5 z{n{`j{e%tADg)8WhlYy5f(bEg`)$R;U<%Qb1tZuUyJA9dv3{%JESagl1kY+GV?D24 z0PwdQg6%5XrXx&;|BO?wL;r22N50v#j=%E=qjJpce(}Hs?fLqXKjZW(9H-{kjUV^U zti^YQ?KcIU!(GrN?lJ}o&>l@;y)_es!2Zj9J|J@1NS4D4N|DCB9_-UqWKddz#nu%8 zpu23YBmlvTW2!(a$us8e`}=WVf?nHY{Y=O1LM z*_zV+x9{se(@NR6@i2cswY-os) z(o0y&x!%8;$a0+KzV@8zVvX^3+79?7msjcBZ+l|~%Hz-LxLZKuwiFe!JH0s*(POu} zPvhmkouhP{Vq>>2kCNjM5ZtqOw7kJJx!Gec$jei(*Ff>5GPvEg%fxlocl`3~&T6y| z_3@C8HMA!g`SMKhF=d2k2nsMkn}UubQ3|%Fcj9<%Q;v(HVWJ-+ z7U>WosUxe`GKiEk>d-GjA19RAzh*^s<>cF!?sK2yhpDK~QpW43A4s&o!>6k8UGRU=K`$XBmC7H4f8 z5MkA#m~!q15>OU~>ap58;$qbzWO2NQu$giVqn6qM`IxK-S4lB*?2j_y%-3cr6EKbN z(nP_*s8w}xm_J110o@uC*iI(+_;Ba+3;V^|>(sYY75;-tJA~C@!G?!oJta{0`pg!D z>?}&RJHdksfH}I z_s(r@VeGRgs#OPJ!2jkhg8cC)UAu+}d8G4K=rI8iaPg*FP{)08omkh;IS8bT>yj)TWB*_p3N<1c_y|Mx+xDW?w?ql1b96eSIMW z3i%aXagPW&Ii=iL-7VIdC|1M*fVXNf?cw%pzJF*Zl`tSJwD--{^FWv2gTfFeotM3O zuCFe(u41;==JVr*{S_wGfT3nUA=h7{8RR^7%+5^`uv7gkw!3$ld{bHZPnBy}+csLY zEu$v}xkaji^=lZXWL9ATA=@NoJ88;E-a*V3D_0?1Oh|Xvn$s|+XCZKnXB*QKy>pd) z5Q1(0;$9C;A_(NL_{>%eOP8k8O$1#!zU@SOS=Om+Dad1R8>q~O_}S{=I@SU+I=tIT zjD;Dh1h580v`|ue14OVe-6T7rfhPmPMriICk*7hkuugm7W9o(^>NohzD$rOqTMP{H zJgud9zgSN2%y!O^6f=bQ50c!L-IO|gtj`)1d4g0K?mkeB^ZCz9X(OKXuE%aW;Jo4! zF029rSL*F*(av2ly25!;UH6TVQYO!lwQ&rX1_2>eXgs90WL9Jfp;XR43}&^ua5H+l z1te7gym4G}0v@fyvYo~-B#v77uGd%59&XNv7XVFF+mh|O+RQF4@l53iaaKbmdqQ^p zgc`=3sE0uC>*^T6ZbAZ3eGPerZx#go!b{E$u|cxPH(au9NQ{yTV90 z1W*_AZAM&`5)ABsi{vaF6`)EQAd5AFy?@wLl_Uk9gRK4x%L=-2hRWc2dCEow5A}Wy!2nx zZQAB2_K=6lc?Y@dz7~@Rma`f1lut7vQIdSD)Q7fWZO^iabB0sAuv5`=wg1Jscq^uAVfJk!*@ zsSrfkcKQAs=oRt}Plu&In} zQ4#6x6Y+Z?BtZqZ|Oi18Y^ z)xuiDJ_|!MX#?5{TErrN+wbWG2d8TMZ^|#Dj!{E!BPX^nT+>ul#xZ{H&g(w;KFL{K zs4?%t6H9A`Vj^8haiYu*B`n;gPG|GnF7|W^!ilC$X-tMo_4CI%siDRe4Yop0q-n!x ztVr>}2n3B_-XQHU*3yswA=o;imMSv@Z7fKIA^6+dV)+{-7Hh&H#mG<@!#8aQh73dN z;>D`4mdNlpwc4YI46D+1Bk(ykdw_Ug6^w|ukrbQCUY8bJOID}FK4wYHeQw7x&1o{6;jJnzw)b1{a~OCJwoagGlk zJ?io>mzW<2Ix2%W=_e@Lc|udEO_7m~nrSuK zDBFtzS$m%8E41Cyre%4!0x64^=T!HvpWIq!LZQEdUw>1RlX4pnwZVCpiW!CCS?v97 zl|*vwA5U);{SXU40#Ec@&14kuL}Ek?ybXxi^7?YonR2Lf6bq)B5zNyot>;5}4(E}^ zM*d7^6M~Vq&mjRF_ym6?<euK3ggZF4|k>+MG{+ftjSqMtE*$ z%`Unl^T}bTm9}Mt%9ttO$EnGy2|H_I0@DVg2rFLgtt4N(qs(E4K@hb2+=P zm)2q)sr=5!fEy_QH6(quD3+W4_4DF*_t2^5%PzOg_SoQF#WLkkW{`=hDE+a5u)vsC z{MMBR%gD_)3LN(LNV0;|=s(&ulsAM*;`7MU8N!X6%Mr2kJ*ranoILr7)OO%J)}bOX zd;U?CL$&9-iCRh&#}|9H3bLA`qAQ-6vQTQDPs_CW{fi# zXEJ2>y@`*jYxu7{!rI2C>y)<5a6sIgCBqSQPF4;^+Fv$ zr781&c&hh8*6WrRG7-zBW8z-(U7Zd*&pNa2_8O)ZD&nK3p zJ8&=ydE^yKshXILNfm|Oxl8_yY>ix{DIh9{gbQbKL)w~4R1XK!myl|6_Sw3dI)|1kEH?Y6Q%j2>v6@KR*bx~)EzK^NCcBGhOIlQteojbL2K)4~4#Y?ZV%;bqI&uPrq@|$O3QpXR zxr1D(D?hxX`A-xN^8+@!U__74?ra(!^&5nw4J3GKt>|!kv0F-cY9SG1k0-ECsop7Y zUa_@NgoLFq80Hp%LR!64YIXNzy&ErK}!Md(yL7DpvaEXZ7#R8Zw5X>_bL8hxiCV<8I*N%(_UPF*(S1$rA<2*A7~ zJs&2#=mI^8mE|>BGLEHzso%RWScj#pgVfFeUClohf)(sd0? zyd;HGacutd;UQJW+7z>G^S`N)rD#8(r)i`?FXl>-xL}b^5WTP~14A#aw z%cUyK1{+DmpZ~ewBF*yUg&JG4jXAmHh1$k8592$ZD><{!`VXP-1Vh!NpH5PO`dwo| zg%h4MXi_4(2A9+Jqo0R4N>gRI5j~wT6_)}I?@T2F!}LxHcw_Lr;R8FWnYn~(LhABK z>y<8q=ze3PzL`YE7lULpcyLY0_362CBr|1cIiZ57+PsoKRdc0r9?g8e3y-Bj07y8` zFTa*h3tdniU65d^AV~-&xK!*(q4-iwwZVzqc_C9%v8y|+mV>prVqcflcg3Jv3HBS{ zXCFJ|!r*=4eHYlJr=M@1BH6}PxT!2GNf?09WjdAkf{#da?ERl3>Q?@=fYcX`tk z=R=a?R1PC(Rk5l~PXF~lP^ZLljen!AfTWr5I%O`bExIwDeli}JFpb*v#ZO|!o1*Vz zhKHY6Hz93{YYB}sz>uUatdJtn2l#s$D!y+nxK*C(v_sc}EMg^x;SH@YMTTBS%I zg+}eXkzhCv&}oavPKhNP^xj+eduA9g+F+%&#x&5%|59$%;E~2#InVjg?j)7a9r?Gr zLFTIqU32R!pTAXso3R0+f69(E37?31op-iwT+_6;*(M&e@nq4u|6wdbp$*aqCbuj+ zAlZjroBi9 z{(W1#16@glKM77Hgkd}>Tm@Ep0A_jc1?8k#L45thTAskqby31{bZ6U<+{)v{DI|_PV?Hw^xQSPA zpd6D3N?g?p0Lp8lRX7deh+){RsqmOeR`lc6UyM$S-F z2T=bb-gdEM&8n6tF6WjwH&J1B@7m#s`uB024C_M0%M&?FW`Aq!RLIX8tY30^f3CCU z<|($Mink$d0G+_#Jtd~UeeuN(fXpqvg}5F4;Qs#3%ntbLDjv(LcwVgg*YTg<_%4da z95nt5RHsDG=?!M54dA4Z{8cmT7#{r2oOpc}OvYx;=I@5p;)Tm2KZo-sxQua+VGmK_ zF$T;B*vPQ}g4G{5tLTgh;4etuf^AkJDi5I4~l3nV<9*W1i+*K(Io|kw*;?p8*Z(?% z%E5~NdEUT{%Hn(=ZLz+}^Ox$Ne1Uuyqk-Z5n>WJy7ee9+m%34PgAKw;&k$D_JYeV( zQG!Q;6F53Bfqc}E>!+5PKC;fFxg~XXqNCdU>41|a^vODVybDchzEd=G3rQAv z*eqM50Z5Bqpe@{p3 zAUFuUXMn2}h8dl2KQXxfFz!%}{x=_WIsi;xrK=?Gpm%W0OElVWjwe+ue!sNT3H6x$ z-EX5&?cWeLEaDSoyhTW%2e$a19gY8ovUlLNwB3>gbKACU+qP}nwr$(CZQHi(-1g3m zo9CXY?*3|w8a>W$So3*jtcaKqiKi~ctZPUvYYD}6iOIC2zmo)p7Sbi;F_8^XV+kag zB0j7OH3)*TqKz&hOf&+@7Sv#AMACzfemQhs{1%m~otSfnCo|7i8sSDElb8BTl96fr zZAl_~%pUSy%m_o4G0iK9^vDcFPAqOmK}bbaLDnmYfGG(pfH4R&{6Ii1#tzto6(vPe z>m}EXJ=S=D<8^VpHN0PQ4AhpevL{>c0LxgfDY=kgx?zObz=1Tiq#%2gGJ$Aec!hRL9}wm+1NAI{9ZS!ZO5qH)sU5TEJ@!%;W`)32xr!c=eT>27}F9`WgJ;ozU# z^mibJ#|Qd!jbub7vjan-)$lZPRkn%|pv#@|Oc{-&qITK+OSI!Ci%n)FWz^-a(L()v}Eg z%UsFv$Hil4D%;>GP<@lB9eK1FFjWOYe1UT|FBTRKS>2k;CagE1%=dx?aWvS$%pnp} zvu6TzD;Wo5ZqQw)LiaTo25vbY;Km1R4~tShAu1u3XCbGOsEc^H3y)fENwBp&@F|#P ztU`0P(ITESJ@DEPJ=XiEDes6Z`#2;oz7L69B2<=Hu$M=-@tWPD<}*Bd(h}X2JzgebNl@5DV3iTL}b}-)I zD*IICY97WC%pj@pdhBLx6p10lazC0}86V2jAy4hdsqr_|K}*cO=v5oOgA)XcWKq8< z{~|l7W5`?MAjX))yi}6>Sn2uT!yjlAvv1Zcw%I_78OGQrBrP`|mK}aFb`#&nmJjyM9q74$pv6 z1GvqD4y@U(9L3*V29MYK56F7}vSZ&}2S~Wj!)x52zGB($Ttqr?x zXRxiA9x~;4$5JdO45O6Lj>DR6Hi6p!92>C!niq9w=ow)`eF~a)fak=L1kI0H^47z% zwxQ;UXI>)$VLSQnMOOj}ubVZQ_7F$`q)mz7fd`U~&!3Zr}#GASbO7idJBMseS3eO7L&KNj6EJWoyO;q*BX zE=|T8gY~I^Z}d2Kc%J9EM1maG`GkQ+aDWZj0X-z{So$;N4d#llOrWyckx1#pu63nN z2$`20o_B5Uo4}Z^>^iO8#CP`@zP3aDo4}ba`Vh{Dq)0 zh$iVQeWPzdJnsg$T3hmoZ^VU16AK)@4fbDa_L6TZA3rRN@m5$;fA+89`p*@ah%SQc z(&bWMzmt7*9Qd^DjPY_6XT@yKLl~?n*jkPEhDGmZhPs0Zm*ZvI;+bK*%JD6LzLOWP zijF7Z6qWHM; zrlBdG0I8LOS~GP*D?#InTZa#@#?PQDk-$~zQFKP>gpdV0>7ZVGh^G8`2(g zXY4$`393Bk=aEdYSQjr|H&BO3h*E}tq3zcvd-2f*oxqy98R<;Dtu2XI<6NICUYg+% zn=UUw&{kiUp`pvbH^FRYXObCw)_t}G{@Daaka@#{q7teEXtJk?xCDA4d7RtEP`c$Y zi~}A_HIqFsBzFpJ>5xm=fES?ayGfS!vCq{*<~y<}&D4@Hy4f97>;}ONHK)<3Ez*E- zM^`+hCXrT`K>H&p^1i3f&!3%U>|NTG~(a+;Arf^oX1dX%Hh!LLguTLIdDhC?Z}aEB&r^ac&`}+j=X;r*+%U_ z|GwP)j0NaeiilFlE96Nb4SLgI%_1>W?YWc0ec^87HT6P!-f);$2&s5LLnwXSIwIYC!DCS< zY1n>JrK<7=(}(`}e)d*m;V?5uQLH&3D;K`z>W)|UVfiNe*xeJHEA7)p?20IvjE&hS z2x^jpq@}}P6h@2Fgxwp)ie!!!XJ}=0m`>Q*+H|H?+|@xB9)P_GF%?FW?l{cqvit($ zu)5*{54+^qczqW5Afivj9k&G?pV`2bN5}g(_$Iphb-T$+jqFU zQ3zQYMRH6&RSsAaDi?lIFSXo%CQP$UsX7Rt045i{a@SoR1}6+u8X=lZIXfV@5M5%( zLyJtKSn?>&V|75m%|CnMFLIZuQK-`2<({VPA?{%+TA0wefof!H8r{96?NKWD!LWKq3lC2FipP^qp9usdWYV?aE|ZgS?1gI5621n8L}$*Z<7}9xh99(-2h? z>Ej@+if2Rad7#b~cqZw~%CkM7YO=j?4xe2(7dzB}5U`_gtg;nbxF6w4T<}z@xE9X| z<160fm_eJR@cGqoMXE;r%(f0!=^3bG5X|CK8C*UrWG|6{kYa5WLKwlJ}E7W$VRhnVTV zOql-@5c9=O(H_i5~L7q_4gh}uA zp?Z`IYKS+8I396nf)5~zP#nC9^Aw=U}L0fQ0K!av&2FARJq!gn39DHA2Kx)FpdO{k%?ob9)UD6!9=w7B~O3=k>%nU~!U z!%`4aQB?GFt>(GQyTmk6K99Oy6%4nUcRJDZt_=|Y5Pj3E=$)zB`R%fSL(E9tqx9c? zz>7_V#upURSNPIUsX??= zjHw^$(rA0ab$*yAB7_$NGRUj4)w0XB>=kpXUSaSF@f#}BOx&tGNv51{L}c6#-V17& zleO}4s$Fmo^owH6d`8x|Iq32o=$~G#9SxLkEXZHKo-zLS0O0@jYW;I~{yjP~>K@wI zD>y%}Yw;Tk#QwwPi}{Ay`1Qf@qo9~1P?*zzGWcMv3^=Y~I})K7oLpJ(Tb6WQ=2te% zFT6@xF~@}XH4&RjyRSa$Tf1RA%_~oTOw(onp#tO{!anbJw_bf;Z~yq-vT?r;62$?Q zCq10$gY1%oYK_=uq1&Pj;XvLD>oDG^;veCHK)y`2tBv|Vdasx5`*wT5?gqU^uKXFk z^JVbnulJQ74MRqTs)RctGJGqC!yEF&6!`C|CE994$jp-dAXF(jJ8XnYv4l9y8%4IARKUs}98U zmK{Aq@dnUKzAp|;c}t9@qf!s)K|}>wqLFe7 zEJcmGa=_3_kS9J*9K*fx^!eMVn2v<>$53oAQGWdfO4x?W@)Cgdk2MuMsu)0Ae!y~$ z=)zVy5C5vJ)7uMR!t>io^N{Mbr#02!0* z%jzw7@NaWvH+SML(L7r+bmj^bL5F&Z_F5A!HL0c1(^#78Eg_tN=Rk+BRw=0TOcWUB zr{D8Kv=~s-KK(W^;^mULq}oHp0jlf=?8{j5+Ff(tV42+&D~`{_y@G6~=bFOwwBztw zC)c6B>rE5QHC~cNqpukv26^<9-MSVHoxV_>i0RD5(n~rBh8DMwe0VySNC=Nb0t#8J zVhMc9SRc335zsu%a9t1rMTZ!-S&U!XRIN)H#crG2EKJA4yEp;J3k)UWt+C}KEu;oZ zYc(Ril{`6(AlM9VTN8U@=7b#MtV@o*HkSBKmku&6eH8>+n5{P*dM22YyzXHjgxqF$ zbTu&!MGh?Je6KW`Lrfl6zx~NO z2IqE*4QVkcO9`)vUfSnz$8(DZ)g^<2OPG{Ia_v7r+7F?NCL=MjtkL#pF!av4hR_zh z-1`UwB*?gPsnv00cqx22Ot9WDZ zEi`2Etvob)>xg7&H*|GW8+Vr+`5R$|(l=OV@+mrEch7|igCckI%J`js825%m6&}fV zL`V5e$=&loO`tilCu|>7dzaB6ZfbBstZce|avR+ba>AqslL@_o z!YW2H87tx54dthwiSj$bhH_K*^C!fwsvWmZX74L%*)}E~aaK3(?S2=TfR_%^rAoy9xb!NO=H%b|&`F4$?fLEj51=jMO&r z&?J6O`-p>e88vlfhJnabBeL+p5)&e&87`Q{q(X@{MVyG)sEUyJ26LA=UE19PQ)Q3y z&yyV+SKEXRFjrVcx7tyC7VT*Df(F;IvC5RH!265r={h1Z@m>t-vv6>!&^PGdw!vW$qCRDDrThmv=JjFhCt<5Pg|&_b3c#tDru z?U&A3-zl>-Q>$~3-{{w8w8`mfcu8=SELwf2!?T~!AfldaG^ba@Q{3aI315rMXon^- z;|oi}Y^xjO#%rkg z(p$+3_S0xt?O2O@SKkOvJn*##rRfJ>2SExijylDi{dcFo z;~qf@L7q%q11vt^y|Fi3?aP9=s@SEn=Qx5t36Qh>xq}D6cox;%FNY1FSnZXQg4e@k z!+(75d!!*efZxZDwzWN=ohZc$oSk?hwkej#-x|lbNfXEWSzB zF(d&TFbtkr_pmJ>i%D)y$?j-pJrR5Oox%;I8)8GjmrEED?_zR-fE=R_P$#hhKAT1# z7=pN&8nX`w@kf4eBm_}bUxAB8xBKUK=??>&GdPEcl(=qcRFH15rNxC5`E4?tL+QK- z2x9ptZSF0>i0!hGKSa;&V@Ym8HU6B-x~!Ju-JbUW<;4`2gwun`u*n@1FM(0o^s*!)Xl{@<;#4U&5jem7F`+-EAfcEyI5E)_ zTm|4x5&!%N?d^{9%PkV{we!fz%eKOYRiPn^}WKf zGa5}DZ)oR{F0NFAKPick%Ed~`*R}`v+0iefC;CYSadg~bo-p@!*hfGdgAxqMF;j*b zADo2*saVg!dr&x<0_X0(?#4v=?6__kM%%wX1jnLT9^JJg%*gUA)iBoGC^9jc$0>kE zSUlS~z_l_8VzS*}mQr#6vil=QI4Bu%^(OT6=@HE!CF5Rjo2+V5lFZE3-3zs4afAy) z6z0cs>=Q^=Aa7(J$%r~b^vBRR#>nK&wvqdV!EuZs%GB{mSb`tD%|HjP5ROA@U+&_V43LWW-^WUb4YQ+Ej zMC`wSrii1Xo#X!-q#L7Z;f1A!!iz-L*_z^RYHg-h>WZqNksaJ=Y!yx@XV|hyI1zh7 zJ*kd4l_bejX?i?tVnLxbB^%I~lGBKqNiKt=sX3@=e&Dh028kD7_X&VI+krXNNEG>X z&}+8qHQRB^^SVvd_nCL53+(@O_Xukq0_Qvkb|8dH?zP__%MOB9dO*bdsDO!Ec3{Np zoh=YJK*qW7N$lenDG)h;WbB(cfF3j#|E>Xg1kEdUdWMm0dI!STH?AKzkT&j}D3Cdj zHt`7=Bmt^N@@NG;OaI21v8QjJI-q3in>4@yx<~kE0eXg^P4Gw=1RX?~v4*a#e+Qkh zhG7n!OY#UBL;=b+@yQxQ9dtJCohuMK;5P0Z5HPzh!q_*ZpE*Fo*f)khxBtiZ+kXH$ z2ru!S2oy93FX=rI%$KlVk|iOWgpp{Bslc3^r2nN}WfsnYgodjKZpN5Jj7t5zpjw4m z^E6(QR_kP0TY`_8-;PE;2fXr9oD?xV91IbCPp2r!5)x^2EdHW`q2Syd9~*O`7;tQ; z^4DRYo;CFd&RU1!=;47peY_%5GD9-tNFm_U%4uPPxoFI=(#1Qo3-8Etn0tFW!Ec36n~C$3TwpmjNizOZ9Lsiy5jPE1Ty22y~0y6c5# zu^|Juzd0vUVk{_Ofj((&B9R00{HYE?-Aa>CSt*1jht4koV*fNhG3Tw-2l0u6 zsLg6t<@xDP`9%sWQ(P~pRY{VaX^@u4nl+vf34l&$u7aX8Ru%ryCYN4ku373fn=P{z zL6J5P^d}fvuJRVrJ}&EY&t}E!#VHCkfH~AG)cyF|ml$YTwm$6EMok0lDx2w<9ZO4* z;1{A$3wAMyyKAaoK^aw?F?F-iejZoinC=RW6vk7H>pytC-zKLy#AhrsjdvM2Q!+z6 zYpgRvW!)Y-3TU3CBW&S1OFVL`TLo)u>(wsvIDdk1CQ!E<2?8YGnBOW*OWch;#GH_s zTuEyrIjZvL&*fED&R*mNvvC%rOmL9SIq0_K`nXK!K9O}%^{*UN?*>WSN1*J2^E?KC zr=UnMX$O6D$@vxh0d6Yhb)jqYx&5=09n-BOn_NalHPKTg4(uKT0q+R0IaL;>oFp~c zs6fOP=*(Pr%FdDtT{p;8BO@oW)EY`t5>;L4%S<*SRms)U#C;!#-ea}}j_bn(VHV$<63YOFHi%=Gst zUU$cr#uF6@%tSbdkhNpi_Uc;jv_(0uVJBnEsA5ZHETVLmgHWy#>$OhYug9v4!j8Z* zsm)s~x-v6GiTQ*IZX>NYayaU)Bbw4(#)>YSd#0MfWwc8Wg*`x}5Z>%HTJ?$=(O6s8 z6WU&4hd_^^t&mq`G*cw==*dF6m_CKxX^PA$-NlkzajfUhtWCyy{dE5s^uCrx?Hq(c z8+TIsd`UW5p-rjL2*O0h5~NTmg$(660*P6p2vk~TTGB3LtT_M~X04hRQ9H-QST|8R z?&R8M_iQ+phiP)(kBvo5Rbo0z-p1xF5^XKm>xnPPmywC3wLMM=EID!R9_mA z^$bctEf$f2IH%jl5P92d2C_06%mIAYQ`hY$0*|q3_2EfJ@msSci?w{6<1T_ z2KkHONVQ{25Dv37N;l4WROq=`GaC8KU){<7XBi)7JtnlWrIm!dgOq&_p z0EA}}`(EcXe2(SsRc?DOuHWZPFl_w*008CNN}RY~jtYEk2ufT238XIseUuSbo>tg* z+J55|U-WzU^<{wSkBoU2NR8SXVGMv1yQh`yoj(<*AAo%^GHEkER?23d(-DmMYnv`u zT;B;N{ZHxD{DEHBUXI!o$C{CNLlz{Va6VjNJ%FJ%4oH&$tveGPpoVfNrdWu1 zNZXw@oFfce2PUp4+~n2|Cg!J)%{VRj1~e|V92krfX{c<8HISaEuR#*D3+h8{q zh#T!qfwU~pwEYCq0qcuNI2IVzEKsjhP`*Xt^ns$g8_m1OH21as$m7TP%+xN;;#9x! zU99>C%<4mFx2oj_IgR7!VT=wL%lEYHp$y&GrxvMPz40oJrcfp;p2V}X2hQ2*vuRLL z4_^|y<7v`aYgC5cYY(Wh6xp3Z!|i+HU)h-Z%c=-KepGfx%dWE5@La@mx5jv7pd2S? zTqJY1Cpp=hBNaP_X~*wD-C;^X*_%|JBy)HtKG~ad6>AYxBy)MkTpTRZ6U zR~#<>N?;?l6OU-`@MW-0m*M1+xqXvvPM73myZDlcXDaXLWq{e7w0|XiJI8n>qHM(x z+=)lGci=KvM@!}3DV?^H1hP9r6<#B}7X4&l94H6M%GxwA(31?ZBO@$y_#28LDih9D zD8*cp@-;{e4HY*CZ~(zh&K)D?9OYHA2XuS##_P_<^5O^ZPwZAVPp5YD zmtijW&lzThe=^Kl#ST`{o&+?>e@OTdC2`>Gk|Qbo>ygies(S5Mqk2+$qE@fG(Yli( z|1ZNFy@M~A8D==ab+y4^W;z?|_xC8OZY*R0_eM`Lci9h~i|bf5c^lDmgP6kPPY~Xm+M4&k zop!u|iH;Ta$%GaGjWmEGCxZ={WFi`NyQ2Q0MACAVc2;9X2cf50FbQlSFl==_yB{9s zk?x2Smf*^mnmU;I^Y$7lrpWF{cT=VAq^2Hdn+w4%UVU6t`EZTl%?14YDq|4hT{h{P zsZ?T63Zu^mYxs>27p?Ro*xp1iwoi&-5MtAPn!#@K0ujr&j%ZTIZS6I$k1OqneSQ7B?SGZMO;Asu1g#^_gat<9be98t2hQX~#V#ROSZ zN@l3eQN|F&MqB56i!|iPmFZG-zQFBCo`=~}Of{1v*A-6RjoTLA&~|spVCT%R z@fe|Gv>_O{?gtN(xBBc<5yK8S5nMQNbGX;Jk2zPk7YluRzF_+*?2x%5C&Z)nA_MHC ziSqU;11#FNV=%|VM_Pf$AQ{OFHS?C>C0UHN6zKgX0+4~z;Er0O)q%FgLOtvQ2-VXRMY{?#X&_27Qee_t`oXqO6+&TA61??DK`Z?74gC)J9_i zwE!dWaGiSViH!LCVh;g&1_(q$cGENoGzxXrp6zOCJJ(!lr8e++vW81QN2)t4n^CSs z#J*2-<%kU(S@ibd-(3Eo){d;IM$w?8=@lK>P2Bc{~s7xI?9f*Qst= z#N~Jfx*}m#RqZm?o*c@iGx5C5jBm(SSLm{(4BaZ^vpiOf6hue%omIvrsjBYZF(xyiX)Hi0-!8wZ%QY!>%1j_L5r@f zFciPcff?vnYAzYwp1bp^um+MMA*BVVQL5Uh1oq^7IH#qt_*LRFA+gekvCpR)gM+?P zPtHewyLKt~jNE+XXia72Rw%8f)}R5~AeCTJ|BkhuvqeV;wnb(UG7Q5>pP195o1L3I zv1ptxUhulCwxPC6PS8azN2&;s5E86yvuZLTT(Viu*#Igs&}<3SCDyb@R}mP@Z|CX_ zUN4Ckd+p^7aLr=YkbI?-njlvCd!n)E5R{JA6>=#?F{^L-Y1E5K=bw zy(vMHb|+WPPiY!LwazK4R;ywv`<#&jQqDPTyg#JbfV)F=(DHZ)Kr~18h;q*+`WkA> zfeGTxuoKJ_;X$&^-%?!Bc99nuwNUG_7wD_)!hhu=$06a1?$4?ZN^^;RC>| zSw>v*1%Qx7m<@iZS5)r>YnW)=zuJT?eqPoZ{~aYl^bT>sC>-f40s3$2!Kbt+F2Xq- zdyF1~AlVQAwXP`U9mhfFvD07zIOFh`#Y(&gVi7x5C$m60f@TU0dDP~;%#&Y-4a*JD z=cyJuqIk#wgzUa#mpw^#kgW24q>A~g1zf^;@aImLGEl(01w`GxcloDAI+@UvZ$R(5 z=L;CLB~7`;hhJ>(x#x=)Ig=K;`gUpq|}gLb#DT6ouni z{%Mz^_~Q;dREK?g2+DnklXhX4>`p}bvn0ZQVHoVLAlhxRGD{E~hWSF;>$c;}cRD9SudnY1m_GEK z`+@vwfYrcoyuUu;vE9rNo`}?OpY%c$W}A)PbRQt7g{ZrbC_7U-@fJFo$+^eg&3MGA zB$85wV*oWJQD6kgfLtiF!-TKgv7zR<`!rnK-8Y5Hx95f`x1J97G|SLD&eB#L+XP)! zpyaw#=R~v#x-xn6F8Mxn4c!sNPl(Z(wo9_ta(T_?o#L75arg>=3K-GFrJMAm|Wd9gL4S8tS3x3eIeU>J?qwu|Zu{!;mQ8JKx#9N9L()Lq-B_!mSU z8A2o}&g%jldM|f03@^M5I^xOvv{6?;Ks<*GQL68oVHE8U-pTS5h!Ts_3)+RwUA*z0 zhCafkFxJd?r|k;vlHEWuy0VBdmQAnrv=U30%36V^q%m!V zpEk%x@31$cf6I8tLEt&1)DG}z`S;?$;#OY>e!%{@*@sNI&QC#p{W^g9^-JbI+wA|> zbYNg;ZSt=bFJt0tpyX`l_+N&E|9Tp#J4#4u$Ui&8#zN}HSQVNgB2Wd0mCc$8TG0S# zK;;6TmQhot5~yrl4sJl^ebr1~a36|)z}^m{m{hE}mU>_5E}jbT=yEK~E}yccFaZ^( zg7|NGI9_x2I9}a&Y592nymi z2WREO1elPVMYJV@F~vYOHNvr!wR&cq*lemT)Mn;9cX*3pKuoSKJVfh(h03rVjB&ov z|2&;+gMbBCdCFdz*66awcG28Aiz5W*yfu?VH&S?2?Md`ZY&sVeAF&9oC$PW2YD=CE zEL@{ST6TD0GDHo6%23u{fV0!CQhE*^?3DpN!epcLdf}C22YPWvC{{-9sxXyDnNdtD z-rThsmA|j?DB##>8CuO|3bfPCQIJOLlkAf$0UDO;NJQ^)8{Ktm{+O2CC>^@qM;H*h z!rkOTmZuF=j9jAhm|C>8iWBH`5l35!6?*Z9CkzY>gt3P%#+sC*nY-OoKOAPb$##Qj zS|Mtwzz1mmmsiapd29vyj1ZnAppRB9hH86>A=^v&`KT_+pL#Wb2p?d|mm)C{j3~;~ zdUAXHMVG~?R6O`ijLtJ`kXu9qj!RE7nH^Zjia?8P<%a03JTq4O z8(YX?md!LNPy4a#1{6YtSJWSaN2jbvkC=h{h(gU4gF8Lz4GD)=x;e&MWAFn%D!^?33EQ!h`9sf% zZ$*$02#tvZaV~|MK?yWP*24}U1se}VaXJvZ)Iq`<<(JpU*Hn^;*cDW0=Pi2elnPre zATsIr_?eM>(LKmfkRJ*|&`sNA6~oNK2wRD0Ip^^?7sxw_y~zZa=g){`C3;FD3*3lB z&9w)*yeASaOX{ZLf9XV^H1nnaSL%6-yx#%a5JNJ+w3xmU6225j{1(ePA8>WZKULLX z86AYTIUc*HEPz6>P)F@;h#hw4*mRnf)ko;j=M;Il7Sczt}u0!9J8Ro|(;fzB_8C#&1UEB#> zpq15+V^=9@$qDC)U{dVs8cjhTgfW6#gc&Aw`6JmuGtco-7Gozi``g>!J@o7w4|gI4 zd?0|gGTVv*mnf2Da?Qa}OsI-oZAf72!#!7t9<>H_b>kGBIhrpmI)#3fn8;TnE&2>S zDVZsYGKz+L={p#kjDcJ#G}rKhjw|BB?|*cVGkDvI!T)6zcESH+K*jj)tudpbp@bv= z{|$QW-`!ZN>RddHMxnpv6iSCziO7LnC;YEmVQ=$ zq=+1Ln#o+FgB(56F_z3UO%iG_?+A+4LDO;Z^oIKt=UKecECU$#?Z6B@%yD zAGptF2$+Mkr>l@Stz(KFc`l(?CBAV#@>bH-ypocM3h1mr$Q0)yq+aG04OWOV5g;^3 zB#@a1Fc+>^Z)9{HV^AqU_S%^nhh&s5gYZ{P-`N$JJW;$H^mwRYh6@p@m**Quu3!i96yc;AgT&g!^omv8<0h~yMji(DYv61V;-0JkzF70zJA%$MKfoq5|CaJnFLm3Q~vkuN&!ms%it zBEn_n=8{~wlZYPghIMa*-xz-mCvQz;BWvwyXdw7$M^`Z-Cf>SkS+cY3R-xhr7mx7L znPf9XV^U8L3W78@4Og#v_LE5kmAR2aPmJ3aX;Ns+v*XFf_TUH_V*n}Oe5%aoT&SMy6)bNbPi7cs2YGfs*YB=$YUt5M(fYl3My^Ag1#h{H#v9gUmzrplfR%Mjo7I4bnB?&WQ zkvKhFo-WuSN{^P!BAfZ5Ic4ctD*znJUWgS5+L=Ke!pmJ9cl~Gb4qfNJTo-(?8|?lG z*xjQ^zL2%l#{_`$82|cVR8F2$McG?@ceYkfLYcSySO!ae*F{+)10x( zA$e=K=4O}Ab;0xdtOt5;>}GfFhL>IvPkJFt@#z5BCm4d4QxXrWWC>7amg2Fo!t7B} zfHkE%gGM}*VBY;8-DPmHU9~EK;WKv05*yy!Xw}Nar|M)*%XUt7A*0|xlbKt@>|h{1 zPT81+DlB(Jv4nbl*k8UX8R#Olqgt6&YO>lQnMEfv#fZ%fqM2d=g_(lvisGZ4pDQ%D zoj>2a(qjAsd1vMjvoZcjgPnVf%Q%T4EN_2C$iVnpB7jzsS=#vYI8afUevAiDr?C)n zty`I*|5(X8KVffxpO0o<*|4CT5_&QPk+DcpghS*N()qpe@|*jW<~336SdjMNA>%~% z;%L~l#zbm>HxZ1ldO*e&PS|SS9CMfY;gzW#vt#5>(_HA;>;qPJ3*W7wf-bOV0z=eQX0_f(ok}DHR+k zK5F^ZKsiUmmqjlA{+~1)$7MB1NfDLBKUi&9N2jn2DlZN@{m|7$e&CLz5pl)NNIpuV zU-ad3D77j%xx^!;Ni%oMq#V1%gKT!m-w(u$P3DVaOX+|1zZ?V9;g*j@TrSU0|4Cv$ z)jzL)DRa<4F&WbDI`6K=8en`$T!~}$kfD=AcMRiSt3PCbT8!4F2eT=rzVsbXjw}&O z9La-lzNiEuEaMY%8ndcTY(oA5q%o#OZ{)O_mEedeAt1lrubJPo@{K?QC#8~1=Tn=Q z96${`KECUZRHmQC5J_=f-lzl4gk=gUuJ`S4D%xYFA$M^OElfSpXx;kJ%=bAn!bnTt z@A^135z(QKDO4t>XEGL{vNWjfLomM$+Ir5>MObacw1whON^8o@Wtwv8w4SzD&sn&m z_^9Dm<4ZRdYy)1 zPJF&R&y@iwGpXt^UAalFskCYm3Q}v@_NY3dj}0dJPFxakgw!odX!zTZIx$PVLcQnM zBU*aOlBG+5>R75cbUAI=T;X#(J-!E^Rg-z|~+GR9Uq$9Ufb-n_q zGbU%U!c&)Hp;=#H2K~o<&Nk#GS+(oztVed~LG0qy;sfZ5K}Lz0V*dLPv+ntGSRj+O zbVA|7Df3&@Pi~3NoKXhx&pX+W(DX&fkQU=A#nWchM&XsRNGb}QJtb$22wX^8rSAlX zK@>|}%UfTx_pNIeiEEB%rt@Hi?dyAOKYzXEbGptW2qJ?fv{;mZ;vji!SK(0vI{<8{ za{jaj^uAMg;jw6QPN@1dN9}F%aN-iZcbyT7`bxt%P|cNwOIJ{jsrcWr-`Fs*wgYEJ zbsHBztrN(Vz)8>hXMELW@6F~;bWLm^sVKN)Nf#Ya(ZoDu!qQ7s$*_}#%-Q54O^}_s$E;ER^jUwO8O_ z%U(I;Nm_;~Eo<EjgOVX?iDqt24BcdibfJ{^{Ye& zlQ!7w1>nvD`x(H214`li7T?FT*U9J1 zTeavd!}$Ws18fgGeCETBPSsH@oW|ys4A^OKUyVw0m1F(^lO`+SndL#y>Cy*v6iT~&6WJ}1fRHk!3l%j%eQ7a>--v^7Jmj1876 zBHdcuvN}&`oUU#YMm+2-jUcIb7uUDh8f%w@xMEnXb)X_l9bR(}LGR>WQu$VCeGckL zChMqq8ACWPry~|GL2Xptr5|k;A=j`2+N}0Q4+DX>gw`J!gFh#jbbAX(s$|p;IA3vj zLyf#dSD6ETAToJzW}9+Z7T9Ou7~NWA=7?oaQR@|ft^RF}-7wK@bT?(|Q_StQXHQ)H zY4+X}?qp+W0d!~XXm60QWdmxC-Y9RDv2H=H_T6aSFk^rE*Bq$>nXvW1cIWP&_p#W0 z$=^g{@j!jq$Jhe!2JVRlz}R`geUZo5g79YUY4j;3x1h=90-dm@_PNKJ_Bmov?h3}| z3}khGp>e1VV2)u8ki^pc!aHzBLLZCox64RU>wm=N1NBZm0=cWf-UIce9*YCw%QZs! z1a3sE4YbPcg>B`~1!iN{1?0lU1KOKyfU8Dge4jKRckouRU*_=Xe8cIXDJCRh-gOokux)VqN{$~{QH`-D8*LC^4MfIzurxN^T)&($`69 z#Sh>=A=p(z_nqEfHLUut8uq`{QU8A+n25WPiM_Lho$Y_~#XCw^D*qzI>L(h#fgx*F zrpAb{)Z4&cs^oL!V+y9P^^SQHg z#GdQvB2Kopa^9Za=JC4W=-YemWb^xc-n#u|>Yx`A*sPAK-0d$NUW`TH9WDZ*0U|}o zD<~_%mx^+%QmzhRsM5`+I#G^a%+i1yGSfM46MkX}-==KMd09BWjf$N|{+4V|T%ZM7 zXML_ZL~nJq0Rq}OOrOqb1(UV*8%EdS(#IZm6n>FrsyQ-m%_Vdq+ag7&dsfQJ)3R>Z zMwel#B2l9zvqkxI+%~r0CsKjww?}A&xgD(%7IZ*!hY;})M^Win){4#FO37`t|~2KhIZ4(#UK;iG$LWKXoG%pWaljmYEVDt zOV*Zx>aC2~d9fm;o>-r)r3~XhqR4z3P6ztzNKet)+645;ZpJFd{zS9SfseV8#SH%VPe zMCSYWrqSCSrZ|)>A%dt`o%1vgfJMO358kF6d$V-CsExS>V$QrVj?_N%TD?eM!#)0H*Pyxy4{KQQ4#M*-08-!ZDg z>6Gx{i3A|jgZtejjw&<#-xl|x{a;r}-|&77JO~Kw{|CJPpVK1H)D8H*(0kkeK<|B0 zz}aCUm653pWvo=7VG@3O(@3F2D8*q>3em2$Odcgv+PMsvaY%3LtgpQl?@Ra!j?{EF zmR&YFpFx?wC0;xnZph{Rm}62;Oz-l%<~!y(<~`nUdVcQl`GDxv*L~`b4nm@cLz|`f zJse8^_ihB&%X}*E+h~yUbO2PIZC{S|Zp0GBo~;nVKGJO@(A9e-=D9WKFXsG*+$|B< zkx)ylVhX3Kx~ROLIdyslp@RG%1UPC)dh{rS)>xgSd_0{NOFhca=rIBxGZ&E#YKUDO zoVu!?4$}kK4{u!7gD4|3VlN8e1R1L(c2gNPU;-u>+pHHY-^MAe#7SLu#5{krR!$V4qKCL|Nbb zE0Y?kIy+RiKusjZw+nMJ|}B^N}!R~hDH!qqXn5ZUJsK7>1l zBd%_2onLxL9QhZGQ=(p6a6UFAVsV7ibplnTQ941Q0x$O5g4vH8e!(3GO~`O*e;r2`wFq2N+t;T`W92Z|8wuKcnv4< z&3JXznGX>~Bh#!EOCrr0fv6ydp>Z?lhtkBj&G&o4S)|@2xiUUKf+ZVns}VW)JMtph zU-}~3f2^6}fMO8#qTH|1s(Rau%Lm$QOPh=k$8y>rTUS;Ke#u8%zR-E7iPuL|S)p~l`8MyzI4xbXP0IMY#ZU~{=QewjL%~g*{ z6%7iZlboamq?N}Z@L=s8jn1XzB)}e7!MTafJPNp%iMQscoB&Kx=oi4Z`43P>ays1c zXxNCgHP*r-+9T7*zjxib=WoQ|nBjAMywu5-D3aZiPRzJKP5f+)+Dk z_938GIQ|;Y=Yr@lLA9uV^GI4kHwcA` zE)p2!EaVz}$Tp7Jz^=<@o{)n}hT_|)J&s=Df%_Sd#KAg>Jq?TZ|5d^GlJ(AN*F z{4&nO09VBHT!rf}cOc#;poPdxA+}f&Jo*w)dr75*MUGMG6#0e`8E>r4N8t4Hm`^Qf zj?geD!&n7w>5>3cBBvDzt=%B~^0>imMC25|Ky=WFyHLqbHSN=om0JpKv8b@HPyYiE zRy-6G&oFY1v2JmnWYnfqvn($pic3Upb_$O=YM~qi7;f2nCTW#6GsFbF&+}K9tD$E3 zWm{fPIG#;``M=JnJSr`}HZ&R90iVw(LIW0+z0s(AURk{;P?Y-d-)S6cj!965#MvxZ zc6UF&ov0q&!v*n2-aG$0dBGB!f=cEaYiC0M0pb52PIss8rL2*;sfCf9iH#|MLDbpS z)?M=Zs26euS||eST`WyZ|F6uWqI8rB=pDC5?!+a&9`Jiigc?J@Gq zT;J|55I>kS%V5q8seY56SLC|{{zvcqs8%t8ynLPVzv@obvjio?-9EJ9rU9=3rg~98 zQ?JOjb6Hq#LP%6@8Up(UQ5Gf(LI93S^>5<)z;omw^SeO;F>5#RiIb|-7X+>njL)UN;!eWGua+e z7@(to;x>a!GOzUSuET@nSn3-P52e- zp1Vt+&Is1+8HFKkK29@oUeZQJ^#(V4$>G( z{qUG5FR(jwC?-z-V>48|01V!iH>p z!#r%_=dJfoFbOF%#JE<(Koqi&m;^UpKT(_z66q*IS(z7d0_nj&r2JUnZF44anA(-3 zOc8&+3sp|lBy-2i6ymh9jxSiz*cvZZ$BgwyOVg;qf1MZib^0>DzWGIq$RHqm|3jHt z7y%e0jQ|SHKnLe<(B9V85(qQ}DB3##zl+q^)XB;I|Ekh#jaOeZb9^82)eBkndB_yO z)j9S^V!3*vKt4|Pu$j2glwNUmJhVKN7W?jId zA*L%Hmp-9Bi@pMvjyG3nWte9cyb@fu?%%z0t{=PIAA1<|Kpmj;BMsrkFI#c-!L60T z4KYed@N(iOSD^`d191rEZXBStLZAP*kp>_p9d6Cz@aOFYqErrq9vpc;ESxKTqln7?r@KgLBNy>SEI9&Azd(ixb?`5vq~P;7=-sJlz|-eK3O?rLr2 zF+}zK0kT4Fp{6=p@(@*gU7_efAz7V@Gh9}ydxFKBR2zn=v)e*|t zDx6puLmy1N>^Lo`AYwR(lLP>3=2<;we1u^KHA=6y*9!%I`3YcMci_T#+mn$?}n-?E;qVvqCUq~>2=PneQ$es^~ z(>3xW!S>Zj&ARQPx?>jh>i+&q*K4pUz=-NW|)_t*}x;q<;r0L zRx_$w%{D9ORN$8%`^@6SC$i`~3r3^Q|GhdMC)FPIgSHrXvXk(}B4f3)BxoShO*_Jx zLiXTYxH!!%nwD>xqR~)&&hq@plw{MTL?yOYs?avnH4&8(3O$9pK_Fn>v{cq~<{cH_ zFyUo!wd&ETC@zg4tZl4aC`FwPXtR zq*fuKQ+?#m3s|6syCYfK zKc%5u;0Ly^kqNkHazGOhuz~)~x)$T7*Ijnp=Td)bZM-tT#^L@9>0wC=gmsT`++7*o zBAU1Uxyp7S;}}dePejj#XTHgek_*T#xS*yWZT6*HdGpmr(r~Bcb5}m-ga}A6E}E0G z0F6Rd)ja{?KMz~?hZ4$ipX92XW*H5H+@bT`dm-^o(? zB#jwoHRwmH+|>IFttMD|tO8br!Khc3SlU-K17*xTQsv`QXrv)jFu+?nHRqaW9FVtb zwWwh-%e$!rhv!fqgHkyZqFX8*@ayl2FFH9Ba}wf7z%eh*pq$HpAuA8r4zG_FFDN z`~RtxDbcO@pv0@=wf)DEp}q^3#-ezARFeAaUW^sZ*%Aol0XpI5ZLZT z9>IE~(zVaeac!_&@~X{u4?8X@ni7rn)zCXf7wcAVVbf}C$S-QBF46*)5WVGdmz!j( z{zKK1V|TXYuMv|iw5}F&Mf%4vC(5xZm{3U)zsuDfX4m1R{F(+)S^;fSVOc zH(BT1D{AD&jtTy~OTCq#T4>+phag2)(iKphrarWu6F>kb!7=`Md4DoC-0yTE*F3Uj z4ByaA7rz)!G0PV#>e9!%&<1cWf`#0x;DXnUDgS8_@KUzGc?Zut@KvGV0|m1l*m1FL{mxYWra->3QoU; zmgWq7zXjLjo%mW~*fWxLM@~Ii^}RK4Jq?9QE49ojN-Au;P)Zx!!y;3t<&zh%NUF!b z?t}5z#k;@j(hXM>27Pw4*Yqezp)GqPcNZ>y{D@P#RrsCx>x#`S8v1#*ki-oqKyp1y zRb2GMIlxwfsjz>qreLoqLWLTZIsy;ut51*G=FqtwkzotcC;PKR(~`&>+vz!x#rN~S zu{Q-97gB5?K|nITI}6_bp|b!On_1eJGAN6PfB(oCIVhW&I2)TPn;P2#Oq5NH{&)Ld ztf8fXrh)c(Mav3bg{8RP4Nl5gRz;2jAuO5{miIGcM_AIvj9z47hR@|Wuh5KzeDDHJ zidyE7jbS)K?MH%U^0&n2^@kw>L{UsA$QqQ;bh zjjw8Cq=>zQf|jN#8aq+BBW(^|=KT7CS-GC5rA_VEMQi&dY}h`ikR>^|Flq{&a7dhqZ+XFK5754iGA&$#})6wLpM(*Ty-H~6xsrtQWoe#@5#M62 z-jRc_ch@?wg;<7APxzxdL*YzUoGEp=FbaWvJpbpn_}s!Ke5t?hnuqpy>)$@*$dnzY z;(BrYq5=o%z)52O5ezXCjqwQCc6e_}sOU5OQtT38yl6^7fNa@HxE^(mcwz)ynMPf1 z$2yAIY!p580Pg(NPw$aXDAhVHfO<}_lD~24?AcJ86XZi;T}o5DmvjlDnu@< z=)@OdoN?I;d@jo)gQ{0JE_F%U3#5aZJ2v0f=}8Mc3@}MLS|$v4s-Eou-qBb&qplL0f1D0L7{}_6_f!Fe|{%CrQZb1CdZb_GQ=LsE?+z zF9mexhIq%}2jhe*NENF*b_F^DiMh_8Soh28BmV&q!h5L-M$$<9PTxXbMgTg_agaNHVZy8vqw>Z6miXLHfeS%XN zIcA>vjt;H4dSxGNn&Hk8Z=!aEepVihKu8DV%6?wdR!!fztYu(SX3r`^N$Xuv??uN zE?@8!pc{h1M*_JjG@LijwSugYx|#Y ztpDe0>_6vaaeIKR5%7Pd+IFd1E1(LY@y!dTE{P$d@8dREQ7uAj$3_@Jh$;Q_@>WVT z;%SUr&vtgf*o}P=kVQikz!G>bjB;&8XHut&TPbRAJ$9UGbG^!5t=Z=HhIBxDt{>ei zu%thX8#u(X(Pf%cgIlK6ot~9k?f?E@_$B7&d*&q!axCWQq|)HuwdMhWWf9YblA`d% zTIE)VNNqZp^4S{2@Y(Gxv{}pD`%>oJ%$Kjk+qz!ahQQ6pyFz#kAFX4+q_>t+QitTc z&L4ss2YppzjlVpw_`v0XE_fH!3035tXnZ-C7LDNc+P9im|T& z3VdJqY)IeeL#r2(oJ&Au zRR+K@%K@N5Qw`lj5jfgUD)JPs0oNLXF2u$iQ{1=a!`Y!{AU^2Xjc*?qO1d6Q_F3*g)F~~go)lPM^N?OS;mlI?Wcb=;iPQO4PrQKtrAtkU~0>ca|B0JX?z9Vrk z?=!vaar`&u9gChB<#|$s2wep(x$as9OQRfHJ_og#TleePDW^GA>>Pv&1U5kkaTzn- z*9dm}!ryR2;xM9ic_-j)$=kb}%a4a5)xsS5HzY$2JrvK;_6W4OhYOk*FA$m({$FYz zVGleFXrAShGom{%kmXIN9Z2uBghw&?fAfbRkHp}}b-ZF#EfMq}e`SLlSd7Y^^i6=9 zlyTKT{H_$?%mOiY1edVm(i@uB3sJ%L$!S}6rzLE={@0RCq>2(l{q8(p zkpBO6p8xlf{m(^Xm&U5fx;WNX6%%w6^dRYvP+eA52p&zYjKYvPKmTEvAFwb8Hk}HGF2j}DsTo464!HrX4!$AQ-R{%6Qg>wmkXC{L@T$1wo`g8aacJrVpM7N+w_u1d_{Ct38obZFx~IWSE`bfp?&J z!)HLPRE}@5x0+ei88>D3kx{kLRhAJv(M8+5jF6euyEQIzDSN_;MZ|4%N!3Z%8UE8C z3*LhT(oulNghxwk^3PgS^3Bp<$-EV zBzRkUYU8Bv)Lk)?==!pQ%6xOS2J8AbrIb~aO`lR5wjNbJy@4TTJ=f0=99t<2t=!*b zxX-G#DM>Uqs=13@OnMF34@Jgqy4lvn3uEQ(5}Jj&>YJ+VF7 zbPJBO-~JR7t$vFRFwEz!G#<*TNZFHVKa+>^drWyP_mJ5|mVxEArLdkKd z4vA9)uf~S*^Y?qKQVyhXV{H54OUhjVB6y!KNK7i!2`2ak2WL(k<%ooiRUCKF#Q}{z z!na6)XO!OeIae{K>G~r|$Eljmk8!CL9c&G0zq+@%Lq3c@1^(dSw7|ubAi=ufQz!cD z@lG}N7*LQ#>h!pAL0xl@LZL)w$`X_{KTf6LX8Di z5dYC)vN4P&^83A1VVGAx{-z>o`G#9aaTwPv(SdTGu9I#bf2%X{*`zZTiPAkqK=hx7 zI9*x}ryw)`pu|u9lv@~n(=hwX7pa%ZqQ+=MyyuH!RjOtU%C#7$HwG+u`&j+{ROasf zWq9-EaHEjX5UtqV9f2Yq>RwQ}gXD!#z?C6qZ9p9w8R!Vr0Zi~8_D0*1Y9+)XYH4`N z@}?ST{rwzcO|2fqkn&o-0NRlAFF=mA0jHJJMbYKoH@mfM{Rp38m!w`pE?X>|`j4mD zVf0Hf)1*%L1(e4#tYUWe`<31>w#6#xTo&EfvwnYN*;Lu99n9Jdn>7br0ns6_alNDC znyOF>K9F%9@6wxm-}ZkX%!A`HU4{w_U;(ri`q6ZKBQ6mf8ToQ7+OChMl#bZk^*LI7 zEV9&b*ec6z4%IUZV^vKL>r@kusxl-L`Zfk5)(!Fc`MBxbLd25c1DT^9@XzdF@6<^) z@}bOqGQE<`hb6;fgp$~I%MQLGvZYj_PzU@ z!rlluf}Zg&++EY43)la!{dvmlUlE(0m6)2scCKd~Oea7qI7eXn9HAb2se&9cRJlnw zuW!qx6|#L={5p-hMG*Pgx(PuVqeFw6qfCZA}@(DM$s8L47HM6)*id55}zl4 zSso=;fl)_6v?(#K%1TUE@G7y}KCqWet+FnWBblHER`l4w>YHQJ8kX29`h>7=8D)rE znAXcFtBbOvPZA--x`P#yNM#~9Tj&K0pgfg_hf_$11sz=e#cw|P(Dl&B7g;L#B=tZd zbPa)M^ z!h6Nhn&PsFYWBB13d?RG>8h3YV5{9>na{4ZhP(P{Z4T4U+dVkMz+DrwY6bN z+O@5mQTAGV6Mma$n;Xl-`BKQVN=b|Kq_Y%)qkx{bSjV|fN0{Y9Ya@6^OpHI(}Y z0JUgwCbgu#bPFdohZdWpJzq=RA7xtB)I}lC2V>Bdww@m=#B5;4t(+u;=Dm10uIv3T zYJNO*7~aQ;q5tko#Rd4rU`D{dYjfn}GZ4A*jSN`s*d|tX8c278RE1z%n9ZN})ukV; zxDJ7Ud!*# zMGP(QSJv}J;d$}aei<)9!HJ00Gf7$s)NLdtGGtaacIxVs{yTi_!2`Ic4XvRraSWWs zazxfgvy^Y6h-PU=SsXfmOW>;l%JwLrBDpsDr{R}Fuf_gRT^dq_u4|g5NgJzNjpt=0 z`EVupLPNO%S|gM3ruKM3tf9`_u2&Jf(cniCSqpErJsUR(u*`mzPX{6vr=Vz$5J8kE z?5@Q07aoP&KwI{gsTG>e7*V2mo@P=ODRU>>eZE1WTZsl+l?f>2%cBbfHM%Ya@4`k< zsw~bI(DBrLOS>b6gsQ?lY}Pl6r8cG!3MCLSu+~LOWNbmU-y0CJgktRU-=hLnWecB~ zwCQF?E$}keLEm{^KOnkBPK&LUv1%xeQk}+rJyMLKAAN?>{TKOy0g8b?SFoUaP&&eZQZ6{zw#zK;Pt`FC-Pai zgmkM=Kf8U%`~F&z$$tRd2NLo?$nr|KR5u2XJ*WRw;jAW?Em6s|C5MRUb?@K2H8{D? zB^r~Reuog2HOEwv9oG7*71r`!Tf`vVybPXil@cNM|J))vnHoC-EP?I}k|t(O|3OKx z2UvO-{Rdh0zj6~MwPAF17Z(N0)6zNM8At^snYB=uh>)?OBjN`sei3}5kBPH8 ziN7jhC7_}!WSyGfoR=cuqDh`Qo|T_EVv;AA*jKjX?Koi8r$=SYIoXkt|8}xu%h@}? z;}>0;KtEH2I_7$1%%Q%Q|0WgfpURIz5VK-HtDWp%0~i2vM7kFy9E9|#BqkB*%x98q zc*VD~^3umYDbVjt++Cv-wS*@DcXhgF2dlgzV2Ge9JVGlps8%_DmFQK26KQ54 z29DsG1nJ=p>!j%^V2=_H0G+(*6Q=RQZ2h_^!%a90OD`r+%Uq|Vy*nQIdpqAI`ok07 z`P8^`Ze%qH6IA_^$cB92&2b*U*>`A=$R`dh?$gX0m*Oj^uuuUZo^2EsjS$2{#_aQ>L(WY!I7_ru zHI+IlD$QjHR1I&yX)S`5>3w_W<!Zz62U=a z@OGQ#sJvR<UL#uPn$V~kTB;O-{0BI@^> zEeFe3ODdIcew}N{brA31<#RL9(ij9`D!b}d4QF#WN&DLN(X&7P*U`p9Pts-Wj4(1| zAZ+0f>TNI$G9(Tkk#Su;fveA#Ae$eZ1l{=7hUUH{R1b;Clz4pYsm>zUo4S2z^pBNk zZbSnwGZ`n6I7BCx=o)p$PaF#>a;Lm42SaV{yuP2b5)cRqPT7uTgSf02n149q>M#nU zwrrOSe<6$G_W`#ZhCCcz&+%ycVfV~-fHWMuqms5pEEs6;_>A*31BUjLe3+gAMm2!> zKpJ?`#9siw?F}Q#CJ(9mhwtuP_E*N!Phwyb2O9sVmzCTPgCf1Vy|V7!w7WOf$gxp46z7w>YB>5-xG^}q$ zABQ;rF2G+>BDseQ*(UaqMHzV>`AEce>uKhfk$#segDTyluuNaLH~z4;Hsej)LE5nh z#pAh%Q6YZ8YxD53mDji8GYTwupf-Xpes%4~1;>y38?@PNF%7bE+&X~OpA(^t3@Sdf z(=h7cN~|inPmGO{!AqH&1@QiRFce>>0*Df~jHvGQWfL1Di)$AJ;6SSEIoQ>NIE8~g zI(uG)-$n3c1jVWesS%E@kqjZhhU^RxvDu-;6XcMs@ED7&fgv9OIkP2#xPk<0xBfmZ zYq!!qm28XeWII7h9l|O-bFEg{wJ~C`2%oW}N6K8%N~mR}+Xe3c=1ogT3b;qgZ*JN# zj8c2`tnP%;>bwklkKs9PGy+BU10T@+a*j@gQ+jTY z!*(`>&RgUREiw~1I%THS_8olKUghTSE#KA4O9F zN4XXsNmByJi!S#B*|`{J<{wd0)3A?t0yX#3?vcYiaa4>4shwiSm@i`PUecvHFw$P( z21~E#pULtJ^E73IA$n{=bRt#hao7o}va)S_fa*J;Z2|+WTDCd6mjRe;AlLB3Z3y54 zC0AkJ;QRyklWu+#@pkEVxYdwPkW92M#nrth8V+~QxUb-3{ypIhjY0fUkUHV zREK0SuX}oePFCQ-Qep%^JCHztSOZbID(mD`$5OsKZE{#-MP-Wcz4whncK%d_KVtdRJ?@(O_=bvA%kNV=6`~p7Gpw_~j;EJV1gg7fFv)+&~jHPc3RDLO$ zU6s<}G=^5Jq?y5&N*a$}7ziMC{go=CTNt5Kv+;X=Z&w#gz5aDJ}jQleI@7^RH$>~8&>cB-`_n|X~0bi*mfg6xVUE@>5v9ZuR zI!`pDk`w-GS~QLV`flpsa0vO}Q8*5ByL;jo3Vc*ejWN8&2O7j(aQ2Akw)3NJp|vL+ z9O)&;(~t#3;n&K=qo>!I3R#-dl3I#drIx-%Q_;_k>yu6DXxY^i|& zk59X}hk5yk(zdo~ucqiyTsbp3fpJcjz1!dl?SIhRhu_w_dLG3Wn$b6R&Fz99TP#%N zf}Y({MZsW;{!QXDS&l%Tx$N-G`B|fvAJvqnEy+Ku7*_E^!0H4&UJRiW$HdYQ`F|4{ zr$BawTaEBOUE)uR3hze;0a}D-nS5APt8!9H;LiOnf#+p> z76~N269Ue|n|Ey;iG0#4a+dBZjAIe6pT`F8AhH;=veR;AF7TfHSLcX(PXnYpPX8b<$!zR3e4L1^oqE9k|mVlS=*9gIuh- zOm8?YCAy4eVLzfS(cH80+B^>F=8_A)?Acv}ZFjfBQXzm~1p^tYJ zvVWy)nr1H{zv2(Zo3=`l64JOWw^;#l0i|k~&@t^y3RUCEK=w)>=W{(V(R8-bD!p^e z?Z(U@*@ZU4fw~?uX##Ob-mRt1OMLzsEah1cfDF4lJQGLy7I-ytU9llbBKHoGQLIKc z6t3aK+3VW-M7 z7!^bJ9H=>^$`~gTCji|V!HM`*maFCfyFI#!iR4ypM+$8}Da^SR!BaDe_K0e7WmH|% zY{PE?yg{j~qT-5JN{-gtkH{0!zshpujgphB!x#A3Qf2n@O|XN`Y4Zg~l@y_8Z25*1 z#jHEOZ7Kf}>u{{L#4pLkNKiI0EF|`s>-B@8?!n21(L3;2&|D3@Y$AvHQmug>Bn9-_ z-P9(|;Rv3@&V@E`66Ek-8m$*C%vACJjp|JiamojeF8f)P^uyoGMM?dZ&&^J; z05c{$X82n3BW9omzDN3_;zp;)M5s~q&Axaxvqkv{q=)IOC2^SDB<7ranch^aUz@3T z*8?Y8Fj`e;lF=W7vNjrH{*g)PutX98JoJIEPb*1WV3F|I%_)F zJWab80js+fMW3-EmY8Dj`*_OR9^FB<&9&bp_k98(hyMdXR%P|B=!%OX`7jLTj^e`ffIh?@ zL+eSUQh4Rfnm~(VJMWjitI~z%19ch24E&Q}fVYKaE+?c!BuF?bMIuwx??167$C5no z4K!VVeYj{&;#*t~X7cPlxb`hwsrSbAHx}+@f-bL(>=H(6|3&kFPl9lYDjqT7De*or zDF@r>N-CM9hggr{A37oF%?$(X`31|18)LLdtj$@!puJN+MG5*w=eQZGlzk;}3?*pW zcYyR`X}~co?H3{~&v)bsfm(0S&oC;i)QZ3BI1R`;0wxL>qnd-X^;%VNj^*epF=qvZ z9c3mQrjNY9S*A=2z=DnPEdQU#n4zl1NV8OJ9<_NqBJq^9ByyXAZ~~0@xOltom?Nfd zm+8uGu&B%ab&6BQH<-O8h1qNKlQyWaW6QyQ^HWt1`EWz}xE!>_1ZZPtU93N+BUlp= z5irKis#u8NZv;kB#2vN_a{l$h>qkP4mj6iaRaMP3#GO-kj5)ZBOh+#B9|HP`Q=1}> zTmnSrZ_lWN{L{xh^ug^Jgwktz-F>q2IyX4Cz8~G24xQaewNVwc$FvBufMsziOL=pF zIM0G#|4x3%m<=MF2MNc!X&K1Y?7cw&eGq=S@m*CG2}gxfi4Yly4Vk2&B=7o}J+Nj# zj6llLnI)+!at@d}uX+$MHhgrn_FyiPv=3{@N6L`U(MPkc^s-3fSi(lDDFqB;#4ecC zn9tInq+n;|s@&wnYQLV9(Obir35wp6G7tg%&I;}h3kD+Oh`Z@4Xn|TSvmf#qUs&G! zLa_>#I4owbS!Of(53isw*D_IS66ib4A#}F~$AX**jizD>j$yfh8O3D=eie!f49PQA z`@gHz?0O?2s9M=8cG4s#-LJ!OwpjyeWQFh6LsQk!b6tsG_IgB5G3Dn@#6I-TE=A5z zOD!?H5Q^xX`^3|Z!?JeUo<*&)1=!^^ z`Nuq);F*a8<&Fu&V7g7wLZQD7tD@Eu&9is%8PQMZ9jBCS$yW5X9MP_;dKqUn5@eL5 z`oJqsjf`)tV7}(0BXMUCXLMzy3O9+7NyK|XS+Rb63~7cA|mkvUj?|D_vHs{>|DIWBBnp(@_o zeqFKSn{XhC#ynkT*wrr1*^%~IpYsFS@;qE;kp9u(17uBJi#wV5UHN!@b;wU=)dzbb z?zOm^iBnt6Q2eL<2gH9csx-nA9bt_wni^@JveyF`41s{_qMk4YKuSHavg!h;P&x)> zlBP~X0l%X2KOD4QzJ)!;O-_nw{{lb5(-{H~zTp0`nno(;WTfRJQ~oiF)H3~1>?2Q8 z!TN^0`c&AJR6RRlpqwI}OS%8?tj{f6_|+=zBP|9q-1f^VE4Lp@K!h{u(p4pCu@u9k zdYZQ*a%xq!F1kps&QZJHhXWnuK|9&2TM(YD!Bh^|b51Fm#I+p2h#{kN^l}trwVn?y z6Y&Ppe+1r=m|`xJ{_9yOf;AyHoz7}7nEF!t#Opu|=dc&&xGuZ$zx#b^QKHLdep|Dl z;(~xk{ST{@{|(CihoR@Uc1+aN#?<`(1ZDqU5ZWP47+2I~^v@hxPZM_=<8)+HX&TDN zPQze9j9Q{0_CfG{EIbU6dB&9CIYam1DDmN`IdaPkpiDc%Y4<$4C7jfEdR7oH-?sfL z>@)1?p|R_EQgg{nBF1}scJ^;3$Mfd3&-2#vmiPUWuf8W(FaC>X9DPiHEHXdm;z~Ho zRhwp@m98pd`Hm7qECfpLzK|CdA&hVzD(w?#B;U|)h1-1z>l;>nj6s%fhWkjM#UMq% zN{|BNEtOZsHecz^>n|7|^04VWCm}Ba1H~IL2m{!i2_c(74bYd008Gt+wN$J+z zL|Y)<-e`GH7KfWOm1D`9T1}%R5{H8by5b^A9yN#Av&zek5N7ZOmlqj~fNOx7}@pBhBzg(D^E&6Gb2M;u+Z}jL$(`k9sbFgen>7VTjz#M&#TP zo==G}74ieDS^q%M(H{-1n$;FHj zt&igh-N|bHa&urViFG1m(i9oBNw(vLT(wtX?T}7aM4NgA@c?srt+izi`F8!%%a`|W zYR+Mti?Jgd9YgfLj0b>Dk+YR-Iy_jJ@fNRdQqg4}-C)4Lu^yP^QD`R4;4>s7fh5Xs zS<1RLa*~Otn#oAqpUETT@>A7vjo4Tk_8j8XGn?_^>LyeIH*9RFw$;anip~lBJ0`G) zQxK21Qp|7i3Wvlk*`|4#8t1R!9Sd6303qmrp zVrFB23=ONSq24C}KO28ftUXm6of4lz)y?!*Dyq!hw(bZdw=q?XGAyDtHcts5^NMrT zp&G}fY*@(C$$qnZW;=mK9hB$y0a_0{o1|j8e<)uG-lXRD0|PD9us66YShR-gDUKPV zS}Opoof*wPjDuv~Sx*p`)gSE{*^#+^onem3T`l^B38>d;n@UCfF)VRzWjbIJ@du}mbJvr_~vJT zFot$Lk82p_g6{u8**iv85{F&B-LY-kwr$%<$JR;5wr$(!*v^TaoERP3=p^0I&EUOv z-kCLb?)_SyYOShO^?&xW_iz7|Og`}^KwbNbWG2sgV-yC86Zt%!RpYzst!dIAQM<-t zYN$Y15P<)w=I)dw-7y4b^s8-i>QHJ-i!T9&K;PABQa?UjAmos?eEYhw2`O+2MpxIT z7$Sq8x8FOnW{$naeryPA&MLKM4(f334&m=Z{? zVR1Iv^+L~5s-D_aH;+0L%oP6#msEBPn|fhiVNALtQE^tftn?p^el986PA(_wCyw5_ zP-S^<71`+ksHjv5o%Pdl)z3pr9u;oNjx1fS=5CWZOFR7_64D)CEy*qg>2l4XN0ZhoN+pJ(+63-5=B z;-(vilG&Aom~GA|BS?okLf@|=zWjrbyP?wp@4`djfU9rL6j&u+xpUDU9w%P(JiF`X zG^w@r$;XALnRq-C((Ebx+|16D0;@{49`=lqr}S6TsO$ok?GR%@P;n_@KD61ojE*rn zLe@RAbPm-ojwW{ANds+5k=#4-iIsp_txh5mw#iPq*^!`ZTv#SDjpH0b3GGUKsT=nr zphR}??6VlxA5t7{c0NU{P;3y|?jub9Az1#XG(-Bmv;b8DceRbzzDmRdW)TBSS`WwF zqIsoq_&9gkKdmE|+(xRZ_*H6uoQDi#bkm*Yn_J$>9;?-dVBa$lIyB*8 zu{0-Y>%@7&os72nX^h|0fG9JR*np_J7A|Q#r%ntMQtZ`qMV?VNy!r8iHzJ)bNzi3VhYHp~lD3hWn7BSvYRz8( z%MJPYKZoXt$%$o7zli_YUs#m?o!R*RJ^}r2*V6xCHZIe!QwFKv_#@$8a78FcNRId% z?T0}@N~o6FbD~X3>0qJ@j~6lOH?}oj93Q1beSAaz`<=@6t(IyO2Z($tjl09%k|Wh8 zvnZ4SdhYR?<8^v{A6r%YhMlN6r`Q6{Zqt%c>O`4rMLU9(lY<`Smg_alv737AgYE(TVbCgYh4ZO)-c6A zp7u>_ZwfevesZpv2wR@6u6l*gGd)kr4Kgp!Od^iCGcv-w8Alh@V7NR8X>R3;(_4Kd zQM`>exCxRDP8IFy%G^j zm2})3zS=vwTzos3)fWit{K^!6gN{769NNhKSn4e!I-D69acYEb>syMlFVzJ}ILfLc z#_@6WYZ#cH=F%Uw@|*uLb<8Kq;4qxREq-^z%E*zmTt*9%Z-;vb+jk-L$rYMl@GEB6 zr0$ZTp=iYJ9ey466^q5Vy@p>qZhxe9^~KP>{8GX9eTGkRxJ28UEhw9!(iM}_^tdFs z)|+ga*4s`EhuXueLY-o6(1e!nyTg>Z-=RyBiWiX!(fxSBzVD0vdyPd`RZ9VX4+>0)1$9V$Xqt7Vn6ipA{^_C@cAVb4KlXIfdGO9_H z*2;t>d81av$@Io8k#>#EcQfp2`(815Rem;@hjGxVlm~s!Tx7! zhv8i_zv35k;+5>%H=h5^6yadv?DAiuX3YNuHz8*7)8c=1{(cP&o*K)TA5LxDSpsdL zj4(VjLy2Lk5(^N6i4riQ5-BJW^p6(DlwK@Z*j7{`5)O4OZVoleS}n`AH61Hu4kX3F z_)K*h8(kW8?x3!vl`hvJoqs#8mQ-025LB(LzK6e`ulb)DK6#&WeU2vrwqL&mdKHhx z@fqj`UfdFLr|)_5Ztadz_^!s?>`DFgPN7(*D{xG=YwrNiJEqjPcLSjNZbrY3N5Jj- zBO+}7-BIK_azwwsYvEUp7)3uEJG6mFh|d!a&+<>m<0cgZ?wH1GM5#;2HlAy7A%Hsg zgz>+syqE5Ktk^^u3Cc2#%5wL_*t^u8a@YQPYynb0Wy+8O*E4(`Di8M@O}!<2ylS&CWll%d9(_5YbQYcBY-?&XL8R&ZSsb*hshxT7)SRe z8ttpj)K3}FesmijE^O57MRjDv*|?{5M<{qM)IJWygp$yh~Zb z?J%v_h^YTeU&}!v0|&La zk$PhrU2qV|19w3#Xo1jDgVpx9k4e06IcInrX3hPXglp=m&6Xd7lUs`Zq%;j@_cDgY zQm!1Bk<8KGtW07C3wFH`=ikh4yIvF-E_~NJN(0OImp?kzcyn zI^LX=g`WMZ3A=7v3){9GauRtL9QySLiPs~ZGA|&nd1K4r-`VKm^q$l5t(6okA7B;4Fef$sbg#eg}R#C5@B$kQx~6zdIihw(*A*rVVD`d*yKkK z)Y6dhYTaIT2t5J~B}=9O$HJC+CmuKVM2^UC!y&x9wx;NX_h4~_h_>v|eIT`bi%*G+ zWQ4*vq=8zM;#8@vquxMl4lzO@t2l^~FSmNYwyae9B)oz@80d{Psqt|k7Y!~AS9~Al zGD=VMM=xfQDgz0_$1-awcrR(m)O~?{XbE+8uK=Q%`h={+rzE3|R}JbVD><7o5{jzb zVn^_FN^+<#t`$0zZq_(;+Qj{MY%LWP_1f=i&*Z$U;_;!Pfe9X7I^8r5jk6~k&jd{+ z;17*XpC3SI21T&APufbGv=k8!^H~4|Yg%xBDhJ#pyhl0i1^+Y9bmufzxm%3Lz0 z7Djf}tGJC?^;Mv;^jJ2P&WpkvN@-QHF_SDJ1VV21HgWZ%Eo_F}hoRV+FJ=q3D^uYgft6pW^tJ{rBF+f1_%@n%dbo=Qwb@OTZZZ== zEhxow%8ga~rPAm4L<2hlM6R(5V>pd2ZEM=c6ptn??+EV-h~?43##CdzneVu4S;^^R z59wnnHyJTVvo+L+4oltxbm>rFUF>^KbjcyfOK>vRk>4aQ$|)D1?SErKT;ils!OG0f z1q1HoJG!%kz*t&n?Rlb394gT`zsS@dPe04zrcC%**Ol*hp(>#s{VDIxtGG&i^lUyO z&?q_HM&TLF=os|X^J zyE>8%ume*uD1Nh_;TfOPtrHOEq}v-WsmD!x))J9ss3tg()~z~OT;ZCJ99dVQAm-(? zCl?>l@huhXXo0iUHe{P@04~@8+Q4t!B2AU6;av?mADzmqxZZ*?2iC92{-6Rd$tjRW;T1{B>3(!Gq;(JHasd9?RS`|J$_YU8rA7U^=btJO=>m%3D!5H&1sW+4B= z5=q9mfE|2F1^RFOH!=4@f{VY$2MjzX>GmNVImo&^QD*si43gl|+oU4oO2@f#@vX4y zwhwP>mL9UgO%QG5ufH&m9-slvCAf^a+PBOR^mRH_7|8_6M@$oq=@+`W(rNb6N`5xP zF0YB)150&6Oyz82^rPs6rnN&ha+26eV^-s)BB6rfIq|uq9{V`|> zF76|5RHCBlj8B8dSz#wAMB5>x!dZ$K2_W;&*?Lc*Z$#$k!o>FqSnH74%l>M;3ZZ=o zLZp~v{K9qp+nW)yvA-V6wXfZ}MALun)iF}2lZ*a2IoV#~N?*>EDjZqN=$v*VwxA>( za)0lT+5O9nK2KV*t@0jTZ_0~KA}Le)lp)+21% z%XP_xIUpSw#?yF5`tow>D6gI*otWDXNG&d4vnEVb7`Y^I36um=!dn5B#p$GnsQdvP zd5Vrm@8voqjA3mFmK_|qC3^;l1)y6m#1oqru#gz~0V8i;rGF8*`@_~Vkh%e~%t6R^t%s z!NaUx-z3ig6_@{~VDy{a4R{lOb z^MYSJaLDiW$Q1ld33UT_e)Ys6c0P$H{gB~(XBhvHOMRyh5Z6<4+BP*Vey}c})hV1% z;UjfUVX*WlhlzGj`OK|>qBs`vLUrbXZn8gL zkXP7GCmMixW|5gn>O`!$xKy-NjC9m_{)v^9<-9~>el@})#C)t?42Mk_d}8b3n*VFC zed*1i!uTEkKaXf16RLJ)Uy$m}uVs(a|ECe{e?)>%{MX=&q>GD_i-wDdncaUd0?V`< zl<}oe{IOVsX05=EIa8@JpwKocxD-L5p&YVac7;?nL5YjcM?GuM$1A)AMFscZWPik$ zW$H9IKS|ZV6v;V>Yo7GW?v?d4{Zu#B`9ox#znD>7>v`*)R+jmBy?+vv``(cTXlpM{ z$1d=s@WTB!E*Dv&8>t>m41f|agY`v(Y<$I;NC(y7c6@p)b$g5! z-uhz*mcUHVLe)Th4ZtW0AMf`Z>tP_?kTZV@uEi=5X3l4dN=H|M)b&oNBn@18U>YNOB#<8U1ktE+- zvlfQiK;zS6El=|5ChpX%(l26=5@UxSC50= z)1#QAN1bQ7{%|ncxuu876vBGOGepREt0tQ`HLgypvvmNOC0~DEOy6W7Us@mQ!c5~P zbzAHF`Xqv+&XdLgY&@ZlcxfiIi|GH?QC%OUunFNd#y;g6R{_5@Q5GfB`K$Wg21d8X zpcstb)|(|X>J9fgN!oR#0rIa^bK7e1CtXW zf+|w%EGwVG!AkKLOtYM(06^Bln!QDn)V&wK_qvwHWt;UK%A99E;n+K0j?0h^qvkwiQj0@loK{o=$A8gzD5%y3ue;3ge!yAS@qm$y%e^Z?r)oa0AWQEuVIG-D zX~P6qRmkN=+zQ2ql6-1V(8x)r*Cfx@DwC90*&^i=VF1C(;pnJ`AwyECl1IItR~KFf zn|8bfyyg~_P0qEBSeSg(RL+6!hNYA|(IF?!;%2vJ21r2T_}hQ{4Efc;F(fRnGt0O4j${rwRrWyAm%VJ)Of zKO$HzXSNX2dsp?I&Z!JlPI6CVy^(6wTr{`9=RpW5*hcI+enXk*YV`;*TogejRQoiF z(w5@e`QX{T`1AOc#6bz;=#D};PgQn7XW&JtiCK0k+yg*i6|#W(wJ@~;So0pYR0J!6_(yhqf9pF zNQj)JmE#P&1{Wy0b0VIHG4S{{x99r}XbB^w*;i(a0BJ^X^QwF}YbZfX+J$2GdpKwe ze22cyf^?-zRvW1CRtbAW1tI3Ofm`qW@eLc7)n92#>P%G;C~;vq8odC?b~ z8A)Kc$eSi&)Apz>w|02x#y9M&1MG93X|y`9wE+qGVIFFFE$Q1csZn)E!GfxAiO)wl zt=R1RGO`wQ-f~Gwm@yloUo9-Hj_|&X zB^F*{hjDAbZi&RLLuQ4FBXgmx|5jHJm+3D*0Jr|dNAw5mKM(+JL-;>`2%nC8;S#>| z-pBtJz4!mosubPqU2XnHp^DM%t^8+8^J9(pxl8wo(E2>wrPTiwOh`O%haCZjNaxJR-5;D zqG1MSM$G_rYMUw*W$RQPNh}@4pJ{pRYRK>jv6G8DxyGSHgU9N}kAZ5}kq2Wd^%s0! z5Ng-=xnrfUSGBRmObz{}!9?1aUOA051AyTkGhfdFN(+1Y`CrbM)y`hZTE-muYfnzQ zh*dvTM0*oKcmm2*Fiqs8rjr|I&IJk8*S}B6K7TCN$12KaQ{>HOg1PEZ>i$&u*ovjK z&X9+ha+j!wnOZxA-|vUm%KGGbU&%|}S=%>0)>1F2Xb6Vl8x=-wIpmPGKaOI#Q3|#v zjWUe%?Jir;=F)X$I9dJbP&60oLI4_a;Tp|mobV;6oE0;={{4c1PS)gWufAZriaGQ+ zdThh{>NCjuA^t@wu;N&dAimC)AvY_=&f>Uc72vrFV?`jkN;bLdcSvSeG}Btqds-C4 zXmMMC^hm=Z9JiBBQ)KC}ig$ZvgCoEye>PUz$jtsaumR>OFs4;z3bZ;`O`%zR%;W107+a&Kt+!e07@{`TVyuf80v{ypS= z867kCMc>(v4$vR+pz!66tgxsZd8Y9FF}nWpUHBB4@=ZpH9!chf?#Z<~AaW9pL zTo6N3-PT?w2B`)Bl#%=^*AhqcPlc;4yDQ1wq|$n4z1Qy`z3%-gZ}VR_xBdFK%6hB* zc8wIetqaYDUHP9wl&c)^SX_0*Y(HDFnuU%)*+uA0G+Qbl%iIW4c_jZ*4Kk%Bdk)p? z@z9?YDP#-jB@Mk8Y5tU|+I5psX^mzJ!SAwCN(xi<#@7H?MJ3-FUc!faRE&VUK#mIUCrd+dfwf9iSr%wv6L5Pi%@m=ZI@EQmhi5qIikv10>(H~KE!Gi9tgWu~Hl@p?4k_8O)GZv^I>eV+(lAuU zYZ5X9lf_4I+M_KP330KZ1#Go-1^Y8u)GtZ451cz0lQsd?aHe-3pTY(o*@n}|>YSxn zOp2EcRn@mnkOex)|-YFI6grUkQS*+`@e2lRSTm{*bcP=TP4^$ML~Y zBCoaKk+xP3CsT=WEH;S`Ls5!dcauSfTC+#6SH!1fbU4=w-*o01m_lE`s3O1|T>52U z+Gf^Vo*(I`sy<{$%HBRXuOfh^Yva*JbOgDJV0Adxf|atjWeQ&ZS3)v+_r&NFXZb!L zNb>~GG@Y7C*%|87s3!e2ne18q(E=lY=`2q}SmWL(rb7!2LUm8?YLT6)o*FE4g@Vn< z%4OczI=hICcjNA=2d84GC7q+a`j};K_?eQs3bz>NOv4OzDbCs|K|MRtLN|zvNDIwm z$F90j&|K2Qum`Rb}% z@Q$&8%b2csaoksVevGv|fL<>-R8KRVD^gC<*m9Mn2&QN!im#L#HC7YW>uL!Fw`fG3 z6K_Mv*FQXDlwlVdLE>P)8kzQ1KSKGYH|TU58|-+wpRhxBI^YGz^&pKsTOy)y%P%n1 zFb}Wj-8=E ziI1{`uXv2ioL*4rB;U4jP+>`K>u*?gUd}Q)nS}5Ak~!TA|3}eofoDGc{tvj1xK9=c zZle~zgSKl#Lo}nr+v4EQeM-nr;1pauNQn+i!GzLN%cLw;b>#fSf!eS)Wxby?PwV*( zt&oNIL7t(IB?^+Z9TUrxjk05C5+>gGn8u!oK0_yL=(tfXOtaDUT&pfc+TLvJPs@<4 z4kHka$N+UaveYc(@Bs#M;8KPy8RJSf6uNTgn7w}pE0F_BYYp~NB8<^I#%1GZl_5NZ z>I#%4V=ZnQ9Gz0t=^o0jk&IE#9>+Ji$R}Ucg`u0FFp9;V%PT%6y_K&S-ds?+>$W!F zJXo93YKX7V4Eu8<*Fs0|RVj6EN!w~;sXRN3!3~m8zFm}hhU$O`e+65g^2On%&+5#7 zx?ipGHQyT(t1^!{(1Rbq7fIv9ls5^#RKCG(U0U}QlO3}P(bm`1o3qn`T;2=T_cQvH z^3#pBtR=4R`}Vvbcz0`N7PNyhYwqPyR3r!_G}JRkIgvdCVJu*>X0rbhe7@1eEpJsAlsV zwX|KeG5<~v@(&O@HXY#ax41fogunM3Pb{Q#oZv}`sh?RVTlaZG^>PA;hb;Gya2sS^ z#q+`JGL{#V^%xT8lf`?=L8jD4-&YMOIbUoGwy*!Pks(!Y**O$TEt;C+F~?kv&o4#V zQg)EYR)z0-=(k8btREe7(I$_ctH7BA{9TH zGu)BlpJRCNaJG7c=QK!A+`Stn(IUZ@b4V7mk;RhLw|E-#Vf-7NfonhYW zuZKKCed}emxeJ>=Xa=?yxMiq2S5BS~eGvp95X0u}DiV!cYtcF+?;IXk)RFxtr5<`H zX~$3kqF`Yp*oyFIC7nDR-Y$unTr#lq5*@=~DyHNt2&Us$0YT1iGm740rm^r+Q(e<{ zP(T*f#@PmZADBZ^==?bf>rpl5(9wz9%tedRJ~92+QB|3=K|rXSGsZn?Dq+~-Hb*pX zN#^Dydd*SivSq+p0KyctN+bJa&OlF|S!Hc4o}}BFa)RnoY>_+01LS7#7G9W*}UYK%G8e%9baq=B~ECh{n3U`S=N9>o^zb*P=n6UIclngREgXzhNH;Bvb82v z2I4VVgqb2|YXW+lCo{&1^DGthV;z%q)ouocyM;-#OBiUN1(w_G3u`tKk*N63!U;Y# z9%DjdjJSBN_+icGLQ_o(bZu{fB(+N&P294a5GJ6+Fr?E@);5PoMyq+Rc= z1pHf)-_aF!?gAk>w*-!(@Se~{LyXf2`s==1v)RtjcBB4nh|oI1*QBx@jX2kqvRm|E zr|&dlZWt0!m~r3M8Kniq&$7s>W9KvWT#C&Fg4|?Mgh-QpanDl2BxX8fH4c zw9$r4+rGa>0wZAf;~pJD3Le)jNK+o!_$KhZcG<|WJ0#YF&5j%ts8@t$?l4A8qPk@ z2~x8{RT$Er%$@#@`|jXwTThPZ#R;#tdzlw!X3aEx8!|N8x6|2`wOp2xRoEnCZ!P|B3qDXjvcbCQ*F&p+P<=utY}0!2K(u;ap?rss+lKf>kO^ zLvt`JVlrbg4CYe1?(^b)cLZLwX`OUTW#I3h#vWY=Vw}9m++gP3l*p6?)c;TA*PJSo z5usnc2waM9-#Gq%i*Em&P^0VRYoP7+5zn!hVrkSf-e%7WARCL3N-jf8Hn)((oWr%#~Pq9WL}tg@Rh%EdRp5)+TNIP8c>40ClUrvdo>8 zV<(J9Cq&<|0T0ACLmX9w)jgko@2@N$-#;sR1K>&^ceF19R*s=j-{6fZf&EgqPA?vW zZxKPCOE&?yFHT`E#Ds4ux*t0**>?d@MoTxJ5smoFxE;foL8NG-E>&=5^%{1}7)f|l zagn5M8%y|TD`zq~3Y}s_>5}H1wImL@vURc{Tr~)~gz21%*5i7ElYJWAi<>uH@%)#1 zcj|5QEU#*`n^#JDS+RMj8qdUiiu3a~9cGOR_rvjbI{oZimfhVjHrOQB4<@xOn8G$= zN~4T(a(rfuu=H0OB`E-Ycc1nn#0|h0Az@3M>gKjV6R%D9LCPVhW&4C+?DHS%*MzypN`L_*prTxOuz0lJVm;TO?s9s zmGANAK1;J}jze-(L~3}OllH!BfPO;q`O03wl@w^jOU>t+EH?1?)r|AK@NK2!zP6r# zt*)JDMeiu{%fvLQtv_Mx-61B6a&aT!0^CV>o0@zjY}vD=(kFOi+~``Y9+2)cFC;;+ z{@Qse+Q-gz9@J{EP6>t7U9N(duuFVoFmC)MZIjtD>)3~{EE+70w`P1Q~`YwfLQt(M$-8FOX}I21(e*r+xCgTU(>xRd zAFo{S@3GlbMd9aeTd*-*+umNOeO^4poDg~!v@l8upAt5)zvu)I@?s>WNjI+=e+o6m za6)fePCs=$pqcx)($c#86>#U0Z*xd1y#>A8^*NlXV1fiI%;t0C0D~qTsJD~q{!BbxX9kFjx|jr{;3_&3CDGz zv}1(I^<12QSX?NtshVPI3aj(X<=fm^Ud9|qr7b;oA4^%Mc$g2qel)9uWwJ7z$=)Ni zT8!K5h223J;Vn}yBUedKbe9<=;siiTDJbmLMgel!l9s5Dm}2)zNwu?@M1OJJu@JEW zkgqxKw0dR|?G$X%6(x9MVnzP@3w+ zi?(t(EpU#0og=hIsqW|=`^)9_DM;#kl$J6XwDpNsn?R|C zO`EAX773*7nJzV#oE8O49yyETuQ8#z9p-E%2cfjRO$(`;hwkeCG#bEN60dtwS%xNS zrn4|>8HH3{snBJGH|;N5PmxMyIt6A&j^8%w*T29}=Hr315l4}<^`_#Qh}k^R-3mQW z^_=G8!~C~OgUg+avh`j@Co{k5UB$x{J3}hjf)iNzzDPdhTUERGSP&NEwAmQ zp{ve6P-8J8+*{R-_vxcRNck3Hy=tQof5qU?H;YL@mm9HbT{K;V#={P^*c>)GhVRzD zS@()M7gHgAD=KN0V~)79S{Nv?qMdchWV4adiW3l)D*9^7?aduvD+q*-Wf`X*--T~k zeUV?gr0EewGZubi^~izdI{OIUPlh|A;HWbW63`znSco|99{U3SVwr)%s@u}UFHiAnj9@=({RugRJ zi@v?#9qTU5t8P>-mZv|~@WE0?Y$)5c#2x=^h9d$X-S$H8kBBTdgfoDBHc#x<9vu z*MspU3n-=tV^VIW`Wb4e9N2`d6R;GRonNPA)tg?yxR`o8Gq*HuY}Yf@40JP0XGfwe z2IDN|mH2;ERCk)0KpUJ>GLYyCiKWV+ZK1)1yme!#7*KjVd0jEhS{^;i3cmUSQvT%C zl|}a9BX3YA$8rn!;nl+6$gU$lma*lhMYHP>#-^|J4B@CTmEx}B5WR#DZv}&J^&h8e z@J~=4sd9*Uj=>BL-hRsMVLnH>aCKdR8(5$*MeUVAIL6_&YVkQ#WG?X2O-inNYdAGZ z9>b*)86d#l7dO;`;bUM%fY%1~?9;~reF#b~1 z?)=r1G`^e~2MOcl1Uo8A&%V1k8-NaAiz;lg)g4Y~g+u)H8gk_8OgmPQ>rlh4x5g>; zI_a5kUP4ljjcN1K`RMc)3UVf zKkwR31H2R;`g?f(T|Dm^!7%g(5Fq~R*?smRed;&k$8W>F1tLLxJY%;gM$%7q)`JDbN;P-h zvn!%T!$#mv19Q$AQ-84D7&asBWVDu^6N~9MS6D+h`Ngk5*14k_B{onO z9xEKn&U=&PC_%rnN5WD3S68!}c6OUnsghpTJSO^1n!Bh;a?Ci7n<@R3>(q{(Q_pP0 z+V;{XhgGkSBBeXgv~huGT`O@7#YNlk<8>yWGbg9uiJW)|9iVq;cK;%z1#o^dldK z!#j{kvZK+kH`;9dI6V0H9Ynlp?lH~Xa2g|yD`q$n`n#8=$YVFpK20S#wg%-wrqUm) z-q;WH2`dA9deSw}jvGf8O7lJqXwg3D9F5?4sEK7FmLeD0zgd;xL;Vnp_V~+l(Zt^` zw0jX4^yBc2If?&AXf@QmrSlc!^2pD%B`oI#ac9(3WzuxQox%fcr9CM^I+_6P+5K|c z9&FG*qS3o^n@bID*mQSIlU*=(;$`w)02cSf-WDhTsui3tVxs`aCRrG=<$b(3#%z3} zD!2(+^B~#=U*--CR-I#bf`1I1F4pb<`Co09*KYMSr?V85Z`eHgA;l5eiCrmvOmiS9 z9PHxO9%GEPDz-MC)`Ok=#z|(+m~D4cD57_5`^V5aKNXohz^EG+4zjGv7ckfXuHO3j5(*D+`^s-C`XGVgZw8rDLI4{Fy`IEAC(eYw6OiZiZmeKFNc_XF zy0V&yP5n_^?g4slexd%U@t-Ddc&tQbh3_!_+s~2M_m_U)=lV>nvzquXh~WJ;y9l9* z%S{>1aX~w#pY1<}k-+<#_aiDhORq%-(3sz8W-YhhM{BzyzzcXNcrqUTHr({@lq&2@ zKD;{T@4TKkj~EdZB%~1(*dSmK7yJrKK!z36N}t~=L#qa&FWERF)!!G5(<2uwCV}y} zAF;hnJ#N2i{j*8~3xIN6GuAosIMv7zSG{ z17nK`{=f=rJW& zq)VDX7hjPfCSMf<;{BHfh_UA_aW;P@pr)Wuf| zVO92p;1@QKNc6=k9-qSZJGWfr5Vl_jEcef?p}Vg}dHKwc`pCT+wCYA}d;hqD3nO1( zjFX%W369pDKHBlV7V9PYKY#s1zp;q-j+uy?{XiTf7js|h0U>6o;Rh|ibz&jWQHLQJ zaFnNS3;-?eRgkP!uR`91dd142&Z&lAL8zZrlcutqEg5(xK&0_M_$8zCigJiRE8mlRgzTIg{ z#?3a>;147Pw2qeN0kCwS9(8yUe)P5xtwo^sr9~KyH2rwv|)A9 z-cS4CKNVsZm_GxTTprPJPS#UuSN3gJougez#dM?W3pHJrZ}7kC-yDqyU{hU8|Dt7 zsZ!_}yoC!{E>o{zvzFF*?gI+OVqjhr2`uf8=#6|eYa_sko*evg`t%(^W`PZ1e*1hT%o***rx9jO|K z@GvJP%T}+oZ(%(@4rSZ{Q}{y& zq}s|K2TgKI&@kqn@7NRNh-~gAIzKjxeI7?DyqnJ5B$#Sx&{A1NTqNB&zQrm;Gk$0? zo7TQ#c(wYt+8LoniCyZ_Hxq^fvYw!WSGMaxR@^awQ-0gG+w>N!glHJOdD}{j1+>OM z6J4~1rb(E`3#kFQ2WOE{KOWKp=oiwlcQ&Gd1oJl>;Ub8%I}(36PHK2*@wP)w3C<7~ ztI!=$MXo0byr1U$4{IolQd~CQkt$};6LI3DlMj~x9xt{gNvIv^>QFuaNXBZpU zCdW^bi{x+HutDFwsa)CTE2^Uc(c-FAO^$BN--wteUqWy+deldW@2KO2r(WPczV)Gb=YS_xN-H|vPPAj%2mHHiWMbC4w$Oj7vi8ntMWGkw+ zbJ1#xMak~f9hfm?_)j)rnm{LyNZud1GBgNFS9+7B@lN@9YrY^OA4*ju#6`$NK1{Z}%b3a^v(LX4i|-ZhBt{pPIa}Mt zer?{Ni7^7pPcIAyH{uu{dn2{@mL#SL8tAb6mUjo^7#$qM-o)@l+w8G7%ZG7I9ulYB z7Ywkq?!zSC9MwSuZ?&6y>e=}mfN z))nToaz>kR%nz6$8VIn#3NJ)jtR4|SIpEkp$%nL#&||04tBl<#sZp>;Wcb8;e+f7diZbu`|Xia(mq zom#88dw$|t3b`9ve~q>Bk|+7n2%-74AtY~r^V0v)64{t|f!G+iILz}(f;KrbM9TmU z76`FRw|^r}-L|W!10GNG&kksDyC_G!u@HoIIciW_#trK>Y(iM)#RAW5D##Rp-rNYS zT^Q3+H|lm;o~0ic&y7y4)7*|?6)n0IYs9xVkkl1sml+V{b%s||XfXuNf{-}j^+ zv;4;bl{<`5x-S{exRN7=E{YQRGdGau&q9C1p#%yg?Go1!m>AUz&($AhzG&XOm;_VD zHO}z!-~SMfd2KRP>sKLbF?F?hul07i3fD!J_;fQ_rT09_?>&c4^f4*{`1_^KUD`odis+ zzz&r#!_tZu8&_;Dk+-fcc?cmqXY@8Nx&OK9F4o)3Kjy1&#OSN)j`ja|2>UOQLk$yC zdka}dOQ-)ZwY>gB*Tnc}TQ>)}>i^r-oRr6HRn%L7)+h=zhF7};x33bASSy$BPBa&9 zDuU^gybGH=9gk(_>-*;RF9cZ+CksFjATqY|4o~m_JBHft!!5hX5hlgA-u>+FGuwHR z`nDWvQ#YP`hda_jSg@#*UYE;yv zI*dZqYHYZpb@TIM(C}kuwDCu^*imhLXHRc@yZyj`{Fiu^9b&n0&U%m@yzz_t7yydQ ziZUsgQ(TYQCWpEl~cEHwSiy)x3iCr6GBzNqz+;CIFBHobdPb@D>3{}gE0>%eB zt1|CfvRNQn6k~x;OsKIp3@WW%Q$YrB&BnIEsEGx7Fz7EGsiA>53Pl_|!ZYW+4Bdy$ zCx)TxH2O34H<+m9EHe+GS9Mj~r0t2srySl;^X230vFKk7*u7ge7331PmYvmrw1{&y zp7}Y*`i^59rZZ0VbW)6Y!S5-T)syE(XN}pml&ZDySnjjNx|9ojw6Qk%BbN-hRkL2S zY%~b;Fvo>iLch~{HUO%jUnRkpP7G(R@&ctI0v9HpZHDn9hf-`gyfCZ}17BNlp~zKZB_3g8QmVt7}WL&9wA+ zQJoqmWJ@RJYN~cQjIjKSh08-Mlza@MQ@eHl7h~_(ooU#$30CZ+V%xTD+qP||V%xTD z+x8vXwkt_Zdi6|qzddXEnJ?Eb*z4T;I!K5Mu4lp(Ry{;Ud)5x|deJTq8)LDEDA*K> zR=t8KhU{5!NZMV!kBVDoR1`ScK_+_TtTklKr8kC&YriK8XMgU2eS5YR$P7(HnjIC} zv{r7MgaNx%cc|Mv_7js6q=%N!-d9(I@Kvm9NTn%;)QDRo-|G3d*u5VRQZmAv=e>?n ze4sX>VDP#UY|47Fb$vCHB$>4v!pN!rFIMeooU+x7iMT_MT3 z9n;IGUb$^%Fk-^^Mn-|vaFl1)+NoEb^YaPkk>#-*+K2K-G8@%4nEx93+qaF^tQ6C5 zyIDOFbW@6N(ziWoS8J-k0E~rZK&lpXYbLIUsamMUf--wGvX(cIkgTzSQ0>uugjI-9M99^@V+A3X07uZi zvdFl+w0vyASY3sMnHsNp$R3TzoMHEc1o943e7wS8H+$^r0j+mkbBQ)oPUP6Mi42Er za`Qn?)!E=pKThNT(D_ME?n%2s=*-Zcs%xS@mq+$Jp==bv_DWW%W2Si1GwcS&YzW}D zgP4Ebn0$68Tq4eZk+#T*=XnEcKqD>UM3VtR#{-3vaYoLFrhf_=byYm26^{)HoEpm? zjAeFYOluT8o6DWV`A%9ikD;j`$Q1Yj6bd4gm1R_p5d5fHQP=qu%6&S=zXf^Z@KPj? zA9bMAs(V5sdr0bbP&*h$y`Hyk98-n0f6YAyTb4``2%b(2*o0W(pz()4Pi z!j^(p*!$z6Z-9nzSCxe#Fho7{z1J8~KS9~H{C?MI^fog+WQ{k*6J4Ql&4Xm5t0z#O zVIO3TWc0gZ(7;@Qcnycj5Cb`ZM^F}{4N$aGu;y%meq>0NsiK-W7*HqRO7eUp<+6`_ zDIiyxU0f~2iF@Rg?=p?(xq4Xr`mYo8F1cK@LEaCC@uZR9X9d& z`SARSj%k`Dhih?A-!Q%*d5-F)QfOX4Cdx`-Noz4cp{rC`$$Cwi9g=vF-=p)W3&1^| z8HKQ01BqRHq|{~zFOf-a2>%scXxiC9-v#>h3-m{z%=sTT+y9Cq2;1Aa7+Tu>4?sah z+8&wl_jmR-i7e!kC`_`tJ;}K-B5IjP0s*9BILADd#1>hXnvujo%Av|}Kf(bxQaMuC zxBoZAum^hz5+Z@9gT~y@;d?LU+{fqNHs)V(4tCsCU0I_R?ffn$jaR5MJJ+wdTfu1s27!zmVY#Z77n4ndbxA|ds1&yY85$!IE>wqdv z+loRz76R8Y76&AtB^AD6m#iq8*YTsZObj^A{FhMU^6ECF)DlH3vZK@Ls!uu+iigK~ zA}E_VR8Jhegzu;P*ZZpa#YWFg%FDTYk|w1(TJ8)&AFW&HKhv|ho`rDJyFy7e*{o>O z`yVcgB9D6Yqo#ws=U-}U1aY2Z=}3o(iV)DN)Ho5j!yS4Pv4UQ+6;Pdy<=@RujJHpY zI|6QvRL@o>jf1#94k(&>hVo%0V~rZTA>}uIq9@GScv9VZ(r#`Xb98xYL#nN`tdrRneVQEBlO}AWZ9`(&$Mweu5{;NP7g%5sw4`Qb{J{3Ogh- zWOgJEWOkgN|C`qLv$)li0`b?cW4d3zc>m+n{I~Ak|5ergzfX=9l!rDTgDiFe`YYk1vDBMJFZbCVmu{F8CA*9?AbnOTlOj;=T5}J_I)EgLSt<==kb1_01 zr$A{TN@?i=m>joK<|R4v^JZl;i!$b=0x+q6-g(6mlSwtjEW76Cy?e*Pq(VN z@5;#Gfi$S-|1w19ef%Dnv$~Ui>p=f!v#0O#rv90Q(AVGReJ}m|MUDRr(#LieiNN|M zkN*u*SDCThR`?k593Xr{<{vG`OX{v8^4ZnkRRi%C}7n$J%$~&`+OwN=kAbJ zcyGv(NU!2<0_EO8iK-${r6Qq#ubZy|O+RKi{5+|`qeiTB|5Qg&P62-jE<+HvcqPm$ zqk=0*GhBr%QrKRBOil*V$wQx0{l(R^GrNh}D=-$&rdpu{>|q zggk6P1-Uhj!H*sgJ`0IXpq8{K>YXlmcB)aY zYxfINra*DmF1m{C)Uw!7ypjk7I_uD|q3oDKupBN5)ZyAUN&5J%+vzycBwTiu{qB2q z!)f4(prhI8j|^>OQSVmvPEvC)FJmoC5I8(D1=U;O6aIi(w;DgLb`8}TplUtY$mdOj z9V>(qu0`Q_QlF0Ri!CTkB5GfP8rnhRmUyyF_dteS$$kiis$(J|5AS@G%1DQ5X?nPZ4`dOwPbF;4RNQIJcss+!Pjl=ogq1+2)fDU3Ba2;% z4*dnFFk?Zv2BD6HnbRYy5*K8(lvGF)tDAAMT|2zr6>`MtN)`92k!L80P&biDH_2vL zGV_zhkD@H4IpXy)Try$WsML+D13N@FH5^?gG$q^MAZ3hVF!~+oBijms!a_DRU~PD9nv*(JYaX?^#x1gI6SNH;;2}A0T5eYi^_=K7v5>dW zz^O}(@|yX(FQZEL`Boq2JpvIWB9A~;aU{DX0t(6WW>3(q9t8D!=o zr8zGa$_l7jW+B!&7gC}`>slkwR+IU#m`H4Gxtm(bBs}xQzetHWriU3xiH1T& zWFw%A3->Yd$C9l{>?8)0i;HE}NxL)$WINQv;MR>NT_5BI)=gc&f&c7>x}tDtvmJYp z-sqNY5wK@EC2i4g{CPl@@pGKR%%fGCs7#p?NE`2}P?;!ABgj+qU>ByYa;2!|v`1W) zRl`BuuHi|f)L=d0E$vpfQ+B>(KK^p*qQs9!$b29?9!AJ|KzuJJ#AkL4^4W@)uV2h~ zkn9*r=02udS#xTdPVO?T(>hcp8+GVUmY?DcPBXSg0cCbAWxUy7KQT)=@&+rDT^3CD z5V&DJ_Ir=c@7PkdsO>!2vN<uhzN+QsyJiBR@-j09!YoL_d}$%N=?w_0bHKd5>(| zWmE2@LO@%m?9z*GG*;YhJx*phvM%KzL8tT1G3?s%>ev6Be zKwgAai{upghPIT^ssFiz1dkZ)9MEW_yM%-bs5*4CR@u*|oSSqOF;vb?%aVOh_4%oe zHm)Ho?aoJu8F$w>k8(GejFM8F1NTnmdb=>%ykvP$*i1X>3f>wtfv-R|c@*)NR)OXi zw)0a*`^cr-O~b!CbIf)}cTP(V!K&@9Mn5(OC$0tw--LsJ0OBe!%{7~EYC z0z>T3@tvm-huwnV;tak6ojNH1?J}4uPmelhx-A^uNI2UfS&&vjY%o4+k-@KPNignL zHFCd~MM3hBj{}Go6-AlEqWGd%qASnYYC@R4X>z@%Q8gk!dQC`*iyv3Vd#MlfTCw+73;1%)F=93Q4h-%%CIa?SYlOaN zYgB7o12awow3qid!e$+$$fxM3n4X!f#a<(-^H3$S3wNxPn}TJo$ToD|&&p(}G;YW# zJ?d_|$1+yV&8O&{gVlfIkgWK{=@3D$k2h+f|3ca`xx4G~J1U%0VYLv!>&U^3+UA0J zUnOYhs8`9_ABw6HP`8>EKFSve;#2HrbZ_IyqqN^y^qZf%nU4Ovv+kT_3ny=aC(kE6?$fMpoU2>!#u6;Ao(-&xmmfaHj49`l z4=)~+8WyrOn=Ve%xkB+2sFaL8eO^lDPHTu*qqplYcp-I?8W&i(TiLtc&)h@cfLher)N-9^KZ)tF5d#+rD3$A8Ptr4Wo`SoBqgIsw?B+xFrqfDX!xa zI%^6Z(isVRJC!oD5=0xbiY-0Xj~B5ot;7W#F>_ffPtbDO^Lzk94mw zgwzWqbL+X~-&8!(Cz&^yCe_5Po3U+7c~P_Gm{{tq$~m+yt*A#m0o5tM2a zD$99hA-1%Nw&akZlGui*!xv_|9r3)|tO<<|ieZqU1>ZxnlozA1#Pnm1Bnkl}d7baN z1GkkQp^W2xv84nf@_D(saZ)z@bg^dKDr+&;K)hHZcv79Q#!0)g)7+e$xrfl*QAOAe z#3ha?k(dk$L+|R?@cD)N1f{<@c$pP5#8k1w%m%Z>T+I)G-g1RxR|{FDSxVWwv(xM_Rxa!})D>YKmo!s_@}!nLPt` ziZ{%e;;?e>{nLWYyOBxl7}(kZ;hP>ZlO8lx|698OCCwPMc34YiWWA~ZnQ#$dpLZ>w z#|NwER&PK0p&xJKeY^QowmpMyD@yv^1*THcna=KbyzDCLajq))Uhl&IUT@IW1E06? z_uRo(dk#|S!R6a8J_%d3eL^2d{L-}5gQz!1J}G*;QQaT1TQ06AQCrrtBUXO)BX2KS z_@CrrUt(5QkjkUtJzCleR{K)FF8B>w{c3+1tj|V2FZd1Ej=*@oZ@Ye&3>exWb4L6- z4Z!zTo-?rW%bhLiseP&DsjC#aEwBEMG`dL~Y-q4HbX4@P{MLRsKq!gX35}M^zts1~ zqJTMe;MZpkD-DkxO^q7R_rQ>0c0Do}jc{s>&*q&RHgd}uZmNvounso2Eia5|G0X(| z^h$-L;A@}nHZ@<-ms6}6g}I@a!je9q3t)niGs9w81dBQkHf7m!>a;?|ZOv5M=zr(y zJaXzZhQMu%s%?*uQ@FV)5VX!{&KpB-|90wh$Cte!%)|1HD8l7{(jQglC`@sR?aOfL zlaf|*@;$zRwsOM78(u9ha+=pH_lb)1Xb!?#-0{RPGWZ7$5_fm2Xm1=~4;r_B536wn zXc7pLI&`GDwI3(VApFbFL_ogCVGn!CuM=mrueFwoZXs%@X__90^f_t@##11{&APENHPrkfsi4r-g5rezda&9c6ALHwwUsX0#o8BYX^17wXyB;y-PzzHYlJrB^o&FE2d3{`q2Red7B)W*}M zWx*070>s26;VVtFvx;OVnb1&Dbh;gI+#{PrI@fS|O>wgv&IvRXHNrVX5T0+4BAHil zVYN_P?kM~D3ylqjHE)vkC8m_Knu7D>J|O*pON&~s%biloUU^^UjuNXD>KVVuXT0zmfBXv!{YUD2sLpa*7tvBTU)@(g>L(rcE!X8pYU{|Y zwx=DODy3vs+@+si{Tufax6>!GG1(35>5SATEp}t_;va8kt5Db2gwXTN-wyXV2RY7T z6c0V|xvX@7-35g&A?0ne2W3gRtA50LB0^uWBHMgcgBZeO0Z~EZW$5& zOnq^;5A2-4(G)~_et6!yzd?yh)itrwD=^36x-8F%J5Km*I*Cj9*r-8@{`*zW2?$JY@vUr_Fvh z_<)z*jQ9}}63c6d^tP>I_dmD2f9<|MFIRVgo(?LpwHadc6p)e%BM-^H;kp>%WM#~UE!2cWgqp!Sv1b^zn(>e?j+@bW-m?g?+b}WboAD9C%R_WkYumhhQN*qN$gx47B#q z;xP6FlE-9NlNFZx6M*bu%+EnFD-S9;%b9GZluihT!of;gI9Ub$61K22&<#uIWipb5 zJyQl1RE2jn1fxo8h~qjgCWlN*sxnT`uQ3dL6T!z!V5&B-@ZuH48G?^qz29AA!>YYn zcA8xqbDYi?Q^T1Aj+y_Zm5v#$1`=*t2BX$c;M@DfV{C=Cu;iFo;M5O^$PY)HSsP1M zm{iF*Ku+Ae$YMmnbX))?8l#z6piGdm(u`()MlE!f&ozRKNR3<& z9)`>y7*@Y^!6%o~-%61~fys(-gF2@XPq>h2bP{?Q( zshDT4Umxz^fKOP7-PX?Jojqnl@Dk&xG^p%yjT$M_adYDl7-J~HXTviA?6K55yw}ataj?Z!eYz4IlS_(C#^S_-Btba@ z`rY7+CQL3gA~6W%fpl#B1~+5hndkSu7l&qEUs?(mf$x;$1_u{9OesJ(r13=-dyj zMCk`R!$$6cHe`%hg&oS_h-lRm&8B$Binl;lVJmsd=RfEhd{4D0>v`(CQu-RcBXbMs)wr(2&mi7IT( zr5Yv)U_oY_Y>7s`(8I1H7_(ZpR{%l6~nT&dM0Nvl*fb;FD~3Uw)Kb{IT@ zSE*yyrz~7`qKYS#-1uh7RumWRy@D9c0Ph~@hgMU99@#dHSSk{`5&2h|IosV--q-9> zFF-~u#$t8WTBe*TR(IH0oHRZ9Al17xIi!DXAw@`4rn+5B(o-?ZxdpqV@1>+Dc|(aV z`0z^v4lDeQgVkon<=45l5k1oMbq4{G2(4~?MfjNooGydx1)gqBncvDmr%=dAI{TQK zWwrZ}w;C3tRU+wP>|)Y_E`j!y7Ft_KcK$VC#$AnUye^sxp5vlfkx}InR*bfpP=lQn z0P|qSN$0ey#`=qC{4yC>BhQa3QH6uwmF{#ZX=|6OjpVdxDruZs|AZ6SMHa!F4Gwv! zl+oT@RH!{8_TE6wd?%qT|M+(1GAD?$Via~sjZf9Dt38!@tXkslpxcrWQK?mA1-}sI zQnQ!u)g@!$i0HveDV{Yrv;LpwT|ZVdOR1nIkggcNE07cmBwM7IOpz|*KNW3_ealyC zNHWcO1g&c^2{3k?xs*^5Q8)P@@>EU`brkL#tIowdea_^Xh^6m$lO2C9Wozg&id z|J{nS;tBA&AwF$`x+B07NF;hozWkEB4GR34Uyy&uXP~mY5&XrA0uL5|dj@v3Ah#vW zH+;s|ctKR6PHI4uo0W))S{VPWIC_%B0O@O!% zPWZ`np>3xv7|rPT1$AqGkX2hr%RASI1!u;21d9r0CBal-kus3R0Lxq}kqKal^r(A?1&x8;5~1>GNk1;c+_Xp(k6 zQdT=d8&wxeoBuAc=+T1mL0--H$EW==e#jU$V~Qj|8Vaecu`d;VP2xW=U;rBt1JyCM zxP})`N*F^J&vu^BESrgEnN(Ul16W)vww02gw#n`i*YYZPHFIwgZ|&Ny+Wvb{YWVLq zCUzl&)Zj13T({?A&u#Axc8=HIBN&n9#6Ja#dS1!}eM%L9JJlun3;L^zyYqF{=l18f z!^gL%9}tXRalm`*R9~szk|lpi{~$X5tDNrxEV6vKJH8`?d@FZ<(>uP~13mKFedI_n zpZ`Jinv(5Px1{T-nYTu_WwLsXee_RAQL1OOdHn)I8q6f^H~X zw5nD1#FAI{2$j;yt#IOHr{Iw;LCdA3a|V6V2|2AeIu*NT(BFqqf%GgGgcsLACNd?s-zgD4`P@=&>GJlHg>r&~D zb)oj^?PI5hOJ2f<#b}sI#Er#pD4Wi`RApRv6EztA;bCd^it8h1;;|Xt$f)F=uD8^M z!nkb&B2zafv0#wBWgHT_T|3K1yj+D44%NY#7Y6Z~Mwy$COk*_y?vW>2De^KHzKa8O zkwy>={VMWeF#{a~4(;Q%!EC6<+px?ajL){*G8=jF2bhZ*iB)s+pcqDaH#wXQNRV9M zGrnSfD-JCl0wdAO_GuI$U}ML45SWSG&+jye@7Dbt^WW%`jccXP3QU7lbWrM8<)9Er zq7d@(z^F4e`}t5!la*@1A8p4AqJw5s7JA9xkN7IDMFI_6+2Ca!HTNSTFxBvC52P*oIrBd2(@QN#xS_*aJ!pC#M$4~ooc_jSox`kw*O&G_Wgu1xT4C(+k*_VsNRlu{R zmjS(+1YNu@N-yG~T_>%92rV;GC^4)q2aSO( z{+2Ol^^F5hq&7+FDS%Q(aO=xaCg$RMCZ`iX9>y7rYcv|8F|TnRC-Gzk-P1>~;D(sO zz_1iaIS)mFtn?PbPco80=W)dpqcDF$i7w3ug(K}R8WpX%z(^fZ=-@?B2ztjr5?=xq z$UN<^)-V>WO~gW?P{&6lOiahRukcy=SUhR%MB$zs!H!;Z_g%`94VJC2pB0}T8sn1} zi5e4H=tjHR@bIia4A`dss-(kW7MF#&ygqGJdSBnyNps-rn*s#aO~Sd2=XuCTNo z1v5YH@aXl1ve)(3g9%Z1LMp{$LcrX-Mll#>3j}PIJ!nF$>iwel4TbQN^~D%Arr@WD zvr#57$F@(KAh`Lwu%t3#OP88F4Ej(;FxlM9;zB66_QIttkRFW)wS0g zwO9@$);Q~eSpoJA;15x2cj*T=?=02HoNlYjzXhUrug0^Rchd(P|1OfB6v|rHJL{B737)^s$qF991unU9dk7sP7sk7JD z?7b(fHg&(6DRGYcl3j*IXWPfr3_ZL-?+iG!T8OT^8z=kjMC}bZ|K`u0mDsbDZx(lA1d6$cf)qvVc{b$bee zwM#|^@$PREbPRr$ipYxti^&*15~c(?6W-_T?G-lCT+R)Poh2BzUFCf>eS+8%8c|K^ zGT3JB*u{sogb;aDa>vc;hrUbbt@0;h6C~#L2n4!M`|$P7o7FFG@0+eyyJ+kjDrC3r z9^x$sYj^0?>XY!heh?qyCf|B5&cC9q48K}XJ4P8bWG^T2j$fqD?NtGg}13s)R?@JIw~KT zOtgekP6I9C+-bqkU@x)5!$Fu_z!_<;?tUHRr)vzdkC0mK{F()ZBYW*VXx_3%Vjf7z zkxSJcNA3Ni4-!)e7v%bObLf~*!AgOIjf%Kpd3ysoVh&uq*BCX5pWq?-J86vmi8{+S z_@AVT=jnj3WCom{VhNQL6rcy%(kSU_b&8#b8%*3N?Oeg3A(cx^kv%xk(Luqu!QGWN z)n4Jl+q*C5KkA3dJ`hVME^<$6nCr%&ybDj+m|{7xLsQZ9rqo($Q1bH_iMFOBSJJZ< zpS*^#RCD2WS$lsafZ%1ke@YJW4mJG=yF{JGhw?Lm>ow!z#veQ-JDz0@@x$BM-@_;@ zO7d{2K8a)W5An@QbiLDp{hC~OWU1%LPx7t?MAmh+;1`Lw!NrworY5f`iWIE#fOwdD zW)njzUT6%U4%wD`rYC^ivw5iMrR%PysCuSq#aD4s5t@|l;A6rc_-XFwo=^IyVPm=c zMYMl)|I~BmZZqO`Myo8@*%zsKMdfN7tL_fWN!XG z*32x!+S49!Y`_LtM;Op&Nd7WaXKFIRCh|?^(Zx9$=gI=MsVAK!DZ_J)mwZiKJWu9J z>Y_&6Y+eVm2!9WB)bLmuQUQ7CO7zb9py?rNy$jH8)o&|bBhV85je^QF0BkO=Qn z!8}AeV%*IVQcBrE{fS&!`xpd!^E(1e76m%Ph;V==flNu(6k29KkI?W)mU&yFRl|~` zgXY*UEXOS+jN4a9J6URj7M*is3)p_@aWY96S0X3KBdQ&5#f!x{ zNYOF{lX^$AIpfy#oXBq_fMMpU<)mamVont%5^%yA7mO;&ioxvkhc01>Blg5Y~Fca=ATl7(YVq*@HW~H)Sami3*EY+p4 zy)t9t`n8Simc-Z1p22KrfrYS^!%+ObxL@?-p?RdoWF$n_smpiiFuSRMsqbXkU!XDZ zAkUpb^S=N{PLf{>e^0(YO$3%x8WD~mq_+Et=RmITcwFf&Bd6w{&;}CRiC|7!-TB#= zl1JK!@z=NSLUwML!K7+X&m(XcjB>oTe#<($3&PCJ#1yy;5w6LH0FdnUXy~b*j7z9z zD?BJZol7sjfBMN9wdak>CxpqaN$;6!-w%Gl7ioQ77h#wkVAKreIM!M534X$Yqnhkf z`4xz}kanE$VGT(%1z7Oo_HWC!EeKYkK(^%zk{kW zC(p3LRD7egI09K=K2+J3bsHkl9kiB>ndAqvvk-6mKtM+ZNkdc z`uZlC-Hd*CeX_23n6)g=rLGCixCfr#fH>1pE*mk(H3O~6JuHfK{-6Tbu1ht|V1+gz z4n0b1GruRKm(G80$EGw{Q1&G4fH~TifOiJ%#B|_U$;}Encp>zpNyP4my9+AXb>`ia zz6yun_g~>`U8mp=OJQH&32aB*>o|Sono&LxhTQzcrS*(5n;c;Js5il)lgCeBd#8x^ zgSDV8p4Ap;$9gn-tWsf@bar2z;}(pems|AF*bB@(^2L5~+y|v~ICEB&%7fX|iKzvp z#Uhv2PLYS*WfY8}wosQg25{(_#Kqkr+!n!g zVRjXqwU!*S_`OO1x1^wzr^^z@Czt16<_%BEl zZ?qiPgX(PQ3A4ZC$x@g%G1LpiSt##Wh4b3hO68349luiNQlW&Z7l!1gZ_NNXvD+y- zRX|=E*~Rfv+2cDh8Z|Lh%fHX;)AhjK{Zabp+EQ}9(b0-8G%VG&7xX=!dMoviO|E7h zS?|pjphtA7)~w2^7O|dU3y1+gX#q+}YL-syREixptgx&TPc%uy7Wt|+m&M$RE*M6z zjovfG*sloHaFWdWRML5EvuNY7cAH1^DmMwCfly?s$i4+kmc(F>yxU1s=sejb2B!}WwEbyFmG|1!p0)o zX9LS8v45ZXN5KK$WQGq8LIKrFY&1`3Y$=soh$kmBs_~2M5X{aiwb#rW=z2K&t~c}q zE&ZP9vj(?pioCog(aCuqMBye#i5AZrYE4d}5zYme%9!&c6Fm<8H!diGAz5Y`}P zG6|3WdHle(>S&c$4F5KW`m9cNM&+^J30gB&?gv6Q7{-O%>=?+Vs?ao@t}9o2ku)I! zw3DKs|0CgV*S1~CM?Vvt&=t?@f`0={DW$@fzY{tX^^5ifh#WgPR7b|KNyd@`5trLY zow}JMJ9c)is`p_sxsY2-(dmDw!Fpfqw|?k(*Z25^;RuvlBe~cHe)tfDsc=2q9k%lc zGVkf4_ot-}x_{nfwcMr2c}%89^GfP8$~ryS*3&**q9$HLSL+Ga2|>A;{T7|feTO&8 z7JH-1ky`DNs)){%yx5J&YPBzi zB?w|Pozi8{1A!xv_0&x@HJmdU-pEWR(>>?JIPi@3SrF{Z@e`@r_p1vwrJV}+Bf~GL zrVru_y@L~A>Sq5aJO8|38WJ1LY6nH|q&V}| zU9kQ_4uF*>hJWz2#`;M9PX0Z>HRgXW?v*R1i0b)K3CsUvy8maD@c%*US)(j#za;S6 z_mV>r8ThNP042UXp&dU}%TX#BL{Jq0Q33_5R@qy!RjP5tO}8TKyNpOt^`IYNPrP83 z3kVSG~AdGnP1*UNn|G=3H;%3XbQ4HF)e$45|qQ!ZOjJW^yR6OK}~ zD^e^aBNM!S8JpyF0h@6nZNOv$mdtpm7FYQVHox7IU9AveJd(A_0QYN4)_wpQhz?gk6`JMYyG5X2;p(rv=aA11haNrol*Y;Q5E6g4a5>Y?{-n29Tm#mY0O;oG9iNk$ zK+NtdMu6?f`aW`3`;?~d#>{SN08M6Nor8{Xwpyy+S&dV3sH-1$OZ>t!N|5|rZ6i3j z%FhLn(Vtw*flM)Nabrw8#D&KJ(EhJ@ohJyH+t3dIwTt}!#}oD6XF|%}=zn^mTGX^1 zkX13h*<_R0ws#h>QRq2u>btmvV&eNeEAIL%v*Qo{%ibzFl7g z17be-gNolIGGX(98PM4k#Uo2L9-(k^;Ers;(9`uJ&Nd#oaP*+}Mvwpx%HXtn=uOVC z&_|p0{!sPxJ1lIyxQb)qVpXX90OUza z(o`j;iM4Q3i;N>Ss1%X}C(sKSzg@>&Rp|CJ`keTQx?Sm>Eu9r>>P3W^>DmH9pd*Y@CHe%#lgVqa(7{ z&1;yTMu)OAhRUW+P8Lp?(T^t-%?j#4%z2q8krl7j$c95qY1vBfu*yS)FXjx-W`&jc zCMwQB91FTZOuDNiBV(-$GFH+W($Gh}q;8b7d8Yk!@*QsgrdO94qMGRUgmPqqKCdWZ z^D@*R4-H{FgUU!@dW?7~5~=*!roAGfaQCmDCSf_zx4%zNDW>vMz`3DZ3C_mqV+m)z?&`6X3tVV}`V*>GUDEyB+3SEzy>JZ^PRQvl|MPM+#Jm8Z zN1JK#L0?Lhj1eK(z(T`xLo`PFM>ug%({5I$O;S`j=;-CHi?N$~?QU=Bo2$n2$$KwH z2kDaZ1QW>{_wAOD&W1F=bW|aH-=OFU;7}bg%XhLlb+!boW(PllIfz!0j^sB*RP)$% z;`;T)>G$|PkMb*wQodTB5_69MN^THYk_9Qlfb?HCOYN)R4yN@?9Z<9FLfirh%>qPT zbCP9UtUgSgn7S;=c#%G-jKXHd8{8?2Y$m)xc(!h8LXHACs*Yq*W7L~?M{;Z;i!x+l zsjTSdHG(2!T_&Tm!q6VrCTli^tXnWg$<4_9Sq-9M2%Uj% z3Zvrr%9`x>xdr#gR)dqyOHXT%|6-SB*Vv76*yB~=YLf<|3|yd6exhL0rIvVpvreNd zI9x0*!x=62=2Ih}P#2$X$~L$J2?N?mDIqdDm~&dp91wgrTIcP-e9^)*h0Y$Cr=5i6 z)V4yp&bO^H|4J;54UV0n2!7ifI`O6QB}P4XLVQtZhq$tT8RfE;B+pTPf37P({jl{E zq%8UYQZ^%B&3^uB;tH`$UTV5wENItf-O#(JTB(NEF+fTX@P^zmRkn;{m~6&35R}Jh$CC@?mFVn7^2-L+RI{;E2>Q)B&&DBa>yH+ z%ZHq?TQ=6HLYQt=_Ix%rWnBuoK_=$q2CnJ3GW`8C6OXAvKCd{8eEKG&3iU`1R9-vK zo0P3Yh%d%0ULY!`aPy*q{v=mp;Q}2yRX&pCKEEJvk9UmzU6{jcwEoFUoHZx?#4bUe zg~cB{1ST@1PnZi48FkPQA-N)bAU*d{4V1`oW{TY_r?3>XUo?ZdyhPG)%c7FQ;#Pbw z<`rx*b}>R%bDh7uX-QP|%@zxZ?d-qis@vp`zlkF)ZY@*k5AE6d{2>L`05=lxXpc z^a1_+Q#TN@D<*xPg$bA_Mvv$pWW*WRv7opoDJAX4;fYl?D_Jz4MIpZ~fMs{wos-#=GU5y-D!{Qr@``u}vr{6Fo!Isf;qw8`PfKo>kO zB%M#IKwv2hr}G>7fVxnrq*xT1N{Qa$fF>g9uw@0?5B85g3_6|Ye><1h35Pw9p;Q&% zd9QA+uCmze#hUp2eE-7l0Rbb>sqs`B)kI(7+3lHOUeeihv|k^@3YuO#yr8P*x*-CZ0_6=mv!WmcrDIbh@SQIJX zw~c$K8H!bo1O!R}8-EyR7z-Lw{z%|cgi~UI^A;7EjN3o*3h)1=D`~|+cLk~|BjZZ$ z;FJ4PI9-z+v-!jXWt_sOt)7{H)o$^|0Wvod8Yj!5xu;$F?1-*y^XTPr(F>{aom{`l zpG?SZ@jSekF(KhgW1-d7KcsB+TZ4}=$DcddZ1u&fVCPM}t!Fx(Z9R)ZoPOnBC=0v; z0%@*#029J34*d=3?rbCK-9%F8!KivT{mc4kk4cU^<6IX%*d7iYLEZ!v-E1Ig5sZ)C zCeNTwn9ftvsm${rQk`s&`EW;zzi26)5mcH(#)9-x zd0u&khTmu&lzR0rR_(*N1E@j+My}nI;y2uX9j%&_@5-j1<2&@HVBtS!yDB)@n>(30 zJ1dzw*gO4?$e$8bTjd{xkKYZM2_Y20KCOZ!5bC24R3Sj^cL5@bh(e%7&4%VsPJH6k zjSU;_8^jmP7nsjwGjk36*L;KiM*)B3b%(osA}Ims(!|ltw)Y(GoO7=8>~`Px_qF>k z<~OovyRBORbi8mw(J?DfK^~N$%vjg^t?)g@-0*cXP)?iB2?uFdc~_fDkJ3YGa9;Zi zx?OinyJ}52LgO_tzch2V7~H8wX44_JH#ychW^FN3`;q3V8!A$7bd*mZ>7Il`W8=j$Ok*ctyP1u3zmLbf})kre^y3}0<^C=JI!j!el6n3f2 zP-T=`E5@@MH0l|a$589!>Oh3c24hqly$TzrO(&h3-!rq>)<>^+n5|DFAwIM7iu-i7 z?W#^2c^ZFL-e$*>VVmM7I^-5;Bti=!nZk3n%puV^MwCnZaipYt(R3MQZ%m}cN(Rr* zaQkjao3wIe{S>OBos>GN$%2tsVwghy6s*Gx^*d(3UuCNF$hJCHylYZ1L~^Rrq@yGV zi^0rv2di2y=VlXaQVO*mHP!5CsAdr=u{_G@F7+Z4)kc1TzPtgMPEf#bgSSpHcxY)X zayZhpE+%uy0J$s0nj#!uDJ2lplFdSv8B$alTk_A@wpnDL;?h$PJg8918RbgQF*k31 zg8}=!@uQ;R&d)1t=AjL-hHBHI>^tJ`XREqC_;gS}SM_ytl?JObb&j2ruoCY^6ym!V zoe9!J>OC<|H~Wc--b0BMwFg(n)?p@|*5W&w364!s=WG|$Q_Q&TXqZO^*XvJw01HG>eV4RC8{l96mA2}F^AqG5}5N2 z`|lIYM6C0hRG(ukOklnn7Dl5~nMbP9Ji|2eFHqv06E|C0Fq{>V#}nc$9l4K&VN?;1 zGKhWh4FA$Vkgcd6m+((U8IOlptP&8y+7ad#iT?r}kTwW8BXC#>V^kT#C@e>A&MWp* z*@#F<76KH&DaDjAQYzn7m@t$K9*IbSGF~yBCBc%#&EwrEL%vZ=OF{Fbz2baz<)K=spoLR)@-Xjaf;S~@QJi~oF=@PIEcFDaW|rBjRU!U*j5qKz_~U zdY8(b~X+=E0sQ6?ZO@|A<`&e!vkMC^G1il*?7N-SR4^fOf4s)M$Y zOv%iZlJ6C5=G;)!_j>NkCVI}z=ITXpl1kG&erUbOQ=s}n%PI4I$|O~J#UP3+GyrZV zzfP;CS*@Grm$~G^+h^kX2K%p%Ai?Xe#EGB%O84^+D*ea(`u~nX`=2%7!uGbdmM%Xp z9*7{u2T8!+57iGOFx(iAx?CO8VvUNR@VPwyz9-^z-TRIkCO-ka1or{$3NA zOy+JehW2-!S))gxz5K5Dt3oO4zrc3-aVYLQ;mj?{CBmv#3k>@b-uc8*FBRd}zGyBp zBy?aa`id2^71~NhNO)>@T*jO7e|8c!Pisw`Oh5rh|vJciCoEjF} z{|6UBe&`|?%Jk5|-J_hyZtQ#m5DIgVKV+XjfFN9GOKnzX8)bIszd2&c2h!Vf2pe|o zbjvMryMSnNdykl1$ND4%=e+KVpzbFijb|W+Q0^Jl?@?(;!kR~d@B2YF?LSTnFdk14mwUcwr$(C zZQHhOb&@}}-LY-kwr%$0taCAI&RMf&uD_e_Vy~)QyXt)&9n*^ke)J2CK!1?`S~%JO z(|Nt0q8Wq*0;2gJ3rE4w$>l$%6jN%_&e);|-)i#GdfLqnm#c@e4kkm-VFJAf#*j?I zPA1xsID#H#B(jIDk{uOGKlJDQQA9f&PN#4xlnM@qgMR(|R~QuD25M=oc*qw(m!^_C^sK^0Q|=ezNnC0cs^Y~I%2Te^m_p` zd(?Ag^f4iNyUdGLg8LyLg9w>>fLV%sUeI3#v$XT>ESO5Oo?9-(KD|Ji%nPoK>T=5cDxJX*tB>@gxQI)?qL1I41#}H9UT$~87U6BLaKvMuOyT*e_}s_nuTsL zP~Zzf9~;uBF%mb*wJ-Evf$N(@><w6UzA4n!26B9x!)N*+8EI_l6p`6zN=p|r)fy2ZC53ay1GdLx<8dlCw-)Wx@9 z^G+NTpG``yw4GP#)`@O+pNxv-%3n;57mnd7mm283X?aneu9wW`;P9L#E-w7 zx>E$vOM~eAN*(5u>m?iAQ=|$6Nq7EL{-L!>OB#y;OZm)GKq|OKw9a{t@%#7x7#t?n z?ujaX7N|Cx|IM5J|2#ncciY*L4}^=((9*wbOiBADFDd}-hPCptE`W3b47LipK8kkZ zPb`UgR3~70ex16yv%@KHHVB!3Bw2vK4I~5!HlKa1zws9XtpE~1NCtw^uCxTCqy)pz z&ds#3HQGTezBXWOm^p+Z0?pG&76<>eDO#NfY*rRKcpVk!J90Sp^o&&x>sKP z?Sl#Yto`_XdAU3DJ3fBD-qe3%L@lo{FJDWPd?^0TQ9aS|uS5JW!f!jp!_WV$)B znWS;piY!NM$YlYXVUu8(OzrN5P_+8H@Tz1pibS3g?VkM@ zlNAS`Ia-Fbn_@Yn@k^Ux!96Lm^5k)!VLXJAs~AG@Ea0J(_9|jJ)G5s(KY17c!7CJ} z|8;UW_Ng(PD*9Vs$G^$B9*c`PP)ySZjn@XC_rS`<}&rA47>qxO0*rI6eTrJ_)~t z^T@3&n6rJ8(ET1{+GVsMZM&9H)BIsM8x(S1H=HMxF#9Cog!^!%?H`En^?345!j8)* z4DLMs^fzbxL2sK-wI)^%V;Iey*KQERd{Duo&OG{ z`vmXrH_-4;LiSqIvGzDjSKb!C!BlLVd$I7&$jHug!td3D77oi?HLSK&%C;p!dt)O1 z{e%|taqh!xpSB6Y8@F-q!!89JPyCyauUW9S0A)Ukj@Bl4EwUHv`b_sn!sA8YZl1~b z_}emZPkp?Di36Dt`L}y!y=i10&p()3HlwAlc*l1T>0K~sUGfhB<~aA7dKb=<7bl*w z{O+UZ7(>xjT=aKG=roo+hrm7ip`DYHK0}21M?VzJ>lK`P_A1uehXL#`p9R95{ZxDG zd))BPYC`{$=FRkK^(_1!+r;zh|ve#S_7_wA~0LaaP-F*R~Jr)sJCB z-x2()zauL9yUhfRy7CZl=dK+>`8QEEk74dA8Oz(zuXN608==RWC@y1OL5o?pfA5Y~ zy|grX6K4}9G_$>3b#6~!9tI5NfW3wnvw&?7cjG2*lh9&gkTnc%?Acm9r8IkG2KU9y zv)MxkxUjuRrGm2qwDVYV5M_C)+W3%QJ+*zLJFljI^(AJjNHAo#BiIpaBm?S;HnAmh zKsC%N*&CTqZ0nv(nXj0j!;Vs-B*vrlO{z#;Q;EQra<_?JeEiR!jH8 z#nfo5H5yBQom7JX4H@f#lCUWS9QB>%IYvu)bMN{8l#_A z-RslMvE2&6?#8{P#T{eV8dq>{Z)?{w(T(o|LC0`^KdJnCS3oh9Pr!a3R~f*5S(_X|!#Qn=0_m_Tt4fq=>gUtf(0fgyE zBpM^!3<4bEUOs5PuOCvRx5{pW4gk@)_ZOZ=`(x>79&hB9djOdgOEwy)Iq25{(gu|S zZDlz#3RQem;-h>BrX+*Z(yE)iNM#*#c*d_-*+5_A*5oP7;lscLE`8@L4Z);$v|5t7 z$MXCJti>BxV|R~Sc5(rEc)gvhML%V>Klfu zA2Rp7g1ItIDZ-UFSv{!cTOg0UmBmzRYS`S;aT8C5-81(RQR`CAlzKa>!uoNv2P=Z@ zx>BqRU{LVQO6Ke~bp3OWfkRCTRS(lUjk}3nD=mS+WjZ|ddmf`lQUE$`@-@pb$o1SDlH-0rfoJE=@VJtv@_uiCJGyxUYV;x@YP<}u0VmT zRD#rrFGIEf;-3g}Htw9p-YYRQ518s>FQfhGMki27{a}K>Ab1psd_?&7j>RWIs?VX@ zo&>_Dzd7%H7cOpKcC>KcbX@)OaS8xDE7eNeJ7K+L1RravYo#-10J+r7VV(a4ANZ6| zLn4D6MJS8ud$BJD?sc6}jr9ZZoa-RD54@3qJa3*5-zbm4!#v_I;Xf)15FKk&8O^)` zeq{BeC}-bD;d*qbv*-bFSo?(eOeAaI-dme&3Wo-Z^8SB$o9;ue88ffq(slEfHsvyq>M(LV?!+^Q;R$NMnmZ@&v+ll;O6ea zBw{jq1JlWenhW6*niMRGcgf2+eX+TDo8%$&@A>OjRX$8}$MKC>abS@6_qQoU@Kg_SrH!O!|Jkk_mf~&SLWe9U$ za53_^BV%u)LcPUfHVrQuj~5+OR@eCh%Ni4bnTi`}*HMKuQjQ=!x#<_VsOqw+PYQJd z=?t)C4`ANgJG4%@_0b;-Z~$#DV$L?_bq}ndD9dq6Snupx>+Qp}XsIp!Wzr={K7f!a zV0Q$cGC3v-IZ7ZSz?nPVnp;TB-#~=)%A^ID8UAjei82r*s`dyWw^%`2cbbIxAaa>S zb5yGJVBxF6QY2pST@(0+n+5Zx#Xywq3|(HHlBZ|!-W}Xr%w4LpiMc207%RocL)MSdG!x?Kig5{1VmQP zjZB|FX#L)mg5a94U9{Zgd`NbnLMjX^WS7m|C z5LTJv6j|8>x|EQg@yP$sAMP!Xl0PL}<~@dw{R~Fu&>Sh1{xH>`ni8IxDVZTfWgNwl zXiLSCYD)pLgeiGIA-#k{3&)tEw7h*VHf4Ygs)Tw+9z@ z&Xqlap6lt~?5Q!qg!UGt*p%Q#lLT6&Ij+xV3vjG+DoKkb7%a-NilHSQq6En4h{k;` z&tYnuMU?Fu3mcvJw0qNJ+jbn#96-N7@zbp;Ksu@A+!@_$wLnYnkmoj!+eO)*BgcyDBG?JXOa=dEqw#k8`&Z^~{1 z(J!bpEZQjJmJUN_@Uc)ERve^PI;Vw`J6wpc{_dO43s}u9NU&xN{lxWPSdu)qQ1)EZ z(L3N#o>G4(=*YGSf^F(mfms?0`;7isqtr%V!E)%7j!G&NQc5b6qGg*zR5Pt8TL9%p z25v{|VdL%V^*lH#F4B^Xm|*_ za5w|l95mB(8}E|@czOvc06tyQF%Grg}tV^DgjSov>oelczOJG*$IDzHe`Q1hC zU6ggFCpv_D)o1He*Dr68GWMW>WefKxaViWusriDEz*|gJUguQQD=K9in_9v38W<@h zW?JTkCjk%%^Bzb>xp+Xd^Dn)8CEiD&%?k48p@M6J)gaXaSyHz%*e9qJ+|9#gP%C#HwT9G<5MFc8^I(E{7nt19tj;<8U^ zUYjcXrgHyecnvGa+ioabXHiZfoMr2jvVv?V#^ln@EuUh_bB4Mjq`^eIrvKD)xiO;^ z=2T8UGiPnxL2~rxZ28_oNJoPoBO|ImFKAQvoFCw&npMTCx@;=c7mGVz)xl(GR}{?zE%}k zNbcefe-U8meTgj!Xv$|bNe2_;94p75`2u>fp%~0NENl18WZ$*do7-#XW+AufPS&z@QnLD+FUj?2pH*?FxasJ?v3%FCtDUHq<;?-`dMx%`VL~WeP9+ zzKnE^tl}PcEUQOFoM`64YK`I-LUO@aVSe-_cSfe8zA8UkC;G4=n%JUyHb4}qN0gzV zzKJYDBWaySmN_lYTvjcI5bfp1snReT-xl-*Pdme%wW242rAtz^AhFUDapB_#&-iH3 z7!*V;>@W7FjK@YdUu;E=rkB!cLL3z>xLh>u^W@(594XFT&EC1lkr*R8+29$F*H_OX z6EEXjJY2;>kd6!_XXqI;-f=~i*SX$Y50$7H=B1CBN6P_7c#VYpzbN#C+Wu}4AARS2 zPVNZetM^_x?!(LPO}a9!E|o8O%)Xdvbdc!VI}nLaHXmzHw9w#fK!Zkrb7mefQ;o68 zJmYuJQDSr0tHS(}<#}O2)i02nVE3mi6KxR4G(P$VhjD`Nd-X~I2H3@P0LE$S!D%uG z`oGhIh+nkl)t0E()y~2`70l~#Zz3OzS&yR)ST=9c<~_gUFMsB~FI0h3S5Ja|kj_u& zoqt7z6sayU-aAU0!l*uamHhs=*7JwLRt%a68uR~R;8aXd#OYql-T&YoVv`^v!&g_< z`|eCCx>Pg`62Ugi{c5uDFq|=E`hZ5FjrgVwM*U0wF>Wq^v^F;1wC+Z_W*9q11#9vB zl4YN{mXc4i2g_s&i2X8YDWoCjaT_~FMqrdvQ*};;xW+GTmx5j8pvq0RzzLnlSg@N1f zakJZ+sB2iuX6a9~xM16+j2FK>WE%N%8l03I&wSf%EB|C%QM<3CsA^YUsQ#G%5R6|5 zzs}n9{+RxWFZb(*{kSrCFmJ^6P3ge5g{EzX<%M)=Ehipt^79R&(x|%{2jlx}MsmUc zE&e-j5Fh;B-AXS|GJ8hOOA{elbuecwAWlXOKX#UDa6K7U-O2xkfccw@b5tg4Z??4V z+?PRP;15}$wVe{JeEb-YkS`raZE1C*5u5fFst!#EXQ-&Gp|SE%rSDmH_6nnRfgb4e zrfIm3zNpl;hQ>xH6tiAhRTEopjx$p2v2Exv=9KCX; z;Tp{)k;q0%T?46^gB18`SWaf@_T++ha-f$ADAijqD1J^@mHP5hb(wXQW?g%IT}73b zr>W?SFp=-n6|UVAXMMawi>-7+?v@*PzbtJXIqqH^>2{3<6%s87o+NCxiOHimCZ8z& zqIvp#ff}J-#+apl>|BAnjN$!-UZHc7^Y10Y8D9tPRc-PryUhlab7PawkBpSQ+bCI7 zpKS^+K1&oYYb88I5V-f~6u=2G1ZgqlS6~AvLiHy!mbIklz)!H1)wJmXi+Xtpu%wd8 zUSm@hD$ZNMFa&Q}fMhZ1FJu9s9Go?px}*mH67S?(A4dIPQ&nyTZ1ie#l?_NU{&jZk zMXMLF-qrDdtIt_BAcxTLwB4rV<2zMFg>KdyZY(ml;0(xhTV<6kTPOmqQ1`<4XVakm z6=|Ld%fM~Pk*DK8($x7K_&X!Oqlw%U8Z~!-eC;nIaOEw)n6`6z1rhcPYXXR$A6Ki5 z78*@OcaD8?6|qmyMm36X&Y^%bq<50(Cb%jZ7x#k*OLjELx`FmzY7d#DRI<`=!dE+e zHt1kC#iC#6zWg|I#G2ymvwH&bI|+m5F@e+0;z?|S_MeBbLxEW?doI|j-=3L&pr3sI zQ5ojd3-^pHLzKNVL~7c6@dzX5Aau{1Ryr-b#5do2o8A~sm{v=?(`wBS9R~D>VH;Am zJP}L6ssD2^nn(-_3zo?Dvd;*F{k?2d`kN74ky}Prc2aP4bv1F&VMPBPs zRXyLRk$u(1DjrJ2qVMXq>ImL%d+a3er>(5bN5-${h8MbfvQJ0T}G zyKl-|A#QxRVoSdIt1t9t3{Zlb!?O2;^Q#R7K78rp@IIreFPuQBH|hP69nvPGyXn$} zq3O3iII$pAHi^M^ZKs~`DBpw=RlL^ zMB^YRlQPr2Mx)YZhF?VZaz+!2DtSE#J=} zDAUNuwvI8vP!X&+s0bc^S)j;^=>vS-F)E5qmWPWh%sq@0K%u3Mv@RLEIy=XEryFGI zhaJ=Ha*EBRiK6*>=p{uCvHcl*cSR;X*XVn=54NmzftTKU`)8jHt08q+Q~1IGdqY+n z>b#^7%yVAnr=g0;+j1dli9>zlXyZMDu-N2MZ(pwRW$BJ4Yn=x++t(X^GQr6kyU@Y> zuoyS?hUXuHK9_7UkmnExy@=8|(`8AaR}kBUFjcrgf`4?y=vk?C4X2(`+M>(29&3%4 zdN5}IV@7Rb(y_GEWwC~%w|>?ef5vD(V#4b{*U^*~j5r~o`G@D%o+zmIY}4=n?-;}}$HvX)l##o@ z7{+ntjAaQqV@OpTQ`VOrZTFZzHEgj>IL;+<4?{&JJjoB6J8QfP;({FWvDiA6F@cP3 zMkbM!G*d5ptq^+o@wk7aPLstCv6Nv=ER7VWkn62_mXm;pd54`Gn%n zI!FUtSwL2DjhgsD`E!EeIU)Nz!l=$TD@soY^XU68K7+K*QV|OD=R6;oVEb#&ydSX! z?_49MwF5rrctv_37TwP*z2qO7e`E1yi&+F0?e;}AmPt-x5>gSWodvjbmI+A%Z>!W{ z7^Q&Iuxc%)t(zNxJjak4XjO?R44<%|)jLO&#C^x8(O(mS3DHERuYd(iC1lxv-lJa^ zCgEQYLio!uSWjc{mD-9OD3Jd9GXSXe21+@IN?)CMEGJ>RLLq(AFm5z8Us86F^VFR7ge{&db#g|j=90#Swo zEE7sTk#fY;UREDn@#Czc)7l^B5?H!;wX{U1-4LJXC_4sPGU|tpA2VvM&y)+~opv+b z&~l(@X~bIebkZrIHs@b*R}L;>^@i`c8b2>krPEklis?^j&Ruqy(`rQg1uXsjPRHfS zpS>(v+Q23F2tPmPb5X!tQ@989zB{%pU21u^0&gYvdtucAG3jEwDUvf%J0lG)r!y!^ zm(z&-2(=BT#wK+krTjUu1#+sBZhci$wx-c{5wRUF z4uE=x^{iM)4uQ+?^}RLdsJG5iMbQz>c5&ns{!GA0&=lS5q6kD~=}08bf2awcRtGs# zLj#Csp~XMxg+S0sPx$4botC+`x(9L}k?4X2!p;B83AE7$NA&L6`V>9%>pe0Jwn0ro zKC1kYMdt>>2s?K7KkV>n$ps)$;Ala@uI7NP;`e|Q9jk!@ySU=1zn{L`*N-(-BLAV8r zkzKik4U(R-ufaYtwg8H6hIl#BY%f^T`;T6|1zP6l?@Y{8i#o^bFtsq=?i877(FV#3 z=?bQh9fZuLw)98LB_GK`&Dz^fPE<^z34hCw4`(W7kzE8+DP}vyOPV4$|0^O3?-U$@ zy~#p5Ek=@pY64$&Rrz3(7H=jU%=l@>MI7ebYE1p<=(JbRQ?ggzZm&lRZE#Un1K%INegt|_m4?QDbc#cErFuKDV#)i{=|^*|b0 z-3BL;VqeMIHSW;YEHdnU8HS4SyiK10dsawkiPFc|o;}Vd;mHvJ>gfFT;YAZ6mv8_mRA_H+U_FI!|$N&@T^{)7zJ1%_-ShdT6fO zc4Hx(d>raC%RoYC(W)8YnNx;^9(v+T(PnA747P{QSA!}Ik=+axe?uZ%;gw&y>cJ5D=UBb*VG zvgzv-O~R>y&Oo1-}YLNPxrPa;+l?1v=rFt=WT%}&2o!NZ)NEEi8IhpA}WzfKWoJM`#7>zKVkAGIq?jr1sf8OMWPTiDL zlFA;?&DeRa0k@lN`96M~Hg46;c~{MSoI=DG58n=KW7J$l=2u16Xfs!*Wv<*L9?XqM zZsTF?fL_HH5ZJ`bQSRUHf^W}S8=!BEKI`%rZ~{}r44C2+v0BNQ36@NFJc?;uGyMqP z`W8phdARE)_M5ottZS~vmv!+*^u{++px_;ICma)wJ8;%>ac-DaTX?ue%8yM@O;9=P zMP31#39J*w{R?S;Qc8&(arUe6{>_~$)Hbj)?^9dq&o)ixk)-nA%*oG8PB=%aaF;DR zrqZ@7H6)}HA48B6f1IOF+R0d%Xl%5F=(oFfUKb^o%WV~7Ud@2x@$(XG(luB&Q%kq@ ztf+?Du13eQi9*a~nYrY?(M3J1ME}f@zv0I_!@iCXpWsu0(EkuBylG{TR+P(M5pmAo zSmZY$C%lnuQCoWTZi+7kuUs-)e>7Jwvm5(*MMN)zdL8(6>%m4br$Wik2uSmw5hZwN zP5X$7^mwS1L&gqj@zK_U@2IdP!~jaGYA5q~sATM@V-5H4bFb1$U76MJ6b3VuTV`MDj=R#qvil3Ke>z zyY{s@S$h*R`vR*E6LS0CnN5UOm~E9K0JtT2u3e}lc~8XD!9o|{x@)`3u{b3P<)5*w zP`;S!ztk4WB2V?eTXcOkJNxO+D1OmLXb+ku}j8XCM?Sk4Na6%{i@FOOfQ$pf=8%Gc{L%bqU7a*u1% z+s$20j z?_XaRcsq2oprw8s7m=Fz-WEa)vFjbm|CNgmT)g}KglIn%To%(H4u;^?!L&JP=Skn) z=lg5_FL)8xJ}%UBRT$Izj2+App<2{~Mxx^fu5JnMlpnD6t&n^L(R&4QIWh1SpXZ9t z`wsNuY5YELG~OM5=0&oBI)jy)yPaRyZb&D)lnvn~(8>sixjG+mW@tfT^~kOO9s3T5n8 zqbJ6b2Z_iggqA=kF4&>NA{AF$c&LalV@&WcGS9=AQ*ro{>qhi*EdNh|NV zm+fDQ)*$Y_++h${qrfKVbr>L5K>Y077_UnlcOTEffvrnmXPN7!NPjL=>!jgSb-mbi zJH)iSfqAGfA%8m>L2SWvJ?u+nhTjQHl8n3 z7Vbi(ju8;cwjP>eAz`A}3C~y+YR#__A^Kq=#V^w_ib=~ieR3*M>C3LfzqKm)h^adG z^bcg(xR@0<4QXjFzl;mxN@IuO^bCL(dS}7=_Zf(HK&rRt%=fWsgo>hw^jM?{n-ri` zban|@cZ&CH?0Cf(m4`Tss@@(nbI?quzC+D})0vJZ-dLX$8S%`CcJlSelRPmz@REB| ziq4}+SM9~_CF52=?^XZy{D+*Ji+6>}0nqLgy*tV1gxLEJ#J`8e2>pvXtr=FGZADnG znOYddEcUcSU1L7+;%~sl+BBEl+SxP%T-_+E%n8pZbgL^_i{4d&tmoy{Po1wIz4-^1 z!It>jy6XXxTjYW(gLhGtjBe2Ml6+9xTzHhp)?=b==wf`oOs>`%y5-U}JuObxBbC0# zJkYx~`Wt9oe@pr;hh{?W-`vrrxWCur(I=&DLnf2%^vGGp$R9mVF{{a%M99VYA@X~T zv@S`Pd>?&3k)*>DtL6-2-by%zRQ&Q-pB9<2v>1ngLrR8xSB8AJ+x*I1VRhwv9ngn1 z0M(&a+RVZktS7i_Vk}A`c9}BU-0wA+6AAph=2>?WYF*MVm*`C)_X!}^F*Yv|@Uy(j zJahycUn=(~D(t`o#J6f@kix6t2YU2lg!7k4>|A2-e^6Uz5Hoz z5?V&>+SKuqTgFq5Z0~73J3c|Zc2ea>;g6E;i9J(4HMVK_Q(r3U?txi_-OAVQ12TwT z%8^Z5Dw5;Nu-?ilQsXP(?xC7635h?RL=TVUwYba}2J0X_+FBW8OyG+{ii@kwTAhYC8Jbo zNO@gQwT!Ykc%ATHIkPM1{aro>MNU|7yTXrt5VV4+iJ2DRH0h*CqnJ2456^wyuCDdVNbU_z|;2Tu!L0SB}^?xw`tX?gDaRX@ z=OK5az&Bw0@S;(oTaeX*4{QeRURH~ArvT2wZrB;-8u#9v>*s#P6DP7UrAcy*!zohT zmDU{*z%BcU>l=*=$*q@g z8^L}(cX9YaC3uuGi*H;0UuUiqXAmkB{yD&C&;HD6b7LPDw?lU6VSW(hO=Y$uSavfI&bq5L8GfI7qQAQUU@9tfztm7M4X|X1X*JLwfqOjE9A`tzfmU zmUX4ChPAd}bu*8ZL`1QwwW^IPetTud#`$&ohA#e=u6xHuHR@-s^Noxd8M=hh7veSV z3HQ!h_l?hu_aEnznJm%*tnb9=_0yjF+t+N2uNIHb_-M3+4LazBf9+@gPH}-h{L#Cl zd)()|A0gPYOJOt&oR}NnDs9{wg17sEoqy4&8xDzrU8-=*h%T%hnXy+w%R9}kvrDfGX$EjZQl_63(c20HTi@#P`CxhuLMOHwlc`Z^P;RK|bgudXyEv%nT_2G2$naQ$UYzRMg=lon zj#GS6l4{-VS%tc_>WbB3&QWUI=d?~k)$gp({R387+X7djn#35XPzznc-Q`&(6o& z4BN4Uw0J8AG7H)t!jT4O?fqOy;Ie7_Q}i32y@ocz~NduljQS0(vpir0M|| z;uCFH+V{%I+D{tr&$4nGD;1)FPNrmoK?$s9o9_l3X&O@|^xE@dhZZrvNPIDht9xp- z<6SMb2zAd_YeTu3G!!Of^N8oLL;6xBzpNFfb}GHSzqEv4M7=tFwOC3Z-oM(}r|_4> zjeIj^`%(O5R*<~fc{?VUPAvi@NHWC0mTR?{^yEk}f5KVMU5R!jZ>dMWUQXwbhp$8Y z82V5s8fMQ-N7FJJ4!eI5DaWBfTPalR%QJOXX^F9&j7{zLMvNb><}rbeoXzfkU?If% z&WS+hVBFM+A-7H5(XcVl;{F3`CNOlL=u9f=?CF{5CI9V_7}EoNz0UH7Ys zi#l<5^^vj@9bXFiW*{}XpU`InqmRUUA|PQzWuQbFeLY$Tps~imj|B7lYoI_OkRywX zz6xI=_P|Re={=aaNRu>Y?Y#Mdhycw@h{oG2^g{vPYkspr_F^CrO%rEKNv8!3(nSRJ zKw!)GK>{WinjhJfX~q7Jlciq|UVJKzc00<@UY-vPa(qT&6ILA{iv~UE%v73&-$^D5 z6=X#720neNvz{Dde&dN^VY?3E?L&U7=2^mjvbGjJ>blTDMOu;81+vPO>$MPU4wNdp zL@=emB}vNsOf*HWz`nu-d#lDNjSu)lg|lBVMX!{7w~1Fb~tZ@{7H=^1d?0C(s+8rHV=EA;JIIqZ6a=+yh-~UqK zgYW;R+^|y|>&cc@q$k&MD+JOMRaEdonTo$RSM60W1%H2LH`BCAZ$!nXZGM=={?`yX z^ck-fFbyUBz8KE0n@g>S4B#ZT|3_5nyKo#DgM zaa1rR{1w*c-iW+fY^j2+Qr+@wD(FDcFJz2cPMDgf>YhD>x;yV)(~nVD8m=zQO+rVV zK^8X<`YxZp_Xl7s=B$egNo1hEUkL55d}RMHLFFH=KgNItZSQJtAm0hSHRM87ber=0rG!WK zlqImIsFpxMZq(3w;pnj?B&jiFoCUWSyF%HVvGv|kv`7*rAO0&gX;^@eeF~s+-mTPdMe|tNTF0ZuV5y9B z5alWn!>np0KG<1u*-m1grKZd0B29{x(sZ@IrG~DuKQ3JN9E)MA!m%H~(^9}n%s(rL zu6nfC^SuCtQ;qhg#9AVABBOwqu(g8f+&^0)g>vAJ*WY-=A=rfJcyhm*5PTXo=d&EQ zcH%$1M>E5;&C?sJ7>4#h0>czeiPTjAi=jlR@&AkpmA5tHU#$FyY)Z^^VMqCXHxUnj zBUw~aruVm;dkZ;`dpxG;9M>y04!?q#xjJ%FXKRVu(>MwE4Mq&KI$#Xw z?`t9^JPJ&3!jp2#qD+ zUr4-;&V19r9~0>49;3&7x}`&(`dPR?=A=aX?_5L{be;C+6cAu*F*Bz()zx& zwb?qu-Ze?Ee6}NT&X{$$Dj*BeQ9AnPmBQ>*Of`iTm~wZ1R9JN^wr}?S%e)=v-|ZD5 zGni_proCG|!fFSX!%~Aygbp*$V~F#74y1-Te6ks#hw@l79no0SD|MkP3scMM;g8=m%HK0~D25=dC3>pPS{9Vnihx;MFY zX}j*sYex1RK{uz}t^%V76qHv%r{4;C7X)o#qp?vm`5qNA!YpHQwkfJeED~?3#GTM2 zv4-QN_6nF?0<11-_q6O53thADV{^Wbk`B|QqAvUfYzCE&l)aY;=fLlcp(;tV#ThiY z${{MsvgKsj(oa9{ZXoz-g(_C)sFs4!lz1tMy(0L6TN2J0FDLrEuNhEXR8?|HESusg z3^TUDCZ=vV!pdgl`UBj?c=7EER>oxQXK~)!4mlb>5CDfLB66F*!W0PU-kXxt%nE{%5KHOy#a1OJmiRIy3RZ+>j7{ev0 zQRRsucMqru06r#cI~iXALIss3;E3aRT@5uBLjGMIDLP9u$G;%+PTKjUUl#YBvOJ%8 zSm<9$_l|0*ago0x0cRT;iLiJ4=k~^ zDiK5JlPZ⋘O7n=^R@K*@;NB30!C&+F26$90vHm#`~Nit~w3+zgr-`shaS@33xjUQXtt%F{+1+ zH|k(AMi>R~5gJ(K7l`tTOMp3X^1^IYiAbZ?a1^j28`&(fJ=mtXAle-1g_w-uD5N&Nr8*gFPC8a7(HGqE$VZQGhmY}@MCNyo{=?AV;xwr$(C?PTU; z*RHSX?Dy2E{r&0xPv1{fcdfPV>-w1tEZ`sc^MqSMkdh|Mqu`AGmitRc`GkpahS!Pa zAqDO;>4T*g?z0ghf+(--6l-U)HGej2iy{N-_?6{z;D(^;yE&w{9Yg2>#&D=5gKlwc zWm$()>=L5#{Ey1!*`~6Hs8WX-CrzlSHh|mI)qKNu3ZpPi)t}Y|9l?Nt4ux1 zl{dtgenyZs%>>`iu6-JvWD(vyjDsCInw081@2>QS7u>P~ZS8*7hJG8j|478#nl4SU@b3yxIy& zUW=w)ievzP1V4Rex17#n@FGN9#VP1aQH(GUyMI(+HyEAs!FBN?BiN;I|HFcM=9X~z z>~bV9hH^S)v^2O)$`|qMH{&N=|FrjyvgL~K3&<2S@~b1TWn2NmrT}gZW zJbHiMTE6CL|93x}{jKKrUpX->-`u?vcM(SWGUl7)2!mbQ)(`DLz88?LFJ1q3sXNv+ z#*UA-;&0^JI9);Y8?Q8gs9#Sv-3v0&->`oVdkVCaJKd@uIR7g5{BF8w1$+N_+?H-HrdRS4e6~u|dtd zu>^&!QYRWkNfjkVnZL3o=(y#3+_+TsLu_*C#3^7*7;r=3`soF8rs;Wes&4_r_v1G( z$%#dyZ>O}Af5^<-6Q#x2RWJz{QcN^u*vE%cR#`L2`c2F<>HAT&l*(pwlcUE}Gv)79 zX^Y`J(yQkj8-5lt&HYN_(`TeAaUsnjPf0jbrWr3s*Cy9~LA8#F2<77g zNn#i&H+Pj5aTT~__)`j?a^wTJO;Jjs|G1vU0ro#qjl@8}Y~F;}?h%Bv*}5yun*dSm$EH6<32-O9YMuj$j>fE9Yi6uxubgmUH zbZw8>VY7tr*cjvGj}|8MYiN;-@Dgp0)NH5qQ3L3^QF{%Ko?hG=Y=)1PyG>;%Fx;uw zHH-@=Rj52pOQGv9Y!aff*l2_;sm%7O4KP-hMZafF=(KX#U@|rfV1q;mFc!U8_)Jjr zewXY1p(pzhnL4`_VO3-nlvhO9AkO5Ojbp{di87{3?UIHEim)Rq)0uS`*hvA1W&Mm4 zyD&raY_`34d=u{^EeYkN)6PR-TNwRD=#0sia!-lL5s2Zc6lew;Ieo~|9H-y4%QN1X~^47-#8rIy5y(Kf{1>W zZ#CGMpEHxGC$3&nQg=AC+POLku9ZNWK-H67We*WCjW*jzBT!U?9l|(=7%wzk-9cnp zkevldoGQXkA+Pl{)^pHZC`z9ia05Cu@w2GRic>Is=lDRl{cxfagv0odoZt+wU(cvF zLU|==32h|k)#!*S3pa-I%euB?xKb-bN@F3L zz_R!UE&<#7rcCjAVcI53DsNr?8MCm$NIn|X(do8VpmMD-isc2h*LE-#SA|W}D_Bp| z4fP>rx}g-~7YD28^wmoCXac0$QZrXwlVo;S6+yiLwaOqMqR|7jOSU*~qa`|V_G;K9 z91gClVzIcMsnZYZhHOp?^h6o1VD2IgI5Z~R{MlFZ-Ik@1yb^JiwUI|IHX#d_@3zV% z8u{?yn++?IP2bh+xJ`AqlA18yBE$^}BViDMA@r;wvkr0krtdPdkbs7ANo0)ET8q?R zHgGZEC;eGiPATrdDlsw!;8QhUUm3uAXcPwM(hEL4LKf)!YfI zQ4J>u*Lldj1!T!2t=3!-Hgnh@mTwO;#5TwLlwWIR5ednOJ(~y27pX|)c%rj_H=1+^ zW5A1tgc)iR$sh8g&?}r>bU{3}26fS>Yo^}aleRn8MN|-y83I#f3&`+|AaXruFx?qv zA>>Y&sOQ{RG^j{Qzmoq;l_OTq02ptw%NW5$LKO9`IDY%|D*7VTbV_lzZ+J2CPU&AkMx z7s=0hC=qX`uG+=GpP2#0ino_4^iSW!hakdOHG~q*TQCX8-ypAIT0WRy#(sXn20vLo zkP8&fDx6pb`1|oam1kqC^zxhNX2uFmbWaA*Bbaws08nl(WP5q#QsIXs*`?N0>>|6X* zlL$5=;QIU|LLHN6u2nrD0|^#wTQk>YuTRElFfB5e5Qh}}(M>qE$Cj5p>Mv zEa`N%W}sLf7Iz=Ps*ujGk1#pv`K~H=4{S7UF5BVWa3Oy`MMUIg8~n1~u%?#4jEuD2 zGE(g$Hz265Eq-x~56_ZQxyXK`jss}Y?|RNh87{aRa12-<#lHeded&HMn?T^JJRPY6 z(#||xGuSgv#?WIK_^3??qx&}l^{aS@G%4#SJuAe+s(N?WA(wu}j-2e;(ojSm?vtwQ z@-&Q&LJdl7F6F&jya(tqRB3p1(5G+KHahkYHc=HdykBz>0lFsfT!d$|(z%P=)=B0{ z-2A}6B@27bHDf@H0NFn?QBhlR)aa3`^m<`iXJ?c9wveGER|dL_bleObYirhNxDf40 z8+8nG=+ivpEz!3DWVC3_E8HtaRw@EYK17O|W0pe9C26xJ)eZS%_W3e-*8%w!rbax? zI|jTCUle)SNh&gM!A zj31fR!Lz_#;df?TxddTJpVG4#cdtJ}0kYp~k7?0Ej?bK==VjDRim{*1oNjBZ%RIrI-g%M3L+)J2hALAP+DY~> zSU1#^ncr1kr@HMhL|Rl&IuTkb`uHnM1== zr1eCSxJqyiFXX3iwcauEBV=3OtsSP6l`aUotHr*?w7CLoZ`bP zZ2h?FCqWRMuZEN1l7gPRqR&I9g;kg5No|O*6@}PYH)bc1kiQmt01>@js|Wn{xWAqB z7+R=DF?W8%nvs%G*wyauC6uIxyH7e z&ubMo-E+!^|6L%FZp!f&uX8#g`Zh%QHjaxqme>XCm*n~4M1d54=%%aZn2D}PTnN^( z$FWZ`zvr=Klz21xEVV2&CmhUboQG|1iQo5VXSBW5`onOL9S-$KUAm4ds!X%IVC&JVW1fK|hL#vfqZBH2db8ecN2$9e&vAV56pW#Lk@uda z+eBQOLM9Y_Nw-(XhKQ_A2meiN?EP#+#%M^br8{oQ?@gqF8yf1z{q8nsiYCH6>ZD#>E#>6EMIGeMi=FhdopcFFCpWCCl}M)| z4W-j@0pX<6*XR)V4@g*E_`}g6XWhafmj-T3=5T?Ryil>Uljp~2`UjZC<4^rin0q(= z-460vzPCfc-ejE#XJgA3AX-JoJI%E|xJ_;+KI+u7g{+y5?RwONmJ_P+GkxgySM ziaok_+8HpCB@!7Jr%Hc#3_z+CXN-bJQ#IAQB_3ch)B@W_`qA0FSO^$8-~Y&w{bA*P;ZWE_bfY{m7VX{TnzE-Ugl6;4E5~6 z>(D*i2c8XbpQ*=*aj)tFPP?w`w&L&!xa}xSt6$vl2W@8St?YYzSMb^ztj`wO;h8Sx z-g(8-H?!@C1;3cS-5boTbq{ZJBsRy?PcR+uRxnAU+Beqivn(d8cckeWO6U0=dF!eR z6A9#Kvw-k4FYeZUQ<_&p3 zMbtHTub&J+=wh%IkW1aKI9wIl_0-RaJk4&tCD<780A`yDb&bE=)vnG*W;n2+FO7{~ z)HJhtV-OBMJK%X&k5(53mjiZ zNiv4Zv+F=~e!YKT`EpEjX%40)0{Q2s@kgQ6vQdsOsS4C(j)xox|2;^srUxiMg;h|d zgL!yDO;d_a9lK%e^6=YNd;6O|?+*su1l}~8?G$fR+?L-t^34v@@3mKqJkqB(F>ESk zzVWjgeOuB+D#EkDj@iTuSJ&_LhbV&Nr*rb;L5AM5*iP=Hm+^m=!-R-TxCk402@Q1sJW)?5>U4oxgL0`IV2bB{+U;+`#QFL?{BHt!9cJbnVN?$^cuGth z)+(E3pF+_QLk+VP2lt-MB8T*i1XhPNewg6nuhuwB^-x3yAbI^xU_ijEixH|Hsk@sw z;wc79Y*9O$y)DEIREo<_4{~qUhgwM$)pWo!+T~Lc6Bk#Y8wh8T+@I2^lpy zDDVt65ZLFYxNjeS7OmHhGT;3~cnwhu?Vy8dL6EO~pgTN~)V4{;p{<%Zrm!G55(aS7nk}_e%jpPx+fWT{K#gcg|Zbr_UwlYNl#h`3W4= zEoa9?cO$}oHTw^;Y^+@8l(GE$_J0J0MRx3#hmhaCp?rbS|5ZAVP)jFJlK+@fjz;w+4@;iIM4gL=m)#FK`f^_>ny#X(B-6_L^fY)7 zJn{KlMV|Tk26Mm0xhJ7#50wSjeAp7F1^YPH>zmEj&s$yAkIzDWe<61qKj`;R_^Awa zMPWvnDPOo-5!)Tr2kuk%=1(an4Alqlr^4O8-;-WuW8IJvF~PWp+Z&+I2FQ>yidgo6 zsiI6u!p^)nX^TR$%4rUWbrnap1eEJ~CjUEpG0^`_(qn3oohpX}lGnq{$dt~`G+N>D z6>BXTX#(~qQ)O-h06;kE^0n_cxX=$BcF3#)r28RBaTqE}SR5WoQaB+r$LRBO%6A1O z8PS(H;a~Tyknw0hxn{c4*4aHcW+XCnw)-JvrpM?q$IgDWTlx#|2;dms%8co>$o56J z%P3LR-@>y(4L={k$wGr9j7jiV%p6J3Chu85x=nQtmQ zX4BO{wK;FHJHXKl;FmitVqdIL*8Kxu(QWBQ+(=`S-aV6AUvers7meunYcUCaI4AX8 z_)(q(e~TUoQg@4^sxsYd?*vhet2HjA*hLKTy2NZ-sm1lS%WR1w%$Exk; zym0XYrlnWR7hUHw5QFVzJd_^oQMpPiatt}fCy^UE{Z|YtzWg^;8lAHV0}To;qEo7s zFGlM(79p2$W%!|-8hRzVib=FO3Q-PTR?C@iz`iIbKvDK#zhfKKw`lkpdlSYn=J-s> zRvfv`nk;~bv~WfRwc;#p*!ec)hq1fW3C^%Ge~d&KVKHTDG9CA8mTGyP8n?$|_+qp+ z*V~;maA~O?0l6eu+{|2>4RQREx?X2R>42lwwAcS9*5Hi{$A^pX(V&ZG>CcfxeS!1z z-p*Xz!zd9f;+)2)s6q@lJC*)k0N`TKNeR#TcG~34iWpE+BzgWU0RKNs#MjG z+DZo;i46k=CpG1JP4OC~Zr9r75sQNPKy-FK zDIcNZ19y2dc5Bh;6HbBHv(sR?I_JDnOo5PEYI0;`{_Cg@oR@0+M*QL36AL;C=4=BPO>4x_t9wNl^a`!^k!5p|5f1Xxkj4xU`Vx_S z)fa1`-Wk0T+xh#m`!5+FVc~WH#D?^SLYy)QzOx#_F87;{4z$HT2&hjyxwiJ7%e1^l z`xw0Fv#n7C=L}hN?K@Sd4k^gGNW&+uw-w0Fk?57hwV@g54@ST?X0*?op7%$^L{!slVOYosRcg}Nck>p5<{&Yr!H5k?I{hm91V{uQ&&eKOWYw{K_oPulYb+}rohl9MaIs}&ow z`Bm%jmRIWQhv)k@hv+hYgfUzo0=eQ8DM$~|3zrtbvwrbvqY2jyfn!|*JW`Pua)m`tiF&)g0eSs<8(wZo2Y%x4rS+kK`G=>_ty!VL?Q@^&1fs?_=P zZZc}fqce@FGP@KZgiIO$kh|7XB6@8+vsDpO6pG`Wu&lLtG#`mIJoQ>+48+%wlVYFc z*_r>*3E|wD+mXTt?mBeBQnH^QnE!j~SBxssVtj^LGO;HY`>o>nitq^|LSZG-3}IU* zA{$K_8H-XBgJpQnZ9A71J}Pr?%I*iT(nFaWJ!fz$Sel>YC9!Aee?X^v@4i$vh#}Rq z*R)QJDqd(A0r%xS{h#Oa*K&FK-Ude0WUw%M9GlDe+7{L{u7pPL9VrOF|>(@Txp@xC}>^T zs9uq$CevsoCP%EC)TFLczNBsKvnWyixcTPmf9N+gaRG2+lj7WY{_OsAI^NhlAIMh(5BV z>Yq90G#d00Qe?zpW1Q&NEy<)O6-z%Xf}LJTT#6rp9v*)l#ATdg5r!r0D;|*4N3Zil z__1a7P5JfHQ7tJY*_6_y<`T=;R|BO~DNT#qOEZf+O9N;-*+EE2fHvt+4<+Hy@piHf z7jm^S_WsCIm&6JDO64SkGPU&6UIbq8bS*;tSIwm8`#sai&kB;dp0hW)3?7nvrqvGkW-5oDly@I*HAfsA$^0H>!`$q?2Q&HG?EYZBmB+t;T z4S;GB$GM{bx@D!!GGCtfpZe93Od*{Zu7X`vv^!&QmuTVC*pLcn(zxicrXB74_-*K@ z(zC#DT0Y&0iym&5y)6{g!SBRzZ&U6P=KQVcQpL=%e^8?F&i7@F zW(5^37EG+I+%KP^TRQ(pksD#*VD=BBKdBi9;uOXbr4ifT`570Ka&DmFu;%-GGTRCy zD1X;MaVCu55I@2uX^KNfg%7B8Z+tO-Q>9(i#?-P7F~(e_W5v6Cd*i`D8k$#?QPiMH zP3J8TU<1?_QuQ_XF4DRqrBfVCBZLIrY6W2wuD%-mE1E|Jjmp5b){}h~Ar)h`S#WLQ z!kHC!6hWefC!O@PC?!N3TgfK>TRt*0c~oW3$txN%A!*maFp|gb`D1qyJ|s#NXqv)% zEp415fxnbaDLbZ&AM`ozu`(04V{-XV!PErLF6?~LdWB>Hlqz+`oW!{68I&dq$51p^ z7_?ru@Xpma-pP``TPZHSM2@e|>rw4w>r9PmLO`%2^bT$#YO8IY5spf4)GQAR)5u?` z)4H&}fdiV7=Ppi=e2ZxfLNNy6-1`1e^Ar)W_cRm)IqOn;9jE&=SvFZ1Tv+n8qp&x{mmq%b8G;Ym#J`yBeX{wIchSbCI)Y(ly>z7wkPs8 zsUo{Z>>k{2NDV=S@A$Y(C?iTq%I?IlOFX6xmJMe%6`rs@8Ui*lIqI$sDqJU_>ePWr zG--bqrsNHk<<(Irzz>`=D8kDQMPPp~k@WH_c^eecz4Q!Hi3#fcz^!jG)}pc&Askox zDN~bVeI+WKfkIsl9qmF>l%RDxa;HUBoFZVuXhZ+#C9|f4R+lAJNMlbwUUtPt7CZK4 zc`9zzoxl}Zhr%h!`z8EHgrx8HEf0k2H9z5jvQJ+$gkawg6`Hvb&@C`xvr|2l2Kh>A zShf&yIaDp4NVqhL3l&+bzJ-e~&rv2-&CkdQE#BCAOB9}SwE1(EyX1>=%}#Sq0Fqi>DgcSec`c~#e*AmKeTvAC z)2ttyml-EOkgt@RMGMr#T$W3Ucz66RCC-+d4u@`NPF*Q?nhl#ogX9YP8oHJLT8j_> z`;x`4LoUUx!@qXzac=al;RI3iz~@DAU6$kuQ%R{~Lti(wsYAtG^SL#GUliiDcEM&? zvlv)_{jU1R)b*?b596#uExU%Kb0NEilrsn8F7~pu`~9OPUNN(9yGH(diAQ@y=XRlhNJvNF@XU0bT=)J}NX7BS_EjSq zoP1p~q@84IP%G@Ve4zi#QUIl0d3jS&;f9NkSD5@3n%isc5AG)OAid+lK{&UM#@(~@ z4MZG*sS!j&VeH{GxHi8OUy{m{?d4a?KR)rW)Hgh%8($4GOG<^!ueW3#RZ;Vl%1=4P z0O=1#W+JM;1v30)PN)GLawPpNo$WqxMI_o0N!pq@*?yqh?P1>zEB1iy6w82S4OSgj zJGK34CEYkYW41I-g>XMp1Qkz3K!SNO9B)B_7-u=RqkX@sG_k^~O2bm=R3XT-G!P5U z6l+T7z&I87e554}x(k1c=hU+@xV46N&78=*Nf+wjUKUkzyb)a*RnqO$C&OfZQzMa6 zO2iLQcjBN>7+pqJbF_@mx&1*2z`AEcaErZX+)edqlbb=W&MTGncVLxOf#dO|5gYp= zV(reeBi=hT1yo{i6k-!O?Txa!nL|AIvNp$>9cR?0n5zqRAw;brs}IgZ^Jiku2mWMV zTkVZbLmffJd5G$fYPlu}FJFHMntn6~j9}kb<>jzxC!4#KBGW=88v>SgaepJ{N}1>; zyPC1B&~V(V7l(~bG=XQ^!L~Lmlu{>`Xr@8YU@)3wg!)-$Bxw=EC%`NQzM0~3=k^Gu z;wf3{3Q~9KpPc#XQWP3%_8kdxZ%Zv9ZI~-~RNEmHAz}<;#uUKbZ%mFR56)X39ho+o zbg+LK*UqBG$6~@p;$v^(aV|Ojj6B~3M6*#PJ!#SL@yG#bBj7#ClWDY*u=yl~+So<3 zCLrokIoqAu2sD2JzQLZ8$du++l22D{t>Rxhi$*0O1vezPAtd7BhmQ^V*aMrzjIxxb zfvi1UZvVE!l8QpXVO7lvk(y~b*+8~R<+!Z=6rS`E17|v(*V&P1;whd9-Y391b4oviXsFOBc$JVV5KC;VZZhG`@}MhNZahya}G62nkdU(KgC>EHxfa;#i|fSm8=$ zCg{m@8UP#z*U)!=Mb@8n>3AUVygSI>;OiTMk>^;|Ru$C-jWtDfuVp;r=T?|YQ~Hf1 zwN?IV`zL?X%154YHoMX@&kmG)iZ1Rk$Osb|iyoHCSOYXJs8Z8&W4$)Aq$GwHkOQT0rGhOzglo44!$n$m`N)lG*`WgO>G&GK4cPB0DsnBhGMqzn%q|b6b8ceEeLn*sX zXVttv1pC@wG2@V=hUoB_(Tq>WQ_9I%2v1gcyl=Sf^@NAqevCTANi!oTzwRAkPnGsi z0`xSZz2w7)1uH{fZyyjh);6U`?N}?8x^tp{W4Ne}sEr1fmTkEG2PIG5YqQx8gMH*# zcxyEEz^B$HVw^jPPxN*70|0(OU18UHw|`V3r)LonOARcPJzH22nCUTNX0g zqlQz8qq?W}>6C|rc5ahs_KEIZ$`Ws5Uarr+K3_jMUZkA78Dh8!K^U(e#!uD5h)2$B zb_eEy{To3!B+y*|Pk;q$6|3p~3VJXZOf<+2& zs3zSlCt6|WWH$m@&G?utUpW)uH0$`Pn)$k#b>`NyX4bP>k`9+)z-_U6;9zu1BDZwR&iPlRgbp$yH)m>MJl6M@zmfAJl9+{k)M)~Vh~+0LkERMRt|Vaz3Z zI@InSTYjd_^;?;(`iB89^o`#6bOC3+tAi~)em2ZkFLyE?THnGwJB*3q2(ONIhfJ?I z#)TdIc_4b!fKL<1l?A}HVl@z-suRz#YP`aF#rfT?b`tS|w3#xDbb2d2RSsE%g;j4r zA?esIvyLDEOIx}tifG?cQ5=CI7(S>}fSQWahRCf56_}StyPPuqr1qsy$X>?r^hPd% zeaNMmsYjccFL%Nz7)V(U?1wmU2&RfD8A}K!gESOZ>tfm-J_0T@-e`8_H(#{+!NZNU zDVfc#j(IgC`jW%7bf&hp2EDc=u6E(`Ms7c@MI^(`8QY3r;!hc*DT` z7Q2Oq9e2p_rVEdy;oE|uF$z!2lT|p**#v&ve48QI3FRARg3Uj4E}ZQp0!M#CFy*vE zgThbL4sSh7jbVFJ?1##B)%4jk86G_*EzC$6WW;z2X=}x~k~J*r-XhAta*pUuhDy5v z2s4KwS$d?;9ozm?8ary5Moh1K_jyHPXLvqz%lE(o6MW{x-n2Lsx8_=NC&1L6fsizG zG2!S}Lu8{0ecc$eju2U&?Lyow`<0^sknOP(y#T$!d9bXm91yx3@Tt`K%4{9-3(fH|%Quc$Co7%5-nSyVVQm9_ z53Z%~^6F`$?d@1_rE06PXZjr4v&}K_e_&gE?bx|CYJ_TpGDFW)j@ZrdEcX7PWEU{v zf|&#kg6%s}7k!bdhf{_UI@4FK(8K+VOLyS1hep69td*>7<72Os%~zkT^%VD1xLb12 zkP17Jg7b%O2w>ks8>^6BVdNTrBSp4UWiLX zOSMO41}UhnzK>Q&kVDJjT~slAJSOg)n;D}wR=HBS5nP>ASwJ15oPCFVNFAe6b%%c_ z1GD$^^T>#ta2O3gYRI?@M_ac;Vse5+{|ixg^dao~4we-q4{JcVr$;-06-Uw6EiA}VBz*FaE;v*^|J{8kJdt2G zBb#^<;o=ikzCoV2V~HQ0XapQV3aL${BY1SfK;E{w{Y!jXD8$jU4o#|hek`Ef%UgE7 zH$CWLnq=g}bw;Us$a_2dZ+}7!9U4)ENP~3~JN}FkrLCP+LFAU7_4socrP2bzC)r#I zdz@anP3PC6>2PmdC zz^}^@8!x}m;xZDGv$WMYqT#kqUY#1a9*<38?m%|CvFxlI}0=rCPxA5<_b zWP8b~6_&^}AHLZYzQI=9;Vemb);stgIj(`@8>IuLR~REwgY8Ut3%`qm01Z)s?zkkJiL zl}$dSAMq}5K`K#M2y*YT1OF)Y_Id46TVgUywOmR!SVz_yG^*VkQq=LDYr=3;SU1t3 zB6I#{&>g3gE-&^ax+Nz2zj;fg>?~a@jchDE&7|xcTwVUVgc#I@byr=)_^Z&w=iy-l zOTrw1jY&dI9vB9L_I2S!mYCeXrtLRVg2bI6t9@n~aqXgZVacMlwQYH$_Ib!=MF|1w ze3@QBrR`-^<>vF+q+Qce*~7*=fRZf%mM!;jKBo1$>wOA%{W|^CSiC%(#|R)9(&$y> zBDz_(KtEeq4Eo?+4U??$6KBMX9hua8{>|cGnkS z^QC_EC71H8*rkd$#h!-!Q0>>68+*THd%Zl4|BD6Iy-*{KBot;adbCXOg6OLgu)T}p zk99D9P0>pP@wa4G;idk-Ps!>vvd4hq1JREOaQN?sdwA^jk|SYm#U#?BXW!q=@hA7S zi!FUNRRwBoGsub&tD;ivh9t!qi|5p!QCsyM8={kIPLT|dU3Y1!Pl#VRO5K5%O%3s! zc`z&?#R?2=m*D*TF&k;f_SNG9UT^Fw+YXyRWkoOR$C4Sqnwa~CQDs#Pj*`zWb2Yf; zS(Ei%@-0`On~(Y}eb}isc>AR%PxyxbkjqG3y)~#bPH-NM0-l&YJB&AAsbkw5iY%D` z2L2E9EVr#Mr z8|J~Yjg*9LMyK1txLgYz(B^Bx&-8FJzaQ(ajSG@YIg|bg2~r$L(@?_%WwA&fjUkJJ z1ySHEo+r9N{L|2h&%} z*=7Zo?c#q|)p>CR?{*z+79XTq)==h%j{Hd>VpL2aKFp6%N@K!|t$=4erw&*YC8$X+ z^opT7;I>PdDAYsgZ*~hrQN@J)kg+9Zg8#^HF7KfO`-kGLw!?vh@9alz`a+ro`@1t0 z$7tV>t+{n{k<)XMlI;RsOBlAG;IuX?ZQ!F?$_&i0ZW-TnCe#atPToZ@lLn^^Dj_Ym zX2VGNcsK{cZA5GTjDE$^p zDO`*w@-@@w*_3S7cv&q8H_+6zv*#Ra7{VNqbn6oG5O$0T-A$|WI~nT_*6^ZkSjq9( zH!h5rDz(g+%sR32I_O`53G+l$$#)IX`SQ3^>~fUwM&W>%Xe1Xn+L_YW2p5|{`Dl+U zh*lTcWDsQr>YOWWp;S-;+S~bux?2`|d32|Nc`XTV+b$ZrRxu5GQhn%Gfh||;lrEo| zBNI)&0FcN@AJOJpuoqo%FPpK1np_x_qe3YHc~+62vqG5>9$Xa8kFv|m(C|(cseDl# zcz7oZJu(+u(iMsw+vY95GmuXx-%IV}M>iK6@{fkNmgR|cWC2)b&7YswzndB4eW0jT zsvt5@PDl=qr+**cChNBa4gkI`Uu&n6SX9&AmCmv%mFFbPO0FSu?>$vl?WEp&E*@7iT+%AH?=x zR7GHke3>~3w;W;VLTHXlg+YV-kSPAzwF08yUnhj;V|9UE{#btcvpHy23 zr?6_hB{x5FG-jKp92L6~eQte`WmunIi#u3Xsn0CKru1Hrbk+Qd~HbE#W^_#JIQafMGoJy@L{9V>Gf6Q#^oA>tOl8SUzU9k7rM=cLNb4 za9>Bh%0Qo8jLj(`@4~ucd{pPvW}JT(6zf=e_u3+-nL3Yfs8t`dZ10j_HGTUhb#xF1 z8R~E}g#8WVplQhfvIZ3fW~Dse@@Km0NPPw5DWtfOBnnk4} zl#)*ACJ2mRlqR4K2=)sZ0&7B@NlH?~%aL_LmW zzoKku-v=mWV;-iMv+8IPHJYiSt8NTmDS4|mw-|oH)_3W>rOwEo^NFL#REqaiax*`Q z?dJh)4CNj|gb(vqMuoa;4N?=j$Z^ox`)Ez6U2DNRTg82f*np*IG~p&s3yFn5q&2N` zDgYw3ulY2M+tct=SX6r@t5M0A$u(Zo2q4?7Xgz)(VzTjf&%{&gww$Neg4>PgvJ2fDQ0zI-;(S7CV2^lW=XvJJT3H0B^o?}vRk7Gp4DGSb z;p&J0vQT(V@S?=e(AI=dxd=%lMlr1BHj&?xD}B?(HiMi+@e_z26A31ZN(YMwBz~&D ziv-X1U~&nP_4a6m`O zN`O>{4PI$h2jrqet_E`Mf{p`Bf&{bWFuSZ^0L>PdTknsersbhY{9kF!4iNJ|eBsvS zk|90p!3PpmQ5F%PIa0q|3u4R?r2-vOL_$ZOo+8j`wSrn+>&huIpSzy{{nll5H!V>Nj7v%_}v*dSjhm2bzr0DAoZecdtscSjB=N0^j->Z+dl z7F4Dk+Kz$zVTa#$4zX~K45uy8M|+-S{fG@kEVnPtsUE#L1yT+@L7}bV|NY(H%rC`EMwLczJ#~MCa zFDy*(LpIpZ5zLsP!lKB|dcHle=)#wR%$ssF%$PB`qnE}t%gHVo=r1cjD$%Nz6)7EB zdoU=59T*}YwVxSzxQyfhkndrwl^BF zAF=pmU>i-x{PB=z)Nx?1Nc!f4CF;0^(!ihidq|h_TL%o7o5RLdc-jlh(GDT|u6{_L zPi8EkViiJNnw+d715;@TFR*N%lT)u1rp%3Ct5i81bnX~9Ce!33+jS;2j9ybD*a6Q~ z2VQ*B>{eeNsd>)X5h_fk>BhG;-cc14iEr8LF#QUI_e)?b6W?Ot6h(8->G^*cd&eeC zqbN%|ot3s#X+3GX(zb2e&Z@L+JZamuZJU+0J=N1Q5pQ?I8$F-yKd{d^_geefYrjMC zBkMnDL)xLePqebcmmSk&ay7^apCd2x1w4kT>JiDVOr#wT zMBLNs5q(|v{8v>n*U4lN|4%z_|D$aGr8WIv5&Tm@-%x8@d?U*qbxh8ag#9bb>{Bk~{>9yW$x zk%^ylVFzql^R*znogg0~Fr2JKiwCH_WK)-t@2?8LS*z7@Wd> zNA;aS`pHAYqZ134IgScAl@Lj;tnp3PFdcq=%BF^@Fr%fmZk}^bQN`)0+fM8?i(c}O zXikR26WvsMOm)T329Q}-OUsMl!FPJSRep~}EmSvQN6Gg5liaxTX!dX7-XtCq&pLU3 zZ#Kx#DC;bMWYH(TP=!18pmjr^J>GwDLfaBLu3U4AcVdOf_0W}85QXJlUB0t+EMX}6 za(?MFr4%ETF_)*y<7g|YLSMp1Ml@Pv$JG}uTUwR(Z=tETGMYHNp+@fQ6dj$dwQI`u z5H$|bmYJmEB6Q{>H+GRrz$=Nlz*6lgf;Oa4?*=eCk5AIF1Y3rj^;k*TyBAJOZ|7Nv zla zk*q;#`4)Fdb{qYMZoq;bU6lqmxjX`e@34W7YnW&Qove8aVqUWz1oZ#|mk%Fd$r^D! zdkA=1b|5$(gj)@p88p?I)7KSsHsn^iVCGiptmjsmgeNG&aZ>S2ej9`*xPwh^=pnv8 zN((7VWN_^e(ur#{7AZ*h!%?cm#`0)UKnws@p_ystC`44mz+}Q0iY1hwjK!d>$&x#< zVAw&U>>`!zZnP$Xv5)kTVEAA?&p)t?JUYWTyM*|ik>HqC;3@}J&Ho1Z zuf>gsTmDPwUvUTgGlrP|$KsZ>`v;WS8QQ42SlawwU#Ln&S03#j_P3B?r;SA&_9HB! z{*JspU_6p2ev#eI0@Sh(Iy+=b(%o>)vRHdj|F9tg+ZbNp50PnSLmH;6gv`&iuIH?^ zbnmOlt_I2;KYVbF;RUc^N0wM#in|~?ORtSy0$GZtwu&h}aAl}soNcYgXTgg%w(a^t zE3M8UU40Gy=RzAzKL@hPW`%5x=ELo7hl8J1Eu)gAwq%>9cc`(V%H^@}WxAon&zlH= zRAKteh33a@*v#aoFd_a{$l;IEd)W#I-B|OZ3aahAn|m~~A2SA0J_c!}SNEiLp}bb> z=k?o;pHe@CYATadX0YGRncR`6?oALb=X<`+_$JflQIAh~(pA8u@R{yCba z3;c(41labi*vB`63E7H z>RewxP-nh0zI{XS;}&+F7VR#4P#Rr7=ZP{d1oM)~o4Sp zWXulGn5|)^P%qCF11*}xokgO9{KP^YLL9+;v4_x$$7W8v<@_KkJY@N!Nls1CmcZc? zfF-I0hoKJ97cy8?E{biDA*Td_S3v*s(ob8eyC1%?@$E4web{U7W8>(iqLpa{J!62s z`q7_I|FuH38#Qf){#ED(6bK0W|5%~_k%aOeSFoYEsf3}OiOv6oQ-LbB|Ed%@+ZIlo zb~JK4GB?1!F&AL;Vif<6PR)R|l11xDv$;+=PdLA9Ygq~FkrIfv*hjh%$2oqIqAoHz zO6X`y2YUYX{r&m%g$A;`Yl#SDuofWBCDCULrv}KytVG4$MDCu=gqVL7O8!+$Mj&U< z2=M8~VzbieGG{{~M>Dl#sTofSdre_1I42>iRZAH!0vNaf7uu&-?+c~z01BJoHgfLL zGU|;glSdts?!46z5N5WgL_au`K8lD$-jXZ36q$FW;9}zNURly63H8VAcr`kZA4r77k=S+2Z>^&sx_i^Yx6R!u_7L?uU&}0 z`EW1LuV}|tOOtmq(iO&r%I-Oc8ZhX${_Kw1h(~=3tCzEGj~wB1*2-k3?6^iAb=T{L~qp)IN>#Df4|SNx0)gDXZW6mXk9K; zcgeN>Td`A}q4@`1ug#-x|C1+Zk$DH;gibBUUF!mWi1$_aGjvCgv=d4Fh9`^aCjsav zaw}{B6%I@L!Kl2;?aD6q9Y@^EPhgN^nox9<<1^+CYxl5f`+(|HYlb``n)+XRT(e0o z|4hb6#J}sHLkZecZMLl1S9@}Bj@v=)AnpiVp=EO?Lyl zBaH7Zji<3Ya*An{Q3cG^TLp>%_K?Bwy1>g{7Z5@}9sv3m(k4JIppp=!i@X#yI`?T| zAejIDAcRn^t;K1sH$(e#c4N|4w|qUxSj0i$_XE;-xA%_swfB41^~d3FLT)gFCLOeX zOnj=5qSl>Wa6uj0-2>vJeL-+iV*Ej3@B-qb17h&diMi;|K#xK&@tacr0zqE0-CN}1 z12iap%Uxwbd^kgj>>qmzFHu^2WokL972twCa(n_cKlMv{m0ye?yMq|icWGbB-(EBP zYoKqVVF7&{2tKr7_7z9&LR5Y3OEb^#J_11)oCdql#EW}>z`kn1)xPGkG_IefK$UBHySE#!(AW=Yf(LNLc>JENf2+{(*$6WG68i2b=GvwQ_Cm ze(>2$$;g8Wvtb>rJyr-rPjazq(@9Un7y?<|-Rfjc^V61VD zZJLP~t{VKc*x){LnMQP3|0&p#Q`p~tdHmC*gRgAWz{`8VmC1Y6eG}qVDkK41YUv=2 zultw`s#R4#=(o5LQFOkAH(hMRQN30|;0{Xc2DqA!FTjj4u}wnR+lRKfWde8k$)u5` z7iVcPdDCmF%PoHdM?%AIya4Q6-s~?O^>5B2d$)IP)Q+v<3?r?ib(o!vnd9OIXEj{v zBc5BTA&ug`!OVRC+T`Tg#f|OEoh}BG^46wvlBt|1KrG9gQrS@-nUHcXB&lFuo%r$D zRaEx{?X|XZKKN=3c0nt)V zt-j^tKp88H$a2+TmUv4H<%b8637#;Lw!~|PI`g(yAu8vO&d%G28Bg>2ZGQ}i@ zf2q>l-7CW)tS0QCxPrCmFo}c$LVR}Pk!!XwG^@%Zu%?0Km7ALuiPT+Y&zY+sm&KEP za)J-&%KFJQHbu(=@zl=bfV4W6k|6ujje$-BkZG19ne-VaM<4sZ^I=LtbV(i}LC*G+vfW2jtsJ{Jxg5L{dsTLkMTN*?vd`8B7*`SxpjY zSxt%wSuW|6dI`2Pr)Dr!X#~7NYTCl(IBw;$U@Lhx@(61gP3nr1G?M5{`26BZwGDIh z?CCBz8BNj(8J()M%4AoYT}p{`85{e?WUKYGS2|+e>}kh>t|)Wm>3C<&SQ#$)X%3W; zIO^#7q@wX83QP7RWICt3TN<({N0^zVRH{ouL`J$Hu#~`p9Dma_)7?)L4cpk@~iS3Aa%}OGc zx~qO_Fxn%XdL!AdLAFh&n$a|rHS>Zk67rss1{2`ewpR+PdDxm^EOez(m-57VxluP z303A**+Q+7p=1FSVuE6!8dk^Q&eF(;&iMyKg*;6Q+eN8#Rx%n2OUyehwASIIk#875 zNkK=6yRZ8>g`5u{>okC_wbbhaK4$~XrD3!~DZ0aWns1J9L@4ypi{X9!b?M+;CVp9& zg>VpRJ@tz3?Q5Wwu5-fK&BcYYk}6*svCw_iIj4r(-|nd);^#S6&hXl3?|DN)`ozlQ zSw6=&v$e;noWj_fQ5Hq(Uiw*t_lF(U!&cqDsY0N%=Yn&R{YbUU$T@#pds6sXM^Mz%d8(U(o1@TvQRV{dkq zx5cC;UO6=6>DvdF&PH61O`Ok0W_Z#lh|DT6!2DFjmX)XBA@?1!$M>9lCX2^s^J(i$ z#yrX~MSZr!@+Tl)+amE-{=ZDpe;pkHby$G|xtHid=66dLUn~<^<&V42n6rHRNIy_& z9EGEm*bURnqT=MwQzxEf!Blmu)7Jz33}f>4I>7jH zML{~#OSt@=phBk+fpOqEGJO~y2)8hmVj#yx(2J0gqe6mNzL zPLFBAt9JeC4Yjhk?75dtuEQ~b9~B}7sh}krz1b587eHa;YYUvjt{4*9-&>y8Ebb~( z++l0FO{8fHy1(I%Tpvl(yt(uTqWv9qvHz}67Y3%OyHarUgMLLB_(7Qq8e6tLU^jLt@hjGj^*5tw1my~VG zp;43C7i~MXlldEFc=JrT)A5F2*v2zuYmx5>e;%K*7uLp#RS!;i@K5RX-`n-N>Ogv| z(_f4{!f#{6nX%W+`{+w_pLSt6%C{MTg3XZF><&Tc-Pr?;c%4S9*8aNL#ze(C?`=%i{i#6YR=iKg97OhDtC^=b@P0AOk zu^Y~$ZqwQdXffHu+tYjd&_)^0aLq~MHru^4+Wq&n z+|_mAyTMn>t3(c7yk34}`lY=&CUr|lfORc90;zLW6wv4dnP}&$>&wX)scI)|XxNoX ze!_2Co5Jd)b}EH?|EYC4UOAtvSw!0)rEZo)$S?_g64A&gk5=8nq6MqtX^S-R|wL))NIG_4%`lZ+SVzq+Q7ZiW@2K95f9Y)@wnF zxktR?NN=;mvE|O5FfX-!T7P&6l^3$562E7_-b7jR_4j{+sQiWHr4jzQ5K71(Aj1En z;QB973;%!N8Aa-v>gfMgHNZl7VdMHc3Py#OmX5OHhT%nw{|qox)MOo&@bZet%PaAo zUpPqc-Ff~bIwWVHI7XfnzWXcerz|IDr^+$F_^_KBD(vd|Sat39T%p}!_4Rr^(*wdA znM1l7J%Mu3pZufRh&g~-Celodb!BkXboI$&|ObBJ+LtI&|TmUMI%0}@gA=wwhd&W9X91yp)Iyz@IHm*~(HxE<=BMlEGEL72@#>Eqv7(Z_3#*99rF$VOavZ z3V+8%XWlOQ1tA!{8P_Q01NJ9Gdg%sH{86Ov?fLZvN> z@+&&)BpybZYxI|3OrF*Jl8%tOz4M5Y$|gk;0He0ESbuy@8(y9jTLygn{R=Bgq4 zy;`-yUrzkEF7|0SQWeKw-vxJg>zU)W~-O2QU^^( zbG4rxpsv~(vVzf>myf1PW0cPoP(ZUis7;eRNr=&9bitLSGRL4dlPli0L9^Z00lTey z%g?WHOUzF)Ov67+3IBdqv22SQZl}m>Gghq~u?b{Tg5_7e#o<@Kh3rti>%h<=Fm>qJfxOQYm`uc+$nv*++oV{%(p7q^~Mm`71tV#c9a6G z95h_5ix2m~hxN<@?oFm3oaqcZV*`#;h|7Ya-Zo*r;Nt5{H|D z%mZ-@ImnPs_erKK(H$^ZE#t`w(9)mBHsjETlqQ1kH}w!+TY;J1RqRqx$fX#sX(%eI z;@Ut=2^F`o*b)h5R^k#j;~~iNr=>Vw!jz*`=-oH$U%w&a5+iK#H7Jpq#xLBBVb&{P%rLuBhpwY1O2 zKS9TW_s5^|SH!Kx1|QWtTCTv3zkDHywDxR;=;V&GOGL6$Tj zIP)%dB8OHINf?LV!EtwQy9|1Bk59ieACc%>T(@PPUB%?fzZN4Ea;5<)1`J)%xd~q& zMO_8=NVV^aRlWR{YQ#0KODBDPXH6bv7DJ%Fo)Gtu1;h355EET_9)(%w1;1UsUJ;>A zspA%($W>eb1x_$|Mhqj0k!bT1h%@ZhqT~!)LF!=y8$UYAvENw@bExq$<*bR`H@z;- z^zT=fnD6dQ59ZhO0~+ds9GH%Y_kHTfp7Ka8wxb_PsfYctYsDx2%5q>3y|?F3&0%Z< zrNbKC0eNmq1DKYM>2cM$@qXV`e`Q&($s?5gwngT|D;n7Crw?6m?~N^WmLw5eID|oU zg~3sD5XzaZtLNO+PYQnz!B|rUS-mlLoE64IyYlv8hL|qKy3-XHMuFM#Vpt*ai6lOg zVG&n2| zklq7>97qx=SVWs>l&2>rUs1cz8dmNpIPI=vugDa!;H6lx!AF~a3epdb=^MIrp8Ec8 zQ~wWmQy{!A&t8&(eE(M*?nutbSI$-vCv^YexNa7+g{biynbVY8)|2^crVE zO>afvSuN@iciJB;!)HsZ-lO_a;xfXf%biv{l06n>O<~MuwvGV>J;fyL&Vq<=Ma_NH z8a)RpB<&5n?2j`pfnR5~C?)>G8n=VxH$))pZfV`_>G$H^tqk9J$)4?|T^8j%N_g9V zc2yGj$qv=$4jp>PB<&{X3CoFski%VnqJwj5=L%UTw&m zjDkhLto0psRbi*I3Y~}*mN30Q#P76@WZ9h#L!S4m(?&i zi08634`(snJ&V!VPr^^QJ4YTntpj9>tL<0FiW$gg`mp72v|X$3;7SY94d_l z2PW!O6)iR@G6Q~$0dj=E7ny1cll<1(WT%9!&)0GFSyoCaxUv~Hq*WZ+cr3+Rww)JueNCEw zF`eJiOz}2T0Zy)ds`zltu?o&DoEtd1*0oH4MWi6P12FD=8`GPqleL6*jN3vW=H#O< z7yb!O#qmu$p@xeA{L z=%X*;?(G58V1&`w1N*bc*xP0a%shz+1eGOK9l=slsO0z)_(h3xcs>jGtzT1s^3aAK z`hudy7EsdFN)UCFF95QguA9004R%MJNV#EdP(O~*-xuT%(Xw^8zt>0 zj${r=g5;)SiCxg774!BLF=CIto`v>?WwU2^sP5V2Mx9OboWXV-U0$ifZmF}%!al;r z_W?Z2U{h!}qzbh)h4osMyt+!B0yS>k^=g$q%|XmJ5QuN%z$DD{im03V#tP5N+D$%r zARKC5U*b5a%D5L+NQbnTqUyzHQGz0Ve_|4iS~ay=w~o-7u^Q5d-oxup@IDKxzg82G zf3?s2T7Iz-YQNvB4NgDt{)XoCh7FY-Y%5${x|VL$8eZ8#*%@-?KPsSGsdd4e2(hkc zD7^#@-R)^EBD9D!XvB;9>Q{^ouvOZ0g@m+O-%2nGLs*ecEI+VBescqX2#w(u{Iygu zMq%9mQ&OIglJI(eI*T;#Bae_{;%tKBD;VDMcMicCNdJtJopQ~}V)f_&I?c*4BU^hi zFNAZyAwEARzMQkdsU6O*7_SdOxZhA2Qp($@6zrMDmNnUHp8gHYr-I};BYDc+A|%6~ zYKb>BwplZYAJLN1rZ&jOOzlM(Q+vWPHf3M`En=t|(M59RUqT%^-TzC((0?>>g$$kl zlj|3=cd|A7KXm_;CY*=%lJEB?-I#hTtv~@82l9SU+-U47*&nn+WQnXGPR{883StN< zj=Sgrk(4MhXdzGZA9TSf;`*IZ&^7?=4)K%HL{@~_YFg`-dbD-(`C}AEiUuewBSZpdTDx zlz-fBogX=i_V|L28i^Wiy4PR)aiguk1g6!rX+&G)X+HK*TizZll*+a<0zBF@ea`lt zrGL2?#pSA(k0z-=Bgf&&T76o?bI&Jx^-RXay=9eQPAkQ(Wnla1{B)L3pPoOo5m+Z^ zcV^=f?>a6^5BW2h3VG&O?FD!k_teuAxizVyFchFq5qq>yiZv6h5KScavrJ~M$*LJs z5@1KI03^bFIOXXSid_bcIW?IDtJ6P8iTJj>7lGSb*DEDTL$DZy=j~0tX>4lc$RK2- zPq(Ja!C7@Yr$bLRx29uTGzb!a>D$}cM_-OXjj{Cb zR;#6GU`RJ#N(?cNLvtqv3Y$qqb5D29BC-=Duk)bCnvOb~NX`7*z&S65$7#|enjYM@ z-KuL;N@9EwfHKkfbYlEJQf>c`Bbw zVJZXz)1^>0@^z1@W3^$bQ|X6sbbX|0xJ-(enWTqQLB(C9=$9uwI$3azb$+UyD-RFR#PvtpPbYT!-&Ynn;H zq$I}^c}G4`!8~)tNOwOS`5N*Pncv|0+gXSF6c3n(G0-s`tyT*r1}=$Ai9A=(DaC-n znqgvv1q!#3I4aiNs>#Vzq7l<*kQ|((L<~A^@>H`v{$)wCR}IyTQ@3;~v$7_u95c?)&L$a(_OLV>X!@8L_<|X(Yig;u>rh+A_X*RrWCYo6GWt*~A<8wZnZi$u4 zdC<#i=M!K=1W;ZZDFiNrY2-!eXB#&jfx7E^O;o6c!9GZ|NI)g$sLTAr2LX z3ofOJWsYuy*Q8)ED4*scwr6PlM%oVg7XGFzY5c(Z7Xv4N2SZgc-Hm<1vCS73MWCu# zI`S{rK+iG>w_UBq#HY#6oY8N0+q*)Toe}1PQ5oa9_@M&lAyXbv(+k(L?nrAj4N~^1)~%-(O3}lsjU3Z& zg%ADn!*a&Ih#y2lm2U$xLjhpE5m4{%=&nTAVGW>6IMdYAqf0@Y4N~;uyNdcg@AE0J zGAVY=N2l;P!w_thI59&&{5A)nJBB?QT)n?2N)EM~%9{+t?l0c|Bynkx3`v^q{=;%P z_DUVx$`9N-ou*;q4sDEmHOfV80%&vcr6Z@c+yk=w442b-#cE{MWFo8#m-Bi_FPAld zw#ypL2J7ikk$iQb(!}Yr#*FI+fA<)!LcaO5U|Cc3lqc0t;$<84hFOI&^?fdL7Cp}) z%@bt%MN-8Y{z<)b_1uwGTMUAQXPHE5?C4{ATD|s2%S~vzNU$M)!K@(adzD`%E>4}x z8Yy_j#S)JAyUQAbI?Q_U+Riy@s-P|^te{eeOPiq5g{g}qT=*tP`?b%8&)RY;+s;uF zMF_Mu&`%-NkSq7#&nA_Ij@hLu>Sp1DV4G%Q?3q*KX8DAzQ%B5J*x3uN7(|tF0SzDx z_F%-NP57w}hd_B9ilr;nQ#X|-+0SY?G?m_cj9B+7#gE;o-ABhxE#?(Q_bOhVxg*KJ zqQ6-BEq~-OUadWLfL2t*lS^@(t|T5Pn8;oej;QabFBDQBJb-; z6cIc{uR3^l%AwCmkUw|o;mw^0yhHwUUKs@BkD0r?R{F?+^i@itCQ#(89KqeyQpBFz zmGTx(9Jq+Jk7Buc3b8X@HYo%n(pEi5<~Iop>sou%XFa*jaO!gjbK_wNCaNzdpsn)#kR;8D`RlF($LuuuC~iPtBeF*v`YWz<1&t;A5WG-K&sovI;6mo7Dz^= zSJao*RRn5Blvy}AFs9dXwI^IOWHAlWWiRg6Wmc8N+i!QxM3f_XZKD_G4XoOl$A1m% zDMdSsWpe0)D$8mXrzLwgD0D<5C#@%|NonZ_#mw7E!VooFwok32m6RTeol0)elU3q% zpq@shhb5HI_M_RwkJ}DOSvb29H+OfH9Y)9eIb@Jvvl}zjcg4e@cZ4~NKI(EeoAc=_ zif&$!d0w5{hvle`b)>wA#o0^156)s&zU}5u#-9||dJeU5r)=wki<5Eh#-sg9ZaUb4-OrW9BGt;S znl*GazMi19wo_PP4*x~WwPpS>pm$iJ6JH;*)lN7~OWcrBEfgYBdBd;eYyR=-_MRUxx@h|c zN4l`47BPl)fh#jv>VV!#vo?S5ju|CLs;+CRnixKG{6vqXDFc0TuXBKuA3alZHB8-TK3HYBJs1 zHUT5J658%d?K`8GpF^W1bQbv6&)WKmeUSa^FhE9#*&Q@N@^7_!O45>S%$ClH8n+dq zCgLg|k-8Dj;#xi!j|0XI&#S6`1~c#g3or&qJCfPoy0TpBcOI3T*$1j#qMz%$Ppk!dGlwhWxu9$2#oY+^w${K_N{h z1_V|PZPDJv4Md-{FUtzjS>S~&srq`OmWvi)hD1G^L23h`pJC=@HI#Nu9p#n~TDp*y zUgEoWiGB7ncEZTMkd5VxVGstf3D_jdOczg}nHm^R()IPP8PxILq7X4r%G0Wl622lC z&GBz>bxnLBi*xUcBhU^7ft!zf`)Ps)E*wOm^s9?b@O{|>r7^AYxn3oE{1F*g|5(3c z#YNFgX)y(&TkPyq12`CAWBOnB2(tY+cE8NZ$3W3OGEeRgBIU3u>G3L#Ps@7;nCm!Itu}dqDBXZ~s+0}6B zEzAebRo{6lmOy{W4LjlB%nkDPz`_17no=ZX5sv(rsZY%a%Y5)avwl{HpqAgJh>V@l zyaNlrlzJx?LTm7Yi_uh{+@ZlBItfurfu4|tl-$*?=t)7i*EYB&W~^u(cdS6_5RTEE zt-;^OI|C!s%71ho*Sbp*TWCig`FGy(1IjfX~Z zRt7)D=O-f0nJyQ+mbnd2T;fKXb@-vh2K;p-@zmfQW za>0opV^0EYmZcoulqNz?`Q8dE?*Nkn?ES^XEjo08M)X_wJL|_=5HOVO1B&ZK_6tkw zm8$;+Xh?ZOpIkR12aAz(FtO(F4V(O^{bEbGc?5_c`00aI&`=c`@8UPOl`pg&@0`b3C}^&^gnB|y zD5oL?*OP;;?FsutGv#uDTsEFa&R?*UC~XMt>?pP>q&z6P__vMN*(f4<{mgdf3K@Eu zedNw5*xC5I39$GhG+1uenJLILBfC8p>{1K_`@ZzdJ@rh4OLBnZh1F&l(7$Gf8k$mXf7b zk*Qh}jFJYGf~12#L#NrkQ}8{MEu5H$@;R`9*vr#fRn18il>VOn<2rBtGYT8>}*&o@!8~>+LKxdH85A0M>7Bz z0ZaO7VspJs=D)kgvweSZ5+juXnI&Zh&!@i%YN><*49gGKdf8~H@}$b_il5t>pLClu zxofLD(_leM4sK>iXrf_m^^7U!Bbcff8V^2kctW#$ahIlZf2sRDdW$yWt>s*b{bgU$ z&A1E>C{J6_bwjVvcfieOJHRC=S4+f&NCOH_z8HTFZu$tbCf;fd?I;RQEQmj-S=J+>UdE9AC$AS?O-^3k6?asWmh)&ec&gyE|Wv+9^$-N5B=by^Tx&NW{Z9umue2oY6%ery8@L{h4r2v zFSCX9)Q+Jq+S*>Jg`I(eLnC$F0Z=_3*z6*muDU2YFg9eV>LMQ@9H@+7GcD)UPX<)j z^Etz?q{y)Jkt#TTpPq2PVyEg1rF@|tEJZH^fKA{0*?@G>F>;nP; z+b9ownX$*2r_ynrHxl5T%sJ)yz55>YW@DZfaiTvaj9jEi*`?9mG-7H}?j5TDLw^7R zxd3q~wOzS3>}QA1PbX@;KTDh>);C!$`(%*^6*geoNfWotHoe~UCpitI+>TU2~-$KQ~@*Of45x3KuvY6<*Vww5wqh zGh=G@>)o%a14)~7Vo0nwK2fVdjU0#aeWCquWlyfy+^hT@eV%On)XO8~n58ts>Vhoip8Z>u$@`TX;S+*+;=>|qLkm7<|+`wRgU-%5d=IF8W zgZAUhuCrFnt%9YLPR+v~GS5pOTHDg{Xgq>sf~?h?)cyk-XBc5Ro+fErI1>F~I_zyy zpA-$Vz9k0UYRg#-cbIj0jkY5T;C<6$7{012%L&COUgeEDPYR%fS~N z-*IXk*jI{&gePWrc(UixxJNN9n}Y)ZIUd)&AN)W&p>ZRVAS*vxU! zl1SCVgsAlx^T#K^I>bwsEgO zYp(S{m2g^rfukwKfhH$^l^6ELF8kDn&bHHwhehXc$R`yde8?Gy=}y;ZDi3;xtDU6a z-aK0`sxG-qJ&d^S?k_KfeT`3Dpl*0Ey)wlR2o}!?zgGF2i7M=L%DWAx?v;Gi=@ded zweY}l!;UehCfz0HX_Vw~gXXZ5R&kspAX3etq3mI+HOHm1Q2%EME#9t*)evjvpqkvb z8s>Y@UPdPdWcH#({-d+oI$lM$T{D+2+Q&oPL)7CSJik~w4G^g71`K7s`y$}&ykEz_ z_Kb`eGV~*GyX|Q3|7`>P6*cuo#Xqhq`hd9GvM`{L`U!X6TpuuAw`NY5hx>~Tb{E`7 z0@v)ACm^opHNPgia}^hN|J(vV8}!`0fQcOo zT?_@L++gv*h95Sxg1o=b7-NQNiaQ z$F`mjSf{?qf=z^ML~z|Hxf!ldGgMfr+!Bz*P(p*$J>@eJ<_FBvLDj7KYo1DETebc@zG4B|nZrq#*cOTt6D}C;}41AzAMN{-q;=u$P6d{@%W0)drZa7 z?2`TI<}*16DF%p`G>BMw7Mv-F9mXGI`AWcsv|w8-wD=M3DFVOxYvcVVbAfvi$7FpA z;~d@>^nbnW6P54{gd0`ScZbp24iY&d@|@19}QG848F@j#~pCHwiDH^tfnb=O)@(A^Alo*4riF0 z_~{#j>`ptKW2+$Ab*d06vZuNp_wl)|)laj9{dBv#cA~U3C&jvG%MY)nh3?7;ms4#A z-*yw5J<`)%3iG-aESOxAkGCpkt|0|Tfa`szIRFS33I29e=a6flEJBFMON}fok2Anf zFY%W59Ea>mn~CX&QJqv~*-P`f`eTaegnlLk{UplHEDB?j4?raVoR~^gl{b*Kh?qCr zk5?-*>1-(0=E?OlQNk7PnL!^zxei(y;&-wjOE-%cDc@2umlY1f0VF_h z;ZQSO0jcrTmVh0Np{k{trIcq_?g6qN5`UJ_SdWgnh{@Q`mLMQVFPUKZd&q__BaG9a z0eNk_xj~oxcsW-b>1f__la{OgK-{-lSz&=@>(PBegx|x$xKD<03Ct(7u&wEn$+E9I(Z$xHzW4D%P?_yCYOb)A4IC?tjT}KgqO&{5}xC zrQaj)u%c>B+Cw*K;K6Pu<`_U5<{H&L~g(W%3H1d(n z={{1e#JMXqXdpZPEn>BM0B;9*sr;>UVEdG~%(4i$)o$y*4 zf_5T#5TNeVpwlx@AP^TsGMc&a{$dHO9&8P81|P?ub`z+9sqO_2GXm-GDDf0}p@05~ z0RI97n-t}Ai4Mi=yE2!s=0*@N^X)2fd>IbnFHYr&Y>VCRq+|viBe8)azw_?|iGW3P zqER=F^2Cb-b#&B?JZB;NQ^Y9~{uOF%uNoGsI|JLuM-mNpV+u#mw{bN-H8uEH*%e`I z=kklEBjBH*;GbT{b7n`U3dcSbP=Cb9$lSw6?ekUpiuUA@?d{Jy!bKo#Qwz=T23IP> z8aL=c>77SRA87%@l@D{GJd$5m=7vGjy4@|{dVo+>Olg!F5fG&Xj0y2nFGG08~*=A+EsvMv3z~HySqE3ySuwv8tE311`(w} z1OZV>8l+P~q+0|eq*J6MCB8-PRq-O?|J%pcSA5v>o6~2`&d&61Et95shWQnk=OOP{ zKTHz9X~hx2Y2s~mQIF_7H0JM}Q*q~td~m$TA7L6~$&t0my+QVwGjxt1k!GE_1%2Yd zj0~}DU(5Z*78x%T<07f95d_BE*|@N5b^ab98oJa&=687qf%G zPU~poH|~9k7SuE|eY*UR&MSBHwbE>hl@QArCyQmank~uugxMOWR9r7^(nhH|Lz8wb z+(i!8blID=9rw@TSSu-{fio+C%?gZ3+M;HYsgOhL^ZoD3G&>>(Vs=Qa1w@h)M;S(b zBoHb|tZ8ynp z(=-ZTi5hU=oQ1We8?Z~pKRaY-9eUscTlS?gSX11N84a78#z3cL5vd^Pl4%*_#zBik z!Q~6#=H{NnkwL!wu|!fj-55O$gN89E)7Dz2wz%At)cQWb9M%#3(k8VNcVoTMwsu(7 zeShM`o;&-K9tj&v4vRZQf~OFTMBaG*A@UGT(un&K{SdfRvXAarBeK;A+&b82Mxq)IRazlzTL(h%?nNscjA%btL`lrr8h`;qQZO zo7?eMtP>w|vR0%hb>kiebjJ;{vfRvuf_yhdNp0jDvOaOwqJpN_#PtogIfZP5HRYR! z{7ymC-L}1svXHcndMBSwI|o^v$6g1i-EbLeqR%BDi*7ox=yeT&s4>-=-$=yRL>|ZO zs7U0wUB;de6Feu`Nha(1RuTVQhRTiG29!rB*0KCLQrYzO(JPRk0*PH6&Bf*z4`M1D za)y#Q30hv3a@4fRJ~EA}HnBl8^med|b`597e<+{zNN-xY zn;a_-V_^>u=AnCIv3z>_G^aF%Rr#&AxoqOMP{>h|;^Bwa+i6?qI#!#XR_wrot?&Bl zwHHA+EUW5#nM;7I#Da|1vqJ>)N+jT6fBNWlTbYXlA~RgFA-LfO#qtD0W;`*1ZduD9 zxzAq6Y+mnFE6^jNQ!~-ji&dzf^SYr2tqNs<(Yh%5HzE<_)^QDMIaHFx-gr8;KAXiE z8^M4R(PN#RV>TVIVpaUvxTB1fj;B0PwgT?s=NqYra`_acO)P2Tz2G+N~F z46|S3-o)oSlET!BAQIn4*?! zr#$v9QR2WB(vGPQ)Uctr>G%&oU5x&a@)PfnYL@e)|ke1wz;fZYyDkCBt zMWhlF^_V!6g+HrE8h}!W4oB}qL`dkrEeVQrW0(^H)6*|J<%xjeC_J@cRjXJ&U>3|7 z;z>=4wzh1w7*eEYyVbvDD5tB|XUjn5|GKgpll8$-3uTNcOHsqLj;?^=el#@oeyE!p z>hoR^O_&Mw@kexx2%M^3crtIL!Ul02>d4>UO!ZL6>$-dIju8+CVmYv-&_zT$rcEAa zBYl+lvwtrxfkq$oJbJCiVj()%DPduR*x zM_ro(chok>yz9HHl5~U4z`*La;9*j=qzbcH;DksbT9_kD0;}-tZlZCa)Ym)XaNQ(M zv&fGlS*$I{y;+s}{5I};j2zhTH+|Isk|FseOi7;?8h(4qkDM5}}g~Kq7Gc*ZG#bbnj1C=VK$2(hxp?VMzpLCm%Aq-m~6biA- z?k)-X2Ja>zouO5*0`Ic0)_B&3kBsj-!Hhj6WLL&G^y`|ui#pP9whQg#>`Be0pD?3? zTY`RffGEkHDSf|AH2&^Jws?Ri9O)Lj#1X>XWBv!ITLe_CJJR>DRnVAC4C@}U&0-$a z@UBVvpy^@afzrhWK29UsLN^avrpBwQG!% zZSV%Y?eL}=qR`Oh4zr_@9L#Bl)P3{v*$Qk?kkv3IK$5sb6CQOb8x=A z_^2=;4ml*_yVn*tWU2lBKyb5@qe@IPyh4%&7*J0wj7u_qmQ7T zHDsG=ZQ~J-`l*T`jK1dNY=%yFXjhEhXlxrm7|H{8%Q|}GQMssYW_Upj?6q2HEyQnRBu=tT)n4tnTq?7`t+G39N!$hUF$^z6@D1_gZ!Yrj)OYw!vN*u=bqj&wkPJeMyMM_VQnL3-UfBA zNavo+o+Lk9YW)I<%N({}NueLRi6&Ma&53U9(*F3#kb;_A7EZ3D#oO$+#_!*nHqF^Q zPiW#Ql$y&;RJ1`r478EO@8ZH^oz2Q^E2ju#CCJ`amRHDR-*)5qFml@6_ZTk{s;wu3 zC~_|Xr^8BF;l+$KBR8R@0*8fopoMn|qbENh+ith}y@&J3Hk<_&z79eXYaH3T z>O+blz0P;x=T5xzxxV0%F`DZw;l4m^k>-9&{r2rg5jb}ylQ73}dJBg&TB!Xf)B*8c z+T(%G6LF7{m9uYB+|ImM@F zKGFpW1}117sSG?u9z*CzLpfAkMe(>wbfvAU`SfKJeshw%xbKiwZo=g24N&ljBhxI4 zCj{8nbr<_o(^?gG+eDMw*d!epG`W|lmC_%e8merX9#dqmO|Kg3MoNg6aw1c53{%LM zW<}z!cXJxD#)M;~h=#*0Zsi6%owY>sDV*jw7HwkQqKAC-W({phc@w{`V^jExVOkxN ziAWRTgnDskCB=m5rrvP`uXdB&v0)SOrqpV%sI99HVou59#yD~}!AF%_)R4Y!j4+Nd zebpSnBH3)a*--7`9Cj$EFq2z(LKJv<7DBqUG+0(PEi26%RJoNpK8<*1hKA;o_6@-i zA`T%$3Wkcv#uhyUc5TpJ%tGMn#D{h5)Gi+mpWb{d@G=Li3zL|t)r>XeMtP{%GxvqM zxXu0AX?kOdz&wfmv7(R`UAp17<+1kocQc6*CHmTuOYLX!3p0neAB7pLCc;MG47}i2 zueTw4)=JJUtgBgLWkA}=nuuUN)Q&1##q{#7+l;%b=-QSXl-S*pdYHR+)!80kzF~N7 zwAaXdW)a(E6xseke5I9U2F&aOlAi2vreONV{Iu5KZKVd*4NGjT56}9i4I}nD>*xiX z{P|fwx@q3EwULA$I;;{MtWgy=5Xdti9BG$QDCA&VqFC)2?-`8VWw^CM(yit(;gey_ z!C|vYwiQ>1^r>o>{MF9-x`E1!hhFr-gIulHBjZy_-ceQi#N+wbRPrMR-4816xXA3l zJz3f4=R6{hPuEO7NgFUZd9P!@_zC_qV{AUY0L3_Dao}dN(xB!03M1%SwV+e(@P!@t zjz;&gPpInB2}wN~@7}3wD<9X}DA=!iI0WiZUrA3f8RQDjnsa1x0{;R=`m?s*%TU<} zKkH(OBM&l+ahtiZmyV2v zw;Z(I1~L(sf;ry4@t5Jnsy3%+Bk zsj|g!%U~7O`RHh-isQjBOR5;g@f-vGz|WJN!FcX?u-SQv2u=Gy*LKEjd2SbHX6;7D z$Q;23${|?cD0ep%$z?VQ+yGfp(jM-l~J@5J0ZXbMRSN4;g_Y;SAidvs#Qa z#&&4J`4ACXHyJRhv{oxEC;T}r(Tbd;Mnz14k)%dEi1YP*264rsrTY*QV6~vB`^qga z>vu-UX99vA6oP*weCbA}<)UFBs2gq7S|t`wosg%WA4eLg(H+G#4eQA`-xFams^5=* zo5=|1mto{C{uW+c?CMmo;#7Ej9W=N%cF3W<_yJM(_TA8(sQ5H;3>)g60D0t(nTgbk z0r#{g71eGMXF9-#tT$8{?+2Su_c(rT#j9eyyG5LQH{o+KEZhfb#EM6Xfmvki%h|37 zD2PbFA2dX$`%q#V*ob8NJX1+>Z8Zfj?^=R(VN3|A@}*$cSekjaeZskpTspJrx}Mj0 zK{pe-piF`8p*k_t!1wdsYuYBUG7|1O9>fp!R5GUbP1Zf8aPlQsI)q@_rnWteRME3I zcom<-*K$TjryOyX8?Mzei)qAb@$%S%L}*61m*v(Z?oAk7|0=5&`freU6ZBS2Xjb%_ zkx>Lt4umdt^b_*8mWP02gmHka1eN}WqUC(W{}l(<7N{BAI^Soow6-H@{ry|*yxlz7 zgQ&vR_y%1FM2xTjQASMSqZ?xg9q)R2S^4Ns8G^Bzpnd($vgi&-LOnQhGVp}JUf$U6 zO}W!s?XvOlp2Nlz_-JG9J-A1=936)lM)W5fono7`$YOH$8;*<=F#`itiGJHOPQa{ z>S?d}9P?;SEO17NsKiZr&J*pU%wxWL$CUdsCaqGKg^?~Vs;yXU&ng9PFyKVrn6Va` zSRo1v{dBXko4o~BqCaEA@R?DgXiJ=&VT-!_JE%!am$IX>JBqlRV_utB3{`g#=(UAH zPO^ehnN5%y6{m|MUz8|4wtUK>w6zE$X#`FqK;9zT2ct7@^m^ElE}$l{RsI99r}owAYK3cpqTN`kp71#QDE zhd0}he`j7O&h5SH?Aqa)=&^!(dKvWA?%K!){%R{Ogp(&)MSEXP^anXzOe>4Hhgp+w64n5)dTruBY`Wg+|$E_S}?x|-pS+c9S@h?8Acz2el@MG-IK}fd&mcf zum?6wh3W--0)Ec+r_Hz6cBqqJD`OSNbt(B+8fBo(^{CCX?hm?R7NM1w+;q)F(cFEf zZhGGc+Ym!g&zeiS9(U^wY03*nIgR^0BKaB}q77h&?;eh)94J1<6SVT|jWf-wnbBr8 zRl=ZG@GbVL#Sa?u=!8>-#af*qndkoaaQH0iq%PToJ|u+z!nURy4a;56t9_hQP1w$y zyin%zeKga1&=l+OQIs;Ux_-FRI?Qhx!3{+U2pkZh2=lGsNDJVI@ltsd$v@2S~9E*L{@O2uyo`O%rr@{F`G+DIn9b1ZL5}QReS9!+Fk(#<$XHISZq zkWC=A%a?ObUcXWII3dMg(eU0`KNWHfp2{Q4+uoj_tEgqeNSL7Ydg}y`{H?gupw$Z$ zZ{HQ`!P3I43=>J|I%pK+2QPfSrGRx7q*s+op3FDR(B=+Pw*K5kmwa<)GuDUH7ujd{vn$xZV@Cr;yJab89{#9TQfW+R_Nw z+|oTLndm?&LxKfU8Q}3&4e@*KfOl4crCNMjTAxKAEy2lEUiyilsVJ^$FR=@JG>D~N z5jAx00KDQ+EG@{Xx5``!>N&BqzJ!N-m;L=ep-(8$8@h?GiQwwu4=c_I{-E5q{O~{$t%n1tv~FSqT6?F$Kw@!ndV`@( z_1xyXZKAyp7bs1-CO6v69*0pZ@8AYqn|ySP5ecq19Q&F1b;)7f?v14S4e7cfJk4^| zwqy1BG7MTPY+lK{d>f=F=t*kU38?3ga~ZJ(H3*6$AmSB|C^7GoF231V82H*cZqqSnbTd>XTfM#mI4e@`@bH@)#O$KzS+?QQd6 z!|vHc3q!1tFECxj(i~BK??7;{mxIG?X(INpM%=^&-}LGk8Dq^$YIP`EWjAG-k$$P2 z9aF_9q5lAWD#r^-whD#KB}^-#tND@)(;1rU#k&O*Bn}UCZBg0W=u=pCAx%LG`siB= zlFY0)iQ*y_c4hdE6V9&c3)XFqHV22=BZEA1*mK&9gI09J_?uPqi@F9LBE?k3@1cu6 zRrpAw%(U>ig8bt>WpUzYlRinZfMHw6y+>qy!-pT)gCh^z_}$wZ`&3gW{6Ztw2eNKL zciLjSMpu@UkVcYS`jSE6x>|`ko84q52fnxH93I3aCj|ApXlb@cI?r*ua_hif=l*Vc z>u}rZm<5$wDM37@x=b{#G)a&j@P1M#HV99U8d%HPFvug=8R-5I^*iDE>1>m7+jW8H zb8QKYULUdWsa>La96fdPAfK|M)IzI~MLkt_`EWK9@@zZ+Gs;xp%@*t+CHF=fVm>6j zXxd(QkZ|M&zg~SC!nk+sDm1^! zUVfA@RWgD?qaa*uXM;oWBs(E@O8&#q1r2)qoTjN`TX*haGT4p+e5caDk8X;(3S&BY z!cY4LXb5eLQV*~{p}jXAqBTMVJ@tzg*Hr3?zzza;y0@0c_@a-(AvgKM&B#KSa(kvn z97jTXqDWD2ro6o7{d$@r_LQgcBun`F57djvDl6VpCz##fm$a6gS31V!XO3xG2u;`p@Poxl`8aS5i4XYVl|mNF+~|>E$0Af<9F?K% z*_zTVwTY@BtjTRT!P->_NYnk#xJt;B#x0m;lt^@?1^~bP)`Wd*xm`M!WpDN*V|{r6)Qi?g4}S)=70eSgC^WaNIv!I^1)zy}pYOv*AQ?$MpT7K$NrPm-(@H zk;X)Mo)U&bdUQ-(C0M~Y%-zSd%Wri`$!KJfcPwu5zd&W(6j_F8O&H}scbRIoT$gQv zV#T8*tWZ0Qn_`w3gL@6bc^Ca{oJgD5n%OfM^=eDE13~B=nW%-^0))b_2Gq$-G1Ur~ zo9YcEtvT`L(_Uk7UHed$`Rc^xaHufLAY(ct{hG{MQTLXgAT7D{-yknhMoa4dLg--< zYLso!d#o_;ueZWIA4o*TQPi%SwR$i?_L^7TVyDEmJ-69c@1w_dXmLqiitkQ*63>K%tHV#bFyFkKiEwyXBiB6V$$7@n;iQ|p8o>I+vlV!{xI;YP{Eu$%RqY8^MoP}bFoH0_Gh^Dq zXQ`bArPXGs5Z7B?q1i+h6njCg7Ma^klIq8W_d2(r%)7{S;?6;>pAaYp6xb{Z78{WP zu4u16t%o~#P{=;Ejl5AaM~JPHJr*vaVtw4*#>9p*7~Vsi#d~7?P~l=38JxqpA0*cu zB8$`bHenyRI@4nK49)Jc+kk6AO{p}4*3;h{@{fHVduZF9HO*y&*c%5@@%HJ`tSp~R z%I07N7O}r?fB_N%I%KQz2xOkY0YPvjDmf!@DlG8=u-1P9nL8Kee{CPEOu*dsS!n;7RSV?&)HRcD@gm zuTf#4-zTdEP@mT;FXX5Pg6K6ksT6R@KXPgCQwN)9E~tl7Fs_7ix#k*HcFHp(ZKLzwy!v~NT9KeZajO%JMZe?tgS;H`E=fQ%z@TQ~Ba_%JHK3MN({CZMR zsZKj5BGe7W=11TS{?p6u$KF0Syc9d3k<0_S@D%XB+y|-FtN>mfnVDPNloj^kY21a| zcvm`ZDNRFC;T`V>5n@^6NxBpa74#+%e>r^4rpsedyVbPdjo#dxWI{fVUT%)9RWyorrLgcrX38whzf7?xY#g_d@B~wq!s{$l z_YO^MOCvf#!tlsl-yDu%M=>qs6!9?O41X$$>To-_IIugRNOzk5F>}MKK~NFXz^(K# zQ|i|RX6!g)Z0O0oQVwOV3hwx5Je9Aoo{<)w;rX*S(X7bHQ%^pp5Lb$Q&VX<%k8_90 zfKAY}*`*=YZq@3E>~MeKCYV@DA5O5DXpV1T3|Wh&4MV+rBZ@B=lp`60MyZ;})O{aNnD(Mv@yCZ`1BB8yA%(gbk?!{S6@esN$eM2$$J*ylVU97k%pQ1QUZf;h~n zN_X5>{Dd2G^$X4Qw+Q+&Nbi*`AUWjo z;0&72GD9DH(Kr_M3QU!%%zrgFCn%kpZ3B{NUIu7kKQSjdO4&{R(`GsQK}R8ZPo-2}6frayRR+x1V-z zzZHq!BE1Wrp~LA%xe$^myKKnf;IrLQz~oQA|ccZ9!khWthaFN0yM z*PrX~p}{swbgaU<92^vRR|2l3JphnDhb)?%$W{)PgHS-Le@I;4Ub$}RM;wP z;t+C~#()wHvA|>KX7Wcz0UOvbUlwCgwuew>C@j!jUppTy_B>HTflB`%9oG3- zs`HcT(&Vdpk#@0@IAp6GQiloYbj-gDM*A@>$CHZ3F?D(@)QyCQr>?SJ9` zu?Z@73~L(u_y`m}$3eTqM)Z|e-`R4*vAe8_5IN{E?n{dTRc7?fg-Fkeg-<}JWO#DYh6plH zRi9U4pA4nIW%gGZ#pSGKCI_20!pmT;Kd5zr4neQqx0I+L|MF#uTBCNfM&qWdL57s7 zuB-eKyM>OZ8j_1fXheFz(!h(jmx($n>jPU`#?U>r8eCW;GZa)3ED>Qj#Dmq$3z#i} zv1(FM>Zq9cJaQd4na+sXc2Oi{W-A}LIUMIeZ`iqt-=C_bPGwGNJ6%@4v!U z=okvkz?>_!XC(5pk&ZKxc3&Eclrf9SBu^Uo5x9Aqv7_8UFyT#ujKkXUN%{}MIF!aO z*sI?6^04U6xk(_*cP5z+iKVK6rM?!gprzVyNB}3t>DKfi#dU1zxJ!v+)p)D>jir{+ z=bJ=N%9RRWkqgV^2M0OgWw7MXoxW@xZEqjJ5kB=6z}5etpQ=Ck{B)RwSCDwhM^Nb4 z`3_R>`#17Jcwatpt`EJSf@#q<%Sh#E($FdMCH>6Fx4^mx840Dz zNxD4eD4GrHEVSyan-Hmx$l^>Y`PRejo00Gw>uKtv!OJGr5idcW2Ya>m=S+4=7pt3SW_#01z_q+s3AjQSF-yD3q~4DAez*l{~A{5-n6;(fV>%=9N8cCtUTu16sdqwvsx zVqt`Lb@FCR2xMTQMyVf#5jB>>)GOe75L$4j#L%-eN+avQKApD=mFILL)(JwnmPqMI zIRt*|(p*c|%#O+Y3=J(E!IN;@S0^qktn&-VqJz@!8qRVEaCtVQF={YAoays5TkxQ} z_Z1%)-BzwWg`Qjf{EEjXcGGGC#0nbreL&vMHs!iZ5?D4o)l%mjR^arO=N`|Owa-)p zn}9v9ZZA;XcT`r-D|AdJ``m34rBIq;^xOxA=D%U;_T-r6S0yH)Ph z>{UFbQSdbDVO|O{>gr=oDt1fjxEr<4$kIhr;yp8R4+(92urqu6bCRNV9rXm})*L)- z>Z;bkmp`cqi#5WldW=Zj(i%mRpZrXX#`;Ky(2%=#S{=)Dw{10}*|yuRZJbkesR?6o zLo=GAYb~mBeuz`&*?~rTjz^WIT<*I7Ez-5@xgnDW8AFbB1q~x;6ck<7+E8!R^0Hh% z>rohXzA}J8rcfF-oR!T%?4n0u_w4p!EMyG5MUbqf>X_tPOYruzTRY};2#vJmUTGjP z*Jf%NH)*sF zuFgp*3HDSNb#5Y_P0-zDXzzGj%b9Pbx~hXJzuU}uTlngOUR-@S z?UGfuv{%!tn7f0Wc!}rhPixWc?W$g`fj0~V+imuGVYmb3yXZT#qq~nw)_nbtPQ6*V zSCEw+*$;L^tMlX}o#kN5&KaR!(!P7`?1wLzcgK^-LEe-i zC#UQw#fA11ZfSZ%6d+bhbqAShMU8dNYQvcD#GRW@3UH`1z$W zu=RZwg~?q=%7#Fe$vgW(*|p622LWU=8@Q`#;?c9^sva*t`zrZcI9CzDN*o79Owjtv zY!ecWwNbUm?H|No$ONL&kf~Kj1L22p?MPc|_)#19I-=)d@iQCXSQ`Ago)z+0PXg`y ztkp3WlX$tK$@mi1p^$;tlsfs9D0fa%N8~ZwM6h0AEvP-oGYMM;UNLgMwII*#3>ThO zbmtK*3?%nT4xeVTML~d9U**VzrgiHpx}9A}UhSd|892K*n|8-egGbh%i7mrN z>(yp?ZrfImf7uadKQh3YXXa}iA!@vN8sJ1rQh_TKtI9ZKH+;Icfe=F5ahshAR0!GR zb%c8J>^L&tS;-(CSgk}|a_;AyW0(B}mYo_|9r?HYPhJfKg^2~YvGR)c+ydpIs1sIp z6gnoW5`7!n1T0AyMT>khT4RtDU0y<@oup63B_$^anau|mCxGN8%co6CtpbNumW9;b zw}gcTBONUv5`Wmg5~gjE!Vgwyn8Nb%@T6iXr<`T5)q~4uK7mj1#!IsoT2D8$-wka? zN3K8c9o_7;gu}#hSn%j`y7WICqyV⪙J}(Os<^FeH|S1Qt!O~;$y(;~ zK;n8)*Etgh1y?uG38wma9mup^iwUd{A=xCMc2I*DSi3gS4gCAt9B3+K8UzV(u>s>< zo2oGPP4xWJ=|0dsChcjF^1}!)hlJ6oumjnl@gxW^)*L^oh><$2=Qm4_fmopxU{l4E4{Tx+QUpFal~lF&F0gzbPPw2|o6cX0A@M=G zVbgX_F`~8d;@Tm2#r%!7`(C%mhmZBZyRil3Q=pK9Ny1=)Gs$Wtd|~FEyKonK=L^YB z?%(NwQE17DtibYJ7ju%HNnUuR!M^3x8pKcc#0kQXY)lKUl;q{$&SH{rm()UbZ%Xo; zxKN}U{Dvb!j_D|B0eSdaY{-*{_bO�vB0{!f)Qf_AjL%>6H`>bb+SDnn;u5AU=S; zp<@RwJ#`TND4v_$6g}i5oe^Ksgm{5qrHE$Y^UA6me7tii8)s306B+T7SMH4;l~^(0 zHGOk2`cuzhT;3(5)x}tcaM76-ha`1A)F@eDaAI;~G7(4JSwPp+x{F5b!)`tz6t0R^ ziFlN7>=)DIqO=qGI$%J+Hg5&5_XefTUh#n(ZfvMhSG0VPnj3HMd>pa!a_Zag95CB> zJ*i0iWZ~DoXW!qS2i~FtCh5R@_xp8Wl5S_>Zf?)~?@al>zf1$%_Wfb|w_gU%IA!{F z#wnAPwYxdf_k&J$9lSuBLI5=P~(Gui2nZw zJ381gdD_{oYs&s$cOslDTcNtC6NxV3>ZWze7EbM8aW=&Xt6^v3MRO4xHS{>E_lEq|M z4ig?p{3QP7F1RqdlbW>0OR5-5;ks`kLdg%nPaeA}9b?srK$cn8^em1Wj0O~~j!4?{&p9`rb-nl`B*oRLTJshn9N zwUkwUZte(usYR}62BzhP*8 z>!}(QBuZwX?mR@u6OXHw^%xHI%_uVNP{n-=UHgRvF8-n+t1=hbEcU_TcGW^3JKj?^ z*7Q0<|Hwe(Ccak-l}Q!ynMY%yCGq!Z$By69JuTC1YkEptC|1@R+Ez7~B|xXv+8Z}n zjTH@f;8$F%)B;yGR$Ek1#OV8M)}oDFk`J9l!NA2M|DL;}oJE{-zTgNUd|ryQ%6xrQ zpd$SlIt}_hA8Zn zqjt1U8AZfL9EZS{XUiq9=~9Sj_h$5_4dU#-`-;%kA+G-x(nFFgp4Qa)f%lZXRxchj z5h%v=cBqzMmTu za^8EY(K8Euem*lDDa^V+(=ra-{l}y z{E~tBDc#Hi^C?Cx7MX?;;#4w`!thZ)znv`3iZSl9`^{Z9*5eGjKHQ_%?B!%;9Jkg< zxichE(6U%^n@CRQ1ZK*KNGWxPA1z`-bfNqR^{(SlfFfmd^zBFWng?GtSc^dT1wU*G zl!tzdM=jcpB-~)8O_1UDStUApHfGNxnLB-o5!k;_KNe?``rP@^;Jxm>!D-y3BbQX{ zvQX9M82D^-c0Qxv935Tr(@_OSytt!y{IOza+W|YtVJe4fXRf~NlUk#n8#m4Bw9zSp@o#pO5LTwfRo_ehgieWy$s$H-uRvIff$+H@ho)UrCTme*xL zYJoP|Fz3~iHx-9?>Z4lzdh+?N-Eu%FrE~4OlHU$4jxm?I-yUZgWwnLoyZJWgmPG(B z$FOYtF|Q=!z+$Od_7?KLK^z7oHrHI@3(qN;D-dJ-fRD`oKh`=rJ2;v zHQzuH-v1I7*o!EW3lrQsn+5~dYw48*+7ZtNb$n-TdV6}yuyUzeHrTSxcT+l^&9%~k zI>)MVd2)H3`{Sl}_47)5@c7+$3Z-{dP28b$8@o4@o)7fPnwXSkyhnM_53C+|^f^uW z{SN$z&}Wgj8F*zmP%sQ|IDichkn_&<*MI!;7wq>RP(ap2&VTrx^?uw21Hud3#&1~R zHVFJeF|eKyG}|DExEBKc|jgb>sQMxo_9`jqVq3JWsnT(8#j z`GK)NNHc+7A=j_Ge;=>4ouloQgQonK@K>pN{*khasTH8KD@hVxAX$4ANe2fn>#I2@ zxj^jTDq^M%c6JWumGh67@~dl~*UsOrbG>r{p8i|=boq8Hwbi#{sjsAZ(Sf|LqH1Mw zwGEN{JHD^#sJx1lv#Eu(?bX5~zd&vGDr$Bnj?U(0Zl+fg{1IHg$`2^{f0myq;C8kS zSBFZR3q!K{4TV3&8QKd}Qm>%$S0UzNKwbC^**``XvJ0al08WLuOiC9W_4-z;e=T}{ zjwu&o;PY!y{bM*Gxxo6;RYb25N$@UEqXafGxJ;0L4kEy`^q=e8Yz=-x=Nj?jqC-9U z4WYjX9T(kUfkWW$t|bBF^Q=%>4cG{eTOy0h@M7UO-0R{#)d4tcn8>{w9F;dv2Y> zHB`!gvepqO1*I=w3Jd)$=4ISo&S_M7sA?%7V=*8>{V2oEjYU`a52)uL;?97s|JmQ< zS9?M68o@sVNCT%bfdHoX{mSsy*13Z8*XS;9;&uC+{uRozbFQ;*SeNT- z+3roTDPRj8fX=x7MF74t{F`1~8$<$VHMrjXeY&JH^QLYF0K9Sk^GTQon((vcScN+Z{L{s~vZztK_x{Ki(AHBR zb^^1c?^lMu%fYqLeu^@e8=+)PF%vsr50SuBiQ_xQxtgTE{0ZfsisOevUk)J`{<;Yd z5Ss)5bJ5QGz(D^Xwtspv=Ma+C=C)>PU%S2(z>nsZK(%{0AOe$a-YOuueE?A60#C9) z$i7-nKLLKj(r~r5b)ir&alA71w3@H-6#c4b=MZf81X~P;guh^7b}8ggp>UD$#zmA@PJigGe|#J2?N|zqSqNMHc`~6#qvPXK8;X zF8zxpepBC+$gJ_X1Jks}z}f`q3x>**{`VlifXbQwCWD%h-B%X}lp_itUsMj}i@!%@ zzg#&@62na&fZBH#$SSHASS!r^dyud8{IvlN2v)(|)kFProQ0n;z03ZZNXKWXkGzYx+k03yx0m^!y!DH|0ayh`q zTQicHfSS4h1Gxww-f%qt2~SgV$MZVlasa_C)h=E@O)@|{V7b7-TJ!Y)u58=DdguxY z16p7J5{cLa@~-VykpJb;mbjap-PaDch?}d`)!na}ZY9JWAm|JL6|d+8K+Ntd0M8$N zNt-x-n?;KOPx!4}fsT~2gNrKwXKL=^;_#cv>?CXrk1BpxSx9lze{$<(&FLJ z0Dp(bzw);{_5T9AoWBmrgoz%2KL^0xg)fLx5eUYA$=^>i^~=#JbZVp00Lyfz__b|* zd*uqWA5Rm{ONH}J-_Me~9LjVtr#}~P;AQ&ezp&auiSl0V$LR}w|>_S zt~d6Niw1%{44}k|7IW|TTEIVT8~o39{Xk;TG_iI2P1BpXBzX7)2>PeMquGnkB5t2u z3s4g9I&SxF8ys`%;iE=)O@iP3h1v2(BH)_!~i_l@2%yc{(ctfIsK{^E?H824H*{}tia|M`Y|Rh67*H%sRaDDDXmT{tgTjVSrGkk1YD>w{w%dkcrl zx$8pNq!f#Uqge$fs$*!?%kT=&SV?SilJ% zzqdfa4EkDJfS3C~4RWzmgE6`m#<`V$OQOFQnq<}?V?F?K7YI!kV}-lvwLpMEz|tA0 z1XRr(9h?CbU9OuP5ub`4KsN#al-31{yKQwXnE%F#9|`7rVe$vFZ;GO?u?7Zk@}Huv z*5+D+P&RP}`rN;Z7~r6XsZ&5ZvEBbyuX1s?7Wp4ud|i%a)IyXN0k0|t76M#+C{vbs zEy(|f^>X!z6zMjY0Kwu2D7G(J$#&(naL!HfXZN}sW)6&d$pWwvc%ak~xnL!^UDty7 z260{KfxKsA9tXlZ6(GTjVbG!HT5$h_!JirXg9)~#w9cKcP!|SD5B>`r77SerQthAB zUldbR%xuE8a6q|=YRn-PCcd^HC9vfZ&P%AHMi-z-Rkfc;6KDPtg5cX z+SS^`*4oSbYb|=YscR#kSKJ4983ybRaH{WDhJQ0CuvHK6%ELb|FnoOv;rqJ%ANla> zOyBwU(P%S?hFui^cnApVV)jUexE8?A3HNuha#;^rBn2u>E|OoT$zssg0{PdRFBcNo z#Hi3AF#hw^4Hs>=6ytj2fn`E~D#U=f$Zt9ch}xs1M}Q90fb#2NtLY5)S`Zgvz;z?f z{<`!15TF2C08lS7W<+u=)UTFxW%U4$hZ>3jbO0%U)NnDvwlK@4NQ^os?@h%sD@5KaNHHbtO$_ z_+l9}K;Qa5hTt!KE&BgSrI(%G_ABc9m@Ih>AWHLKf7=KCU&b>0KPml>s6UP4yE$I& zUUY-hvco_!q6S<{{a<9@JHx+N`+o#KH<-VO=xl)n``}o>2!LlDlov*@uX}L>rvJR% zaQ<$>xh8+R!q_A;m*pK22uKj=ud@IFr{9zO-xe2#;{*#S0oudC`*j6{gWq3ZeYMPM zuPfevIs9@8a5pZ%5iS-<H$gtYKHGuhJWw>m3BQqRaI#mOWmy%id8Y978Mg*HFXyT64q4|6cJDyLNm>Vz=cq2 zVbcVwXN0-pkeY^QB-2w1DgS`3)~j8`DBX5T63|S{v|`r9YKH`sV1I|_d-s0#oqNu? zbZ6c$I&;6@`Tm~oocp`?R0%|z(}&BiJ5qu$Cw+uz@3(GoEH797O(4)sz%`q#8?J?) zT?o0&2-gurZnHxmmzma?(|m{B}^|a8uvm^H0Nx`w@9`66^e?CvbeF>K=SUVxO+yc08GpcK4Z^!`~+do_cG?uq)vfQ%`N7%;F!!`9QZ3dUIULulku+U z5{T|GBlwHDZdCyy zOgDaDoeW(UJr;IrfWK*XG^ei$1oMw*$92W7>=RjVc^=fD@=xEcB3~n$@yGv%yt)$~ zLG~Va zKt+DH?VUyvGYqdsEqlNlada4p&{&EXqypr8ipF*+OgZIWdClxog?#A1k3y8(qXMDV z#pD71_xFAAh7CK8lkm1OZ7Ry{Re`X~?=2bj@pla8v z)`g6)MnuN5SRq9d^kbuc9y&kT$Gm?2y$VF*I%E^w7Q7Lvf~RGZI2c=Adtu5x=yMIf zP@>mQ;TCaB0%e;$zIZ?C4*DL$#-rB*DhOKL5t3XxDuhKF3L&p$4L-3Swdtux6*V`~CY86uJzoiNst-lhm+u z+)gQBwyv@8?cG4@g88NrwBEC=qbYJ_-yPf5uoP{<@6mJ8ay|eQ8vVuzX>)Ww)0vYW zbY0ipnXndX?p0V-P>6;{)F56<+Q=jA_}WQ8rRiJtU&OkXFQy@Yy!&6LO6#y zX6SM&(%D{lPWO!9R7l8+O}{uNb}8b#1<^pO^aoF?f$AJ1pv1m;a!dPeY!{JHuZNQL3ndvo^f$~Te*^8eq-7X86@yWNqcIjb8lCNuhKJ z3w6(@V^S}|D%cP*SmpE2g>nsCP%~#1*ZZX)2fYelnaL!qz0X;aZ@@F79kcek`d7tBZ%wS%Xhy(0@x-Yvepj|3kLNT3k zwCl>taLFF@oT$w1%WkXfkjfS=;4=M#N5sS8VR$k}8z}oNeYi7z&xxfM=N;MFg`llR zze#Cr@qPJNJ?4~7tUk4QG3LrrK+z{!JAC+1e14~MFVi^%@yk~ImiUFKFyM5A6*XY? zKr2XkVt`qx**Kyora!c52a}X)73kknEWTE2?D>K0-I8EvMHe|mfmV=oOJF`wLDcrr zNMvImx*DodG`6*Q9jE)7Hq#!<$YE^erGL&`@hGEbi)qbsLk1>pMP2D1;QSn;D~as3 z#s~@0O(Nx#^z8IS-AEQ-1A_vmHKSUWB7JFmoS7tnHC50oKIZ>W6`6oUzA@CeF@0|g zA4}s1u%1PuNjYh%e3x||pY`|-6uPr8W~|GK_-nWQBUuGSpHr$7+^b*b-(CzWwt86c zICe^TVWMfKd&sH8zKvLSD0w92z77Nm-QKT$NQR|OybYUqVs!z0H7IkVzQ@i9sX1eOm-eyq%_9t`_*8CY?#XW3Hv&s#p{^==vM#^9+rtWb-r22err=_ zMHS3dh=N16MSUGIFf}^u;5M%f$Dz*sKuIP6=66Jf;8_N=DZ)({k$7v~=JdJ+z&_^D zyC&7h@WmwBf1df@&Rn4_?NoXef zE~hU{M(f%W0Wu`QlT)gqWgjkjCLVg>6$ry-bJY+Tnu}K`rO9??Hh$8KnzI+~r?bNB zAQ^~7`l|Qc{lf4fg4bpw4&D3T94>=$`bolET|cWdF$NhqILHVPHC_gwL|Q%R822ak z(o_>-fG&sYVr2j_zN*jN6%!fXV->7Jw+Z&d$zAfsg7C$wgS*zX1*8A#KiIgW&q|VExFVLHC72QywuvA zuWor2ky_(pOo#FLDj?Ez?uPnf{j;(65d}@4i2JhJLRP5&2-v8$nqH-SAUegv-{;rJ zM0-q3DnwM@!ZTMAu=w)@KW@5P6Bb}G>#YOohTsl3eqg^+`9t$(DDrn9?UM-Z^)1$M zEznQdfd|#qUllIKzRLnM95A%|vfIXQ>#H4jFayW-CeDu0b9Z?M{km?q+v<1g?oRAJ z_Sbl`@$_Bo=9QJYoAqb$Y9bJF7`^Kj_7_Uryx2FESG=ne;X2Vs`!2ij<`CDt-Jd<* z<#{vei`fZB=2EpSCN`uN3sTuD4<08Txq$d*9=SBI+hQFax| Date: Mon, 13 Apr 2015 20:43:24 -0700 Subject: [PATCH 013/144] [Minor][SparkR] Minor refactor and removes redundancy related to cleanClosure. 1. Only use `cleanClosure` in creation of RRDDs. Normally, user and developer do not need to call `cleanClosure` in their function definition. 2. Removes redundant code (e.g. unnecessary wrapper functions) related to `cleanClosure`. Author: hlin09 Closes #5495 from hlin09/cleanClosureFix and squashes the following commits: 74ec303 [hlin09] Minor refactor and removes redundancy. --- R/pkg/R/RDD.R | 16 ++++------------ R/pkg/R/pairRDD.R | 4 ---- 2 files changed, 4 insertions(+), 16 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index d6a75007a6ad7..820027ef67e3b 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -85,7 +85,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) if (!inherits(prev, "PipelinedRDD") || !isPipelinable(prev)) { # This transformation is the first in its stage: - .Object@func <- func + .Object@func <- cleanClosure(func) .Object@prev_jrdd <- getJRDD(prev) .Object@env$prev_serializedMode <- prev@env$serializedMode # NOTE: We use prev_serializedMode to track the serialization mode of prev_JRDD @@ -94,7 +94,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) pipelinedFunc <- function(split, iterator) { func(split, prev@func(split, iterator)) } - .Object@func <- pipelinedFunc + .Object@func <- cleanClosure(pipelinedFunc) .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline # Get the serialization mode of the parent RDD .Object@env$prev_serializedMode <- prev@env$prev_serializedMode @@ -144,17 +144,13 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), return(rdd@env$jrdd_val) } - computeFunc <- function(split, part) { - rdd@func(split, part) - } - packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - serializedFuncArr <- serialize(computeFunc, connection = NULL) + serializedFuncArr <- serialize(rdd@func, connection = NULL) prev_jrdd <- rdd@prev_jrdd @@ -551,11 +547,7 @@ setMethod("mapPartitions", setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { - FUN <- cleanClosure(FUN) - closureCapturingFunc <- function(split, part) { - FUN(split, part) - } - PipelinedRDD(X, closureCapturingFunc) + PipelinedRDD(X, FUN) }) #' @rdname lapplyPartitionsWithIndex diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index c2396c32a7548..739d399f0820f 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -694,10 +694,6 @@ setMethod("cogroup", for (i in 1:rddsLen) { rdds[[i]] <- lapply(rdds[[i]], function(x) { list(x[[1]], list(i, x[[2]])) }) - # TODO(hao): As issue [SparkR-142] mentions, the right value of i - # will not be captured into UDF if getJRDD is not invoked. - # It should be resolved together with that issue. - getJRDD(rdds[[i]]) # Capture the closure. } union.rdd <- Reduce(unionRDD, rdds) group.func <- function(vlist) { From 971b95b0c9002bd541bcbe0da54a9967ba22588f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 13 Apr 2015 21:18:05 -0700 Subject: [PATCH 014/144] [SPARK-5957][ML] better handling of parameters The design doc was posted on the JIRA page. Python changes will be in a follow-up PR. jkbradley 1. Use codegen for shared params. 1. Move shared params to package `ml.param.shared`. 1. Set default values in `Params` instead of in `Param`. 1. Add a few methods to `Params` and `ParamMap`. 1. Move schema handling to `SchemaUtils` from `Params`. - [x] check visibility of the methods added Author: Xiangrui Meng Closes #5431 from mengxr/SPARK-5957 and squashes the following commits: d19236d [Xiangrui Meng] fix test 26ae2d7 [Xiangrui Meng] re-gen code and mark clear protected 38b78c7 [Xiangrui Meng] update Param.toString and remove Params.explain() 409e2d5 [Xiangrui Meng] address comments 2d637bd [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5957 eec2264 [Xiangrui Meng] make get* public in Params 4090d95 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5957 4fee9e7 [Xiangrui Meng] re-gen shared params 2737c2d [Xiangrui Meng] rename SharedParamCodeGen to SharedParamsCodeGen e938f81 [Xiangrui Meng] update code to set default parameter values 28ed322 [Xiangrui Meng] merge master 55be1f3 [Xiangrui Meng] merge master d63b5cc [Xiangrui Meng] fix examples 29b004c [Xiangrui Meng] update ParamsSuite 94fd98e [Xiangrui Meng] fix explain params 48d0e84 [Xiangrui Meng] add remove and update explainParams 4ac6348 [Xiangrui Meng] move schema utils to SchemaUtils add a few methods to Params 0d9594e [Xiangrui Meng] add getOrElse to ParamMap eeeffe8 [Xiangrui Meng] map ++ paramMap => extractValues 0d3fc5b [Xiangrui Meng] setDefault after param a9dbf59 [Xiangrui Meng] minor updates d9302b8 [Xiangrui Meng] generate default values 1c72579 [Xiangrui Meng] pass test compile abb7a3b [Xiangrui Meng] update default values handling dcab97a [Xiangrui Meng] add codegen for shared params --- .../examples/ml/JavaDeveloperApiExample.java | 4 +- .../examples/ml/DeveloperApiExample.scala | 6 +- .../scala/org/apache/spark/ml/Estimator.scala | 2 +- .../scala/org/apache/spark/ml/Pipeline.scala | 10 +- .../org/apache/spark/ml/Transformer.scala | 5 +- .../spark/ml/classification/Classifier.scala | 17 +- .../classification/LogisticRegression.scala | 18 +- .../ProbabilisticClassifier.scala | 11 +- .../BinaryClassificationEvaluator.scala | 15 +- .../apache/spark/ml/feature/HashingTF.scala | 6 +- .../apache/spark/ml/feature/Normalizer.scala | 7 +- .../spark/ml/feature/StandardScaler.scala | 9 +- .../spark/ml/feature/StringIndexer.scala | 10 +- .../apache/spark/ml/feature/Tokenizer.scala | 16 +- .../spark/ml/feature/VectorAssembler.scala | 7 +- .../spark/ml/feature/VectorIndexer.scala | 25 +- .../spark/ml/impl/estimator/Predictor.scala | 16 +- .../org/apache/spark/ml/param/params.scala | 236 ++++++++++------ .../ml/param/shared/SharedParamsCodeGen.scala | 169 ++++++++++++ .../spark/ml/param/shared/sharedParams.scala | 259 ++++++++++++++++++ .../apache/spark/ml/param/sharedParams.scala | 173 ------------ .../apache/spark/ml/recommendation/ALS.scala | 49 ++-- .../ml/regression/LinearRegression.scala | 8 +- .../spark/ml/tuning/CrossValidator.scala | 18 +- .../apache/spark/ml/util/SchemaUtils.scala | 61 +++++ .../apache/spark/ml/param/ParamsSuite.scala | 47 +++- .../apache/spark/ml/param/TestParams.scala | 12 +- 27 files changed, 820 insertions(+), 396 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 19d0eb216848e..eaf00d09f550d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -116,7 +116,7 @@ class MyJavaLogisticRegression */ IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); - int getMaxIter() { return (Integer) get(maxIter); } + int getMaxIter() { return (Integer) getOrDefault(maxIter); } public MyJavaLogisticRegression() { setMaxIter(100); @@ -211,7 +211,7 @@ public Vector predictRaw(Vector features) { public MyJavaLogisticRegressionModel copy() { MyJavaLogisticRegressionModel m = new MyJavaLogisticRegressionModel(parent_, fittingParamMap_, weights_); - Params$.MODULE$.inheritValues(this.paramMap(), this, m); + Params$.MODULE$.inheritValues(this.extractParamMap(), this, m); return m; } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index df26798e41b7b..2245fa429fda3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -99,7 +99,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { * class since the maxIter parameter is only used during training (not in the Model). */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") - def getMaxIter: Int = get(maxIter) + def getMaxIter: Int = getOrDefault(maxIter) } /** @@ -174,11 +174,11 @@ private class MyLogisticRegressionModel( * Create a copy of the model. * The copy is shallow, except for the embedded paramMap, which gets a deep copy. * - * This is used for the defaul implementation of [[transform()]]. + * This is used for the default implementation of [[transform()]]. */ override protected def copy(): MyLogisticRegressionModel = { val m = new MyLogisticRegressionModel(parent, fittingParamMap, weights) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index eff7ef925dfbd..d6b3503ebdd9a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -40,7 +40,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { */ @varargs def fit(dataset: DataFrame, paramPairs: ParamPair[_]*): M = { - val map = new ParamMap().put(paramPairs: _*) + val map = ParamMap(paramPairs: _*) fit(dataset, map) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index a455341a1f723..8eddf79cdfe28 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -84,7 +84,7 @@ class Pipeline extends Estimator[PipelineModel] { /** param for pipeline stages */ val stages: Param[Array[PipelineStage]] = new Param(this, "stages", "stages of the pipeline") def setStages(value: Array[PipelineStage]): this.type = { set(stages, value); this } - def getStages: Array[PipelineStage] = get(stages) + def getStages: Array[PipelineStage] = getOrDefault(stages) /** * Fits the pipeline to the input dataset with additional parameters. If a stage is an @@ -101,7 +101,7 @@ class Pipeline extends Estimator[PipelineModel] { */ override def fit(dataset: DataFrame, paramMap: ParamMap): PipelineModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val theStages = map(stages) // Search for the last estimator. var indexOfLastEstimator = -1 @@ -138,7 +138,7 @@ class Pipeline extends Estimator[PipelineModel] { } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val theStages = map(stages) require(theStages.toSet.size == theStages.size, "Cannot have duplicate components in a pipeline.") @@ -177,14 +177,14 @@ class PipelineModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap + val map = fittingParamMap ++ extractParamMap(paramMap) transformSchema(dataset.schema, map, logging = true) stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, map)) } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap + val map = fittingParamMap ++ extractParamMap(paramMap) stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, map)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 9a5848684b179..7fb87fe452ee6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -22,6 +22,7 @@ import scala.annotation.varargs import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -86,7 +87,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O protected def validateInputType(inputType: DataType): Unit = {} override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType validateInputType(inputType) if (schema.fieldNames.contains(map(outputCol))) { @@ -99,7 +100,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) dataset.withColumn(map(outputCol), callUDF(this.createTransformFunc(map), outputDataType, dataset(map(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index c5fc89f935432..29339c98f51cf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,12 +17,14 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} +import org.apache.spark.ml.param.{ParamMap, Params} +import org.apache.spark.ml.param.shared.HasRawPredictionCol +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.functions._ import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -42,8 +44,8 @@ private[spark] trait ClassifierParams extends PredictorParams fitting: Boolean, featuresDataType: DataType): StructType = { val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(rawPredictionCol), new VectorUDT) + val map = extractParamMap(paramMap) + SchemaUtils.appendColumn(parentSchema, map(rawPredictionCol), new VectorUDT) } } @@ -67,8 +69,7 @@ private[spark] abstract class Classifier[ with ClassifierParams { /** @group setParam */ - def setRawPredictionCol(value: String): E = - set(rawPredictionCol, value).asInstanceOf[E] + def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] // TODO: defaultEvaluator (follow-up PR) } @@ -109,7 +110,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 34625745dd0a8..cc8b0721cf2b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel @@ -31,8 +31,10 @@ import org.apache.spark.storage.StorageLevel * Params for logistic regression. */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams - with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold + with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold { + setDefault(regParam -> 0.1, maxIter -> 100, threshold -> 0.5) +} /** * :: AlphaComponent :: @@ -45,10 +47,6 @@ class LogisticRegression extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { - setRegParam(0.1) - setMaxIter(100) - setThreshold(0.5) - /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) @@ -100,8 +98,6 @@ class LogisticRegressionModel private[ml] ( extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with LogisticRegressionParams { - setThreshold(0.5) - /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) @@ -123,7 +119,7 @@ class LogisticRegressionModel private[ml] ( // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Output selected columns only. // This is a bit complicated since it tries to avoid repeated computation. @@ -184,7 +180,7 @@ class LogisticRegressionModel private[ml] ( * The behavior of this can be adjusted using [[threshold]]. */ override protected def predict(features: Vector): Double = { - if (score(features) > paramMap(threshold)) 1 else 0 + if (score(features) > getThreshold) 1 else 0 } override protected def predictProbabilities(features: Vector): Vector = { @@ -199,7 +195,7 @@ class LogisticRegressionModel private[ml] ( override protected def copy(): LogisticRegressionModel = { val m = new LogisticRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(this.extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index bd8caac855981..10404548ccfde 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} +import org.apache.spark.ml.param.{ParamMap, Params} +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} - /** * Params for probabilistic classification. */ @@ -37,8 +38,8 @@ private[classification] trait ProbabilisticClassifierParams fitting: Boolean, featuresDataType: DataType): StructType = { val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(probabilityCol), new VectorUDT) + val map = extractParamMap(paramMap) + SchemaUtils.appendColumn(parentSchema, map(probabilityCol), new VectorUDT) } } @@ -102,7 +103,7 @@ private[spark] abstract class ProbabilisticClassificationModel[ // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 2360f4479f1c2..c865eb9fe092d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -20,12 +20,13 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Evaluator import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.DoubleType - /** * :: AlphaComponent :: * @@ -40,10 +41,10 @@ class BinaryClassificationEvaluator extends Evaluator with Params * @group param */ val metricName: Param[String] = new Param(this, "metricName", - "metric name in evaluation (areaUnderROC|areaUnderPR)", Some("areaUnderROC")) + "metric name in evaluation (areaUnderROC|areaUnderPR)") /** @group getParam */ - def getMetricName: String = get(metricName) + def getMetricName: String = getOrDefault(metricName) /** @group setParam */ def setMetricName(value: String): this.type = set(metricName, value) @@ -54,12 +55,14 @@ class BinaryClassificationEvaluator extends Evaluator with Params /** @group setParam */ def setLabelCol(value: String): this.type = set(labelCol, value) + setDefault(metricName -> "areaUnderROC") + override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val schema = dataset.schema - checkInputColumn(schema, map(rawPredictionCol), new VectorUDT) - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, map(rawPredictionCol), new VectorUDT) + SchemaUtils.checkColumnType(schema, map(labelCol), DoubleType) // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. val scoreAndLabels = dataset.select(map(rawPredictionCol), map(labelCol)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index fc4e12773c46d..b20f2fc49a8f6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -35,14 +35,16 @@ class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { * number of features * @group param */ - val numFeatures = new IntParam(this, "numFeatures", "number of features", Some(1 << 18)) + val numFeatures = new IntParam(this, "numFeatures", "number of features") /** @group getParam */ - def getNumFeatures: Int = get(numFeatures) + def getNumFeatures: Int = getOrDefault(numFeatures) /** @group setParam */ def setNumFeatures(value: Int): this.type = set(numFeatures, value) + setDefault(numFeatures -> (1 << 18)) + override protected def createTransformFunc(paramMap: ParamMap): Iterable[_] => Vector = { val hashingTF = new feature.HashingTF(paramMap(numFeatures)) hashingTF.transform diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index 05f91dc9105fe..decaeb0da6246 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -35,14 +35,16 @@ class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { * Normalization in L^p^ space, p = 2 by default. * @group param */ - val p = new DoubleParam(this, "p", "the p norm value", Some(2)) + val p = new DoubleParam(this, "p", "the p norm value") /** @group getParam */ - def getP: Double = get(p) + def getP: Double = getOrDefault(p) /** @group setParam */ def setP(value: Double): this.type = set(p, value) + setDefault(p -> 2.0) + override protected def createTransformFunc(paramMap: ParamMap): Vector => Vector = { val normalizer = new feature.Normalizer(paramMap(p)) normalizer.transform @@ -50,4 +52,3 @@ class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { override protected def outputDataType: DataType = new VectorUDT() } - diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 1142aa4f8e73d..1b102619b3524 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ @@ -47,7 +48,7 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP override def fit(dataset: DataFrame, paramMap: ParamMap): StandardScalerModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val input = dataset.select(map(inputCol)).map { case Row(v: Vector) => v } val scaler = new feature.StandardScaler().fit(input) val model = new StandardScalerModel(this, map, scaler) @@ -56,7 +57,7 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], s"Input column ${map(inputCol)} must be a vector column") @@ -86,13 +87,13 @@ class StandardScalerModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val scale = udf((v: Vector) => { scaler.transform(v) } : Vector) dataset.withColumn(map(outputCol), scale(col(map(inputCol)))) } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], s"Input column ${map(inputCol)} must be a vector column") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 61e6742e880d8..4d960df357fe9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -22,6 +22,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StringType, StructType} @@ -34,8 +36,8 @@ private[feature] trait StringIndexerBase extends Params with HasInputCol with Ha /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - checkInputColumn(schema, map(inputCol), StringType) + val map = extractParamMap(paramMap) + SchemaUtils.checkColumnType(schema, map(inputCol), StringType) val inputFields = schema.fields val outputColName = map(outputCol) require(inputFields.forall(_.name != outputColName), @@ -64,7 +66,7 @@ class StringIndexer extends Estimator[StringIndexerModel] with StringIndexerBase // TODO: handle unseen labels override def fit(dataset: DataFrame, paramMap: ParamMap): StringIndexerModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val counts = dataset.select(map(inputCol)).map(_.getString(0)).countByValue() val labels = counts.toSeq.sortBy(-_._2).map(_._1).toArray val model = new StringIndexerModel(this, map, labels) @@ -105,7 +107,7 @@ class StringIndexerModel private[ml] ( def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val indexer = udf { label: String => if (labelToIndex.contains(label)) { labelToIndex(label) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 68401e36950bd..376a004858b4c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -56,39 +56,39 @@ class RegexTokenizer extends UnaryTransformer[String, Seq[String], RegexTokenize * param for minimum token length, default is one to avoid returning empty strings * @group param */ - val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length", Some(1)) + val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length") /** @group setParam */ def setMinTokenLength(value: Int): this.type = set(minTokenLength, value) /** @group getParam */ - def getMinTokenLength: Int = get(minTokenLength) + def getMinTokenLength: Int = getOrDefault(minTokenLength) /** * param sets regex as splitting on gaps (true) or matching tokens (false) * @group param */ - val gaps: BooleanParam = new BooleanParam( - this, "gaps", "Set regex to match gaps or tokens", Some(false)) + val gaps: BooleanParam = new BooleanParam(this, "gaps", "Set regex to match gaps or tokens") /** @group setParam */ def setGaps(value: Boolean): this.type = set(gaps, value) /** @group getParam */ - def getGaps: Boolean = get(gaps) + def getGaps: Boolean = getOrDefault(gaps) /** * param sets regex pattern used by tokenizer * @group param */ - val pattern: Param[String] = new Param( - this, "pattern", "regex pattern used for tokenizing", Some("\\p{L}+|[^\\p{L}\\s]+")) + val pattern: Param[String] = new Param(this, "pattern", "regex pattern used for tokenizing") /** @group setParam */ def setPattern(value: String): this.type = set(pattern, value) /** @group getParam */ - def getPattern: String = get(pattern) + def getPattern: String = getOrDefault(pattern) + + setDefault(minTokenLength -> 1, gaps -> false, pattern -> "\\p{L}+|[^\\p{L}\\s]+") override protected def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { str => val re = paramMap(pattern).r diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index d1b8f7e6e9295..e567e069e7c0b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -22,7 +22,8 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkException import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Transformer -import org.apache.spark.ml.param.{HasInputCols, HasOutputCol, ParamMap} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} import org.apache.spark.sql.{Column, DataFrame, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute @@ -44,7 +45,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val assembleFunc = udf { r: Row => VectorAssembler.assemble(r.toSeq: _*) } @@ -61,7 +62,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputColNames = map(inputCols) val outputColName = map(outputCol) val inputDataTypes = inputColNames.map(name => schema(name).dataType) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 8760960e19272..452faa06e2021 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -18,10 +18,12 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute.{BinaryAttribute, NumericAttribute, NominalAttribute, Attribute, AttributeGroup} -import org.apache.spark.ml.param.{HasInputCol, HasOutputCol, IntParam, ParamMap, Params} +import org.apache.spark.ml.param.{IntParam, ParamMap, Params} +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, VectorUDT} import org.apache.spark.sql.{Row, DataFrame} import org.apache.spark.sql.functions.callUDF @@ -40,11 +42,12 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu */ val maxCategories = new IntParam(this, "maxCategories", "Threshold for the number of values a categorical feature can take." + - " If a feature is found to have > maxCategories values, then it is declared continuous.", - Some(20)) + " If a feature is found to have > maxCategories values, then it is declared continuous.") /** @group getParam */ - def getMaxCategories: Int = get(maxCategories) + def getMaxCategories: Int = getOrDefault(maxCategories) + + setDefault(maxCategories -> 20) } /** @@ -101,7 +104,7 @@ class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerPara override def fit(dataset: DataFrame, paramMap: ParamMap): VectorIndexerModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val firstRow = dataset.select(map(inputCol)).take(1) require(firstRow.length == 1, s"VectorIndexer cannot be fit on an empty dataset.") val numFeatures = firstRow(0).getAs[Vector](0).size @@ -120,12 +123,12 @@ class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerPara override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { // We do not transfer feature metadata since we do not know what types of features we will // produce in transform(). - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val dataType = new VectorUDT require(map.contains(inputCol), s"VectorIndexer requires input column parameter: $inputCol") require(map.contains(outputCol), s"VectorIndexer requires output column parameter: $outputCol") - checkInputColumn(schema, map(inputCol), dataType) - addOutputColumn(schema, map(outputCol), dataType) + SchemaUtils.checkColumnType(schema, map(inputCol), dataType) + SchemaUtils.appendColumn(schema, map(outputCol), dataType) } } @@ -320,7 +323,7 @@ class VectorIndexerModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val newField = prepOutputField(dataset.schema, map) val newCol = callUDF(transformFunc, new VectorUDT, dataset(map(inputCol))) // For now, just check the first row of inputCol for vector length. @@ -334,13 +337,13 @@ class VectorIndexerModel private[ml] ( } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val dataType = new VectorUDT require(map.contains(inputCol), s"VectorIndexerModel requires input column parameter: $inputCol") require(map.contains(outputCol), s"VectorIndexerModel requires output column parameter: $outputCol") - checkInputColumn(schema, map(inputCol), dataType) + SchemaUtils.checkColumnType(schema, map(inputCol), dataType) val origAttrGroup = AttributeGroup.fromStructField(schema(map(inputCol))) val origNumFeatures: Option[Int] = if (origAttrGroup.attributes.nonEmpty) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index dfb89cc8d4af3..195333a5cc47f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -18,8 +18,10 @@ package org.apache.spark.ml.impl.estimator import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{VectorUDT, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD @@ -53,14 +55,14 @@ private[spark] trait PredictorParams extends Params paramMap: ParamMap, fitting: Boolean, featuresDataType: DataType): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector - checkInputColumn(schema, map(featuresCol), featuresDataType) + SchemaUtils.checkColumnType(schema, map(featuresCol), featuresDataType) if (fitting) { // TODO: Allow other numeric types - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, map(labelCol), DoubleType) } - addOutputColumn(schema, map(predictionCol), DoubleType) + SchemaUtils.appendColumn(schema, map(predictionCol), DoubleType) } } @@ -98,7 +100,7 @@ private[spark] abstract class Predictor[ // This handles a few items such as schema validation. // Developers only need to implement train(). transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val model = train(dataset, map) Params.inheritValues(map, this, model) // copy params to model model @@ -141,7 +143,7 @@ private[spark] abstract class Predictor[ * and put it in an RDD with strong types. */ protected def extractLabeledPoints(dataset: DataFrame, paramMap: ParamMap): RDD[LabeledPoint] = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) dataset.select(map(labelCol), map(featuresCol)) .map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) @@ -201,7 +203,7 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 7d5178d0abb2d..849c60433c777 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -17,15 +17,14 @@ package org.apache.spark.ml.param +import java.lang.reflect.Modifier +import java.util.NoSuchElementException + import scala.annotation.varargs import scala.collection.mutable -import java.lang.reflect.Modifier - import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.Identifiable -import org.apache.spark.sql.types.{DataType, StructField, StructType} - /** * :: AlphaComponent :: @@ -38,12 +37,7 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} * @tparam T param value type */ @AlphaComponent -class Param[T] ( - val parent: Params, - val name: String, - val doc: String, - val defaultValue: Option[T] = None) - extends Serializable { +class Param[T] (val parent: Params, val name: String, val doc: String) extends Serializable { /** * Creates a param pair with the given value (for Java). @@ -55,58 +49,55 @@ class Param[T] ( */ def ->(value: T): ParamPair[T] = ParamPair(this, value) + /** + * Converts this param's name, doc, and optionally its default value and the user-supplied + * value in its parent to string. + */ override def toString: String = { - if (defaultValue.isDefined) { - s"$name: $doc (default: ${defaultValue.get})" + val valueStr = if (parent.isDefined(this)) { + val defaultValueStr = parent.getDefault(this).map("default: " + _) + val currentValueStr = parent.get(this).map("current: " + _) + (defaultValueStr ++ currentValueStr).mkString("(", ", ", ")") } else { - s"$name: $doc" + "(undefined)" } + s"$name: $doc $valueStr" } } // specialize primitive-typed params because Java doesn't recognize scala.Double, scala.Int, ... /** Specialized version of [[Param[Double]]] for Java. */ -class DoubleParam(parent: Params, name: String, doc: String, defaultValue: Option[Double]) - extends Param[Double](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class DoubleParam(parent: Params, name: String, doc: String) + extends Param[Double](parent, name, doc) { override def w(value: Double): ParamPair[Double] = super.w(value) } /** Specialized version of [[Param[Int]]] for Java. */ -class IntParam(parent: Params, name: String, doc: String, defaultValue: Option[Int]) - extends Param[Int](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class IntParam(parent: Params, name: String, doc: String) + extends Param[Int](parent, name, doc) { override def w(value: Int): ParamPair[Int] = super.w(value) } /** Specialized version of [[Param[Float]]] for Java. */ -class FloatParam(parent: Params, name: String, doc: String, defaultValue: Option[Float]) - extends Param[Float](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class FloatParam(parent: Params, name: String, doc: String) + extends Param[Float](parent, name, doc) { override def w(value: Float): ParamPair[Float] = super.w(value) } /** Specialized version of [[Param[Long]]] for Java. */ -class LongParam(parent: Params, name: String, doc: String, defaultValue: Option[Long]) - extends Param[Long](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class LongParam(parent: Params, name: String, doc: String) + extends Param[Long](parent, name, doc) { override def w(value: Long): ParamPair[Long] = super.w(value) } /** Specialized version of [[Param[Boolean]]] for Java. */ -class BooleanParam(parent: Params, name: String, doc: String, defaultValue: Option[Boolean]) - extends Param[Boolean](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class BooleanParam(parent: Params, name: String, doc: String) + extends Param[Boolean](parent, name, doc) { override def w(value: Boolean): ParamPair[Boolean] = super.w(value) } @@ -124,8 +115,11 @@ case class ParamPair[T](param: Param[T], value: T) @AlphaComponent trait Params extends Identifiable with Serializable { - /** Returns all params. */ - def params: Array[Param[_]] = { + /** + * Returns all params sorted by their names. The default implementation uses Java reflection to + * list all public methods that have no arguments and return [[Param]]. + */ + lazy val params: Array[Param[_]] = { val methods = this.getClass.getMethods methods.filter { m => Modifier.isPublic(m.getModifiers) && @@ -153,25 +147,29 @@ trait Params extends Identifiable with Serializable { def explainParams(): String = params.mkString("\n") /** Checks whether a param is explicitly set. */ - def isSet(param: Param[_]): Boolean = { - require(param.parent.eq(this)) + final def isSet(param: Param[_]): Boolean = { + shouldOwn(param) paramMap.contains(param) } + /** Checks whether a param is explicitly set or has a default value. */ + final def isDefined(param: Param[_]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) || paramMap.contains(param) + } + /** Gets a param by its name. */ - private[ml] def getParam(paramName: String): Param[Any] = { - val m = this.getClass.getMethod(paramName) - assert(Modifier.isPublic(m.getModifiers) && - classOf[Param[_]].isAssignableFrom(m.getReturnType) && - m.getParameterTypes.isEmpty) - m.invoke(this).asInstanceOf[Param[Any]] + def getParam(paramName: String): Param[Any] = { + params.find(_.name == paramName).getOrElse { + throw new NoSuchElementException(s"Param $paramName does not exist.") + }.asInstanceOf[Param[Any]] } /** * Sets a parameter in the embedded param map. */ - protected def set[T](param: Param[T], value: T): this.type = { - require(param.parent.eq(this)) + protected final def set[T](param: Param[T], value: T): this.type = { + shouldOwn(param) paramMap.put(param.asInstanceOf[Param[Any]], value) this } @@ -179,52 +177,102 @@ trait Params extends Identifiable with Serializable { /** * Sets a parameter (by name) in the embedded param map. */ - private[ml] def set(param: String, value: Any): this.type = { + protected final def set(param: String, value: Any): this.type = { set(getParam(param), value) } /** - * Gets the value of a parameter in the embedded param map. + * Optionally returns the user-supplied value of a param. + */ + final def get[T](param: Param[T]): Option[T] = { + shouldOwn(param) + paramMap.get(param) + } + + /** + * Clears the user-supplied value for the input param. + */ + protected final def clear(param: Param[_]): this.type = { + shouldOwn(param) + paramMap.remove(param) + this + } + + /** + * Gets the value of a param in the embedded param map or its default value. Throws an exception + * if neither is set. + */ + final def getOrDefault[T](param: Param[T]): T = { + shouldOwn(param) + get(param).orElse(getDefault(param)).get + } + + /** + * Sets a default value for a param. + * @param param param to set the default value. Make sure that this param is initialized before + * this method gets called. + * @param value the default value */ - protected def get[T](param: Param[T]): T = { - require(param.parent.eq(this)) - paramMap(param) + protected final def setDefault[T](param: Param[T], value: T): this.type = { + shouldOwn(param) + defaultParamMap.put(param, value) + this } /** - * Internal param map. + * Sets default values for a list of params. + * @param paramPairs a list of param pairs that specify params and their default values to set + * respectively. Make sure that the params are initialized before this method + * gets called. */ - protected val paramMap: ParamMap = ParamMap.empty + protected final def setDefault(paramPairs: ParamPair[_]*): this.type = { + paramPairs.foreach { p => + setDefault(p.param.asInstanceOf[Param[Any]], p.value) + } + this + } /** - * Check whether the given schema contains an input column. - * @param colName Input column name - * @param dataType Input column DataType + * Gets the default value of a parameter. */ - protected def checkInputColumn(schema: StructType, colName: String, dataType: DataType): Unit = { - val actualDataType = schema(colName).dataType - require(actualDataType.equals(dataType), s"Input column $colName must be of type $dataType" + - s" but was actually $actualDataType. Column param description: ${getParam(colName)}") + final def getDefault[T](param: Param[T]): Option[T] = { + shouldOwn(param) + defaultParamMap.get(param) } /** - * Add an output column to the given schema. - * This fails if the given output column already exists. - * @param schema Initial schema (not modified) - * @param colName Output column name. If this column name is an empy String "", this method - * returns the initial schema, unchanged. This allows users to disable output - * columns. - * @param dataType Output column DataType - */ - protected def addOutputColumn( - schema: StructType, - colName: String, - dataType: DataType): StructType = { - if (colName.length == 0) return schema - val fieldNames = schema.fieldNames - require(!fieldNames.contains(colName), s"Output column $colName already exists.") - val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) - StructType(outputFields) + * Tests whether the input param has a default value set. + */ + final def hasDefault[T](param: Param[T]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) + } + + /** + * Extracts the embedded default param values and user-supplied values, and then merges them with + * extra values from input into a flat param map, where the latter value is used if there exist + * conflicts, i.e., with ordering: default param values < user-supplied values < extraParamMap. + */ + protected final def extractParamMap(extraParamMap: ParamMap): ParamMap = { + defaultParamMap ++ paramMap ++ extraParamMap + } + + /** + * [[extractParamMap]] with no extra values. + */ + protected final def extractParamMap(): ParamMap = { + extractParamMap(ParamMap.empty) + } + + /** Internal param map for user-supplied values. */ + private val paramMap: ParamMap = ParamMap.empty + + /** Internal param map for default values. */ + private val defaultParamMap: ParamMap = ParamMap.empty + + /** Validates that the input param belongs to this instance. */ + private def shouldOwn(param: Param[_]): Unit = { + require(param.parent.eq(this), s"Param $param does not belong to $this.") } } @@ -261,12 +309,13 @@ private[spark] object Params { * A param to value map. */ @AlphaComponent -class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) extends Serializable { +final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) + extends Serializable { /** * Creates an empty param map. */ - def this() = this(mutable.Map.empty[Param[Any], Any]) + def this() = this(mutable.Map.empty) /** * Puts a (param, value) pair (overwrites if the input param exists). @@ -288,12 +337,17 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Optionally returns the value associated with a param or its default. + * Optionally returns the value associated with a param. */ def get[T](param: Param[T]): Option[T] = { - map.get(param.asInstanceOf[Param[Any]]) - .orElse(param.defaultValue) - .asInstanceOf[Option[T]] + map.get(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + + /** + * Returns the value associated with a param or a default value. + */ + def getOrElse[T](param: Param[T], default: T): T = { + get(param).getOrElse(default) } /** @@ -301,10 +355,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * Raises a NoSuchElementException if there is no value associated with the input param. */ def apply[T](param: Param[T]): T = { - val value = get(param) - if (value.isDefined) { - value.get - } else { + get(param).getOrElse { throw new NoSuchElementException(s"Cannot find param ${param.name}.") } } @@ -316,6 +367,13 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten map.contains(param.asInstanceOf[Param[Any]]) } + /** + * Removes a key from this map and returns its value associated previously as an option. + */ + def remove[T](param: Param[T]): Option[T] = { + map.remove(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + /** * Filters this param map for the given parent. */ @@ -325,7 +383,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Make a copy of this param map. + * Creates a copy of this param map. */ def copy: ParamMap = new ParamMap(map.clone()) @@ -337,7 +395,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten /** * Returns a new param map that contains parameters in this map and the given map, - * where the latter overwrites this if there exists conflicts. + * where the latter overwrites this if there exist conflicts. */ def ++(other: ParamMap): ParamMap = { // TODO: Provide a better method name for Java users. @@ -363,7 +421,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Number of param pairs in this set. + * Number of param pairs in this map. */ def size: Int = map.size } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala new file mode 100644 index 0000000000000..95d7e64790c79 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -0,0 +1,169 @@ +/* + * 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.ml.param.shared + +import java.io.PrintWriter + +import scala.reflect.ClassTag + +/** + * Code generator for shared params (sharedParams.scala). Run under the Spark folder with + * {{{ + * build/sbt "mllib/runMain org.apache.spark.ml.param.shared.SharedParamsCodeGen" + * }}} + */ +private[shared] object SharedParamsCodeGen { + + def main(args: Array[String]): Unit = { + val params = Seq( + ParamDesc[Double]("regParam", "regularization parameter"), + ParamDesc[Int]("maxIter", "max number of iterations"), + ParamDesc[String]("featuresCol", "features column name", Some("\"features\"")), + ParamDesc[String]("labelCol", "label column name", Some("\"label\"")), + ParamDesc[String]("predictionCol", "prediction column name", Some("\"prediction\"")), + ParamDesc[String]("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", + Some("\"rawPrediction\"")), + ParamDesc[String]("probabilityCol", + "column name for predicted class conditional probabilities", Some("\"probability\"")), + ParamDesc[Double]("threshold", "threshold in binary classification prediction"), + ParamDesc[String]("inputCol", "input column name"), + ParamDesc[Array[String]]("inputCols", "input column names"), + ParamDesc[String]("outputCol", "output column name"), + ParamDesc[Int]("checkpointInterval", "checkpoint interval"), + ParamDesc[Boolean]("fitIntercept", "whether to fit an intercept term", Some("true"))) + + val code = genSharedParams(params) + val file = "src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala" + val writer = new PrintWriter(file) + writer.write(code) + writer.close() + } + + /** Description of a param. */ + private case class ParamDesc[T: ClassTag]( + name: String, + doc: String, + defaultValueStr: Option[String] = None) { + + require(name.matches("[a-z][a-zA-Z0-9]*"), s"Param name $name is invalid.") + require(doc.nonEmpty) // TODO: more rigorous on doc + + def paramTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + c match { + case _ if c == classOf[Int] => "IntParam" + case _ if c == classOf[Long] => "LongParam" + case _ if c == classOf[Float] => "FloatParam" + case _ if c == classOf[Double] => "DoubleParam" + case _ if c == classOf[Boolean] => "BooleanParam" + case _ => s"Param[${getTypeString(c)}]" + } + } + + def valueTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + getTypeString(c) + } + + private def getTypeString(c: Class[_]): String = { + c match { + case _ if c == classOf[Int] => "Int" + case _ if c == classOf[Long] => "Long" + case _ if c == classOf[Float] => "Float" + case _ if c == classOf[Double] => "Double" + case _ if c == classOf[Boolean] => "Boolean" + case _ if c == classOf[String] => "String" + case _ if c.isArray => s"Array[${getTypeString(c.getComponentType)}]" + } + } + } + + /** Generates the HasParam trait code for the input param. */ + private def genHasParamTrait(param: ParamDesc[_]): String = { + val name = param.name + val Name = name(0).toUpper +: name.substring(1) + val Param = param.paramTypeName + val T = param.valueTypeName + val doc = param.doc + val defaultValue = param.defaultValueStr + val defaultValueDoc = defaultValue.map { v => + s" (default: $v)" + }.getOrElse("") + val setDefault = defaultValue.map { v => + s""" + | setDefault($name, $v) + |""".stripMargin + }.getOrElse("") + + s""" + |/** + | * :: DeveloperApi :: + | * Trait for shared param $name$defaultValueDoc. + | */ + |@DeveloperApi + |trait Has$Name extends Params { + | + | /** + | * Param for $doc. + | * @group param + | */ + | final val $name: $Param = new $Param(this, "$name", "$doc") + |$setDefault + | /** @group getParam */ + | final def get$Name: $T = getOrDefault($name) + |} + |""".stripMargin + } + + /** Generates Scala source code for the input params with header. */ + private def genSharedParams(params: Seq[ParamDesc[_]]): String = { + val header = + """/* + | * 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.ml.param.shared + | + |import org.apache.spark.annotation.DeveloperApi + |import org.apache.spark.ml.param._ + | + |// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + | + |// scalastyle:off + |""".stripMargin + + val footer = "// scalastyle:on\n" + + val traits = params.map(genHasParamTrait).mkString + + header + traits + footer + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala new file mode 100644 index 0000000000000..72b08bf276483 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -0,0 +1,259 @@ +/* + * 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.ml.param.shared + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.param._ + +// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + +// scalastyle:off + +/** + * :: DeveloperApi :: + * Trait for shared param regParam. + */ +@DeveloperApi +trait HasRegParam extends Params { + + /** + * Param for regularization parameter. + * @group param + */ + final val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") + + /** @group getParam */ + final def getRegParam: Double = getOrDefault(regParam) +} + +/** + * :: DeveloperApi :: + * Trait for shared param maxIter. + */ +@DeveloperApi +trait HasMaxIter extends Params { + + /** + * Param for max number of iterations. + * @group param + */ + final val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") + + /** @group getParam */ + final def getMaxIter: Int = getOrDefault(maxIter) +} + +/** + * :: DeveloperApi :: + * Trait for shared param featuresCol (default: "features"). + */ +@DeveloperApi +trait HasFeaturesCol extends Params { + + /** + * Param for features column name. + * @group param + */ + final val featuresCol: Param[String] = new Param[String](this, "featuresCol", "features column name") + + setDefault(featuresCol, "features") + + /** @group getParam */ + final def getFeaturesCol: String = getOrDefault(featuresCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param labelCol (default: "label"). + */ +@DeveloperApi +trait HasLabelCol extends Params { + + /** + * Param for label column name. + * @group param + */ + final val labelCol: Param[String] = new Param[String](this, "labelCol", "label column name") + + setDefault(labelCol, "label") + + /** @group getParam */ + final def getLabelCol: String = getOrDefault(labelCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param predictionCol (default: "prediction"). + */ +@DeveloperApi +trait HasPredictionCol extends Params { + + /** + * Param for prediction column name. + * @group param + */ + final val predictionCol: Param[String] = new Param[String](this, "predictionCol", "prediction column name") + + setDefault(predictionCol, "prediction") + + /** @group getParam */ + final def getPredictionCol: String = getOrDefault(predictionCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param rawPredictionCol (default: "rawPrediction"). + */ +@DeveloperApi +trait HasRawPredictionCol extends Params { + + /** + * Param for raw prediction (a.k.a. confidence) column name. + * @group param + */ + final val rawPredictionCol: Param[String] = new Param[String](this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name") + + setDefault(rawPredictionCol, "rawPrediction") + + /** @group getParam */ + final def getRawPredictionCol: String = getOrDefault(rawPredictionCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param probabilityCol (default: "probability"). + */ +@DeveloperApi +trait HasProbabilityCol extends Params { + + /** + * Param for column name for predicted class conditional probabilities. + * @group param + */ + final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "column name for predicted class conditional probabilities") + + setDefault(probabilityCol, "probability") + + /** @group getParam */ + final def getProbabilityCol: String = getOrDefault(probabilityCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param threshold. + */ +@DeveloperApi +trait HasThreshold extends Params { + + /** + * Param for threshold in binary classification prediction. + * @group param + */ + final val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction") + + /** @group getParam */ + final def getThreshold: Double = getOrDefault(threshold) +} + +/** + * :: DeveloperApi :: + * Trait for shared param inputCol. + */ +@DeveloperApi +trait HasInputCol extends Params { + + /** + * Param for input column name. + * @group param + */ + final val inputCol: Param[String] = new Param[String](this, "inputCol", "input column name") + + /** @group getParam */ + final def getInputCol: String = getOrDefault(inputCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param inputCols. + */ +@DeveloperApi +trait HasInputCols extends Params { + + /** + * Param for input column names. + * @group param + */ + final val inputCols: Param[Array[String]] = new Param[Array[String]](this, "inputCols", "input column names") + + /** @group getParam */ + final def getInputCols: Array[String] = getOrDefault(inputCols) +} + +/** + * :: DeveloperApi :: + * Trait for shared param outputCol. + */ +@DeveloperApi +trait HasOutputCol extends Params { + + /** + * Param for output column name. + * @group param + */ + final val outputCol: Param[String] = new Param[String](this, "outputCol", "output column name") + + /** @group getParam */ + final def getOutputCol: String = getOrDefault(outputCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param checkpointInterval. + */ +@DeveloperApi +trait HasCheckpointInterval extends Params { + + /** + * Param for checkpoint interval. + * @group param + */ + final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") + + /** @group getParam */ + final def getCheckpointInterval: Int = getOrDefault(checkpointInterval) +} + +/** + * :: DeveloperApi :: + * Trait for shared param fitIntercept (default: true). + */ +@DeveloperApi +trait HasFitIntercept extends Params { + + /** + * Param for whether to fit an intercept term. + * @group param + */ + final val fitIntercept: BooleanParam = new BooleanParam(this, "fitIntercept", "whether to fit an intercept term") + + setDefault(fitIntercept, true) + + /** @group getParam */ + final def getFitIntercept: Boolean = getOrDefault(fitIntercept) +} +// scalastyle:on diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala deleted file mode 100644 index 07e6eb417763d..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ /dev/null @@ -1,173 +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.ml.param - -/* NOTE TO DEVELOPERS: - * If you mix these parameter traits into your algorithm, please add a setter method as well - * so that users may use a builder pattern: - * val myLearner = new MyLearner().setParam1(x).setParam2(y)... - */ - -private[ml] trait HasRegParam extends Params { - /** - * param for regularization parameter - * @group param - */ - val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") - - /** @group getParam */ - def getRegParam: Double = get(regParam) -} - -private[ml] trait HasMaxIter extends Params { - /** - * param for max number of iterations - * @group param - */ - val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") - - /** @group getParam */ - def getMaxIter: Int = get(maxIter) -} - -private[ml] trait HasFeaturesCol extends Params { - /** - * param for features column name - * @group param - */ - val featuresCol: Param[String] = - new Param(this, "featuresCol", "features column name", Some("features")) - - /** @group getParam */ - def getFeaturesCol: String = get(featuresCol) -} - -private[ml] trait HasLabelCol extends Params { - /** - * param for label column name - * @group param - */ - val labelCol: Param[String] = new Param(this, "labelCol", "label column name", Some("label")) - - /** @group getParam */ - def getLabelCol: String = get(labelCol) -} - -private[ml] trait HasPredictionCol extends Params { - /** - * param for prediction column name - * @group param - */ - val predictionCol: Param[String] = - new Param(this, "predictionCol", "prediction column name", Some("prediction")) - - /** @group getParam */ - def getPredictionCol: String = get(predictionCol) -} - -private[ml] trait HasRawPredictionCol extends Params { - /** - * param for raw prediction column name - * @group param - */ - val rawPredictionCol: Param[String] = - new Param(this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name", - Some("rawPrediction")) - - /** @group getParam */ - def getRawPredictionCol: String = get(rawPredictionCol) -} - -private[ml] trait HasProbabilityCol extends Params { - /** - * param for predicted class conditional probabilities column name - * @group param - */ - val probabilityCol: Param[String] = - new Param(this, "probabilityCol", "column name for predicted class conditional probabilities", - Some("probability")) - - /** @group getParam */ - def getProbabilityCol: String = get(probabilityCol) -} - -private[ml] trait HasFitIntercept extends Params { - /** - * param for fitting the intercept term, defaults to true - * @group param - */ - val fitIntercept: BooleanParam = - new BooleanParam(this, "fitIntercept", "indicates whether to fit an intercept term", Some(true)) - - /** @group getParam */ - def getFitIntercept: Boolean = get(fitIntercept) -} - -private[ml] trait HasThreshold extends Params { - /** - * param for threshold in (binary) prediction - * @group param - */ - val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in prediction") - - /** @group getParam */ - def getThreshold: Double = get(threshold) -} - -private[ml] trait HasInputCol extends Params { - /** - * param for input column name - * @group param - */ - val inputCol: Param[String] = new Param(this, "inputCol", "input column name") - - /** @group getParam */ - def getInputCol: String = get(inputCol) -} - -private[ml] trait HasInputCols extends Params { - /** - * Param for input column names. - */ - val inputCols: Param[Array[String]] = new Param(this, "inputCols", "input column names") - - /** @group getParam */ - def getInputCols: Array[String] = get(inputCols) -} - -private[ml] trait HasOutputCol extends Params { - /** - * param for output column name - * @group param - */ - val outputCol: Param[String] = new Param(this, "outputCol", "output column name") - - /** @group getParam */ - def getOutputCol: String = get(outputCol) -} - -private[ml] trait HasCheckpointInterval extends Params { - /** - * param for checkpoint interval - * @group param - */ - val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") - - /** @group getParam */ - def getCheckpointInterval: Int = get(checkpointInterval) -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 52c9e95d6012f..bd793beba35b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -34,6 +34,7 @@ import org.apache.spark.{Logging, Partitioner} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -54,86 +55,88 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR * Param for rank of the matrix factorization. * @group param */ - val rank = new IntParam(this, "rank", "rank of the factorization", Some(10)) + val rank = new IntParam(this, "rank", "rank of the factorization") /** @group getParam */ - def getRank: Int = get(rank) + def getRank: Int = getOrDefault(rank) /** * Param for number of user blocks. * @group param */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", Some(10)) + val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks") /** @group getParam */ - def getNumUserBlocks: Int = get(numUserBlocks) + def getNumUserBlocks: Int = getOrDefault(numUserBlocks) /** * Param for number of item blocks. * @group param */ val numItemBlocks = - new IntParam(this, "numItemBlocks", "number of item blocks", Some(10)) + new IntParam(this, "numItemBlocks", "number of item blocks") /** @group getParam */ - def getNumItemBlocks: Int = get(numItemBlocks) + def getNumItemBlocks: Int = getOrDefault(numItemBlocks) /** * Param to decide whether to use implicit preference. * @group param */ - val implicitPrefs = - new BooleanParam(this, "implicitPrefs", "whether to use implicit preference", Some(false)) + val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") /** @group getParam */ - def getImplicitPrefs: Boolean = get(implicitPrefs) + def getImplicitPrefs: Boolean = getOrDefault(implicitPrefs) /** * Param for the alpha parameter in the implicit preference formulation. * @group param */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", Some(1.0)) + val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference") /** @group getParam */ - def getAlpha: Double = get(alpha) + def getAlpha: Double = getOrDefault(alpha) /** * Param for the column name for user ids. * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids", Some("user")) + val userCol = new Param[String](this, "userCol", "column name for user ids") /** @group getParam */ - def getUserCol: String = get(userCol) + def getUserCol: String = getOrDefault(userCol) /** * Param for the column name for item ids. * @group param */ - val itemCol = - new Param[String](this, "itemCol", "column name for item ids", Some("item")) + val itemCol = new Param[String](this, "itemCol", "column name for item ids") /** @group getParam */ - def getItemCol: String = get(itemCol) + def getItemCol: String = getOrDefault(itemCol) /** * Param for the column name for ratings. * @group param */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings", Some("rating")) + val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") /** @group getParam */ - def getRatingCol: String = get(ratingCol) + def getRatingCol: String = getOrDefault(ratingCol) /** * Param for whether to apply nonnegativity constraints. * @group param */ val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares", Some(false)) + this, "nonnegative", "whether to use nonnegative constraint for least squares") /** @group getParam */ - val getNonnegative: Boolean = get(nonnegative) + def getNonnegative: Boolean = getOrDefault(nonnegative) + + setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, + implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", + ratingCol -> "rating", nonnegative -> false) /** * Validates and transforms the input schema. @@ -142,7 +145,7 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR * @return output schema */ protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) assert(schema(map(userCol)).dataType == IntegerType) assert(schema(map(itemCol)).dataType== IntegerType) val ratingType = schema(map(ratingCol)).dataType @@ -171,7 +174,7 @@ class ALSModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { import dataset.sqlContext.implicits._ - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val users = userFactors.toDF("id", "features") val items = itemFactors.toDF("id", "features") @@ -283,7 +286,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { setCheckpointInterval(10) override def fit(dataset: DataFrame, paramMap: ParamMap): ALSModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val ratings = dataset .select(col(map(userCol)), col(map(itemCol)), col(map(ratingCol)).cast(FloatType)) .map { row => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 65f6627a0c351..26ca7459c4fdf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -18,7 +18,8 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} +import org.apache.spark.ml.param.{Params, ParamMap} +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.sql.DataFrame @@ -41,8 +42,7 @@ private[regression] trait LinearRegressionParams extends RegressorParams class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegressionModel] with LinearRegressionParams { - setRegParam(0.1) - setMaxIter(100) + setDefault(regParam -> 0.1, maxIter -> 100) /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) @@ -93,7 +93,7 @@ class LinearRegressionModel private[ml] ( override protected def copy(): LinearRegressionModel = { val m = new LinearRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 2eb1dac56f1e9..4bb4ed813c006 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.types.StructType * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ private[ml] trait CrossValidatorParams extends Params { + /** * param for the estimator to be cross-validated * @group param @@ -38,7 +39,7 @@ private[ml] trait CrossValidatorParams extends Params { val estimator: Param[Estimator[_]] = new Param(this, "estimator", "estimator for selection") /** @group getParam */ - def getEstimator: Estimator[_] = get(estimator) + def getEstimator: Estimator[_] = getOrDefault(estimator) /** * param for estimator param maps @@ -48,7 +49,7 @@ private[ml] trait CrossValidatorParams extends Params { new Param(this, "estimatorParamMaps", "param maps for the estimator") /** @group getParam */ - def getEstimatorParamMaps: Array[ParamMap] = get(estimatorParamMaps) + def getEstimatorParamMaps: Array[ParamMap] = getOrDefault(estimatorParamMaps) /** * param for the evaluator for selection @@ -57,17 +58,18 @@ private[ml] trait CrossValidatorParams extends Params { val evaluator: Param[Evaluator] = new Param(this, "evaluator", "evaluator for selection") /** @group getParam */ - def getEvaluator: Evaluator = get(evaluator) + def getEvaluator: Evaluator = getOrDefault(evaluator) /** * param for number of folds for cross validation * @group param */ - val numFolds: IntParam = - new IntParam(this, "numFolds", "number of folds for cross validation", Some(3)) + val numFolds: IntParam = new IntParam(this, "numFolds", "number of folds for cross validation") /** @group getParam */ - def getNumFolds: Int = get(numFolds) + def getNumFolds: Int = getOrDefault(numFolds) + + setDefault(numFolds -> 3) } /** @@ -92,7 +94,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP def setNumFolds(value: Int): this.type = set(numFolds, value) override def fit(dataset: DataFrame, paramMap: ParamMap): CrossValidatorModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val schema = dataset.schema transformSchema(dataset.schema, paramMap, logging = true) val sqlCtx = dataset.sqlContext @@ -130,7 +132,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) map(estimator).transformSchema(schema, paramMap) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala new file mode 100644 index 0000000000000..0383bf0b382b7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -0,0 +1,61 @@ +/* + * 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.ml.util + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.types.{DataType, StructField, StructType} + +/** + * :: DeveloperApi :: + * Utils for handling schemas. + */ +@DeveloperApi +object SchemaUtils { + + // TODO: Move the utility methods to SQL. + + /** + * Check whether the given schema contains a column of the required data type. + * @param colName column name + * @param dataType required column data type + */ + def checkColumnType(schema: StructType, colName: String, dataType: DataType): Unit = { + val actualDataType = schema(colName).dataType + require(actualDataType.equals(dataType), + s"Column $colName must be of type $dataType but was actually $actualDataType.") + } + + /** + * Appends a new column to the input schema. This fails if the given output column already exists. + * @param schema input schema + * @param colName new column name. If this column name is an empty string "", this method returns + * the input schema unchanged. This allows users to disable output columns. + * @param dataType new column data type + * @return new schema with the input column appended + */ + def appendColumn( + schema: StructType, + colName: String, + dataType: DataType): StructType = { + if (colName.isEmpty) return schema + val fieldNames = schema.fieldNames + require(!fieldNames.contains(colName), s"Column $colName already exists.") + val outputFields = schema.fields :+ StructField(colName, dataType, nullable = false) + StructType(outputFields) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala index 1ce2987612378..88ea679eeaad5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala @@ -21,19 +21,25 @@ import org.scalatest.FunSuite class ParamsSuite extends FunSuite { - val solver = new TestParams() - import solver.{inputCol, maxIter} - test("param") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + assert(maxIter.name === "maxIter") assert(maxIter.doc === "max number of iterations") - assert(maxIter.defaultValue.get === 100) assert(maxIter.parent.eq(solver)) - assert(maxIter.toString === "maxIter: max number of iterations (default: 100)") - assert(inputCol.defaultValue === None) + assert(maxIter.toString === "maxIter: max number of iterations (default: 10)") + + solver.setMaxIter(5) + assert(maxIter.toString === "maxIter: max number of iterations (default: 10, current: 5)") + + assert(inputCol.toString === "inputCol: input column name (undefined)") } test("param pair") { + val solver = new TestParams() + import solver.maxIter + val pair0 = maxIter -> 5 val pair1 = maxIter.w(5) val pair2 = ParamPair(maxIter, 5) @@ -44,10 +50,12 @@ class ParamsSuite extends FunSuite { } test("param map") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val map0 = ParamMap.empty assert(!map0.contains(maxIter)) - assert(map0(maxIter) === maxIter.defaultValue.get) map0.put(maxIter, 10) assert(map0.contains(maxIter)) assert(map0(maxIter) === 10) @@ -78,23 +86,39 @@ class ParamsSuite extends FunSuite { } test("params") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val params = solver.params - assert(params.size === 2) + assert(params.length === 2) assert(params(0).eq(inputCol), "params must be ordered by name") assert(params(1).eq(maxIter)) + + assert(!solver.isSet(maxIter)) + assert(solver.isDefined(maxIter)) + assert(solver.getMaxIter === 10) + solver.setMaxIter(100) + assert(solver.isSet(maxIter)) + assert(solver.getMaxIter === 100) + assert(!solver.isSet(inputCol)) + assert(!solver.isDefined(inputCol)) + intercept[NoSuchElementException](solver.getInputCol) + assert(solver.explainParams() === Seq(inputCol, maxIter).mkString("\n")) + assert(solver.getParam("inputCol").eq(inputCol)) assert(solver.getParam("maxIter").eq(maxIter)) - intercept[NoSuchMethodException] { + intercept[NoSuchElementException] { solver.getParam("abc") } - assert(!solver.isSet(inputCol)) + intercept[IllegalArgumentException] { solver.validate() } solver.validate(ParamMap(inputCol -> "input")) solver.setInputCol("input") assert(solver.isSet(inputCol)) + assert(solver.isDefined(inputCol)) assert(solver.getInputCol === "input") solver.validate() intercept[IllegalArgumentException] { @@ -104,5 +128,8 @@ class ParamsSuite extends FunSuite { intercept[IllegalArgumentException] { solver.validate() } + + solver.clearMaxIter() + assert(!solver.isSet(maxIter)) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index ce52f2f230085..8f9ab687c05cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -20,17 +20,21 @@ package org.apache.spark.ml.param /** A subclass of Params for testing. */ class TestParams extends Params { - val maxIter = new IntParam(this, "maxIter", "max number of iterations", Some(100)) + val maxIter = new IntParam(this, "maxIter", "max number of iterations") def setMaxIter(value: Int): this.type = { set(maxIter, value); this } - def getMaxIter: Int = get(maxIter) + def getMaxIter: Int = getOrDefault(maxIter) val inputCol = new Param[String](this, "inputCol", "input column name") def setInputCol(value: String): this.type = { set(inputCol, value); this } - def getInputCol: String = get(inputCol) + def getInputCol: String = getOrDefault(inputCol) + + setDefault(maxIter -> 10) override def validate(paramMap: ParamMap): Unit = { - val m = this.paramMap ++ paramMap + val m = extractParamMap(paramMap) require(m(maxIter) >= 0) require(m.contains(inputCol)) } + + def clearMaxIter(): this.type = clear(maxIter) } From 77eeb10fd1f5beb558d6278d9443d267fc6c2f62 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 13 Apr 2015 22:31:44 -0700 Subject: [PATCH 015/144] [WIP][HOTFIX][SPARK-4123]: Fix bug in PR dependency (all deps. removed issue) We're seeing a bug sporadically in the new PR dependency comparison test whereby it notes that *all* dependencies are removed. This happens when the current PR is built, but the final, sorted, dependency file is left blank. I believe this is an error either in the way the `git checkout` calls have been or an error within the `mvn` build for that PR (again, likely related to the `git checkout`). As such I've set the checkouts to now force (with `-f` flag) which is more in line with what Jenkins currently does on the initial checkout. Setting this as a WIP for now to trigger the build process myriad times to see if the issue still arises. Author: Brennon York Closes #5443 from brennonyork/HOTFIX2-SPARK-4123 and squashes the following commits: f2186be [Brennon York] added output for the various git commit refs 3f073d6 [Brennon York] removed the git checkouts piping to dev null 07765a6 [Brennon York] updated the diff logic to reference the filenames rather than hardlink e3f63c7 [Brennon York] added '-f' to the checkout flags for git 710c8d1 [Brennon York] added 30 minutes to the test benchmark --- dev/run-tests-jenkins | 4 ++++ dev/tests/pr_new_dependencies.sh | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index f6372835a6dbf..3c1c91a111357 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -161,6 +161,10 @@ pr_message="" # Ensure we save off the current HEAD to revert to current_pr_head="`git rev-parse HEAD`" +echo "HEAD: `git rev-parse HEAD`" +echo "GHPRB: $ghprbActualCommit" +echo "SHA1: $sha1" + # Run pull request tests for t in "${PR_TESTS[@]}"; do this_test="${FWDIR}/dev/tests/${t}.sh" diff --git a/dev/tests/pr_new_dependencies.sh b/dev/tests/pr_new_dependencies.sh index 370c7cc737bbd..fdfb3c62aff58 100755 --- a/dev/tests/pr_new_dependencies.sh +++ b/dev/tests/pr_new_dependencies.sh @@ -39,12 +39,12 @@ CURR_CP_FILE="my-classpath.txt" MASTER_CP_FILE="master-classpath.txt" # First switch over to the master branch -git checkout master &>/dev/null +git checkout -f master # Find and copy all pom.xml files into a *.gate file that we can check # against through various `git` changes find -name "pom.xml" -exec cp {} {}.gate \; # Switch back to the current PR -git checkout "${current_pr_head}" &>/dev/null +git checkout -f "${current_pr_head}" # Check if any *.pom files from the current branch are different from the master difference_q="" @@ -71,7 +71,7 @@ else sort > ${CURR_CP_FILE} # Checkout the master branch to compare against - git checkout master &>/dev/null + git checkout -f master ${MVN_BIN} clean package dependency:build-classpath -DskipTests 2>/dev/null | \ sed -n -e '/Building Spark Project Assembly/,$p' | \ @@ -84,7 +84,7 @@ else rev | \ sort > ${MASTER_CP_FILE} - DIFF_RESULTS="`diff my-classpath.txt master-classpath.txt`" + DIFF_RESULTS="`diff ${CURR_CP_FILE} ${MASTER_CP_FILE}`" if [ -z "${DIFF_RESULTS}" ]; then echo " * This patch does not change any dependencies." From 628a72f70ed06b8d7aee81cfb16070eb2c87b9cd Mon Sep 17 00:00:00 2001 From: Punyashloka Biswal Date: Tue, 14 Apr 2015 11:43:06 +0100 Subject: [PATCH 016/144] [SPARK-6731] Bump version of apache commons-math3 Version 3.1.1 is two years old and the newer version includes approximate percentile statistics (among other things). Author: Punyashloka Biswal Closes #5380 from punya/patch-1 and squashes the following commits: 226622b [Punyashloka Biswal] Bump version of apache commons-math3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d8881c213bf07..0b8d664dbfe1b 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ 1.8.3 1.1.0 4.2.6 - 3.1.1 + 3.4.1 ${project.build.directory}/spark-test-classpath.txt 2.10.4 2.10 From 51b306b930cfe03ad21af72a3a6ef31e6e626235 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Tue, 14 Apr 2015 12:39:56 +0100 Subject: [PATCH 017/144] SPARK-6878 [CORE] Fix for sum on empty RDD fails with exception Author: Erik van Oosten Closes #5489 from erikvanoosten/master and squashes the following commits: 1c91954 [Erik van Oosten] Rewrote double range matcher to an exact equality assert (SPARK-6878) f1708c9 [Erik van Oosten] Fix for sum on empty RDD fails with exception (SPARK-6878) --- .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 2 +- .../test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 29ca3e9c4bd04..843a893235e56 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.StatCounter class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Add up the elements in this RDD. */ def sum(): Double = { - self.reduce(_ + _) + self.fold(0.0)(_ + _) } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 97079382c716f..01039b9449daf 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -22,6 +22,12 @@ import org.scalatest.FunSuite import org.apache.spark._ class DoubleRDDSuite extends FunSuite with SharedSparkContext { + test("sum") { + assert(sc.parallelize(Seq.empty[Double]).sum() === 0.0) + assert(sc.parallelize(Seq(1.0)).sum() === 1.0) + assert(sc.parallelize(Seq(1.0, 2.0)).sum() === 3.0) + } + // Verify tests on the histogram functionality. We test with both evenly // and non-evenly spaced buckets as the bucket lookup function changes. test("WorksOnEmpty") { From 320bca4508e890b874c2eb7abb76a30ef14c932f Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 14 Apr 2015 11:48:12 -0700 Subject: [PATCH 018/144] [SPARK-6081] Support fetching http/https uris in driver runner. Currently if passed uris such as http/https, it won't able to fetch them as it only calls HadoopFs get. This fix utilizes the existing util method to fetch remote uris as well. Author: Timothy Chen Closes #4832 from tnachen/driver_remote and squashes the following commits: aa52cd6 [Timothy Chen] Support fetching remote uris in driver runner. --- .../spark/deploy/worker/DriverRunner.scala | 21 ++++++++++++------- .../apache/spark/deploy/worker/Worker.scala | 3 ++- .../spark/deploy/JsonProtocolSuite.scala | 7 ++++--- .../deploy/worker/DriverRunnerTest.scala | 7 ++++--- 4 files changed, 23 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e0948e16ef354..ef7a703bffe67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -24,14 +24,14 @@ import scala.collection.JavaConversions._ import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.hadoop.fs.{FileUtil, Path} +import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SecurityManager} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.util.{Clock, SystemClock} +import org.apache.spark.util.{Utils, Clock, SystemClock} /** * Manages the execution of one driver, including automatically restarting the driver on failure. @@ -44,7 +44,8 @@ private[deploy] class DriverRunner( val sparkHome: File, val driverDesc: DriverDescription, val worker: ActorRef, - val workerUrl: String) + val workerUrl: String, + val securityManager: SecurityManager) extends Logging { @volatile private var process: Option[Process] = None @@ -136,12 +137,9 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { - val jarPath = new Path(driverDesc.jarUrl) val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val jarFileSystem = jarPath.getFileSystem(hadoopConf) - val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) @@ -149,7 +147,14 @@ private[deploy] class DriverRunner( if (!localJarFile.exists()) { // May already exist if running multiple workers on one node logInfo(s"Copying user jar $jarPath to $destPath") - FileUtil.copy(jarFileSystem, jarPath, destPath, false, hadoopConf) + Utils.fetchFile( + driverDesc.jarUrl, + driverDir, + conf, + securityManager, + hadoopConf, + System.currentTimeMillis(), + useCache = false) } if (!localJarFile.exists()) { // Verify copy succeeded 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 c4c24a7866aa3..3ee2eb69e8a4e 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 @@ -436,7 +436,8 @@ private[worker] class Worker( sparkHome, driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), self, - akkaUrl) + akkaUrl, + securityMgr) drivers(driverId) = driver driver.start() 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 2071701b313db..b58d62567afe1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} class JsonProtocolSuite extends FunSuite { @@ -124,8 +124,9 @@ class JsonProtocolSuite extends FunSuite { } def createDriverRunner(): DriverRunner = { - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - createDriverDesc(), null, "akka://worker") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + createDriverDesc(), null, "akka://worker", new SecurityManager(conf)) } def assertValidJson(json: JValue) { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index aa6e4874cecde..2159fd8c16c6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -25,7 +25,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.FunSuite -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.util.Clock @@ -33,8 +33,9 @@ class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - driverDescription, null, "akka://1.2.3.4/worker/") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + driverDescription, null, "akka://1.2.3.4/worker/", new SecurityManager(conf)) } private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { From f63b44a5c201d9678738a906462be9a6d7e3e8f8 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 14 Apr 2015 12:02:11 -0700 Subject: [PATCH 019/144] [SPARK-6894]spark.executor.extraLibraryOptions => spark.executor.extraLibraryPath https://issues.apache.org/jira/browse/SPARK-6894 cc vanzin Author: WangTaoTheTonic Closes #5506 from WangTaoTheTonic/SPARK-6894 and squashes the following commits: 4b7ced7 [WangTaoTheTonic] spark.executor.extraLibraryOptions => spark.executor.extraLibraryPath --- .../src/main/java/org/apache/spark/launcher/SparkLauncher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index b566507ee6061..d4cfeacb6ef18 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -52,7 +52,7 @@ public class SparkLauncher { /** Configuration key for the executor VM options. */ public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; /** Configuration key for the executor native library path. */ - public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions"; + public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryPath"; /** Configuration key for the number of executor CPU cores. */ public static final String EXECUTOR_CORES = "spark.executor.cores"; From dcf8a9f331c6193a62bbc9282bdc99663e23ca19 Mon Sep 17 00:00:00 2001 From: pankaj arora Date: Tue, 14 Apr 2015 12:06:46 -0700 Subject: [PATCH 020/144] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure Fixed null check when all the dependent stages are cancelled due to previous stage failure. This happens when one of the executor node goes down and all the dependent stages are cancelled. Author: pankaj arora Closes #5494 from pankajarora12/NEWBRANCH and squashes the following commits: 55ba5e3 [pankaj arora] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure 4575720 [pankaj arora] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 +------ 1 file changed, 1 insertion(+), 6 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 508fe7b3303ca..4a32f8936fb0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -818,12 +818,7 @@ class DAGScheduler( } } - val properties = if (jobIdToActiveJob.contains(jobId)) { - jobIdToActiveJob(stage.jobId).properties - } else { - // this stage will be assigned to "default" pool - null - } + val properties = jobIdToActiveJob.get(stage.jobId).map(_.properties).orNull runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are From 25998e4d73bcc95ac85d9af71adfdc726ec89568 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Tue, 14 Apr 2015 12:56:47 -0700 Subject: [PATCH 021/144] [SPARK-2033] Automatically cleanup checkpoint Author: GuoQiang Li Closes #855 from witgo/cleanup_checkpoint_date and squashes the following commits: 1649850 [GuoQiang Li] review commit c0087e0 [GuoQiang Li] Automatically cleanup checkpoint --- .../org/apache/spark/ContextCleaner.scala | 44 ++++++++++++----- .../apache/spark/rdd/RDDCheckpointData.scala | 27 ++++++++-- .../apache/spark/ContextCleanerSuite.scala | 49 ++++++++++++++++++- 3 files changed, 102 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 9b05c9623b704..715b259057569 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -22,7 +22,7 @@ import java.lang.ref.{ReferenceQueue, WeakReference} import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.util.Utils /** @@ -33,6 +33,7 @@ private case class CleanRDD(rddId: Int) extends CleanupTask private case class CleanShuffle(shuffleId: Int) extends CleanupTask private case class CleanBroadcast(broadcastId: Long) extends CleanupTask private case class CleanAccum(accId: Long) extends CleanupTask +private case class CleanCheckpoint(rddId: Int) extends CleanupTask /** * A WeakReference associated with a CleanupTask. @@ -94,12 +95,12 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { @volatile private var stopped = false /** Attach a listener object to get information of when objects are cleaned. */ - def attachListener(listener: CleanerListener) { + def attachListener(listener: CleanerListener): Unit = { listeners += listener } /** Start the cleaner. */ - def start() { + def start(): Unit = { cleaningThread.setDaemon(true) cleaningThread.setName("Spark Context Cleaner") cleaningThread.start() @@ -108,7 +109,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** * Stop the cleaning thread and wait until the thread has finished running its current task. */ - def stop() { + def stop(): Unit = { stopped = true // Interrupt the cleaning thread, but wait until the current task has finished before // doing so. This guards against the race condition where a cleaning thread may @@ -121,7 +122,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a RDD for cleanup when it is garbage collected. */ - def registerRDDForCleanup(rdd: RDD[_]) { + def registerRDDForCleanup(rdd: RDD[_]): Unit = { registerForCleanup(rdd, CleanRDD(rdd.id)) } @@ -130,17 +131,22 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a ShuffleDependency for cleanup when it is garbage collected. */ - def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]) { + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]): Unit = { registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) } /** Register a Broadcast for cleanup when it is garbage collected. */ - def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { + def registerBroadcastForCleanup[T](broadcast: Broadcast[T]): Unit = { registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) } + /** Register a RDDCheckpointData for cleanup when it is garbage collected. */ + def registerRDDCheckpointDataForCleanup[T](rdd: RDD[_], parentId: Int): Unit = { + registerForCleanup(rdd, CleanCheckpoint(parentId)) + } + /** Register an object for cleanup. */ - private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) } @@ -164,6 +170,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) case CleanAccum(accId) => doCleanupAccum(accId, blocking = blockOnCleanupTasks) + case CleanCheckpoint(rddId) => + doCleanCheckpoint(rddId) } } } @@ -175,7 +183,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform RDD cleanup. */ - def doCleanupRDD(rddId: Int, blocking: Boolean) { + def doCleanupRDD(rddId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning RDD " + rddId) sc.unpersistRDD(rddId, blocking) @@ -187,7 +195,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform shuffle cleanup, asynchronously. */ - def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { + def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning shuffle " + shuffleId) mapOutputTrackerMaster.unregisterShuffle(shuffleId) @@ -200,7 +208,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform broadcast cleanup. */ - def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { + def doCleanupBroadcast(broadcastId: Long, blocking: Boolean): Unit = { try { logDebug(s"Cleaning broadcast $broadcastId") broadcastManager.unbroadcast(broadcastId, true, blocking) @@ -212,7 +220,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform accumulator cleanup. */ - def doCleanupAccum(accId: Long, blocking: Boolean) { + def doCleanupAccum(accId: Long, blocking: Boolean): Unit = { try { logDebug("Cleaning accumulator " + accId) Accumulators.remove(accId) @@ -223,6 +231,18 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } + /** Perform checkpoint cleanup. */ + def doCleanCheckpoint(rddId: Int): Unit = { + try { + logDebug("Cleaning rdd checkpoint data " + rddId) + RDDCheckpointData.clearRDDCheckpointData(sc, rddId) + logInfo("Cleaned rdd checkpoint data " + rddId) + } + catch { + case e: Exception => logError("Error cleaning rdd checkpoint data " + rddId, e) + } + } + private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 6afd63d537d75..1722c27e55003 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, Partition, SerializableWritable, SparkException} +import org.apache.spark._ import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} /** @@ -83,7 +83,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } // Create the output path for the checkpoint - val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) + val path = RDDCheckpointData.rddCheckpointDataPath(rdd.context, rdd.id).get val fs = path.getFileSystem(rdd.context.hadoopConfiguration) if (!fs.mkdirs(path)) { throw new SparkException("Failed to create checkpoint path " + path) @@ -92,8 +92,13 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) // Save to file, and reload it as an RDD val broadcastedConf = rdd.context.broadcast( new SerializableWritable(rdd.context.hadoopConfiguration)) - rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) + if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) { + rdd.context.cleaner.foreach { cleaner => + cleaner.registerRDDCheckpointDataForCleanup(newRDD, rdd.id) + } + } + rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) if (newRDD.partitions.length != rdd.partitions.length) { throw new SparkException( "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.length + ") has different " + @@ -130,5 +135,17 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -// Used for synchronization -private[spark] object RDDCheckpointData +private[spark] object RDDCheckpointData { + def rddCheckpointDataPath(sc: SparkContext, rddId: Int): Option[Path] = { + sc.checkpointDir.map { dir => new Path(dir, "rdd-" + rddId) } + } + + def clearRDDCheckpointData(sc: SparkContext, rddId: Int): Unit = { + rddCheckpointDataPath(sc, rddId).foreach { path => + val fs = path.getFileSystem(sc.hadoopConfiguration) + if (fs.exists(path)) { + fs.delete(path, true) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 1de169d964d23..097e7076e5391 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,7 +28,8 @@ import org.scalatest.concurrent.{PatienceConfiguration, Eventually} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.storage._ import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.shuffle.sort.SortShuffleManager @@ -205,6 +206,52 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { postGCTester.assertCleanup() } + test("automatically cleanup checkpoint") { + val checkpointDir = java.io.File.createTempFile("temp", "") + checkpointDir.deleteOnExit() + checkpointDir.delete() + var rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + var rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).isDefined) + val path = RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get + val fs = path.getFileSystem(sc.hadoopConfiguration) + assert(fs.exists(path)) + + // the checkpoint is not cleaned by default (without the configuration set) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + sc.stop() + val conf = new SparkConf().setMaster("local[2]").setAppName("cleanupCheckpoint"). + set("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + sc = new SparkContext(conf) + rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + // Test that GC causes checkpoint data cleanup after dereferencing the RDD + postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(!fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + } + test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly From 8f8dc45f6d4c8d7b740eaa3d2ea09d0b531af9dd Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 14 Apr 2015 13:32:06 -0700 Subject: [PATCH 022/144] SPARK-1706: Allow multiple executors per worker in Standalone mode resubmit of https://github.com/apache/spark/pull/636 for a totally different algorithm https://issues.apache.org/jira/browse/SPARK-1706 In current implementation, the user has to start multiple workers in a server for starting multiple executors in a server, which introduces additional overhead due to the more JVM processes... In this patch, I changed the scheduling logic in master to enable the user to start multiple executor processes within the same JVM process. 1. user configure spark.executor.maxCoreNumPerExecutor to suggest the maximum core he/she would like to allocate to each executor 2. Master assigns the executors to the workers with the major consideration on the memoryPerExecutor and the worker.freeMemory, and tries to allocate as many as possible cores to the executor ```min(min(memoryPerExecutor, worker.freeCore), maxLeftCoreToAssign)``` where ```maxLeftCoreToAssign = maxExecutorCanAssign * maxCoreNumPerExecutor``` --------------------------------------- Other small changes include change memoryPerSlave in ApplicationDescription to memoryPerExecutor, as "Slave" is overrided to represent both worker and executor in the documents... (we have some discussion on this before?) Author: CodingCat Closes #731 from CodingCat/SPARK-1706-2 and squashes the following commits: 6dee808 [CodingCat] change filter predicate fbeb7e5 [CodingCat] address the comments 940cb42 [CodingCat] avoid unnecessary allocation b8ca561 [CodingCat] revert a change 45967b4 [CodingCat] remove unused method 2eeff77 [CodingCat] stylistic fixes 12a1b32 [CodingCat] change the semantic of coresPerExecutor to exact core number f035423 [CodingCat] stylistic fix d9c1685 [CodingCat] remove unused var f595bd6 [CodingCat] recover some unintentional changes 63b3df9 [CodingCat] change the description of the parameter in the submit script 4cf61f1 [CodingCat] improve the code and docs ff011e2 [CodingCat] start multiple executors on the worker by rewriting startExeuctor logic 2c2bcc5 [CodingCat] fix wrong usage info 497ec2c [CodingCat] address andrew's comments 878402c [CodingCat] change the launching executor code f64a28d [CodingCat] typo fix 387f4ec [CodingCat] bug fix 35c462c [CodingCat] address Andrew's comments 0b64fea [CodingCat] fix compilation issue 19d3da7 [CodingCat] address the comments 5b81466 [CodingCat] remove outdated comments ec7d421 [CodingCat] test commit e5efabb [CodingCat] more java docs and consolidate canUse function a26096d [CodingCat] stylistic fix a5d629a [CodingCat] java doc b34ec0c [CodingCat] make master support multiple executors per worker --- .../spark/deploy/ApplicationDescription.scala | 9 +- .../apache/spark/deploy/JsonProtocol.scala | 4 +- .../org/apache/spark/deploy/SparkSubmit.scala | 2 + .../spark/deploy/SparkSubmitArguments.scala | 5 +- .../spark/deploy/master/ApplicationInfo.scala | 8 +- .../apache/spark/deploy/master/Master.scala | 117 +++++++++--------- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 7 +- docs/configuration.md | 11 ++ 10 files changed, 96 insertions(+), 73 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 b7ae9c1fc0a23..ae99432f5ce86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -22,12 +22,13 @@ import java.net.URI private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], - val memoryPerSlave: Int, + val memoryPerExecutorMB: Int, val command: Command, var appUiUrl: String, val eventLogDir: Option[URI] = None, // short name of compression codec used when writing event logs, if any (e.g. lzf) - val eventLogCodec: Option[String] = None) + val eventLogCodec: Option[String] = None, + val coresPerExecutor: Option[Int] = None) extends Serializable { val user = System.getProperty("user.name", "") @@ -35,13 +36,13 @@ private[spark] class ApplicationDescription( def copy( name: String = name, maxCores: Option[Int] = maxCores, - memoryPerSlave: Int = memoryPerSlave, + memoryPerExecutorMB: Int = memoryPerExecutorMB, command: Command = command, appUiUrl: String = appUiUrl, eventLogDir: Option[URI] = eventLogDir, eventLogCodec: Option[String] = eventLogCodec): ApplicationDescription = new ApplicationDescription( - name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir, eventLogCodec) + name, maxCores, memoryPerExecutorMB, command, appUiUrl, eventLogDir, eventLogCodec) override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index dfc5b97e6a6c8..2954f932b4f41 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -46,7 +46,7 @@ private[deploy] object JsonProtocol { ("name" -> obj.desc.name) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ - ("memoryperslave" -> obj.desc.memoryPerSlave) ~ + ("memoryperslave" -> obj.desc.memoryPerExecutorMB) ~ ("submitdate" -> obj.submitDate.toString) ~ ("state" -> obj.state.toString) ~ ("duration" -> obj.duration) @@ -55,7 +55,7 @@ private[deploy] object JsonProtocol { def writeApplicationDescription(obj: ApplicationDescription): JObject = { ("name" -> obj.name) ~ ("cores" -> obj.maxCores) ~ - ("memoryperslave" -> obj.memoryPerSlave) ~ + ("memoryperslave" -> obj.memoryPerExecutorMB) ~ ("user" -> obj.user) ~ ("command" -> obj.command.toString) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 60bc243ebf40a..296a0764b8baf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -406,6 +406,8 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), // Other options + OptionAssigner(args.executorCores, STANDALONE, ALL_DEPLOY_MODES, + sysProp = "spark.executor.cores"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, 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 03ecf3fd99ec5..faa8780288ea3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -482,10 +482,13 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. | + | Spark standalone and YARN only: + | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, + | or all available cores on the worker in standalone mode) + | | YARN-only: | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). - | --executor-cores NUM Number of cores per executor (Default: 1). | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index bc5b293379f2b..f59d550d4f3b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -75,9 +75,11 @@ private[deploy] class ApplicationInfo( } } - private[master] def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): - ExecutorDesc = { - val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) + private[master] def addExecutor( + worker: WorkerInfo, + cores: Int, + useID: Option[Int] = None): ExecutorDesc = { + val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerExecutorMB) executors(exec.id) = exec coresGranted += cores exec 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 9a5d5877da86d..c5a6b1beac9be 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 @@ -524,52 +524,28 @@ private[master] class Master( } /** - * Can an app use the given worker? True if the worker has enough memory and we haven't already - * launched an executor for the app on it (right now the standalone backend doesn't like having - * two executors on the same worker). - */ - private def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { - worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) - } - - /** - * Schedule the currently available resources among waiting apps. This method will be called - * every time a new app joins or resource availability changes. + * Schedule executors to be launched on the workers. + * + * There are two modes of launching executors. The first attempts to spread out an application's + * executors on as many workers as possible, while the second does the opposite (i.e. launch them + * on as few workers as possible). The former is usually better for data locality purposes and is + * the default. + * + * The number of cores assigned to each executor is configurable. When this is explicitly set, + * multiple executors from the same application may be launched on the same worker if the worker + * has enough cores and memory. Otherwise, each executor grabs all the cores available on the + * worker by default, in which case only one executor may be launched on each worker. */ - private def schedule() { - if (state != RecoveryState.ALIVE) { return } - - // First schedule drivers, they take strict precedence over applications - // Randomization helps balance drivers - 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 - // explored all alive workers. - var launched = false - var numWorkersVisited = 0 - while (numWorkersVisited < numWorkersAlive && !launched) { - val worker = shuffledAliveWorkers(curPos) - numWorkersVisited += 1 - if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { - launchDriver(worker, driver) - waitingDrivers -= driver - launched = true - } - curPos = (curPos + 1) % numWorkersAlive - } - } - + private def startExecutorsOnWorkers(): Unit = { // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app // in the queue, then the second app, etc. if (spreadOutApps) { - // Try to spread out each app among all the nodes, until it has all its cores + // Try to spread out each app among all the workers, until it has all its cores for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(app, _)).sortBy(_.coresFree).reverse + .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && + worker.coresFree >= app.desc.coresPerExecutor.getOrElse(1)) + .sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) @@ -582,32 +558,61 @@ private[master] class Master( pos = (pos + 1) % numUsable } // Now that we've decided how many cores to give on each node, let's actually give them - for (pos <- 0 until numUsable) { - if (assigned(pos) > 0) { - val exec = app.addExecutor(usableWorkers(pos), assigned(pos)) - launchExecutor(usableWorkers(pos), exec) - app.state = ApplicationState.RUNNING - } + for (pos <- 0 until numUsable if assigned(pos) > 0) { + allocateWorkerResourceToExecutors(app, assigned(pos), usableWorkers(pos)) } } } else { - // Pack each app into as few nodes as possible until we've assigned all its cores + // Pack each app into as few workers as possible until we've assigned all its cores for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) { for (app <- waitingApps if app.coresLeft > 0) { - if (canUse(app, worker)) { - val coresToUse = math.min(worker.coresFree, app.coresLeft) - if (coresToUse > 0) { - val exec = app.addExecutor(worker, coresToUse) - launchExecutor(worker, exec) - app.state = ApplicationState.RUNNING - } - } + allocateWorkerResourceToExecutors(app, app.coresLeft, worker) + } + } + } + } + + /** + * Allocate a worker's resources to one or more executors. + * @param app the info of the application which the executors belong to + * @param coresToAllocate cores on this worker to be allocated to this application + * @param worker the worker info + */ + private def allocateWorkerResourceToExecutors( + app: ApplicationInfo, + coresToAllocate: Int, + worker: WorkerInfo): Unit = { + val memoryPerExecutor = app.desc.memoryPerExecutorMB + val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(coresToAllocate) + var coresLeft = coresToAllocate + while (coresLeft >= coresPerExecutor && worker.memoryFree >= memoryPerExecutor) { + val exec = app.addExecutor(worker, coresPerExecutor) + coresLeft -= coresPerExecutor + launchExecutor(worker, exec) + app.state = ApplicationState.RUNNING + } + } + + /** + * Schedule the currently available resources among waiting apps. This method will be called + * every time a new app joins or resource availability changes. + */ + private def schedule(): Unit = { + if (state != RecoveryState.ALIVE) { return } + // Drivers take strict precedence over executors + val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers + for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { + for (driver <- waitingDrivers) { + if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { + launchDriver(worker, driver) + waitingDrivers -= driver } } } + startExecutorsOnWorkers() } - private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { + private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc): Unit = { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 761aa8f7b1ef6..273f077bd8f57 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -94,7 +94,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")

  • Executor Memory: - {Utils.megabytesToString(app.desc.memoryPerSlave)} + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)}
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 45412a35e9a7d..399f07399a0aa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -208,8 +208,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {app.coresGranted} - - {Utils.megabytesToString(app.desc.memoryPerSlave)} + + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)} {UIUtils.formatDate(app.submitDate)} {app.desc.user} 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 7eb3fdc19b5b8..ed5b7c1088196 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 @@ -82,12 +82,11 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", 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, sc.eventLogCodec) - + val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, + command, appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() - waitForRegistration() } diff --git a/docs/configuration.md b/docs/configuration.md index 7169ec295ef7f..d9e9e67026cbb 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -723,6 +723,17 @@ Apart from these, the following properties are also available, and may be useful this duration will be cleared as well. + + spark.executor.cores + 1 in YARN mode, all the available cores on the worker in standalone mode. + + The number of cores to use on each executor. For YARN and standalone mode only. + + In standalone mode, setting this parameter allows an application to run multiple executors on + the same worker, provided that there are enough cores on that worker. Otherwise, only one + executor per application will run on each worker. + + spark.default.parallelism From b075e4b720221a8204cae93468065a6708348830 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 13:34:44 -0700 Subject: [PATCH 023/144] [SPARK-6700] [yarn] Re-enable flaky test. Test runs have been successful on jenkins. So let's re-enable the test and look out for any failures, and fix things appropriately. Author: Marcelo Vanzin Closes #5459 from vanzin/SPARK-6700 and squashes the following commits: 2ead85b [Marcelo Vanzin] WIP: re-enable flaky test to catch failure in jenkins. --- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index c06c0105670c0..76952e3341cc4 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -144,7 +144,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } // Enable this once fix SPARK-6700 - ignore("run Python application in yarn-cluster mode") { + test("run Python application in yarn-cluster mode") { val primaryPyFile = new File(tempDir, "test.py") Files.write(TEST_PYFILE, primaryPyFile, UTF_8) val pyFile = new File(tempDir, "test2.py") From 6adb8bcbf0a1a7bfe2990de18c59c66cd7a0aeb8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 14 Apr 2015 13:40:07 -0700 Subject: [PATCH 024/144] [SPARK-6905] Upgrade to snappy-java 1.1.1.7 We should upgrade our snappy-java dependency to 1.1.1.7 in order to include a fix for a bug that results in worse compression in SnappyOutputStream (see https://github.com/xerial/snappy-java/issues/100). Author: Josh Rosen Closes #5512 from JoshRosen/snappy-1.1.1.7 and squashes the following commits: f1ac0f8 [Josh Rosen] Upgrade to snappy-java 1.1.1.7. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0b8d664dbfe1b..261292d5b6cde 100644 --- a/pom.xml +++ b/pom.xml @@ -156,7 +156,7 @@ 3.6.3 1.8.8 2.4.4 - 1.1.1.6 + 1.1.1.7 1.1.2 ${java.home} From 65774370a1275e25cd8a3357e397d116767793a9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 13:41:38 -0700 Subject: [PATCH 025/144] [SPARK-5808] [build] Package pyspark files in sbt assembly. This turned out to be more complicated than I wanted because the layout of python/ doesn't really follow the usual maven conventions. So some extra code is needed to copy just the right things. Author: Marcelo Vanzin Closes #5461 from vanzin/SPARK-5808 and squashes the following commits: 7153dac [Marcelo Vanzin] Only try to create resource dir if it doesn't already exist. ee90e84 [Marcelo Vanzin] [SPARK-5808] [build] Package pyspark files in sbt assembly. --- project/SparkBuild.scala | 60 +++++++++++++++++++++++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f51f4b58f97a..09b4976d10c26 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -import java.io.File +import java.io._ import scala.util.Properties import scala.collection.JavaConversions._ @@ -166,6 +166,9 @@ object SparkBuild extends PomBuild { /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) + /* Package pyspark artifacts in the main assembly. */ + enable(PySparkAssembly.settings)(assembly) + /* Enable unidoc only for the root spark project */ enable(Unidoc.settings)(spark) @@ -316,6 +319,7 @@ object Hive { } object Assembly { + import sbtassembly.AssemblyUtils._ import sbtassembly.Plugin._ import AssemblyKeys._ @@ -347,6 +351,60 @@ object Assembly { ) } +object PySparkAssembly { + import sbtassembly.Plugin._ + import AssemblyKeys._ + + lazy val settings = Seq( + unmanagedJars in Compile += { BuildCommons.sparkHome / "python/lib/py4j-0.8.2.1-src.zip" }, + // Use a resource generator to copy all .py files from python/pyspark into a managed directory + // to be included in the assembly. We can't just add "python/" to the assembly's resource dir + // list since that will copy unneeded / unwanted files. + resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File => + val dst = new File(outDir, "pyspark") + if (!dst.isDirectory()) { + require(dst.mkdirs()) + } + + val src = new File(BuildCommons.sparkHome, "python/pyspark") + copy(src, dst) + } + ) + + private def copy(src: File, dst: File): Seq[File] = { + src.listFiles().flatMap { f => + val child = new File(dst, f.getName()) + if (f.isDirectory()) { + child.mkdir() + copy(f, child) + } else if (f.getName().endsWith(".py")) { + var in: Option[FileInputStream] = None + var out: Option[FileOutputStream] = None + try { + in = Some(new FileInputStream(f)) + out = Some(new FileOutputStream(child)) + + val bytes = new Array[Byte](1024) + var read = 0 + while (read >= 0) { + read = in.get.read(bytes) + if (read > 0) { + out.get.write(bytes, 0, read) + } + } + + Some(child) + } finally { + in.foreach(_.close()) + out.foreach(_.close()) + } + } else { + None + } + } + } +} + object Unidoc { import BuildCommons._ From 4d4b24927417b2c17810e94d6d46c37491c68869 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 14 Apr 2015 14:00:49 -0700 Subject: [PATCH 026/144] [SPARK-6769][YARN][TEST] Usage of the ListenerBus in YarnClusterSuite is wrong In YarnClusterSuite, a test case uses `SaveExecutorInfo` to handle ExecutorAddedEvent as follows. ``` private class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo } } ... listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf() .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) sc.addSparkListener(listener) val status = new File(args(0)) var result = "failure" try { val data = sc.parallelize(1 to 4, 4).collect().toSet assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) data should be (Set(1, 2, 3, 4)) result = "success" } finally { sc.stop() Files.write(result, status, UTF_8) } ``` But, the usage is wrong because Executors will spawn during initializing SparkContext and SparkContext#addSparkListener should be invoked after the initialization, thus after Executors spawn, so SaveExecutorInfo cannot handle ExecutorAddedEvent. Following code refers the result of the handling ExecutorAddedEvent. Because of the reason above, we cannot reach the assertion. ``` // verify log urls are present listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) } ``` Author: Kousuke Saruta Closes #5417 from sarutak/SPARK-6769 and squashes the following commits: 8adc8ba [Kousuke Saruta] Fixed compile error e258530 [Kousuke Saruta] Fixed style 591cf3e [Kousuke Saruta] Fixed style 48ec89a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 860c965 [Kousuke Saruta] Simplified code 207d325 [Kousuke Saruta] Added findListenersByClass method to ListenerBus 2408c84 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 2d7e409 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 3874adf [Kousuke Saruta] Fixed the usage of listener bus in LogUrlsStandaloneSuite 153a91b [Kousuke Saruta] Fixed the usage of listener bus in YarnClusterSuite --- .../org/apache/spark/util/ListenerBus.scala | 8 ++++++++ .../spark/deploy/LogUrlsStandaloneSuite.scala | 20 ++++++++++--------- .../spark/deploy/yarn/YarnClusterSuite.scala | 17 +++++++++------- 3 files changed, 29 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index d60b8b9a31a9b..a725767d08cc2 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -19,9 +19,12 @@ package org.apache.spark.util import java.util.concurrent.CopyOnWriteArrayList +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark.Logging +import org.apache.spark.scheduler.SparkListener /** * An event bus which posts events to its listeners. @@ -64,4 +67,9 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { */ def onPostEvent(listener: L, event: E): Unit + private[spark] def findListenersByClass[T <: L : ClassTag](): Seq[T] = { + val c = implicitly[ClassTag[T]].runtimeClass + listeners.filter(_.getClass == c).map(_.asInstanceOf[T]).toSeq + } + } diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index 9cdb42814ca32..c93d16f8a1586 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import java.net.URL +import scala.collection.JavaConversions._ import scala.collection.mutable import scala.io.Source @@ -65,16 +66,17 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { new MySparkConf().setAll(getAll) } } - val conf = new MySparkConf() + val conf = new MySparkConf().set( + "spark.extraListeners", classOf[SaveExecutorInfo].getName) sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - val listener = new SaveExecutorInfo - sc.addSparkListener(listener) - // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) info.logUrlMap.values.foreach { logUrl => @@ -82,12 +84,12 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { } } } +} - private class SaveExecutorInfo extends SparkListener { - val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() +private[spark] class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor: SparkListenerExecutorAdded) { - addedExecutorInfos(executor.executorId) = executor.executorInfo - } + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 76952e3341cc4..a18c94d4ab4a8 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException, TestUtils} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} +import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListener, SparkListenerExecutorAdded} import org.apache.spark.util.Utils /** @@ -282,10 +282,10 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } -private class SaveExecutorInfo extends SparkListener { +private[spark] class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor : SparkListenerExecutorAdded) { + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo } } @@ -293,7 +293,6 @@ private class SaveExecutorInfo extends SparkListener { private object YarnClusterDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 - var listener: SaveExecutorInfo = null def main(args: Array[String]): Unit = { if (args.length != 1) { @@ -306,10 +305,9 @@ private object YarnClusterDriver extends Logging with Matchers { System.exit(1) } - listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) - sc.addSparkListener(listener) val status = new File(args(0)) var result = "failure" try { @@ -323,7 +321,12 @@ private object YarnClusterDriver extends Logging with Matchers { } // verify log urls are present - listener.addedExecutorInfos.values.foreach { info => + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) + val executorInfos = listener.addedExecutorInfos.values + assert(executorInfos.nonEmpty) + executorInfos.foreach { info => assert(info.logUrlMap.nonEmpty) } } From a76b921a923ac37d3c73ee18d24df4bb611daba3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 14 Apr 2015 14:07:25 -0700 Subject: [PATCH 027/144] Revert "[SPARK-6352] [SQL] Add DirectParquetOutputCommitter" This reverts commit b29663eeea440b1d1a288d41b5ddf67e77c5bd54. I'm reverting this because it broke test compilation for the Hadoop 1.x profiles. --- .../DirectParquetOutputCommitter.scala | 66 ------------------- .../sql/parquet/ParquetTableOperations.scala | 22 ------- .../spark/sql/parquet/ParquetIOSuite.scala | 21 ------ 3 files changed, 109 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala deleted file mode 100644 index 25a66cb488103..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala +++ /dev/null @@ -1,66 +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.sql.parquet - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter - -import parquet.Log -import parquet.hadoop.util.ContextUtil -import parquet.hadoop.{ParquetFileReader, ParquetFileWriter, ParquetOutputCommitter} - -private[parquet] class DirectParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) - extends ParquetOutputCommitter(outputPath, context) { - val LOG = Log.getLog(classOf[ParquetOutputCommitter]) - - override def getWorkPath(): Path = outputPath - override def abortTask(taskContext: TaskAttemptContext): Unit = {} - override def commitTask(taskContext: TaskAttemptContext): Unit = {} - override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true - override def setupJob(jobContext: JobContext): Unit = {} - override def setupTask(taskContext: TaskAttemptContext): Unit = {} - - override def commitJob(jobContext: JobContext) { - try { - val configuration = ContextUtil.getConfiguration(jobContext) - val fileSystem = outputPath.getFileSystem(configuration) - val outputStatus = fileSystem.getFileStatus(outputPath) - val footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus) - try { - ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers) - if (configuration.getBoolean("mapreduce.fileoutputcommitter.marksuccessfuljobs", true)) { - val successPath = new Path(outputPath, FileOutputCommitter.SUCCEEDED_FILE_NAME) - fileSystem.create(successPath).close() - } - } catch { - case e: Exception => { - LOG.warn("could not write summary file for " + outputPath, e) - val metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE) - if (fileSystem.exists(metadataPath)) { - fileSystem.delete(metadataPath, true) - } - } - } - } catch { - case e: Exception => LOG.warn("could not write summary file for " + outputPath, e) - } - } - -} - 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 3724bda829d30..1c868da23e060 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 @@ -379,8 +379,6 @@ private[sql] case class InsertIntoParquetTable( */ private[parquet] class AppendingParquetOutputFormat(offset: Int) extends parquet.hadoop.ParquetOutputFormat[Row] { - var committer: OutputCommitter = null - // override to accept existing directories as valid output directory override def checkOutputSpecs(job: JobContext): Unit = {} @@ -405,26 +403,6 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] } - - // override to create output committer from configuration - override def getOutputCommitter(context: TaskAttemptContext): OutputCommitter = { - if (committer == null) { - val output = getOutputPath(context) - val cls = context.getConfiguration.getClass("spark.sql.parquet.output.committer.class", - classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) - val ctor = cls.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) - committer = ctor.newInstance(output, context).asInstanceOf[ParquetOutputCommitter] - } - committer - } - - // FileOutputFormat.getOutputPath takes JobConf in hadoop-1 but JobContext in hadoop-2 - private def getOutputPath(context: TaskAttemptContext): Path = { - context.getConfiguration().get("mapred.output.dir") match { - case null => null - case name => new Path(name) - } - } } /** 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 index 4d0bf7cf99cdf..97c0f439acf13 100644 --- 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 @@ -381,27 +381,6 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } } - - test("SPARK-6352 DirectParquetOutputCommitter") { - // Write to a parquet file and let it fail. - // _temporary should be missing if direct output committer works. - try { - configuration.set("spark.sql.parquet.output.committer.class", - "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") - sqlContext.udf.register("div0", (x: Int) => x / 0) - withTempPath { dir => - intercept[org.apache.spark.SparkException] { - sqlContext.sql("select div0(1)").saveAsParquetFile(dir.getCanonicalPath) - } - val path = new Path(dir.getCanonicalPath, "_temporary") - val fs = path.getFileSystem(configuration) - assert(!fs.exists(path)) - } - } - finally { - configuration.unset("spark.sql.parquet.output.committer.class") - } - } } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From 6de282e2de3cb69f9b746d03fde581429248824a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 14 Apr 2015 16:51:36 -0700 Subject: [PATCH 028/144] [SPARK-6796][Streaming][WebUI] Add "Active Batches" and "Completed Batches" lists to StreamingPage This PR adds two lists, `Active Batches` and `Completed Batches`. Here is the screenshot: ![batch_list](https://cloud.githubusercontent.com/assets/1000778/7060458/d8898572-deb3-11e4-938b-6f8602c71a9f.png) Due to [SPARK-6766](https://issues.apache.org/jira/browse/SPARK-6766), I need to merge #5414 in my local machine to get the above screenshot. Author: zsxwing Closes #5434 from zsxwing/SPARK-6796 and squashes the following commits: be50fc6 [zsxwing] Fix the code style 51b792e [zsxwing] Fix the unit test 6f3078e [zsxwing] Make 'startTime' readable f40e0a9 [zsxwing] Merge branch 'master' into SPARK-6796 2525336 [zsxwing] Rename 'Processed batches' and 'Waiting batches' and also add links a69c091 [zsxwing] Show the number of total completed batches too a12ad7b [zsxwing] Change 'records' to 'events' in the UI 86b5e7f [zsxwing] Make BatchTableBase abstract b248787 [zsxwing] Add tests to verify the new tables d18ab7d [zsxwing] Fix the code style 6ceffb3 [zsxwing] Add "Active Batches" and "Completed Batches" lists to StreamingPage --- .../spark/streaming/ui/AllBatchesTable.scala | 114 ++++++++++++++++++ .../spark/streaming/ui/StreamingPage.scala | 44 +++++-- .../spark/streaming/UISeleniumSuite.scala | 11 ++ 3 files changed, 159 insertions(+), 10 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala new file mode 100644 index 0000000000000..df1c0a10704c3 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala @@ -0,0 +1,114 @@ +/* + * 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.streaming.ui + +import scala.xml.Node + +import org.apache.spark.streaming.scheduler.BatchInfo +import org.apache.spark.ui.UIUtils + +private[ui] abstract class BatchTableBase(tableId: String) { + + protected def columns: Seq[Node] = { + Batch Time + Input Size + Scheduling Delay + Processing Time + } + + protected def baseRow(batch: BatchInfo): Seq[Node] = { + val batchTime = batch.batchTime.milliseconds + val formattedBatchTime = UIUtils.formatDate(batch.batchTime.milliseconds) + val eventCount = batch.receivedBlockInfo.values.map { + receivers => receivers.map(_.numRecords).sum + }.sum + val schedulingDelay = batch.schedulingDelay + val formattedSchedulingDelay = schedulingDelay.map(UIUtils.formatDuration).getOrElse("-") + val processingTime = batch.processingDelay + val formattedProcessingTime = processingTime.map(UIUtils.formatDuration).getOrElse("-") + + {formattedBatchTime} + {eventCount.toString} events + + {formattedSchedulingDelay} + + + {formattedProcessingTime} + + } + + private def batchTable: Seq[Node] = { + + + {columns} + + + {renderRows} + +
    + } + + def toNodeSeq: Seq[Node] = { + batchTable + } + + /** + * Return HTML for all rows of this table. + */ + protected def renderRows: Seq[Node] +} + +private[ui] class ActiveBatchTable(runningBatches: Seq[BatchInfo], waitingBatches: Seq[BatchInfo]) + extends BatchTableBase("active-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Status + + override protected def renderRows: Seq[Node] = { + // The "batchTime"s of "waitingBatches" must be greater than "runningBatches"'s, so display + // waiting batches before running batches + waitingBatches.flatMap(batch => {waitingBatchRow(batch)}) ++ + runningBatches.flatMap(batch => {runningBatchRow(batch)}) + } + + private def runningBatchRow(batch: BatchInfo): Seq[Node] = { + baseRow(batch) ++ processing + } + + private def waitingBatchRow(batch: BatchInfo): Seq[Node] = { + baseRow(batch) ++ queued + } +} + +private[ui] class CompletedBatchTable(batches: Seq[BatchInfo]) + extends BatchTableBase("completed-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Total Delay + + override protected def renderRows: Seq[Node] = { + batches.flatMap(batch => {completedBatchRow(batch)}) + } + + private def completedBatchRow(batch: BatchInfo): Seq[Node] = { + val totalDelay = batch.totalDelay + val formattedTotalDelay = totalDelay.map(UIUtils.formatDuration).getOrElse("-") + baseRow(batch) ++ + + {formattedTotalDelay} + + } +} 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 b6dcb62bfeec8..07fa285642eec 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 @@ -41,7 +41,8 @@ private[ui] class StreamingPage(parent: StreamingTab) generateBasicStats() ++

    ++

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ generateReceiverStats() ++ - generateBatchStatsTable() + generateBatchStatsTable() ++ + generateBatchListTables() } UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } @@ -49,9 +50,10 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Generate basic stats of the streaming program */ private def generateBasicStats(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime + // scalastyle:off
    + // scalastyle:on } /** Generate stats of data received by the receivers in the streaming program */ @@ -86,10 +89,10 @@ private[ui] class StreamingPage(parent: StreamingTab) "Receiver", "Status", "Location", - "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", - "Minimum rate\n[records/sec]", - "Median rate\n[records/sec]", - "Maximum rate\n[records/sec]", + "Events in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", + "Minimum rate\n[events/sec]", + "Median rate\n[events/sec]", + "Maximum rate\n[events/sec]", "Last Error" ) val dataRows = (0 until listener.numReceivers).map { receiverId => @@ -190,5 +193,26 @@ private[ui] class StreamingPage(parent: StreamingTab) } UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) } + + private def generateBatchListTables(): Seq[Node] = { + val runningBatches = listener.runningBatches.sortBy(_.batchTime.milliseconds).reverse + val waitingBatches = listener.waitingBatches.sortBy(_.batchTime.milliseconds).reverse + val completedBatches = listener.retainedCompletedBatches. + sortBy(_.batchTime.milliseconds).reverse + + val activeBatchesContent = { +

    Active Batches ({runningBatches.size + waitingBatches.size})

    ++ + new ActiveBatchTable(runningBatches, waitingBatches).toNodeSeq + } + + val completedBatchesContent = { +

    + Completed Batches (last {completedBatches.size} out of {listener.numTotalCompletedBatches}) +

    ++ + new CompletedBatchTable(completedBatches).toNodeSeq + } + + activeBatchesContent ++ completedBatchesContent + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 998426ebb82e5..205ddf6dbe9b0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -75,6 +75,17 @@ class UISeleniumSuite val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq statisticText should contain("Network receivers:") statisticText should contain("Batch interval:") + + val h4Text = findAll(cssSelector("h4")).map(_.text).toSeq + h4Text should contain("Active Batches (0)") + h4Text should contain("Completed Batches (last 0 out of 0)") + + findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Status") + } + findAll(cssSelector("""#completed-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Total Delay") + } } ssc.stop(false) From 9717389365772d218cd7c67f9a13c3440f3c6791 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 18:51:39 -0700 Subject: [PATCH 029/144] [SPARK-6890] [core] Fix launcher lib work with SPARK_PREPEND_CLASSES. The fix for SPARK-6406 broke the case where sub-processes are launched when SPARK_PREPEND_CLASSES is set, because the code now would only add the launcher's build directory to the sub-process's classpath instead of the complete assembly. This patch fixes the problem by having the launch scripts stash the assembly's location in an environment variable. This is not the prettiest solution, but it avoids having to plumb that location all the way through the Worker code that launches executors. The env variable is always set by the launch scripts, so users cannot override it. Author: Marcelo Vanzin Closes #5504 from vanzin/SPARK-6890 and squashes the following commits: 7aec921 [Marcelo Vanzin] Fix tests. ff87a60 [Marcelo Vanzin] Merge branch 'master' into SPARK-6890 31d3ce8 [Marcelo Vanzin] [SPARK-6890] [core] Fix launcher lib work with SPARK_PREPEND_CLASSES. --- bin/spark-class | 11 ++++- bin/spark-class2.cmd | 11 ++++- .../launcher/AbstractCommandBuilder.java | 44 +++++++++++++++++-- .../spark/launcher/CommandBuilderUtils.java | 1 + .../SparkSubmitCommandBuilderSuite.java | 15 ++++--- 5 files changed, 71 insertions(+), 11 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index c03946d92e2e4..c49d97ce5cf25 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -82,13 +82,22 @@ if [ $(command -v "$JAR_CMD") ] ; then fi fi +LAUNCH_CLASSPATH="$SPARK_ASSEMBLY_JAR" + +# Add the launcher build dir to the classpath if requested. +if [ -n "$SPARK_PREPEND_CLASSES" ]; then + LAUNCH_CLASSPATH="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" +fi + +export _SPARK_ASSEMBLY="$SPARK_ASSEMBLY_JAR" + # The launcher library will print arguments separated by a NULL character, to allow arguments with # characters that would be otherwise interpreted by the shell. Read that in a while loop, populating # an array that will be used to exec the final command. CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <("$RUNNER" -cp "$SPARK_ASSEMBLY_JAR" org.apache.spark.launcher.Main "$@") +done < <("$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@") if [ "${CMD[0]}" = "usage" ]; then "${CMD[@]}" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 4b3401d745f2a..3d068dd3a2739 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -46,13 +46,22 @@ if "%SPARK_ASSEMBLY_JAR%"=="0" ( exit /b 1 ) +set LAUNCH_CLASSPATH=%SPARK_ASSEMBLY_JAR% + +rem Add the launcher build dir to the classpath if requested. +if not "x%SPARK_PREPEND_CLASSES%"=="x" ( + set LAUNCH_CLASSPATH=%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCH_CLASSPATH% +) + +set _SPARK_ASSEMBLY=%SPARK_ASSEMBLY_JAR% + rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. -for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %SPARK_ASSEMBLY_JAR% org.apache.spark.launcher.Main %*"') do ( +for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCH_CLASSPATH% org.apache.spark.launcher.Main %*"') do ( set SPARK_CMD=%%i ) %SPARK_CMD% diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index d8279145d8e90..b8f02b961113d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -186,12 +186,24 @@ List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); } - final String assembly = AbstractCommandBuilder.class.getProtectionDomain().getCodeSource(). - getLocation().getPath(); + // We can't rely on the ENV_SPARK_ASSEMBLY variable to be set. Certain situations, such as + // when running unit tests, or user code that embeds Spark and creates a SparkContext + // with a local or local-cluster master, will cause this code to be called from an + // environment where that env variable is not guaranteed to exist. + // + // For the testing case, we rely on the test code to set and propagate the test classpath + // appropriately. + // + // For the user code case, we fall back to looking for the Spark assembly under SPARK_HOME. + // That duplicates some of the code in the shell scripts that look for the assembly, though. + String assembly = getenv(ENV_SPARK_ASSEMBLY); + if (assembly == null && isEmpty(getenv("SPARK_TESTING"))) { + assembly = findAssembly(); + } addToClassPath(cp, assembly); - // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only - // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate + // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only + // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate // "lib_managed/jars/" with the datanucleus jars when Spark is built with Hive File libdir; if (new File(sparkHome, "RELEASE").isFile()) { @@ -299,6 +311,30 @@ String getenv(String key) { return firstNonEmpty(childEnv.get(key), System.getenv(key)); } + private String findAssembly() { + String sparkHome = getSparkHome(); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + } else { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", getScalaVersion())); + } + + final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); + FileFilter filter = new FileFilter() { + @Override + public boolean accept(File file) { + return file.isFile() && re.matcher(file.getName()).matches(); + } + }; + File[] assemblies = libdir.listFiles(filter); + checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); + checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); + return assemblies[0].getAbsolutePath(); + } + private String getConfDir() { String confDir = getenv("SPARK_CONF_DIR"); return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index f4ebc25bdd32b..8028e42ffb483 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -30,6 +30,7 @@ class CommandBuilderUtils { static final String DEFAULT_MEM = "512m"; static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; static final String ENV_SPARK_HOME = "SPARK_HOME"; + static final String ENV_SPARK_ASSEMBLY = "_SPARK_ASSEMBLY"; /** Returns whether the given string is null or empty. */ static boolean isEmpty(String s) { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 626116a9e7477..97043a76cc612 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -98,7 +98,7 @@ public void testShellCliParser() throws Exception { parser.NAME, "appName"); - List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List args = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch"); assertEquals(expected, args.subList(args.size() - expected.size(), args.size())); } @@ -110,7 +110,7 @@ public void testAlternateSyntaxParsing() throws Exception { parser.MASTER + "=foo", parser.DEPLOY_MODE + "=bar"); - List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List cmd = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); assertEquals("foo", findArgValue(cmd, parser.MASTER)); assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE)); @@ -153,7 +153,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { String deployMode = isDriver ? "client" : "cluster"; SparkSubmitCommandBuilder launcher = - new SparkSubmitCommandBuilder(Collections.emptyList()); + newCommandBuilder(Collections.emptyList()); launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); launcher.master = "yarn"; @@ -273,10 +273,15 @@ private boolean findInStringList(String list, String sep, String needle) { return contains(needle, list.split(sep)); } - private List buildCommand(List args, Map env) throws Exception { + private SparkSubmitCommandBuilder newCommandBuilder(List args) { SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); - return builder.buildCommand(env); + builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_ASSEMBLY, "dummy"); + return builder; + } + + private List buildCommand(List args, Map env) throws Exception { + return newCommandBuilder(args).buildCommand(env); } } From 30a6e0dcc0bd298731c1387546779cddcc16bc72 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 18:52:48 -0700 Subject: [PATCH 030/144] [SPARK-5634] [core] Show correct message in HS when no incomplete apps f... ...ound. Author: Marcelo Vanzin Closes #5515 from vanzin/SPARK-5634 and squashes the following commits: f74ecf1 [Marcelo Vanzin] [SPARK-5634] [core] Show correct message in HS when no incomplete apps found. --- .../scala/org/apache/spark/deploy/history/HistoryPage.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 6e432d63c6b5a..3781b4e8c12bd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -90,6 +90,8 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") ++ appTable + } else if (requestedIncomplete) { +

    No incomplete applications found!

    } else {

    No completed applications found!

    ++

    Did you specify the correct logging directory? From 6be918942c4078692d169d72fa9c358f6e98e85e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 14 Apr 2015 23:47:16 -0700 Subject: [PATCH 031/144] [SPARK-6871][SQL] WITH clause in CTE can not following another WITH clause JIRA https://issues.apache.org/jira/browse/SPARK-6871 Author: Liang-Chi Hsieh Closes #5480 from viirya/no_cte_after_cte and squashes the following commits: 4da3712 [Liang-Chi Hsieh] Create new test. 40b38ed [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_cte_after_cte 0edf568 [Liang-Chi Hsieh] for comments. 6591b79 [Liang-Chi Hsieh] WITH clause in CTE can not following another WITH clause. --- .../apache/spark/sql/catalyst/SqlParser.scala | 18 +++++++++--------- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 15 insertions(+), 9 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 bc8d3751f6616..9a3531ceb3343 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -121,14 +121,14 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { } protected lazy val start: Parser[LogicalPlan] = - ( (select | ("(" ~> select <~ ")")) * - ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } - | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } - | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} - | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } - ) - | insert - | cte + start1 | insert | cte + + protected lazy val start1: Parser[LogicalPlan] = + (select | ("(" ~> select <~ ")")) * + ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } + | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } + | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} + | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) protected lazy val select: Parser[LogicalPlan] = @@ -159,7 +159,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { } protected lazy val cte: Parser[LogicalPlan] = - WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start <~ ")"), ",") ~ start ^^ { + WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start1 <~ ")"), ",") ~ (start1 | insert) ^^ { case r ~ s => With(s, r.map({case n ~ s => (n, Subquery(n, s))}).toMap) } 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 73fb791c3ead7..0174aaee94246 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 @@ -431,6 +431,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } + test("Allow only a single WITH clause per query") { + intercept[RuntimeException] { + sql("with q1 as (select * from testData) with q2 as (select * from q1) select * from q2") + } + } + test("date row") { checkAnswer(sql( """select cast("2015-01-28" as date) from testData limit 1"""), From 29aabdd6c20197adb16706823a8c7f48a0074352 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 15 Apr 2015 10:23:53 +0100 Subject: [PATCH 032/144] [HOTFIX] [SPARK-6896] [SQL] fix compile error in hive-thriftserver SPARK-6440 #5424 import guava but did not promote guava dependency to compile level. [INFO] compiler plugin: BasicArtifact(org.scalamacros,paradise_2.10.4,2.0.1,null) [info] Compiling 8 Scala sources to /root/projects/spark/sql/hive-thriftserver/target/scala-2.10/classes... [error] bad symbolic reference. A signature in Utils.class refers to term util [error] in package com.google.common which is not available. [error] It may be completely missing from the current classpath, or the version on [error] the classpath might be incompatible with the version used when compiling Utils.class. [error] [error] while compiling: /root/projects/spark/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala [error] during phase: erasure [error] library version: version 2.10.4 [error] compiler version: version 2.10.4 [error] reconstructed args: -deprecation -classpath Author: Daoyuan Wang Closes #5507 from adrian-wang/guava and squashes the following commits: c337dad [Daoyuan Wang] fix compile error --- sql/hive-thriftserver/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a96b1ffc26966..f38c796241df1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -44,7 +44,6 @@ com.google.guava guava - runtime ${hive.group} From 6c5ed8a6d552abd967d27cdb94b68d46ccb57221 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 15 Apr 2015 15:17:58 +0100 Subject: [PATCH 033/144] SPARK-6861 [BUILD] Scalastyle config prevents building Maven child modules alone Move scalastyle-config.xml to dev/ (SBT config still doesn't work) to fix running mvn targets from subdirs; make scalastyle a verify stage target again in Maven; output results in target not project root; update to scalastyle 0.7.0 Author: Sean Owen Closes #5471 from srowen/SPARK-6861 and squashes the following commits: acac637 [Sean Owen] Oops, add back execution but leave it at the default verify phase 35a4fd2 [Sean Owen] Revert change to scalastyle-config.xml location, but return scalastyle Maven check to verify phase instead of package to get it farther out of the way, since the Maven invocation is optional c4fb42c [Sean Owen] Move scalastyle-config.xml to dev/ (SBT config still doesn't work) to fix running mvn targets from subdirs; make scalastyle a verify stage target again in Maven; output results in target not project root; update to scalastyle 0.7.0 --- pom.xml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 261292d5b6cde..bcc2f57f1af5d 100644 --- a/pom.xml +++ b/pom.xml @@ -1447,7 +1447,7 @@ org.scalastyle scalastyle-maven-plugin - 0.4.0 + 0.7.0 false true @@ -1456,13 +1456,12 @@ ${basedir}/src/main/scala ${basedir}/src/test/scala scalastyle-config.xml - scalastyle-output.xml + ${basedir}/target/scalastyle-output.xml ${project.build.sourceEncoding} ${project.reporting.outputEncoding} - package check From f11288d5272bc18585b8cad4ee3bd59eade7c296 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Apr 2015 12:58:02 -0700 Subject: [PATCH 034/144] [SPARK-6886] [PySpark] fix big closure with shuffle Currently, the created broadcast object will have same life cycle as RDD in Python. For multistage jobs, an PythonRDD will be created in JVM and the RDD in Python may be GCed, then the broadcast will be destroyed in JVM before the PythonRDD. This PR change to use PythonRDD to track the lifecycle of the broadcast object. It also have a refactor about getNumPartitions() to avoid unnecessary creation of PythonRDD, which could be heavy. cc JoshRosen Author: Davies Liu Closes #5496 from davies/big_closure and squashes the following commits: 9a0ea4c [Davies Liu] fix big closure with shuffle --- python/pyspark/rdd.py | 15 +++++---------- python/pyspark/tests.py | 6 ++---- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c9ac95d117574..93e658eded9e2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1197,7 +1197,7 @@ def take(self, num): [91, 92, 93] """ items = [] - totalParts = self._jrdd.partitions().size() + totalParts = self.getNumPartitions() partsScanned = 0 while len(items) < num and partsScanned < totalParts: @@ -1260,7 +1260,7 @@ def isEmpty(self): >>> sc.parallelize([1]).isEmpty() False """ - return self._jrdd.partitions().size() == 0 or len(self.take(1)) == 0 + return self.getNumPartitions() == 0 or len(self.take(1)) == 0 def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None): """ @@ -2235,11 +2235,9 @@ def _prepare_for_python_RDD(sc, command, obj=None): ser = CloudPickleSerializer() pickled_command = ser.dumps((command, sys.version_info[:2])) if len(pickled_command) > (1 << 20): # 1M + # The broadcast will have same life cycle as created PythonRDD broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) - # tracking the life cycle by obj - if obj is not None: - obj._broadcast = broadcast broadcast_vars = ListConverter().convert( [x._jbroadcast for x in sc._pickled_broadcast_vars], sc._gateway._gateway_client) @@ -2294,12 +2292,9 @@ def pipeline_func(split, iterator): self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self.partitioner = prev.partitioner if self.preservesPartitioning else None - self._broadcast = None - def __del__(self): - if self._broadcast: - self._broadcast.unpersist() - self._broadcast = None + def getNumPartitions(self): + return self._prev_jrdd.partitions().size() @property def _jrdd(self): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b938b9ce12395..ee67e80d539f8 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -550,10 +550,8 @@ def test_large_closure(self): data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) self.assertEquals(N, rdd.first()) - self.assertTrue(rdd._broadcast is not None) - rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1) - self.assertEqual(1, rdd.first()) - self.assertTrue(rdd._broadcast is None) + # regression test for SPARK-6886 + self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count()) def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) From b75b3070740803480d235b0c9a86673721344f30 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:00:19 -0700 Subject: [PATCH 035/144] [SPARK-6730][SQL] Allow using keyword as identifier in OPTIONS JIRA: https://issues.apache.org/jira/browse/SPARK-6730 It is very possible that keyword will be used as identifier in `OPTIONS`, this pr makes it works. However, another approach is that we can request that `OPTIONS` can't include keywords and has to use alternative identifier (e.g. table -> cassandraTable) if needed. If so, please let me know to close this pr. Thanks. Author: Liang-Chi Hsieh Closes #5520 from viirya/relax_options and squashes the following commits: 339fd68 [Liang-Chi Hsieh] Use regex parser. 92be11c [Liang-Chi Hsieh] Allow using keyword as identifier in OPTIONS. --- .../scala/org/apache/spark/sql/sources/ddl.scala | 15 ++++++++++++++- .../apache/spark/sql/sources/DDLTestSuite.scala | 11 ++++++----- 2 files changed, 20 insertions(+), 6 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 319de710fbc3e..2e861b84b7133 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import scala.util.matching.Regex import scala.language.implicitConversions import org.apache.spark.Logging @@ -155,7 +156,19 @@ private[sql] class DDLParser( protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")} - protected lazy val pair: Parser[(String, String)] = ident ~ stringLit ^^ { case k ~ v => (k,v) } + override implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( + s"identifier matching regex ${regex}", { + case lexical.Identifier(str) if regex.unapplySeq(str).isDefined => str + case lexical.Keyword(str) if regex.unapplySeq(str).isDefined => str + } + ) + + protected lazy val optionName: Parser[String] = "[_a-zA-Z][a-zA-Z0-9]*".r ^^ { + case name => name + } + + protected lazy val pair: Parser[(String, String)] = + optionName ~ stringLit ^^ { case k ~ v => (k,v) } protected lazy val column: Parser[StructField] = ident ~ dataType ~ (COMMENT ~> stringLit).? ^^ { case columnName ~ typ ~ cm => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 3f24a497390c1..ca25751b9583d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -25,17 +25,17 @@ class DDLScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext) + SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt, parameters("Table"))(sqlContext) } } -case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) +case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { override def schema: StructType = StructType(Seq( StructField("intType", IntegerType, nullable = false, - new MetadataBuilder().putString("comment", "test comment").build()), + new MetadataBuilder().putString("comment", s"test comment $table").build()), StructField("stringType", StringType, nullable = false), StructField("dateType", DateType, nullable = false), StructField("timestampType", TimestampType, nullable = false), @@ -73,7 +73,8 @@ class DDLTestSuite extends DataSourceTest { |USING org.apache.spark.sql.sources.DDLScanSource |OPTIONS ( | From '1', - | To '10' + | To '10', + | Table 'test1' |) """.stripMargin) } @@ -81,7 +82,7 @@ class DDLTestSuite extends DataSourceTest { sqlTest( "describe ddlPeople", Seq( - Row("intType", "int", "test comment"), + Row("intType", "int", "test comment test1"), Row("stringType", "string", ""), Row("dateType", "date", ""), Row("timestampType", "timestamp", ""), From e3e4e9a38b25174ed8bb460ba2b375813ebf3b4b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:01:29 -0700 Subject: [PATCH 036/144] [SPARK-6800][SQL] Update doc for JDBCRelation's columnPartition JIRA https://issues.apache.org/jira/browse/SPARK-6800 Author: Liang-Chi Hsieh Closes #5488 from viirya/fix_jdbc_where and squashes the following commits: 51386c8 [Liang-Chi Hsieh] Update code comment. 1dcc929 [Liang-Chi Hsieh] Update document. 3eb74d6 [Liang-Chi Hsieh] Revert and modify doc. df11783 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_jdbc_where 3e7db15 [Liang-Chi Hsieh] Fix wrong logic to generate WHERE clause for JDBC. --- docs/sql-programming-guide.md | 5 ++++- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 332618edf0c55..03500867df70f 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1371,7 +1371,10 @@ the Data Sources API. The following options are supported: These options must all be specified if any of them is specified. They describe how to partition the table when reading in parallel from multiple workers. - partitionColumn must be a numeric column from the table in question. + partitionColumn must be a numeric column from the table in question. Notice + that lowerBound and upperBound are just used to decide the + partition stride, not for filtering the rows in table. So all rows in the table will be + partitioned and returned. 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 c25ef58e6f62a..b237fe684cdc1 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 @@ -873,8 +873,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * passed to this function. * * @param columnName the name of a column of integral type that will be used for partitioning. - * @param lowerBound the minimum value of `columnName` to retrieve - * @param upperBound the maximum value of `columnName` to retrieve + * @param lowerBound the minimum value of `columnName` used to decide partition stride + * @param upperBound the maximum value of `columnName` used to decide partition stride * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split * evenly into this many partitions * From 785f95586b951d7b05481ee925fb95c20c4d6b6f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 15 Apr 2015 13:04:03 -0700 Subject: [PATCH 037/144] [SPARK-6887][SQL] ColumnBuilder misses FloatType https://issues.apache.org/jira/browse/SPARK-6887 Author: Yin Huai Closes #5499 from yhuai/inMemFloat and squashes the following commits: 84cba38 [Yin Huai] Add test. 4b75ba6 [Yin Huai] Add FloatType back. --- .../spark/sql/columnar/ColumnBuilder.scala | 1 + .../org/apache/spark/sql/QueryTest.scala | 3 + .../columnar/InMemoryColumnarQuerySuite.scala | 59 ++++++++++++++++++- 3 files changed, 62 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index c881747751520..00ed70430b84d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -153,6 +153,7 @@ private[sql] object ColumnBuilder { val builder: ColumnBuilder = dataType match { case IntegerType => new IntColumnBuilder case LongType => new LongColumnBuilder + case FloatType => new FloatColumnBuilder case DoubleType => new DoubleColumnBuilder case BooleanType => new BooleanColumnBuilder case ByteType => new ByteColumnBuilder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 9a81fc5d72819..59f9508444f25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -104,9 +104,12 @@ object QueryTest { // Converts data to types that we can do equality comparison using Scala collections. // For BigDecimal type, the Scala type has a better definition of equality test (similar to // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. val converted: Seq[Row] = answer.map { s => Row.fromSeq(s.toSeq.map { case d: java.math.BigDecimal => BigDecimal(d) + case b: Array[Byte] => b.toSeq case o => o }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 479210d1c9c43..56591d9dba29e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.columnar +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.types.{DecimalType, Decimal} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY @@ -132,4 +134,59 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM test_fixed_decimal"), (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) } + + test("test different data types") { + // Create the schema. + val struct = + StructType( + StructField("f1", FloatType, true) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val dataTypes = + Seq(StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType.Unlimited, DecimalType(6, 5), + DateType, TimestampType, + ArrayType(IntegerType), MapType(StringType, LongType), struct) + val fields = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + val allColumns = fields.map(_.name).mkString(",") + val schema = StructType(fields) + + // Create a RDD for the schema + val rdd = + sparkContext.parallelize((1 to 100), 10).map { i => + Row( + s"str${i}: test cache.", + s"binary${i}: test cache.".getBytes("UTF-8"), + null, + i % 2 == 0, + i.toByte, + i.toShort, + i, + Long.MaxValue - i.toLong, + (i + 0.25).toFloat, + (i + 0.75), + BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), + new Date(i), + new Timestamp(i), + (1 to i).toSeq, + (0 to i).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, + Row((i - 0.25).toFloat, (1 to i).toSeq)) + } + createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") + // Cache the table. + sql("cache table InMemoryCache_different_data_types") + // Make sure the table is indeed cached. + val tableScan = table("InMemoryCache_different_data_types").queryExecution.executedPlan + assert( + isCached("InMemoryCache_different_data_types"), + "InMemoryCache_different_data_types should be cached.") + // Issue a query and check the results. + checkAnswer( + sql(s"SELECT DISTINCT ${allColumns} FROM InMemoryCache_different_data_types"), + table("InMemoryCache_different_data_types").collect()) + dropTempTable("InMemoryCache_different_data_types") + } } From 85842760dc4616577162f44cc0fa9db9bd23bd9c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Apr 2015 13:06:38 -0700 Subject: [PATCH 038/144] [SPARK-6638] [SQL] Improve performance of StringType in SQL This PR change the internal representation for StringType from java.lang.String to UTF8String, which is implemented use ArrayByte. This PR should not break any public API, Row.getString() will still return java.lang.String. This is the first step of improve the performance of String in SQL. cc rxin Author: Davies Liu Closes #5350 from davies/string and squashes the following commits: 3b7bfa8 [Davies Liu] fix schema of AddJar 2772f0d [Davies Liu] fix new test failure 6d776a9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 59025c8 [Davies Liu] address comments from @marmbrus 341ec2c [Davies Liu] turn off scala style check in UTF8StringSuite 744788f [Davies Liu] Merge branch 'master' of github.com:apache/spark into string b04a19c [Davies Liu] add comment for getString/setString 08d897b [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 5116b43 [Davies Liu] rollback unrelated changes 1314a37 [Davies Liu] address comments from Yin 867bf50 [Davies Liu] fix String filter push down 13d9d42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 2089d24 [Davies Liu] add hashcode check back ac18ae6 [Davies Liu] address comment fd11364 [Davies Liu] optimize UTF8String 8d17f21 [Davies Liu] fix hive compatibility tests e5fa5b8 [Davies Liu] remove clone in UTF8String 28f3d81 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 28d6f32 [Davies Liu] refactor 537631c [Davies Liu] some comment about Date 9f4c194 [Davies Liu] convert data type for data source 956b0a4 [Davies Liu] fix hive tests 73e4363 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 9dc32d1 [Davies Liu] fix some hive tests 23a766c [Davies Liu] refactor 8b45864 [Davies Liu] fix codegen with UTF8String bb52e44 [Davies Liu] fix scala style c7dd4d2 [Davies Liu] fix some catalyst tests 38c303e [Davies Liu] fix python sql tests 5f9e120 [Davies Liu] fix sql tests 6b499ac [Davies Liu] fix style a85fb27 [Davies Liu] refactor d32abd1 [Davies Liu] fix utf8 for python api 4699c3a [Davies Liu] use Array[Byte] in UTF8String 21f67c6 [Davies Liu] cleanup 685fd07 [Davies Liu] use UTF8String instead of String for StringType --- python/pyspark/sql/dataframe.py | 10 +- .../main/scala/org/apache/spark/sql/Row.scala | 3 +- .../sql/catalyst/CatalystTypeConverters.scala | 37 +++ .../spark/sql/catalyst/ScalaReflection.scala | 1 + .../catalyst/analysis/HiveTypeCoercion.scala | 6 +- .../spark/sql/catalyst/expressions/Cast.scala | 36 +-- .../expressions/SpecificMutableRow.scala | 12 +- .../expressions/codegen/CodeGenerator.scala | 32 ++- .../codegen/GenerateProjection.scala | 46 ++-- .../sql/catalyst/expressions/generators.scala | 7 +- .../sql/catalyst/expressions/literals.scala | 7 +- .../sql/catalyst/expressions/predicates.scala | 3 +- .../spark/sql/catalyst/expressions/rows.scala | 14 +- .../expressions/stringOperations.scala | 90 ++++---- .../sql/catalyst/optimizer/Optimizer.scala | 21 +- .../apache/spark/sql/types/DateUtils.scala | 1 + .../apache/spark/sql/types/UTF8String.scala | 214 ++++++++++++++++++ .../apache/spark/sql/types/dataTypes.scala | 6 +- .../ExpressionEvaluationSuite.scala | 90 ++++---- .../GeneratedMutableEvaluationSuite.scala | 4 +- .../spark/sql/types/UTF8StringSuite.scala | 70 ++++++ .../org/apache/spark/sql/SQLContext.scala | 1 + .../spark/sql/columnar/ColumnStats.scala | 6 +- .../spark/sql/columnar/ColumnType.scala | 20 +- .../spark/sql/execution/ExistingRDD.scala | 31 ++- .../apache/spark/sql/execution/commands.scala | 13 +- .../spark/sql/execution/debug/package.scala | 2 +- .../spark/sql/execution/pythonUdfs.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 4 +- .../apache/spark/sql/jdbc/JDBCRelation.scala | 2 + .../org/apache/spark/sql/jdbc/jdbc.scala | 5 +- .../apache/spark/sql/json/JSONRelation.scala | 8 +- .../org/apache/spark/sql/json/JsonRDD.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 19 +- .../spark/sql/parquet/ParquetFilters.scala | 12 +- .../sql/parquet/ParquetTableSupport.scala | 7 +- .../apache/spark/sql/parquet/newParquet.scala | 11 +- .../sql/sources/DataSourceStrategy.scala | 37 +-- .../apache/spark/sql/sources/interfaces.scala | 10 + .../scala/org/apache/spark/sql/RowSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 8 +- .../sql/columnar/ColumnarTestUtils.scala | 4 +- .../spark/sql/sources/TableScanSuite.scala | 10 +- .../spark/sql/hive/HiveInspectors.scala | 22 +- .../spark/sql/hive/HiveStrategies.scala | 13 +- .../hive/execution/ScriptTransformation.scala | 17 +- .../spark/sql/hive/execution/commands.scala | 10 +- .../org/apache/spark/sql/hive/Shim12.scala | 4 +- .../org/apache/spark/sql/hive/Shim13.scala | 36 ++- 50 files changed, 742 insertions(+), 298 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ef91a9c4f522d..f2c3b74a185cf 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -456,7 +456,7 @@ def join(self, other, joinExprs=None, joinType=None): One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] + [Row(name=None, height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] """ if joinExprs is None: @@ -637,9 +637,9 @@ def groupBy(self, *cols): >>> df.groupBy().avg().collect() [Row(AVG(age)=3.5)] >>> df.groupBy('name').agg({'age': 'mean'}).collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] >>> df.groupBy(df.name).avg().collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] """ jcols = ListConverter().convert([_to_java_column(c) for c in cols], self._sc._gateway._gateway_client) @@ -867,11 +867,11 @@ def agg(self, *exprs): >>> gdf = df.groupBy(df.name) >>> gdf.agg({"*": "count"}).collect() - [Row(name=u'Bob', COUNT(1)=1), Row(name=u'Alice', COUNT(1)=1)] + [Row(name=u'Alice', COUNT(1)=1), Row(name=u'Bob', COUNT(1)=1)] >>> from pyspark.sql import functions as F >>> gdf.agg(F.min(df.age)).collect() - [Row(MIN(age)=5), Row(MIN(age)=2)] + [Row(MIN(age)=2), Row(MIN(age)=5)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index d794f034f5578..ac8a782976465 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.util.hashing.MurmurHash3 import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.{StructType, DateUtils} +import org.apache.spark.sql.types.StructType object Row { /** @@ -257,6 +257,7 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ + // TODO(davies): This is not the right default implementation, we use Int as Date internally def getDate(i: Int): java.sql.Date = apply(i).asInstanceOf[java.sql.Date] /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 91976fef6dc0d..d4f9fdacda4fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -77,6 +77,9 @@ object CatalystTypeConverters { } new GenericRowWithSchema(ar, structType) + case (d: String, _) => + UTF8String(d) + case (d: BigDecimal, _) => Decimal(d) @@ -175,6 +178,11 @@ object CatalystTypeConverters { case other => other } + case dataType: StringType => (item: Any) => extractOption(item) match { + case s: String => UTF8String(s) + case other => other + } + case _ => (item: Any) => extractOption(item) match { case d: BigDecimal => Decimal(d) @@ -184,6 +192,26 @@ object CatalystTypeConverters { } } + /** + * Converts Scala objects to catalyst rows / types. + * + * Note: This should be called before do evaluation on Row + * (It does not support UDT) + * This is used to create an RDD or test results with correct types for Catalyst. + */ + def convertToCatalyst(a: Any): Any = a match { + case s: String => UTF8String(s) + case d: java.sql.Date => DateUtils.fromJavaDate(d) + case d: BigDecimal => Decimal(d) + case d: java.math.BigDecimal => Decimal(d) + case seq: Seq[Any] => seq.map(convertToCatalyst) + case r: Row => Row(r.toSeq.map(convertToCatalyst): _*) + case arr: Array[Any] => arr.toSeq.map(convertToCatalyst).toArray + case m: Map[Any, Any] => + m.map { case (k, v) => (convertToCatalyst(k), convertToCatalyst(v)) }.toMap + case other => other + } + /** * Converts Catalyst types used internally in rows to standard Scala types * This method is slow, and for batch conversion you should be using converter @@ -211,6 +239,9 @@ object CatalystTypeConverters { case (i: Int, DateType) => DateUtils.toJavaDate(i) + case (s: UTF8String, StringType) => + s.toString() + case (other, _) => other } @@ -262,6 +293,12 @@ object CatalystTypeConverters { case other => other } + case StringType => + (item: Any) => item match { + case s: UTF8String => s.toString() + case other => other + } + case other => (item: Any) => item } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 01d5c1512201a..d9521953cad73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -138,6 +138,7 @@ trait ScalaReflection { // The data type can be determined without ambiguity. case obj: BooleanType.JvmType => BooleanType case obj: BinaryType.JvmType => BinaryType + case obj: String => StringType case obj: StringType.JvmType => StringType case obj: ByteType.JvmType => ByteType case obj: ShortType.JvmType => ShortType 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 3aeb964994d37..35c7f00d4e42a 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 @@ -115,7 +115,7 @@ trait HiveTypeCoercion { * the appropriate numeric equivalent. */ object ConvertNaNs extends Rule[LogicalPlan] { - val stringNaN = Literal.create("NaN", StringType) + val stringNaN = Literal("NaN") def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { @@ -563,6 +563,10 @@ trait HiveTypeCoercion { case Sum(e @ TimestampType()) => Sum(Cast(e, DoubleType)) case Average(e @ TimestampType()) => Average(Cast(e, DoubleType)) + // Compatible with Hive + case Substring(e, start, len) if e.dataType != StringType => + Substring(Cast(e, StringType), start, len) + // Coalesce should return the first non-null value, which could be any column // from the list. So we need to make sure the return type is deterministic and // compatible with every child column. 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 31f1a5fdc7e53..adf941ab2a45f 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 @@ -21,7 +21,6 @@ import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.types._ /** Cast the child expression to the target data type. */ @@ -112,21 +111,21 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { - case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) - case DateType => buildCast[Int](_, d => DateUtils.toString(d)) - case TimestampType => buildCast[Timestamp](_, timestampToString) - case _ => buildCast[Any](_, _.toString) + case BinaryType => buildCast[Array[Byte]](_, UTF8String(_)) + case DateType => buildCast[Int](_, d => UTF8String(DateUtils.toString(d))) + case TimestampType => buildCast[Timestamp](_, t => UTF8String(timestampToString(t))) + case _ => buildCast[Any](_, o => UTF8String(o.toString)) } // BinaryConverter private[this] def castToBinary(from: DataType): Any => Any = from match { - case StringType => buildCast[String](_, _.getBytes("UTF-8")) + case StringType => buildCast[UTF8String](_, _.getBytes) } // UDFToBoolean private[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, _.length() != 0) + buildCast[UTF8String](_, _.length() != 0) case TimestampType => buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) case DateType => @@ -151,8 +150,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // TimestampConverter private[this] def castToTimestamp(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => { + buildCast[UTF8String](_, utfs => { // Throw away extra if more than 9 decimal places + val s = utfs.toString val periodIdx = s.indexOf(".") var n = s if (periodIdx != -1 && n.length() - periodIdx > 9) { @@ -227,8 +227,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => - try DateUtils.fromJavaDate(Date.valueOf(s)) + buildCast[UTF8String](_, s => + try DateUtils.fromJavaDate(Date.valueOf(s.toString)) catch { case _: java.lang.IllegalArgumentException => null } ) case TimestampType => @@ -245,7 +245,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toLong catch { + buildCast[UTF8String](_, s => try s.toString.toLong catch { case _: NumberFormatException => null }) case BooleanType => @@ -261,7 +261,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // IntConverter private[this] def castToInt(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toInt catch { + buildCast[UTF8String](_, s => try s.toString.toInt catch { case _: NumberFormatException => null }) case BooleanType => @@ -277,7 +277,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ShortConverter private[this] def castToShort(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toShort catch { + buildCast[UTF8String](_, s => try s.toString.toShort catch { case _: NumberFormatException => null }) case BooleanType => @@ -293,7 +293,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ByteConverter private[this] def castToByte(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toByte catch { + buildCast[UTF8String](_, s => try s.toString.toByte catch { case _: NumberFormatException => null }) case BooleanType => @@ -323,7 +323,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { case StringType => - buildCast[String](_, s => try changePrecision(Decimal(s.toDouble), target) catch { + buildCast[UTF8String](_, s => try { + changePrecision(Decimal(s.toString.toDouble), target) + } catch { case _: NumberFormatException => null }) case BooleanType => @@ -348,7 +350,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DoubleConverter private[this] def castToDouble(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toDouble catch { + buildCast[UTF8String](_, s => try s.toString.toDouble catch { case _: NumberFormatException => null }) case BooleanType => @@ -364,7 +366,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // FloatConverter private[this] def castToFloat(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toFloat catch { + buildCast[UTF8String](_, s => try s.toString.toFloat catch { case _: NumberFormatException => null }) case BooleanType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 47b6f358ed1b1..3475ed05f4454 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -230,13 +230,17 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR new GenericRow(newValues) } - override def update(ordinal: Int, value: Any): Unit = { - if (value == null) setNullAt(ordinal) else values(ordinal).update(value) + override def update(ordinal: Int, value: Any) { + if (value == null) { + setNullAt(ordinal) + } else { + values(ordinal).update(value) + } } - override def setString(ordinal: Int, value: String): Unit = update(ordinal, value) + override def setString(ordinal: Int, value: String): Unit = update(ordinal, UTF8String(value)) - override def getString(ordinal: Int): String = apply(ordinal).asInstanceOf[String] + override def getString(ordinal: Int): String = apply(ordinal).toString override def setInt(ordinal: Int, value: Int): Unit = { val currentValue = values(ordinal).asInstanceOf[MutableInt] 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 d141354a0f427..be2c101d63a63 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 @@ -216,10 +216,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val $primitiveTerm: ${termForType(dataType)} = $value """.children - case expressions.Literal(value: String, dataType) => + case expressions.Literal(value: UTF8String, dataType) => q""" val $nullTerm = ${value == null} - val $primitiveTerm: ${termForType(dataType)} = $value + val $primitiveTerm: ${termForType(dataType)} = + org.apache.spark.sql.types.UTF8String(${value.getBytes}) """.children case expressions.Literal(value: Int, dataType) => @@ -243,11 +244,14 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) """.children case Cast(child @ DateType(), StringType) => - child.castOrNull(c => q"org.apache.spark.sql.types.DateUtils.toString($c)", StringType) + child.castOrNull(c => + q"""org.apache.spark.sql.types.UTF8String( + org.apache.spark.sql.types.DateUtils.toString($c))""", + StringType) case Cast(child @ NumericType(), IntegerType) => child.castOrNull(c => q"$c.toInt", IntegerType) @@ -272,9 +276,18 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - ${eval.primitiveTerm}.toString + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.toString) """.children + case EqualTo(e1: BinaryType, e2: BinaryType) => + (e1, e2).evaluateAs (BooleanType) { + case (eval1, eval2) => + q""" + java.util.Arrays.equals($eval1.asInstanceOf[Array[Byte]], + $eval2.asInstanceOf[Array[Byte]]) + """ + } + case EqualTo(e1, e2) => (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 == $eval2" } @@ -597,7 +610,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val localLogger = log val localLoggerTree = reify { localLogger } q""" - $localLoggerTree.debug(${e.toString} + ": " + (if($nullTerm) "null" else $primitiveTerm)) + $localLoggerTree.debug( + ${e.toString} + ": " + (if ($nullTerm) "null" else $primitiveTerm.toString)) """ :: Nil } else { Nil @@ -608,6 +622,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { dataType match { + case StringType => q"$inputRow($ordinal).asInstanceOf[org.apache.spark.sql.types.UTF8String]" case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)" case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]" } @@ -619,6 +634,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin ordinal: Int, value: TermName) = { dataType match { + case StringType => q"$destinationRow.update($ordinal, $value)" case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" case _ => q"$destinationRow.update($ordinal, $value)" } @@ -642,13 +658,13 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case DoubleType => "Double" case FloatType => "Float" case BooleanType => "Boolean" - case StringType => "String" + case StringType => "org.apache.spark.sql.types.UTF8String" } protected def defaultPrimitive(dt: DataType) = dt match { case BooleanType => ru.Literal(Constant(false)) case FloatType => ru.Literal(Constant(-1.0.toFloat)) - case StringType => ru.Literal(Constant("")) + case StringType => q"""org.apache.spark.sql.types.UTF8String("")""" case ShortType => ru.Literal(Constant(-1.toShort)) case LongType => ru.Literal(Constant(-1L)) case ByteType => ru.Literal(Constant(-1.toByte)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 69397a73a8880..6f572ff959fb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -111,36 +111,54 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { val specificAccessorFunctions = NativeType.all.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // getString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) return $elementName" :: Nil case _ => Nil } - - q""" - override def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + // Row() need this interface to compile + case StringType => + q""" + override def getString(i: Int): String = { + $accessorFailure + }""" + case other => + q""" + override def ${accessorForType(dataType)}(i: Int): ${termForType(dataType)} = { + ..$ifStatements; + $accessorFailure + }""" + } } val specificMutatorFunctions = NativeType.all.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // setString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) { nullBits($i) = false; $elementName = value; return }" :: Nil case _ => Nil } - - q""" - override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + case StringType => + // MutableRow() need this interface to compile + q""" + override def setString(i: Int, value: String) { + $accessorFailure + }""" + case other => + q""" + override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}) { + ..$ifStatements; + $accessorFailure + }""" + } } val hashValues = expressions.zipWithIndex.map { case (e,i) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 860b72fad38b3..67caadb839ff9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map -import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees} import org.apache.spark.sql.types._ /** @@ -85,8 +85,11 @@ case class UserDefinedGenerator( override protected def makeOutput(): Seq[Attribute] = schema override def eval(input: Row): TraversableOnce[Row] = { + // TODO(davies): improve this + // Convert the objects into Scala Type before calling function, we need schema to support UDT + val inputSchema = StructType(children.map(e => StructField(e.simpleString, e.dataType, true))) val inputRow = new InterpretedProjection(children) - function(inputRow(input)) + function(CatalystTypeConverters.convertToScala(inputRow(input), inputSchema).asInstanceOf[Row]) } override def toString: String = s"UserDefinedGenerator(${children.mkString(",")})" 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 0e2d593e94124..18cba4cc46707 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types._ object Literal { @@ -29,7 +30,7 @@ object Literal { case f: Float => Literal(f, FloatType) case b: Byte => Literal(b, ByteType) case s: Short => Literal(s, ShortType) - case s: String => Literal(s, StringType) + case s: String => Literal(UTF8String(s), StringType) case b: Boolean => Literal(b, BooleanType) case d: BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) @@ -42,7 +43,9 @@ object Literal { throw new RuntimeException("Unsupported literal type " + v.getClass + " " + v) } - def create(v: Any, dataType: DataType): Literal = Literal(v, dataType) + def create(v: Any, dataType: DataType): Literal = { + Literal(CatalystTypeConverters.convertToCatalyst(v), dataType) + } } /** 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 7e47cb3fffe12..fcd6352079b4d 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 @@ -179,8 +179,7 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison val r = right.eval(input) if (r == null) null else if (left.dataType != BinaryType) l == r - else BinaryType.ordering.compare( - l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) == 0 + else java.util.Arrays.equals(l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 0a275b84086cf..1b62e17ff47fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{StructType, NativeType} +import org.apache.spark.sql.types.{UTF8String, StructType, NativeType} /** @@ -37,6 +37,7 @@ trait MutableRow extends Row { def setByte(ordinal: Int, value: Byte) def setFloat(ordinal: Int, value: Float) def setString(ordinal: Int, value: String) + // TODO(davies): add setDate() and setDecimal() } /** @@ -114,9 +115,15 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { } override def getString(i: Int): String = { - values(i).asInstanceOf[String] + values(i) match { + case null => null + case s: String => s + case utf8: UTF8String => utf8.toString + } } + // TODO(davies): add getDate and getDecimal + // Custom hashCode function that matches the efficient code generated version. override def hashCode: Int = { var result: Int = 37 @@ -189,8 +196,7 @@ class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { override def setFloat(ordinal: Int, value: Float): Unit = { values(ordinal) = value } override def setInt(ordinal: Int, value: Int): Unit = { values(ordinal) = value } override def setLong(ordinal: Int, value: Long): Unit = { values(ordinal) = value } - override def setString(ordinal: Int, value: String): Unit = { values(ordinal) = value } - + override def setString(ordinal: Int, value: String) { values(ordinal) = UTF8String(value)} override def setNullAt(i: Int): Unit = { values(i) = null } override def setShort(ordinal: Int, value: Short): Unit = { values(ordinal) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index acfbbace608ef..d597bf7ce756a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -19,11 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern -import scala.collection.IndexedSeqOptimized - - import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.types.{BinaryType, BooleanType, DataType, StringType} +import org.apache.spark.sql.types._ trait StringRegexExpression { self: BinaryExpression => @@ -60,38 +57,17 @@ trait StringRegexExpression { if(r == null) { null } else { - val regex = pattern(r.asInstanceOf[String]) + val regex = pattern(r.asInstanceOf[UTF8String].toString) if(regex == null) { null } else { - matches(regex, l.asInstanceOf[String]) + matches(regex, l.asInstanceOf[UTF8String].toString) } } } } } -trait CaseConversionExpression { - self: UnaryExpression => - - type EvaluatedType = Any - - def convert(v: String): String - - override def foldable: Boolean = child.foldable - def nullable: Boolean = child.nullable - def dataType: DataType = StringType - - override def eval(input: Row): Any = { - val evaluated = child.eval(input) - if (evaluated == null) { - null - } else { - convert(evaluated.toString) - } - } -} - /** * Simple RegEx pattern matching function */ @@ -134,12 +110,33 @@ case class RLike(left: Expression, right: Expression) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) } +trait CaseConversionExpression { + self: UnaryExpression => + + type EvaluatedType = Any + + def convert(v: UTF8String): UTF8String + + override def foldable: Boolean = child.foldable + def nullable: Boolean = child.nullable + def dataType: DataType = StringType + + override def eval(input: Row): Any = { + val evaluated = child.eval(input) + if (evaluated == null) { + null + } else { + convert(evaluated.asInstanceOf[UTF8String]) + } + } +} + /** * A function that converts the characters of a string to uppercase. */ case class Upper(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toUpperCase() + override def convert(v: UTF8String): UTF8String = v.toUpperCase override def toString: String = s"Upper($child)" } @@ -149,7 +146,7 @@ case class Upper(child: Expression) extends UnaryExpression with CaseConversionE */ case class Lower(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toLowerCase() + override def convert(v: UTF8String): UTF8String = v.toLowerCase override def toString: String = s"Lower($child)" } @@ -162,15 +159,16 @@ trait StringComparison { override def nullable: Boolean = left.nullable || right.nullable - def compare(l: String, r: String): Boolean + def compare(l: UTF8String, r: UTF8String): Boolean override def eval(input: Row): Any = { - val leftEval = left.eval(input).asInstanceOf[String] + val leftEval = left.eval(input) if(leftEval == null) { null } else { - val rightEval = right.eval(input).asInstanceOf[String] - if (rightEval == null) null else compare(leftEval, rightEval) + val rightEval = right.eval(input) + if (rightEval == null) null + else compare(leftEval.asInstanceOf[UTF8String], rightEval.asInstanceOf[UTF8String]) } } @@ -184,7 +182,7 @@ trait StringComparison { */ case class Contains(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.contains(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) } /** @@ -192,7 +190,7 @@ case class Contains(left: Expression, right: Expression) */ case class StartsWith(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.startsWith(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) } /** @@ -200,7 +198,7 @@ case class StartsWith(left: Expression, right: Expression) */ case class EndsWith(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.endsWith(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) } /** @@ -224,9 +222,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends override def children: Seq[Expression] = str :: pos :: len :: Nil @inline - def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) - (implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = { - val len = str.length + def slicePos(startPos: Int, sliceLen: Int, length: () => Int): (Int, Int) = { // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and // negative indices for start positions. If a start index i is greater than 0, it // refers to element i-1 in the sequence. If a start index i is less than 0, it refers @@ -235,7 +231,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends val start = startPos match { case pos if pos > 0 => pos - 1 - case neg if neg < 0 => len + neg + case neg if neg < 0 => length() + neg case _ => 0 } @@ -244,12 +240,11 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends case x => start + x } - str.slice(start, end) + (start, end) } override def eval(input: Row): Any = { val string = str.eval(input) - val po = pos.eval(input) val ln = len.eval(input) @@ -257,11 +252,14 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends null } else { val start = po.asInstanceOf[Int] - val length = ln.asInstanceOf[Int] - + val length = ln.asInstanceOf[Int] string match { - case ba: Array[Byte] => slice(ba, start, length) - case other => slice(other.toString, start, length) + case ba: Array[Byte] => + val (st, end) = slicePos(start, length, () => ba.length) + ba.slice(st, end) + case s: UTF8String => + val (st, end) = slicePos(start, length, () => s.length) + s.slice(st, end) } } } 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 93e69d409cb91..7c80634d2c852 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 @@ -198,14 +198,19 @@ object LikeSimplification extends Rule[LogicalPlan] { val equalTo = "([^_%]*)".r def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case Like(l, Literal(startsWith(pattern), StringType)) if !pattern.endsWith("\\") => - StartsWith(l, Literal(pattern)) - case Like(l, Literal(endsWith(pattern), StringType)) => - EndsWith(l, Literal(pattern)) - case Like(l, Literal(contains(pattern), StringType)) if !pattern.endsWith("\\") => - Contains(l, Literal(pattern)) - case Like(l, Literal(equalTo(pattern), StringType)) => - EqualTo(l, Literal(pattern)) + case Like(l, Literal(utf, StringType)) => + utf.toString match { + case startsWith(pattern) if !pattern.endsWith("\\") => + StartsWith(l, Literal(pattern)) + case endsWith(pattern) => + EndsWith(l, Literal(pattern)) + case contains(pattern) if !pattern.endsWith("\\") => + Contains(l, Literal(pattern)) + case equalTo(pattern) => + EqualTo(l, Literal(pattern)) + case _ => + Like(l, Literal.create(utf, StringType)) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala index 504fb05842505..d36a49159b87f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala @@ -40,6 +40,7 @@ object DateUtils { millisToDays(d.getTime) } + // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisLocal: Long): Int = { ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala new file mode 100644 index 0000000000000..fc02ba6c9c43e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala @@ -0,0 +1,214 @@ +/* +* 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.types + +import java.util.Arrays + +/** + * A UTF-8 String, as internal representation of StringType in SparkSQL + * + * A String encoded in UTF-8 as an Array[Byte], which can be used for comparison, + * search, see http://en.wikipedia.org/wiki/UTF-8 for details. + * + * Note: This is not designed for general use cases, should not be used outside SQL. + */ + +final class UTF8String extends Ordered[UTF8String] with Serializable { + + private[this] var bytes: Array[Byte] = _ + + /** + * Update the UTF8String with String. + */ + def set(str: String): UTF8String = { + bytes = str.getBytes("utf-8") + this + } + + /** + * Update the UTF8String with Array[Byte], which should be encoded in UTF-8 + */ + def set(bytes: Array[Byte]): UTF8String = { + this.bytes = bytes + this + } + + /** + * Return the number of bytes for a code point with the first byte as `b` + * @param b The first byte of a code point + */ + @inline + private[this] def numOfBytes(b: Byte): Int = { + val offset = (b & 0xFF) - 192 + if (offset >= 0) UTF8String.bytesOfCodePointInUTF8(offset) else 1 + } + + /** + * Return the number of code points in it. + * + * This is only used by Substring() when `start` is negative. + */ + def length(): Int = { + var len = 0 + var i: Int = 0 + while (i < bytes.length) { + i += numOfBytes(bytes(i)) + len += 1 + } + len + } + + def getBytes: Array[Byte] = { + bytes + } + + /** + * Return a substring of this, + * @param start the position of first code point + * @param until the position after last code point + */ + def slice(start: Int, until: Int): UTF8String = { + if (until <= start || start >= bytes.length || bytes == null) { + new UTF8String + } + + var c = 0 + var i: Int = 0 + while (c < start && i < bytes.length) { + i += numOfBytes(bytes(i)) + c += 1 + } + var j = i + while (c < until && j < bytes.length) { + j += numOfBytes(bytes(j)) + c += 1 + } + UTF8String(Arrays.copyOfRange(bytes, i, j)) + } + + def contains(sub: UTF8String): Boolean = { + val b = sub.getBytes + if (b.length == 0) { + return true + } + var i: Int = 0 + while (i <= bytes.length - b.length) { + // In worst case, it's O(N*K), but should works fine with SQL + if (bytes(i) == b(0) && Arrays.equals(Arrays.copyOfRange(bytes, i, i + b.length), b)) { + return true + } + i += 1 + } + false + } + + def startsWith(prefix: UTF8String): Boolean = { + val b = prefix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, 0, b.length), b) + } + + def endsWith(suffix: UTF8String): Boolean = { + val b = suffix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, bytes.length - b.length, bytes.length), b) + } + + def toUpperCase(): UTF8String = { + // upper case depends on locale, fallback to String. + UTF8String(toString().toUpperCase) + } + + def toLowerCase(): UTF8String = { + // lower case depends on locale, fallback to String. + UTF8String(toString().toLowerCase) + } + + override def toString(): String = { + new String(bytes, "utf-8") + } + + override def clone(): UTF8String = new UTF8String().set(this.bytes) + + override def compare(other: UTF8String): Int = { + var i: Int = 0 + val b = other.getBytes + while (i < bytes.length && i < b.length) { + val res = bytes(i).compareTo(b(i)) + if (res != 0) return res + i += 1 + } + bytes.length - b.length + } + + override def compareTo(other: UTF8String): Int = { + compare(other) + } + + override def equals(other: Any): Boolean = other match { + case s: UTF8String => + Arrays.equals(bytes, s.getBytes) + case s: String => + // This is only used for Catalyst unit tests + // fail fast + bytes.length >= s.length && length() == s.length && toString() == s + case _ => + false + } + + override def hashCode(): Int = { + Arrays.hashCode(bytes) + } +} + +object UTF8String { + // number of tailing bytes in a UTF8 sequence for a code point + // see http://en.wikipedia.org/wiki/UTF-8, 192-256 of Byte 1 + private[types] val bytesOfCodePointInUTF8: Array[Int] = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 4, 4, 4, 4, 4, 4, 4, 4, + 5, 5, 5, 5, + 6, 6, 6, 6) + + /** + * Create a UTF-8 String from String + */ + def apply(s: String): UTF8String = { + if (s != null) { + new UTF8String().set(s) + } else{ + null + } + } + + /** + * Create a UTF-8 String from Array[Byte], which should be encoded in UTF-8 + */ + def apply(bytes: Array[Byte]): UTF8String = { + if (bytes != null) { + new UTF8String().set(bytes) + } else { + null + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index cdf2bc68d9c5e..c6fb22c26bd3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -350,7 +350,7 @@ class StringType private() extends NativeType with PrimitiveType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = String + private[sql] type JvmType = UTF8String @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] @@ -1196,8 +1196,8 @@ abstract class UserDefinedType[UserType] extends DataType with Serializable { /** * Convert the user type to a SQL datum * - * TODO: Can we make this take obj: UserType? The issue is in ScalaReflection.convertToCatalyst, - * where we need to convert Any to UserType. + * TODO: Can we make this take obj: UserType? The issue is in + * CatalystTypeConverters.convertToCatalyst, where we need to convert Any to UserType. */ def serialize(obj: Any): Any 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 d4362a91d992c..76298f03c94ae 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 @@ -25,8 +25,9 @@ import org.scalactic.TripleEqualsSupport.Spread import org.scalatest.FunSuite import org.scalatest.Matchers._ -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField +import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.types._ @@ -59,6 +60,10 @@ class ExpressionEvaluationBaseSuite extends FunSuite { class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { + def create_row(values: Any*): Row = { + new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray) + } + test("literals") { checkEvaluation(Literal(1), 1) checkEvaluation(Literal(true), true) @@ -265,24 +270,23 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("LIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) - checkEvaluation("abdef" like regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a_%b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("**"))) - checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a_b"))) - checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) - - checkEvaluation(Literal.create(null, StringType) like regEx, null, - new GenericRow(Array[Any]("bc%"))) + checkEvaluation("abcd" like regEx, null, create_row(null)) + checkEvaluation("abdef" like regEx, true, create_row("abdef")) + checkEvaluation("a_%b" like regEx, true, create_row("a\\__b")) + checkEvaluation("addb" like regEx, true, create_row("a_%b")) + checkEvaluation("addb" like regEx, false, create_row("a\\__b")) + checkEvaluation("addb" like regEx, false, create_row("a%\\%b")) + checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b")) + checkEvaluation("addb" like regEx, true, create_row("a%")) + checkEvaluation("addb" like regEx, false, create_row("**")) + checkEvaluation("abc" like regEx, true, create_row("a%")) + checkEvaluation("abc" like regEx, false, create_row("b%")) + checkEvaluation("abc" like regEx, false, create_row("bc%")) + checkEvaluation("a\nb" like regEx, true, create_row("a_b")) + checkEvaluation("ab" like regEx, true, create_row("a%b")) + checkEvaluation("a\nb" like regEx, true, create_row("a%b")) + + checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%")) } test("RLIKE literal Regular Expression") { @@ -313,14 +317,14 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("RLIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abdef" rlike regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("abbbbc" rlike regEx, true, new GenericRow(Array[Any]("a.*c"))) - checkEvaluation("fofo" rlike regEx, true, new GenericRow(Array[Any]("^fo"))) - checkEvaluation("fo\no" rlike regEx, true, new GenericRow(Array[Any]("^fo\no$"))) - checkEvaluation("Bn" rlike regEx, true, new GenericRow(Array[Any]("^Ba*n"))) + checkEvaluation("abdef" rlike regEx, true, create_row("abdef")) + checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c")) + checkEvaluation("fofo" rlike regEx, true, create_row("^fo")) + checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$")) + checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n")) intercept[java.util.regex.PatternSyntaxException] { - evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) + evaluate("abbbbc" rlike regEx, create_row("**")) } } @@ -763,7 +767,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("null checking") { - val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) + val row = create_row("^Ba*n", null, true, null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) val c3 = 'a.boolean.at(2) @@ -803,7 +807,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("case when") { - val row = new GenericRow(Array[Any](null, false, true, "a", "b", "c")) + val row = create_row(null, false, true, "a", "b", "c") val c1 = 'a.boolean.at(0) val c2 = 'a.boolean.at(1) val c3 = 'a.boolean.at(2) @@ -846,13 +850,13 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("complex type") { - val row = new GenericRow(Array[Any]( - "^Ba*n", // 0 - null.asInstanceOf[String], // 1 - new GenericRow(Array[Any]("aa", "bb")), // 2 - Map("aa"->"bb"), // 3 - Seq("aa", "bb") // 4 - )) + val row = create_row( + "^Ba*n", // 0 + null.asInstanceOf[UTF8String], // 1 + create_row("aa", "bb"), // 2 + Map("aa"->"bb"), // 3 + Seq("aa", "bb") // 4 + ) val typeS = StructType( StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil @@ -909,7 +913,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("arithmetic") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -934,7 +938,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("fractional arithmetic") { - val row = new GenericRow(Array[Any](1.1, 2.0, 3.1, null)) + val row = create_row(1.1, 2.0, 3.1, null) val c1 = 'a.double.at(0) val c2 = 'a.double.at(1) val c3 = 'a.double.at(2) @@ -958,7 +962,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("BinaryComparison") { - val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) + val row = create_row(1, 2, 3, null, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -988,7 +992,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("StringComparison") { - val row = new GenericRow(Array[Any]("abc", null)) + val row = create_row("abc", null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) @@ -1009,7 +1013,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("Substring") { - val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) + val row = create_row("example", "example".toArray.map(_.toByte)) val s = 'a.string.at(0) @@ -1053,7 +1057,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { // substring(null, _, _) -> null checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), - null, new GenericRow(Array[Any](null))) + null, create_row(null)) // substring(_, null, _) -> null checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), @@ -1102,20 +1106,20 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("SQRT") { val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24)) val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble)) - val rowSequence = inputSequence.map(l => new GenericRow(Array[Any](l.toDouble))) + val rowSequence = inputSequence.map(l => create_row(l.toDouble)) val d = 'a.double.at(0) for ((row, expected) <- rowSequence zip expectedResults) { checkEvaluation(Sqrt(d), expected, row) } - checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null)) checkEvaluation(Sqrt(-1), null, EmptyRow) checkEvaluation(Sqrt(-1.5), null, EmptyRow) } test("Bitwise operations") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala index 275ea2627ebcd..bcc0c404d2cfb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.codegen._ /** @@ -43,7 +43,7 @@ class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { } val actual = plan(inputRow) - val expectedRow = new GenericRow(Array[Any](expected)) + val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) if (actual.hashCode() != expectedRow.hashCode()) { fail( s""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala new file mode 100644 index 0000000000000..a22aa6f244c48 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala @@ -0,0 +1,70 @@ +/* +* 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.types + +import org.scalatest.FunSuite + +// scalastyle:off +class UTF8StringSuite extends FunSuite { + test("basic") { + def check(str: String, len: Int) { + + assert(UTF8String(str).length == len) + assert(UTF8String(str.getBytes("utf8")).length() == len) + + assert(UTF8String(str) == str) + assert(UTF8String(str.getBytes("utf8")) == str) + assert(UTF8String(str).toString == str) + assert(UTF8String(str.getBytes("utf8")).toString == str) + assert(UTF8String(str.getBytes("utf8")) == UTF8String(str)) + + assert(UTF8String(str).hashCode() == UTF8String(str.getBytes("utf8")).hashCode()) + } + + check("hello", 5) + check("世 界", 3) + } + + test("contains") { + assert(UTF8String("hello").contains(UTF8String("ello"))) + assert(!UTF8String("hello").contains(UTF8String("vello"))) + assert(UTF8String("大千世界").contains(UTF8String("千世"))) + assert(!UTF8String("大千世界").contains(UTF8String("世千"))) + } + + test("prefix") { + assert(UTF8String("hello").startsWith(UTF8String("hell"))) + assert(!UTF8String("hello").startsWith(UTF8String("ell"))) + assert(UTF8String("大千世界").startsWith(UTF8String("大千"))) + assert(!UTF8String("大千世界").startsWith(UTF8String("千"))) + } + + test("suffix") { + assert(UTF8String("hello").endsWith(UTF8String("ello"))) + assert(!UTF8String("hello").endsWith(UTF8String("ellov"))) + assert(UTF8String("大千世界").endsWith(UTF8String("世界"))) + assert(!UTF8String("大千世界").endsWith(UTF8String("世"))) + } + + test("slice") { + assert(UTF8String("hello").slice(1, 3) == UTF8String("el")) + assert(UTF8String("大千世界").slice(0, 1) == UTF8String("大")) + assert(UTF8String("大千世界").slice(1, 3) == UTF8String("千世")) + assert(UTF8String("大千世界").slice(3, 5) == UTF8String("界")) + } +} 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 b237fe684cdc1..89a4faf35e0d2 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 @@ -1195,6 +1195,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case FloatType => true case DateType => true case TimestampType => true + case StringType => true case ArrayType(_, _) => true case MapType(_, _, _) => true case StructType(_) => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 87a6631da8300..b0f983c180673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -216,13 +216,13 @@ private[sql] class IntColumnStats extends ColumnStats { } private[sql] class StringColumnStats extends ColumnStats { - protected var upper: String = null - protected var lower: String = null + protected var upper: UTF8String = null + protected var lower: UTF8String = null override def gatherStats(row: Row, ordinal: Int): Unit = { super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { - val value = row.getString(ordinal) + val value = row(ordinal).asInstanceOf[UTF8String] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value sizeInBytes += STRING.actualSize(row, ordinal) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index c47497e0662d9..1b9e0df2dcb5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.reflect.runtime.universe.TypeTag @@ -312,26 +312,28 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { row.getString(ordinal).getBytes("utf-8").length + 4 } - override def append(v: String, buffer: ByteBuffer): Unit = { - val stringBytes = v.getBytes("utf-8") + override def append(v: UTF8String, buffer: ByteBuffer): Unit = { + val stringBytes = v.getBytes buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } - override def extract(buffer: ByteBuffer): String = { + override def extract(buffer: ByteBuffer): UTF8String = { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) - new String(stringBytes, "utf-8") + UTF8String(stringBytes) } - override def setField(row: MutableRow, ordinal: Int, value: String): Unit = { - row.setString(ordinal, value) + override def setField(row: MutableRow, ordinal: Int, value: UTF8String): Unit = { + row.update(ordinal, value) } - override def getField(row: Row, ordinal: Int): String = row.getString(ordinal) + override def getField(row: Row, ordinal: Int): UTF8String = { + row(ordinal).asInstanceOf[UTF8String] + } override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { - to.setString(toOrdinal, from.getString(fromOrdinal)) + to.update(toOrdinal, from(fromOrdinal)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 656bdd7212f56..1fd387eec7e57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow, SpecificMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SQLContext} /** * :: DeveloperApi :: @@ -54,6 +54,33 @@ object RDDConversions { } } } + + /** + * Convert the objects inside Row into the types Catalyst expected. + */ + def rowToRowRdd(data: RDD[Row], schema: StructType): RDD[Row] = { + data.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val bufferedIterator = iterator.buffered + val mutableRow = new GenericMutableRow(bufferedIterator.head.toSeq.toArray) + val schemaFields = schema.fields.toArray + val converters = schemaFields.map { + f => CatalystTypeConverters.createToCatalystConverter(f.dataType) + } + bufferedIterator.map { r => + var i = 0 + while (i < mutableRow.length) { + mutableRow(i) = converters(i)(r(i)) + i += 1 + } + + mutableRow + } + } + } + } } /** Logical plan node for scanning data from an RDD. */ 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 fad7a281dc1e2..99f24910fd61f 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 @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.{BooleanType, StructField, StructType, StringType} -import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -61,7 +62,11 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { override def executeTake(limit: Int): Array[Row] = sideEffectResult.take(limit).toArray - override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) + override def execute(): RDD[Row] = { + val converted = sideEffectResult.map(r => + CatalystTypeConverters.convertToCatalyst(r, schema).asInstanceOf[Row]) + sqlContext.sparkContext.parallelize(converted, 1) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index e916e68e58b5d..710787096e6cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -164,7 +164,7 @@ package object debug { case (_: Long, LongType) => case (_: Int, IntegerType) => - case (_: String, StringType) => + case (_: UTF8String, StringType) => case (_: Float, FloatType) => case (_: Byte, ByteType) => case (_: Short, ShortType) => 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 5b308d88d4cdf..7a43bfd8bc8d9 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 @@ -140,6 +140,7 @@ object EvaluatePython { case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) case (date: Int, DateType) => DateUtils.toJavaDate(date) + case (s: UTF8String, StringType) => s.toString // Pyrolite can handle Timestamp and Decimal case (other, _) => other @@ -192,7 +193,8 @@ object EvaluatePython { case (c: Long, IntegerType) => c.toInt case (c: Int, LongType) => c.toLong case (c: Double, FloatType) => c.toFloat - case (c, StringType) if !c.isInstanceOf[String] => c.toString + case (c: String, StringType) => UTF8String(c) + case (c, StringType) if !c.isInstanceOf[String] => UTF8String(c.toString) case (c, _) => c } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 463e1dcc268bc..b9022fcd9e3ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -233,7 +233,7 @@ private[sql] class JDBCRDD( * Converts value to SQL expression. */ private def compileValue(value: Any): Any = value match { - case stringValue: String => s"'${escapeSql(stringValue)}'" + case stringValue: UTF8String => s"'${escapeSql(stringValue.toString)}'" case _ => value } @@ -349,12 +349,14 @@ private[sql] class JDBCRDD( val pos = i + 1 conversions(i) match { case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) + // TODO(davies): convert Date into Int case DateConversion => mutableRow.update(i, rs.getDate(pos)) case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) case IntegerConversion => mutableRow.setInt(i, rs.getInt(pos)) case LongConversion => mutableRow.setLong(i, rs.getLong(pos)) + // TODO(davies): use getBytes for better performance, if the encoding is UTF-8 case StringConversion => mutableRow.setString(i, rs.getString(pos)) case TimestampConversion => mutableRow.update(i, rs.getTimestamp(pos)) case BinaryConversion => mutableRow.update(i, rs.getBytes(pos)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 4fa84dc076f7e..99b755c9f25d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -130,6 +130,8 @@ private[sql] case class JDBCRelation( extends BaseRelation with PrunedFilteredScan { + override val needConversion: Boolean = false + override val schema: StructType = JDBCRDD.resolveTable(url, table, properties) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala index 34f864f5fda7a..d4e0abc040bc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -18,11 +18,8 @@ package org.apache.spark.sql import java.sql.{Connection, DriverManager, PreparedStatement} -import org.apache.spark.{Logging, Partition} -import org.apache.spark.sql._ -import org.apache.spark.sql.sources.LogicalRelation -import org.apache.spark.sql.jdbc.{JDBCPartitioningInfo, JDBCRelation, JDBCPartition} +import org.apache.spark.Logging import org.apache.spark.sql.types._ package object jdbc { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f4c99b4b56606..e3352d02787fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.json import java.io.IOException import org.apache.hadoop.fs.Path + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row - -import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} private[sql] class DefaultSource @@ -113,6 +113,8 @@ private[sql] case class JSONRelation( // TODO: Support partitioned JSON relation. private def baseRDD = sqlContext.sparkContext.textFile(path) + override val needConversion: Boolean = false + override val schema = userSpecifiedSchema.getOrElse( JsonRDD.nullTypeToStringType( JsonRDD.inferSchema( 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 b1e8521383756..29de7401dda71 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 @@ -409,7 +409,7 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case StringType => toString(value) + case StringType => UTF8String(toString(value)) case _ if value == null || value == "" => null // guard the non string type case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 43ca359b51735..bc108e37dfb0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -219,8 +219,8 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, value.getBytes) - protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - updateField(fieldIndex, value) + protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + updateField(fieldIndex, UTF8String(value)) protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, readTimestamp(value)) @@ -418,8 +418,8 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, value.getBytes) - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - current.setString(fieldIndex, value) + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + current.update(fieldIndex, UTF8String(value)) override protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, readTimestamp(value)) @@ -475,19 +475,18 @@ private[parquet] class CatalystPrimitiveConverter( private[parquet] class CatalystPrimitiveStringConverter(parent: CatalystConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { - private[this] var dict: Array[String] = null + private[this] var dict: Array[Array[Byte]] = null override def hasDictionarySupport: Boolean = true override def setDictionary(dictionary: Dictionary):Unit = - dict = Array.tabulate(dictionary.getMaxId + 1) {dictionary.decodeToBinary(_).toStringUsingUTF8} - + dict = Array.tabulate(dictionary.getMaxId + 1) { dictionary.decodeToBinary(_).getBytes } override def addValueFromDictionary(dictionaryId: Int): Unit = parent.updateString(fieldIndex, dict(dictionaryId)) override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value.toStringUsingUTF8) + parent.updateString(fieldIndex, value.getBytes) } private[parquet] object CatalystArrayConverter { @@ -714,9 +713,9 @@ private[parquet] class CatalystNativeArrayConverter( elements += 1 } - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = { + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] + buffer(elements) = UTF8String(value).asInstanceOf[NativeType] elements += 1 } 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 0357dcc4688be..5eb1c6abc2432 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 @@ -55,7 +55,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), @@ -76,7 +76,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), @@ -94,7 +94,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -111,7 +111,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -128,7 +128,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -145,7 +145,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) 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 5a1b15490d273..e05a4c20b0d41 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 @@ -198,10 +198,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { if (value != null) { schema match { case StringType => writer.addBinary( - Binary.fromByteArray( - value.asInstanceOf[String].getBytes("utf-8") - ) - ) + Binary.fromByteArray(value.asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(value.asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) @@ -349,7 +346,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { index: Int): Unit = { ctype match { case StringType => writer.addBinary( - Binary.fromByteArray(record(index).asInstanceOf[String].getBytes("utf-8"))) + Binary.fromByteArray(record(index).asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(record(index).asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(record.getInt(index)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 20fdf5e58ef82..af7b3c81ae7b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -33,7 +33,6 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.{InputSplit, Job, JobContext} - import parquet.filter2.predicate.FilterApi import parquet.format.converter.ParquetMetadataConverter import parquet.hadoop.metadata.CompressionCodecName @@ -45,13 +44,13 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, expressions} import org.apache.spark.sql.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{IntegerType, StructField, StructType, _} import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext, SaveMode} -import org.apache.spark.{Logging, Partition => SparkPartition, SerializableWritable, SparkException, TaskContext} +import org.apache.spark.{Logging, SerializableWritable, SparkException, TaskContext, Partition => SparkPartition} /** * Allows creation of Parquet based tables using the syntax: @@ -409,6 +408,9 @@ private[sql] case class ParquetRelation2( file.getName == ParquetFileWriter.PARQUET_METADATA_FILE } + // Skip type conversion + override val needConversion: Boolean = false + // TODO Should calculate per scan size // It's common that a query only scans a fraction of a large Parquet file. Returning size of the // whole Parquet file disables some optimizations in this case (e.g. broadcast join). @@ -550,7 +552,8 @@ private[sql] case class ParquetRelation2( baseRDD.mapPartitionsWithInputSplit { case (split: ParquetInputSplit, iterator) => val partValues = selectedPartitions.collectFirst { - case p if split.getPath.getParent.toString == p.path => p.values + case p if split.getPath.getParent.toString == p.path => + CatalystTypeConverters.convertToCatalyst(p.values).asInstanceOf[Row] }.get val requiredPartOrdinal = partitionKeyLocations.keys.toSeq diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 34d048e426d10..b3d71f687a60a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.{UTF8String, StringType} import org.apache.spark.sql.{Row, Strategy, execution, sources} /** @@ -53,7 +54,7 @@ private[sql] object DataSourceStrategy extends Strategy { (a, _) => t.buildScan(a)) :: Nil case l @ LogicalRelation(t: TableScan) => - execution.PhysicalRDD(l.output, t.buildScan()) :: Nil + createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => @@ -102,20 +103,30 @@ private[sql] object DataSourceStrategy extends Strategy { projectList.asInstanceOf[Seq[Attribute]] // Safe due to if above. .map(relation.attributeMap) // Match original case of attributes. - val scan = - execution.PhysicalRDD( - projectList.map(_.toAttribute), + val scan = createPhysicalRDD(relation.relation, projectList.map(_.toAttribute), scanBuilder(requestedColumns, pushedFilters)) filterCondition.map(execution.Filter(_, scan)).getOrElse(scan) } else { val requestedColumns = (projectSet ++ filterSet).map(relation.attributeMap).toSeq - val scan = - execution.PhysicalRDD(requestedColumns, scanBuilder(requestedColumns, pushedFilters)) + val scan = createPhysicalRDD(relation.relation, requestedColumns, + scanBuilder(requestedColumns, pushedFilters)) execution.Project(projectList, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)) } } + private[this] def createPhysicalRDD( + relation: BaseRelation, + output: Seq[Attribute], + rdd: RDD[Row]): SparkPlan = { + val converted = if (relation.needConversion) { + execution.RDDConversions.rowToRowRdd(rdd, relation.schema) + } else { + rdd + } + execution.PhysicalRDD(output, converted) + } + /** * Selects Catalyst predicate [[Expression]]s which are convertible into data source [[Filter]]s, * and convert them. @@ -167,14 +178,14 @@ private[sql] object DataSourceStrategy extends Strategy { case expressions.Not(child) => translate(child).map(sources.Not) - case expressions.StartsWith(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringStartsWith(a.name, v)) + case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringStartsWith(a.name, v.toString)) - case expressions.EndsWith(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringEndsWith(a.name, v)) + case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringEndsWith(a.name, v.toString)) - case expressions.Contains(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringContains(a.name, v)) + case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringContains(a.name, v.toString)) case _ => None } 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 8f9946a5a801e..ca53dcdb92c52 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 @@ -126,6 +126,16 @@ abstract class BaseRelation { * could lead to execution plans that are suboptimal (i.e. broadcasting a very large table). */ def sizeInBytes: Long = sqlContext.conf.defaultSizeInBytes + + /** + * Whether does it need to convert the objects in Row to internal representation, for example: + * java.lang.String -> UTF8String + * java.lang.Decimal -> Decimal + * + * Note: The internal representation is not stable across releases and thus data sources outside + * of Spark SQL should leave this as true. + */ + def needConversion: Boolean = true } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 36465cc2fa11a..bf6cf1321a056 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -30,7 +30,7 @@ class RowSuite extends FunSuite { test("create row") { val expected = new GenericMutableRow(4) expected.update(0, 2147483647) - expected.update(1, "this is a string") + expected.setString(1, "this is a string") expected.update(2, false) expected.update(3, null) val actual1 = Row(2147483647, "this is a string", false, null) 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 0174aaee94246..4c48dca44498b 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 @@ -17,18 +17,14 @@ package org.apache.spark.sql -import org.apache.spark.sql.execution.GeneratedAggregate -import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ -import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types._ - -import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} +import org.apache.spark.sql.types._ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 5f08834f73c6b..c86ef338fc644 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -65,7 +65,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(FIXED_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(STRING, UTF8String("hello"), 4 + "hello".getBytes("utf-8").length) checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) @@ -108,8 +108,8 @@ class ColumnTypeSuite extends FunSuite with Logging { testNativeColumnType[StringType.type]( STRING, - (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes("utf-8") + (buffer: ByteBuffer, string: UTF8String) => { + val bytes = string.getBytes buffer.putInt(bytes.length) buffer.put(bytes) }, @@ -117,7 +117,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes) - new String(bytes, "utf-8") + UTF8String(bytes) }) testColumnType[BinaryType.type, Array[Byte]]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index b301818a008e7..f76314b9dab5e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.types.{Decimal, DataType, NativeType} +import org.apache.spark.sql.types.{UTF8String, DataType, Decimal, NativeType} object ColumnarTestUtils { def makeNullRow(length: Int): GenericMutableRow = { @@ -48,7 +48,7 @@ object ColumnarTestUtils { case FLOAT => Random.nextFloat() case DOUBLE => Random.nextDouble() case FIXED_DECIMAL(precision, scale) => Decimal(Random.nextLong() % 100, precision, scale) - case STRING => Random.nextString(Random.nextInt(32)) + case STRING => UTF8String(Random.nextString(Random.nextInt(32))) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) case DATE => Random.nextInt() 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 60c8c00bda4d5..3b47b8adf313b 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 @@ -74,7 +74,7 @@ case class AllDataTypesScan( i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -82,7 +82,7 @@ case class AllDataTypesScan( Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) } } } @@ -103,7 +103,7 @@ class TableScanSuite extends DataSourceTest { i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -111,7 +111,7 @@ class TableScanSuite extends DataSourceTest { Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) }.toSeq before { @@ -266,7 +266,7 @@ class TableScanSuite extends DataSourceTest { sqlTest( "SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema", - (1 to 10).map(i => Row(Seq(new Date((i + 2) * 8640000)))).toSeq) + (1 to 10).map(i => Row(Seq(new Date(1970, 1, i + 1)))).toSeq) test("Caching") { // Cached Query Execution 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 921c6194c7b76..74ae984f34866 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 @@ -34,7 +34,7 @@ import scala.collection.JavaConversions._ * 1. The Underlying data type in catalyst and in Hive * In catalyst: * Primitive => - * java.lang.String + * UTF8String * int / scala.Int * boolean / scala.Boolean * float / scala.Float @@ -239,9 +239,10 @@ private[hive] trait HiveInspectors { */ def unwrap(data: Any, oi: ObjectInspector): Any = oi match { case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null - case poi: WritableConstantStringObjectInspector => poi.getWritableConstantValue.toString + case poi: WritableConstantStringObjectInspector => + UTF8String(poi.getWritableConstantValue.toString) case poi: WritableConstantHiveVarcharObjectInspector => - poi.getWritableConstantValue.getHiveVarchar.getValue + UTF8String(poi.getWritableConstantValue.getHiveVarchar.getValue) case poi: WritableConstantHiveDecimalObjectInspector => HiveShim.toCatalystDecimal( PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector, @@ -284,10 +285,13 @@ private[hive] trait HiveInspectors { 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 + UTF8String(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue) + case hvoi: HiveVarcharObjectInspector => + UTF8String(hvoi.getPrimitiveJavaObject(data).getValue) case x: StringObjectInspector if x.preferWritable() => - x.getPrimitiveWritableObject(data).toString + UTF8String(x.getPrimitiveWritableObject(data).toString) + case x: StringObjectInspector => + UTF8String(x.getPrimitiveJavaObject(data)) 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) @@ -340,7 +344,9 @@ private[hive] trait HiveInspectors { */ protected def wrapperFor(oi: ObjectInspector): Any => Any = oi match { case _: JavaHiveVarcharObjectInspector => - (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) + (o: Any) => + val s = o.asInstanceOf[UTF8String].toString + new HiveVarchar(s, s.size) case _: JavaHiveDecimalObjectInspector => (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal) @@ -409,7 +415,7 @@ private[hive] trait HiveInspectors { 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 _: StringObjectInspector => a.asInstanceOf[UTF8String].toString() case _: IntObjectInspector if x.preferWritable() => HiveShim.getIntWritable(a) case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer] case _: BooleanObjectInspector if x.preferWritable() => HiveShim.getBooleanWritable(a) 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 1ccb0c279c60e..a6f4fbe8aba06 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,24 +17,21 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.catalyst.expressions.Row - import scala.collection.JavaConversions._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate +import org.apache.spark.sql.catalyst.expressions.{Row, _} 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.sources.DescribeCommand -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, _} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, CreateTableUsing} +import org.apache.spark.sql.sources.{CreateTableUsing, CreateTableUsingAsSelect, DescribeCommand} import org.apache.spark.sql.types.StringType @@ -131,7 +128,7 @@ private[hive] trait HiveStrategies { val partitionValues = part.getValues var i = 0 while (i < partitionValues.size()) { - inputData(i) = partitionValues(i) + inputData(i) = CatalystTypeConverters.convertToCatalyst(partitionValues(i)) i += 1 } pruningCondition(inputData) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 8efed7f0299bf..cab0fdd35723a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.hive.execution -import java.io.{BufferedReader, InputStreamReader} -import java.io.{DataInputStream, DataOutputStream, EOFException} +import java.io.{BufferedReader, DataInputStream, DataOutputStream, EOFException, InputStreamReader} import java.util.Properties import scala.collection.JavaConversions._ @@ -28,12 +27,13 @@ import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils /** @@ -121,14 +121,13 @@ case class ScriptTransformation( if (outputSerde == null) { val prevLine = curLine curLine = reader.readLine() - if (!ioschema.schemaLess) { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"))) .asInstanceOf[Array[Any]]) } else { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2)) .asInstanceOf[Array[Any]]) } } else { 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 902a12785e3e9..a40a1e53117cd 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 @@ -22,11 +22,11 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ /** * Analyzes the given table in the current database to generate statistics, which will be @@ -76,6 +76,12 @@ case class DropTable( private[hive] case class AddJar(path: String) extends RunnableCommand { + override val output: Seq[Attribute] = { + val schema = StructType( + StructField("result", IntegerType, false) :: Nil) + schema.toAttributes + } + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") 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 0ed93c2c5b1fa..33e96eaabfbf6 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 @@ -41,7 +41,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.sql.types.{Decimal, DecimalType} +import org.apache.spark.sql.types.{UTF8String, Decimal, DecimalType} private[hive] case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable { @@ -135,7 +135,7 @@ private[hive] object HiveShim { PrimitiveCategory.VOID, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) 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 7577309900209..d331c210e8939 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 @@ -17,37 +17,35 @@ package org.apache.spark.sql.hive -import java.util -import java.util.{ArrayList => JArrayList} -import java.util.Properties import java.rmi.server.UID +import java.util.{Properties, ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions +import com.esotericsoftware.kryo.Kryo import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.{NullWritable, Writable} -import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory} -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, PrimitiveObjectInspector, ObjectInspector} -import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} -import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorConverters, PrimitiveObjectInspector} +import org.apache.hadoop.hive.serde2.typeinfo.{DecimalTypeInfo, TypeInfo, TypeInfoFactory} +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo} +import org.apache.hadoop.io.{NullWritable, Writable} +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging -import org.apache.spark.sql.types.{Decimal, DecimalType} - +import org.apache.spark.sql.types.{Decimal, DecimalType, UTF8String} /** * This class provides the UDF creation and also the UDF instance serialization and @@ -63,18 +61,14 @@ private[hive] case class HiveFunctionWrapper(var functionClassName: String) // for Serialization def this() = this(null) - import java.io.{OutputStream, InputStream} - import com.esotericsoftware.kryo.Kryo import org.apache.spark.util.Utils._ - import org.apache.hadoop.hive.ql.exec.Utilities - import org.apache.hadoop.hive.ql.exec.UDF @transient private val methodDeSerialize = { val method = classOf[Utilities].getDeclaredMethod( "deserializeObjectByKryo", classOf[Kryo], - classOf[InputStream], + classOf[java.io.InputStream], classOf[Class[_]]) method.setAccessible(true) @@ -87,7 +81,7 @@ private[hive] case class HiveFunctionWrapper(var functionClassName: String) "serializeObjectByKryo", classOf[Kryo], classOf[Object], - classOf[OutputStream]) + classOf[java.io.OutputStream]) method.setAccessible(true) method @@ -224,7 +218,7 @@ private[hive] object HiveShim { TypeInfoFactory.voidTypeInfo, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) From cf38fe04f8782ff4573ae106ec0de8e8d183cb2b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:15:58 -0700 Subject: [PATCH 039/144] [SPARK-6844][SQL] Clean up accumulators used in InMemoryRelation when it is uncached JIRA: https://issues.apache.org/jira/browse/SPARK-6844 Author: Liang-Chi Hsieh Closes #5475 from viirya/cache_memory_leak and squashes the following commits: 0b41235 [Liang-Chi Hsieh] fix style. dc1d5d5 [Liang-Chi Hsieh] For comments. 78af229 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cache_memory_leak 26c9bb6 [Liang-Chi Hsieh] Add configuration to enable in-memory table scan accumulators. 1c3b06e [Liang-Chi Hsieh] Clean up accumulators used in InMemoryRelation when it is uncached. --- .../org/apache/spark/sql/CacheManager.scala | 2 +- .../columnar/InMemoryColumnarTableScan.scala | 47 ++++++++++++++----- .../apache/spark/sql/CachedTableSuite.scala | 18 +++++++ .../columnar/PartitionBatchPruningSuite.scala | 2 + 4 files changed, 55 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index ca4a127120b37..18584c2dcf797 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -112,7 +112,7 @@ private[sql] class CacheManager(sqlContext: SQLContext) extends Logging { val planToCache = query.queryExecution.analyzed val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) require(dataIndex >= 0, s"Table $query is not cached.") - cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData(dataIndex).cachedRepresentation.uncache(blocking) cachedData.remove(dataIndex) } 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 6eee0c86d6a1c..d9b6fb43ab83d 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 @@ -19,13 +19,15 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import org.apache.spark.Accumulator +import org.apache.spark.{Accumulable, Accumulator, Accumulators} import org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row +import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -53,11 +55,16 @@ private[sql] case class InMemoryRelation( child: SparkPlan, tableName: Option[String])( private var _cachedColumnBuffers: RDD[CachedBatch] = null, - private var _statistics: Statistics = null) + private var _statistics: Statistics = null, + private var _batchStats: Accumulable[ArrayBuffer[Row], Row] = null) extends LogicalPlan with MultiInstanceRelation { - private val batchStats = - child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + private val batchStats: Accumulable[ArrayBuffer[Row], Row] = + if (_batchStats == null) { + child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + } else { + _batchStats + } val partitionStatistics = new PartitionStatistics(output) @@ -161,7 +168,7 @@ private[sql] case class InMemoryRelation( def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, statisticsToBePropagated) + _cachedColumnBuffers, statisticsToBePropagated, batchStats) } override def children: Seq[LogicalPlan] = Seq.empty @@ -175,13 +182,20 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, - statisticsToBePropagated).asInstanceOf[this.type] + statisticsToBePropagated, + batchStats).asInstanceOf[this.type] } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, statisticsToBePropagated) + Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) + + private[sql] def uncache(blocking: Boolean): Unit = { + Accumulators.remove(batchStats.id) + cachedColumnBuffers.unpersist(blocking) + _cachedColumnBuffers = null + } } private[sql] case class InMemoryColumnarTableScan( @@ -244,15 +258,20 @@ private[sql] case class InMemoryColumnarTableScan( } } + lazy val enableAccumulators: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + // Accumulators used for testing purposes - val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) - val readBatches: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readBatches: Accumulator[Int] = sparkContext.accumulator(0) private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning override def execute(): RDD[Row] = { - readPartitions.setValue(0) - readBatches.setValue(0) + if (enableAccumulators) { + readPartitions.setValue(0) + readBatches.setValue(0) + } relation.cachedColumnBuffers.mapPartitions { cachedBatchIterator => val partitionFilter = newPredicate( @@ -302,7 +321,7 @@ private[sql] case class InMemoryColumnarTableScan( } } - if (rows.hasNext) { + if (rows.hasNext && enableAccumulators) { readPartitions += 1 } @@ -321,7 +340,9 @@ private[sql] case class InMemoryColumnarTableScan( logInfo(s"Skipping partition based on stats $statsString") false } else { - readBatches += 1 + if (enableAccumulators) { + readBatches += 1 + } true } } 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 f7b5f08beb92f..01e3b8671071e 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 @@ -22,6 +22,7 @@ import scala.language.{implicitConversions, postfixOps} import org.scalatest.concurrent.Eventually._ +import org.apache.spark.Accumulators import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ @@ -297,4 +298,21 @@ class CachedTableSuite extends QueryTest { sql("Clear CACHE") assert(cacheManager.isEmpty) } + + test("Clear accumulators when uncacheTable to prevent memory leaking") { + val accsSize = Accumulators.originals.size + + sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") + sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + cacheTable("t1") + cacheTable("t2") + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + uncacheTable("t1") + uncacheTable("t2") + + assert(accsSize >= Accumulators.originals.size) + } } 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 e57bb06e7263b..2a0b701cad7fa 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 @@ -39,6 +39,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be // Enable in-memory partition pruning setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") + // Enable in-memory table scan accumulators + setConf("spark.sql.inMemoryTableScanStatistics.enable", "true") } override protected def afterAll(): Unit = { From 557a797a273f1668065806cba53e19e6134a66d3 Mon Sep 17 00:00:00 2001 From: sboeschhuawei Date: Wed, 15 Apr 2015 13:28:10 -0700 Subject: [PATCH 040/144] [SPARK-6937][MLLIB] Fixed bug in PICExample in which the radius were not being accepted on c... Tiny bug in PowerIterationClusteringExample in which radius not accepted from command line Author: sboeschhuawei Closes #5531 from javadba/picsub and squashes the following commits: 2aab8cf [sboeschhuawei] Fixed bug in PICExample in which the radius were not being accepted on command line --- .../examples/mllib/PowerIterationClusteringExample.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 9f22d40c15f3f..6d8b806569dfd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -65,7 +65,7 @@ object PowerIterationClusteringExample { def main(args: Array[String]) { val defaultParams = Params() - val parser = new OptionParser[Params]("PIC Circles") { + val parser = new OptionParser[Params]("PowerIterationClusteringExample") { head("PowerIterationClusteringExample: an example PIC app using concentric circles.") opt[Int]('k', "k") .text(s"number of circles (/clusters), default: ${defaultParams.k}") @@ -76,9 +76,9 @@ object PowerIterationClusteringExample { opt[Int]("maxIterations") .text(s"number of iterations, default: ${defaultParams.maxIterations}") .action((x, c) => c.copy(maxIterations = x)) - opt[Int]('r', "r") + opt[Double]('r', "r") .text(s"radius of outermost circle, default: ${defaultParams.outerRadius}") - .action((x, c) => c.copy(numPoints = x)) + .action((x, c) => c.copy(outerRadius = x)) } parser.parse(args, defaultParams).map { params => @@ -154,3 +154,4 @@ object PowerIterationClusteringExample { coeff * math.exp(expCoeff * ssquares) } } + From 4754e16f4746ebd882b2ce7f1efc6e4d4408922c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Apr 2015 13:39:12 -0700 Subject: [PATCH 041/144] [SPARK-6898][SQL] completely support special chars in column names Even if we wrap column names in backticks like `` `a#$b.c` ``, we still handle the "." inside column name specially. I think it's fragile to use a special char to split name parts, why not put name parts in `UnresolvedAttribute` directly? Author: Wenchen Fan This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5511 from cloud-fan/6898 and squashes the following commits: 48e3e57 [Wenchen Fan] more style fix 820dc45 [Wenchen Fan] do not ignore newName in UnresolvedAttribute d81ad43 [Wenchen Fan] fix style 11699d6 [Wenchen Fan] completely support special chars in column names --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +-- .../sql/catalyst/analysis/Analyzer.scala | 13 ++++----- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 ++++- .../sql/catalyst/analysis/unresolved.scala | 14 ++++++++-- .../catalyst/plans/logical/LogicalPlan.scala | 27 +++++++++---------- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 -- .../org/apache/spark/sql/DataFrame.scala | 4 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 13 ++++++--- .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- 9 files changed, 52 insertions(+), 33 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 9a3531ceb3343..0af969cc5cc67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -381,13 +381,13 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | "(" ~> expression <~ ")" | function | dotExpressionHeader - | ident ^^ UnresolvedAttribute + | ident ^^ {case i => UnresolvedAttribute.quoted(i)} | signedPrimary | "~" ~> expression ^^ BitwiseNot ) protected lazy val dotExpressionHeader: Parser[Expression] = (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { - case i1 ~ i2 ~ rest => UnresolvedAttribute((Seq(i1, i2) ++ rest).mkString(".")) + case i1 ~ i2 ~ rest => UnresolvedAttribute(Seq(i1, i2) ++ rest) } } 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 8b68b0df35f48..cb49e5ad5586f 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 @@ -297,14 +297,15 @@ class Analyzer( case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressionsUp { - case u @ UnresolvedAttribute(name) if resolver(name, VirtualColumn.groupingIdName) && + case u @ UnresolvedAttribute(nameParts) if nameParts.length == 1 && + resolver(nameParts(0), VirtualColumn.groupingIdName) && q.isInstanceOf[GroupingAnalytics] => // Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics q.asInstanceOf[GroupingAnalytics].gid - case u @ UnresolvedAttribute(name) => + case u @ UnresolvedAttribute(nameParts) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = - withPosition(u) { q.resolveChildren(name, resolver).getOrElse(u) } + withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) } logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => @@ -383,12 +384,12 @@ class Analyzer( child: LogicalPlan, grandchild: LogicalPlan): (Seq[SortOrder], Seq[Attribute]) = { // Find any attributes that remain unresolved in the sort. - val unresolved: Seq[String] = - ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) + val unresolved: Seq[Seq[String]] = + ordering.flatMap(_.collect { case UnresolvedAttribute(nameParts) => nameParts }) // Create a map from name, to resolved attributes, when the desired name can be found // prior to the projection. - val resolved: Map[String, NamedExpression] = + val resolved: Map[Seq[String], NamedExpression] = unresolved.flatMap(u => grandchild.resolve(u, resolver).map(a => u -> a)).toMap // Construct a set that contains all of the attributes that we need to evaluate the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index fa02111385c06..1155dac28fc78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -46,8 +46,12 @@ trait CheckAnalysis { operator transformExpressionsUp { case a: Attribute if !a.resolved => if (operator.childrenResolved) { + val nameParts = a match { + case UnresolvedAttribute(nameParts) => nameParts + case _ => Seq(a.name) + } // Throw errors for specific problems with get field. - operator.resolveChildren(a.name, resolver, throwErrors = true) + operator.resolveChildren(nameParts, resolver, throwErrors = true) } val from = operator.inputSet.map(_.name).mkString(", ") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 300e9ba187bc5..3f567e3e8b2a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -49,7 +49,12 @@ case class UnresolvedRelation( /** * Holds the name of an attribute that has yet to be resolved. */ -case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { +case class UnresolvedAttribute(nameParts: Seq[String]) + extends Attribute with trees.LeafNode[Expression] { + + def name: String = + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") @@ -59,7 +64,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def newInstance(): UnresolvedAttribute = this override def withNullability(newNullability: Boolean): UnresolvedAttribute = this override def withQualifiers(newQualifiers: Seq[String]): UnresolvedAttribute = this - override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute(name) + override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -68,6 +73,11 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def toString: String = s"'$name" } +object UnresolvedAttribute { + def apply(name: String): UnresolvedAttribute = new UnresolvedAttribute(name.split("\\.")) + def quoted(name: String): UnresolvedAttribute = new UnresolvedAttribute(Seq(name)) +} + case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 579a0fb8d3f93..ae4620a4e5abf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -19,12 +19,11 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedGetField, Resolver} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, EliminateSubQueries, Resolver} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.types.{ArrayType, StructType, StructField} abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { @@ -111,10 +110,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ def resolveChildren( - name: String, + nameParts: Seq[String], resolver: Resolver, throwErrors: Boolean = false): Option[NamedExpression] = - resolve(name, children.flatMap(_.output), resolver, throwErrors) + resolve(nameParts, children.flatMap(_.output), resolver, throwErrors) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this @@ -122,10 +121,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * `[scope].AttributeName.[nested].[fields]...`. */ def resolve( - name: String, + nameParts: Seq[String], resolver: Resolver, throwErrors: Boolean = false): Option[NamedExpression] = - resolve(name, output, resolver, throwErrors) + resolve(nameParts, output, resolver, throwErrors) /** * Resolve the given `name` string against the given attribute, returning either 0 or 1 match. @@ -135,7 +134,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsTableColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { assert(nameParts.length > 1) @@ -155,7 +154,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { if (resolver(attribute.name, nameParts.head)) { @@ -167,13 +166,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { /** Performs attribute resolution given a name and a sequence of possible attributes. */ protected def resolve( - name: String, + nameParts: Seq[String], input: Seq[Attribute], resolver: Resolver, throwErrors: Boolean): Option[NamedExpression] = { - val parts = name.split("\\.") - // A sequence of possible candidate matches. // Each candidate is a tuple. The first element is a resolved attribute, followed by a list // of parts that are to be resolved. @@ -182,9 +179,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // and the second element will be List("c"). var candidates: Seq[(Attribute, List[String])] = { // If the name has 2 or more parts, try to resolve it as `table.column` first. - if (parts.length > 1) { + if (nameParts.length > 1) { input.flatMap { option => - resolveAsTableColumn(parts, resolver, option) + resolveAsTableColumn(nameParts, resolver, option) } } else { Seq.empty @@ -194,10 +191,12 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // If none of attributes match `table.column` pattern, we try to resolve it as a column. if (candidates.isEmpty) { candidates = input.flatMap { candidate => - resolveAsColumn(parts, resolver, candidate) + resolveAsColumn(nameParts, resolver, candidate) } } + def name = UnresolvedAttribute(nameParts).name + candidates.distinct match { // One match, no nested fields, use it. case Seq((a, Nil)) => Some(a) 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 6e3d6b9263e86..e10ddfdf5127c 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 @@ -27,8 +27,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import scala.collection.immutable - class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) val caseInsensitiveCatalog = new SimpleCatalog(false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 94ae2d65fd0e4..3235f85d5bbd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -158,7 +158,7 @@ class DataFrame private[sql]( } protected[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { + queryExecution.analyzed.resolve(colName.split("\\."), sqlContext.analyzer.resolver).getOrElse { throw new AnalysisException( s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") } @@ -166,7 +166,7 @@ class DataFrame private[sql]( protected[sql] def numericColumns: Seq[Expression] = { schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get + queryExecution.analyzed.resolve(n.name.split("\\."), sqlContext.analyzer.resolver).get } } 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 4c48dca44498b..d739e550f3e56 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 @@ -19,14 +19,13 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.TestData._ import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} import org.apache.spark.sql.types._ - class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { // Make sure the tables are loaded. TestData @@ -1125,7 +1124,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { val data = sparkContext.parallelize( Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) jsonRDD(data).registerTempTable("records") - sql("SELECT `key?number1` FROM records") + sql("SELECT `key?number1`, `key.number2` FROM records") } test("SPARK-3814 Support Bitwise & operator") { @@ -1225,4 +1224,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY c0.a"), Row(1)) checkAnswer(sql("SELECT b[0].a FROM t ORDER BY c0.a"), Row(1)) } + + test("SPARK-6898: complete support for special chars in column names") { + jsonRDD(sparkContext.makeRDD( + """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) + .registerTempTable("t") + + checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) + } } 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 53a204b8c2932..fd305eb480e63 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 @@ -1101,7 +1101,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token(".", qualifier :: Token(attr, Nil) :: Nil) => nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => - UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) + UnresolvedAttribute(qualifierName :+ cleanIdentifier(attr)) case other => UnresolvedGetField(other, attr) } From 585638e81ce09a72b9e7f95d38e0d432cfa02456 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 15 Apr 2015 14:06:10 -0700 Subject: [PATCH 042/144] [SPARK-2213] [SQL] sort merge join for spark sql Thanks for the initial work from Ishiihara in #3173 This PR introduce a new join method of sort merge join, which firstly ensure that keys of same value are in the same partition, and inside each partition the Rows are sorted by key. Then we can run down both sides together, find matched rows using [sort merge join](http://en.wikipedia.org/wiki/Sort-merge_join). In this way, we don't have to store the whole hash table of one side as hash join, thus we have less memory usage. Also, this PR would benefit from #3438 , making the sorting phrase much more efficient. We introduced a new configuration of "spark.sql.planner.sortMergeJoin" to switch between this(`true`) and ShuffledHashJoin(`false`), probably we want the default value of it be `false` at first. Author: Daoyuan Wang Author: Michael Armbrust This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5208 from adrian-wang/smj and squashes the following commits: 2493b9f [Daoyuan Wang] fix style 5049d88 [Daoyuan Wang] propagate rowOrdering for RangePartitioning f91a2ae [Daoyuan Wang] yin's comment: use external sort if option is enabled, add comments f515cd2 [Daoyuan Wang] yin's comment: outputOrdering, join suite refine ec8061b [Daoyuan Wang] minor change 413fd24 [Daoyuan Wang] Merge pull request #3 from marmbrus/pr/5208 952168a [Michael Armbrust] add type 5492884 [Michael Armbrust] copy when ordering 7ddd656 [Michael Armbrust] Cleanup addition of ordering requirements b198278 [Daoyuan Wang] inherit ordering in project c8e82a3 [Daoyuan Wang] fix style 6e897dd [Daoyuan Wang] hide boundReference from manually construct RowOrdering for key compare in smj 8681d73 [Daoyuan Wang] refactor Exchange and fix copy for sorting 2875ef2 [Daoyuan Wang] fix changed configuration 61d7f49 [Daoyuan Wang] add omitted comment 00a4430 [Daoyuan Wang] fix bug 078d69b [Daoyuan Wang] address comments: add comments, do sort in shuffle, and others 3af6ba5 [Daoyuan Wang] use buffer for only one side 171001f [Daoyuan Wang] change default outputordering 47455c9 [Daoyuan Wang] add apache license ... a28277f [Daoyuan Wang] fix style 645c70b [Daoyuan Wang] address comments using sort 068c35d [Daoyuan Wang] fix new style and add some tests 925203b [Daoyuan Wang] address comments 07ce92f [Daoyuan Wang] fix ArrayIndexOutOfBound 42fca0e [Daoyuan Wang] code clean e3ec096 [Daoyuan Wang] fix comment style.. 2edd235 [Daoyuan Wang] fix outputpartitioning 57baa40 [Daoyuan Wang] fix sort eval bug 303b6da [Daoyuan Wang] fix several errors 95db7ad [Daoyuan Wang] fix brackets for if-statement 4464f16 [Daoyuan Wang] fix error 880d8e9 [Daoyuan Wang] sort merge join for spark sql --- .../spark/sql/catalyst/expressions/rows.scala | 10 +- .../plans/physical/partitioning.scala | 13 ++ .../scala/org/apache/spark/sql/SQLConf.scala | 8 + .../org/apache/spark/sql/SQLContext.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 148 ++++++++++++--- .../spark/sql/execution/SparkPlan.scala | 6 + .../spark/sql/execution/SparkStrategies.scala | 11 +- .../spark/sql/execution/basicOperators.scala | 10 ++ .../sql/execution/joins/SortMergeJoin.scala | 169 ++++++++++++++++++ .../org/apache/spark/sql/JoinSuite.scala | 28 ++- .../SortMergeCompatibilitySuite.scala | 162 +++++++++++++++++ 11 files changed, 534 insertions(+), 33 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala create mode 100644 sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 1b62e17ff47fd..b6ec7d3417ef8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{UTF8String, StructType, NativeType} - +import org.apache.spark.sql.types.{UTF8String, DataType, StructType, NativeType} /** * An extended interface to [[Row]] that allows the values for each column to be updated. Setting @@ -239,3 +238,10 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { return 0 } } + +object RowOrdering { + def forSchema(dataTypes: Seq[DataType]): RowOrdering = + new RowOrdering(dataTypes.zipWithIndex.map { + case(dt, index) => new SortOrder(BoundReference(index, dt, nullable = true), Ascending) + }) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 288c11f69fe22..fb4217a44807b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -94,6 +94,9 @@ sealed trait Partitioning { * only compatible if the `numPartitions` of them is the same. */ def compatibleWith(other: Partitioning): Boolean + + /** Returns the expressions that are used to key the partitioning. */ + def keyExpressions: Seq[Expression] } case class UnknownPartitioning(numPartitions: Int) extends Partitioning { @@ -106,6 +109,8 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case UnknownPartitioning(_) => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object SinglePartition extends Partitioning { @@ -117,6 +122,8 @@ case object SinglePartition extends Partitioning { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object BroadcastPartitioning extends Partitioning { @@ -128,6 +135,8 @@ case object BroadcastPartitioning extends Partitioning { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } /** @@ -158,6 +167,8 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } + override def keyExpressions: Seq[Expression] = expressions + override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } @@ -200,6 +211,8 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case _ => false } + override def keyExpressions: Seq[Expression] = ordering.map(_.child) + override def eval(input: Row): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } 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 ee641bdfeb2d7..5c65f04ee8497 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 @@ -47,6 +47,7 @@ private[spark] object SQLConf { // 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. val EXTERNAL_SORT = "spark.sql.planner.externalSort" + val SORTMERGE_JOIN = "spark.sql.planner.sortMergeJoin" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -128,6 +129,13 @@ private[sql] class SQLConf extends Serializable { /** When true the planner will use the external sort, which may spill to disk. */ private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean + /** + * Sort merge join would sort the two side of join first, and then iterate both sides together + * only once to get all matches. Using sort merge join can save a lot of memory usage compared + * to HashJoin. + */ + private[spark] def sortMergeJoinEnabled: Boolean = getConf(SORTMERGE_JOIN, "false").toBoolean + /** * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode * that evaluates expressions found in queries. In general this custom code runs much faster 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 89a4faf35e0d2..f9f3eb2e03817 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 @@ -1081,7 +1081,7 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] val prepareForExecution = new RuleExecutor[SparkPlan] { val batches = - Batch("Add exchange", Once, AddExchange(self)) :: Nil + Batch("Add exchange", Once, EnsureRequirements(self)) :: Nil } protected[sql] def openSession(): SQLSession = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 437408d30bfd2..518fc9e57c708 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -19,24 +19,42 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions.{Attribute, RowOrdering} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair +object Exchange { + /** + * Returns true when the ordering expressions are a subset of the key. + * if true, ShuffledRDD can use `setKeyOrdering(orderingKey)` to sort within [[Exchange]]. + */ + def canSortWithShuffle(partitioning: Partitioning, desiredOrdering: Seq[SortOrder]): Boolean = { + desiredOrdering.map(_.child).toSet.subsetOf(partitioning.keyExpressions.toSet) + } +} + /** * :: DeveloperApi :: + * Performs a shuffle that will result in the desired `newPartitioning`. Optionally sorts each + * resulting partition based on expressions from the partition key. It is invalid to construct an + * exchange operator with a `newOrdering` that cannot be calculated using the partitioning key. */ @DeveloperApi -case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { +case class Exchange( + newPartitioning: Partitioning, + newOrdering: Seq[SortOrder], + child: SparkPlan) + extends UnaryNode { override def outputPartitioning: Partitioning = newPartitioning + override def outputOrdering: Seq[SortOrder] = newOrdering + override def output: Seq[Attribute] = child.output /** We must copy rows when sort based shuffle is on */ @@ -45,6 +63,20 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una private val bypassMergeThreshold = child.sqlContext.sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + private val keyOrdering = { + if (newOrdering.nonEmpty) { + val key = newPartitioning.keyExpressions + val boundOrdering = newOrdering.map { o => + val ordinal = key.indexOf(o.child) + if (ordinal == -1) sys.error(s"Invalid ordering on $o requested for $newPartitioning") + o.copy(child = BoundReference(ordinal, o.child.dataType, o.child.nullable)) + } + new RowOrdering(boundOrdering) + } else { + null // Ordering will not be used + } + } + override def execute(): RDD[Row] = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => @@ -56,7 +88,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una // we can avoid the defensive copies to improve performance. In the long run, we probably // want to include information in shuffle dependencies to indicate whether elements in the // source RDD should be copied. - val rdd = if (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) { + val willMergeSort = sortBasedShuffleOn && numPartitions > bypassMergeThreshold + + val rdd = if (willMergeSort || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => val hashExpressions = newMutableProjection(expressions, child.output)() iter.map(r => (hashExpressions(r).copy(), r.copy())) @@ -69,12 +103,17 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una } } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Row, Row](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Row, Row](rdd, part) + } shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => - val rdd = if (sortBasedShuffleOn) { + val rdd = if (sortBasedShuffleOn || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => iter.map(row => (row.copy(), null))} } else { child.execute().mapPartitions { iter => @@ -87,7 +126,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una implicit val ordering = new RowOrdering(sortingExpressions, child.output) val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Null, Null](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Null, Null](rdd, part) + } shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._1) @@ -120,27 +164,34 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una * Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]] * of input data meets the * [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for - * each operator by inserting [[Exchange]] Operators where required. + * each operator by inserting [[Exchange]] Operators where required. Also ensure that the + * required input partition ordering requirements are met. */ -private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPlan] { +private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[SparkPlan] { // TODO: Determine the number of partitions. def numPartitions: Int = sqlContext.conf.numShufflePartitions def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => - // Check if every child's outputPartitioning satisfies the corresponding + // True iff every child's outputPartitioning satisfies the corresponding // required data distribution. def meetsRequirements: Boolean = - !operator.requiredChildDistribution.zip(operator.children).map { + operator.requiredChildDistribution.zip(operator.children).forall { case (required, child) => val valid = child.outputPartitioning.satisfies(required) logDebug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid - }.exists(!_) + } - // Check if outputPartitionings of children are compatible with each other. + // True iff any of the children are incorrectly sorted. + def needsAnySort: Boolean = + operator.requiredChildOrdering.zip(operator.children).exists { + case (required, child) => required.nonEmpty && required != child.outputOrdering + } + + // True iff outputPartitionings of children are compatible with each other. // It is possible that every child satisfies its required data distribution // but two children have incompatible outputPartitionings. For example, // A dataset is range partitioned by "a.asc" (RangePartitioning) and another @@ -157,28 +208,69 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl case Seq(a,b) => a compatibleWith b }.exists(!_) - // Check if the partitioning we want to ensure is the same as the child's output - // partitioning. If so, we do not need to add the Exchange operator. - def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan): SparkPlan = - if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child + // Adds Exchange or Sort operators as required + def addOperatorsIfNecessary( + partitioning: Partitioning, + rowOrdering: Seq[SortOrder], + child: SparkPlan): SparkPlan = { + val needSort = rowOrdering.nonEmpty && child.outputOrdering != rowOrdering + val needsShuffle = child.outputPartitioning != partitioning + val canSortWithShuffle = Exchange.canSortWithShuffle(partitioning, rowOrdering) + + if (needSort && needsShuffle && canSortWithShuffle) { + Exchange(partitioning, rowOrdering, child) + } else { + val withShuffle = if (needsShuffle) { + Exchange(partitioning, Nil, child) + } else { + child + } - if (meetsRequirements && compatible) { + val withSort = if (needSort) { + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, withShuffle) + } else { + Sort(rowOrdering, global = false, withShuffle) + } + } else { + withShuffle + } + + withSort + } + } + + if (meetsRequirements && compatible && !needsAnySort) { operator } else { // At least one child does not satisfies its required data distribution or // at least one child's outputPartitioning is not compatible with another child's // outputPartitioning. In this case, we need to add Exchange operators. - val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { - case (AllTuples, child) => - addExchangeIfNecessary(SinglePartition, child) - case (ClusteredDistribution(clustering), child) => - addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child) - case (OrderedDistribution(ordering), child) => - addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child) - case (UnspecifiedDistribution, child) => child - case (dist, _) => sys.error(s"Don't know how to ensure $dist") + val requirements = + (operator.requiredChildDistribution, operator.requiredChildOrdering, operator.children) + + val fixedChildren = requirements.zipped.map { + case (AllTuples, rowOrdering, child) => + addOperatorsIfNecessary(SinglePartition, rowOrdering, child) + case (ClusteredDistribution(clustering), rowOrdering, child) => + addOperatorsIfNecessary(HashPartitioning(clustering, numPartitions), rowOrdering, child) + case (OrderedDistribution(ordering), rowOrdering, child) => + addOperatorsIfNecessary(RangePartitioning(ordering, numPartitions), rowOrdering, child) + + case (UnspecifiedDistribution, Seq(), child) => + child + case (UnspecifiedDistribution, rowOrdering, child) => + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, child) + } else { + Sort(rowOrdering, global = false, child) + } + + case (dist, ordering, _) => + sys.error(s"Don't know how to ensure $dist with ordering $ordering") } - operator.withNewChildren(repartitionedChildren) + + operator.withNewChildren(fixedChildren) } } } 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 fabcf6b4a0570..e159ffe66cb24 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 @@ -72,6 +72,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) + /** Specifies how data is ordered in each partition. */ + def outputOrdering: Seq[SortOrder] = Nil + + /** Specifies sort order for each partition requirements on the input data for this operator. */ + def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil) + /** * Runs this query returning the result as an RDD. */ 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 5b99e40c2f491..e687d01f57520 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 @@ -90,6 +90,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { left.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildLeft) + // If the sort merge join option is set, we want to use sort merge join prior to hashjoin + // for now let's support inner join first, then add outer join + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if sqlContext.conf.sortMergeJoinEnabled => + val mergeJoin = + joins.SortMergeJoin(leftKeys, rightKeys, planLater(left), planLater(right)) + condition.map(Filter(_, mergeJoin)).getOrElse(mergeJoin) :: Nil + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => val buildSide = if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { @@ -309,7 +317,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.OneRowRelation => execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => - execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil + execution.Exchange( + HashPartitioning(expressions, numPartitions), Nil, planLater(child)) :: Nil case e @ EvaluatePython(udf, child, _) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index f8221f41bc6c3..308dae236a5ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -41,6 +41,8 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends val resuableProjection = buildProjection() iter.map(resuableProjection) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -55,6 +57,8 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def execute(): RDD[Row] = child.execute().mapPartitions { iter => iter.filter(conditionEvaluator) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -147,6 +151,8 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. override def execute(): RDD[Row] = sparkContext.makeRDD(collectData(), 1) + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -172,6 +178,8 @@ case class Sort( } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -202,6 +210,8 @@ case class ExternalSort( } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala new file mode 100644 index 0000000000000..b5123668ba11e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -0,0 +1,169 @@ +/* + * 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 java.util.NoSuchElementException + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * Performs an sort merge join of two child relations. + */ +@DeveloperApi +case class SortMergeJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def output: Seq[Attribute] = left.output ++ right.output + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + // this is to manually construct an ordering that can be used to compare keys from both sides + private val keyOrdering: RowOrdering = RowOrdering.forSchema(leftKeys.map(_.dataType)) + + override def outputOrdering: Seq[SortOrder] = requiredOrders(leftKeys) + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + requiredOrders(leftKeys) :: requiredOrders(rightKeys) :: Nil + + @transient protected lazy val leftKeyGenerator = newProjection(leftKeys, left.output) + @transient protected lazy val rightKeyGenerator = newProjection(rightKeys, right.output) + + private def requiredOrders(keys: Seq[Expression]): Seq[SortOrder] = + keys.map(SortOrder(_, Ascending)) + + override def execute(): RDD[Row] = { + val leftResults = left.execute().map(_.copy()) + val rightResults = right.execute().map(_.copy()) + + leftResults.zipPartitions(rightResults) { (leftIter, rightIter) => + new Iterator[Row] { + // Mutable per row objects. + private[this] val joinRow = new JoinedRow5 + private[this] var leftElement: Row = _ + private[this] var rightElement: Row = _ + private[this] var leftKey: Row = _ + private[this] var rightKey: Row = _ + private[this] var rightMatches: CompactBuffer[Row] = _ + private[this] var rightPosition: Int = -1 + private[this] var stop: Boolean = false + private[this] var matchKey: Row = _ + + // initialize iterator + initialize() + + override final def hasNext: Boolean = nextMatchingPair() + + override final def next(): Row = { + if (hasNext) { + // we are using the buffered right rows and run down left iterator + val joinedRow = joinRow(leftElement, rightMatches(rightPosition)) + rightPosition += 1 + if (rightPosition >= rightMatches.size) { + rightPosition = 0 + fetchLeft() + if (leftElement == null || keyOrdering.compare(leftKey, matchKey) != 0) { + stop = false + rightMatches = null + } + } + joinedRow + } else { + // no more result + throw new NoSuchElementException + } + } + + private def fetchLeft() = { + if (leftIter.hasNext) { + leftElement = leftIter.next() + leftKey = leftKeyGenerator(leftElement) + } else { + leftElement = null + } + } + + private def fetchRight() = { + if (rightIter.hasNext) { + rightElement = rightIter.next() + rightKey = rightKeyGenerator(rightElement) + } else { + rightElement = null + } + } + + private def initialize() = { + fetchLeft() + fetchRight() + } + + /** + * Searches the right iterator for the next rows that have matches in left side, and store + * them in a buffer. + * + * @return true if the search is successful, and false if the right iterator runs out of + * tuples. + */ + private def nextMatchingPair(): Boolean = { + if (!stop && rightElement != null) { + // run both side to get the first match pair + while (!stop && leftElement != null && rightElement != null) { + val comparing = keyOrdering.compare(leftKey, rightKey) + // for inner join, we need to filter those null keys + stop = comparing == 0 && !leftKey.anyNull + if (comparing > 0 || rightKey.anyNull) { + fetchRight() + } else if (comparing < 0 || leftKey.anyNull) { + fetchLeft() + } + } + rightMatches = new CompactBuffer[Row]() + if (stop) { + stop = false + // iterate the right side to buffer all rows that matches + // as the records should be ordered, exit when we meet the first that not match + while (!stop && rightElement != null) { + rightMatches += rightElement + fetchRight() + stop = keyOrdering.compare(leftKey, rightKey) != 0 + } + if (rightMatches.size > 0) { + rightPosition = 0 + matchKey = leftKey + } + } + } + rightMatches != null && rightMatches.size > 0 + } + } + } + } +} 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 e4dee87849fd4..037d392c1f929 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 @@ -51,6 +51,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j case j: BroadcastLeftSemiJoinHash => j + case j: SortMergeJoin => j } assert(operators.size === 1) @@ -62,6 +63,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("join operator selection") { cacheManager.clearCache() + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]), ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]), @@ -91,17 +93,41 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", classOf[BroadcastNestedLoopJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } } test("broadcasted hash join operator selection") { cacheManager.clearCache() sql("CACHE TABLE testData") + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), ("SELECT * FROM testData join testData2 ON key = a and key = 2", classOf[BroadcastHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key = 2", classOf[BroadcastHashJoin]) + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key = 2", + classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } sql("UNCACHE TABLE testData") } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala new file mode 100644 index 0000000000000..65d070bd3cbde --- /dev/null +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala @@ -0,0 +1,162 @@ +/* + * 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.sql.SQLConf +import org.apache.spark.sql.hive.test.TestHive + +/** + * Runs the test cases that are included in the hive distribution with sort merge join is true. + */ +class SortMergeCompatibilitySuite extends HiveCompatibilitySuite { + override def beforeAll() { + super.beforeAll() + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "true") + } + + override def afterAll() { + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "false") + super.afterAll() + } + + override def whiteList = Seq( + "auto_join0", + "auto_join1", + "auto_join10", + "auto_join11", + "auto_join12", + "auto_join13", + "auto_join14", + "auto_join14_hadoop20", + "auto_join15", + "auto_join17", + "auto_join18", + "auto_join19", + "auto_join2", + "auto_join20", + "auto_join21", + "auto_join22", + "auto_join23", + "auto_join24", + "auto_join25", + "auto_join26", + "auto_join27", + "auto_join28", + "auto_join3", + "auto_join30", + "auto_join31", + "auto_join32", + "auto_join4", + "auto_join5", + "auto_join6", + "auto_join7", + "auto_join8", + "auto_join9", + "auto_join_filters", + "auto_join_nulls", + "auto_join_reordering_values", + "auto_smb_mapjoin_14", + "auto_sortmerge_join_1", + "auto_sortmerge_join_10", + "auto_sortmerge_join_11", + "auto_sortmerge_join_12", + "auto_sortmerge_join_13", + "auto_sortmerge_join_14", + "auto_sortmerge_join_15", + "auto_sortmerge_join_16", + "auto_sortmerge_join_2", + "auto_sortmerge_join_3", + "auto_sortmerge_join_4", + "auto_sortmerge_join_5", + "auto_sortmerge_join_6", + "auto_sortmerge_join_7", + "auto_sortmerge_join_8", + "auto_sortmerge_join_9", + "correlationoptimizer1", + "correlationoptimizer10", + "correlationoptimizer11", + "correlationoptimizer13", + "correlationoptimizer14", + "correlationoptimizer15", + "correlationoptimizer2", + "correlationoptimizer3", + "correlationoptimizer4", + "correlationoptimizer6", + "correlationoptimizer7", + "correlationoptimizer8", + "correlationoptimizer9", + "join0", + "join1", + "join10", + "join11", + "join12", + "join13", + "join14", + "join14_hadoop20", + "join15", + "join16", + "join17", + "join18", + "join19", + "join2", + "join20", + "join21", + "join22", + "join23", + "join24", + "join25", + "join26", + "join27", + "join28", + "join29", + "join3", + "join30", + "join31", + "join32", + "join32_lessSize", + "join33", + "join34", + "join35", + "join36", + "join37", + "join38", + "join39", + "join4", + "join40", + "join41", + "join5", + "join6", + "join7", + "join8", + "join9", + "join_1to1", + "join_array", + "join_casesensitive", + "join_empty", + "join_filters", + "join_hive_626", + "join_map_ppr", + "join_nulls", + "join_nullsafe", + "join_rc", + "join_reorder2", + "join_reorder3", + "join_reorder4", + "join_star" + ) +} From d5f1b9650b6e46cf6a9d61f01cda0df0cda5b1c9 Mon Sep 17 00:00:00 2001 From: Isaias Barroso Date: Wed, 15 Apr 2015 22:40:52 +0100 Subject: [PATCH 043/144] [SPARK-2312] Logging Unhandled messages The previous solution has changed based on https://github.com/apache/spark/pull/2048 discussions. Author: Isaias Barroso Closes #2055 from isaias/SPARK-2312 and squashes the following commits: f61d9e6 [Isaias Barroso] Change Log level for unhandled message to debug f341777 [Isaias Barroso] [SPARK-2312] Logging Unhandled messages --- .../scala/org/apache/spark/util/ActorLogReceive.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala index 332d0cbb2dc0c..81a7cbde01ce5 100644 --- a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala +++ b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala @@ -43,7 +43,13 @@ private[spark] trait ActorLogReceive { private val _receiveWithLogging = receiveWithLogging - override def isDefinedAt(o: Any): Boolean = _receiveWithLogging.isDefinedAt(o) + override def isDefinedAt(o: Any): Boolean = { + val handled = _receiveWithLogging.isDefinedAt(o) + if (!handled) { + log.debug(s"Received unexpected actor system event: $o") + } + handled + } override def apply(o: Any): Unit = { if (log.isDebugEnabled) { From 8a53de16fc8208358b76d0f3d45538e0304bcc8e Mon Sep 17 00:00:00 2001 From: Max Seiden Date: Wed, 15 Apr 2015 16:15:11 -0700 Subject: [PATCH 044/144] [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators There were a few places where new SparkSqlSerializer instances were created with new, empty SparkConfs resulting in user specified registrators sometimes not getting initialized. The fix is to try and pull a conf from the SparkEnv, and construct a new conf (that loads defaults) if one cannot be found. The changes touched: 1) SparkSqlSerializer's resource pool (this appears to fix the issue in the comment) 2) execution.Exchange (for all of the partitioners) 3) execution.Limit (for the HashPartitioner) A few tests were added to ColumnTypeSuite, ensuring that a custom registrator and serde is initialized and used when in-memory columns are written. Author: Max Seiden This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5237 from mhseiden/sql_udt_kryo and squashes the following commits: 3175c2f [Max Seiden] [SPARK-5277][SQL] - address code review comments e5011fb [Max Seiden] [SPARK-5277][SQL] - SparkSqlSerializer does not register user specified KryoRegistrators --- .../apache/spark/sql/execution/Exchange.scala | 9 +-- .../sql/execution/SparkSqlSerializer.scala | 7 +-- .../spark/sql/execution/basicOperators.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 62 ++++++++++++++++++- 4 files changed, 68 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 518fc9e57c708..69a620e1ec929 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -78,6 +78,8 @@ case class Exchange( } override def execute(): RDD[Row] = attachTree(this , "execute") { + lazy val sparkConf = child.sqlContext.sparkContext.getConf + newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. @@ -109,7 +111,7 @@ case class Exchange( } else { new ShuffledRDD[Row, Row, Row](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => @@ -132,8 +134,7 @@ case class Exchange( } else { new ShuffledRDD[Row, Null, Null](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) - + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._1) case SinglePartition => @@ -151,7 +152,7 @@ case class Exchange( } val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 914f387dec78f..eea15aff5dbcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -65,12 +65,9 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co private[execution] class KryoResourcePool(size: Int) extends ResourcePool[SerializerInstance](size) { - val ser: KryoSerializer = { + val ser: SparkSqlSerializer = { val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) - // TODO (lian) Using KryoSerializer here is workaround, needs further investigation - // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization - // related error. - new KryoSerializer(sparkConf) + new SparkSqlSerializer(sparkConf) } def newInstance(): SerializerInstance = ser.newInstance() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 308dae236a5ed..d286fe81bee5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -121,7 +121,7 @@ case class Limit(limit: Int, child: SparkPlan) } val part = new HashPartitioner(1) val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(child.sqlContext.sparkContext.getConf)) shuffled.mapPartitions(_.take(limit).map(_._2)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index c86ef338fc644..b48bed1871c50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer import java.sql.Timestamp +import com.esotericsoftware.kryo.{Serializer, Kryo} +import com.esotericsoftware.kryo.io.{Input, Output} +import org.apache.spark.serializer.KryoRegistrator import org.scalatest.FunSuite -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -73,7 +76,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(BINARY, binary, 4 + 4) val generic = Map(1 -> "a") - checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 11) + checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 8) } testNativeColumnType[BooleanType.type]( @@ -158,6 +161,41 @@ class ColumnTypeSuite extends FunSuite with Logging { } } + test("CUSTOM") { + val conf = new SparkConf() + conf.set("spark.kryo.registrator", "org.apache.spark.sql.columnar.Registrator") + val serializer = new SparkSqlSerializer(conf).newInstance() + + val buffer = ByteBuffer.allocate(512) + val obj = CustomClass(Int.MaxValue,Long.MaxValue) + val serializedObj = serializer.serialize(obj).array() + + GENERIC.append(serializer.serialize(obj).array(), buffer) + buffer.rewind() + + val length = buffer.getInt + assert(length === serializedObj.length) + assert(13 == length) // id (1) + int (4) + long (8) + + val genericSerializedObj = SparkSqlSerializer.serialize(obj) + assert(length != genericSerializedObj.length) + assert(length < genericSerializedObj.length) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + serializer.deserialize(ByteBuffer.wrap(bytes)) + } + + buffer.rewind() + buffer.putInt(serializedObj.length).put(serializedObj) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + buffer.rewind() + serializer.deserialize(ByteBuffer.wrap(GENERIC.extract(buffer))) + } + } + def testNativeColumnType[T <: NativeType]( columnType: NativeColumnType[T], putter: (ByteBuffer, T#JvmType) => Unit, @@ -229,3 +267,23 @@ class ColumnTypeSuite extends FunSuite with Logging { } } } + +private[columnar] final case class CustomClass(a: Int, b: Long) + +private[columnar] object CustomerSerializer extends Serializer[CustomClass] { + override def write(kryo: Kryo, output: Output, t: CustomClass) { + output.writeInt(t.a) + output.writeLong(t.b) + } + override def read(kryo: Kryo, input: Input, aClass: Class[CustomClass]): CustomClass = { + val a = input.readInt() + val b = input.readLong() + CustomClass(a,b) + } +} + +private[columnar] final class Registrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[CustomClass], CustomerSerializer) + } +} From 52c3439a8a107ce1fc10e4f0b59fd7881e851622 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Wed, 15 Apr 2015 21:52:25 -0700 Subject: [PATCH 045/144] SPARK-6938: All require statements now have an informative error message. This pr adds informative error messages to all require statements in the Vectors class that did not previously have them. This references [SPARK-6938](https://issues.apache.org/jira/browse/SPARK-6938). Author: Juliet Hougland Closes #5532 from jhlch/SPARK-6938 and squashes the following commits: ab321bb [Juliet Hougland] Remove braces from string interpolation when not required. 1221f94 [Juliet Hougland] All require statements now have an informative error message. --- .../org/apache/spark/mllib/linalg/Vectors.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 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 328dbe2ce11fa..4ef171f4f0419 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 @@ -227,7 +227,7 @@ object Vectors { * @param elements vector elements in (index, value) pairs. */ def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { - require(size > 0) + require(size > 0, "The size of the requested sparse vector must be greater than 0.") val (indices, values) = elements.sortBy(_._1).unzip var prev = -1 @@ -235,7 +235,8 @@ object Vectors { require(prev < i, s"Found duplicate indices: $i.") prev = i } - require(prev < size) + require(prev < size, s"You may not write an element to index $prev because the declared " + + s"size of your vector is $size") new SparseVector(size, indices.toArray, values.toArray) } @@ -309,7 +310,8 @@ object Vectors { * @return norm in L^p^ space. */ def norm(vector: Vector, p: Double): Double = { - require(p >= 1.0) + require(p >= 1.0, "To compute the p-norm of the vector, we require that you specify a p>=1. " + + s"You specified p=$p.") val values = vector match { case DenseVector(vs) => vs case SparseVector(n, ids, vs) => vs @@ -360,7 +362,8 @@ object Vectors { * @return squared distance between two Vectors. */ def sqdist(v1: Vector, v2: Vector): Double = { - require(v1.size == v2.size, "vector dimension mismatch") + require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + + s"=${v2.size}.") var squaredDistance = 0.0 (v1, v2) match { case (v1: SparseVector, v2: SparseVector) => @@ -518,7 +521,9 @@ class SparseVector( val indices: Array[Int], val values: Array[Double]) extends Vector { - require(indices.length == values.length) + require(indices.length == values.length, "Sparse vectors require that the dimension of the" + + s" indices match the dimension of the values. You provided ${indices.size} indices and " + + s" ${values.size} values.") override def toString: String = "(%s,%s,%s)".format(size, indices.mkString("[", ",", "]"), values.mkString("[", ",", "]")) From 57cd1e86d1d450f85fc9e296aff498a940452113 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 15 Apr 2015 23:49:42 -0700 Subject: [PATCH 046/144] [SPARK-6893][ML] default pipeline parameter handling in python Same as #5431 but for Python. jkbradley Author: Xiangrui Meng Closes #5534 from mengxr/SPARK-6893 and squashes the following commits: d3b519b [Xiangrui Meng] address comments ebaccc6 [Xiangrui Meng] style update fce244e [Xiangrui Meng] update explainParams with test 4d6b07a [Xiangrui Meng] add tests 5294500 [Xiangrui Meng] update default param handling in python --- .../org/apache/spark/ml/Identifiable.scala | 2 +- .../apache/spark/ml/param/TestParams.scala | 9 +- python/pyspark/ml/classification.py | 3 +- python/pyspark/ml/feature.py | 19 +-- python/pyspark/ml/param/__init__.py | 146 +++++++++++++++--- ...d_params.py => _shared_params_code_gen.py} | 42 ++--- python/pyspark/ml/param/shared.py | 106 ++++++------- python/pyspark/ml/pipeline.py | 6 +- python/pyspark/ml/tests.py | 52 ++++++- python/pyspark/ml/util.py | 4 +- python/pyspark/ml/wrapper.py | 2 +- 11 files changed, 270 insertions(+), 121 deletions(-) rename python/pyspark/ml/param/{_gen_shared_params.py => _shared_params_code_gen.py} (70%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala index a50090671ae48..a1d49095c24ac 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala @@ -25,7 +25,7 @@ import java.util.UUID private[ml] trait Identifiable extends Serializable { /** - * A unique id for the object. The default implementation concatenates the class name, "-", and 8 + * A unique id for the object. The default implementation concatenates the class name, "_", and 8 * random hex chars. */ private[ml] val uid: String = diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index 8f9ab687c05cb..641b64b42a5e7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -17,16 +17,13 @@ package org.apache.spark.ml.param +import org.apache.spark.ml.param.shared.{HasInputCol, HasMaxIter} + /** A subclass of Params for testing. */ -class TestParams extends Params { +class TestParams extends Params with HasMaxIter with HasInputCol { - val maxIter = new IntParam(this, "maxIter", "max number of iterations") def setMaxIter(value: Int): this.type = { set(maxIter, value); this } - def getMaxIter: Int = getOrDefault(maxIter) - - val inputCol = new Param[String](this, "inputCol", "input column name") def setInputCol(value: String): this.type = { set(inputCol, value); this } - def getInputCol: String = getOrDefault(inputCol) setDefault(maxIter -> 10) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 7f42de531f3b4..d7bc09fd77adb 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -59,6 +59,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxIter=100, regParam=0.1) """ super(LogisticRegression, self).__init__() + self._setDefault(maxIter=100, regParam=0.1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -71,7 +72,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre Sets params for logistic regression. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def _create_model(self, java_model): return LogisticRegressionModel(java_model) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1cfcd019dfb18..263fe2a5bcc41 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -52,22 +52,22 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): _java_class = "org.apache.spark.ml.feature.Tokenizer" @keyword_only - def __init__(self, inputCol="input", outputCol="output"): + def __init__(self, inputCol=None, outputCol=None): """ - __init__(self, inputCol="input", outputCol="output") + __init__(self, inputCol=None, outputCol=None) """ super(Tokenizer, self).__init__() kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, inputCol="input", outputCol="output"): + def setParams(self, inputCol=None, outputCol=None): """ setParams(self, inputCol="input", outputCol="output") Sets params for this Tokenizer. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) @inherit_doc @@ -91,22 +91,23 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): _java_class = "org.apache.spark.ml.feature.HashingTF" @keyword_only - def __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None) """ super(HashingTF, self).__init__() + self._setDefault(numFeatures=1 << 18) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None) Sets params for this HashingTF. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) if __name__ == "__main__": diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index e3a53dd780c4c..5c62620562a84 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -25,23 +25,21 @@ class Param(object): """ - A param with self-contained documentation and optionally default value. + A param with self-contained documentation. """ - def __init__(self, parent, name, doc, defaultValue=None): - if not isinstance(parent, Identifiable): - raise ValueError("Parent must be identifiable but got type %s." % type(parent).__name__) + def __init__(self, parent, name, doc): + if not isinstance(parent, Params): + raise ValueError("Parent must be a Params but got type %s." % type(parent).__name__) self.parent = parent self.name = str(name) self.doc = str(doc) - self.defaultValue = defaultValue def __str__(self): - return str(self.parent) + "-" + self.name + return str(self.parent) + "__" + self.name def __repr__(self): - return "Param(parent=%r, name=%r, doc=%r, defaultValue=%r)" % \ - (self.parent, self.name, self.doc, self.defaultValue) + return "Param(parent=%r, name=%r, doc=%r)" % (self.parent, self.name, self.doc) class Params(Identifiable): @@ -52,26 +50,128 @@ class Params(Identifiable): __metaclass__ = ABCMeta - def __init__(self): - super(Params, self).__init__() - #: embedded param map - self.paramMap = {} + #: internal param map for user-supplied values param map + paramMap = {} + + #: internal param map for default values + defaultParamMap = {} @property def params(self): """ - Returns all params. The default implementation uses - :py:func:`dir` to get all attributes of type + Returns all params ordered by name. The default implementation + uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ return filter(lambda attr: isinstance(attr, Param), [getattr(self, x) for x in dir(self) if x != "params"]) - def _merge_params(self, params): - paramMap = self.paramMap.copy() - paramMap.update(params) + def _explain(self, param): + """ + Explains a single param and returns its name, doc, and optional + default value and user-supplied value in a string. + """ + param = self._resolveParam(param) + values = [] + if self.isDefined(param): + if param in self.defaultParamMap: + values.append("default: %s" % self.defaultParamMap[param]) + if param in self.paramMap: + values.append("current: %s" % self.paramMap[param]) + else: + values.append("undefined") + valueStr = "(" + ", ".join(values) + ")" + return "%s: %s %s" % (param.name, param.doc, valueStr) + + def explainParams(self): + """ + Returns the documentation of all params with their optionally + default values and user-supplied values. + """ + return "\n".join([self._explain(param) for param in self.params]) + + def getParam(self, paramName): + """ + Gets a param by its name. + """ + param = getattr(self, paramName) + if isinstance(param, Param): + return param + else: + raise ValueError("Cannot find param with name %s." % paramName) + + def isSet(self, param): + """ + Checks whether a param is explicitly set by user. + """ + param = self._resolveParam(param) + return param in self.paramMap + + def hasDefault(self, param): + """ + Checks whether a param has a default value. + """ + param = self._resolveParam(param) + return param in self.defaultParamMap + + def isDefined(self, param): + """ + Checks whether a param is explicitly set by user or has a default value. + """ + return self.isSet(param) or self.hasDefault(param) + + def getOrDefault(self, param): + """ + Gets the value of a param in the user-supplied param map or its + default value. Raises an error if either is set. + """ + if isinstance(param, Param): + if param in self.paramMap: + return self.paramMap[param] + else: + return self.defaultParamMap[param] + elif isinstance(param, str): + return self.getOrDefault(self.getParam(param)) + else: + raise KeyError("Cannot recognize %r as a param." % param) + + def extractParamMap(self, extraParamMap={}): + """ + Extracts the embedded default param values and user-supplied + values, and then merges them with extra values from input into + a flat param map, where the latter value is used if there exist + conflicts, i.e., with ordering: default param values < + user-supplied values < extraParamMap. + :param extraParamMap: extra param values + :return: merged param map + """ + paramMap = self.defaultParamMap.copy() + paramMap.update(self.paramMap) + paramMap.update(extraParamMap) return paramMap + def _shouldOwn(self, param): + """ + Validates that the input param belongs to this Params instance. + """ + if param.parent is not self: + raise ValueError("Param %r does not belong to %r." % (param, self)) + + def _resolveParam(self, param): + """ + Resolves a param and validates the ownership. + :param param: param name or the param instance, which must + belong to this Params instance + :return: resolved param instance + """ + if isinstance(param, Param): + self._shouldOwn(param) + return param + elif isinstance(param, str): + return self.getParam(param) + else: + raise ValueError("Cannot resolve %r as a param." % param) + @staticmethod def _dummy(): """ @@ -81,10 +181,18 @@ def _dummy(): dummy.uid = "undefined" return dummy - def _set_params(self, **kwargs): + def _set(self, **kwargs): """ - Sets params. + Sets user-supplied params. """ for param, value in kwargs.iteritems(): self.paramMap[getattr(self, param)] = value return self + + def _setDefault(self, **kwargs): + """ + Sets default params. + """ + for param, value in kwargs.iteritems(): + self.defaultParamMap[getattr(self, param)] = value + return self diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_shared_params_code_gen.py similarity index 70% rename from python/pyspark/ml/param/_gen_shared_params.py rename to python/pyspark/ml/param/_shared_params_code_gen.py index 5eb81106f116c..55f422497672f 100644 --- a/python/pyspark/ml/param/_gen_shared_params.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -32,29 +32,34 @@ # limitations under the License. #""" +# Code generator for shared params (shared.py). Run under this folder with: +# python _shared_params_code_gen.py > shared.py -def _gen_param_code(name, doc, defaultValue): + +def _gen_param_code(name, doc, defaultValueStr): """ Generates Python code for a shared param class. :param name: param name :param doc: param doc - :param defaultValue: string representation of the param + :param defaultValueStr: string representation of the default value :return: code string """ # TODO: How to correctly inherit instance attributes? template = '''class Has$Name(Params): """ - Params with $name. + Mixin for param $name: $doc. """ # a placeholder to make it appear in the generated doc - $name = Param(Params._dummy(), "$name", "$doc", $defaultValue) + $name = Param(Params._dummy(), "$name", "$doc") def __init__(self): super(Has$Name, self).__init__() #: param for $doc - self.$name = Param(self, "$name", "$doc", $defaultValue) + self.$name = Param(self, "$name", "$doc") + if $defaultValueStr is not None: + self._setDefault($name=$defaultValueStr) def set$Name(self, value): """ @@ -67,32 +72,29 @@ def get$Name(self): """ Gets the value of $name or its default value. """ - if self.$name in self.paramMap: - return self.paramMap[self.$name] - else: - return self.$name.defaultValue''' + return self.getOrDefault(self.$name)''' - upperCamelName = name[0].upper() + name[1:] + Name = name[0].upper() + name[1:] return template \ .replace("$name", name) \ - .replace("$Name", upperCamelName) \ + .replace("$Name", Name) \ .replace("$doc", doc) \ - .replace("$defaultValue", defaultValue) + .replace("$defaultValueStr", str(defaultValueStr)) if __name__ == "__main__": print header - print "\n# DO NOT MODIFY. The code is generated by _gen_shared_params.py.\n" + print "\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n" print "from pyspark.ml.param import Param, Params\n\n" shared = [ - ("maxIter", "max number of iterations", "100"), - ("regParam", "regularization constant", "0.1"), + ("maxIter", "max number of iterations", None), + ("regParam", "regularization constant", None), ("featuresCol", "features column name", "'features'"), ("labelCol", "label column name", "'label'"), ("predictionCol", "prediction column name", "'prediction'"), - ("inputCol", "input column name", "'input'"), - ("outputCol", "output column name", "'output'"), - ("numFeatures", "number of features", "1 << 18")] + ("inputCol", "input column name", None), + ("outputCol", "output column name", None), + ("numFeatures", "number of features", None)] code = [] - for name, doc, defaultValue in shared: - code.append(_gen_param_code(name, doc, defaultValue)) + for name, doc, defaultValueStr in shared: + code.append(_gen_param_code(name, doc, defaultValueStr)) print "\n\n\n".join(code) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 586822f2de423..13b6749998ad0 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -15,23 +15,25 @@ # limitations under the License. # -# DO NOT MODIFY. The code is generated by _gen_shared_params.py. +# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py. from pyspark.ml.param import Param, Params class HasMaxIter(Params): """ - Params with maxIter. + Mixin for param maxIter: max number of iterations. """ # a placeholder to make it appear in the generated doc - maxIter = Param(Params._dummy(), "maxIter", "max number of iterations", 100) + maxIter = Param(Params._dummy(), "maxIter", "max number of iterations") def __init__(self): super(HasMaxIter, self).__init__() #: param for max number of iterations - self.maxIter = Param(self, "maxIter", "max number of iterations", 100) + self.maxIter = Param(self, "maxIter", "max number of iterations") + if None is not None: + self._setDefault(maxIter=None) def setMaxIter(self, value): """ @@ -44,24 +46,23 @@ def getMaxIter(self): """ Gets the value of maxIter or its default value. """ - if self.maxIter in self.paramMap: - return self.paramMap[self.maxIter] - else: - return self.maxIter.defaultValue + return self.getOrDefault(self.maxIter) class HasRegParam(Params): """ - Params with regParam. + Mixin for param regParam: regularization constant. """ # a placeholder to make it appear in the generated doc - regParam = Param(Params._dummy(), "regParam", "regularization constant", 0.1) + regParam = Param(Params._dummy(), "regParam", "regularization constant") def __init__(self): super(HasRegParam, self).__init__() #: param for regularization constant - self.regParam = Param(self, "regParam", "regularization constant", 0.1) + self.regParam = Param(self, "regParam", "regularization constant") + if None is not None: + self._setDefault(regParam=None) def setRegParam(self, value): """ @@ -74,24 +75,23 @@ def getRegParam(self): """ Gets the value of regParam or its default value. """ - if self.regParam in self.paramMap: - return self.paramMap[self.regParam] - else: - return self.regParam.defaultValue + return self.getOrDefault(self.regParam) class HasFeaturesCol(Params): """ - Params with featuresCol. + Mixin for param featuresCol: features column name. """ # a placeholder to make it appear in the generated doc - featuresCol = Param(Params._dummy(), "featuresCol", "features column name", 'features') + featuresCol = Param(Params._dummy(), "featuresCol", "features column name") def __init__(self): super(HasFeaturesCol, self).__init__() #: param for features column name - self.featuresCol = Param(self, "featuresCol", "features column name", 'features') + self.featuresCol = Param(self, "featuresCol", "features column name") + if 'features' is not None: + self._setDefault(featuresCol='features') def setFeaturesCol(self, value): """ @@ -104,24 +104,23 @@ def getFeaturesCol(self): """ Gets the value of featuresCol or its default value. """ - if self.featuresCol in self.paramMap: - return self.paramMap[self.featuresCol] - else: - return self.featuresCol.defaultValue + return self.getOrDefault(self.featuresCol) class HasLabelCol(Params): """ - Params with labelCol. + Mixin for param labelCol: label column name. """ # a placeholder to make it appear in the generated doc - labelCol = Param(Params._dummy(), "labelCol", "label column name", 'label') + labelCol = Param(Params._dummy(), "labelCol", "label column name") def __init__(self): super(HasLabelCol, self).__init__() #: param for label column name - self.labelCol = Param(self, "labelCol", "label column name", 'label') + self.labelCol = Param(self, "labelCol", "label column name") + if 'label' is not None: + self._setDefault(labelCol='label') def setLabelCol(self, value): """ @@ -134,24 +133,23 @@ def getLabelCol(self): """ Gets the value of labelCol or its default value. """ - if self.labelCol in self.paramMap: - return self.paramMap[self.labelCol] - else: - return self.labelCol.defaultValue + return self.getOrDefault(self.labelCol) class HasPredictionCol(Params): """ - Params with predictionCol. + Mixin for param predictionCol: prediction column name. """ # a placeholder to make it appear in the generated doc - predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name", 'prediction') + predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name") def __init__(self): super(HasPredictionCol, self).__init__() #: param for prediction column name - self.predictionCol = Param(self, "predictionCol", "prediction column name", 'prediction') + self.predictionCol = Param(self, "predictionCol", "prediction column name") + if 'prediction' is not None: + self._setDefault(predictionCol='prediction') def setPredictionCol(self, value): """ @@ -164,24 +162,23 @@ def getPredictionCol(self): """ Gets the value of predictionCol or its default value. """ - if self.predictionCol in self.paramMap: - return self.paramMap[self.predictionCol] - else: - return self.predictionCol.defaultValue + return self.getOrDefault(self.predictionCol) class HasInputCol(Params): """ - Params with inputCol. + Mixin for param inputCol: input column name. """ # a placeholder to make it appear in the generated doc - inputCol = Param(Params._dummy(), "inputCol", "input column name", 'input') + inputCol = Param(Params._dummy(), "inputCol", "input column name") def __init__(self): super(HasInputCol, self).__init__() #: param for input column name - self.inputCol = Param(self, "inputCol", "input column name", 'input') + self.inputCol = Param(self, "inputCol", "input column name") + if None is not None: + self._setDefault(inputCol=None) def setInputCol(self, value): """ @@ -194,24 +191,23 @@ def getInputCol(self): """ Gets the value of inputCol or its default value. """ - if self.inputCol in self.paramMap: - return self.paramMap[self.inputCol] - else: - return self.inputCol.defaultValue + return self.getOrDefault(self.inputCol) class HasOutputCol(Params): """ - Params with outputCol. + Mixin for param outputCol: output column name. """ # a placeholder to make it appear in the generated doc - outputCol = Param(Params._dummy(), "outputCol", "output column name", 'output') + outputCol = Param(Params._dummy(), "outputCol", "output column name") def __init__(self): super(HasOutputCol, self).__init__() #: param for output column name - self.outputCol = Param(self, "outputCol", "output column name", 'output') + self.outputCol = Param(self, "outputCol", "output column name") + if None is not None: + self._setDefault(outputCol=None) def setOutputCol(self, value): """ @@ -224,24 +220,23 @@ def getOutputCol(self): """ Gets the value of outputCol or its default value. """ - if self.outputCol in self.paramMap: - return self.paramMap[self.outputCol] - else: - return self.outputCol.defaultValue + return self.getOrDefault(self.outputCol) class HasNumFeatures(Params): """ - Params with numFeatures. + Mixin for param numFeatures: number of features. """ # a placeholder to make it appear in the generated doc - numFeatures = Param(Params._dummy(), "numFeatures", "number of features", 1 << 18) + numFeatures = Param(Params._dummy(), "numFeatures", "number of features") def __init__(self): super(HasNumFeatures, self).__init__() #: param for number of features - self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) + self.numFeatures = Param(self, "numFeatures", "number of features") + if None is not None: + self._setDefault(numFeatures=None) def setNumFeatures(self, value): """ @@ -254,7 +249,4 @@ def getNumFeatures(self): """ Gets the value of numFeatures or its default value. """ - if self.numFeatures in self.paramMap: - return self.paramMap[self.numFeatures] - else: - return self.numFeatures.defaultValue + return self.getOrDefault(self.numFeatures) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 83880a5afcd1d..d94ecfff09f66 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -124,10 +124,10 @@ def setParams(self, stages=[]): Sets params for Pipeline. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def fit(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) stages = paramMap[self.stages] for stage in stages: if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): @@ -164,7 +164,7 @@ def __init__(self, transformers): self.transformers = transformers def transform(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for t in self.transformers: dataset = t.transform(dataset, paramMap) return dataset diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index b627c2b4e930b..3a42bcf723894 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -33,6 +33,7 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import DataFrame from pyspark.ml.param import Param +from pyspark.ml.param.shared import HasMaxIter, HasInputCol from pyspark.ml.pipeline import Transformer, Estimator, Pipeline @@ -46,7 +47,7 @@ class MockTransformer(Transformer): def __init__(self): super(MockTransformer, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None @@ -62,7 +63,7 @@ class MockEstimator(Estimator): def __init__(self): super(MockEstimator, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None self.model = None @@ -111,5 +112,52 @@ def test_pipeline(self): self.assertEqual(6, dataset.index) +class TestParams(HasMaxIter, HasInputCol): + """ + A subclass of Params mixed with HasMaxIter and HasInputCol. + """ + + def __init__(self): + super(TestParams, self).__init__() + self._setDefault(maxIter=10) + + +class ParamTests(PySparkTestCase): + + def test_param(self): + testParams = TestParams() + maxIter = testParams.maxIter + self.assertEqual(maxIter.name, "maxIter") + self.assertEqual(maxIter.doc, "max number of iterations") + self.assertTrue(maxIter.parent is testParams) + + def test_params(self): + testParams = TestParams() + maxIter = testParams.maxIter + inputCol = testParams.inputCol + + params = testParams.params + self.assertEqual(params, [inputCol, maxIter]) + + self.assertTrue(testParams.hasDefault(maxIter)) + self.assertFalse(testParams.isSet(maxIter)) + self.assertTrue(testParams.isDefined(maxIter)) + self.assertEqual(testParams.getMaxIter(), 10) + testParams.setMaxIter(100) + self.assertTrue(testParams.isSet(maxIter)) + self.assertEquals(testParams.getMaxIter(), 100) + + self.assertFalse(testParams.hasDefault(inputCol)) + self.assertFalse(testParams.isSet(inputCol)) + self.assertFalse(testParams.isDefined(inputCol)) + with self.assertRaises(KeyError): + testParams.getInputCol() + + self.assertEquals( + testParams.explainParams(), + "\n".join(["inputCol: input column name (undefined)", + "maxIter: max number of iterations (default: 10, current: 100)"])) + + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 6f7f39c40eb5a..d3cb100a9efa5 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -40,8 +40,8 @@ class Identifiable(object): def __init__(self): #: A unique id for the object. The default implementation - #: concatenates the class name, "-", and 8 random hex chars. - self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] + #: concatenates the class name, "_", and 8 random hex chars. + self.uid = type(self).__name__ + "_" + uuid.uuid4().hex[:8] def __repr__(self): return self.uid diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 31a66b3d2f730..394f23c5e9b12 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -64,7 +64,7 @@ def _transfer_params_to_java(self, params, java_obj): :param params: additional params (overwriting embedded values) :param java_obj: Java object to receive the params """ - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for param in self.params: if param in paramMap: java_obj.set(param.name, paramMap[param]) From 8370550593f3549e90ace446961281dad0cd7498 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 16 Apr 2015 10:39:02 +0100 Subject: [PATCH 047/144] [Streaming][minor] Remove additional quote and unneeded imports Author: jerryshao Closes #5540 from jerryshao/minor-fix and squashes the following commits: ebaa646 [jerryshao] Minor fix --- .../apache/spark/examples/streaming/DirectKafkaWordCount.scala | 2 +- .../main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index 1c8a20bf8f1ae..11a8cf09533ce 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -41,7 +41,7 @@ object DirectKafkaWordCount { | is a list of one or more Kafka brokers | is a list of one or more kafka topics to consume from | - """".stripMargin) + """.stripMargin) System.exit(1) } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index a0b8a0c565210..a1b4a12e5d6a0 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -23,10 +23,9 @@ import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskC import org.apache.spark.rdd.RDD import org.apache.spark.util.NextIterator -import java.util.Properties import kafka.api.{FetchRequestBuilder, FetchResponse} import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.consumer.{ConsumerConfig, SimpleConsumer} +import kafka.consumer.SimpleConsumer import kafka.message.{MessageAndMetadata, MessageAndOffset} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties From 6179a948371897cecb7322ebda366c2de8ecaedd Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 16 Apr 2015 10:45:32 +0100 Subject: [PATCH 048/144] SPARK-4783 [CORE] System.exit() calls in SparkContext disrupt applications embedding Spark Avoid `System.exit(1)` in `TaskSchedulerImpl` and convert to `SparkException`; ensure scheduler calls `sc.stop()` even when this exception is thrown. CC mateiz aarondav as those who may have last touched this code. Author: Sean Owen Closes #5492 from srowen/SPARK-4783 and squashes the following commits: 60dc682 [Sean Owen] Avoid System.exit(1) in TaskSchedulerImpl and convert to SparkException; ensure scheduler calls sc.stop() even when this exception is thrown --- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 5 ++--- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 9 ++++++--- 2 files changed, 8 insertions(+), 6 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 2362cc7240039..ecc8bf189986d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -394,7 +394,7 @@ private[spark] class TaskSchedulerImpl( def error(message: String) { synchronized { - if (activeTaskSets.size > 0) { + if (activeTaskSets.nonEmpty) { // Have each task set throw a SparkException with the error for ((taskSetId, manager) <- activeTaskSets) { try { @@ -407,8 +407,7 @@ private[spark] class TaskSchedulerImpl( // No task sets are active but we still got an error. Just exit since this // must mean the error is during registration. // It might be good to do something smarter here in the future. - logError("Exiting due to error from cluster scheduler: " + message) - System.exit(1) + throw new SparkException(s"Exiting due to error from cluster scheduler: $message") } } } 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 ed5b7c1088196..ccf1dc5af6120 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 @@ -118,9 +118,12 @@ private[spark] class SparkDeploySchedulerBackend( notifyContext() if (!stopping) { logError("Application has been killed. Reason: " + reason) - scheduler.error(reason) - // Ensure the application terminates, as we can no longer run jobs. - sc.stop() + try { + scheduler.error(reason) + } finally { + // Ensure the application terminates, as we can no longer run jobs. + sc.stop() + } } } From de4fa6b6d12e2bee0307ffba2abfca0c33f15e45 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 16 Apr 2015 10:48:31 +0100 Subject: [PATCH 049/144] [SPARK-4194] [core] Make SparkContext initialization exception-safe. SparkContext has a very long constructor, where multiple things are initialized, multiple threads are spawned, and multiple opportunities for exceptions to be thrown exist. If one of these happens at an innoportune time, lots of garbage tends to stick around. This patch re-organizes SparkContext so that its internal state is initialized in a big "try" block. The fields keeping state are now completely private to SparkContext, and are "vars", because Scala doesn't allow you to initialize a val later. The existing API interface is kept by turning vals into defs (which works because Scala guarantees the same binary interface for those). On top of that, a few things in other areas were changed to avoid more things leaking: - Executor was changed to explicitly wait for the heartbeat thread to stop. LocalBackend was changed to wait for the "StopExecutor" message to be received, since otherwise there could be a race between that message arriving and the actor system being shut down. - ConnectionManager could possibly hang during shutdown, because an interrupt at the wrong moment could cause the selector thread to still call select and then wait forever. So also wake up the selector so that this situation is avoided. Author: Marcelo Vanzin Closes #5335 from vanzin/SPARK-4194 and squashes the following commits: 746b661 [Marcelo Vanzin] Fix borked merge. 80fc00e [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 408dada [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 2621609 [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 6b73fcb [Marcelo Vanzin] Scalastyle. c671c46 [Marcelo Vanzin] Fix merge. 3979aad [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 8caa8b3 [Marcelo Vanzin] [SPARK-4194] [core] Make SparkContext initialization exception-safe. 071f16e [Marcelo Vanzin] Nits. 27456b9 [Marcelo Vanzin] More exception safety. a0b0881 [Marcelo Vanzin] Stop alloc manager before scheduler. 5545d83 [Marcelo Vanzin] [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops. --- .../scala/org/apache/spark/SparkContext.scala | 505 ++++++++++-------- .../org/apache/spark/executor/Executor.scala | 33 +- .../spark/network/nio/ConnectionManager.scala | 7 +- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- .../spark/scheduler/local/LocalBackend.scala | 19 +- .../ExecutorAllocationManagerSuite.scala | 6 - 6 files changed, 329 insertions(+), 244 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3f1a7dd99d635..e106c5c4bef60 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -31,6 +31,7 @@ import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -50,9 +51,10 @@ import org.apache.spark.executor.{ExecutorEndpoint, TriggerThreadDump} import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.io.CompressionCodec +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ -import org.apache.spark.rpc.RpcAddress +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} @@ -192,8 +194,42 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // log out Spark Version in Spark driver log logInfo(s"Running Spark version $SPARK_VERSION") - private[spark] val conf = config.clone() - conf.validateSettings() + /* ------------------------------------------------------------------------------------- * + | Private variables. These variables keep the internal state of the context, and are | + | not accessible by the outside world. They're mutable since we want to initialize all | + | of them to some neutral value ahead of time, so that calling "stop()" while the | + | constructor is still running is safe. | + * ------------------------------------------------------------------------------------- */ + + private var _conf: SparkConf = _ + private var _eventLogDir: Option[URI] = None + private var _eventLogCodec: Option[String] = None + private var _env: SparkEnv = _ + private var _metadataCleaner: MetadataCleaner = _ + private var _jobProgressListener: JobProgressListener = _ + private var _statusTracker: SparkStatusTracker = _ + private var _progressBar: Option[ConsoleProgressBar] = None + private var _ui: Option[SparkUI] = None + private var _hadoopConfiguration: Configuration = _ + private var _executorMemory: Int = _ + private var _schedulerBackend: SchedulerBackend = _ + private var _taskScheduler: TaskScheduler = _ + private var _heartbeatReceiver: RpcEndpointRef = _ + @volatile private var _dagScheduler: DAGScheduler = _ + private var _applicationId: String = _ + private var _eventLogger: Option[EventLoggingListener] = None + private var _executorAllocationManager: Option[ExecutorAllocationManager] = None + private var _cleaner: Option[ContextCleaner] = None + private var _listenerBusStarted: Boolean = false + private var _jars: Seq[String] = _ + private var _files: Seq[String] = _ + + /* ------------------------------------------------------------------------------------- * + | Accessors and public fields. These provide access to the internal state of the | + | context. | + * ------------------------------------------------------------------------------------- */ + + private[spark] def conf: SparkConf = _conf /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -201,65 +237,24 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ def getConf: SparkConf = conf.clone() - if (!conf.contains("spark.master")) { - throw new SparkException("A master URL must be set in your configuration") - } - if (!conf.contains("spark.app.name")) { - throw new SparkException("An application name must be set in your configuration") - } - - if (conf.getBoolean("spark.logConf", false)) { - logInfo("Spark configuration:\n" + conf.toDebugString) - } - - // Set Spark driver host and port system properties - conf.setIfMissing("spark.driver.host", Utils.localHostName()) - conf.setIfMissing("spark.driver.port", "0") - - val jars: Seq[String] = - conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val files: Seq[String] = - conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val master = conf.get("spark.master") - val appName = conf.get("spark.app.name") + def jars: Seq[String] = _jars + def files: Seq[String] = _files + def master: String = _conf.get("spark.master") + def appName: String = _conf.get("spark.app.name") - private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) - private[spark] val eventLogDir: Option[URI] = { - if (isEventLogEnabled) { - val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) - .stripSuffix("/") - Some(Utils.resolveURI(unresolvedDir)) - } else { - None - } - } - private[spark] val eventLogCodec: Option[String] = { - val compress = conf.getBoolean("spark.eventLog.compress", false) - if (compress && isEventLogEnabled) { - Some(CompressionCodec.getCodecName(conf)).map(CompressionCodec.getShortName) - } else { - None - } - } + private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) + private[spark] def eventLogDir: Option[URI] = _eventLogDir + private[spark] def eventLogCodec: Option[String] = _eventLogCodec // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() - conf.set("spark.tachyonStore.folderName", tachyonFolderName) - val isLocal = (master == "local" || master.startsWith("local[")) - - if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + def isLocal: Boolean = (master == "local" || master.startsWith("local[")) // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - - // Create the Spark execution environment (cache, map output tracker, etc) - // This function allows components created by SparkEnv to be mocked in unit tests: private[spark] def createSparkEnv( conf: SparkConf, @@ -268,8 +263,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli SparkEnv.createDriverEnv(conf, isLocal, listenerBus) } - private[spark] val env = createSparkEnv(conf, isLocal, listenerBus) - SparkEnv.set(env) + private[spark] def env: SparkEnv = _env // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() @@ -277,35 +271,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] - private[spark] val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - + private[spark] def metadataCleaner: MetadataCleaner = _metadataCleaner + private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener - private[spark] val jobProgressListener = new JobProgressListener(conf) - listenerBus.addListener(jobProgressListener) + def statusTracker: SparkStatusTracker = _statusTracker - val statusTracker = new SparkStatusTracker(this) + private[spark] def progressBar: Option[ConsoleProgressBar] = _progressBar - private[spark] val progressBar: Option[ConsoleProgressBar] = - if (conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { - Some(new ConsoleProgressBar(this)) - } else { - None - } - - // Initialize the Spark UI - private[spark] val ui: Option[SparkUI] = - if (conf.getBoolean("spark.ui.enabled", true)) { - Some(SparkUI.createLiveUI(this, conf, listenerBus, jobProgressListener, - env.securityManager,appName)) - } else { - // For tests, do not enable the UI - None - } - - // Bind the UI before starting the task scheduler to communicate - // the bound port to the cluster manager properly - ui.foreach(_.bind()) + private[spark] def ui: Option[SparkUI] = _ui /** * A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. @@ -313,134 +286,248 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * '''Note:''' As it will be reused in all Hadoop RDDs, it's better not to modify it unless you * plan to set some global configurations for all Hadoop RDDs. */ - val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) + def hadoopConfiguration: Configuration = _hadoopConfiguration + + private[spark] def executorMemory: Int = _executorMemory + + // Environment variables to pass to our executors. + private[spark] val executorEnvs = HashMap[String, String]() + + // Set SPARK_USER for user who is running SparkContext. + val sparkUser = Utils.getCurrentUserName() - // Add each JAR given through the constructor - if (jars != null) { - jars.foreach(addJar) + private[spark] def schedulerBackend: SchedulerBackend = _schedulerBackend + private[spark] def schedulerBackend_=(sb: SchedulerBackend): Unit = { + _schedulerBackend = sb } - if (files != null) { - files.foreach(addFile) + private[spark] def taskScheduler: TaskScheduler = _taskScheduler + private[spark] def taskScheduler_=(ts: TaskScheduler): Unit = { + _taskScheduler = ts } + private[spark] def dagScheduler: DAGScheduler = _dagScheduler + private[spark] def dagScheduler_=(ds: DAGScheduler): Unit = { + _dagScheduler = ds + } + + def applicationId: String = _applicationId + + def metricsSystem: MetricsSystem = if (_env != null) _env.metricsSystem else null + + private[spark] def eventLogger: Option[EventLoggingListener] = _eventLogger + + private[spark] def executorAllocationManager: Option[ExecutorAllocationManager] = + _executorAllocationManager + + private[spark] def cleaner: Option[ContextCleaner] = _cleaner + + private[spark] var checkpointDir: Option[String] = None + + // Thread Local variable that can be used by users to pass information down the stack + private val localProperties = new InheritableThreadLocal[Properties] { + override protected def childValue(parent: Properties): Properties = new Properties(parent) + override protected def initialValue(): Properties = new Properties() + } + + /* ------------------------------------------------------------------------------------- * + | Initialization. This code initializes the context in a manner that is exception-safe. | + | All internal fields holding state are initialized here, and any error prompts the | + | stop() method to be called. | + * ------------------------------------------------------------------------------------- */ + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value } - private[spark] val executorMemory = conf.getOption("spark.executor.memory") - .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) - .orElse(Option(System.getenv("SPARK_MEM")).map(warnSparkMem)) - .map(Utils.memoryStringToMb) - .getOrElse(512) + try { + _conf = config.clone() + _conf.validateSettings() - // Environment variables to pass to our executors. - private[spark] val executorEnvs = HashMap[String, String]() + if (!_conf.contains("spark.master")) { + throw new SparkException("A master URL must be set in your configuration") + } + if (!_conf.contains("spark.app.name")) { + throw new SparkException("An application name must be set in your configuration") + } - // Convert java options to env vars as a work around - // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) - value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value - } - Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => - executorEnvs("SPARK_PREPEND_CLASSES") = v - } - // The Mesos scheduler backend relies on this environment variable to set executor memory. - // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv + if (_conf.getBoolean("spark.logConf", false)) { + logInfo("Spark configuration:\n" + _conf.toDebugString) + } - // Set SPARK_USER for user who is running SparkContext. - val sparkUser = Utils.getCurrentUserName() - executorEnvs("SPARK_USER") = sparkUser + // Set Spark driver host and port system properties + _conf.setIfMissing("spark.driver.host", Utils.localHostName()) + _conf.setIfMissing("spark.driver.port", "0") - // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will - // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) - private val heartbeatReceiver = env.rpcEnv.setupEndpoint( - HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) + _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - // Create and start the scheduler - private[spark] var (schedulerBackend, taskScheduler) = - SparkContext.createTaskScheduler(this, master) + _jars =_conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)) + .toSeq.flatten - heartbeatReceiver.send(TaskSchedulerIsSet) + _eventLogDir = + if (isEventLogEnabled) { + val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) + .stripSuffix("/") + Some(Utils.resolveURI(unresolvedDir)) + } else { + None + } - @volatile private[spark] var dagScheduler: DAGScheduler = _ - try { - dagScheduler = new DAGScheduler(this) - } catch { - case e: Exception => { - try { - stop() - } finally { - throw new SparkException("Error while constructing DAGScheduler", e) + _eventLogCodec = { + val compress = _conf.getBoolean("spark.eventLog.compress", false) + if (compress && isEventLogEnabled) { + Some(CompressionCodec.getCodecName(_conf)).map(CompressionCodec.getShortName) + } else { + None } } - } - // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's - // constructor - taskScheduler.start() + _conf.set("spark.tachyonStore.folderName", tachyonFolderName) - val applicationId: String = taskScheduler.applicationId() - conf.set("spark.app.id", applicationId) + if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") - env.blockManager.initialize(applicationId) + // Create the Spark execution environment (cache, map output tracker, etc) + _env = createSparkEnv(_conf, isLocal, listenerBus) + SparkEnv.set(_env) - val metricsSystem = env.metricsSystem + _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) - // 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))) + _jobProgressListener = new JobProgressListener(_conf) + listenerBus.addListener(jobProgressListener) - // Optionally log Spark events - private[spark] val eventLogger: Option[EventLoggingListener] = { - if (isEventLogEnabled) { - val logger = - new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) - logger.start() - listenerBus.addListener(logger) - Some(logger) - } else None - } + _statusTracker = new SparkStatusTracker(this) - // Optionally scale number of executors dynamically based on workload. Exposed for testing. - private val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) - private val dynamicAllocationTesting = conf.getBoolean("spark.dynamicAllocation.testing", false) - private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = - if (dynamicAllocationEnabled) { - assert(supportDynamicAllocation, - "Dynamic allocation of executors is currently only supported in YARN mode") - Some(new ExecutorAllocationManager(this, listenerBus, conf)) - } else { - None + _progressBar = + if (_conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { + Some(new ConsoleProgressBar(this)) + } else { + None + } + + _ui = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener, + _env.securityManager,appName)) + } else { + // For tests, do not enable the UI + None + } + // Bind the UI before starting the task scheduler to communicate + // the bound port to the cluster manager properly + _ui.foreach(_.bind()) + + _hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(_conf) + + // Add each JAR given through the constructor + if (jars != null) { + jars.foreach(addJar) } - executorAllocationManager.foreach(_.start()) - private[spark] val cleaner: Option[ContextCleaner] = { - if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { - Some(new ContextCleaner(this)) - } else { - None + if (files != null) { + files.foreach(addFile) } - } - cleaner.foreach(_.start()) - setupAndStartListenerBus() - postEnvironmentUpdate() - postApplicationStart() + _executorMemory = _conf.getOption("spark.executor.memory") + .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) + .orElse(Option(System.getenv("SPARK_MEM")) + .map(warnSparkMem)) + .map(Utils.memoryStringToMb) + .getOrElse(512) + + // Convert java options to env vars as a work around + // since we can't set env vars directly in sbt. + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) + value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { + executorEnvs(envKey) = value + } + Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => + executorEnvs("SPARK_PREPEND_CLASSES") = v + } + // The Mesos scheduler backend relies on this environment variable to set executor memory. + // TODO: Set this only in the Mesos scheduler. + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + executorEnvs ++= _conf.getExecutorEnv + executorEnvs("SPARK_USER") = sparkUser + + // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will + // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) + _heartbeatReceiver = env.rpcEnv.setupEndpoint( + HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) + + // Create and start the scheduler + val (sched, ts) = SparkContext.createTaskScheduler(this, master) + _schedulerBackend = sched + _taskScheduler = ts + _dagScheduler = new DAGScheduler(this) + _heartbeatReceiver.send(TaskSchedulerIsSet) + + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + _taskScheduler.start() + + _applicationId = _taskScheduler.applicationId() + _conf.set("spark.app.id", _applicationId) + _env.blockManager.initialize(_applicationId) + + // 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))) + + _eventLogger = + if (isEventLogEnabled) { + val logger = + new EventLoggingListener(_applicationId, _eventLogDir.get, _conf, _hadoopConfiguration) + logger.start() + listenerBus.addListener(logger) + Some(logger) + } else { + None + } - private[spark] var checkpointDir: Option[String] = None + // Optionally scale number of executors dynamically based on workload. Exposed for testing. + val dynamicAllocationEnabled = _conf.getBoolean("spark.dynamicAllocation.enabled", false) + _executorAllocationManager = + if (dynamicAllocationEnabled) { + assert(supportDynamicAllocation, + "Dynamic allocation of executors is currently only supported in YARN mode") + Some(new ExecutorAllocationManager(this, listenerBus, _conf)) + } else { + None + } + _executorAllocationManager.foreach(_.start()) - // Thread Local variable that can be used by users to pass information down the stack - private val localProperties = new InheritableThreadLocal[Properties] { - override protected def childValue(parent: Properties): Properties = new Properties(parent) - override protected def initialValue(): Properties = new Properties() + _cleaner = + if (_conf.getBoolean("spark.cleaner.referenceTracking", true)) { + Some(new ContextCleaner(this)) + } else { + None + } + _cleaner.foreach(_.start()) + + setupAndStartListenerBus() + postEnvironmentUpdate() + postApplicationStart() + + // Post init + _taskScheduler.postStartHook() + _env.metricsSystem.registerSource(new DAGSchedulerSource(dagScheduler)) + _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) + } catch { + case NonFatal(e) => + logError("Error initializing SparkContext.", e) + try { + stop() + } catch { + case NonFatal(inner) => + logError("Error stopping SparkContext after init error.", inner) + } finally { + throw e + } } /** @@ -544,19 +631,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } - // Post init - taskScheduler.postStartHook() - - private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) - private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - - private def initDriverMetrics() { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) - } - - initDriverMetrics() - // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. @@ -1146,7 +1220,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * this application is supported. This is currently only available for YARN. */ private[spark] def supportDynamicAllocation = - master.contains("yarn") || dynamicAllocationTesting + master.contains("yarn") || _conf.getBoolean("spark.dynamicAllocation.testing", false) /** * :: DeveloperApi :: @@ -1163,7 +1237,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * This is currently only supported in YARN mode. Return whether the request is received. */ private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = { - assert(master.contains("yarn") || dynamicAllocationTesting, + assert(supportDynamicAllocation, "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => @@ -1403,28 +1477,40 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def stop() { // Use the stopping variable to ensure no contention for the stop scenario. // Still track the stopped variable for use elsewhere in the code. - if (!stopped.compareAndSet(false, true)) { logInfo("SparkContext already stopped.") return } - + postApplicationEnd() - ui.foreach(_.stop()) - env.metricsSystem.report() - metadataCleaner.cancel() - cleaner.foreach(_.stop()) - executorAllocationManager.foreach(_.stop()) - dagScheduler.stop() - dagScheduler = null - listenerBus.stop() - eventLogger.foreach(_.stop()) - env.rpcEnv.stop(heartbeatReceiver) - progressBar.foreach(_.stop()) - taskScheduler = null + _ui.foreach(_.stop()) + if (env != null) { + env.metricsSystem.report() + } + if (metadataCleaner != null) { + metadataCleaner.cancel() + } + _cleaner.foreach(_.stop()) + _executorAllocationManager.foreach(_.stop()) + if (_dagScheduler != null) { + _dagScheduler.stop() + _dagScheduler = null + } + if (_listenerBusStarted) { + listenerBus.stop() + _listenerBusStarted = false + } + _eventLogger.foreach(_.stop()) + if (env != null && _heartbeatReceiver != null) { + env.rpcEnv.stop(_heartbeatReceiver) + } + _progressBar.foreach(_.stop()) + _taskScheduler = null // TODO: Cache.stop()? - env.stop() - SparkEnv.set(null) + if (_env != null) { + _env.stop() + SparkEnv.set(null) + } SparkContext.clearActiveContext() logInfo("Successfully stopped SparkContext") } @@ -1749,6 +1835,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } listenerBus.start(this) + _listenerBusStarted = true } /** Post the application start event */ @@ -2152,7 +2239,7 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, 1) + val backend = new LocalBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) (backend, scheduler) @@ -2164,7 +2251,7 @@ object SparkContext extends Logging { throw new SparkException(s"Asked to run locally with $threadCount threads") } val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) @@ -2174,7 +2261,7 @@ object SparkContext extends Logging { // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) 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 516f619529c48..1b5fdeba28ee2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,7 +21,7 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, Executors, TimeUnit} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -60,8 +60,6 @@ private[spark] class Executor( private val conf = env.conf - @volatile private var isStopped = false - // No ip or host:port - just hostname Utils.checkHost(executorHostname, "Expected executed slave to be a hostname") // must not have port specified. @@ -114,6 +112,10 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] + // Executor for the heartbeat task. + private val heartbeater = Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("driver-heartbeater")) + startDriverHeartbeater() def launchTask( @@ -138,7 +140,8 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() env.rpcEnv.stop(executorEndpoint) - isStopped = true + heartbeater.shutdown() + heartbeater.awaitTermination(10, TimeUnit.SECONDS) threadPool.shutdown() if (!isLocal) { env.stop() @@ -432,23 +435,17 @@ private[spark] class Executor( } /** - * Starts a thread to report heartbeat and partial metrics for active tasks to driver. - * This thread stops running when the executor is stopped. + * Schedules a task to report heartbeat and partial metrics for active tasks to driver. */ private def startDriverHeartbeater(): Unit = { val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") - val thread = new Thread() { - override def run() { - // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) - while (!isStopped) { - reportHeartBeat() - Thread.sleep(intervalMs) - } - } + + // Wait a random interval so the heartbeats don't end up in sync + val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] + + val heartbeatTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat()) } - thread.setDaemon(true) - thread.setName("driver-heartbeater") - thread.start() + heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) } } 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 8e3c30fc3d781..5a74c13b38bf7 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 @@ -86,11 +86,11 @@ private[nio] class ConnectionManager( conf.get("spark.network.timeout", "120s")) // 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 + // + // 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) @@ -989,6 +989,7 @@ private[nio] class ConnectionManager( def stop() { ackTimeoutMonitor.stop() + selector.wakeup() selectorThread.interrupt() selectorThread.join() selector.close() 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 ecc8bf189986d..13a52d836f32f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -142,11 +142,10 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") - import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } - } + }(sc.env.actorSystem.dispatcher) } } 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 70a477a6895cc..50ba0b9d5a612 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 @@ -20,12 +20,12 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer import java.util.concurrent.{Executors, TimeUnit} -import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcCallContext, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.util.Utils private case class ReviveOffers() @@ -71,11 +71,15 @@ private[spark] class LocalEndpoint( case KillTask(taskId, interruptThread) => executor.killTask(taskId, interruptThread) + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case StopExecutor => executor.stop() + context.reply(true) } + def reviveOffers() { val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) val tasks = scheduler.resourceOffers(offers).flatten @@ -104,8 +108,11 @@ private[spark] class LocalEndpoint( * master all run in the same JVM. It sits behind a TaskSchedulerImpl and handles launching tasks * on a single Executor (created by the LocalBackend) running locally. */ -private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) - extends SchedulerBackend with ExecutorBackend { +private[spark] class LocalBackend( + conf: SparkConf, + scheduler: TaskSchedulerImpl, + val totalCores: Int) + extends SchedulerBackend with ExecutorBackend with Logging { private val appId = "local-" + System.currentTimeMillis var localEndpoint: RpcEndpointRef = null @@ -116,7 +123,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: } override def stop() { - localEndpoint.send(StopExecutor) + localEndpoint.sendWithReply(StopExecutor) } override def reviveOffers() { diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 6b3049b28cd5e..22acc270b983e 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -56,19 +56,13 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext wit // Min < 0 val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") intercept[SparkException] { contexts += new SparkContext(conf1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Max < 0 val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") intercept[SparkException] { contexts += new SparkContext(conf2) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Both min and max, but min > max intercept[SparkException] { createSparkContext(2, 1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Both min and max, and min == max val sc1 = createSparkContext(1, 1) From 3ae37b93a7c299bd8b22a36248035bca5de3422f Mon Sep 17 00:00:00 2001 From: Jin Adachi Date: Thu, 16 Apr 2015 23:41:04 +0800 Subject: [PATCH 050/144] [SPARK-6694][SQL]SparkSQL CLI must be able to specify an option --database on the command line. SparkSQL CLI has an option --database as follows. But, the option --database is ignored. ``` $ spark-sql --help : CLI options: : --database Specify the database to use ``` Author: Jin Adachi Author: adachij Closes #5345 from adachij2002/SPARK-6694 and squashes the following commits: 8659084 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 0301eb9 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 df81086 [Jin Adachi] Modify code style. 846f83e [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 dbe8c63 [Jin Adachi] Change file permission to 644. 7b58f42 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 c581d06 [Jin Adachi] Add an option --database test db56122 [Jin Adachi] Merge branch 'SPARK-6694' of https://github.com/adachij2002/spark into SPARK-6694 ee09fa5 [adachij] Merge branch 'master' into SPARK-6694 c804c03 [adachij] SparkSQL CLI must be able to specify an option --database on the command line. --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 3 ++ .../sql/hive/thriftserver/CliSuite.scala | 45 +++++++++++++++---- 2 files changed, 39 insertions(+), 9 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 62c061bef690a..85281c6d73a3b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -145,6 +145,9 @@ private[hive] object SparkSQLCLIDriver { case e: UnsupportedEncodingException => System.exit(3) } + // use the specified database if specified + cli.processSelectDatabase(sessionState); + // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) 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 6d1d7c3a4e698..b070fa8eaa469 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 @@ -25,22 +25,31 @@ import scala.concurrent.{Await, Promise} import scala.sys.process.{Process, ProcessLogger} import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging import org.apache.spark.util.Utils -class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { +class CliSuite extends FunSuite with BeforeAndAfter with Logging { + val warehousePath = Utils.createTempDir() + val metastorePath = Utils.createTempDir() + + before { + warehousePath.delete() + metastorePath.delete() + } + + after { + warehousePath.delete() + metastorePath.delete() + } + def runCliWithin( timeout: FiniteDuration, extraArgs: Seq[String] = Seq.empty)( - queriesAndExpectedAnswers: (String, String)*) { + queriesAndExpectedAnswers: (String, String)*): Unit = { val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip - val warehousePath = Utils.createTempDir() - warehousePath.delete() - val metastorePath = Utils.createTempDir() - metastorePath.delete() val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) val command = { @@ -95,8 +104,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { """.stripMargin, cause) throw cause } finally { - warehousePath.delete() - metastorePath.delete() process.destroy() } } @@ -124,4 +131,24 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { test("Single command with -e") { runCliWithin(1.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK") } + + test("Single command with --database") { + runCliWithin(1.minute)( + "CREATE DATABASE hive_test_db;" + -> "OK", + "USE hive_test_db;" + -> "OK", + "CREATE TABLE hive_test(key INT, val STRING);" + -> "OK", + "SHOW TABLES;" + -> "Time taken: " + ) + + runCliWithin(1.minute, Seq("--database", "hive_test_db", "-e", "SHOW TABLES;"))( + "" + -> "OK", + "" + -> "hive_test" + ) + } } From ef3fb801ae971656ed9cd1b0ab95bc5a1548adbd Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Apr 2015 13:45:55 -0500 Subject: [PATCH 051/144] [SPARK-6934][Core] Use 'spark.akka.askTimeout' for the ask timeout Fixed my mistake in #4588 Author: zsxwing Closes #5529 from zsxwing/SPARK-6934 and squashes the following commits: 9890b2d [zsxwing] Use 'spark.akka.askTimeout' for the ask timeout --- core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index e259867c14040..f2c1c86af767e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -284,7 +284,7 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) private[this] val maxRetries = conf.getInt("spark.akka.num.retries", 3) private[this] val retryWaitMs = conf.getLong("spark.akka.retry.wait", 3000) - private[this] val defaultTimeout = conf.getLong("spark.akka.lookupTimeout", 30) seconds + private[this] val defaultAskTimeout = conf.getLong("spark.akka.askTimeout", 30) seconds /** * return the address for the [[RpcEndpointRef]] @@ -304,7 +304,8 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) * * This method only sends the message once and never retries. */ - def sendWithReply[T: ClassTag](message: Any): Future[T] = sendWithReply(message, defaultTimeout) + def sendWithReply[T: ClassTag](message: Any): Future[T] = + sendWithReply(message, defaultAskTimeout) /** * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a `Future` to @@ -327,7 +328,7 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) * @tparam T type of the reply message * @return the reply message from the corresponding [[RpcEndpoint]] */ - def askWithReply[T: ClassTag](message: Any): T = askWithReply(message, defaultTimeout) + def askWithReply[T: ClassTag](message: Any): T = askWithReply(message, defaultAskTimeout) /** * Send a message to the corresponding [[RpcEndpoint.receive]] and get its result within a From 55f553a979db925aa0c3559f7e80b99d2bf3feb4 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 16 Apr 2015 13:06:34 -0700 Subject: [PATCH 052/144] [SPARK-6855] [SPARKR] Set R includes to get the right collate order. This prevents tools like devtools::document creating invalid collate orders Author: Shivaram Venkataraman Closes #5462 from shivaram/collate-order and squashes the following commits: f3db562 [Shivaram Venkataraman] Set R includes to get the right collate order. This prevents tools like devtools::document creating invalid collate orders --- R/pkg/DESCRIPTION | 6 +++--- R/pkg/R/DataFrame.R | 2 +- R/pkg/R/column.R | 2 +- R/pkg/R/group.R | 3 +++ R/pkg/R/jobj.R | 3 +++ R/pkg/R/pairRDD.R | 2 ++ 6 files changed, 13 insertions(+), 5 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1842b97d43651..052f68c6c24e2 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -17,19 +17,19 @@ License: Apache License (== 2.0) Collate: 'generics.R' 'jobj.R' - 'SQLTypes.R' 'RDD.R' 'pairRDD.R' + 'SQLTypes.R' 'column.R' 'group.R' 'DataFrame.R' 'SQLContext.R' + 'backend.R' 'broadcast.R' + 'client.R' 'context.R' 'deserialize.R' 'serialize.R' 'sparkR.R' - 'backend.R' - 'client.R' 'utils.R' 'zzz.R' diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index feafd56909a67..044fdb4d01223 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -17,7 +17,7 @@ # DataFrame.R - DataFrame class and methods implemented in S4 OO classes -#' @include jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +#' @include generics.R jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R NULL setOldClass("jobj") diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index e196305186b9a..b282001d8b6b5 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -17,7 +17,7 @@ # Column Class -#' @include generics.R jobj.R +#' @include generics.R jobj.R SQLTypes.R NULL setOldClass("jobj") diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 09fc0a7abe48a..855fbdfc7c4ca 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -17,6 +17,9 @@ # group.R - GroupedData class and methods implemented in S4 OO classes +#' @include generics.R jobj.R SQLTypes.R column.R +NULL + setOldClass("jobj") #' @title S4 class that represents a GroupedData diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R index 4180f146b7fbc..a8a25230b636d 100644 --- a/R/pkg/R/jobj.R +++ b/R/pkg/R/jobj.R @@ -18,6 +18,9 @@ # References to objects that exist on the JVM backend # are maintained using the jobj. +#' @include generics.R +NULL + # Maintain a reference count of Java object references # This allows us to GC the java object when it is safe .validJobjs <- new.env(parent = emptyenv()) diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 739d399f0820f..5d64822859d1f 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -16,6 +16,8 @@ # # Operations supported on RDDs contains pairs (i.e key, value) +#' @include generics.R jobj.R RDD.R +NULL ############ Actions and Transformations ############ From 04e44b37cc04f62fbf9e08c7076349e0a4d12ea8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Apr 2015 16:20:57 -0700 Subject: [PATCH 053/144] [SPARK-4897] [PySpark] Python 3 support This PR update PySpark to support Python 3 (tested with 3.4). Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped. TODO: ec2/spark-ec2.py is not fully tested with python3. Author: Davies Liu Author: twneale Author: Josh Rosen Closes #5173 from davies/python3 and squashes the following commits: d7d6323 [Davies Liu] fix tests 6c52a98 [Davies Liu] fix mllib test 99e334f [Davies Liu] update timeout b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 cafd5ec [Davies Liu] adddress comments from @mengxr bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 179fc8d [Davies Liu] tuning flaky tests 8c8b957 [Davies Liu] fix ResourceWarning in Python 3 5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 4006829 [Davies Liu] fix test 2fc0066 [Davies Liu] add python3 path 71535e9 [Davies Liu] fix xrange and divide 5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ed498c8 [Davies Liu] fix compatibility with python 3 820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ad7c374 [Davies Liu] fix mllib test and warning ef1fc2f [Davies Liu] fix tests 4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 59bb492 [Davies Liu] fix tests 1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ca0fdd3 [Davies Liu] fix code style 9563a15 [Davies Liu] add imap back for python 2 0b1ec04 [Davies Liu] make python examples work with Python 3 d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 a716d34 [Davies Liu] test with python 3.4 f1700e8 [Davies Liu] fix test in python3 671b1db [Davies Liu] fix test in python3 692ff47 [Davies Liu] fix flaky test 7b9699f [Davies Liu] invalidate import cache for Python 3.3+ 9c58497 [Davies Liu] fix kill worker 309bfbf [Davies Liu] keep compatibility 5707476 [Davies Liu] cleanup, fix hash of string in 3.3+ 8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 f53e1f0 [Davies Liu] fix tests 70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3 a39167e [Davies Liu] support customize class in __main__ 814c77b [Davies Liu] run unittests with python 3 7f4476e [Davies Liu] mllib tests passed d737924 [Davies Liu] pass ml tests 375ea17 [Davies Liu] SQL tests pass 6cc42a9 [Davies Liu] rename 431a8de [Davies Liu] streaming tests pass 78901a7 [Davies Liu] fix hash of serializer in Python 3 24b2f2e [Davies Liu] pass all RDD tests 35f48fe [Davies Liu] run future again 1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py 6e3c21d [Davies Liu] make cloudpickle work with Python3 2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run 1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out 7354371 [twneale] buffer --> memoryview I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work. b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?). f40d925 [twneale] xrange --> range e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206 79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper 2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3 854be27 [Josh Rosen] Run `futurize` on Python code: 7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py. --- bin/pyspark | 1 + bin/spark-submit | 3 + bin/spark-submit2.cmd | 3 + dev/run-tests | 2 + dev/run-tests-jenkins | 2 +- ec2/spark_ec2.py | 262 ++++---- examples/src/main/python/als.py | 15 +- examples/src/main/python/avro_inputformat.py | 9 +- .../src/main/python/cassandra_inputformat.py | 8 +- .../src/main/python/cassandra_outputformat.py | 6 +- examples/src/main/python/hbase_inputformat.py | 8 +- .../src/main/python/hbase_outputformat.py | 6 +- examples/src/main/python/kmeans.py | 11 +- .../src/main/python/logistic_regression.py | 20 +- .../ml/simple_text_classification_pipeline.py | 20 +- .../src/main/python/mllib/correlations.py | 19 +- .../src/main/python/mllib/dataset_example.py | 13 +- .../main/python/mllib/decision_tree_runner.py | 29 +- .../python/mllib/gaussian_mixture_model.py | 9 +- .../python/mllib/gradient_boosted_trees.py | 7 +- examples/src/main/python/mllib/kmeans.py | 5 +- .../main/python/mllib/logistic_regression.py | 9 +- .../python/mllib/random_forest_example.py | 9 +- .../python/mllib/random_rdd_generation.py | 21 +- .../src/main/python/mllib/sampled_rdds.py | 29 +- examples/src/main/python/mllib/word2vec.py | 5 +- examples/src/main/python/pagerank.py | 16 +- .../src/main/python/parquet_inputformat.py | 7 +- examples/src/main/python/pi.py | 5 +- examples/src/main/python/sort.py | 6 +- examples/src/main/python/sql.py | 4 +- examples/src/main/python/status_api_demo.py | 10 +- .../main/python/streaming/hdfs_wordcount.py | 3 +- .../main/python/streaming/kafka_wordcount.py | 3 +- .../python/streaming/network_wordcount.py | 3 +- .../recoverable_network_wordcount.py | 11 +- .../python/streaming/sql_network_wordcount.py | 5 +- .../streaming/stateful_network_wordcount.py | 3 +- .../src/main/python/transitive_closure.py | 10 +- examples/src/main/python/wordcount.py | 6 +- .../MatrixFactorizationModelWrapper.scala | 9 +- .../mllib/api/python/PythonMLLibAPI.scala | 39 +- python/pyspark/accumulators.py | 9 +- python/pyspark/broadcast.py | 37 +- python/pyspark/cloudpickle.py | 577 +++++------------- python/pyspark/conf.py | 9 +- python/pyspark/context.py | 42 +- python/pyspark/daemon.py | 36 +- python/pyspark/heapq3.py | 24 +- python/pyspark/java_gateway.py | 2 +- python/pyspark/join.py | 1 + python/pyspark/ml/classification.py | 4 +- python/pyspark/ml/feature.py | 22 +- python/pyspark/ml/param/__init__.py | 8 +- .../ml/param/_shared_params_code_gen.py | 10 +- python/pyspark/mllib/__init__.py | 11 +- python/pyspark/mllib/classification.py | 7 +- python/pyspark/mllib/clustering.py | 18 +- python/pyspark/mllib/common.py | 19 +- python/pyspark/mllib/feature.py | 18 +- python/pyspark/mllib/fpm.py | 2 + python/pyspark/mllib/linalg.py | 48 +- python/pyspark/mllib/rand.py | 33 +- python/pyspark/mllib/recommendation.py | 7 +- python/pyspark/mllib/stat/_statistics.py | 25 +- python/pyspark/mllib/tests.py | 20 +- python/pyspark/mllib/tree.py | 15 +- python/pyspark/mllib/util.py | 26 +- python/pyspark/profiler.py | 10 +- python/pyspark/rdd.py | 189 +++--- python/pyspark/rddsampler.py | 4 +- python/pyspark/serializers.py | 101 ++- python/pyspark/shell.py | 16 +- python/pyspark/shuffle.py | 126 ++-- python/pyspark/sql/__init__.py | 15 +- python/pyspark/sql/{types.py => _types.py} | 49 +- python/pyspark/sql/context.py | 32 +- python/pyspark/sql/dataframe.py | 63 +- python/pyspark/sql/functions.py | 6 +- python/pyspark/sql/tests.py | 11 +- python/pyspark/statcounter.py | 4 +- python/pyspark/streaming/context.py | 5 +- python/pyspark/streaming/dstream.py | 51 +- python/pyspark/streaming/kafka.py | 8 +- python/pyspark/streaming/tests.py | 39 +- python/pyspark/streaming/util.py | 6 +- python/pyspark/tests.py | 327 +++++----- python/pyspark/worker.py | 16 +- python/run-tests | 15 +- python/test_support/userlib-0.1-py2.7.egg | Bin 1945 -> 0 bytes python/test_support/userlib-0.1.zip | Bin 0 -> 668 bytes 91 files changed, 1398 insertions(+), 1396 deletions(-) rename python/pyspark/sql/{types.py => _types.py} (97%) delete mode 100644 python/test_support/userlib-0.1-py2.7.egg create mode 100644 python/test_support/userlib-0.1.zip diff --git a/bin/pyspark b/bin/pyspark index 776b28dc41099..8acad6113797d 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -89,6 +89,7 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR + export PYTHONHASHSEED=0 if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else diff --git a/bin/spark-submit b/bin/spark-submit index bcff78edd51ca..0e0afe71a0f05 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,6 +19,9 @@ SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +# disable randomized hash for string in Python 3.3+ +export PYTHONHASHSEED=0 + # Only define a usage function if an upstream script hasn't done so. if ! type -t usage >/dev/null 2>&1; then usage() { diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index 08ddb185742d2..d3fc4a5cc3f6e 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -20,6 +20,9 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. +rem disable randomized hash for string in Python 3.3+ +set PYTHONHASHSEED=0 + set CLASS=org.apache.spark.deploy.SparkSubmit call %~dp0spark-class2.cmd %CLASS% %* set SPARK_ERROR_LEVEL=%ERRORLEVEL% diff --git a/dev/run-tests b/dev/run-tests index bb21ab6c9aa04..861d1671182c2 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -235,6 +235,8 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS +# add path for python 3 in jenkins +export PATH="${PATH}:/home/anaonda/envs/py3k/bin" ./python/run-tests echo "" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 3c1c91a111357..030f2cdddb350 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -47,7 +47,7 @@ COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" -TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout +TESTS_TIMEOUT="150m" # format: http://linux.die.net/man/1/timeout # Array to capture all tests to run on the pull request. These tests are held under the #+ dev/tests/ directory. diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0c1f24761d0de..87c0818279713 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -19,7 +19,7 @@ # limitations under the License. # -from __future__ import with_statement +from __future__ import with_statement, print_function import hashlib import itertools @@ -37,12 +37,17 @@ import tempfile import textwrap import time -import urllib2 import warnings from datetime import datetime from optparse import OptionParser from sys import stderr +if sys.version < "3": + from urllib2 import urlopen, Request, HTTPError +else: + from urllib.request import urlopen, Request + from urllib.error import HTTPError + SPARK_EC2_VERSION = "1.2.1" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -88,10 +93,10 @@ def setup_external_libs(libs): SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib") if not os.path.exists(SPARK_EC2_LIB_DIR): - print "Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( + print("Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( path=SPARK_EC2_LIB_DIR - ) - print "This should be a one-time operation." + )) + print("This should be a one-time operation.") os.mkdir(SPARK_EC2_LIB_DIR) for lib in libs: @@ -100,8 +105,8 @@ def setup_external_libs(libs): if not os.path.isdir(lib_dir): tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz") - print " - Downloading {lib}...".format(lib=lib["name"]) - download_stream = urllib2.urlopen( + print(" - Downloading {lib}...".format(lib=lib["name"])) + download_stream = urlopen( "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format( prefix=PYPI_URL_PREFIX, first_letter=lib["name"][:1], @@ -113,13 +118,13 @@ def setup_external_libs(libs): tgz_file.write(download_stream.read()) with open(tgz_file_path) as tar: if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: - print >> stderr, "ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]) + print("ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]), file=stderr) sys.exit(1) tar = tarfile.open(tgz_file_path) tar.extractall(path=SPARK_EC2_LIB_DIR) tar.close() os.remove(tgz_file_path) - print " - Finished downloading {lib}.".format(lib=lib["name"]) + print(" - Finished downloading {lib}.".format(lib=lib["name"])) sys.path.insert(1, lib_dir) @@ -299,12 +304,12 @@ def parse_args(): if home_dir is None or not os.path.isfile(home_dir + '/.boto'): if not os.path.isfile('/etc/boto.cfg'): if os.getenv('AWS_ACCESS_KEY_ID') is None: - print >> stderr, ("ERROR: The environment variable AWS_ACCESS_KEY_ID " + - "must be set") + print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", + file=stderr) sys.exit(1) if os.getenv('AWS_SECRET_ACCESS_KEY') is None: - print >> stderr, ("ERROR: The environment variable AWS_SECRET_ACCESS_KEY " + - "must be set") + print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", + file=stderr) sys.exit(1) return (opts, action, cluster_name) @@ -316,7 +321,7 @@ def get_or_make_group(conn, name, vpc_id): if len(group) > 0: return group[0] else: - print "Creating security group " + name + print("Creating security group " + name) return conn.create_security_group(name, "Spark EC2 group", vpc_id) @@ -324,18 +329,19 @@ 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) + print("Don't know about Spark version: {v}".format(v=version), file=stderr) 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 = 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) + response = urlopen(request) + except HTTPError as e: + print("Couldn't validate Spark commit: {url}".format(url=github_commit_url), + file=stderr) + print("Received HTTP response code of {code}.".format(code=e.code), file=stderr) sys.exit(1) return version @@ -394,8 +400,7 @@ def get_spark_ami(opts): instance_type = EC2_INSTANCE_TYPES[opts.instance_type] else: instance_type = "pvm" - print >> stderr,\ - "Don't recognize %s, assuming type is pvm" % opts.instance_type + print("Don't recognize %s, assuming type is pvm" % opts.instance_type, file=stderr) # URL prefix from which to fetch AMI information ami_prefix = "{r}/{b}/ami-list".format( @@ -404,10 +409,10 @@ def get_spark_ami(opts): ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type) try: - ami = urllib2.urlopen(ami_path).read().strip() - print "Spark AMI: " + ami + ami = urlopen(ami_path).read().strip() + print("Spark AMI: " + ami) except: - print >> stderr, "Could not resolve AMI at: " + ami_path + print("Could not resolve AMI at: " + ami_path, file=stderr) sys.exit(1) return ami @@ -419,11 +424,11 @@ def get_spark_ami(opts): # Fails if there already instances running in the cluster's groups. def launch_cluster(conn, opts, cluster_name): if opts.identity_file is None: - print >> stderr, "ERROR: Must provide an identity file (-i) for ssh connections." + print("ERROR: Must provide an identity file (-i) for ssh connections.", file=stderr) sys.exit(1) if opts.key_pair is None: - print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." + print("ERROR: Must provide a key pair name (-k) to use on instances.", file=stderr) sys.exit(1) user_data_content = None @@ -431,7 +436,7 @@ def launch_cluster(conn, opts, cluster_name): with open(opts.user_data) as user_data_file: user_data_content = user_data_file.read() - print "Setting up security groups..." + print("Setting up security groups...") 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 @@ -497,8 +502,8 @@ def launch_cluster(conn, opts, cluster_name): existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if existing_slaves or (existing_masters and not opts.use_existing_master): - print >> stderr, ("ERROR: There are already instances running in " + - "group %s or %s" % (master_group.name, slave_group.name)) + print("ERROR: There are already instances running in group %s or %s" % + (master_group.name, slave_group.name), file=stderr) sys.exit(1) # Figure out Spark AMI @@ -511,12 +516,12 @@ def launch_cluster(conn, opts, cluster_name): 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..." + print("Launching instances...") try: image = conn.get_all_images(image_ids=[opts.ami])[0] except: - print >> stderr, "Could not find AMI " + opts.ami + print("Could not find AMI " + opts.ami, file=stderr) sys.exit(1) # Create block device mapping so that we can add EBS volumes if asked to. @@ -542,8 +547,8 @@ def launch_cluster(conn, opts, cluster_name): # Launch slaves if opts.spot_price is not None: # Launch spot instances with the requested price - print ("Requesting %d slaves as spot instances with price $%.3f" % - (opts.slaves, opts.spot_price)) + print("Requesting %d slaves as spot instances with price $%.3f" % + (opts.slaves, opts.spot_price)) zones = get_zones(conn, opts) num_zones = len(zones) i = 0 @@ -566,7 +571,7 @@ def launch_cluster(conn, opts, cluster_name): my_req_ids += [req.id for req in slave_reqs] i += 1 - print "Waiting for spot instances to be granted..." + print("Waiting for spot instances to be granted...") try: while True: time.sleep(10) @@ -579,24 +584,24 @@ def launch_cluster(conn, opts, cluster_name): if i in id_to_req and id_to_req[i].state == "active": active_instance_ids.append(id_to_req[i].instance_id) if len(active_instance_ids) == opts.slaves: - print "All %d slaves granted" % opts.slaves + print("All %d slaves granted" % opts.slaves) reservations = conn.get_all_reservations(active_instance_ids) slave_nodes = [] for r in reservations: slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves) + print("%d of %d slaves granted, waiting longer" % ( + len(active_instance_ids), opts.slaves)) except: - print "Canceling spot instance requests" + print("Canceling spot instance requests") conn.cancel_spot_instance_requests(my_req_ids) # Log a warning if any of these requests actually launched instances: (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) running = len(master_nodes) + len(slave_nodes) if running: - print >> stderr, ("WARNING: %d instances are still running" % running) + print(("WARNING: %d instances are still running" % running), file=stderr) sys.exit(0) else: # Launch non-spot instances @@ -618,16 +623,16 @@ def launch_cluster(conn, opts, cluster_name): placement_group=opts.placement_group, user_data=user_data_content) slave_nodes += slave_res.instances - print "Launched {s} slave{plural_s} in {z}, regid = {r}".format( - s=num_slaves_this_zone, - plural_s=('' if num_slaves_this_zone == 1 else 's'), - z=zone, - r=slave_res.id) + print("Launched {s} slave{plural_s} in {z}, regid = {r}".format( + s=num_slaves_this_zone, + plural_s=('' if num_slaves_this_zone == 1 else 's'), + z=zone, + r=slave_res.id)) i += 1 # Launch or resume masters if existing_masters: - print "Starting master..." + print("Starting master...") for inst in existing_masters: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -650,10 +655,10 @@ def launch_cluster(conn, opts, cluster_name): user_data=user_data_content) master_nodes = master_res.instances - print "Launched master in %s, regid = %s" % (zone, master_res.id) + print("Launched master in %s, regid = %s" % (zone, master_res.id)) # This wait time corresponds to SPARK-4983 - print "Waiting for AWS to propagate instance metadata..." + print("Waiting for AWS to propagate instance metadata...") time.sleep(5) # Give the instances descriptive names for master in master_nodes: @@ -674,8 +679,8 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): Get the EC2 instances in an existing cluster if available. Returns a tuple of lists of EC2 instance objects for the masters and slaves. """ - print "Searching for existing cluster {c} in region {r}...".format( - c=cluster_name, r=opts.region) + print("Searching for existing cluster {c} in region {r}...".format( + c=cluster_name, r=opts.region)) def get_instances(group_names): """ @@ -693,16 +698,15 @@ def get_instances(group_names): slave_instances = get_instances([cluster_name + "-slaves"]) if any((master_instances, slave_instances)): - print "Found {m} master{plural_m}, {s} slave{plural_s}.".format( - m=len(master_instances), - plural_m=('' if len(master_instances) == 1 else 's'), - s=len(slave_instances), - plural_s=('' if len(slave_instances) == 1 else 's')) + print("Found {m} master{plural_m}, {s} slave{plural_s}.".format( + m=len(master_instances), + plural_m=('' if len(master_instances) == 1 else 's'), + s=len(slave_instances), + plural_s=('' if len(slave_instances) == 1 else 's'))) if not master_instances and die_on_error: - print >> sys.stderr, \ - "ERROR: Could not find a master for cluster {c} in region {r}.".format( - c=cluster_name, r=opts.region) + print("ERROR: Could not find a master for cluster {c} in region {r}.".format( + c=cluster_name, r=opts.region), file=sys.stderr) sys.exit(1) return (master_instances, slave_instances) @@ -713,7 +717,7 @@ def get_instances(group_names): def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = get_dns_name(master_nodes[0], opts.private_ips) if deploy_ssh_key: - print "Generating cluster's SSH key on master..." + print("Generating cluster's SSH key on master...") key_setup = """ [ -f ~/.ssh/id_rsa ] || (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa && @@ -721,10 +725,10 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): """ ssh(master, opts, key_setup) dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) - print "Transferring cluster's SSH key to slaves..." + print("Transferring cluster's SSH key to slaves...") for slave in slave_nodes: slave_address = get_dns_name(slave, opts.private_ips) - print slave_address + print(slave_address) ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', @@ -738,8 +742,8 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - print "Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( - r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch) + print("Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( + r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch)) ssh( host=master, opts=opts, @@ -749,7 +753,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): b=opts.spark_ec2_git_branch) ) - print "Deploying files to master..." + print("Deploying files to master...") deploy_files( conn=conn, root_dir=SPARK_EC2_DIR + "/" + "deploy.generic", @@ -760,25 +764,25 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): ) if opts.deploy_root_dir is not None: - print "Deploying {s} to master...".format(s=opts.deploy_root_dir) + print("Deploying {s} to master...".format(s=opts.deploy_root_dir)) deploy_user_files( root_dir=opts.deploy_root_dir, opts=opts, master_nodes=master_nodes ) - print "Running setup on master..." + print("Running setup on master...") setup_spark_cluster(master, opts) - print "Done!" + print("Done!") def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") - print "Spark standalone cluster started at http://%s:8080" % master + print("Spark standalone cluster started at http://%s:8080" % master) if opts.ganglia: - print "Ganglia started at http://%s:5080/ganglia" % master + print("Ganglia started at http://%s:5080/ganglia" % master) def is_ssh_available(host, opts, print_ssh_output=True): @@ -795,7 +799,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): if s.returncode != 0 and print_ssh_output: # extra leading newline is for spacing in wait_for_cluster_state() - print textwrap.dedent("""\n + print(textwrap.dedent("""\n Warning: SSH connection error. (This could be temporary.) Host: {h} SSH return code: {r} @@ -804,7 +808,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): h=host, r=s.returncode, o=cmd_output.strip() - ) + )) return s.returncode == 0 @@ -865,10 +869,10 @@ def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): sys.stdout.write("\n") end_time = datetime.now() - print "Cluster is now in '{s}' state. Waited {t} seconds.".format( + print("Cluster is now in '{s}' state. Waited {t} seconds.".format( s=cluster_state, t=(end_time - start_time).seconds - ) + )) # Get number of local disks available for a given EC2 instance type. @@ -916,8 +920,8 @@ def get_num_disks(instance_type): if instance_type in disks_by_instance: return disks_by_instance[instance_type] else: - print >> stderr, ("WARNING: Don't know number of disks on instance type %s; assuming 1" - % instance_type) + print("WARNING: Don't know number of disks on instance type %s; assuming 1" + % instance_type, file=stderr) return 1 @@ -951,7 +955,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) tachyon_v = "" - print "Deploying Spark via git hash; Tachyon won't be set up" + print("Deploying Spark via git hash; Tachyon won't be set up") modules = filter(lambda x: x != "tachyon", modules) master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] @@ -1067,8 +1071,8 @@ def ssh(host, opts, command): "--key-pair parameters and try again.".format(host)) else: raise e - print >> stderr, \ - "Error executing remote command, retrying after 30 seconds: {0}".format(e) + print("Error executing remote command, retrying after 30 seconds: {0}".format(e), + file=stderr) time.sleep(30) tries = tries + 1 @@ -1107,8 +1111,8 @@ def ssh_write(host, opts, command, arguments): elif tries > 5: raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: - print >> stderr, \ - "Error {0} while executing remote command, retrying after 30 seconds".format(status) + print("Error {0} while executing remote command, retrying after 30 seconds". + format(status), file=stderr) time.sleep(30) tries = tries + 1 @@ -1162,42 +1166,41 @@ def real_main(): if opts.identity_file is not None: if not os.path.exists(opts.identity_file): - print >> stderr,\ - "ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file) + print("ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file), + file=stderr) sys.exit(1) file_mode = os.stat(opts.identity_file).st_mode if not (file_mode & S_IRUSR) or not oct(file_mode)[-2:] == '00': - print >> stderr, "ERROR: The identity file must be accessible only by you." - print >> stderr, 'You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file) + print("ERROR: The identity file must be accessible only by you.", file=stderr) + print('You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file), + file=stderr) sys.exit(1) if opts.instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, "Warning: Unrecognized EC2 instance type for instance-type: {t}".format( - t=opts.instance_type) + print("Warning: Unrecognized EC2 instance type for instance-type: {t}".format( + t=opts.instance_type), file=stderr) if opts.master_instance_type != "": if opts.master_instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, \ - "Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( - t=opts.master_instance_type) + print("Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( + t=opts.master_instance_type), file=stderr) # Since we try instance types even if we can't resolve them, we check if they resolve first # and, if they do, see if they resolve to the same virtualization type. if opts.instance_type in EC2_INSTANCE_TYPES and \ opts.master_instance_type in EC2_INSTANCE_TYPES: if EC2_INSTANCE_TYPES[opts.instance_type] != \ EC2_INSTANCE_TYPES[opts.master_instance_type]: - print >> stderr, \ - "Error: spark-ec2 currently does not support having a master and slaves " + \ - "with different AMI virtualization types." - print >> stderr, "master instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.master_instance_type]) - print >> stderr, "slave instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.instance_type]) + print("Error: spark-ec2 currently does not support having a master and slaves " + "with different AMI virtualization types.", file=stderr) + print("master instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.master_instance_type]), file=stderr) + print("slave instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.instance_type]), file=stderr) sys.exit(1) if opts.ebs_vol_num > 8: - print >> stderr, "ebs-vol-num cannot be greater than 8" + print("ebs-vol-num cannot be greater than 8", file=stderr) sys.exit(1) # Prevent breaking ami_prefix (/, .git and startswith checks) @@ -1206,23 +1209,22 @@ def real_main(): opts.spark_ec2_git_repo.endswith(".git") or \ not opts.spark_ec2_git_repo.startswith("https://github.com") or \ not opts.spark_ec2_git_repo.endswith("spark-ec2"): - print >> stderr, "spark-ec2-git-repo must be a github repo and it must not have a " \ - "trailing / or .git. " \ - "Furthermore, we currently only support forks named spark-ec2." + print("spark-ec2-git-repo must be a github repo and it must not have a trailing / or .git. " + "Furthermore, we currently only support forks named spark-ec2.", file=stderr) sys.exit(1) if not (opts.deploy_root_dir is None or (os.path.isabs(opts.deploy_root_dir) and os.path.isdir(opts.deploy_root_dir) and os.path.exists(opts.deploy_root_dir))): - print >> stderr, "--deploy-root-dir must be an absolute path to a directory that exists " \ - "on the local file system" + print("--deploy-root-dir must be an absolute path to a directory that exists " + "on the local file system", file=stderr) sys.exit(1) try: conn = ec2.connect_to_region(opts.region) except Exception as e: - print >> stderr, (e) + print((e), file=stderr) sys.exit(1) # Select an AZ at random if it was not specified. @@ -1231,7 +1233,7 @@ def real_main(): if action == "launch": if opts.slaves <= 0: - print >> sys.stderr, "ERROR: You have to start at least 1 slave" + print("ERROR: You have to start at least 1 slave", file=sys.stderr) sys.exit(1) if opts.resume: (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) @@ -1250,18 +1252,18 @@ def real_main(): conn, opts, cluster_name, die_on_error=False) if any(master_nodes + slave_nodes): - print "The following instances will be terminated:" + print("The following instances will be terminated:") for inst in master_nodes + slave_nodes: - print "> %s" % get_dns_name(inst, opts.private_ips) - print "ALL DATA ON ALL NODES WILL BE LOST!!" + print("> %s" % get_dns_name(inst, opts.private_ips)) + print("ALL DATA ON ALL NODES WILL BE LOST!!") msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) response = raw_input(msg) if response == "y": - print "Terminating master..." + print("Terminating master...") for inst in master_nodes: inst.terminate() - print "Terminating slaves..." + print("Terminating slaves...") for inst in slave_nodes: inst.terminate() @@ -1274,16 +1276,16 @@ def real_main(): cluster_instances=(master_nodes + slave_nodes), cluster_state='terminated' ) - print "Deleting security groups (this will take some time)..." + print("Deleting security groups (this will take some time)...") attempt = 1 while attempt <= 3: - print "Attempt %d" % attempt + print("Attempt %d" % attempt) groups = [g for g in conn.get_all_security_groups() if g.name in group_names] success = True # Delete individual rules in all groups before deleting groups to # remove dependencies between them for group in groups: - print "Deleting rules in security group " + group.name + print("Deleting rules in security group " + group.name) for rule in group.rules: for grant in rule.grants: success &= group.revoke(ip_protocol=rule.ip_protocol, @@ -1298,10 +1300,10 @@ def real_main(): try: # It is needed to use group_id to make it work with VPC conn.delete_security_group(group_id=group.id) - print "Deleted security group %s" % group.name + print("Deleted security group %s" % group.name) except boto.exception.EC2ResponseError: success = False - print "Failed to delete security group %s" % group.name + print("Failed to delete security group %s" % group.name) # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails @@ -1311,17 +1313,16 @@ def real_main(): attempt += 1 if not success: - print "Failed to delete all security groups after 3 tries." - print "Try re-running in a few minutes." + print("Failed to delete all security groups after 3 tries.") + print("Try re-running in a few minutes.") elif action == "login": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) if not master_nodes[0].public_dns_name and not opts.private_ips: - print "Master has no public DNS name. Maybe you meant to specify " \ - "--private-ips?" + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") else: master = get_dns_name(master_nodes[0], opts.private_ips) - print "Logging into master " + master + "..." + print("Logging into master " + master + "...") proxy_opt = [] if opts.proxy_port is not None: proxy_opt = ['-D', opts.proxy_port] @@ -1336,19 +1337,18 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Rebooting slaves..." + print("Rebooting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: - print "Rebooting " + inst.id + print("Rebooting " + inst.id) inst.reboot() elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) if not master_nodes[0].public_dns_name and not opts.private_ips: - print "Master has no public DNS name. Maybe you meant to specify " \ - "--private-ips?" + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") else: - print get_dns_name(master_nodes[0], opts.private_ips) + print(get_dns_name(master_nodes[0], opts.private_ips)) elif action == "stop": response = raw_input( @@ -1361,11 +1361,11 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Stopping master..." + print("Stopping master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.stop() - print "Stopping slaves..." + print("Stopping slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: if inst.spot_instance_request_id: @@ -1375,11 +1375,11 @@ def real_main(): elif action == "start": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print "Starting slaves..." + print("Starting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() - print "Starting master..." + print("Starting master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -1403,15 +1403,15 @@ def real_main(): setup_cluster(conn, master_nodes, slave_nodes, opts, False) else: - print >> stderr, "Invalid action: %s" % action + print("Invalid action: %s" % action, file=stderr) sys.exit(1) def main(): try: real_main() - except UsageError, e: - print >> stderr, "\nError:\n", e + except UsageError as e: + print("\nError:\n", e, file=stderr) sys.exit(1) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 70b6146e39a87..1c3a787bd0e94 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -21,7 +21,8 @@ This example requires numpy (http://www.numpy.org/) """ -from os.path import realpath +from __future__ import print_function + import sys import numpy as np @@ -57,9 +58,9 @@ def update(i, vec, mat, ratings): Usage: als [M] [U] [F] [iterations] [partitions]" """ - print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an + print("""WARN: This is a naive implementation of ALS and is given as an example. Please use the ALS method found in pyspark.mllib.recommendation for more - conventional use.""" + conventional use.""", file=sys.stderr) sc = SparkContext(appName="PythonALS") M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 @@ -68,8 +69,8 @@ def update(i, vec, mat, ratings): ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2 - print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \ - (M, U, F, ITERATIONS, partitions) + print("Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % + (M, U, F, ITERATIONS, partitions)) R = matrix(rand(M, F)) * matrix(rand(U, F).T) ms = matrix(rand(M, F)) @@ -95,7 +96,7 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) error = rmse(R, ms, us) - print "Iteration %d:" % i - print "\nRMSE: %5.4f\n" % error + print("Iteration %d:" % i) + print("\nRMSE: %5.4f\n" % error) sc.stop() diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index 4626bbb7e3b02..da368ac628a49 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -15,9 +15,12 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext +from functools import reduce """ Read data file users.avro in local Spark distro: @@ -49,7 +52,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 2 and len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: avro_inputformat [reader_schema_file] Run with example jar: @@ -57,7 +60,7 @@ /path/to/examples/avro_inputformat.py [reader_schema_file] Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -77,6 +80,6 @@ conf=conf) output = avro_rdd.map(lambda x: x[0]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index 05f34b74df45a..93ca0cfcc9302 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, """ + print(""" Usage: cassandra_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ /path/to/examples/cassandra_inputformat.py Assumes you have some data in Cassandra already, running on , in and - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -77,6 +79,6 @@ conf=conf) output = cass_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index d144539e58b8f..5d643eac92f94 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -46,7 +48,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: cassandra_outputformat Run with example jar: @@ -60,7 +62,7 @@ ... fname text, ... lname text ... ); - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index 3b16010f1cb97..e17819d5feb76 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: hbase_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ /path/to/examples/hbase_inputformat.py
    Assumes you have some data in HBase already, running on , in
    - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -74,6 +76,6 @@ conf=conf) output = hbase_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index abb425b1f886a..9e5641789a976 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -40,7 +42,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: hbase_outputformat
    Run with example jar: @@ -48,7 +50,7 @@ /path/to/examples/hbase_outputformat.py Assumes you have created
    with column family in HBase running on already - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 86ef6f32c84e8..19391506463f0 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -22,6 +22,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -47,12 +48,12 @@ def closestPoint(p, centers): if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of KMeans Clustering and is given + print("""WARN: This is a naive implementation of KMeans Clustering and is given as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on - how to use MLlib's KMeans implementation.""" + how to use MLlib's KMeans implementation.""", file=sys.stderr) sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) @@ -69,13 +70,13 @@ def closestPoint(p, centers): pointStats = closest.reduceByKey( lambda (x1, y1), (x2, y2): (x1 + x2, y1 + y2)) newPoints = pointStats.map( - lambda (x, (y, z)): (x, y / z)).collect() + lambda xy: (xy[0], xy[1][0] / xy[1][1])).collect() tempDist = sum(np.sum((kPoints[x] - y) ** 2) for (x, y) in newPoints) for (x, y) in newPoints: kPoints[x] = y - print "Final centers: " + str(kPoints) + print("Final centers: " + str(kPoints)) sc.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 3aa56b0528168..b318b7d87bfdc 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -22,10 +22,8 @@ In practice, one may prefer to use the LogisticRegression algorithm in MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. """ +from __future__ import print_function -from collections import namedtuple -from math import exp -from os.path import realpath import sys import numpy as np @@ -42,19 +40,19 @@ def readPointBatch(iterator): strs = list(iterator) matrix = np.zeros((len(strs), D + 1)) - for i in xrange(len(strs)): - matrix[i] = np.fromstring(strs[i].replace(',', ' '), dtype=np.float32, sep=' ') + for i, s in enumerate(strs): + matrix[i] = np.fromstring(s.replace(',', ' '), dtype=np.float32, sep=' ') return [matrix] if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of Logistic Regression and is + print("""WARN: This is a naive implementation of Logistic Regression and is given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py - to see how MLlib's implementation is used.""" + to see how MLlib's implementation is used.""", file=sys.stderr) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() @@ -62,7 +60,7 @@ def readPointBatch(iterator): # Initialize w to a random value w = 2 * np.random.ranf(size=D) - 1 - print "Initial w: " + str(w) + print("Initial w: " + str(w)) # Compute logistic regression gradient for a matrix of data points def gradient(matrix, w): @@ -76,9 +74,9 @@ def add(x, y): return x for i in range(iterations): - print "On iteration %i" % (i + 1) + print("On iteration %i" % (i + 1)) w -= points.map(lambda m: gradient(m, w)).reduce(add) - print "Final w: " + str(w) + print("Final w: " + str(w)) sc.stop() diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index c73edb7fd6b20..fab21f003b233 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression @@ -37,10 +39,10 @@ # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0L, "a b c d e spark", 1.0), - (1L, "b d", 0.0), - (2L, "spark f g h", 1.0), - (3L, "hadoop mapreduce", 0.0)]) \ + training = sc.parallelize([(0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0)]) \ .map(lambda x: LabeledDocument(*x)).toDF() # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. @@ -54,16 +56,16 @@ # Prepare test documents, which are unlabeled. Document = Row("id", "text") - test = sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ + test = sc.parallelize([(4, "spark i j k"), + (5, "l m n"), + (6, "mapreduce spark"), + (7, "apache hadoop")]) \ .map(lambda x: Document(*x)).toDF() # Make predictions on test documents and print columns of interest. prediction = model.transform(test) selected = prediction.select("id", "text", "prediction") for row in selected.collect(): - print row + print(row) sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py index 4218eca822a99..0e13546b88e67 100755 --- a/examples/src/main/python/mllib/correlations.py +++ b/examples/src/main/python/mllib/correlations.py @@ -18,6 +18,7 @@ """ Correlations using MLlib. """ +from __future__ import print_function import sys @@ -29,7 +30,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: correlations ()" + print("Usage: correlations ()", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonCorrelations") if len(sys.argv) == 2: @@ -41,20 +42,20 @@ points = MLUtils.loadLibSVMFile(sc, filepath)\ .map(lambda lp: LabeledPoint(lp.label, lp.features.toArray())) - print - print 'Summary of data file: ' + filepath - print '%d data points' % points.count() + print() + print('Summary of data file: ' + filepath) + print('%d data points' % points.count()) # Statistics (correlations) - print - print 'Correlation (%s) between label and each feature' % corrType - print 'Feature\tCorrelation' + print() + print('Correlation (%s) between label and each feature' % corrType) + print('Feature\tCorrelation') numFeatures = points.take(1)[0].features.size labelRDD = points.map(lambda lp: lp.label) for i in range(numFeatures): featureRDD = points.map(lambda lp: lp.features[i]) corr = Statistics.corr(labelRDD, featureRDD, corrType) - print '%d\t%g' % (i, corr) - print + print('%d\t%g' % (i, corr)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py index fcbf56cbf0c52..e23ecc0c5d302 100644 --- a/examples/src/main/python/mllib/dataset_example.py +++ b/examples/src/main/python/mllib/dataset_example.py @@ -19,6 +19,7 @@ An example of how to use DataFrame as a dataset for ML. Run with:: bin/spark-submit examples/src/main/python/mllib/dataset_example.py """ +from __future__ import print_function import os import sys @@ -32,16 +33,16 @@ def summarize(dataset): - print "schema: %s" % dataset.schema().json() + print("schema: %s" % dataset.schema().json()) labels = dataset.map(lambda r: r.label) - print "label average: %f" % labels.mean() + print("label average: %f" % labels.mean()) features = dataset.map(lambda r: r.features) summary = Statistics.colStats(features) - print "features average: %r" % summary.mean() + print("features average: %r" % summary.mean()) if __name__ == "__main__": if len(sys.argv) > 2: - print >> sys.stderr, "Usage: dataset_example.py " + print("Usage: dataset_example.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="DatasetExample") sqlContext = SQLContext(sc) @@ -54,9 +55,9 @@ def summarize(dataset): summarize(dataset0) tempdir = tempfile.NamedTemporaryFile(delete=False).name os.unlink(tempdir) - print "Save dataset as a Parquet file to %s." % tempdir + print("Save dataset as a Parquet file to %s." % tempdir) dataset0.saveAsParquetFile(tempdir) - print "Load it back and summarize it again." + print("Load it back and summarize it again.") dataset1 = sqlContext.parquetFile(tempdir).setName("dataset1").cache() summarize(dataset1) shutil.rmtree(tempdir) diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index fccabd841b139..513ed8fd51450 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import numpy import os @@ -83,18 +84,17 @@ def reindexClassLabels(data): numClasses = len(classCounts) # origToNewLabels: class --> index in 0,...,numClasses-1 if (numClasses < 2): - print >> sys.stderr, \ - "Dataset for classification should have at least 2 classes." + \ - " The given dataset had only %d classes." % numClasses + print("Dataset for classification should have at least 2 classes." + " The given dataset had only %d classes." % numClasses, file=sys.stderr) exit(1) origToNewLabels = dict([(sortedClasses[i], i) for i in range(0, numClasses)]) - print "numClasses = %d" % numClasses - print "Per-class example fractions, counts:" - print "Class\tFrac\tCount" + print("numClasses = %d" % numClasses) + print("Per-class example fractions, counts:") + print("Class\tFrac\tCount") for c in sortedClasses: frac = classCounts[c] / (numExamples + 0.0) - print "%g\t%g\t%d" % (c, frac, classCounts[c]) + print("%g\t%g\t%d" % (c, frac, classCounts[c])) if (sortedClasses[0] == 0 and sortedClasses[-1] == numClasses - 1): return (data, origToNewLabels) @@ -105,8 +105,7 @@ def reindexClassLabels(data): def usage(): - print >> sys.stderr, \ - "Usage: decision_tree_runner [libsvm format data filepath]" + print("Usage: decision_tree_runner [libsvm format data filepath]", file=sys.stderr) exit(1) @@ -133,13 +132,13 @@ def usage(): model = DecisionTree.trainClassifier(reindexedData, numClasses=numClasses, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. - print "Trained DecisionTree for classification:" - print " Model numNodes: %d" % model.numNodes() - print " Model depth: %d" % model.depth() - print " Training accuracy: %g" % getAccuracy(model, reindexedData) + print("Trained DecisionTree for classification:") + print(" Model numNodes: %d" % model.numNodes()) + print(" Model depth: %d" % model.depth()) + print(" Training accuracy: %g" % getAccuracy(model, reindexedData)) if model.numNodes() < 20: - print model.toDebugString() + print(model.toDebugString()) else: - print model + print(model) sc.stop() diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index a2cd626c9f19d..2cb8010cdc07f 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -18,7 +18,8 @@ """ A Gaussian Mixture Model clustering program using MLlib. """ -import sys +from __future__ import print_function + import random import argparse import numpy as np @@ -59,7 +60,7 @@ def parseVector(line): model = GaussianMixture.train(data, args.k, args.convergenceTol, args.maxIterations, args.seed) for i in range(args.k): - print ("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, - "sigma = ", model.gaussians[i].sigma.toArray()) - print ("Cluster labels (first 100): ", model.predict(data).take(100)) + print(("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, + "sigma = ", model.gaussians[i].sigma.toArray())) + print(("Cluster labels (first 100): ", model.predict(data).take(100))) sc.stop() diff --git a/examples/src/main/python/mllib/gradient_boosted_trees.py b/examples/src/main/python/mllib/gradient_boosted_trees.py index e647773ad9060..781bd61c9d2b5 100644 --- a/examples/src/main/python/mllib/gradient_boosted_trees.py +++ b/examples/src/main/python/mllib/gradient_boosted_trees.py @@ -18,6 +18,7 @@ """ Gradient boosted Trees classification and regression using MLlib. """ +from __future__ import print_function import sys @@ -34,7 +35,7 @@ def testClassification(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() \ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count() \ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification ensemble model:') @@ -49,7 +50,7 @@ def testRegression(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() \ + testMSE = labelsAndPredictions.map(lambda vp: (vp[0] - vp[1]) * (vp[0] - vp[1])).sum() \ / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression ensemble model:') @@ -58,7 +59,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: gradient_boosted_trees" + print("Usage: gradient_boosted_trees", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonGradientBoostedTrees") diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index 2eeb1abeeb12b..f901a87fa63ac 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -34,12 +35,12 @@ def parseVector(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) sc = SparkContext(appName="KMeans") lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector) k = int(sys.argv[2]) model = KMeans.train(data, k) - print "Final centers: " + str(model.clusterCenters) + print("Final centers: " + str(model.clusterCenters)) sc.stop() diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 8cae27fc4a52d..d4f1d34e2d8cf 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -20,11 +20,10 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function -from math import exp import sys -import numpy as np from pyspark import SparkContext from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.classification import LogisticRegressionWithSGD @@ -42,12 +41,12 @@ def parsePoint(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).map(parsePoint) iterations = int(sys.argv[2]) model = LogisticRegressionWithSGD.train(points, iterations) - print "Final weights: " + str(model.weights) - print "Final intercept: " + str(model.intercept) + print("Final weights: " + str(model.weights)) + print("Final intercept: " + str(model.intercept)) sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_example.py b/examples/src/main/python/mllib/random_forest_example.py index d3c24f7664329..4cfdad868c66e 100755 --- a/examples/src/main/python/mllib/random_forest_example.py +++ b/examples/src/main/python/mllib/random_forest_example.py @@ -22,6 +22,7 @@ For information on multiclass classification, please refer to the decision_tree_runner.py example. """ +from __future__ import print_function import sys @@ -43,7 +44,7 @@ def testClassification(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count()\ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count()\ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification forest model:') @@ -62,8 +63,8 @@ def testRegression(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum()\ - / float(testData.count()) + testMSE = labelsAndPredictions.map(lambda v_p1: (v_p1[0] - v_p1[1]) * (v_p1[0] - v_p1[1]))\ + .sum() / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression forest model:') print(model.toDebugString()) @@ -71,7 +72,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: random_forest_example" + print("Usage: random_forest_example", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonRandomForestExample") diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py index 1e8892741e714..729bae30b152c 100755 --- a/examples/src/main/python/mllib/random_rdd_generation.py +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -18,6 +18,7 @@ """ Randomly generated RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: random_rdd_generation" + print("Usage: random_rdd_generation", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonRandomRDDGeneration") @@ -37,19 +38,19 @@ # Example: RandomRDDs.normalRDD normalRDD = RandomRDDs.normalRDD(sc, numExamples) - print 'Generated RDD of %d examples sampled from the standard normal distribution'\ - % normalRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples sampled from the standard normal distribution' + % normalRDD.count()) + print(' First 5 samples:') for sample in normalRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() # Example: RandomRDDs.normalVectorRDD normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows=numExamples, numCols=2) - print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count()) + print(' First 5 samples:') for sample in normalVectorRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py index 92af3af5ebd1e..b7033ab7daeb3 100755 --- a/examples/src/main/python/mllib/sampled_rdds.py +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -18,6 +18,7 @@ """ Randomly sampled RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: sampled_rdds " + print("Usage: sampled_rdds ", file=sys.stderr) exit(-1) if len(sys.argv) == 2: datapath = sys.argv[1] @@ -41,24 +42,24 @@ examples = MLUtils.loadLibSVMFile(sc, datapath) numExamples = examples.count() if numExamples == 0: - print >> sys.stderr, "Error: Data file had no samples to load." + print("Error: Data file had no samples to load.", file=sys.stderr) exit(1) - print 'Loaded data with %d examples from file: %s' % (numExamples, datapath) + print('Loaded data with %d examples from file: %s' % (numExamples, datapath)) # Example: RDD.sample() and RDD.takeSample() expectedSampleSize = int(numExamples * fraction) - print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ - % (fraction, expectedSampleSize) + print('Sampling RDD using fraction %g. Expected sample size = %d.' + % (fraction, expectedSampleSize)) sampledRDD = examples.sample(withReplacement=True, fraction=fraction) - print ' RDD.sample(): sample has %d examples' % sampledRDD.count() + print(' RDD.sample(): sample has %d examples' % sampledRDD.count()) sampledArray = examples.takeSample(withReplacement=True, num=expectedSampleSize) - print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) + print(' RDD.takeSample(): sample has %d examples' % len(sampledArray)) - print + print() # Example: RDD.sampleByKey() keyedRDD = examples.map(lambda lp: (int(lp.label), lp.features)) - print ' Keyed data using label (Int) as key ==> Orig' + print(' Keyed data using label (Int) as key ==> Orig') # Count examples per label in original data. keyCountsA = keyedRDD.countByKey() @@ -69,18 +70,18 @@ sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement=True, fractions=fractions) keyCountsB = sampledByKeyRDD.countByKey() sizeB = sum(keyCountsB.values()) - print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ - % sizeB + print(' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' + % sizeB) # Compare samples - print ' \tFractions of examples with key' - print 'Key\tOrig\tSample' + print(' \tFractions of examples with key') + print('Key\tOrig\tSample') for k in sorted(keyCountsA.keys()): fracA = keyCountsA[k] / float(numExamples) if sizeB != 0: fracB = keyCountsB.get(k, 0) / float(sizeB) else: fracB = 0 - print '%d\t%g\t%g' % (k, fracA, fracB) + print('%d\t%g\t%g' % (k, fracA, fracB)) sc.stop() diff --git a/examples/src/main/python/mllib/word2vec.py b/examples/src/main/python/mllib/word2vec.py index 99fef4276a369..40d1b887927e0 100644 --- a/examples/src/main/python/mllib/word2vec.py +++ b/examples/src/main/python/mllib/word2vec.py @@ -23,6 +23,7 @@ # grep -o -E '\w+(\W+\w+){0,15}' text8 > text8_lines # This was done so that the example can be run in local mode +from __future__ import print_function import sys @@ -34,7 +35,7 @@ if __name__ == "__main__": if len(sys.argv) < 2: - print USAGE + print(USAGE) sys.exit("Argument for file not provided") file_path = sys.argv[1] sc = SparkContext(appName='Word2Vec') @@ -46,5 +47,5 @@ synonyms = model.findSynonyms('china', 40) for word, cosine_distance in synonyms: - print "{}: {}".format(word, cosine_distance) + print("{}: {}".format(word, cosine_distance)) sc.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index a5f25d78c1146..2fdc9773d4eb1 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -19,6 +19,7 @@ This is an example implementation of PageRank. For more conventional use, Please refer to PageRank implementation provided by graphx """ +from __future__ import print_function import re import sys @@ -42,11 +43,12 @@ def parseNeighbors(urls): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: pagerank " + print("Usage: pagerank ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of PageRank and is - given as an example! Please refer to PageRank implementation provided by graphx""" + print("""WARN: This is a naive implementation of PageRank and is + given as an example! Please refer to PageRank implementation provided by graphx""", + file=sys.stderr) # Initialize the spark context. sc = SparkContext(appName="PythonPageRank") @@ -62,19 +64,19 @@ def parseNeighbors(urls): links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() # Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - ranks = links.map(lambda (url, neighbors): (url, 1.0)) + ranks = links.map(lambda url_neighbors: (url_neighbors[0], 1.0)) # Calculates and updates URL ranks continuously using PageRank algorithm. - for iteration in xrange(int(sys.argv[2])): + for iteration in range(int(sys.argv[2])): # Calculates URL contributions to the rank of other URLs. contribs = links.join(ranks).flatMap( - lambda (url, (urls, rank)): computeContribs(urls, rank)) + lambda url_urls_rank: computeContribs(url_urls_rank[1][0], url_urls_rank[1][1])) # Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(add).mapValues(lambda rank: rank * 0.85 + 0.15) # Collects all URL ranks and dump them to console. for (link, rank) in ranks.collect(): - print "%s has rank: %s." % (link, rank) + print("%s has rank: %s." % (link, rank)) sc.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index fa4c20ab20281..96ddac761d698 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,14 +36,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, """ + print(""" Usage: parquet_inputformat.py Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \\ /path/to/examples/parquet_inputformat.py Assumes you have Parquet data stored in . - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -56,6 +57,6 @@ valueConverter='org.apache.spark.examples.pythonconverters.IndexedRecordToJavaConverter') output = parquet_rdd.map(lambda x: x[1]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index a7c74e969cdb9..92e5cf45abc8b 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,7 +36,7 @@ def f(_): y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n + 1), partitions).map(f).reduce(add) - print "Pi is roughly %f" % (4.0 * count / n) + count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + print("Pi is roughly %f" % (4.0 * count / n)) sc.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index bb686f17518a0..f6b0ecb02c100 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -22,7 +24,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: sort " + print("Usage: sort ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonSort") lines = sc.textFile(sys.argv[1], 1) @@ -33,6 +35,6 @@ # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() for (num, unitcount) in output: - print num + print(num) sc.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index d89361f324917..87d7b088f077b 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import os from pyspark import SparkContext @@ -68,6 +70,6 @@ teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") for each in teenagers.collect(): - print each[0] + print(each[0]) sc.stop() diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py index a33bdc475a06d..49b7902185aaa 100644 --- a/examples/src/main/python/status_api_demo.py +++ b/examples/src/main/python/status_api_demo.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import time import threading import Queue @@ -52,15 +54,15 @@ def run(): ids = status.getJobIdsForGroup() for id in ids: job = status.getJobInfo(id) - print "Job", id, "status: ", job.status + print("Job", id, "status: ", job.status) for sid in job.stageIds: info = status.getStageInfo(sid) if info: - print "Stage %d: %d tasks total (%d active, %d complete)" % \ - (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks) + print("Stage %d: %d tasks total (%d active, %d complete)" % + (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks)) time.sleep(1) - print "Job results are:", result.get() + print("Job results are:", result.get()) sc.stop() if __name__ == "__main__": diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py index f7ffb5379681e..f815dd26823d1 100644 --- a/examples/src/main/python/streaming/hdfs_wordcount.py +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -25,6 +25,7 @@ Then create a text file in `localdir` and the words in the file will get counted. """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: hdfs_wordcount.py " + print("Usage: hdfs_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingHDFSWordCount") diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py index 51e1ff822fc55..b178e7899b5e1 100644 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -27,6 +27,7 @@ spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ localhost:2181 test` """ +from __future__ import print_function import sys @@ -36,7 +37,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kafka_wordcount.py " + print("Usage: kafka_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingKafkaWordCount") diff --git a/examples/src/main/python/streaming/network_wordcount.py b/examples/src/main/python/streaming/network_wordcount.py index cfa9c1ff5bfbc..2b48bcfd55db0 100644 --- a/examples/src/main/python/streaming/network_wordcount.py +++ b/examples/src/main/python/streaming/network_wordcount.py @@ -25,6 +25,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999` """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: network_wordcount.py " + print("Usage: network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index fc6827c82bf9b..ac91f0a06b172 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -35,6 +35,7 @@ checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from the checkpoint data. """ +from __future__ import print_function import os import sys @@ -46,7 +47,7 @@ def createContext(host, port, outputPath): # If you do not see this printed, that means the StreamingContext has been loaded # from the new checkpoint - print "Creating new context" + print("Creating new context") if os.path.exists(outputPath): os.remove(outputPath) sc = SparkContext(appName="PythonStreamingRecoverableNetworkWordCount") @@ -60,8 +61,8 @@ def createContext(host, port, outputPath): def echo(time, rdd): counts = "Counts at time %s %s" % (time, rdd.collect()) - print counts - print "Appending to " + os.path.abspath(outputPath) + print(counts) + print("Appending to " + os.path.abspath(outputPath)) with open(outputPath, 'a') as f: f.write(counts + "\n") @@ -70,8 +71,8 @@ def echo(time, rdd): if __name__ == "__main__": if len(sys.argv) != 5: - print >> sys.stderr, "Usage: recoverable_network_wordcount.py "\ - " " + print("Usage: recoverable_network_wordcount.py " + " ", file=sys.stderr) exit(-1) host, port, checkpoint, output = sys.argv[1:] ssc = StreamingContext.getOrCreate(checkpoint, diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index f89bc562d856b..da90c07dbd82f 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -27,6 +27,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999` """ +from __future__ import print_function import os import sys @@ -44,7 +45,7 @@ def getSqlContextInstance(sparkContext): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: sql_network_wordcount.py " + print("Usage: sql_network_wordcount.py ", file=sys.stderr) exit(-1) host, port = sys.argv[1:] sc = SparkContext(appName="PythonSqlNetworkWordCount") @@ -57,7 +58,7 @@ def getSqlContextInstance(sparkContext): # Convert RDDs of the words DStream to DataFrame and run SQL query def process(time, rdd): - print "========= %s =========" % str(time) + print("========= %s =========" % str(time)) try: # Get the singleton instance of SQLContext diff --git a/examples/src/main/python/streaming/stateful_network_wordcount.py b/examples/src/main/python/streaming/stateful_network_wordcount.py index 18a9a5a452ffb..16ef646b7c42e 100644 --- a/examples/src/main/python/streaming/stateful_network_wordcount.py +++ b/examples/src/main/python/streaming/stateful_network_wordcount.py @@ -29,6 +29,7 @@ `$ bin/spark-submit examples/src/main/python/streaming/stateful_network_wordcount.py \ localhost 9999` """ +from __future__ import print_function import sys @@ -37,7 +38,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: stateful_network_wordcount.py " + print("Usage: stateful_network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingStatefulNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 00a281bfb6506..7bf5fb6ddfe29 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from random import Random @@ -49,20 +51,20 @@ def generateGraph(): # the graph to obtain the path (x, z). # Because join() joins on keys, the edges are stored in reversed order. - edges = tc.map(lambda (x, y): (y, x)) + edges = tc.map(lambda x_y: (x_y[1], x_y[0])) - oldCount = 0L + oldCount = 0 nextCount = tc.count() while True: oldCount = nextCount # Perform the join, obtaining an RDD of (y, (z, x)) pairs, # then project the result to obtain the new (x, z) paths. - new_edges = tc.join(edges).map(lambda (_, (a, b)): (b, a)) + new_edges = tc.join(edges).map(lambda __a_b: (__a_b[1][1], __a_b[1][0])) tc = tc.union(new_edges).distinct().cache() nextCount = tc.count() if nextCount == oldCount: break - print "TC has %i edges" % tc.count() + print("TC has %i edges" % tc.count()) sc.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index ae6cd13b83d92..7c0143607b61d 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from operator import add @@ -23,7 +25,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: wordcount " + print("Usage: wordcount ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonWordCount") lines = sc.textFile(sys.argv[1], 1) @@ -32,6 +34,6 @@ .reduceByKey(add) output = counts.collect() for (word, count) in output: - print "%s: %i" % (word, count) + print("%s: %i" % (word, count)) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala index ecd3b16598438..534edac56bc5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.api.python import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.recommendation.{MatrixFactorizationModel, Rating} import org.apache.spark.rdd.RDD @@ -31,10 +32,14 @@ private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorization predict(SerDe.asTupleRDD(userAndProducts.rdd)) def getUserFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + SerDe.fromTuple2RDD(userFeatures.map { + case (user, feature) => (user, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) } def getProductFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + SerDe.fromTuple2RDD(productFeatures.map { + case (product, feature) => (product, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index ab15f0f36a14b..f976d2f97b043 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -28,7 +28,6 @@ import scala.reflect.ClassTag import net.razorvine.pickle._ -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ @@ -40,15 +39,15 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.stat.test.ChiSqTestResult -import org.apache.spark.mllib.tree.{GradientBoostedTrees, RandomForest, DecisionTree} -import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Algo, Strategy} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.mllib.tree.configuration.{Algo, BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.loss.Losses -import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel, RandomForestModel, DecisionTreeModel} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel, RandomForestModel} +import org.apache.spark.mllib.tree.{DecisionTree, GradientBoostedTrees, RandomForest} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -279,7 +278,7 @@ private[python] class PythonMLLibAPI extends Serializable { data: JavaRDD[LabeledPoint], lambda: Double): JList[Object] = { val model = NaiveBayes.train(data.rdd, lambda) - List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta). + List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta.map(Vectors.dense)). map(_.asInstanceOf[Object]).asJava } @@ -335,7 +334,7 @@ private[python] class PythonMLLibAPI extends Serializable { mu += model.gaussians(i).mu sigma += model.gaussians(i).sigma } - List(wt.toArray, mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava + List(Vectors.dense(wt.toArray), mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava } finally { data.rdd.unpersist(blocking = false) } @@ -346,20 +345,20 @@ private[python] class PythonMLLibAPI extends Serializable { */ def predictSoftGMM( data: JavaRDD[Vector], - wt: Object, + wt: Vector, mu: Array[Object], - si: Array[Object]): RDD[Array[Double]] = { + si: Array[Object]): RDD[Vector] = { - val weight = wt.asInstanceOf[Array[Double]] + val weight = wt.toArray val mean = mu.map(_.asInstanceOf[DenseVector]) val sigma = si.map(_.asInstanceOf[DenseMatrix]) val gaussians = Array.tabulate(weight.length){ i => new MultivariateGaussian(mean(i), sigma(i)) } val model = new GaussianMixtureModel(weight, gaussians) - model.predictSoft(data) + model.predictSoft(data).map(Vectors.dense) } - + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -936,6 +935,14 @@ private[spark] object SerDe extends Serializable { out.write(code) } + protected def getBytes(obj: Object): Array[Byte] = { + if (obj.getClass.isArray) { + obj.asInstanceOf[Array[Byte]] + } else { + obj.asInstanceOf[String].getBytes(LATIN1) + } + } + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } @@ -961,7 +968,7 @@ private[spark] object SerDe extends Serializable { if (args.length != 1) { throw new PickleException("should be 1") } - val bytes = args(0).asInstanceOf[String].getBytes(LATIN1) + val bytes = getBytes(args(0)) val bb = ByteBuffer.wrap(bytes, 0, bytes.length) bb.order(ByteOrder.nativeOrder()) val db = bb.asDoubleBuffer() @@ -994,7 +1001,7 @@ private[spark] object SerDe extends Serializable { if (args.length != 3) { throw new PickleException("should be 3") } - val bytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val bytes = getBytes(args(2)) val n = bytes.length / 8 val values = new Array[Double](n) val order = ByteOrder.nativeOrder() @@ -1031,8 +1038,8 @@ private[spark] object SerDe extends Serializable { throw new PickleException("should be 3") } val size = args(0).asInstanceOf[Int] - val indiceBytes = args(1).asInstanceOf[String].getBytes(LATIN1) - val valueBytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val indiceBytes = getBytes(args(1)) + val valueBytes = getBytes(args(2)) val n = indiceBytes.length / 4 val indices = new Array[Int](n) val values = new Array[Double](n) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index ccbca67656c8d..7271809e43880 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -54,7 +54,7 @@ ... def zero(self, value): ... return [0.0] * len(value) ... def addInPlace(self, val1, val2): -... for i in xrange(len(val1)): +... for i in range(len(val1)): ... val1[i] += val2[i] ... return val1 >>> va = sc.accumulator([1.0, 2.0, 3.0], VectorAccumulatorParam()) @@ -86,9 +86,13 @@ Exception:... """ +import sys import select import struct -import SocketServer +if sys.version < '3': + import SocketServer +else: + import socketserver as SocketServer import threading from pyspark.cloudpickle import CloudPickler from pyspark.serializers import read_int, PickleSerializer @@ -247,6 +251,7 @@ class AccumulatorServer(SocketServer.TCPServer): def shutdown(self): self.server_shutdown = True SocketServer.TCPServer.shutdown(self) + self.server_close() def _start_update_server(): diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 6b8a8b256a891..3de4615428bb6 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -16,10 +16,15 @@ # import os -import cPickle +import sys import gc from tempfile import NamedTemporaryFile +if sys.version < '3': + import cPickle as pickle +else: + import pickle + unicode = str __all__ = ['Broadcast'] @@ -70,33 +75,19 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None): self._path = path def dump(self, value, f): - if isinstance(value, basestring): - if isinstance(value, unicode): - f.write('U') - value = value.encode('utf8') - else: - f.write('S') - f.write(value) - else: - f.write('P') - cPickle.dump(value, f, 2) + pickle.dump(value, f, 2) f.close() return f.name def load(self, path): with open(path, 'rb', 1 << 20) as f: - flag = f.read(1) - data = f.read() - if flag == 'P': - # cPickle.loads() may create lots of objects, disable GC - # temporary for better performance - gc.disable() - try: - return cPickle.loads(data) - finally: - gc.enable() - else: - return data.decode('utf8') if flag == 'U' else data + # pickle.load() may create lots of objects, disable GC + # temporary for better performance + gc.disable() + try: + return pickle.load(f) + finally: + gc.enable() @property def value(self): diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index bb0783555aa77..9ef93071d2e77 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -40,164 +40,126 @@ NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ - +from __future__ import print_function import operator import os +import io import pickle import struct import sys import types from functools import partial import itertools -from copy_reg import _extension_registry, _inverted_registry, _extension_cache -import new import dis import traceback -import platform - -PyImp = platform.python_implementation() - -import logging -cloudLog = logging.getLogger("Cloud.Transport") +if sys.version < '3': + from pickle import Pickler + try: + from cStringIO import StringIO + except ImportError: + from StringIO import StringIO + PY3 = False +else: + types.ClassType = type + from pickle import _Pickler as Pickler + from io import BytesIO as StringIO + PY3 = True #relevant opcodes -STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) -DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) -LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) +STORE_GLOBAL = dis.opname.index('STORE_GLOBAL') +DELETE_GLOBAL = dis.opname.index('DELETE_GLOBAL') +LOAD_GLOBAL = dis.opname.index('LOAD_GLOBAL') GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] +HAVE_ARGUMENT = dis.HAVE_ARGUMENT +EXTENDED_ARG = dis.EXTENDED_ARG -HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) -EXTENDED_ARG = chr(dis.EXTENDED_ARG) - -if PyImp == "PyPy": - # register builtin type in `new` - new.method = types.MethodType - -try: - from cStringIO import StringIO -except ImportError: - from StringIO import StringIO -# These helper functions were copied from PiCloud's util module. def islambda(func): - return getattr(func,'func_name') == '' + return getattr(func,'__name__') == '' -def xrange_params(xrangeobj): - """Returns a 3 element tuple describing the xrange start, step, and len - respectively - Note: Only guarentees that elements of xrange are the same. parameters may - be different. - e.g. xrange(1,1) is interpretted as xrange(0,0); both behave the same - though w/ iteration - """ - - xrange_len = len(xrangeobj) - if not xrange_len: #empty - return (0,1,0) - start = xrangeobj[0] - if xrange_len == 1: #one element - return start, 1, 1 - return (start, xrangeobj[1] - xrangeobj[0], xrange_len) - -#debug variables intended for developer use: -printSerialization = False -printMemoization = False +_BUILTIN_TYPE_NAMES = {} +for k, v in types.__dict__.items(): + if type(v) is type: + _BUILTIN_TYPE_NAMES[v] = k -useForcedImports = True #Should I use forced imports for tracking? +def _builtin_type(name): + return getattr(types, name) -class CloudPickler(pickle.Pickler): +class CloudPickler(Pickler): - dispatch = pickle.Pickler.dispatch.copy() - savedForceImports = False - savedDjangoEnv = False #hack tro transport django environment + dispatch = Pickler.dispatch.copy() - def __init__(self, file, protocol=None, min_size_to_save= 0): - pickle.Pickler.__init__(self,file,protocol) - self.modules = set() #set of modules needed to depickle - self.globals_ref = {} # map ids to dictionary. used to ensure that functions can share global env + def __init__(self, file, protocol=None): + Pickler.__init__(self, file, protocol) + # set of modules to unpickle + self.modules = set() + # map ids to dictionary. used to ensure that functions can share global env + self.globals_ref = {} def dump(self, obj): - # note: not thread safe - # minimal side-effects, so not fixing - recurse_limit = 3000 - base_recurse = sys.getrecursionlimit() - if base_recurse < recurse_limit: - sys.setrecursionlimit(recurse_limit) self.inject_addons() try: - return pickle.Pickler.dump(self, obj) - except RuntimeError, e: + return Pickler.dump(self, obj) + except RuntimeError as e: if 'recursion' in e.args[0]: - msg = """Could not pickle object as excessively deep recursion required. - Try _fast_serialization=2 or contact PiCloud support""" + msg = """Could not pickle object as excessively deep recursion required.""" raise pickle.PicklingError(msg) - finally: - new_recurse = sys.getrecursionlimit() - if new_recurse == recurse_limit: - sys.setrecursionlimit(base_recurse) + + def save_memoryview(self, obj): + """Fallback to save_string""" + Pickler.save_string(self, str(obj)) def save_buffer(self, obj): """Fallback to save_string""" - pickle.Pickler.save_string(self,str(obj)) - dispatch[buffer] = save_buffer + Pickler.save_string(self,str(obj)) + if PY3: + dispatch[memoryview] = save_memoryview + else: + dispatch[buffer] = save_buffer - #block broken objects - def save_unsupported(self, obj, pack=None): + def save_unsupported(self, obj): raise pickle.PicklingError("Cannot pickle objects of type %s" % type(obj)) dispatch[types.GeneratorType] = save_unsupported - #python2.6+ supports slice pickling. some py2.5 extensions might as well. We just test it - try: - slice(0,1).__reduce__() - except TypeError: #can't pickle - - dispatch[slice] = save_unsupported - - #itertools objects do not pickle! + # itertools objects do not pickle! for v in itertools.__dict__.values(): if type(v) is type: dispatch[v] = save_unsupported - - def save_dict(self, obj): - """hack fix - If the dict is a global, deal with it in a special way - """ - #print 'saving', obj - if obj is __builtins__: - self.save_reduce(_get_module_builtins, (), obj=obj) - else: - pickle.Pickler.save_dict(self, obj) - dispatch[pickle.DictionaryType] = save_dict - - - def save_module(self, obj, pack=struct.pack): + def save_module(self, obj): """ Save a module as an import """ - #print 'try save import', obj.__name__ self.modules.add(obj) - self.save_reduce(subimport,(obj.__name__,), obj=obj) - dispatch[types.ModuleType] = save_module #new type + self.save_reduce(subimport, (obj.__name__,), obj=obj) + dispatch[types.ModuleType] = save_module - def save_codeobject(self, obj, pack=struct.pack): + def save_codeobject(self, obj): """ Save a code object """ - #print 'try to save codeobj: ', obj - args = ( - obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, - obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars - ) + if PY3: + args = ( + obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, + obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, obj.co_varnames, + obj.co_filename, obj.co_name, obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, + obj.co_cellvars + ) + else: + args = ( + obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, + obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars + ) self.save_reduce(types.CodeType, args, obj=obj) - dispatch[types.CodeType] = save_codeobject #new type + dispatch[types.CodeType] = save_codeobject - def save_function(self, obj, name=None, pack=struct.pack): + def save_function(self, obj, name=None): """ Registered with the dispatch to handle all function types. Determines what kind of function obj is (e.g. lambda, defined at @@ -205,12 +167,14 @@ def save_function(self, obj, name=None, pack=struct.pack): """ write = self.write - name = obj.__name__ + if name is None: + name = obj.__name__ modname = pickle.whichmodule(obj, name) - #print 'which gives %s %s %s' % (modname, obj, name) + # print('which gives %s %s %s' % (modname, obj, name)) try: themodule = sys.modules[modname] - except KeyError: # eval'd items such as namedtuple give invalid items for their function __module__ + except KeyError: + # eval'd items such as namedtuple give invalid items for their function __module__ modname = '__main__' if modname == '__main__': @@ -221,37 +185,18 @@ def save_function(self, obj, name=None, pack=struct.pack): if getattr(themodule, name, None) is obj: return self.save_global(obj, name) - if not self.savedDjangoEnv: - #hack for django - if we detect the settings module, we transport it - django_settings = os.environ.get('DJANGO_SETTINGS_MODULE', '') - if django_settings: - django_mod = sys.modules.get(django_settings) - if django_mod: - cloudLog.debug('Transporting django settings %s during save of %s', django_mod, name) - self.savedDjangoEnv = True - self.modules.add(django_mod) - write(pickle.MARK) - self.save_reduce(django_settings_load, (django_mod.__name__,), obj=django_mod) - write(pickle.POP_MARK) - - # if func is lambda, def'ed at prompt, is in main, or is nested, then # we'll pickle the actual function object rather than simply saving a # reference (as is done in default pickler), via save_function_tuple. - if islambda(obj) or obj.func_code.co_filename == '' or themodule is None: - #Force server to import modules that have been imported in main - modList = None - if themodule is None and not self.savedForceImports: - mainmod = sys.modules['__main__'] - if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'): - modList = list(mainmod.___pyc_forcedImports__) - self.savedForceImports = True - self.save_function_tuple(obj, modList) + if islambda(obj) or obj.__code__.co_filename == '' or themodule is None: + #print("save global", islambda(obj), obj.__code__.co_filename, modname, themodule) + self.save_function_tuple(obj) return - else: # func is nested + else: + # func is nested klass = getattr(themodule, name, None) if klass is None or klass is not obj: - self.save_function_tuple(obj, [themodule]) + self.save_function_tuple(obj) return if obj.__dict__: @@ -266,7 +211,7 @@ def save_function(self, obj, name=None, pack=struct.pack): self.memoize(obj) dispatch[types.FunctionType] = save_function - def save_function_tuple(self, func, forced_imports): + def save_function_tuple(self, func): """ Pickles an actual func object. A func comprises: code, globals, defaults, closure, and dict. We @@ -281,19 +226,6 @@ def save_function_tuple(self, func, forced_imports): save = self.save write = self.write - # save the modules (if any) - if forced_imports: - write(pickle.MARK) - save(_modules_to_main) - #print 'forced imports are', forced_imports - - forced_names = map(lambda m: m.__name__, forced_imports) - save((forced_names,)) - - #save((forced_imports,)) - write(pickle.REDUCE) - write(pickle.POP_MARK) - code, f_globals, defaults, closure, dct, base_globals = self.extract_func_data(func) save(_fill_function) # skeleton function updater @@ -318,6 +250,8 @@ def extract_code_globals(co): Find all globals names read or written to by codeblock co """ code = co.co_code + if not PY3: + code = [ord(c) for c in code] names = co.co_names out_names = set() @@ -327,18 +261,18 @@ def extract_code_globals(co): while i < n: op = code[i] - i = i+1 + i += 1 if op >= HAVE_ARGUMENT: - oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg + oparg = code[i] + code[i+1] * 256 + extended_arg extended_arg = 0 - i = i+2 + i += 2 if op == EXTENDED_ARG: - extended_arg = oparg*65536L + extended_arg = oparg*65536 if op in GLOBAL_OPS: out_names.add(names[oparg]) - #print 'extracted', out_names, ' from ', names - if co.co_consts: # see if nested function have any global refs + # see if nested function have any global refs + if co.co_consts: for const in co.co_consts: if type(const) is types.CodeType: out_names |= CloudPickler.extract_code_globals(const) @@ -350,46 +284,28 @@ def extract_func_data(self, func): Turn the function into a tuple of data necessary to recreate it: code, globals, defaults, closure, dict """ - code = func.func_code + code = func.__code__ # extract all global ref's - func_global_refs = CloudPickler.extract_code_globals(code) + func_global_refs = self.extract_code_globals(code) # process all variables referenced by global environment f_globals = {} for var in func_global_refs: - #Some names, such as class functions are not global - we don't need them - if func.func_globals.has_key(var): - f_globals[var] = func.func_globals[var] + if var in func.__globals__: + f_globals[var] = func.__globals__[var] # defaults requires no processing - defaults = func.func_defaults - - def get_contents(cell): - try: - return cell.cell_contents - except ValueError, e: #cell is empty error on not yet assigned - raise pickle.PicklingError('Function to be pickled has free variables that are referenced before assignment in enclosing scope') - + defaults = func.__defaults__ # process closure - if func.func_closure: - closure = map(get_contents, func.func_closure) - else: - closure = [] + closure = [c.cell_contents for c in func.__closure__] if func.__closure__ else [] # save the dict - dct = func.func_dict - - if printSerialization: - outvars = ['code: ' + str(code) ] - outvars.append('globals: ' + str(f_globals)) - outvars.append('defaults: ' + str(defaults)) - outvars.append('closure: ' + str(closure)) - print 'function ', func, 'is extracted to: ', ', '.join(outvars) + dct = func.__dict__ - base_globals = self.globals_ref.get(id(func.func_globals), {}) - self.globals_ref[id(func.func_globals)] = base_globals + base_globals = self.globals_ref.get(id(func.__globals__), {}) + self.globals_ref[id(func.__globals__)] = base_globals return (code, f_globals, defaults, closure, dct, base_globals) @@ -400,8 +316,9 @@ def save_builtin_function(self, obj): dispatch[types.BuiltinFunctionType] = save_builtin_function def save_global(self, obj, name=None, pack=struct.pack): - write = self.write - memo = self.memo + if obj.__module__ == "__builtin__" or obj.__module__ == "builtins": + if obj in _BUILTIN_TYPE_NAMES: + return self.save_reduce(_builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) if name is None: name = obj.__name__ @@ -410,98 +327,57 @@ def save_global(self, obj, name=None, pack=struct.pack): if modname is None: modname = pickle.whichmodule(obj, name) - try: - __import__(modname) - themodule = sys.modules[modname] - except (ImportError, KeyError, AttributeError): #should never occur - raise pickle.PicklingError( - "Can't pickle %r: Module %s cannot be found" % - (obj, modname)) - if modname == '__main__': themodule = None - - if themodule: + else: + __import__(modname) + themodule = sys.modules[modname] self.modules.add(themodule) - sendRef = True - typ = type(obj) - #print 'saving', obj, typ - try: - try: #Deal with case when getattribute fails with exceptions - klass = getattr(themodule, name) - except (AttributeError): - if modname == '__builtin__': #new.* are misrepeported - modname = 'new' - __import__(modname) - themodule = sys.modules[modname] - try: - klass = getattr(themodule, name) - except AttributeError, a: - # print themodule, name, obj, type(obj) - raise pickle.PicklingError("Can't pickle builtin %s" % obj) - else: - raise + if hasattr(themodule, name) and getattr(themodule, name) is obj: + return Pickler.save_global(self, obj, name) - except (ImportError, KeyError, AttributeError): - if typ == types.TypeType or typ == types.ClassType: - sendRef = False - else: #we can't deal with this - raise - else: - if klass is not obj and (typ == types.TypeType or typ == types.ClassType): - sendRef = False - if not sendRef: - #note: Third party types might crash this - add better checks! - d = dict(obj.__dict__) #copy dict proxy to a dict - if not isinstance(d.get('__dict__', None), property): # don't extract dict that are properties - d.pop('__dict__',None) - d.pop('__weakref__',None) + typ = type(obj) + if typ is not obj and isinstance(obj, (type, types.ClassType)): + d = dict(obj.__dict__) # copy dict proxy to a dict + if not isinstance(d.get('__dict__', None), property): + # don't extract dict that are properties + d.pop('__dict__', None) + d.pop('__weakref__', None) # hack as __new__ is stored differently in the __dict__ new_override = d.get('__new__', None) if new_override: d['__new__'] = obj.__new__ - self.save_reduce(type(obj),(obj.__name__,obj.__bases__, - d),obj=obj) - #print 'internal reduce dask %s %s' % (obj, d) - return - - if self.proto >= 2: - code = _extension_registry.get((modname, name)) - if code: - assert code > 0 - if code <= 0xff: - write(pickle.EXT1 + chr(code)) - elif code <= 0xffff: - write("%c%c%c" % (pickle.EXT2, code&0xff, code>>8)) - else: - write(pickle.EXT4 + pack("> sys.stderr, 'Cloud not import django settings %s:' % (name) - print_exec(sys.stderr) - if modified_env: - del os.environ['DJANGO_SETTINGS_MODULE'] - else: - #add project directory to sys,path: - if hasattr(module,'__file__'): - dirname = os.path.split(module.__file__)[0] + '/' - sys.path.append(dirname) # restores function attributes def _restore_attr(obj, attr): @@ -851,13 +636,16 @@ def _restore_attr(obj, attr): setattr(obj, key, val) return obj + def _get_module_builtins(): return pickle.__builtins__ + def print_exec(stream): ei = sys.exc_info() traceback.print_exception(ei[0], ei[1], ei[2], None, stream) + def _modules_to_main(modList): """Force every module in modList to be placed into main""" if not modList: @@ -868,22 +656,16 @@ def _modules_to_main(modList): if type(modname) is str: try: mod = __import__(modname) - except Exception, i: #catch all... - sys.stderr.write('warning: could not import %s\n. Your function may unexpectedly error due to this import failing; \ -A version mismatch is likely. Specific error was:\n' % modname) + except Exception as e: + sys.stderr.write('warning: could not import %s\n. ' + 'Your function may unexpectedly error due to this import failing;' + 'A version mismatch is likely. Specific error was:\n' % modname) print_exec(sys.stderr) else: - setattr(main,mod.__name__, mod) - else: - #REVERSE COMPATIBILITY FOR CLOUD CLIENT 1.5 (WITH EPD) - #In old version actual module was sent - setattr(main,modname.__name__, modname) + setattr(main, mod.__name__, mod) -#object generators: -def _build_xrange(start, step, len): - """Built xrange explicitly""" - return xrange(start, start + step*len, step) +#object generators: def _genpartial(func, args, kwds): if not args: args = () @@ -891,22 +673,26 @@ def _genpartial(func, args, kwds): kwds = {} return partial(func, *args, **kwds) + def _fill_function(func, globals, defaults, dict): """ Fills in the rest of function data into the skeleton function object that were created via _make_skel_func(). """ - func.func_globals.update(globals) - func.func_defaults = defaults - func.func_dict = dict + func.__globals__.update(globals) + func.__defaults__ = defaults + func.__dict__ = dict return func + def _make_cell(value): - return (lambda: value).func_closure[0] + return (lambda: value).__closure__[0] + def _reconstruct_closure(values): return tuple([_make_cell(v) for v in values]) + def _make_skel_func(code, closures, base_globals = None): """ Creates a skeleton function object that contains just the provided code and the correct number of cells in func_closure. All other @@ -928,40 +714,3 @@ def _make_skel_func(code, closures, base_globals = None): def _getobject(modname, attribute): mod = __import__(modname, fromlist=[attribute]) return mod.__dict__[attribute] - -def _generateImage(size, mode, str_rep): - """Generate image from string representation""" - import Image - i = Image.new(mode, size) - i.fromstring(str_rep) - return i - -def _lazyloadImage(fp): - import Image - fp.seek(0) #works in almost any case - return Image.open(fp) - -"""Timeseries""" -def _genTimeSeries(reduce_args, state): - import scikits.timeseries.tseries as ts - from numpy import ndarray - from numpy.ma import MaskedArray - - - time_series = ts._tsreconstruct(*reduce_args) - - #from setstate modified - (ver, shp, typ, isf, raw, msk, flv, dsh, dtm, dtyp, frq, infodict) = state - #print 'regenerating %s' % dtyp - - MaskedArray.__setstate__(time_series, (ver, shp, typ, isf, raw, msk, flv)) - _dates = time_series._dates - #_dates.__setstate__((ver, dsh, typ, isf, dtm, frq)) #use remote typ - ndarray.__setstate__(_dates,(dsh,dtyp, isf, dtm)) - _dates.freq = frq - _dates._cachedinfo.update(dict(full=None, hasdups=None, steps=None, - toobj=None, toord=None, tostr=None)) - # Update the _optinfo dictionary - time_series._optinfo.update(infodict) - return time_series - diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index dc7cd0bce56f3..924da3eecf214 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -44,7 +44,7 @@ >>> conf.get("spark.executorEnv.VAR1") u'value1' ->>> print conf.toDebugString() +>>> print(conf.toDebugString()) spark.executorEnv.VAR1=value1 spark.executorEnv.VAR3=value3 spark.executorEnv.VAR4=value4 @@ -56,6 +56,13 @@ __all__ = ['SparkConf'] +import sys +import re + +if sys.version > '3': + unicode = str + __doc__ = re.sub(r"(\W|^)[uU](['])", r'\1\2', __doc__) + class SparkConf(object): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 78dccc40470e3..1dc2fec0ae5c8 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import os import shutil import sys @@ -32,11 +34,14 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler +if sys.version > '3': + xrange = range + __all__ = ['SparkContext'] @@ -133,7 +138,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if sparkHome: self._conf.setSparkHome(sparkHome) if environment: - for key, value in environment.iteritems(): + for key, value in environment.items(): self._conf.setExecutorEnv(key, value) for key, value in DEFAULT_CONFIGS.items(): self._conf.setIfMissing(key, value) @@ -153,6 +158,10 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if k.startswith("spark.executorEnv."): varName = k[len("spark.executorEnv."):] self.environment[varName] = v + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + # disable randomness of hash of string in worker, if this is not + # launched by spark-submit + self.environment["PYTHONHASHSEED"] = "0" # Create the Java SparkContext through Py4J self._jsc = jsc or self._initialize_context(self._conf._jconf) @@ -323,7 +332,7 @@ def parallelize(self, c, numSlices=None): start0 = c[0] def getStart(split): - return start0 + (split * size / numSlices) * step + return start0 + int((split * size / numSlices)) * step def f(split, iterator): return xrange(getStart(split), getStart(split + 1), step) @@ -357,6 +366,7 @@ def pickleFile(self, name, minPartitions=None): minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.objectFile(name, minPartitions), self) + @ignore_unicode_prefix def textFile(self, name, minPartitions=None, use_unicode=True): """ Read a text file from HDFS, a local file system (available on all @@ -369,7 +379,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello world!") + ... _ = testFile.write("Hello world!") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello world!'] @@ -378,6 +388,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): return RDD(self._jsc.textFile(name, minPartitions), self, UTF8Deserializer(use_unicode)) + @ignore_unicode_prefix def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): """ Read a directory of text files from HDFS, a local file system @@ -411,9 +422,9 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): >>> dirPath = os.path.join(tempdir, "files") >>> os.mkdir(dirPath) >>> with open(os.path.join(dirPath, "1.txt"), "w") as file1: - ... file1.write("1") + ... _ = file1.write("1") >>> with open(os.path.join(dirPath, "2.txt"), "w") as file2: - ... file2.write("2") + ... _ = file2.write("2") >>> textFiles = sc.wholeTextFiles(dirPath) >>> sorted(textFiles.collect()) [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] @@ -456,7 +467,7 @@ def _dictToJavaMap(self, d): jm = self._jvm.java.util.HashMap() if not d: d = {} - for k, v in d.iteritems(): + for k, v in d.items(): jm[k] = v return jm @@ -608,6 +619,7 @@ def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) return RDD(jrdd, self, input_deserializer) + @ignore_unicode_prefix def union(self, rdds): """ Build the union of a list of RDDs. @@ -618,7 +630,7 @@ def union(self, rdds): >>> path = os.path.join(tempdir, "union-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello") + ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello'] @@ -677,7 +689,7 @@ def addFile(self, path): >>> from pyspark import SparkFiles >>> path = os.path.join(tempdir, "test.txt") >>> with open(path, "w") as testFile: - ... testFile.write("100") + ... _ = testFile.write("100") >>> sc.addFile(path) >>> def func(iterator): ... with open(SparkFiles.get("test.txt")) as testFile: @@ -705,11 +717,13 @@ def addPyFile(self, path): """ self.addFile(path) (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix - if filename[-4:].lower() in self.PACKAGE_EXTENSIONS: self._python_includes.append(filename) # for tests in local mode sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() def setCheckpointDir(self, dirName): """ @@ -744,7 +758,7 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): The application can use L{SparkContext.cancelJobGroup} to cancel all running jobs in this group. - >>> import thread, threading + >>> import threading >>> from time import sleep >>> result = "Not Set" >>> lock = threading.Lock() @@ -763,10 +777,10 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): ... sleep(5) ... sc.cancelJobGroup("job_to_cancel") >>> supress = lock.acquire() - >>> supress = thread.start_new_thread(start_job, (10,)) - >>> supress = thread.start_new_thread(stop_job, tuple()) + >>> supress = threading.Thread(target=start_job, args=(10,)).start() + >>> supress = threading.Thread(target=stop_job).start() >>> supress = lock.acquire() - >>> print result + >>> print(result) Cancelled If interruptOnCancel is set to true for the job group, then job cancellation will result diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 93885985fe377..7f06d4288c872 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -24,9 +24,10 @@ import traceback import time import gc -from errno import EINTR, ECHILD, EAGAIN +from errno import EINTR, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT + from pyspark.worker import main as worker_main from pyspark.serializers import read_int, write_int @@ -53,8 +54,8 @@ def worker(sock): # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because # otherwise writes also cause a seek that makes us miss data on the read side. - infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + infile = os.fdopen(os.dup(sock.fileno()), "rb", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "wb", 65536) exit_code = 0 try: worker_main(infile, outfile) @@ -68,17 +69,6 @@ def worker(sock): return exit_code -# Cleanup zombie children -def cleanup_dead_children(): - try: - while True: - pid, _ = os.waitpid(0, os.WNOHANG) - if not pid: - break - except: - pass - - def manager(): # Create a new process group to corral our children os.setpgid(0, 0) @@ -88,8 +78,12 @@ def manager(): listen_sock.bind(('127.0.0.1', 0)) listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() - write_int(listen_port, sys.stdout) - sys.stdout.flush() + + # re-open stdin/stdout in 'wb' mode + stdin_bin = os.fdopen(sys.stdin.fileno(), 'rb', 4) + stdout_bin = os.fdopen(sys.stdout.fileno(), 'wb', 4) + write_int(listen_port, stdout_bin) + stdout_bin.flush() def shutdown(code): signal.signal(SIGTERM, SIG_DFL) @@ -101,6 +95,7 @@ def handle_sigterm(*args): shutdown(1) signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP + signal.signal(SIGCHLD, SIG_IGN) reuse = os.environ.get("SPARK_REUSE_WORKER") @@ -115,12 +110,9 @@ def handle_sigterm(*args): else: raise - # cleanup in signal handler will cause deadlock - cleanup_dead_children() - if 0 in ready_fds: try: - worker_pid = read_int(sys.stdin) + worker_pid = read_int(stdin_bin) except EOFError: # Spark told us to exit by closing stdin shutdown(0) @@ -145,7 +137,7 @@ def handle_sigterm(*args): time.sleep(1) pid = os.fork() # error here will shutdown daemon else: - outfile = sock.makefile('w') + outfile = sock.makefile(mode='wb') write_int(e.errno, outfile) # Signal that the fork failed outfile.flush() outfile.close() @@ -157,7 +149,7 @@ def handle_sigterm(*args): listen_sock.close() try: # Acknowledge that the fork was successful - outfile = sock.makefile("w") + outfile = sock.makefile(mode="wb") write_int(os.getpid(), outfile) outfile.flush() outfile.close() diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py index bc441f138f7fc..4ef2afe03544f 100644 --- a/python/pyspark/heapq3.py +++ b/python/pyspark/heapq3.py @@ -627,51 +627,49 @@ def merge(iterables, key=None, reverse=False): if key is None: for order, it in enumerate(map(iter, iterables)): try: - next = it.next - h_append([next(), order * direction, next]) + h_append([next(it), order * direction, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - value, order, next = s = h[0] + value, order, it = s = h[0] yield value - s[0] = next() # raises StopIteration when exhausted + s[0] = next(it) # raises StopIteration when exhausted _heapreplace(h, s) # restore heap condition except StopIteration: _heappop(h) # remove empty iterator if h: # fast case when only a single iterator remains - value, order, next = h[0] + value, order, it = h[0] yield value - for value in next.__self__: + for value in it: yield value return for order, it in enumerate(map(iter, iterables)): try: - next = it.next - value = next() - h_append([key(value), order * direction, value, next]) + value = next(it) + h_append([key(value), order * direction, value, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - key_value, order, value, next = s = h[0] + key_value, order, value, it = s = h[0] yield value - value = next() + value = next(it) s[0] = key(value) s[2] = value _heapreplace(h, s) except StopIteration: _heappop(h) if h: - key_value, order, value, next = h[0] + key_value, order, value, it = h[0] yield value - for value in next.__self__: + for value in it: yield value diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2a5e84a7dfdb4..45bc38f7e61f8 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -69,7 +69,7 @@ def preexec_func(): if callback_socket in readable: gateway_connection = callback_socket.accept()[0] # Determine which ephemeral port the server started on: - gateway_port = read_int(gateway_connection.makefile()) + gateway_port = read_int(gateway_connection.makefile(mode="rb")) gateway_connection.close() callback_socket.close() if gateway_port is None: diff --git a/python/pyspark/join.py b/python/pyspark/join.py index c3491defb2b29..94df3990164d6 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -32,6 +32,7 @@ """ from pyspark.resultiterable import ResultIterable +from functools import reduce def _do_python_join(rdd, other, numPartitions, dispatch): diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d7bc09fd77adb..45754bc9d4b10 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -39,10 +39,10 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> lr = LogisticRegression(maxIter=5, regParam=0.01) >>> model = lr.fit(df) >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0))]).toDF() - >>> print model.transform(test0).head().prediction + >>> model.transform(test0).head().prediction 0.0 >>> test1 = sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))]).toDF() - >>> print model.transform(test1).head().prediction + >>> model.transform(test1).head().prediction 1.0 >>> lr.setParams("vector") Traceback (most recent call last): diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 263fe2a5bcc41..4e4614b859ac6 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,6 +15,7 @@ # limitations under the License. # +from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures from pyspark.ml.util import keyword_only from pyspark.ml.wrapper import JavaTransformer @@ -24,6 +25,7 @@ @inherit_doc +@ignore_unicode_prefix class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): """ A tokenizer that converts the input string to lowercase and then @@ -32,15 +34,15 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(text="a b c")]).toDF() >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) >>> # Change a parameter. - >>> print tokenizer.setParams(outputCol="tokens").transform(df).head() + >>> tokenizer.setParams(outputCol="tokens").transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Temporarily modify a parameter. - >>> print tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() + >>> tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Must use keyword arguments to specify params. >>> tokenizer.setParams("text") @@ -79,13 +81,13 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(words=["a", "b", "c"])]).toDF() >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") - >>> print hashingTF.transform(df).head().features - (10,[7,8,9],[1.0,1.0,1.0]) - >>> print hashingTF.setParams(outputCol="freqs").transform(df).head().freqs - (10,[7,8,9],[1.0,1.0,1.0]) + >>> hashingTF.transform(df).head().features + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) + >>> hashingTF.setParams(outputCol="freqs").transform(df).head().freqs + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} - >>> print hashingTF.transform(df, params).head().vector - (5,[2,3,4],[1.0,1.0,1.0]) + >>> hashingTF.transform(df, params).head().vector + SparseVector(5, {2: 1.0, 3: 1.0, 4: 1.0}) """ _java_class = "org.apache.spark.ml.feature.HashingTF" diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 5c62620562a84..9fccb65675185 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -63,8 +63,8 @@ def params(self): uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ - return filter(lambda attr: isinstance(attr, Param), - [getattr(self, x) for x in dir(self) if x != "params"]) + return list(filter(lambda attr: isinstance(attr, Param), + [getattr(self, x) for x in dir(self) if x != "params"])) def _explain(self, param): """ @@ -185,7 +185,7 @@ def _set(self, **kwargs): """ Sets user-supplied params. """ - for param, value in kwargs.iteritems(): + for param, value in kwargs.items(): self.paramMap[getattr(self, param)] = value return self @@ -193,6 +193,6 @@ def _setDefault(self, **kwargs): """ Sets default params. """ - for param, value in kwargs.iteritems(): + for param, value in kwargs.items(): self.defaultParamMap[getattr(self, param)] = value return self diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 55f422497672f..6a3192465d66d 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + header = """# # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -82,9 +84,9 @@ def get$Name(self): .replace("$defaultValueStr", str(defaultValueStr)) if __name__ == "__main__": - print header - print "\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n" - print "from pyspark.ml.param import Param, Params\n\n" + print(header) + print("\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n") + print("from pyspark.ml.param import Param, Params\n\n") shared = [ ("maxIter", "max number of iterations", None), ("regParam", "regularization constant", None), @@ -97,4 +99,4 @@ def get$Name(self): code = [] for name, doc, defaultValueStr in shared: code.append(_gen_param_code(name, doc, defaultValueStr)) - print "\n\n\n".join(code) + print("\n\n\n".join(code)) diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index f2ef573fe9f6f..07507b2ad0d05 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -18,6 +18,7 @@ """ Python bindings for MLlib. """ +from __future__ import absolute_import # MLlib currently needs NumPy 1.4+, so complain if lower @@ -29,7 +30,9 @@ 'recommendation', 'regression', 'stat', 'tree', 'util'] import sys -import rand as random -random.__name__ = 'random' -random.RandomRDDs.__module__ = __name__ + '.random' -sys.modules[__name__ + '.random'] = random +from . import rand as random +modname = __name__ + '.random' +random.__name__ = modname +random.RandomRDDs.__module__ = modname +sys.modules[modname] = random +del modname, sys diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 2466e8ac43458..eda0b60f8b1e7 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -510,9 +510,10 @@ def save(self, sc, path): def load(cls, sc, path): java_model = sc._jvm.org.apache.spark.mllib.classification.NaiveBayesModel.load( sc._jsc.sc(), path) - py_labels = _java2py(sc, java_model.labels()) - py_pi = _java2py(sc, java_model.pi()) - py_theta = _java2py(sc, java_model.theta()) + # Can not unpickle array.array from Pyrolite in Python3 with "bytes" + py_labels = _java2py(sc, java_model.labels(), "latin1") + py_pi = _java2py(sc, java_model.pi(), "latin1") + py_theta = _java2py(sc, java_model.theta(), "latin1") return NaiveBayesModel(py_labels, py_pi, numpy.array(py_theta)) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 464f49aeee3cd..abbb7cf60eece 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,6 +15,12 @@ # limitations under the License. # +import sys +import array as pyarray + +if sys.version > '3': + xrange = range + from numpy import array from pyspark import RDD @@ -55,8 +61,8 @@ class KMeansModel(Saveable, Loader): True >>> model.predict(sparse_data[2]) == model.predict(sparse_data[3]) True - >>> type(model.clusterCenters) - + >>> isinstance(model.clusterCenters, list) + True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) @@ -90,7 +96,7 @@ def predict(self, x): return best def save(self, sc, path): - java_centers = _py2java(sc, map(_convert_to_vector, self.centers)) + java_centers = _py2java(sc, [_convert_to_vector(c) for c in self.centers]) java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel(java_centers) java_model.save(sc._jsc.sc(), path) @@ -133,7 +139,7 @@ class GaussianMixtureModel(object): ... 5.7048, 4.6567, 5.5026, ... 4.5605, 5.2043, 6.2734]).reshape(5, 3)) >>> model = GaussianMixture.train(clusterdata_2, 2, convergenceTol=0.0001, - ... maxIterations=150, seed=10) + ... maxIterations=150, seed=10) >>> labels = model.predict(clusterdata_2).collect() >>> labels[0]==labels[1]==labels[2] True @@ -168,8 +174,8 @@ def predictSoft(self, x): if isinstance(x, RDD): means, sigmas = zip(*[(g.mu, g.sigma) for g in self.gaussians]) membership_matrix = callMLlibFunc("predictSoftGMM", x.map(_convert_to_vector), - self.weights, means, sigmas) - return membership_matrix + _convert_to_vector(self.weights), means, sigmas) + return membership_matrix.map(lambda x: pyarray.array('d', x)) class GaussianMixture(object): diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index a539d2f2846f9..ba6058978880a 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -15,6 +15,11 @@ # limitations under the License. # +import sys +if sys.version >= '3': + long = int + unicode = str + import py4j.protocol from py4j.protocol import Py4JJavaError from py4j.java_gateway import JavaObject @@ -36,7 +41,7 @@ def _new_smart_decode(obj): if isinstance(obj, float): - s = unicode(obj) + s = str(obj) return _float_str_mapping.get(s, s) return _old_smart_decode(obj) @@ -74,15 +79,15 @@ def _py2java(sc, obj): obj = ListConverter().convert([_py2java(sc, x) for x in obj], sc._gateway._gateway_client) elif isinstance(obj, JavaObject): pass - elif isinstance(obj, (int, long, float, bool, basestring)): + elif isinstance(obj, (int, long, float, bool, bytes, unicode)): pass else: - bytes = bytearray(PickleSerializer().dumps(obj)) - obj = sc._jvm.SerDe.loads(bytes) + data = bytearray(PickleSerializer().dumps(obj)) + obj = sc._jvm.SerDe.loads(data) return obj -def _java2py(sc, r): +def _java2py(sc, r, encoding="bytes"): if isinstance(r, JavaObject): clsName = r.getClass().getSimpleName() # convert RDD into JavaRDD @@ -102,8 +107,8 @@ def _java2py(sc, r): except Py4JJavaError: pass # not pickable - if isinstance(r, bytearray): - r = PickleSerializer().loads(str(r)) + if isinstance(r, (bytearray, bytes)): + r = PickleSerializer().loads(bytes(r), encoding=encoding) return r diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 8be819aceec24..1140539a24e95 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -23,12 +23,17 @@ import sys import warnings import random +import binascii +if sys.version >= '3': + basestring = str + unicode = str from py4j.protocol import Py4JJavaError -from pyspark import RDD, SparkContext +from pyspark import SparkContext +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper -from pyspark.mllib.linalg import Vectors, Vector, _convert_to_vector +from pyspark.mllib.linalg import Vectors, _convert_to_vector __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] @@ -206,7 +211,7 @@ class HashingTF(object): >>> htf = HashingTF(100) >>> doc = "a a b b c d".split(" ") >>> htf.transform(doc) - SparseVector(100, {1: 1.0, 14: 1.0, 31: 2.0, 44: 2.0}) + SparseVector(100, {...}) """ def __init__(self, numFeatures=1 << 20): """ @@ -360,6 +365,7 @@ def getVectors(self): return self.call("getVectors") +@ignore_unicode_prefix class Word2Vec(object): """ Word2Vec creates vector representation of words in a text corpus. @@ -382,7 +388,7 @@ class Word2Vec(object): >>> sentence = "a b " * 100 + "a c " * 10 >>> localDoc = [sentence, sentence] >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) - >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> model = Word2Vec().setVectorSize(10).setSeed(42).fit(doc) >>> syms = model.findSynonyms("a", 2) >>> [s[0] for s in syms] @@ -400,7 +406,7 @@ def __init__(self): self.learningRate = 0.025 self.numPartitions = 1 self.numIterations = 1 - self.seed = random.randint(0, sys.maxint) + self.seed = random.randint(0, sys.maxsize) self.minCount = 5 def setVectorSize(self, vectorSize): @@ -459,7 +465,7 @@ def fit(self, data): raise TypeError("data should be an RDD of list of string") jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), float(self.learningRate), int(self.numPartitions), - int(self.numIterations), long(self.seed), + int(self.numIterations), int(self.seed), int(self.minCount)) return Word2VecModel(jmodel) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index 3aa6d79d7093c..628ccc01cf3cc 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -16,12 +16,14 @@ # from pyspark import SparkContext +from pyspark.rdd import ignore_unicode_prefix from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc __all__ = ['FPGrowth', 'FPGrowthModel'] @inherit_doc +@ignore_unicode_prefix class FPGrowthModel(JavaModelWrapper): """ diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index a80320c52d1d0..38b3aa3ad460e 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -25,7 +25,13 @@ import sys import array -import copy_reg + +if sys.version >= '3': + basestring = str + xrange = range + import copyreg as copy_reg +else: + import copy_reg import numpy as np @@ -57,7 +63,7 @@ def fast_pickle_array(ar): def _convert_to_vector(l): if isinstance(l, Vector): return l - elif type(l) in (array.array, np.array, np.ndarray, list, tuple): + elif type(l) in (array.array, np.array, np.ndarray, list, tuple, xrange): return DenseVector(l) elif _have_scipy and scipy.sparse.issparse(l): assert l.shape[1] == 1, "Expected column vector" @@ -88,7 +94,7 @@ def _vector_size(v): """ if isinstance(v, Vector): return len(v) - elif type(v) in (array.array, list, tuple): + elif type(v) in (array.array, list, tuple, xrange): return len(v) elif type(v) == np.ndarray: if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): @@ -193,7 +199,7 @@ class DenseVector(Vector): DenseVector([1.0, 0.0]) """ def __init__(self, ar): - if isinstance(ar, basestring): + if isinstance(ar, bytes): ar = np.frombuffer(ar, dtype=np.float64) elif not isinstance(ar, np.ndarray): ar = np.array(ar, dtype=np.float64) @@ -321,11 +327,13 @@ def func(self, other): __sub__ = _delegate("__sub__") __mul__ = _delegate("__mul__") __div__ = _delegate("__div__") + __truediv__ = _delegate("__truediv__") __mod__ = _delegate("__mod__") __radd__ = _delegate("__radd__") __rsub__ = _delegate("__rsub__") __rmul__ = _delegate("__rmul__") __rdiv__ = _delegate("__rdiv__") + __rtruediv__ = _delegate("__rtruediv__") __rmod__ = _delegate("__rmod__") @@ -344,12 +352,12 @@ def __init__(self, size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print SparseVector(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> SparseVector(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ self.size = int(size) assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" @@ -361,8 +369,8 @@ def __init__(self, size, *args): self.indices = np.array([p[0] for p in pairs], dtype=np.int32) self.values = np.array([p[1] for p in pairs], dtype=np.float64) else: - if isinstance(args[0], basestring): - assert isinstance(args[1], str), "values should be string too" + if isinstance(args[0], bytes): + assert isinstance(args[1], bytes), "values should be string too" if args[0]: self.indices = np.frombuffer(args[0], np.int32) self.values = np.frombuffer(args[1], np.float64) @@ -591,12 +599,12 @@ def sparse(size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print Vectors.sparse(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> Vectors.sparse(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ return SparseVector(size, *args) @@ -645,7 +653,7 @@ def _convert_to_array(array_like, dtype): """ Convert Matrix attributes which are array-like or buffer to array. """ - if isinstance(array_like, basestring): + if isinstance(array_like, bytes): return np.frombuffer(array_like, dtype=dtype) return np.asarray(array_like, dtype=dtype) @@ -677,7 +685,7 @@ def toArray(self): def toSparse(self): """Convert to SparseMatrix""" indices = np.nonzero(self.values)[0] - colCounts = np.bincount(indices / self.numRows) + colCounts = np.bincount(indices // self.numRows) colPtrs = np.cumsum(np.hstack( (0, colCounts, np.zeros(self.numCols - colCounts.size)))) values = self.values[indices] diff --git a/python/pyspark/mllib/rand.py b/python/pyspark/mllib/rand.py index 20ee9d78bf5b0..06fbc0eb6aef0 100644 --- a/python/pyspark/mllib/rand.py +++ b/python/pyspark/mllib/rand.py @@ -88,10 +88,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): :param seed: Random seed (default: a random long integer). :return: RDD of float comprised of i.i.d. samples ~ N(0.0, 1.0). - >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - 0.0) < 0.1 True >>> abs(stats.stdev() - 1.0) < 0.1 @@ -118,10 +118,10 @@ def logNormalRDD(sc, mean, std, size, numPartitions=None, seed=None): >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2L) + >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> from math import sqrt @@ -145,10 +145,10 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Pois(mean). >>> mean = 100.0 - >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -171,10 +171,10 @@ def exponentialRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Exp(mean). >>> mean = 2.0 - >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -202,10 +202,10 @@ def gammaRDD(sc, shape, scale, size, numPartitions=None, seed=None): >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2L) + >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> abs(stats.stdev() - expStd) < 0.5 @@ -254,7 +254,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): :return: RDD of Vector with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. >>> import numpy as np - >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - 0.0) < 0.1 @@ -286,8 +286,8 @@ def logNormalVectorRDD(sc, mean, std, numRows, numCols, numPartitions=None, seed >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> mat = np.matrix(RandomRDDs.logNormalVectorRDD(sc, mean, std, \ - 100, 100, seed=1L).collect()) + >>> m = RandomRDDs.logNormalVectorRDD(sc, mean, std, 100, 100, seed=1).collect() + >>> mat = np.matrix(m) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 @@ -315,7 +315,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> import numpy as np >>> mean = 100.0 - >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -345,7 +345,7 @@ def exponentialVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=No >>> import numpy as np >>> mean = 0.5 - >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -380,8 +380,7 @@ def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed= >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, \ - 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index c5c4c13dae105..80e0a356bb78a 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -15,6 +15,7 @@ # limitations under the License. # +import array from collections import namedtuple from pyspark import SparkContext @@ -104,14 +105,14 @@ def predictAll(self, user_product): assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" first = user_product.first() assert len(first) == 2, "user_product should be RDD of (user, product)" - user_product = user_product.map(lambda (u, p): (int(u), int(p))) + user_product = user_product.map(lambda u_p: (int(u_p[0]), int(u_p[1]))) return self.call("predict", user_product) def userFeatures(self): - return self.call("getUserFeatures") + return self.call("getUserFeatures").mapValues(lambda v: array.array('d', v)) def productFeatures(self): - return self.call("getProductFeatures") + return self.call("getProductFeatures").mapValues(lambda v: array.array('d', v)) @classmethod def load(cls, sc, path): diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index 1d83e9d483f8e..b475be4b4d953 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark import RDD +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import Matrix, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -38,7 +38,7 @@ def variance(self): return self.call("variance").toArray() def count(self): - return self.call("count") + return int(self.call("count")) def numNonzeros(self): return self.call("numNonzeros").toArray() @@ -78,7 +78,7 @@ def colStats(rdd): >>> cStats.variance() array([ 4., 13., 0., 25.]) >>> cStats.count() - 3L + 3 >>> cStats.numNonzeros() array([ 3., 2., 0., 3.]) >>> cStats.max() @@ -124,20 +124,20 @@ def corr(x, y=None, method=None): >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) >>> pearsonCorr = Statistics.corr(rdd) - >>> print str(pearsonCorr).replace('nan', 'NaN') + >>> print(str(pearsonCorr).replace('nan', 'NaN')) [[ 1. 0.05564149 NaN 0.40047142] [ 0.05564149 1. NaN 0.91359586] [ NaN NaN 1. NaN] [ 0.40047142 0.91359586 NaN 1. ]] >>> spearmanCorr = Statistics.corr(rdd, method="spearman") - >>> print str(spearmanCorr).replace('nan', 'NaN') + >>> print(str(spearmanCorr).replace('nan', 'NaN')) [[ 1. 0.10540926 NaN 0.4 ] [ 0.10540926 1. NaN 0.9486833 ] [ NaN NaN 1. NaN] [ 0.4 0.9486833 NaN 1. ]] >>> try: ... Statistics.corr(rdd, "spearman") - ... print "Method name as second argument without 'method=' shouldn't be allowed." + ... print("Method name as second argument without 'method=' shouldn't be allowed.") ... except TypeError: ... pass """ @@ -153,6 +153,7 @@ def corr(x, y=None, method=None): return callMLlibFunc("corr", x.map(float), y.map(float), method) @staticmethod + @ignore_unicode_prefix def chiSqTest(observed, expected=None): """ .. note:: Experimental @@ -188,11 +189,11 @@ def chiSqTest(observed, expected=None): >>> from pyspark.mllib.linalg import Vectors, Matrices >>> observed = Vectors.dense([4, 6, 5]) >>> pearson = Statistics.chiSqTest(observed) - >>> print pearson.statistic + >>> print(pearson.statistic) 0.4 >>> pearson.degreesOfFreedom 2 - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.8187 >>> pearson.method u'pearson' @@ -202,12 +203,12 @@ def chiSqTest(observed, expected=None): >>> observed = Vectors.dense([21, 38, 43, 80]) >>> expected = Vectors.dense([3, 5, 7, 20]) >>> pearson = Statistics.chiSqTest(observed, expected) - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.0027 >>> 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)) - >>> print round(chi.statistic, 4) + >>> print(round(chi.statistic, 4)) 21.9958 >>> data = [LabeledPoint(0.0, Vectors.dense([0.5, 10.0])), @@ -218,9 +219,9 @@ def chiSqTest(observed, expected=None): ... LabeledPoint(1.0, Vectors.dense([3.5, 40.0])),] >>> rdd = sc.parallelize(data, 4) >>> chi = Statistics.chiSqTest(rdd) - >>> print chi[0].statistic + >>> print(chi[0].statistic) 0.75 - >>> print chi[1].statistic + >>> print(chi[1].statistic) 1.5 """ if isinstance(observed, RDD): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8eaddcf8b9b5e..c6ed5acd1770e 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -72,11 +72,11 @@ class VectorTests(PySparkTestCase): def _test_serialize(self, v): self.assertEqual(v, ser.loads(ser.dumps(v))) jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) - nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) + nv = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvec))) self.assertEqual(v, nv) vs = [v] * 100 jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) - nvs = ser.loads(str(self.sc._jvm.SerDe.dumps(jvecs))) + nvs = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvecs))) self.assertEqual(vs, nvs) def test_serialize(self): @@ -412,11 +412,11 @@ def test_col_norms(self): self.assertEqual(10, len(summary.normL1())) self.assertEqual(10, len(summary.normL2())) - data2 = self.sc.parallelize(xrange(10)).map(lambda x: Vectors.dense(x)) + data2 = self.sc.parallelize(range(10)).map(lambda x: Vectors.dense(x)) summary2 = Statistics.colStats(data2) self.assertEqual(array([45.0]), summary2.normL1()) import math - expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, xrange(10)))) + expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, range(10)))) self.assertTrue(math.fabs(summary2.normL2()[0] - expectedNormL2) < 1e-14) @@ -438,11 +438,11 @@ def test_serialization(self): def test_infer_schema(self): sqlCtx = SQLContext(self.sc) rdd = self.sc.parallelize([LabeledPoint(1.0, self.dv1), LabeledPoint(0.0, self.sv1)]) - srdd = sqlCtx.inferSchema(rdd) - schema = srdd.schema + df = rdd.toDF() + schema = df.schema field = [f for f in schema.fields if f.name == "features"][0] self.assertEqual(field.dataType, self.udt) - vectors = srdd.map(lambda p: p.features).collect() + vectors = df.map(lambda p: p.features).collect() self.assertEqual(len(vectors), 2) for v in vectors: if isinstance(v, SparseVector): @@ -695,7 +695,7 @@ def test_right_number_of_results(self): class SerDeTest(PySparkTestCase): def test_to_java_object_rdd(self): # SPARK-6660 - data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0L) + data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0) self.assertEqual(_to_java_object_rdd(data).count(), 10) @@ -771,7 +771,7 @@ def test_model_transform(self): if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index a7a4d2aaf855b..0fe6e4fabe43a 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -163,14 +163,16 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, ... LabeledPoint(1.0, [3.0]) ... ] >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) - >>> print model, # it already has newline + >>> print(model) DecisionTreeModel classifier of depth 1 with 3 nodes - >>> print model.toDebugString(), # it already has newline + + >>> print(model.toDebugString()) DecisionTreeModel classifier of depth 1 with 3 nodes If (feature 0 <= 0.0) Predict: 0.0 Else (feature 0 > 0.0) Predict: 1.0 + >>> model.predict(array([1.0])) 1.0 >>> model.predict(array([0.0])) @@ -318,9 +320,10 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, 3 >>> model.totalNumNodes() 7 - >>> print model, + >>> print(model) TreeEnsembleModel classifier with 3 trees - >>> print model.toDebugString(), + + >>> print(model.toDebugString()) TreeEnsembleModel classifier with 3 trees Tree 0: @@ -335,6 +338,7 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, Predict: 0.0 Else (feature 0 > 1.0) Predict: 1.0 + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) @@ -483,8 +487,9 @@ def trainClassifier(cls, data, categoricalFeaturesInfo, 100 >>> model.totalNumNodes() 300 - >>> print model, # it already has newline + >>> print(model) # it already has newline TreeEnsembleModel classifier with 100 trees + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index c5c3468eb95e9..16a90db146ef0 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -15,10 +15,14 @@ # limitations under the License. # +import sys import numpy as np import warnings -from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper, inherit_doc +if sys.version > '3': + xrange = range + +from pyspark.mllib.common import callMLlibFunc, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector @@ -94,22 +98,16 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None >>> from pyspark.mllib.util import MLUtils >>> from pyspark.mllib.regression import LabeledPoint >>> tempFile = NamedTemporaryFile(delete=True) - >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") + >>> _ = tempFile.write(b"+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() >>> tempFile.close() - >>> type(examples[0]) == LabeledPoint - True - >>> print examples[0] - (1.0,(6,[0,2,4],[1.0,2.0,3.0])) - >>> type(examples[1]) == LabeledPoint - True - >>> print examples[1] - (-1.0,(6,[],[])) - >>> type(examples[2]) == LabeledPoint - True - >>> print examples[2] - (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) + >>> examples[0] + LabeledPoint(1.0, (6,[0,2,4],[1.0,2.0,3.0])) + >>> examples[1] + LabeledPoint(-1.0, (6,[],[])) + >>> examples[2] + LabeledPoint(-1.0, (6,[1,3,5],[4.0,5.0,6.0])) """ from pyspark.mllib.regression import LabeledPoint if multiclass is not None: diff --git a/python/pyspark/profiler.py b/python/pyspark/profiler.py index 4408996db0790..d18daaabfcb3c 100644 --- a/python/pyspark/profiler.py +++ b/python/pyspark/profiler.py @@ -84,11 +84,11 @@ class Profiler(object): >>> from pyspark import BasicProfiler >>> class MyCustomProfiler(BasicProfiler): ... def show(self, id): - ... print "My custom profiles for RDD:%s" % id + ... print("My custom profiles for RDD:%s" % id) ... >>> conf = SparkConf().set("spark.python.profile", "true") >>> sc = SparkContext('local', 'test', conf=conf, profiler_cls=MyCustomProfiler) - >>> sc.parallelize(list(range(1000))).map(lambda x: 2 * x).take(10) + >>> sc.parallelize(range(1000)).map(lambda x: 2 * x).take(10) [0, 2, 4, 6, 8, 10, 12, 14, 16, 18] >>> sc.show_profiles() My custom profiles for RDD:1 @@ -111,9 +111,9 @@ def show(self, id): """ Print the profile stats to stdout, id is the RDD id """ stats = self.stats() if stats: - print "=" * 60 - print "Profile of RDD" % id - print "=" * 60 + print("=" * 60) + print("Profile of RDD" % id) + print("=" * 60) stats.sort_stats("time", "cumulative").print_stats() def dump(self, id, path): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 93e658eded9e2..d9cdbb666f92a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -16,21 +16,29 @@ # import copy -from collections import defaultdict -from itertools import chain, ifilter, imap -import operator import sys +import os +import re +import operator import shlex -from subprocess import Popen, PIPE -from tempfile import NamedTemporaryFile -from threading import Thread import warnings import heapq import bisect import random import socket +from subprocess import Popen, PIPE +from tempfile import NamedTemporaryFile +from threading import Thread +from collections import defaultdict +from itertools import chain +from functools import reduce from math import sqrt, log, isinf, isnan, pow, ceil +if sys.version > '3': + basestring = unicode = str +else: + from itertools import imap as map, ifilter as filter + from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer @@ -50,20 +58,21 @@ __all__ = ["RDD"] -# TODO: for Python 3.3+, PYTHONHASHSEED should be reset to disable randomized -# hash for string def portable_hash(x): """ - This function returns consistant hash code for builtin types, especially + This function returns consistent hash code for builtin types, especially for None and tuple with None. - The algrithm is similar to that one used by CPython 2.7 + The algorithm is similar to that one used by CPython 2.7 >>> portable_hash(None) 0 >>> portable_hash((None, 1)) & 0xffffffff 219750521 """ + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + raise Exception("Randomness of hash of string should be disabled via PYTHONHASHSEED") + if x is None: return 0 if isinstance(x, tuple): @@ -71,7 +80,7 @@ def portable_hash(x): for i in x: h ^= portable_hash(i) h *= 1000003 - h &= sys.maxint + h &= sys.maxsize h ^= len(x) if h == -1: h = -2 @@ -123,6 +132,19 @@ def _load_from_socket(port, serializer): sock.close() +def ignore_unicode_prefix(f): + """ + Ignore the 'u' prefix of string in doc tests, to make it works + in both python 2 and 3 + """ + if sys.version >= '3': + # the representation of unicode string in Python 3 does not have prefix 'u', + # so remove the prefix 'u' for doc tests + literal_re = re.compile(r"(\W|^)[uU](['])", re.UNICODE) + f.__doc__ = literal_re.sub(r'\1\2', f.__doc__) + return f + + class Partitioner(object): def __init__(self, numPartitions, partitionFunc): self.numPartitions = numPartitions @@ -251,7 +273,7 @@ def map(self, f, preservesPartitioning=False): [('a', 1), ('b', 1), ('c', 1)] """ def func(_, iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitionsWithIndex(func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): @@ -266,7 +288,7 @@ def flatMap(self, f, preservesPartitioning=False): [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -329,7 +351,7 @@ def filter(self, f): [2, 4] """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def distinct(self, numPartitions=None): @@ -341,7 +363,7 @@ def distinct(self, numPartitions=None): """ return self.map(lambda x: (x, None)) \ .reduceByKey(lambda x, _: x, numPartitions) \ - .map(lambda (x, _): x) + .map(lambda x: x[0]) def sample(self, withReplacement, fraction, seed=None): """ @@ -354,8 +376,8 @@ def sample(self, withReplacement, fraction, seed=None): :param seed: seed for the random number generator >>> rdd = sc.parallelize(range(100), 4) - >>> rdd.sample(False, 0.1, 81).count() - 10 + >>> 6 <= rdd.sample(False, 0.1, 81).count() <= 14 + True """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) @@ -368,12 +390,14 @@ def randomSplit(self, weights, seed=None): :param seed: random seed :return: split RDDs in a list - >>> rdd = sc.parallelize(range(5), 1) + >>> rdd = sc.parallelize(range(500), 1) >>> rdd1, rdd2 = rdd.randomSplit([2, 3], 17) - >>> rdd1.collect() - [1, 3] - >>> rdd2.collect() - [0, 2, 4] + >>> len(rdd1.collect() + rdd2.collect()) + 500 + >>> 150 < rdd1.count() < 250 + True + >>> 250 < rdd2.count() < 350 + True """ s = float(sum(weights)) cweights = [0.0] @@ -416,7 +440,7 @@ def takeSample(self, withReplacement, num, seed=None): rand.shuffle(samples) return samples - maxSampleSize = sys.maxint - int(numStDev * sqrt(sys.maxint)) + maxSampleSize = sys.maxsize - int(numStDev * sqrt(sys.maxsize)) if num > maxSampleSize: raise ValueError( "Sample size cannot be greater than %d." % maxSampleSize) @@ -430,7 +454,7 @@ def takeSample(self, withReplacement, num, seed=None): # See: scala/spark/RDD.scala while len(samples) < num: # TODO: add log warning for when more than one iteration was run - seed = rand.randint(0, sys.maxint) + seed = rand.randint(0, sys.maxsize) samples = self.sample(withReplacement, fraction, seed).collect() rand.shuffle(samples) @@ -507,7 +531,7 @@ def intersection(self, other): """ return self.map(lambda v: (v, None)) \ .cogroup(other.map(lambda v: (v, None))) \ - .filter(lambda (k, vs): all(vs)) \ + .filter(lambda k_vs: all(k_vs[1])) \ .keys() def _reserialize(self, serializer=None): @@ -549,7 +573,7 @@ def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=p def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda k_v: keyfunc(k_v[0]), reverse=(not ascending))) return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True) @@ -579,7 +603,7 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda kv: keyfunc(kv[0]), reverse=(not ascending))) if numPartitions == 1: if self.getNumPartitions() > 1: @@ -594,12 +618,12 @@ def sortPartition(iterator): return self # empty RDD maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner fraction = min(maxSampleSize / max(rddSize, 1), 1.0) - samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() + samples = self.sample(False, fraction, 1).map(lambda kv: kv[0]).collect() samples = sorted(samples, key=keyfunc) # we have numPartitions many parts but one of the them has # an implicit boundary - bounds = [samples[len(samples) * (i + 1) / numPartitions] + bounds = [samples[int(len(samples) * (i + 1) / numPartitions)] for i in range(0, numPartitions - 1)] def rangePartitioner(k): @@ -662,12 +686,13 @@ def groupBy(self, f, numPartitions=None): """ return self.map(lambda x: (f(x), x)).groupByKey(numPartitions) + @ignore_unicode_prefix def pipe(self, command, env={}): """ Return an RDD created by piping elements to a forked external process. >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() - ['1', '2', '', '3'] + [u'1', u'2', u'', u'3'] """ def func(iterator): pipe = Popen( @@ -675,17 +700,18 @@ def func(iterator): def pipe_objs(out): for obj in iterator: - out.write(str(obj).rstrip('\n') + '\n') + s = str(obj).rstrip('\n') + '\n' + out.write(s.encode('utf-8')) out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() - return (x.rstrip('\n') for x in iter(pipe.stdout.readline, '')) + return (x.rstrip(b'\n').decode('utf-8') for x in iter(pipe.stdout.readline, b'')) return self.mapPartitions(func) def foreach(self, f): """ Applies a function to all elements of this RDD. - >>> def f(x): print x + >>> def f(x): print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) """ def processPartition(iterator): @@ -700,7 +726,7 @@ def foreachPartition(self, f): >>> def f(iterator): ... for x in iterator: - ... print x + ... print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreachPartition(f) """ def func(it): @@ -874,7 +900,7 @@ def aggregatePartition(iterator): # aggregation. while numPartitions > scale + numPartitions / scale: numPartitions /= scale - curNumPartitions = numPartitions + curNumPartitions = int(numPartitions) def mapPartition(i, iterator): for obj in iterator: @@ -984,7 +1010,7 @@ def histogram(self, buckets): (('a', 'b', 'c'), [2, 2]) """ - if isinstance(buckets, (int, long)): + if isinstance(buckets, int): if buckets < 1: raise ValueError("number of buckets must be >= 1") @@ -1020,6 +1046,7 @@ def minmax(a, b): raise ValueError("Can not generate buckets with infinite value") # keep them as integer if possible + inc = int(inc) if inc * buckets != maxv - minv: inc = (maxv - minv) * 1.0 / buckets @@ -1137,7 +1164,7 @@ def countPartition(iterator): yield counts def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] += v return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) @@ -1378,8 +1405,8 @@ def saveAsPickleFile(self, path, batchSize=10): >>> tmpFile = NamedTemporaryFile(delete=True) >>> tmpFile.close() >>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3) - >>> sorted(sc.pickleFile(tmpFile.name, 5).collect()) - [1, 2, 'rdd', 'spark'] + >>> sorted(sc.pickleFile(tmpFile.name, 5).map(str).collect()) + ['1', '2', 'rdd', 'spark'] """ if batchSize == 0: ser = AutoBatchedSerializer(PickleSerializer()) @@ -1387,6 +1414,7 @@ def saveAsPickleFile(self, path, batchSize=10): ser = BatchedSerializer(PickleSerializer(), batchSize) self._reserialize(ser)._jrdd.saveAsObjectFile(path) + @ignore_unicode_prefix def saveAsTextFile(self, path, compressionCodecClass=None): """ Save this RDD as a text file, using string representations of elements. @@ -1418,12 +1446,13 @@ def saveAsTextFile(self, path, compressionCodecClass=None): >>> codec = "org.apache.hadoop.io.compress.GzipCodec" >>> sc.parallelize(['foo', 'bar']).saveAsTextFile(tempFile3.name, codec) >>> from fileinput import input, hook_compressed - >>> ''.join(sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed))) - 'bar\\nfoo\\n' + >>> result = sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed)) + >>> b''.join(result).decode('utf-8') + u'bar\\nfoo\\n' """ def func(split, iterator): for x in iterator: - if not isinstance(x, basestring): + if not isinstance(x, (unicode, bytes)): x = unicode(x) if isinstance(x, unicode): x = x.encode("utf-8") @@ -1458,7 +1487,7 @@ def keys(self): >>> m.collect() [1, 3] """ - return self.map(lambda (k, v): k) + return self.map(lambda x: x[0]) def values(self): """ @@ -1468,7 +1497,7 @@ def values(self): >>> m.collect() [2, 4] """ - return self.map(lambda (k, v): v) + return self.map(lambda x: x[1]) def reduceByKey(self, func, numPartitions=None): """ @@ -1507,7 +1536,7 @@ def reducePartition(iterator): yield m def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] = func(m1[k], v) if k in m1 else v return m1 return self.mapPartitions(reducePartition).reduce(mergeMaps) @@ -1604,8 +1633,8 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash): >>> pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x)) >>> sets = pairs.partitionBy(2).glom().collect() - >>> set(sets[0]).intersection(set(sets[1])) - set([]) + >>> len(set(sets[0]).intersection(set(sets[1]))) + 0 """ if numPartitions is None: numPartitions = self._defaultReducePartitions() @@ -1637,22 +1666,22 @@ def add_shuffle_key(split, iterator): if (c % 1000 == 0 and get_used_memory() > limit or c > batch): n, size = len(buckets), 0 - for split in buckets.keys(): + for split in list(buckets.keys()): yield pack_long(split) d = outputSerializer.dumps(buckets[split]) del buckets[split] yield d size += len(d) - avg = (size / n) >> 20 + avg = int(size / n) >> 20 # let 1M < avg < 10M if avg < 1: batch *= 1.5 elif avg > 10: - batch = max(batch / 1.5, 1) + batch = max(int(batch / 1.5), 1) c = 0 - for split, items in buckets.iteritems(): + for split, items in buckets.items(): yield pack_long(split) yield outputSerializer.dumps(items) @@ -1707,7 +1736,7 @@ def combineLocally(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ if spill else InMemoryMerger(agg) merger.mergeValues(iterator) - return merger.iteritems() + return merger.items() locally_combined = self.mapPartitions(combineLocally, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) @@ -1716,7 +1745,7 @@ def _mergeCombiners(iterator): merger = ExternalMerger(agg, memory, serializer) \ if spill else InMemoryMerger(agg) merger.mergeCombiners(iterator) - return merger.iteritems() + return merger.items() return shuffled.mapPartitions(_mergeCombiners, preservesPartitioning=True) @@ -1745,7 +1774,7 @@ def foldByKey(self, zeroValue, func, numPartitions=None): >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> from operator import add - >>> rdd.foldByKey(0, add).collect() + >>> sorted(rdd.foldByKey(0, add).collect()) [('a', 2), ('b', 1)] """ def createZero(): @@ -1769,10 +1798,10 @@ def groupByKey(self, numPartitions=None): sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. - >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.groupByKey().mapValues(len).collect()) + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(rdd.groupByKey().mapValues(len).collect()) [('a', 2), ('b', 1)] - >>> sorted(x.groupByKey().mapValues(list).collect()) + >>> sorted(rdd.groupByKey().mapValues(list).collect()) [('a', [1, 1]), ('b', [1])] """ def createCombiner(x): @@ -1795,7 +1824,7 @@ def combine(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ if spill else InMemoryMerger(agg) merger.mergeValues(iterator) - return merger.iteritems() + return merger.items() locally_combined = self.mapPartitions(combine, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) @@ -1804,7 +1833,7 @@ def groupByKey(it): merger = ExternalGroupBy(agg, memory, serializer)\ if spill else InMemoryMerger(agg) merger.mergeCombiners(it) - return merger.iteritems() + return merger.items() return shuffled.mapPartitions(groupByKey, True).mapValues(ResultIterable) @@ -1819,7 +1848,7 @@ def flatMapValues(self, f): >>> x.flatMapValues(f).collect() [('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')] """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def mapValues(self, f): @@ -1833,7 +1862,7 @@ def mapValues(self, f): >>> x.mapValues(f).collect() [('a', 3), ('b', 1)] """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def groupWith(self, other, *others): @@ -1844,8 +1873,7 @@ def groupWith(self, other, *others): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) >>> z = sc.parallelize([("b", 42)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1]), list(y[2]), list(y[3])))), \ - sorted(list(w.groupWith(x, y, z).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(w.groupWith(x, y, z).collect()))] [('a', ([5], [1], [2], [])), ('b', ([6], [4], [], [42]))] """ @@ -1860,7 +1888,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(x.cogroup(y).collect()))] [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup((self, other), numPartitions) @@ -1896,8 +1924,9 @@ def subtractByKey(self, other, numPartitions=None): >>> sorted(x.subtractByKey(y).collect()) [('b', 4), ('b', 5)] """ - def filter_func((key, vals)): - return vals[0] and not vals[1] + def filter_func(pair): + key, (val1, val2) = pair + return val1 and not val2 return self.cogroup(other, numPartitions).filter(filter_func).flatMapValues(lambda x: x[0]) def subtract(self, other, numPartitions=None): @@ -1919,8 +1948,8 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) - [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] + >>> [(x, list(map(list, y))) for x, y in sorted(x.cogroup(y).collect())] + [(0, [[0], [0]]), (1, [[1], [1]]), (2, [[], [2]]), (3, [[], [3]]), (4, [[2], [4]])] """ return self.map(lambda x: (f(x), x)) @@ -2049,17 +2078,18 @@ def name(self): """ Return the name of this RDD. """ - name_ = self._jrdd.name() - if name_: - return name_.encode('utf-8') + n = self._jrdd.name() + if n: + return n + @ignore_unicode_prefix def setName(self, name): """ Assign a name to this RDD. - >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1 = sc.parallelize([1, 2]) >>> rdd1.setName('RDD1').name() - 'RDD1' + u'RDD1' """ self._jrdd.setName(name) return self @@ -2121,7 +2151,7 @@ def lookup(self, key): >>> sorted.lookup(1024) [] """ - values = self.filter(lambda (k, v): k == key).values() + values = self.filter(lambda kv: kv[0] == key).values() if self.partitioner is not None: return self.ctx.runJob(values, lambda x: x, [self.partitioner(key)], False) @@ -2159,7 +2189,7 @@ def sumApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) + >>> r = sum(range(1000)) >>> (rdd.sumApprox(1000) - r) / r < 0.05 True """ @@ -2176,7 +2206,7 @@ def meanApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) / 1000.0 + >>> r = sum(range(1000)) / 1000.0 >>> (rdd.meanApprox(1000) - r) / r < 0.05 True """ @@ -2201,10 +2231,10 @@ def countApproxDistinct(self, relativeSD=0.05): It must be greater than 0.000017. >>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct() - >>> 950 < n < 1050 + >>> 900 < n < 1100 True >>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct() - >>> 18 < n < 22 + >>> 16 < n < 24 True """ if relativeSD < 0.000017: @@ -2223,8 +2253,7 @@ def toLocalIterator(self): >>> [x for x in rdd.toLocalIterator()] [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """ - partitions = xrange(self.getNumPartitions()) - for partition in partitions: + for partition in range(self.getNumPartitions()): rows = self.context.runJob(self, lambda x: x, [partition]) for row in rows: yield row diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 459e1427803cb..fe8f87324804b 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -23,7 +23,7 @@ class RDDSamplerBase(object): def __init__(self, withReplacement, seed=None): - self._seed = seed if seed is not None else random.randint(0, sys.maxint) + self._seed = seed if seed is not None else random.randint(0, sys.maxsize) self._withReplacement = withReplacement self._random = None @@ -31,7 +31,7 @@ def initRandomGenerator(self, split): self._random = random.Random(self._seed ^ split) # mixing because the initial seeds are close to each other - for _ in xrange(10): + for _ in range(10): self._random.randint(0, 1) def getUniformSample(self): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 4afa82f4b2973..d8cdcda3a3783 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -49,16 +49,24 @@ >>> sc.stop() """ -import cPickle -from itertools import chain, izip, product +import sys +from itertools import chain, product import marshal import struct -import sys import types import collections import zlib import itertools +if sys.version < '3': + import cPickle as pickle + protocol = 2 + from itertools import izip as zip +else: + import pickle + protocol = 3 + xrange = range + from pyspark import cloudpickle @@ -97,7 +105,7 @@ def _load_stream_without_unbatching(self, stream): # subclasses should override __eq__ as appropriate. def __eq__(self, other): - return isinstance(other, self.__class__) + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not self.__eq__(other) @@ -212,10 +220,6 @@ def load_stream(self, stream): def _load_stream_without_unbatching(self, stream): return self.serializer.load_stream(stream) - def __eq__(self, other): - return (isinstance(other, BatchedSerializer) and - other.serializer == self.serializer and other.batchSize == self.batchSize) - def __repr__(self): return "BatchedSerializer(%s, %d)" % (str(self.serializer), self.batchSize) @@ -233,14 +237,14 @@ def __init__(self, serializer, batchSize=10): def _batched(self, iterator): n = self.batchSize for key, values in iterator: - for i in xrange(0, len(values), n): + for i in range(0, len(values), n): yield key, values[i:i + n] def load_stream(self, stream): return self.serializer.load_stream(stream) def __repr__(self): - return "FlattenedValuesSerializer(%d)" % self.batchSize + return "FlattenedValuesSerializer(%s, %d)" % (self.serializer, self.batchSize) class AutoBatchedSerializer(BatchedSerializer): @@ -270,12 +274,8 @@ def dump_stream(self, iterator, stream): elif size > best * 10 and batch > 1: batch /= 2 - def __eq__(self, other): - return (isinstance(other, AutoBatchedSerializer) and - other.serializer == self.serializer and other.bestSize == self.bestSize) - def __repr__(self): - return "AutoBatchedSerializer(%s)" % str(self.serializer) + return "AutoBatchedSerializer(%s)" % self.serializer class CartesianDeserializer(FramedSerializer): @@ -285,6 +285,7 @@ class CartesianDeserializer(FramedSerializer): """ def __init__(self, key_ser, val_ser): + FramedSerializer.__init__(self) self.key_ser = key_ser self.val_ser = val_ser @@ -293,7 +294,7 @@ def prepare_keys_values(self, stream): val_stream = self.val_ser._load_stream_without_unbatching(stream) key_is_batched = isinstance(self.key_ser, BatchedSerializer) val_is_batched = isinstance(self.val_ser, BatchedSerializer) - for (keys, vals) in izip(key_stream, val_stream): + for (keys, vals) in zip(key_stream, val_stream): keys = keys if key_is_batched else [keys] vals = vals if val_is_batched else [vals] yield (keys, vals) @@ -303,10 +304,6 @@ def load_stream(self, stream): for pair in product(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, CartesianDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "CartesianDeserializer(%s, %s)" % \ (str(self.key_ser), str(self.val_ser)) @@ -318,22 +315,14 @@ class PairDeserializer(CartesianDeserializer): Deserializes the JavaRDD zip() of two PythonRDDs. """ - def __init__(self, key_ser, val_ser): - self.key_ser = key_ser - self.val_ser = val_ser - def load_stream(self, stream): for (keys, vals) in self.prepare_keys_values(stream): if len(keys) != len(vals): raise ValueError("Can not deserialize RDD with different number of items" " in pair: (%d, %d)" % (len(keys), len(vals))) - for pair in izip(keys, vals): + for pair in zip(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, PairDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "PairDeserializer(%s, %s)" % (str(self.key_ser), str(self.val_ser)) @@ -382,8 +371,8 @@ def _hijack_namedtuple(): global _old_namedtuple # or it will put in closure def _copy_func(f): - return types.FunctionType(f.func_code, f.func_globals, f.func_name, - f.func_defaults, f.func_closure) + return types.FunctionType(f.__code__, f.__globals__, f.__name__, + f.__defaults__, f.__closure__) _old_namedtuple = _copy_func(collections.namedtuple) @@ -392,15 +381,15 @@ def namedtuple(*args, **kwargs): return _hack_namedtuple(cls) # replace namedtuple with new one - collections.namedtuple.func_globals["_old_namedtuple"] = _old_namedtuple - collections.namedtuple.func_globals["_hack_namedtuple"] = _hack_namedtuple - collections.namedtuple.func_code = namedtuple.func_code + collections.namedtuple.__globals__["_old_namedtuple"] = _old_namedtuple + collections.namedtuple.__globals__["_hack_namedtuple"] = _hack_namedtuple + collections.namedtuple.__code__ = namedtuple.__code__ collections.namedtuple.__hijack = 1 # hack the cls already generated by namedtuple # those created in other module can be pickled as normal, # so only hack those in __main__ module - for n, o in sys.modules["__main__"].__dict__.iteritems(): + for n, o in sys.modules["__main__"].__dict__.items(): if (type(o) is type and o.__base__ is tuple and hasattr(o, "_fields") and "__reduce__" not in o.__dict__): @@ -413,7 +402,7 @@ def namedtuple(*args, **kwargs): class PickleSerializer(FramedSerializer): """ - Serializes objects using Python's cPickle serializer: + Serializes objects using Python's pickle serializer: http://docs.python.org/2/library/pickle.html @@ -422,10 +411,14 @@ class PickleSerializer(FramedSerializer): """ def dumps(self, obj): - return cPickle.dumps(obj, 2) + return pickle.dumps(obj, protocol) - def loads(self, obj): - return cPickle.loads(obj) + if sys.version >= '3': + def loads(self, obj, encoding="bytes"): + return pickle.loads(obj, encoding=encoding) + else: + def loads(self, obj, encoding=None): + return pickle.loads(obj) class CloudPickleSerializer(PickleSerializer): @@ -454,7 +447,7 @@ def loads(self, obj): class AutoSerializer(FramedSerializer): """ - Choose marshal or cPickle as serialization protocol automatically + Choose marshal or pickle as serialization protocol automatically """ def __init__(self): @@ -463,19 +456,19 @@ def __init__(self): def dumps(self, obj): if self._type is not None: - return 'P' + cPickle.dumps(obj, -1) + return b'P' + pickle.dumps(obj, -1) try: - return 'M' + marshal.dumps(obj) + return b'M' + marshal.dumps(obj) except Exception: - self._type = 'P' - return 'P' + cPickle.dumps(obj, -1) + self._type = b'P' + return b'P' + pickle.dumps(obj, -1) def loads(self, obj): _type = obj[0] - if _type == 'M': + if _type == b'M': return marshal.loads(obj[1:]) - elif _type == 'P': - return cPickle.loads(obj[1:]) + elif _type == b'P': + return pickle.loads(obj[1:]) else: raise ValueError("invalid sevialization type: %s" % _type) @@ -495,8 +488,8 @@ 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 + def __repr__(self): + return "CompressedSerializer(%s)" % self.serializer class UTF8Deserializer(Serializer): @@ -505,7 +498,7 @@ class UTF8Deserializer(Serializer): Deserializes streams written by String.getBytes. """ - def __init__(self, use_unicode=False): + def __init__(self, use_unicode=True): self.use_unicode = use_unicode def loads(self, stream): @@ -526,13 +519,13 @@ def load_stream(self, stream): except EOFError: return - def __eq__(self, other): - return isinstance(other, UTF8Deserializer) and self.use_unicode == other.use_unicode + def __repr__(self): + return "UTF8Deserializer(%s)" % self.use_unicode def read_long(stream): length = stream.read(8) - if length == "": + if not length: raise EOFError return struct.unpack("!q", length)[0] @@ -547,7 +540,7 @@ def pack_long(value): def read_int(stream): length = stream.read(4) - if length == "": + if not length: raise EOFError return struct.unpack("!i", length)[0] diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 81aa970a32f76..144cdf0b0cdd5 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -21,13 +21,6 @@ This file is designed to be launched as a PYTHONSTARTUP script. """ -import sys -if sys.version_info[0] != 2: - print("Error: Default Python used is Python%s" % sys.version_info.major) - print("\tSet env variable PYSPARK_PYTHON to Python2 binary and re-run it.") - sys.exit(1) - - import atexit import os import platform @@ -53,9 +46,14 @@ try: # Try to access HiveConf, it will raise exception if Hive is not added sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - sqlCtx = sqlContext = HiveContext(sc) + sqlContext = HiveContext(sc) except py4j.protocol.Py4JError: - sqlCtx = sqlContext = SQLContext(sc) + sqlContext = SQLContext(sc) +except TypeError: + sqlContext = SQLContext(sc) + +# for compatibility +sqlCtx = sqlContext print("""Welcome to ____ __ diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 8a6fc627eb383..b54baa57ec28a 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -78,8 +78,8 @@ def _get_local_dirs(sub): # global stats -MemoryBytesSpilled = 0L -DiskBytesSpilled = 0L +MemoryBytesSpilled = 0 +DiskBytesSpilled = 0 class Aggregator(object): @@ -126,7 +126,7 @@ def mergeCombiners(self, iterator): """ Merge the combined items by mergeCombiner """ raise NotImplementedError - def iteritems(self): + def items(self): """ Return the merged items ad iterator """ raise NotImplementedError @@ -156,9 +156,9 @@ def mergeCombiners(self, iterator): for k, v in iterator: d[k] = comb(d[k], v) if k in d else v - def iteritems(self): - """ Return the merged items as iterator """ - return self.data.iteritems() + def items(self): + """ Return the merged items ad iterator """ + return iter(self.data.items()) def _compressed_serializer(self, serializer=None): @@ -208,15 +208,15 @@ class ExternalMerger(Merger): >>> agg = SimpleAggregator(lambda x, y: x + y) >>> merger = ExternalMerger(agg, 10) >>> N = 10000 - >>> merger.mergeValues(zip(xrange(N), xrange(N))) + >>> merger.mergeValues(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) + >>> sum(v for k,v in merger.items()) 49995000 >>> merger = ExternalMerger(agg, 10) - >>> merger.mergeCombiners(zip(xrange(N), xrange(N))) + >>> merger.mergeCombiners(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) + >>> sum(v for k,v in merger.items()) 49995000 """ @@ -335,10 +335,10 @@ def _spill(self): # above limit at the first time. # open all the files for writing - streams = [open(os.path.join(path, str(i)), 'w') + streams = [open(os.path.join(path, str(i)), 'wb') for i in range(self.partitions)] - for k, v in self.data.iteritems(): + for k, v in self.data.items(): h = self._partition(k) # put one item in batch, make it compatible with load_stream # it will increase the memory if dump them in batch @@ -354,9 +354,9 @@ def _spill(self): else: for i in range(self.partitions): p = os.path.join(path, str(i)) - with open(p, "w") as f: + with open(p, "wb") as f: # dump items in batch - self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.serializer.dump_stream(iter(self.pdata[i].items()), f) self.pdata[i].clear() DiskBytesSpilled += os.path.getsize(p) @@ -364,10 +364,10 @@ def _spill(self): gc.collect() # release the memory as much as possible MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 - def iteritems(self): + def items(self): """ Return all merged items as iterator """ if not self.pdata and not self.spills: - return self.data.iteritems() + return iter(self.data.items()) return self._external_items() def _external_items(self): @@ -398,7 +398,8 @@ def _merged_items(self, index): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), 0) + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) # limit the total partitions if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS @@ -408,7 +409,7 @@ def _merged_items(self, index): gc.collect() # release the memory as much as possible return self._recursive_merged_items(index) - return self.data.iteritems() + return self.data.items() def _recursive_merged_items(self, index): """ @@ -426,7 +427,8 @@ def _recursive_merged_items(self, index): for j in range(self.spills): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) - m.mergeCombiners(self.serializer.load_stream(open(p)), 0) + with open(p, 'rb') as f: + m.mergeCombiners(self.serializer.load_stream(f), 0) if get_used_memory() > limit: m._spill() @@ -451,7 +453,7 @@ class ExternalSorter(object): >>> sorter = ExternalSorter(1) # 1M >>> import random - >>> l = range(1024) + >>> l = list(range(1024)) >>> random.shuffle(l) >>> sorted(l) == list(sorter.sorted(l)) True @@ -499,9 +501,16 @@ def sorted(self, iterator, key=None, reverse=False): # sort them inplace will save memory current_chunk.sort(key=key, reverse=reverse) path = self._get_path(len(chunks)) - with open(path, 'w') as f: + with open(path, 'wb') as f: self.serializer.dump_stream(current_chunk, f) - chunks.append(self.serializer.load_stream(open(path))) + + def load(f): + for v in self.serializer.load_stream(f): + yield v + # close the file explicit once we consume all the items + # to avoid ResourceWarning in Python3 + f.close() + chunks.append(load(open(path, 'rb'))) current_chunk = [] gc.collect() limit = self._next_limit() @@ -527,7 +536,7 @@ class ExternalList(object): ExternalList can have many items which cannot be hold in memory in the same time. - >>> l = ExternalList(range(100)) + >>> l = ExternalList(list(range(100))) >>> len(l) 100 >>> l.append(10) @@ -555,11 +564,11 @@ def __init__(self, values): def __getstate__(self): if self._file is not None: self._file.flush() - f = os.fdopen(os.dup(self._file.fileno())) - f.seek(0) - serialized = f.read() + with os.fdopen(os.dup(self._file.fileno()), "rb") as f: + f.seek(0) + serialized = f.read() else: - serialized = '' + serialized = b'' return self.values, self.count, serialized def __setstate__(self, item): @@ -575,7 +584,7 @@ def __iter__(self): if self._file is not None: self._file.flush() # read all items from disks first - with os.fdopen(os.dup(self._file.fileno()), 'r') as f: + with os.fdopen(os.dup(self._file.fileno()), 'rb') as f: f.seek(0) for v in self._ser.load_stream(f): yield v @@ -598,11 +607,16 @@ def _open_file(self): d = dirs[id(self) % len(dirs)] if not os.path.exists(d): os.makedirs(d) - p = os.path.join(d, str(id)) - self._file = open(p, "w+", 65536) + p = os.path.join(d, str(id(self))) + self._file = open(p, "wb+", 65536) self._ser = BatchedSerializer(CompressedSerializer(PickleSerializer()), 1024) os.unlink(p) + def __del__(self): + if self._file: + self._file.close() + self._file = None + def _spill(self): """ dump the values into disk """ global MemoryBytesSpilled, DiskBytesSpilled @@ -651,33 +665,28 @@ class GroupByKey(object): """ Group a sorted iterator as [(k1, it1), (k2, it2), ...] - >>> k = [i/3 for i in range(6)] + >>> k = [i // 3 for i in range(6)] >>> v = [[i] for i in range(6)] - >>> g = GroupByKey(iter(zip(k, v))) + >>> g = GroupByKey(zip(k, v)) >>> [(k, list(it)) for k, it in g] [(0, [0, 1, 2]), (1, [3, 4, 5])] """ def __init__(self, iterator): - self.iterator = iter(iterator) - self.next_item = None + self.iterator = iterator def __iter__(self): - return self - - def next(self): - key, value = self.next_item if self.next_item else next(self.iterator) - values = ExternalListOfList([value]) - try: - while True: - k, v = next(self.iterator) - if k != key: - self.next_item = (k, v) - break + key, values = None, None + for k, v in self.iterator: + if values is not None and k == key: values.append(v) - except StopIteration: - self.next_item = None - return key, values + else: + if values is not None: + yield (key, values) + key = k + values = ExternalListOfList([v]) + if values is not None: + yield (key, values) class ExternalGroupBy(ExternalMerger): @@ -744,7 +753,7 @@ def _spill(self): # above limit at the first time. # open all the files for writing - streams = [open(os.path.join(path, str(i)), 'w') + streams = [open(os.path.join(path, str(i)), 'wb') for i in range(self.partitions)] # If the number of keys is small, then the overhead of sort is small @@ -756,7 +765,7 @@ def _spill(self): h = self._partition(k) self.serializer.dump_stream([(k, self.data[k])], streams[h]) else: - for k, v in self.data.iteritems(): + for k, v in self.data.items(): h = self._partition(k) self.serializer.dump_stream([(k, v)], streams[h]) @@ -771,14 +780,14 @@ def _spill(self): else: for i in range(self.partitions): p = os.path.join(path, str(i)) - with open(p, "w") as f: + with open(p, "wb") as f: # dump items in batch if self._sorted: # sort by key only (stable) - sorted_items = sorted(self.pdata[i].iteritems(), key=operator.itemgetter(0)) + sorted_items = sorted(self.pdata[i].items(), key=operator.itemgetter(0)) self.serializer.dump_stream(sorted_items, f) else: - self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.serializer.dump_stream(self.pdata[i].items(), f) self.pdata[i].clear() DiskBytesSpilled += os.path.getsize(p) @@ -792,7 +801,7 @@ def _merged_items(self, index): # if the memory can not hold all the partition, # then use sort based merge. Because of compression, # the data on disks will be much smaller than needed memory - if (size >> 20) >= self.memory_limit / 10: + if size >= self.memory_limit << 17: # * 1M / 8 return self._merge_sorted_items(index) self.data = {} @@ -800,15 +809,18 @@ def _merged_items(self, index): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), 0) - return self.data.iteritems() + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) + return self.data.items() def _merge_sorted_items(self, index): """ load a partition from disk, then sort and group by key """ def load_partition(j): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) - return self.serializer.load_stream(open(p, 'r', 65536)) + with open(p, 'rb', 65536) as f: + for v in self.serializer.load_stream(f): + yield v disk_items = [load_partition(j) for j in range(self.spills)] diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 65abb24eed823..6d54b9e49ed10 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -37,9 +37,22 @@ - L{types} List of data types available. """ +from __future__ import absolute_import + +# fix the module name conflict for Python 3+ +import sys +from . import _types as types +modname = __name__ + '.types' +types.__name__ = modname +# update the __module__ for all objects, make them picklable +for v in types.__dict__.values(): + if hasattr(v, "__module__") and v.__module__.endswith('._types'): + v.__module__ = modname +sys.modules[modname] = types +del modname, sys -from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.types import Row +from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD, DataFrameNaFunctions __all__ = [ diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/_types.py similarity index 97% rename from python/pyspark/sql/types.py rename to python/pyspark/sql/_types.py index ef76d84c00481..492c0cbdcf693 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/_types.py @@ -15,6 +15,7 @@ # limitations under the License. # +import sys import decimal import datetime import keyword @@ -25,6 +26,9 @@ from array import array from operator import itemgetter +if sys.version >= "3": + long = int + unicode = str __all__ = [ "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", @@ -410,7 +414,7 @@ def fromJson(cls, json): split = pyUDT.rfind(".") pyModule = pyUDT[:split] pyClass = pyUDT[split+1:] - m = __import__(pyModule, globals(), locals(), [pyClass], -1) + m = __import__(pyModule, globals(), locals(), [pyClass]) UDT = getattr(m, pyClass) return UDT() @@ -419,10 +423,9 @@ def __eq__(self, other): _all_primitive_types = dict((v.typeName(), v) - for v in globals().itervalues() - if type(v) is PrimitiveTypeSingleton and - v.__base__ == PrimitiveType) - + for v in list(globals().values()) + if (type(v) is type or type(v) is PrimitiveTypeSingleton) + and v.__base__ == PrimitiveType) _all_complex_types = dict((v.typeName(), v) for v in [ArrayType, MapType, StructType]) @@ -486,10 +489,10 @@ def _parse_datatype_json_string(json_string): def _parse_datatype_json_value(json_value): - if type(json_value) is unicode: + if not isinstance(json_value, dict): if json_value in _all_primitive_types.keys(): return _all_primitive_types[json_value]() - elif json_value == u'decimal': + elif json_value == 'decimal': return DecimalType() elif _FIXED_DECIMAL.match(json_value): m = _FIXED_DECIMAL.match(json_value) @@ -511,10 +514,8 @@ def _parse_datatype_json_value(json_value): type(None): NullType, bool: BooleanType, int: LongType, - long: LongType, float: DoubleType, str: StringType, - unicode: StringType, bytearray: BinaryType, decimal.Decimal: DecimalType, datetime.date: DateType, @@ -522,6 +523,12 @@ def _parse_datatype_json_value(json_value): datetime.time: TimestampType, } +if sys.version < "3": + _type_mappings.update({ + unicode: StringType, + long: LongType, + }) + def _infer_type(obj): """Infer the DataType from obj @@ -541,7 +548,7 @@ def _infer_type(obj): return dataType() if isinstance(obj, dict): - for key, value in obj.iteritems(): + for key, value in obj.items(): if key is not None and value is not None: return MapType(_infer_type(key), _infer_type(value), True) else: @@ -565,10 +572,10 @@ def _infer_schema(row): items = sorted(row.items()) elif isinstance(row, (tuple, list)): - if hasattr(row, "_fields"): # namedtuple - items = zip(row._fields, tuple(row)) - elif hasattr(row, "__fields__"): # Row + if hasattr(row, "__fields__"): # Row items = zip(row.__fields__, tuple(row)) + elif hasattr(row, "_fields"): # namedtuple + items = zip(row._fields, tuple(row)) else: names = ['_%d' % i for i in range(1, len(row) + 1)] items = zip(names, row) @@ -647,7 +654,7 @@ def converter(obj): if isinstance(obj, dict): return tuple(c(obj.get(n)) for n, c in zip(names, converters)) elif isinstance(obj, tuple): - if hasattr(obj, "_fields") or hasattr(obj, "__fields__"): + if hasattr(obj, "__fields__") or hasattr(obj, "_fields"): return tuple(c(v) for c, v in zip(converters, obj)) elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs d = dict(obj) @@ -733,12 +740,12 @@ def _create_converter(dataType): if isinstance(dataType, ArrayType): conv = _create_converter(dataType.elementType) - return lambda row: map(conv, row) + return lambda row: [conv(v) for v in row] elif isinstance(dataType, MapType): kconv = _create_converter(dataType.keyType) vconv = _create_converter(dataType.valueType) - return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems()) + return lambda row: dict((kconv(k), vconv(v)) for k, v in row.items()) elif isinstance(dataType, NullType): return lambda x: None @@ -881,7 +888,7 @@ def _infer_schema_type(obj, dataType): >>> _infer_schema_type(row, schema) StructType...a,ArrayType...b,MapType(StringType,...c,LongType... """ - if dataType is NullType(): + if isinstance(dataType, NullType): return _infer_type(obj) if not obj: @@ -892,7 +899,7 @@ def _infer_schema_type(obj, dataType): return ArrayType(eType, True) elif isinstance(dataType, MapType): - k, v = obj.iteritems().next() + k, v = next(iter(obj.items())) return MapType(_infer_schema_type(k, dataType.keyType), _infer_schema_type(v, dataType.valueType)) @@ -935,7 +942,7 @@ def _verify_type(obj, dataType): >>> _verify_type(None, StructType([])) >>> _verify_type("", StringType()) >>> _verify_type(0, LongType()) - >>> _verify_type(range(3), ArrayType(ShortType())) + >>> _verify_type(list(range(3)), ArrayType(ShortType())) >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... @@ -976,7 +983,7 @@ def _verify_type(obj, dataType): _verify_type(i, dataType.elementType) elif isinstance(dataType, MapType): - for k, v in obj.iteritems(): + for k, v in obj.items(): _verify_type(k, dataType.keyType) _verify_type(v, dataType.valueType) @@ -1213,6 +1220,8 @@ def __getattr__(self, item): return self[idx] except IndexError: raise AttributeError(item) + except ValueError: + raise AttributeError(item) def __reduce__(self): if hasattr(self, "__fields__"): diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e8529a8f8e3a4..c90afc326ca0e 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -15,14 +15,19 @@ # limitations under the License. # +import sys import warnings import json -from itertools import imap + +if sys.version >= '3': + basestring = unicode = str +else: + from itertools import imap as map from py4j.protocol import Py4JError from py4j.java_collections import MapConverter -from pyspark.rdd import RDD, _prepare_for_python_RDD +from pyspark.rdd import RDD, _prepare_for_python_RDD, ignore_unicode_prefix from pyspark.serializers import AutoBatchedSerializer, PickleSerializer from pyspark.sql.types import Row, StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter @@ -62,31 +67,27 @@ class SQLContext(object): A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. - When created, :class:`SQLContext` adds a method called ``toDF`` to :class:`RDD`, - which could be used to convert an RDD into a DataFrame, it's a shorthand for - :func:`SQLContext.createDataFrame`. - :param sparkContext: The :class:`SparkContext` backing this SQLContext. :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instantiate a new SQLContext in the JVM, instead we make all calls to this object. """ + @ignore_unicode_prefix def __init__(self, sparkContext, sqlContext=None): """Creates a new SQLContext. >>> from datetime import datetime >>> sqlContext = SQLContext(sc) - >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, + >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() >>> df.registerTempTable("allTypes") >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() - [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] - >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, - ... x.row.a, x.list)).collect() - [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] + [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] + >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() + [(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ self._sc = sparkContext self._jsc = self._sc._jsc @@ -122,6 +123,7 @@ def udf(self): """Returns a :class:`UDFRegistration` for UDF registration.""" return UDFRegistration(self) + @ignore_unicode_prefix def registerFunction(self, name, f, returnType=StringType()): """Registers a lambda function as a UDF so it can be used in SQL statements. @@ -147,7 +149,7 @@ def registerFunction(self, name, f, returnType=StringType()): >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] """ - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) pickled_cmd, bvars, env, includes = _prepare_for_python_RDD(self._sc, command, self) @@ -185,6 +187,7 @@ def _inferSchema(self, rdd, samplingRatio=None): schema = rdd.map(_infer_schema).reduce(_merge_type) return schema + @ignore_unicode_prefix def inferSchema(self, rdd, samplingRatio=None): """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ @@ -195,6 +198,7 @@ def inferSchema(self, rdd, samplingRatio=None): return self.createDataFrame(rdd, None, samplingRatio) + @ignore_unicode_prefix def applySchema(self, rdd, schema): """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ @@ -208,6 +212,7 @@ def applySchema(self, rdd, schema): return self.createDataFrame(rdd, schema) + @ignore_unicode_prefix def createDataFrame(self, data, schema=None, samplingRatio=None): """ Creates a :class:`DataFrame` from an :class:`RDD` of :class:`tuple`/:class:`list`, @@ -380,6 +385,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): df = self._ssql_ctx.jsonFile(path, scala_datatype) return DataFrame(df, self) + @ignore_unicode_prefix def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): """Loads an RDD storing one JSON object per string as a :class:`DataFrame`. @@ -477,6 +483,7 @@ def createExternalTable(self, tableName, path=None, source=None, joptions) return DataFrame(df, self) + @ignore_unicode_prefix def sql(self, sqlQuery): """Returns a :class:`DataFrame` representing the result of the given query. @@ -497,6 +504,7 @@ def table(self, tableName): """ return DataFrame(self._ssql_ctx.table(tableName), self) + @ignore_unicode_prefix def tables(self, dbName=None): """Returns a :class:`DataFrame` containing names of tables in the given database. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f2c3b74a185cf..d76504f986270 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -16,14 +16,19 @@ # import sys -import itertools import warnings import random +if sys.version >= '3': + basestring = unicode = str + long = int +else: + from itertools import imap as map + from py4j.java_collections import ListConverter, MapConverter from pyspark.context import SparkContext -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -65,19 +70,20 @@ def __init__(self, jdf, sql_ctx): self._sc = sql_ctx and sql_ctx._sc self.is_cached = False self._schema = None # initialized lazily + self._lazy_rdd = None @property def rdd(self): """Returns the content as an :class:`pyspark.RDD` of :class:`Row`. """ - if not hasattr(self, '_lazy_rdd'): + if self._lazy_rdd is None: jrdd = self._jdf.javaToPython() rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer())) schema = self.schema def applySchema(it): cls = _create_cls(schema) - return itertools.imap(cls, it) + return map(cls, it) self._lazy_rdd = rdd.mapPartitions(applySchema) @@ -89,13 +95,14 @@ def na(self): """ return DataFrameNaFunctions(self) - def toJSON(self, use_unicode=False): + @ignore_unicode_prefix + def toJSON(self, use_unicode=True): """Converts a :class:`DataFrame` into a :class:`RDD` of string. Each row is turned into a JSON document as one element in the returned RDD. >>> df.toJSON().first() - '{"age":2,"name":"Alice"}' + u'{"age":2,"name":"Alice"}' """ rdd = self._jdf.toJSON() return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) @@ -228,7 +235,7 @@ def printSchema(self): |-- name: string (nullable = true) """ - print (self._jdf.schema().treeString()) + print(self._jdf.schema().treeString()) def explain(self, extended=False): """Prints the (logical and physical) plans to the console for debugging purpose. @@ -250,9 +257,9 @@ def explain(self, extended=False): == RDD == """ if extended: - print self._jdf.queryExecution().toString() + print(self._jdf.queryExecution().toString()) else: - print self._jdf.queryExecution().executedPlan().toString() + print(self._jdf.queryExecution().executedPlan().toString()) def isLocal(self): """Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally @@ -270,7 +277,7 @@ def show(self, n=20): 2 Alice 5 Bob """ - print self._jdf.showString(n).encode('utf8', 'ignore') + print(self._jdf.showString(n)) def __repr__(self): return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) @@ -279,10 +286,11 @@ def count(self): """Returns the number of rows in this :class:`DataFrame`. >>> df.count() - 2L + 2 """ - return self._jdf.count() + return int(self._jdf.count()) + @ignore_unicode_prefix def collect(self): """Returns all the records as a list of :class:`Row`. @@ -295,6 +303,7 @@ def collect(self): cls = _create_cls(self.schema) return [cls(r) for r in rs] + @ignore_unicode_prefix def limit(self, num): """Limits the result count to the number specified. @@ -306,6 +315,7 @@ def limit(self, num): jdf = self._jdf.limit(num) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def take(self, num): """Returns the first ``num`` rows as a :class:`list` of :class:`Row`. @@ -314,6 +324,7 @@ def take(self, num): """ return self.limit(num).collect() + @ignore_unicode_prefix def map(self, f): """ Returns a new :class:`RDD` by applying a the ``f`` function to each :class:`Row`. @@ -324,6 +335,7 @@ def map(self, f): """ return self.rdd.map(f) + @ignore_unicode_prefix def flatMap(self, f): """ Returns a new :class:`RDD` by first applying the ``f`` function to each :class:`Row`, and then flattening the results. @@ -353,7 +365,7 @@ def foreach(self, f): This is a shorthand for ``df.rdd.foreach()``. >>> def f(person): - ... print person.name + ... print(person.name) >>> df.foreach(f) """ return self.rdd.foreach(f) @@ -365,7 +377,7 @@ def foreachPartition(self, f): >>> def f(people): ... for person in people: - ... print person.name + ... print(person.name) >>> df.foreachPartition(f) """ return self.rdd.foreachPartition(f) @@ -412,7 +424,7 @@ def distinct(self): """Returns a new :class:`DataFrame` containing the distinct rows in this :class:`DataFrame`. >>> df.distinct().count() - 2L + 2 """ return DataFrame(self._jdf.distinct(), self.sql_ctx) @@ -420,10 +432,10 @@ def sample(self, withReplacement, fraction, seed=None): """Returns a sampled subset of this :class:`DataFrame`. >>> df.sample(False, 0.5, 97).count() - 1L + 1 """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction - seed = seed if seed is not None else random.randint(0, sys.maxint) + seed = seed if seed is not None else random.randint(0, sys.maxsize) rdd = self._jdf.sample(withReplacement, fraction, long(seed)) return DataFrame(rdd, self.sql_ctx) @@ -437,6 +449,7 @@ def dtypes(self): return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields] @property + @ignore_unicode_prefix def columns(self): """Returns all column names as a list. @@ -445,6 +458,7 @@ def columns(self): """ return [f.name for f in self.schema.fields] + @ignore_unicode_prefix def join(self, other, joinExprs=None, joinType=None): """Joins with another :class:`DataFrame`, using the given join expression. @@ -470,6 +484,7 @@ def join(self, other, joinExprs=None, joinType=None): jdf = self._jdf.join(other._jdf, joinExprs._jc, joinType) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def sort(self, *cols): """Returns a new :class:`DataFrame` sorted by the specified column(s). @@ -513,6 +528,7 @@ def describe(self, *cols): jdf = self._jdf.describe(self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols)) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def head(self, n=None): """ Returns the first ``n`` rows as a list of :class:`Row`, @@ -528,6 +544,7 @@ def head(self, n=None): return rs[0] if rs else None return self.take(n) + @ignore_unicode_prefix def first(self): """Returns the first row as a :class:`Row`. @@ -536,6 +553,7 @@ def first(self): """ return self.head() + @ignore_unicode_prefix def __getitem__(self, item): """Returns the column as a :class:`Column`. @@ -567,6 +585,7 @@ def __getattr__(self, name): jc = self._jdf.apply(name) return Column(jc) + @ignore_unicode_prefix def select(self, *cols): """Projects a set of expressions and returns a new :class:`DataFrame`. @@ -598,6 +617,7 @@ def selectExpr(self, *expr): jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def filter(self, condition): """Filters rows using the given condition. @@ -626,6 +646,7 @@ def filter(self, condition): where = filter + @ignore_unicode_prefix def groupBy(self, *cols): """Groups the :class:`DataFrame` using the specified columns, so we can run aggregation on them. See :class:`GroupedData` @@ -775,6 +796,7 @@ def fillna(self, value, subset=None): cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) return DataFrame(self._jdf.na().fill(value, cols), self.sql_ctx) + @ignore_unicode_prefix def withColumn(self, colName, col): """Returns a new :class:`DataFrame` by adding a column. @@ -786,6 +808,7 @@ def withColumn(self, colName, col): """ return self.select('*', col.alias(colName)) + @ignore_unicode_prefix def withColumnRenamed(self, existing, new): """REturns a new :class:`DataFrame` by renaming an existing column. @@ -852,6 +875,7 @@ def __init__(self, jdf, sql_ctx): self._jdf = jdf self.sql_ctx = sql_ctx + @ignore_unicode_prefix def agg(self, *exprs): """Compute aggregates and returns the result as a :class:`DataFrame`. @@ -1041,11 +1065,13 @@ def __init__(self, jc): __sub__ = _bin_op("minus") __mul__ = _bin_op("multiply") __div__ = _bin_op("divide") + __truediv__ = _bin_op("divide") __mod__ = _bin_op("mod") __radd__ = _bin_op("plus") __rsub__ = _reverse_op("minus") __rmul__ = _bin_op("multiply") __rdiv__ = _reverse_op("divide") + __rtruediv__ = _reverse_op("divide") __rmod__ = _reverse_op("mod") # logistic operators @@ -1075,6 +1101,7 @@ def __init__(self, jc): startswith = _bin_op("startsWith") endswith = _bin_op("endsWith") + @ignore_unicode_prefix def substr(self, startPos, length): """ Return a :class:`Column` which is a substring of the column @@ -1097,6 +1124,7 @@ def substr(self, startPos, length): __getslice__ = substr + @ignore_unicode_prefix def inSet(self, *cols): """ A boolean expression that is evaluated to true if the value of this expression is contained by the evaluated values of the arguments. @@ -1131,6 +1159,7 @@ def alias(self, alias): """ return Column(getattr(self._jc, "as")(alias)) + @ignore_unicode_prefix def cast(self, dataType): """ Convert the column into type `dataType` diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index daeb6916b58bc..1d6536952810f 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -18,8 +18,10 @@ """ A collections of builtin functions """ +import sys -from itertools import imap +if sys.version < "3": + from itertools import imap as map from py4j.java_collections import ListConverter @@ -116,7 +118,7 @@ def __init__(self, func, returnType): def _create_judf(self): f = self.func # put it in closure `func` - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) sc = SparkContext._active_spark_context diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index b3a6a2c6a9229..7c09a0cfe30ab 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -157,13 +157,13 @@ def test_udf2(self): self.assertEqual(4, res[0]) def test_udf_with_array_type(self): - d = [Row(l=range(3), d={"key": range(5)})] + d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) self.sqlCtx.createDataFrame(rdd).registerTempTable("test") self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType()) [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect() - self.assertEqual(range(3), l1) + self.assertEqual(list(range(3)), l1) self.assertEqual(1, l2) def test_broadcast_in_udf(self): @@ -266,7 +266,7 @@ def test_infer_nested_schema(self): def test_apply_schema(self): from datetime import date, datetime - rdd = self.sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + rdd = self.sc.parallelize([(127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), datetime(2010, 1, 1, 1, 1, 1), {"a": 1}, (2,), [1, 2, 3], None)]) schema = StructType([ @@ -309,7 +309,7 @@ def test_apply_schema(self): def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] df = self.sc.parallelize(d).toDF() - k, v = df.head().m.items()[0] + k, v = list(df.head().m.items())[0] self.assertEqual(1, k.i) self.assertEqual("", v.s) @@ -554,6 +554,9 @@ def setUpClass(cls): except py4j.protocol.Py4JError: cls.sqlCtx = None return + except TypeError: + cls.sqlCtx = None + return os.unlink(cls.tempdir.name) _scala_HiveContext =\ cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc()) diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 1e597d64e03fe..944fa414b0c0e 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -31,7 +31,7 @@ class StatCounter(object): def __init__(self, values=[]): - self.n = 0L # Running count of our values + self.n = 0 # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) self.maxValue = float("-inf") @@ -87,7 +87,7 @@ def copy(self): return copy.deepcopy(self) def count(self): - return self.n + return int(self.n) def mean(self): return self.mu diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 2c73083c9f9a8..4590c58839266 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -14,6 +14,9 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +from __future__ import print_function + import os import sys @@ -157,7 +160,7 @@ def getOrCreate(cls, checkpointPath, setupFunc): try: jssc = gw.jvm.JavaStreamingContext(checkpointPath) except Exception: - print >>sys.stderr, "failed to load StreamingContext from checkpoint" + print("failed to load StreamingContext from checkpoint", file=sys.stderr) raise jsc = jssc.sparkContext() diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 3fa42444239f7..ff097985fae3e 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -15,11 +15,15 @@ # limitations under the License. # -from itertools import chain, ifilter, imap +import sys import operator import time +from itertools import chain from datetime import datetime +if sys.version < "3": + from itertools import imap as map, ifilter as filter + from py4j.protocol import Py4JJavaError from pyspark import RDD @@ -76,7 +80,7 @@ def filter(self, f): Return a new DStream containing only the elements that satisfy predicate. """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def flatMap(self, f, preservesPartitioning=False): @@ -85,7 +89,7 @@ def flatMap(self, f, preservesPartitioning=False): this DStream, and then flattening the results """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def map(self, f, preservesPartitioning=False): @@ -93,7 +97,7 @@ def map(self, f, preservesPartitioning=False): Return a new DStream by applying a function to each element of DStream. """ def func(iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitions(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -150,7 +154,7 @@ def foreachRDD(self, func): """ Apply a function to each RDD in this DStream. """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: old_func = func func = lambda t, rdd: old_func(rdd) jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer) @@ -165,14 +169,14 @@ def pprint(self, num=10): """ def takeAndPrint(time, rdd): taken = rdd.take(num + 1) - print "-------------------------------------------" - print "Time: %s" % time - print "-------------------------------------------" + print("-------------------------------------------") + print("Time: %s" % time) + print("-------------------------------------------") for record in taken[:num]: - print record + print(record) if len(taken) > num: - print "..." - print + print("...") + print() self.foreachRDD(takeAndPrint) @@ -181,7 +185,7 @@ def mapValues(self, f): Return a new DStream by applying a map function to the value of each key-value pairs in this DStream without changing the key. """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def flatMapValues(self, f): @@ -189,7 +193,7 @@ def flatMapValues(self, f): Return a new DStream by applying a flatmap function to the value of each key-value pairs in this DStream without changing the key. """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def glom(self): @@ -286,10 +290,10 @@ def transform(self, func): `func` can have one argument of `rdd`, or have two arguments of (`time`, `rdd`) """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: oldfunc = func func = lambda t, rdd: oldfunc(rdd) - assert func.func_code.co_argcount == 2, "func should take one or two arguments" + assert func.__code__.co_argcount == 2, "func should take one or two arguments" return TransformedDStream(self, func) def transformWith(self, func, other, keepSerializer=False): @@ -300,10 +304,10 @@ def transformWith(self, func, other, keepSerializer=False): `func` can have two arguments of (`rdd_a`, `rdd_b`) or have three arguments of (`time`, `rdd_a`, `rdd_b`) """ - if func.func_code.co_argcount == 2: + if func.__code__.co_argcount == 2: oldfunc = func func = lambda t, a, b: oldfunc(a, b) - assert func.func_code.co_argcount == 3, "func should take two or three arguments" + assert func.__code__.co_argcount == 3, "func should take two or three arguments" jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer, other._jrdd_deserializer) dstream = self._sc._jvm.PythonTransformed2DStream(self._jdstream.dstream(), other._jdstream.dstream(), jfunc) @@ -460,7 +464,7 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio keyed = self.map(lambda x: (1, x)) reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration, 1) - return reduced.map(lambda (k, v): v) + return reduced.map(lambda kv: kv[1]) def countByWindow(self, windowDuration, slideDuration): """ @@ -489,7 +493,7 @@ def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=Non keyed = self.map(lambda x: (x, 1)) counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub, windowDuration, slideDuration, numPartitions) - return counted.filter(lambda (k, v): v > 0).count() + return counted.filter(lambda kv: kv[1] > 0).count() def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None): """ @@ -548,7 +552,8 @@ def reduceFunc(t, a, b): def invReduceFunc(t, a, b): b = b.reduceByKey(func, numPartitions) joined = a.leftOuterJoin(b, numPartitions) - return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1) + return joined.mapValues(lambda kv: invFunc(kv[0], kv[1]) + if kv[1] is not None else kv[0]) jreduceFunc = TransformFunction(self._sc, reduceFunc, reduced._jrdd_deserializer) if invReduceFunc: @@ -579,9 +584,9 @@ def reduceFunc(t, a, b): g = b.groupByKey(numPartitions).mapValues(lambda vs: (list(vs), None)) else: g = a.cogroup(b.partitionBy(numPartitions), numPartitions) - g = g.mapValues(lambda (va, vb): (list(vb), list(va)[0] if len(va) else None)) - state = g.mapValues(lambda (vs, s): updateFunc(vs, s)) - return state.filter(lambda (k, v): v is not None) + g = g.mapValues(lambda ab: (list(ab[1]), list(ab[0])[0] if len(ab[0]) else None)) + state = g.mapValues(lambda vs_s: updateFunc(vs_s[0], vs_s[1])) + return state.filter(lambda k_v: k_v[1] is not None) jreduceFunc = TransformFunction(self._sc, reduceFunc, self._sc.serializer, self._jrdd_deserializer) diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index f083ed149effb..7a7b6e1d9a527 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -67,10 +67,10 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") helper = helperClass.newInstance() jstream = helper.createStream(ssc._jssc, jparam, jtopics, jlevel) - except Py4JJavaError, e: + except Py4JJavaError as e: # TODO: use --jar once it also work on driver if 'ClassNotFoundException' in str(e.java_exception): - print """ + print(""" ________________________________________________________________________________________________ Spark Streaming's Kafka libraries not found in class path. Try one of the following. @@ -88,8 +88,8 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, ________________________________________________________________________________________________ -""" % (ssc.sparkContext.version, ssc.sparkContext.version) +""" % (ssc.sparkContext.version, ssc.sparkContext.version)) raise e ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) stream = DStream(jstream, ssc, ser) - return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v))) + return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 9b4635e49020b..06d22154373bc 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -22,6 +22,7 @@ import unittest import tempfile import struct +from functools import reduce from py4j.java_collections import MapConverter @@ -51,7 +52,7 @@ def wait_for(self, result, n): while len(result) < n and time.time() - start_time < self.timeout: time.sleep(0.01) if len(result) < n: - print "timeout after", self.timeout + print("timeout after", self.timeout) def _take(self, dstream, n): """ @@ -131,7 +132,7 @@ def test_map(self): def func(dstream): return dstream.map(str) - expected = map(lambda x: map(str, x), input) + expected = [list(map(str, x)) for x in input] self._test_func(input, func, expected) def test_flatMap(self): @@ -140,8 +141,8 @@ def test_flatMap(self): def func(dstream): return dstream.flatMap(lambda x: (x, x * 2)) - expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))), - input) + expected = [list(chain.from_iterable((map(lambda y: [y, y * 2], x)))) + for x in input] self._test_func(input, func, expected) def test_filter(self): @@ -150,7 +151,7 @@ def test_filter(self): def func(dstream): return dstream.filter(lambda x: x % 2 == 0) - expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input) + expected = [[y for y in x if y % 2 == 0] for x in input] self._test_func(input, func, expected) def test_count(self): @@ -159,7 +160,7 @@ def test_count(self): def func(dstream): return dstream.count() - expected = map(lambda x: [len(x)], input) + expected = [[len(x)] for x in input] self._test_func(input, func, expected) def test_reduce(self): @@ -168,7 +169,7 @@ def test_reduce(self): def func(dstream): return dstream.reduce(operator.add) - expected = map(lambda x: [reduce(operator.add, x)], input) + expected = [[reduce(operator.add, x)] for x in input] self._test_func(input, func, expected) def test_reduceByKey(self): @@ -185,27 +186,27 @@ def func(dstream): def test_mapValues(self): """Basic operation test for DStream.mapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 2), (3, 3)], + [(0, 4), (1, 1), (2, 2), (3, 3)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.mapValues(lambda x: x + 10) expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)], - [("", 14), (1, 11), (2, 12), (3, 13)], + [(0, 14), (1, 11), (2, 12), (3, 13)], [(1, 11), (2, 11), (3, 11), (4, 11)]] self._test_func(input, func, expected, sort=True) def test_flatMapValues(self): """Basic operation test for DStream.flatMapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 1), (3, 1)], + [(0, 4), (1, 1), (2, 1), (3, 1)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.flatMapValues(lambda x: (x, x + 10)) expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12), ("c", 1), ("c", 11), ("d", 1), ("d", 11)], - [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], + [(0, 4), (0, 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]] self._test_func(input, func, expected) @@ -233,7 +234,7 @@ def f(iterator): def test_countByValue(self): """Basic operation test for DStream.countByValue.""" - input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]] + input = [list(range(1, 5)) * 2, list(range(5, 7)) + list(range(5, 9)), ["a", "a", "b", ""]] def func(dstream): return dstream.countByValue() @@ -285,7 +286,7 @@ def test_union(self): def func(d1, d2): return d1.union(d2) - expected = [range(6), range(6), range(6)] + expected = [list(range(6)), list(range(6)), list(range(6))] self._test_func(input1, func, expected, input2=input2) def test_cogroup(self): @@ -424,7 +425,7 @@ class StreamingContextTests(PySparkStreamingTestCase): duration = 0.1 def _add_input_stream(self): - inputs = map(lambda x: range(1, x), range(101)) + inputs = [range(1, x) for x in range(101)] stream = self.ssc.queueStream(inputs) self._collect(stream, 1, block=False) @@ -441,7 +442,7 @@ def test_stop_multiple_times(self): self.ssc.stop() def test_queue_stream(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) result = self._collect(dstream, 3) self.assertEqual(input, result) @@ -457,13 +458,13 @@ def test_text_file_stream(self): with open(os.path.join(d, name), "w") as f: f.writelines(["%d\n" % i for i in range(10)]) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], result) + self.assertEqual([list(range(10)), list(range(10))], result) def test_binary_records_stream(self): d = tempfile.mkdtemp() self.ssc = StreamingContext(self.sc, self.duration) dstream = self.ssc.binaryRecordsStream(d, 10).map( - lambda v: struct.unpack("10b", str(v))) + lambda v: struct.unpack("10b", bytes(v))) result = self._collect(dstream, 2, block=False) self.ssc.start() for name in ('a', 'b'): @@ -471,10 +472,10 @@ def test_binary_records_stream(self): with open(os.path.join(d, name), "wb") as f: f.write(bytearray(range(10))) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], map(lambda v: list(v[0]), result)) + self.assertEqual([list(range(10)), list(range(10))], [list(v[0]) for v in result]) def test_union(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) dstream2 = self.ssc.queueStream(input) dstream3 = self.ssc.union(dstream, dstream2) diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py index 86ee5aa04f252..34291f30a5652 100644 --- a/python/pyspark/streaming/util.py +++ b/python/pyspark/streaming/util.py @@ -91,9 +91,9 @@ def dumps(self, id): except Exception: traceback.print_exc() - def loads(self, bytes): + def loads(self, data): try: - f, deserializers = self.serializer.loads(str(bytes)) + f, deserializers = self.serializer.loads(bytes(data)) return TransformFunction(self.ctx, f, *deserializers) except Exception: traceback.print_exc() @@ -116,7 +116,7 @@ def rddToFileName(prefix, suffix, timestamp): """ if isinstance(timestamp, datetime): seconds = time.mktime(timestamp.timetuple()) - timestamp = long(seconds * 1000) + timestamp.microsecond / 1000 + timestamp = int(seconds * 1000) + timestamp.microsecond // 1000 if suffix is None: return prefix + "-" + str(timestamp) else: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index ee67e80d539f8..75f39d9e75f38 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -19,8 +19,8 @@ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. """ + from array import array -from fileinput import input from glob import glob import os import re @@ -45,6 +45,9 @@ sys.exit(1) else: import unittest + if sys.version_info[0] >= 3: + xrange = range + basestring = str from pyspark.conf import SparkConf @@ -52,7 +55,9 @@ from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer + CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer, \ + PairDeserializer, CartesianDeserializer, AutoBatchedSerializer, AutoSerializer, \ + FlattenedValuesSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark import shuffle from pyspark.profiler import BasicProfiler @@ -81,7 +86,7 @@ class MergerTests(unittest.TestCase): def setUp(self): self.N = 1 << 12 self.l = [i for i in xrange(self.N)] - self.data = zip(self.l, self.l) + self.data = list(zip(self.l, self.l)) self.agg = Aggregator(lambda x: [x], lambda x, y: x.append(y) or x, lambda x, y: x.extend(y) or x) @@ -89,45 +94,45 @@ def setUp(self): def test_in_memory(self): m = InMemoryMerger(self.agg) m.mergeValues(self.data) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = InMemoryMerger(self.agg) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + m.mergeCombiners(map(lambda x_y: (x_y[0], [x_y[1]]), self.data)) + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_small_dataset(self): m = ExternalMerger(self.agg, 1000) m.mergeValues(self.data) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 1000) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + m.mergeCombiners(map(lambda x_y1: (x_y1[0], [x_y1[1]]), self.data)) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_medium_dataset(self): - m = ExternalMerger(self.agg, 30) + m = ExternalMerger(self.agg, 20) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 10) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3)) + m.mergeCombiners(map(lambda x_y2: (x_y2[0], [x_y2[1]]), self.data * 3)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N)) * 3) def test_huge_dataset(self): - m = ExternalMerger(self.agg, 10, partitions=3) - m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) + m = ExternalMerger(self.agg, 5, partitions=3) + m.mergeCombiners(map(lambda k_v: (k_v[0], [str(k_v[1])]), self.data * 10)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(len(v) for k, v in m.iteritems()), + self.assertEqual(sum(len(v) for k, v in m.items()), self.N * 10) m._cleanup() @@ -144,55 +149,55 @@ def gen_gs(N, step=1): self.assertEqual(1, len(list(gen_gs(1)))) self.assertEqual(2, len(list(gen_gs(2)))) self.assertEqual(100, len(list(gen_gs(100)))) - self.assertEqual(range(1, 101), [k for k, _ in gen_gs(100)]) - self.assertTrue(all(range(k) == list(vs) for k, vs in gen_gs(100))) + self.assertEqual(list(range(1, 101)), [k for k, _ in gen_gs(100)]) + self.assertTrue(all(list(range(k)) == list(vs) for k, vs in gen_gs(100))) for k, vs in gen_gs(50002, 10000): self.assertEqual(k, len(vs)) - self.assertEqual(range(k), list(vs)) + self.assertEqual(list(range(k)), list(vs)) ser = PickleSerializer() l = ser.loads(ser.dumps(list(gen_gs(50002, 30000)))) for k, vs in l: self.assertEqual(k, len(vs)) - self.assertEqual(range(k), list(vs)) + self.assertEqual(list(range(k)), list(vs)) class SorterTests(unittest.TestCase): def test_in_memory_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1024) - self.assertEquals(sorted(l), list(sorter.sorted(l))) - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) def test_external_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1) - self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) self.assertGreater(shuffle.DiskBytesSpilled, 0) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) def test_external_sort_in_rdd(self): conf = SparkConf().set("spark.python.worker.memory", "1m") sc = SparkContext(conf=conf) - l = range(10240) + l = list(range(10240)) random.shuffle(l) - rdd = sc.parallelize(l, 10) - self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect()) + rdd = sc.parallelize(l, 2) + self.assertEqual(sorted(l), rdd.sortBy(lambda x: x).collect()) sc.stop() @@ -200,11 +205,11 @@ class SerializationTestCase(unittest.TestCase): def test_namedtuple(self): from collections import namedtuple - from cPickle import dumps, loads + from pickle import dumps, loads P = namedtuple("P", "x y") p1 = P(1, 3) p2 = loads(dumps(p1, 2)) - self.assertEquals(p1, p2) + self.assertEqual(p1, p2) def test_itemgetter(self): from operator import itemgetter @@ -246,7 +251,7 @@ def test_pickling_file_handles(self): ser = CloudPickleSerializer() out1 = sys.stderr out2 = ser.loads(ser.dumps(out1)) - self.assertEquals(out1, out2) + self.assertEqual(out1, out2) def test_func_globals(self): @@ -263,19 +268,36 @@ def __reduce__(self): def foo(): sys.exit(0) - self.assertTrue("exit" in foo.func_code.co_names) + self.assertTrue("exit" in foo.__code__.co_names) ser.dumps(foo) def test_compressed_serializer(self): ser = CompressedSerializer(PickleSerializer()) - from StringIO import StringIO + try: + from StringIO import StringIO + except ImportError: + from io import BytesIO as StringIO io = StringIO() ser.dump_stream(["abc", u"123", range(5)], io) io.seek(0) self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(io))) ser.dump_stream(range(1000), io) io.seek(0) - self.assertEqual(["abc", u"123", range(5)] + range(1000), list(ser.load_stream(io))) + self.assertEqual(["abc", u"123", range(5)] + list(range(1000)), list(ser.load_stream(io))) + io.close() + + def test_hash_serializer(self): + hash(NoOpSerializer()) + hash(UTF8Deserializer()) + hash(PickleSerializer()) + hash(MarshalSerializer()) + hash(AutoSerializer()) + hash(BatchedSerializer(PickleSerializer())) + hash(AutoBatchedSerializer(MarshalSerializer())) + hash(PairDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CartesianDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CompressedSerializer(PickleSerializer())) + hash(FlattenedValuesSerializer(PickleSerializer())) class PySparkTestCase(unittest.TestCase): @@ -340,7 +362,7 @@ def test_checkpoint_and_restore(self): self.assertTrue(flatMappedRDD.getCheckpointFile() is not None) recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(), flatMappedRDD._jrdd_deserializer) - self.assertEquals([1, 2, 3, 4], recovered.collect()) + self.assertEqual([1, 2, 3, 4], recovered.collect()) class AddFileTests(PySparkTestCase): @@ -356,8 +378,7 @@ def test_add_py_file(self): def func(x): from userlibrary import UserClass return UserClass().hello() - self.assertRaises(Exception, - self.sc.parallelize(range(2)).map(func).first) + self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) log4j.LogManager.getRootLogger().setLevel(old_level) # Add the file, so the job should now succeed: @@ -372,7 +393,7 @@ def test_add_file_locally(self): download_path = SparkFiles.get("hello.txt") self.assertNotEqual(path, download_path) with open(download_path) as test_file: - self.assertEquals("Hello World!\n", test_file.readline()) + self.assertEqual("Hello World!\n", test_file.readline()) def test_add_py_file_locally(self): # To ensure that we're actually testing addPyFile's effects, check that @@ -381,7 +402,7 @@ def func(): from userlibrary import UserClass self.assertRaises(ImportError, func) path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") - self.sc.addFile(path) + self.sc.addPyFile(path) from userlibrary import UserClass self.assertEqual("Hello World!", UserClass().hello()) @@ -391,7 +412,7 @@ def test_add_egg_file_locally(self): def func(): from userlib import UserClass self.assertRaises(ImportError, func) - path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1-py2.7.egg") + path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip") self.sc.addPyFile(path) from userlib import UserClass self.assertEqual("Hello World from inside a package!", UserClass().hello()) @@ -427,8 +448,9 @@ def test_save_as_textfile_with_unicode(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode("utf-8")) def test_save_as_textfile_with_utf8(self): x = u"\u00A1Hola, mundo!" @@ -436,19 +458,20 @@ def test_save_as_textfile_with_utf8(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode('utf8')) def test_transforming_cartesian_result(self): # Regression test for SPARK-1034 rdd1 = self.sc.parallelize([1, 2]) rdd2 = self.sc.parallelize([3, 4]) cart = rdd1.cartesian(rdd2) - result = cart.map(lambda (x, y): x + y).collect() + result = cart.map(lambda x_y3: x_y3[0] + x_y3[1]).collect() def test_transforming_pickle_file(self): # Regression test for SPARK-2601 - data = self.sc.parallelize(["Hello", "World!"]) + data = self.sc.parallelize([u"Hello", u"World!"]) tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsPickleFile(tempFile.name) @@ -461,13 +484,13 @@ def test_cartesian_on_textfile(self): a = self.sc.textFile(path) result = a.cartesian(a).collect() (x, y) = result[0] - self.assertEqual("Hello World!", x.strip()) - self.assertEqual("Hello World!", y.strip()) + self.assertEqual(u"Hello World!", x.strip()) + self.assertEqual(u"Hello World!", y.strip()) def test_deleting_input_files(self): # Regression test for SPARK-1025 tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name) filtered_data = data.filter(lambda x: True) @@ -510,21 +533,21 @@ def test_namedtuple_in_rdd(self): jon = Person(1, "Jon", "Doe") jane = Person(2, "Jane", "Doe") theDoes = self.sc.parallelize([jon, jane]) - self.assertEquals([jon, jane], theDoes.collect()) + self.assertEqual([jon, jane], theDoes.collect()) def test_large_broadcast(self): N = 100000 data = [[float(i) for i in range(300)] for i in range(N)] bdata = self.sc.broadcast(data) # 270MB m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() - self.assertEquals(N, m) + self.assertEqual(N, m) def test_multiple_broadcasts(self): N = 1 << 21 b1 = self.sc.broadcast(set(range(N))) # multiple blocks in JVM - r = range(1 << 15) + r = list(range(1 << 15)) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -535,7 +558,7 @@ def test_multiple_broadcasts(self): self.assertEqual(checksum, csum) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -549,7 +572,7 @@ def test_large_closure(self): N = 1000000 data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) - self.assertEquals(N, rdd.first()) + self.assertEqual(N, rdd.first()) # regression test for SPARK-6886 self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count()) @@ -590,15 +613,15 @@ def test_zip_with_different_number_of_items(self): # same total number of items, but different distributions a = self.sc.parallelize([2, 3], 2).flatMap(range) b = self.sc.parallelize([3, 2], 2).flatMap(range) - self.assertEquals(a.count(), b.count()) + self.assertEqual(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) def test_count_approx_distinct(self): rdd = self.sc.parallelize(range(1000)) - self.assertTrue(950 < rdd.countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.04) < 1050) + self.assertTrue(950 < rdd.countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.03) < 1050) rdd = self.sc.parallelize([i % 20 for i in range(1000)], 7) self.assertTrue(18 < rdd.countApproxDistinct() < 22) @@ -612,59 +635,59 @@ def test_count_approx_distinct(self): def test_histogram(self): # empty rdd = self.sc.parallelize([]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) self.assertRaises(ValueError, lambda: rdd.histogram(1)) # out of range rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram((0, 4, 10))[1]) # in range with one bucket rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals([4], rdd.histogram([0, 10])[1]) - self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([4], rdd.histogram([0, 10])[1]) + self.assertEqual([3, 1], rdd.histogram([0, 4, 10])[1]) # in range with one bucket exact match - self.assertEquals([4], rdd.histogram([1, 4])[1]) + self.assertEqual([4], rdd.histogram([1, 4])[1]) # out of range with two buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 5, 10])[1]) # out of range with two uneven buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) # in range with two buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two bucket and None rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two uneven buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 11])[1]) # mixed range with two uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) - self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([4, 3], rdd.histogram([0, 5, 11])[1]) # mixed range with four uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # mixed range with uneven buckets and NaN rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, None, float('nan')]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # out of range with infinite buckets rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) - self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + self.assertEqual([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) # invalid buckets self.assertRaises(ValueError, lambda: rdd.histogram([])) @@ -674,25 +697,25 @@ def test_histogram(self): # without buckets rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + self.assertEqual(([1, 4], [4]), rdd.histogram(1)) # without buckets single element rdd = self.sc.parallelize([1]) - self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + self.assertEqual(([1, 1], [1]), rdd.histogram(1)) # without bucket no range rdd = self.sc.parallelize([1] * 4) - self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + self.assertEqual(([1, 1], [4]), rdd.histogram(1)) # without buckets basic two rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + self.assertEqual(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) # without buckets with more requested than elements rdd = self.sc.parallelize([1, 2]) buckets = [1 + 0.2 * i for i in range(6)] hist = [1, 0, 0, 0, 1] - self.assertEquals((buckets, hist), rdd.histogram(5)) + self.assertEqual((buckets, hist), rdd.histogram(5)) # invalid RDDs rdd = self.sc.parallelize([1, float('inf')]) @@ -702,15 +725,8 @@ def test_histogram(self): # string rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) - self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) - self.assertRaises(TypeError, lambda: rdd.histogram(2)) - - # mixed RDD - rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) - self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) - self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertEqual([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEqual((["ab", "ef"], [5]), rdd.histogram(1)) self.assertRaises(TypeError, lambda: rdd.histogram(2)) def test_repartitionAndSortWithinPartitions(self): @@ -718,31 +734,31 @@ def test_repartitionAndSortWithinPartitions(self): repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2) partitions = repartitioned.glom().collect() - self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) - self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + self.assertEqual(partitions[0], [(0, 5), (0, 8), (2, 6)]) + self.assertEqual(partitions[1], [(1, 3), (3, 8), (3, 8)]) def test_distinct(self): rdd = self.sc.parallelize((1, 2, 3)*10, 10) - self.assertEquals(rdd.getNumPartitions(), 10) - self.assertEquals(rdd.distinct().count(), 3) + self.assertEqual(rdd.getNumPartitions(), 10) + self.assertEqual(rdd.distinct().count(), 3) result = rdd.distinct(5) - self.assertEquals(result.getNumPartitions(), 5) - self.assertEquals(result.count(), 3) + self.assertEqual(result.getNumPartitions(), 5) + self.assertEqual(result.count(), 3) def test_external_group_by_key(self): - self.sc._conf.set("spark.python.worker.memory", "5m") + self.sc._conf.set("spark.python.worker.memory", "1m") N = 200001 kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x)) gkv = kv.groupByKey().cache() self.assertEqual(3, gkv.count()) - filtered = gkv.filter(lambda (k, vs): k == 1) + filtered = gkv.filter(lambda kv: kv[0] == 1) self.assertEqual(1, filtered.count()) - self.assertEqual([(1, N/3)], filtered.mapValues(len).collect()) - self.assertEqual([(N/3, N/3)], + self.assertEqual([(1, N // 3)], filtered.mapValues(len).collect()) + self.assertEqual([(N // 3, N // 3)], filtered.values().map(lambda x: (len(x), len(list(x)))).collect()) result = filtered.collect()[0][1] - self.assertEqual(N/3, len(result)) - self.assertTrue(isinstance(result.data, shuffle.ExternalList)) + self.assertEqual(N // 3, len(result)) + self.assertTrue(isinstance(result.data, shuffle.ExternalListOfList)) def test_sort_on_empty_rdd(self): self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect()) @@ -767,7 +783,7 @@ def test_null_in_rdd(self): rdd = RDD(jrdd, self.sc, UTF8Deserializer()) self.assertEqual([u"a", None, u"b"], rdd.collect()) rdd = RDD(jrdd, self.sc, NoOpSerializer()) - self.assertEqual(["a", None, "b"], rdd.collect()) + self.assertEqual([b"a", None, b"b"], rdd.collect()) def test_multiple_python_java_RDD_conversions(self): # Regression test for SPARK-5361 @@ -813,14 +829,14 @@ def test_narrow_dependency_in_join(self): self.sc.setJobGroup("test3", "test", True) d = sorted(parted.cogroup(parted).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test3")[0] self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds)) self.sc.setJobGroup("test4", "test", True) d = sorted(parted.cogroup(rdd).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test4")[0] self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) @@ -906,6 +922,7 @@ def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ints = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfint/", @@ -954,15 +971,16 @@ def test_sequencefiles(self): en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] self.assertEqual(nulls, en) - maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) + maps = self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable").collect() em = [(1, {}), (1, {3.0: u'bb'}), (2, {1.0: u'aa'}), (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] - self.assertEqual(maps, em) + for v in maps: + self.assertTrue(v in em) # arrays get pickled to tuples by default tuples = sorted(self.sc.sequenceFile( @@ -1089,8 +1107,8 @@ def test_converters(self): def test_binary_files(self): path = os.path.join(self.tempdir.name, "binaryfiles") os.mkdir(path) - data = "short binary data" - with open(os.path.join(path, "part-0000"), 'w') as f: + data = b"short binary data" + with open(os.path.join(path, "part-0000"), 'wb') as f: f.write(data) [(p, d)] = self.sc.binaryFiles(path).collect() self.assertTrue(p.endswith("part-0000")) @@ -1103,7 +1121,7 @@ def test_binary_records(self): for i in range(100): f.write('%04d' % i) result = self.sc.binaryRecords(path, 4).map(int).collect() - self.assertEqual(range(100), result) + self.assertEqual(list(range(100)), result) class OutputFormatTests(ReusedPySparkTestCase): @@ -1115,6 +1133,7 @@ def setUp(self): def tearDown(self): shutil.rmtree(self.tempdir.name, ignore_errors=True) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] @@ -1155,8 +1174,9 @@ def test_sequencefiles(self): (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/") - maps = sorted(self.sc.sequenceFile(basepath + "/sfmap/").collect()) - self.assertEqual(maps, em) + maps = self.sc.sequenceFile(basepath + "/sfmap/").collect() + for v in maps: + self.assertTrue(v, em) def test_oldhadoop(self): basepath = self.tempdir.name @@ -1168,12 +1188,13 @@ def test_oldhadoop(self): "org.apache.hadoop.mapred.SequenceFileOutputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable") - result = sorted(self.sc.hadoopFile( + result = self.sc.hadoopFile( basepath + "/oldhadoop/", "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) - self.assertEqual(result, dict_data) + "org.apache.hadoop.io.MapWritable").collect() + for v in result: + self.assertTrue(v, dict_data) conf = { "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", @@ -1183,12 +1204,13 @@ def test_oldhadoop(self): } self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) input_conf = {"mapred.input.dir": basepath + "/olddataset/"} - old_dataset = sorted(self.sc.hadoopRDD( + result = self.sc.hadoopRDD( "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable", - conf=input_conf).collect()) - self.assertEqual(old_dataset, dict_data) + conf=input_conf).collect() + for v in result: + self.assertTrue(v, dict_data) def test_newhadoop(self): basepath = self.tempdir.name @@ -1223,6 +1245,7 @@ def test_newhadoop(self): conf=input_conf).collect()) self.assertEqual(new_dataset, data) + @unittest.skipIf(sys.version >= "3", "serialize of array") def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays @@ -1303,7 +1326,7 @@ def test_reserialization(self): basepath = self.tempdir.name x = range(1, 5) y = range(1001, 1005) - data = zip(x, y) + data = list(zip(x, y)) rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y)) rdd.saveAsSequenceFile(basepath + "/reserialize/sequence") result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) @@ -1354,7 +1377,7 @@ def connect(self, port): sock = socket(AF_INET, SOCK_STREAM) sock.connect(('127.0.0.1', port)) # send a split index of -1 to shutdown the worker - sock.send("\xFF\xFF\xFF\xFF") + sock.send(b"\xFF\xFF\xFF\xFF") sock.close() return True @@ -1395,7 +1418,6 @@ def test_termination_sigterm(self): class WorkerTests(PySparkTestCase): - def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) temp.close() @@ -1410,7 +1432,7 @@ def sleep(x): # start job in background thread def run(): - self.sc.parallelize(range(1)).foreach(sleep) + self.sc.parallelize(range(1), 1).foreach(sleep) import threading t = threading.Thread(target=run) t.daemon = True @@ -1419,7 +1441,8 @@ def run(): daemon_pid, worker_pid = 0, 0 while True: if os.path.exists(path): - data = open(path).read().split(' ') + with open(path) as f: + data = f.read().split(' ') daemon_pid, worker_pid = map(int, data) break time.sleep(0.1) @@ -1455,7 +1478,7 @@ def raise_exception(_): def test_after_jvm_exception(self): tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name, 1) filtered_data = data.filter(lambda x: True) @@ -1577,12 +1600,12 @@ def test_single_script(self): |from pyspark import SparkContext | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect() + |print(sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) def test_script_with_local_functions(self): """Submit and test a single script file calling a global function""" @@ -1593,12 +1616,12 @@ def test_script_with_local_functions(self): | return x * 3 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[3, 6, 9]", out) + self.assertIn("[3, 6, 9]", out.decode('utf-8')) def test_module_dependency(self): """Submit and test a script with a dependency on another module""" @@ -1607,7 +1630,7 @@ def test_module_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1617,7 +1640,7 @@ def test_module_dependency(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_module_dependency_on_cluster(self): """Submit and test a script with a dependency on another module on a cluster""" @@ -1626,7 +1649,7 @@ def test_module_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1637,7 +1660,7 @@ def test_module_dependency_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency(self): """Submit and test a script with a dependency on a Spark Package""" @@ -1646,14 +1669,14 @@ def test_package_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", "file:" + self.programDir, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency_on_cluster(self): """Submit and test a script with a dependency on a Spark Package on a cluster""" @@ -1662,7 +1685,7 @@ def test_package_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", @@ -1670,7 +1693,7 @@ def test_package_dependency_on_cluster(self): "local-cluster[1,1,512]", script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_single_script_on_cluster(self): """Submit and test a single script on a cluster""" @@ -1681,7 +1704,7 @@ def test_single_script_on_cluster(self): | return x * 2 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) # this will fail if you have different spark.executor.memory # in conf/spark-defaults.conf @@ -1690,7 +1713,7 @@ def test_single_script_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) class ContextTests(unittest.TestCase): @@ -1765,7 +1788,7 @@ class SciPyTests(PySparkTestCase): def test_serialize(self): from scipy.special import gammaln x = range(1, 5) - expected = map(gammaln, x) + expected = list(map(gammaln, x)) observed = self.sc.parallelize(x).map(gammaln).collect() self.assertEqual(expected, observed) @@ -1786,11 +1809,11 @@ def test_statcounter_array(self): if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") if not _have_numpy: - print "NOTE: Skipping NumPy tests as it does not seem to be installed" + print("NOTE: Skipping NumPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") if not _have_numpy: - print "NOTE: NumPy tests were skipped as it does not seem to be installed" + print("NOTE: NumPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 452d6fabdcc17..fbdaf3a5814cd 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -18,6 +18,7 @@ """ Worker that receives input from Piped RDD. """ +from __future__ import print_function import os import sys import time @@ -37,9 +38,9 @@ def report_times(outfile, boot, init, finish): write_int(SpecialLengths.TIMING_DATA, outfile) - write_long(1000 * boot, outfile) - write_long(1000 * init, outfile) - write_long(1000 * finish, outfile) + write_long(int(1000 * boot), outfile) + write_long(int(1000 * init), outfile) + write_long(int(1000 * finish), outfile) def add_path(path): @@ -72,6 +73,9 @@ def main(infile, outfile): for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) add_path(os.path.join(spark_files_dir, filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) @@ -106,14 +110,14 @@ def process(): except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc(), outfile) + write_with_length(traceback.format_exc().encode("utf-8"), outfile) except IOError: # JVM close the socket pass except Exception: # Write the error to stderr if it happened while serializing - print >> sys.stderr, "PySpark worker failed with exception:" - print >> sys.stderr, traceback.format_exc() + print("PySpark worker failed with exception:", file=sys.stderr) + print(traceback.format_exc(), file=sys.stderr) exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) diff --git a/python/run-tests b/python/run-tests index f3a07d8aba562..ed3e819ef30c1 100755 --- a/python/run-tests +++ b/python/run-tests @@ -66,7 +66,7 @@ function run_core_tests() { function run_sql_tests() { echo "Run sql tests ..." - run_test "pyspark/sql/types.py" + run_test "pyspark/sql/_types.py" run_test "pyspark/sql/context.py" run_test "pyspark/sql/dataframe.py" run_test "pyspark/sql/functions.py" @@ -136,6 +136,19 @@ run_mllib_tests run_ml_tests run_streaming_tests +# Try to test with Python 3 +if [ $(which python3.4) ]; then + export PYSPARK_PYTHON="python3.4" + echo "Testing with Python3.4 version:" + $PYSPARK_PYTHON --version + + run_core_tests + run_sql_tests + run_mllib_tests + run_ml_tests + run_streaming_tests +fi + # Try to test with PyPy if [ $(which pypy) ]; then export PYSPARK_PYTHON="pypy" diff --git a/python/test_support/userlib-0.1-py2.7.egg b/python/test_support/userlib-0.1-py2.7.egg deleted file mode 100644 index 1674c9cb2227e160aec55c74e9ba1ddba850f580..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1945 zcmWIWW@Zs#U|`^2Sdq%-{OPC4zkVQZ3lQ@IacOaCQBG!(er{rBo?bzvZr?^OCPM+2 z-!(^Pyju5>Ifws9kif>ilh5?x&a}Ph`t|+UfpA{q4-(mHcXF|6ID3k0S`rmJ^Pu27 zX2pP|PDj;+_Dt5>5pw-nXZI!d?|Lntzt1q@eSGhRp0S;^U`hJyZ6+W7I>sGuc2o~I z;$-<*VantEcQTf&4@>aB-66eQMnKwrV%L&?hZb7x-!-%B+8>4hxZ7LO_?&A_9oL=$ zbn|W?2Kfl)_W1bByv&mLc%b`}lmDGLz4eV;d4LhKhymk9W&!bThE5TMq?8kh2`5rh zI+9Wnd=is5t|hQ}rTk-O;&oG4-6HJKBDP%2^0|tb`0vbyFVikRj(5)S6>%#*g z4>4Xznml`c(5%T>?3=boqzIt-qG4rZelQ~gLjn^6g8-5*pfQlVbi#eF!v-Slm#_J* zt#~NyvVK)UwfP*@J=O@3Im_WVolA58N~>g61`pSxe0_vE)< z^ZdsjUjEk5J^74tL*7Q7qYqkq3~ia@R@xk@{p5N=T`@m+{)dK?i}POB9N~!PDabP2 znxJE9d~uu8)sXNlS7bEW#Y9#FH7t}rbS~rCraZfArw&~@*7N!K)x8r0<{n>Yd*t

    waS|`|O?l3T$=A=7qdQyq0*MfDE zS-ZRgRd|K9^1{m+-Ws>d>^fX$@A>54M%x1?ijPX2D3<;<+wF7fk=$4Lh4pO9-@=?k}n_0dd;3x|M^$py6en$SghCm=SC!>>%c6)9BG#!3k-N^AQnc7 z0HkE(t$pfz=n3C5XM#_h(LJHB7099Mr(1N+Q%CO^6IygS{lCb@1vG^Nh{b@|)!kj! z)6dOcza+mPJ}0#-HAkN)2L?f8-Bp`(* z^5{v|nXIY_yAzriEjv{+f4c7E5CpoBkx7IZ6hzp|O`rrAY-t2hu#yOxhmkd7E4Uz9 zfrW769wg03=`R`G1oT1!VL}ry>7ZGUq8nR^N9bk-CO(*MB>T~=M^EGk0|J2tz!MQl zl1DcKJ*gwi=tnjKmhkau2c>%$*wT0qSv$5|fNm6eCO{bVAK56REP-wUdYVO;Fo^{z YJYXp}z?+o~q=XX)eSyWk87qhf04R`+K>z>% diff --git a/python/test_support/userlib-0.1.zip b/python/test_support/userlib-0.1.zip new file mode 100644 index 0000000000000000000000000000000000000000..496e1349aa9674cfa4f232ea8b5e17fcdb49f4fd GIT binary patch literal 668 zcmWIWW@h1H0D&_>m2O}Ll;B{HVJIz5Ey~GE(hm*cWMDr3PAF9Jy-;Xr1vdjD%U4DQ z1{M*Z+5or-kw6n(?`LOH28x2PFp>%J@tJv%$WKZ+A#ed zj*Lul%(&u00_Z;w5MX%g2%-@&!wQKRw17l54KrRKrU7G+VN0VEl4&Th1T+{+3_%RW k6#>WwZ$dH{9vwhKLD7NXT2?loY6fN?`~#%BKu%)-0QB3qpa1{> literal 0 HcmV?d00001 From 5fe43433529346788e8c343d338a5b7dc169cf58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=91=E5=B3=A4?= Date: Thu, 16 Apr 2015 17:32:42 -0700 Subject: [PATCH 054/144] SPARK-6927 [SQL] Sorting Error when codegen on MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix this error by adding BinaryType comparor in GenerateOrdering. JIRA https://issues.apache.org/jira/browse/SPARK-6927 Author: 云峤 Closes #5524 from kaka1992/fix-codegen-sort and squashes the following commits: d7e2afe [云峤] fix codegen sorting error --- .../codegen/GenerateOrdering.scala | 14 ++++++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 20 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 0db29eb404bd1..fc2a2b60703e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NumericType} +import org.apache.spark.sql.types.{BinaryType, StringType, NumericType} /** * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of @@ -43,6 +43,18 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit val evalB = expressionEvaluator(order.child) val compare = order.child.dataType match { + case BinaryType => + q""" + val x = ${if (order.direction == Ascending) evalA.primitiveTerm else evalB.primitiveTerm} + val y = ${if (order.direction != Ascending) evalB.primitiveTerm else evalA.primitiveTerm} + var i = 0 + while (i < x.length && i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + i = i+1 + } + return x.length - y.length + """ case _: NumericType => q""" val comp = ${evalA.primitiveTerm} - ${evalB.primitiveTerm} 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 d739e550f3e56..9e02e69fda3f2 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 @@ -398,6 +398,26 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { setConf(SQLConf.EXTERNAL_SORT, before.toString) } + test("SPARK-6927 sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.EXTERNAL_SORT, "false") + setConf(SQLConf.CODEGEN_ENABLED, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + + test("SPARK-6927 external sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.CODEGEN_ENABLED, "true") + setConf(SQLConf.EXTERNAL_SORT, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + test("limit") { checkAnswer( sql("SELECT * FROM testData LIMIT 10"), From 6183b5e2caedd074073d0f6cb6609a634e2f5194 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Apr 2015 17:33:57 -0700 Subject: [PATCH 055/144] [SPARK-6911] [SQL] improve accessor for nested types Support access columns by index in Python: ``` >>> df[df[0] > 3].collect() [Row(age=5, name=u'Bob')] ``` Access items in ArrayType or MapType ``` >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() >>> df.select(df.l[0], df.d["key"]).show() ``` Access field in StructType ``` >>> df.select(df.r.getField("b")).show() >>> df.select(df.r.a).show() ``` Author: Davies Liu Closes #5513 from davies/access and squashes the following commits: e04d5a0 [Davies Liu] Update run-tests-jenkins 7ada9eb [Davies Liu] update timeout d125ac4 [Davies Liu] check column name, improve scala tests 6b62540 [Davies Liu] fix test db15b42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into access 6c32e79 [Davies Liu] add scala tests 11f1df3 [Davies Liu] improve accessor for nested types --- python/pyspark/sql/dataframe.py | 49 +++++++++++++++++-- python/pyspark/sql/tests.py | 18 +++++++ .../scala/org/apache/spark/sql/Column.scala | 7 +-- .../org/apache/spark/sql/DataFrameSuite.scala | 6 +++ .../scala/org/apache/spark/sql/TestData.scala | 9 ++-- 5 files changed, 76 insertions(+), 13 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index d76504f986270..b9a3e6cfe7f49 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -563,16 +563,23 @@ def __getitem__(self, item): [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] >>> df[ df.age > 3 ].collect() [Row(age=5, name=u'Bob')] + >>> df[df[0] > 3].collect() + [Row(age=5, name=u'Bob')] """ if isinstance(item, basestring): + if item not in self.columns: + raise IndexError("no such column: %s" % item) jc = self._jdf.apply(item) return Column(jc) elif isinstance(item, Column): return self.filter(item) - elif isinstance(item, list): + elif isinstance(item, (list, tuple)): return self.select(*item) + elif isinstance(item, int): + jc = self._jdf.apply(self.columns[item]) + return Column(jc) else: - raise IndexError("unexpected index: %s" % item) + raise TypeError("unexpected type: %s" % type(item)) def __getattr__(self, name): """Returns the :class:`Column` denoted by ``name``. @@ -580,8 +587,8 @@ def __getattr__(self, name): >>> df.select(df.age).collect() [Row(age=2), Row(age=5)] """ - if name.startswith("__"): - raise AttributeError(name) + if name not in self.columns: + raise AttributeError("No such column: %s" % name) jc = self._jdf.apply(name) return Column(jc) @@ -1093,7 +1100,39 @@ def __init__(self, jc): # container operators __contains__ = _bin_op("contains") __getitem__ = _bin_op("getItem") - getField = _bin_op("getField", "An expression that gets a field by name in a StructField.") + + def getItem(self, key): + """An expression that gets an item at position `ordinal` out of a list, + or gets an item by key out of a dict. + + >>> df = sc.parallelize([([1, 2], {"key": "value"})]).toDF(["l", "d"]) + >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() + l[0] d[key] + 1 value + >>> df.select(df.l[0], df.d["key"]).show() + l[0] d[key] + 1 value + """ + return self[key] + + def getField(self, name): + """An expression that gets a field by name in a StructField. + + >>> from pyspark.sql import Row + >>> df = sc.parallelize([Row(r=Row(a=1, b="b"))]).toDF() + >>> df.select(df.r.getField("b")).show() + r.b + b + >>> df.select(df.r.a).show() + r.a + 1 + """ + return Column(self._jc.getField(name)) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + return self.getField(item) # string methods rlike = _bin_op("rlike") diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 7c09a0cfe30ab..6691e8c8dc44b 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -426,6 +426,24 @@ def test_help_command(self): pydoc.render_doc(df.foo) pydoc.render_doc(df.take(1)) + def test_access_column(self): + df = self.df + self.assertTrue(isinstance(df.key, Column)) + self.assertTrue(isinstance(df['key'], Column)) + self.assertTrue(isinstance(df[0], Column)) + self.assertRaises(IndexError, lambda: df[2]) + self.assertRaises(IndexError, lambda: df["bad_key"]) + self.assertRaises(TypeError, lambda: df[{}]) + + def test_access_nested_types(self): + df = self.sc.parallelize([Row(l=[1], r=Row(a=1, b="b"), d={"k": "v"})]).toDF() + self.assertEqual(1, df.select(df.l[0]).first()[0]) + self.assertEqual(1, df.select(df.l.getItem(0)).first()[0]) + self.assertEqual(1, df.select(df.r.a).first()[0]) + self.assertEqual("b", df.select(df.r.getField("b")).first()[0]) + self.assertEqual("v", df.select(df.d["k"]).first()[0]) + self.assertEqual("v", df.select(df.d.getItem("k")).first()[0]) + def test_infer_long_type(self): longrow = [Row(f1='a', f2=100000000000000)] df = self.sc.parallelize(longrow).toDF() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 3cd7adf8cab5e..edb229c059e6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -515,14 +515,15 @@ class Column(protected[sql] val expr: Expression) extends Logging { def rlike(literal: String): Column = RLike(expr, lit(literal).expr) /** - * An expression that gets an item at position `ordinal` out of an array. + * An expression that gets an item at position `ordinal` out of an array, + * or gets a value by key `key` in a [[MapType]]. * * @group expr_ops */ - def getItem(ordinal: Int): Column = GetItem(expr, Literal(ordinal)) + def getItem(key: Any): Column = GetItem(expr, Literal(key)) /** - * An expression that gets a field by name in a [[StructField]]. + * An expression that gets a field by name in a [[StructType]]. * * @group expr_ops */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index b26e22f6229fe..34b2cb054a3e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -86,6 +86,12 @@ class DataFrameSuite extends QueryTest { TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, oldSetting.toString) } + test("access complex data") { + assert(complexData.filter(complexData("a").getItem(0) === 2).count() == 1) + assert(complexData.filter(complexData("m").getItem("1") === 1).count() == 1) + assert(complexData.filter(complexData("s").getField("key") === 1).count() == 1) + } + test("table scan") { checkAnswer( testData, 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 637f59b2e68ca..225b51bd73d6c 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 @@ -20,9 +20,8 @@ package org.apache.spark.sql import java.sql.Timestamp import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test._ import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.test._ case class TestData(key: Int, value: String) @@ -199,11 +198,11 @@ object TestData { Salary(1, 1000.0) :: Nil).toDF() salary.registerTempTable("salary") - case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) + case class ComplexData(m: Map[String, Int], s: TestData, a: Seq[Int], b: Boolean) val complexData = TestSQLContext.sparkContext.parallelize( - ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true) - :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false) + ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1), true) + :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2), false) :: Nil).toDF() complexData.registerTempTable("complexData") } From d96608674f6c2ff3abb13c65d80c1a3872206710 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 16 Apr 2015 17:35:51 -0700 Subject: [PATCH 056/144] [SQL][Minor] Fix foreachUp of treenode `foreachUp` should runs the given function recursively on [[children]] then on this node(just like transformUp). The current implementation does not follow this. This will leads to checkanalysis do not check from bottom of logical tree. Author: scwf Author: Fei Wang Closes #5518 from scwf/patch-1 and squashes the following commits: 18e28b2 [scwf] added a test case 1ccbfa8 [Fei Wang] fix foreachUp --- .../apache/spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../spark/sql/catalyst/trees/TreeNodeSuite.scala | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index a2df51e598a2b..97502ed3afe72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -85,7 +85,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param f the function to be applied to each node in the tree. */ def foreachUp(f: BaseType => Unit): Unit = { - children.foreach(_.foreach(f)) + children.foreach(_.foreachUp(f)) f(this) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 4eb8708335dcf..6b393327cc97a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -117,5 +117,17 @@ class TreeNodeSuite extends FunSuite { assert(transformed.origin.startPosition.isDefined) } + test("foreach up") { + val actual = new ArrayBuffer[String]() + val expected = Seq("1", "2", "3", "4", "-", "*", "+") + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + expression foreachUp { + case b: BinaryExpression => actual.append(b.symbol); + case l: Literal => actual.append(l.toString); + } + + assert(expected === actual) + } + } From 1e43851d6455f65b850ea0327d0e92f65395d23f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 16 Apr 2015 17:50:20 -0700 Subject: [PATCH 057/144] [SPARK-6899][SQL] Fix type mismatch when using codegen with Average on DecimalType JIRA https://issues.apache.org/jira/browse/SPARK-6899 Author: Liang-Chi Hsieh Closes #5517 from viirya/fix_codegen_average and squashes the following commits: 8ae5f65 [Liang-Chi Hsieh] Add the case of DecimalType.Unlimited to Average. --- .../spark/sql/catalyst/expressions/aggregates.scala | 2 +- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) 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 14a855054b94d..f3830c6d3bcf2 100644 --- 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 @@ -326,7 +326,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN override def asPartial: SplitEvaluation = { child.dataType match { - case DecimalType.Fixed(_, _) => + case DecimalType.Fixed(_, _) | DecimalType.Unlimited => // Turn the child to unlimited decimals for calculation, before going back to fixed val partialSum = Alias(Sum(Cast(child, DecimalType.Unlimited)), "PartialSum")() val partialCount = Alias(Count(child), "PartialCount")() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 34b2cb054a3e7..44a7d1e7bbb6a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -537,4 +537,13 @@ class DataFrameSuite extends QueryTest { val df = TestSQLContext.createDataFrame(rowRDD, schema) df.rdd.collect() } + + test("SPARK-6899") { + val originalValue = TestSQLContext.conf.codegenEnabled + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, "true") + checkAnswer( + decimalData.agg(avg('a)), + Row(new java.math.BigDecimal(2.0))) + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) + } } From e5949c287ed19e78b6eecc61c3e88a07ad452eb9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Apr 2015 17:59:49 -0700 Subject: [PATCH 058/144] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver Otherwise we cannot add jars with drivers after the fact. Author: Michael Armbrust Closes #5543 from marmbrus/jdbcClassloader and squashes the following commits: d9930f3 [Michael Armbrust] fix imports 73d0614 [Michael Armbrust] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver --- .../main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 99b755c9f25d0..5f480083d5a49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils /** * Data corresponding to one partition of a JDBCRDD. @@ -99,7 +100,7 @@ private[sql] class DefaultSource extends RelationProvider { val upperBound = parameters.getOrElse("upperBound", null) val numPartitions = parameters.getOrElse("numPartitions", null) - if (driver != null) Class.forName(driver) + if (driver != null) Utils.getContextOrSparkClassLoader.loadClass(driver) if (partitionColumn != null && (lowerBound == null || upperBound == null || numPartitions == null)) { From 8220d5265f1bbea9dfdaeec4f2d06d7fe24c0bc3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Apr 2015 21:49:26 -0500 Subject: [PATCH 059/144] [SPARK-6972][SQL] Add Coalesce to DataFrame Author: Michael Armbrust Closes #5545 from marmbrus/addCoalesce and squashes the following commits: 9fdf3f6 [Michael Armbrust] [SPARK-6972][SQL] Add Coalesce to DataFrame --- .../scala/org/apache/spark/sql/DataFrame.scala | 14 ++++++++++++++ .../main/scala/org/apache/spark/sql/RDDApi.scala | 2 ++ .../org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ 3 files changed, 24 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 3235f85d5bbd2..17c21f6e3a0e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -908,6 +908,20 @@ class DataFrame private[sql]( schema, needsConversion = false) } + /** + * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. + * Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g. + * if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of + * the 100 new partitions will claim 10 of the current partitions. + * @group rdd + */ + override def coalesce(numPartitions: Int): DataFrame = { + sqlContext.createDataFrame( + queryExecution.toRdd.coalesce(numPartitions), + schema, + needsConversion = false) + } + /** * Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. * @group dfops diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala index ba4373f0124b4..63dbab19947c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala @@ -61,5 +61,7 @@ private[sql] trait RDDApi[T] { def repartition(numPartitions: Int): DataFrame + def coalesce(numPartitions: Int): DataFrame + def distinct: DataFrame } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 44a7d1e7bbb6a..3250ab476aeb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -178,6 +178,14 @@ class DataFrameSuite extends QueryTest { testData.select('key).collect().toSeq) } + test("coalesce") { + assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) + + checkAnswer( + testData.select('key).coalesce(1).select('key), + testData.select('key).collect().toSeq) + } + test("groupBy") { checkAnswer( testData2.groupBy("a").agg($"a", sum($"b")), From f7a25644ed5b3b49fe7f33743bec3d95cdf7913e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 17 Apr 2015 11:02:31 +0100 Subject: [PATCH 060/144] SPARK-6846 [WEBUI] Stage kill URL easy to accidentally trigger and possibility for security issue kill endpoints now only accept a POST (kill stage, master kill app, master kill driver); kill link now POSTs Author: Sean Owen Closes #5528 from srowen/SPARK-6846 and squashes the following commits: 137ac9f [Sean Owen] Oops, fix scalastyle line length probelm 7c5f961 [Sean Owen] Add Imran's test of kill link 59f447d [Sean Owen] kill endpoints now only accept a POST (kill stage, master kill app, master kill driver); kill link now POSTs --- .../org/apache/spark/ui/static/webui.css | 6 +-- .../spark/deploy/master/ui/MasterPage.scala | 28 +++++++------ .../spark/deploy/master/ui/MasterWebUI.scala | 8 ++-- .../org/apache/spark/ui/JettyUtils.scala | 17 +++++++- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../org/apache/spark/ui/jobs/StageTable.scala | 27 ++++++------- .../org/apache/spark/ui/UISeleniumSuite.scala | 40 +++++++++++++------ 7 files changed, 78 insertions(+), 52 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 6c37cc8b98236..4910744d1d790 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 @@ -85,17 +85,13 @@ table.sortable td { filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } -span.kill-link { +a.kill-link { margin-right: 2px; margin-left: 20px; color: gray; float: right; } -span.kill-link a { - color: gray; -} - span.expand-details { font-size: 10pt; cursor: pointer; diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 399f07399a0aa..1f2c3fdbfb2bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -190,12 +190,14 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def appRow(app: ApplicationInfo): Seq[Node] = { val killLink = if (parent.killEnabled && (app.state == ApplicationState.RUNNING || app.state == ApplicationState.WAITING)) { - val killLinkUri = s"app/kill?id=${app.id}&terminate=true" - val confirm = "return window.confirm(" + - s"'Are you sure you want to kill application ${app.id} ?');" - - (kill) - + val confirm = + s"if (window.confirm('Are you sure you want to kill application ${app.id} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" +
    + + + (kill) + }

    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 1b670418ab1ff..bb11e0642ddc6 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 @@ -43,10 +43,10 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler( - createRedirectHandler("/app/kill", "/", masterPage.handleAppKillRequest)) - attachHandler( - createRedirectHandler("/driver/kill", "/", masterPage.handleDriverKillRequest)) + attachHandler(createRedirectHandler( + "/app/kill", "/", masterPage.handleAppKillRequest, httpMethod = "POST")) + attachHandler(createRedirectHandler( + "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethod = "POST")) } /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 95f254a9ef22a..a091ca650c60c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -114,10 +114,23 @@ private[spark] object JettyUtils extends Logging { srcPath: String, destPath: String, beforeRedirect: HttpServletRequest => Unit = x => (), - basePath: String = ""): ServletContextHandler = { + basePath: String = "", + httpMethod: String = "GET"): ServletContextHandler = { val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { - override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + override def doGet(request: HttpServletRequest, response: HttpServletResponse): Unit = { + httpMethod match { + case "GET" => doRequest(request, response) + case _ => response.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } + override def doPost(request: HttpServletRequest, response: HttpServletResponse): Unit = { + httpMethod match { + case "POST" => doRequest(request, response) + case _ => response.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } + private def doRequest(request: HttpServletRequest, response: HttpServletResponse): Unit = { beforeRedirect(request) // Make sure we don't end up with "//" in the middle val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString 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 adfa6bbada256..580ab8b1325f8 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -55,8 +55,8 @@ private[spark] class SparkUI private ( attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) - attachHandler( - createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) + attachHandler(createRedirectHandler( + "/stages/stage/kill", "/stages", stagesTab.handleKillRequest, httpMethod = "POST")) } initialize() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5865850fa09b5..cb72890a0fd20 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -73,20 +73,21 @@ private[ui] class StageTableBase( } private def makeDescription(s: StageInfo): Seq[Node] = { - // scalastyle:off + val basePathUri = UIUtils.prependBaseUri(basePath) + val killLink = if (killEnabled) { - val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" - .format(UIUtils.prependBaseUri(basePath), s.stageId) - val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" - .format(s.stageId) - - (kill) - + val killLinkUri = s"$basePathUri/stages/stage/kill/" + val confirm = + s"if (window.confirm('Are you sure you want to kill stage ${s.stageId} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" + + + + (kill) + } - // scalastyle:on - val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" - .format(UIUtils.prependBaseUri(basePath), s.stageId, s.attemptId) + val nameLinkUri = s"$basePathUri/stages/stage?id=${s.stageId}&attempt=${s.attemptId}" val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -98,11 +99,9 @@ private[ui] class StageTableBase( diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 1cb594633f331..eb9db550fd74c 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui +import java.net.{HttpURLConnection, URL} import javax.servlet.http.HttpServletRequest import scala.collection.JavaConversions._ @@ -56,12 +57,13 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before * Create a test SparkContext with the SparkUI enabled. * It is safe to `get` the SparkUI directly from the SparkContext returned here. */ - private def newSparkContext(): SparkContext = { + private def newSparkContext(killEnabled: Boolean = true): SparkContext = { val conf = new SparkConf() .setMaster("local") .setAppName("test") .set("spark.ui.enabled", "true") .set("spark.ui.port", "0") + .set("spark.ui.killEnabled", killEnabled.toString) val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -128,21 +130,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } test("spark.ui.killEnabled should properly control kill button display") { - def getSparkContext(killEnabled: Boolean): SparkContext = { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.killEnabled", killEnabled.toString) - new SparkContext(conf) - } - def hasKillLink: Boolean = find(className("kill-link")).isDefined def runSlowJob(sc: SparkContext) { sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() } - withSpark(getSparkContext(killEnabled = true)) { sc => + withSpark(newSparkContext(killEnabled = true)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") @@ -150,7 +143,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } - withSpark(getSparkContext(killEnabled = false)) { sc => + withSpark(newSparkContext(killEnabled = false)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") @@ -233,7 +226,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // because someone could change the error message and cause this test to pass by accident. // Instead, it's safer to check that each row contains a link to a stage details page. findAll(cssSelector("tbody tr")).foreach { row => - val link = row.underlying.findElement(By.xpath(".//a")) + val link = row.underlying.findElement(By.xpath("./td/div/a")) link.getAttribute("href") should include ("stage") } } @@ -356,4 +349,25 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } } + + test("kill stage is POST only") { + def getResponseCode(url: URL, method: String): Int = { + val connection = url.openConnection().asInstanceOf[HttpURLConnection] + connection.setRequestMethod(method) + connection.connect() + val code = connection.getResponseCode() + connection.disconnect() + code + } + + withSpark(newSparkContext(killEnabled = true)) { sc => + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + val url = new URL( + sc.ui.get.appUIAddress.stripSuffix("/") + "/stages/stage/kill/?id=0&terminate=true") + getResponseCode(url, "GET") should be (405) + getResponseCode(url, "POST") should be (200) + } + } + } } From 4527761bcd6501c362baf2780905a0018b9a74ba Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Apr 2015 11:06:01 +0100 Subject: [PATCH 061/144] [SPARK-6046] [core] Reorganize deprecated config support in SparkConf. This change tries to follow the chosen way for handling deprecated configs in SparkConf: all values (old and new) are kept in the conf object, and newer names take precedence over older ones when retrieving the value. Warnings are logged when config options are set, which generally happens on the driver node (where the logs are most visible). Author: Marcelo Vanzin Closes #5514 from vanzin/SPARK-6046 and squashes the following commits: 9371529 [Marcelo Vanzin] Avoid math. 6cf3f11 [Marcelo Vanzin] Review feedback. 2445d48 [Marcelo Vanzin] Fix (and cleanup) update interval initialization. b6824be [Marcelo Vanzin] Clean up the other deprecated config use also. ab20351 [Marcelo Vanzin] Update FsHistoryProvider to only retrieve new config key. 2c93209 [Marcelo Vanzin] [SPARK-6046] [core] Reorganize deprecated config support in SparkConf. --- .../scala/org/apache/spark/SparkConf.scala | 174 ++++++++++-------- .../deploy/history/FsHistoryProvider.scala | 9 +- .../org/apache/spark/executor/Executor.scala | 5 +- .../org/apache/spark/SparkConfSuite.scala | 22 +++ docs/monitoring.md | 6 +- .../org/apache/spark/deploy/yarn/Client.scala | 3 +- 6 files changed, 124 insertions(+), 95 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 390e631647bd6..b0186e9a007b8 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -68,6 +68,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { if (value == null) { throw new NullPointerException("null value for " + key) } + logDeprecationWarning(key) settings.put(key, value) this } @@ -134,13 +135,15 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Set multiple parameters together */ def setAll(settings: Traversable[(String, String)]): SparkConf = { - this.settings.putAll(settings.toMap.asJava) + settings.foreach { case (k, v) => set(k, v) } this } /** Set a parameter if it isn't already configured */ def setIfMissing(key: String, value: String): SparkConf = { - settings.putIfAbsent(key, value) + if (settings.putIfAbsent(key, value) == null) { + logDeprecationWarning(key) + } this } @@ -174,8 +177,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } - /** - * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no * suffix is provided then seconds are assumed. * @throws NoSuchElementException */ @@ -183,36 +186,35 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { Utils.timeStringAsSeconds(get(key)) } - /** - * Get a time parameter as seconds, falling back to a default if not set. If no + /** + * Get a time parameter as seconds, falling back to a default if not set. If no * suffix is provided then seconds are assumed. - * */ def getTimeAsSeconds(key: String, defaultValue: String): Long = { Utils.timeStringAsSeconds(get(key, defaultValue)) } - /** - * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no - * suffix is provided then milliseconds are assumed. + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. * @throws NoSuchElementException */ def getTimeAsMs(key: String): Long = { Utils.timeStringAsMs(get(key)) } - /** - * Get a time parameter as milliseconds, falling back to a default if not set. If no - * suffix is provided then milliseconds are assumed. + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. */ def getTimeAsMs(key: String, defaultValue: String): Long = { Utils.timeStringAsMs(get(key, defaultValue)) } - + /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { - Option(settings.get(key)) + Option(settings.get(key)).orElse(getDeprecatedConfig(key, this)) } /** Get all parameters as a list of pairs */ @@ -379,13 +381,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } } - - // Warn against the use of deprecated configs - deprecatedConfigs.values.foreach { dc => - if (contains(dc.oldName)) { - dc.warn() - } - } } /** @@ -400,19 +395,44 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { private[spark] object SparkConf extends Logging { + /** + * Maps deprecated config keys to information about the deprecation. + * + * The extra information is logged as a warning when the config is present in the user's + * configuration. + */ private val deprecatedConfigs: Map[String, DeprecatedConfig] = { val configs = Seq( - DeprecatedConfig("spark.files.userClassPathFirst", "spark.executor.userClassPathFirst", - "1.3"), - DeprecatedConfig("spark.yarn.user.classpath.first", null, "1.3", - "Use spark.{driver,executor}.userClassPathFirst instead."), - DeprecatedConfig("spark.history.fs.updateInterval", - "spark.history.fs.update.interval.seconds", - "1.3", "Use spark.history.fs.update.interval.seconds instead"), - DeprecatedConfig("spark.history.updateInterval", - "spark.history.fs.update.interval.seconds", - "1.3", "Use spark.history.fs.update.interval.seconds instead")) - configs.map { x => (x.oldName, x) }.toMap + DeprecatedConfig("spark.yarn.user.classpath.first", "1.3", + "Please use spark.{driver,executor}.userClassPathFirst instead.")) + Map(configs.map { cfg => (cfg.key -> cfg) }:_*) + } + + /** + * Maps a current config key to alternate keys that were used in previous version of Spark. + * + * The alternates are used in the order defined in this map. If deprecated configs are + * present in the user's configuration, a warning is logged. + */ + private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( + "spark.executor.userClassPathFirst" -> Seq( + AlternateConfig("spark.files.userClassPathFirst", "1.3")), + "spark.history.fs.update.interval" -> Seq( + AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), + AlternateConfig("spark.history.fs.updateInterval", "1.3"), + AlternateConfig("spark.history.updateInterval", "1.3")) + ) + + /** + * A view of `configsWithAlternatives` that makes it more efficient to look up deprecated + * config keys. + * + * Maps the deprecated config name to a 2-tuple (new config name, alternate config info). + */ + private val allAlternatives: Map[String, (String, AlternateConfig)] = { + configsWithAlternatives.keys.flatMap { key => + configsWithAlternatives(key).map { cfg => (cfg.key -> (key -> cfg)) } + }.toMap } /** @@ -443,61 +463,57 @@ private[spark] object SparkConf extends Logging { } /** - * Translate the configuration key if it is deprecated and has a replacement, otherwise just - * returns the provided key. - * - * @param userKey Configuration key from the user / caller. - * @param warn Whether to print a warning if the key is deprecated. Warnings will be printed - * only once for each key. + * Looks for available deprecated keys for the given config option, and return the first + * value available. */ - private def translateConfKey(userKey: String, warn: Boolean = false): String = { - deprecatedConfigs.get(userKey) - .map { deprecatedKey => - if (warn) { - deprecatedKey.warn() - } - deprecatedKey.newName.getOrElse(userKey) - }.getOrElse(userKey) + def getDeprecatedConfig(key: String, conf: SparkConf): Option[String] = { + configsWithAlternatives.get(key).flatMap { alts => + alts.collectFirst { case alt if conf.contains(alt.key) => + val value = conf.get(alt.key) + alt.translation.map(_(value)).getOrElse(value) + } + } } /** - * Holds information about keys that have been deprecated or renamed. + * Logs a warning message if the given config key is deprecated. + */ + def logDeprecationWarning(key: String): Unit = { + deprecatedConfigs.get(key).foreach { cfg => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"may be removed in the future. ${cfg.deprecationMessage}") + } + + allAlternatives.get(key).foreach { case (newKey, cfg) => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"and may be removed in the future. Please use the new key '$newKey' instead.") + } + } + + /** + * Holds information about keys that have been deprecated and do not have a replacement. * - * @param oldName Old configuration key. - * @param newName New configuration key, or `null` if key has no replacement, in which case the - * deprecated key will be used (but the warning message will still be printed). + * @param key The deprecated key. * @param version Version of Spark where key was deprecated. - * @param deprecationMessage Message to include in the deprecation warning; mandatory when - * `newName` is not provided. + * @param deprecationMessage Message to include in the deprecation warning. */ private case class DeprecatedConfig( - oldName: String, - _newName: String, + key: String, version: String, - deprecationMessage: String = null) { - - private val warned = new AtomicBoolean(false) - val newName = Option(_newName) + deprecationMessage: String) - if (newName == null && (deprecationMessage == null || deprecationMessage.isEmpty())) { - throw new IllegalArgumentException("Need new config name or deprecation message.") - } - - def warn(): Unit = { - if (warned.compareAndSet(false, true)) { - if (newName != null) { - val message = Option(deprecationMessage).getOrElse( - s"Please use the alternative '$newName' instead.") - logWarning( - s"The configuration option '$oldName' has been replaced as of Spark $version and " + - s"may be removed in the future. $message") - } else { - logWarning( - s"The configuration option '$oldName' has been deprecated as of Spark $version and " + - s"may be removed in the future. $deprecationMessage") - } - } - } + /** + * Information about an alternate configuration key that has been deprecated. + * + * @param key The deprecated config key. + * @param version The Spark version in which the key was deprecated. + * @param translation A translation function for converting old config values into new ones. + */ + private case class AlternateConfig( + key: String, + version: String, + translation: Option[String => String] = None) - } } 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 9d40d8c8fd7a8..985545742df67 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 @@ -49,11 +49,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private val NOT_STARTED = "" // Interval between each check for event log updates - private val UPDATE_INTERVAL_MS = conf.getOption("spark.history.fs.update.interval.seconds") - .orElse(conf.getOption("spark.history.fs.updateInterval")) - .orElse(conf.getOption("spark.history.updateInterval")) - .map(_.toInt) - .getOrElse(10) * 1000 + private val UPDATE_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.update.interval", "10s") // Interval between each cleaner checks for event logs to delete private val CLEAN_INTERVAL_MS = conf.getLong("spark.history.fs.cleaner.interval.seconds", @@ -130,8 +126,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // Disable the background thread during tests. if (!conf.contains("spark.testing")) { // A task that periodically checks for event log updates on disk. - pool.scheduleAtFixedRate(getRunner(checkForLogs), 0, UPDATE_INTERVAL_MS, - TimeUnit.MILLISECONDS) + pool.scheduleAtFixedRate(getRunner(checkForLogs), 0, UPDATE_INTERVAL_S, TimeUnit.SECONDS) if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) { // A task that periodically cleans event logs on disk. 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 1b5fdeba28ee2..327d155b38c22 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -89,10 +89,7 @@ private[spark] class Executor( ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME, new ExecutorEndpoint(env.rpcEnv, executorId)) // Whether to load classes in user jars before those in Spark jars - private val userClassPathFirst: Boolean = { - conf.getBoolean("spark.executor.userClassPathFirst", - conf.getBoolean("spark.files.userClassPathFirst", false)) - } + private val userClassPathFirst = conf.getBoolean("spark.executor.userClassPathFirst", false) // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index e08210ae60d17..7d87ba5fd2610 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -197,6 +197,28 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro serializer.newInstance().serialize(new StringBuffer()) } + test("deprecated configs") { + val conf = new SparkConf() + val newName = "spark.history.fs.update.interval" + + assert(!conf.contains(newName)) + + conf.set("spark.history.updateInterval", "1") + assert(conf.get(newName) === "1") + + conf.set("spark.history.fs.updateInterval", "2") + assert(conf.get(newName) === "2") + + conf.set("spark.history.fs.update.interval.seconds", "3") + assert(conf.get(newName) === "3") + + conf.set(newName, "4") + assert(conf.get(newName) === "4") + + val count = conf.getAll.filter { case (k, v) => k.startsWith("spark.history.") }.size + assert(count === 4) + } + } class Class1 {} diff --git a/docs/monitoring.md b/docs/monitoring.md index 6816671ffbf46..2a130224591ca 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -86,10 +86,10 @@ follows: - - + + 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 1091ff54b0463..52e4dee46c535 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 @@ -1052,8 +1052,7 @@ object Client extends Logging { if (isDriver) { conf.getBoolean("spark.driver.userClassPathFirst", false) } else { - conf.getBoolean("spark.executor.userClassPathFirst", - conf.getBoolean("spark.files.userClassPathFirst", false)) + conf.getBoolean("spark.executor.userClassPathFirst", false) } } From f6a9a57a72767f48fcc02e5fda4d6eafa67aebde Mon Sep 17 00:00:00 2001 From: Punya Biswal Date: Fri, 17 Apr 2015 11:08:37 +0100 Subject: [PATCH 062/144] [SPARK-6952] Handle long args when detecting PID reuse sbin/spark-daemon.sh used ps -p "$TARGET_PID" -o args= to figure out whether the process running with the expected PID is actually a Spark daemon. When running with a large classpath, the output of ps gets truncated and the check fails spuriously. This weakens the check to see if it's a java command (which is something we do in other parts of the script) rather than looking for the specific main class name. This means that SPARK-4832 might happen under a slightly broader range of circumstances (a java program happened to reuse the same PID), but it seems worthwhile compared to failing consistently with a large classpath. Author: Punya Biswal Closes #5535 from punya/feature/SPARK-6952 and squashes the following commits: 7ea12d1 [Punya Biswal] Handle long args when detecting PID reuse --- sbin/spark-daemon.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index d8e0facb81169..de762acc8fa0e 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -129,7 +129,7 @@ run_command() { if [ -f "$pid" ]; then TARGET_ID="$(cat "$pid")" - if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then echo "$command running as process $TARGET_ID. Stop it first." exit 1 fi @@ -163,7 +163,7 @@ run_command() { echo "$newpid" > "$pid" sleep 2 # Check if the process has died; in that case we'll tail the log so the user can see - if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then + if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then echo "failed to launch $command:" tail -2 "$log" | sed 's/^/ /' echo "full log in $log" From dc48ba9f9f7449dd2f12cbad288b65c8119d9284 Mon Sep 17 00:00:00 2001 From: linweizhong Date: Fri, 17 Apr 2015 12:04:02 +0100 Subject: [PATCH 063/144] [SPARK-6604][PySpark]Specify ip of python server scoket In driver now will start a server socket and use a wildcard ip, use 127.0.0.0 is more reasonable, as we only use it by local Python process. /cc davies Author: linweizhong Closes #5256 from Sephiroth-Lin/SPARK-6604 and squashes the following commits: 7b3c633 [linweizhong] rephrase --- core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b1ffba4c546bf..7409dc2d866f6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -604,7 +604,7 @@ private[spark] object PythonRDD extends Logging { * The thread will terminate after all the data are sent or any exceptions happen. */ private def serveIterator[T](items: Iterator[T], threadName: String): Int = { - val serverSocket = new ServerSocket(0, 1) + val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) // Close the socket if no connection in 3 seconds serverSocket.setSoTimeout(3000) From c84d91692aa25c01882bcc3f9fd5de3cfa786195 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 17 Apr 2015 11:29:27 -0500 Subject: [PATCH 064/144] [SPARK-6957] [SPARK-6958] [SQL] improve API compatibility to pandas ``` select(['cola', 'colb']) groupby(['colA', 'colB']) groupby([df.colA, df.colB]) df.sort('A', ascending=True) df.sort(['A', 'B'], ascending=True) df.sort(['A', 'B'], ascending=[1, 0]) ``` cc rxin Author: Davies Liu Closes #5544 from davies/compatibility and squashes the following commits: 4944058 [Davies Liu] add docstrings adb2816 [Davies Liu] Merge branch 'master' of github.com:apache/spark into compatibility bcbbcab [Davies Liu] support ascending as list 8dabdf0 [Davies Liu] improve API compatibility to pandas --- python/pyspark/sql/dataframe.py | 96 ++++++++++++++++++++++----------- python/pyspark/sql/functions.py | 11 ++-- python/pyspark/sql/tests.py | 2 +- 3 files changed, 70 insertions(+), 39 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index b9a3e6cfe7f49..326d22e72f104 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -485,13 +485,17 @@ def join(self, other, joinExprs=None, joinType=None): return DataFrame(jdf, self.sql_ctx) @ignore_unicode_prefix - def sort(self, *cols): + def sort(self, *cols, **kwargs): """Returns a new :class:`DataFrame` sorted by the specified column(s). - :param cols: list of :class:`Column` to sort by. + :param cols: list of :class:`Column` or column names to sort by. + :param ascending: sort by ascending order or not, could be bool, int + or list of bool, int (default: True). >>> df.sort(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> df.sort("age", ascending=False).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] >>> df.orderBy(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] >>> from pyspark.sql.functions import * @@ -499,16 +503,42 @@ def sort(self, *cols): [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] >>> df.orderBy(desc("age"), "name").collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> df.orderBy(["age", "name"], ascending=[0, 1]).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] """ if not cols: raise ValueError("should sort by at least one column") - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols)) + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + jcols = [_to_java_column(c) for c in cols] + ascending = kwargs.get('ascending', True) + if isinstance(ascending, (bool, int)): + if not ascending: + jcols = [jc.desc() for jc in jcols] + elif isinstance(ascending, list): + jcols = [jc if asc else jc.desc() + for asc, jc in zip(ascending, jcols)] + else: + raise TypeError("ascending can only be bool or list, but got %s" % type(ascending)) + + jdf = self._jdf.sort(self._jseq(jcols)) return DataFrame(jdf, self.sql_ctx) orderBy = sort + def _jseq(self, cols, converter=None): + """Return a JVM Seq of Columns from a list of Column or names""" + return _to_seq(self.sql_ctx._sc, cols, converter) + + def _jcols(self, *cols): + """Return a JVM Seq of Columns from a list of Column or column names + + If `cols` has only one list in it, cols[0] will be used as the list. + """ + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + return self._jseq(cols, _to_java_column) + def describe(self, *cols): """Computes statistics for numeric columns. @@ -523,9 +553,7 @@ def describe(self, *cols): min 2 max 5 """ - cols = ListConverter().convert(cols, - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.describe(self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols)) + jdf = self._jdf.describe(self._jseq(cols)) return DataFrame(jdf, self.sql_ctx) @ignore_unicode_prefix @@ -607,9 +635,7 @@ def select(self, *cols): >>> df.select(df.name, (df.age + 10).alias('age')).collect() [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] """ - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.select(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.select(self._jcols(*cols)) return DataFrame(jdf, self.sql_ctx) def selectExpr(self, *expr): @@ -620,8 +646,9 @@ def selectExpr(self, *expr): >>> df.selectExpr("age * 2", "abs(age)").collect() [Row((age * 2)=4, Abs(age)=2), Row((age * 2)=10, Abs(age)=5)] """ - jexpr = ListConverter().convert(expr, self._sc._gateway._gateway_client) - jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) + if len(expr) == 1 and isinstance(expr[0], list): + expr = expr[0] + jdf = self._jdf.selectExpr(self._jseq(expr)) return DataFrame(jdf, self.sql_ctx) @ignore_unicode_prefix @@ -659,6 +686,8 @@ def groupBy(self, *cols): so we can run aggregation on them. See :class:`GroupedData` for all the available aggregate functions. + :func:`groupby` is an alias for :func:`groupBy`. + :param cols: list of columns to group by. Each element should be a column name (string) or an expression (:class:`Column`). @@ -668,12 +697,14 @@ def groupBy(self, *cols): [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] >>> df.groupBy(df.name).avg().collect() [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] + >>> df.groupBy(['name', df.age]).count().collect() + [Row(name=u'Bob', age=5, count=1), Row(name=u'Alice', age=2, count=1)] """ - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.groupBy(self._jcols(*cols)) return GroupedData(jdf, self.sql_ctx) + groupby = groupBy + def agg(self, *exprs): """ Aggregate on the entire :class:`DataFrame` without groups (shorthand for ``df.groupBy.agg()``). @@ -744,9 +775,7 @@ def dropna(self, how='any', thresh=None, subset=None): if thresh is None: thresh = len(subset) if how == 'any' else 1 - cols = ListConverter().convert(subset, self.sql_ctx._sc._gateway._gateway_client) - cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) - return DataFrame(self._jdf.na().drop(thresh, cols), self.sql_ctx) + return DataFrame(self._jdf.na().drop(thresh, self._jseq(subset)), self.sql_ctx) def fillna(self, value, subset=None): """Replace null values, alias for ``na.fill()``. @@ -799,9 +828,7 @@ def fillna(self, value, subset=None): elif not isinstance(subset, (list, tuple)): raise ValueError("subset should be a list or tuple of column names") - cols = ListConverter().convert(subset, self.sql_ctx._sc._gateway._gateway_client) - cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) - return DataFrame(self._jdf.na().fill(value, cols), self.sql_ctx) + return DataFrame(self._jdf.na().fill(value, self._jseq(subset)), self.sql_ctx) @ignore_unicode_prefix def withColumn(self, colName, col): @@ -862,10 +889,8 @@ def _api(self): def df_varargs_api(f): def _api(self, *args): - jargs = ListConverter().convert(args, - self.sql_ctx._sc._gateway._gateway_client) name = f.__name__ - jdf = getattr(self._jdf, name)(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jargs)) + jdf = getattr(self._jdf, name)(_to_seq(self.sql_ctx._sc, args)) return DataFrame(jdf, self.sql_ctx) _api.__name__ = f.__name__ _api.__doc__ = f.__doc__ @@ -912,9 +937,8 @@ def agg(self, *exprs): else: # Columns assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" - jcols = ListConverter().convert([c._jc for c in exprs[1:]], - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.agg(exprs[0]._jc, self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.agg(exprs[0]._jc, + _to_seq(self.sql_ctx._sc, [c._jc for c in exprs[1:]])) return DataFrame(jdf, self.sql_ctx) @dfapi @@ -1006,6 +1030,19 @@ def _to_java_column(col): return jcol +def _to_seq(sc, cols, converter=None): + """ + Convert a list of Column (or names) into a JVM Seq of Column. + + An optional `converter` could be used to convert items in `cols` + into JVM Column objects. + """ + if converter: + cols = [converter(c) for c in cols] + jcols = ListConverter().convert(cols, sc._gateway._gateway_client) + return sc._jvm.PythonUtils.toSeq(jcols) + + def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): @@ -1177,8 +1214,7 @@ def inSet(self, *cols): cols = cols[0] cols = [c._jc if isinstance(c, Column) else _create_column_from_literal(c) for c in cols] sc = SparkContext._active_spark_context - jcols = ListConverter().convert(cols, sc._gateway._gateway_client) - jc = getattr(self._jc, "in")(sc._jvm.PythonUtils.toSeq(jcols)) + jc = getattr(self._jc, "in")(_to_seq(sc, cols)) return Column(jc) # order diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 1d6536952810f..bb47923f24b82 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -23,13 +23,11 @@ if sys.version < "3": from itertools import imap as map -from py4j.java_collections import ListConverter - from pyspark import SparkContext from pyspark.rdd import _prepare_for_python_RDD from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.sql.types import StringType -from pyspark.sql.dataframe import Column, _to_java_column +from pyspark.sql.dataframe import Column, _to_java_column, _to_seq __all__ = ['countDistinct', 'approxCountDistinct', 'udf'] @@ -87,8 +85,7 @@ def countDistinct(col, *cols): [Row(c=2)] """ sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], sc._gateway._gateway_client) - jc = sc._jvm.functions.countDistinct(_to_java_column(col), sc._jvm.PythonUtils.toSeq(jcols)) + jc = sc._jvm.functions.countDistinct(_to_java_column(col), _to_seq(sc, cols, _to_java_column)) return Column(jc) @@ -138,9 +135,7 @@ def __del__(self): def __call__(self, *cols): sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - sc._gateway._gateway_client) - jc = self._judf.apply(sc._jvm.PythonUtils.toSeq(jcols)) + jc = self._judf.apply(_to_seq(sc, cols, _to_java_column)) return Column(jc) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 6691e8c8dc44b..aa3aa1d164d9f 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -282,7 +282,7 @@ def test_apply_schema(self): StructField("struct1", StructType([StructField("b", ShortType(), False)]), False), StructField("list1", ArrayType(ByteType(), False), False), StructField("null1", DoubleType(), True)]) - df = self.sqlCtx.applySchema(rdd, schema) + df = self.sqlCtx.createDataFrame(rdd, schema) results = df.map(lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, x.float1, x.date1, x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1)) r = (127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), From 50ab8a6543ad5c31e89c16df374d0cb13222fd1e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Apr 2015 14:21:51 -0500 Subject: [PATCH 065/144] [SPARK-2669] [yarn] Distribute client configuration to AM. Currently, when Spark launches the Yarn AM, the process will use the local Hadoop configuration on the node where the AM launches, if one is present. A more correct approach is to use the same configuration used to launch the Spark job, since the user may have made modifications (such as adding app-specific configs). The approach taken here is to use the distributed cache to make all files in the Hadoop configuration directory available to the AM. This is a little overkill since only the AM needs them (the executors use the broadcast Hadoop configuration from the driver), but is the easier approach. Even though only a few files in that directory may end up being used, all of them are uploaded. This allows supporting use cases such as when auxiliary configuration files are used for SSL configuration, or when uploading a Hive configuration directory. Not all of these may be reflected in a o.a.h.conf.Configuration object, but may be needed when a driver in cluster mode instantiates, for example, a HiveConf object instead. Author: Marcelo Vanzin Closes #4142 from vanzin/SPARK-2669 and squashes the following commits: f5434b9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 013f0fb [Marcelo Vanzin] Review feedback. f693152 [Marcelo Vanzin] Le sigh. ed45b7d [Marcelo Vanzin] Zip all config files and upload them as an archive. 5927b6b [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 cbb9fb3 [Marcelo Vanzin] Remove stale test. e3e58d0 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 e3d0613 [Marcelo Vanzin] Review feedback. 34bdbd8 [Marcelo Vanzin] Fix test. 022a688 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 a77ddd5 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 79221c7 [Marcelo Vanzin] [SPARK-2669] [yarn] Distribute client configuration to AM. --- docs/running-on-yarn.md | 6 +- .../org/apache/spark/deploy/yarn/Client.scala | 125 +++++++++++++++--- .../spark/deploy/yarn/ExecutorRunnable.scala | 2 +- .../spark/deploy/yarn/ClientSuite.scala | 29 ++-- .../spark/deploy/yarn/YarnClusterSuite.scala | 6 +- 5 files changed, 132 insertions(+), 36 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 853c9f26b0ec9..0968fc5ad632b 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -211,7 +211,11 @@ Most of the configs are the same for Spark on YARN as for other deployment modes # Launching Spark on YARN Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. -These configs are used to write to the dfs and connect to the YARN ResourceManager. +These configs are used to write to the dfs and connect to the YARN ResourceManager. The +configuration contained in this directory will be distributed to the YARN cluster so that all +containers used by the application use the same configuration. If the configuration references +Java system properties or environment variables not managed by YARN, they should also be set in the +Spark application's configuration (driver, executors, and the AM when running in client mode). There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. 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 52e4dee46c535..019afbd1a1743 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 @@ -17,15 +17,18 @@ package org.apache.spark.deploy.yarn +import java.io.{File, FileOutputStream} import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer +import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.reflect.runtime.universe import scala.util.{Try, Success, Failure} import com.google.common.base.Objects +import com.google.common.io.Files import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration @@ -77,12 +80,6 @@ private[spark] class Client( def stop(): Unit = yarnClient.stop() - /* ------------------------------------------------------------------------------------- * - | The following methods have much in common in the stable and alpha versions of Client, | - | but cannot be implemented in the parent trait due to subtle API differences across | - | hadoop versions. | - * ------------------------------------------------------------------------------------- */ - /** * Submit an application running our ApplicationMaster to the ResourceManager. * @@ -223,6 +220,10 @@ private[spark] class Client( val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) val nns = getNameNodesToAccess(sparkConf) + dst + // Used to keep track of URIs added to the distributed cache. If the same URI is added + // multiple times, YARN will fail to launch containers for the app with an internal + // error. + val distributedUris = new HashSet[String] obtainTokensForNamenodes(nns, hadoopConf, credentials) obtainTokenForHiveMetastore(hadoopConf, credentials) @@ -241,6 +242,17 @@ private[spark] class Client( "for alternatives.") } + def addDistributedUri(uri: URI): Boolean = { + val uriStr = uri.toString() + if (distributedUris.contains(uriStr)) { + logWarning(s"Resource $uri added multiple times to distributed cache.") + false + } else { + distributedUris += uriStr + true + } + } + /** * Copy the given main resource to the distributed cache if the scheme is not "local". * Otherwise, set the corresponding key in our SparkConf to handle it downstream. @@ -258,11 +270,13 @@ private[spark] class Client( if (!localPath.isEmpty()) { val localURI = new URI(localPath) if (localURI.getScheme != LOCAL_SCHEME) { - val src = getQualifiedLocalPath(localURI, hadoopConf) - val destPath = copyFileToRemote(dst, src, replication) - val destFs = FileSystem.get(destPath.toUri(), hadoopConf) - distCacheMgr.addResource(destFs, hadoopConf, destPath, - localResources, LocalResourceType.FILE, destName, statCache) + if (addDistributedUri(localURI)) { + val src = getQualifiedLocalPath(localURI, hadoopConf) + val destPath = copyFileToRemote(dst, src, replication) + val destFs = FileSystem.get(destPath.toUri(), hadoopConf) + distCacheMgr.addResource(destFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, destName, statCache) + } } else if (confKey != null) { // If the resource is intended for local use only, handle this downstream // by setting the appropriate property @@ -271,6 +285,13 @@ private[spark] class Client( } } + createConfArchive().foreach { file => + require(addDistributedUri(file.toURI())) + val destPath = copyFileToRemote(dst, new Path(file.toURI()), replication) + distCacheMgr.addResource(fs, hadoopConf, destPath, localResources, LocalResourceType.ARCHIVE, + LOCALIZED_HADOOP_CONF_DIR, statCache, appMasterOnly = true) + } + /** * Do the same for any additional resources passed in through ClientArguments. * Each resource category is represented by a 3-tuple of: @@ -288,13 +309,15 @@ private[spark] class Client( flist.split(',').foreach { file => val localURI = new URI(file.trim()) if (localURI.getScheme != LOCAL_SCHEME) { - val localPath = new Path(localURI) - val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) - val destPath = copyFileToRemote(dst, localPath, replication) - distCacheMgr.addResource( - fs, hadoopConf, destPath, localResources, resType, linkname, statCache) - if (addToClasspath) { - cachedSecondaryJarLinks += linkname + if (addDistributedUri(localURI)) { + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyFileToRemote(dst, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destPath, localResources, resType, linkname, statCache) + if (addToClasspath) { + cachedSecondaryJarLinks += linkname + } } } else if (addToClasspath) { // Resource is intended for local use only and should be added to the class path @@ -310,6 +333,57 @@ private[spark] class Client( localResources } + /** + * Create an archive with the Hadoop config files for distribution. + * + * These are only used by the AM, since executors will use the configuration object broadcast by + * the driver. The files are zipped and added to the job as an archive, so that YARN will explode + * it when distributing to the AM. This directory is then added to the classpath of the AM + * process, just to make sure that everybody is using the same default config. + * + * This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR + * shows up in the classpath before YARN_CONF_DIR. + * + * Currently this makes a shallow copy of the conf directory. If there are cases where a + * Hadoop config directory contains subdirectories, this code will have to be fixed. + */ + private def createConfArchive(): Option[File] = { + val hadoopConfFiles = new HashMap[String, File]() + Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey => + sys.env.get(envKey).foreach { path => + val dir = new File(path) + if (dir.isDirectory()) { + dir.listFiles().foreach { file => + if (!hadoopConfFiles.contains(file.getName())) { + hadoopConfFiles(file.getName()) = file + } + } + } + } + } + + if (!hadoopConfFiles.isEmpty) { + val hadoopConfArchive = File.createTempFile(LOCALIZED_HADOOP_CONF_DIR, ".zip", + new File(Utils.getLocalDir(sparkConf))) + + val hadoopConfStream = new ZipOutputStream(new FileOutputStream(hadoopConfArchive)) + try { + hadoopConfStream.setLevel(0) + hadoopConfFiles.foreach { case (name, file) => + hadoopConfStream.putNextEntry(new ZipEntry(name)) + Files.copy(file, hadoopConfStream) + hadoopConfStream.closeEntry() + } + } finally { + hadoopConfStream.close() + } + + Some(hadoopConfArchive) + } else { + None + } + } + /** * Set up the environment for launching our ApplicationMaster container. */ @@ -317,7 +391,7 @@ private[spark] class Client( logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - populateClasspath(args, yarnConf, sparkConf, env, extraCp) + populateClasspath(args, yarnConf, sparkConf, env, true, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -718,6 +792,9 @@ object Client extends Logging { // Distribution-defined classpath to add to processes val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH" + // Subdirectory where the user's hadoop config files will be placed. + val LOCALIZED_HADOOP_CONF_DIR = "__hadoop_conf__" + /** * Find the user-defined Spark jar if configured, or return the jar containing this * class if not. @@ -831,11 +908,19 @@ object Client extends Logging { conf: Configuration, sparkConf: SparkConf, env: HashMap[String, String], + isAM: Boolean, extraClassPath: Option[String] = None): Unit = { extraClassPath.foreach(addClasspathEntry(_, env)) addClasspathEntry( YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env ) + + if (isAM) { + addClasspathEntry( + YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR + + LOCALIZED_HADOOP_CONF_DIR, env) + } + if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { val userClassPath = if (args != null) { 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 b06069c07f451..9d04d241dae9e 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 @@ -277,7 +277,7 @@ class ExecutorRunnable( private def prepareEnvironment(container: Container): HashMap[String, String] = { val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - Client.populateClasspath(null, yarnConf, sparkConf, env, extraCp) + Client.populateClasspath(null, yarnConf, sparkConf, env, false, extraCp) sparkConf.getExecutorEnv.foreach { case (key, value) => // This assumes each executor environment variable set here is a path diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index c1b94ac9c5bdd..a51c2005cb472 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -20,6 +20,11 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ HashMap => MutableHashMap } +import scala.reflect.ClassTag +import scala.util.Try + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig @@ -30,11 +35,6 @@ import org.mockito.Matchers._ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ HashMap => MutableHashMap } -import scala.reflect.ClassTag -import scala.util.Try - import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils @@ -93,7 +93,7 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - Client.populateClasspath(args, conf, sparkConf, env) + Client.populateClasspath(args, conf, sparkConf, env, true) val cp = env("CLASSPATH").split(":|;|") s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => @@ -104,13 +104,16 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { cp should not contain (uri.getPath()) } }) - if (classOf[Environment].getMethods().exists(_.getName == "$$")) { - cp should contain("{{PWD}}") - } else if (Utils.isWindows) { - cp should contain("%PWD%") - } else { - cp should contain(Environment.PWD.$()) - } + val pwdVar = + if (classOf[Environment].getMethods().exists(_.getName == "$$")) { + "{{PWD}}" + } else if (Utils.isWindows) { + "%PWD%" + } else { + Environment.PWD.$() + } + cp should contain(pwdVar) + cp should contain (s"$pwdVar${Path.SEPARATOR}${Client.LOCALIZED_HADOOP_CONF_DIR}") cp should not contain (Client.SPARK_JAR) cp should not contain (Client.APP_JAR) } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index a18c94d4ab4a8..3877da4120e7c 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -77,6 +77,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit private var yarnCluster: MiniYARNCluster = _ private var tempDir: File = _ private var fakeSparkJar: File = _ + private var hadoopConfDir: File = _ private var logConfDir: File = _ override def beforeAll() { @@ -120,6 +121,9 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + hadoopConfDir = new File(tempDir, Client.LOCALIZED_HADOOP_CONF_DIR) + assert(hadoopConfDir.mkdir()) + File.createTempFile("token", ".txt", hadoopConfDir) } override def afterAll() { @@ -258,7 +262,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit appArgs Utils.executeAndGetOutput(argv, - extraEnvironment = Map("YARN_CONF_DIR" -> tempDir.getAbsolutePath())) + extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) } /** From a83571acc938582865efb41645aa1e414f339e46 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 17 Apr 2015 13:15:36 -0700 Subject: [PATCH 066/144] [SPARK-6113] [ml] Stabilize DecisionTree API MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This is a PR for cleaning up and finalizing the DecisionTree API. PRs for ensembles will follow once this is merged. ### Goal Here is the description copied from the JIRA (for both trees and ensembles): > **Issue**: The APIs for DecisionTree and ensembles (RandomForests and GradientBoostedTrees) have been experimental for a long time. The API has become very convoluted because trees and ensembles have many, many variants, some of which we have added incrementally without a long-term design. > **Proposal**: This JIRA is for discussing changes required to finalize the APIs. After we discuss, I will make a PR to update the APIs and make them non-Experimental. This will require making many breaking changes; see the design doc for details. > **[Design doc](https://docs.google.com/document/d/1rJ_DZinyDG3PkYkAKSsQlY0QgCeefn4hUv7GsPkzBP4)** : This outlines current issues and the proposed API. Overall code layout: * The old API in mllib.tree.* will remain the same. * The new API will reside in ml.classification.* and ml.regression.* ### Summary of changes Old API * Exactly the same, except I made 1 method in Loss private (but that is not a breaking change since that method was introduced after the Spark 1.3 release). New APIs * Under Pipeline API * The new API preserves functionality, except: * New API does NOT store prob (probability of label in classification). I want to have it store the full vector of probabilities but feel that should be in a later PR. * Use abstractions for parameters, estimators, and models to avoid code duplication * Limit parameters to relevant algorithms * For enum-like types, only expose Strings * We can make these pluggable later on by adding new parameters. That is a far-future item. Test suites * I organized DecisionTreeSuite, but I made absolutely no changes to the tests themselves. * The test suites for the new API only test (a) similarity with the results of the old API and (b) elements of the new API. * After code is moved to this new API, we should move the tests from the old suites which test the internals. ### Details #### Changed names Parameters * useNodeIdCache -> cacheNodeIds #### Other changes * Split: Changed categories to set instead of list #### Non-decision tree changes * AttributeGroup * Added parentheses to toMetadata, toStructField methods (These were removed in a previous PR, but I ran into 1 issue with the Scala compiler not being able to disambiguate between a toMetadata method with no parentheses and a toMetadata method which takes 1 argument.) * Attributes * Renamed: toMetadata -> toMetadataImpl * Added toMetadata methods which return ML metadata (keyed with “ML_ATTR”) * NominalAttribute: Added getNumValues method which examines both numValues and values. * Params.inheritValues: Checks whether the parent param really belongs to the child (to allow Estimator-Model pairs with different sets of parameters) ### Questions for reviewers * Is "DecisionTreeClassificationModel" too long a name? * Is this OK in the docs? ``` class DecisionTreeRegressor extends TreeRegressor[DecisionTreeRegressionModel] with DecisionTreeParams[DecisionTreeRegressor] with TreeRegressorParams[DecisionTreeRegressor] ``` ### Future We should open up the abstractions at some point. E.g., it would be useful to be able to set tree-related parameters in 1 place and then pass those to multiple tree-based algorithms. Follow-up JIRAs will be (in this order): * Tree ensembles * Deprecate old tree code * Move DecisionTree implementation code to new API. * Move tests from the old suites which test the internals. * Update programming guide * Python API * Change RandomForest* to always use bootstrapping, even when numTrees = 1 * Provide the probability of the predicted label for classification. After we move code to the new API and update it to maintain probabilities for all labels, then we can add the probabilities to the new API. CC: mengxr manishamde codedeft chouqin MechCoder Author: Joseph K. Bradley Closes #5530 from jkbradley/dt-api-dt and squashes the following commits: 6aae255 [Joseph K. Bradley] Changed tree abstractions not to take type parameters, and for setters to return this.type instead ec17947 [Joseph K. Bradley] Updates based on code review. Main changes were: moving public types from ml.impl.tree to ml.tree, modifying CategoricalSplit to take an Array of categories but store a Set internally, making more types sealed or final 5626c81 [Joseph K. Bradley] style fixes f8fbd24 [Joseph K. Bradley] imported reorg of DecisionTreeSuite from old PR. small cleanups 7ef63ed [Joseph K. Bradley] Added DecisionTreeRegressor, test suites, and example (for real this time) e11673f [Joseph K. Bradley] Added DecisionTreeRegressor, test suites, and example 119f407 [Joseph K. Bradley] added DecisionTreeClassifier example 0bdc486 [Joseph K. Bradley] fixed issues after param PR was merged f9fbb60 [Joseph K. Bradley] Done with DecisionTreeClassifier, but no save/load yet. Need to add example as well 2532c9a [Joseph K. Bradley] partial move to spark.ml API, not done yet c72c1a0 [Joseph K. Bradley] Copied changes for common items, plus DecisionTreeClassifier from original PR --- .../examples/ml/DecisionTreeExample.scala | 322 +++++++++++++++ .../spark/ml/attribute/AttributeGroup.scala | 10 +- .../spark/ml/attribute/attributes.scala | 43 +- .../DecisionTreeClassifier.scala | 155 ++++++++ .../spark/ml/feature/StringIndexer.scala | 2 +- .../spark/ml/impl/tree/treeParams.scala | 300 ++++++++++++++ .../scala/org/apache/spark/ml/package.scala | 12 + .../org/apache/spark/ml/param/params.scala | 3 +- .../ml/regression/DecisionTreeRegressor.scala | 145 +++++++ .../scala/org/apache/spark/ml/tree/Node.scala | 205 ++++++++++ .../org/apache/spark/ml/tree/Split.scala | 151 +++++++ .../org/apache/spark/ml/tree/treeModels.scala | 60 +++ .../apache/spark/ml/util/MetadataUtils.scala | 82 ++++ .../spark/mllib/tree/DecisionTree.scala | 5 +- .../mllib/tree/GradientBoostedTrees.scala | 12 +- .../spark/mllib/tree/RandomForest.scala | 2 +- .../tree/configuration/BoostingStrategy.scala | 10 +- .../spark/mllib/tree/loss/AbsoluteError.scala | 5 +- .../spark/mllib/tree/loss/LogLoss.scala | 5 +- .../apache/spark/mllib/tree/loss/Loss.scala | 4 +- .../spark/mllib/tree/loss/SquaredError.scala | 5 +- .../mllib/tree/model/DecisionTreeModel.scala | 4 +- .../apache/spark/mllib/tree/model/Node.scala | 2 +- .../mllib/tree/model/treeEnsembleModels.scala | 32 +- .../JavaDecisionTreeClassifierSuite.java | 98 +++++ .../JavaDecisionTreeRegressorSuite.java | 97 +++++ .../ml/attribute/AttributeGroupSuite.scala | 4 +- .../spark/ml/attribute/AttributeSuite.scala | 42 +- .../DecisionTreeClassifierSuite.scala | 274 +++++++++++++ .../spark/ml/feature/VectorIndexerSuite.scala | 2 +- .../org/apache/spark/ml/impl/TreeTests.scala | 132 +++++++ .../DecisionTreeRegressorSuite.scala | 91 +++++ .../spark/mllib/tree/DecisionTreeSuite.scala | 373 +++++++++--------- 33 files changed, 2426 insertions(+), 263 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala create mode 100644 mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala new file mode 100644 index 0000000000000..d4cc8dede07ef --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -0,0 +1,322 @@ +/* + * 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.ml + +import scala.collection.mutable +import scala.language.reflectiveCalls + +import scopt.OptionParser + +import org.apache.spark.ml.tree.DecisionTreeModel +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.examples.mllib.AbstractParams +import org.apache.spark.ml.{Pipeline, PipelineStage} +import org.apache.spark.ml.classification.{DecisionTreeClassificationModel, DecisionTreeClassifier} +import org.apache.spark.ml.feature.{VectorIndexer, StringIndexer} +import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.evaluation.{RegressionMetrics, MulticlassMetrics} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.{SQLContext, DataFrame} + + +/** + * An example runner for decision trees. Run with + * {{{ + * ./bin/run-example ml.DecisionTreeExample [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DecisionTreeExample { + + case class Params( + input: String = null, + testInput: String = "", + dataFormat: String = "libsvm", + algo: String = "Classification", + maxDepth: Int = 5, + maxBins: Int = 32, + minInstancesPerNode: Int = 1, + minInfoGain: Double = 0.0, + numTrees: Int = 1, + featureSubsetStrategy: String = "auto", + fracTest: Double = 0.2, + cacheNodeIds: Boolean = false, + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DecisionTreeExample") { + head("DecisionTreeExample: an example decision tree app.") + opt[String]("algo") + .text(s"algorithm (Classification, Regression), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = x)) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + opt[Int]("minInstancesPerNode") + .text(s"min number of instances required at child nodes to create the parent split," + + s" default: ${defaultParams.minInstancesPerNode}") + .action((x, c) => c.copy(minInstancesPerNode = x)) + opt[Double]("minInfoGain") + .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") + .action((x, c) => c.copy(minInfoGain = x)) + opt[Double]("fracTest") + .text(s"fraction of data to hold out for testing. If given option testInput, " + + s"this option is ignored. default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[Boolean]("cacheNodeIds") + .text(s"whether to use node Id cache during training, " + + s"default: ${defaultParams.cacheNodeIds}") + .action((x, c) => c.copy(cacheNodeIds = x)) + opt[String]("checkpointDir") + .text(s"checkpoint directory where intermediate node Id caches will be stored, " + + s"default: ${defaultParams.checkpointDir match { + case Some(strVal) => strVal + case None => "None" + }}") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"how often to checkpoint the node Id cache, " + + s"default: ${defaultParams.checkpointInterval}") + .action((x, c) => c.copy(checkpointInterval = x)) + opt[String]("testInput") + .text(s"input path to test dataset. If given, option fracTest is ignored." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) + arg[String]("") + .text("input path to labeled examples") + .required() + .action((x, c) => c.copy(input = x)) + checkConfig { params => + if (params.fracTest < 0 || params.fracTest > 1) { + failure(s"fracTest ${params.fracTest} value incorrect; should be in [0,1].") + } else { + success + } + } + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + /** Load a dataset from the given path, using the given format */ + private[ml] def loadData( + sc: SparkContext, + path: String, + format: String, + expectedNumFeatures: Option[Int] = None): RDD[LabeledPoint] = { + format match { + case "dense" => MLUtils.loadLabeledPoints(sc, path) + case "libsvm" => expectedNumFeatures match { + case Some(numFeatures) => MLUtils.loadLibSVMFile(sc, path, numFeatures) + case None => MLUtils.loadLibSVMFile(sc, path) + } + case _ => throw new IllegalArgumentException(s"Bad data format: $format") + } + } + + /** + * Load training and test data from files. + * @param input Path to input dataset. + * @param dataFormat "libsvm" or "dense" + * @param testInput Path to test dataset. + * @param algo Classification or Regression + * @param fracTest Fraction of input data to hold out for testing. Ignored if testInput given. + * @return (training dataset, test dataset) + */ + private[ml] def loadDatasets( + sc: SparkContext, + input: String, + dataFormat: String, + testInput: String, + algo: String, + fracTest: Double): (DataFrame, DataFrame) = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Load training data + val origExamples: RDD[LabeledPoint] = loadData(sc, input, dataFormat) + + // Load or create test set + val splits: Array[RDD[LabeledPoint]] = if (testInput != "") { + // Load testInput. + val numFeatures = origExamples.take(1)(0).features.size + val origTestExamples: RDD[LabeledPoint] = loadData(sc, input, dataFormat, Some(numFeatures)) + Array(origExamples, origTestExamples) + } else { + // Split input into training, test. + origExamples.randomSplit(Array(1.0 - fracTest, fracTest), seed = 12345) + } + + // For classification, convert labels to Strings since we will index them later with + // StringIndexer. + def labelsToStrings(data: DataFrame): DataFrame = { + algo.toLowerCase match { + case "classification" => + data.withColumn("labelString", data("label").cast(StringType)) + case "regression" => + data + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + } + val dataframes = splits.map(_.toDF()).map(labelsToStrings).map(_.cache()) + + (dataframes(0), dataframes(1)) + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"DecisionTreeExample with $params") + val sc = new SparkContext(conf) + params.checkpointDir.foreach(sc.setCheckpointDir) + val algo = params.algo.toLowerCase + + println(s"DecisionTreeExample with parameters:\n$params") + + // Load training and test data and cache it. + val (training: DataFrame, test: DataFrame) = + loadDatasets(sc, params.input, params.dataFormat, params.testInput, algo, params.fracTest) + + val numTraining = training.count() + val numTest = test.count() + val numFeatures = training.select("features").first().getAs[Vector](0).size + println("Loaded data:") + println(s" numTraining = $numTraining, numTest = $numTest") + println(s" numFeatures = $numFeatures") + + // Set up Pipeline + val stages = new mutable.ArrayBuffer[PipelineStage]() + // (1) For classification, re-index classes. + val labelColName = if (algo == "classification") "indexedLabel" else "label" + if (algo == "classification") { + val labelIndexer = new StringIndexer().setInputCol("labelString").setOutputCol(labelColName) + stages += labelIndexer + } + // (2) Identify categorical features using VectorIndexer. + // Features with more than maxCategories values will be treated as continuous. + val featuresIndexer = new VectorIndexer().setInputCol("features") + .setOutputCol("indexedFeatures").setMaxCategories(10) + stages += featuresIndexer + // (3) Learn DecisionTree + val dt = algo match { + case "classification" => + new DecisionTreeClassifier().setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + case "regression" => + new DecisionTreeRegressor().setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + stages += dt + val pipeline = new Pipeline().setStages(stages.toArray) + + // Fit the Pipeline + val startTime = System.nanoTime() + val pipelineModel = pipeline.fit(training) + val elapsedTime = (System.nanoTime() - startTime) / 1e9 + println(s"Training time: $elapsedTime seconds") + + // Get the trained Decision Tree from the fitted PipelineModel + val treeModel: DecisionTreeModel = algo match { + case "classification" => + pipelineModel.getModel[DecisionTreeClassificationModel]( + dt.asInstanceOf[DecisionTreeClassifier]) + case "regression" => + pipelineModel.getModel[DecisionTreeRegressionModel](dt.asInstanceOf[DecisionTreeRegressor]) + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + if (treeModel.numNodes < 20) { + println(treeModel.toDebugString) // Print full model. + } else { + println(treeModel) // Print model summary. + } + + // Predict on training + val trainingFullPredictions = pipelineModel.transform(training).cache() + val trainingPredictions = trainingFullPredictions.select("prediction") + .map(_.getDouble(0)) + val trainingLabels = trainingFullPredictions.select(labelColName).map(_.getDouble(0)) + // Predict on test data + val testFullPredictions = pipelineModel.transform(test).cache() + val testPredictions = testFullPredictions.select("prediction") + .map(_.getDouble(0)) + val testLabels = testFullPredictions.select(labelColName).map(_.getDouble(0)) + + // For classification, print number of classes for reference. + if (algo == "classification") { + val numClasses = + MetadataUtils.getNumClasses(trainingFullPredictions.schema(labelColName)) match { + case Some(n) => n + case None => throw new RuntimeException( + "DecisionTreeExample had unknown failure when indexing labels for classification.") + } + println(s"numClasses = $numClasses.") + } + + // Evaluate model on training, test data + algo match { + case "classification" => + val trainingAccuracy = + new MulticlassMetrics(trainingPredictions.zip(trainingLabels)).precision + println(s"Train accuracy = $trainingAccuracy") + val testAccuracy = + new MulticlassMetrics(testPredictions.zip(testLabels)).precision + println(s"Test accuracy = $testAccuracy") + case "regression" => + val trainingRMSE = + new RegressionMetrics(trainingPredictions.zip(trainingLabels)).rootMeanSquaredError + println(s"Training root mean squared error (RMSE) = $trainingRMSE") + val testRMSE = + new RegressionMetrics(testPredictions.zip(testLabels)).rootMeanSquaredError + println(s"Test root mean squared error (RMSE) = $testRMSE") + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index aa27a668f1695..d7dee8fed2a55 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -117,12 +117,12 @@ class AttributeGroup private ( case numeric: NumericAttribute => // Skip default numeric attributes. if (numeric.withoutIndex != NumericAttribute.defaultAttr) { - numericMetadata += numeric.toMetadata(withType = false) + numericMetadata += numeric.toMetadataImpl(withType = false) } case nominal: NominalAttribute => - nominalMetadata += nominal.toMetadata(withType = false) + nominalMetadata += nominal.toMetadataImpl(withType = false) case binary: BinaryAttribute => - binaryMetadata += binary.toMetadata(withType = false) + binaryMetadata += binary.toMetadataImpl(withType = false) } val attrBldr = new MetadataBuilder if (numericMetadata.nonEmpty) { @@ -151,7 +151,7 @@ class AttributeGroup private ( } /** Converts to ML metadata */ - def toMetadata: Metadata = toMetadata(Metadata.empty) + def toMetadata(): Metadata = toMetadata(Metadata.empty) /** Converts to a StructField with some existing metadata. */ def toStructField(existingMetadata: Metadata): StructField = { @@ -159,7 +159,7 @@ class AttributeGroup private ( } /** Converts to a StructField. */ - def toStructField: StructField = toStructField(Metadata.empty) + def toStructField(): StructField = toStructField(Metadata.empty) override def equals(other: Any): Boolean = { other match { diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala index 00b7566aab434..5717d6ec2eaec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -68,21 +68,32 @@ sealed abstract class Attribute extends Serializable { * Converts this attribute to [[Metadata]]. * @param withType whether to include the type info */ - private[attribute] def toMetadata(withType: Boolean): Metadata + private[attribute] def toMetadataImpl(withType: Boolean): Metadata /** * Converts this attribute to [[Metadata]]. For numeric attributes, the type info is excluded to * save space, because numeric type is the default attribute type. For nominal and binary * attributes, the type info is included. */ - private[attribute] def toMetadata(): Metadata = { + private[attribute] def toMetadataImpl(): Metadata = { if (attrType == AttributeType.Numeric) { - toMetadata(withType = false) + toMetadataImpl(withType = false) } else { - toMetadata(withType = true) + toMetadataImpl(withType = true) } } + /** Converts to ML metadata with some existing metadata. */ + def toMetadata(existingMetadata: Metadata): Metadata = { + new MetadataBuilder() + .withMetadata(existingMetadata) + .putMetadata(AttributeKeys.ML_ATTR, toMetadataImpl()) + .build() + } + + /** Converts to ML metadata */ + def toMetadata(): Metadata = toMetadata(Metadata.empty) + /** * Converts to a [[StructField]] with some existing metadata. * @param existingMetadata existing metadata to carry over @@ -90,7 +101,7 @@ sealed abstract class Attribute extends Serializable { def toStructField(existingMetadata: Metadata): StructField = { val newMetadata = new MetadataBuilder() .withMetadata(existingMetadata) - .putMetadata(AttributeKeys.ML_ATTR, withoutName.withoutIndex.toMetadata()) + .putMetadata(AttributeKeys.ML_ATTR, withoutName.withoutIndex.toMetadataImpl()) .build() StructField(name.get, DoubleType, nullable = false, newMetadata) } @@ -98,7 +109,7 @@ sealed abstract class Attribute extends Serializable { /** Converts to a [[StructField]]. */ def toStructField(): StructField = toStructField(Metadata.empty) - override def toString: String = toMetadata(withType = true).toString + override def toString: String = toMetadataImpl(withType = true).toString } /** Trait for ML attribute factories. */ @@ -210,7 +221,7 @@ class NumericAttribute private[ml] ( override def isNominal: Boolean = false /** Convert this attribute to metadata. */ - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder() if (withType) bldr.putString(TYPE, attrType.name) @@ -353,6 +364,20 @@ class NominalAttribute private[ml] ( /** Copy without the `numValues`. */ def withoutNumValues: NominalAttribute = copy(numValues = None) + /** + * Get the number of values, either from `numValues` or from `values`. + * Return None if unknown. + */ + def getNumValues: Option[Int] = { + if (numValues.nonEmpty) { + numValues + } else if (values.nonEmpty) { + Some(values.get.length) + } else { + None + } + } + /** Creates a copy of this attribute with optional changes. */ private def copy( name: Option[String] = name, @@ -363,7 +388,7 @@ class NominalAttribute private[ml] ( new NominalAttribute(name, index, isOrdinal, numValues, values) } - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder() if (withType) bldr.putString(TYPE, attrType.name) @@ -465,7 +490,7 @@ class BinaryAttribute private[ml] ( new BinaryAttribute(name, index, values) } - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder if (withType) bldr.putString(TYPE, attrType.name) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala new file mode 100644 index 0000000000000..3855e396b5534 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -0,0 +1,155 @@ +/* + * 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.ml.classification + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{Predictor, PredictionModel} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.{Params, ParamMap} +import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * for classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + */ +@AlphaComponent +final class DecisionTreeClassifier + extends Predictor[Vector, DecisionTreeClassifier, DecisionTreeClassificationModel] + with DecisionTreeParams + with TreeClassifierParams { + + // Override parameter setters from parent trait for Java API compatibility. + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = + super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = + super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + override protected def train( + dataset: DataFrame, + paramMap: ParamMap): DecisionTreeClassificationModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema(paramMap(featuresCol))) + val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema(paramMap(labelCol))) match { + case Some(n: Int) => n + case None => throw new IllegalArgumentException("DecisionTreeClassifier was given input" + + s" with invalid label column, without the number of classes specified.") + // TODO: Automatically index labels. + } + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, paramMap) + val strategy = getOldStrategy(categoricalFeatures, numClasses) + val oldModel = OldDecisionTree.train(oldDataset, strategy) + DecisionTreeClassificationModel.fromOld(oldModel, this, paramMap, categoricalFeatures) + } + + /** (private[ml]) Create a Strategy instance to use with the old API. */ + override private[ml] def getOldStrategy( + categoricalFeatures: Map[Int, Int], + numClasses: Int): OldStrategy = { + val strategy = super.getOldStrategy(categoricalFeatures, numClasses) + strategy.algo = OldAlgo.Classification + strategy.setImpurity(getOldImpurity) + strategy + } +} + +object DecisionTreeClassifier { + /** Accessor for supported impurities */ + final val supportedImpurities: Array[String] = TreeClassifierParams.supportedImpurities +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + */ +@AlphaComponent +final class DecisionTreeClassificationModel private[ml] ( + override val parent: DecisionTreeClassifier, + override val fittingParamMap: ParamMap, + override val rootNode: Node) + extends PredictionModel[Vector, DecisionTreeClassificationModel] + with DecisionTreeModel with Serializable { + + require(rootNode != null, + "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") + + override protected def predict(features: Vector): Double = { + rootNode.predict(features) + } + + override protected def copy(): DecisionTreeClassificationModel = { + val m = new DecisionTreeClassificationModel(parent, fittingParamMap, rootNode) + Params.inheritValues(this.extractParamMap(), this, m) + m + } + + override def toString: String = { + s"DecisionTreeClassificationModel of depth $depth with $numNodes nodes" + } + + /** (private[ml]) Convert to a model in the old API */ + private[ml] def toOld: OldDecisionTreeModel = { + new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Classification) + } +} + +private[ml] object DecisionTreeClassificationModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldDecisionTreeModel, + parent: DecisionTreeClassifier, + fittingParamMap: ParamMap, + categoricalFeatures: Map[Int, Int]): DecisionTreeClassificationModel = { + require(oldModel.algo == OldAlgo.Classification, + s"Cannot convert non-classification DecisionTreeModel (old API) to" + + s" DecisionTreeClassificationModel (new API). Algo is: ${oldModel.algo}") + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) + new DecisionTreeClassificationModel(parent, fittingParamMap, rootNode) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 4d960df357fe9..23956c512c8a6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -118,7 +118,7 @@ class StringIndexerModel private[ml] ( } val outputColName = map(outputCol) val metadata = NominalAttribute.defaultAttr - .withName(outputColName).withValues(labels).toStructField().metadata + .withName(outputColName).withValues(labels).toMetadata() dataset.select(col("*"), indexer(dataset(map(inputCol))).as(outputColName, metadata)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala new file mode 100644 index 0000000000000..6f4509f03d033 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala @@ -0,0 +1,300 @@ +/* + * 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.ml.impl.tree + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.impl.estimator.PredictorParams +import org.apache.spark.ml.param._ +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.impurity.{Gini => OldGini, Entropy => OldEntropy, + Impurity => OldImpurity, Variance => OldVariance} + + +/** + * :: DeveloperApi :: + * Parameters for Decision Tree-based algorithms. + * + * Note: Marked as private and DeveloperApi since this may be made public in the future. + */ +@DeveloperApi +private[ml] trait DecisionTreeParams extends PredictorParams { + + /** + * Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (default = 5) + * @group param + */ + final val maxDepth: IntParam = + new IntParam(this, "maxDepth", "Maximum depth of the tree." + + " E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.") + + /** + * Maximum number of bins used for discretizing continuous features and for choosing how to split + * on features at each node. More bins give higher granularity. + * Must be >= 2 and >= number of categories in any categorical feature. + * (default = 32) + * @group param + */ + final val maxBins: IntParam = new IntParam(this, "maxBins", "Max number of bins for" + + " discretizing continuous features. Must be >=2 and >= number of categories for any" + + " categorical feature.") + + /** + * Minimum number of instances each child must have after split. + * If a split causes the left or right child to have fewer than minInstancesPerNode, + * the split will be discarded as invalid. + * Should be >= 1. + * (default = 1) + * @group param + */ + final val minInstancesPerNode: IntParam = new IntParam(this, "minInstancesPerNode", "Minimum" + + " number of instances each child must have after split. If a split causes the left or right" + + " child to have fewer than minInstancesPerNode, the split will be discarded as invalid." + + " Should be >= 1.") + + /** + * Minimum information gain for a split to be considered at a tree node. + * (default = 0.0) + * @group param + */ + final val minInfoGain: DoubleParam = new DoubleParam(this, "minInfoGain", + "Minimum information gain for a split to be considered at a tree node.") + + /** + * Maximum memory in MB allocated to histogram aggregation. + * (default = 256 MB) + * @group expertParam + */ + final val maxMemoryInMB: IntParam = new IntParam(this, "maxMemoryInMB", + "Maximum memory in MB allocated to histogram aggregation.") + + /** + * If false, the algorithm will pass trees to executors to match instances with nodes. + * If true, the algorithm will cache node IDs for each instance. + * Caching can speed up training of deeper trees. + * (default = false) + * @group expertParam + */ + final val cacheNodeIds: BooleanParam = new BooleanParam(this, "cacheNodeIds", "If false, the" + + " algorithm will pass trees to executors to match instances with nodes. If true, the" + + " algorithm will cache node IDs for each instance. Caching can speed up training of deeper" + + " trees.") + + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be >= 1. + * (default = 10) + * @group expertParam + */ + final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "Specifies" + + " how often to checkpoint the cached node IDs. E.g. 10 means that the cache will get" + + " checkpointed every 10 iterations. This is only used if cacheNodeIds is true and if the" + + " checkpoint directory is set in the SparkContext. Must be >= 1.") + + setDefault(maxDepth -> 5, maxBins -> 32, minInstancesPerNode -> 1, minInfoGain -> 0.0, + maxMemoryInMB -> 256, cacheNodeIds -> false, checkpointInterval -> 10) + + /** @group setParam */ + def setMaxDepth(value: Int): this.type = { + require(value >= 0, s"maxDepth parameter must be >= 0. Given bad value: $value") + set(maxDepth, value) + this.asInstanceOf[this.type] + } + + /** @group getParam */ + def getMaxDepth: Int = getOrDefault(maxDepth) + + /** @group setParam */ + def setMaxBins(value: Int): this.type = { + require(value >= 2, s"maxBins parameter must be >= 2. Given bad value: $value") + set(maxBins, value) + this + } + + /** @group getParam */ + def getMaxBins: Int = getOrDefault(maxBins) + + /** @group setParam */ + def setMinInstancesPerNode(value: Int): this.type = { + require(value >= 1, s"minInstancesPerNode parameter must be >= 1. Given bad value: $value") + set(minInstancesPerNode, value) + this + } + + /** @group getParam */ + def getMinInstancesPerNode: Int = getOrDefault(minInstancesPerNode) + + /** @group setParam */ + def setMinInfoGain(value: Double): this.type = { + set(minInfoGain, value) + this + } + + /** @group getParam */ + def getMinInfoGain: Double = getOrDefault(minInfoGain) + + /** @group expertSetParam */ + def setMaxMemoryInMB(value: Int): this.type = { + require(value > 0, s"maxMemoryInMB parameter must be > 0. Given bad value: $value") + set(maxMemoryInMB, value) + this + } + + /** @group expertGetParam */ + def getMaxMemoryInMB: Int = getOrDefault(maxMemoryInMB) + + /** @group expertSetParam */ + def setCacheNodeIds(value: Boolean): this.type = { + set(cacheNodeIds, value) + this + } + + /** @group expertGetParam */ + def getCacheNodeIds: Boolean = getOrDefault(cacheNodeIds) + + /** @group expertSetParam */ + def setCheckpointInterval(value: Int): this.type = { + require(value >= 1, s"checkpointInterval parameter must be >= 1. Given bad value: $value") + set(checkpointInterval, value) + this + } + + /** @group expertGetParam */ + def getCheckpointInterval: Int = getOrDefault(checkpointInterval) + + /** + * Create a Strategy instance to use with the old API. + * NOTE: The caller should set impurity and subsamplingRate (which is set to 1.0, + * the default for single trees). + */ + private[ml] def getOldStrategy( + categoricalFeatures: Map[Int, Int], + numClasses: Int): OldStrategy = { + val strategy = OldStrategy.defaultStategy(OldAlgo.Classification) + strategy.checkpointInterval = getCheckpointInterval + strategy.maxBins = getMaxBins + strategy.maxDepth = getMaxDepth + strategy.maxMemoryInMB = getMaxMemoryInMB + strategy.minInfoGain = getMinInfoGain + strategy.minInstancesPerNode = getMinInstancesPerNode + strategy.useNodeIdCache = getCacheNodeIds + strategy.numClasses = numClasses + strategy.categoricalFeaturesInfo = categoricalFeatures + strategy.subsamplingRate = 1.0 // default for individual trees + strategy + } +} + +/** + * (private trait) Parameters for Decision Tree-based classification algorithms. + */ +private[ml] trait TreeClassifierParams extends Params { + + /** + * Criterion used for information gain calculation (case-insensitive). + * Supported: "entropy" and "gini". + * (default = gini) + * @group param + */ + val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + + " information gain calculation (case-insensitive). Supported options:" + + s" ${TreeClassifierParams.supportedImpurities.mkString(", ")}") + + setDefault(impurity -> "gini") + + /** @group setParam */ + def setImpurity(value: String): this.type = { + val impurityStr = value.toLowerCase + require(TreeClassifierParams.supportedImpurities.contains(impurityStr), + s"Tree-based classifier was given unrecognized impurity: $value." + + s" Supported options: ${TreeClassifierParams.supportedImpurities.mkString(", ")}") + set(impurity, impurityStr) + this + } + + /** @group getParam */ + def getImpurity: String = getOrDefault(impurity) + + /** Convert new impurity to old impurity. */ + private[ml] def getOldImpurity: OldImpurity = { + getImpurity match { + case "entropy" => OldEntropy + case "gini" => OldGini + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException( + s"TreeClassifierParams was given unrecognized impurity: $impurity.") + } + } +} + +private[ml] object TreeClassifierParams { + // These options should be lowercase. + val supportedImpurities: Array[String] = Array("entropy", "gini").map(_.toLowerCase) +} + +/** + * (private trait) Parameters for Decision Tree-based regression algorithms. + */ +private[ml] trait TreeRegressorParams extends Params { + + /** + * Criterion used for information gain calculation (case-insensitive). + * Supported: "variance". + * (default = variance) + * @group param + */ + val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + + " information gain calculation (case-insensitive). Supported options:" + + s" ${TreeRegressorParams.supportedImpurities.mkString(", ")}") + + setDefault(impurity -> "variance") + + /** @group setParam */ + def setImpurity(value: String): this.type = { + val impurityStr = value.toLowerCase + require(TreeRegressorParams.supportedImpurities.contains(impurityStr), + s"Tree-based regressor was given unrecognized impurity: $value." + + s" Supported options: ${TreeRegressorParams.supportedImpurities.mkString(", ")}") + set(impurity, impurityStr) + this + } + + /** @group getParam */ + def getImpurity: String = getOrDefault(impurity) + + /** Convert new impurity to old impurity. */ + protected def getOldImpurity: OldImpurity = { + getImpurity match { + case "variance" => OldVariance + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException( + s"TreeRegressorParams was given unrecognized impurity: $impurity") + } + } +} + +private[ml] object TreeRegressorParams { + // These options should be lowercase. + val supportedImpurities: Array[String] = Array("variance").map(_.toLowerCase) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/package.scala b/mllib/src/main/scala/org/apache/spark/ml/package.scala index b45bd1499b72e..ac75e9de1a8f2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/package.scala @@ -32,6 +32,18 @@ package org.apache.spark * @groupname getParam Parameter getters * @groupprio getParam 6 * + * @groupname expertParam (expert-only) Parameters + * @groupdesc expertParam A list of advanced, expert-only (hyper-)parameter keys this algorithm can + * take. Users can set and get the parameter values through setters and getters, + * respectively. + * @groupprio expertParam 7 + * + * @groupname expertSetParam (expert-only) Parameter setters + * @groupprio expertSetParam 8 + * + * @groupname expertGetParam (expert-only) Parameter getters + * @groupprio expertGetParam 9 + * * @groupname Ungrouped Members * @groupprio Ungrouped 0 */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 849c60433c777..ddc5907e7facd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -296,8 +296,9 @@ private[spark] object Params { paramMap: ParamMap, parent: E, child: M): Unit = { + val childParams = child.params.map(_.name).toSet parent.params.foreach { param => - if (paramMap.contains(param)) { + if (paramMap.contains(param) && childParams.contains(param.name)) { child.set(child.getParam(param.name), paramMap(param)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala new file mode 100644 index 0000000000000..49a8b77acf960 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -0,0 +1,145 @@ +/* + * 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.ml.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.{Params, ParamMap} +import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * for regression. + * It supports both continuous and categorical features. + */ +@AlphaComponent +final class DecisionTreeRegressor + extends Predictor[Vector, DecisionTreeRegressor, DecisionTreeRegressionModel] + with DecisionTreeParams + with TreeRegressorParams { + + // Override parameter setters from parent trait for Java API compatibility. + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = + super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + override protected def train( + dataset: DataFrame, + paramMap: ParamMap): DecisionTreeRegressionModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema(paramMap(featuresCol))) + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, paramMap) + val strategy = getOldStrategy(categoricalFeatures) + val oldModel = OldDecisionTree.train(oldDataset, strategy) + DecisionTreeRegressionModel.fromOld(oldModel, this, paramMap, categoricalFeatures) + } + + /** (private[ml]) Create a Strategy instance to use with the old API. */ + private[ml] def getOldStrategy(categoricalFeatures: Map[Int, Int]): OldStrategy = { + val strategy = super.getOldStrategy(categoricalFeatures, numClasses = 0) + strategy.algo = OldAlgo.Regression + strategy.setImpurity(getOldImpurity) + strategy + } +} + +object DecisionTreeRegressor { + /** Accessor for supported impurities */ + final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression. + * It supports both continuous and categorical features. + * @param rootNode Root of the decision tree + */ +@AlphaComponent +final class DecisionTreeRegressionModel private[ml] ( + override val parent: DecisionTreeRegressor, + override val fittingParamMap: ParamMap, + override val rootNode: Node) + extends PredictionModel[Vector, DecisionTreeRegressionModel] + with DecisionTreeModel with Serializable { + + require(rootNode != null, + "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") + + override protected def predict(features: Vector): Double = { + rootNode.predict(features) + } + + override protected def copy(): DecisionTreeRegressionModel = { + val m = new DecisionTreeRegressionModel(parent, fittingParamMap, rootNode) + Params.inheritValues(this.extractParamMap(), this, m) + m + } + + override def toString: String = { + s"DecisionTreeRegressionModel of depth $depth with $numNodes nodes" + } + + /** Convert to a model in the old API */ + private[ml] def toOld: OldDecisionTreeModel = { + new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Regression) + } +} + +private[ml] object DecisionTreeRegressionModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldDecisionTreeModel, + parent: DecisionTreeRegressor, + fittingParamMap: ParamMap, + categoricalFeatures: Map[Int, Int]): DecisionTreeRegressionModel = { + require(oldModel.algo == OldAlgo.Regression, + s"Cannot convert non-regression DecisionTreeModel (old API) to" + + s" DecisionTreeRegressionModel (new API). Algo is: ${oldModel.algo}") + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) + new DecisionTreeRegressionModel(parent, fittingParamMap, rootNode) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala new file mode 100644 index 0000000000000..d6e2203d9f937 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.tree + +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.model.{InformationGainStats => OldInformationGainStats, + Node => OldNode, Predict => OldPredict} + + +/** + * Decision tree node interface. + */ +sealed abstract class Node extends Serializable { + + // TODO: Add aggregate stats (once available). This will happen after we move the DecisionTree + // code into the new API and deprecate the old API. + + /** Prediction this node makes (or would make, if it is an internal node) */ + def prediction: Double + + /** Impurity measure at this node (for training data) */ + def impurity: Double + + /** Recursive prediction helper method */ + private[ml] def predict(features: Vector): Double = prediction + + /** + * Get the number of nodes in tree below this node, including leaf nodes. + * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. + */ + private[tree] def numDescendants: Int + + /** + * Recursive print function. + * @param indentFactor The number of spaces to add to each level of indentation. + */ + private[tree] def subtreeToString(indentFactor: Int = 0): String + + /** + * Get depth of tree from this node. + * E.g.: Depth 0 means this is a leaf node. Depth 1 means 1 internal and 2 leaf nodes. + */ + private[tree] def subtreeDepth: Int + + /** + * Create a copy of this node in the old Node format, recursively creating child nodes as needed. + * @param id Node ID using old format IDs + */ + private[ml] def toOld(id: Int): OldNode +} + +private[ml] object Node { + + /** + * Create a new Node from the old Node format, recursively creating child nodes as needed. + */ + def fromOld(oldNode: OldNode, categoricalFeatures: Map[Int, Int]): Node = { + if (oldNode.isLeaf) { + // TODO: Once the implementation has been moved to this API, then include sufficient + // statistics here. + new LeafNode(prediction = oldNode.predict.predict, impurity = oldNode.impurity) + } else { + val gain = if (oldNode.stats.nonEmpty) { + oldNode.stats.get.gain + } else { + 0.0 + } + new InternalNode(prediction = oldNode.predict.predict, impurity = oldNode.impurity, + gain = gain, leftChild = fromOld(oldNode.leftNode.get, categoricalFeatures), + rightChild = fromOld(oldNode.rightNode.get, categoricalFeatures), + split = Split.fromOld(oldNode.split.get, categoricalFeatures)) + } + } +} + +/** + * Decision tree leaf node. + * @param prediction Prediction this node makes + * @param impurity Impurity measure at this node (for training data) + */ +final class LeafNode private[ml] ( + override val prediction: Double, + override val impurity: Double) extends Node { + + override def toString: String = s"LeafNode(prediction = $prediction, impurity = $impurity)" + + override private[ml] def predict(features: Vector): Double = prediction + + override private[tree] def numDescendants: Int = 0 + + override private[tree] def subtreeToString(indentFactor: Int = 0): String = { + val prefix: String = " " * indentFactor + prefix + s"Predict: $prediction\n" + } + + override private[tree] def subtreeDepth: Int = 0 + + override private[ml] def toOld(id: Int): OldNode = { + // NOTE: We do NOT store 'prob' in the new API currently. + new OldNode(id, new OldPredict(prediction, prob = 0.0), impurity, isLeaf = true, + None, None, None, None) + } +} + +/** + * Internal Decision Tree node. + * @param prediction Prediction this node would make if it were a leaf node + * @param impurity Impurity measure at this node (for training data) + * @param gain Information gain value. + * Values < 0 indicate missing values; this quirk will be removed with future updates. + * @param leftChild Left-hand child node + * @param rightChild Right-hand child node + * @param split Information about the test used to split to the left or right child. + */ +final class InternalNode private[ml] ( + override val prediction: Double, + override val impurity: Double, + val gain: Double, + val leftChild: Node, + val rightChild: Node, + val split: Split) extends Node { + + override def toString: String = { + s"InternalNode(prediction = $prediction, impurity = $impurity, split = $split)" + } + + override private[ml] def predict(features: Vector): Double = { + if (split.shouldGoLeft(features)) { + leftChild.predict(features) + } else { + rightChild.predict(features) + } + } + + override private[tree] def numDescendants: Int = { + 2 + leftChild.numDescendants + rightChild.numDescendants + } + + override private[tree] def subtreeToString(indentFactor: Int = 0): String = { + val prefix: String = " " * indentFactor + prefix + s"If (${InternalNode.splitToString(split, left=true)})\n" + + leftChild.subtreeToString(indentFactor + 1) + + prefix + s"Else (${InternalNode.splitToString(split, left=false)})\n" + + rightChild.subtreeToString(indentFactor + 1) + } + + override private[tree] def subtreeDepth: Int = { + 1 + math.max(leftChild.subtreeDepth, rightChild.subtreeDepth) + } + + override private[ml] def toOld(id: Int): OldNode = { + assert(id.toLong * 2 < Int.MaxValue, "Decision Tree could not be converted from new to old API" + + " since the old API does not support deep trees.") + // NOTE: We do NOT store 'prob' in the new API currently. + new OldNode(id, new OldPredict(prediction, prob = 0.0), impurity, isLeaf = false, + Some(split.toOld), Some(leftChild.toOld(OldNode.leftChildIndex(id))), + Some(rightChild.toOld(OldNode.rightChildIndex(id))), + Some(new OldInformationGainStats(gain, impurity, leftChild.impurity, rightChild.impurity, + new OldPredict(leftChild.prediction, prob = 0.0), + new OldPredict(rightChild.prediction, prob = 0.0)))) + } +} + +private object InternalNode { + + /** + * Helper method for [[Node.subtreeToString()]]. + * @param split Split to print + * @param left Indicates whether this is the part of the split going to the left, + * or that going to the right. + */ + private def splitToString(split: Split, left: Boolean): String = { + val featureStr = s"feature ${split.featureIndex}" + split match { + case contSplit: ContinuousSplit => + if (left) { + s"$featureStr <= ${contSplit.threshold}" + } else { + s"$featureStr > ${contSplit.threshold}" + } + case catSplit: CategoricalSplit => + val categoriesStr = catSplit.getLeftCategories.mkString("{", ",", "}") + if (left) { + s"$featureStr in $categoriesStr" + } else { + s"$featureStr not in $categoriesStr" + } + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala new file mode 100644 index 0000000000000..cb940f62990ed --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -0,0 +1,151 @@ +/* + * 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.ml.tree + +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} +import org.apache.spark.mllib.tree.model.{Split => OldSplit} + + +/** + * Interface for a "Split," which specifies a test made at a decision tree node + * to choose the left or right path. + */ +sealed trait Split extends Serializable { + + /** Index of feature which this split tests */ + def featureIndex: Int + + /** Return true (split to left) or false (split to right) */ + private[ml] def shouldGoLeft(features: Vector): Boolean + + /** Convert to old Split format */ + private[tree] def toOld: OldSplit +} + +private[ml] object Split { + + def fromOld(oldSplit: OldSplit, categoricalFeatures: Map[Int, Int]): Split = { + oldSplit.featureType match { + case OldFeatureType.Categorical => + new CategoricalSplit(featureIndex = oldSplit.feature, + leftCategories = oldSplit.categories.toArray, categoricalFeatures(oldSplit.feature)) + case OldFeatureType.Continuous => + new ContinuousSplit(featureIndex = oldSplit.feature, threshold = oldSplit.threshold) + } + } +} + +/** + * Split which tests a categorical feature. + * @param featureIndex Index of the feature to test + * @param leftCategories If the feature value is in this set of categories, then the split goes + * left. Otherwise, it goes right. + * @param numCategories Number of categories for this feature. + */ +final class CategoricalSplit( + override val featureIndex: Int, + leftCategories: Array[Double], + private val numCategories: Int) + extends Split { + + require(leftCategories.forall(cat => 0 <= cat && cat < numCategories), "Invalid leftCategories" + + s" (should be in range [0, $numCategories)): ${leftCategories.mkString(",")}") + + /** + * If true, then "categories" is the set of categories for splitting to the left, and vice versa. + */ + private val isLeft: Boolean = leftCategories.length <= numCategories / 2 + + /** Set of categories determining the splitting rule, along with [[isLeft]]. */ + private val categories: Set[Double] = { + if (isLeft) { + leftCategories.toSet + } else { + setComplement(leftCategories.toSet) + } + } + + override private[ml] def shouldGoLeft(features: Vector): Boolean = { + if (isLeft) { + categories.contains(features(featureIndex)) + } else { + !categories.contains(features(featureIndex)) + } + } + + override def equals(o: Any): Boolean = { + o match { + case other: CategoricalSplit => featureIndex == other.featureIndex && + isLeft == other.isLeft && categories == other.categories + case _ => false + } + } + + override private[tree] def toOld: OldSplit = { + val oldCats = if (isLeft) { + categories + } else { + setComplement(categories) + } + OldSplit(featureIndex, threshold = 0.0, OldFeatureType.Categorical, oldCats.toList) + } + + /** Get sorted categories which split to the left */ + def getLeftCategories: Array[Double] = { + val cats = if (isLeft) categories else setComplement(categories) + cats.toArray.sorted + } + + /** Get sorted categories which split to the right */ + def getRightCategories: Array[Double] = { + val cats = if (isLeft) setComplement(categories) else categories + cats.toArray.sorted + } + + /** [0, numCategories) \ cats */ + private def setComplement(cats: Set[Double]): Set[Double] = { + Range(0, numCategories).map(_.toDouble).filter(cat => !cats.contains(cat)).toSet + } +} + +/** + * Split which tests a continuous feature. + * @param featureIndex Index of the feature to test + * @param threshold If the feature value is <= this threshold, then the split goes left. + * Otherwise, it goes right. + */ +final class ContinuousSplit(override val featureIndex: Int, val threshold: Double) extends Split { + + override private[ml] def shouldGoLeft(features: Vector): Boolean = { + features(featureIndex) <= threshold + } + + override def equals(o: Any): Boolean = { + o match { + case other: ContinuousSplit => + featureIndex == other.featureIndex && threshold == other.threshold + case _ => + false + } + } + + override private[tree] def toOld: OldSplit = { + OldSplit(featureIndex, threshold, OldFeatureType.Continuous, List.empty[Double]) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala new file mode 100644 index 0000000000000..8e3bc3849dcf0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.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.ml.tree + +import org.apache.spark.annotation.AlphaComponent + + +/** + * :: AlphaComponent :: + * + * Abstraction for Decision Tree models. + * + * TODO: Add support for predicting probabilities and raw predictions + */ +@AlphaComponent +trait DecisionTreeModel { + + /** Root of the decision tree */ + def rootNode: Node + + /** Number of nodes in tree, including leaf nodes. */ + def numNodes: Int = { + 1 + rootNode.numDescendants + } + + /** + * Depth of the tree. + * E.g.: Depth 0 means 1 leaf node. Depth 1 means 1 internal node and 2 leaf nodes. + */ + lazy val depth: Int = { + rootNode.subtreeDepth + } + + /** Summary of the model */ + override def toString: String = { + // Implementing classes should generally override this method to be more descriptive. + s"DecisionTreeModel of depth $depth with $numNodes nodes" + } + + /** Full description of model */ + def toDebugString: String = { + val header = toString + "\n" + header + rootNode.subtreeToString(2) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala new file mode 100644 index 0000000000000..c84c8b4eb744f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -0,0 +1,82 @@ +/* + * 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.ml.util + +import scala.collection.immutable.HashMap + +import org.apache.spark.annotation.Experimental +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, BinaryAttribute, NominalAttribute, + NumericAttribute} +import org.apache.spark.sql.types.StructField + + +/** + * :: Experimental :: + * + * Helper utilities for tree-based algorithms + */ +@Experimental +object MetadataUtils { + + /** + * Examine a schema to identify the number of classes in a label column. + * Returns None if the number of labels is not specified, or if the label column is continuous. + */ + def getNumClasses(labelSchema: StructField): Option[Int] = { + Attribute.fromStructField(labelSchema) match { + case numAttr: NumericAttribute => None + case binAttr: BinaryAttribute => Some(2) + case nomAttr: NominalAttribute => nomAttr.getNumValues + } + } + + /** + * Examine a schema to identify categorical (Binary and Nominal) features. + * + * @param featuresSchema Schema of the features column. + * If a feature does not have metadata, it is assumed to be continuous. + * If a feature is Nominal, then it must have the number of values + * specified. + * @return Map: feature index --> number of categories. + * The map's set of keys will be the set of categorical feature indices. + */ + def getCategoricalFeatures(featuresSchema: StructField): Map[Int, Int] = { + val metadata = AttributeGroup.fromStructField(featuresSchema) + if (metadata.attributes.isEmpty) { + HashMap.empty[Int, Int] + } else { + metadata.attributes.get.zipWithIndex.flatMap { case (attr, idx) => + if (attr == null) { + Iterator() + } else { + attr match { + case numAttr: NumericAttribute => Iterator() + case binAttr: BinaryAttribute => Iterator(idx -> 2) + case nomAttr: NominalAttribute => + nomAttr.getNumValues match { + case Some(numValues: Int) => Iterator(idx -> numValues) + case None => throw new IllegalArgumentException(s"Feature $idx is marked as" + + " Nominal (categorical), but it does not have the number of values specified.") + } + } + } + }.toMap + } + } + +} 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 b9d0c56dd1ea3..dfe3a0b6913ef 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 @@ -1147,7 +1147,10 @@ object DecisionTree extends Serializable with Logging { } } - assert(splits.length > 0) + // TODO: Do not fail; just ignore the useless feature. + assert(splits.length > 0, + s"DecisionTree could not handle feature $featureIndex since it had only 1 unique value." + + " Please remove this feature and then try again.") // set number of splits accordingly metadata.setNumSplits(featureIndex, splits.length) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index c02c79f094b66..0e31c7ed58df8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -81,11 +81,11 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) /** * Method to validate a gradient boosting model * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @param validationInput Validation dataset: - RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - Should be different from and follow the same distribution as input. - e.g., these two datasets could be created from an original dataset - by using [[org.apache.spark.rdd.RDD.randomSplit()]] + * @param validationInput Validation dataset. + * This dataset should be different from the training dataset, + * but it should follow the same distribution. + * E.g., these two datasets could be created from an original dataset + * by using [[org.apache.spark.rdd.RDD.randomSplit()]] * @return a gradient boosted trees model that can be used for prediction */ def runWithValidation( @@ -194,8 +194,6 @@ object GradientBoostedTrees extends Logging { val firstTreeWeight = 1.0 baseLearners(0) = firstTreeModel baseLearnerWeights(0) = firstTreeWeight - val startingModel = new GradientBoostedTreesModel( - Regression, Array(firstTreeModel), baseLearnerWeights.slice(0, 1)) var predError: RDD[(Double, Double)] = GradientBoostedTreesModel. computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index db01f2e229e5a..055e60c7d9c95 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -249,7 +249,7 @@ private class RandomForest ( nodeIdCache.get.deleteAllCheckpoints() } catch { case e:IOException => - logWarning(s"delete all chackpoints failed. Error reason: ${e.getMessage}") + logWarning(s"delete all checkpoints failed. Error reason: ${e.getMessage}") } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index 664c8df019233..2d6b01524ff3d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -89,14 +89,14 @@ object BoostingStrategy { * @return Configuration for boosting algorithm */ def defaultParams(algo: Algo): BoostingStrategy = { - val treeStragtegy = Strategy.defaultStategy(algo) - treeStragtegy.maxDepth = 3 + val treeStrategy = Strategy.defaultStategy(algo) + treeStrategy.maxDepth = 3 algo match { case Algo.Classification => - treeStragtegy.numClasses = 2 - new BoostingStrategy(treeStragtegy, LogLoss) + treeStrategy.numClasses = 2 + new BoostingStrategy(treeStrategy, LogLoss) case Algo.Regression => - new BoostingStrategy(treeStragtegy, SquaredError) + new BoostingStrategy(treeStrategy, SquaredError) case _ => throw new IllegalArgumentException(s"$algo is not supported by boosting.") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index 6f570b4e09c79..2bdef73c4a8f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -45,9 +45,8 @@ object AbsoluteError extends Loss { if (label - prediction < 0) 1.0 else -1.0 } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val err = label - prediction math.abs(err) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 24ee9f3d51293..778c24526de70 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -47,10 +47,9 @@ object LogLoss extends Loss { - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val margin = 2.0 * label * prediction // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable. 2.0 * MLUtils.log1pExp(-margin) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index d3b82b752fa0d..64ffccbce073f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -22,6 +22,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: * Trait for adding "pluggable" loss functions for the gradient boosting algorithm. @@ -57,6 +58,5 @@ trait Loss extends Serializable { * @param label True label. * @return Measure of model error on datapoint. */ - def computeError(prediction: Double, label: Double): Double - + private[mllib] def computeError(prediction: Double, label: Double): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index 58857ae15e93e..a5582d3ef3324 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -45,9 +45,8 @@ object SquaredError extends Loss { 2.0 * (prediction - label) } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val err = prediction - label err * err } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index c9bafd60fba4d..331af428533de 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -113,11 +113,13 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable DecisionTreeModel.SaveLoadV1_0.save(sc, path, this) } - override protected def formatVersion: String = "1.0" + override protected def formatVersion: String = DecisionTreeModel.formatVersion } object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { + private[spark] def formatVersion: String = "1.0" + private[tree] object SaveLoadV1_0 { def thisFormatVersion: String = "1.0" diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 4f72bb8014cc0..708ba04b567d3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -175,7 +175,7 @@ class Node ( } } -private[tree] object Node { +private[spark] object Node { /** * Return a node with the given node id (but nothing else set). diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index fef3d2acb202a..8341219bfa71c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -38,6 +38,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.util.Utils + /** * :: Experimental :: * Represents a random forest model. @@ -47,7 +48,7 @@ import org.apache.spark.util.Utils */ @Experimental class RandomForestModel(override val algo: Algo, override val trees: Array[DecisionTreeModel]) - extends TreeEnsembleModel(algo, trees, Array.fill(trees.size)(1.0), + extends TreeEnsembleModel(algo, trees, Array.fill(trees.length)(1.0), combiningStrategy = if (algo == Classification) Vote else Average) with Saveable { @@ -58,11 +59,13 @@ class RandomForestModel(override val algo: Algo, override val trees: Array[Decis RandomForestModel.SaveLoadV1_0.thisClassName) } - override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override protected def formatVersion: String = RandomForestModel.formatVersion } object RandomForestModel extends Loader[RandomForestModel] { + private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override def load(sc: SparkContext, path: String): RandomForestModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName @@ -102,15 +105,13 @@ class GradientBoostedTreesModel( extends TreeEnsembleModel(algo, trees, treeWeights, combiningStrategy = Sum) with Saveable { - require(trees.size == treeWeights.size) + require(trees.length == treeWeights.length) override def save(sc: SparkContext, path: String): Unit = { TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, GradientBoostedTreesModel.SaveLoadV1_0.thisClassName) } - override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion - /** * Method to compute error or loss for every iteration of gradient boosting. * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] @@ -138,7 +139,7 @@ class GradientBoostedTreesModel( evaluationArray(0) = predictionAndError.values.mean() val broadcastTrees = sc.broadcast(trees) - (1 until numIterations).map { nTree => + (1 until numIterations).foreach { nTree => predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => val currentTree = broadcastTrees.value(nTree) val currentTreeWeight = localTreeWeights(nTree) @@ -155,6 +156,7 @@ class GradientBoostedTreesModel( evaluationArray } + override protected def formatVersion: String = GradientBoostedTreesModel.formatVersion } object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { @@ -200,17 +202,17 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { loss: Loss): RDD[(Double, Double)] = { val newPredError = data.zip(predictionAndError).mapPartitions { iter => - iter.map { - case (lp, (pred, error)) => { - val newPred = pred + tree.predict(lp.features) * treeWeight - val newError = loss.computeError(newPred, lp.label) - (newPred, newError) - } + iter.map { case (lp, (pred, error)) => + val newPred = pred + tree.predict(lp.features) * treeWeight + val newError = loss.computeError(newPred, lp.label) + (newPred, newError) } } newPredError } + private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName @@ -340,12 +342,12 @@ private[tree] sealed class TreeEnsembleModel( } /** - * Get number of trees in forest. + * Get number of trees in ensemble. */ - def numTrees: Int = trees.size + def numTrees: Int = trees.length /** - * Get total number of nodes, summed over all trees in the forest. + * Get total number of nodes, summed over all trees in the ensemble. */ def totalNumNodes: Int = trees.map(_.numNodes).sum } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java new file mode 100644 index 0000000000000..43b8787f9dd7e --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -0,0 +1,98 @@ +/* + * 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.ml.classification; + +import java.io.File; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.util.Utils; + + +public class JavaDecisionTreeClassifierSuite implements Serializable { + + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaDecisionTreeClassifierSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + DecisionTreeClassifier dt = new DecisionTreeClassifier() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (int i = 0; i < DecisionTreeClassifier.supportedImpurities().length; ++i) { + dt.setImpurity(DecisionTreeClassifier.supportedImpurities()[i]); + } + DecisionTreeClassificationModel model = dt.fit(dataFrame); + + model.transform(dataFrame); + model.numNodes(); + model.depth(); + model.toDebugString(); + + /* + // TODO: Add test once save/load are implemented. + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model3.save(sc.sc(), path); + DecisionTreeClassificationModel sameModel = + DecisionTreeClassificationModel.load(sc.sc(), path); + TreeTests.checkEqual(model3, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java new file mode 100644 index 0000000000000..a3a339004f31c --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -0,0 +1,97 @@ +/* + * 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.ml.regression; + +import java.io.File; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.util.Utils; + + +public class JavaDecisionTreeRegressorSuite implements Serializable { + + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaDecisionTreeRegressorSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + DecisionTreeRegressor dt = new DecisionTreeRegressor() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (int i = 0; i < DecisionTreeRegressor.supportedImpurities().length; ++i) { + dt.setImpurity(DecisionTreeRegressor.supportedImpurities()[i]); + } + DecisionTreeRegressionModel model = dt.fit(dataFrame); + + model.transform(dataFrame); + model.numNodes(); + model.depth(); + model.toDebugString(); + + /* + // TODO: Add test once save/load are implemented. + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model2.save(sc.sc(), path); + DecisionTreeRegressionModel sameModel = DecisionTreeRegressionModel.load(sc.sc(), path); + TreeTests.checkEqual(model2, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala index 0dcfe5a2002dc..17ddd335deb6d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala @@ -44,7 +44,7 @@ class AttributeGroupSuite extends FunSuite { group("abc") } assert(group === AttributeGroup.fromMetadata(group.toMetadataImpl, group.name)) - assert(group === AttributeGroup.fromStructField(group.toStructField)) + assert(group === AttributeGroup.fromStructField(group.toStructField())) } test("attribute group without attributes") { @@ -54,7 +54,7 @@ class AttributeGroupSuite extends FunSuite { assert(group0.size === 10) assert(group0.attributes.isEmpty) assert(group0 === AttributeGroup.fromMetadata(group0.toMetadataImpl, group0.name)) - assert(group0 === AttributeGroup.fromStructField(group0.toStructField)) + assert(group0 === AttributeGroup.fromStructField(group0.toStructField())) val group1 = new AttributeGroup("item") assert(group1.name === "item") diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala index 6ec35b03656f9..3e1a7196e37cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala @@ -36,9 +36,9 @@ class AttributeSuite extends FunSuite { assert(attr.max.isEmpty) assert(attr.std.isEmpty) assert(attr.sparsity.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = false) === metadata) - assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = false) === metadata) + assert(attr.toMetadataImpl(withType = true) === metadataWithType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === Attribute.fromMetadata(metadataWithType)) intercept[NoSuchElementException] { @@ -59,9 +59,9 @@ class AttributeSuite extends FunSuite { assert(!attr.isNominal) assert(attr.name === Some(name)) assert(attr.index === Some(index)) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = false) === metadata) - assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = false) === metadata) + assert(attr.toMetadataImpl(withType = true) === metadataWithType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === Attribute.fromMetadata(metadataWithType)) val field = attr.toStructField() @@ -81,7 +81,7 @@ class AttributeSuite extends FunSuite { assert(attr2.max === Some(1.0)) assert(attr2.std === Some(0.5)) assert(attr2.sparsity === Some(0.3)) - assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadataImpl())) } test("bad numeric attributes") { @@ -105,9 +105,9 @@ class AttributeSuite extends FunSuite { assert(attr.values.isEmpty) assert(attr.numValues.isEmpty) assert(attr.isOrdinal.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) intercept[NoSuchElementException] { @@ -135,9 +135,9 @@ class AttributeSuite extends FunSuite { assert(attr.values === Some(values)) assert(attr.indexOf("medium") === 1) assert(attr.getValue(1) === "medium") - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) @@ -147,8 +147,8 @@ class AttributeSuite extends FunSuite { assert(attr2.index.isEmpty) assert(attr2.values.get === Array("small", "medium", "large", "x-large")) assert(attr2.indexOf("x-large") === 3) - assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) - assert(attr2 === NominalAttribute.fromMetadata(attr2.toMetadata(withType = false))) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadataImpl())) + assert(attr2 === NominalAttribute.fromMetadata(attr2.toMetadataImpl(withType = false))) } test("bad nominal attributes") { @@ -168,9 +168,9 @@ class AttributeSuite extends FunSuite { assert(attr.name.isEmpty) assert(attr.index.isEmpty) assert(attr.values.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) intercept[NoSuchElementException] { @@ -196,9 +196,9 @@ class AttributeSuite extends FunSuite { assert(attr.name === Some(name)) assert(attr.index === Some(index)) assert(attr.values.get === values) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala new file mode 100644 index 0000000000000..af88595df5245 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -0,0 +1,274 @@ +/* + * 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.ml.classification + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, + DecisionTreeSuite => OldDecisionTreeSuite} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +class DecisionTreeClassifierSuite extends FunSuite with MLlibTestSparkContext { + + import DecisionTreeClassifierSuite.compareAPIs + + private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ + private var orderedLabeledPointsWithLabel0RDD: RDD[LabeledPoint] = _ + private var orderedLabeledPointsWithLabel1RDD: RDD[LabeledPoint] = _ + private var categoricalDataPointsForMulticlassRDD: RDD[LabeledPoint] = _ + private var continuousDataPointsForMulticlassRDD: RDD[LabeledPoint] = _ + private var categoricalDataPointsForMulticlassForOrderedFeaturesRDD: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + categoricalDataPointsRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + orderedLabeledPointsWithLabel0RDD = + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()) + orderedLabeledPointsWithLabel1RDD = + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()) + categoricalDataPointsForMulticlassRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()) + continuousDataPointsForMulticlassRDD = + sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()) + categoricalDataPointsForMulticlassForOrderedFeaturesRDD = sc.parallelize( + OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures()) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + test("Binary classification stump with ordered categorical features") { + val dt = new DecisionTreeClassifier() + .setImpurity("gini") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3, 1-> 3) + val numClasses = 2 + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures, numClasses) + } + + test("Binary classification stump with fixed labels 0,1 for Entropy,Gini") { + val dt = new DecisionTreeClassifier() + .setMaxDepth(3) + .setMaxBins(100) + val numClasses = 2 + Array(orderedLabeledPointsWithLabel0RDD, orderedLabeledPointsWithLabel1RDD).foreach { rdd => + DecisionTreeClassifier.supportedImpurities.foreach { impurity => + dt.setImpurity(impurity) + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + } + } + + test("Multiclass classification stump with 3-ary (unordered) categorical features") { + val rdd = categoricalDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 3 + val categoricalFeatures = Map(0 -> 3, 1 -> 3) + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(3.0))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Binary classification stump with 2 continuous features") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0))))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Multiclass classification stump with unordered categorical features," + + " with just enough bins") { + val maxBins = 2 * (math.pow(2, 3 - 1).toInt - 1) // just enough bins to allow unordered features + val rdd = categoricalDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(maxBins) + val categoricalFeatures = Map(0 -> 3, 1 -> 3) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification stump with continuous features") { + val rdd = continuousDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Multiclass classification stump with continuous + unordered categorical features") { + val rdd = continuousDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification stump with 10-ary (ordered) categorical features") { + val rdd = categoricalDataPointsForMulticlassForOrderedFeaturesRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 10, 1 -> 10) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification tree with 10-ary (ordered) categorical features," + + " with just enough bins") { + val rdd = categoricalDataPointsForMulticlassForOrderedFeaturesRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(10) + val categoricalFeatures = Map(0 -> 10, 1 -> 10) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("split must satisfy min instances per node requirements") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + .setMinInstancesPerNode(2) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("do not choose split that does not satisfy min instance per node requirements") { + // if a split does not satisfy min instances per node requirements, + // this split is invalid, even though the information gain of split is large. + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxBins(2) + .setMaxDepth(2) + .setMinInstancesPerNode(2) + val categoricalFeatures = Map(0 -> 2, 1-> 2) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("split must satisfy min info gain requirements") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) + val rdd = sc.parallelize(arr) + + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + .setMinInfoGain(1.0) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: Reinstate test once save/load are implemented + /* + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val oldModel = OldDecisionTreeSuite.createModel(OldAlgo.Classification) + val newModel = DecisionTreeClassificationModel.fromOld(oldModel) + + // Save model, load it back, and compare. + try { + newModel.save(sc, path) + val sameNewModel = DecisionTreeClassificationModel.load(sc, path) + TreeTests.checkEqual(newModel, sameNewModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + */ +} + +private[ml] object DecisionTreeClassifierSuite extends FunSuite { + + /** + * Train 2 decision trees on the given dataset, one using the old API and one using the new API. + * Convert the old tree to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + dt: DecisionTreeClassifier, + categoricalFeatures: Map[Int, Int], + numClasses: Int): Unit = { + val oldStrategy = dt.getOldStrategy(categoricalFeatures, numClasses) + val oldTree = OldDecisionTree.train(data, oldStrategy) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) + val newTree = dt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldTreeAsNew = DecisionTreeClassificationModel.fromOld(oldTree, newTree.parent, + newTree.fittingParamMap, categoricalFeatures) + TreeTests.checkEqual(oldTreeAsNew, newTree) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index 81ef831c42e55..1b261b2643854 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -228,7 +228,7 @@ class VectorIndexerSuite extends FunSuite with MLlibTestSparkContext { } val attrGroup = new AttributeGroup("features", featureAttributes) val densePoints1WithMeta = - densePoints1.select(densePoints1("features").as("features", attrGroup.toMetadata)) + densePoints1.select(densePoints1("features").as("features", attrGroup.toMetadata())) val vectorIndexer = getIndexer.setMaxCategories(2) val model = vectorIndexer.fit(densePoints1WithMeta) // Check that ML metadata are preserved. diff --git a/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala new file mode 100644 index 0000000000000..2e57d4ce37f1d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala @@ -0,0 +1,132 @@ +/* + * 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.ml.impl + +import scala.collection.JavaConverters._ + +import org.scalatest.FunSuite + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.tree.{DecisionTreeModel, InternalNode, LeafNode, Node} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SQLContext, DataFrame} + + +private[ml] object TreeTests extends FunSuite { + + /** + * Convert the given data to a DataFrame, and set the features and label metadata. + * @param data Dataset. Categorical features and labels must already have 0-based indices. + * This must be non-empty. + * @param categoricalFeatures Map: categorical feature index -> number of distinct values + * @param numClasses Number of classes label can take. If 0, mark as continuous. + * @return DataFrame with metadata + */ + def setMetadata( + data: RDD[LabeledPoint], + categoricalFeatures: Map[Int, Int], + numClasses: Int): DataFrame = { + val sqlContext = new SQLContext(data.sparkContext) + import sqlContext.implicits._ + val df = data.toDF() + val numFeatures = data.first().features.size + val featuresAttributes = Range(0, numFeatures).map { feature => + if (categoricalFeatures.contains(feature)) { + NominalAttribute.defaultAttr.withIndex(feature).withNumValues(categoricalFeatures(feature)) + } else { + NumericAttribute.defaultAttr.withIndex(feature) + } + }.toArray + val featuresMetadata = new AttributeGroup("features", featuresAttributes).toMetadata() + val labelAttribute = if (numClasses == 0) { + NumericAttribute.defaultAttr.withName("label") + } else { + NominalAttribute.defaultAttr.withName("label").withNumValues(numClasses) + } + val labelMetadata = labelAttribute.toMetadata() + df.select(df("features").as("features", featuresMetadata), + df("label").as("label", labelMetadata)) + } + + /** Java-friendly version of [[setMetadata()]] */ + def setMetadata( + data: JavaRDD[LabeledPoint], + categoricalFeatures: java.util.Map[java.lang.Integer, java.lang.Integer], + numClasses: Int): DataFrame = { + setMetadata(data.rdd, categoricalFeatures.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numClasses) + } + + /** + * Check if the two trees are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + * If the trees are not equal, this prints the two trees and throws an exception. + */ + def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + try { + checkEqual(a.rootNode, b.rootNode) + } catch { + case ex: Exception => + throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + + "TREE A:\n" + a.toDebugString + "\n" + + "TREE B:\n" + b.toDebugString + "\n", ex) + } + } + + /** + * Return true iff the two nodes and their descendants are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + */ + private def checkEqual(a: Node, b: Node): Unit = { + assert(a.prediction === b.prediction) + assert(a.impurity === b.impurity) + (a, b) match { + case (aye: InternalNode, bee: InternalNode) => + assert(aye.split === bee.split) + checkEqual(aye.leftChild, bee.leftChild) + checkEqual(aye.rightChild, bee.rightChild) + case (aye: LeafNode, bee: LeafNode) => // do nothing + case _ => + throw new AssertionError("Found mismatched nodes") + } + } + + // TODO: Reinstate after adding ensembles + /** + * Check if the two models are exactly the same. + * If the models are not equal, this throws an exception. + */ + /* + def checkEqual(a: TreeEnsembleModel, b: TreeEnsembleModel): Unit = { + try { + a.getTrees.zip(b.getTrees).foreach { case (treeA, treeB) => + TreeTests.checkEqual(treeA, treeB) + } + assert(a.getTreeWeights === b.getTreeWeights) + } catch { + case ex: Exception => throw new AssertionError( + "checkEqual failed since the two tree ensembles were not identical") + } + } + */ +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala new file mode 100644 index 0000000000000..0b40fe33fae9d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.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.ml.regression + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, + DecisionTreeSuite => OldDecisionTreeSuite} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +class DecisionTreeRegressorSuite extends FunSuite with MLlibTestSparkContext { + + import DecisionTreeRegressorSuite.compareAPIs + + private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + categoricalDataPointsRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + test("Regression stump with 3-ary (ordered) categorical features") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3, 1-> 3) + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) + } + + test("Regression stump with binary (ordered) categorical features") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 2, 1-> 2) + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: test("model save/load") +} + +private[ml] object DecisionTreeRegressorSuite extends FunSuite { + + /** + * Train 2 decision trees on the given dataset, one using the old API and one using the new API. + * Convert the old tree to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + dt: DecisionTreeRegressor, + categoricalFeatures: Map[Int, Int]): Unit = { + val oldStrategy = dt.getOldStrategy(categoricalFeatures) + val oldTree = OldDecisionTree.train(data, oldStrategy) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) + val newTree = dt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldTreeAsNew = DecisionTreeRegressionModel.fromOld(oldTree, newTree.parent, + newTree.fittingParamMap, categoricalFeatures) + TreeTests.checkEqual(oldTreeAsNew, newTree) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 4c162df810bb2..249b8eae19b17 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -36,6 +36,10 @@ import org.apache.spark.util.Utils class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { + ///////////////////////////////////////////////////////////////////////////// + // Tests examining individual elements of training + ///////////////////////////////////////////////////////////////////////////// + test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) @@ -254,6 +258,165 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(bins(0).length === 0) } + test("Avoid aggregation on the last level") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue leaf nodes into node queue + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } + + test("Avoid aggregation if impurity is 0.0") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue a node into node queue if its impurity is 0.0 + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } + + test("Second level node building with vs. without groups") { + val arr = DecisionTreeSuite.generateOrderedLabeledPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + + // Train a 1-node model + val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, + numClasses = 2, maxBins = 100) + val modelOneNode = DecisionTree.train(rdd, strategyOneNode) + val rootNode1 = modelOneNode.topNode.deepCopy() + val rootNode2 = modelOneNode.topNode.deepCopy() + assert(rootNode1.leftNode.nonEmpty) + assert(rootNode1.rightNode.nonEmpty) + + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + // Single group second level tree construction. + val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) + val treeToNodeToIndexInfo = Map((0, Map( + (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), + (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + val children1 = new Array[Node](2) + children1(0) = rootNode1.leftNode.get + children1(1) = rootNode1.rightNode.get + + // Train one second-level node at a time. + val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) + val treeToNodeToIndexInfoA = Map((0, Map( + (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) + val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) + val treeToNodeToIndexInfoB = Map((0, Map( + (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) + val children2 = new Array[Node](2) + children2(0) = rootNode2.leftNode.get + children2(1) = rootNode2.rightNode.get + + // Verify whether the splits obtained using single group and multiple group level + // construction strategies are the same. + for (i <- 0 until 2) { + assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) + assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) + assert(children1(i).split === children2(i).split) + assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) + val stats1 = children1(i).stats.get + val stats2 = children2(i).stats.get + assert(stats1.gain === stats2.gain) + assert(stats1.impurity === stats2.impurity) + assert(stats1.leftImpurity === stats2.leftImpurity) + assert(stats1.rightImpurity === stats2.rightImpurity) + assert(children1(i).predict.predict === children2(i).predict.predict) + } + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// test("Binary classification stump with ordered categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() @@ -438,76 +601,6 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(rootNode.predict.predict === 1) } - test("Second level node building with vs. without groups") { - val arr = DecisionTreeSuite.generateOrderedLabeledPoints() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - - // Train a 1-node model - val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, - numClasses = 2, maxBins = 100) - val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val rootNode1 = modelOneNode.topNode.deepCopy() - val rootNode2 = modelOneNode.topNode.deepCopy() - assert(rootNode1.leftNode.nonEmpty) - assert(rootNode1.rightNode.nonEmpty) - - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - // Single group second level tree construction. - val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) - val treeToNodeToIndexInfo = Map((0, Map( - (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), - (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - val children1 = new Array[Node](2) - children1(0) = rootNode1.leftNode.get - children1(1) = rootNode1.rightNode.get - - // Train one second-level node at a time. - val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) - val treeToNodeToIndexInfoA = Map((0, Map( - (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) - val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) - val treeToNodeToIndexInfoB = Map((0, Map( - (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) - val children2 = new Array[Node](2) - children2(0) = rootNode2.leftNode.get - children2(1) = rootNode2.rightNode.get - - // Verify whether the splits obtained using single group and multiple group level - // construction strategies are the same. - for (i <- 0 until 2) { - assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) - assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) - assert(children1(i).split === children2(i).split) - assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) - val stats1 = children1(i).stats.get - val stats2 = children2(i).stats.get - assert(stats1.gain === stats2.gain) - assert(stats1.impurity === stats2.impurity) - assert(stats1.leftImpurity === stats2.leftImpurity) - assert(stats1.rightImpurity === stats2.rightImpurity) - assert(children1(i).predict.predict === children2(i).predict.predict) - } - } - test("Multiclass classification stump with 3-ary (unordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) @@ -528,11 +621,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) - arr(2) = new LabeledPoint(1.0, Vectors.dense(2.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(3.0))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 2) @@ -544,11 +637,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("Binary classification stump with 2 continuous features") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0))))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -668,11 +761,10 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("split must satisfy min instances per node requirements") { - val arr = new Array[LabeledPoint](3) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) - + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, numClasses = 2, minInstancesPerNode = 2) @@ -695,11 +787,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { test("do not choose split that does not satisfy min instance per node requirements") { // if a split does not satisfy min instances per node requirements, // this split is invalid, even though the information gain of split is large. - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0, 1.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) - arr(3) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, @@ -715,10 +807,10 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("split must satisfy min info gain requirements") { - val arr = new Array[LabeledPoint](3) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, @@ -739,91 +831,9 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(gain == InformationGainStats.invalidInformationGainStats) } - test("Avoid aggregation on the last level") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue leaf nodes into node queue - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } - - test("Avoid aggregation if impurity is 0.0") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue a node into node queue if its impurity is 0.0 - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// test("Node.subtreeIterator") { val model = DecisionTreeSuite.createModel(Classification) @@ -996,8 +1006,9 @@ object DecisionTreeSuite extends FunSuite { /** * Create a tree model. This is deterministic and contains a variety of node and feature types. + * TODO: Update this to be a correct tree (with matching probabilities, impurities, etc.) */ - private[tree] def createModel(algo: Algo): DecisionTreeModel = { + private[mllib] def createModel(algo: Algo): DecisionTreeModel = { val topNode = createInternalNode(id = 1, Continuous) val (node2, node3) = (createLeafNode(id = 2), createInternalNode(id = 3, Categorical)) val (node6, node7) = (createLeafNode(id = 6), createLeafNode(id = 7)) @@ -1017,7 +1028,7 @@ object DecisionTreeSuite extends FunSuite { * make mistakes such as creating loops of Nodes. * If the trees are not equal, this prints the two trees and throws an exception. */ - private[tree] def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + private[mllib] def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { try { assert(a.algo === b.algo) checkEqual(a.topNode, b.topNode) From 59e206deb7346148412bbf5ba4ab626718fadf18 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Fri, 17 Apr 2015 13:42:19 -0700 Subject: [PATCH 067/144] [SPARK-6807] [SparkR] Merge recent SparkR-pkg changes This PR pulls in recent changes in SparkR-pkg, including cartesian, intersection, sampleByKey, subtract, subtractByKey, except, and some API for StructType and StructField. Author: cafreeman Author: Davies Liu Author: Zongheng Yang Author: Shivaram Venkataraman Author: Shivaram Venkataraman Author: Sun Rui Closes #5436 from davies/R3 and squashes the following commits: c2b09be [Davies Liu] SQLTypes -> schema a5a02f2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R3 168b7fe [Davies Liu] sort generics b1fe460 [Davies Liu] fix conflict in README.md e74c04e [Davies Liu] fix schema.R 4f5ac09 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R5 41f8184 [Davies Liu] rm man ae78312 [Davies Liu] Merge pull request #237 from sun-rui/SPARKR-154_3 1bdcb63 [Zongheng Yang] Updates to README.md. 5a553e7 [cafreeman] Use object attribute instead of argument 71372d9 [cafreeman] Update docs and examples 8526d2e71 [cafreeman] Remove `tojson` functions 6ef5f2d [cafreeman] Fix spacing 7741d66 [cafreeman] Rename the SQL DataType function 141efd8 [Shivaram Venkataraman] Merge pull request #245 from hqzizania/upstream 9387402 [Davies Liu] fix style 40199eb [Shivaram Venkataraman] Move except into sorted position 07d0dbc [Sun Rui] [SPARKR-244] Fix test failure after integration of subtract() and subtractByKey() for RDD. 7e8caa3 [Shivaram Venkataraman] Merge pull request #246 from hlin09/fixCombineByKey ed66c81 [cafreeman] Update `subtract` to work with `generics.R` f3ba785 [cafreeman] Fixed duplicate export 275deb4 [cafreeman] Update `NAMESPACE` and tests 1a3b63d [cafreeman] new version of `CreateDF` 836c4bf [cafreeman] Update `createDataFrame` and `toDF` be5d5c1 [cafreeman] refactor schema functions 40338a4 [Zongheng Yang] Merge pull request #244 from sun-rui/SPARKR-154_5 20b97a6 [Zongheng Yang] Merge pull request #234 from hqzizania/assist ba54e34 [Shivaram Venkataraman] Merge pull request #238 from sun-rui/SPARKR-154_4 c9497a3 [Shivaram Venkataraman] Merge pull request #208 from lythesia/master b317aa7 [Zongheng Yang] Merge pull request #243 from hqzizania/master 136a07e [Zongheng Yang] Merge pull request #242 from hqzizania/stats cd66603 [cafreeman] new line at EOF 8b76e81 [Shivaram Venkataraman] Merge pull request #233 from redbaron/fail-early-on-missing-dep 7dd81b7 [cafreeman] Documentation 0e2a94f [cafreeman] Define functions for schema and fields --- R/pkg/DESCRIPTION | 2 +- R/pkg/NAMESPACE | 20 +- R/pkg/R/DataFrame.R | 18 +- R/pkg/R/RDD.R | 205 ++++++++++++------ R/pkg/R/SQLContext.R | 44 +--- R/pkg/R/SQLTypes.R | 64 ------ R/pkg/R/column.R | 2 +- R/pkg/R/generics.R | 46 +++- R/pkg/R/group.R | 2 +- R/pkg/R/pairRDD.R | 192 +++++++++++++--- R/pkg/R/schema.R | 162 ++++++++++++++ R/pkg/R/serialize.R | 9 +- R/pkg/R/utils.R | 80 +++++++ R/pkg/inst/tests/test_rdd.R | 193 ++++++++++++++--- R/pkg/inst/tests/test_shuffle.R | 12 + R/pkg/inst/tests/test_sparkSQL.R | 35 +-- R/pkg/inst/worker/worker.R | 59 ++++- .../scala/org/apache/spark/api/r/RRDD.scala | 131 +++++------ .../scala/org/apache/spark/api/r/SerDe.scala | 14 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 32 ++- 20 files changed, 971 insertions(+), 351 deletions(-) delete mode 100644 R/pkg/R/SQLTypes.R create mode 100644 R/pkg/R/schema.R diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 052f68c6c24e2..1c1779a763c7e 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -19,7 +19,7 @@ Collate: 'jobj.R' 'RDD.R' 'pairRDD.R' - 'SQLTypes.R' + 'schema.R' 'column.R' 'group.R' 'DataFrame.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index a354cdce74afa..80283643861ac 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -5,6 +5,7 @@ exportMethods( "aggregateByKey", "aggregateRDD", "cache", + "cartesian", "checkpoint", "coalesce", "cogroup", @@ -28,6 +29,7 @@ exportMethods( "fullOuterJoin", "glom", "groupByKey", + "intersection", "join", "keyBy", "keys", @@ -52,11 +54,14 @@ exportMethods( "reduceByKeyLocally", "repartition", "rightOuterJoin", + "sampleByKey", "sampleRDD", "saveAsTextFile", "saveAsObjectFile", "sortBy", "sortByKey", + "subtract", + "subtractByKey", "sumRDD", "take", "takeOrdered", @@ -95,6 +100,7 @@ exportClasses("DataFrame") exportMethods("columns", "distinct", "dtypes", + "except", "explain", "filter", "groupBy", @@ -118,7 +124,6 @@ exportMethods("columns", "show", "showDF", "sortDF", - "subtract", "toJSON", "toRDD", "unionAll", @@ -178,5 +183,14 @@ export("cacheTable", "toDF", "uncacheTable") -export("print.structType", - "print.structField") +export("sparkRSQL.init", + "sparkRHive.init") + +export("structField", + "structField.jobj", + "structField.character", + "print.structField", + "structType", + "structType.jobj", + "structType.structField", + "print.structType") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 044fdb4d01223..861fe1c78b0db 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -17,7 +17,7 @@ # DataFrame.R - DataFrame class and methods implemented in S4 OO classes -#' @include generics.R jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +#' @include generics.R jobj.R schema.R RDD.R pairRDD.R column.R group.R NULL setOldClass("jobj") @@ -1141,15 +1141,15 @@ setMethod("intersect", dataFrame(intersected) }) -#' Subtract +#' except #' #' Return a new DataFrame containing rows in this DataFrame #' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL. #' #' @param x A Spark DataFrame #' @param y A Spark DataFrame -#' @return A DataFrame containing the result of the subtract operation. -#' @rdname subtract +#' @return A DataFrame containing the result of the except operation. +#' @rdname except #' @export #' @examples #'\dontrun{ @@ -1157,13 +1157,15 @@ setMethod("intersect", #' sqlCtx <- sparkRSQL.init(sc) #' df1 <- jsonFile(sqlCtx, path) #' df2 <- jsonFile(sqlCtx, path2) -#' subtractDF <- subtract(df, df2) +#' exceptDF <- except(df, df2) #' } -setMethod("subtract", +#' @rdname except +#' @export +setMethod("except", signature(x = "DataFrame", y = "DataFrame"), function(x, y) { - subtracted <- callJMethod(x@sdf, "except", y@sdf) - dataFrame(subtracted) + excepted <- callJMethod(x@sdf, "except", y@sdf) + dataFrame(excepted) }) #' Save the contents of the DataFrame to a data source diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 820027ef67e3b..128431334ca52 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -730,6 +730,7 @@ setMethod("take", index <- -1 jrdd <- getJRDD(x) numPartitions <- numPartitions(x) + serializedModeRDD <- getSerializedMode(x) # TODO(shivaram): Collect more than one partition based on size # estimates similar to the scala version of `take`. @@ -748,13 +749,14 @@ setMethod("take", elems <- convertJListToRList(partition, flatten = TRUE, logicalUpperBound = size, - serializedMode = getSerializedMode(x)) - # TODO: Check if this append is O(n^2)? + serializedMode = serializedModeRDD) + resList <- append(resList, elems) } resList }) + #' First #' #' Return the first element of an RDD @@ -1092,21 +1094,42 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { if (num < length(part)) { # R limitation: order works only on primitive types! ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) - list(part[ord[1:num]]) + part[ord[1:num]] } else { - list(part) + part } } - reduceFunc <- function(elems, part) { - newElems <- append(elems, part) - # R limitation: order works only on primitive types! - ord <- order(unlist(newElems, recursive = FALSE), decreasing = !ascending) - newElems[ord[1:num]] - } - newRdd <- mapPartitions(x, partitionFunc) - reduce(newRdd, reduceFunc) + + resList <- list() + index <- -1 + jrdd <- getJRDD(newRdd) + numPartitions <- numPartitions(newRdd) + serializedModeRDD <- getSerializedMode(newRdd) + + while (TRUE) { + index <- index + 1 + + if (index >= numPartitions) { + ord <- order(unlist(resList, recursive = FALSE), decreasing = !ascending) + resList <- resList[ord[1:num]] + break + } + + # a JList of byte arrays + partitionArr <- callJMethod(jrdd, "collectPartitions", as.list(as.integer(index))) + partition <- partitionArr[[1]] + + # elems is capped to have at most `num` elements + elems <- convertJListToRList(partition, + flatten = TRUE, + logicalUpperBound = num, + serializedMode = serializedModeRDD) + + resList <- append(resList, elems) + } + resList } #' Returns the first N elements from an RDD in ascending order. @@ -1465,67 +1488,105 @@ setMethod("zipRDD", stop("Can only zip RDDs which have the same number of partitions.") } - if (getSerializedMode(x) != getSerializedMode(other) || - getSerializedMode(x) == "byte") { - # Append the number of elements in each partition to that partition so that we can later - # check if corresponding partitions of both RDDs have the same number of elements. - # - # Note that this appending also serves the purpose of reserialization, because even if - # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded - # as a single byte array. For example, partitions of an RDD generated from partitionBy() - # may be encoded as multiple byte arrays. - appendLength <- function(part) { - part[[length(part) + 1]] <- length(part) + 1 - part - } - x <- lapplyPartition(x, appendLength) - other <- lapplyPartition(other, appendLength) - } + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "zip", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) - zippedJRDD <- callJMethod(getJRDD(x), "zip", getJRDD(other)) - # The zippedRDD's elements are of scala Tuple2 type. The serialized - # flag Here is used for the elements inside the tuples. - serializerMode <- getSerializedMode(x) - zippedRDD <- RDD(zippedJRDD, serializerMode) + mergePartitions(rdd, TRUE) + }) + +#' Cartesian product of this RDD and another one. +#' +#' Return the Cartesian product of this RDD and another one, +#' that is, the RDD of all pairs of elements (a, b) where a +#' is in this and b is in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @return A new RDD which is the Cartesian product of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2) +#' sortByKey(cartesian(rdd, rdd)) +#' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +#'} +#' @rdname cartesian +#' @aliases cartesian,RDD,RDD-method +setMethod("cartesian", + signature(x = "RDD", other = "RDD"), + function(x, other) { + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "cartesian", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) - partitionFunc <- function(split, part) { - len <- length(part) - if (len > 0) { - if (serializerMode == "byte") { - lengthOfValues <- part[[len]] - lengthOfKeys <- part[[len - lengthOfValues]] - stopifnot(len == lengthOfKeys + lengthOfValues) - - # check if corresponding partitions of both RDDs have the same number of elements. - if (lengthOfKeys != lengthOfValues) { - stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") - } - - if (lengthOfKeys > 1) { - keys <- part[1 : (lengthOfKeys - 1)] - values <- part[(lengthOfKeys + 1) : (len - 1)] - } else { - keys <- list() - values <- list() - } - } else { - # Keys, values must have same length here, because this has - # been validated inside the JavaRDD.zip() function. - keys <- part[c(TRUE, FALSE)] - values <- part[c(FALSE, TRUE)] - } - mapply( - function(k, v) { - list(k, v) - }, - keys, - values, - SIMPLIFY = FALSE, - USE.NAMES = FALSE) - } else { - part - } + mergePartitions(rdd, FALSE) + }) + +#' Subtract an RDD with another RDD. +#' +#' Return an RDD with the elements from this that are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the elements from this that are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +#' rdd2 <- parallelize(sc, list(2, 4)) +#' collect(subtract(rdd1, rdd2)) +#' # list(1, 1, 3) +#'} +#' @rdname subtract +#' @aliases subtract,RDD +setMethod("subtract", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + mapFunction <- function(e) { list(e, NA) } + rdd1 <- map(x, mapFunction) + rdd2 <- map(other, mapFunction) + keys(subtractByKey(rdd1, rdd2, numPartitions)) + }) + +#' Intersection of this RDD and another one. +#' +#' Return the intersection of this RDD and another one. +#' The output will not contain any duplicate elements, +#' even if the input RDDs did. Performs a hash partition +#' across the cluster. +#' Note that this method performs a shuffle internally. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions The number of partitions in the result RDD. +#' @return An RDD which is the intersection of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +#' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +#' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +#' # list(1, 2, 3) +#'} +#' @rdname intersection +#' @aliases intersection,RDD +setMethod("intersection", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + rdd1 <- map(x, function(v) { list(v, NA) }) + rdd2 <- map(other, function(v) { list(v, NA) }) + + filterFunction <- function(elem) { + iters <- elem[[2]] + all(as.vector( + lapply(iters, function(iter) { length(iter) > 0 }), mode = "logical")) } - - PipelinedRDD(zippedRDD, partitionFunc) + + keys(filterRDD(cogroup(rdd1, rdd2, numPartitions = numPartitions), filterFunction)) }) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 930ada22f4c38..4f05ba524a01a 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -54,9 +54,9 @@ infer_type <- function(x) { # StructType types <- lapply(x, infer_type) fields <- lapply(1:length(x), function(i) { - list(name = names[[i]], type = types[[i]], nullable = TRUE) + structField(names[[i]], types[[i]], TRUE) }) - list(type = "struct", fields = fields) + do.call(structType, fields) } } else if (length(x) > 1) { list(type = "array", elementType = type, containsNull = TRUE) @@ -65,30 +65,6 @@ infer_type <- function(x) { } } -#' dump the schema into JSON string -tojson <- function(x) { - if (is.list(x)) { - names <- names(x) - if (!is.null(names)) { - items <- lapply(names, function(n) { - safe_n <- gsub('"', '\\"', n) - paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') - }) - d <- paste(items, collapse = ', ') - paste('{', d, '}', sep = '') - } else { - l <- paste(lapply(x, tojson), collapse = ', ') - paste('[', l, ']', sep = '') - } - } else if (is.character(x)) { - paste('"', x, '"', sep = '') - } else if (is.logical(x)) { - if (x) "true" else "false" - } else { - stop(paste("unexpected type:", class(x))) - } -} - #' Create a DataFrame from an RDD #' #' Converts an RDD to a DataFrame by infer the types. @@ -134,7 +110,7 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { stop(paste("unexpected type:", class(data))) } - if (is.null(schema) || is.null(names(schema))) { + if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { row <- first(rdd) names <- if (is.null(schema)) { names(row) @@ -143,7 +119,7 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { } if (is.null(names)) { names <- lapply(1:length(row), function(x) { - paste("_", as.character(x), sep = "") + paste("_", as.character(x), sep = "") }) } @@ -159,20 +135,18 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { types <- lapply(row, infer_type) fields <- lapply(1:length(row), function(i) { - list(name = names[[i]], type = types[[i]], nullable = TRUE) + structField(names[[i]], types[[i]], TRUE) }) - schema <- list(type = "struct", fields = fields) + schema <- do.call(structType, fields) } - stopifnot(class(schema) == "list") - stopifnot(schema$type == "struct") - stopifnot(class(schema$fields) == "list") - schemaString <- tojson(schema) + stopifnot(class(schema) == "structType") + # schemaString <- tojson(schema) jrdd <- getJRDD(lapply(rdd, function(x) x), "row") srdd <- callJMethod(jrdd, "rdd") sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", - srdd, schemaString, sqlCtx) + srdd, schema$jobj, sqlCtx) dataFrame(sdf) } diff --git a/R/pkg/R/SQLTypes.R b/R/pkg/R/SQLTypes.R deleted file mode 100644 index 962fba5b3cf03..0000000000000 --- a/R/pkg/R/SQLTypes.R +++ /dev/null @@ -1,64 +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. -# - -# Utility functions for handling SparkSQL DataTypes. - -# Handler for StructType -structType <- function(st) { - obj <- structure(new.env(parent = emptyenv()), class = "structType") - obj$jobj <- st - obj$fields <- function() { lapply(callJMethod(st, "fields"), structField) } - obj -} - -#' Print a Spark StructType. -#' -#' This function prints the contents of a StructType returned from the -#' SparkR JVM backend. -#' -#' @param x A StructType object -#' @param ... further arguments passed to or from other methods -print.structType <- function(x, ...) { - fieldsList <- lapply(x$fields(), function(i) { i$print() }) - print(fieldsList) -} - -# Handler for StructField -structField <- function(sf) { - obj <- structure(new.env(parent = emptyenv()), class = "structField") - obj$jobj <- sf - obj$name <- function() { callJMethod(sf, "name") } - obj$dataType <- function() { callJMethod(sf, "dataType") } - obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } - obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } - obj$nullable <- function() { callJMethod(sf, "nullable") } - obj$print <- function() { paste("StructField(", - paste(obj$name(), obj$dataType.toString(), obj$nullable(), sep = ", "), - ")", sep = "") } - obj -} - -#' Print a Spark StructField. -#' -#' This function prints the contents of a StructField returned from the -#' SparkR JVM backend. -#' -#' @param x A StructField object -#' @param ... further arguments passed to or from other methods -print.structField <- function(x, ...) { - cat(x$print()) -} diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index b282001d8b6b5..95fb9ff0887b6 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -17,7 +17,7 @@ # Column Class -#' @include generics.R jobj.R SQLTypes.R +#' @include generics.R jobj.R schema.R NULL setOldClass("jobj") diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 5fb1ccaa84ee2..6c6233390134c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -230,6 +230,10 @@ setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") } ############ Binary Functions ############# +#' @rdname cartesian +#' @export +setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) + #' @rdname countByKey #' @export setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) @@ -238,6 +242,11 @@ setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) #' @export setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) +#' @rdname intersection +#' @export +setGeneric("intersection", function(x, other, numPartitions = 1L) { + standardGeneric("intersection") }) + #' @rdname keys #' @export setGeneric("keys", function(x) { standardGeneric("keys") }) @@ -250,12 +259,18 @@ setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) #' @export setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) +#' @rdname sampleByKey +#' @export +setGeneric("sampleByKey", + function(x, withReplacement, fractions, seed) { + standardGeneric("sampleByKey") + }) + #' @rdname values #' @export setGeneric("values", function(x) { standardGeneric("values") }) - ############ Shuffle Functions ############ #' @rdname aggregateByKey @@ -330,9 +345,24 @@ setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("ri #' @rdname sortByKey #' @export -setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1L) { - standardGeneric("sortByKey") -}) +setGeneric("sortByKey", + function(x, ascending = TRUE, numPartitions = 1L) { + standardGeneric("sortByKey") + }) + +#' @rdname subtract +#' @export +setGeneric("subtract", + function(x, other, numPartitions = 1L) { + standardGeneric("subtract") + }) + +#' @rdname subtractByKey +#' @export +setGeneric("subtractByKey", + function(x, other, numPartitions = 1L) { + standardGeneric("subtractByKey") + }) ################### Broadcast Variable Methods ################# @@ -357,6 +387,10 @@ setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) #' @export setGeneric("explain", function(x, ...) { standardGeneric("explain") }) +#' @rdname except +#' @export +setGeneric("except", function(x, y) { standardGeneric("except") }) + #' @rdname filter #' @export setGeneric("filter", function(x, condition) { standardGeneric("filter") }) @@ -434,10 +468,6 @@ setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) #' @export setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) -#' @rdname subtract -#' @export -setGeneric("subtract", function(x, y) { standardGeneric("subtract") }) - #' @rdname tojson #' @export setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 855fbdfc7c4ca..02237b3672d6b 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -17,7 +17,7 @@ # group.R - GroupedData class and methods implemented in S4 OO classes -#' @include generics.R jobj.R SQLTypes.R column.R +#' @include generics.R jobj.R schema.R column.R NULL setOldClass("jobj") diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 5d64822859d1f..13efebc11c46e 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -430,7 +430,7 @@ setMethod("combineByKey", pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - item$hash <- as.character(item[[1]]) + item$hash <- as.character(hashCode(item[[1]])) updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner) }) convertEnvsToList(keys, combiners) @@ -443,7 +443,7 @@ setMethod("combineByKey", pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - item$hash <- as.character(item[[1]]) + item$hash <- as.character(hashCode(item[[1]])) updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity) }) convertEnvsToList(keys, combiners) @@ -452,19 +452,19 @@ setMethod("combineByKey", }) #' Aggregate a pair RDD by each key. -#' +#' #' Aggregate the values of each key in an RDD, using given combine functions #' and a neutral "zero value". This function can return a different result type, #' U, than the type of the values in this RDD, V. Thus, we need one operation -#' for merging a V into a U and one operation for merging two U's, The former -#' operation is used for merging values within a partition, and the latter is -#' used for merging values between partitions. To avoid memory allocation, both -#' of these functions are allowed to modify and return their first argument +#' for merging a V into a U and one operation for merging two U's, The former +#' operation is used for merging values within a partition, and the latter is +#' used for merging values between partitions. To avoid memory allocation, both +#' of these functions are allowed to modify and return their first argument #' instead of creating a new U. -#' +#' #' @param x An RDD. #' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the values of each key. It may return +#' @param seqOp A function to aggregate the values of each key. It may return #' a different result type from the type of the values. #' @param combOp A function to aggregate results of seqOp. #' @return An RDD containing the aggregation result. @@ -476,7 +476,7 @@ setMethod("combineByKey", #' zeroValue <- list(0, 0) #' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } #' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) #' # list(list(1, list(3, 2)), list(2, list(7, 2))) #'} #' @rdname aggregateByKey @@ -493,12 +493,12 @@ setMethod("aggregateByKey", }) #' Fold a pair RDD by each key. -#' +#' #' Aggregate the values of each key in an RDD, using an associative function "func" -#' and a neutral "zero value" which may be added to the result an arbitrary -#' number of times, and must not change the result (e.g., 0 for addition, or +#' and a neutral "zero value" which may be added to the result an arbitrary +#' number of times, and must not change the result (e.g., 0 for addition, or #' 1 for multiplication.). -#' +#' #' @param x An RDD. #' @param zeroValue A neutral "zero value". #' @param func An associative function for folding values of each key. @@ -548,11 +548,11 @@ setMethod("join", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(FALSE, FALSE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), doJoin) }) @@ -568,8 +568,8 @@ setMethod("join", #' @param y An RDD to be joined. Should be an RDD where each element is #' list(K, V). #' @param numPartitions Number of partitions to create. -#' @return For each element (k, v) in x, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +#' @return For each element (k, v) in x, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) #' if no elements in rdd2 have key k. #' @examples #'\dontrun{ @@ -586,11 +586,11 @@ setMethod("leftOuterJoin", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(FALSE, TRUE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) @@ -623,18 +623,18 @@ setMethod("rightOuterJoin", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(TRUE, FALSE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Full outer join two RDDs #' #' @description -#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). #' The key types of the two RDDs should be the same. #' #' @param x An RDD to be joined. Should be an RDD where each element is @@ -644,7 +644,7 @@ setMethod("rightOuterJoin", #' @param numPartitions Number of partitions to create. #' @return For each element (k, v) in x and (k, w) in y, the resulting RDD #' will contain all pairs (k, (v, w)) for both (k, v) in x and -#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements #' in x/y have key k. #' @examples #'\dontrun{ @@ -683,7 +683,7 @@ setMethod("fullOuterJoin", #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) #' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' cogroup(rdd1, rdd2, numPartitions = 2L) +#' cogroup(rdd1, rdd2, numPartitions = 2L) #' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) #'} #' @rdname cogroup @@ -694,7 +694,7 @@ setMethod("cogroup", rdds <- list(...) rddsLen <- length(rdds) for (i in 1:rddsLen) { - rdds[[i]] <- lapply(rdds[[i]], + rdds[[i]] <- lapply(rdds[[i]], function(x) { list(x[[1]], list(i, x[[2]])) }) } union.rdd <- Reduce(unionRDD, rdds) @@ -719,7 +719,7 @@ setMethod("cogroup", } }) } - cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), + cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), group.func) }) @@ -741,18 +741,18 @@ setMethod("sortByKey", signature(x = "RDD"), function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { rangeBounds <- list() - + if (numPartitions > 1) { rddSize <- count(x) # constant from Spark's RangePartitioner maxSampleSize <- numPartitions * 20 fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) - + samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L))) - + # Note: the built-in R sort() function only works on atomic vectors samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) - + if (length(samples) > 0) { rangeBounds <- lapply(seq_len(numPartitions - 1), function(i) { @@ -764,24 +764,146 @@ setMethod("sortByKey", rangePartitionFunc <- function(key) { partition <- 0 - + # TODO: Use binary search instead of linear search, similar with Spark while (partition < length(rangeBounds) && key > rangeBounds[[partition + 1]]) { partition <- partition + 1 } - + if (ascending) { partition } else { numPartitions - partition - 1 } } - + partitionFunc <- function(part) { sortKeyValueList(part, decreasing = !ascending) } - + newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) lapplyPartition(newRDD, partitionFunc) }) +#' Subtract a pair RDD with another pair RDD. +#' +#' Return an RDD with the pairs from x whose keys are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the pairs from x whose keys are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), +#' list("b", 5), list("a", 2))) +#' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +#' collect(subtractByKey(rdd1, rdd2)) +#' # list(list("b", 4), list("b", 5)) +#'} +#' @rdname subtractByKey +#' @aliases subtractByKey,RDD +setMethod("subtractByKey", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + filterFunction <- function(elem) { + iters <- elem[[2]] + (length(iters[[1]]) > 0) && (length(iters[[2]]) == 0) + } + + flatMapValues(filterRDD(cogroup(x, + other, + numPartitions = numPartitions), + filterFunction), + function (v) { v[[1]] }) + }) + +#' Return a subset of this RDD sampled by key. +#' +#' @description +#' \code{sampleByKey} Create a sample of this RDD using variable sampling rates +#' for different keys as specified by fractions, a key to sampling rate map. +#' +#' @param x The RDD to sample elements by key, where each element is +#' list(K, V) or c(K, V). +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3000) +#' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) +#' else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +#' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +#' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +#' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +#' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +#' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +#' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +#' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +#' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +#' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +#' fractions <- list(a = 0.2, b = 0.1) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +#'} +#' @rdname sampleByKey +#' @aliases sampleByKey,RDD-method +setMethod("sampleByKey", + signature(x = "RDD", withReplacement = "logical", + fractions = "vector", seed = "integer"), + function(x, withReplacement, fractions, seed) { + + for (elem in fractions) { + if (elem < 0.0) { + stop(paste("Negative fraction value ", fractions[which(fractions == elem)])) + } + } + + # The sampler: takes a partition and returns its sampled version. + samplingFunc <- function(split, part) { + set.seed(bitwXor(seed, split)) + res <- vector("list", length(part)) + len <- 0 + + # mixing because the initial seeds are close to each other + runif(10) + + for (elem in part) { + if (elem[[1]] %in% names(fractions)) { + frac <- as.numeric(fractions[which(elem[[1]] == names(fractions))]) + if (withReplacement) { + count <- rpois(1, frac) + if (count > 0) { + res[(len + 1):(len + count)] <- rep(list(elem), count) + len <- len + count + } + } else { + if (runif(1) < frac) { + len <- len + 1 + res[[len]] <- elem + } + } + } else { + stop("KeyError: \"", elem[[1]], "\"") + } + } + + # TODO(zongheng): look into the performance of the current + # implementation. Look into some iterator package? Note that + # Scala avoids many calls to creating an empty list and PySpark + # similarly achieves this using `yield'. (duplicated from sampleRDD) + if (len > 0) { + res[1:len] + } else { + list() + } + } + + lapplyPartitionsWithIndex(x, samplingFunc) + }) diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R new file mode 100644 index 0000000000000..e442119086b17 --- /dev/null +++ b/R/pkg/R/schema.R @@ -0,0 +1,162 @@ +# +# 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. +# + +# A set of S3 classes and methods that support the SparkSQL `StructType` and `StructField +# datatypes. These are used to create and interact with DataFrame schemas. + +#' structType +#' +#' Create a structType object that contains the metadata for a DataFrame. Intended for +#' use with createDataFrame and toDF. +#' +#' @param x a structField object (created with the field() function) +#' @param ... additional structField objects +#' @return a structType object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' schema <- structType(structField("a", "integer"), structField("b", "string")) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } +structType <- function(x, ...) { + UseMethod("structType", x) +} + +structType.jobj <- function(x) { + obj <- structure(list(), class = "structType") + obj$jobj <- x + obj$fields <- function() { lapply(callJMethod(obj$jobj, "fields"), structField) } + obj +} + +structType.structField <- function(x, ...) { + fields <- list(x, ...) + if (!all(sapply(fields, inherits, "structField"))) { + stop("All arguments must be structField objects.") + } + sfObjList <- lapply(fields, function(field) { + field$jobj + }) + stObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createStructType", + listToSeq(sfObjList)) + structType(stObj) +} + +#' Print a Spark StructType. +#' +#' This function prints the contents of a StructType returned from the +#' SparkR JVM backend. +#' +#' @param x A StructType object +#' @param ... further arguments passed to or from other methods +print.structType <- function(x, ...) { + cat("StructType\n", + sapply(x$fields(), function(field) { paste("|-", "name = \"", field$name(), + "\", type = \"", field$dataType.toString(), + "\", nullable = ", field$nullable(), "\n", + sep = "") }) + , sep = "") +} + +#' structField +#' +#' Create a structField object that contains the metadata for a single field in a schema. +#' +#' @param x The name of the field +#' @param type The data type of the field +#' @param nullable A logical vector indicating whether or not the field is nullable +#' @return a structField object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' field1 <- structField("a", "integer", TRUE) +#' field2 <- structField("b", "string", TRUE) +#' schema <- structType(field1, field2) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } + +structField <- function(x, ...) { + UseMethod("structField", x) +} + +structField.jobj <- function(x) { + obj <- structure(list(), class = "structField") + obj$jobj <- x + obj$name <- function() { callJMethod(x, "name") } + obj$dataType <- function() { callJMethod(x, "dataType") } + obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } + obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } + obj$nullable <- function() { callJMethod(x, "nullable") } + obj +} + +structField.character <- function(x, type, nullable = TRUE) { + if (class(x) != "character") { + stop("Field name must be a string.") + } + if (class(type) != "character") { + stop("Field type must be a string.") + } + if (class(nullable) != "logical") { + stop("nullable must be either TRUE or FALSE") + } + options <- c("byte", + "integer", + "double", + "numeric", + "character", + "string", + "binary", + "raw", + "logical", + "boolean", + "timestamp", + "date") + dataType <- if (type %in% options) { + type + } else { + stop(paste("Unsupported type for Dataframe:", type)) + } + sfObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createStructField", + x, + dataType, + nullable) + structField(sfObj) +} + +#' Print a Spark StructField. +#' +#' This function prints the contents of a StructField returned from the +#' SparkR JVM backend. +#' +#' @param x A StructField object +#' @param ... further arguments passed to or from other methods +print.structField <- function(x, ...) { + cat("StructField(name = \"", x$name(), + "\", type = \"", x$dataType.toString(), + "\", nullable = ", x$nullable(), + ")", + sep = "") +} diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 8a9c0c652ce24..c53d0a961016f 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -69,8 +69,9 @@ writeJobj <- function(con, value) { } writeString <- function(con, value) { - writeInt(con, as.integer(nchar(value) + 1)) - writeBin(value, con, endian = "big") + utfVal <- enc2utf8(value) + writeInt(con, as.integer(nchar(utfVal, type = "bytes") + 1)) + writeBin(utfVal, con, endian = "big") } writeInt <- function(con, value) { @@ -189,7 +190,3 @@ writeArgs <- function(con, args) { } } } - -writeStrings <- function(con, stringList) { - writeLines(unlist(stringList), con) -} diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index c337fb0751e72..23305d3c67074 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -465,3 +465,83 @@ cleanClosure <- function(func, checkedFuncs = new.env()) { } func } + +# Append partition lengths to each partition in two input RDDs if needed. +# param +# x An RDD. +# Other An RDD. +# return value +# A list of two result RDDs. +appendPartitionLengths <- function(x, other) { + if (getSerializedMode(x) != getSerializedMode(other) || + getSerializedMode(x) == "byte") { + # Append the number of elements in each partition to that partition so that we can later + # know the boundary of elements from x and other. + # + # Note that this appending also serves the purpose of reserialization, because even if + # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded + # as a single byte array. For example, partitions of an RDD generated from partitionBy() + # may be encoded as multiple byte arrays. + appendLength <- function(part) { + len <- length(part) + part[[len + 1]] <- len + 1 + part + } + x <- lapplyPartition(x, appendLength) + other <- lapplyPartition(other, appendLength) + } + list (x, other) +} + +# Perform zip or cartesian between elements from two RDDs in each partition +# param +# rdd An RDD. +# zip A boolean flag indicating this call is for zip operation or not. +# return value +# A result RDD. +mergePartitions <- function(rdd, zip) { + serializerMode <- getSerializedMode(rdd) + partitionFunc <- function(split, part) { + len <- length(part) + if (len > 0) { + if (serializerMode == "byte") { + lengthOfValues <- part[[len]] + lengthOfKeys <- part[[len - lengthOfValues]] + stopifnot(len == lengthOfKeys + lengthOfValues) + + # For zip operation, check if corresponding partitions of both RDDs have the same number of elements. + if (zip && lengthOfKeys != lengthOfValues) { + stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + } + + if (lengthOfKeys > 1) { + keys <- part[1 : (lengthOfKeys - 1)] + } else { + keys <- list() + } + if (lengthOfValues > 1) { + values <- part[(lengthOfKeys + 1) : (len - 1)] + } else { + values <- list() + } + + if (!zip) { + return(mergeCompactLists(keys, values)) + } + } else { + keys <- part[c(TRUE, FALSE)] + values <- part[c(FALSE, TRUE)] + } + mapply( + function(k, v) { list(k, v) }, + keys, + values, + SIMPLIFY = FALSE, + USE.NAMES = FALSE) + } else { + part + } + } + + PipelinedRDD(rdd, partitionFunc) +} diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index b76e4db03e715..3ba7d1716302a 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -35,7 +35,7 @@ test_that("get number of partitions in RDD", { test_that("first on RDD", { expect_true(first(rdd) == 1) newrdd <- lapply(rdd, function(x) x + 1) - expect_true(first(newrdd) == 2) + expect_true(first(newrdd) == 2) }) test_that("count and length on RDD", { @@ -48,7 +48,7 @@ test_that("count by values and keys", { actual <- countByValue(mods) expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - + actual <- countByKey(intRdd) expected <- list(list(2L, 2L), list(1L, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -82,11 +82,11 @@ test_that("filterRDD on RDD", { filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) actual <- collect(filtered.rdd) expect_equal(actual, list(2, 4, 6, 8, 10)) - + filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) actual <- collect(filtered.rdd) expect_equal(actual, list(list(1L, -1))) - + # Filter out all elements. filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) actual <- collect(filtered.rdd) @@ -96,7 +96,7 @@ test_that("filterRDD on RDD", { test_that("lookup on RDD", { vals <- lookup(intRdd, 1L) expect_equal(vals, list(-1, 200)) - + vals <- lookup(intRdd, 3L) expect_equal(vals, list()) }) @@ -110,7 +110,7 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { }) rdd2 <- lapply(rdd2, function(x) x + x) actual <- collect(rdd2) - expected <- list(24, 24, 24, 24, 24, + expected <- list(24, 24, 24, 24, 24, 168, 170, 172, 174, 176) expect_equal(actual, expected) }) @@ -248,10 +248,10 @@ test_that("flatMapValues() on pairwise RDDs", { l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) actual <- collect(flatMapValues(l, function(x) { x })) expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) - + # Generate x to x+1 for every value actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) })) - expect_equal(actual, + expect_equal(actual, list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) }) @@ -348,7 +348,7 @@ test_that("top() on RDDs", { rdd <- parallelize(sc, l) actual <- top(rdd, 6L) expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6]) - + l <- list("e", "d", "c", "d", "a") rdd <- parallelize(sc, l) actual <- top(rdd, 3L) @@ -358,7 +358,7 @@ test_that("top() on RDDs", { test_that("fold() on RDDs", { actual <- fold(rdd, 0, "+") expect_equal(actual, Reduce("+", nums, 0)) - + rdd <- parallelize(sc, list()) actual <- fold(rdd, 0, "+") expect_equal(actual, 0) @@ -371,7 +371,7 @@ test_that("aggregateRDD() on RDDs", { combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) expect_equal(actual, list(10, 4)) - + rdd <- parallelize(sc, list()) actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) expect_equal(actual, list(0, 0)) @@ -380,13 +380,13 @@ test_that("aggregateRDD() on RDDs", { test_that("zipWithUniqueId() on RDDs", { rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) actual <- collect(zipWithUniqueId(rdd)) - expected <- list(list("a", 0), list("b", 3), list("c", 1), + expected <- list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) expect_equal(actual, expected) - + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) actual <- collect(zipWithUniqueId(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) }) @@ -394,13 +394,13 @@ test_that("zipWithUniqueId() on RDDs", { test_that("zipWithIndex() on RDDs", { rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) actual <- collect(zipWithIndex(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) - + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) actual <- collect(zipWithIndex(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) }) @@ -427,12 +427,12 @@ test_that("pipeRDD() on RDDs", { actual <- collect(pipeRDD(rdd, "more")) expected <- as.list(as.character(1:10)) expect_equal(actual, expected) - + trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) actual <- collect(pipeRDD(trailed.rdd, "sort")) expected <- list("", "1", "2", "3") expect_equal(actual, expected) - + rev.nums <- 9:0 rev.rdd <- parallelize(sc, rev.nums, 2L) actual <- collect(pipeRDD(rev.rdd, "sort")) @@ -446,11 +446,11 @@ test_that("zipRDD() on RDDs", { actual <- collect(zipRDD(rdd1, rdd2)) expect_equal(actual, list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) - + mockFile = c("Spark is pretty.", "Spark is awesome.") fileName <- tempfile(pattern="spark-test", fileext=".tmp") writeLines(mockFile, fileName) - + rdd <- textFile(sc, fileName, 1) actual <- collect(zipRDD(rdd, rdd)) expected <- lapply(mockFile, function(x) { list(x ,x) }) @@ -465,10 +465,125 @@ test_that("zipRDD() on RDDs", { actual <- collect(zipRDD(rdd, rdd1)) expected <- lapply(mockFile, function(x) { list(x, x) }) expect_equal(actual, expected) - + + unlink(fileName) +}) + +test_that("cartesian() on RDDs", { + rdd <- parallelize(sc, 1:3) + actual <- collect(cartesian(rdd, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(1, 1), list(1, 2), list(1, 3), + list(2, 1), list(2, 2), list(2, 3), + list(3, 1), list(3, 2), list(3, 3))) + + # test case where one RDD is empty + emptyRdd <- parallelize(sc, list()) + actual <- collect(cartesian(rdd, emptyRdd)) + expect_equal(actual, list()) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + actual <- collect(cartesian(rdd, rdd)) + expected <- list( + list("Spark is awesome.", "Spark is pretty."), + list("Spark is awesome.", "Spark is awesome."), + list("Spark is pretty.", "Spark is pretty."), + list("Spark is pretty.", "Spark is awesome.")) + expect_equal(sortKeyValueList(actual), expected) + + rdd1 <- parallelize(sc, 0:1) + actual <- collect(cartesian(rdd1, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(0, "Spark is pretty."), + list(0, "Spark is awesome."), + list(1, "Spark is pretty."), + list(1, "Spark is awesome."))) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collect(cartesian(rdd, rdd1)) + expect_equal(sortKeyValueList(actual), expected) + unlink(fileName) }) +test_that("subtract() on RDDs", { + l <- list(1, 1, 2, 2, 3, 4) + rdd1 <- parallelize(sc, l) + + # subtract by itself + actual <- collect(subtract(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtract by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + l) + + rdd2 <- parallelize(sc, list(2, 4)) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + list(1, 1, 3)) + + l <- list("a", "a", "b", "b", "c", "d") + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list("b", "d")) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="character"))), + list("a", "a", "c")) +}) + +test_that("subtractByKey() on pairwise RDDs", { + l <- list(list("a", 1), list("b", 4), + list("b", 5), list("a", 2)) + rdd1 <- parallelize(sc, l) + + # subtractByKey by itself + actual <- collect(subtractByKey(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtractByKey by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(l)) + + rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list("b", 4), list("b", 5))) + + l <- list(list(1, 1), list(2, 4), + list(2, 5), list(1, 2)) + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list(list(1, 3), list(3, 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list(2, 4), list(2, 5))) +}) + +test_that("intersection() on RDDs", { + # intersection with self + actual <- collect(intersection(rdd, rdd)) + expect_equal(sort(as.integer(actual)), nums) + + # intersection with an empty RDD + emptyRdd <- parallelize(sc, list()) + actual <- collect(intersection(rdd, emptyRdd)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) + rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) + actual <- collect(intersection(rdd1, rdd2)) + expect_equal(sort(as.integer(actual)), 1:3) +}) + test_that("join() on pairwise RDDs", { rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) @@ -596,9 +711,9 @@ test_that("sortByKey() on pairwise RDDs", { sortedRdd3 <- sortByKey(rdd3) actual <- collect(sortedRdd3) expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) - + # test on the boundary cases - + # boundary case 1: the RDD to be sorted has only 1 partition rdd4 <- parallelize(sc, l, 1L) sortedRdd4 <- sortByKey(rdd4) @@ -623,7 +738,7 @@ test_that("sortByKey() on pairwise RDDs", { rdd7 <- parallelize(sc, l3, 2L) sortedRdd7 <- sortByKey(rdd7) actual <- collect(sortedRdd7) - expect_equal(actual, l3) + expect_equal(actual, l3) }) test_that("collectAsMap() on a pairwise RDD", { @@ -634,12 +749,36 @@ test_that("collectAsMap() on a pairwise RDD", { rdd <- parallelize(sc, list(list("a", 1), list("b", 2))) vals <- collectAsMap(rdd) expect_equal(vals, list(a = 1, b = 2)) - + rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4))) vals <- collectAsMap(rdd) expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4)) - + rdd <- parallelize(sc, list(list(1, "a"), list(2, "b"))) vals <- collectAsMap(rdd) expect_equal(vals, list(`1` = "a", `2` = "b")) }) + +test_that("sampleByKey() on pairwise RDDs", { + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list("a", x) else list("b", x) }) + fractions <- list(a = 0.2, b = 0.1) + sample <- sampleByKey(pairsRDD, FALSE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")), TRUE) + expect_equal(50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")), TRUE) + expect_equal(lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0, TRUE) + expect_equal(lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000, TRUE) + expect_equal(lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0, TRUE) + expect_equal(lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000, TRUE) + + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list(2, x) else list(3, x) }) + fractions <- list(`2` = 0.2, `3` = 0.1) + sample <- sampleByKey(pairsRDD, TRUE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, 2)) && 300 > length(lookup(sample, 2)), TRUE) + expect_equal(50 < length(lookup(sample, 3)) && 150 > length(lookup(sample, 3)), TRUE) + expect_equal(lookup(sample, 2)[which.min(lookup(sample, 2))] >= 0, TRUE) + expect_equal(lookup(sample, 2)[which.max(lookup(sample, 2))] <= 2000, TRUE) + expect_equal(lookup(sample, 3)[which.min(lookup(sample, 3))] >= 0, TRUE) + expect_equal(lookup(sample, 3)[which.max(lookup(sample, 3))] <= 2000, TRUE) +}) diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/inst/tests/test_shuffle.R index d1da8232aea81..d7dedda553c56 100644 --- a/R/pkg/inst/tests/test_shuffle.R +++ b/R/pkg/inst/tests/test_shuffle.R @@ -87,6 +87,18 @@ test_that("combineByKey for doubles", { expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) }) +test_that("combineByKey for characters", { + stringKeyRDD <- parallelize(sc, + list(list("max", 1L), list("min", 2L), + list("other", 3L), list("max", 4L)), 2L) + reduced <- combineByKey(stringKeyRDD, + function(x) { x }, "+", "+", 2L) + actual <- collect(reduced) + + expected <- list(list("max", 5L), list("min", 2L), list("other", 3L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + test_that("aggregateByKey", { # test aggregateByKey for int keys rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index cf5cf6d1692af..25831ae2d9e18 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -44,9 +44,8 @@ test_that("infer types", { expect_equal(infer_type(list(1L, 2L)), list(type = 'array', elementType = "integer", containsNull = TRUE)) expect_equal(infer_type(list(a = 1L, b = "2")), - list(type = "struct", - fields = list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)))) + structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE))) e <- new.env() assign("a", 1L, envir = e) expect_equal(infer_type(e), @@ -54,6 +53,18 @@ test_that("infer types", { valueContainsNull = TRUE)) }) +test_that("structType and structField", { + testField <- structField("a", "string") + expect_true(inherits(testField, "structField")) + expect_true(testField$name() == "a") + expect_true(testField$nullable()) + + testSchema <- structType(testField, structField("b", "integer")) + expect_true(inherits(testSchema, "structType")) + expect_true(inherits(testSchema$fields()[[2]], "structField")) + expect_true(testSchema$fields()[[1]]$dataType.toString() == "StringType") +}) + test_that("create DataFrame from RDD", { rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) df <- createDataFrame(sqlCtx, rdd, list("a", "b")) @@ -66,9 +77,8 @@ test_that("create DataFrame from RDD", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - fields <- list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)) - schema <- list(type = "struct", fields = fields) + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) df <- createDataFrame(sqlCtx, rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) @@ -94,9 +104,8 @@ test_that("toDF", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - fields <- list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)) - schema <- list(type = "struct", fields = fields) + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) df <- toDF(rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) @@ -635,7 +644,7 @@ test_that("isLocal()", { expect_false(isLocal(df)) }) -test_that("unionAll(), subtract(), and intersect() on a DataFrame", { +test_that("unionAll(), except(), and intersect() on a DataFrame", { df <- jsonFile(sqlCtx, jsonPath) lines <- c("{\"name\":\"Bob\", \"age\":24}", @@ -650,10 +659,10 @@ test_that("unionAll(), subtract(), and intersect() on a DataFrame", { expect_true(count(unioned) == 6) expect_true(first(unioned)$name == "Michael") - subtracted <- sortDF(subtract(df, df2), desc(df$age)) + excepted <- sortDF(except(df, df2), desc(df$age)) expect_true(inherits(unioned, "DataFrame")) - expect_true(count(subtracted) == 2) - expect_true(first(subtracted)$name == "Justin") + expect_true(count(excepted) == 2) + expect_true(first(excepted)$name == "Justin") intersected <- sortDF(intersect(df, df2), df$age) expect_true(inherits(unioned, "DataFrame")) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index c6542928e8ddd..014bf7bd7b3fe 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -17,6 +17,23 @@ # Worker class +# Get current system time +currentTimeSecs <- function() { + as.numeric(Sys.time()) +} + +# Get elapsed time +elapsedSecs <- function() { + proc.time()[3] +} + +# Constants +specialLengths <- list(END_OF_STERAM = 0L, TIMING_DATA = -1L) + +# Timing R process boot +bootTime <- currentTimeSecs() +bootElap <- elapsedSecs() + rLibDir <- Sys.getenv("SPARKR_RLIBDIR") # Set libPaths to include SparkR package as loadNamespace needs this # TODO: Figure out if we can avoid this by not loading any objects that require @@ -37,7 +54,7 @@ serializer <- SparkR:::readString(inputCon) # Include packages as required packageNames <- unserialize(SparkR:::readRaw(inputCon)) for (pkg in packageNames) { - suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE)) + suppressPackageStartupMessages(library(as.character(pkg), character.only=TRUE)) } # read function dependencies @@ -46,6 +63,9 @@ computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) env <- environment(computeFunc) parent.env(env) <- .GlobalEnv # Attach under global environment. +# Timing init envs for computing +initElap <- elapsedSecs() + # Read and set broadcast variables numBroadcastVars <- SparkR:::readInt(inputCon) if (numBroadcastVars > 0) { @@ -56,6 +76,9 @@ if (numBroadcastVars > 0) { } } +# Timing broadcast +broadcastElap <- elapsedSecs() + # If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int # as number of partitions to create. numPartitions <- SparkR:::readInt(inputCon) @@ -73,14 +96,23 @@ if (isEmpty != 0) { } else if (deserializer == "row") { data <- SparkR:::readDeserializeRows(inputCon) } + # Timing reading input data for execution + inputElap <- elapsedSecs() + output <- computeFunc(partition, data) + # Timing computing + computeElap <- elapsedSecs() + if (serializer == "byte") { SparkR:::writeRawSerialize(outputCon, output) } else if (serializer == "row") { SparkR:::writeRowSerialize(outputCon, output) } else { - SparkR:::writeStrings(outputCon, output) + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) } + # Timing output + outputElap <- elapsedSecs() } else { if (deserializer == "byte") { # Now read as many characters as described in funcLen @@ -90,6 +122,8 @@ if (isEmpty != 0) { } else if (deserializer == "row") { data <- SparkR:::readDeserializeRows(inputCon) } + # Timing reading input data for execution + inputElap <- elapsedSecs() res <- new.env() @@ -107,6 +141,8 @@ if (isEmpty != 0) { res[[bucket]] <- acc } invisible(lapply(data, hashTupleToEnvir)) + # Timing computing + computeElap <- elapsedSecs() # Step 2: write out all of the environment as key-value pairs. for (name in ls(res)) { @@ -116,13 +152,26 @@ if (isEmpty != 0) { length(res[[name]]$data) <- res[[name]]$counter SparkR:::writeRawSerialize(outputCon, res[[name]]$data) } + # Timing output + outputElap <- elapsedSecs() } +} else { + inputElap <- broadcastElap + computeElap <- broadcastElap + outputElap <- broadcastElap } +# Report timing +SparkR:::writeInt(outputCon, specialLengths$TIMING_DATA) +SparkR:::writeDouble(outputCon, bootTime) +SparkR:::writeDouble(outputCon, initElap - bootElap) # init +SparkR:::writeDouble(outputCon, broadcastElap - initElap) # broadcast +SparkR:::writeDouble(outputCon, inputElap - broadcastElap) # input +SparkR:::writeDouble(outputCon, computeElap - inputElap) # compute +SparkR:::writeDouble(outputCon, outputElap - computeElap) # output + # End of output -if (serializer %in% c("byte", "row")) { - SparkR:::writeInt(outputCon, 0L) -} +SparkR:::writeInt(outputCon, specialLengths$END_OF_STERAM) close(outputCon) close(inputCon) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 5fa4d483b8342..6fea5e1144f2f 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -42,10 +42,15 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( rLibDir: String, broadcastVars: Array[Broadcast[Object]]) extends RDD[U](parent) with Logging { + protected var dataStream: DataInputStream = _ + private var bootTime: Double = _ override def getPartitions: Array[Partition] = parent.partitions override def compute(partition: Partition, context: TaskContext): Iterator[U] = { + // Timing start + bootTime = System.currentTimeMillis / 1000.0 + // The parent may be also an RRDD, so we should launch it first. val parentIterator = firstParent[T].iterator(partition, context) @@ -69,7 +74,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // the socket used to receive the output of task val outSocket = serverSocket.accept() val inputStream = new BufferedInputStream(outSocket.getInputStream) - val dataStream = openDataStream(inputStream) + dataStream = new DataInputStream(inputStream) serverSocket.close() try { @@ -155,6 +160,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( } else if (deserializer == SerializationFormats.ROW) { dataOut.write(elem.asInstanceOf[Array[Byte]]) } else if (deserializer == SerializationFormats.STRING) { + // write string(for StringRRDD) printOut.println(elem) } } @@ -180,9 +186,41 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( }.start() } - protected def openDataStream(input: InputStream): Closeable + protected def readData(length: Int): U - protected def read(): U + protected def read(): U = { + try { + val length = dataStream.readInt() + + length match { + case SpecialLengths.TIMING_DATA => + // Timing data from R worker + val boot = dataStream.readDouble - bootTime + val init = dataStream.readDouble + val broadcast = dataStream.readDouble + val input = dataStream.readDouble + val compute = dataStream.readDouble + val output = dataStream.readDouble + logInfo( + ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + + "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + + "total = %.3f s").format( + boot, + init, + broadcast, + input, + compute, + output, + boot + init + broadcast + input + compute + output)) + read() + case length if length >= 0 => + readData(length) + } + } catch { + case eof: EOFException => + throw new SparkException("R worker exited unexpectedly (cranshed)", eof) + } + } } /** @@ -202,31 +240,16 @@ private class PairwiseRRDD[T: ClassTag]( SerializationFormats.BYTE, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: DataInputStream = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new DataInputStream(input) - dataStream - } - - override protected def read(): (Int, Array[Byte]) = { - try { - val length = dataStream.readInt() - - length match { - case length if length == 2 => - val hashedKey = dataStream.readInt() - val contentPairsLength = dataStream.readInt() - val contentPairs = new Array[Byte](contentPairsLength) - dataStream.readFully(contentPairs) - (hashedKey, contentPairs) - case _ => null // End of input - } - } catch { - case eof: EOFException => { - throw new SparkException("R worker exited unexpectedly (crashed)", eof) - } - } + override protected def readData(length: Int): (Int, Array[Byte]) = { + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null + } } lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this) @@ -247,28 +270,13 @@ private class RRDD[T: ClassTag]( parent, -1, func, deserializer, serializer, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: DataInputStream = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new DataInputStream(input) - dataStream - } - - override protected def read(): Array[Byte] = { - try { - val length = dataStream.readInt() - - length match { - case length if length > 0 => - val obj = new Array[Byte](length) - dataStream.readFully(obj, 0, length) - obj - case _ => null - } - } catch { - case eof: EOFException => { - throw new SparkException("R worker exited unexpectedly (crashed)", eof) - } + override protected def readData(length: Int): Array[Byte] = { + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj) + obj + case _ => null } } @@ -289,26 +297,21 @@ private class StringRRDD[T: ClassTag]( parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: BufferedReader = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new BufferedReader(new InputStreamReader(input)) - dataStream - } - - override protected def read(): String = { - try { - dataStream.readLine() - } catch { - case e: IOException => { - throw new SparkException("R worker exited unexpectedly (crashed)", e) - } + override protected def readData(length: Int): String = { + length match { + case length if length > 0 => + SerDe.readStringBytes(dataStream, length) + case _ => null } } lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) } +private object SpecialLengths { + val TIMING_DATA = -1 +} + private[r] class BufferedStreamThread( in: InputStream, name: String, diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index ccb2a371f4e48..371dfe454d1a2 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -85,13 +85,17 @@ private[spark] object SerDe { in.readDouble() } + def readStringBytes(in: DataInputStream, len: Int): String = { + val bytes = new Array[Byte](len) + in.readFully(bytes) + assert(bytes(len - 1) == 0) + val str = new String(bytes.dropRight(1), "UTF-8") + str + } + def readString(in: DataInputStream): String = { val len = in.readInt() - val asciiBytes = new Array[Byte](len) - in.readFully(asciiBytes) - assert(asciiBytes(len - 1) == 0) - val str = new String(asciiBytes.dropRight(1).map(_.toChar)) - str + readStringBytes(in, len) } def readBoolean(in: DataInputStream): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index d1ea7cc3e9162..ae77f72998a22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -23,7 +23,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode} private[r] object SQLUtils { @@ -39,8 +39,34 @@ private[r] object SQLUtils { arr.toSeq } - def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = { - val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] + def createStructType(fields : Seq[StructField]): StructType = { + StructType(fields) + } + + def getSQLDataType(dataType: String): DataType = { + dataType match { + case "byte" => org.apache.spark.sql.types.ByteType + case "integer" => org.apache.spark.sql.types.IntegerType + case "double" => org.apache.spark.sql.types.DoubleType + case "numeric" => org.apache.spark.sql.types.DoubleType + case "character" => org.apache.spark.sql.types.StringType + case "string" => org.apache.spark.sql.types.StringType + case "binary" => org.apache.spark.sql.types.BinaryType + case "raw" => org.apache.spark.sql.types.BinaryType + case "logical" => org.apache.spark.sql.types.BooleanType + case "boolean" => org.apache.spark.sql.types.BooleanType + case "timestamp" => org.apache.spark.sql.types.TimestampType + case "date" => org.apache.spark.sql.types.DateType + case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + } + } + + def createStructField(name: String, dataType: String, nullable: Boolean): StructField = { + val dtObj = getSQLDataType(dataType) + StructField(name, dtObj, nullable) + } + + def createDF(rdd: RDD[Array[Byte]], schema: StructType, sqlContext: SQLContext): DataFrame = { val num = schema.fields.size val rowRDD = rdd.map(bytesToRow) sqlContext.createDataFrame(rowRDD, schema) From d305e686b3d73213784bd75cdad7d168b22a1dc4 Mon Sep 17 00:00:00 2001 From: Olivier Girardot Date: Fri, 17 Apr 2015 16:23:10 -0500 Subject: [PATCH 068/144] SPARK-6988 : Fix documentation regarding DataFrames using the Java API This patch includes : * adding how to use map after an sql query using javaRDD * fixing the first few java examples that were written in Scala Thank you for your time, Olivier. Author: Olivier Girardot Closes #5564 from ogirardot/branch-1.3 and squashes the following commits: 9f8d60e [Olivier Girardot] SPARK-6988 : Fix documentation regarding DataFrames using the Java API (cherry picked from commit 6b528dc139da594ef2e651d84bd91fe0f738a39d) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 03500867df70f..d49233714a0bb 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -193,8 +193,8 @@ df.groupBy("age").count().show()
    {% highlight java %} -val sc: JavaSparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +JavaSparkContext sc // An existing SparkContext. +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc) // Create the DataFrame DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); @@ -308,8 +308,8 @@ val df = sqlContext.sql("SELECT * FROM table")
    {% highlight java %} -val sqlContext = ... // An existing SQLContext -val df = sqlContext.sql("SELECT * FROM table") +SQLContext sqlContext = ... // An existing SQLContext +DataFrame df = sqlContext.sql("SELECT * FROM table") {% endhighlight %}
    @@ -435,7 +435,7 @@ DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AN // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. -List teenagerNames = teenagers.map(new Function() { +List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } @@ -599,7 +599,7 @@ DataFrame results = sqlContext.sql("SELECT name FROM people"); // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. -List names = results.map(new Function() { +List names = results.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } @@ -860,7 +860,7 @@ DataFrame parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); -List teenagerNames = teenagers.map(new Function() { +List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } From a452c59210cf2c8ff8601cdb11401eea6dc14973 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 17 Apr 2015 16:30:13 -0500 Subject: [PATCH 069/144] Minor fix to SPARK-6958: Improve Python docstring for DataFrame.sort. As a follow up PR to #5544. cc davies Author: Reynold Xin Closes #5558 from rxin/sort-doc-improvement and squashes the following commits: f4c276f [Reynold Xin] Review feedback. d2dcf24 [Reynold Xin] Minor fix to SPARK-6958: Improve Python docstring for DataFrame.sort. --- python/pyspark/sql/dataframe.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 326d22e72f104..d70c5b0a6930c 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -489,8 +489,9 @@ def sort(self, *cols, **kwargs): """Returns a new :class:`DataFrame` sorted by the specified column(s). :param cols: list of :class:`Column` or column names to sort by. - :param ascending: sort by ascending order or not, could be bool, int - or list of bool, int (default: True). + :param ascending: boolean or list of boolean (default True). + Sort ascending vs. descending. Specify list for multiple sort orders. + If a list is specified, length of the list must equal length of the `cols`. >>> df.sort(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] @@ -519,7 +520,7 @@ def sort(self, *cols, **kwargs): jcols = [jc if asc else jc.desc() for asc, jc in zip(ascending, jcols)] else: - raise TypeError("ascending can only be bool or list, but got %s" % type(ascending)) + raise TypeError("ascending can only be boolean or list, but got %s" % type(ascending)) jdf = self._jdf.sort(self._jseq(jcols)) return DataFrame(jdf, self.sql_ctx) From c5ed510135aee3a1a0402057b3b5229892aa6f3a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 17 Apr 2015 18:28:42 -0700 Subject: [PATCH 070/144] [SPARK-6703][Core] Provide a way to discover existing SparkContext's I've added a static getOrCreate method to the static SparkContext object that allows one to either retrieve a previously created SparkContext or to instantiate a new one with the provided config. The method accepts an optional SparkConf to make usage intuitive. Still working on a test for this, basically want to create a new context from scratch, then ensure that subsequent calls don't overwrite that. Author: Ilya Ganelin Closes #5501 from ilganeli/SPARK-6703 and squashes the following commits: db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation --- .../scala/org/apache/spark/SparkContext.scala | 49 ++++++++++++++++--- .../org/apache/spark/SparkContextSuite.scala | 20 ++++++++ project/MimaExcludes.scala | 4 ++ 3 files changed, 66 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e106c5c4bef60..86269eac52db0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -23,7 +23,7 @@ import java.io._ import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Properties, UUID} -import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean, AtomicInteger} import java.util.UUID.randomUUID import scala.collection.{Map, Set} @@ -1887,11 +1887,12 @@ object SparkContext extends Logging { private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object() /** - * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`. + * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. */ - private var activeContext: Option[SparkContext] = None + private val activeContext: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null) /** * Points to a partially-constructed SparkContext if some thread is in the SparkContext @@ -1926,7 +1927,8 @@ object SparkContext extends Logging { logWarning(warnMsg) } - activeContext.foreach { ctx => + if (activeContext.get() != null) { + val ctx = activeContext.get() val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + " To ignore this error, set spark.driver.allowMultipleContexts = true. " + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" @@ -1941,6 +1943,39 @@ object SparkContext extends Logging { } } + /** + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. + */ + def getOrCreate(config: SparkConf): SparkContext = { + // Synchronize to ensure that multiple create requests don't trigger an exception + // from assertNoOtherContextIsRunning within setActiveContext + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + if (activeContext.get() == null) { + setActiveContext(new SparkContext(config), allowMultipleContexts = false) + } + activeContext.get() + } + } + + /** + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. + * + * This method allows not passing a SparkConf (useful if just retrieving). + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. + */ + def getOrCreate(): SparkContext = { + getOrCreate(new SparkConf()) + } + /** * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is * running. Throws an exception if a running context is detected and logs a warning if another @@ -1967,7 +2002,7 @@ object SparkContext extends Logging { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { assertNoOtherContextIsRunning(sc, allowMultipleContexts) contextBeingConstructed = None - activeContext = Some(sc) + activeContext.set(sc) } } @@ -1978,7 +2013,7 @@ object SparkContext extends Logging { */ private[spark] def clearActiveContext(): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - activeContext = None + activeContext.set(null) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 94be1c6d6397c..728558a424780 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -67,6 +67,26 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { } } + test("Test getOrCreate") { + var sc2: SparkContext = null + SparkContext.clearActiveContext() + val conf = new SparkConf().setAppName("test").setMaster("local") + + sc = SparkContext.getOrCreate(conf) + + assert(sc.getConf.get("spark.app.name").equals("test")) + sc2 = SparkContext.getOrCreate(new SparkConf().setAppName("test2").setMaster("local")) + assert(sc2.getConf.get("spark.app.name").equals("test")) + assert(sc === sc2) + assert(sc eq sc2) + + // Try creating second context to confirm that it's still possible, if desired + sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true")) + + sc2.stop() + } + test("BytesWritable implicit conversion is correct") { // Regression test for SPARK-3121 val bytesWritable = new BytesWritable() diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1564babefa62f..7ef363a2f07ad 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -68,6 +68,10 @@ object MimaExcludes { // SPARK-6693 add tostring with max lines and width for matrix ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.toString") + )++ Seq( + // SPARK-6703 Add getOrCreate method to SparkContext + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.SparkContext.org$apache$spark$SparkContext$$activeContext") ) case v if v.startsWith("1.3") => From 6fbeb82e13db7117d8f216e6148632490a4bc5be Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Fri, 17 Apr 2015 18:30:55 -0700 Subject: [PATCH 071/144] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Defined executorCores from "spark.mesos.executor.cores" - Changed the amount of mesosExecutor's cores to executorCores. - Added new configuration option on running-on-mesos.md Author: Jongyoul Lee Closes #5063 from jongyoul/SPARK-6350 and squashes the following commits: 9238d6e [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs - Changed configuration name - Made mesosExecutorCores private 2d41241 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 89edb4f [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 8ba7694 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 7549314 [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed docs 4ae7b0c [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Removed TODO c27efce [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed Mesos*Suite for supporting integer WorkerOffers - Fixed Documentation 1fe4c03 [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Change available resources of cpus to integer value beacuse WorkerOffer support the amount cpus as integer value 5f3767e [Jongyoul Lee] Revert "[SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode" 4b7c69e [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Changed configruation name and description from "spark.mesos.executor.cores" to "spark.executor.frameworkCores" 0556792 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Defined executorCores from "spark.mesos.executor.cores" - Changed the amount of mesosExecutor's cores to executorCores. - Added new configuration option on running-on-mesos.md --- .../cluster/mesos/MesosSchedulerBackend.scala | 14 +++++++------- .../cluster/mesos/MesosSchedulerBackendSuite.scala | 4 ++-- docs/running-on-mesos.md | 10 ++++++++++ 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index b381436839227..d9d62b0e287ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -67,6 +67,8 @@ private[spark] class MesosSchedulerBackend( // The listener bus to publish executor added/removed events. val listenerBus = sc.listenerBus + + private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1) @volatile var appId: String = _ @@ -139,7 +141,7 @@ private[spark] class MesosSchedulerBackend( .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder() - .setValue(scheduler.CPUS_PER_TASK).build()) + .setValue(mesosExecutorCores).build()) .build() val memory = Resource.newBuilder() .setName("mem") @@ -220,10 +222,9 @@ private[spark] class MesosSchedulerBackend( val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task - cpus >= 2 * scheduler.CPUS_PER_TASK) || + cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK)) || (slaveIdsWithExecutors.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK) } @@ -232,10 +233,9 @@ private[spark] class MesosSchedulerBackend( val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { - // If the executor doesn't exist yet, subtract CPU for executor - // TODO(pwendell): Should below just subtract "1"? - getResource(o.getResourcesList, "cpus").toInt - - scheduler.CPUS_PER_TASK + // If the Mesos executor has not been started on this slave yet, set aside a few + // cores for the Mesos executor by offering fewer cores to the Spark executor + (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt } new WorkerOffer( o.getSlaveId.getValue, diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index a311512e82c5e..cdd7be0fbe5dd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -118,12 +118,12 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(0).getSlaveId.getValue, mesosOffers.get(0).getHostname, - 2 + (minCpu - backend.mesosExecutorCores).toInt )) expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(2).getSlaveId.getValue, mesosOffers.get(2).getHostname, - 2 + (minCpu - backend.mesosExecutorCores).toInt )) val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index c984639bd34cf..594bf78b67713 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -210,6 +210,16 @@ See the [configuration page](configuration.html) for information on Spark config Note that total amount of cores the executor will request in total will not exceed the spark.cores.max setting. +
    + + + + From 1991337336596f94698e79c2366f065c374128ab Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Apr 2015 19:02:07 -0700 Subject: [PATCH 072/144] [SPARK-5933] [core] Move config deprecation warnings to SparkConf. I didn't find many deprecated configs after a grep-based search, but the ones I could find were moved to the centralized location in SparkConf. While there, I deprecated a couple more HS configs that mentioned time units. Author: Marcelo Vanzin Closes #5562 from vanzin/SPARK-5933 and squashes the following commits: dcb617e7 [Marcelo Vanzin] [SPARK-5933] [core] Move config deprecation warnings to SparkConf. --- .../main/scala/org/apache/spark/SparkConf.scala | 17 ++++++++++++++--- .../main/scala/org/apache/spark/SparkEnv.scala | 10 ++-------- .../deploy/history/FsHistoryProvider.scala | 15 +++------------ .../scala/org/apache/spark/SparkConfSuite.scala | 3 +++ docs/monitoring.md | 15 +++++++-------- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +-------- 6 files changed, 30 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b0186e9a007b8..e3a649d755450 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -403,6 +403,9 @@ private[spark] object SparkConf extends Logging { */ private val deprecatedConfigs: Map[String, DeprecatedConfig] = { val configs = Seq( + DeprecatedConfig("spark.cache.class", "0.8", + "The spark.cache.class property is no longer being used! Specify storage levels using " + + "the RDD.persist() method instead."), DeprecatedConfig("spark.yarn.user.classpath.first", "1.3", "Please use spark.{driver,executor}.userClassPathFirst instead.")) Map(configs.map { cfg => (cfg.key -> cfg) }:_*) @@ -420,7 +423,15 @@ private[spark] object SparkConf extends Logging { "spark.history.fs.update.interval" -> Seq( AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), AlternateConfig("spark.history.fs.updateInterval", "1.3"), - AlternateConfig("spark.history.updateInterval", "1.3")) + AlternateConfig("spark.history.updateInterval", "1.3")), + "spark.history.fs.cleaner.interval" -> Seq( + AlternateConfig("spark.history.fs.cleaner.interval.seconds", "1.4")), + "spark.history.fs.cleaner.maxAge" -> Seq( + AlternateConfig("spark.history.fs.cleaner.maxAge.seconds", "1.4")), + "spark.yarn.am.waitTime" -> Seq( + AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3", + // Translate old value to a duration, with 10s wait time per try. + translation = s => s"${s.toLong * 10}s")) ) /** @@ -470,7 +481,7 @@ private[spark] object SparkConf extends Logging { configsWithAlternatives.get(key).flatMap { alts => alts.collectFirst { case alt if conf.contains(alt.key) => val value = conf.get(alt.key) - alt.translation.map(_(value)).getOrElse(value) + if (alt.translation != null) alt.translation(value) else value } } } @@ -514,6 +525,6 @@ private[spark] object SparkConf extends Logging { private case class AlternateConfig( key: String, version: String, - translation: Option[String => String] = None) + translation: String => String = null) } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 0171488e09562..959aefabd8de4 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -103,7 +103,7 @@ class SparkEnv ( // actorSystem.awaitTermination() // Note that blockTransferService is stopped by BlockManager since it is started by it. - + // If we only stop sc, but the driver process still run as a services then we need to delete // the tmp dir, if not, it will create too many tmp dirs. // We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the @@ -375,12 +375,6 @@ object SparkEnv extends Logging { "." } - // Warn about deprecated spark.cache.class property - if (conf.contains("spark.cache.class")) { - logWarning("The spark.cache.class property is no longer being used! Specify storage " + - "levels using the RDD.persist() method instead.") - } - val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { new OutputCommitCoordinator(conf) } @@ -406,7 +400,7 @@ object SparkEnv extends Logging { shuffleMemoryManager, outputCommitCoordinator, conf) - + // Add a reference to tmp dir created by driver, we will delete this tmp dir when stop() is // called, and we only need to do it for driver. Because driver may run as a service, and if we // don't delete this tmp dir when sc is stopped, then will create too many tmp dirs. 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 985545742df67..47bdd7749ec3d 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 @@ -52,8 +52,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private val UPDATE_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.update.interval", "10s") // Interval between each cleaner checks for event logs to delete - private val CLEAN_INTERVAL_MS = conf.getLong("spark.history.fs.cleaner.interval.seconds", - DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S) * 1000 + private val CLEAN_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.cleaner.interval", "1d") private val logDir = conf.getOption("spark.history.fs.logDirectory") .map { d => Utils.resolveURI(d).toString } @@ -130,8 +129,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) { // A task that periodically cleans event logs on disk. - pool.scheduleAtFixedRate(getRunner(cleanLogs), 0, CLEAN_INTERVAL_MS, - TimeUnit.MILLISECONDS) + pool.scheduleAtFixedRate(getRunner(cleanLogs), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS) } } } @@ -270,8 +268,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis try { val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) .getOrElse(Seq[FileStatus]()) - val maxAge = conf.getLong("spark.history.fs.cleaner.maxAge.seconds", - DEFAULT_SPARK_HISTORY_FS_MAXAGE_S) * 1000 + val maxAge = conf.getTimeAsSeconds("spark.history.fs.cleaner.maxAge", "7d") * 1000 val now = System.currentTimeMillis() val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() @@ -417,12 +414,6 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private object FsHistoryProvider { val DEFAULT_LOG_DIR = "file:/tmp/spark-events" - - // One day - val DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S = Duration(1, TimeUnit.DAYS).toSeconds - - // One week - val DEFAULT_SPARK_HISTORY_FS_MAXAGE_S = Duration(7, TimeUnit.DAYS).toSeconds } private class FsApplicationHistoryInfo( diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 7d87ba5fd2610..8e6c200c4ba00 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -217,6 +217,9 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro val count = conf.getAll.filter { case (k, v) => k.startsWith("spark.history.") }.size assert(count === 4) + + conf.set("spark.yarn.applicationMaster.waitTries", "42") + assert(conf.getTimeAsSeconds("spark.yarn.am.waitTime") === 420) } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 2a130224591ca..8a85928d6d44d 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -153,19 +153,18 @@ follows: - - + + - - + +
    @@ -223,12 +225,14 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { (driver.state == DriverState.RUNNING || driver.state == DriverState.SUBMITTED || driver.state == DriverState.RELAUNCHING)) { - val killLinkUri = s"driver/kill?id=${driver.id}&terminate=true" - val confirm = "return window.confirm(" + - s"'Are you sure you want to kill driver ${driver.id} ?');" - - (kill) - + val confirm = + s"if (window.confirm('Are you sure you want to kill driver ${driver.id} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" +
    + + + (kill) +
    }
    {driver.id} {killLink}
    spark.history.fs.update.interval.seconds10spark.history.fs.update.interval10s - The period, in seconds, at which information displayed by this history server is updated. + The period at which information displayed by this history server is updated. Each update checks for any changes made to the event logs in persisted storage.
    spark.mesos.mesosExecutor.cores1.0 + (Fine-grained mode only) Number of cores to give each Mesos executor. This does not + include the cores used to run the Spark tasks. In other words, even if no Spark task + is being run, each Mesos executor will occupy the number of cores configured here. + The value can be a floating point number. +
    spark.mesos.executor.home driver side SPARK_HOME
    spark.history.fs.cleaner.interval.seconds86400spark.history.fs.cleaner.interval1d - How often the job history cleaner checks for files to delete, in seconds. Defaults to 86400 (one day). - Files are only deleted if they are older than spark.history.fs.cleaner.maxAge.seconds. + How often the job history cleaner checks for files to delete. + Files are only deleted if they are older than spark.history.fs.cleaner.maxAge.
    spark.history.fs.cleaner.maxAge.seconds3600 * 24 * 7spark.history.fs.cleaner.maxAge7d - Job history files older than this many seconds will be deleted when the history cleaner runs. - Defaults to 3600 * 24 * 7 (1 week). + Job history files older than this will be deleted when the history cleaner runs.
    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 c357b7ae9d4da..f7a84207e9da6 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 @@ -373,14 +373,7 @@ private[spark] class ApplicationMaster( private def waitForSparkContextInitialized(): SparkContext = { logInfo("Waiting for spark context initialization") 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.getTimeAsMs("spark.yarn.am.waitTime", - s"${waitTries.getOrElse(100000L)}ms") + val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") val deadline = System.currentTimeMillis() + totalWaitTime while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { From d850b4bd3a294dd245881e03f7f94bf970a7ee79 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 17 Apr 2015 19:17:06 -0700 Subject: [PATCH 073/144] [SPARK-6975][Yarn] Fix argument validation error `numExecutors` checking is failed when dynamic allocation is enabled with default configuration. Details can be seen is [SPARK-6975](https://issues.apache.org/jira/browse/SPARK-6975). sryza, please help me to review this, not sure is this the correct way, I think previous you change this part :) Author: jerryshao Closes #5551 from jerryshao/SPARK-6975 and squashes the following commits: 4335da1 [jerryshao] Change according to the comments 77bdcbd [jerryshao] Fix argument validation error --- .../org/apache/spark/deploy/yarn/ClientArguments.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 da6798cb1b279..1423533470fc0 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 @@ -103,9 +103,13 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) * This is intended to be called only after the provided arguments have been parsed. */ private def validateArgs(): Unit = { - if (numExecutors <= 0) { + if (numExecutors < 0 || (!isDynamicAllocationEnabled && numExecutors == 0)) { throw new IllegalArgumentException( - "You must specify at least 1 executor!\n" + getUsageMessage()) + s""" + |Number of executors was $numExecutors, but must be at least 1 + |(or 0 if dynamic executor allocation is enabled). + |${getUsageMessage()} + """.stripMargin) } if (executorCores < sparkConf.getInt("spark.task.cpus", 1)) { throw new SparkException("Executor cores must not be less than " + From 5f095d56054d57c54d81db1d36cd46312810fb6a Mon Sep 17 00:00:00 2001 From: Olivier Girardot Date: Sat, 18 Apr 2015 00:31:01 -0700 Subject: [PATCH 074/144] SPARK-6992 : Fix documentation example for Spark SQL on StructType This patch is fixing the Java examples for Spark SQL when defining programmatically a Schema and mapping Rows. Author: Olivier Girardot Closes #5569 from ogirardot/branch-1.3 and squashes the following commits: c29e58d [Olivier Girardot] SPARK-6992 : Fix documentation example for Spark SQL on StructType (cherry picked from commit c9b1ba4b16a7afe93d45bf75b128cc0dd287ded0) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d49233714a0bb..b2022546268a7 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -555,13 +555,16 @@ by `SQLContext`. For example: {% highlight java %} -// Import factory methods provided by DataType. -import org.apache.spark.sql.types.DataType; +import org.apache.spark.api.java.function.Function; +// Import factory methods provided by DataTypes. +import org.apache.spark.sql.types.DataTypes; // Import StructType and StructField import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.StructField; // Import Row. import org.apache.spark.sql.Row; +// Import RowFactory. +import org.apache.spark.sql.RowFactory; // sc is an existing JavaSparkContext. SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); @@ -575,16 +578,16 @@ String schemaString = "name age"; // Generate the schema based on the string of schema List fields = new ArrayList(); for (String fieldName: schemaString.split(" ")) { - fields.add(DataType.createStructField(fieldName, DataType.StringType, true)); + fields.add(DataTypes.createStructField(fieldName, DataTypes.StringType, true)); } -StructType schema = DataType.createStructType(fields); +StructType schema = DataTypes.createStructType(fields); // Convert records of the RDD (people) to Rows. JavaRDD rowRDD = people.map( new Function() { public Row call(String record) throws Exception { String[] fields = record.split(","); - return Row.create(fields[0], fields[1].trim()); + return RowFactory.create(fields[0], fields[1].trim()); } }); From 327ebf0cb5e236579bece057eda27b21aed0e2dc Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sat, 18 Apr 2015 10:14:56 +0100 Subject: [PATCH 075/144] [core] [minor] Make sure ConnectionManager stops. My previous fix (force a selector wakeup) didn't seem to work since I ran into the hang again. So change the code a bit to be more explicit about the condition when the selector thread should exit. Author: Marcelo Vanzin Closes #5566 from vanzin/conn-mgr-hang and squashes the following commits: ddb2c03 [Marcelo Vanzin] [core] [minor] Make sure ConnectionManager stops. --- .../spark/network/nio/ConnectionManager.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 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 5a74c13b38bf7..1a68e621eaee7 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 @@ -188,6 +188,7 @@ private[nio] class ConnectionManager( private val writeRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() private val readRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() + @volatile private var isActive = true private val selectorThread = new Thread("connection-manager-thread") { override def run(): Unit = ConnectionManager.this.run() } @@ -342,7 +343,7 @@ private[nio] class ConnectionManager( def run() { try { - while(!selectorThread.isInterrupted) { + while (isActive) { while (!registerRequests.isEmpty) { val conn: SendingConnection = registerRequests.dequeue() addListeners(conn) @@ -398,7 +399,7 @@ private[nio] class ConnectionManager( } catch { // Explicitly only dealing with CancelledKeyException here since other exceptions // should be dealt with differently. - case e: CancelledKeyException => { + case e: CancelledKeyException => // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() @@ -420,8 +421,11 @@ private[nio] class ConnectionManager( } } } - } - 0 + 0 + + case e: ClosedSelectorException => + logDebug("Failed select() as selector is closed.", e) + return } if (selectedKeysCount == 0) { @@ -988,11 +992,11 @@ private[nio] class ConnectionManager( } def stop() { + isActive = false ackTimeoutMonitor.stop() - selector.wakeup() + selector.close() selectorThread.interrupt() selectorThread.join() - selector.close() val connections = connectionsByKey.values connections.foreach(_.close()) if (connectionsByKey.size != 0) { From 28683b4df5de06373b867068b9b8adfbcaf93176 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 18 Apr 2015 16:46:28 -0700 Subject: [PATCH 076/144] [SPARK-6219] Reuse pep8.py Per the discussion in the comments on [this commit](https://github.com/apache/spark/commit/f17d43b033d928dbc46aef8e367aa08902e698ad#commitcomment-10780649), this PR allows the Python lint script to reuse `pep8.py` when possible. Author: Nicholas Chammas Closes #5561 from nchammas/save-dem-pep8-bytes and squashes the following commits: b7c91e6 [Nicholas Chammas] reuse pep8.py --- dev/.gitignore | 1 + dev/lint-python | 21 +++++++++++---------- 2 files changed, 12 insertions(+), 10 deletions(-) create mode 100644 dev/.gitignore diff --git a/dev/.gitignore b/dev/.gitignore new file mode 100644 index 0000000000000..4a6027429e0d3 --- /dev/null +++ b/dev/.gitignore @@ -0,0 +1 @@ +pep8*.py diff --git a/dev/lint-python b/dev/lint-python index fded654893a7c..f50d149dc4d44 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -32,18 +32,19 @@ compile_status="${PIPESTATUS[0]}" #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 #+ TODOs: #+ - Download pep8 from PyPI. It's more "official". -PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" -PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.6.2/pep8.py" +PEP8_VERSION="1.6.2" +PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8-$PEP8_VERSION.py" +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/$PEP8_VERSION/pep8.py" -# if [ ! -e "$PEP8_SCRIPT_PATH" ]; then -curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" -curl_status="$?" +if [ ! -e "$PEP8_SCRIPT_PATH" ]; then + curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" + curl_status="$?" -if [ "$curl_status" -ne 0 ]; then - echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." - exit "$curl_status" + if [ "$curl_status" -ne 0 ]; then + echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." + exit "$curl_status" + fi fi -# fi # There is no need to write this output to a file #+ first, but we do so so that the check status can @@ -65,7 +66,7 @@ else echo "Python lint checks passed." fi -rm "$PEP8_SCRIPT_PATH" +# rm "$PEP8_SCRIPT_PATH" rm "$PYTHON_LINT_REPORT_PATH" exit "$lint_status" From 729885ec6b4be61144d04821f1a6e8d2134eea00 Mon Sep 17 00:00:00 2001 From: Gaurav Nanda Date: Sat, 18 Apr 2015 17:20:46 -0700 Subject: [PATCH 077/144] Fixed doc Just fixed a doc. Author: Gaurav Nanda Closes #5576 from gaurav324/master and squashes the following commits: 8a7323f [Gaurav Nanda] Fixed doc --- docs/mllib-linear-methods.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 9270741d439d9..2b2be4d9d0273 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -377,7 +377,7 @@ references. Here is an [detailed mathematical derivation](http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297). -For multiclass classification problems, the algorithm will outputs a multinomial logistic regression +For multiclass classification problems, the algorithm will output a multinomial logistic regression model, which contains $K - 1$ binary logistic regression models regressed against the first class. Given a new data points, $K - 1$ models will be run, and the class with largest probability will be chosen as the predicted class. From 8fbd45c74e762dd6b071ea58a60f5bb649f74042 Mon Sep 17 00:00:00 2001 From: Olivier Girardot Date: Sat, 18 Apr 2015 18:21:44 -0700 Subject: [PATCH 078/144] SPARK-6993 : Add default min, max methods for JavaDoubleRDD The default method will use Guava's Ordering instead of java.util.Comparator.naturalOrder() because it's not available in Java 7, only in Java 8. Author: Olivier Girardot Closes #5571 from ogirardot/master and squashes the following commits: 7fe2e9e [Olivier Girardot] SPARK-6993 : Add default min, max methods for JavaDoubleRDD --- .../org/apache/spark/api/java/JavaDoubleRDD.scala | 14 ++++++++++++++ .../test/java/org/apache/spark/JavaAPISuite.java | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 79e4ebf2db578..61af867b11b9c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -163,6 +163,20 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) /** Add up the elements in this RDD. */ def sum(): JDouble = srdd.sum() + /** + * Returns the minimum element from this RDD as defined by + * the default comparator natural order. + * @return the minimum of the RDD + */ + def min(): JDouble = min(com.google.common.collect.Ordering.natural()) + + /** + * Returns the maximum element from this RDD as defined by + * the default comparator natural order. + * @return the maximum of the RDD + */ + def max(): JDouble = max(com.google.common.collect.Ordering.natural()) + /** * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and * count of the RDD's elements in one operation. diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index d4b5bb519157c..8a4f2a08fe701 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -761,6 +761,20 @@ public void min() { Assert.assertEquals(1.0, max, 0.001); } + @Test + public void naturalMax() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(); + Assert.assertTrue(4.0 == max); + } + + @Test + public void naturalMin() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(); + Assert.assertTrue(1.0 == max); + } + @Test public void takeOrdered() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); From 0424da68d4c81dc3a9944d8485feb1233c6633c4 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sun, 19 Apr 2015 09:37:09 +0100 Subject: [PATCH 079/144] [SPARK-6963][CORE]Flaky test: o.a.s.ContextCleanerSuite automatically cleanup checkpoint cc andrewor14 Author: GuoQiang Li Closes #5548 from witgo/SPARK-6963 and squashes the following commits: 964aea7 [GuoQiang Li] review commits b08b3c9 [GuoQiang Li] Flaky test: o.a.s.ContextCleanerSuite automatically cleanup checkpoint --- .../org/apache/spark/ContextCleaner.scala | 2 ++ .../apache/spark/ContextCleanerSuite.scala | 21 +++++++++++++------ 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 715b259057569..37198d887b07b 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -236,6 +236,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { try { logDebug("Cleaning rdd checkpoint data " + rddId) RDDCheckpointData.clearRDDCheckpointData(sc, rddId) + listeners.foreach(_.checkpointCleaned(rddId)) logInfo("Cleaned rdd checkpoint data " + rddId) } catch { @@ -260,4 +261,5 @@ private[spark] trait CleanerListener { def shuffleCleaned(shuffleId: Int) def broadcastCleaned(broadcastId: Long) def accumCleaned(accId: Long) + def checkpointCleaned(rddId: Long) } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 097e7076e5391..c7868ddcf770f 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -224,7 +224,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { assert(fs.exists(path)) // the checkpoint is not cleaned by default (without the configuration set) - var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Nil) rdd = null // Make RDD out of scope runGC() postGCTester.assertCleanup() @@ -245,7 +245,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) // Test that GC causes checkpoint data cleanup after dereferencing the RDD - postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Seq(rddId)) rdd = null // Make RDD out of scope runGC() postGCTester.assertCleanup() @@ -406,12 +406,14 @@ class CleanerTester( sc: SparkContext, rddIds: Seq[Int] = Seq.empty, shuffleIds: Seq[Int] = Seq.empty, - broadcastIds: Seq[Long] = Seq.empty) + broadcastIds: Seq[Long] = Seq.empty, + checkpointIds: Seq[Long] = Seq.empty) extends Logging { val toBeCleanedRDDIds = new HashSet[Int] with SynchronizedSet[Int] ++= rddIds val toBeCleanedShuffleIds = new HashSet[Int] with SynchronizedSet[Int] ++= shuffleIds val toBeCleanedBroadcstIds = new HashSet[Long] with SynchronizedSet[Long] ++= broadcastIds + val toBeCheckpointIds = new HashSet[Long] with SynchronizedSet[Long] ++= checkpointIds val isDistributed = !sc.isLocal val cleanerListener = new CleanerListener { @@ -427,12 +429,17 @@ class CleanerTester( def broadcastCleaned(broadcastId: Long): Unit = { toBeCleanedBroadcstIds -= broadcastId - logInfo("Broadcast" + broadcastId + " cleaned") + logInfo("Broadcast " + broadcastId + " cleaned") } def accumCleaned(accId: Long): Unit = { logInfo("Cleaned accId " + accId + " cleaned") } + + def checkpointCleaned(rddId: Long): Unit = { + toBeCheckpointIds -= rddId + logInfo("checkpoint " + rddId + " cleaned") + } } val MAX_VALIDATION_ATTEMPTS = 10 @@ -456,7 +463,8 @@ class CleanerTester( /** Verify that RDDs, shuffles, etc. occupy resources */ private def preCleanupValidate() { - assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty, "Nothing to cleanup") + assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty || + checkpointIds.nonEmpty, "Nothing to cleanup") // Verify the RDDs have been persisted and blocks are present rddIds.foreach { rddId => @@ -547,7 +555,8 @@ class CleanerTester( private def isAllCleanedUp = toBeCleanedRDDIds.isEmpty && toBeCleanedShuffleIds.isEmpty && - toBeCleanedBroadcstIds.isEmpty + toBeCleanedBroadcstIds.isEmpty && + toBeCheckpointIds.isEmpty private def getRDDBlocks(rddId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { From fa73da024000386eecef79573e8ac96d6f05b2c7 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 19 Apr 2015 20:33:51 -0700 Subject: [PATCH 080/144] [SPARK-6998][MLlib] Make StreamingKMeans 'Serializable' If `StreamingKMeans` is not `Serializable`, we cannot do checkpoint for applications that using `StreamingKMeans`. So we should make it `Serializable`. Author: zsxwing Closes #5582 from zsxwing/SPARK-6998 and squashes the following commits: 67c2a14 [zsxwing] Make StreamingKMeans 'Serializable' --- .../org/apache/spark/mllib/clustering/StreamingKMeans.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f483fd1c7d2cf..d4606fda37b0d 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 @@ -165,7 +165,7 @@ class StreamingKMeansModel( class StreamingKMeans( var k: Int, var decayFactor: Double, - var timeUnit: String) extends Logging { + var timeUnit: String) extends Logging with Serializable { def this() = this(2, 1.0, StreamingKMeans.BATCHES) From d8e1b7b06c499289ff3ce5ec91ff354493a17c48 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 19 Apr 2015 20:35:43 -0700 Subject: [PATCH 081/144] [SPARK-6983][Streaming] Update ReceiverTrackerActor to use the new Rpc interface A subtask of [SPARK-5293](https://issues.apache.org/jira/browse/SPARK-5293) Author: zsxwing Closes #5557 from zsxwing/SPARK-6983 and squashes the following commits: e777e9f [zsxwing] Update ReceiverTrackerActor to use the new Rpc interface --- .../scala/org/apache/spark/rpc/RpcEnv.scala | 2 +- .../receiver/ReceiverSupervisorImpl.scala | 52 +++++---------- .../streaming/scheduler/ReceiverInfo.scala | 4 +- .../streaming/scheduler/ReceiverTracker.scala | 64 ++++++++++--------- 4 files changed, 52 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index f2c1c86af767e..cba038ca355d7 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -258,7 +258,7 @@ private[spark] trait RpcEndpoint { final def stop(): Unit = { val _self = self if (_self != null) { - rpcEnv.stop(self) + rpcEnv.stop(_self) } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 8f2f1fef76874..89af40330b9d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -21,18 +21,16 @@ import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await -import akka.actor.{ActorRef, Actor, Props} -import akka.pattern.ask import com.google.common.base.Throwables import org.apache.hadoop.conf.Configuration import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.rpc.{RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.Time import org.apache.spark.streaming.scheduler._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{RpcUtils, Utils} /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -63,37 +61,23 @@ private[streaming] class ReceiverSupervisorImpl( } - /** Remote Akka actor for the ReceiverTracker */ - private val trackerActor = { - val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.getInt("spark.driver.port", 7077) - val url = AkkaUtils.address( - AkkaUtils.protocol(env.actorSystem), - SparkEnv.driverActorSystemName, - ip, - port, - "ReceiverTracker") - env.actorSystem.actorSelection(url) - } - - /** Timeout for Akka actor messages */ - private val askTimeout = AkkaUtils.askTimeout(env.conf) + /** Remote RpcEndpointRef for the ReceiverTracker */ + private val trackerEndpoint = RpcUtils.makeDriverRef("ReceiverTracker", env.conf, env.rpcEnv) - /** Akka actor for receiving messages from the ReceiverTracker in the driver */ - private val actor = env.actorSystem.actorOf( - Props(new Actor { + /** RpcEndpointRef for receiving messages from the ReceiverTracker in the driver */ + private val endpoint = env.rpcEnv.setupEndpoint( + "Receiver-" + streamId + "-" + System.currentTimeMillis(), new ThreadSafeRpcEndpoint { + override val rpcEnv: RpcEnv = env.rpcEnv override def receive: PartialFunction[Any, Unit] = { case StopReceiver => logInfo("Received stop signal") - stop("Stopped by driver", None) + ReceiverSupervisorImpl.this.stop("Stopped by driver", None) case CleanupOldBlocks(threshTime) => logDebug("Received delete old batch signal") cleanupOldBlocks(threshTime) } - - def ref: ActorRef = self - }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) + }) /** Unique block ids if one wants to add blocks directly */ private val newBlockId = new AtomicLong(System.currentTimeMillis()) @@ -162,15 +146,14 @@ private[streaming] class ReceiverSupervisorImpl( logDebug(s"Pushed block $blockId in ${(System.currentTimeMillis - time)} ms") val blockInfo = ReceivedBlockInfo(streamId, numRecords, blockStoreResult) - val future = trackerActor.ask(AddBlock(blockInfo))(askTimeout) - Await.result(future, askTimeout) + trackerEndpoint.askWithReply[Boolean](AddBlock(blockInfo)) logDebug(s"Reported block $blockId") } /** Report error to the receiver tracker */ def reportError(message: String, error: Throwable) { val errorString = Option(error).map(Throwables.getStackTraceAsString).getOrElse("") - trackerActor ! ReportError(streamId, message, errorString) + trackerEndpoint.send(ReportError(streamId, message, errorString)) logWarning("Reported error " + message + " - " + error) } @@ -180,22 +163,19 @@ private[streaming] class ReceiverSupervisorImpl( override protected def onStop(message: String, error: Option[Throwable]) { blockGenerator.stop() - env.actorSystem.stop(actor) + env.rpcEnv.stop(endpoint) } override protected def onReceiverStart() { val msg = RegisterReceiver( - streamId, receiver.getClass.getSimpleName, Utils.localHostName(), actor) - val future = trackerActor.ask(msg)(askTimeout) - Await.result(future, askTimeout) + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), endpoint) + trackerEndpoint.askWithReply[Boolean](msg) } override protected def onReceiverStop(message: String, error: Option[Throwable]) { logInfo("Deregistering receiver " + streamId) val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") - val future = trackerActor.ask( - DeregisterReceiver(streamId, message, errorString))(askTimeout) - Await.result(future, askTimeout) + trackerEndpoint.askWithReply[Boolean](DeregisterReceiver(streamId, message, errorString)) logInfo("Stopped receiver " + streamId) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala index d7e39c528c519..52f08b9c9de68 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.streaming.scheduler -import akka.actor.ActorRef import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rpc.RpcEndpointRef /** * :: DeveloperApi :: @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi case class ReceiverInfo( streamId: Int, name: String, - private[streaming] val actor: ActorRef, + private[streaming] val endpoint: RpcEndpointRef, active: Boolean, location: String, lastErrorMessage: String = "", 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 98900473138fe..c4ead6f30a63d 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 @@ -17,13 +17,11 @@ package org.apache.spark.streaming.scheduler - import scala.collection.mutable.{HashMap, SynchronizedMap} import scala.language.existentials -import akka.actor._ - import org.apache.spark.{Logging, SerializableWritable, SparkEnv, SparkException} +import org.apache.spark.rpc._ import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{CleanupOldBlocks, Receiver, ReceiverSupervisorImpl, StopReceiver} @@ -36,7 +34,7 @@ private[streaming] case class RegisterReceiver( streamId: Int, typ: String, host: String, - receiverActor: ActorRef + receiverEndpoint: RpcEndpointRef ) extends ReceiverTrackerMessage private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) extends ReceiverTrackerMessage @@ -67,19 +65,19 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false ) private val listenerBus = ssc.scheduler.listenerBus - // actor is created when generator starts. + // endpoint is created when generator starts. // This not being null means the tracker has been started and not stopped - private var actor: ActorRef = null + private var endpoint: RpcEndpointRef = null - /** Start the actor and receiver execution thread. */ + /** Start the endpoint and receiver execution thread. */ def start(): Unit = synchronized { - if (actor != null) { + if (endpoint != null) { throw new SparkException("ReceiverTracker already started") } if (!receiverInputStreams.isEmpty) { - actor = ssc.env.actorSystem.actorOf(Props(new ReceiverTrackerActor), - "ReceiverTracker") + endpoint = ssc.env.rpcEnv.setupEndpoint( + "ReceiverTracker", new ReceiverTrackerEndpoint(ssc.env.rpcEnv)) if (!skipReceiverLaunch) receiverExecutor.start() logInfo("ReceiverTracker started") } @@ -87,13 +85,13 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Stop the receiver execution thread. */ def stop(graceful: Boolean): Unit = synchronized { - if (!receiverInputStreams.isEmpty && actor != null) { + if (!receiverInputStreams.isEmpty && endpoint != null) { // First, stop the receivers if (!skipReceiverLaunch) receiverExecutor.stop(graceful) - // Finally, stop the actor - ssc.env.actorSystem.stop(actor) - actor = null + // Finally, stop the endpoint + ssc.env.rpcEnv.stop(endpoint) + endpoint = null receivedBlockTracker.stop() logInfo("ReceiverTracker stopped") } @@ -129,8 +127,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Signal the receivers to delete old block data if (ssc.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false)) { logInfo(s"Cleanup old received batch data: $cleanupThreshTime") - receiverInfo.values.flatMap { info => Option(info.actor) } - .foreach { _ ! CleanupOldBlocks(cleanupThreshTime) } + receiverInfo.values.flatMap { info => Option(info.endpoint) } + .foreach { _.send(CleanupOldBlocks(cleanupThreshTime)) } } } @@ -139,23 +137,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false streamId: Int, typ: String, host: String, - receiverActor: ActorRef, - sender: ActorRef + receiverEndpoint: RpcEndpointRef, + senderAddress: RpcAddress ) { if (!receiverInputStreamIds.contains(streamId)) { throw new SparkException("Register received for unexpected id " + streamId) } receiverInfo(streamId) = ReceiverInfo( - streamId, s"${typ}-${streamId}", receiverActor, true, host) + streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) - logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) + logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) } /** Deregister a receiver */ private def deregisterReceiver(streamId: Int, message: String, error: String) { val newReceiverInfo = receiverInfo.get(streamId) match { case Some(oldInfo) => - oldInfo.copy(actor = null, active = false, lastErrorMessage = message, lastError = error) + oldInfo.copy(endpoint = null, active = false, lastErrorMessage = message, lastError = error) case None => logWarning("No prior receiver info") ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) @@ -199,19 +197,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receivedBlockTracker.hasUnallocatedReceivedBlocks } - /** Actor to receive messages from the receivers. */ - private class ReceiverTrackerActor extends Actor { + /** RpcEndpoint to receive messages from the receivers. */ + private class ReceiverTrackerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint { + override def receive: PartialFunction[Any, Unit] = { - case RegisterReceiver(streamId, typ, host, receiverActor) => - registerReceiver(streamId, typ, host, receiverActor, sender) - sender ! true - case AddBlock(receivedBlockInfo) => - sender ! addBlock(receivedBlockInfo) case ReportError(streamId, message, error) => reportError(streamId, message, error) + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RegisterReceiver(streamId, typ, host, receiverEndpoint) => + registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) + context.reply(true) + case AddBlock(receivedBlockInfo) => + context.reply(addBlock(receivedBlockInfo)) case DeregisterReceiver(streamId, message, error) => deregisterReceiver(streamId, message, error) - sender ! true + context.reply(true) } } @@ -314,8 +316,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Stops the receivers. */ private def stopReceivers() { // Signal the receivers to stop - receiverInfo.values.flatMap { info => Option(info.actor)} - .foreach { _ ! StopReceiver } + receiverInfo.values.flatMap { info => Option(info.endpoint)} + .foreach { _.send(StopReceiver) } logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } From c776ee8a6fdcdc463746a815b7686e4e33a874a9 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 19 Apr 2015 20:48:36 -0700 Subject: [PATCH 082/144] [SPARK-6979][Streaming] Replace JobScheduler.eventActor and JobGenerator.eventActor with EventLoop Title says it all. cc rxin tdas Author: zsxwing Closes #5554 from zsxwing/SPARK-6979 and squashes the following commits: 5304350 [zsxwing] Fix NotSerializableException e9d3479 [zsxwing] Add blank lines 633e279 [zsxwing] Fix NotSerializableException e496ace [zsxwing] Replace JobGenerator.eventActor with EventLoop ec6ec58 [zsxwing] Fix the import order ce0fa73 [zsxwing] Replace JobScheduler.eventActor with EventLoop --- .../mllib/clustering/StreamingKMeans.scala | 3 +- .../streaming/scheduler/JobGenerator.scala | 38 +++++++++--------- .../streaming/scheduler/JobScheduler.scala | 40 ++++++++++--------- 3 files changed, 42 insertions(+), 39 deletions(-) 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 d4606fda37b0d..812014a041719 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 @@ -20,8 +20,7 @@ package org.apache.spark.mllib.clustering import scala.reflect.ClassTag import org.apache.spark.Logging -import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.{Experimental, DeveloperApi} +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 58e56638a2dca..2467d50839add 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -19,12 +19,10 @@ package org.apache.spark.streaming.scheduler import scala.util.{Failure, Success, Try} -import akka.actor.{ActorRef, Props, Actor} - import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Clock, ManualClock, Utils} +import org.apache.spark.util.{Clock, EventLoop, ManualClock} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -58,7 +56,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } private val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, - longTime => eventActor ! GenerateJobs(new Time(longTime)), "JobGenerator") + longTime => eventLoop.post(GenerateJobs(new Time(longTime))), "JobGenerator") // This is marked lazy so that this is initialized after checkpoint duration has been set // in the context and the generator has been started. @@ -70,22 +68,26 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { null } - // eventActor is created when generator starts. + // eventLoop is created when generator starts. // This not being null means the scheduler has been started and not stopped - private var eventActor: ActorRef = null + private var eventLoop: EventLoop[JobGeneratorEvent] = null // last batch whose completion,checkpointing and metadata cleanup has been completed private var lastProcessedBatch: Time = null /** Start generation of jobs */ def start(): Unit = synchronized { - if (eventActor != null) return // generator has already been started + if (eventLoop != null) return // generator has already been started + + eventLoop = new EventLoop[JobGeneratorEvent]("JobGenerator") { + override protected def onReceive(event: JobGeneratorEvent): Unit = processEvent(event) - eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { - override def receive: PartialFunction[Any, Unit] = { - case event: JobGeneratorEvent => processEvent(event) + override protected def onError(e: Throwable): Unit = { + jobScheduler.reportError("Error in job generator", e) } - }), "JobGenerator") + } + eventLoop.start() + if (ssc.isCheckpointPresent) { restart() } else { @@ -99,7 +101,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { * checkpoints written. */ def stop(processReceivedData: Boolean): Unit = synchronized { - if (eventActor == null) return // generator has already been stopped + if (eventLoop == null) return // generator has already been stopped if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") @@ -146,9 +148,9 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { graph.stop() } - // Stop the actor and checkpoint writer + // Stop the event loop and checkpoint writer if (shouldCheckpoint) checkpointWriter.stop() - ssc.env.actorSystem.stop(eventActor) + eventLoop.stop() logInfo("Stopped JobGenerator") } @@ -156,7 +158,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { * Callback called when a batch has been completely processed. */ def onBatchCompletion(time: Time) { - eventActor ! ClearMetadata(time) + eventLoop.post(ClearMetadata(time)) } /** @@ -164,7 +166,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { */ def onCheckpointCompletion(time: Time, clearCheckpointDataLater: Boolean) { if (clearCheckpointDataLater) { - eventActor ! ClearCheckpointData(time) + eventLoop.post(ClearCheckpointData(time)) } } @@ -247,7 +249,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } - eventActor ! DoCheckpoint(time, clearCheckpointDataLater = false) + eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = false)) } /** Clear DStream metadata for the given `time`. */ @@ -257,7 +259,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // If checkpointing is enabled, then checkpoint, // else mark batch to be fully processed if (shouldCheckpoint) { - eventActor ! DoCheckpoint(time, clearCheckpointDataLater = true) + eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = true)) } else { // If checkpointing is not enabled, then delete metadata information about // received blocks (block data not saved in any case). Otherwise, wait for 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 95f1857b4c377..508b89278dcba 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 @@ -17,13 +17,15 @@ package org.apache.spark.streaming.scheduler -import scala.util.{Failure, Success, Try} -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 scala.collection.JavaConversions._ +import scala.util.{Failure, Success} + +import org.apache.spark.Logging import org.apache.spark.rdd.PairRDDFunctions import org.apache.spark.streaming._ +import org.apache.spark.util.EventLoop private[scheduler] sealed trait JobSchedulerEvent @@ -46,20 +48,20 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { val listenerBus = new StreamingListenerBus() // These two are created only when scheduler starts. - // eventActor not being null means the scheduler has been started and not stopped + // eventLoop not being null means the scheduler has been started and not stopped var receiverTracker: ReceiverTracker = null - private var eventActor: ActorRef = null - + private var eventLoop: EventLoop[JobSchedulerEvent] = null def start(): Unit = synchronized { - if (eventActor != null) return // scheduler has already been started + if (eventLoop != null) return // scheduler has already been started logDebug("Starting JobScheduler") - eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { - override def receive: PartialFunction[Any, Unit] = { - case event: JobSchedulerEvent => processEvent(event) - } - }), "JobScheduler") + eventLoop = new EventLoop[JobSchedulerEvent]("JobScheduler") { + override protected def onReceive(event: JobSchedulerEvent): Unit = processEvent(event) + + override protected def onError(e: Throwable): Unit = reportError("Error in job scheduler", e) + } + eventLoop.start() listenerBus.start(ssc.sparkContext) receiverTracker = new ReceiverTracker(ssc) @@ -69,7 +71,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } def stop(processAllReceivedData: Boolean): Unit = synchronized { - if (eventActor == null) return // scheduler has already been stopped + if (eventLoop == null) return // scheduler has already been stopped logDebug("Stopping JobScheduler") // First, stop receiving @@ -96,8 +98,8 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // Stop everything else listenerBus.stop() - ssc.env.actorSystem.stop(eventActor) - eventActor = null + eventLoop.stop() + eventLoop = null logInfo("Stopped JobScheduler") } @@ -117,7 +119,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } def reportError(msg: String, e: Throwable) { - eventActor ! ErrorReported(msg, e) + eventLoop.post(ErrorReported(msg, e)) } private def processEvent(event: JobSchedulerEvent) { @@ -172,14 +174,14 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private class JobHandler(job: Job) extends Runnable { def run() { - eventActor ! JobStarted(job) + eventLoop.post(JobStarted(job)) // 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) + eventLoop.post(JobCompleted(job)) } } } From 6fe690d5a8216ba7efde4b52e7a19fb00814341c Mon Sep 17 00:00:00 2001 From: dobashim Date: Mon, 20 Apr 2015 00:03:23 -0400 Subject: [PATCH 083/144] [doc][mllib] Fix typo of the page title in Isotonic regression documents * Fix the page title in Isotonic regression documents (Naive Bayes -> Isotonic regression) * Add a newline character at the end of the file Author: dobashim Closes #5581 from dobashim/master and squashes the following commits: d54a041 [dobashim] Fix typo of the page title in Isotonic regression documents --- docs/mllib-isotonic-regression.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/mllib-isotonic-regression.md b/docs/mllib-isotonic-regression.md index 12fb29d426741..b521c2f27cd6e 100644 --- a/docs/mllib-isotonic-regression.md +++ b/docs/mllib-isotonic-regression.md @@ -1,6 +1,6 @@ --- layout: global -title: Naive Bayes - MLlib +title: Isotonic regression - MLlib displayTitle: MLlib - Regression --- @@ -152,4 +152,4 @@ Double meanSquaredError = new JavaDoubleRDD(predictionAndLabel.map( System.out.println("Mean Squared Error = " + meanSquaredError); {% endhighlight %} - \ No newline at end of file + From 1be207078cef48c5935595969bf9f6b1ec1334ca Mon Sep 17 00:00:00 2001 From: jrabary Date: Mon, 20 Apr 2015 09:47:56 -0700 Subject: [PATCH 084/144] [SPARK-5924] Add the ability to specify withMean or withStd parameters with StandarScaler The current implementation call the default constructor of mllib.feature.StandarScaler without the possibility to specify withMean or withStd options. Author: jrabary Closes #4704 from jrabary/master and squashes the following commits: fae8568 [jrabary] style fix 8896b0e [jrabary] Comments fix ef96d73 [jrabary] style fix 8e52607 [jrabary] style fix edd9d48 [jrabary] Fix default param initialization 17e1a76 [jrabary] Fix default param initialization 298f405 [jrabary] Typo fix 45ed914 [jrabary] Add withMean and withStd params to StandarScaler --- .../spark/ml/feature/StandardScaler.scala | 32 ++++++++++++++++--- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 1b102619b3524..447851ec034d6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -30,7 +30,22 @@ import org.apache.spark.sql.types.{StructField, StructType} /** * Params for [[StandardScaler]] and [[StandardScalerModel]]. */ -private[feature] trait StandardScalerParams extends Params with HasInputCol with HasOutputCol +private[feature] trait StandardScalerParams extends Params with HasInputCol with HasOutputCol { + + /** + * False by default. Centers the data with mean before scaling. + * It will build a dense output, so this does not work on sparse input + * and will raise an exception. + * @group param + */ + val withMean: BooleanParam = new BooleanParam(this, "withMean", "Center data with mean") + + /** + * True by default. Scales the data to unit standard deviation. + * @group param + */ + val withStd: BooleanParam = new BooleanParam(this, "withStd", "Scale to unit standard deviation") +} /** * :: AlphaComponent :: @@ -40,18 +55,27 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with @AlphaComponent class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerParams { + setDefault(withMean -> false, withStd -> true) + /** @group setParam */ def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ def setOutputCol(value: String): this.type = set(outputCol, value) - + + /** @group setParam */ + def setWithMean(value: Boolean): this.type = set(withMean, value) + + /** @group setParam */ + def setWithStd(value: Boolean): this.type = set(withStd, value) + override def fit(dataset: DataFrame, paramMap: ParamMap): StandardScalerModel = { transformSchema(dataset.schema, paramMap, logging = true) val map = extractParamMap(paramMap) val input = dataset.select(map(inputCol)).map { case Row(v: Vector) => v } - val scaler = new feature.StandardScaler().fit(input) - val model = new StandardScalerModel(this, map, scaler) + val scaler = new feature.StandardScaler(withMean = map(withMean), withStd = map(withStd)) + val scalerModel = scaler.fit(input) + val model = new StandardScalerModel(this, map, scalerModel) Params.inheritValues(map, this, model) model } From 968ad972175390bb0a96918fd3c7b318d70fa466 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 20 Apr 2015 09:54:21 -0700 Subject: [PATCH 085/144] [SPARK-7003] Improve reliability of connection failure detection between Netty block transfer service endpoints Currently we rely on the assumption that an exception will be raised and the channel closed if two endpoints cannot communicate over a Netty TCP channel. However, this guarantee does not hold in all network environments, and [SPARK-6962](https://issues.apache.org/jira/browse/SPARK-6962) seems to point to a case where only the server side of the connection detected a fault. This patch improves robustness of fetch/rpc requests by having an explicit timeout in the transport layer which closes the connection if there is a period of inactivity while there are outstanding requests. NB: This patch is actually only around 50 lines added if you exclude the testing-related code. Author: Aaron Davidson Closes #5584 from aarondav/timeout and squashes the following commits: 8699680 [Aaron Davidson] Address Reynold's comments 37ce656 [Aaron Davidson] [SPARK-7003] Improve reliability of connection failure detection between Netty block transfer service endpoints --- .../spark/network/TransportContext.java | 5 +- .../client/TransportResponseHandler.java | 14 +- .../server/TransportChannelHandler.java | 33 ++- .../spark/network/util/MapConfigProvider.java | 41 +++ .../apache/spark/network/util/NettyUtils.java | 2 +- .../RequestTimeoutIntegrationSuite.java | 277 ++++++++++++++++++ .../network/TransportClientFactorySuite.java | 21 +- 7 files changed, 375 insertions(+), 18 deletions(-) create mode 100644 network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java create mode 100644 network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java index f0a89c9d9116c..3fe69b1bd8851 100644 --- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import io.netty.channel.Channel; import io.netty.channel.socket.SocketChannel; +import io.netty.handler.timeout.IdleStateHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,6 +107,7 @@ public TransportChannelHandler initializePipeline(SocketChannel channel) { .addLast("encoder", encoder) .addLast("frameDecoder", NettyUtils.createFrameDecoder()) .addLast("decoder", decoder) + .addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000)) // NOTE: Chunks are currently guaranteed to be returned in the order of request, but this // would require more logic to guarantee if this were not part of the same event loop. .addLast("handler", channelHandler); @@ -126,7 +128,8 @@ private TransportChannelHandler createChannelHandler(Channel channel) { TransportClient client = new TransportClient(channel, responseHandler); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, rpcHandler); - return new TransportChannelHandler(client, responseHandler, requestHandler); + return new TransportChannelHandler(client, responseHandler, requestHandler, + conf.connectionTimeoutMs()); } public TransportConf getConf() { return conf; } diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 2044afb0d85db..94fc21af5e606 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; -import com.google.common.annotations.VisibleForTesting; import io.netty.channel.Channel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,13 +50,18 @@ public class TransportResponseHandler extends MessageHandler { private final Map outstandingRpcs; + /** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */ + private final AtomicLong timeOfLastRequestNs; + public TransportResponseHandler(Channel channel) { this.channel = channel; this.outstandingFetches = new ConcurrentHashMap(); this.outstandingRpcs = new ConcurrentHashMap(); + this.timeOfLastRequestNs = new AtomicLong(0); } public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) { + timeOfLastRequestNs.set(System.nanoTime()); outstandingFetches.put(streamChunkId, callback); } @@ -65,6 +70,7 @@ public void removeFetchRequest(StreamChunkId streamChunkId) { } public void addRpcRequest(long requestId, RpcResponseCallback callback) { + timeOfLastRequestNs.set(System.nanoTime()); outstandingRpcs.put(requestId, callback); } @@ -161,8 +167,12 @@ public void handle(ResponseMessage message) { } /** Returns total number of outstanding requests (fetch requests + rpcs) */ - @VisibleForTesting public int numOutstandingRequests() { return outstandingFetches.size() + outstandingRpcs.size(); } + + /** Returns the time in nanoseconds of when the last request was sent out. */ + public long getTimeOfLastRequestNs() { + return timeOfLastRequestNs.get(); + } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index e491367fa4528..8e0ee709e38e3 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -19,6 +19,8 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +42,11 @@ * Client. * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler, * for the Client's responses to the Server's requests. + * + * This class also handles timeouts from a {@link io.netty.handler.timeout.IdleStateHandler}. + * We consider a connection timed out if there are outstanding fetch or RPC requests but no traffic + * on the channel for at least `requestTimeoutMs`. Note that this is duplex traffic; we will not + * timeout if the client is continuously sending but getting no responses, for simplicity. */ public class TransportChannelHandler extends SimpleChannelInboundHandler { private final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class); @@ -47,14 +54,17 @@ public class TransportChannelHandler extends SimpleChannelInboundHandler 0; + boolean isActuallyOverdue = + System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; + if (e.state() == IdleState.ALL_IDLE && hasInFlightRequests && isActuallyOverdue) { + String address = NettyUtils.getRemoteAddress(ctx.channel()); + logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + + "requests. Assuming connection is dead; please adjust spark.network.timeout if this " + + "is wrong.", address, requestTimeoutNs / 1000 / 1000); + ctx.close(); + } + } + } } diff --git a/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java b/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java new file mode 100644 index 0000000000000..668d2356b955d --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import com.google.common.collect.Maps; + +import java.util.Map; +import java.util.NoSuchElementException; + +/** ConfigProvider based on a Map (copied in the constructor). */ +public class MapConfigProvider extends ConfigProvider { + private final Map config; + + public MapConfigProvider(Map config) { + this.config = Maps.newHashMap(config); + } + + @Override + public String get(String name) { + String value = config.get(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java index dabd6261d2aa0..26c6399ce7dbc 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -98,7 +98,7 @@ public static ByteToMessageDecoder createFrameDecoder() { return new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 8, -8, 8); } - /** Returns the remote address on the channel or "<remote address>" if none exists. */ + /** Returns the remote address on the channel or "<unknown remote>" if none exists. */ public static String getRemoteAddress(Channel channel) { if (channel != null && channel.remoteAddress() != null) { return channel.remoteAddress().toString(); diff --git a/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java new file mode 100644 index 0000000000000..84ebb337e6d54 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; +import org.junit.*; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +/** + * Suite which ensures that requests that go without a response for the network timeout period are + * failed, and the connection closed. + * + * In this suite, we use 2 seconds as the connection timeout, with some slack given in the tests, + * to ensure stability in different test environments. + */ +public class RequestTimeoutIntegrationSuite { + + private TransportServer server; + private TransportClientFactory clientFactory; + + private StreamManager defaultManager; + private TransportConf conf; + + // A large timeout that "shouldn't happen", for the sake of faulty tests not hanging forever. + private final int FOREVER = 60 * 1000; + + @Before + public void setUp() throws Exception { + Map configMap = Maps.newHashMap(); + configMap.put("spark.shuffle.io.connectionTimeout", "2s"); + conf = new TransportConf(new MapConfigProvider(configMap)); + + defaultManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + throw new UnsupportedOperationException(); + } + }; + } + + @After + public void tearDown() { + if (server != null) { + server.close(); + } + if (clientFactory != null) { + clientFactory.close(); + } + } + + // Basic suite: First request completes quickly, and second waits for longer than network timeout. + @Test + public void timeoutInactiveRequests() throws Exception { + final Semaphore semaphore = new Semaphore(1); + final byte[] response = new byte[16]; + RpcHandler handler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + try { + semaphore.tryAcquire(FOREVER, TimeUnit.MILLISECONDS); + callback.onSuccess(response); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // First completes quickly (semaphore starts at 1). + TestCallback callback0 = new TestCallback(); + synchronized (callback0) { + client.sendRpc(new byte[0], callback0); + callback0.wait(FOREVER); + assert (callback0.success.length == response.length); + } + + // Second times out after 2 seconds, with slack. Must be IOException. + TestCallback callback1 = new TestCallback(); + synchronized (callback1) { + client.sendRpc(new byte[0], callback1); + callback1.wait(4 * 1000); + assert (callback1.failure != null); + assert (callback1.failure instanceof IOException); + } + semaphore.release(); + } + + // A timeout will cause the connection to be closed, invalidating the current TransportClient. + // It should be the case that requesting a client from the factory produces a new, valid one. + @Test + public void timeoutCleanlyClosesClient() throws Exception { + final Semaphore semaphore = new Semaphore(0); + final byte[] response = new byte[16]; + RpcHandler handler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + try { + semaphore.tryAcquire(FOREVER, TimeUnit.MILLISECONDS); + callback.onSuccess(response); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + + // First request should eventually fail. + TransportClient client0 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback0 = new TestCallback(); + synchronized (callback0) { + client0.sendRpc(new byte[0], callback0); + callback0.wait(FOREVER); + assert (callback0.failure instanceof IOException); + assert (!client0.isActive()); + } + + // Increment the semaphore and the second request should succeed quickly. + semaphore.release(2); + TransportClient client1 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback1 = new TestCallback(); + synchronized (callback1) { + client1.sendRpc(new byte[0], callback1); + callback1.wait(FOREVER); + assert (callback1.success.length == response.length); + assert (callback1.failure == null); + } + } + + // The timeout is relative to the LAST request sent, which is kinda weird, but still. + // This test also makes sure the timeout works for Fetch requests as well as RPCs. + @Test + public void furtherRequestsDelay() throws Exception { + final byte[] response = new byte[16]; + final StreamManager manager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS); + return new NioManagedBuffer(ByteBuffer.wrap(response)); + } + }; + RpcHandler handler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + @Override + public StreamManager getStreamManager() { + return manager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // Send one request, which will eventually fail. + TestCallback callback0 = new TestCallback(); + client.fetchChunk(0, 0, callback0); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + // Send a second request before the first has failed. + TestCallback callback1 = new TestCallback(); + client.fetchChunk(0, 1, callback1); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + synchronized (callback0) { + // not complete yet, but should complete soon + assert (callback0.success == null && callback0.failure == null); + callback0.wait(2 * 1000); + assert (callback0.failure instanceof IOException); + } + + synchronized (callback1) { + // failed at same time as previous + assert (callback0.failure instanceof IOException); + } + } + + /** + * Callback which sets 'success' or 'failure' on completion. + * Additionally notifies all waiters on this callback when invoked. + */ + class TestCallback implements RpcResponseCallback, ChunkReceivedCallback { + + byte[] success; + Throwable failure; + + @Override + public void onSuccess(byte[] response) { + synchronized(this) { + success = response; + this.notifyAll(); + } + } + + @Override + public void onFailure(Throwable e) { + synchronized(this) { + failure = e; + this.notifyAll(); + } + } + + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + synchronized(this) { + try { + success = buffer.nioByteBuffer().array(); + this.notifyAll(); + } catch (IOException e) { + // weird + } + } + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + synchronized(this) { + failure = e; + this.notifyAll(); + } + } + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 416dc1b969fa4..35de5e57ccb98 100644 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -20,10 +20,11 @@ import java.io.IOException; import java.util.Collections; import java.util.HashSet; -import java.util.NoSuchElementException; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.collect.Maps; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -36,9 +37,9 @@ import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.util.ConfigProvider; -import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class TransportClientFactorySuite { @@ -70,16 +71,10 @@ public void tearDown() { */ private void testClientReuse(final int maxConnections, boolean concurrent) throws IOException, InterruptedException { - TransportConf conf = new TransportConf(new ConfigProvider() { - @Override - public String get(String name) { - if (name.equals("spark.shuffle.io.numConnectionsPerPeer")) { - return Integer.toString(maxConnections); - } else { - throw new NoSuchElementException(); - } - } - }); + + Map configMap = Maps.newHashMap(); + configMap.put("spark.shuffle.io.numConnectionsPerPeer", Integer.toString(maxConnections)); + TransportConf conf = new TransportConf(new MapConfigProvider(configMap)); RpcHandler rpcHandler = new NoOpRpcHandler(); TransportContext context = new TransportContext(conf, rpcHandler); From 77176619a97d07811ab20e1dde4677359d85eb33 Mon Sep 17 00:00:00 2001 From: Elisey Zanko Date: Mon, 20 Apr 2015 10:44:09 -0700 Subject: [PATCH 086/144] [SPARK-6661] Python type errors should print type, not object Author: Elisey Zanko Closes #5361 from 31z4/spark-6661 and squashes the following commits: 73c5d79 [Elisey Zanko] Python type errors should print type, not object --- python/pyspark/accumulators.py | 2 +- python/pyspark/context.py | 2 +- python/pyspark/ml/param/__init__.py | 2 +- python/pyspark/ml/pipeline.py | 4 ++-- python/pyspark/mllib/linalg.py | 4 ++-- python/pyspark/mllib/regression.py | 2 +- python/pyspark/mllib/tests.py | 6 ++++-- python/pyspark/sql/_types.py | 12 ++++++------ python/pyspark/sql/context.py | 8 ++++---- python/pyspark/sql/dataframe.py | 2 +- 10 files changed, 23 insertions(+), 21 deletions(-) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 7271809e43880..0d21a132048a5 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -83,7 +83,7 @@ >>> sc.accumulator([1.0, 2.0, 3.0]) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... -Exception:... +TypeError:... """ import sys diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 1dc2fec0ae5c8..6a743ac8bd600 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -671,7 +671,7 @@ def accumulator(self, value, accum_param=None): elif isinstance(value, complex): accum_param = accumulators.COMPLEX_ACCUMULATOR_PARAM else: - raise Exception("No default accumulator param for type %s" % type(value)) + raise TypeError("No default accumulator param for type %s" % type(value)) SparkContext._next_accum_id += 1 return Accumulator(SparkContext._next_accum_id - 1, value, accum_param) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 9fccb65675185..49c20b4cf70cf 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -30,7 +30,7 @@ class Param(object): def __init__(self, parent, name, doc): if not isinstance(parent, Params): - raise ValueError("Parent must be a Params but got type %s." % type(parent).__name__) + raise TypeError("Parent must be a Params but got type %s." % type(parent)) self.parent = parent self.name = str(name) self.doc = str(doc) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index d94ecfff09f66..7c1ec3026da6f 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -131,8 +131,8 @@ def fit(self, dataset, params={}): stages = paramMap[self.stages] for stage in stages: if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): - raise ValueError( - "Cannot recognize a pipeline stage of type %s." % type(stage).__name__) + raise TypeError( + "Cannot recognize a pipeline stage of type %s." % type(stage)) indexOfLastEstimator = -1 for i, stage in enumerate(stages): if isinstance(stage, Estimator): diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 38b3aa3ad460e..ec8c879ea9389 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -145,7 +145,7 @@ def serialize(self, obj): values = [float(v) for v in obj] return (1, None, None, values) else: - raise ValueError("cannot serialize %r of type %r" % (obj, type(obj))) + raise TypeError("cannot serialize %r of type %r" % (obj, type(obj))) def deserialize(self, datum): assert len(datum) == 4, \ @@ -561,7 +561,7 @@ def __getitem__(self, index): inds = self.indices vals = self.values if not isinstance(index, int): - raise ValueError( + raise TypeError( "Indices must be of type integer, got type %s" % type(index)) if index < 0: index += self.size diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index cd7310a64f4ae..a0117c57133ae 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -170,7 +170,7 @@ def _regression_train_wrapper(train_func, modelClass, data, initial_weights): from pyspark.mllib.classification import LogisticRegressionModel first = data.first() if not isinstance(first, LabeledPoint): - raise ValueError("data should be an RDD of LabeledPoint, but got %s" % first) + raise TypeError("data should be an RDD of LabeledPoint, but got %s" % type(first)) if initial_weights is None: initial_weights = [0.0] * len(data.first().features) if (modelClass == LogisticRegressionModel): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index c6ed5acd1770e..849c88341a967 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -135,8 +135,10 @@ def test_sparse_vector_indexing(self): self.assertEquals(sv[-1], 2) self.assertEquals(sv[-2], 0) self.assertEquals(sv[-4], 0) - for ind in [4, -5, 7.8]: + for ind in [4, -5]: self.assertRaises(ValueError, sv.__getitem__, ind) + for ind in [7.8, '1']: + self.assertRaises(TypeError, sv.__getitem__, ind) def test_matrix_indexing(self): mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) @@ -450,7 +452,7 @@ def test_infer_schema(self): elif isinstance(v, DenseVector): self.assertEqual(v, self.dv1) else: - raise ValueError("expecting a vector but got %r of type %r" % (v, type(v))) + raise TypeError("expecting a vector but got %r of type %r" % (v, type(v))) @unittest.skipIf(not _have_scipy, "SciPy not installed") diff --git a/python/pyspark/sql/_types.py b/python/pyspark/sql/_types.py index 492c0cbdcf693..110d1152fbdb6 100644 --- a/python/pyspark/sql/_types.py +++ b/python/pyspark/sql/_types.py @@ -562,8 +562,8 @@ def _infer_type(obj): else: try: return _infer_schema(obj) - except ValueError: - raise ValueError("not supported type: %s" % type(obj)) + except TypeError: + raise TypeError("not supported type: %s" % type(obj)) def _infer_schema(row): @@ -584,7 +584,7 @@ def _infer_schema(row): items = sorted(row.__dict__.items()) else: - raise ValueError("Can not infer schema for type: %s" % type(row)) + raise TypeError("Can not infer schema for type: %s" % type(row)) fields = [StructField(k, _infer_type(v), True) for k, v in items] return StructType(fields) @@ -696,7 +696,7 @@ def _merge_type(a, b): return a elif type(a) is not type(b): # TODO: type cast (such as int -> long) - raise TypeError("Can not merge type %s and %s" % (a, b)) + raise TypeError("Can not merge type %s and %s" % (type(a), type(b))) # same type if isinstance(a, StructType): @@ -773,7 +773,7 @@ def convert_struct(obj): elif hasattr(obj, "__dict__"): # object d = obj.__dict__ else: - raise ValueError("Unexpected obj: %s" % obj) + raise TypeError("Unexpected obj type: %s" % type(obj)) if convert_fields: return tuple([conv(d.get(name)) for name, conv in zip(names, converters)]) @@ -912,7 +912,7 @@ def _infer_schema_type(obj, dataType): return StructType(fields) else: - raise ValueError("Unexpected dataType: %s" % dataType) + raise TypeError("Unexpected dataType: %s" % type(dataType)) _acceptable_types = { diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index c90afc326ca0e..acf3c114548c0 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -208,7 +208,7 @@ def applySchema(self, rdd, schema): raise TypeError("Cannot apply schema to DataFrame") if not isinstance(schema, StructType): - raise TypeError("schema should be StructType, but got %s" % schema) + raise TypeError("schema should be StructType, but got %s" % type(schema)) return self.createDataFrame(rdd, schema) @@ -281,7 +281,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): # data could be list, tuple, generator ... rdd = self._sc.parallelize(data) except Exception: - raise ValueError("cannot create an RDD from type: %s" % type(data)) + raise TypeError("cannot create an RDD from type: %s" % type(data)) else: rdd = data @@ -293,8 +293,8 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): if isinstance(schema, (list, tuple)): first = rdd.first() if not isinstance(first, (list, tuple)): - raise ValueError("each row in `rdd` should be list or tuple, " - "but got %r" % type(first)) + raise TypeError("each row in `rdd` should be list or tuple, " + "but got %r" % type(first)) row_cls = Row(*schema) schema = self._inferSchema(rdd.map(lambda r: row_cls(*r)), samplingRatio) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index d70c5b0a6930c..75c181c0c7f5e 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -608,7 +608,7 @@ def __getitem__(self, item): jc = self._jdf.apply(self.columns[item]) return Column(jc) else: - raise TypeError("unexpected type: %s" % type(item)) + raise TypeError("unexpected item type: %s" % type(item)) def __getattr__(self, name): """Returns the :class:`Column` denoted by ``name``. From 1ebceaa55bec28850a48fb28b4cf7b44c8448a78 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 20 Apr 2015 10:47:37 -0700 Subject: [PATCH 087/144] [Minor][MLlib] Incorrect path to test data is used in DecisionTreeExample It should load from `testInput` instead of `input` for test data. Author: Liang-Chi Hsieh Closes #5594 from viirya/use_testinput and squashes the following commits: 5e8b174 [Liang-Chi Hsieh] Fix style. b60b475 [Liang-Chi Hsieh] Use testInput. --- .../org/apache/spark/examples/ml/DecisionTreeExample.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index d4cc8dede07ef..921b396e799e7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -173,7 +173,8 @@ object DecisionTreeExample { val splits: Array[RDD[LabeledPoint]] = if (testInput != "") { // Load testInput. val numFeatures = origExamples.take(1)(0).features.size - val origTestExamples: RDD[LabeledPoint] = loadData(sc, input, dataFormat, Some(numFeatures)) + val origTestExamples: RDD[LabeledPoint] = + loadData(sc, testInput, dataFormat, Some(numFeatures)) Array(origExamples, origTestExamples) } else { // Split input into training, test. From 97fda73db4efda2ba5b12937954de428258a5b56 Mon Sep 17 00:00:00 2001 From: Eric Chiang Date: Mon, 20 Apr 2015 13:11:21 -0700 Subject: [PATCH 088/144] fixed doc The contribution is my original work. I license the work to the project under the project's open source license. Small typo in the programming guide. Author: Eric Chiang Closes #5599 from ericchiang/docs-typo and squashes the following commits: 1177942 [Eric Chiang] fixed doc --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index f4fabb0927b66..27816515c5de2 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1093,7 +1093,7 @@ for details. ### Shuffle operations Certain operations within Spark trigger an event known as the shuffle. The shuffle is Spark's -mechanism for re-distributing data so that is grouped differently across partitions. This typically +mechanism for re-distributing data so that it's grouped differently across partitions. This typically involves copying data across executors and machines, making the shuffle a complex and costly operation. From 517bdf36aecdc94ef569b68f0a96892e707b5c7b Mon Sep 17 00:00:00 2001 From: BenFradet Date: Mon, 20 Apr 2015 13:46:55 -0700 Subject: [PATCH 089/144] [doc][streaming] Fixed broken link in mllib section The commit message is pretty self-explanatory. Author: BenFradet Closes #5600 from BenFradet/master and squashes the following commits: 108492d [BenFradet] [doc][streaming] Fixed broken link in mllib section --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 262512a639046..2f2fea53168a3 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1588,7 +1588,7 @@ See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more abo *** ## MLlib Operations -You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. (Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. +You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. [Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. *** From ce7ddabbcd330b19f6d0c17082304dfa6e1621b2 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 20 Apr 2015 18:42:50 -0700 Subject: [PATCH 090/144] [SPARK-6368][SQL] Build a specialized serializer for Exchange operator. JIRA: https://issues.apache.org/jira/browse/SPARK-6368 Author: Yin Huai Closes #5497 from yhuai/serializer2 and squashes the following commits: da562c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2 50e0c3d [Yin Huai] When no filed is emitted to shuffle, use SparkSqlSerializer for now. 9f1ed92 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2 6d07678 [Yin Huai] Address comments. 4273b8c [Yin Huai] Enabled SparkSqlSerializer2. 09e587a [Yin Huai] Remove TODO. 791b96a [Yin Huai] Use UTF8String. 60a1487 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2 3e09655 [Yin Huai] Use getAs for Date column. 43b9fb4 [Yin Huai] Test. 8297732 [Yin Huai] Fix test. c9373c8 [Yin Huai] Support DecimalType. 2379eeb [Yin Huai] ASF header. 39704ab [Yin Huai] Specialized serializer for Exchange. --- .../scala/org/apache/spark/sql/SQLConf.scala | 4 + .../apache/spark/sql/execution/Exchange.scala | 59 ++- .../sql/execution/SparkSqlSerializer2.scala | 421 ++++++++++++++++++ .../execution/SparkSqlSerializer2Suite.scala | 195 ++++++++ 4 files changed, 673 insertions(+), 6 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.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 5c65f04ee8497..4fc5de7e824fe 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 @@ -64,6 +64,8 @@ private[spark] object SQLConf { // Set to false when debugging requires the ability to look at invalid query plans. val DATAFRAME_EAGER_ANALYSIS = "spark.sql.eagerAnalysis" + val USE_SQL_SERIALIZER2 = "spark.sql.useSerializer2" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -147,6 +149,8 @@ private[sql] class SQLConf extends Serializable { */ private[spark] def codegenEnabled: Boolean = getConf(CODEGEN_ENABLED, "false").toBoolean + private[spark] def useSqlSerializer2: Boolean = getConf(USE_SQL_SERIALIZER2, "true").toBoolean + /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to * a broadcast value during the physical executions of join operations. Setting this to -1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 69a620e1ec929..5b2e46962cd3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} +import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner} import org.apache.spark.rdd.{RDD, ShuffledRDD} +import org.apache.spark.serializer.Serializer import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.types.DataType import org.apache.spark.util.MutablePair object Exchange { @@ -77,9 +79,48 @@ case class Exchange( } } - override def execute(): RDD[Row] = attachTree(this , "execute") { - lazy val sparkConf = child.sqlContext.sparkContext.getConf + @transient private lazy val sparkConf = child.sqlContext.sparkContext.getConf + + def serializer( + keySchema: Array[DataType], + valueSchema: Array[DataType], + numPartitions: Int): Serializer = { + // In ExternalSorter's spillToMergeableFile function, key-value pairs are written out + // through write(key) and then write(value) instead of write((key, value)). Because + // SparkSqlSerializer2 assumes that objects passed in are Product2, we cannot safely use + // it when spillToMergeableFile in ExternalSorter will be used. + // So, we will not use SparkSqlSerializer2 when + // - Sort-based shuffle is enabled and the number of reducers (numPartitions) is greater + // then the bypassMergeThreshold; or + // - newOrdering is defined. + val cannotUseSqlSerializer2 = + (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) || newOrdering.nonEmpty + + // It is true when there is no field that needs to be write out. + // For now, we will not use SparkSqlSerializer2 when noField is true. + val noField = + (keySchema == null || keySchema.length == 0) && + (valueSchema == null || valueSchema.length == 0) + + val useSqlSerializer2 = + child.sqlContext.conf.useSqlSerializer2 && // SparkSqlSerializer2 is enabled. + !cannotUseSqlSerializer2 && // Safe to use Serializer2. + SparkSqlSerializer2.support(keySchema) && // The schema of key is supported. + SparkSqlSerializer2.support(valueSchema) && // The schema of value is supported. + !noField + + val serializer = if (useSqlSerializer2) { + logInfo("Using SparkSqlSerializer2.") + new SparkSqlSerializer2(keySchema, valueSchema) + } else { + logInfo("Using SparkSqlSerializer.") + new SparkSqlSerializer(sparkConf) + } + + serializer + } + override def execute(): RDD[Row] = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. @@ -111,7 +152,10 @@ case class Exchange( } else { new ShuffledRDD[Row, Row, Row](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) + val keySchema = expressions.map(_.dataType).toArray + val valueSchema = child.output.map(_.dataType).toArray + shuffled.setSerializer(serializer(keySchema, valueSchema, numPartitions)) + shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => @@ -134,7 +178,9 @@ case class Exchange( } else { new ShuffledRDD[Row, Null, Null](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) + val keySchema = child.output.map(_.dataType).toArray + shuffled.setSerializer(serializer(keySchema, null, numPartitions)) + shuffled.map(_._1) case SinglePartition => @@ -152,7 +198,8 @@ case class Exchange( } val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) - shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) + val valueSchema = child.output.map(_.dataType).toArray + shuffled.setSerializer(serializer(null, valueSchema, 1)) shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala new file mode 100644 index 0000000000000..cec97de2cd8e4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala @@ -0,0 +1,421 @@ +/* + * 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 java.io._ +import java.math.{BigDecimal, BigInteger} +import java.nio.ByteBuffer +import java.sql.Timestamp + +import scala.reflect.ClassTag + +import org.apache.spark.serializer._ +import org.apache.spark.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow +import org.apache.spark.sql.types._ + +/** + * The serialization stream for [[SparkSqlSerializer2]]. It assumes that the object passed in + * its `writeObject` are [[Product2]]. The serialization functions for the key and value of the + * [[Product2]] are constructed based on their schemata. + * The benefit of this serialization stream is that compared with general-purpose serializers like + * Kryo and Java serializer, it can significantly reduce the size of serialized and has a lower + * allocation cost, which can benefit the shuffle operation. Right now, its main limitations are: + * 1. It does not support complex types, i.e. Map, Array, and Struct. + * 2. It assumes that the objects passed in are [[Product2]]. So, it cannot be used when + * [[org.apache.spark.util.collection.ExternalSorter]]'s merge sort operation is used because + * the objects passed in the serializer are not in the type of [[Product2]]. Also also see + * the comment of the `serializer` method in [[Exchange]] for more information on it. + */ +private[sql] class Serializer2SerializationStream( + keySchema: Array[DataType], + valueSchema: Array[DataType], + out: OutputStream) + extends SerializationStream with Logging { + + val rowOut = new DataOutputStream(out) + val writeKey = SparkSqlSerializer2.createSerializationFunction(keySchema, rowOut) + val writeValue = SparkSqlSerializer2.createSerializationFunction(valueSchema, rowOut) + + def writeObject[T: ClassTag](t: T): SerializationStream = { + val kv = t.asInstanceOf[Product2[Row, Row]] + writeKey(kv._1) + writeValue(kv._2) + + this + } + + def flush(): Unit = { + rowOut.flush() + } + + def close(): Unit = { + rowOut.close() + } +} + +/** + * The corresponding deserialization stream for [[Serializer2SerializationStream]]. + */ +private[sql] class Serializer2DeserializationStream( + keySchema: Array[DataType], + valueSchema: Array[DataType], + in: InputStream) + extends DeserializationStream with Logging { + + val rowIn = new DataInputStream(new BufferedInputStream(in)) + + val key = if (keySchema != null) new SpecificMutableRow(keySchema) else null + val value = if (valueSchema != null) new SpecificMutableRow(valueSchema) else null + val readKey = SparkSqlSerializer2.createDeserializationFunction(keySchema, rowIn, key) + val readValue = SparkSqlSerializer2.createDeserializationFunction(valueSchema, rowIn, value) + + def readObject[T: ClassTag](): T = { + readKey() + readValue() + + (key, value).asInstanceOf[T] + } + + def close(): Unit = { + rowIn.close() + } +} + +private[sql] class ShuffleSerializerInstance( + keySchema: Array[DataType], + valueSchema: Array[DataType]) + extends SerializerInstance { + + def serialize[T: ClassTag](t: T): ByteBuffer = + throw new UnsupportedOperationException("Not supported.") + + def deserialize[T: ClassTag](bytes: ByteBuffer): T = + throw new UnsupportedOperationException("Not supported.") + + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = + throw new UnsupportedOperationException("Not supported.") + + def serializeStream(s: OutputStream): SerializationStream = { + new Serializer2SerializationStream(keySchema, valueSchema, s) + } + + def deserializeStream(s: InputStream): DeserializationStream = { + new Serializer2DeserializationStream(keySchema, valueSchema, s) + } +} + +/** + * SparkSqlSerializer2 is a special serializer that creates serialization function and + * deserialization function based on the schema of data. It assumes that values passed in + * are key/value pairs and values returned from it are also key/value pairs. + * The schema of keys is represented by `keySchema` and that of values is represented by + * `valueSchema`. + */ +private[sql] class SparkSqlSerializer2(keySchema: Array[DataType], valueSchema: Array[DataType]) + extends Serializer + with Logging + with Serializable{ + + def newInstance(): SerializerInstance = new ShuffleSerializerInstance(keySchema, valueSchema) +} + +private[sql] object SparkSqlSerializer2 { + + final val NULL = 0 + final val NOT_NULL = 1 + + /** + * Check if rows with the given schema can be serialized with ShuffleSerializer. + */ + def support(schema: Array[DataType]): Boolean = { + if (schema == null) return true + + var i = 0 + while (i < schema.length) { + schema(i) match { + case udt: UserDefinedType[_] => return false + case array: ArrayType => return false + case map: MapType => return false + case struct: StructType => return false + case _ => + } + i += 1 + } + + return true + } + + /** + * The util function to create the serialization function based on the given schema. + */ + def createSerializationFunction(schema: Array[DataType], out: DataOutputStream): Row => Unit = { + (row: Row) => + // If the schema is null, the returned function does nothing when it get called. + if (schema != null) { + var i = 0 + while (i < schema.length) { + schema(i) match { + // When we write values to the underlying stream, we also first write the null byte + // first. Then, if the value is not null, we write the contents out. + + case NullType => // Write nothing. + + case BooleanType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeBoolean(row.getBoolean(i)) + } + + case ByteType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeByte(row.getByte(i)) + } + + case ShortType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeShort(row.getShort(i)) + } + + case IntegerType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeInt(row.getInt(i)) + } + + case LongType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeLong(row.getLong(i)) + } + + case FloatType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeFloat(row.getFloat(i)) + } + + case DoubleType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeDouble(row.getDouble(i)) + } + + case decimal: DecimalType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val value = row.apply(i).asInstanceOf[Decimal] + val javaBigDecimal = value.toJavaBigDecimal + // First, write out the unscaled value. + val bytes: Array[Byte] = javaBigDecimal.unscaledValue().toByteArray + out.writeInt(bytes.length) + out.write(bytes) + // Then, write out the scale. + out.writeInt(javaBigDecimal.scale()) + } + + case DateType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeInt(row.getAs[Int](i)) + } + + case TimestampType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val timestamp = row.getAs[java.sql.Timestamp](i) + val time = timestamp.getTime + val nanos = timestamp.getNanos + out.writeLong(time - (nanos / 1000000)) // Write the milliseconds value. + out.writeInt(nanos) // Write the nanoseconds part. + } + + case StringType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val bytes = row.getAs[UTF8String](i).getBytes + out.writeInt(bytes.length) + out.write(bytes) + } + + case BinaryType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val bytes = row.getAs[Array[Byte]](i) + out.writeInt(bytes.length) + out.write(bytes) + } + } + i += 1 + } + } + } + + /** + * The util function to create the deserialization function based on the given schema. + */ + def createDeserializationFunction( + schema: Array[DataType], + in: DataInputStream, + mutableRow: SpecificMutableRow): () => Unit = { + () => { + // If the schema is null, the returned function does nothing when it get called. + if (schema != null) { + var i = 0 + while (i < schema.length) { + schema(i) match { + // When we read values from the underlying stream, we also first read the null byte + // first. Then, if the value is not null, we update the field of the mutable row. + + case NullType => mutableRow.setNullAt(i) // Read nothing. + + case BooleanType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setBoolean(i, in.readBoolean()) + } + + case ByteType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setByte(i, in.readByte()) + } + + case ShortType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setShort(i, in.readShort()) + } + + case IntegerType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setInt(i, in.readInt()) + } + + case LongType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setLong(i, in.readLong()) + } + + case FloatType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setFloat(i, in.readFloat()) + } + + case DoubleType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setDouble(i, in.readDouble()) + } + + case decimal: DecimalType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + // First, read in the unscaled value. + val length = in.readInt() + val bytes = new Array[Byte](length) + in.readFully(bytes) + val unscaledVal = new BigInteger(bytes) + // Then, read the scale. + val scale = in.readInt() + // Finally, create the Decimal object and set it in the row. + mutableRow.update(i, Decimal(new BigDecimal(unscaledVal, scale))) + } + + case DateType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.update(i, in.readInt()) + } + + case TimestampType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + val time = in.readLong() // Read the milliseconds value. + val nanos = in.readInt() // Read the nanoseconds part. + val timestamp = new Timestamp(time) + timestamp.setNanos(nanos) + mutableRow.update(i, timestamp) + } + + case StringType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + val length = in.readInt() + val bytes = new Array[Byte](length) + in.readFully(bytes) + mutableRow.update(i, UTF8String(bytes)) + } + + case BinaryType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + val length = in.readInt() + val bytes = new Array[Byte](length) + in.readFully(bytes) + mutableRow.update(i, bytes) + } + } + i += 1 + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala new file mode 100644 index 0000000000000..27f063d73a9a9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala @@ -0,0 +1,195 @@ +/* + * 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 java.sql.{Timestamp, Date} + +import org.scalatest.{FunSuite, BeforeAndAfterAll} + +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.ShuffleDependency +import org.apache.spark.sql.types._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.{MyDenseVectorUDT, QueryTest} + +class SparkSqlSerializer2DataTypeSuite extends FunSuite { + // Make sure that we will not use serializer2 for unsupported data types. + def checkSupported(dataType: DataType, isSupported: Boolean): Unit = { + val testName = + s"${if (dataType == null) null else dataType.toString} is " + + s"${if (isSupported) "supported" else "unsupported"}" + + test(testName) { + assert(SparkSqlSerializer2.support(Array(dataType)) === isSupported) + } + } + + checkSupported(null, isSupported = true) + checkSupported(NullType, isSupported = true) + checkSupported(BooleanType, isSupported = true) + checkSupported(ByteType, isSupported = true) + checkSupported(ShortType, isSupported = true) + checkSupported(IntegerType, isSupported = true) + checkSupported(LongType, isSupported = true) + checkSupported(FloatType, isSupported = true) + checkSupported(DoubleType, isSupported = true) + checkSupported(DateType, isSupported = true) + checkSupported(TimestampType, isSupported = true) + checkSupported(StringType, isSupported = true) + checkSupported(BinaryType, isSupported = true) + checkSupported(DecimalType(10, 5), isSupported = true) + checkSupported(DecimalType.Unlimited, isSupported = true) + + // For now, ArrayType, MapType, and StructType are not supported. + checkSupported(ArrayType(DoubleType, true), isSupported = false) + checkSupported(ArrayType(StringType, false), isSupported = false) + checkSupported(MapType(IntegerType, StringType, true), isSupported = false) + checkSupported(MapType(IntegerType, ArrayType(DoubleType), false), isSupported = false) + checkSupported(StructType(StructField("a", IntegerType, true) :: Nil), isSupported = false) + // UDTs are not supported right now. + checkSupported(new MyDenseVectorUDT, isSupported = false) +} + +abstract class SparkSqlSerializer2Suite extends QueryTest with BeforeAndAfterAll { + var allColumns: String = _ + val serializerClass: Class[Serializer] = + classOf[SparkSqlSerializer2].asInstanceOf[Class[Serializer]] + var numShufflePartitions: Int = _ + var useSerializer2: Boolean = _ + + override def beforeAll(): Unit = { + numShufflePartitions = conf.numShufflePartitions + useSerializer2 = conf.useSqlSerializer2 + + sql("set spark.sql.useSerializer2=true") + + val supportedTypes = + Seq(StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType.Unlimited, DecimalType(6, 5), + DateType, TimestampType) + + val fields = supportedTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + allColumns = fields.map(_.name).mkString(",") + val schema = StructType(fields) + + // Create a RDD with all data types supported by SparkSqlSerializer2. + val rdd = + sparkContext.parallelize((1 to 1000), 10).map { i => + Row( + s"str${i}: test serializer2.", + s"binary${i}: test serializer2.".getBytes("UTF-8"), + null, + i % 2 == 0, + i.toByte, + i.toShort, + i, + Long.MaxValue - i.toLong, + (i + 0.25).toFloat, + (i + 0.75), + BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), + new Date(i), + new Timestamp(i)) + } + + createDataFrame(rdd, schema).registerTempTable("shuffle") + + super.beforeAll() + } + + override def afterAll(): Unit = { + dropTempTable("shuffle") + sql(s"set spark.sql.shuffle.partitions=$numShufflePartitions") + sql(s"set spark.sql.useSerializer2=$useSerializer2") + super.afterAll() + } + + def checkSerializer[T <: Serializer]( + executedPlan: SparkPlan, + expectedSerializerClass: Class[T]): Unit = { + executedPlan.foreach { + case exchange: Exchange => + val shuffledRDD = exchange.execute().firstParent.asInstanceOf[ShuffledRDD[_, _, _]] + val dependency = shuffledRDD.getDependencies.head.asInstanceOf[ShuffleDependency[_, _, _]] + val serializerNotSetMessage = + s"Expected $expectedSerializerClass as the serializer of Exchange. " + + s"However, the serializer was not set." + val serializer = dependency.serializer.getOrElse(fail(serializerNotSetMessage)) + assert(serializer.getClass === expectedSerializerClass) + case _ => // Ignore other nodes. + } + } + + test("key schema and value schema are not nulls") { + val df = sql(s"SELECT DISTINCT ${allColumns} FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + checkAnswer( + df, + table("shuffle").collect()) + } + + test("value schema is null") { + val df = sql(s"SELECT col0 FROM shuffle ORDER BY col0") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + assert( + df.map(r => r.getString(0)).collect().toSeq === + table("shuffle").select("col0").map(r => r.getString(0)).collect().sorted.toSeq) + } + + test("no map output field") { + val df = sql(s"SELECT 1 + 1 FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, classOf[SparkSqlSerializer]) + } +} + +/** Tests SparkSqlSerializer2 with sort based shuffle without sort merge. */ +class SparkSqlSerializer2SortShuffleSuite extends SparkSqlSerializer2Suite { + override def beforeAll(): Unit = { + super.beforeAll() + // Sort merge will not be triggered. + sql("set spark.sql.shuffle.partitions = 200") + } + + test("key schema is null") { + val aggregations = allColumns.split(",").map(c => s"COUNT($c)").mkString(",") + val df = sql(s"SELECT $aggregations FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + checkAnswer( + df, + Row(1000, 1000, 0, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000)) + } +} + +/** For now, we will use SparkSqlSerializer for sort based shuffle with sort merge. */ +class SparkSqlSerializer2SortMergeShuffleSuite extends SparkSqlSerializer2Suite { + + // We are expecting SparkSqlSerializer. + override val serializerClass: Class[Serializer] = + classOf[SparkSqlSerializer].asInstanceOf[Class[Serializer]] + + override def beforeAll(): Unit = { + super.beforeAll() + // To trigger the sort merge. + sql("set spark.sql.shuffle.partitions = 201") + } +} From c736220dac51cf73181fdd7f621c960c4e7bf0c2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 20 Apr 2015 18:54:01 -0700 Subject: [PATCH 091/144] [SPARK-6635][SQL] DataFrame.withColumn should replace columns with identical column names JIRA https://issues.apache.org/jira/browse/SPARK-6635 Author: Liang-Chi Hsieh Closes #5541 from viirya/replace_with_column and squashes the following commits: b539c7b [Liang-Chi Hsieh] For comment. 72f35b1 [Liang-Chi Hsieh] DataFrame.withColumn can replace original column with identical column name. --- .../scala/org/apache/spark/sql/DataFrame.scala | 14 +++++++++++++- .../org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 17c21f6e3a0e9..45f5da387692e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -747,7 +747,19 @@ class DataFrame private[sql]( * Returns a new [[DataFrame]] by adding a column. * @group dfops */ - def withColumn(colName: String, col: Column): DataFrame = select(Column("*"), col.as(colName)) + def withColumn(colName: String, col: Column): DataFrame = { + val resolver = sqlContext.analyzer.resolver + val replaced = schema.exists(f => resolver(f.name, colName)) + if (replaced) { + val colNames = schema.map { field => + val name = field.name + if (resolver(name, colName)) col.as(colName) else Column(name) + } + select(colNames :_*) + } else { + select(Column("*"), col.as(colName)) + } + } /** * Returns a new [[DataFrame]] with a column renamed. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 3250ab476aeb4..b9b6a400ae195 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -473,6 +473,14 @@ class DataFrameSuite extends QueryTest { assert(df.schema.map(_.name).toSeq === Seq("key", "value", "newCol")) } + test("replace column using withColumn") { + val df2 = TestSQLContext.sparkContext.parallelize(Array(1, 2, 3)).toDF("x") + val df3 = df2.withColumn("x", df2("x") + 1) + checkAnswer( + df3.select("x"), + Row(2) :: Row(3) :: Row(4) :: Nil) + } + test("withColumnRenamed") { val df = testData.toDF().withColumn("newCol", col("key") + 1) .withColumnRenamed("value", "valueRenamed") From 8136810dfad12008ac300116df7bc8448740f1ae Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 20 Apr 2015 23:18:42 -0700 Subject: [PATCH 092/144] [SPARK-6490][Core] Add spark.rpc.* and deprecate spark.akka.* Deprecated `spark.akka.num.retries`, `spark.akka.retry.wait`, `spark.akka.askTimeout`, `spark.akka.lookupTimeout`, and added `spark.rpc.num.retries`, `spark.rpc.retry.wait`, `spark.rpc.askTimeout`, `spark.rpc.lookupTimeout`. Author: zsxwing Closes #5595 from zsxwing/SPARK-6490 and squashes the following commits: e0d80a9 [zsxwing] Use getTimeAsMs and getTimeAsSeconds and other minor fixes 31dbe69 [zsxwing] Add spark.rpc.* and deprecate spark.akka.* --- .../scala/org/apache/spark/SparkConf.scala | 10 ++++++- .../org/apache/spark/deploy/Client.scala | 6 ++--- .../spark/deploy/client/AppClient.scala | 4 +-- .../apache/spark/deploy/master/Master.scala | 4 +-- .../spark/deploy/master/ui/MasterWebUI.scala | 4 +-- .../deploy/rest/StandaloneRestServer.scala | 8 +++--- .../spark/deploy/worker/ui/WorkerWebUI.scala | 4 +-- .../scala/org/apache/spark/rpc/RpcEnv.scala | 10 +++---- .../cluster/YarnSchedulerBackend.scala | 4 +-- .../spark/storage/BlockManagerMaster.scala | 4 +-- .../org/apache/spark/util/AkkaUtils.scala | 26 +++---------------- .../org/apache/spark/util/RpcUtils.scala | 23 ++++++++++++++++ .../apache/spark/MapOutputTrackerSuite.scala | 4 +-- .../org/apache/spark/SparkConfSuite.scala | 24 ++++++++++++++++- .../org/apache/spark/rpc/RpcEnvSuite.scala | 4 +-- 15 files changed, 86 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index e3a649d755450..c1996e08756a6 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -431,7 +431,15 @@ private[spark] object SparkConf extends Logging { "spark.yarn.am.waitTime" -> Seq( AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3", // Translate old value to a duration, with 10s wait time per try. - translation = s => s"${s.toLong * 10}s")) + translation = s => s"${s.toLong * 10}s")), + "spark.rpc.numRetries" -> Seq( + AlternateConfig("spark.akka.num.retries", "1.4")), + "spark.rpc.retry.wait" -> Seq( + AlternateConfig("spark.akka.retry.wait", "1.4")), + "spark.rpc.askTimeout" -> Seq( + AlternateConfig("spark.akka.askTimeout", "1.4")), + "spark.rpc.lookupTimeout" -> Seq( + AlternateConfig("spark.akka.lookupTimeout", "1.4")) ) /** diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 8d13b2a2cd4f3..c2c3e9a9e4827 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -27,7 +27,7 @@ import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, RpcUtils, Utils} /** * Proxy that relays messages to the driver. @@ -36,7 +36,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with ActorLogReceive with Logging { var masterActor: ActorSelection = _ - val timeout = AkkaUtils.askTimeout(conf) + val timeout = RpcUtils.askTimeout(conf) override def preStart(): Unit = { masterActor = context.actorSelection( @@ -155,7 +155,7 @@ object Client { if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) { conf.set("spark.akka.logLifecycleEvents", "true") } - conf.set("spark.akka.askTimeout", "10") + conf.set("spark.rpc.askTimeout", "10") conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING")) Logger.getRootLogger.setLevel(driverArgs.logLevel) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 4f06d7f96c46e..43c8a934c311a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.util.{ActorLogReceive, Utils, AkkaUtils} +import org.apache.spark.util.{ActorLogReceive, RpcUtils, Utils, AkkaUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, @@ -193,7 +193,7 @@ private[spark] class AppClient( def stop() { if (actor != null) { try { - val timeout = AkkaUtils.askTimeout(conf) + val timeout = RpcUtils.askTimeout(conf) val future = actor.ask(StopAppClient)(timeout) Await.result(future, timeout) } catch { 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 c5a6b1beac9be..ff2eed6dee70a 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 @@ -47,7 +47,7 @@ import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.metrics.MetricsSystem import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, RpcUtils, SignalLogger, Utils} private[master] class Master( host: String, @@ -931,7 +931,7 @@ private[deploy] object Master extends Logging { securityManager = securityMgr) val actor = actorSystem.actorOf( Props(classOf[Master], host, boundPort, webUiPort, securityMgr, conf), actorName) - val timeout = AkkaUtils.askTimeout(conf) + val timeout = RpcUtils.askTimeout(conf) val portsRequest = actor.ask(BoundPortsRequest)(timeout) val portsResponse = Await.result(portsRequest, timeout).asInstanceOf[BoundPortsResponse] (actorSystem, boundPort, portsResponse.webUIPort, portsResponse.restPort) 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 bb11e0642ddc6..aad9c87bdb987 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 @@ -21,7 +21,7 @@ import org.apache.spark.Logging import org.apache.spark.deploy.master.Master import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.RpcUtils /** * Web UI server for the standalone master. @@ -31,7 +31,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { val masterActorRef = master.self - val timeout = AkkaUtils.askTimeout(master.conf) + val timeout = RpcUtils.askTimeout(master.conf) val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) initialize() diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 4f19af59f409f..2d6b8d4204795 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -32,7 +32,7 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{AkkaUtils, RpcUtils, Utils} import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ @@ -223,7 +223,7 @@ private[rest] class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) } protected def handleKill(submissionId: String): KillSubmissionResponse = { - val askTimeout = AkkaUtils.askTimeout(conf) + val askTimeout = RpcUtils.askTimeout(conf) val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse]( DeployMessages.RequestKillDriver(submissionId), masterActor, askTimeout) val k = new KillSubmissionResponse @@ -257,7 +257,7 @@ private[rest] class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) } protected def handleStatus(submissionId: String): SubmissionStatusResponse = { - val askTimeout = AkkaUtils.askTimeout(conf) + val askTimeout = RpcUtils.askTimeout(conf) val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse]( DeployMessages.RequestDriverStatus(submissionId), masterActor, askTimeout) val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } @@ -321,7 +321,7 @@ private[rest] class SubmitRequestServlet( responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { requestMessage match { case submitRequest: CreateSubmissionRequest => - val askTimeout = AkkaUtils.askTimeout(conf) + val askTimeout = RpcUtils.askTimeout(conf) val driverDescription = buildDriverDescription(submitRequest) val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout) 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 de6423beb543e..b3bb5f911dbd7 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 @@ -25,7 +25,7 @@ import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.worker.ui.WorkerWebUI._ import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.RpcUtils /** * Web UI server for the standalone worker. @@ -38,7 +38,7 @@ class WorkerWebUI( extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") with Logging { - private[ui] val timeout = AkkaUtils.askTimeout(worker.conf) + private[ui] val timeout = RpcUtils.askTimeout(worker.conf) initialize() diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index cba038ca355d7..a5336b7563802 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -25,7 +25,7 @@ import scala.language.postfixOps import scala.reflect.ClassTag import org.apache.spark.{Logging, SparkException, SecurityManager, SparkConf} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{RpcUtils, Utils} /** * An RPC environment. [[RpcEndpoint]]s need to register itself with a name to [[RpcEnv]] to @@ -38,7 +38,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] abstract class RpcEnv(conf: SparkConf) { - private[spark] val defaultLookupTimeout = AkkaUtils.lookupTimeout(conf) + private[spark] val defaultLookupTimeout = RpcUtils.lookupTimeout(conf) /** * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used to implement @@ -282,9 +282,9 @@ trait ThreadSafeRpcEndpoint extends RpcEndpoint private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) extends Serializable with Logging { - private[this] val maxRetries = conf.getInt("spark.akka.num.retries", 3) - private[this] val retryWaitMs = conf.getLong("spark.akka.retry.wait", 3000) - private[this] val defaultAskTimeout = conf.getLong("spark.akka.askTimeout", 30) seconds + private[this] val maxRetries = RpcUtils.numRetries(conf) + private[this] val retryWaitMs = RpcUtils.retryWaitMs(conf) + private[this] val defaultAskTimeout = RpcUtils.askTimeout(conf) /** * return the address for the [[RpcEndpointRef]] diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index f72566c370a6f..1406a36a669c5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -24,7 +24,7 @@ import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.ui.JettyUtils -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{RpcUtils, Utils} import scala.util.control.NonFatal @@ -46,7 +46,7 @@ private[spark] abstract class YarnSchedulerBackend( private val yarnSchedulerEndpoint = rpcEnv.setupEndpoint( YarnSchedulerBackend.ENDPOINT_NAME, new YarnSchedulerEndpoint(rpcEnv)) - private implicit val askTimeout = AkkaUtils.askTimeout(sc.conf) + private implicit val askTimeout = RpcUtils.askTimeout(sc.conf) /** * Request executors from the ApplicationMaster by specifying the total number desired. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index ceacf043029f3..c798843bd5d8a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -23,7 +23,7 @@ import scala.concurrent.ExecutionContext.Implicits.global import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.RpcUtils private[spark] class BlockManagerMaster( @@ -32,7 +32,7 @@ class BlockManagerMaster( isDriver: Boolean) extends Logging { - val timeout = AkkaUtils.askTimeout(conf) + val timeout = RpcUtils.askTimeout(conf) /** Remove a dead executor from the driver endpoint. This is only called on the driver side. */ def removeExecutor(execId: String) { 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 8e8cc7cc6389e..b725df3b44596 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await -import scala.concurrent.duration.{Duration, FiniteDuration} +import scala.concurrent.duration.FiniteDuration import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -125,16 +125,6 @@ private[spark] object AkkaUtils extends Logging { (actorSystem, boundPort) } - /** Returns the default Spark timeout to use for Akka ask operations. */ - def askTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getLong("spark.akka.askTimeout", 30), "seconds") - } - - /** Returns the default Spark timeout to use for Akka remote actor lookup. */ - def lookupTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getLong("spark.akka.lookupTimeout", 30), "seconds") - } - private val AKKA_MAX_FRAME_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 /** Returns the configured max frame size for Akka messages in bytes. */ @@ -150,16 +140,6 @@ private[spark] object AkkaUtils extends Logging { /** Space reserved for extra data in an Akka message besides serialized task or task result. */ val reservedSizeBytes = 200 * 1024 - /** Returns the configured number of times to retry connecting */ - def numRetries(conf: SparkConf): Int = { - conf.getInt("spark.akka.num.retries", 3) - } - - /** Returns the configured number of milliseconds to wait on each retry */ - def retryWaitMs(conf: SparkConf): Int = { - conf.getInt("spark.akka.retry.wait", 3000) - } - /** * Send a message to the given actor and get its result within a default timeout, or * throw a SparkException if this fails. @@ -216,7 +196,7 @@ private[spark] object AkkaUtils extends Logging { val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name) - val timeout = AkkaUtils.lookupTimeout(conf) + val timeout = RpcUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } @@ -230,7 +210,7 @@ private[spark] object AkkaUtils extends Logging { val executorActorSystemName = SparkEnv.executorActorSystemName Utils.checkHost(host, "Expected hostname") val url = address(protocol(actorSystem), executorActorSystemName, host, port, name) - val timeout = AkkaUtils.lookupTimeout(conf) + val timeout = RpcUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala index 6665b17c3d5df..5ae793e0e87a3 100644 --- a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -17,6 +17,9 @@ package org.apache.spark.util +import scala.concurrent.duration._ +import scala.language.postfixOps + import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} @@ -32,4 +35,24 @@ object RpcUtils { Utils.checkHost(driverHost, "Expected hostname") rpcEnv.setupEndpointRef(driverActorSystemName, RpcAddress(driverHost, driverPort), name) } + + /** Returns the configured number of times to retry connecting */ + def numRetries(conf: SparkConf): Int = { + conf.getInt("spark.rpc.numRetries", 3) + } + + /** Returns the configured number of milliseconds to wait on each retry */ + def retryWaitMs(conf: SparkConf): Long = { + conf.getTimeAsMs("spark.rpc.retry.wait", "3s") + } + + /** Returns the default Spark timeout to use for RPC ask operations. */ + def askTimeout(conf: SparkConf): FiniteDuration = { + conf.getTimeAsSeconds("spark.rpc.askTimeout", "30s") seconds + } + + /** Returns the default Spark timeout to use for RPC remote endpoint lookup. */ + def lookupTimeout(conf: SparkConf): FiniteDuration = { + conf.getTimeAsSeconds("spark.rpc.lookupTimeout", "30s") seconds + } } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 6295d34be5ca9..6ed057a7cab97 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -154,7 +154,7 @@ class MapOutputTrackerSuite extends FunSuite { test("remote fetch below akka frame size") { val newConf = new SparkConf newConf.set("spark.akka.frameSize", "1") - newConf.set("spark.akka.askTimeout", "1") // Fail fast + newConf.set("spark.rpc.askTimeout", "1") // Fail fast val masterTracker = new MapOutputTrackerMaster(conf) val rpcEnv = createRpcEnv("spark") @@ -180,7 +180,7 @@ class MapOutputTrackerSuite extends FunSuite { test("remote fetch exceeds akka frame size") { val newConf = new SparkConf newConf.set("spark.akka.frameSize", "1") - newConf.set("spark.akka.askTimeout", "1") // Fail fast + newConf.set("spark.rpc.askTimeout", "1") // Fail fast val masterTracker = new MapOutputTrackerMaster(conf) val rpcEnv = createRpcEnv("test") diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 8e6c200c4ba00..d7d8014a20498 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -19,11 +19,13 @@ package org.apache.spark import java.util.concurrent.{TimeUnit, Executors} +import scala.concurrent.duration._ +import scala.language.postfixOps import scala.util.{Try, Random} import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} -import org.apache.spark.util.ResetSystemProperties +import org.apache.spark.util.{RpcUtils, ResetSystemProperties} import com.esotericsoftware.kryo.Kryo class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { @@ -222,6 +224,26 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(conf.getTimeAsSeconds("spark.yarn.am.waitTime") === 420) } + test("akka deprecated configs") { + val conf = new SparkConf() + + assert(!conf.contains("spark.rpc.num.retries")) + assert(!conf.contains("spark.rpc.retry.wait")) + assert(!conf.contains("spark.rpc.askTimeout")) + assert(!conf.contains("spark.rpc.lookupTimeout")) + + conf.set("spark.akka.num.retries", "1") + assert(RpcUtils.numRetries(conf) === 1) + + conf.set("spark.akka.retry.wait", "2") + assert(RpcUtils.retryWaitMs(conf) === 2L) + + conf.set("spark.akka.askTimeout", "3") + assert(RpcUtils.askTimeout(conf) === (3 seconds)) + + conf.set("spark.akka.lookupTimeout", "4") + assert(RpcUtils.lookupTimeout(conf) === (4 seconds)) + } } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index ada07ef11cd7a..5fbda37c7cb88 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -155,8 +155,8 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { }) val conf = new SparkConf() - conf.set("spark.akka.retry.wait", "0") - conf.set("spark.akka.num.retries", "1") + conf.set("spark.rpc.retry.wait", "0") + conf.set("spark.rpc.num.retries", "1") val anotherEnv = createRpcEnv(conf, "remote", 13345) // Use anotherEnv to find out the RpcEndpointRef val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-timeout") From ab9128fb7ec7ca479dc91e7cc7c775e8d071eafa Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 21 Apr 2015 00:08:18 -0700 Subject: [PATCH 093/144] [SPARK-6949] [SQL] [PySpark] Support Date/Timestamp in Column expression This PR enable auto_convert in JavaGateway, then we could register a converter for a given types, for example, date and datetime. There are two bugs related to auto_convert, see [1] and [2], we workaround it in this PR. [1] https://github.com/bartdag/py4j/issues/160 [2] https://github.com/bartdag/py4j/issues/161 cc rxin JoshRosen Author: Davies Liu Closes #5570 from davies/py4j_date and squashes the following commits: eb4fa53 [Davies Liu] fix tests in python 3 d17d634 [Davies Liu] rollback changes in mllib 2e7566d [Davies Liu] convert tuple into ArrayList ceb3779 [Davies Liu] Update rdd.py 3c373f3 [Davies Liu] support date and datetime by auto_convert cb094ff [Davies Liu] enable auto convert --- python/pyspark/context.py | 6 +----- python/pyspark/java_gateway.py | 15 ++++++++++++++- python/pyspark/rdd.py | 3 +++ python/pyspark/sql/_types.py | 27 +++++++++++++++++++++++++++ python/pyspark/sql/context.py | 13 ++++--------- python/pyspark/sql/dataframe.py | 18 ++++-------------- python/pyspark/sql/tests.py | 11 +++++++++++ python/pyspark/streaming/context.py | 11 +++-------- python/pyspark/streaming/kafka.py | 7 ++----- python/pyspark/streaming/tests.py | 6 +----- 10 files changed, 70 insertions(+), 47 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6a743ac8bd600..b006120eb266d 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -23,8 +23,6 @@ from threading import Lock from tempfile import NamedTemporaryFile -from py4j.java_collections import ListConverter - from pyspark import accumulators from pyspark.accumulators import Accumulator from pyspark.broadcast import Broadcast @@ -643,7 +641,6 @@ def union(self, rdds): rdds = [x._reserialize() for x in rdds] first = rdds[0]._jrdd rest = [x._jrdd for x in rdds[1:]] - rest = ListConverter().convert(rest, self._gateway._gateway_client) return RDD(self._jsc.union(first, rest), self, rdds[0]._jrdd_deserializer) def broadcast(self, value): @@ -846,13 +843,12 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): """ if partitions is None: partitions = range(rdd._jrdd.partitions().size()) - javaPartitions = ListConverter().convert(partitions, self._gateway._gateway_client) # Implementation note: This is implemented as a mapPartitions followed # by runJob() in order to avoid having to pass a Python lambda into # SparkContext#runJob. mappedRDD = rdd.mapPartitions(partitionFunc) - port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, + port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, partitions, allowLocal) return list(_load_from_socket(port, mappedRDD._jrdd_deserializer)) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 45bc38f7e61f8..3cee4ea6e3a35 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -17,17 +17,30 @@ import atexit import os +import sys import select import signal import shlex import socket import platform from subprocess import Popen, PIPE + +if sys.version >= '3': + xrange = range + from py4j.java_gateway import java_import, JavaGateway, GatewayClient +from py4j.java_collections import ListConverter from pyspark.serializers import read_int +# patching ListConverter, or it will convert bytearray into Java ArrayList +def can_convert_list(self, obj): + return isinstance(obj, (list, tuple, xrange)) + +ListConverter.can_convert = can_convert_list + + def launch_gateway(): if "PYSPARK_GATEWAY_PORT" in os.environ: gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) @@ -92,7 +105,7 @@ def killChild(): atexit.register(killChild) # Connect to the gateway - gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=False) + gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=True) # Import the classes used by PySpark java_import(gateway.jvm, "org.apache.spark.SparkConf") diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d9cdbb666f92a..d254deb527d10 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2267,6 +2267,9 @@ def _prepare_for_python_RDD(sc, command, obj=None): # The broadcast will have same life cycle as created PythonRDD broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) + # There is a bug in py4j.java_gateway.JavaClass with auto_convert + # https://github.com/bartdag/py4j/issues/161 + # TODO: use auto_convert once py4j fix the bug broadcast_vars = ListConverter().convert( [x._jbroadcast for x in sc._pickled_broadcast_vars], sc._gateway._gateway_client) diff --git a/python/pyspark/sql/_types.py b/python/pyspark/sql/_types.py index 110d1152fbdb6..95fb91ad43457 100644 --- a/python/pyspark/sql/_types.py +++ b/python/pyspark/sql/_types.py @@ -17,6 +17,7 @@ import sys import decimal +import time import datetime import keyword import warnings @@ -30,6 +31,9 @@ long = int unicode = str +from py4j.protocol import register_input_converter +from py4j.java_gateway import JavaClass + __all__ = [ "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", @@ -1237,6 +1241,29 @@ def __repr__(self): return "" % ", ".join(self) +class DateConverter(object): + def can_convert(self, obj): + return isinstance(obj, datetime.date) + + def convert(self, obj, gateway_client): + Date = JavaClass("java.sql.Date", gateway_client) + return Date.valueOf(obj.strftime("%Y-%m-%d")) + + +class DatetimeConverter(object): + def can_convert(self, obj): + return isinstance(obj, datetime.datetime) + + def convert(self, obj, gateway_client): + Timestamp = JavaClass("java.sql.Timestamp", gateway_client) + return Timestamp(int(time.mktime(obj.timetuple())) * 1000 + obj.microsecond // 1000) + + +# datetime is a subclass of date, we should register DatetimeConverter first +register_input_converter(DatetimeConverter()) +register_input_converter(DateConverter()) + + def _test(): import doctest from pyspark.context import SparkContext diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index acf3c114548c0..f6f107ca32d2f 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -25,7 +25,6 @@ from itertools import imap as map from py4j.protocol import Py4JError -from py4j.java_collections import MapConverter from pyspark.rdd import RDD, _prepare_for_python_RDD, ignore_unicode_prefix from pyspark.serializers import AutoBatchedSerializer, PickleSerializer @@ -442,15 +441,13 @@ def load(self, path=None, source=None, schema=None, **options): if source is None: source = self.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") - joptions = MapConverter().convert(options, - self._sc._gateway._gateway_client) if schema is None: - df = self._ssql_ctx.load(source, joptions) + df = self._ssql_ctx.load(source, options) else: if not isinstance(schema, StructType): raise TypeError("schema should be StructType") scala_datatype = self._ssql_ctx.parseDataType(schema.json()) - df = self._ssql_ctx.load(source, scala_datatype, joptions) + df = self._ssql_ctx.load(source, scala_datatype, options) return DataFrame(df, self) def createExternalTable(self, tableName, path=None, source=None, @@ -471,16 +468,14 @@ def createExternalTable(self, tableName, path=None, source=None, if source is None: source = self.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") - joptions = MapConverter().convert(options, - self._sc._gateway._gateway_client) if schema is None: - df = self._ssql_ctx.createExternalTable(tableName, source, joptions) + df = self._ssql_ctx.createExternalTable(tableName, source, options) else: if not isinstance(schema, StructType): raise TypeError("schema should be StructType") scala_datatype = self._ssql_ctx.parseDataType(schema.json()) df = self._ssql_ctx.createExternalTable(tableName, source, scala_datatype, - joptions) + options) return DataFrame(df, self) @ignore_unicode_prefix diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 75c181c0c7f5e..ca9bf8efb945c 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -25,8 +25,6 @@ else: from itertools import imap as map -from py4j.java_collections import ListConverter, MapConverter - from pyspark.context import SparkContext from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer @@ -186,9 +184,7 @@ def saveAsTable(self, tableName, source=None, mode="error", **options): source = self.sql_ctx.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") jmode = self._java_save_mode(mode) - joptions = MapConverter().convert(options, - self.sql_ctx._sc._gateway._gateway_client) - self._jdf.saveAsTable(tableName, source, jmode, joptions) + self._jdf.saveAsTable(tableName, source, jmode, options) def save(self, path=None, source=None, mode="error", **options): """Saves the contents of the :class:`DataFrame` to a data source. @@ -211,9 +207,7 @@ def save(self, path=None, source=None, mode="error", **options): source = self.sql_ctx.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") jmode = self._java_save_mode(mode) - joptions = MapConverter().convert(options, - self._sc._gateway._gateway_client) - self._jdf.save(source, jmode, joptions) + self._jdf.save(source, jmode, options) @property def schema(self): @@ -819,7 +813,6 @@ def fillna(self, value, subset=None): value = float(value) if isinstance(value, dict): - value = MapConverter().convert(value, self.sql_ctx._sc._gateway._gateway_client) return DataFrame(self._jdf.na().fill(value), self.sql_ctx) elif subset is None: return DataFrame(self._jdf.na().fill(value), self.sql_ctx) @@ -932,9 +925,7 @@ def agg(self, *exprs): """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): - jmap = MapConverter().convert(exprs[0], - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.agg(jmap) + jdf = self._jdf.agg(exprs[0]) else: # Columns assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" @@ -1040,8 +1031,7 @@ def _to_seq(sc, cols, converter=None): """ if converter: cols = [converter(c) for c in cols] - jcols = ListConverter().convert(cols, sc._gateway._gateway_client) - return sc._jvm.PythonUtils.toSeq(jcols) + return sc._jvm.PythonUtils.toSeq(cols) def _unary_op(name, doc="unary operator"): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index aa3aa1d164d9f..23e84283679e1 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -26,6 +26,7 @@ import tempfile import pickle import functools +import datetime import py4j @@ -464,6 +465,16 @@ def test_infer_long_type(self): self.assertEqual(_infer_type(2**61), LongType()) self.assertEqual(_infer_type(2**71), LongType()) + def test_filter_with_datetime(self): + time = datetime.datetime(2015, 4, 17, 23, 1, 2, 3000) + date = time.date() + row = Row(date=date, time=time) + df = self.sqlCtx.createDataFrame([row]) + self.assertEqual(1, df.filter(df.date == date).count()) + self.assertEqual(1, df.filter(df.time == time).count()) + self.assertEqual(0, df.filter(df.date > date).count()) + self.assertEqual(0, df.filter(df.time > time).count()) + def test_dropna(self): schema = StructType([ StructField("name", StringType(), True), diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 4590c58839266..ac5ba69e8dbbb 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -20,7 +20,6 @@ import os import sys -from py4j.java_collections import ListConverter from py4j.java_gateway import java_import, JavaObject from pyspark import RDD, SparkConf @@ -305,9 +304,7 @@ def queueStream(self, rdds, oneAtATime=True, default=None): rdds = [self._sc.parallelize(input) for input in rdds] self._check_serializers(rdds) - jrdds = ListConverter().convert([r._jrdd for r in rdds], - SparkContext._gateway._gateway_client) - queue = self._jvm.PythonDStream.toRDDQueue(jrdds) + queue = self._jvm.PythonDStream.toRDDQueue([r._jrdd for r in rdds]) if default: default = default._reserialize(rdds[0]._jrdd_deserializer) jdstream = self._jssc.queueStream(queue, oneAtATime, default._jrdd) @@ -322,8 +319,7 @@ def transform(self, dstreams, transformFunc): the transform function parameter will be the same as the order of corresponding DStreams in the list. """ - jdstreams = ListConverter().convert([d._jdstream for d in dstreams], - SparkContext._gateway._gateway_client) + jdstreams = [d._jdstream for d in dstreams] # change the final serializer to sc.serializer func = TransformFunction(self._sc, lambda t, *rdds: transformFunc(rdds).map(lambda x: x), @@ -346,6 +342,5 @@ def union(self, *dstreams): if len(set(s._slideDuration for s in dstreams)) > 1: raise ValueError("All DStreams should have same slide duration") first = dstreams[0] - jrest = ListConverter().convert([d._jdstream for d in dstreams[1:]], - SparkContext._gateway._gateway_client) + jrest = [d._jdstream for d in dstreams[1:]] return DStream(self._jssc.union(first._jdstream, jrest), self, first._jrdd_deserializer) diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index 7a7b6e1d9a527..8d610d6569b4a 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -15,8 +15,7 @@ # limitations under the License. # -from py4j.java_collections import MapConverter -from py4j.java_gateway import java_import, Py4JError, Py4JJavaError +from py4j.java_gateway import Py4JJavaError from pyspark.storagelevel import StorageLevel from pyspark.serializers import PairDeserializer, NoOpSerializer @@ -57,8 +56,6 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, }) if not isinstance(topics, dict): raise TypeError("topics should be dict") - jtopics = MapConverter().convert(topics, ssc.sparkContext._gateway._gateway_client) - jparam = MapConverter().convert(kafkaParams, ssc.sparkContext._gateway._gateway_client) jlevel = ssc._sc._getJavaStorageLevel(storageLevel) try: @@ -66,7 +63,7 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") helper = helperClass.newInstance() - jstream = helper.createStream(ssc._jssc, jparam, jtopics, jlevel) + jstream = helper.createStream(ssc._jssc, kafkaParams, topics, jlevel) except Py4JJavaError as e: # TODO: use --jar once it also work on driver if 'ClassNotFoundException' in str(e.java_exception): diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 06d22154373bc..33f958a601f3a 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -24,8 +24,6 @@ import struct from functools import reduce -from py4j.java_collections import MapConverter - from pyspark.context import SparkConf, SparkContext, RDD from pyspark.streaming.context import StreamingContext from pyspark.streaming.kafka import KafkaUtils @@ -581,11 +579,9 @@ def test_kafka_stream(self): """Test the Python Kafka stream API.""" topic = "topic1" sendData = {"a": 3, "b": 5, "c": 10} - jSendData = MapConverter().convert(sendData, - self.ssc.sparkContext._gateway._gateway_client) self._kafkaTestUtils.createTopic(topic) - self._kafkaTestUtils.sendMessages(topic, jSendData) + self._kafkaTestUtils.sendMessages(topic, sendData) stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(), "test-streaming-consumer", {topic: 1}, From 1f2f723b0daacbb9e70ec42c19a84470af1d7765 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 21 Apr 2015 00:14:16 -0700 Subject: [PATCH 094/144] [SPARK-5990] [MLLIB] Model import/export for IsotonicRegression Model import/export for IsotonicRegression Author: Yanbo Liang Closes #5270 from yanboliang/spark-5990 and squashes the following commits: 872028d [Yanbo Liang] fix code style f80ec1b [Yanbo Liang] address comments 49600cc [Yanbo Liang] address comments 429ff7d [Yanbo Liang] store each interval as a record 2b2f5a1 [Yanbo Liang] Model import/export for IsotonicRegression --- .../mllib/regression/IsotonicRegression.scala | 78 ++++++++++++++++++- .../regression/IsotonicRegressionSuite.scala | 21 +++++ 2 files changed, 98 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index cb70852e3cc8d..1d7617046b6c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -23,9 +23,16 @@ import java.util.Arrays.binarySearch import scala.collection.mutable.ArrayBuffer +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.sql.{DataFrame, SQLContext} /** * :: Experimental :: @@ -42,7 +49,7 @@ import org.apache.spark.rdd.RDD class IsotonicRegressionModel ( val boundaries: Array[Double], val predictions: Array[Double], - val isotonic: Boolean) extends Serializable { + val isotonic: Boolean) extends Serializable with Saveable { private val predictionOrd = if (isotonic) Ordering[Double] else Ordering[Double].reverse @@ -124,6 +131,75 @@ class IsotonicRegressionModel ( predictions(foundIndex) } } + + override def save(sc: SparkContext, path: String): Unit = { + IsotonicRegressionModel.SaveLoadV1_0.save(sc, path, boundaries, predictions, isotonic) + } + + override protected def formatVersion: String = "1.0" +} + +object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { + + import org.apache.spark.mllib.util.Loader._ + + private object SaveLoadV1_0 { + + def thisFormatVersion: String = "1.0" + + /** Hard-code class name string in case it changes in the future */ + def thisClassName: String = "org.apache.spark.mllib.regression.IsotonicRegressionModel" + + /** Model data for model import/export */ + case class Data(boundary: Double, prediction: Double) + + def save( + sc: SparkContext, + path: String, + boundaries: Array[Double], + predictions: Array[Double], + isotonic: Boolean): Unit = { + val sqlContext = new SQLContext(sc) + + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ + ("isotonic" -> isotonic))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) + + sqlContext.createDataFrame( + boundaries.toSeq.zip(predictions).map { case (b, p) => Data(b, p) } + ).saveAsParquetFile(dataPath(path)) + } + + def load(sc: SparkContext, path: String): (Array[Double], Array[Double]) = { + val sqlContext = new SQLContext(sc) + val dataRDD = sqlContext.parquetFile(dataPath(path)) + + checkSchema[Data](dataRDD.schema) + val dataArray = dataRDD.select("boundary", "prediction").collect() + val (boundaries, predictions) = dataArray.map { x => + (x.getDouble(0), x.getDouble(1)) + }.toList.sortBy(_._1).unzip + (boundaries.toArray, predictions.toArray) + } + } + + override def load(sc: SparkContext, path: String): IsotonicRegressionModel = { + implicit val formats = DefaultFormats + val (loadedClassName, version, metadata) = loadMetadata(sc, path) + val isotonic = (metadata \ "isotonic").extract[Boolean] + val classNameV1_0 = SaveLoadV1_0.thisClassName + (loadedClassName, version) match { + case (className, "1.0") if className == classNameV1_0 => + val (boundaries, predictions) = SaveLoadV1_0.load(sc, path) + new IsotonicRegressionModel(boundaries, predictions, isotonic) + case _ => throw new Exception( + s"IsotonicRegressionModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)" + ) + } + } } /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala index 7ef45248281e9..8e12340bbd9d6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.{Matchers, FunSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class IsotonicRegressionSuite extends FunSuite with MLlibTestSparkContext with Matchers { @@ -73,6 +74,26 @@ class IsotonicRegressionSuite extends FunSuite with MLlibTestSparkContext with M assert(model.isotonic) } + test("model save/load") { + val boundaries = Array(0.0, 1.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0) + val predictions = Array(1, 2, 2, 6, 16.5, 16.5, 17.0, 18.0) + val model = new IsotonicRegressionModel(boundaries, predictions, true) + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = IsotonicRegressionModel.load(sc, path) + assert(model.boundaries === sameModel.boundaries) + assert(model.predictions === sameModel.predictions) + assert(model.isotonic === model.isotonic) + } finally { + Utils.deleteRecursively(tempDir) + } + } + test("isotonic regression with size 0") { val model = runIsotonicRegression(Seq(), true) From 5fea3e5c36450658d8b767dd3c06dac2251a0e0c Mon Sep 17 00:00:00 2001 From: David McGuire Date: Tue, 21 Apr 2015 07:21:10 -0400 Subject: [PATCH 095/144] [SPARK-6985][streaming] Receiver maxRate over 1000 causes a StackOverflowError A simple truncation in integer division (on rates over 1000 messages / second) causes the existing implementation to sleep for 0 milliseconds, then call itself recursively; this causes what is essentially an infinite recursion, since the base case of the calculated amount of time having elapsed can't be reached before available stack space is exhausted. A fix to this truncation error is included in this patch. However, even with the defect patched, the accuracy of the existing implementation is abysmal (the error bounds of the original test were effectively [-30%, +10%], although this fact was obscured by hard-coded error margins); as such, when the error bounds were tightened down to [-5%, +5%], the existing implementation failed to meet the new, tightened, requirements. Therefore, an industry-vetted solution (from Guava) was used to get the adapted tests to pass. Author: David McGuire Closes #5559 from dmcguire81/master and squashes the following commits: d29d2e0 [David McGuire] Back out to +/-5% error margins, for flexibility in timing 8be6934 [David McGuire] Fix spacing per code review 90e98b9 [David McGuire] Address scalastyle errors 29011bd [David McGuire] Further ratchet down the error margins b33b796 [David McGuire] Eliminate dependency on even distribution by BlockGenerator 8f2934b [David McGuire] Remove arbitrary thread timing / cooperation code 70ee310 [David McGuire] Use Thread.yield(), since Thread.sleep(0) is system-dependent 82ee46d [David McGuire] Replace guard clause with nested conditional 2794717 [David McGuire] Replace the RateLimiter with the Guava implementation 38f3ca8 [David McGuire] Ratchet down the error rate to +/- 5%; tests fail 24b1bc0 [David McGuire] Fix truncation in integer division causing infinite recursion d6e1079 [David McGuire] Stack overflow error in RateLimiter on rates over 1000/s --- .../streaming/receiver/RateLimiter.scala | 33 +++---------------- .../spark/streaming/ReceiverSuite.scala | 29 +++++++++------- 2 files changed, 21 insertions(+), 41 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala index e4f6ba626ebbf..97db9ded83367 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.receiver import org.apache.spark.{Logging, SparkConf} -import java.util.concurrent.TimeUnit._ +import com.google.common.util.concurrent.{RateLimiter=>GuavaRateLimiter} /** Provides waitToPush() method to limit the rate at which receivers consume data. * @@ -33,37 +33,12 @@ import java.util.concurrent.TimeUnit._ */ private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging { - private var lastSyncTime = System.nanoTime - private var messagesWrittenSinceSync = 0L private val desiredRate = conf.getInt("spark.streaming.receiver.maxRate", 0) - private val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS) + private lazy val rateLimiter = GuavaRateLimiter.create(desiredRate) def waitToPush() { - if( desiredRate <= 0 ) { - return - } - val now = System.nanoTime - val elapsedNanosecs = math.max(now - lastSyncTime, 1) - val rate = messagesWrittenSinceSync.toDouble * 1000000000 / elapsedNanosecs - if (rate < desiredRate) { - // It's okay to write; just update some variables and return - messagesWrittenSinceSync += 1 - if (now > lastSyncTime + SYNC_INTERVAL) { - // Sync interval has passed; let's resync - lastSyncTime = now - messagesWrittenSinceSync = 1 - } - } else { - // Calculate how much time we should sleep to bring ourselves to the desired rate. - val targetTimeInMillis = messagesWrittenSinceSync * 1000 / desiredRate - val elapsedTimeInMillis = elapsedNanosecs / 1000000 - val sleepTimeInMillis = targetTimeInMillis - elapsedTimeInMillis - if (sleepTimeInMillis > 0) { - logTrace("Natural rate is " + rate + " per second but desired rate is " + - desiredRate + ", sleeping for " + sleepTimeInMillis + " ms to compensate.") - Thread.sleep(sleepTimeInMillis) - } - waitToPush() + if (desiredRate > 0) { + rateLimiter.acquire() } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 91261a9db7360..e7aee6eadbfc7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -158,7 +158,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener val blockIntervalMs = 100 - val maxRate = 100 + val maxRate = 1001 val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms"). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) @@ -176,7 +176,6 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { blockGenerator.addData(count) generatedData += count count += 1 - Thread.sleep(1) } blockGenerator.stop() @@ -185,25 +184,31 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { assert(blockGeneratorListener.arrayBuffers.size > 0, "No blocks received") assert(recordedData.toSet === generatedData.toSet, "Received data not same") - // recordedData size should be close to the expected rate - val minExpectedMessages = expectedMessages - 3 - val maxExpectedMessages = expectedMessages + 1 + // recordedData size should be close to the expected rate; use an error margin proportional to + // the value, so that rate changes don't cause a brittle test + val minExpectedMessages = expectedMessages - 0.05 * expectedMessages + val maxExpectedMessages = expectedMessages + 0.05 * expectedMessages val numMessages = recordedData.size assert( numMessages >= minExpectedMessages && numMessages <= maxExpectedMessages, s"#records received = $numMessages, not between $minExpectedMessages and $maxExpectedMessages" ) - val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 3 - val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 1 + // XXX Checking every block would require an even distribution of messages across blocks, + // which throttling code does not control. Therefore, test against the average. + val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 0.05 * expectedMessagesPerBlock + val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 0.05 * expectedMessagesPerBlock val receivedBlockSizes = recordedBlocks.map { _.size }.mkString(",") + + // the first and last block may be incomplete, so we slice them out + val validBlocks = recordedBlocks.drop(1).dropRight(1) + val averageBlockSize = validBlocks.map(block => block.size).sum / validBlocks.size + assert( - // the first and last block may be incomplete, so we slice them out - recordedBlocks.drop(1).dropRight(1).forall { block => - block.size >= minExpectedMessagesPerBlock && block.size <= maxExpectedMessagesPerBlock - }, + averageBlockSize >= minExpectedMessagesPerBlock && + averageBlockSize <= maxExpectedMessagesPerBlock, s"# records in received blocks = [$receivedBlockSizes], not between " + - s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock" + s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock, on average" ) } From c035c0f2d72f2a303b86fe0037ec43d756fff060 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 21 Apr 2015 11:01:18 -0700 Subject: [PATCH 096/144] [SPARK-5360] [SPARK-6606] Eliminate duplicate objects in serialized CoGroupedRDD CoGroupPartition, part of CoGroupedRDD, includes references to each RDD that the CoGroupedRDD narrowly depends on, and a reference to the ShuffleHandle. The partition is serialized separately from the RDD, so when the RDD and partition arrive on the worker, the references in the partition and in the RDD no longer point to the same object. This is a relatively minor performance issue (the closure can be 2x larger than it needs to be because the rdds and partitions are serialized twice; see numbers below) but is more annoying as a developer issue (this is where I ran into): if any state is stored in the RDD or ShuffleHandle on the worker side, subtle bugs can appear due to the fact that the references to the RDD / ShuffleHandle in the RDD and in the partition point to separate objects. I'm not sure if this is enough of a potential future problem to fix this old and central part of the code, so hoping to get input from others here. I did some simple experiments to see how much this effects closure size. For this example: $ val a = sc.parallelize(1 to 10).map((_, 1)) $ val b = sc.parallelize(1 to 2).map(x => (x, 2*x)) $ a.cogroup(b).collect() the closure was 1902 bytes with current Spark, and 1129 bytes after my change. The difference comes from eliminating duplicate serialization of the shuffle handle. For this example: $ val sortedA = a.sortByKey() $ val sortedB = b.sortByKey() $ sortedA.cogroup(sortedB).collect() the closure was 3491 bytes with current Spark, and 1333 bytes after my change. Here, the difference comes from eliminating duplicate serialization of the two RDDs for the narrow dependencies. The ShuffleHandle includes the ShuffleDependency, so this difference will get larger if a ShuffleDependency includes a serializer, a key ordering, or an aggregator (all set to None by default). It would also get bigger for a big RDD -- although I can't think of any examples where the RDD object gets large. The difference is not affected by the size of the function the user specifies, which (based on my understanding) is typically the source of large task closures. Author: Kay Ousterhout Closes #4145 from kayousterhout/SPARK-5360 and squashes the following commits: 85156c3 [Kay Ousterhout] Better comment the narrowDeps parameter cff0209 [Kay Ousterhout] Fixed spelling issue 658e1af [Kay Ousterhout] [SPARK-5360] Eliminate duplicate objects in serialized CoGroupedRDD --- .../org/apache/spark/rdd/CoGroupedRDD.scala | 43 +++++++++++-------- .../org/apache/spark/rdd/SubtractedRDD.scala | 30 +++++++------ 2 files changed, 44 insertions(+), 29 deletions(-) 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 7021a339e879b..658e8c8b89318 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -29,15 +29,16 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap, CompactBuffer} import org.apache.spark.util.Utils import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.ShuffleHandle - -private[spark] sealed trait CoGroupSplitDep extends Serializable +/** The references to rdd and splitIndex are transient because redundant information is stored + * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from + * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the + * task closure. */ private[spark] case class NarrowCoGroupSplitDep( - rdd: RDD[_], - splitIndex: Int, + @transient rdd: RDD[_], + @transient splitIndex: Int, var split: Partition - ) extends CoGroupSplitDep { + ) extends Serializable { @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { @@ -47,9 +48,16 @@ private[spark] case class NarrowCoGroupSplitDep( } } -private[spark] case class ShuffleCoGroupSplitDep(handle: ShuffleHandle) extends CoGroupSplitDep - -private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) +/** + * Stores information about the narrow dependencies used by a CoGroupedRdd. + * + * @param narrowDeps maps to the dependencies variable in the parent RDD: for each one to one + * dependency in dependencies, narrowDeps has a NarrowCoGroupSplitDep (describing + * the partition for that dependency) at the corresponding index. The size of + * narrowDeps should always be equal to the number of parents. + */ +private[spark] class CoGroupPartition( + idx: Int, val narrowDeps: Array[Option[NarrowCoGroupSplitDep]]) extends Partition with Serializable { override val index: Int = idx override def hashCode(): Int = idx @@ -105,9 +113,9 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: // Assume each RDD contributed a single dependency, and get it dependencies(j) match { case s: ShuffleDependency[_, _, _] => - new ShuffleCoGroupSplitDep(s.shuffleHandle) + None case _ => - new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) + Some(new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))) } }.toArray) } @@ -120,20 +128,21 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val sparkConf = SparkEnv.get.conf val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true) val split = s.asInstanceOf[CoGroupPartition] - val numRdds = split.deps.length + val numRdds = dependencies.length // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] - for ((dep, depNum) <- split.deps.zipWithIndex) dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => + for ((dep, depNum) <- dependencies.zipWithIndex) dep match { + case oneToOneDependency: OneToOneDependency[Product2[K, Any]] => + val dependencyPartition = split.narrowDeps(depNum).get.split // Read them from the parent - val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]] + val it = oneToOneDependency.rdd.iterator(dependencyPartition, context) rddIterators += ((it, depNum)) - case ShuffleCoGroupSplitDep(handle) => + case shuffleDependency: ShuffleDependency[_, _, _] => // Read map outputs of shuffle val it = SparkEnv.get.shuffleManager - .getReader(handle, split.index, split.index + 1, context) + .getReader(shuffleDependency.shuffleHandle, split.index, split.index + 1, context) .read() rddIterators += ((it, depNum)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index e9d745588ee9a..633aeba3bbae6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -81,9 +81,9 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( array(i) = new CoGroupPartition(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) => dependencies(j) match { case s: ShuffleDependency[_, _, _] => - new ShuffleCoGroupSplitDep(s.shuffleHandle) + None case _ => - new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) + Some(new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))) } }.toArray) } @@ -105,20 +105,26 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( seq } } - def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit): Unit = dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => - rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) + def integrate(depNum: Int, op: Product2[K, V] => Unit) = { + dependencies(depNum) match { + case oneToOneDependency: OneToOneDependency[_] => + val dependencyPartition = partition.narrowDeps(depNum).get.split + oneToOneDependency.rdd.iterator(dependencyPartition, context) + .asInstanceOf[Iterator[Product2[K, V]]].foreach(op) - case ShuffleCoGroupSplitDep(handle) => - val iter = SparkEnv.get.shuffleManager - .getReader(handle, partition.index, partition.index + 1, context) - .read() - iter.foreach(op) + case shuffleDependency: ShuffleDependency[_, _, _] => + val iter = SparkEnv.get.shuffleManager + .getReader( + shuffleDependency.shuffleHandle, partition.index, partition.index + 1, context) + .read() + iter.foreach(op) + } } + // the first dep is rdd1; add all values to the map - integrate(partition.deps(0), t => getSeq(t._1) += t._2) + integrate(0, t => getSeq(t._1) += t._2) // the second dep is rdd2; remove all of its keys - integrate(partition.deps(1), t => map.remove(t._1)) + integrate(1, t => map.remove(t._1)) map.iterator.map { t => t._2.iterator.map { (t._1, _) } }.flatten } From c25ca7c5a1f2a4f88f40b0c5cdbfa927c186cfa8 Mon Sep 17 00:00:00 2001 From: emres Date: Tue, 21 Apr 2015 16:39:56 -0400 Subject: [PATCH 097/144] SPARK-3276 Added a new configuration spark.streaming.minRememberDuration SPARK-3276 Added a new configuration parameter ``spark.streaming.minRememberDuration``, with a default value of 1 minute. So that when a Spark Streaming application is started, an arbitrary number of minutes can be taken as threshold for remembering. Author: emres Closes #5438 from emres/SPARK-3276 and squashes the following commits: 766f938 [emres] SPARK-3276 Switched to using newly added getTimeAsSeconds method. affee1d [emres] SPARK-3276 Changed the property name and variable name for minRememberDuration c9d58ca [emres] SPARK-3276 Minor code re-formatting. 1c53ba9 [emres] SPARK-3276 Started to use ssc.conf rather than ssc.sparkContext.getConf, and also getLong method directly. bfe0acb [emres] SPARK-3276 Moved the minRememberDurationMin to the class daccc82 [emres] SPARK-3276 Changed the property name to reflect the unit of value and reduced number of fields. 43cc1ce [emres] SPARK-3276 Added a new configuration parameter spark.streaming.minRemember duration, with a default value of 1 minute. --- .../streaming/dstream/FileInputDStream.scala | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 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 66d519171fd76..eca69f00188e4 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 @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.SerializableWritable +import org.apache.spark.{SparkConf, SerializableWritable} import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.streaming._ import org.apache.spark.util.{TimeStampedHashMap, Utils} @@ -63,7 +63,7 @@ import org.apache.spark.util.{TimeStampedHashMap, Utils} * the streaming app. * - If a file is to be visible in the directory listings, it must be visible within a certain * duration of the mod time of the file. This duration is the "remember window", which is set to - * 1 minute (see `FileInputDStream.MIN_REMEMBER_DURATION`). Otherwise, the file will never be + * 1 minute (see `FileInputDStream.minRememberDuration`). Otherwise, the file will never be * selected as the mod time will be less than the ignore threshold when it becomes visible. * - Once a file is visible, the mod time cannot change. If it does due to appends, then the * processing semantics are undefined. @@ -80,6 +80,15 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( private val serializableConfOpt = conf.map(new SerializableWritable(_)) + /** + * Minimum duration of remembering the information of selected files. Defaults to 60 seconds. + * + * Files with mod times older than this "window" of remembering will be ignored. So if new + * files are visible within this window, then the file will get selected in the next batch. + */ + private val minRememberDurationS = + Seconds(ssc.conf.getTimeAsSeconds("spark.streaming.minRememberDuration", "60s")) + // This is a def so that it works during checkpoint recovery: private def clock = ssc.scheduler.clock @@ -95,7 +104,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( * This would allow us to filter away not-too-old files which have already been recently * selected and processed. */ - private val numBatchesToRemember = FileInputDStream.calculateNumBatchesToRemember(slideDuration) + private val numBatchesToRemember = FileInputDStream + .calculateNumBatchesToRemember(slideDuration, minRememberDurationS) private val durationToRemember = slideDuration * numBatchesToRemember remember(durationToRemember) @@ -330,20 +340,14 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( private[streaming] object FileInputDStream { - /** - * Minimum duration of remembering the information of selected files. Files with mod times - * older than this "window" of remembering will be ignored. So if new files are visible - * within this window, then the file will get selected in the next batch. - */ - private val MIN_REMEMBER_DURATION = Minutes(1) - def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") /** * Calculate the number of last batches to remember, such that all the files selected in - * at least last MIN_REMEMBER_DURATION duration can be remembered. + * at least last minRememberDurationS duration can be remembered. */ - def calculateNumBatchesToRemember(batchDuration: Duration): Int = { - math.ceil(MIN_REMEMBER_DURATION.milliseconds.toDouble / batchDuration.milliseconds).toInt + def calculateNumBatchesToRemember(batchDuration: Duration, + minRememberDurationS: Duration): Int = { + math.ceil(minRememberDurationS.milliseconds.toDouble / batchDuration.milliseconds).toInt } } From 45c47fa4176ea75886a58f5d73c44afcb29aa629 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Tue, 21 Apr 2015 14:36:50 -0700 Subject: [PATCH 098/144] [SPARK-6845] [MLlib] [PySpark] Add isTranposed flag to DenseMatrix Since sparse matrices now support a isTransposed flag for row major data, DenseMatrices should do the same. Author: MechCoder Closes #5455 from MechCoder/spark-6845 and squashes the following commits: 525c370 [MechCoder] minor 004a37f [MechCoder] Cast boolean to int 151f3b6 [MechCoder] [WIP] Add isTransposed to pickle DenseMatrix cc0b90a [MechCoder] [SPARK-6845] Add isTranposed flag to DenseMatrix --- .../mllib/api/python/PythonMLLibAPI.scala | 13 +++-- python/pyspark/mllib/linalg.py | 49 +++++++++++++------ python/pyspark/mllib/tests.py | 16 ++++++ 3 files changed, 58 insertions(+), 20 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index f976d2f97b043..6237b64c8f984 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -985,8 +985,10 @@ private[spark] object SerDe extends Serializable { val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] val bytes = new Array[Byte](8 * m.values.size) val order = ByteOrder.nativeOrder() + val isTransposed = if (m.isTransposed) 1 else 0 ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + out.write(Opcodes.MARK) out.write(Opcodes.BININT) out.write(PickleUtils.integer_to_bytes(m.numRows)) out.write(Opcodes.BININT) @@ -994,19 +996,22 @@ private[spark] object SerDe extends Serializable { out.write(Opcodes.BINSTRING) out.write(PickleUtils.integer_to_bytes(bytes.length)) out.write(bytes) - out.write(Opcodes.TUPLE3) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) } def construct(args: Array[Object]): Object = { - if (args.length != 3) { - throw new PickleException("should be 3") + if (args.length != 4) { + throw new PickleException("should be 4") } val bytes = getBytes(args(2)) val n = bytes.length / 8 val values = new Array[Double](n) val order = ByteOrder.nativeOrder() ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) - new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values) + val isTransposed = args(3).asInstanceOf[Int] == 1 + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) } } diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index ec8c879ea9389..cc9a4cf8ba170 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -638,9 +638,10 @@ class Matrix(object): Represents a local matrix. """ - def __init__(self, numRows, numCols): + def __init__(self, numRows, numCols, isTransposed=False): self.numRows = numRows self.numCols = numCols + self.isTransposed = isTransposed def toArray(self): """ @@ -662,14 +663,16 @@ class DenseMatrix(Matrix): """ Column-major dense matrix. """ - def __init__(self, numRows, numCols, values): - Matrix.__init__(self, numRows, numCols) + def __init__(self, numRows, numCols, values, isTransposed=False): + Matrix.__init__(self, numRows, numCols, isTransposed) values = self._convert_to_array(values, np.float64) assert len(values) == numRows * numCols self.values = values def __reduce__(self): - return DenseMatrix, (self.numRows, self.numCols, self.values.tostring()) + return DenseMatrix, ( + self.numRows, self.numCols, self.values.tostring(), + int(self.isTransposed)) def toArray(self): """ @@ -680,15 +683,23 @@ def toArray(self): array([[ 0., 2.], [ 1., 3.]]) """ - return self.values.reshape((self.numRows, self.numCols), order='F') + if self.isTransposed: + return np.asfortranarray( + self.values.reshape((self.numRows, self.numCols))) + else: + return self.values.reshape((self.numRows, self.numCols), order='F') def toSparse(self): """Convert to SparseMatrix""" - indices = np.nonzero(self.values)[0] + if self.isTransposed: + values = np.ravel(self.toArray(), order='F') + else: + values = self.values + indices = np.nonzero(values)[0] colCounts = np.bincount(indices // self.numRows) colPtrs = np.cumsum(np.hstack( (0, colCounts, np.zeros(self.numCols - colCounts.size)))) - values = self.values[indices] + values = values[indices] rowIndices = indices % self.numRows return SparseMatrix(self.numRows, self.numCols, colPtrs, rowIndices, values) @@ -701,21 +712,28 @@ def __getitem__(self, indices): if j >= self.numCols or j < 0: raise ValueError("Column index %d is out of range [0, %d)" % (j, self.numCols)) - return self.values[i + j * self.numRows] + + if self.isTransposed: + return self.values[i * self.numCols + j] + else: + return self.values[i + j * self.numRows] def __eq__(self, other): - return (isinstance(other, DenseMatrix) and - self.numRows == other.numRows and - self.numCols == other.numCols and - all(self.values == other.values)) + if (not isinstance(other, DenseMatrix) or + self.numRows != other.numRows or + self.numCols != other.numCols): + return False + + self_values = np.ravel(self.toArray(), order='F') + other_values = np.ravel(other.toArray(), order='F') + return all(self_values == other_values) class SparseMatrix(Matrix): """Sparse Matrix stored in CSC format.""" def __init__(self, numRows, numCols, colPtrs, rowIndices, values, isTransposed=False): - Matrix.__init__(self, numRows, numCols) - self.isTransposed = isTransposed + Matrix.__init__(self, numRows, numCols, isTransposed) self.colPtrs = self._convert_to_array(colPtrs, np.int32) self.rowIndices = self._convert_to_array(rowIndices, np.int32) self.values = self._convert_to_array(values, np.float64) @@ -777,8 +795,7 @@ def toArray(self): return A def toDense(self): - densevals = np.reshape( - self.toArray(), (self.numRows * self.numCols), order='F') + densevals = np.ravel(self.toArray(), order='F') return DenseMatrix(self.numRows, self.numCols, densevals) # TODO: More efficient implementation: diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 849c88341a967..8f89e2cee0592 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -195,6 +195,22 @@ def test_sparse_matrix(self): self.assertEquals(expected[i][j], sm1t[i, j]) self.assertTrue(array_equal(sm1t.toArray(), expected)) + def test_dense_matrix_is_transposed(self): + mat1 = DenseMatrix(3, 2, [0, 4, 1, 6, 3, 9], isTransposed=True) + mat = DenseMatrix(3, 2, [0, 1, 3, 4, 6, 9]) + self.assertEquals(mat1, mat) + + expected = [[0, 4], [1, 6], [3, 9]] + for i in range(3): + for j in range(2): + self.assertEquals(mat1[i, j], expected[i][j]) + self.assertTrue(array_equal(mat1.toArray(), expected)) + + sm = mat1.toSparse() + self.assertTrue(array_equal(sm.rowIndices, [1, 2, 0, 1, 2])) + self.assertTrue(array_equal(sm.colPtrs, [0, 2, 5])) + self.assertTrue(array_equal(sm.values, [1, 3, 4, 6, 9])) + class ListTests(PySparkTestCase): From 04bf34e34f22e3d7e972fe755251774fc6a6d52e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 21 Apr 2015 14:43:46 -0700 Subject: [PATCH 099/144] [SPARK-7011] Build(compilation) fails with scala 2.11 option, because a protected[sql] type is accessed in ml package. [This](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala#L58) is where it is used and fails compilations at. Author: Prashant Sharma Closes #5593 from ScrapCodes/SPARK-7011/build-fix and squashes the following commits: e6d57a3 [Prashant Sharma] [SPARK-7011] Build fails with scala 2.11 option, because a protected[sql] type is accessed in ml package. --- .../src/main/scala/org/apache/spark/sql/types/dataTypes.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index c6fb22c26bd3c..a108413497829 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -299,7 +299,7 @@ class NullType private() extends DataType { case object NullType extends NullType -protected[sql] object NativeType { +protected[spark] object NativeType { val all = Seq( IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) @@ -327,7 +327,7 @@ protected[sql] object PrimitiveType { } } -protected[sql] abstract class NativeType extends DataType { +protected[spark] abstract class NativeType extends DataType { private[sql] type JvmType @transient private[sql] val tag: TypeTag[JvmType] private[sql] val ordering: Ordering[JvmType] From 2e8c6ca47df14681c1110f0736234ce76a3eca9b Mon Sep 17 00:00:00 2001 From: vidmantas zemleris Date: Tue, 21 Apr 2015 14:47:09 -0700 Subject: [PATCH 100/144] [SPARK-6994] Allow to fetch field values by name in sql.Row It looked weird that up to now there was no way in Spark's Scala API to access fields of `DataFrame/sql.Row` by name, only by their index. This tries to solve this issue. Author: vidmantas zemleris Closes #5573 from vidma/features/row-with-named-fields and squashes the following commits: 6145ae3 [vidmantas zemleris] [SPARK-6994][SQL] Allow to fetch field values by name on Row 9564ebb [vidmantas zemleris] [SPARK-6994][SQL] Add fieldIndex to schema (StructType) --- .../main/scala/org/apache/spark/sql/Row.scala | 32 +++++++++ .../spark/sql/catalyst/expressions/rows.scala | 2 + .../apache/spark/sql/types/dataTypes.scala | 9 +++ .../scala/org/apache/spark/sql/RowTest.scala | 71 +++++++++++++++++++ .../spark/sql/types/DataTypeSuite.scala | 13 ++++ .../scala/org/apache/spark/sql/RowSuite.scala | 10 +++ 6 files changed, 137 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index ac8a782976465..4190b7ffe1c8f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -306,6 +306,38 @@ trait Row extends Serializable { */ def getAs[T](i: Int): T = apply(i).asInstanceOf[T] + /** + * Returns the value of a given fieldName. + * + * @throws UnsupportedOperationException when schema is not defined. + * @throws IllegalArgumentException when fieldName do not exist. + * @throws ClassCastException when data type does not match. + */ + def getAs[T](fieldName: String): T = getAs[T](fieldIndex(fieldName)) + + /** + * Returns the index of a given field name. + * + * @throws UnsupportedOperationException when schema is not defined. + * @throws IllegalArgumentException when fieldName do not exist. + */ + def fieldIndex(name: String): Int = { + throw new UnsupportedOperationException("fieldIndex on a Row without schema is undefined.") + } + + /** + * Returns a Map(name -> value) for the requested fieldNames + * + * @throws UnsupportedOperationException when schema is not defined. + * @throws IllegalArgumentException when fieldName do not exist. + * @throws ClassCastException when data type does not match. + */ + def getValuesMap[T](fieldNames: Seq[String]): Map[String, T] = { + fieldNames.map { name => + name -> getAs[T](name) + }.toMap + } + override def toString(): String = s"[${this.mkString(",")}]" /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index b6ec7d3417ef8..981373477a4bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -181,6 +181,8 @@ class GenericRowWithSchema(values: Array[Any], override val schema: StructType) /** No-arg constructor for serialization. */ protected def this() = this(null, null) + + override def fieldIndex(name: String): Int = schema.fieldIndex(name) } class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index a108413497829..7cd7bd1914c95 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -1025,6 +1025,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru private lazy val fieldNamesSet: Set[String] = fieldNames.toSet private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap + private lazy val nameToIndex: Map[String, Int] = fieldNames.zipWithIndex.toMap /** * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not @@ -1049,6 +1050,14 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru StructType(fields.filter(f => names.contains(f.name))) } + /** + * Returns index of a given field + */ + def fieldIndex(name: String): Int = { + nameToIndex.getOrElse(name, + throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) + } + protected[sql] def toAttributes: Seq[AttributeReference] = map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala new file mode 100644 index 0000000000000..bbb9739e9cc76 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -0,0 +1,71 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema} +import org.apache.spark.sql.types._ +import org.scalatest.{Matchers, FunSpec} + +class RowTest extends FunSpec with Matchers { + + val schema = StructType( + StructField("col1", StringType) :: + StructField("col2", StringType) :: + StructField("col3", IntegerType) :: Nil) + val values = Array("value1", "value2", 1) + + val sampleRow: Row = new GenericRowWithSchema(values, schema) + val noSchemaRow: Row = new GenericRow(values) + + describe("Row (without schema)") { + it("throws an exception when accessing by fieldName") { + intercept[UnsupportedOperationException] { + noSchemaRow.fieldIndex("col1") + } + intercept[UnsupportedOperationException] { + noSchemaRow.getAs("col1") + } + } + } + + describe("Row (with schema)") { + it("fieldIndex(name) returns field index") { + sampleRow.fieldIndex("col1") shouldBe 0 + sampleRow.fieldIndex("col3") shouldBe 2 + } + + it("getAs[T] retrieves a value by fieldname") { + sampleRow.getAs[String]("col1") shouldBe "value1" + sampleRow.getAs[Int]("col3") shouldBe 1 + } + + it("Accessing non existent field throws an exception") { + intercept[IllegalArgumentException] { + sampleRow.getAs[String]("non_existent") + } + } + + it("getValuesMap() retrieves values of multiple fields as a Map(field -> value)") { + val expected = Map( + "col1" -> "value1", + "col2" -> "value2" + ) + sampleRow.getValuesMap(List("col1", "col2")) shouldBe expected + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index a1341ea13d810..d797510f36685 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -56,6 +56,19 @@ class DataTypeSuite extends FunSuite { } } + test("extract field index from a StructType") { + val struct = StructType( + StructField("a", LongType) :: + StructField("b", FloatType) :: Nil) + + assert(struct.fieldIndex("a") === 0) + assert(struct.fieldIndex("b") === 1) + + intercept[IllegalArgumentException] { + struct.fieldIndex("non_existent") + } + } + def checkDataTypeJsonRepr(dataType: DataType): Unit = { test(s"JSON - $dataType") { assert(DataType.fromJson(dataType.json) === dataType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index bf6cf1321a056..fb3ba4bc1b908 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -62,4 +62,14 @@ class RowSuite extends FunSuite { val de = instance.deserialize(ser).asInstanceOf[Row] assert(de === row) } + + test("get values by field name on Row created via .toDF") { + val row = Seq((1, Seq(1))).toDF("a", "b").first() + assert(row.getAs[Int]("a") === 1) + assert(row.getAs[Seq[Int]]("b") === Seq(1)) + + intercept[IllegalArgumentException]{ + row.getAs[Int]("c") + } + } } From 03fd92167107f1d061c1a7ef216468b508546ac7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 21 Apr 2015 14:48:02 -0700 Subject: [PATCH 101/144] [SQL][minor] make it more clear that we only need to re-throw GetField exception for UnresolvedAttribute For `GetField` outside `UnresolvedAttribute`, we will throw exception in `Analyzer`. Author: Wenchen Fan Closes #5588 from cloud-fan/tmp and squashes the following commits: 7ac74d2 [Wenchen Fan] small refactor --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 1155dac28fc78..a986dd5387c38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -46,12 +46,11 @@ trait CheckAnalysis { operator transformExpressionsUp { case a: Attribute if !a.resolved => if (operator.childrenResolved) { - val nameParts = a match { - case UnresolvedAttribute(nameParts) => nameParts - case _ => Seq(a.name) + a match { + case UnresolvedAttribute(nameParts) => + // Throw errors for specific problems with get field. + operator.resolveChildren(nameParts, resolver, throwErrors = true) } - // Throw errors for specific problems with get field. - operator.resolveChildren(nameParts, resolver, throwErrors = true) } val from = operator.inputSet.map(_.name).mkString(", ") From 6265cba00f6141575b4be825735d77d4cea500ab Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 21 Apr 2015 14:48:42 -0700 Subject: [PATCH 102/144] [SPARK-6969][SQL] Refresh the cached table when REFRESH TABLE is used https://issues.apache.org/jira/browse/SPARK-6969 Author: Yin Huai Closes #5583 from yhuai/refreshTableRefreshDataCache and squashes the following commits: 1e5142b [Yin Huai] Add todo. 92b2498 [Yin Huai] Minor updates. 367df92 [Yin Huai] Recache data in the command of REFRESH TABLE. --- .../org/apache/spark/sql/sources/ddl.scala | 17 +++++++ .../spark/sql/hive/CachedTableSuite.scala | 50 ++++++++++++++++++- 2 files changed, 66 insertions(+), 1 deletion(-) 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 2e861b84b7133..78d494184e759 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 @@ -347,7 +347,24 @@ private[sql] case class RefreshTable(databaseName: String, tableName: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { + // Refresh the given table's metadata first. sqlContext.catalog.refreshTable(databaseName, tableName) + + // If this table is cached as a InMemoryColumnarRelation, drop the original + // cached version and make the new version cached lazily. + val logicalPlan = sqlContext.catalog.lookupRelation(Seq(databaseName, tableName)) + // Use lookupCachedData directly since RefreshTable also takes databaseName. + val isCached = sqlContext.cacheManager.lookupCachedData(logicalPlan).nonEmpty + if (isCached) { + // Create a data frame to represent the table. + // TODO: Use uncacheTable once it supports database name. + val df = DataFrame(sqlContext, logicalPlan) + // Uncache the logicalPlan. + sqlContext.cacheManager.tryUncacheQuery(df, blocking = true) + // Cache it again. + sqlContext.cacheManager.cacheQuery(df, Some(tableName)) + } + Seq.empty[Row] } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index c188264072a84..fc6c3c35037b0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.hive +import java.io.File + import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} +import org.apache.spark.sql.{SaveMode, AnalysisException, DataFrame, QueryTest} import org.apache.spark.storage.RDDBlockId +import org.apache.spark.util.Utils class CachedTableSuite extends QueryTest { @@ -155,4 +158,49 @@ class CachedTableSuite extends QueryTest { assertCached(table("udfTest")) uncacheTable("udfTest") } + + test("REFRESH TABLE also needs to recache the data (data source tables)") { + val tempPath: File = Utils.createTempDir() + tempPath.delete() + table("src").save(tempPath.toString, "parquet", SaveMode.Overwrite) + sql("DROP TABLE IF EXISTS refreshTable") + createExternalTable("refreshTable", tempPath.toString, "parquet") + checkAnswer( + table("refreshTable"), + table("src").collect()) + // Cache the table. + sql("CACHE TABLE refreshTable") + assertCached(table("refreshTable")) + // Append new data. + table("src").save(tempPath.toString, "parquet", SaveMode.Append) + // We are still using the old data. + assertCached(table("refreshTable")) + checkAnswer( + table("refreshTable"), + table("src").collect()) + // Refresh the table. + sql("REFRESH TABLE refreshTable") + // We are using the new data. + assertCached(table("refreshTable")) + checkAnswer( + table("refreshTable"), + table("src").unionAll(table("src")).collect()) + + // Drop the table and create it again. + sql("DROP TABLE refreshTable") + createExternalTable("refreshTable", tempPath.toString, "parquet") + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + // Refresh the table. REFRESH TABLE command should not make a uncached + // table cached. + sql("REFRESH TABLE refreshTable") + checkAnswer( + table("refreshTable"), + table("src").unionAll(table("src")).collect()) + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + + sql("DROP TABLE refreshTable") + Utils.deleteRecursively(tempPath) + } } From 2a24bf92e6d36e876bad6a8b4e0ff12c407ebb8a Mon Sep 17 00:00:00 2001 From: Punya Biswal Date: Tue, 21 Apr 2015 14:50:02 -0700 Subject: [PATCH 103/144] [SPARK-6996][SQL] Support map types in java beans liancheng mengxr this is similar to #5146. Author: Punya Biswal Closes #5578 from punya/feature/SPARK-6996 and squashes the following commits: d56c3e0 [Punya Biswal] Fix imports c7e308b [Punya Biswal] Support java iterable types in POJOs 5e00685 [Punya Biswal] Support map types in java beans --- .../sql/catalyst/CatalystTypeConverters.scala | 20 ++++ .../apache/spark/sql/JavaTypeInference.scala | 110 ++++++++++++++++++ .../org/apache/spark/sql/SQLContext.scala | 52 +-------- .../apache/spark/sql/JavaDataFrameSuite.java | 57 +++++++-- 4 files changed, 180 insertions(+), 59 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/JavaTypeInference.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index d4f9fdacda4fb..a13e2f36a1a1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst +import java.lang.{Iterable => JavaIterable} import java.util.{Map => JavaMap} import scala.collection.mutable.HashMap @@ -49,6 +50,16 @@ object CatalystTypeConverters { case (s: Seq[_], arrayType: ArrayType) => s.map(convertToCatalyst(_, arrayType.elementType)) + case (jit: JavaIterable[_], arrayType: ArrayType) => { + val iter = jit.iterator + var listOfItems: List[Any] = List() + while (iter.hasNext) { + val item = iter.next() + listOfItems :+= convertToCatalyst(item, arrayType.elementType) + } + listOfItems + } + case (s: Array[_], arrayType: ArrayType) => s.toSeq.map(convertToCatalyst(_, arrayType.elementType)) @@ -124,6 +135,15 @@ object CatalystTypeConverters { extractOption(item) match { case a: Array[_] => a.toSeq.map(elementConverter) case s: Seq[_] => s.map(elementConverter) + case i: JavaIterable[_] => { + val iter = i.iterator + var convertedIterable: List[Any] = List() + while (iter.hasNext) { + val item = iter.next() + convertedIterable :+= elementConverter(item) + } + convertedIterable + } case null => null } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/JavaTypeInference.scala b/sql/core/src/main/scala/org/apache/spark/sql/JavaTypeInference.scala new file mode 100644 index 0000000000000..db484c5f50074 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/JavaTypeInference.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 + +import java.beans.Introspector +import java.lang.{Iterable => JIterable} +import java.util.{Iterator => JIterator, Map => JMap} + +import com.google.common.reflect.TypeToken + +import org.apache.spark.sql.types._ + +import scala.language.existentials + +/** + * Type-inference utilities for POJOs and Java collections. + */ +private [sql] object JavaTypeInference { + + private val iterableType = TypeToken.of(classOf[JIterable[_]]) + private val mapType = TypeToken.of(classOf[JMap[_, _]]) + private val iteratorReturnType = classOf[JIterable[_]].getMethod("iterator").getGenericReturnType + private val nextReturnType = classOf[JIterator[_]].getMethod("next").getGenericReturnType + private val keySetReturnType = classOf[JMap[_, _]].getMethod("keySet").getGenericReturnType + private val valuesReturnType = classOf[JMap[_, _]].getMethod("values").getGenericReturnType + + /** + * Infers the corresponding SQL data type of a Java type. + * @param typeToken Java type + * @return (SQL data type, nullable) + */ + private [sql] def inferDataType(typeToken: TypeToken[_]): (DataType, Boolean) = { + // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. + typeToken.getRawType match { + case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => + (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) + + case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + + case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) + case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) + + case _ if typeToken.isArray => + val (dataType, nullable) = inferDataType(typeToken.getComponentType) + (ArrayType(dataType, nullable), true) + + case _ if iterableType.isAssignableFrom(typeToken) => + val (dataType, nullable) = inferDataType(elementType(typeToken)) + (ArrayType(dataType, nullable), true) + + case _ if mapType.isAssignableFrom(typeToken) => + val typeToken2 = typeToken.asInstanceOf[TypeToken[_ <: JMap[_, _]]] + val mapSupertype = typeToken2.getSupertype(classOf[JMap[_, _]]) + val keyType = elementType(mapSupertype.resolveType(keySetReturnType)) + val valueType = elementType(mapSupertype.resolveType(valuesReturnType)) + val (keyDataType, _) = inferDataType(keyType) + val (valueDataType, nullable) = inferDataType(valueType) + (MapType(keyDataType, valueDataType, nullable), true) + + case _ => + val beanInfo = Introspector.getBeanInfo(typeToken.getRawType) + val properties = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") + val fields = properties.map { property => + val returnType = typeToken.method(property.getReadMethod).getReturnType + val (dataType, nullable) = inferDataType(returnType) + new StructField(property.getName, dataType, nullable) + } + (new StructType(fields), true) + } + } + + private def elementType(typeToken: TypeToken[_]): TypeToken[_] = { + val typeToken2 = typeToken.asInstanceOf[TypeToken[_ <: JIterable[_]]] + val iterableSupertype = typeToken2.getSupertype(classOf[JIterable[_]]) + val iteratorType = iterableSupertype.resolveType(iteratorReturnType) + val itemType = iteratorType.resolveType(nextReturnType) + itemType + } +} 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 f9f3eb2e03817..bcd20c06c6dca 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 @@ -25,6 +25,8 @@ import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag +import com.google.common.reflect.TypeToken + import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDD @@ -1222,56 +1224,12 @@ class SQLContext(@transient val sparkContext: SparkContext) * Returns a Catalyst Schema for the given java bean class. */ protected def getSchema(beanClass: Class[_]): Seq[AttributeReference] = { - val (dataType, _) = inferDataType(beanClass) + val (dataType, _) = JavaTypeInference.inferDataType(TypeToken.of(beanClass)) dataType.asInstanceOf[StructType].fields.map { f => AttributeReference(f.name, f.dataType, f.nullable)() } } - /** - * Infers the corresponding SQL data type of a Java class. - * @param clazz Java class - * @return (SQL data type, nullable) - */ - private def inferDataType(clazz: Class[_]): (DataType, Boolean) = { - // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. - clazz match { - case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => - (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) - - case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) - case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) - case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) - case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) - case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) - case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) - case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) - case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) - - case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) - case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) - case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) - case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) - case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) - case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) - case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) - - case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) - case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) - case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) - - case c: Class[_] if c.isArray => - val (dataType, nullable) = inferDataType(c.getComponentType) - (ArrayType(dataType, nullable), true) - - case _ => - val beanInfo = Introspector.getBeanInfo(clazz) - val properties = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") - val fields = properties.map { property => - val (dataType, nullable) = inferDataType(property.getPropertyType) - new StructField(property.getName, dataType, nullable) - } - (new StructType(fields), true) - } - } } + + diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 6d0fbe83c2f36..fc3ed4a708d46 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -17,23 +17,28 @@ package test.org.apache.spark.sql; -import java.io.Serializable; -import java.util.Arrays; - -import scala.collection.Seq; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import com.google.common.collect.ImmutableMap; +import com.google.common.primitives.Ints; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.*; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.TestData$; import org.apache.spark.sql.test.TestSQLContext; import org.apache.spark.sql.test.TestSQLContext$; import org.apache.spark.sql.types.*; +import org.junit.*; + +import scala.collection.JavaConversions; +import scala.collection.Seq; +import scala.collection.mutable.Buffer; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import static org.apache.spark.sql.functions.*; @@ -106,6 +111,8 @@ public void testShow() { public static class Bean implements Serializable { private double a = 0.0; private Integer[] b = new Integer[]{0, 1}; + private Map c = ImmutableMap.of("hello", new int[] { 1, 2 }); + private List d = Arrays.asList("floppy", "disk"); public double getA() { return a; @@ -114,6 +121,14 @@ public double getA() { public Integer[] getB() { return b; } + + public Map getC() { + return c; + } + + public List getD() { + return d; + } } @Test @@ -127,7 +142,15 @@ public void testCreateDataFrameFromJavaBeans() { Assert.assertEquals( new StructField("b", new ArrayType(IntegerType$.MODULE$, true), true, Metadata.empty()), schema.apply("b")); - Row first = df.select("a", "b").first(); + ArrayType valueType = new ArrayType(DataTypes.IntegerType, false); + MapType mapType = new MapType(DataTypes.StringType, valueType, true); + Assert.assertEquals( + new StructField("c", mapType, true, Metadata.empty()), + schema.apply("c")); + Assert.assertEquals( + new StructField("d", new ArrayType(DataTypes.StringType, true), true, Metadata.empty()), + schema.apply("d")); + Row first = df.select("a", "b", "c", "d").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); // Now Java lists and maps are converetd to Scala Seq's and Map's. Once we get a Seq below, // verify that it has the expected length, and contains expected elements. @@ -136,5 +159,15 @@ public void testCreateDataFrameFromJavaBeans() { for (int i = 0; i < result.length(); i++) { Assert.assertEquals(bean.getB()[i], result.apply(i)); } + Buffer outputBuffer = (Buffer) first.getJavaMap(2).get("hello"); + Assert.assertArrayEquals( + bean.getC().get("hello"), + Ints.toArray(JavaConversions.asJavaList(outputBuffer))); + Seq d = first.getAs(3); + Assert.assertEquals(bean.getD().size(), d.length()); + for (int i = 0; i < d.length(); i++) { + Assert.assertEquals(bean.getD().get(i), d.apply(i)); + } } + } From 7662ec23bb6c4d4fe4c857b6928eaed0a97d3c04 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 21 Apr 2015 15:11:15 -0700 Subject: [PATCH 104/144] [SPARK-5817] [SQL] Fix bug of udtf with column names It's a bug while do query like: ```sql select d from (select explode(array(1,1)) d from src limit 1) t ``` And it will throws exception like: ``` org.apache.spark.sql.AnalysisException: cannot resolve 'd' given input columns _c0; line 1 pos 7 at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:48) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:45) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:50) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:249) at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$transformExpressionUp$1(QueryPlan.scala:103) at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2$$anonfun$apply$2.apply(QueryPlan.scala:117) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2.apply(QueryPlan.scala:116) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) ``` To solve the bug, it requires code refactoring for UDTF The major changes are about: * Simplifying the UDTF development, UDTF will manage the output attribute names any more, instead, the `logical.Generate` will handle that properly. * UDTF will be asked for the output schema (data types) during the logical plan analyzing. Author: Cheng Hao Closes #4602 from chenghao-intel/explode_bug and squashes the following commits: c2a5132 [Cheng Hao] add back resolved for Alias 556e982 [Cheng Hao] revert the unncessary change 002c361 [Cheng Hao] change the rule of resolved for Generate 04ae500 [Cheng Hao] add qualifier only for generator output 5ee5d2c [Cheng Hao] prepend the new qualifier d2e8b43 [Cheng Hao] Update the code as feedback ca5e7f4 [Cheng Hao] shrink the commits --- .../sql/catalyst/analysis/Analyzer.scala | 57 ++++++++++++++++++- .../sql/catalyst/analysis/CheckAnalysis.scala | 12 ++++ .../spark/sql/catalyst/dsl/package.scala | 3 +- .../sql/catalyst/expressions/generators.scala | 49 ++++------------ .../expressions/namedExpressions.scala | 2 + .../sql/catalyst/optimizer/Optimizer.scala | 8 +-- .../plans/logical/basicOperators.scala | 37 +++++++----- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../optimizer/FilterPushdownSuite.scala | 8 +-- .../org/apache/spark/sql/DataFrame.scala | 21 +++++-- .../apache/spark/sql/execution/Generate.scala | 22 ++----- .../spark/sql/execution/SparkStrategies.scala | 5 +- .../apache/spark/sql/hive/HiveContext.scala | 1 - .../org/apache/spark/sql/hive/HiveQl.scala | 37 ++++++------ .../org/apache/spark/sql/hive/hiveUdfs.scala | 38 +------------ ... output-0-d1f244bce64f22b34ad5bf9fd360b632 | 1 + ...mn name-0-7ac701cf43e73e9e416888e4df694348 | 0 ...mn name-1-5cdf9d51fc0e105e365d82e7611e37f3 | 0 ...mn name-2-f963396461294e06cb7cafe22a1419e4 | 3 + ...n names-0-46bdb27b3359dc81d8c246b9f69d4b82 | 0 ...n names-1-cdf6989f3b055257f1692c3bbd80dc73 | 0 ...n names-2-ab3954b69d7a991bc801a509c3166cc5 | 3 + ...mn name-0-7ac701cf43e73e9e416888e4df694348 | 0 ...mn name-1-26599718c322ff4f9740040c066d8292 | 0 ...mn name-2-f963396461294e06cb7cafe22a1419e4 | 3 + .../sql/hive/execution/HiveQuerySuite.scala | 40 ++++++++++++- 26 files changed, 207 insertions(+), 145 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/Specify the udtf output-0-d1f244bce64f22b34ad5bf9fd360b632 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator with column name-0-7ac701cf43e73e9e416888e4df694348 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator with column name-1-5cdf9d51fc0e105e365d82e7611e37f3 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator with column name-2-f963396461294e06cb7cafe22a1419e4 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator with multiple column names-0-46bdb27b3359dc81d8c246b9f69d4b82 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator with multiple column names-1-cdf6989f3b055257f1692c3bbd80dc73 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator with multiple column names-2-ab3954b69d7a991bc801a509c3166cc5 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator without column name-0-7ac701cf43e73e9e416888e4df694348 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator without column name-1-26599718c322ff4f9740040c066d8292 create mode 100644 sql/hive/src/test/resources/golden/insert table with generator without column name-2-f963396461294e06cb7cafe22a1419e4 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 cb49e5ad5586f..5e42b409dcc59 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.sql.AnalysisException -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._ @@ -59,6 +58,7 @@ class Analyzer( ResolveReferences :: ResolveGroupingAnalytics :: ResolveSortReferences :: + ResolveGenerate :: ImplicitGenerate :: ResolveFunctions :: GlobalAggregates :: @@ -474,8 +474,59 @@ class Analyzer( */ object ImplicitGenerate extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case Project(Seq(Alias(g: Generator, _)), child) => - Generate(g, join = false, outer = false, None, child) + case Project(Seq(Alias(g: Generator, name)), child) => + Generate(g, join = false, outer = false, + qualifier = None, UnresolvedAttribute(name) :: Nil, child) + case Project(Seq(MultiAlias(g: Generator, names)), child) => + Generate(g, join = false, outer = false, + qualifier = None, names.map(UnresolvedAttribute(_)), child) + } + } + + /** + * Resolve the Generate, if the output names specified, we will take them, otherwise + * we will try to provide the default names, which follow the same rule with Hive. + */ + object ResolveGenerate extends Rule[LogicalPlan] { + // Construct the output attributes for the generator, + // The output attribute names can be either specified or + // auto generated. + private def makeGeneratorOutput( + generator: Generator, + generatorOutput: Seq[Attribute]): Seq[Attribute] = { + val elementTypes = generator.elementTypes + + if (generatorOutput.length == elementTypes.length) { + generatorOutput.zip(elementTypes).map { + case (a, (t, nullable)) if !a.resolved => + AttributeReference(a.name, t, nullable)() + case (a, _) => a + } + } else if (generatorOutput.length == 0) { + elementTypes.zipWithIndex.map { + // keep the default column names as Hive does _c0, _c1, _cN + case ((t, nullable), i) => AttributeReference(s"_c$i", t, nullable)() + } + } else { + throw new AnalysisException( + s""" + |The number of aliases supplied in the AS clause does not match + |the number of columns output by the UDTF expected + |${elementTypes.size} aliases but got ${generatorOutput.size} + """.stripMargin) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case p: Generate if !p.child.resolved || !p.generator.resolved => p + case p: Generate if p.resolved == false => + // if the generator output names are not specified, we will use the default ones. + Generate( + p.generator, + join = p.join, + outer = p.outer, + p.qualifier, + makeGeneratorOutput(p.generator, p.generatorOutput), p.child) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index a986dd5387c38..2381689e17525 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -38,6 +38,12 @@ trait CheckAnalysis { throw new AnalysisException(msg) } + def containsMultipleGenerators(exprs: Seq[Expression]): Boolean = { + exprs.flatMap(_.collect { + case e: Generator => true + }).length >= 1 + } + def checkAnalysis(plan: LogicalPlan): Unit = { // We transform up and order the rules so as to catch the first possible failure instead // of the result of cascading resolution failures. @@ -110,6 +116,12 @@ trait CheckAnalysis { failAnalysis( s"unresolved operator ${operator.simpleString}") + case p @ Project(exprs, _) if containsMultipleGenerators(exprs) => + failAnalysis( + s"""Only a single table generating function is allowed in a SELECT clause, found: + | ${exprs.map(_.prettyString).mkString(",")}""".stripMargin) + + case _ => // Analysis successful! } } 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 21c15ad14fd19..4e5c64bb63c9f 100644 --- 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 @@ -284,12 +284,13 @@ package object dsl { seed: Int = (math.random * 1000).toInt): LogicalPlan = Sample(fraction, withReplacement, seed, logicalPlan) + // TODO specify the output column names def generate( generator: Generator, join: Boolean = false, outer: Boolean = false, alias: Option[String] = None): LogicalPlan = - Generate(generator, join, outer, None, logicalPlan) + Generate(generator, join = join, outer = outer, alias, Nil, logicalPlan) def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan = InsertIntoTable( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 67caadb839ff9..9a6cb048af5ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -42,47 +42,30 @@ abstract class Generator extends Expression { override type EvaluatedType = TraversableOnce[Row] - override lazy val dataType = - ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))) + // TODO ideally we should return the type of ArrayType(StructType), + // however, we don't keep the output field names in the Generator. + override def dataType: DataType = throw new UnsupportedOperationException override def nullable: Boolean = false /** - * Should be overridden by specific generators. Called only once for each instance to ensure - * that rule application does not change the output schema of a generator. + * The output element data types in structure of Seq[(DataType, Nullable)] + * TODO we probably need to add more information like metadata etc. */ - protected def makeOutput(): Seq[Attribute] - - private var _output: Seq[Attribute] = null - - def output: Seq[Attribute] = { - if (_output == null) { - _output = makeOutput() - } - _output - } + def elementTypes: Seq[(DataType, Boolean)] /** Should be implemented by child classes to perform specific Generators. */ override def eval(input: Row): TraversableOnce[Row] - - /** Overridden `makeCopy` also copies the attributes that are produced by this generator. */ - override def makeCopy(newArgs: Array[AnyRef]): this.type = { - val copy = super.makeCopy(newArgs) - copy._output = _output - copy - } } /** * A generator that produces its output using the provided lambda function. */ case class UserDefinedGenerator( - schema: Seq[Attribute], + elementTypes: Seq[(DataType, Boolean)], function: Row => TraversableOnce[Row], children: Seq[Expression]) - extends Generator{ - - override protected def makeOutput(): Seq[Attribute] = schema + extends Generator { override def eval(input: Row): TraversableOnce[Row] = { // TODO(davies): improve this @@ -98,30 +81,18 @@ case class UserDefinedGenerator( /** * Given an input array produces a sequence of rows for each value in the array. */ -case class Explode(attributeNames: Seq[String], child: Expression) +case class Explode(child: Expression) extends Generator with trees.UnaryNode[Expression] { override lazy val resolved = child.resolved && (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - private lazy val elementTypes = child.dataType match { + override def elementTypes: Seq[(DataType, Boolean)] = child.dataType match { case ArrayType(et, containsNull) => (et, containsNull) :: Nil case MapType(kt, vt, valueContainsNull) => (kt, false) :: (vt, valueContainsNull) :: Nil } - // TODO: Move this pattern into Generator. - protected def makeOutput() = - if (attributeNames.size == elementTypes.size) { - attributeNames.zip(elementTypes).map { - case (n, (t, nullable)) => AttributeReference(n, t, nullable)() - } - } else { - elementTypes.zipWithIndex.map { - case ((t, nullable), i) => AttributeReference(s"c_$i", t, nullable)() - } - } - override def eval(input: Row): TraversableOnce[Row] = { child.dataType match { case ArrayType(_, _) => 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 bcbcbeb31c7b5..afcb2ce8b9cb4 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 @@ -112,6 +112,8 @@ case class Alias(child: Expression, name: String)( extends NamedExpression with trees.UnaryNode[Expression] { override type EvaluatedType = Any + // Alias(Generator, xx) need to be transformed into Generate(generator, ...) + override lazy val resolved = childrenResolved && !child.isInstanceOf[Generator] override def eval(input: Row): Any = child.eval(input) 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 7c80634d2c852..2d03fbfb0d311 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 @@ -482,16 +482,16 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { object PushPredicateThroughGenerate extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case filter @ Filter(condition, - generate @ Generate(generator, join, outer, alias, grandChild)) => + case filter @ Filter(condition, g: Generate) => // Predicates that reference attributes produced by the `Generate` operator cannot // be pushed below the operator. val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { - conjunct => conjunct.references subsetOf grandChild.outputSet + conjunct => conjunct.references subsetOf g.child.outputSet } if (pushDown.nonEmpty) { val pushDownPredicate = pushDown.reduce(And) - val withPushdown = generate.copy(child = Filter(pushDownPredicate, grandChild)) + val withPushdown = Generate(g.generator, join = g.join, outer = g.outer, + g.qualifier, g.generatorOutput, Filter(pushDownPredicate, g.child)) stayUp.reduceOption(And).map(Filter(_, withPushdown)).getOrElse(withPushdown) } else { filter 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 17522976dc2c9..bbc94a7ab3398 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 @@ -40,34 +40,43 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional * programming with one important additional feature, which allows the input rows to be joined with * their output. + * @param generator the generator expression * @param join when true, each output row is implicitly joined with the input tuple that produced * it. * @param outer when true, each input row will be output at least once, even if the output of the * given `generator` is empty. `outer` has no effect when `join` is false. - * @param alias when set, this string is applied to the schema of the output of the transformation - * as a qualifier. + * @param qualifier Qualifier for the attributes of generator(UDTF) + * @param generatorOutput The output schema of the Generator. + * @param child Children logical plan node */ case class Generate( generator: Generator, join: Boolean, outer: Boolean, - alias: Option[String], + qualifier: Option[String], + generatorOutput: Seq[Attribute], child: LogicalPlan) extends UnaryNode { - protected def generatorOutput: Seq[Attribute] = { - val output = alias - .map(a => generator.output.map(_.withQualifiers(a :: Nil))) - .getOrElse(generator.output) - if (join && outer) { - output.map(_.withNullability(true)) - } else { - output - } + override lazy val resolved: Boolean = { + generator.resolved && + childrenResolved && + generator.elementTypes.length == generatorOutput.length && + !generatorOutput.exists(!_.resolved) } - override def output: Seq[Attribute] = - if (join) child.output ++ generatorOutput else generatorOutput + // we don't want the gOutput to be taken as part of the expressions + // as that will cause exceptions like unresolved attributes etc. + override def expressions: Seq[Expression] = generator :: Nil + + def output: Seq[Attribute] = { + val qualified = qualifier.map(q => + // prepend the new qualifier to the existed one + generatorOutput.map(a => a.withQualifiers(q +: a.qualifiers)) + ).getOrElse(generatorOutput) + + if (join) child.output ++ qualified else qualified + } } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { 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 e10ddfdf5127c..7c249215bd6b6 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 @@ -90,7 +90,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(!Project(Seq(UnresolvedAttribute("a")), testRelation).resolved) - val explode = Explode(Nil, AttributeReference("a", IntegerType, nullable = true)()) + val explode = Explode(AttributeReference("a", IntegerType, nullable = true)()) assert(!Project(Seq(Alias(explode, "explode")()), testRelation).resolved) assert(!Project(Seq(Alias(Count(Literal(1)), "count")()), testRelation).resolved) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 1448098c770aa..45cf695d20b01 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -454,21 +454,21 @@ class FilterPushdownSuite extends PlanTest { test("generate: predicate referenced no generated column") { val originalQuery = { testRelationWithArrayType - .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")) + .generate(Explode('c_arr), true, false, Some("arr")) .where(('b >= 5) && ('a > 6)) } val optimized = Optimize(originalQuery.analyze) val correctAnswer = { testRelationWithArrayType .where(('b >= 5) && ('a > 6)) - .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")).analyze + .generate(Explode('c_arr), true, false, Some("arr")).analyze } comparePlans(optimized, correctAnswer) } test("generate: part of conjuncts referenced generated column") { - val generator = Explode(Seq("c"), 'c_arr) + val generator = Explode('c_arr) val originalQuery = { testRelationWithArrayType .generate(generator, true, false, Some("arr")) @@ -499,7 +499,7 @@ class FilterPushdownSuite extends PlanTest { test("generate: all conjuncts referenced generated column") { val originalQuery = { testRelationWithArrayType - .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")) + .generate(Explode('c_arr), true, false, Some("arr")) .where(('c > 6) || ('b > 5)).analyze } val optimized = Optimize(originalQuery) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 45f5da387692e..03d9834d1d131 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -34,7 +34,7 @@ import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, ResolvedStar} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, ResolvedStar} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ @@ -711,12 +711,16 @@ class DataFrame private[sql]( */ def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] - val attributes = schema.toAttributes + + val elementTypes = schema.toAttributes.map { attr => (attr.dataType, attr.nullable) } + val names = schema.toAttributes.map(_.name) + val rowFunction = f.andThen(_.map(CatalystTypeConverters.convertToCatalyst(_, schema).asInstanceOf[Row])) - val generator = UserDefinedGenerator(attributes, rowFunction, input.map(_.expr)) + val generator = UserDefinedGenerator(elementTypes, rowFunction, input.map(_.expr)) - Generate(generator, join = true, outer = false, None, logicalPlan) + Generate(generator, join = true, outer = false, + qualifier = None, names.map(UnresolvedAttribute(_)), logicalPlan) } /** @@ -733,12 +737,17 @@ class DataFrame private[sql]( : DataFrame = { val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil + // TODO handle the metadata? + val elementTypes = attributes.map { attr => (attr.dataType, attr.nullable) } + val names = attributes.map(_.name) + def rowFunction(row: Row): TraversableOnce[Row] = { f(row(0).asInstanceOf[A]).map(o => Row(CatalystTypeConverters.convertToCatalyst(o, dataType))) } - val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) + val generator = UserDefinedGenerator(elementTypes, rowFunction, apply(inputColumn).expr :: Nil) - Generate(generator, join = true, outer = false, None, logicalPlan) + Generate(generator, join = true, outer = false, + qualifier = None, names.map(UnresolvedAttribute(_)), logicalPlan) } ///////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 12271048bb39c..5201e20a10565 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -27,44 +27,34 @@ import org.apache.spark.sql.catalyst.expressions._ * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional * programming with one important additional feature, which allows the input rows to be joined with * their output. + * @param generator the generator expression * @param join when true, each output row is implicitly joined with the input tuple that produced * it. * @param outer when true, each input row will be output at least once, even if the output of the * given `generator` is empty. `outer` has no effect when `join` is false. + * @param output the output attributes of this node, which constructed in analysis phase, + * and we can not change it, as the parent node bound with it already. */ @DeveloperApi case class Generate( generator: Generator, join: Boolean, outer: Boolean, + output: Seq[Attribute], child: SparkPlan) extends UnaryNode { - // This must be a val since the generator output expr ids are not preserved by serialization. - protected val generatorOutput: Seq[Attribute] = { - if (join && outer) { - generator.output.map(_.withNullability(true)) - } else { - generator.output - } - } - - // This must be a val since the generator output expr ids are not preserved by serialization. - override val output = - if (join) child.output ++ generatorOutput else generatorOutput - val boundGenerator = BindReferences.bindReference(generator, child.output) override def execute(): RDD[Row] = { if (join) { child.execute().mapPartitions { iter => - val nullValues = Seq.fill(generator.output.size)(Literal(null)) + val nullValues = Seq.fill(generator.elementTypes.size)(Literal(null)) // Used to produce rows with no matches when outer = true. val outerProjection = newProjection(child.output ++ nullValues, child.output) - val joinProjection = - newProjection(child.output ++ generatorOutput, child.output ++ generatorOutput) + val joinProjection = newProjection(output, output) val joinedRow = new JoinedRow iter.flatMap {row => 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 e687d01f57520..030ef118f75d4 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 @@ -312,8 +312,9 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Except(planLater(left), planLater(right)) :: Nil case logical.Intersect(left, right) => execution.Intersect(planLater(left), planLater(right)) :: Nil - case logical.Generate(generator, join, outer, _, child) => - execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil + case g @ logical.Generate(generator, join, outer, _, _, child) => + execution.Generate( + generator, join = join, outer = outer, g.output, planLater(child)) :: Nil case logical.OneRowRelation => execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => 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 7c6a7df2bd01e..c4a73b3004076 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 @@ -249,7 +249,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: - ResolveUdtfsAlias :: sources.PreInsertCastAndRename :: Nil } 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 fd305eb480e63..85061f22772dd 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 @@ -725,12 +725,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val alias = getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText - Generate( - nodesToGenerator(clauses), - join = true, - outer = false, - Some(alias.toLowerCase), - withWhere) + val (generator, attributes) = nodesToGenerator(clauses) + Generate( + generator, + join = true, + outer = false, + Some(alias.toLowerCase), + attributes.map(UnresolvedAttribute(_)), + withWhere) }.getOrElse(withWhere) // The projection of the query can either be a normal projection, an aggregation @@ -833,12 +835,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val alias = getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText - Generate( - nodesToGenerator(clauses), - join = true, - outer = isOuter.nonEmpty, - Some(alias.toLowerCase), - nodeToRelation(relationClause)) + val (generator, attributes) = nodesToGenerator(clauses) + Generate( + generator, + join = true, + outer = isOuter.nonEmpty, + Some(alias.toLowerCase), + attributes.map(UnresolvedAttribute(_)), + nodeToRelation(relationClause)) /* All relations, possibly with aliases or sampling clauses. */ case Token("TOK_TABREF", clauses) => @@ -1311,7 +1315,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val explode = "(?i)explode".r - def nodesToGenerator(nodes: Seq[Node]): Generator = { + def nodesToGenerator(nodes: Seq[Node]): (Generator, Seq[String]) = { val function = nodes.head val attributes = nodes.flatMap { @@ -1321,7 +1325,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C function match { case Token("TOK_FUNCTION", Token(explode(), Nil) :: child :: Nil) => - Explode(attributes, nodeToExpr(child)) + (Explode(nodeToExpr(child)), attributes) case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => val functionInfo: FunctionInfo = @@ -1329,10 +1333,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C sys.error(s"Couldn't find function $functionName")) val functionClassName = functionInfo.getFunctionClass.getName - HiveGenericUdtf( + (HiveGenericUdtf( new HiveFunctionWrapper(functionClassName), - attributes, - children.map(nodeToExpr)) + children.map(nodeToExpr)), attributes) case a: ASTNode => throw new NotImplementedError( 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 47305571e579e..4b6f0ad75f54f 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 @@ -66,7 +66,7 @@ private[hive] abstract class HiveFunctionRegistry } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveUdaf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), Nil, children) + HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } @@ -266,7 +266,6 @@ private[hive] case class HiveUdaf( */ private[hive] case class HiveGenericUdtf( funcWrapper: HiveFunctionWrapper, - aliasNames: Seq[String], children: Seq[Expression]) extends Generator with HiveInspectors { @@ -282,23 +281,8 @@ private[hive] case class HiveGenericUdtf( @transient protected lazy val udtInput = new Array[AnyRef](children.length) - protected lazy val outputDataTypes = outputInspector.getAllStructFieldRefs.map { - field => inspectorToDataType(field.getFieldObjectInspector) - } - - override protected def makeOutput() = { - // Use column names when given, otherwise _c1, _c2, ... _cn. - if (aliasNames.size == outputDataTypes.size) { - aliasNames.zip(outputDataTypes).map { - case (attrName, attrDataType) => - AttributeReference(attrName, attrDataType, nullable = true)() - } - } else { - outputDataTypes.zipWithIndex.map { - case (attrDataType, i) => - AttributeReference(s"_c$i", attrDataType, nullable = true)() - } - } + lazy val elementTypes = outputInspector.getAllStructFieldRefs.map { + field => (inspectorToDataType(field.getFieldObjectInspector), true) } override def eval(input: Row): TraversableOnce[Row] = { @@ -333,22 +317,6 @@ private[hive] case class HiveGenericUdtf( } } -/** - * Resolve Udtfs Alias. - */ -private[spark] object ResolveUdtfsAlias extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p @ Project(projectList, _) - if projectList.exists(_.isInstanceOf[MultiAlias]) && projectList.size != 1 => - throw new TreeNodeException(p, "only single Generator supported for SELECT clause") - - case Project(Seq(Alias(udtf @ HiveGenericUdtf(_, _, _), name)), child) => - Generate(udtf.copy(aliasNames = Seq(name)), join = false, outer = false, None, child) - case Project(Seq(MultiAlias(udtf @ HiveGenericUdtf(_, _, _), names)), child) => - Generate(udtf.copy(aliasNames = names), join = false, outer = false, None, child) - } -} - private[hive] case class HiveUdafFunction( funcWrapper: HiveFunctionWrapper, exprs: Seq[Expression], diff --git a/sql/hive/src/test/resources/golden/Specify the udtf output-0-d1f244bce64f22b34ad5bf9fd360b632 b/sql/hive/src/test/resources/golden/Specify the udtf output-0-d1f244bce64f22b34ad5bf9fd360b632 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/Specify the udtf output-0-d1f244bce64f22b34ad5bf9fd360b632 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/insert table with generator with column name-0-7ac701cf43e73e9e416888e4df694348 b/sql/hive/src/test/resources/golden/insert table with generator with column name-0-7ac701cf43e73e9e416888e4df694348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with column name-1-5cdf9d51fc0e105e365d82e7611e37f3 b/sql/hive/src/test/resources/golden/insert table with generator with column name-1-5cdf9d51fc0e105e365d82e7611e37f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with column name-2-f963396461294e06cb7cafe22a1419e4 b/sql/hive/src/test/resources/golden/insert table with generator with column name-2-f963396461294e06cb7cafe22a1419e4 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert table with generator with column name-2-f963396461294e06cb7cafe22a1419e4 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-0-46bdb27b3359dc81d8c246b9f69d4b82 b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-0-46bdb27b3359dc81d8c246b9f69d4b82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-1-cdf6989f3b055257f1692c3bbd80dc73 b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-1-cdf6989f3b055257f1692c3bbd80dc73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-2-ab3954b69d7a991bc801a509c3166cc5 b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-2-ab3954b69d7a991bc801a509c3166cc5 new file mode 100644 index 0000000000000..0c7520f2090dd --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-2-ab3954b69d7a991bc801a509c3166cc5 @@ -0,0 +1,3 @@ +86 val_86 +238 val_238 +311 val_311 diff --git a/sql/hive/src/test/resources/golden/insert table with generator without column name-0-7ac701cf43e73e9e416888e4df694348 b/sql/hive/src/test/resources/golden/insert table with generator without column name-0-7ac701cf43e73e9e416888e4df694348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator without column name-1-26599718c322ff4f9740040c066d8292 b/sql/hive/src/test/resources/golden/insert table with generator without column name-1-26599718c322ff4f9740040c066d8292 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator without column name-2-f963396461294e06cb7cafe22a1419e4 b/sql/hive/src/test/resources/golden/insert table with generator without column name-2-f963396461294e06cb7cafe22a1419e4 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert table with generator without column name-2-f963396461294e06cb7cafe22a1419e4 @@ -0,0 +1,3 @@ +1 +2 +3 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 300b1f7920473..ac10b173307d8 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 @@ -27,7 +27,7 @@ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.spark.{SparkFiles, SparkException} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ @@ -67,6 +67,40 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } + createQueryTest("insert table with generator with column name", + """ + | CREATE TABLE gen_tmp (key Int); + | INSERT OVERWRITE TABLE gen_tmp + | SELECT explode(array(1,2,3)) AS val FROM src LIMIT 3; + | SELECT key FROM gen_tmp ORDER BY key ASC; + """.stripMargin) + + createQueryTest("insert table with generator with multiple column names", + """ + | CREATE TABLE gen_tmp (key Int, value String); + | INSERT OVERWRITE TABLE gen_tmp + | SELECT explode(map(key, value)) as (k1, k2) FROM src LIMIT 3; + | SELECT key, value FROM gen_tmp ORDER BY key, value ASC; + """.stripMargin) + + createQueryTest("insert table with generator without column name", + """ + | CREATE TABLE gen_tmp (key Int); + | INSERT OVERWRITE TABLE gen_tmp + | SELECT explode(array(1,2,3)) FROM src LIMIT 3; + | SELECT key FROM gen_tmp ORDER BY key ASC; + """.stripMargin) + + test("multiple generator in projection") { + intercept[AnalysisException] { + sql("SELECT explode(map(key, value)), key FROM src").collect() + } + + intercept[AnalysisException] { + sql("SELECT explode(map(key, value)) as k1, k2, key FROM src").collect() + } + } + createQueryTest("! operator", """ |SELECT a FROM ( @@ -456,7 +490,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("lateral view2", "SELECT * FROM src LATERAL VIEW explode(array(1,2)) tbl") - createQueryTest("lateral view3", "FROM src SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX") @@ -478,6 +511,9 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("lateral view6", "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v") + createQueryTest("Specify the udtf output", + "SELECT d FROM (SELECT explode(array(1,1)) d FROM src LIMIT 1) t") + test("sampling") { sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") sql("SELECT * FROM src TABLESAMPLE(100 PERCENT) s") From f83c0f112d04173f4fc2c5eaf0f9cb11d9180077 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 21 Apr 2015 16:24:15 -0700 Subject: [PATCH 105/144] [SPARK-3386] Share and reuse SerializerInstances in shuffle paths This patch modifies several shuffle-related code paths to share and re-use SerializerInstances instead of creating new ones. Some serializers, such as KryoSerializer or SqlSerializer, can be fairly expensive to create or may consume moderate amounts of memory, so it's probably best to avoid unnecessary serializer creation in hot code paths. The key change in this patch is modifying `getDiskWriter()` / `DiskBlockObjectWriter` to accept `SerializerInstance`s instead of `Serializer`s (which are factories for instances). This allows the disk writer's creator to decide whether the serializer instance can be shared or re-used. The rest of the patch modifies several write and read paths to use shared serializers. One big win is in `ShuffleBlockFetcherIterator`, where we used to create a new serializer per received block. Similarly, the shuffle write path used to create a new serializer per file even though in many cases only a single thread would be writing to a file at a time. I made a small serializer reuse optimization in CoarseGrainedExecutorBackend as well, since it seemed like a small and obvious improvement. Author: Josh Rosen Closes #5606 from JoshRosen/SPARK-3386 and squashes the following commits: f661ce7 [Josh Rosen] Remove thread local; add comment instead 64f8398 [Josh Rosen] Use ThreadLocal for serializer instance in CoarseGrainedExecutorBackend aeb680e [Josh Rosen] [SPARK-3386] Reuse SerializerInstance in shuffle code paths --- .../executor/CoarseGrainedExecutorBackend.scala | 6 +++++- .../spark/shuffle/FileShuffleBlockManager.scala | 6 ++++-- .../org/apache/spark/storage/BlockManager.scala | 8 ++++---- .../apache/spark/storage/BlockObjectWriter.scala | 6 +++--- .../storage/ShuffleBlockFetcherIterator.scala | 6 ++++-- .../util/collection/ExternalAppendOnlyMap.scala | 6 ++---- .../spark/util/collection/ExternalSorter.scala | 14 +++++++++----- .../spark/storage/BlockObjectWriterSuite.scala | 6 +++--- 8 files changed, 34 insertions(+), 24 deletions(-) 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 8300f9f2190b9..8af46f3327adb 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -30,6 +30,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.{SignalLogger, Utils} private[spark] class CoarseGrainedExecutorBackend( @@ -47,6 +48,10 @@ private[spark] class CoarseGrainedExecutorBackend( var executor: Executor = null @volatile var driver: Option[RpcEndpointRef] = None + // If this CoarseGrainedExecutorBackend is changed to support multiple threads, then this may need + // to be changed so that we don't share the serializer instance across threads + private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() + override def onStart() { import scala.concurrent.ExecutionContext.Implicits.global logInfo("Connecting to driver: " + driverUrl) @@ -83,7 +88,6 @@ private[spark] class CoarseGrainedExecutorBackend( logError("Received LaunchTask command but executor was null") System.exit(1) } else { - val ser = env.closureSerializer.newInstance() val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskId = taskDesc.taskId, attemptNumber = taskDesc.attemptNumber, diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 5be3ed771e534..538e150ead05a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -113,11 +113,12 @@ class FileShuffleBlockManager(conf: SparkConf) private var fileGroup: ShuffleFileGroup = null val openStartTime = System.nanoTime + val serializerInstance = serializer.newInstance() val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) { fileGroup = getUnusedFileGroup() Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) - blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize, + blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializerInstance, bufferSize, writeMetrics) } } else { @@ -133,7 +134,8 @@ class FileShuffleBlockManager(conf: SparkConf) logWarning(s"Failed to remove existing shuffle file $blockFile") } } - blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize, writeMetrics) + blockManager.getDiskWriter(blockId, blockFile, serializerInstance, bufferSize, + writeMetrics) } } // Creating the file to write to and creating a disk writer both involve interacting with diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1aa0ef18de118..145a9c1ae3391 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -37,7 +37,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo import org.apache.spark.rpc.RpcEnv -import org.apache.spark.serializer.Serializer +import org.apache.spark.serializer.{SerializerInstance, Serializer} import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.util._ @@ -646,13 +646,13 @@ private[spark] class BlockManager( def getDiskWriter( blockId: BlockId, file: File, - serializer: Serializer, + serializerInstance: SerializerInstance, bufferSize: Int, writeMetrics: ShuffleWriteMetrics): BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) - new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites, - writeMetrics) + new DiskBlockObjectWriter(blockId, file, serializerInstance, bufferSize, compressStream, + syncWrites, writeMetrics) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 0dfc91dfaff85..14833791f7a4d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -21,7 +21,7 @@ import java.io.{BufferedOutputStream, FileOutputStream, File, OutputStream} import java.nio.channels.FileChannel import org.apache.spark.Logging -import org.apache.spark.serializer.{SerializationStream, Serializer} +import org.apache.spark.serializer.{SerializerInstance, SerializationStream} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.util.Utils @@ -71,7 +71,7 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { private[spark] class DiskBlockObjectWriter( blockId: BlockId, file: File, - serializer: Serializer, + serializerInstance: SerializerInstance, bufferSize: Int, compressStream: OutputStream => OutputStream, syncWrites: Boolean, @@ -134,7 +134,7 @@ private[spark] class DiskBlockObjectWriter( ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() bs = compressStream(new BufferedOutputStream(ts, bufferSize)) - objOut = serializer.newInstance().serializeStream(bs) + objOut = serializerInstance.serializeStream(bs) initialized = true this } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 8f28ef49a8a6f..f3379521d55e2 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -27,7 +27,7 @@ import org.apache.spark.{Logging, TaskContext} import org.apache.spark.network.BlockTransferService import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.serializer.Serializer +import org.apache.spark.serializer.{SerializerInstance, Serializer} import org.apache.spark.util.{CompletionIterator, Utils} /** @@ -106,6 +106,8 @@ final class ShuffleBlockFetcherIterator( private[this] val shuffleMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + private[this] val serializerInstance: SerializerInstance = serializer.newInstance() + /** * Whether the iterator is still active. If isZombie is true, the callback interface will no * longer place fetched blocks into [[results]]. @@ -299,7 +301,7 @@ final class ShuffleBlockFetcherIterator( // the scheduler gets a FetchFailedException. Try(buf.createInputStream()).map { is0 => val is = blockManager.wrapForCompression(blockId, is0) - val iter = serializer.newInstance().deserializeStream(is).asIterator + val iter = serializerInstance.deserializeStream(is).asIterator CompletionIterator[Any, Iterator[Any]](iter, { // Once the iterator is exhausted, release the buffer and set currentResult to null // so we don't release it again in cleanup. diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 9ff4744593d4d..30dd7f22e494f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -151,8 +151,7 @@ class ExternalAppendOnlyMap[K, V, C]( override protected[this] def spill(collection: SizeTracker): Unit = { val (blockId, file) = diskBlockManager.createTempLocalBlock() curWriteMetrics = new ShuffleWriteMetrics() - var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, - curWriteMetrics) + var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) var objectsWritten = 0 // List of batch sizes (bytes) in the order they are written to disk @@ -179,8 +178,7 @@ class ExternalAppendOnlyMap[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() curWriteMetrics = new ShuffleWriteMetrics() - writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, - curWriteMetrics) + writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) } } if (objectsWritten > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 035f3767ff554..79a1a8a0dae38 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -272,7 +272,8 @@ private[spark] class ExternalSorter[K, V, C]( // createTempShuffleBlock here; see SPARK-3426 for more context. val (blockId, file) = diskBlockManager.createTempShuffleBlock() curWriteMetrics = new ShuffleWriteMetrics() - var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) + var writer = blockManager.getDiskWriter( + blockId, file, serInstance, fileBufferSize, curWriteMetrics) var objectsWritten = 0 // Objects written since the last flush // List of batch sizes (bytes) in the order they are written to disk @@ -308,7 +309,8 @@ private[spark] class ExternalSorter[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() curWriteMetrics = new ShuffleWriteMetrics() - writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) + writer = blockManager.getDiskWriter( + blockId, file, serInstance, fileBufferSize, curWriteMetrics) } } if (objectsWritten > 0) { @@ -358,7 +360,9 @@ private[spark] class ExternalSorter[K, V, C]( // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use // createTempShuffleBlock here; see SPARK-3426 for more context. val (blockId, file) = diskBlockManager.createTempShuffleBlock() - blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics).open() + val writer = blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, + curWriteMetrics) + writer.open() } // Creating the file to write to and creating a disk writer both involve interacting with // the disk, and can take a long time in aggregate when we open many files, so should be @@ -749,8 +753,8 @@ private[spark] class ExternalSorter[K, V, C]( // partition and just write everything directly. for ((id, elements) <- this.partitionedIterator) { if (elements.hasNext) { - val writer = blockManager.getDiskWriter( - blockId, outputFile, ser, fileBufferSize, context.taskMetrics.shuffleWriteMetrics.get) + val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, + context.taskMetrics.shuffleWriteMetrics.get) for (elem <- elements) { writer.write(elem) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala index 78bbc4ec2c620..003a728cb84a0 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala @@ -30,7 +30,7 @@ class BlockObjectWriterSuite extends FunSuite { val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, - new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) writer.write(Long.box(20)) // Record metrics update on every write @@ -52,7 +52,7 @@ class BlockObjectWriterSuite extends FunSuite { val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, - new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) writer.write(Long.box(20)) // Record metrics update on every write @@ -75,7 +75,7 @@ class BlockObjectWriterSuite extends FunSuite { val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, - new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) writer.open() writer.close() From a70e849c7f9e3df5e86113d45b8c4537597cfb29 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 21 Apr 2015 16:35:37 -0700 Subject: [PATCH 106/144] [minor] [build] Set java options when generating mima ignores. The default java options make the call to GenerateMIMAIgnore take forever to run since it's gc'ing all the time. Improve things by setting the perm gen size / max heap size to larger values. Author: Marcelo Vanzin Closes #5615 from vanzin/gen-mima-fix and squashes the following commits: f44e921 [Marcelo Vanzin] [minor] [build] Set java options when generating mima ignores. --- dev/mima | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/dev/mima b/dev/mima index bed5cd042634e..2952fa65d42ff 100755 --- a/dev/mima +++ b/dev/mima @@ -27,16 +27,21 @@ cd "$FWDIR" echo -e "q\n" | build/sbt oldDeps/update rm -f .generated-mima* +generate_mima_ignore() { + SPARK_JAVA_OPTS="-XX:MaxPermSize=1g -Xmx2g" \ + ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +} + # Generate Mima Ignore is called twice, first with latest built jars # on the classpath and then again with previous version jars on the classpath. # Because of a bug in GenerateMIMAIgnore that when old jars are ahead on classpath # it did not process the new classes (which are in assembly jar). -./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +generate_mima_ignore export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" -./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +generate_mima_ignore echo -e "q\n" | build/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" ret_val=$? From 7fe6142cd3c39ec79899878c3deca9d5130d05b1 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Tue, 21 Apr 2015 16:42:45 -0700 Subject: [PATCH 107/144] [SPARK-6065] [MLlib] Optimize word2vec.findSynonyms using blas calls 1. Use blas calls to find the dot product between two vectors. 2. Prevent re-computing the L2 norm of the given vector for each word in model. Author: MechCoder Closes #5467 from MechCoder/spark-6065 and squashes the following commits: dd0b0b2 [MechCoder] Preallocate wordVectors ffc9240 [MechCoder] Minor 6b74c81 [MechCoder] Switch back to native blas calls da1642d [MechCoder] Explicit types and indexing 64575b0 [MechCoder] Save indexedmap and a wordvecmat instead of matrix fbe0108 [MechCoder] Made the following changes 1. Calculate norms during initialization. 2. Use Blas calls from linalg.blas 1350cf3 [MechCoder] [SPARK-6065] Optimize word2vec.findSynonynms using blas calls --- .../apache/spark/mllib/feature/Word2Vec.scala | 57 +++++++++++++++++-- 1 file changed, 51 insertions(+), 6 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 b2d9053f70145..98e83112f52ae 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 @@ -34,7 +34,7 @@ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vector, Vectors, DenseMatrix, BLAS, DenseVector} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd._ import org.apache.spark.util.Utils @@ -429,7 +429,36 @@ class Word2Vec extends Serializable with Logging { */ @Experimental class Word2VecModel private[mllib] ( - private val model: Map[String, Array[Float]]) extends Serializable with Saveable { + model: Map[String, Array[Float]]) extends Serializable with Saveable { + + // wordList: Ordered list of words obtained from model. + private val wordList: Array[String] = model.keys.toArray + + // wordIndex: Maps each word to an index, which can retrieve the corresponding + // vector from wordVectors (see below). + private val wordIndex: Map[String, Int] = wordList.zip(0 until model.size).toMap + + // vectorSize: Dimension of each word's vector. + private val vectorSize = model.head._2.size + private val numWords = wordIndex.size + + // wordVectors: Array of length numWords * vectorSize, vector corresponding to the word + // mapped with index i can be retrieved by the slice + // (ind * vectorSize, ind * vectorSize + vectorSize) + // wordVecNorms: Array of length numWords, each value being the Euclidean norm + // of the wordVector. + private val (wordVectors: Array[Float], wordVecNorms: Array[Double]) = { + val wordVectors = new Array[Float](vectorSize * numWords) + val wordVecNorms = new Array[Double](numWords) + var i = 0 + while (i < numWords) { + val vec = model.get(wordList(i)).get + Array.copy(vec, 0, wordVectors, i * vectorSize, vectorSize) + wordVecNorms(i) = blas.snrm2(vectorSize, vec, 1) + i += 1 + } + (wordVectors, wordVecNorms) + } private def cosineSimilarity(v1: Array[Float], v2: Array[Float]): Double = { require(v1.length == v2.length, "Vectors should have the same length") @@ -443,7 +472,7 @@ class Word2VecModel private[mllib] ( override protected def formatVersion = "1.0" def save(sc: SparkContext, path: String): Unit = { - Word2VecModel.SaveLoadV1_0.save(sc, path, model) + Word2VecModel.SaveLoadV1_0.save(sc, path, getVectors) } /** @@ -479,9 +508,23 @@ class Word2VecModel private[mllib] ( */ def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { require(num > 0, "Number of similar words should > 0") - // TODO: optimize top-k + val fVector = vector.toArray.map(_.toFloat) - model.mapValues(vec => cosineSimilarity(fVector, vec)) + val cosineVec = Array.fill[Float](numWords)(0) + val alpha: Float = 1 + val beta: Float = 0 + + blas.sgemv( + "T", vectorSize, numWords, alpha, wordVectors, vectorSize, fVector, 1, beta, cosineVec, 1) + + // Need not divide with the norm of the given vector since it is constant. + val updatedCosines = new Array[Double](numWords) + var ind = 0 + while (ind < numWords) { + updatedCosines(ind) = cosineVec(ind) / wordVecNorms(ind) + ind += 1 + } + wordList.zip(updatedCosines) .toSeq .sortBy(- _._2) .take(num + 1) @@ -493,7 +536,9 @@ class Word2VecModel private[mllib] ( * Returns a map of words to their vector representations. */ def getVectors: Map[String, Array[Float]] = { - model + wordIndex.map { case (word, ind) => + (word, wordVectors.slice(vectorSize * ind, vectorSize * ind + vectorSize)) + } } } From 686dd742e11f6ad0078b7ff9b30b83a118fd8109 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 21 Apr 2015 16:44:52 -0700 Subject: [PATCH 108/144] [SPARK-7036][MLLIB] ALS.train should support DataFrames in PySpark SchemaRDD works with ALS.train in 1.2, so we should continue support DataFrames for compatibility. coderxiang Author: Xiangrui Meng Closes #5619 from mengxr/SPARK-7036 and squashes the following commits: dfcaf5a [Xiangrui Meng] ALS.train should support DataFrames in PySpark --- python/pyspark/mllib/recommendation.py | 36 +++++++++++++++++++------- 1 file changed, 26 insertions(+), 10 deletions(-) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 80e0a356bb78a..4b7d17d64e947 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -22,6 +22,7 @@ from pyspark.rdd import RDD from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc from pyspark.mllib.util import JavaLoader, JavaSaveable +from pyspark.sql import DataFrame __all__ = ['MatrixFactorizationModel', 'ALS', 'Rating'] @@ -78,18 +79,23 @@ class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): True >>> model = ALS.train(ratings, 1, nonnegative=True, seed=10) - >>> model.predict(2,2) + >>> model.predict(2, 2) + 3.8... + + >>> df = sqlContext.createDataFrame([Rating(1, 1, 1.0), Rating(1, 2, 2.0), Rating(2, 1, 2.0)]) + >>> model = ALS.train(df, 1, nonnegative=True, seed=10) + >>> model.predict(2, 2) 3.8... >>> model = ALS.trainImplicit(ratings, 1, nonnegative=True, seed=10) - >>> model.predict(2,2) + >>> model.predict(2, 2) 0.4... >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) >>> sameModel = MatrixFactorizationModel.load(sc, path) - >>> sameModel.predict(2,2) + >>> sameModel.predict(2, 2) 0.4... >>> sameModel.predictAll(testset).collect() [Rating(... @@ -125,13 +131,20 @@ class ALS(object): @classmethod def _prepare(cls, ratings): - assert isinstance(ratings, RDD), "ratings should be RDD" + if isinstance(ratings, RDD): + pass + elif isinstance(ratings, DataFrame): + ratings = ratings.rdd + else: + raise TypeError("Ratings should be represented by either an RDD or a DataFrame, " + "but got %s." % type(ratings)) first = ratings.first() - if not isinstance(first, Rating): - if isinstance(first, (tuple, list)): - ratings = ratings.map(lambda x: Rating(*x)) - else: - raise ValueError("rating should be RDD of Rating or tuple/list") + if isinstance(first, Rating): + pass + elif isinstance(first, (tuple, list)): + ratings = ratings.map(lambda x: Rating(*x)) + else: + raise TypeError("Expect a Rating or a tuple/list, but got %s." % type(first)) return ratings @classmethod @@ -152,8 +165,11 @@ def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alp def _test(): import doctest import pyspark.mllib.recommendation + from pyspark.sql import SQLContext globs = pyspark.mllib.recommendation.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + sc = SparkContext('local[4]', 'PythonTest') + globs['sc'] = sc + globs['sqlContext'] = SQLContext(sc) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: From ae036d08170202074b266afd17ce34b689c70b0c Mon Sep 17 00:00:00 2001 From: Alain Date: Tue, 21 Apr 2015 16:46:17 -0700 Subject: [PATCH 109/144] [Minor][MLLIB] Fix a minor formatting bug in toString method in Node.scala add missing comma and space Author: Alain Closes #5621 from AiHe/tree-node-issue and squashes the following commits: 159a7bb [Alain] [Minor][MLLIB] Fix a minor formatting bug in toString methods in Node.scala (cherry picked from commit 4508f01890a723f80d631424ff8eda166a13a727) Signed-off-by: Xiangrui Meng --- .../src/main/scala/org/apache/spark/mllib/tree/model/Node.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 708ba04b567d3..86390a20cb5cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -52,7 +52,7 @@ class Node ( override def toString: String = { "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + - "impurity = " + impurity + "split = " + split + ", stats = " + stats + "impurity = " + impurity + ", split = " + split + ", stats = " + stats } /** From b063a61b9852cf9b9d2c905332d2ecb2fd716cc4 Mon Sep 17 00:00:00 2001 From: mweindel Date: Tue, 21 Apr 2015 20:19:33 -0400 Subject: [PATCH 110/144] Avoid warning message about invalid refuse_seconds value in Mesos >=0.21... Starting with version 0.21.0, Apache Mesos is very noisy if the filter parameter refuse_seconds is set to an invalid value like `-1`. I have seen systems with millions of log lines like ``` W0420 18:00:48.773059 32352 hierarchical_allocator_process.hpp:589] Using the default value of 'refuse_seconds' to create the refused resources filter because the input value is negative ``` in the Mesos master INFO and WARNING log files. Therefore the CoarseMesosSchedulerBackend should set the default value for refuse seconds (i.e. 5 seconds) directly. This is no problem for the fine-grained MesosSchedulerBackend, as it uses the value 1 second for this parameter. Author: mweindel Closes #5597 from MartinWeindel/master and squashes the following commits: 2f99ffd [mweindel] Avoid warning message about invalid refuse_seconds value in Mesos >=0.21. --- .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index b037a4966ced0..82f652dae0378 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -207,7 +207,7 @@ private[spark] class CoarseMesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { synchronized { - val filters = Filters.newBuilder().setRefuseSeconds(-1).build() + val filters = Filters.newBuilder().setRefuseSeconds(5).build() for (offer <- offers) { val slaveId = offer.getSlaveId.toString From e72c16e30d85cdc394d318b5551698885cfda9b8 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 21 Apr 2015 20:33:57 -0400 Subject: [PATCH 111/144] [SPARK-6014] [core] Revamp Spark shutdown hooks, fix shutdown races. This change adds some new utility code to handle shutdown hooks in Spark. The main goal is to take advantage of Hadoop 2.x's API for shutdown hooks, which allows Spark to register a hook that will run before the one that cleans up HDFS clients, and thus avoids some races that would cause exceptions to show up and other issues such as failure to properly close event logs. Unfortunately, Hadoop 1.x does not have such APIs, so in that case correctness is still left to chance. Author: Marcelo Vanzin Closes #5560 from vanzin/SPARK-6014 and squashes the following commits: edfafb1 [Marcelo Vanzin] Better scaladoc. fcaeedd [Marcelo Vanzin] Merge branch 'master' into SPARK-6014 e7039dc [Marcelo Vanzin] [SPARK-6014] [core] Revamp Spark shutdown hooks, fix shutdown races. --- .../spark/deploy/history/HistoryServer.scala | 6 +- .../spark/deploy/worker/ExecutorRunner.scala | 12 +- .../spark/storage/DiskBlockManager.scala | 18 +-- .../spark/storage/TachyonBlockManager.scala | 24 ++-- .../scala/org/apache/spark/util/Utils.scala | 136 +++++++++++++++--- .../org/apache/spark/util/UtilsSuite.scala | 32 +++-- .../hive/thriftserver/HiveThriftServer2.scala | 9 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 9 +- .../spark/deploy/yarn/ApplicationMaster.scala | 63 ++++---- 9 files changed, 195 insertions(+), 114 deletions(-) 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 72f6048239297..56bef57e55392 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 @@ -27,7 +27,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.SignalLogger +import org.apache.spark.util.{SignalLogger, Utils} /** * A web server that renders SparkUIs of completed applications. @@ -194,9 +194,7 @@ object HistoryServer extends Logging { val server = new HistoryServer(conf, provider, securityManager, port) server.bind() - Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") { - override def run(): Unit = server.stop() - }) + Utils.addShutdownHook { () => server.stop() } // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } 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 7d5acabb95a48..7aa85b732fc87 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 @@ -28,6 +28,7 @@ import com.google.common.io.Files import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.util.Utils import org.apache.spark.util.logging.FileAppender /** @@ -61,7 +62,7 @@ private[deploy] class ExecutorRunner( // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. - private var shutdownHook: Thread = null + private var shutdownHook: AnyRef = null private[worker] def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { @@ -69,12 +70,7 @@ private[deploy] class ExecutorRunner( } workerThread.start() // Shutdown hook that kills actors on shutdown. - shutdownHook = new Thread() { - override def run() { - killProcess(Some("Worker shutting down")) - } - } - Runtime.getRuntime.addShutdownHook(shutdownHook) + shutdownHook = Utils.addShutdownHook { () => killProcess(Some("Worker shutting down")) } } /** @@ -106,7 +102,7 @@ private[deploy] class ExecutorRunner( workerThread = null state = ExecutorState.KILLED try { - Runtime.getRuntime.removeShutdownHook(shutdownHook) + Utils.removeShutdownHook(shutdownHook) } catch { case e: IllegalStateException => None } 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 2883137872600..7ea5e54f9e1fe 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -138,25 +138,17 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } } - private def addShutdownHook(): Thread = { - val shutdownHook = new Thread("delete Spark local dirs") { - override def run(): Unit = Utils.logUncaughtExceptions { - logDebug("Shutdown hook called") - DiskBlockManager.this.doStop() - } + private def addShutdownHook(): AnyRef = { + Utils.addShutdownHook { () => + logDebug("Shutdown hook called") + DiskBlockManager.this.doStop() } - Runtime.getRuntime.addShutdownHook(shutdownHook) - shutdownHook } /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { // Remove the shutdown hook. It causes memory leaks if we leave it around. - try { - Runtime.getRuntime.removeShutdownHook(shutdownHook) - } catch { - case e: IllegalStateException => None - } + Utils.removeShutdownHook(shutdownHook) doStop() } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index af873034215a9..951897cead996 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -135,21 +135,19 @@ private[spark] class TachyonBlockManager( private def addShutdownHook() { tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { - override def run(): Unit = Utils.logUncaughtExceptions { - logDebug("Shutdown hook called") - tachyonDirs.foreach { tachyonDir => - try { - if (!Utils.hasRootAsShutdownDeleteDir(tachyonDir)) { - Utils.deleteRecursively(tachyonDir, client) - } - } catch { - case e: Exception => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) + Utils.addShutdownHook { () => + logDebug("Shutdown hook called") + tachyonDirs.foreach { tachyonDir => + try { + if (!Utils.hasRootAsShutdownDeleteDir(tachyonDir)) { + Utils.deleteRecursively(tachyonDir, client) } + } catch { + case e: Exception => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } - client.close() } - }) + client.close() + } } } 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 1029b0f9fce1e..7b0de1ae55b78 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,7 +21,7 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.{Properties, Locale, Random, UUID} +import java.util.{PriorityQueue, Properties, Locale, Random, UUID} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection @@ -30,7 +30,7 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.util.Try +import scala.util.{Failure, Success, Try} import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.{ByteStreams, Files} @@ -64,9 +64,15 @@ private[spark] object CallSite { private[spark] object Utils extends Logging { val random = new Random() + val DEFAULT_SHUTDOWN_PRIORITY = 100 + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null + + private val shutdownHooks = new SparkShutdownHookManager() + shutdownHooks.install() + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -176,18 +182,16 @@ private[spark] object Utils extends Logging { private val shutdownDeleteTachyonPaths = new scala.collection.mutable.HashSet[String]() // Add a shutdown hook to delete the temp dirs when the JVM exits - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dirs") { - override def run(): Unit = Utils.logUncaughtExceptions { - logDebug("Shutdown hook called") - shutdownDeletePaths.foreach { dirPath => - try { - Utils.deleteRecursively(new File(dirPath)) - } catch { - case e: Exception => logError(s"Exception while deleting Spark temp dir: $dirPath", e) - } + addShutdownHook { () => + logDebug("Shutdown hook called") + shutdownDeletePaths.foreach { dirPath => + try { + Utils.deleteRecursively(new File(dirPath)) + } catch { + case e: Exception => logError(s"Exception while deleting Spark temp dir: $dirPath", e) } } - }) + } // Register the path to be deleted via shutdown hook def registerShutdownDeleteDir(file: File) { @@ -613,7 +617,7 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeoutMs = + val timeoutMs = conf.getTimeAsSeconds("spark.files.fetchTimeout", "60s").toInt * 1000 uc.setConnectTimeout(timeoutMs) uc.setReadTimeout(timeoutMs) @@ -1172,7 +1176,7 @@ private[spark] object Utils extends Logging { /** * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the * default UncaughtExceptionHandler - * + * * NOTE: This method is to be called by the spark-started JVM process. */ def tryOrExit(block: => Unit) { @@ -1185,11 +1189,11 @@ private[spark] object Utils extends Logging { } /** - * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught * exception - * - * NOTE: This method is to be called by the driver-side components to avoid stopping the - * user-started JVM process completely; in contrast, tryOrExit is to be called in the + * + * NOTE: This method is to be called by the driver-side components to avoid stopping the + * user-started JVM process completely; in contrast, tryOrExit is to be called in the * spark-started JVM process . */ def tryOrStopSparkContext(sc: SparkContext)(block: => Unit) { @@ -2132,6 +2136,102 @@ private[spark] object Utils extends Logging { .getOrElse(UserGroupInformation.getCurrentUser().getShortUserName()) } + /** + * Adds a shutdown hook with default priority. + * + * @param hook The code to run during shutdown. + * @return A handle that can be used to unregister the shutdown hook. + */ + def addShutdownHook(hook: () => Unit): AnyRef = { + addShutdownHook(DEFAULT_SHUTDOWN_PRIORITY, hook) + } + + /** + * Adds a shutdown hook with the given priority. Hooks with lower priority values run + * first. + * + * @param hook The code to run during shutdown. + * @return A handle that can be used to unregister the shutdown hook. + */ + def addShutdownHook(priority: Int, hook: () => Unit): AnyRef = { + shutdownHooks.add(priority, hook) + } + + /** + * Remove a previously installed shutdown hook. + * + * @param ref A handle returned by `addShutdownHook`. + * @return Whether the hook was removed. + */ + def removeShutdownHook(ref: AnyRef): Boolean = { + shutdownHooks.remove(ref) + } + +} + +private [util] class SparkShutdownHookManager { + + private val hooks = new PriorityQueue[SparkShutdownHook]() + private var shuttingDown = false + + /** + * Install a hook to run at shutdown and run all registered hooks in order. Hadoop 1.x does not + * have `ShutdownHookManager`, so in that case we just use the JVM's `Runtime` object and hope for + * the best. + */ + def install(): Unit = { + val hookTask = new Runnable() { + override def run(): Unit = runAll() + } + Try(Class.forName("org.apache.hadoop.util.ShutdownHookManager")) match { + case Success(shmClass) => + val fsPriority = classOf[FileSystem].getField("SHUTDOWN_HOOK_PRIORITY").get() + .asInstanceOf[Int] + val shm = shmClass.getMethod("get").invoke(null) + shm.getClass().getMethod("addShutdownHook", classOf[Runnable], classOf[Int]) + .invoke(shm, hookTask, Integer.valueOf(fsPriority + 30)) + + case Failure(_) => + Runtime.getRuntime.addShutdownHook(new Thread(hookTask, "Spark Shutdown Hook")); + } + } + + def runAll(): Unit = synchronized { + shuttingDown = true + while (!hooks.isEmpty()) { + Utils.logUncaughtExceptions(hooks.poll().run()) + } + } + + def add(priority: Int, hook: () => Unit): AnyRef = synchronized { + checkState() + val hookRef = new SparkShutdownHook(priority, hook) + hooks.add(hookRef) + hookRef + } + + def remove(ref: AnyRef): Boolean = synchronized { + checkState() + hooks.remove(ref) + } + + private def checkState(): Unit = { + if (shuttingDown) { + throw new IllegalStateException("Shutdown hooks cannot be modified during shutdown.") + } + } + +} + +private class SparkShutdownHook(private val priority: Int, hook: () => Unit) + extends Comparable[SparkShutdownHook] { + + override def compareTo(other: SparkShutdownHook): Int = { + other.priority - priority + } + + def run(): Unit = hook() + } /** 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 fb97e650ff95c..1ba99803f5a0e 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.util -import scala.util.Random - import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.text.DecimalFormatSymbols import java.util.concurrent.TimeUnit import java.util.Locale +import java.util.PriorityQueue + +import scala.collection.mutable.ListBuffer +import scala.util.Random import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files @@ -36,14 +38,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { - + test("timeConversion") { // Test -1 assert(Utils.timeStringAsSeconds("-1") === -1) - + // Test zero assert(Utils.timeStringAsSeconds("0") === 0) - + assert(Utils.timeStringAsSeconds("1") === 1) assert(Utils.timeStringAsSeconds("1s") === 1) assert(Utils.timeStringAsSeconds("1000ms") === 1) @@ -52,7 +54,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(Utils.timeStringAsSeconds("1min") === TimeUnit.MINUTES.toSeconds(1)) assert(Utils.timeStringAsSeconds("1h") === TimeUnit.HOURS.toSeconds(1)) assert(Utils.timeStringAsSeconds("1d") === TimeUnit.DAYS.toSeconds(1)) - + assert(Utils.timeStringAsMs("1") === 1) assert(Utils.timeStringAsMs("1ms") === 1) assert(Utils.timeStringAsMs("1000us") === 1) @@ -61,7 +63,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) - + // Test invalid strings intercept[NumberFormatException] { Utils.timeStringAsMs("This breaks 600s") @@ -79,7 +81,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { Utils.timeStringAsMs("This 123s breaks") } } - + test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") assert(Utils.bytesToString(1500) === "1500.0 B") @@ -466,4 +468,18 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { val newFileName = new File(testFileDir, testFileName) assert(newFileName.isFile()) } + + test("shutdown hook manager") { + val manager = new SparkShutdownHookManager() + val output = new ListBuffer[Int]() + + val hook1 = manager.add(1, () => output += 1) + manager.add(3, () => output += 3) + manager.add(2, () => output += 2) + manager.add(4, () => output += 4) + manager.remove(hook1) + + manager.runAll() + assert(output.toList === List(4, 3, 2)) + } } 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 c3a3f8c0f41df..832596fc8bee5 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 @@ -28,6 +28,7 @@ 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} +import org.apache.spark.util.Utils /** * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a @@ -57,13 +58,7 @@ object HiveThriftServer2 extends Logging { logInfo("Starting SparkContext") SparkSQLEnv.init() - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.stop() - } - } - ) + Utils.addShutdownHook { () => SparkSQLEnv.stop() } try { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 85281c6d73a3b..7e307bb4ad1e8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -40,6 +40,7 @@ import org.apache.thrift.transport.TSocket import org.apache.spark.Logging import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.util.Utils private[hive] object SparkSQLCLIDriver { private var prompt = "spark-sql" @@ -101,13 +102,7 @@ private[hive] object SparkSQLCLIDriver { SessionState.start(sessionState) // Clean up after we exit - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.stop() - } - } - ) + Utils.addShutdownHook { () => SparkSQLEnv.stop() } // "-h" option has been passed, so connect to Hive thrift server. if (sessionState.getHost != null) { 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 f7a84207e9da6..93ae45133ce24 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 @@ -25,7 +25,6 @@ import java.net.{Socket, URL} import java.util.concurrent.atomic.AtomicReference import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -95,44 +94,38 @@ private[spark] class ApplicationMaster( logInfo("ApplicationAttemptId: " + appAttemptId) val fs = FileSystem.get(yarnConf) - val cleanupHook = new Runnable { - override def run() { - // If the SparkContext is still registered, shut it down as a best case effort in case - // users do not call sc.stop or do System.exit(). - val sc = sparkContextRef.get() - if (sc != null) { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() - } - val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) - val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts - - if (!finished) { - // This happens when the user application calls System.exit(). We have the choice - // of either failing or succeeding at this point. We report success to avoid - // retrying applications that have succeeded (System.exit(0)), which means that - // applications that explicitly exit with a non-zero status will also show up as - // succeeded in the RM UI. - finish(finalStatus, - ApplicationMaster.EXIT_SUCCESS, - "Shutdown hook called before final status was reported.") - } - if (!unregistered) { - // we only want to unregister if we don't want the RM to retry - if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { - unregister(finalStatus, finalMsg) - cleanupStagingDir(fs) - } + Utils.addShutdownHook { () => + // If the SparkContext is still registered, shut it down as a best case effort in case + // users do not call sc.stop or do System.exit(). + val sc = sparkContextRef.get() + if (sc != null) { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() + } + val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) + val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts + + if (!finished) { + // This happens when the user application calls System.exit(). We have the choice + // of either failing or succeeding at this point. We report success to avoid + // retrying applications that have succeeded (System.exit(0)), which means that + // applications that explicitly exit with a non-zero status will also show up as + // succeeded in the RM UI. + finish(finalStatus, + ApplicationMaster.EXIT_SUCCESS, + "Shutdown hook called before final status was reported.") + } + + if (!unregistered) { + // we only want to unregister if we don't want the RM to retry + if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { + unregister(finalStatus, finalMsg) + cleanupStagingDir(fs) } } } - // Use higher priority than FileSystem. - assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) - ShutdownHookManager - .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) - // Call this to force generation of secret so it gets populated into the // Hadoop UGI. This has to happen before the startUserApplication which does a // doAs in order for the credentials to be passed on to the executor containers. @@ -546,8 +539,6 @@ private[spark] class ApplicationMaster( object ApplicationMaster extends Logging { - val SHUTDOWN_HOOK_PRIORITY: Int = 30 - // exit codes for different causes, no reason behind the values private val EXIT_SUCCESS = 0 private val EXIT_UNCAUGHT_EXCEPTION = 10 From 3134c3fe495862b7687b5aa00d3344d09cd5e08e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 21 Apr 2015 17:49:55 -0700 Subject: [PATCH 112/144] [SPARK-6953] [PySpark] speed up python tests This PR try to speed up some python tests: ``` tests.py 144s -> 103s -41s mllib/classification.py 24s -> 17s -7s mllib/regression.py 27s -> 15s -12s mllib/tree.py 27s -> 13s -14s mllib/tests.py 64s -> 31s -33s streaming/tests.py 185s -> 84s -101s ``` Considering python3, the total saving will be 558s (almost 10 minutes) (core, and streaming run three times, mllib runs twice). During testing, it will show used time for each test file: ``` Run core tests ... Running test: pyspark/rdd.py ... ok (22s) Running test: pyspark/context.py ... ok (16s) Running test: pyspark/conf.py ... ok (4s) Running test: pyspark/broadcast.py ... ok (4s) Running test: pyspark/accumulators.py ... ok (4s) Running test: pyspark/serializers.py ... ok (6s) Running test: pyspark/profiler.py ... ok (5s) Running test: pyspark/shuffle.py ... ok (1s) Running test: pyspark/tests.py ... ok (103s) 144s ``` Author: Reynold Xin Author: Xiangrui Meng Closes #5605 from rxin/python-tests-speed and squashes the following commits: d08542d [Reynold Xin] Merge pull request #14 from mengxr/SPARK-6953 89321ee [Xiangrui Meng] fix seed in tests 3ad2387 [Reynold Xin] Merge pull request #5427 from davies/python_tests --- python/pyspark/mllib/classification.py | 17 ++--- python/pyspark/mllib/regression.py | 25 ++++--- python/pyspark/mllib/tests.py | 69 +++++++++--------- python/pyspark/mllib/tree.py | 15 ++-- python/pyspark/shuffle.py | 7 +- python/pyspark/sql/tests.py | 4 +- python/pyspark/streaming/tests.py | 63 ++++++++++------- python/pyspark/tests.py | 96 ++++++++++++++++---------- python/run-tests | 13 ++-- 9 files changed, 182 insertions(+), 127 deletions(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index eda0b60f8b1e7..a70c664a71fdb 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -86,7 +86,7 @@ class LogisticRegressionModel(LinearClassificationModel): ... LabeledPoint(0.0, [0.0, 1.0]), ... LabeledPoint(1.0, [1.0, 0.0]), ... ] - >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(data)) + >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(data), iterations=10) >>> lrm.predict([1.0, 0.0]) 1 >>> lrm.predict([0.0, 1.0]) @@ -95,7 +95,7 @@ class LogisticRegressionModel(LinearClassificationModel): [1, 0] >>> lrm.clearThreshold() >>> lrm.predict([0.0, 1.0]) - 0.123... + 0.279... >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), @@ -103,7 +103,7 @@ class LogisticRegressionModel(LinearClassificationModel): ... LabeledPoint(0.0, SparseVector(2, {0: 1.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] - >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(sparse_data)) + >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(sparse_data), iterations=10) >>> lrm.predict(array([0.0, 1.0])) 1 >>> lrm.predict(array([1.0, 0.0])) @@ -129,7 +129,8 @@ class LogisticRegressionModel(LinearClassificationModel): ... LabeledPoint(1.0, [1.0, 0.0, 0.0]), ... LabeledPoint(2.0, [0.0, 0.0, 1.0]) ... ] - >>> mcm = LogisticRegressionWithLBFGS.train(data=sc.parallelize(multi_class_data), numClasses=3) + >>> data = sc.parallelize(multi_class_data) + >>> mcm = LogisticRegressionWithLBFGS.train(data, iterations=10, numClasses=3) >>> mcm.predict([0.0, 0.5, 0.0]) 0 >>> mcm.predict([0.8, 0.0, 0.0]) @@ -298,7 +299,7 @@ def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType ... LabeledPoint(0.0, [0.0, 1.0]), ... LabeledPoint(1.0, [1.0, 0.0]), ... ] - >>> lrm = LogisticRegressionWithLBFGS.train(sc.parallelize(data)) + >>> lrm = LogisticRegressionWithLBFGS.train(sc.parallelize(data), iterations=10) >>> lrm.predict([1.0, 0.0]) 1 >>> lrm.predict([0.0, 1.0]) @@ -330,14 +331,14 @@ class SVMModel(LinearClassificationModel): ... LabeledPoint(1.0, [2.0]), ... LabeledPoint(1.0, [3.0]) ... ] - >>> svm = SVMWithSGD.train(sc.parallelize(data)) + >>> svm = SVMWithSGD.train(sc.parallelize(data), iterations=10) >>> svm.predict([1.0]) 1 >>> svm.predict(sc.parallelize([[1.0]])).collect() [1] >>> svm.clearThreshold() >>> svm.predict(array([1.0])) - 1.25... + 1.44... >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {0: -1.0})), @@ -345,7 +346,7 @@ class SVMModel(LinearClassificationModel): ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] - >>> svm = SVMWithSGD.train(sc.parallelize(sparse_data)) + >>> svm = SVMWithSGD.train(sc.parallelize(sparse_data), iterations=10) >>> svm.predict(SparseVector(2, {1: 1.0})) 1 >>> svm.predict(SparseVector(2, {0: -1.0})) diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index a0117c57133ae..4bc6351bdf02f 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -108,7 +108,8 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=np.array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=np.array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -135,12 +136,13 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, SparseVector(1, {0: 2.0})), ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=100, step=1.0, + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, ... miniBatchFraction=1.0, initialWeights=array([1.0]), regParam=0.1, regType="l2", ... intercept=True, validateData=True) >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 @@ -238,7 +240,7 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LassoWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=10, initialWeights=array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -265,12 +267,13 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(3.0, SparseVector(1, {0: 2.0})), ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True - >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=100, step=1.0, + >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, ... validateData=True) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 @@ -321,7 +324,8 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -348,12 +352,13 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, SparseVector(1, {0: 2.0})), ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True - >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=100, step=1.0, + >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, ... validateData=True) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 @@ -396,7 +401,7 @@ def _test(): from pyspark import SparkContext import pyspark.mllib.regression globs = pyspark.mllib.regression.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8f89e2cee0592..1b008b93bc137 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -36,6 +36,7 @@ else: import unittest +from pyspark import SparkContext from pyspark.mllib.common import _to_java_object_rdd from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ DenseMatrix, SparseMatrix, Vectors, Matrices @@ -47,7 +48,6 @@ from pyspark.mllib.feature import StandardScaler from pyspark.serializers import PickleSerializer from pyspark.sql import SQLContext -from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase _have_scipy = False try: @@ -58,6 +58,12 @@ pass ser = PickleSerializer() +sc = SparkContext('local[4]', "MLlib tests") + + +class MLlibTestCase(unittest.TestCase): + def setUp(self): + self.sc = sc def _squared_distance(a, b): @@ -67,7 +73,7 @@ def _squared_distance(a, b): return b.squared_distance(a) -class VectorTests(PySparkTestCase): +class VectorTests(MLlibTestCase): def _test_serialize(self, v): self.assertEqual(v, ser.loads(ser.dumps(v))) @@ -212,7 +218,7 @@ def test_dense_matrix_is_transposed(self): self.assertTrue(array_equal(sm.values, [1, 3, 4, 6, 9])) -class ListTests(PySparkTestCase): +class ListTests(MLlibTestCase): """ Test MLlib algorithms on plain lists, to make sure they're passed through @@ -255,7 +261,7 @@ def test_gmm(self): [-6, -7], ]) clusters = GaussianMixture.train(data, 2, convergenceTol=0.001, - maxIterations=100, seed=56) + maxIterations=10, seed=56) labels = clusters.predict(data).collect() self.assertEquals(labels[0], labels[1]) self.assertEquals(labels[2], labels[3]) @@ -266,9 +272,9 @@ def test_gmm_deterministic(self): y = range(0, 100, 10) data = self.sc.parallelize([[a, b] for a, b in zip(x, y)]) clusters1 = GaussianMixture.train(data, 5, convergenceTol=0.001, - maxIterations=100, seed=63) + maxIterations=10, seed=63) clusters2 = GaussianMixture.train(data, 5, convergenceTol=0.001, - maxIterations=100, seed=63) + maxIterations=10, seed=63) for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEquals(round(c1, 7), round(c2, 7)) @@ -287,13 +293,13 @@ def test_classification(self): temp_dir = tempfile.mkdtemp() - lr_model = LogisticRegressionWithSGD.train(rdd) + lr_model = LogisticRegressionWithSGD.train(rdd, iterations=10) self.assertTrue(lr_model.predict(features[0]) <= 0) self.assertTrue(lr_model.predict(features[1]) > 0) self.assertTrue(lr_model.predict(features[2]) <= 0) self.assertTrue(lr_model.predict(features[3]) > 0) - svm_model = SVMWithSGD.train(rdd) + svm_model = SVMWithSGD.train(rdd, iterations=10) self.assertTrue(svm_model.predict(features[0]) <= 0) self.assertTrue(svm_model.predict(features[1]) > 0) self.assertTrue(svm_model.predict(features[2]) <= 0) @@ -307,7 +313,7 @@ def test_classification(self): categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories dt_model = DecisionTree.trainClassifier( - rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo, maxBins=4) self.assertTrue(dt_model.predict(features[0]) <= 0) self.assertTrue(dt_model.predict(features[1]) > 0) self.assertTrue(dt_model.predict(features[2]) <= 0) @@ -319,7 +325,8 @@ def test_classification(self): self.assertEqual(same_dt_model.toDebugString(), dt_model.toDebugString()) rf_model = RandomForest.trainClassifier( - rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=100) + rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=10, + maxBins=4, seed=1) self.assertTrue(rf_model.predict(features[0]) <= 0) self.assertTrue(rf_model.predict(features[1]) > 0) self.assertTrue(rf_model.predict(features[2]) <= 0) @@ -331,7 +338,7 @@ def test_classification(self): self.assertEqual(same_rf_model.toDebugString(), rf_model.toDebugString()) gbt_model = GradientBoostedTrees.trainClassifier( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numIterations=4) self.assertTrue(gbt_model.predict(features[0]) <= 0) self.assertTrue(gbt_model.predict(features[1]) > 0) self.assertTrue(gbt_model.predict(features[2]) <= 0) @@ -360,19 +367,19 @@ def test_regression(self): rdd = self.sc.parallelize(data) features = [p.features.tolist() for p in data] - lr_model = LinearRegressionWithSGD.train(rdd) + lr_model = LinearRegressionWithSGD.train(rdd, iterations=10) self.assertTrue(lr_model.predict(features[0]) <= 0) self.assertTrue(lr_model.predict(features[1]) > 0) self.assertTrue(lr_model.predict(features[2]) <= 0) self.assertTrue(lr_model.predict(features[3]) > 0) - lasso_model = LassoWithSGD.train(rdd) + lasso_model = LassoWithSGD.train(rdd, iterations=10) self.assertTrue(lasso_model.predict(features[0]) <= 0) self.assertTrue(lasso_model.predict(features[1]) > 0) self.assertTrue(lasso_model.predict(features[2]) <= 0) self.assertTrue(lasso_model.predict(features[3]) > 0) - rr_model = RidgeRegressionWithSGD.train(rdd) + rr_model = RidgeRegressionWithSGD.train(rdd, iterations=10) self.assertTrue(rr_model.predict(features[0]) <= 0) self.assertTrue(rr_model.predict(features[1]) > 0) self.assertTrue(rr_model.predict(features[2]) <= 0) @@ -380,35 +387,35 @@ def test_regression(self): categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories dt_model = DecisionTree.trainRegressor( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, maxBins=4) self.assertTrue(dt_model.predict(features[0]) <= 0) self.assertTrue(dt_model.predict(features[1]) > 0) self.assertTrue(dt_model.predict(features[2]) <= 0) self.assertTrue(dt_model.predict(features[3]) > 0) rf_model = RandomForest.trainRegressor( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=100, seed=1) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=10, maxBins=4, seed=1) self.assertTrue(rf_model.predict(features[0]) <= 0) self.assertTrue(rf_model.predict(features[1]) > 0) self.assertTrue(rf_model.predict(features[2]) <= 0) self.assertTrue(rf_model.predict(features[3]) > 0) gbt_model = GradientBoostedTrees.trainRegressor( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numIterations=4) self.assertTrue(gbt_model.predict(features[0]) <= 0) self.assertTrue(gbt_model.predict(features[1]) > 0) self.assertTrue(gbt_model.predict(features[2]) <= 0) self.assertTrue(gbt_model.predict(features[3]) > 0) try: - LinearRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) - LassoWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) - RidgeRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) + LinearRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0]), iterations=10) + LassoWithSGD.train(rdd, initialWeights=array([1.0, 1.0]), iterations=10) + RidgeRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0]), iterations=10) except ValueError: self.fail() -class StatTests(PySparkTestCase): +class StatTests(MLlibTestCase): # SPARK-4023 def test_col_with_different_rdds(self): # numpy @@ -438,7 +445,7 @@ def test_col_norms(self): self.assertTrue(math.fabs(summary2.normL2()[0] - expectedNormL2) < 1e-14) -class VectorUDTTests(PySparkTestCase): +class VectorUDTTests(MLlibTestCase): dv0 = DenseVector([]) dv1 = DenseVector([1.0, 2.0]) @@ -472,7 +479,7 @@ def test_infer_schema(self): @unittest.skipIf(not _have_scipy, "SciPy not installed") -class SciPyTests(PySparkTestCase): +class SciPyTests(MLlibTestCase): """ Test both vector operations and MLlib algorithms with SciPy sparse matrices, @@ -613,7 +620,7 @@ def test_regression(self): self.assertTrue(dt_model.predict(features[3]) > 0) -class ChiSqTestTests(PySparkTestCase): +class ChiSqTestTests(MLlibTestCase): def test_goodness_of_fit(self): from numpy import inf @@ -711,13 +718,13 @@ def test_right_number_of_results(self): self.assertIsNotNone(chi[1000]) -class SerDeTest(PySparkTestCase): +class SerDeTest(MLlibTestCase): def test_to_java_object_rdd(self): # SPARK-6660 data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0) self.assertEqual(_to_java_object_rdd(data).count(), 10) -class FeatureTest(PySparkTestCase): +class FeatureTest(MLlibTestCase): def test_idf_model(self): data = [ Vectors.dense([1, 2, 6, 0, 2, 3, 1, 1, 0, 0, 3]), @@ -730,13 +737,8 @@ def test_idf_model(self): self.assertEqual(len(idf), 11) -class Word2VecTests(PySparkTestCase): +class Word2VecTests(MLlibTestCase): def test_word2vec_setters(self): - data = [ - ["I", "have", "a", "pen"], - ["I", "like", "soccer", "very", "much"], - ["I", "live", "in", "Tokyo"] - ] model = Word2Vec() \ .setVectorSize(2) \ .setLearningRate(0.01) \ @@ -765,7 +767,7 @@ def test_word2vec_get_vectors(self): self.assertEquals(len(model.getVectors()), 3) -class StandardScalerTests(PySparkTestCase): +class StandardScalerTests(MLlibTestCase): def test_model_setters(self): data = [ [1.0, 2.0, 3.0], @@ -793,3 +795,4 @@ def test_model_transform(self): unittest.main() if not _have_scipy: print("NOTE: SciPy tests were skipped as it does not seem to be installed") + sc.stop() diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 0fe6e4fabe43a..cfcbea573fd22 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -482,13 +482,13 @@ def trainClassifier(cls, data, categoricalFeaturesInfo, ... LabeledPoint(1.0, [3.0]) ... ] >>> - >>> model = GradientBoostedTrees.trainClassifier(sc.parallelize(data), {}) + >>> model = GradientBoostedTrees.trainClassifier(sc.parallelize(data), {}, numIterations=10) >>> model.numTrees() - 100 + 10 >>> model.totalNumNodes() - 300 + 30 >>> print(model) # it already has newline - TreeEnsembleModel classifier with 100 trees + TreeEnsembleModel classifier with 10 trees >>> model.predict([2.0]) 1.0 @@ -541,11 +541,12 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] >>> - >>> model = GradientBoostedTrees.trainRegressor(sc.parallelize(sparse_data), {}) + >>> data = sc.parallelize(sparse_data) + >>> model = GradientBoostedTrees.trainRegressor(data, {}, numIterations=10) >>> model.numTrees() - 100 + 10 >>> model.totalNumNodes() - 102 + 12 >>> model.predict(SparseVector(2, {1: 1.0})) 1.0 >>> model.predict(SparseVector(2, {0: 1.0})) diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index b54baa57ec28a..1d0b16cade8bb 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -486,7 +486,7 @@ def sorted(self, iterator, key=None, reverse=False): goes above the limit. """ global MemoryBytesSpilled, DiskBytesSpilled - batch, limit = 100, self._next_limit() + batch, limit = 100, self.memory_limit chunks, current_chunk = [], [] iterator = iter(iterator) while True: @@ -497,7 +497,7 @@ def sorted(self, iterator, key=None, reverse=False): break used_memory = get_used_memory() - if used_memory > self.memory_limit: + if used_memory > limit: # sort them inplace will save memory current_chunk.sort(key=key, reverse=reverse) path = self._get_path(len(chunks)) @@ -513,13 +513,14 @@ def load(f): chunks.append(load(open(path, 'rb'))) current_chunk = [] gc.collect() + batch //= 2 limit = self._next_limit() MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 DiskBytesSpilled += os.path.getsize(path) os.unlink(path) # data will be deleted after close elif not chunks: - batch = min(batch * 2, 10000) + batch = min(int(batch * 1.5), 10000) current_chunk.sort(key=key, reverse=reverse) if not chunks: diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 23e84283679e1..fe43c374f1cb1 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -109,7 +109,7 @@ def setUpClass(cls): os.unlink(cls.tempdir.name) cls.sqlCtx = SQLContext(cls.sc) cls.testData = [Row(key=i, value=str(i)) for i in range(100)] - rdd = cls.sc.parallelize(cls.testData) + rdd = cls.sc.parallelize(cls.testData, 2) cls.df = rdd.toDF() @classmethod @@ -303,7 +303,7 @@ def test_apply_schema(self): abstract = "byte1 short1 float1 time1 map1{} struct1(b) list1[]" schema = _parse_schema_abstract(abstract) typedSchema = _infer_schema_type(rdd.first(), schema) - df = self.sqlCtx.applySchema(rdd, typedSchema) + df = self.sqlCtx.createDataFrame(rdd, typedSchema) r = (127, -32768, 1.0, datetime(2010, 1, 1, 1, 1, 1), {"a": 1}, Row(b=2), [1, 2, 3]) self.assertEqual(r, tuple(df.first())) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 33f958a601f3a..5fa1e5ef081ab 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -16,14 +16,23 @@ # import os +import sys from itertools import chain import time import operator -import unittest import tempfile import struct from functools import reduce +if sys.version_info[:2] <= (2, 6): + try: + import unittest2 as unittest + except ImportError: + sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier') + sys.exit(1) +else: + import unittest + from pyspark.context import SparkConf, SparkContext, RDD from pyspark.streaming.context import StreamingContext from pyspark.streaming.kafka import KafkaUtils @@ -31,19 +40,25 @@ class PySparkStreamingTestCase(unittest.TestCase): - timeout = 20 # seconds - duration = 1 + timeout = 4 # seconds + duration = .2 - def setUp(self): - class_name = self.__class__.__name__ + @classmethod + def setUpClass(cls): + class_name = cls.__name__ conf = SparkConf().set("spark.default.parallelism", 1) - self.sc = SparkContext(appName=class_name, conf=conf) - self.sc.setCheckpointDir("/tmp") - # TODO: decrease duration to speed up tests + cls.sc = SparkContext(appName=class_name, conf=conf) + cls.sc.setCheckpointDir("/tmp") + + @classmethod + def tearDownClass(cls): + cls.sc.stop() + + def setUp(self): self.ssc = StreamingContext(self.sc, self.duration) def tearDown(self): - self.ssc.stop() + self.ssc.stop(False) def wait_for(self, result, n): start_time = time.time() @@ -363,13 +378,13 @@ def func(dstream): class WindowFunctionTests(PySparkStreamingTestCase): - timeout = 20 + timeout = 5 def test_window(self): input = [range(1), range(2), range(3), range(4), range(5)] def func(dstream): - return dstream.window(3, 1).count() + return dstream.window(.6, .2).count() expected = [[1], [3], [6], [9], [12], [9], [5]] self._test_func(input, func, expected) @@ -378,7 +393,7 @@ def test_count_by_window(self): input = [range(1), range(2), range(3), range(4), range(5)] def func(dstream): - return dstream.countByWindow(3, 1) + return dstream.countByWindow(.6, .2) expected = [[1], [3], [6], [9], [12], [9], [5]] self._test_func(input, func, expected) @@ -387,7 +402,7 @@ def test_count_by_window_large(self): input = [range(1), range(2), range(3), range(4), range(5), range(6)] def func(dstream): - return dstream.countByWindow(5, 1) + return dstream.countByWindow(1, .2) expected = [[1], [3], [6], [10], [15], [20], [18], [15], [11], [6]] self._test_func(input, func, expected) @@ -396,7 +411,7 @@ def test_count_by_value_and_window(self): input = [range(1), range(2), range(3), range(4), range(5), range(6)] def func(dstream): - return dstream.countByValueAndWindow(5, 1) + return dstream.countByValueAndWindow(1, .2) expected = [[1], [2], [3], [4], [5], [6], [6], [6], [6], [6]] self._test_func(input, func, expected) @@ -405,7 +420,7 @@ def test_group_by_key_and_window(self): input = [[('a', i)] for i in range(5)] def func(dstream): - return dstream.groupByKeyAndWindow(3, 1).mapValues(list) + return dstream.groupByKeyAndWindow(.6, .2).mapValues(list) expected = [[('a', [0])], [('a', [0, 1])], [('a', [0, 1, 2])], [('a', [1, 2, 3])], [('a', [2, 3, 4])], [('a', [3, 4])], [('a', [4])]] @@ -436,8 +451,8 @@ def test_stop_only_streaming_context(self): def test_stop_multiple_times(self): self._add_input_stream() self.ssc.start() - self.ssc.stop() - self.ssc.stop() + self.ssc.stop(False) + self.ssc.stop(False) def test_queue_stream(self): input = [list(range(i + 1)) for i in range(3)] @@ -495,10 +510,7 @@ def func(rdds): self.assertEqual([2, 3, 1], self._take(dstream, 3)) -class CheckpointTests(PySparkStreamingTestCase): - - def setUp(self): - pass +class CheckpointTests(unittest.TestCase): def test_get_or_create(self): inputd = tempfile.mkdtemp() @@ -518,12 +530,12 @@ def setup(): return ssc cpd = tempfile.mkdtemp("test_streaming_cps") - self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup) + ssc = StreamingContext.getOrCreate(cpd, setup) ssc.start() def check_output(n): while not os.listdir(outputd): - time.sleep(0.1) + time.sleep(0.01) time.sleep(1) # make sure mtime is larger than the previous one with open(os.path.join(inputd, str(n)), 'w') as f: f.writelines(["%d\n" % i for i in range(10)]) @@ -553,12 +565,15 @@ def check_output(n): ssc.stop(True, True) time.sleep(1) - self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup) + ssc = StreamingContext.getOrCreate(cpd, setup) ssc.start() check_output(3) + ssc.stop(True, True) class KafkaStreamTests(PySparkStreamingTestCase): + timeout = 20 # seconds + duration = 1 def setUp(self): super(KafkaStreamTests, self).setUp() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 75f39d9e75f38..ea63a396da5b8 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -31,7 +31,6 @@ import time import zipfile import random -import itertools import threading import hashlib @@ -49,6 +48,11 @@ xrange = range basestring = str +if sys.version >= "3": + from io import StringIO +else: + from StringIO import StringIO + from pyspark.conf import SparkConf from pyspark.context import SparkContext @@ -196,7 +200,7 @@ def test_external_sort_in_rdd(self): sc = SparkContext(conf=conf) l = list(range(10240)) random.shuffle(l) - rdd = sc.parallelize(l, 2) + rdd = sc.parallelize(l, 4) self.assertEqual(sorted(l), rdd.sortBy(lambda x: x).collect()) sc.stop() @@ -300,6 +304,18 @@ def test_hash_serializer(self): hash(FlattenedValuesSerializer(PickleSerializer())) +class QuietTest(object): + def __init__(self, sc): + self.log4j = sc._jvm.org.apache.log4j + + def __enter__(self): + self.old_level = self.log4j.LogManager.getRootLogger().getLevel() + self.log4j.LogManager.getRootLogger().setLevel(self.log4j.Level.FATAL) + + def __exit__(self, exc_type, exc_val, exc_tb): + self.log4j.LogManager.getRootLogger().setLevel(self.old_level) + + class PySparkTestCase(unittest.TestCase): def setUp(self): @@ -371,15 +387,11 @@ def test_add_py_file(self): # To ensure that we're actually testing addPyFile's effects, check that # this job fails due to `userlibrary` not being on the Python path: # disable logging in log4j temporarily - log4j = self.sc._jvm.org.apache.log4j - old_level = log4j.LogManager.getRootLogger().getLevel() - log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) - def func(x): from userlibrary import UserClass return UserClass().hello() - self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) - log4j.LogManager.getRootLogger().setLevel(old_level) + with QuietTest(self.sc): + self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) # Add the file, so the job should now succeed: path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") @@ -496,7 +508,8 @@ def test_deleting_input_files(self): filtered_data = data.filter(lambda x: True) self.assertEqual(1, filtered_data.count()) os.unlink(tempFile.name) - self.assertRaises(Exception, lambda: filtered_data.count()) + with QuietTest(self.sc): + self.assertRaises(Exception, lambda: filtered_data.count()) def test_sampling_default_seed(self): # Test for SPARK-3995 (default seed setting) @@ -536,9 +549,9 @@ def test_namedtuple_in_rdd(self): self.assertEqual([jon, jane], theDoes.collect()) def test_large_broadcast(self): - N = 100000 + N = 10000 data = [[float(i) for i in range(300)] for i in range(N)] - bdata = self.sc.broadcast(data) # 270MB + bdata = self.sc.broadcast(data) # 27MB m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() self.assertEqual(N, m) @@ -569,7 +582,7 @@ def test_multiple_broadcasts(self): self.assertEqual(checksum, csum) def test_large_closure(self): - N = 1000000 + N = 200000 data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) self.assertEqual(N, rdd.first()) @@ -604,17 +617,18 @@ def test_zip_with_different_number_of_items(self): # different number of partitions b = self.sc.parallelize(range(100, 106), 3) self.assertRaises(ValueError, lambda: a.zip(b)) - # different number of batched items in JVM - b = self.sc.parallelize(range(100, 104), 2) - self.assertRaises(Exception, lambda: a.zip(b).count()) - # different number of items in one pair - b = self.sc.parallelize(range(100, 106), 2) - self.assertRaises(Exception, lambda: a.zip(b).count()) - # same total number of items, but different distributions - a = self.sc.parallelize([2, 3], 2).flatMap(range) - b = self.sc.parallelize([3, 2], 2).flatMap(range) - self.assertEqual(a.count(), b.count()) - self.assertRaises(Exception, lambda: a.zip(b).count()) + with QuietTest(self.sc): + # different number of batched items in JVM + b = self.sc.parallelize(range(100, 104), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # different number of items in one pair + b = self.sc.parallelize(range(100, 106), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # same total number of items, but different distributions + a = self.sc.parallelize([2, 3], 2).flatMap(range) + b = self.sc.parallelize([3, 2], 2).flatMap(range) + self.assertEqual(a.count(), b.count()) + self.assertRaises(Exception, lambda: a.zip(b).count()) def test_count_approx_distinct(self): rdd = self.sc.parallelize(range(1000)) @@ -877,7 +891,12 @@ def test_profiler(self): func_names = [func_name for fname, n, func_name in stat_list] self.assertTrue("heavy_foo" in func_names) + old_stdout = sys.stdout + sys.stdout = io = StringIO() self.sc.show_profiles() + self.assertTrue("heavy_foo" in io.getvalue()) + sys.stdout = old_stdout + d = tempfile.gettempdir() self.sc.dump_profiles(d) self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) @@ -901,7 +920,7 @@ def show(self, id): def do_computation(self): def heavy_foo(x): - for i in range(1 << 20): + for i in range(1 << 18): x = 1 rdd = self.sc.parallelize(range(100)) @@ -1417,7 +1436,7 @@ def test_termination_sigterm(self): self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) -class WorkerTests(PySparkTestCase): +class WorkerTests(ReusedPySparkTestCase): def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) temp.close() @@ -1432,7 +1451,10 @@ def sleep(x): # start job in background thread def run(): - self.sc.parallelize(range(1), 1).foreach(sleep) + try: + self.sc.parallelize(range(1), 1).foreach(sleep) + except Exception: + pass import threading t = threading.Thread(target=run) t.daemon = True @@ -1473,7 +1495,8 @@ def test_after_exception(self): def raise_exception(_): raise Exception() rdd = self.sc.parallelize(range(100), 1) - self.assertRaises(Exception, lambda: rdd.foreach(raise_exception)) + with QuietTest(self.sc): + self.assertRaises(Exception, lambda: rdd.foreach(raise_exception)) self.assertEqual(100, rdd.map(str).count()) def test_after_jvm_exception(self): @@ -1484,7 +1507,8 @@ def test_after_jvm_exception(self): filtered_data = data.filter(lambda x: True) self.assertEqual(1, filtered_data.count()) os.unlink(tempFile.name) - self.assertRaises(Exception, lambda: filtered_data.count()) + with QuietTest(self.sc): + self.assertRaises(Exception, lambda: filtered_data.count()) rdd = self.sc.parallelize(range(100), 1) self.assertEqual(100, rdd.map(str).count()) @@ -1522,14 +1546,11 @@ def test_with_different_versions_of_python(self): rdd.count() version = sys.version_info sys.version_info = (2, 0, 0) - log4j = self.sc._jvm.org.apache.log4j - old_level = log4j.LogManager.getRootLogger().getLevel() - log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) try: - self.assertRaises(Py4JJavaError, lambda: rdd.count()) + with QuietTest(self.sc): + self.assertRaises(Py4JJavaError, lambda: rdd.count()) finally: sys.version_info = version - log4j.LogManager.getRootLogger().setLevel(old_level) class SparkSubmitTests(unittest.TestCase): @@ -1751,9 +1772,14 @@ def test_with_stop(self): def test_progress_api(self): with SparkContext() as sc: sc.setJobGroup('test_progress_api', '', True) - rdd = sc.parallelize(range(10)).map(lambda x: time.sleep(100)) - t = threading.Thread(target=rdd.collect) + + def run(): + try: + rdd.count() + except Exception: + pass + t = threading.Thread(target=run) t.daemon = True t.start() # wait for scheduler to start diff --git a/python/run-tests b/python/run-tests index ed3e819ef30c1..88b63b84fdc27 100755 --- a/python/run-tests +++ b/python/run-tests @@ -28,6 +28,7 @@ cd "$FWDIR/python" FAILED=0 LOG_FILE=unit-tests.log +START=$(date +"%s") rm -f $LOG_FILE @@ -35,8 +36,8 @@ rm -f $LOG_FILE rm -rf metastore warehouse function run_test() { - echo "Running test: $1" | tee -a $LOG_FILE - + echo -en "Running test: $1 ... " | tee -a $LOG_FILE + start=$(date +"%s") SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 > $LOG_FILE 2>&1 FAILED=$((PIPESTATUS[0]||$FAILED)) @@ -48,6 +49,9 @@ function run_test() { echo "Had test failures; see logs." echo -en "\033[0m" # No color exit -1 + else + now=$(date +"%s") + echo "ok ($(($now - $start))s)" fi } @@ -161,9 +165,8 @@ if [ $(which pypy) ]; then fi if [[ $FAILED == 0 ]]; then - echo -en "\033[32m" # Green - echo "Tests passed." - echo -en "\033[0m" # No color + now=$(date +"%s") + echo -e "\033[32mTests passed \033[0min $(($now - $START)) seconds" fi # TODO: in the long-run, it would be nice to use a test runner like `nose`. From 41ef78a94105bb995bb14d15d47cbb6ca1638f62 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 21 Apr 2015 17:52:52 -0700 Subject: [PATCH 113/144] Closes #5427 From a0761ec7063f984dcadc8d154f83dd9cfd1c5e0b Mon Sep 17 00:00:00 2001 From: texasmichelle Date: Tue, 21 Apr 2015 18:08:29 -0700 Subject: [PATCH 114/144] [SPARK-1684] [PROJECT INFRA] Merge script should standardize SPARK-XXX prefix Cleans up the pull request title in the merge script to follow conventions outlined in the wiki under Contributing Code. https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-ContributingCode [MODULE] SPARK-XXXX: Description Author: texasmichelle Closes #5149 from texasmichelle/master and squashes the following commits: 9b6b0a7 [texasmichelle] resolved variable scope issue 7d5fa20 [texasmichelle] only prompt if title has been modified 8c195bb [texasmichelle] removed erroneous line 4f1ed46 [texasmichelle] Deque removal, logic simplifications, & prompt user to pick a title (orig or modified) df73f6a [texasmichelle] reworked regex's to enforce brackets around JIRA ref 43b5aed [texasmichelle] Merge remote-tracking branch 'apache/master' 25229c6 [texasmichelle] Merge remote-tracking branch 'apache/master' aa20a6e [texasmichelle] Move code into main() and add doctest for new text parsing method 48520ba [texasmichelle] SPARK-1684: Corrected import statement 042099d [texasmichelle] SPARK-1684 Merge script should standardize SPARK-XXX prefix 8f4a7d1 [texasmichelle] SPARK-1684 Merge script should standardize SPARK-XXX prefix --- dev/merge_spark_pr.py | 199 +++++++++++++++++++++++++++++------------- 1 file changed, 140 insertions(+), 59 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 3062e9c3c6651..b69cd15f99f63 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -55,8 +55,6 @@ # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" -os.chdir(SPARK_HOME) - def get_json(url): try: @@ -85,10 +83,6 @@ def continue_maybe(prompt): if result.lower() != "y": fail("Okay, exiting") - -original_head = run_cmd("git rev-parse HEAD")[:8] - - def clean_up(): print "Restoring head pointer to %s" % original_head run_cmd("git checkout %s" % original_head) @@ -101,7 +95,7 @@ def clean_up(): # merge the requested PR and return the merge hash -def merge_pr(pr_num, target_ref): +def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): pr_branch_name = "%s_MERGE_PR_%s" % (BRANCH_PREFIX, pr_num) target_branch_name = "%s_MERGE_PR_%s_%s" % (BRANCH_PREFIX, pr_num, target_ref.upper()) run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) @@ -274,7 +268,7 @@ def get_version_json(version_str): asf_jira.transition_issue( jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) - print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + print "Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) def resolve_jira_issues(title, merge_branches, comment): @@ -286,68 +280,155 @@ def resolve_jira_issues(title, merge_branches, comment): resolve_jira_issue(merge_branches, comment, jira_id) -branches = get_json("%s/branches" % GITHUB_API_BASE) -branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) -# Assumes branch names can be sorted lexicographically -latest_branch = sorted(branch_names, reverse=True)[0] - -pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") -pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) -pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) +def standardize_jira_ref(text): + """ + Standardize the [SPARK-XXXXX] [MODULE] prefix + Converts "[SPARK-XXX][mllib] Issue", "[MLLib] SPARK-XXX. Issue" or "SPARK XXX [MLLIB]: Issue" to "[SPARK-XXX] [MLLIB] Issue" + + >>> standardize_jira_ref("[SPARK-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful") + '[SPARK-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful' + >>> standardize_jira_ref("[SPARK-4123][Project Infra][WIP]: Show new dependencies added in pull requests") + '[SPARK-4123] [PROJECT INFRA] [WIP] Show new dependencies added in pull requests' + >>> standardize_jira_ref("[MLlib] Spark 5954: Top by key") + '[SPARK-5954] [MLLIB] Top by key' + >>> standardize_jira_ref("[SPARK-979] a LRU scheduler for load balancing in TaskSchedulerImpl") + '[SPARK-979] a LRU scheduler for load balancing in TaskSchedulerImpl' + >>> standardize_jira_ref("SPARK-1094 Support MiMa for reporting binary compatibility accross versions.") + '[SPARK-1094] Support MiMa for reporting binary compatibility accross versions.' + >>> standardize_jira_ref("[WIP] [SPARK-1146] Vagrant support for Spark") + '[SPARK-1146] [WIP] Vagrant support for Spark' + >>> standardize_jira_ref("SPARK-1032. If Yarn app fails before registering, app master stays aroun...") + '[SPARK-1032] If Yarn app fails before registering, app master stays aroun...' + >>> standardize_jira_ref("[SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser.") + '[SPARK-6250] [SPARK-6146] [SPARK-5911] [SQL] Types are now reserved words in DDL parser.' + >>> standardize_jira_ref("Additional information for users building from source code") + 'Additional information for users building from source code' + """ + jira_refs = [] + components = [] + + # If the string is compliant, no need to process any further + if (re.search(r'^\[SPARK-[0-9]{3,6}\] (\[[A-Z0-9_\s,]+\] )+\S+', text)): + return text + + # Extract JIRA ref(s): + pattern = re.compile(r'(SPARK[-\s]*[0-9]{3,6})+', re.IGNORECASE) + for ref in pattern.findall(text): + # Add brackets, replace spaces with a dash, & convert to uppercase + jira_refs.append('[' + re.sub(r'\s+', '-', ref.upper()) + ']') + text = text.replace(ref, '') + + # Extract spark component(s): + # Look for alphanumeric chars, spaces, dashes, periods, and/or commas + pattern = re.compile(r'(\[[\w\s,-\.]+\])', re.IGNORECASE) + for component in pattern.findall(text): + components.append(component.upper()) + text = text.replace(component, '') + + # Cleanup any remaining symbols: + pattern = re.compile(r'^\W+(.*)', re.IGNORECASE) + if (pattern.search(text) is not None): + text = pattern.search(text).groups()[0] + + # Assemble full text (JIRA ref(s), module(s), remaining text) + clean_text = ' '.join(jira_refs).strip() + " " + ' '.join(components).strip() + " " + text.strip() + + # Replace multiple spaces with a single space, e.g. if no jira refs and/or components were included + clean_text = re.sub(r'\s+', ' ', clean_text.strip()) + + return clean_text + +def main(): + global original_head + + os.chdir(SPARK_HOME) + original_head = run_cmd("git rev-parse HEAD")[:8] + + branches = get_json("%s/branches" % GITHUB_API_BASE) + branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) + # Assumes branch names can be sorted lexicographically + latest_branch = sorted(branch_names, reverse=True)[0] + + pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") + pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) + pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) + + url = pr["url"] + + # Decide whether to use the modified title or not + modified_title = standardize_jira_ref(pr["title"]) + if modified_title != pr["title"]: + print "I've re-written the title as follows to match the standard format:" + print "Original: %s" % pr["title"] + print "Modified: %s" % modified_title + result = raw_input("Would you like to use the modified title? (y/n): ") + if result.lower() == "y": + title = modified_title + print "Using modified title:" + else: + title = pr["title"] + print "Using original title:" + print title + else: + title = pr["title"] -url = pr["url"] -title = pr["title"] -body = pr["body"] -target_ref = pr["base"]["ref"] -user_login = pr["user"]["login"] -base_ref = pr["head"]["ref"] -pr_repo_desc = "%s/%s" % (user_login, base_ref) + body = pr["body"] + target_ref = pr["base"]["ref"] + user_login = pr["user"]["login"] + base_ref = pr["head"]["ref"] + pr_repo_desc = "%s/%s" % (user_login, base_ref) -# Merged pull requests don't appear as merged in the GitHub API; -# Instead, they're closed by asfgit. -merge_commits = \ - [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] + # Merged pull requests don't appear as merged in the GitHub API; + # Instead, they're closed by asfgit. + merge_commits = \ + [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] -if merge_commits: - merge_hash = merge_commits[0]["commit_id"] - message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] + if merge_commits: + merge_hash = merge_commits[0]["commit_id"] + message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] - print "Pull request %s has already been merged, assuming you want to backport" % pr_num - commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', + print "Pull request %s has already been merged, assuming you want to backport" % pr_num + commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', "%s^{commit}" % merge_hash]).strip() != "" - if not commit_is_downloaded: - fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) + if not commit_is_downloaded: + fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) - print "Found commit %s:\n%s" % (merge_hash, message) - cherry_pick(pr_num, merge_hash, latest_branch) - sys.exit(0) + print "Found commit %s:\n%s" % (merge_hash, message) + cherry_pick(pr_num, merge_hash, latest_branch) + sys.exit(0) -if not bool(pr["mergeable"]): - msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ - "Continue? (experts only!)" - continue_maybe(msg) + if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) -print ("\n=== Pull Request #%s ===" % pr_num) -print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( - title, pr_repo_desc, target_ref, url)) -continue_maybe("Proceed with merging pull request #%s?" % pr_num) + print ("\n=== Pull Request #%s ===" % pr_num) + print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( + title, pr_repo_desc, target_ref, url)) + continue_maybe("Proceed with merging pull request #%s?" % pr_num) -merged_refs = [target_ref] + merged_refs = [target_ref] -merge_hash = merge_pr(pr_num, target_ref) + merge_hash = merge_pr(pr_num, target_ref, title, body, pr_repo_desc) -pick_prompt = "Would you like to pick %s into another branch?" % merge_hash -while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": - merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + pick_prompt = "Would you like to pick %s into another branch?" % merge_hash + while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] -if JIRA_IMPORTED: - if JIRA_USERNAME and JIRA_PASSWORD: - continue_maybe("Would you like to update an associated JIRA?") - jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) - resolve_jira_issues(title, merged_refs, jira_comment) + if JIRA_IMPORTED: + if JIRA_USERNAME and JIRA_PASSWORD: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira_issues(title, merged_refs, jira_comment) + else: + print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Exiting without trying to close the associated JIRA." else: - print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." print "Exiting without trying to close the associated JIRA." -else: - print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." - print "Exiting without trying to close the associated JIRA." + +if __name__ == "__main__": + import doctest + doctest.testmod() + + main() From 3a3f7100f4ead9b7ac50e9711ac50b603ebf6bea Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 21 Apr 2015 18:37:53 -0700 Subject: [PATCH 115/144] [SPARK-6490][Docs] Add docs for rpc configurations Added docs for rpc configurations and also fixed two places that should have been fixed in #5595. Author: zsxwing Closes #5607 from zsxwing/SPARK-6490-docs and squashes the following commits: 25a6736 [zsxwing] Increase the default timeout to 120s 6e37c30 [zsxwing] Update docs 5577540 [zsxwing] Use spark.network.timeout as the default timeout if it presents 4f07174 [zsxwing] Fix unit tests 1c2cf26 [zsxwing] Add docs for rpc configurations --- .../org/apache/spark/util/RpcUtils.scala | 6 ++-- .../org/apache/spark/SparkConfSuite.scala | 2 +- .../org/apache/spark/rpc/RpcEnvSuite.scala | 2 +- docs/configuration.md | 34 +++++++++++++++++-- 4 files changed, 38 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala index 5ae793e0e87a3..f16cc8e7e42c6 100644 --- a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -48,11 +48,13 @@ object RpcUtils { /** Returns the default Spark timeout to use for RPC ask operations. */ def askTimeout(conf: SparkConf): FiniteDuration = { - conf.getTimeAsSeconds("spark.rpc.askTimeout", "30s") seconds + conf.getTimeAsSeconds("spark.rpc.askTimeout", + conf.get("spark.network.timeout", "120s")) seconds } /** Returns the default Spark timeout to use for RPC remote endpoint lookup. */ def lookupTimeout(conf: SparkConf): FiniteDuration = { - conf.getTimeAsSeconds("spark.rpc.lookupTimeout", "30s") seconds + conf.getTimeAsSeconds("spark.rpc.lookupTimeout", + conf.get("spark.network.timeout", "120s")) seconds } } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index d7d8014a20498..272e6af0514e4 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -227,7 +227,7 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro test("akka deprecated configs") { val conf = new SparkConf() - assert(!conf.contains("spark.rpc.num.retries")) + assert(!conf.contains("spark.rpc.numRetries")) assert(!conf.contains("spark.rpc.retry.wait")) assert(!conf.contains("spark.rpc.askTimeout")) assert(!conf.contains("spark.rpc.lookupTimeout")) diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 5fbda37c7cb88..44c88b00c442a 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -156,7 +156,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val conf = new SparkConf() conf.set("spark.rpc.retry.wait", "0") - conf.set("spark.rpc.num.retries", "1") + conf.set("spark.rpc.numRetries", "1") val anotherEnv = createRpcEnv(conf, "remote", 13345) // Use anotherEnv to find out the RpcEndpointRef val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-timeout") diff --git a/docs/configuration.md b/docs/configuration.md index d9e9e67026cbb..d587b91124cb8 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -963,8 +963,9 @@ Apart from these, the following properties are also available, and may be useful Default timeout for all network interactions. 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, + spark.shuffle.io.connectionTimeout, spark.rpc.askTimeout or + spark.rpc.lookupTimeout if they are not configured. @@ -982,6 +983,35 @@ Apart from these, the following properties are also available, and may be useful This is only relevant for the Spark shell. + + spark.rpc.numRetries + 3 + Number of times to retry before an RPC task gives up. + An RPC task will run at most times of this number. + + + + + spark.rpc.retry.wait + 3s + + Duration for an RPC ask operation to wait before retrying. + + + + spark.rpc.askTimeout + 120s + + Duration for an RPC ask operation to wait before timing out. + + + + spark.rpc.lookupTimeout + 120s + Duration for an RPC remote endpoint lookup operation to wait before timing out. + + + #### Scheduling From 70f9f8ff38560967f2c84de77263a5455c45c495 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 21 Apr 2015 21:04:04 -0700 Subject: [PATCH 116/144] [MINOR] Comment improvements in ExternalSorter. 1. Clearly specifies the contract/interactions for users of this class. 2. Minor fix in one doc to avoid ambiguity. Author: Patrick Wendell Closes #5620 from pwendell/cleanup and squashes the following commits: 8d8f44f [Patrick Wendell] [Minor] Comment improvements in ExternalSorter. --- .../util/collection/ExternalSorter.scala | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 79a1a8a0dae38..79a695fb62086 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -53,7 +53,18 @@ import org.apache.spark.storage.{BlockObjectWriter, BlockId} * probably want to pass None as the ordering to avoid extra sorting. On the other hand, if you do * want to do combining, having an Ordering is more efficient than not having it. * - * At a high level, this class works as follows: + * Users interact with this class in the following way: + * + * 1. Instantiate an ExternalSorter. + * + * 2. Call insertAll() with a set of records. + * + * 3. Request an iterator() back to traverse sorted/aggregated records. + * - or - + * Invoke writePartitionedFile() to create a file containing sorted/aggregated outputs + * that can be used in Spark's sort shuffle. + * + * At a high level, this class works internally as follows: * * - We repeatedly fill up buffers of in-memory data, using either a SizeTrackingAppendOnlyMap if * we want to combine by key, or an simple SizeTrackingBuffer if we don't. Inside these buffers, @@ -65,11 +76,11 @@ import org.apache.spark.storage.{BlockObjectWriter, BlockId} * aggregation. For each file, we track how many objects were in each partition in memory, so we * don't have to write out the partition ID for every element. * - * - When the user requests an iterator, the spilled files are merged, along with any remaining - * in-memory data, using the same sort order defined above (unless both sorting and aggregation - * are disabled). If we need to aggregate by key, we either use a total ordering from the - * ordering parameter, or read the keys with the same hash code and compare them with each other - * for equality to merge values. + * - When the user requests an iterator or file output, the spilled files are merged, along with + * any remaining in-memory data, using the same sort order defined above (unless both sorting + * and aggregation are disabled). If we need to aggregate by key, we either use a total ordering + * from the ordering parameter, or read the keys with the same hash code and compare them with + * each other for equality to merge values. * * - Users are expected to call stop() at the end to delete all the intermediate files. * @@ -259,8 +270,8 @@ private[spark] class ExternalSorter[K, V, C]( * Spill our in-memory collection to a sorted file that we can merge later (normal code path). * We add this file into spilledFiles to find it later. * - * Alternatively, if bypassMergeSort is true, we spill to separate files for each partition. - * See spillToPartitionedFiles() for that code path. + * This should not be invoked if bypassMergeSort is true. In that case, spillToPartitionedFiles() + * is used to write files for each partition. * * @param collection whichever collection we're using (map or buffer) */ From 607eff0edfc10a1473fa9713a0500bf09f105c13 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 21 Apr 2015 21:44:44 -0700 Subject: [PATCH 117/144] [SPARK-6113] [ML] Small cleanups after original tree API PR This does a few clean-ups. With this PR, all spark.ml tree components have ```private[ml]``` constructors. CC: mengxr Author: Joseph K. Bradley Closes #5567 from jkbradley/dt-api-dt2 and squashes the following commits: 2263b5b [Joseph K. Bradley] Added note about tree example issue. bb9f610 [Joseph K. Bradley] Small cleanups after original tree API PR --- .../examples/ml/DecisionTreeExample.scala | 25 ++++++++++++++----- .../spark/ml/impl/tree/treeParams.scala | 4 +-- .../org/apache/spark/ml/tree/Split.scala | 7 +++--- 3 files changed, 25 insertions(+), 11 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index 921b396e799e7..2cd515c89d3d2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -44,6 +44,13 @@ import org.apache.spark.sql.{SQLContext, DataFrame} * {{{ * ./bin/run-example ml.DecisionTreeExample [options] * }}} + * Note that Decision Trees can take a large amount of memory. If the run-example command above + * fails, try running via spark-submit and specifying the amount of memory as at least 1g. + * For local mode, run + * {{{ + * ./bin/spark-submit --class org.apache.spark.examples.ml.DecisionTreeExample --driver-memory 1g + * [examples JAR path] [options] + * }}} * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ object DecisionTreeExample { @@ -70,7 +77,7 @@ object DecisionTreeExample { val parser = new OptionParser[Params]("DecisionTreeExample") { head("DecisionTreeExample: an example decision tree app.") opt[String]("algo") - .text(s"algorithm (Classification, Regression), default: ${defaultParams.algo}") + .text(s"algorithm (classification, regression), default: ${defaultParams.algo}") .action((x, c) => c.copy(algo = x)) opt[Int]("maxDepth") .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") @@ -222,18 +229,23 @@ object DecisionTreeExample { // (1) For classification, re-index classes. val labelColName = if (algo == "classification") "indexedLabel" else "label" if (algo == "classification") { - val labelIndexer = new StringIndexer().setInputCol("labelString").setOutputCol(labelColName) + val labelIndexer = new StringIndexer() + .setInputCol("labelString") + .setOutputCol(labelColName) stages += labelIndexer } // (2) Identify categorical features using VectorIndexer. // Features with more than maxCategories values will be treated as continuous. - val featuresIndexer = new VectorIndexer().setInputCol("features") - .setOutputCol("indexedFeatures").setMaxCategories(10) + val featuresIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(10) stages += featuresIndexer // (3) Learn DecisionTree val dt = algo match { case "classification" => - new DecisionTreeClassifier().setFeaturesCol("indexedFeatures") + new DecisionTreeClassifier() + .setFeaturesCol("indexedFeatures") .setLabelCol(labelColName) .setMaxDepth(params.maxDepth) .setMaxBins(params.maxBins) @@ -242,7 +254,8 @@ object DecisionTreeExample { .setCacheNodeIds(params.cacheNodeIds) .setCheckpointInterval(params.checkpointInterval) case "regression" => - new DecisionTreeRegressor().setFeaturesCol("indexedFeatures") + new DecisionTreeRegressor() + .setFeaturesCol("indexedFeatures") .setLabelCol(labelColName) .setMaxDepth(params.maxDepth) .setMaxBins(params.maxBins) diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala index 6f4509f03d033..eb2609faef05a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala @@ -117,7 +117,7 @@ private[ml] trait DecisionTreeParams extends PredictorParams { def setMaxDepth(value: Int): this.type = { require(value >= 0, s"maxDepth parameter must be >= 0. Given bad value: $value") set(maxDepth, value) - this.asInstanceOf[this.type] + this } /** @group getParam */ @@ -283,7 +283,7 @@ private[ml] trait TreeRegressorParams extends Params { def getImpurity: String = getOrDefault(impurity) /** Convert new impurity to old impurity. */ - protected def getOldImpurity: OldImpurity = { + private[ml] def getOldImpurity: OldImpurity = { getImpurity match { case "variance" => OldVariance case _ => diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index cb940f62990ed..708c769087dd0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -38,7 +38,7 @@ sealed trait Split extends Serializable { private[tree] def toOld: OldSplit } -private[ml] object Split { +private[tree] object Split { def fromOld(oldSplit: OldSplit, categoricalFeatures: Map[Int, Int]): Split = { oldSplit.featureType match { @@ -58,7 +58,7 @@ private[ml] object Split { * left. Otherwise, it goes right. * @param numCategories Number of categories for this feature. */ -final class CategoricalSplit( +final class CategoricalSplit private[ml] ( override val featureIndex: Int, leftCategories: Array[Double], private val numCategories: Int) @@ -130,7 +130,8 @@ final class CategoricalSplit( * @param threshold If the feature value is <= this threshold, then the split goes left. * Otherwise, it goes right. */ -final class ContinuousSplit(override val featureIndex: Int, val threshold: Double) extends Split { +final class ContinuousSplit private[ml] (override val featureIndex: Int, val threshold: Double) + extends Split { override private[ml] def shouldGoLeft(features: Vector): Boolean = { features(featureIndex) <= threshold From bdc5c16e76c5d0bc147408353b2ba4faa8e914fc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 21 Apr 2015 22:34:31 -0700 Subject: [PATCH 118/144] [SPARK-6889] [DOCS] CONTRIBUTING.md updates to accompany contribution doc updates Part of the SPARK-6889 doc updates, to accompany wiki updates at https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark See draft text at https://docs.google.com/document/d/1tB9-f9lmxhC32QlOo4E8Z7eGDwHx1_Q3O8uCmRXQTo8/edit# Author: Sean Owen Closes #5623 from srowen/SPARK-6889 and squashes the following commits: 03773b1 [Sean Owen] Part of the SPARK-6889 doc updates, to accompany wiki updates at https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark --- CONTRIBUTING.md | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index b6c6b050fa331..f10d7e277eea3 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,12 +1,16 @@ ## Contributing to Spark -Contributions via GitHub pull requests are gladly accepted from their original -author. Along with any pull requests, please state that the contribution is -your original work and that you license the work to the project under the -project's open source license. Whether or not you state this explicitly, by -submitting any copyrighted material via pull request, email, or other means -you agree to license the material under the project's open source license and -warrant that you have the legal authority to do so. +*Before opening a pull request*, review the +[Contributing to Spark wiki](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark). +It lists steps that are required before creating a PR. In particular, consider: + +- Is the change important and ready enough to ask the community to spend time reviewing? +- Have you searched for existing, related JIRAs and pull requests? +- Is this a new feature that can stand alone as a package on http://spark-packages.org ? +- Is the change being proposed clearly explained and motivated? -Please see the [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) -for more information. +When you contribute code, you affirm that the contribution is your original work and that you +license the work to the project under the project's open source license. Whether or not you +state this explicitly, by submitting any copyrighted material via pull request, email, or +other means you agree to license the material under the project's open source license and +warrant that you have the legal authority to do so. From 33b85620f910c404873d362d27cca1223084913a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Apr 2015 11:08:59 -0700 Subject: [PATCH 119/144] [SPARK-7052][Core] Add ThreadUtils and move thread methods from Utils to ThreadUtils As per rxin 's suggestion in https://github.com/apache/spark/pull/5392/files#r28757176 What's more, there is a race condition in the global shared `daemonThreadFactoryBuilder`. `daemonThreadFactoryBuilder` may be modified by multiple threads. This PR removed the global `daemonThreadFactoryBuilder` and created a new `ThreadFactoryBuilder` every time. Author: zsxwing Closes #5631 from zsxwing/thread-utils and squashes the following commits: 9fe5b0e [zsxwing] Add ThreadUtils and move thread methods from Utils to ThreadUtils --- .../spark/ExecutorAllocationManager.scala | 8 +-- .../org/apache/spark/HeartbeatReceiver.scala | 11 ++- .../deploy/history/FsHistoryProvider.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 7 +- .../spark/network/nio/ConnectionManager.scala | 12 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../spark/scheduler/TaskResultGetter.scala | 4 +- .../CoarseGrainedSchedulerBackend.scala | 6 +- .../cluster/YarnSchedulerBackend.scala | 4 +- .../spark/scheduler/local/LocalBackend.scala | 8 +-- .../storage/BlockManagerMasterEndpoint.scala | 4 +- .../storage/BlockManagerSlaveEndpoint.scala | 4 +- .../org/apache/spark/util/ThreadUtils.scala | 67 +++++++++++++++++++ .../scala/org/apache/spark/util/Utils.scala | 29 -------- .../apache/spark/util/ThreadUtilsSuite.scala | 57 ++++++++++++++++ .../streaming/kafka/KafkaInputDStream.scala | 5 +- .../kafka/ReliableKafkaReceiver.scala | 4 +- .../receiver/ReceivedBlockHandler.scala | 4 +- .../streaming/util/WriteAheadLogManager.scala | 4 +- 19 files changed, 170 insertions(+), 76 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/ThreadUtils.scala create mode 100644 core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 4e7bf51fc0622..b986fa87dc2f4 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -17,12 +17,12 @@ package org.apache.spark -import java.util.concurrent.{Executors, TimeUnit} +import java.util.concurrent.TimeUnit import scala.collection.mutable import org.apache.spark.scheduler._ -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{ThreadUtils, Clock, SystemClock, Utils} /** * An agent that dynamically allocates and removes executors based on the workload. @@ -132,8 +132,8 @@ private[spark] class ExecutorAllocationManager( private val listener = new ExecutorAllocationListener // Executor that handles the scheduling task. - private val executor = Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("spark-dynamic-executor-allocation")) + private val executor = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-dynamic-executor-allocation") /** * Verify that the settings specified through the config are valid. diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index e3bd16f1cbf24..68d05d5b02537 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -17,7 +17,7 @@ package org.apache.spark -import java.util.concurrent.{ScheduledFuture, TimeUnit, Executors} +import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable @@ -25,7 +25,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEnv, RpcCallContext} import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler.{SlaveLost, TaskScheduler} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * A heartbeat from executors to the driver. This is a shared message used by several internal @@ -76,11 +76,10 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) private var timeoutCheckingTask: ScheduledFuture[_] = null - private val timeoutCheckingThread = Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("heartbeat-timeout-checking-thread")) + private val timeoutCheckingThread = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("heartbeat-timeout-checking-thread") - private val killExecutorThread = Executors.newSingleThreadExecutor( - Utils.namedThreadFactory("kill-executor-thread")) + private val killExecutorThread = ThreadUtils.newDaemonSingleThreadExecutor("kill-executor-thread") override def onStart(): Unit = { timeoutCheckingTask = timeoutCheckingThread.scheduleAtFixedRate(new Runnable { 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 47bdd7749ec3d..9847d5944a390 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 @@ -32,7 +32,7 @@ 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 +import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.{Logging, SecurityManager, SparkConf} @@ -99,7 +99,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private val replayExecutor: ExecutorService = { if (!conf.contains("spark.testing")) { - Executors.newSingleThreadExecutor(Utils.namedThreadFactory("log-replay-executor")) + ThreadUtils.newDaemonSingleThreadExecutor("log-replay-executor") } else { MoreExecutors.sameThreadExecutor() } 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 327d155b38c22..5fc04df5d6a40 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,7 +21,7 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, Executors, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -76,7 +76,7 @@ private[spark] class Executor( } // Start worker thread pool - private val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") + private val threadPool = ThreadUtils.newDaemonCachedThreadPool("Executor task launch worker") private val executorSource = new ExecutorSource(threadPool, executorId) if (!isLocal) { @@ -110,8 +110,7 @@ private[spark] class Executor( private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] // Executor for the heartbeat task. - private val heartbeater = Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("driver-heartbeater")) + private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") startDriverHeartbeater() 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 1a68e621eaee7..16e905982cf64 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 @@ -36,7 +36,7 @@ import io.netty.util.{Timeout, TimerTask, HashedWheelTimer} import org.apache.spark._ import org.apache.spark.network.sasl.{SparkSaslClient, SparkSaslServer} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} import scala.util.Try import scala.util.control.NonFatal @@ -79,7 +79,7 @@ private[nio] class ConnectionManager( private val selector = SelectorProvider.provider.openSelector() private val ackTimeoutMonitor = - new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) + new HashedWheelTimer(ThreadUtils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = conf.getTimeAsSeconds("spark.core.connection.ack.wait.timeout", @@ -102,7 +102,7 @@ private[nio] class ConnectionManager( handlerThreadCount, conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-message-executor")) { + ThreadUtils.namedThreadFactory("handle-message-executor")) { override def afterExecute(r: Runnable, t: Throwable): Unit = { super.afterExecute(r, t) @@ -117,7 +117,7 @@ private[nio] class ConnectionManager( ioThreadCount, conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-read-write-executor")) { + ThreadUtils.namedThreadFactory("handle-read-write-executor")) { override def afterExecute(r: Runnable, t: Throwable): Unit = { super.afterExecute(r, t) @@ -134,7 +134,7 @@ private[nio] class ConnectionManager( connectThreadCount, conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-connect-executor")) { + ThreadUtils.namedThreadFactory("handle-connect-executor")) { override def afterExecute(r: Runnable, t: Throwable): Unit = { super.afterExecute(r, t) @@ -160,7 +160,7 @@ private[nio] class ConnectionManager( private val registerRequests = new SynchronizedQueue[SendingConnection] implicit val futureExecContext = ExecutionContext.fromExecutor( - Utils.newDaemonCachedThreadPool("Connection manager future execution context")) + ThreadUtils.newDaemonCachedThreadPool("Connection manager future execution context")) @volatile private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message] = null 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 4a32f8936fb0e..8c4bff4e83afc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.util.Properties -import java.util.concurrent.{TimeUnit, Executors} +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} @@ -129,7 +129,7 @@ class DAGScheduler( private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false) private val messageScheduler = - Executors.newScheduledThreadPool(1, Utils.namedThreadFactory("dag-scheduler-message")) + ThreadUtils.newDaemonSingleThreadScheduledExecutor("dag-scheduler-message") private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) 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 3938580aeea59..391827c1d2156 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. @@ -35,7 +35,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul extends Logging { private val THREADS = sparkEnv.conf.getInt("spark.resultGetter.threads", 4) - private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( + private val getTaskResultExecutor = ThreadUtils.newDaemonFixedThreadPool( THREADS, "task-result-getter") protected val serializer = new ThreadLocal[SerializerInstance] { 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 63987dfb32695..9656fb76858ea 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 @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import java.util.concurrent.{TimeUnit, Executors} +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -26,7 +26,7 @@ import org.apache.spark.rpc._ import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{SerializableBuffer, AkkaUtils, Utils} +import org.apache.spark.util.{ThreadUtils, SerializableBuffer, AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -73,7 +73,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp private val addressToExecutorId = new HashMap[RpcAddress, String] private val reviveThread = - Executors.newSingleThreadScheduledExecutor(Utils.namedThreadFactory("driver-revive-thread")) + ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") override def onStart() { // Periodically revive offers to allow delay scheduling to work diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 1406a36a669c5..d987c7d563579 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -24,7 +24,7 @@ import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.ui.JettyUtils -import org.apache.spark.util.{RpcUtils, Utils} +import org.apache.spark.util.{ThreadUtils, RpcUtils} import scala.util.control.NonFatal @@ -97,7 +97,7 @@ private[spark] abstract class YarnSchedulerBackend( private var amEndpoint: Option[RpcEndpointRef] = None private val askAmThreadPool = - Utils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-thread-pool") + ThreadUtils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-thread-pool") implicit val askAmExecutor = ExecutionContext.fromExecutor(askAmThreadPool) override def receive: PartialFunction[Any, Unit] = { 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 50ba0b9d5a612..ac5b524517818 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 @@ -18,14 +18,14 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer -import java.util.concurrent.{Executors, TimeUnit} +import java.util.concurrent.TimeUnit import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcCallContext, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} private case class ReviveOffers() @@ -47,8 +47,8 @@ private[spark] class LocalEndpoint( private val totalCores: Int) extends ThreadSafeRpcEndpoint with Logging { - private val reviveThread = Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("local-revive-thread")) + private val reviveThread = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("local-revive-thread") private var freeCores = totalCores diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 28c73a7d543ff..4682167912ff0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -28,7 +28,7 @@ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * BlockManagerMasterEndpoint is an [[ThreadSafeRpcEndpoint]] on the master node to track statuses @@ -51,7 +51,7 @@ class BlockManagerMasterEndpoint( // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - private val askThreadPool = Utils.newDaemonCachedThreadPool("block-manager-ask-thread-pool") + private val askThreadPool = ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool") private implicit val askExecutionContext = ExecutionContext.fromExecutorService(askThreadPool) override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala index 8980fa8eb70e2..543df4e1350dd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.rpc.{RpcEnv, RpcCallContext, RpcEndpoint} -import org.apache.spark.util.Utils +import org.apache.spark.util.ThreadUtils import org.apache.spark.{Logging, MapOutputTracker, SparkEnv} import org.apache.spark.storage.BlockManagerMessages._ @@ -36,7 +36,7 @@ class BlockManagerSlaveEndpoint( extends RpcEndpoint with Logging { private val asyncThreadPool = - Utils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool") + ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool") private implicit val asyncExecutionContext = ExecutionContext.fromExecutorService(asyncThreadPool) // Operations that involve removing blocks may be slow and should be done asynchronously diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala new file mode 100644 index 0000000000000..098a4b79496b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.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.util + +import java.util.concurrent._ + +import com.google.common.util.concurrent.ThreadFactoryBuilder + +private[spark] object ThreadUtils { + + /** + * Create a thread factory that names threads with a prefix and also sets the threads to daemon. + */ + def namedThreadFactory(prefix: String): ThreadFactory = { + new ThreadFactoryBuilder().setDaemon(true).setNameFormat(prefix + "-%d").build() + } + + /** + * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. + */ + def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] + } + + /** + * Wrapper over newFixedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. + */ + def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor] + } + + /** + * Wrapper over newSingleThreadExecutor. + */ + def newDaemonSingleThreadExecutor(threadName: String): ExecutorService = { + val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() + Executors.newSingleThreadExecutor(threadFactory) + } + + /** + * Wrapper over newSingleThreadScheduledExecutor. + */ + def newDaemonSingleThreadScheduledExecutor(threadName: String): ScheduledExecutorService = { + val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() + Executors.newSingleThreadScheduledExecutor(threadFactory) + } +} 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 7b0de1ae55b78..2feb7341b159b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -35,7 +35,6 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.{ByteStreams, Files} import com.google.common.net.InetAddresses -import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} @@ -897,34 +896,6 @@ private[spark] object Utils extends Logging { hostPortParseResults.get(hostPort) } - private val daemonThreadFactoryBuilder: ThreadFactoryBuilder = - new ThreadFactoryBuilder().setDaemon(true) - - /** - * Create a thread factory that names threads with a prefix and also sets the threads to daemon. - */ - def namedThreadFactory(prefix: String): ThreadFactory = { - daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() - } - - /** - * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a - * unique, sequentially assigned integer. - */ - def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { - val threadFactory = namedThreadFactory(prefix) - Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] - } - - /** - * Wrapper over newFixedThreadPool. Thread names are formatted as prefix-ID, where ID is a - * unique, sequentially assigned integer. - */ - def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = { - val threadFactory = namedThreadFactory(prefix) - Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor] - } - /** * Return the string to tell how long has passed in milliseconds. */ diff --git a/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala new file mode 100644 index 0000000000000..a3aa3e953fbec --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.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.concurrent.{CountDownLatch, TimeUnit} + +import org.scalatest.FunSuite + +class ThreadUtilsSuite extends FunSuite { + + test("newDaemonSingleThreadExecutor") { + val executor = ThreadUtils.newDaemonSingleThreadExecutor("this-is-a-thread-name") + @volatile var threadName = "" + executor.submit(new Runnable { + override def run(): Unit = { + threadName = Thread.currentThread().getName() + } + }) + executor.shutdown() + executor.awaitTermination(10, TimeUnit.SECONDS) + assert(threadName === "this-is-a-thread-name") + } + + test("newDaemonSingleThreadScheduledExecutor") { + val executor = ThreadUtils.newDaemonSingleThreadScheduledExecutor("this-is-a-thread-name") + try { + val latch = new CountDownLatch(1) + @volatile var threadName = "" + executor.schedule(new Runnable { + override def run(): Unit = { + threadName = Thread.currentThread().getName() + latch.countDown() + } + }, 1, TimeUnit.MILLISECONDS) + latch.await(10, TimeUnit.SECONDS) + assert(threadName === "this-is-a-thread-name") + } finally { + executor.shutdownNow() + } + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 4d26b640e8d74..cca0fac0234e1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.Utils +import org.apache.spark.util.ThreadUtils /** * Input stream that pulls messages from a Kafka Broker. @@ -111,7 +111,8 @@ class KafkaReceiver[ val topicMessageStreams = consumerConnector.createMessageStreams( topics, keyDecoder, valueDecoder) - val executorPool = Utils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") + val executorPool = + ThreadUtils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") try { // Start the messages handler for each partition topicMessageStreams.values.foreach { streams => diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index c4a44c1822c39..ea87e960379f1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -33,7 +33,7 @@ import org.I0Itec.zkclient.ZkClient import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} -import org.apache.spark.util.Utils +import org.apache.spark.util.ThreadUtils /** * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager without loss. @@ -121,7 +121,7 @@ class ReliableKafkaReceiver[ zkClient = new ZkClient(consumerConfig.zkConnect, consumerConfig.zkSessionTimeoutMs, consumerConfig.zkConnectionTimeoutMs, ZKStringSerializer) - messageHandlerThreadPool = Utils.newDaemonFixedThreadPool( + messageHandlerThreadPool = ThreadUtils.newDaemonFixedThreadPool( topics.values.sum, "KafkaMessageHandler") blockGenerator.start() 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 dcdc27d29c270..297bf04c0c25e 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 @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage._ import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogManager} -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{ThreadUtils, Clock, SystemClock} /** Trait that represents the metadata related to storage of blocks */ private[streaming] trait ReceivedBlockStoreResult { @@ -150,7 +150,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler( // For processing futures used in parallel block storing into block manager and write ahead log // # threads = 2, so that both writing to BM and WAL can proceed in parallel implicit private val executionContext = ExecutionContext.fromExecutorService( - Utils.newDaemonFixedThreadPool(2, this.getClass.getSimpleName)) + ThreadUtils.newDaemonFixedThreadPool(2, this.getClass.getSimpleName)) /** * This implementation stores the block into the block manager as well as a write ahead log. 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 6bdfe45dc7f83..38a93cc3c9a1f 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 @@ -25,7 +25,7 @@ import scala.language.postfixOps import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.Logging -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{ThreadUtils, Clock, SystemClock} import WriteAheadLogManager._ /** @@ -60,7 +60,7 @@ private[streaming] class WriteAheadLogManager( if (callerName.nonEmpty) s" for $callerName" else "" private val threadpoolName = s"WriteAheadLogManager $callerNameTag" implicit private val executionContext = ExecutionContext.fromExecutorService( - Utils.newDaemonFixedThreadPool(1, threadpoolName)) + ThreadUtils.newDaemonSingleThreadExecutor(threadpoolName)) override protected val logName = s"WriteAheadLogManager $callerNameTag" private var currentLogPath: Option[String] = None From cdf0328684f70ddcd49b23c23c1532aeb9caa44e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 22 Apr 2015 11:18:01 -0700 Subject: [PATCH 120/144] [SQL] Rename some apply functions. I was looking at the code gen code and got confused by a few of use cases of apply, in particular apply on objects. So I went ahead and changed a few of them. Hopefully slightly more clear with a proper verb. Author: Reynold Xin Closes #5624 from rxin/apply-rename and squashes the following commits: ee45034 [Reynold Xin] [SQL] Rename some apply functions. --- .../sql/catalyst/AbstractSparkSQLParser.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../expressions/codegen/CodeGenerator.scala | 6 +-- .../codegen/GenerateMutableProjection.scala | 2 +- .../codegen/GenerateOrdering.scala | 2 +- .../codegen/GeneratePredicate.scala | 2 +- .../codegen/GenerateProjection.scala | 2 +- .../sql/catalyst/expressions/predicates.scala | 6 +-- .../sql/catalyst/rules/RuleExecutor.scala | 2 +- .../spark/sql/catalyst/SqlParserSuite.scala | 9 ++-- .../sql/catalyst/analysis/AnalysisSuite.scala | 22 ++++---- .../analysis/DecimalPrecisionSuite.scala | 6 +-- .../GeneratedEvaluationSuite.scala | 10 ++-- .../GeneratedMutableEvaluationSuite.scala | 8 +-- .../BooleanSimplificationSuite.scala | 2 +- .../optimizer/CombiningLimitsSuite.scala | 4 +- .../optimizer/ConstantFoldingSuite.scala | 14 ++--- .../ConvertToLocalRelationSuite.scala | 2 +- .../ExpressionOptimizationSuite.scala | 2 +- .../optimizer/FilterPushdownSuite.scala | 52 +++++++++---------- .../optimizer/LikeSimplificationSuite.scala | 8 +-- .../catalyst/optimizer/OptimizeInSuite.scala | 4 +- ...mplifyCaseConversionExpressionsSuite.scala | 8 +-- .../optimizer/UnionPushdownSuite.scala | 7 ++- .../catalyst/trees/RuleExecutorSuite.scala | 6 +-- .../org/apache/spark/sql/SQLContext.scala | 12 ++--- .../spark/sql/execution/SparkPlan.scala | 10 ++-- .../joins/BroadcastNestedLoopJoin.scala | 2 +- .../sql/execution/joins/LeftSemiJoinBNL.scala | 2 +- .../apache/spark/sql/parquet/newParquet.scala | 2 +- .../org/apache/spark/sql/sources/ddl.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 4 +- .../spark/sql/hive/HiveStrategies.scala | 4 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- 35 files changed, 117 insertions(+), 117 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala index 3823584287741..1f3c02478bd68 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala @@ -32,7 +32,7 @@ private[sql] object KeywordNormalizer { private[sql] abstract class AbstractSparkSQLParser extends StandardTokenParsers with PackratParsers { - def apply(input: String): LogicalPlan = { + def parse(input: String): LogicalPlan = { // Initialize the Keywords. lexical.initialize(reservedWords) phrase(start)(new lexical.Scanner(input)) match { 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 4e5c64bb63c9f..5d5aba9644ff7 100644 --- 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 @@ -296,7 +296,7 @@ package object dsl { InsertIntoTable( analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite, false) - def analyze: LogicalPlan = EliminateSubQueries(analysis.SimpleAnalyzer(logicalPlan)) + def analyze: LogicalPlan = EliminateSubQueries(analysis.SimpleAnalyzer.execute(logicalPlan)) } object plans { // scalastyle:ignore 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 be2c101d63a63..eeffedb558c1b 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 @@ -98,11 +98,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin }) /** Generates the requested evaluator binding the given expression(s) to the inputSchema. */ - def apply(expressions: InType, inputSchema: Seq[Attribute]): OutType = - apply(bind(expressions, inputSchema)) + def generate(expressions: InType, inputSchema: Seq[Attribute]): OutType = + generate(bind(expressions, inputSchema)) /** Generates the requested evaluator given already bound expression(s). */ - def apply(expressions: InType): OutType = cache.get(canonicalize(expressions)) + def generate(expressions: InType): OutType = cache.get(canonicalize(expressions)) /** * Returns a term name that is unique within this instance of a `CodeGenerator`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index a419fd7ecb39b..840260703ab74 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -30,7 +30,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => Mu val mutableRowName = newTermName("mutableRow") protected def canonicalize(in: Seq[Expression]): Seq[Expression] = - in.map(ExpressionCanonicalizer(_)) + in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = in.map(BindReferences.bindReference(_, inputSchema)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index fc2a2b60703e4..b129c0d898bb7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -30,7 +30,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit import scala.reflect.runtime.universe._ protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] = - in.map(ExpressionCanonicalizer(_).asInstanceOf[SortOrder]) + in.map(ExpressionCanonicalizer.execute(_).asInstanceOf[SortOrder]) protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] = in.map(BindReferences.bindReference(_, inputSchema)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index 2a0935c790cf3..40e163024360e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -26,7 +26,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] { import scala.reflect.runtime.{universe => ru} import scala.reflect.runtime.universe._ - protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer(in) + protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer.execute(in) protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = BindReferences.bindReference(in, inputSchema) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 6f572ff959fb4..d491babc2bff0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -31,7 +31,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { import scala.reflect.runtime.universe._ protected def canonicalize(in: Seq[Expression]): Seq[Expression] = - in.map(ExpressionCanonicalizer(_)) + in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = in.map(BindReferences.bindReference(_, inputSchema)) 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 fcd6352079b4d..46522eb9c1264 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 @@ -23,10 +23,10 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, NativeType} object InterpretedPredicate { - def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = - apply(BindReferences.bindReference(expression, inputSchema)) + def create(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = + create(BindReferences.bindReference(expression, inputSchema)) - def apply(expression: Expression): (Row => Boolean) = { + def create(expression: Expression): (Row => Boolean) = { (r: Row) => expression.eval(r).asInstanceOf[Boolean] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index c441f0bf24d85..3f9858b0c4a43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -45,7 +45,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { * Executes the batches of rules defined by the subclass. The batches are executed serially * using the defined execution strategy. Within each batch, rules are also executed serially. */ - def apply(plan: TreeType): TreeType = { + def execute(plan: TreeType): TreeType = { var curPlan = plan batches.foreach { batch => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala index 1a0a0e6154ad2..a652c70560990 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala @@ -49,13 +49,14 @@ class SqlParserSuite extends FunSuite { test("test long keyword") { val parser = new SuperLongKeywordTestParser - assert(TestCommand("NotRealCommand") === parser("ThisIsASuperLongKeyWordTest NotRealCommand")) + assert(TestCommand("NotRealCommand") === + parser.parse("ThisIsASuperLongKeyWordTest NotRealCommand")) } test("test case insensitive") { val parser = new CaseInsensitiveTestParser - assert(TestCommand("NotRealCommand") === parser("EXECUTE NotRealCommand")) - assert(TestCommand("NotRealCommand") === parser("execute NotRealCommand")) - assert(TestCommand("NotRealCommand") === parser("exEcute NotRealCommand")) + assert(TestCommand("NotRealCommand") === parser.parse("EXECUTE NotRealCommand")) + assert(TestCommand("NotRealCommand") === parser.parse("execute NotRealCommand")) + assert(TestCommand("NotRealCommand") === parser.parse("exEcute NotRealCommand")) } } 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 7c249215bd6b6..971e1ff5ec2b8 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 @@ -42,10 +42,10 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { def caseSensitiveAnalyze(plan: LogicalPlan): Unit = - caseSensitiveAnalyzer.checkAnalysis(caseSensitiveAnalyzer(plan)) + caseSensitiveAnalyzer.checkAnalysis(caseSensitiveAnalyzer.execute(plan)) def caseInsensitiveAnalyze(plan: LogicalPlan): Unit = - caseInsensitiveAnalyzer.checkAnalysis(caseInsensitiveAnalyzer(plan)) + caseInsensitiveAnalyzer.checkAnalysis(caseInsensitiveAnalyzer.execute(plan)) val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) val testRelation2 = LocalRelation( @@ -82,7 +82,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { a.select(UnresolvedStar(None)).select('a).unionAll(b.select(UnresolvedStar(None))) } - assert(caseInsensitiveAnalyzer(plan).resolved) + assert(caseInsensitiveAnalyzer.execute(plan).resolved) } test("check project's resolved") { @@ -98,11 +98,11 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { test("analyze project") { assert( - caseSensitiveAnalyzer(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + caseSensitiveAnalyzer.execute(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) assert( - caseSensitiveAnalyzer( + caseSensitiveAnalyzer.execute( Project(Seq(UnresolvedAttribute("TbL.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) @@ -115,13 +115,13 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(e.getMessage().toLowerCase.contains("cannot resolve")) assert( - caseInsensitiveAnalyzer( + caseInsensitiveAnalyzer.execute( Project(Seq(UnresolvedAttribute("TbL.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) assert( - caseInsensitiveAnalyzer( + caseInsensitiveAnalyzer.execute( Project(Seq(UnresolvedAttribute("tBl.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) @@ -134,13 +134,13 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(e.getMessage == "Table Not Found: tAbLe") assert( - caseSensitiveAnalyzer(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) + caseSensitiveAnalyzer.execute(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) assert( - caseInsensitiveAnalyzer(UnresolvedRelation(Seq("tAbLe"), None)) === testRelation) + caseInsensitiveAnalyzer.execute(UnresolvedRelation(Seq("tAbLe"), None)) === testRelation) assert( - caseInsensitiveAnalyzer(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) + caseInsensitiveAnalyzer.execute(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) } def errorTest( @@ -219,7 +219,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) - val plan = caseInsensitiveAnalyzer( + val plan = caseInsensitiveAnalyzer.execute( testRelation2.select( 'a / Literal(2) as 'div1, 'a / 'b as 'div2, 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 67bec999dfbd1..36b03d1c65e28 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 @@ -48,12 +48,12 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { private def checkType(expression: Expression, expectedType: DataType): Unit = { val plan = Project(Seq(Alias(expression, "c")()), relation) - assert(analyzer(plan).schema.fields(0).dataType === expectedType) + assert(analyzer.execute(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 { + val comparison = analyzer.execute(plan).collect { case Project(Alias(e: BinaryComparison, _) :: Nil, _) => e }.head assert(comparison.left.dataType === expectedType) @@ -64,7 +64,7 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { val plan = Union(Project(Seq(Alias(left, "l")()), relation), Project(Seq(Alias(right, "r")()), relation)) - val (l, r) = analyzer(plan).collect { + val (l, r) = analyzer.execute(plan).collect { case Union(left, right) => (left.output.head, right.output.head) }.head assert(l.dataType === expectedType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala index ef3114fd4dbab..b5ebe4b38e337 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala @@ -29,7 +29,7 @@ class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { expected: Any, inputRow: Row = EmptyRow): Unit = { val plan = try { - GenerateMutableProjection(Alias(expression, s"Optimized($expression)")() :: Nil)() + GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)() } catch { case e: Throwable => val evaluated = GenerateProjection.expressionEvaluator(expression) @@ -56,10 +56,10 @@ class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { val futures = (1 to 20).map { _ => future { - GeneratePredicate(EqualTo(Literal(1), Literal(1))) - GenerateProjection(EqualTo(Literal(1), Literal(1)) :: Nil) - GenerateMutableProjection(EqualTo(Literal(1), Literal(1)) :: Nil) - GenerateOrdering(Add(Literal(1), Literal(1)).asc :: Nil) + GeneratePredicate.generate(EqualTo(Literal(1), Literal(1))) + GenerateProjection.generate(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateMutableProjection.generate(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateOrdering.generate(Add(Literal(1), Literal(1)).asc :: Nil) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala index bcc0c404d2cfb..97af2e0fd0502 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -25,13 +25,13 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ */ class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { override def checkEvaluation( - expression: Expression, - expected: Any, - inputRow: Row = EmptyRow): Unit = { + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { lazy val evaluated = GenerateProjection.expressionEvaluator(expression) val plan = try { - GenerateProjection(Alias(expression, s"Optimized($expression)")() :: Nil) + GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil) } catch { case e: Throwable => fail( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 72f06e26e05f1..6255578d7fa57 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -61,7 +61,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { def checkCondition(input: Expression, expected: Expression): Unit = { val plan = testRelation.where(input).analyze - val actual = Optimize(plan).expressions.head + val actual = Optimize.execute(plan).expressions.head compareConditions(actual, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index e2ae0d25db1a5..2d16d668fd522 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -44,7 +44,7 @@ class CombiningLimitsSuite extends PlanTest { .limit(10) .limit(5) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) @@ -61,7 +61,7 @@ class CombiningLimitsSuite extends PlanTest { .limit(7) .limit(5) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 4396bd0dda9a9..14b28e8402610 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -47,7 +47,7 @@ class ConstantFoldingSuite extends PlanTest { .subquery('y) .select('a) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a.attr) @@ -74,7 +74,7 @@ class ConstantFoldingSuite extends PlanTest { Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)) )(Literal(9) / Literal(3) as Symbol("9/3")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -99,7 +99,7 @@ class ConstantFoldingSuite extends PlanTest { Literal(2) * 'a + Literal(4) as Symbol("c3"), 'a * (Literal(3) + Literal(4)) as Symbol("c4")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -127,7 +127,7 @@ class ConstantFoldingSuite extends PlanTest { (Literal(1) === Literal(1) || 'b > 1) && (Literal(1) === Literal(2) || 'b < 10))) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -144,7 +144,7 @@ class ConstantFoldingSuite extends PlanTest { Cast(Literal("2"), IntegerType) + Literal(3) + 'a as Symbol("c1"), Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as Symbol("c2")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -163,7 +163,7 @@ class ConstantFoldingSuite extends PlanTest { Rand + Literal(1) as Symbol("c1"), Sum('a) as Symbol("c2")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -210,7 +210,7 @@ class ConstantFoldingSuite extends PlanTest { Contains("abc", Literal.create(null, StringType)) as 'c20 ) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala index cf42d43823399..6841bd9890c97 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala @@ -49,7 +49,7 @@ class ConvertToLocalRelationSuite extends PlanTest { UnresolvedAttribute("a").as("a1"), (UnresolvedAttribute("b") + 1).as("b1")) - val optimized = Optimize(projectOnLocal.analyze) + val optimized = Optimize.execute(projectOnLocal.analyze) comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala index 2f3704be59a9d..a4a3a66b8b229 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -30,7 +30,7 @@ class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { expected: Any, inputRow: Row = EmptyRow): Unit = { val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, OneRowRelation) - val optimizedPlan = DefaultOptimizer(plan) + val optimizedPlan = DefaultOptimizer.execute(plan) super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 45cf695d20b01..aa9708b164efa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -50,7 +50,7 @@ class FilterPushdownSuite extends PlanTest { .subquery('y) .select('a) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a.attr) @@ -65,7 +65,7 @@ class FilterPushdownSuite extends PlanTest { .groupBy('a)('a, Count('b)) .select('a) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) @@ -81,7 +81,7 @@ class FilterPushdownSuite extends PlanTest { .groupBy('a)('a as 'c, Count('b)) .select('c) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) @@ -98,7 +98,7 @@ class FilterPushdownSuite extends PlanTest { .select('a) .where('a === 1) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where('a === 1) @@ -115,7 +115,7 @@ class FilterPushdownSuite extends PlanTest { .where('e === 1) .analyze - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where('a + 'b === 1) @@ -131,7 +131,7 @@ class FilterPushdownSuite extends PlanTest { .where('a === 1) .where('a === 2) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where('a === 1 && 'a === 2) @@ -152,7 +152,7 @@ class FilterPushdownSuite extends PlanTest { .where("y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val right = testRelation.where('b === 2) val correctAnswer = @@ -170,7 +170,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val right = testRelation val correctAnswer = @@ -188,7 +188,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val right = testRelation.where('b === 2) val correctAnswer = @@ -206,7 +206,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val correctAnswer = left.join(y, LeftOuter).where("y.b".attr === 2).analyze @@ -223,7 +223,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val right = testRelation.where('b === 2).subquery('d) val correctAnswer = x.join(right, RightOuter).where("x.b".attr === 1).analyze @@ -240,7 +240,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('d) val correctAnswer = left.join(y, LeftOuter, Some("d.b".attr === 1)).where("y.b".attr === 2).analyze @@ -257,7 +257,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val right = testRelation.where('b === 2).subquery('d) val correctAnswer = x.join(right, RightOuter, Some("d.b".attr === 1)).where("x.b".attr === 2).analyze @@ -274,7 +274,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('l) val right = testRelation.where('b === 1).subquery('r) val correctAnswer = @@ -292,7 +292,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val right = testRelation.where('b === 2).subquery('r) val correctAnswer = x.join(right, RightOuter, Some("r.b".attr === 1)).where("x.b".attr === 2).analyze @@ -309,7 +309,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('l) val right = testRelation.where('b === 1).subquery('r) val correctAnswer = @@ -327,7 +327,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.subquery('l) val right = testRelation.where('b === 2).subquery('r) val correctAnswer = @@ -346,7 +346,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('l) val right = testRelation.where('b === 1).subquery('r) val correctAnswer = @@ -365,7 +365,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('a === 3).subquery('l) val right = testRelation.where('b === 2).subquery('r) val correctAnswer = @@ -382,7 +382,7 @@ class FilterPushdownSuite extends PlanTest { val originalQuery = { x.join(y, condition = Some("x.b".attr === "y.b".attr)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) comparePlans(analysis.EliminateSubQueries(originalQuery.analyze), optimized) } @@ -396,7 +396,7 @@ class FilterPushdownSuite extends PlanTest { .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("y.a".attr === 1)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('a === 1).subquery('x) val right = testRelation.where('a === 1).subquery('y) val correctAnswer = @@ -415,7 +415,7 @@ class FilterPushdownSuite extends PlanTest { .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('a === 1).subquery('x) val right = testRelation.subquery('y) val correctAnswer = @@ -436,7 +436,7 @@ class FilterPushdownSuite extends PlanTest { ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val lleft = testRelation.where('a >= 3).subquery('z) val left = testRelation.where('a === 1).subquery('x) val right = testRelation.subquery('y) @@ -457,7 +457,7 @@ class FilterPushdownSuite extends PlanTest { .generate(Explode('c_arr), true, false, Some("arr")) .where(('b >= 5) && ('a > 6)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = { testRelationWithArrayType .where(('b >= 5) && ('a > 6)) @@ -474,7 +474,7 @@ class FilterPushdownSuite extends PlanTest { .generate(generator, true, false, Some("arr")) .where(('b >= 5) && ('c > 6)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val referenceResult = { testRelationWithArrayType .where('b >= 5) @@ -502,7 +502,7 @@ class FilterPushdownSuite extends PlanTest { .generate(Explode('c_arr), true, false, Some("arr")) .where(('c > 6) || ('b > 5)).analyze } - val optimized = Optimize(originalQuery) + val optimized = Optimize.execute(originalQuery) comparePlans(optimized, originalQuery) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala index b10577c8001e2..b3df487c84dc8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala @@ -41,7 +41,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where(('a like "abc%") || ('a like "abc\\%")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(StartsWith('a, "abc") || ('a like "abc\\%")) .analyze @@ -54,7 +54,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where('a like "%xyz") - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(EndsWith('a, "xyz")) .analyze @@ -67,7 +67,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where(('a like "%mn%") || ('a like "%mn\\%")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(Contains('a, "mn") || ('a like "%mn\\%")) .analyze @@ -80,7 +80,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where(('a like "") || ('a like "abc")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(('a === "") || ('a === "abc")) .analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index 966bc9ada1e6e..3eb399e68e70c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -49,7 +49,7 @@ class OptimizeInSuite extends PlanTest { .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2)))) .analyze - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(InSet(UnresolvedAttribute("a"), HashSet[Any]() + 1 + 2)) @@ -64,7 +64,7 @@ class OptimizeInSuite extends PlanTest { .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2), UnresolvedAttribute("b")))) .analyze - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2), UnresolvedAttribute("b")))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala index 22992fb6f50d4..6b1e53cd42b24 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala @@ -41,7 +41,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Upper(Upper('a)) as 'u) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Upper('a) as 'u) @@ -55,7 +55,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Upper(Lower('a)) as 'u) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Upper('a) as 'u) @@ -69,7 +69,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Lower(Upper('a)) as 'l) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Lower('a) as 'l) .analyze @@ -82,7 +82,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Lower(Lower('a)) as 'l) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Lower('a) as 'l) .analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala index a54751dfa9a12..a3ad200800b02 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -41,7 +40,7 @@ class UnionPushdownSuite extends PlanTest { test("union: filter to each side") { val query = testUnion.where('a === 1) - val optimized = Optimize(query.analyze) + val optimized = Optimize.execute(query.analyze) val correctAnswer = Union(testRelation.where('a === 1), testRelation2.where('d === 1)).analyze @@ -52,7 +51,7 @@ class UnionPushdownSuite extends PlanTest { test("union: project to each side") { val query = testUnion.select('b) - val optimized = Optimize(query.analyze) + val optimized = Optimize.execute(query.analyze) val correctAnswer = Union(testRelation.select('b), testRelation2.select('e)).analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala index 4b2d45584045f..2a641c63f87bb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala @@ -34,7 +34,7 @@ class RuleExecutorSuite extends FunSuite { val batches = Batch("once", Once, DecrementLiterals) :: Nil } - assert(ApplyOnce(Literal(10)) === Literal(9)) + assert(ApplyOnce.execute(Literal(10)) === Literal(9)) } test("to fixed point") { @@ -42,7 +42,7 @@ class RuleExecutorSuite extends FunSuite { val batches = Batch("fixedPoint", FixedPoint(100), DecrementLiterals) :: Nil } - assert(ToFixedPoint(Literal(10)) === Literal(0)) + assert(ToFixedPoint.execute(Literal(10)) === Literal(0)) } test("to maxIterations") { @@ -50,6 +50,6 @@ class RuleExecutorSuite extends FunSuite { val batches = Batch("fixedPoint", FixedPoint(10), DecrementLiterals) :: Nil } - assert(ToFixedPoint(Literal(100)) === Literal(90)) + assert(ToFixedPoint.execute(Literal(100)) === Literal(90)) } } 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 bcd20c06c6dca..a279b0f07c38a 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 @@ -132,16 +132,16 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer @transient - protected[sql] val ddlParser = new DDLParser(sqlParser.apply(_)) + protected[sql] val ddlParser = new DDLParser(sqlParser.parse(_)) @transient protected[sql] val sqlParser = { val fallback = new catalyst.SqlParser - new SparkSQLParser(fallback(_)) + new SparkSQLParser(fallback.parse(_)) } protected[sql] def parseSql(sql: String): LogicalPlan = { - ddlParser(sql, false).getOrElse(sqlParser(sql)) + ddlParser.parse(sql, false).getOrElse(sqlParser.parse(sql)) } protected[sql] def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) @@ -1120,12 +1120,12 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] class QueryExecution(val logical: LogicalPlan) { def assertAnalyzed(): Unit = analyzer.checkAnalysis(analyzed) - lazy val analyzed: LogicalPlan = analyzer(logical) + lazy val analyzed: LogicalPlan = analyzer.execute(logical) lazy val withCachedData: LogicalPlan = { assertAnalyzed() cacheManager.useCachedData(analyzed) } - lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData) + lazy val optimizedPlan: LogicalPlan = optimizer.execute(withCachedData) // TODO: Don't just pick the first one... lazy val sparkPlan: SparkPlan = { @@ -1134,7 +1134,7 @@ class SQLContext(@transient val sparkContext: SparkContext) } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. - lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan) + lazy val executedPlan: SparkPlan = prepareForExecution.execute(sparkPlan) /** Internal version of the RDD. Avoids copies and has no schema */ lazy val toRdd: RDD[Row] = executedPlan.execute() 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 e159ffe66cb24..59c89800da00f 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 @@ -144,7 +144,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ log.debug( s"Creating Projection: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") if (codegenEnabled) { - GenerateProjection(expressions, inputSchema) + GenerateProjection.generate(expressions, inputSchema) } else { new InterpretedProjection(expressions, inputSchema) } @@ -156,7 +156,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ log.debug( s"Creating MutableProj: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") if(codegenEnabled) { - GenerateMutableProjection(expressions, inputSchema) + GenerateMutableProjection.generate(expressions, inputSchema) } else { () => new InterpretedMutableProjection(expressions, inputSchema) } @@ -166,15 +166,15 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ protected def newPredicate( expression: Expression, inputSchema: Seq[Attribute]): (Row) => Boolean = { if (codegenEnabled) { - GeneratePredicate(expression, inputSchema) + GeneratePredicate.generate(expression, inputSchema) } else { - InterpretedPredicate(expression, inputSchema) + InterpretedPredicate.create(expression, inputSchema) } } protected def newOrdering(order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[Row] = { if (codegenEnabled) { - GenerateOrdering(order, inputSchema) + GenerateOrdering.generate(order, inputSchema) } else { new RowOrdering(order, inputSchema) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index 83b1a83765153..56200f6b8c8a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -59,7 +59,7 @@ case class BroadcastNestedLoopJoin( } @transient private lazy val boundCondition = - InterpretedPredicate( + InterpretedPredicate.create( condition .map(c => BindReferences.bindReference(c, left.output ++ right.output)) .getOrElse(Literal(true))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 1fa7e7bd0406c..e06f63f94b78b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -45,7 +45,7 @@ case class LeftSemiJoinBNL( override def right: SparkPlan = broadcast @transient private lazy val boundCondition = - InterpretedPredicate( + InterpretedPredicate.create( condition .map(c => BindReferences.bindReference(c, left.output ++ right.output)) .getOrElse(Literal(true))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index af7b3c81ae7b2..88466f52bd4e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -611,7 +611,7 @@ private[sql] case class ParquetRelation2( val rawPredicate = partitionPruningPredicates.reduceOption(expressions.And).getOrElse(Literal(true)) - val boundPredicate = InterpretedPredicate(rawPredicate transform { + val boundPredicate = InterpretedPredicate.create(rawPredicate transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) BoundReference(index, partitionColumns(index).dataType, nullable = true) 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 78d494184e759..e7a0685e013d8 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 @@ -38,9 +38,9 @@ private[sql] class DDLParser( parseQuery: String => LogicalPlan) extends AbstractSparkSQLParser with DataTypeParser with Logging { - def apply(input: String, exceptionOnError: Boolean): Option[LogicalPlan] = { + def parse(input: String, exceptionOnError: Boolean): Option[LogicalPlan] = { try { - Some(apply(input)) + Some(parse(input)) } catch { case ddlException: DDLException => throw ddlException case _ if !exceptionOnError => None 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 c4a73b3004076..dd06b2620c5ee 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 @@ -93,7 +93,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { if (conf.dialect == "sql") { super.sql(substituted) } else if (conf.dialect == "hiveql") { - val ddlPlan = ddlParserWithHiveQL(sqlText, exceptionOnError = false) + val ddlPlan = ddlParserWithHiveQL.parse(sqlText, exceptionOnError = false) DataFrame(this, ddlPlan.getOrElse(HiveQl.parseSql(substituted))) } else { sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") 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 85061f22772dd..0ea6d57b816c6 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 @@ -144,7 +144,7 @@ private[hive] object HiveQl { protected val hqlParser = { val fallback = new ExtendedHiveQlParser - new SparkSQLParser(fallback(_)) + new SparkSQLParser(fallback.parse(_)) } /** @@ -240,7 +240,7 @@ private[hive] object HiveQl { /** Returns a LogicalPlan for a given HiveQL string. */ - def parseSql(sql: String): LogicalPlan = hqlParser(sql) + def parseSql(sql: String): LogicalPlan = hqlParser.parse(sql) val errorRegEx = "line (\\d+):(\\d+) (.*)".r 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 a6f4fbe8aba06..be9249a8b1f44 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 @@ -119,9 +119,9 @@ private[hive] trait HiveStrategies { val inputData = new GenericMutableRow(relation.partitionKeys.size) val pruningCondition = if (codegenEnabled) { - GeneratePredicate(castedPredicate) + GeneratePredicate.generate(castedPredicate) } else { - InterpretedPredicate(castedPredicate) + InterpretedPredicate.create(castedPredicate) } val partitions = relation.hiveQlPartitions.filter { part => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 6570fa1043900..9f17bca083d13 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -185,7 +185,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. - analyzer(logical) + analyzer.execute(logical) } } From fbe7106d75c6a1624d10793fba6759703bc5c6e6 Mon Sep 17 00:00:00 2001 From: szheng79 Date: Wed, 22 Apr 2015 13:02:55 -0700 Subject: [PATCH 121/144] [SPARK-7039][SQL]JDBCRDD: Add support on type NVARCHAR Issue: https://issues.apache.org/jira/browse/SPARK-7039 Add support to column type NVARCHAR in Sql Server java.sql.Types: http://docs.oracle.com/javase/7/docs/api/java/sql/Types.html Author: szheng79 Closes #5618 from szheng79/patch-1 and squashes the following commits: 10da99c [szheng79] Update JDBCRDD.scala eab0bd8 [szheng79] Add support on type NVARCHAR --- sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index b9022fcd9e3ad..8b1edec20feee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -60,6 +60,7 @@ private[sql] object JDBCRDD extends Logging { case java.sql.Types.NCLOB => StringType case java.sql.Types.NULL => null case java.sql.Types.NUMERIC => DecimalType.Unlimited + case java.sql.Types.NVARCHAR => StringType case java.sql.Types.OTHER => null case java.sql.Types.REAL => DoubleType case java.sql.Types.REF => StringType From baf865ddc2cff9b99d6aeab9861e030da511257f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 22 Apr 2015 15:26:58 -0700 Subject: [PATCH 122/144] [SPARK-7059][SQL] Create a DataFrame join API to facilitate equijoin. Author: Reynold Xin Closes #5638 from rxin/joinUsing and squashes the following commits: 13e9cc9 [Reynold Xin] Code review + Python. b1bd914 [Reynold Xin] [SPARK-7059][SQL] Create a DataFrame join API to facilitate equijoin and self join. --- python/pyspark/sql/dataframe.py | 9 ++++- .../org/apache/spark/sql/DataFrame.scala | 37 +++++++++++++++++ .../org/apache/spark/sql/DataFrameSuite.scala | 40 ++++++++++++++----- 3 files changed, 74 insertions(+), 12 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ca9bf8efb945c..c8c30ce4022c8 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -459,16 +459,23 @@ def join(self, other, joinExprs=None, joinType=None): The following performs a full outer join between ``df1`` and ``df2``. :param other: Right side of the join - :param joinExprs: Join expression + :param joinExprs: a string for join column name, or a join expression (Column). + If joinExprs is a string indicating the name of the join column, + the column must exist on both sides, and this performs an inner equi-join. :param joinType: str, default 'inner'. One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() [Row(name=None, height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] + + >>> df.join(df2, 'name').select(df.name, df2.height).collect() + [Row(name=u'Bob', height=85)] """ if joinExprs is None: jdf = self._jdf.join(other._jdf) + elif isinstance(joinExprs, basestring): + jdf = self._jdf.join(other._jdf, joinExprs) else: assert isinstance(joinExprs, Column), "joinExprs should be Column" if joinType is None: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 03d9834d1d131..ca6ae482eb2ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -342,6 +342,43 @@ class DataFrame private[sql]( Join(logicalPlan, right.logicalPlan, joinType = Inner, None) } + /** + * Inner equi-join with another [[DataFrame]] using the given column. + * + * Different from other join functions, the join column will only appear once in the output, + * i.e. similar to SQL's `JOIN USING` syntax. + * + * {{{ + * // Joining df1 and df2 using the column "user_id" + * df1.join(df2, "user_id") + * }}} + * + * Note that if you perform a self-join using this function without aliasing the input + * [[DataFrame]]s, you will NOT be able to reference any columns after the join, since + * there is no way to disambiguate which side of the join you would like to reference. + * + * @param right Right side of the join operation. + * @param usingColumn Name of the column to join on. This column must exist on both sides. + * @group dfops + */ + def join(right: DataFrame, usingColumn: String): DataFrame = { + // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right + // by creating a new instance for one of the branch. + val joined = sqlContext.executePlan( + Join(logicalPlan, right.logicalPlan, joinType = Inner, None)).analyzed.asInstanceOf[Join] + + // Project only one of the join column. + val joinedCol = joined.right.resolve(usingColumn) + Project( + joined.output.filterNot(_ == joinedCol), + Join( + joined.left, + joined.right, + joinType = Inner, + Some(EqualTo(joined.left.resolve(usingColumn), joined.right.resolve(usingColumn)))) + ) + } + /** * Inner join with another [[DataFrame]], using the given join expression. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index b9b6a400ae195..5ec06d448e50f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -109,15 +109,6 @@ class DataFrameSuite extends QueryTest { assert(testData.head(2).head.schema === testData.schema) } - test("self join") { - val df1 = testData.select(testData("key")).as('df1) - val df2 = testData.select(testData("key")).as('df2) - - checkAnswer( - df1.join(df2, $"df1.key" === $"df2.key"), - sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) - } - test("simple explode") { val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDF("words") @@ -127,8 +118,35 @@ class DataFrameSuite extends QueryTest { ) } - test("self join with aliases") { - val df = Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str") + test("join - join using") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + val df2 = Seq(1, 2, 3).map(i => (i, (i + 1).toString)).toDF("int", "str") + + checkAnswer( + df.join(df2, "int"), + Row(1, "1", "2") :: Row(2, "2", "3") :: Row(3, "3", "4") :: Nil) + } + + test("join - join using self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + + // self join + checkAnswer( + df.join(df, "int"), + Row(1, "1", "1") :: Row(2, "2", "2") :: Row(3, "3", "3") :: Nil) + } + + test("join - self join") { + val df1 = testData.select(testData("key")).as('df1) + val df2 = testData.select(testData("key")).as('df2) + + checkAnswer( + df1.join(df2, $"df1.key" === $"df2.key"), + sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) + } + + test("join - using aliases after self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") checkAnswer( df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) From f4f39981f4f5e88c30eec7d0b107e2c3cdc268c9 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 22 Apr 2015 17:22:26 -0700 Subject: [PATCH 123/144] [SPARK-6827] [MLLIB] Wrap FPGrowthModel.freqItemsets and make it consistent with Java API Make PySpark ```FPGrowthModel.freqItemsets``` consistent with Java/Scala API like ```MatrixFactorizationModel.userFeatures``` It return a RDD with each tuple is composed of an array and a long value. I think it's difficult to implement namedtuples to wrap the output because items of freqItemsets can be any type with arbitrary length which is tedious to impelement corresponding SerDe function. Author: Yanbo Liang Closes #5614 from yanboliang/spark-6827 and squashes the following commits: da8c404 [Yanbo Liang] use namedtuple 5532e78 [Yanbo Liang] Wrap FPGrowthModel.freqItemsets and make it consistent with Java API --- python/pyspark/mllib/fpm.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index 628ccc01cf3cc..d8df02bdbaba9 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -15,6 +15,10 @@ # limitations under the License. # +import numpy +from numpy import array +from collections import namedtuple + from pyspark import SparkContext from pyspark.rdd import ignore_unicode_prefix from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc @@ -36,14 +40,14 @@ class FPGrowthModel(JavaModelWrapper): >>> rdd = sc.parallelize(data, 2) >>> model = FPGrowth.train(rdd, 0.6, 2) >>> sorted(model.freqItemsets().collect()) - [([u'a'], 4), ([u'c'], 3), ([u'c', u'a'], 3)] + [FreqItemset(items=[u'a'], freq=4), FreqItemset(items=[u'c'], freq=3), ... """ def freqItemsets(self): """ - Get the frequent itemsets of this model + Returns the frequent itemsets of this model. """ - return self.call("getFreqItemsets") + return self.call("getFreqItemsets").map(lambda x: (FPGrowth.FreqItemset(x[0], x[1]))) class FPGrowth(object): @@ -67,6 +71,11 @@ def train(cls, data, minSupport=0.3, numPartitions=-1): model = callMLlibFunc("trainFPGrowthModel", data, float(minSupport), int(numPartitions)) return FPGrowthModel(model) + class FreqItemset(namedtuple("FreqItemset", ["items", "freq"])): + """ + Represents an (items, freq) tuple. + """ + def _test(): import doctest From 04525c077c638a7e615c294ba988e35036554f5f Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 22 Apr 2015 19:14:28 -0700 Subject: [PATCH 124/144] [SPARK-6967] [SQL] fix date type convertion in jdbcrdd This pr convert java.sql.Date type into Int for JDBCRDD. Author: Daoyuan Wang Closes #5590 from adrian-wang/datebug and squashes the following commits: f897b81 [Daoyuan Wang] add a test case 3c9184c [Daoyuan Wang] fix date type convertion in jdbcrdd --- .../src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala | 4 ++-- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 7 +++++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 8b1edec20feee..b975191d41963 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -350,8 +350,8 @@ private[sql] class JDBCRDD( val pos = i + 1 conversions(i) match { case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) - // TODO(davies): convert Date into Int - case DateConversion => mutableRow.update(i, rs.getDate(pos)) + case DateConversion => + mutableRow.update(i, DateUtils.fromJavaDate(rs.getDate(pos))) case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 3596b183d4328..db096af4535a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -249,6 +249,13 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { assert(rows(0).getAs[java.sql.Timestamp](2).getNanos === 543543543) } + test("test DATE types") { + val rows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.TIMETYPES").collect() + val cachedRows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.TIMETYPES").cache().collect() + assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) + assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) + } + test("H2 floating-point types") { val rows = sql("SELECT * FROM flttypes").collect() assert(rows(0).getDouble(0) === 1.00000000000000022) // Yes, I meant ==. From b69c4f9b2e8544f1b178db2aefbcaa166f76cb7a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 22 Apr 2015 21:24:22 -0700 Subject: [PATCH 125/144] Disable flaky test: ReceiverSuite "block generator throttling". --- .../test/scala/org/apache/spark/streaming/ReceiverSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index e7aee6eadbfc7..b84129fd70dd4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -155,7 +155,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { assert(recordedData.toSet === generatedData.toSet) } - test("block generator throttling") { + ignore("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener val blockIntervalMs = 100 val maxRate = 1001 From 1b85e08509a0e19dc35b6ab869977254156cdaf1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 22 Apr 2015 21:35:12 -0700 Subject: [PATCH 126/144] [MLlib] UnaryTransformer nullability should not depend on PrimitiveType. Author: Reynold Xin Closes #5644 from rxin/mllib-nullable and squashes the following commits: a727e5b [Reynold Xin] [MLlib] UnaryTransformer nullability should not depend on primitive types. --- mllib/src/main/scala/org/apache/spark/ml/Transformer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 7fb87fe452ee6..0acda71ec6045 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -94,7 +94,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O throw new IllegalArgumentException(s"Output column ${map(outputCol)} already exists.") } val outputFields = schema.fields :+ - StructField(map(outputCol), outputDataType, !outputDataType.isPrimitive) + StructField(map(outputCol), outputDataType, nullable = false) StructType(outputFields) } From d20686066e978dd12e618e3978f109f05bc412fe Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 22 Apr 2015 21:35:42 -0700 Subject: [PATCH 127/144] [SPARK-7066][MLlib] VectorAssembler should use NumericType not NativeType. Author: Reynold Xin Closes #5642 from rxin/mllib-native-type and squashes the following commits: e23af5b [Reynold Xin] Remove StringType 7cbb205 [Reynold Xin] [SPARK-7066][MLlib] VectorAssembler should use NumericType and StringType, not NativeType. --- .../scala/org/apache/spark/ml/feature/VectorAssembler.scala | 5 +++-- .../main/scala/org/apache/spark/sql/types/dataTypes.scala | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index e567e069e7c0b..fd16d3d6c268b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -55,7 +55,8 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { schema(c).dataType match { case DoubleType => UnresolvedAttribute(c) case t if t.isInstanceOf[VectorUDT] => UnresolvedAttribute(c) - case _: NativeType => Alias(Cast(UnresolvedAttribute(c), DoubleType), s"${c}_double_$uid")() + case _: NumericType => + Alias(Cast(UnresolvedAttribute(c), DoubleType), s"${c}_double_$uid")() } } dataset.select(col("*"), assembleFunc(new Column(CreateStruct(args))).as(map(outputCol))) @@ -67,7 +68,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { val outputColName = map(outputCol) val inputDataTypes = inputColNames.map(name => schema(name).dataType) inputDataTypes.foreach { - case _: NativeType => + case _: NumericType => case t if t.isInstanceOf[VectorUDT] => case other => throw new IllegalArgumentException(s"Data type $other is not supported.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 7cd7bd1914c95..ddf9d664c6826 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -299,7 +299,7 @@ class NullType private() extends DataType { case object NullType extends NullType -protected[spark] object NativeType { +protected[sql] object NativeType { val all = Seq( IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) @@ -327,7 +327,7 @@ protected[sql] object PrimitiveType { } } -protected[spark] abstract class NativeType extends DataType { +protected[sql] abstract class NativeType extends DataType { private[sql] type JvmType @transient private[sql] val tag: TypeTag[JvmType] private[sql] val ordering: Ordering[JvmType] From 03e85b4a11899f37424cd6e1f8d71f1d704c90bb Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 22 Apr 2015 21:42:09 -0700 Subject: [PATCH 128/144] [SPARK-7046] Remove InputMetrics from BlockResult This is a code cleanup. The BlockResult class originally contained an InputMetrics object so that InputMetrics could directly be used as the InputMetrics for the whole task. Now we copy the fields out of here, and the presence of this object is confusing because it's only a partial input metrics (it doesn't include the records read). Because this object is no longer useful (and is confusing), it should be removed. Author: Kay Ousterhout Closes #5627 from kayousterhout/SPARK-7046 and squashes the following commits: bf64bbe [Kay Ousterhout] Import fix a08ca19 [Kay Ousterhout] [SPARK-7046] Remove InputMetrics from BlockResult --- .../main/scala/org/apache/spark/CacheManager.scala | 5 ++--- .../org/apache/spark/storage/BlockManager.scala | 9 +++------ .../org/apache/spark/storage/BlockManagerSuite.scala | 12 ++++++------ 3 files changed, 11 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index a96d754744a05..4d20c7369376e 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -44,10 +44,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { blockManager.get(key) match { case Some(blockResult) => // Partition is already materialized, so just return its values - val inputMetrics = blockResult.inputMetrics val existingMetrics = context.taskMetrics - .getInputMetricsForReadMethod(inputMetrics.readMethod) - existingMetrics.incBytesRead(inputMetrics.bytesRead) + .getInputMetricsForReadMethod(blockResult.readMethod) + existingMetrics.incBytesRead(blockResult.bytes) val iter = blockResult.data.asInstanceOf[Iterator[T]] new InterruptibleIterator[T](context, iter) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 145a9c1ae3391..55718e584c195 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,7 +29,7 @@ import scala.util.Random import sun.nio.ch.DirectBuffer import org.apache.spark._ -import org.apache.spark.executor._ +import org.apache.spark.executor.{DataReadMethod, ShuffleWriteMetrics} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} @@ -50,11 +50,8 @@ private[spark] case class ArrayValues(buffer: Array[Any]) extends BlockValues /* Class for returning a fetched block and associated metrics. */ private[spark] class BlockResult( val data: Iterator[Any], - readMethod: DataReadMethod.Value, - bytes: Long) { - val inputMetrics = new InputMetrics(readMethod) - inputMetrics.incBytesRead(bytes) -} + val readMethod: DataReadMethod.Value, + val bytes: Long) /** * Manager running on every node (driver and executors) which provides interfaces for putting and 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 545722b050ee8..7d82a7c66ad1a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -428,19 +428,19 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) - assert(list1Get.get.inputMetrics.bytesRead === list1SizeEstimate) - assert(list1Get.get.inputMetrics.readMethod === DataReadMethod.Memory) + assert(list1Get.get.bytes === list1SizeEstimate) + assert(list1Get.get.readMethod === DataReadMethod.Memory) val list2MemoryGet = store.get("list2memory") assert(list2MemoryGet.isDefined, "list2memory expected to be in store") assert(list2MemoryGet.get.data.size === 3) - assert(list2MemoryGet.get.inputMetrics.bytesRead === list2SizeEstimate) - assert(list2MemoryGet.get.inputMetrics.readMethod === DataReadMethod.Memory) + assert(list2MemoryGet.get.bytes === list2SizeEstimate) + assert(list2MemoryGet.get.readMethod === DataReadMethod.Memory) val list2DiskGet = store.get("list2disk") assert(list2DiskGet.isDefined, "list2memory expected to be in store") assert(list2DiskGet.get.data.size === 3) // We don't know the exact size of the data on disk, but it should certainly be > 0. - assert(list2DiskGet.get.inputMetrics.bytesRead > 0) - assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod.Disk) + assert(list2DiskGet.get.bytes > 0) + assert(list2DiskGet.get.readMethod === DataReadMethod.Disk) } test("in-memory LRU storage") { From d9e70f331fc3999d615ede49fc69a993dc65f272 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 22 Apr 2015 22:18:56 -0700 Subject: [PATCH 129/144] [HOTFIX][SQL] Fix broken cached test Added in #5475. Pointed as broken in #5639. /cc marmbrus Author: Liang-Chi Hsieh Closes #5640 from viirya/fix_cached_test and squashes the following commits: c0cf69a [Liang-Chi Hsieh] Fix broken cached test. --- .../apache/spark/sql/CachedTableSuite.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) 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 01e3b8671071e..0772e5e187425 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 @@ -300,19 +300,26 @@ class CachedTableSuite extends QueryTest { } test("Clear accumulators when uncacheTable to prevent memory leaking") { - val accsSize = Accumulators.originals.size - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") - cacheTable("t1") - cacheTable("t2") + + Accumulators.synchronized { + val accsSize = Accumulators.originals.size + cacheTable("t1") + cacheTable("t2") + assert((accsSize + 2) == Accumulators.originals.size) + } + sql("SELECT * FROM t1").count() sql("SELECT * FROM t2").count() sql("SELECT * FROM t1").count() sql("SELECT * FROM t2").count() - uncacheTable("t1") - uncacheTable("t2") - assert(accsSize >= Accumulators.originals.size) + Accumulators.synchronized { + val accsSize = Accumulators.originals.size + uncacheTable("t1") + uncacheTable("t2") + assert((accsSize - 2) == Accumulators.originals.size) + } } } From 2d33323cadbf58dd1d05ffff998d18cad6a896cd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 22 Apr 2015 23:54:48 -0700 Subject: [PATCH 130/144] [MLlib] Add support for BooleanType to VectorAssembler. Author: Reynold Xin Closes #5648 from rxin/vectorAssembler-boolean and squashes the following commits: 1bf3d40 [Reynold Xin] [MLlib] Add support for BooleanType to VectorAssembler. --- .../scala/org/apache/spark/ml/feature/VectorAssembler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index fd16d3d6c268b..7b2a451ca5ee5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -55,7 +55,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { schema(c).dataType match { case DoubleType => UnresolvedAttribute(c) case t if t.isInstanceOf[VectorUDT] => UnresolvedAttribute(c) - case _: NumericType => + case _: NumericType | BooleanType => Alias(Cast(UnresolvedAttribute(c), DoubleType), s"${c}_double_$uid")() } } @@ -68,7 +68,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { val outputColName = map(outputCol) val inputDataTypes = inputColNames.map(name => schema(name).dataType) inputDataTypes.foreach { - case _: NumericType => + case _: NumericType | BooleanType => case t if t.isInstanceOf[VectorUDT] => case other => throw new IllegalArgumentException(s"Data type $other is not supported.") From 29163c520087e89ca322521db1dd8656d86a6f0e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 22 Apr 2015 23:55:20 -0700 Subject: [PATCH 131/144] [SPARK-7068][SQL] Remove PrimitiveType Author: Reynold Xin Closes #5646 from rxin/remove-primitive-type and squashes the following commits: 01b673d [Reynold Xin] [SPARK-7068][SQL] Remove PrimitiveType --- .../apache/spark/sql/types/dataTypes.scala | 70 ++++++++----------- .../spark/sql/parquet/ParquetConverter.scala | 11 +-- .../sql/parquet/ParquetTableOperations.scala | 2 +- .../spark/sql/parquet/ParquetTypes.scala | 6 +- .../apache/spark/sql/parquet/newParquet.scala | 13 ++-- 5 files changed, 48 insertions(+), 54 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index ddf9d664c6826..42e26e05996dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -41,6 +41,21 @@ import org.apache.spark.util.Utils object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) + private val nonDecimalNameToType = { + (Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all) + .map(t => t.typeName -> t).toMap + } + + /** Given the string representation of a type, return its DataType */ + private def nameToType(name: String): DataType = { + val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r + name match { + case "decimal" => DecimalType.Unlimited + case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) + case other => nonDecimalNameToType(other) + } + } + private object JSortedObject { def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { case JObject(seq) => Some(seq.toList.sortBy(_._1)) @@ -51,7 +66,7 @@ object DataType { // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side. private def parseDataType(json: JValue): DataType = json match { case JString(name) => - PrimitiveType.nameToType(name) + nameToType(name) case JSortedObject( ("containsNull", JBool(n)), @@ -190,13 +205,11 @@ object DataType { equalsIgnoreNullability(leftKeyType, rightKeyType) && equalsIgnoreNullability(leftValueType, rightValueType) case (StructType(leftFields), StructType(rightFields)) => - leftFields.size == rightFields.size && - leftFields.zip(rightFields) - .forall{ - case (left, right) => - left.name == right.name && equalsIgnoreNullability(left.dataType, right.dataType) - } - case (left, right) => left == right + leftFields.length == rightFields.length && + leftFields.zip(rightFields).forall { case (l, r) => + l.name == r.name && equalsIgnoreNullability(l.dataType, r.dataType) + } + case (l, r) => l == r } } @@ -225,12 +238,11 @@ object DataType { equalsIgnoreCompatibleNullability(fromValue, toValue) case (StructType(fromFields), StructType(toFields)) => - fromFields.size == toFields.size && - fromFields.zip(toFields).forall { - case (fromField, toField) => - fromField.name == toField.name && - (toField.nullable || !fromField.nullable) && - equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType) + fromFields.length == toFields.length && + fromFields.zip(toFields).forall { case (fromField, toField) => + fromField.name == toField.name && + (toField.nullable || !fromField.nullable) && + equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType) } case (fromDataType, toDataType) => fromDataType == toDataType @@ -256,8 +268,6 @@ abstract class DataType { /** The default size of a value of this data type. */ def defaultSize: Int - def isPrimitive: Boolean = false - def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase private[sql] def jsonValue: JValue = typeName @@ -307,26 +317,6 @@ protected[sql] object NativeType { } -protected[sql] trait PrimitiveType extends DataType { - override def isPrimitive: Boolean = true -} - - -protected[sql] object PrimitiveType { - private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all - private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap - - /** Given the string representation of a type, return its DataType */ - private[sql] def nameToType(name: String): DataType = { - val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r - name match { - case "decimal" => DecimalType.Unlimited - case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) - case other => nonDecimalNameToType(other) - } - } -} - protected[sql] abstract class NativeType extends DataType { private[sql] type JvmType @transient private[sql] val tag: TypeTag[JvmType] @@ -346,7 +336,7 @@ protected[sql] abstract class NativeType extends DataType { * @group dataType */ @DeveloperApi -class StringType private() extends NativeType with PrimitiveType { +class StringType private() extends NativeType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. @@ -373,7 +363,7 @@ case object StringType extends StringType * @group dataType */ @DeveloperApi -class BinaryType private() extends NativeType with PrimitiveType { +class BinaryType private() extends NativeType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. @@ -407,7 +397,7 @@ case object BinaryType extends BinaryType *@group dataType */ @DeveloperApi -class BooleanType private() extends NativeType with PrimitiveType { +class BooleanType private() extends NativeType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. @@ -492,7 +482,7 @@ case object DateType extends DateType * * @group dataType */ -abstract class NumericType extends NativeType with PrimitiveType { +abstract class NumericType extends NativeType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index bc108e37dfb0f..116424539da11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -146,7 +146,8 @@ private[sql] object CatalystConverter { } } // All other primitive types use the default converter - case ctype: PrimitiveType => { // note: need the type tag here! + case ctype: DataType if ParquetTypesConverter.isPrimitiveType(ctype) => { + // note: need the type tag here! new CatalystPrimitiveConverter(parent, fieldIndex) } case _ => throw new RuntimeException( @@ -324,9 +325,9 @@ private[parquet] class CatalystGroupConverter( override def start(): Unit = { current = ArrayBuffer.fill(size)(null) - converters.foreach { - converter => if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer + converters.foreach { converter => + if (!converter.isPrimitive) { + converter.asInstanceOf[CatalystConverter].clearBuffer() } } } @@ -612,7 +613,7 @@ private[parquet] class CatalystArrayConverter( override def start(): Unit = { if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer + converter.asInstanceOf[CatalystConverter].clearBuffer() } } 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 1c868da23e060..a938b77578686 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 @@ -268,7 +268,7 @@ private[sql] case class InsertIntoParquetTable( val job = new Job(sqlContext.sparkContext.hadoopConfiguration) val writeSupport = - if (child.output.map(_.dataType).forall(_.isPrimitive)) { + if (child.output.map(_.dataType).forall(ParquetTypesConverter.isPrimitiveType)) { log.debug("Initializing MutableRowWriteSupport") classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport] } else { 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 60e1bec4db8e5..1dc819b5d7b9b 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 @@ -48,8 +48,10 @@ private[parquet] case class ParquetTypeInfo( length: Option[Int] = None) private[parquet] object ParquetTypesConverter extends Logging { - def isPrimitiveType(ctype: DataType): Boolean = - classOf[PrimitiveType] isAssignableFrom ctype.getClass + def isPrimitiveType(ctype: DataType): Boolean = ctype match { + case _: NumericType | BooleanType | StringType | BinaryType => true + case _: DataType => false + } def toPrimitiveDataType( parquetType: ParquetPrimitiveType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 88466f52bd4e9..85e60733bc57a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -634,12 +634,13 @@ private[sql] case class ParquetRelation2( // before calling execute(). val job = new Job(sqlContext.sparkContext.hadoopConfiguration) - val writeSupport = if (parquetSchema.map(_.dataType).forall(_.isPrimitive)) { - log.debug("Initializing MutableRowWriteSupport") - classOf[MutableRowWriteSupport] - } else { - classOf[RowWriteSupport] - } + val writeSupport = + if (parquetSchema.map(_.dataType).forall(ParquetTypesConverter.isPrimitiveType)) { + log.debug("Initializing MutableRowWriteSupport") + classOf[MutableRowWriteSupport] + } else { + classOf[RowWriteSupport] + } ParquetOutputFormat.setWriteSupportClass(job, writeSupport) From f60bece14f98450b4a71b00d7b58525f06e1f9ed Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 23 Apr 2015 01:43:40 -0700 Subject: [PATCH 132/144] [SPARK-7069][SQL] Rename NativeType -> AtomicType. Also renamed JvmType to InternalType. Author: Reynold Xin Closes #5651 from rxin/native-to-atomic-type and squashes the following commits: cbd4028 [Reynold Xin] [SPARK-7069][SQL] Rename NativeType -> AtomicType. --- .../spark/sql/catalyst/ScalaReflection.scala | 24 ++-- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../expressions/codegen/CodeGenerator.scala | 18 ++- .../codegen/GenerateProjection.scala | 4 +- .../sql/catalyst/expressions/predicates.scala | 10 +- .../spark/sql/catalyst/expressions/rows.scala | 6 +- .../apache/spark/sql/types/dataTypes.scala | 114 +++++++++--------- .../spark/sql/columnar/ColumnAccessor.scala | 2 +- .../spark/sql/columnar/ColumnBuilder.scala | 4 +- .../spark/sql/columnar/ColumnType.scala | 6 +- .../CompressibleColumnAccessor.scala | 4 +- .../CompressibleColumnBuilder.scala | 4 +- .../compression/CompressionScheme.scala | 10 +- .../compression/compressionSchemes.scala | 42 +++---- .../org/apache/spark/sql/json/JsonRDD.scala | 6 +- .../spark/sql/parquet/ParquetConverter.scala | 12 +- .../sql/parquet/ParquetTableSupport.scala | 2 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 6 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 8 +- .../sql/columnar/ColumnarTestUtils.scala | 6 +- .../compression/DictionaryEncodingSuite.scala | 4 +- .../compression/IntegralDeltaSuite.scala | 6 +- .../compression/RunLengthEncodingSuite.scala | 4 +- .../TestCompressibleColumnBuilder.scala | 6 +- 24 files changed, 159 insertions(+), 153 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index d9521953cad73..c52965507c715 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp - import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation @@ -110,7 +108,7 @@ trait ScalaReflection { StructField(p.name.toString, dataType, nullable) }), nullable = true) case t if t <:< typeOf[String] => Schema(StringType, nullable = true) - case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< typeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) @@ -136,20 +134,20 @@ trait ScalaReflection { def typeOfObject: PartialFunction[Any, DataType] = { // The data type can be determined without ambiguity. - case obj: BooleanType.JvmType => BooleanType - case obj: BinaryType.JvmType => BinaryType + case obj: Boolean => BooleanType + case obj: Array[Byte] => BinaryType case obj: String => StringType - case obj: StringType.JvmType => StringType - case obj: ByteType.JvmType => ByteType - case obj: ShortType.JvmType => ShortType - case obj: IntegerType.JvmType => IntegerType - case obj: LongType.JvmType => LongType - case obj: FloatType.JvmType => FloatType - case obj: DoubleType.JvmType => DoubleType + case obj: UTF8String => StringType + case obj: Byte => ByteType + case obj: Short => ShortType + case obj: Int => IntegerType + case obj: Long => LongType + case obj: Float => FloatType + case obj: Double => DoubleType case obj: java.sql.Date => DateType case obj: java.math.BigDecimal => DecimalType.Unlimited case obj: Decimal => DecimalType.Unlimited - case obj: TimestampType.JvmType => TimestampType + case obj: java.sql.Timestamp => TimestampType case null => NullType // For other cases, there is no obvious mapping from the type of the given object to a // Catalyst data type. A user should provide his/her specific rules 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 566b34f7c3a6a..140ccd8d3796f 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 @@ -346,7 +346,7 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { } lazy val ordering = left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } @@ -391,7 +391,7 @@ case class MinOf(left: Expression, right: Expression) extends Expression { } lazy val ordering = left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } 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 eeffedb558c1b..cbe520347385d 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 @@ -623,7 +623,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { dataType match { case StringType => q"$inputRow($ordinal).asInstanceOf[org.apache.spark.sql.types.UTF8String]" - case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)" + case dt: DataType if isNativeType(dt) => q"$inputRow.${accessorForType(dt)}($ordinal)" case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]" } } @@ -635,7 +635,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin value: TermName) = { dataType match { case StringType => q"$destinationRow.update($ordinal, $value)" - case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" + case dt: DataType if isNativeType(dt) => + q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" case _ => q"$destinationRow.update($ordinal, $value)" } } @@ -675,7 +676,18 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } protected def termForType(dt: DataType) = dt match { - case n: NativeType => n.tag + case n: AtomicType => n.tag case _ => typeTag[Any] } + + /** + * List of data types that have special accessors and setters in [[Row]]. + */ + protected val nativeTypes = + Seq(IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) + + /** + * Returns true if the data type has a special accessor and setter in [[Row]]. + */ + protected def isNativeType(dt: DataType) = nativeTypes.contains(dt) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index d491babc2bff0..584f938445c8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -109,7 +109,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { q"override def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }" } - val specificAccessorFunctions = NativeType.all.map { dataType => + val specificAccessorFunctions = nativeTypes.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { // getString() is not used by expressions case (e, i) if e.dataType == dataType && dataType != StringType => @@ -135,7 +135,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { } } - val specificMutatorFunctions = NativeType.all.map { dataType => + val specificMutatorFunctions = nativeTypes.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { // setString() is not used by expressions case (e, i) if e.dataType == dataType && dataType != StringType => 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 46522eb9c1264..9cb00cb2732ff 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, NativeType} +import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, AtomicType} object InterpretedPredicate { def create(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = @@ -211,7 +211,7 @@ case class LessThan(left: Expression, right: Expression) extends BinaryCompariso s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } @@ -240,7 +240,7 @@ case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryCo s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } @@ -269,7 +269,7 @@ case class GreaterThan(left: Expression, right: Expression) extends BinaryCompar s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } @@ -298,7 +298,7 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 981373477a4bc..5fd892c42e69c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{UTF8String, DataType, StructType, NativeType} +import org.apache.spark.sql.types.{UTF8String, DataType, StructType, AtomicType} /** * An extended interface to [[Row]] that allows the values for each column to be updated. Setting @@ -227,9 +227,9 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { return if (order.direction == Ascending) 1 else -1 } else { val comparison = order.dataType match { - case n: NativeType if order.direction == Ascending => + case n: AtomicType if order.direction == Ascending => n.ordering.asInstanceOf[Ordering[Any]].compare(left, right) - case n: NativeType if order.direction == Descending => + case n: AtomicType if order.direction == Descending => n.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right) case other => sys.error(s"Type $other does not support ordered operations") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 42e26e05996dd..87c7b7599366a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -42,7 +42,8 @@ object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) private val nonDecimalNameToType = { - (Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all) + Seq(NullType, DateType, TimestampType, BinaryType, + IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) .map(t => t.typeName -> t).toMap } @@ -309,22 +310,17 @@ class NullType private() extends DataType { case object NullType extends NullType -protected[sql] object NativeType { - val all = Seq( - IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) - - def unapply(dt: DataType): Boolean = all.contains(dt) -} - - -protected[sql] abstract class NativeType extends DataType { - private[sql] type JvmType - @transient private[sql] val tag: TypeTag[JvmType] - private[sql] val ordering: Ordering[JvmType] +/** + * An internal type used to represent everything that is not null, UDTs, arrays, structs, and maps. + */ +protected[sql] abstract class AtomicType extends DataType { + private[sql] type InternalType + @transient private[sql] val tag: TypeTag[InternalType] + private[sql] val ordering: Ordering[InternalType] @transient private[sql] val classTag = ScalaReflectionLock.synchronized { val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[JvmType](mirror.runtimeClass(tag.tpe)) + ClassTag[InternalType](mirror.runtimeClass(tag.tpe)) } } @@ -336,13 +332,13 @@ protected[sql] abstract class NativeType extends DataType { * @group dataType */ @DeveloperApi -class StringType private() extends NativeType { +class StringType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = UTF8String - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] type InternalType = UTF8String + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val ordering = implicitly[Ordering[InternalType]] /** * The default size of a value of the StringType is 4096 bytes. @@ -363,13 +359,13 @@ case object StringType extends StringType * @group dataType */ @DeveloperApi -class BinaryType private() extends NativeType { +class BinaryType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Array[Byte] - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = new Ordering[JvmType] { + private[sql] type InternalType = Array[Byte] + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val ordering = new Ordering[InternalType] { def compare(x: Array[Byte], y: Array[Byte]): Int = { for (i <- 0 until x.length; if i < y.length) { val res = x(i).compareTo(y(i)) @@ -397,13 +393,13 @@ case object BinaryType extends BinaryType *@group dataType */ @DeveloperApi -class BooleanType private() extends NativeType { +class BooleanType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Boolean - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] type InternalType = Boolean + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val ordering = implicitly[Ordering[InternalType]] /** * The default size of a value of the BooleanType is 1 byte. @@ -424,15 +420,15 @@ case object BooleanType extends BooleanType * @group dataType */ @DeveloperApi -class TimestampType private() extends NativeType { +class TimestampType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Timestamp + private[sql] type InternalType = Timestamp - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val ordering = new Ordering[JvmType] { + private[sql] val ordering = new Ordering[InternalType] { def compare(x: Timestamp, y: Timestamp): Int = x.compareTo(y) } @@ -455,15 +451,15 @@ case object TimestampType extends TimestampType * @group dataType */ @DeveloperApi -class DateType private() extends NativeType { +class DateType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "DateType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Int + private[sql] type InternalType = Int - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = implicitly[Ordering[InternalType]] /** * The default size of a value of the DateType is 4 bytes. @@ -482,13 +478,13 @@ case object DateType extends DateType * * @group dataType */ -abstract class NumericType extends NativeType { +abstract class NumericType extends AtomicType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets // desugared by the compiler into an argument to the objects constructor. This means there is no // longer an no argument constructor and thus the JVM cannot serialize the object anymore. - private[sql] val numeric: Numeric[JvmType] + private[sql] val numeric: Numeric[InternalType] } @@ -507,7 +503,7 @@ protected[sql] object IntegralType { protected[sql] sealed abstract class IntegralType extends NumericType { - private[sql] val integral: Integral[JvmType] + private[sql] val integral: Integral[InternalType] } @@ -522,11 +518,11 @@ class LongType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "LongType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Long - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] type InternalType = Long + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } private[sql] val numeric = implicitly[Numeric[Long]] private[sql] val integral = implicitly[Integral[Long]] - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = implicitly[Ordering[InternalType]] /** * The default size of a value of the LongType is 8 bytes. @@ -552,11 +548,11 @@ class IntegerType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Int - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] type InternalType = Int + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } private[sql] val numeric = implicitly[Numeric[Int]] private[sql] val integral = implicitly[Integral[Int]] - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = implicitly[Ordering[InternalType]] /** * The default size of a value of the IntegerType is 4 bytes. @@ -582,11 +578,11 @@ class ShortType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Short - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] type InternalType = Short + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } private[sql] val numeric = implicitly[Numeric[Short]] private[sql] val integral = implicitly[Integral[Short]] - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = implicitly[Ordering[InternalType]] /** * The default size of a value of the ShortType is 2 bytes. @@ -612,11 +608,11 @@ class ByteType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Byte - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] type InternalType = Byte + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } private[sql] val numeric = implicitly[Numeric[Byte]] private[sql] val integral = implicitly[Integral[Byte]] - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = implicitly[Ordering[InternalType]] /** * The default size of a value of the ByteType is 1 byte. @@ -641,8 +637,8 @@ protected[sql] object FractionalType { protected[sql] sealed abstract class FractionalType extends NumericType { - private[sql] val fractional: Fractional[JvmType] - private[sql] val asIntegral: Integral[JvmType] + private[sql] val fractional: Fractional[InternalType] + private[sql] val asIntegral: Integral[InternalType] } @@ -665,8 +661,8 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT /** No-arg constructor for kryo. */ protected def this() = this(null) - private[sql] type JvmType = Decimal - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] type InternalType = Decimal + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } private[sql] val numeric = Decimal.DecimalIsFractional private[sql] val fractional = Decimal.DecimalIsFractional private[sql] val ordering = Decimal.DecimalIsFractional @@ -743,11 +739,11 @@ class DoubleType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Double - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] type InternalType = Double + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } private[sql] val numeric = implicitly[Numeric[Double]] private[sql] val fractional = implicitly[Fractional[Double]] - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = implicitly[Ordering[InternalType]] private[sql] val asIntegral = DoubleAsIfIntegral /** @@ -772,11 +768,11 @@ class FloatType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Float - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] type InternalType = Float + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } private[sql] val numeric = implicitly[Numeric[Float]] private[sql] val fractional = implicitly[Fractional[Float]] - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = implicitly[Ordering[InternalType]] private[sql] val asIntegral = FloatAsIfIntegral /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index f615fb33a7c35..64449b2659b4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -61,7 +61,7 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( protected def underlyingBuffer = buffer } -private[sql] abstract class NativeColumnAccessor[T <: NativeType]( +private[sql] abstract class NativeColumnAccessor[T <: AtomicType]( override protected val buffer: ByteBuffer, override protected val columnType: NativeColumnType[T]) extends BasicColumnAccessor(buffer, columnType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 00ed70430b84d..aa10af400c815 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -84,10 +84,10 @@ private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType]( extends BasicColumnBuilder[T, JvmType](columnStats, columnType) with NullableColumnBuilder -private[sql] abstract class NativeColumnBuilder[T <: NativeType]( +private[sql] abstract class NativeColumnBuilder[T <: AtomicType]( override val columnStats: ColumnStats, override val columnType: NativeColumnType[T]) - extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType) + extends BasicColumnBuilder[T, T#InternalType](columnStats, columnType) with NullableColumnBuilder with AllCompressionSchemes with CompressibleColumnBuilder[T] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 1b9e0df2dcb5e..20be5ca9d0046 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -101,16 +101,16 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( override def toString: String = getClass.getSimpleName.stripSuffix("$") } -private[sql] abstract class NativeColumnType[T <: NativeType]( +private[sql] abstract class NativeColumnType[T <: AtomicType]( val dataType: T, typeId: Int, defaultSize: Int) - extends ColumnType[T, T#JvmType](typeId, defaultSize) { + extends ColumnType[T, T#InternalType](typeId, defaultSize) { /** * Scala TypeTag. Can be used to create primitive arrays and hash tables. */ - def scalaTag: TypeTag[dataType.JvmType] = dataType.tag + def scalaTag: TypeTag[dataType.InternalType] = dataType.tag } private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index d0b602a834dfe..cb205defbb1ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.columnar.compression import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType -private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor { +private[sql] trait CompressibleColumnAccessor[T <: AtomicType] extends ColumnAccessor { this: NativeColumnAccessor[T] => private var decoder: Decoder[T] = _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index b9cfc5df550d1..8e2a1af6dae78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -22,7 +22,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType /** * A stackable trait that builds optionally compressed byte buffer for a column. Memory layout of @@ -41,7 +41,7 @@ import org.apache.spark.sql.types.NativeType * header body * }}} */ -private[sql] trait CompressibleColumnBuilder[T <: NativeType] +private[sql] trait CompressibleColumnBuilder[T <: AtomicType] extends ColumnBuilder with Logging { this: NativeColumnBuilder[T] with WithCompressionSchemes => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index 879d29bcfa6f6..17c2d9b111188 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -22,9 +22,9 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType -private[sql] trait Encoder[T <: NativeType] { +private[sql] trait Encoder[T <: AtomicType] { def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {} def compressedSize: Int @@ -38,7 +38,7 @@ private[sql] trait Encoder[T <: NativeType] { def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer } -private[sql] trait Decoder[T <: NativeType] { +private[sql] trait Decoder[T <: AtomicType] { def next(row: MutableRow, ordinal: Int): Unit def hasNext: Boolean @@ -49,9 +49,9 @@ private[sql] trait CompressionScheme { def supports(columnType: ColumnType[_, _]): Boolean - def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] + def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] - def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] + def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] } private[sql] trait WithCompressionSchemes { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 8727d71c48bb7..534ae90ddbc8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -35,16 +35,16 @@ private[sql] case object PassThrough extends CompressionScheme { override def supports(columnType: ColumnType[_, _]): Boolean = true - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } - override def decoder[T <: NativeType]( + override def decoder[T <: AtomicType]( buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] = { new this.Decoder(buffer, columnType) } - class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { + class Encoder[T <: AtomicType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { override def uncompressedSize: Int = 0 override def compressedSize: Int = 0 @@ -56,7 +56,7 @@ private[sql] case object PassThrough extends CompressionScheme { } } - class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { override def next(row: MutableRow, ordinal: Int): Unit = { @@ -70,11 +70,11 @@ private[sql] case object PassThrough extends CompressionScheme { private[sql] case object RunLengthEncoding extends CompressionScheme { override val typeId = 1 - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } - override def decoder[T <: NativeType]( + override def decoder[T <: AtomicType]( buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] = { new this.Decoder(buffer, columnType) } @@ -84,7 +84,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { case _ => false } - class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { + class Encoder[T <: AtomicType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { private var _uncompressedSize = 0 private var _compressedSize = 0 @@ -152,12 +152,12 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { } } - class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { private var run = 0 private var valueCount = 0 - private var currentValue: T#JvmType = _ + private var currentValue: T#InternalType = _ override def next(row: MutableRow, ordinal: Int): Unit = { if (valueCount == run) { @@ -181,12 +181,12 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { // 32K unique values allowed val MAX_DICT_SIZE = Short.MaxValue - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : Decoder[T] = { new this.Decoder(buffer, columnType) } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } @@ -195,7 +195,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { case _ => false } - class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { + class Encoder[T <: AtomicType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary // overflows. private var _uncompressedSize = 0 @@ -208,7 +208,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { private var count = 0 // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself. - private var values = new mutable.ArrayBuffer[T#JvmType](1024) + private var values = new mutable.ArrayBuffer[T#InternalType](1024) // The dictionary that maps a value to the encoded short integer. private val dictionary = mutable.HashMap.empty[Any, Short] @@ -268,14 +268,14 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { override def compressedSize: Int = if (overflow) Int.MaxValue else dictionarySize + count * 2 } - class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { private val dictionary = { // TODO Can we clean up this mess? Maybe move this to `DataType`? implicit val classTag = { val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) + ClassTag[T#InternalType](mirror.runtimeClass(columnType.scalaTag.tpe)) } Array.fill(buffer.getInt()) { @@ -296,12 +296,12 @@ private[sql] case object BooleanBitSet extends CompressionScheme { val BITS_PER_LONG = 64 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : compression.Decoder[T] = { new this.Decoder(buffer).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new this.Encoder).asInstanceOf[compression.Encoder[T]] } @@ -384,12 +384,12 @@ private[sql] case object BooleanBitSet extends CompressionScheme { private[sql] case object IntDelta extends CompressionScheme { override def typeId: Int = 4 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : compression.Decoder[T] = { new Decoder(buffer, INT).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new Encoder).asInstanceOf[compression.Encoder[T]] } @@ -464,12 +464,12 @@ private[sql] case object IntDelta extends CompressionScheme { private[sql] case object LongDelta extends CompressionScheme { override def typeId: Int = 5 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : compression.Decoder[T] = { new Decoder(buffer, LONG).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new Encoder).asInstanceOf[compression.Encoder[T]] } 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 29de7401dda71..6e94e7056eb0b 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 @@ -183,7 +183,7 @@ private[sql] object JsonRDD extends Logging { private def typeOfPrimitiveValue: PartialFunction[Any, DataType] = { // For Integer values, use LongType by default. val useLongType: PartialFunction[Any, DataType] = { - case value: IntegerType.JvmType => LongType + case value: IntegerType.InternalType => LongType } useLongType orElse ScalaReflection.typeOfObject orElse { @@ -411,11 +411,11 @@ private[sql] object JsonRDD extends Logging { desiredType match { case StringType => UTF8String(toString(value)) case _ if value == null || value == "" => null // guard the non string type - case IntegerType => value.asInstanceOf[IntegerType.JvmType] + case IntegerType => value.asInstanceOf[IntegerType.InternalType] case LongType => toLong(value) case DoubleType => toDouble(value) case DecimalType() => toDecimal(value) - case BooleanType => value.asInstanceOf[BooleanType.JvmType] + case BooleanType => value.asInstanceOf[BooleanType.InternalType] case NullType => null case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 116424539da11..36cb5e03bbca7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -90,7 +90,7 @@ private[sql] object CatalystConverter { createConverter(field.copy(dataType = udt.sqlType), fieldIndex, parent) } // For native JVM types we use a converter with native arrays - case ArrayType(elementType: NativeType, false) => { + case ArrayType(elementType: AtomicType, false) => { new CatalystNativeArrayConverter(elementType, fieldIndex, parent) } // This is for other types of arrays, including those with nested fields @@ -118,19 +118,19 @@ private[sql] object CatalystConverter { case ShortType => { new CatalystPrimitiveConverter(parent, fieldIndex) { override def addInt(value: Int): Unit = - parent.updateShort(fieldIndex, value.asInstanceOf[ShortType.JvmType]) + parent.updateShort(fieldIndex, value.asInstanceOf[ShortType.InternalType]) } } case ByteType => { new CatalystPrimitiveConverter(parent, fieldIndex) { override def addInt(value: Int): Unit = - parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.JvmType]) + parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.InternalType]) } } case DateType => { new CatalystPrimitiveConverter(parent, fieldIndex) { override def addInt(value: Int): Unit = - parent.updateDate(fieldIndex, value.asInstanceOf[DateType.JvmType]) + parent.updateDate(fieldIndex, value.asInstanceOf[DateType.InternalType]) } } case d: DecimalType => { @@ -637,13 +637,13 @@ private[parquet] class CatalystArrayConverter( * @param capacity The (initial) capacity of the buffer */ private[parquet] class CatalystNativeArrayConverter( - val elementType: NativeType, + val elementType: AtomicType, val index: Int, protected[parquet] val parent: CatalystConverter, protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) extends CatalystConverter { - type NativeType = elementType.JvmType + type NativeType = elementType.InternalType private var buffer: Array[NativeType] = elementType.classTag.newArray(capacity) 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 e05a4c20b0d41..c45c431438efc 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 @@ -189,7 +189,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { case t @ StructType(_) => writeStruct( t, value.asInstanceOf[CatalystConverter.StructScalaType[_]]) - case _ => writePrimitive(schema.asInstanceOf[NativeType], value) + case _ => writePrimitive(schema.asInstanceOf[AtomicType], value) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index fec487f1d2c82..7cefcf44061ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -34,7 +34,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) - def testColumnStats[T <: NativeType, U <: ColumnStats]( + def testColumnStats[T <: AtomicType, U <: ColumnStats]( columnStatsClass: Class[U], columnType: NativeColumnType[T], initialStatistics: Row): Unit = { @@ -55,8 +55,8 @@ class ColumnStatsSuite extends FunSuite { val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) rows.foreach(columnStats.gatherStats(_, 0)) - val values = rows.take(10).map(_(0).asInstanceOf[T#JvmType]) - val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] + val values = rows.take(10).map(_(0).asInstanceOf[T#InternalType]) + val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#InternalType]] val stats = columnStats.collectedStatistics assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index b48bed1871c50..1e105e259dce7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -196,12 +196,12 @@ class ColumnTypeSuite extends FunSuite with Logging { } } - def testNativeColumnType[T <: NativeType]( + def testNativeColumnType[T <: AtomicType]( columnType: NativeColumnType[T], - putter: (ByteBuffer, T#JvmType) => Unit, - getter: (ByteBuffer) => T#JvmType): Unit = { + putter: (ByteBuffer, T#InternalType) => Unit, + getter: (ByteBuffer) => T#InternalType): Unit = { - testColumnType[T, T#JvmType](columnType, putter, getter) + testColumnType[T, T#InternalType](columnType, putter, getter) } def testColumnType[T <: DataType, JvmType]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index f76314b9dab5e..75d993e563e06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.types.{UTF8String, DataType, Decimal, NativeType} +import org.apache.spark.sql.types.{UTF8String, DataType, Decimal, AtomicType} object ColumnarTestUtils { def makeNullRow(length: Int): GenericMutableRow = { @@ -91,9 +91,9 @@ object ColumnarTestUtils { row } - def makeUniqueValuesAndSingleValueRows[T <: NativeType]( + def makeUniqueValuesAndSingleValueRows[T <: AtomicType]( columnType: NativeColumnType[T], - count: Int): (Seq[T#JvmType], Seq[GenericMutableRow]) = { + count: Int): (Seq[T#InternalType], Seq[GenericMutableRow]) = { val values = makeUniqueRandomValues(columnType, count) val rows = values.map { value => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index c82d9799359c7..64b70552eb047 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -24,14 +24,14 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new IntColumnStats, INT) testDictionaryEncoding(new LongColumnStats, LONG) testDictionaryEncoding(new StringColumnStats, STRING) - def testDictionaryEncoding[T <: NativeType]( + def testDictionaryEncoding[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T]) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 88011631ee4e3..bfd99f143bedc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -33,7 +33,7 @@ class IntegralDeltaSuite extends FunSuite { columnType: NativeColumnType[I], scheme: CompressionScheme) { - def skeleton(input: Seq[I#JvmType]) { + def skeleton(input: Seq[I#InternalType]) { // ------------- // Tests encoder // ------------- @@ -120,13 +120,13 @@ class IntegralDeltaSuite extends FunSuite { case LONG => Seq(2: Long, 1: Long, 2: Long, 130: Long) } - skeleton(input.map(_.asInstanceOf[I#JvmType])) + skeleton(input.map(_.asInstanceOf[I#InternalType])) } test(s"$scheme: long random series") { // Have to workaround with `Any` since no `ClassTag[I#JvmType]` available here. val input = Array.fill[Any](10000)(makeRandomValue(columnType)) - skeleton(input.map(_.asInstanceOf[I#JvmType])) + skeleton(input.map(_.asInstanceOf[I#InternalType])) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 08df1db375097..fde7a4595be0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new NoopColumnStats, BOOLEAN) @@ -32,7 +32,7 @@ class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new LongColumnStats, LONG) testRunLengthEncoding(new StringColumnStats, STRING) - def testRunLengthEncoding[T <: NativeType]( + def testRunLengthEncoding[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T]) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index fc8ff3b41d0e6..5268dfe0aa03e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.columnar.compression import org.apache.spark.sql.columnar._ -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType -class TestCompressibleColumnBuilder[T <: NativeType]( +class TestCompressibleColumnBuilder[T <: AtomicType]( override val columnStats: ColumnStats, override val columnType: NativeColumnType[T], override val schemes: Seq[CompressionScheme]) @@ -32,7 +32,7 @@ class TestCompressibleColumnBuilder[T <: NativeType]( } object TestCompressibleColumnBuilder { - def apply[T <: NativeType]( + def apply[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T], scheme: CompressionScheme): TestCompressibleColumnBuilder[T] = { From a7d65d38f934c5c751ba32aa7ab648c6d16044ab Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 23 Apr 2015 16:45:26 +0530 Subject: [PATCH 133/144] [HOTFIX] [SQL] Fix compilation for scala 2.11. Author: Prashant Sharma Closes #5652 from ScrapCodes/hf/compilation-fix-scala-2.11 and squashes the following commits: 819ff06 [Prashant Sharma] [HOTFIX] Fix compilation for scala 2.11. --- .../test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index fc3ed4a708d46..e02c84872c628 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -162,7 +162,7 @@ public void testCreateDataFrameFromJavaBeans() { Buffer outputBuffer = (Buffer) first.getJavaMap(2).get("hello"); Assert.assertArrayEquals( bean.getC().get("hello"), - Ints.toArray(JavaConversions.asJavaList(outputBuffer))); + Ints.toArray(JavaConversions.bufferAsJavaList(outputBuffer))); Seq d = first.getAs(3); Assert.assertEquals(bean.getD().size(), d.length()); for (int i = 0; i < d.length(); i++) { From 975f53e4f978759db7639cd08498ad8cd0ae2a56 Mon Sep 17 00:00:00 2001 From: Prabeesh K Date: Thu, 23 Apr 2015 10:33:13 -0700 Subject: [PATCH 134/144] [minor][streaming]fixed scala string interpolation error Author: Prabeesh K Closes #5653 from prabeesh/fix and squashes the following commits: 9d7a9f5 [Prabeesh K] fixed scala string interpolation error --- .../org/apache/spark/examples/streaming/MQTTWordCount.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f40caad322f59..85b9a54b40baf 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 @@ -56,7 +56,7 @@ object MQTTPublisher { while (true) { try { msgtopic.publish(message) - println(s"Published data. topic: {msgtopic.getName()}; Message: {message}") + println(s"Published data. topic: ${msgtopic.getName()}; Message: $message") } catch { case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => Thread.sleep(10) From cc48e6387abdd909921cb58e0588cdf226556bcd Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 23 Apr 2015 10:35:22 -0700 Subject: [PATCH 135/144] [SPARK-7044] [SQL] Fix the deadlock in script transformation Author: Cheng Hao Closes #5625 from chenghao-intel/transform and squashes the following commits: 5ec1dd2 [Cheng Hao] fix the deadlock issue in ScriptTransform --- .../hive/execution/ScriptTransformation.scala | 33 ++++++++++++------- .../sql/hive/execution/SQLQuerySuite.scala | 8 +++++ 2 files changed, 29 insertions(+), 12 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index cab0fdd35723a..3eddda3b28c66 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -145,20 +145,29 @@ case class ScriptTransformation( val dataOutputStream = new DataOutputStream(outputStream) val outputProjection = new InterpretedProjection(input, child.output) - iter - .map(outputProjection) - .foreach { row => - if (inputSerde == null) { - val data = row.mkString("", ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), - ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") - - outputStream.write(data) - } else { - val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) - prepareWritable(writable).write(dataOutputStream) + // Put the write(output to the pipeline) into a single thread + // and keep the collector as remain in the main thread. + // otherwise it will causes deadlock if the data size greater than + // the pipeline / buffer capacity. + new Thread(new Runnable() { + override def run(): Unit = { + iter + .map(outputProjection) + .foreach { row => + if (inputSerde == null) { + val data = row.mkString("", ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), + ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") + + outputStream.write(data) + } else { + val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + prepareWritable(writable).write(dataOutputStream) + } } + outputStream.close() } - outputStream.close() + }).start() + iterator } } 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 47b4cb9ca61ff..4f8d0ac0e7656 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 @@ -561,4 +561,12 @@ class SQLQuerySuite extends QueryTest { sql("select d from dn union all select d * 2 from dn") .queryExecution.analyzed } + + test("test script transform") { + val data = (1 to 100000).map { i => (i, i, i) } + data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + assert(100000 === + sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans") + .queryExecution.toRdd.count()) + } } From 534f2a43625fbf1a3a65d09550a19875cd1dce43 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 23 Apr 2015 11:29:34 -0700 Subject: [PATCH 136/144] [SPARK-6752][Streaming] Allow StreamingContext to be recreated from checkpoint and existing SparkContext Currently if you want to create a StreamingContext from checkpoint information, the system will create a new SparkContext. This prevent StreamingContext to be recreated from checkpoints in managed environments where SparkContext is precreated. The solution in this PR: Introduce the following methods on StreamingContext 1. `new StreamingContext(checkpointDirectory, sparkContext)` Recreate StreamingContext from checkpoint using the provided SparkContext 2. `StreamingContext.getOrCreate(checkpointDirectory, sparkContext, createFunction: SparkContext => StreamingContext)` If checkpoint file exists, then recreate StreamingContext using the provided SparkContext (that is, call 1.), else create StreamingContext using the provided createFunction TODO: the corresponding Java and Python API has to be added as well. Author: Tathagata Das Closes #5428 from tdas/SPARK-6752 and squashes the following commits: 94db63c [Tathagata Das] Fix long line. 524f519 [Tathagata Das] Many changes based on PR comments. eabd092 [Tathagata Das] Added Function0, Java API and unit tests for StreamingContext.getOrCreate 36a7823 [Tathagata Das] Minor changes. 204814e [Tathagata Das] Added StreamingContext.getOrCreate with existing SparkContext --- .../spark/api/java/function/Function0.java | 27 +++ .../apache/spark/streaming/Checkpoint.scala | 26 ++- .../spark/streaming/StreamingContext.scala | 85 ++++++++-- .../api/java/JavaStreamingContext.scala | 119 ++++++++++++- .../apache/spark/streaming/JavaAPISuite.java | 145 ++++++++++++---- .../spark/streaming/CheckpointSuite.scala | 3 +- .../streaming/StreamingContextSuite.scala | 159 ++++++++++++++++++ 7 files changed, 503 insertions(+), 61 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/api/java/function/Function0.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function0.java b/core/src/main/java/org/apache/spark/api/java/function/Function0.java new file mode 100644 index 0000000000000..38e410c5debe6 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/Function0.java @@ -0,0 +1,27 @@ +/* + * 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.java.function; + +import java.io.Serializable; + +/** + * A zero-argument function that returns an R. + */ +public interface Function0 extends Serializable { + public R call() throws Exception; +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 0a50485118588..7bfae253c3a0c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -77,7 +77,8 @@ object Checkpoint extends Logging { } /** Get checkpoint files present in the give directory, ordered by oldest-first */ - def getCheckpointFiles(checkpointDir: String, fs: FileSystem): Seq[Path] = { + def getCheckpointFiles(checkpointDir: String, fsOption: Option[FileSystem] = None): Seq[Path] = { + def sortFunc(path1: Path, path2: Path): Boolean = { val (time1, bk1) = path1.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } val (time2, bk2) = path2.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } @@ -85,6 +86,7 @@ object Checkpoint extends Logging { } val path = new Path(checkpointDir) + val fs = fsOption.getOrElse(path.getFileSystem(new Configuration())) if (fs.exists(path)) { val statuses = fs.listStatus(path) if (statuses != null) { @@ -160,7 +162,7 @@ class CheckpointWriter( } // Delete old checkpoint files - val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs) + val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)) if (allCheckpointFiles.size > 10) { allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach(file => { logInfo("Deleting " + file) @@ -234,15 +236,24 @@ class CheckpointWriter( private[streaming] object CheckpointReader extends Logging { - def read(checkpointDir: String, conf: SparkConf, hadoopConf: Configuration): Option[Checkpoint] = - { + /** + * Read checkpoint files present in the given checkpoint directory. If there are no checkpoint + * files, then return None, else try to return the latest valid checkpoint object. If no + * checkpoint files could be read correctly, then return None (if ignoreReadError = true), + * or throw exception (if ignoreReadError = false). + */ + def read( + checkpointDir: String, + conf: SparkConf, + hadoopConf: Configuration, + ignoreReadError: Boolean = false): Option[Checkpoint] = { val checkpointPath = new Path(checkpointDir) // TODO(rxin): Why is this a def?! def fs: FileSystem = checkpointPath.getFileSystem(hadoopConf) // Try to find the checkpoint files - val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs).reverse + val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)).reverse if (checkpointFiles.isEmpty) { return None } @@ -282,7 +293,10 @@ object CheckpointReader extends Logging { }) // If none of checkpoint files could be read, then throw exception - throw new SparkException("Failed to read checkpoint from directory " + checkpointPath) + if (!ignoreReadError) { + throw new SparkException(s"Failed to read checkpoint from directory $checkpointPath") + } + None } } 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 f57f295874645..90c8b47aebce0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -107,6 +107,19 @@ class StreamingContext private[streaming] ( */ def this(path: String) = this(path, new Configuration) + /** + * Recreate a StreamingContext from a checkpoint file using an existing SparkContext. + * @param path Path to the directory that was specified as the checkpoint directory + * @param sparkContext Existing SparkContext + */ + def this(path: String, sparkContext: SparkContext) = { + this( + sparkContext, + CheckpointReader.read(path, sparkContext.conf, sparkContext.hadoopConfiguration).get, + null) + } + + if (sc_ == null && cp_ == null) { throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") @@ -115,10 +128,12 @@ class StreamingContext private[streaming] ( private[streaming] val isCheckpointPresent = (cp_ != null) private[streaming] val sc: SparkContext = { - if (isCheckpointPresent) { + if (sc_ != null) { + sc_ + } else if (isCheckpointPresent) { new SparkContext(cp_.createSparkConf()) } else { - sc_ + throw new SparkException("Cannot create StreamingContext without a SparkContext") } } @@ -129,7 +144,7 @@ class StreamingContext private[streaming] ( private[streaming] val conf = sc.conf - private[streaming] val env = SparkEnv.get + private[streaming] val env = sc.env private[streaming] val graph: DStreamGraph = { if (isCheckpointPresent) { @@ -174,7 +189,9 @@ class StreamingContext private[streaming] ( /** Register streaming source to metrics system */ private val streamingSource = new StreamingSource(this) - SparkEnv.get.metricsSystem.registerSource(streamingSource) + assert(env != null) + assert(env.metricsSystem != null) + env.metricsSystem.registerSource(streamingSource) /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { @@ -621,19 +638,59 @@ object StreamingContext extends Logging { hadoopConf: Configuration = new Configuration(), createOnError: Boolean = false ): StreamingContext = { - val checkpointOption = try { - CheckpointReader.read(checkpointPath, new SparkConf(), hadoopConf) - } catch { - case e: Exception => - if (createOnError) { - None - } else { - throw e - } - } + val checkpointOption = CheckpointReader.read( + checkpointPath, new SparkConf(), hadoopConf, createOnError) checkpointOption.map(new StreamingContext(null, _, null)).getOrElse(creatingFunc()) } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc` on the provided `sparkContext`. Note + * that the SparkConf configuration in the checkpoint data will not be restored as the + * SparkContext has already been created. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new StreamingContext using the given SparkContext + * @param sparkContext SparkContext using which the StreamingContext will be created + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: SparkContext => StreamingContext, + sparkContext: SparkContext + ): StreamingContext = { + getOrCreate(checkpointPath, creatingFunc, sparkContext, createOnError = false) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc` on the provided `sparkContext`. Note + * that the SparkConf configuration in the checkpoint data will not be restored as the + * SparkContext has already been created. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new StreamingContext using the given SparkContext + * @param sparkContext SparkContext using which the StreamingContext will be created + * @param createOnError Whether to create a new StreamingContext if there is an + * error in reading checkpoint data. By default, an exception will be + * thrown on error. + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: SparkContext => StreamingContext, + sparkContext: SparkContext, + createOnError: Boolean + ): StreamingContext = { + val checkpointOption = CheckpointReader.read( + checkpointPath, sparkContext.conf, sparkContext.hadoopConfiguration, createOnError) + checkpointOption.map(new StreamingContext(sparkContext, _, null)) + .getOrElse(creatingFunc(sparkContext)) + } + /** * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 4095a7cc84946..572d7d8e8753d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -32,13 +32,14 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.api.java.function.{Function0 => JFunction0} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.scheduler.StreamingListener -import org.apache.hadoop.conf.Configuration -import org.apache.spark.streaming.dstream.{PluggableInputDStream, ReceiverInputDStream, DStream} +import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver +import org.apache.hadoop.conf.Configuration /** * A Java-friendly version of [[org.apache.spark.streaming.StreamingContext]] which is the main @@ -655,6 +656,7 @@ object JavaStreamingContext { * @param checkpointPath Checkpoint directory used in an earlier JavaStreamingContext program * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext */ + @deprecated("use getOrCreate without JavaStreamingContextFactor", "1.4.0") def getOrCreate( checkpointPath: String, factory: JavaStreamingContextFactory @@ -676,6 +678,7 @@ object JavaStreamingContext { * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible * file system */ + @deprecated("use getOrCreate without JavaStreamingContextFactory", "1.4.0") def getOrCreate( checkpointPath: String, hadoopConf: Configuration, @@ -700,6 +703,7 @@ object JavaStreamingContext { * @param createOnError Whether to create a new JavaStreamingContext if there is an * error in reading checkpoint data. */ + @deprecated("use getOrCreate without JavaStreamingContextFactory", "1.4.0") def getOrCreate( checkpointPath: String, hadoopConf: Configuration, @@ -712,6 +716,117 @@ object JavaStreamingContext { new JavaStreamingContext(ssc) } + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier JavaStreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction0[JavaStreamingContext] + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + creatingFunc.call().ssc + }) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible + * file system + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction0[JavaStreamingContext], + hadoopConf: Configuration + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + creatingFunc.call().ssc + }, hadoopConf) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible + * file system + * @param createOnError Whether to create a new JavaStreamingContext if there is an + * error in reading checkpoint data. + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction0[JavaStreamingContext], + hadoopConf: Configuration, + createOnError: Boolean + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + creatingFunc.call().ssc + }, hadoopConf, createOnError) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param sparkContext SparkContext using which the StreamingContext will be created + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction[JavaSparkContext, JavaStreamingContext], + sparkContext: JavaSparkContext + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, (sparkContext: SparkContext) => { + creatingFunc.call(new JavaSparkContext(sparkContext)).ssc + }, sparkContext.sc) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param sparkContext SparkContext using which the StreamingContext will be created + * @param createOnError Whether to create a new JavaStreamingContext if there is an + * error in reading checkpoint data. + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction[JavaSparkContext, JavaStreamingContext], + sparkContext: JavaSparkContext, + createOnError: Boolean + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, (sparkContext: SparkContext) => { + creatingFunc.call(new JavaSparkContext(sparkContext)).ssc + }, sparkContext.sc, createOnError) + new JavaStreamingContext(ssc) + } + /** * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. 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 90340753a4eed..cb2e8380b4933 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -22,10 +22,12 @@ import java.nio.charset.Charset; import java.util.*; +import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; + import scala.Tuple2; import org.junit.Assert; @@ -45,6 +47,7 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.*; import org.apache.spark.util.Utils; +import org.apache.spark.SparkConf; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -929,7 +932,7 @@ public void testPairMap() { // Maps pair -> pair of different type public Tuple2 call(Tuple2 in) throws Exception { return in.swap(); } - }); + }); JavaTestUtils.attachTestOutputStream(reversed); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -987,12 +990,12 @@ public void testPairMap2() { // Maps pair -> single JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaDStream reversed = pairStream.map( - new Function, Integer>() { - @Override - public Integer call(Tuple2 in) throws Exception { - return in._2(); - } - }); + new Function, Integer>() { + @Override + public Integer call(Tuple2 in) throws Exception { + return in._2(); + } + }); JavaTestUtils.attachTestOutputStream(reversed); List> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1123,7 +1126,7 @@ public void testCombineByKey() { JavaPairDStream combined = pairStream.combineByKey( new Function() { - @Override + @Override public Integer call(Integer i) throws Exception { return i; } @@ -1144,14 +1147,14 @@ public void testCountByValue() { Arrays.asList("hello")); List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2("hello", 1L), - new Tuple2("world", 1L)), - Arrays.asList( - new Tuple2("hello", 1L), - new Tuple2("moon", 1L)), - Arrays.asList( - new Tuple2("hello", 1L))); + Arrays.asList( + new Tuple2("hello", 1L), + new Tuple2("world", 1L)), + Arrays.asList( + new Tuple2("hello", 1L), + new Tuple2("moon", 1L)), + Arrays.asList( + new Tuple2("hello", 1L))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream counted = stream.countByValue(); @@ -1249,17 +1252,17 @@ public void testUpdateStateByKey() { JavaPairDStream updated = pairStream.updateStateByKey( new Function2, Optional, Optional>() { - @Override - public Optional call(List values, Optional state) { - int out = 0; - if (state.isPresent()) { - out = out + state.get(); - } - for (Integer v: values) { - out = out + v; + @Override + public Optional call(List values, Optional state) { + int out = 0; + if (state.isPresent()) { + out = out + state.get(); + } + for (Integer v : values) { + out = out + v; + } + return Optional.of(out); } - return Optional.of(out); - } }); JavaTestUtils.attachTestOutputStream(updated); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1292,17 +1295,17 @@ public void testUpdateStateByKeyWithInitial() { JavaPairDStream updated = pairStream.updateStateByKey( new Function2, Optional, Optional>() { - @Override - public Optional call(List values, Optional state) { - int out = 0; - if (state.isPresent()) { - out = out + state.get(); - } - for (Integer v: values) { - out = out + v; + @Override + public Optional call(List values, Optional state) { + int out = 0; + if (state.isPresent()) { + out = out + state.get(); + } + for (Integer v : values) { + out = out + v; + } + return Optional.of(out); } - return Optional.of(out); - } }, new HashPartitioner(1), initialRDD); JavaTestUtils.attachTestOutputStream(updated); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1328,7 +1331,7 @@ public void testReduceByKeyAndWindowWithInverse() { JavaPairDStream reduceWindowed = pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), - new Duration(2000), new Duration(1000)); + new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(reduceWindowed); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1707,6 +1710,74 @@ public Integer call(String s) throws Exception { Utils.deleteRecursively(tempDir); } + @SuppressWarnings("unchecked") + @Test + public void testContextGetOrCreate() throws InterruptedException { + + final SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("newContext", "true"); + + File emptyDir = Files.createTempDir(); + emptyDir.deleteOnExit(); + StreamingContextSuite contextSuite = new StreamingContextSuite(); + String corruptedCheckpointDir = contextSuite.createCorruptedCheckpoint(); + String checkpointDir = contextSuite.createValidCheckpoint(); + + // Function to create JavaStreamingContext without any output operations + // (used to detect the new context) + final MutableBoolean newContextCreated = new MutableBoolean(false); + Function0 creatingFunc = new Function0() { + public JavaStreamingContext call() { + newContextCreated.setValue(true); + return new JavaStreamingContext(conf, Seconds.apply(1)); + } + }; + + newContextCreated.setValue(false); + ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc); + Assert.assertTrue("new context not created", newContextCreated.isTrue()); + ssc.stop(); + + newContextCreated.setValue(false); + ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc, + new org.apache.hadoop.conf.Configuration(), true); + Assert.assertTrue("new context not created", newContextCreated.isTrue()); + ssc.stop(); + + newContextCreated.setValue(false); + ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc, + new org.apache.hadoop.conf.Configuration()); + Assert.assertTrue("old context not recovered", newContextCreated.isFalse()); + ssc.stop(); + + // Function to create JavaStreamingContext using existing JavaSparkContext + // without any output operations (used to detect the new context) + Function creatingFunc2 = + new Function() { + public JavaStreamingContext call(JavaSparkContext context) { + newContextCreated.setValue(true); + return new JavaStreamingContext(context, Seconds.apply(1)); + } + }; + + JavaSparkContext sc = new JavaSparkContext(conf); + newContextCreated.setValue(false); + ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc2, sc); + Assert.assertTrue("new context not created", newContextCreated.isTrue()); + ssc.stop(false); + + newContextCreated.setValue(false); + ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc2, sc, true); + Assert.assertTrue("new context not created", newContextCreated.isTrue()); + ssc.stop(false); + + newContextCreated.setValue(false); + ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc2, sc); + Assert.assertTrue("old context not recovered", newContextCreated.isFalse()); + ssc.stop(); + } /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD @SuppressWarnings("unchecked") 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 54c30440a6e8d..6b0a3f91d4d06 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -430,9 +430,8 @@ class CheckpointSuite extends TestSuiteBase { 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) + assert(Checkpoint.getCheckpointFiles(checkpointDir).size === 6) } ssc.stop() // Check that we shut down while the third batch was being processed diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 58353a5f97c8a..4f193322ad33e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -17,8 +17,10 @@ package org.apache.spark.streaming +import java.io.File import java.util.concurrent.atomic.AtomicInteger +import org.apache.commons.io.FileUtils import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts import org.scalatest.concurrent.Eventually._ @@ -330,6 +332,139 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } + test("getOrCreate") { + val conf = new SparkConf().setMaster(master).setAppName(appName) + + // Function to create StreamingContext that has a config to identify it to be new context + var newContextCreated = false + def creatingFunction(): StreamingContext = { + newContextCreated = true + new StreamingContext(conf, batchDuration) + } + + // Call ssc.stop after a body of code + def testGetOrCreate(body: => Unit): Unit = { + newContextCreated = false + try { + body + } finally { + if (ssc != null) { + ssc.stop() + } + ssc = null + } + } + + val emptyPath = Utils.createTempDir().getAbsolutePath() + + // getOrCreate should create new context with empty path + testGetOrCreate { + ssc = StreamingContext.getOrCreate(emptyPath, creatingFunction _) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + } + + val corrutedCheckpointPath = createCorruptedCheckpoint() + + // getOrCreate should throw exception with fake checkpoint file and createOnError = false + intercept[Exception] { + ssc = StreamingContext.getOrCreate(corrutedCheckpointPath, creatingFunction _) + } + + // getOrCreate should throw exception with fake checkpoint file + intercept[Exception] { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, createOnError = false) + } + + // getOrCreate should create new context with fake checkpoint file and createOnError = true + testGetOrCreate { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, createOnError = true) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + } + + val checkpointPath = createValidCheckpoint() + + // getOrCreate should recover context with checkpoint path, and recover old configuration + testGetOrCreate { + ssc = StreamingContext.getOrCreate(checkpointPath, creatingFunction _) + assert(ssc != null, "no context created") + assert(!newContextCreated, "old context not recovered") + assert(ssc.conf.get("someKey") === "someValue") + } + } + + test("getOrCreate with existing SparkContext") { + val conf = new SparkConf().setMaster(master).setAppName(appName) + sc = new SparkContext(conf) + + // Function to create StreamingContext that has a config to identify it to be new context + var newContextCreated = false + def creatingFunction(sparkContext: SparkContext): StreamingContext = { + newContextCreated = true + new StreamingContext(sparkContext, batchDuration) + } + + // Call ssc.stop(stopSparkContext = false) after a body of cody + def testGetOrCreate(body: => Unit): Unit = { + newContextCreated = false + try { + body + } finally { + if (ssc != null) { + ssc.stop(stopSparkContext = false) + } + ssc = null + } + } + + val emptyPath = Utils.createTempDir().getAbsolutePath() + + // getOrCreate should create new context with empty path + testGetOrCreate { + ssc = StreamingContext.getOrCreate(emptyPath, creatingFunction _, sc, createOnError = true) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + assert(ssc.sparkContext === sc, "new StreamingContext does not use existing SparkContext") + } + + val corrutedCheckpointPath = createCorruptedCheckpoint() + + // getOrCreate should throw exception with fake checkpoint file and createOnError = false + intercept[Exception] { + ssc = StreamingContext.getOrCreate(corrutedCheckpointPath, creatingFunction _, sc) + } + + // getOrCreate should throw exception with fake checkpoint file + intercept[Exception] { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, sc, createOnError = false) + } + + // getOrCreate should create new context with fake checkpoint file and createOnError = true + testGetOrCreate { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, sc, createOnError = true) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + assert(ssc.sparkContext === sc, "new StreamingContext does not use existing SparkContext") + } + + val checkpointPath = createValidCheckpoint() + + // StreamingContext.getOrCreate should recover context with checkpoint path + testGetOrCreate { + ssc = StreamingContext.getOrCreate(checkpointPath, creatingFunction _, sc) + assert(ssc != null, "no context created") + assert(!newContextCreated, "old context not recovered") + assert(ssc.sparkContext === sc, "new StreamingContext does not use existing SparkContext") + assert(!ssc.conf.contains("someKey"), + "recovered StreamingContext unexpectedly has old config") + } + } + test("DStream and generated RDD creation sites") { testPackage.test() } @@ -339,6 +474,30 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val inputStream = new TestInputStream(s, input, 1) inputStream } + + def createValidCheckpoint(): String = { + val testDirectory = Utils.createTempDir().getAbsolutePath() + val checkpointDirectory = Utils.createTempDir().getAbsolutePath() + val conf = new SparkConf().setMaster(master).setAppName(appName) + conf.set("someKey", "someValue") + ssc = new StreamingContext(conf, batchDuration) + ssc.checkpoint(checkpointDirectory) + ssc.textFileStream(testDirectory).foreachRDD { rdd => rdd.count() } + ssc.start() + eventually(timeout(10000 millis)) { + assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) + } + ssc.stop() + checkpointDirectory + } + + def createCorruptedCheckpoint(): String = { + val checkpointDirectory = Utils.createTempDir().getAbsolutePath() + val fakeCheckpointFile = Checkpoint.checkpointFile(checkpointDirectory, Time(1000)) + FileUtils.write(new File(fakeCheckpointFile.toString()), "blablabla") + assert(Checkpoint.getCheckpointFiles(checkpointDirectory).nonEmpty) + checkpointDirectory + } } class TestException(msg: String) extends Exception(msg) From c1213e6a92e126ad886d9804cedaf6db3618e602 Mon Sep 17 00:00:00 2001 From: Vinod K C Date: Thu, 23 Apr 2015 12:00:23 -0700 Subject: [PATCH 137/144] [SPARK-7055][SQL]Use correct ClassLoader for JDBC Driver in JDBCRDD.getConnector Author: Vinod K C Closes #5633 from vinodkc/use_correct_classloader_driverload and squashes the following commits: 73c5380 [Vinod K C] Use correct ClassLoader for JDBC Driver --- .../src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index b975191d41963..f326510042122 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow} import org.apache.spark.sql.types._ import org.apache.spark.sql.sources._ +import org.apache.spark.util.Utils private[sql] object JDBCRDD extends Logging { /** @@ -152,7 +153,7 @@ private[sql] object JDBCRDD extends Logging { def getConnector(driver: String, url: String, properties: Properties): () => Connection = { () => { try { - if (driver != null) Class.forName(driver) + if (driver != null) Utils.getContextOrSparkClassLoader.loadClass(driver) } catch { case e: ClassNotFoundException => { logWarning(s"Couldn't find class $driver", e); From 6afde2c7810c363083d0a699b1de02b54c13e6a9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 23 Apr 2015 13:19:03 -0700 Subject: [PATCH 138/144] [SPARK-7058] Include RDD deserialization time in "task deserialization time" metric The web UI's "task deserialization time" metric is slightly misleading because it does not capture the time taken to deserialize the broadcasted RDD. Author: Josh Rosen Closes #5635 from JoshRosen/SPARK-7058 and squashes the following commits: ed90f75 [Josh Rosen] Update UI tooltip a3743b4 [Josh Rosen] Update comments. 4f52910 [Josh Rosen] Roll back whitespace change e9cf9f4 [Josh Rosen] Remove unused variable 9f32e55 [Josh Rosen] Expose executorDeserializeTime on Task instead of pushing runtime calculation into Task. 21f5b47 [Josh Rosen] Don't double-count the broadcast deserialization time in task runtime 1752f0e [Josh Rosen] [SPARK-7058] Incorporate RDD deserialization time in task deserialization time metric --- .../main/scala/org/apache/spark/executor/Executor.scala | 8 ++++++-- .../scala/org/apache/spark/scheduler/ResultTask.scala | 2 ++ .../scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 2 ++ core/src/main/scala/org/apache/spark/scheduler/Task.scala | 7 +++++++ core/src/main/scala/org/apache/spark/ui/ToolTips.scala | 4 +++- 5 files changed, 20 insertions(+), 3 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 5fc04df5d6a40..f57e215c3f2ed 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -220,8 +220,12 @@ private[spark] class Executor( val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { - m.setExecutorDeserializeTime(taskStart - deserializeStartTime) - m.setExecutorRunTime(taskFinish - taskStart) + // Deserialization happens in two parts: first, we deserialize a Task object, which + // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. + m.setExecutorDeserializeTime( + (taskStart - deserializeStartTime) + task.executorDeserializeTime) + // We need to subtract Task.run()'s deserialization time to avoid double-counting + m.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) m.setJvmGCTime(computeTotalGcTime() - startGCTime) m.setResultSerializationTime(afterSerialization - beforeSerialization) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index e074ce6ebff0b..c9a124113961f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -53,9 +53,11 @@ private[spark] class ResultTask[T, U]( override def runTask(context: TaskContext): U = { // Deserialize the RDD and the func using the broadcast variables. + val deserializeStartTime = System.currentTimeMillis() val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, func) = ser.deserialize[(RDD[T], (TaskContext, Iterator[T]) => U)]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime metrics = Some(context.taskMetrics) func(context, rdd.iterator(partition, context)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 6c7d00069acb2..bd3dd23dfe1ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -56,9 +56,11 @@ private[spark] class ShuffleMapTask( override def runTask(context: TaskContext): MapStatus = { // Deserialize the RDD using the broadcast variable. + val deserializeStartTime = System.currentTimeMillis() val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null 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 8b592867ee31d..b09b19e2ac9e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -87,11 +87,18 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex // initialized when kill() is invoked. @volatile @transient private var _killed = false + protected var _executorDeserializeTime: Long = 0 + /** * Whether the task has been killed. */ def killed: Boolean = _killed + /** + * Returns the amount of time spent deserializing the RDD and function to be run. + */ + def executorDeserializeTime: Long = _executorDeserializeTime + /** * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark * code and user code to properly handle the flag. This function should be idempotent so it can diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index cae6870c2ab20..24f3236456248 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -24,7 +24,9 @@ private[spark] object ToolTips { scheduler delay is large, consider decreasing the size of tasks or decreasing the size of task results.""" - val TASK_DESERIALIZATION_TIME = "Time spent deserializing the task closure on the executor." + val TASK_DESERIALIZATION_TIME = + """Time spent deserializing the task closure on the executor, including the time to read the + broadcasted task.""" val SHUFFLE_READ_BLOCKED_TIME = "Time that the task spent blocked waiting for shuffle data to be read from remote machines." From 3e91cc273d281053618bfa032bc610e2cf8d8e78 Mon Sep 17 00:00:00 2001 From: wizz Date: Thu, 23 Apr 2015 14:00:07 -0700 Subject: [PATCH 139/144] [SPARK-7085][MLlib] Fix miniBatchFraction parameter in train method called with 4 arguments Author: wizz Closes #5658 from kuromatsu-nobuyuki/SPARK-7085 and squashes the following commits: 6ec2d21 [wizz] Fix miniBatchFraction parameter in train method called with 4 arguments --- .../org/apache/spark/mllib/regression/RidgeRegression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8838ca8c14718..309f9af466457 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 @@ -171,7 +171,7 @@ object RidgeRegressionWithSGD { numIterations: Int, stepSize: Double, regParam: Double): RidgeRegressionModel = { - train(input, numIterations, stepSize, regParam, 0.01) + train(input, numIterations, stepSize, regParam, 1.0) } /** From baa83a9a6769c5e119438d65d7264dceb8d743d5 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 23 Apr 2015 17:20:17 -0400 Subject: [PATCH 140/144] [SPARK-6879] [HISTORYSERVER] check if app is completed before clean it up https://issues.apache.org/jira/browse/SPARK-6879 Use `applications` to replace `FileStatus`, and check if the app is completed before clean it up. If an exception was throwed, add it to `applications` to wait for the next loop. Author: WangTaoTheTonic Closes #5491 from WangTaoTheTonic/SPARK-6879 and squashes the following commits: 4a533eb [WangTaoTheTonic] treat ACE specially cb45105 [WangTaoTheTonic] rebase d4d5251 [WangTaoTheTonic] per Marcelo's comments d7455d8 [WangTaoTheTonic] slightly change when delete file b0abca5 [WangTaoTheTonic] use global var to store apps to clean 94adfe1 [WangTaoTheTonic] leave expired apps alone to be deleted 9872a9d [WangTaoTheTonic] use the right path fdef4d6 [WangTaoTheTonic] check if app is completed before clean it up --- .../deploy/history/FsHistoryProvider.scala | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) 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 9847d5944a390..a94ebf6e53750 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 @@ -35,7 +35,6 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.{Logging, SecurityManager, SparkConf} - /** * A class that provides application history from event logs stored in the file system. * This provider checks for new finished applications in the background periodically and @@ -76,6 +75,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] = new mutable.LinkedHashMap() + // List of applications to be deleted by event log cleaner. + private var appsToClean = new mutable.ListBuffer[FsApplicationHistoryInfo] + // Constants used to parse Spark 1.0.0 log directories. private[history] val LOG_PREFIX = "EVENT_LOG_" private[history] val SPARK_VERSION_PREFIX = EventLoggingListener.SPARK_VERSION_KEY + "_" @@ -266,34 +268,40 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def cleanLogs(): Unit = { try { - val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) - .getOrElse(Seq[FileStatus]()) val maxAge = conf.getTimeAsSeconds("spark.history.fs.cleaner.maxAge", "7d") * 1000 val now = System.currentTimeMillis() val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + // Scan all logs from the log directory. + // Only completed applications older than the specified max age will be deleted. applications.values.foreach { info => - if (now - info.lastUpdated <= maxAge) { + if (now - info.lastUpdated <= maxAge || !info.completed) { appsToRetain += (info.id -> info) + } else { + appsToClean += info } } applications = appsToRetain - // Scan all logs from the log directory. - // Only directories older than the specified max age will be deleted - statusList.foreach { dir => + val leftToClean = new mutable.ListBuffer[FsApplicationHistoryInfo] + appsToClean.foreach { info => try { - if (now - dir.getModificationTime() > maxAge) { - // if path is a directory and set to true, - // the directory is deleted else throws an exception - fs.delete(dir.getPath, true) + val path = new Path(logDir, info.logPath) + if (fs.exists(path)) { + fs.delete(path, true) } } catch { - case t: IOException => logError(s"IOException in cleaning logs of $dir", t) + case e: AccessControlException => + logInfo(s"No permission to delete ${info.logPath}, ignoring.") + case t: IOException => + logError(s"IOException in cleaning logs of ${info.logPath}", t) + leftToClean += info } } + + appsToClean = leftToClean } catch { case t: Exception => logError("Exception in cleaning logs", t) } From 6d0749cae301ee4bf37632d657de48e75548a523 Mon Sep 17 00:00:00 2001 From: Tijo Thomas Date: Thu, 23 Apr 2015 17:23:15 -0400 Subject: [PATCH 141/144] [SPARK-7087] [BUILD] Fix path issue change version script Author: Tijo Thomas Closes #5656 from tijoparacka/FIX_PATHISSUE_CHANGE_VERSION_SCRIPT and squashes the following commits: ab4f4b1 [Tijo Thomas] removed whitespace 24478c9 [Tijo Thomas] modified to provide the spark base dir while searching for pom and also while changing the vesrion no 7b8e10b [Tijo Thomas] Modified for providing the base directories while finding the list of pom files and also while changing the version no --- dev/change-version-to-2.10.sh | 6 +++--- dev/change-version-to-2.11.sh | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/change-version-to-2.10.sh b/dev/change-version-to-2.10.sh index 15e0c73b4295e..c4adb1f96b7d3 100755 --- a/dev/change-version-to-2.10.sh +++ b/dev/change-version-to-2.10.sh @@ -18,9 +18,9 @@ # # Note that this will not necessarily work as intended with non-GNU sed (e.g. OS X) - -find . -name 'pom.xml' | grep -v target \ +BASEDIR=$(dirname $0)/.. +find $BASEDIR -name 'pom.xml' | grep -v target \ | xargs -I {} sed -i -e 's/\(artifactId.*\)_2.11/\1_2.10/g' {} # Also update in parent POM -sed -i -e '0,/2.112.102.112.10 in parent POM -sed -i -e '0,/2.102.112.102.11 Date: Thu, 23 Apr 2015 14:46:54 -0700 Subject: [PATCH 142/144] [SPARK-7070] [MLLIB] LDA.setBeta should call setTopicConcentration. jkbradley Author: Xiangrui Meng Closes #5649 from mengxr/SPARK-7070 and squashes the following commits: c66023c [Xiangrui Meng] setBeta should call setTopicConcentration --- .../scala/org/apache/spark/mllib/clustering/LDA.scala | 2 +- .../org/apache/spark/mllib/clustering/LDASuite.scala | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 9d63a08e211bc..d006b39acb213 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -177,7 +177,7 @@ class LDA private ( def getBeta: Double = getTopicConcentration /** Alias for [[setTopicConcentration()]] */ - def setBeta(beta: Double): this.type = setBeta(beta) + def setBeta(beta: Double): this.type = setTopicConcentration(beta) /** * Maximum number of iterations for learning. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 15de10fd13a19..cc747dabb9968 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -123,6 +123,14 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { assert(termVertexIds.map(i => LDA.index2term(i.toLong)) === termIds) assert(termVertexIds.forall(i => LDA.isTermVertex((i.toLong, 0)))) } + + test("setter alias") { + val lda = new LDA().setAlpha(2.0).setBeta(3.0) + assert(lda.getAlpha === 2.0) + assert(lda.getDocConcentration === 2.0) + assert(lda.getBeta === 3.0) + assert(lda.getTopicConcentration === 3.0) + } } private[clustering] object LDASuite { From 6220d933e5ce4ba890f5d6a50a69b95d319dafb4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 23 Apr 2015 14:48:19 -0700 Subject: [PATCH 143/144] [SQL] Break dataTypes.scala into multiple files. It was over 1000 lines of code, making it harder to find all the types. Only moved code around, and didn't change any. Author: Reynold Xin Closes #5670 from rxin/break-types and squashes the following commits: 8c59023 [Reynold Xin] Check in missing files. dcd5193 [Reynold Xin] [SQL] Break dataTypes.scala into multiple files. --- .../apache/spark/sql/types/ArrayType.scala | 74 + .../apache/spark/sql/types/BinaryType.scala | 63 + .../apache/spark/sql/types/BooleanType.scala | 51 + .../org/apache/spark/sql/types/ByteType.scala | 54 + .../org/apache/spark/sql/types/DataType.scala | 353 +++++ .../org/apache/spark/sql/types/DateType.scala | 54 + .../apache/spark/sql/types/DecimalType.scala | 110 ++ .../apache/spark/sql/types/DoubleType.scala | 53 + .../apache/spark/sql/types/FloatType.scala | 53 + .../apache/spark/sql/types/IntegerType.scala | 54 + .../org/apache/spark/sql/types/LongType.scala | 54 + .../org/apache/spark/sql/types/MapType.scala | 79 ++ .../org/apache/spark/sql/types/NullType.scala | 39 + .../apache/spark/sql/types/ShortType.scala | 53 + .../apache/spark/sql/types/StringType.scala | 50 + .../apache/spark/sql/types/StructField.scala | 54 + .../apache/spark/sql/types/StructType.scala | 263 ++++ .../spark/sql/types/TimestampType.scala | 57 + .../spark/sql/types/UserDefinedType.scala | 81 ++ .../apache/spark/sql/types/dataTypes.scala | 1224 ----------------- 20 files changed, 1649 insertions(+), 1224 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala new file mode 100644 index 0000000000000..b116163faccad --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -0,0 +1,74 @@ +/* + * 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.types + +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.DeveloperApi + + +object ArrayType { + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, containsNull = true) +} + + +/** + * :: DeveloperApi :: + * The data type for collections of multiple values. + * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * + * Please use [[DataTypes.createArrayType()]] to create a specific instance. + * + * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and + * `containsNull: Boolean`. The field of `elementType` is used to specify the type of + * array elements. The field of `containsNull` is used to specify if the array has `null` values. + * + * @param elementType The data type of values. + * @param containsNull Indicates if values have `null` values + * + * @group dataType + */ +@DeveloperApi +case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { + + /** No-arg constructor for kryo. */ + protected def this() = this(null, false) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append( + s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") + DataType.buildFormattedString(elementType, s"$prefix |", builder) + } + + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("elementType" -> elementType.jsonValue) ~ + ("containsNull" -> containsNull) + + /** + * The default size of a value of the ArrayType is 100 * the default size of the element type. + * (We assume that there are 100 elements). + */ + override def defaultSize: Int = 100 * elementType.defaultSize + + override def simpleString: String = s"array<${elementType.simpleString}>" + + private[spark] override def asNullable: ArrayType = + ArrayType(elementType.asNullable, containsNull = true) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala new file mode 100644 index 0000000000000..a581a9e9468ef --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala @@ -0,0 +1,63 @@ +/* + * 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.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Array[Byte]` values. + * Please use the singleton [[DataTypes.BinaryType]]. + * + * @group dataType + */ +@DeveloperApi +class BinaryType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + + private[sql] type InternalType = Array[Byte] + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + + private[sql] val ordering = new Ordering[InternalType] { + def compare(x: Array[Byte], y: Array[Byte]): Int = { + for (i <- 0 until x.length; if i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + } + x.length - y.length + } + } + + /** + * The default size of a value of the BinaryType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + private[spark] override def asNullable: BinaryType = this +} + + +case object BinaryType extends BinaryType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala new file mode 100644 index 0000000000000..a7f228cefa57a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala @@ -0,0 +1,51 @@ +/* + * 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.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. + * + *@group dataType + */ +@DeveloperApi +class BooleanType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Boolean + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the BooleanType is 1 byte. + */ + override def defaultSize: Int = 1 + + private[spark] override def asNullable: BooleanType = this +} + + +case object BooleanType extends BooleanType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala new file mode 100644 index 0000000000000..4d8685796ec76 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala @@ -0,0 +1,54 @@ +/* + * 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.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. + * + * @group dataType + */ +@DeveloperApi +class ByteType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Byte + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Byte]] + private[sql] val integral = implicitly[Integral[Byte]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the ByteType is 1 byte. + */ + override def defaultSize: Int = 1 + + override def simpleString: String = "tinyint" + + private[spark] override def asNullable: ByteType = this +} + +case object ByteType extends ByteType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala new file mode 100644 index 0000000000000..e6bfcd9adfeb1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -0,0 +1,353 @@ +/* + * 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.types + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.{TypeTag, runtimeMirror} +import scala.util.parsing.combinator.RegexParsers + +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.util.Utils + + +/** + * :: DeveloperApi :: + * The base type of all Spark SQL data types. + * + * @group dataType + */ +@DeveloperApi +abstract class DataType { + /** Matches any expression that evaluates to this DataType */ + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType == this => true + case _ => false + } + + /** The default size of a value of this data type. */ + def defaultSize: Int + + def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase + + private[sql] def jsonValue: JValue = typeName + + def json: String = compact(render(jsonValue)) + + def prettyJson: String = pretty(render(jsonValue)) + + def simpleString: String = typeName + + /** Check if `this` and `other` are the same data type when ignoring nullability + * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + */ + private[spark] def sameType(other: DataType): Boolean = + DataType.equalsIgnoreNullability(this, other) + + /** Returns the same data type but set all nullability fields are true + * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + */ + private[spark] def asNullable: DataType +} + + +/** + * An internal type used to represent everything that is not null, UDTs, arrays, structs, and maps. + */ +protected[sql] abstract class AtomicType extends DataType { + private[sql] type InternalType + @transient private[sql] val tag: TypeTag[InternalType] + private[sql] val ordering: Ordering[InternalType] + + @transient private[sql] val classTag = ScalaReflectionLock.synchronized { + val mirror = runtimeMirror(Utils.getSparkClassLoader) + ClassTag[InternalType](mirror.runtimeClass(tag.tpe)) + } +} + + +/** + * :: DeveloperApi :: + * Numeric data types. + * + * @group dataType + */ +abstract class NumericType extends AtomicType { + // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for + // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a + // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets + // desugared by the compiler into an argument to the objects constructor. This means there is no + // longer an no argument constructor and thus the JVM cannot serialize the object anymore. + private[sql] val numeric: Numeric[InternalType] +} + + +private[sql] object NumericType { + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] +} + + +/** Matcher for any expressions that evaluate to [[IntegralType]]s */ +private[sql] object IntegralType { + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[IntegralType] => true + case _ => false + } +} + + +private[sql] abstract class IntegralType extends NumericType { + private[sql] val integral: Integral[InternalType] +} + + + +/** Matcher for any expressions that evaluate to [[FractionalType]]s */ +private[sql] object FractionalType { + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[FractionalType] => true + case _ => false + } +} + + +private[sql] abstract class FractionalType extends NumericType { + private[sql] val fractional: Fractional[InternalType] + private[sql] val asIntegral: Integral[InternalType] +} + + +object DataType { + + def fromJson(json: String): DataType = parseDataType(parse(json)) + + @deprecated("Use DataType.fromJson instead", "1.2.0") + def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) + + private val nonDecimalNameToType = { + Seq(NullType, DateType, TimestampType, BinaryType, + IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) + .map(t => t.typeName -> t).toMap + } + + /** Given the string representation of a type, return its DataType */ + private def nameToType(name: String): DataType = { + val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r + name match { + case "decimal" => DecimalType.Unlimited + case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) + case other => nonDecimalNameToType(other) + } + } + + private object JSortedObject { + def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { + case JObject(seq) => Some(seq.toList.sortBy(_._1)) + case _ => None + } + } + + // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side. + private def parseDataType(json: JValue): DataType = json match { + case JString(name) => + nameToType(name) + + case JSortedObject( + ("containsNull", JBool(n)), + ("elementType", t: JValue), + ("type", JString("array"))) => + ArrayType(parseDataType(t), n) + + case JSortedObject( + ("keyType", k: JValue), + ("type", JString("map")), + ("valueContainsNull", JBool(n)), + ("valueType", v: JValue)) => + MapType(parseDataType(k), parseDataType(v), n) + + case JSortedObject( + ("fields", JArray(fields)), + ("type", JString("struct"))) => + StructType(fields.map(parseStructField)) + + case JSortedObject( + ("class", JString(udtClass)), + ("pyClass", _), + ("sqlType", _), + ("type", JString("udt"))) => + Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] + } + + private def parseStructField(json: JValue): StructField = json match { + case JSortedObject( + ("metadata", metadata: JObject), + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata)) + // Support reading schema when 'metadata' is missing. + case JSortedObject( + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable) + } + + private object CaseClassStringParser extends RegexParsers { + protected lazy val primitiveType: Parser[DataType] = + ( "StringType" ^^^ StringType + | "FloatType" ^^^ FloatType + | "IntegerType" ^^^ IntegerType + | "ByteType" ^^^ ByteType + | "ShortType" ^^^ ShortType + | "DoubleType" ^^^ DoubleType + | "LongType" ^^^ LongType + | "BinaryType" ^^^ BinaryType + | "BooleanType" ^^^ BooleanType + | "DateType" ^^^ DateType + | "DecimalType()" ^^^ DecimalType.Unlimited + | fixedDecimalType + | "TimestampType" ^^^ TimestampType + ) + + protected lazy val fixedDecimalType: Parser[DataType] = + ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { + case precision ~ scale => DecimalType(precision.toInt, scale.toInt) + } + + protected lazy val arrayType: Parser[DataType] = + "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { + case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) + } + + protected lazy val mapType: Parser[DataType] = + "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { + case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) + } + + protected lazy val structField: Parser[StructField] = + ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { + case name ~ tpe ~ nullable => + StructField(name, tpe, nullable = nullable) + } + + protected lazy val boolVal: Parser[Boolean] = + ( "true" ^^^ true + | "false" ^^^ false + ) + + protected lazy val structType: Parser[DataType] = + "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { + case fields => StructType(fields) + } + + protected lazy val dataType: Parser[DataType] = + ( arrayType + | mapType + | structType + | primitiveType + ) + + /** + * Parses a string representation of a DataType. + * + * TODO: Generate parser as pickler... + */ + def apply(asString: String): DataType = parseAll(dataType, asString) match { + case Success(result, _) => result + case failure: NoSuccess => + throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") + } + } + + protected[types] def buildFormattedString( + dataType: DataType, + prefix: String, + builder: StringBuilder): Unit = { + dataType match { + case array: ArrayType => + array.buildFormattedString(prefix, builder) + case struct: StructType => + struct.buildFormattedString(prefix, builder) + case map: MapType => + map.buildFormattedString(prefix, builder) + case _ => + } + } + + /** + * Compares two types, ignoring nullability of ArrayType, MapType, StructType. + */ + private[types] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { + (left, right) match { + case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => + equalsIgnoreNullability(leftElementType, rightElementType) + case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) => + equalsIgnoreNullability(leftKeyType, rightKeyType) && + equalsIgnoreNullability(leftValueType, rightValueType) + case (StructType(leftFields), StructType(rightFields)) => + leftFields.length == rightFields.length && + leftFields.zip(rightFields).forall { case (l, r) => + l.name == r.name && equalsIgnoreNullability(l.dataType, r.dataType) + } + case (l, r) => l == r + } + } + + /** + * Compares two types, ignoring compatible nullability of ArrayType, MapType, StructType. + * + * Compatible nullability is defined as follows: + * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` + * if and only if `to.containsNull` is true, or both of `from.containsNull` and + * `to.containsNull` are false. + * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` + * if and only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and + * `to.valueContainsNull` are false. + * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` + * if and only if for all every pair of fields, `to.nullable` is true, or both + * of `fromField.nullable` and `toField.nullable` are false. + */ + private[sql] def equalsIgnoreCompatibleNullability(from: DataType, to: DataType): Boolean = { + (from, to) match { + case (ArrayType(fromElement, fn), ArrayType(toElement, tn)) => + (tn || !fn) && equalsIgnoreCompatibleNullability(fromElement, toElement) + + case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => + (tn || !fn) && + equalsIgnoreCompatibleNullability(fromKey, toKey) && + equalsIgnoreCompatibleNullability(fromValue, toValue) + + case (StructType(fromFields), StructType(toFields)) => + fromFields.length == toFields.length && + fromFields.zip(toFields).forall { case (fromField, toField) => + fromField.name == toField.name && + (toField.nullable || !fromField.nullable) && + equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType) + } + + case (fromDataType, toDataType) => fromDataType == toDataType + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala new file mode 100644 index 0000000000000..03f0644bc784c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala @@ -0,0 +1,54 @@ +/* + * 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.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `java.sql.Date` values. + * Please use the singleton [[DataTypes.DateType]]. + * + * @group dataType + */ +@DeveloperApi +class DateType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "DateType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Int + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the DateType is 4 bytes. + */ + override def defaultSize: Int = 4 + + private[spark] override def asNullable: DateType = this +} + + +case object DateType extends DateType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala new file mode 100644 index 0000000000000..0f8cecd28f7df --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.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.types + +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock +import org.apache.spark.sql.catalyst.expressions.Expression + + +/** Precision parameters for a Decimal */ +case class PrecisionInfo(precision: Int, scale: Int) + + +/** + * :: DeveloperApi :: + * The data type representing `java.math.BigDecimal` values. + * A Decimal that might have fixed precision and scale, or unlimited values for these. + * + * Please use [[DataTypes.createDecimalType()]] to create a specific instance. + * + * @group dataType + */ +@DeveloperApi +case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { + + /** No-arg constructor for kryo. */ + protected def this() = this(null) + + private[sql] type InternalType = Decimal + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = Decimal.DecimalIsFractional + private[sql] val fractional = Decimal.DecimalIsFractional + private[sql] val ordering = Decimal.DecimalIsFractional + private[sql] val asIntegral = Decimal.DecimalAsIfIntegral + + def precision: Int = precisionInfo.map(_.precision).getOrElse(-1) + + def scale: Int = precisionInfo.map(_.scale).getOrElse(-1) + + override def typeName: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" + case None => "decimal" + } + + override def toString: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)" + case None => "DecimalType()" + } + + /** + * The default size of a value of the DecimalType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + override def simpleString: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" + case None => "decimal(10,0)" + } + + private[spark] override def asNullable: DecimalType = this +} + + +/** Extra factory methods and pattern matchers for Decimals */ +object DecimalType { + val Unlimited: DecimalType = DecimalType(None) + + object Fixed { + def unapply(t: DecimalType): Option[(Int, Int)] = + t.precisionInfo.map(p => (p.precision, p.scale)) + } + + object Expression { + def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { + case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale)) + case _ => None + } + } + + def apply(): DecimalType = Unlimited + + def apply(precision: Int, scale: Int): DecimalType = + DecimalType(Some(PrecisionInfo(precision, scale))) + + def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType] + + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType] + + def isFixed(dataType: DataType): Boolean = dataType match { + case DecimalType.Fixed(_, _) => true + case _ => false + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala new file mode 100644 index 0000000000000..66766623213c9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -0,0 +1,53 @@ +/* + * 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.types + +import scala.math.{Ordering, Fractional, Numeric} +import scala.math.Numeric.DoubleAsIfIntegral +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. + * + * @group dataType + */ +@DeveloperApi +class DoubleType private() extends FractionalType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Double + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Double]] + private[sql] val fractional = implicitly[Fractional[Double]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + private[sql] val asIntegral = DoubleAsIfIntegral + + /** + * The default size of a value of the DoubleType is 8 bytes. + */ + override def defaultSize: Int = 8 + + private[spark] override def asNullable: DoubleType = this +} + +case object DoubleType extends DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala new file mode 100644 index 0000000000000..1d5a2f4f6f86c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -0,0 +1,53 @@ +/* + * 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.types + +import scala.math.Numeric.FloatAsIfIntegral +import scala.math.{Ordering, Fractional, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. + * + * @group dataType + */ +@DeveloperApi +class FloatType private() extends FractionalType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Float + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Float]] + private[sql] val fractional = implicitly[Fractional[Float]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + private[sql] val asIntegral = FloatAsIfIntegral + + /** + * The default size of a value of the FloatType is 4 bytes. + */ + override def defaultSize: Int = 4 + + private[spark] override def asNullable: FloatType = this +} + +case object FloatType extends FloatType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala new file mode 100644 index 0000000000000..74e464c082873 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala @@ -0,0 +1,54 @@ +/* + * 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.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. + * + * @group dataType + */ +@DeveloperApi +class IntegerType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Int + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Int]] + private[sql] val integral = implicitly[Integral[Int]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the IntegerType is 4 bytes. + */ + override def defaultSize: Int = 4 + + override def simpleString: String = "int" + + private[spark] override def asNullable: IntegerType = this +} + +case object IntegerType extends IntegerType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala new file mode 100644 index 0000000000000..390675782e5fd --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala @@ -0,0 +1,54 @@ +/* + * 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.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. + * + * @group dataType + */ +@DeveloperApi +class LongType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "LongType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Long + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Long]] + private[sql] val integral = implicitly[Integral[Long]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the LongType is 8 bytes. + */ + override def defaultSize: Int = 8 + + override def simpleString: String = "bigint" + + private[spark] override def asNullable: LongType = this +} + + +case object LongType extends LongType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala new file mode 100644 index 0000000000000..cfdf493074415 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.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.types + +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ + + +/** + * :: DeveloperApi :: + * The data type for Maps. Keys in a map are not allowed to have `null` values. + * + * Please use [[DataTypes.createMapType()]] to create a specific instance. + * + * @param keyType The data type of map keys. + * @param valueType The data type of map values. + * @param valueContainsNull Indicates if map values have `null` values. + * + * @group dataType + */ +case class MapType( + keyType: DataType, + valueType: DataType, + valueContainsNull: Boolean) extends DataType { + + /** No-arg constructor for kryo. */ + def this() = this(null, null, false) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"$prefix-- key: ${keyType.typeName}\n") + builder.append(s"$prefix-- value: ${valueType.typeName} " + + s"(valueContainsNull = $valueContainsNull)\n") + DataType.buildFormattedString(keyType, s"$prefix |", builder) + DataType.buildFormattedString(valueType, s"$prefix |", builder) + } + + override private[sql] def jsonValue: JValue = + ("type" -> typeName) ~ + ("keyType" -> keyType.jsonValue) ~ + ("valueType" -> valueType.jsonValue) ~ + ("valueContainsNull" -> valueContainsNull) + + /** + * The default size of a value of the MapType is + * 100 * (the default size of the key type + the default size of the value type). + * (We assume that there are 100 elements). + */ + override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize) + + override def simpleString: String = s"map<${keyType.simpleString},${valueType.simpleString}>" + + private[spark] override def asNullable: MapType = + MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) +} + + +object MapType { + /** + * Construct a [[MapType]] object with the given key type and value type. + * The `valueContainsNull` is true. + */ + def apply(keyType: DataType, valueType: DataType): MapType = + MapType(keyType: DataType, valueType: DataType, valueContainsNull = true) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala new file mode 100644 index 0000000000000..b64b07431fa96 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.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.sql.types + +import org.apache.spark.annotation.DeveloperApi + + +/** + * :: DeveloperApi :: + * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. + * + * @group dataType + */ +@DeveloperApi +class NullType private() extends DataType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "NullType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + override def defaultSize: Int = 1 + + private[spark] override def asNullable: NullType = this +} + +case object NullType extends NullType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala new file mode 100644 index 0000000000000..73e9ec780b0af --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala @@ -0,0 +1,53 @@ +/* + * 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.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. + * + * @group dataType + */ +@DeveloperApi +class ShortType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Short + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Short]] + private[sql] val integral = implicitly[Integral[Short]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the ShortType is 2 bytes. + */ + override def defaultSize: Int = 2 + + override def simpleString: String = "smallint" + + private[spark] override def asNullable: ShortType = this +} + +case object ShortType extends ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala new file mode 100644 index 0000000000000..134ab0af4e0de --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -0,0 +1,50 @@ +/* + * 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.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. + * + * @group dataType + */ +@DeveloperApi +class StringType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "StringType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = UTF8String + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the StringType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + private[spark] override def asNullable: StringType = this +} + +case object StringType extends StringType + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala new file mode 100644 index 0000000000000..83570a5eaee61 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -0,0 +1,54 @@ +/* + * 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.types + +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ + +/** + * A field inside a StructType. + * @param name The name of this field. + * @param dataType The data type of this field. + * @param nullable Indicates if values of this field can be `null` values. + * @param metadata The metadata of this field. The metadata should be preserved during + * transformation if the content of the column is not modified, e.g, in selection. + */ +case class StructField( + name: String, + dataType: DataType, + nullable: Boolean = true, + metadata: Metadata = Metadata.empty) { + + /** No-arg constructor for kryo. */ + protected def this() = this(null, null) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") + DataType.buildFormattedString(dataType, s"$prefix |", builder) + } + + // override the default toString to be compatible with legacy parquet files. + override def toString: String = s"StructField($name,$dataType,$nullable)" + + private[sql] def jsonValue: JValue = { + ("name" -> name) ~ + ("type" -> dataType.jsonValue) ~ + ("nullable" -> nullable) ~ + ("metadata" -> metadata.jsonValue) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala new file mode 100644 index 0000000000000..d80ffca18ec9a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -0,0 +1,263 @@ +/* + * 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.types + +import scala.collection.mutable.ArrayBuffer +import scala.math.max + +import org.json4s.JsonDSL._ + +import org.apache.spark.SparkException +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} + + +/** + * :: DeveloperApi :: + * A [[StructType]] object can be constructed by + * {{{ + * StructType(fields: Seq[StructField]) + * }}} + * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. + * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. + * If a provided name does not have a matching field, it will be ignored. For the case + * of extracting a single StructField, a `null` will be returned. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val struct = + * StructType( + * StructField("a", IntegerType, true) :: + * StructField("b", LongType, false) :: + * StructField("c", BooleanType, false) :: Nil) + * + * // Extract a single StructField. + * val singleField = struct("b") + * // singleField: StructField = StructField(b,LongType,false) + * + * // This struct does not have a field called "d". null will be returned. + * val nonExisting = struct("d") + * // nonExisting: StructField = null + * + * // Extract multiple StructFields. Field names are provided in a set. + * // A StructType object will be returned. + * val twoFields = struct(Set("b", "c")) + * // twoFields: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * + * // Any names without matching fields will be ignored. + * // For the case shown below, "d" will be ignored and + * // it is treated as struct(Set("b", "c")). + * val ignoreNonExisting = struct(Set("b", "c", "d")) + * // ignoreNonExisting: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * }}} + * + * A [[org.apache.spark.sql.Row]] object is used as a value of the StructType. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val innerStruct = + * StructType( + * StructField("f1", IntegerType, true) :: + * StructField("f2", LongType, false) :: + * StructField("f3", BooleanType, false) :: Nil) + * + * val struct = StructType( + * StructField("a", innerStruct, true) :: Nil) + * + * // Create a Row with the schema defined by struct + * val row = Row(Row(1, 2, true)) + * // row: Row = [[1,2,true]] + * }}} + * + * @group dataType + */ +@DeveloperApi +case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { + + /** No-arg constructor for kryo. */ + protected def this() = this(null) + + /** Returns all field names in an array. */ + def fieldNames: Array[String] = fields.map(_.name) + + private lazy val fieldNamesSet: Set[String] = fieldNames.toSet + private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap + private lazy val nameToIndex: Map[String, Int] = fieldNames.zipWithIndex.toMap + + /** + * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not + * have a name matching the given name, `null` will be returned. + */ + def apply(name: String): StructField = { + nameToField.getOrElse(name, + throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) + } + + /** + * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the + * original order of fields. Those names which do not have matching fields will be ignored. + */ + def apply(names: Set[String]): StructType = { + val nonExistFields = names -- fieldNamesSet + if (nonExistFields.nonEmpty) { + throw new IllegalArgumentException( + s"Field ${nonExistFields.mkString(",")} does not exist.") + } + // Preserve the original order of fields. + StructType(fields.filter(f => names.contains(f.name))) + } + + /** + * Returns index of a given field + */ + def fieldIndex(name: String): Int = { + nameToIndex.getOrElse(name, + throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) + } + + protected[sql] def toAttributes: Seq[AttributeReference] = + map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + + def treeString: String = { + val builder = new StringBuilder + builder.append("root\n") + val prefix = " |" + fields.foreach(field => field.buildFormattedString(prefix, builder)) + + builder.toString() + } + + def printTreeString(): Unit = println(treeString) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + fields.foreach(field => field.buildFormattedString(prefix, builder)) + } + + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("fields" -> map(_.jsonValue)) + + override def apply(fieldIndex: Int): StructField = fields(fieldIndex) + + override def length: Int = fields.length + + override def iterator: Iterator[StructField] = fields.iterator + + /** + * The default size of a value of the StructType is the total default sizes of all field types. + */ + override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum + + override def simpleString: String = { + val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") + s"struct<${fieldTypes.mkString(",")}>" + } + + /** + * Merges with another schema (`StructType`). For a struct field A from `this` and a struct field + * B from `that`, + * + * 1. If A and B have the same name and data type, they are merged to a field C with the same name + * and data type. C is nullable if and only if either A or B is nullable. + * 2. If A doesn't exist in `that`, it's included in the result schema. + * 3. If B doesn't exist in `this`, it's also included in the result schema. + * 4. Otherwise, `this` and `that` are considered as conflicting schemas and an exception would be + * thrown. + */ + private[sql] def merge(that: StructType): StructType = + StructType.merge(this, that).asInstanceOf[StructType] + + private[spark] override def asNullable: StructType = { + val newFields = fields.map { + case StructField(name, dataType, nullable, metadata) => + StructField(name, dataType.asNullable, nullable = true, metadata) + } + + StructType(newFields) + } +} + + +object StructType { + + def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) + + def apply(fields: java.util.List[StructField]): StructType = { + StructType(fields.toArray.asInstanceOf[Array[StructField]]) + } + + protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = + StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + + private[sql] def merge(left: DataType, right: DataType): DataType = + (left, right) match { + case (ArrayType(leftElementType, leftContainsNull), + ArrayType(rightElementType, rightContainsNull)) => + ArrayType( + merge(leftElementType, rightElementType), + leftContainsNull || rightContainsNull) + + case (MapType(leftKeyType, leftValueType, leftContainsNull), + MapType(rightKeyType, rightValueType, rightContainsNull)) => + MapType( + merge(leftKeyType, rightKeyType), + merge(leftValueType, rightValueType), + leftContainsNull || rightContainsNull) + + case (StructType(leftFields), StructType(rightFields)) => + val newFields = ArrayBuffer.empty[StructField] + + leftFields.foreach { + case leftField @ StructField(leftName, leftType, leftNullable, _) => + rightFields + .find(_.name == leftName) + .map { case rightField @ StructField(_, rightType, rightNullable, _) => + leftField.copy( + dataType = merge(leftType, rightType), + nullable = leftNullable || rightNullable) + } + .orElse(Some(leftField)) + .foreach(newFields += _) + } + + rightFields + .filterNot(f => leftFields.map(_.name).contains(f.name)) + .foreach(newFields += _) + + StructType(newFields) + + case (DecimalType.Fixed(leftPrecision, leftScale), + DecimalType.Fixed(rightPrecision, rightScale)) => + DecimalType( + max(leftScale, rightScale) + max(leftPrecision - leftScale, rightPrecision - rightScale), + max(leftScale, rightScale)) + + case (leftUdt: UserDefinedType[_], rightUdt: UserDefinedType[_]) + if leftUdt.userClass == rightUdt.userClass => leftUdt + + case (leftType, rightType) if leftType == rightType => + leftType + + case _ => + throw new SparkException(s"Failed to merge incompatible data types $left and $right") + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala new file mode 100644 index 0000000000000..aebabfc475925 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.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.sql.types + +import java.sql.Timestamp + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `java.sql.Timestamp` values. + * Please use the singleton [[DataTypes.TimestampType]]. + * + * @group dataType + */ +@DeveloperApi +class TimestampType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Timestamp + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + + private[sql] val ordering = new Ordering[InternalType] { + def compare(x: Timestamp, y: Timestamp): Int = x.compareTo(y) + } + + /** + * The default size of a value of the TimestampType is 12 bytes. + */ + override def defaultSize: Int = 12 + + private[spark] override def asNullable: TimestampType = this +} + +case object TimestampType extends TimestampType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala new file mode 100644 index 0000000000000..6b20505c6009a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala @@ -0,0 +1,81 @@ +/* + * 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.types + +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.DeveloperApi + +/** + * ::DeveloperApi:: + * The data type for User Defined Types (UDTs). + * + * This interface allows a user to make their own classes more interoperable with SparkSQL; + * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create + * a `DataFrame` which has class X in the schema. + * + * For SparkSQL to recognize UDTs, the UDT must be annotated with + * [[SQLUserDefinedType]]. + * + * The conversion via `serialize` occurs when instantiating a `DataFrame` from another RDD. + * The conversion via `deserialize` occurs when reading from a `DataFrame`. + */ +@DeveloperApi +abstract class UserDefinedType[UserType] extends DataType with Serializable { + + /** Underlying storage type for this UDT */ + def sqlType: DataType + + /** Paired Python UDT class, if exists. */ + def pyUDT: String = null + + /** + * Convert the user type to a SQL datum + * + * TODO: Can we make this take obj: UserType? The issue is in + * CatalystTypeConverters.convertToCatalyst, where we need to convert Any to UserType. + */ + def serialize(obj: Any): Any + + /** Convert a SQL datum to the user type */ + def deserialize(datum: Any): UserType + + override private[sql] def jsonValue: JValue = { + ("type" -> "udt") ~ + ("class" -> this.getClass.getName) ~ + ("pyClass" -> pyUDT) ~ + ("sqlType" -> sqlType.jsonValue) + } + + /** + * Class object for the UserType + */ + def userClass: java.lang.Class[UserType] + + /** + * The default size of a value of the UserDefinedType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + /** + * For UDT, asNullable will not change the nullability of its internal sqlType and just returns + * itself. + */ + private[spark] override def asNullable: UserDefinedType[UserType] = this +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala deleted file mode 100644 index 87c7b7599366a..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ /dev/null @@ -1,1224 +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.sql.types - -import java.sql.Timestamp - -import scala.collection.mutable.ArrayBuffer -import scala.math._ -import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} -import scala.reflect.ClassTag -import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} -import scala.util.parsing.combinator.RegexParsers - -import org.json4s._ -import org.json4s.JsonAST.JValue -import org.json4s.JsonDSL._ -import org.json4s.jackson.JsonMethods._ - -import org.apache.spark.SparkException -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.ScalaReflectionLock -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} -import org.apache.spark.util.Utils - - -object DataType { - def fromJson(json: String): DataType = parseDataType(parse(json)) - - private val nonDecimalNameToType = { - Seq(NullType, DateType, TimestampType, BinaryType, - IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) - .map(t => t.typeName -> t).toMap - } - - /** Given the string representation of a type, return its DataType */ - private def nameToType(name: String): DataType = { - val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r - name match { - case "decimal" => DecimalType.Unlimited - case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) - case other => nonDecimalNameToType(other) - } - } - - private object JSortedObject { - def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { - case JObject(seq) => Some(seq.toList.sortBy(_._1)) - case _ => None - } - } - - // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side. - private def parseDataType(json: JValue): DataType = json match { - case JString(name) => - nameToType(name) - - case JSortedObject( - ("containsNull", JBool(n)), - ("elementType", t: JValue), - ("type", JString("array"))) => - ArrayType(parseDataType(t), n) - - case JSortedObject( - ("keyType", k: JValue), - ("type", JString("map")), - ("valueContainsNull", JBool(n)), - ("valueType", v: JValue)) => - MapType(parseDataType(k), parseDataType(v), n) - - case JSortedObject( - ("fields", JArray(fields)), - ("type", JString("struct"))) => - StructType(fields.map(parseStructField)) - - case JSortedObject( - ("class", JString(udtClass)), - ("pyClass", _), - ("sqlType", _), - ("type", JString("udt"))) => - Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] - } - - private def parseStructField(json: JValue): StructField = json match { - case JSortedObject( - ("metadata", metadata: JObject), - ("name", JString(name)), - ("nullable", JBool(nullable)), - ("type", dataType: JValue)) => - StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata)) - // Support reading schema when 'metadata' is missing. - case JSortedObject( - ("name", JString(name)), - ("nullable", JBool(nullable)), - ("type", dataType: JValue)) => - StructField(name, parseDataType(dataType), nullable) - } - - @deprecated("Use DataType.fromJson instead", "1.2.0") - def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) - - private object CaseClassStringParser extends RegexParsers { - protected lazy val primitiveType: Parser[DataType] = - ( "StringType" ^^^ StringType - | "FloatType" ^^^ FloatType - | "IntegerType" ^^^ IntegerType - | "ByteType" ^^^ ByteType - | "ShortType" ^^^ ShortType - | "DoubleType" ^^^ DoubleType - | "LongType" ^^^ LongType - | "BinaryType" ^^^ BinaryType - | "BooleanType" ^^^ BooleanType - | "DateType" ^^^ DateType - | "DecimalType()" ^^^ DecimalType.Unlimited - | fixedDecimalType - | "TimestampType" ^^^ TimestampType - ) - - protected lazy val fixedDecimalType: Parser[DataType] = - ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { - case precision ~ scale => DecimalType(precision.toInt, scale.toInt) - } - - protected lazy val arrayType: Parser[DataType] = - "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { - case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) - } - - protected lazy val mapType: Parser[DataType] = - "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { - case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) - } - - protected lazy val structField: Parser[StructField] = - ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { - case name ~ tpe ~ nullable => - StructField(name, tpe, nullable = nullable) - } - - protected lazy val boolVal: Parser[Boolean] = - ( "true" ^^^ true - | "false" ^^^ false - ) - - protected lazy val structType: Parser[DataType] = - "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { - case fields => StructType(fields) - } - - protected lazy val dataType: Parser[DataType] = - ( arrayType - | mapType - | structType - | primitiveType - ) - - /** - * Parses a string representation of a DataType. - * - * TODO: Generate parser as pickler... - */ - def apply(asString: String): DataType = parseAll(dataType, asString) match { - case Success(result, _) => result - case failure: NoSuccess => - throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") - } - } - - protected[types] def buildFormattedString( - dataType: DataType, - prefix: String, - builder: StringBuilder): Unit = { - dataType match { - case array: ArrayType => - array.buildFormattedString(prefix, builder) - case struct: StructType => - struct.buildFormattedString(prefix, builder) - case map: MapType => - map.buildFormattedString(prefix, builder) - case _ => - } - } - - /** - * Compares two types, ignoring nullability of ArrayType, MapType, StructType. - */ - private[types] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { - (left, right) match { - case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => - equalsIgnoreNullability(leftElementType, rightElementType) - case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) => - equalsIgnoreNullability(leftKeyType, rightKeyType) && - equalsIgnoreNullability(leftValueType, rightValueType) - case (StructType(leftFields), StructType(rightFields)) => - leftFields.length == rightFields.length && - leftFields.zip(rightFields).forall { case (l, r) => - l.name == r.name && equalsIgnoreNullability(l.dataType, r.dataType) - } - case (l, r) => l == r - } - } - - /** - * Compares two types, ignoring compatible nullability of ArrayType, MapType, StructType. - * - * Compatible nullability is defined as follows: - * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` - * if and only if `to.containsNull` is true, or both of `from.containsNull` and - * `to.containsNull` are false. - * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` - * if and only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and - * `to.valueContainsNull` are false. - * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` - * if and only if for all every pair of fields, `to.nullable` is true, or both - * of `fromField.nullable` and `toField.nullable` are false. - */ - private[sql] def equalsIgnoreCompatibleNullability(from: DataType, to: DataType): Boolean = { - (from, to) match { - case (ArrayType(fromElement, fn), ArrayType(toElement, tn)) => - (tn || !fn) && equalsIgnoreCompatibleNullability(fromElement, toElement) - - case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => - (tn || !fn) && - equalsIgnoreCompatibleNullability(fromKey, toKey) && - equalsIgnoreCompatibleNullability(fromValue, toValue) - - case (StructType(fromFields), StructType(toFields)) => - fromFields.length == toFields.length && - fromFields.zip(toFields).forall { case (fromField, toField) => - fromField.name == toField.name && - (toField.nullable || !fromField.nullable) && - equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType) - } - - case (fromDataType, toDataType) => fromDataType == toDataType - } - } -} - - -/** - * :: DeveloperApi :: - * The base type of all Spark SQL data types. - * - * @group dataType - */ -@DeveloperApi -abstract class DataType { - /** Matches any expression that evaluates to this DataType */ - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType == this => true - case _ => false - } - - /** The default size of a value of this data type. */ - def defaultSize: Int - - def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase - - private[sql] def jsonValue: JValue = typeName - - def json: String = compact(render(jsonValue)) - - def prettyJson: String = pretty(render(jsonValue)) - - def simpleString: String = typeName - - /** Check if `this` and `other` are the same data type when ignoring nullability - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). - */ - private[spark] def sameType(other: DataType): Boolean = - DataType.equalsIgnoreNullability(this, other) - - /** Returns the same data type but set all nullability fields are true - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). - */ - private[spark] def asNullable: DataType -} - -/** - * :: DeveloperApi :: - * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. - * - * @group dataType - */ -@DeveloperApi -class NullType private() extends DataType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "NullType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - override def defaultSize: Int = 1 - - private[spark] override def asNullable: NullType = this -} - -case object NullType extends NullType - - -/** - * An internal type used to represent everything that is not null, UDTs, arrays, structs, and maps. - */ -protected[sql] abstract class AtomicType extends DataType { - private[sql] type InternalType - @transient private[sql] val tag: TypeTag[InternalType] - private[sql] val ordering: Ordering[InternalType] - - @transient private[sql] val classTag = ScalaReflectionLock.synchronized { - val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[InternalType](mirror.runtimeClass(tag.tpe)) - } -} - - -/** - * :: DeveloperApi :: - * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. - * - * @group dataType - */ -@DeveloperApi -class StringType private() extends AtomicType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "StringType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = UTF8String - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val ordering = implicitly[Ordering[InternalType]] - - /** - * The default size of a value of the StringType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - private[spark] override def asNullable: StringType = this -} - -case object StringType extends StringType - - -/** - * :: DeveloperApi :: - * The data type representing `Array[Byte]` values. - * Please use the singleton [[DataTypes.BinaryType]]. - * - * @group dataType - */ -@DeveloperApi -class BinaryType private() extends AtomicType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Array[Byte] - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val ordering = new Ordering[InternalType] { - def compare(x: Array[Byte], y: Array[Byte]): Int = { - for (i <- 0 until x.length; if i < y.length) { - val res = x(i).compareTo(y(i)) - if (res != 0) return res - } - x.length - y.length - } - } - - /** - * The default size of a value of the BinaryType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - private[spark] override def asNullable: BinaryType = this -} - -case object BinaryType extends BinaryType - - -/** - * :: DeveloperApi :: - * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. - * - *@group dataType - */ -@DeveloperApi -class BooleanType private() extends AtomicType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Boolean - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val ordering = implicitly[Ordering[InternalType]] - - /** - * The default size of a value of the BooleanType is 1 byte. - */ - override def defaultSize: Int = 1 - - private[spark] override def asNullable: BooleanType = this -} - -case object BooleanType extends BooleanType - - -/** - * :: DeveloperApi :: - * The data type representing `java.sql.Timestamp` values. - * Please use the singleton [[DataTypes.TimestampType]]. - * - * @group dataType - */ -@DeveloperApi -class TimestampType private() extends AtomicType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Timestamp - - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - - private[sql] val ordering = new Ordering[InternalType] { - def compare(x: Timestamp, y: Timestamp): Int = x.compareTo(y) - } - - /** - * The default size of a value of the TimestampType is 12 bytes. - */ - override def defaultSize: Int = 12 - - private[spark] override def asNullable: TimestampType = this -} - -case object TimestampType extends TimestampType - - -/** - * :: DeveloperApi :: - * The data type representing `java.sql.Date` values. - * Please use the singleton [[DataTypes.DateType]]. - * - * @group dataType - */ -@DeveloperApi -class DateType private() extends AtomicType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "DateType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Int - - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - - private[sql] val ordering = implicitly[Ordering[InternalType]] - - /** - * The default size of a value of the DateType is 4 bytes. - */ - override def defaultSize: Int = 4 - - private[spark] override def asNullable: DateType = this -} - -case object DateType extends DateType - - -/** - * :: DeveloperApi :: - * Numeric data types. - * - * @group dataType - */ -abstract class NumericType extends AtomicType { - // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for - // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a - // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets - // desugared by the compiler into an argument to the objects constructor. This means there is no - // longer an no argument constructor and thus the JVM cannot serialize the object anymore. - private[sql] val numeric: Numeric[InternalType] -} - - -protected[sql] object NumericType { - def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] -} - - -/** Matcher for any expressions that evaluate to [[IntegralType]]s */ -protected[sql] object IntegralType { - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType.isInstanceOf[IntegralType] => true - case _ => false - } -} - - -protected[sql] sealed abstract class IntegralType extends NumericType { - private[sql] val integral: Integral[InternalType] -} - - -/** - * :: DeveloperApi :: - * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. - * - * @group dataType - */ -@DeveloperApi -class LongType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "LongType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Long - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val numeric = implicitly[Numeric[Long]] - private[sql] val integral = implicitly[Integral[Long]] - private[sql] val ordering = implicitly[Ordering[InternalType]] - - /** - * The default size of a value of the LongType is 8 bytes. - */ - override def defaultSize: Int = 8 - - override def simpleString: String = "bigint" - - private[spark] override def asNullable: LongType = this -} - -case object LongType extends LongType - - -/** - * :: DeveloperApi :: - * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. - * - * @group dataType - */ -@DeveloperApi -class IntegerType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Int - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val numeric = implicitly[Numeric[Int]] - private[sql] val integral = implicitly[Integral[Int]] - private[sql] val ordering = implicitly[Ordering[InternalType]] - - /** - * The default size of a value of the IntegerType is 4 bytes. - */ - override def defaultSize: Int = 4 - - override def simpleString: String = "int" - - private[spark] override def asNullable: IntegerType = this -} - -case object IntegerType extends IntegerType - - -/** - * :: DeveloperApi :: - * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. - * - * @group dataType - */ -@DeveloperApi -class ShortType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Short - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val numeric = implicitly[Numeric[Short]] - private[sql] val integral = implicitly[Integral[Short]] - private[sql] val ordering = implicitly[Ordering[InternalType]] - - /** - * The default size of a value of the ShortType is 2 bytes. - */ - override def defaultSize: Int = 2 - - override def simpleString: String = "smallint" - - private[spark] override def asNullable: ShortType = this -} - -case object ShortType extends ShortType - - -/** - * :: DeveloperApi :: - * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. - * - * @group dataType - */ -@DeveloperApi -class ByteType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Byte - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val numeric = implicitly[Numeric[Byte]] - private[sql] val integral = implicitly[Integral[Byte]] - private[sql] val ordering = implicitly[Ordering[InternalType]] - - /** - * The default size of a value of the ByteType is 1 byte. - */ - override def defaultSize: Int = 1 - - override def simpleString: String = "tinyint" - - private[spark] override def asNullable: ByteType = this -} - -case object ByteType extends ByteType - - -/** Matcher for any expressions that evaluate to [[FractionalType]]s */ -protected[sql] object FractionalType { - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType.isInstanceOf[FractionalType] => true - case _ => false - } -} - - -protected[sql] sealed abstract class FractionalType extends NumericType { - private[sql] val fractional: Fractional[InternalType] - private[sql] val asIntegral: Integral[InternalType] -} - - -/** Precision parameters for a Decimal */ -case class PrecisionInfo(precision: Int, scale: Int) - - -/** - * :: DeveloperApi :: - * The data type representing `java.math.BigDecimal` values. - * A Decimal that might have fixed precision and scale, or unlimited values for these. - * - * Please use [[DataTypes.createDecimalType()]] to create a specific instance. - * - * @group dataType - */ -@DeveloperApi -case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { - - /** No-arg constructor for kryo. */ - protected def this() = this(null) - - private[sql] type InternalType = Decimal - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val numeric = Decimal.DecimalIsFractional - private[sql] val fractional = Decimal.DecimalIsFractional - private[sql] val ordering = Decimal.DecimalIsFractional - private[sql] val asIntegral = Decimal.DecimalAsIfIntegral - - def precision: Int = precisionInfo.map(_.precision).getOrElse(-1) - - def scale: Int = precisionInfo.map(_.scale).getOrElse(-1) - - override def typeName: String = precisionInfo match { - case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" - case None => "decimal" - } - - override def toString: String = precisionInfo match { - case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)" - case None => "DecimalType()" - } - - /** - * The default size of a value of the DecimalType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - override def simpleString: String = precisionInfo match { - case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" - case None => "decimal(10,0)" - } - - private[spark] override def asNullable: DecimalType = this -} - - -/** Extra factory methods and pattern matchers for Decimals */ -object DecimalType { - val Unlimited: DecimalType = DecimalType(None) - - object Fixed { - def unapply(t: DecimalType): Option[(Int, Int)] = - t.precisionInfo.map(p => (p.precision, p.scale)) - } - - object Expression { - def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { - case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale)) - case _ => None - } - } - - def apply(): DecimalType = Unlimited - - def apply(precision: Int, scale: Int): DecimalType = - DecimalType(Some(PrecisionInfo(precision, scale))) - - def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType] - - def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType] - - def isFixed(dataType: DataType): Boolean = dataType match { - case DecimalType.Fixed(_, _) => true - case _ => false - } -} - - -/** - * :: DeveloperApi :: - * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. - * - * @group dataType - */ -@DeveloperApi -class DoubleType private() extends FractionalType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Double - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val numeric = implicitly[Numeric[Double]] - private[sql] val fractional = implicitly[Fractional[Double]] - private[sql] val ordering = implicitly[Ordering[InternalType]] - private[sql] val asIntegral = DoubleAsIfIntegral - - /** - * The default size of a value of the DoubleType is 8 bytes. - */ - override def defaultSize: Int = 8 - - private[spark] override def asNullable: DoubleType = this -} - -case object DoubleType extends DoubleType - - -/** - * :: DeveloperApi :: - * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. - * - * @group dataType - */ -@DeveloperApi -class FloatType private() extends FractionalType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type InternalType = Float - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } - private[sql] val numeric = implicitly[Numeric[Float]] - private[sql] val fractional = implicitly[Fractional[Float]] - private[sql] val ordering = implicitly[Ordering[InternalType]] - private[sql] val asIntegral = FloatAsIfIntegral - - /** - * The default size of a value of the FloatType is 4 bytes. - */ - override def defaultSize: Int = 4 - - private[spark] override def asNullable: FloatType = this -} - -case object FloatType extends FloatType - - -object ArrayType { - /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ - def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) -} - - -/** - * :: DeveloperApi :: - * The data type for collections of multiple values. - * Internally these are represented as columns that contain a ``scala.collection.Seq``. - * - * Please use [[DataTypes.createArrayType()]] to create a specific instance. - * - * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and - * `containsNull: Boolean`. The field of `elementType` is used to specify the type of - * array elements. The field of `containsNull` is used to specify if the array has `null` values. - * - * @param elementType The data type of values. - * @param containsNull Indicates if values have `null` values - * - * @group dataType - */ -@DeveloperApi -case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { - - /** No-arg constructor for kryo. */ - protected def this() = this(null, false) - - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append( - s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") - DataType.buildFormattedString(elementType, s"$prefix |", builder) - } - - override private[sql] def jsonValue = - ("type" -> typeName) ~ - ("elementType" -> elementType.jsonValue) ~ - ("containsNull" -> containsNull) - - /** - * The default size of a value of the ArrayType is 100 * the default size of the element type. - * (We assume that there are 100 elements). - */ - override def defaultSize: Int = 100 * elementType.defaultSize - - override def simpleString: String = s"array<${elementType.simpleString}>" - - private[spark] override def asNullable: ArrayType = - ArrayType(elementType.asNullable, containsNull = true) -} - - -/** - * A field inside a StructType. - * @param name The name of this field. - * @param dataType The data type of this field. - * @param nullable Indicates if values of this field can be `null` values. - * @param metadata The metadata of this field. The metadata should be preserved during - * transformation if the content of the column is not modified, e.g, in selection. - */ -case class StructField( - name: String, - dataType: DataType, - nullable: Boolean = true, - metadata: Metadata = Metadata.empty) { - - /** No-arg constructor for kryo. */ - protected def this() = this(null, null) - - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") - DataType.buildFormattedString(dataType, s"$prefix |", builder) - } - - // override the default toString to be compatible with legacy parquet files. - override def toString: String = s"StructField($name,$dataType,$nullable)" - - private[sql] def jsonValue: JValue = { - ("name" -> name) ~ - ("type" -> dataType.jsonValue) ~ - ("nullable" -> nullable) ~ - ("metadata" -> metadata.jsonValue) - } -} - - -object StructType { - protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = - StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - - def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) - - def apply(fields: java.util.List[StructField]): StructType = { - StructType(fields.toArray.asInstanceOf[Array[StructField]]) - } - - private[sql] def merge(left: DataType, right: DataType): DataType = - (left, right) match { - case (ArrayType(leftElementType, leftContainsNull), - ArrayType(rightElementType, rightContainsNull)) => - ArrayType( - merge(leftElementType, rightElementType), - leftContainsNull || rightContainsNull) - - case (MapType(leftKeyType, leftValueType, leftContainsNull), - MapType(rightKeyType, rightValueType, rightContainsNull)) => - MapType( - merge(leftKeyType, rightKeyType), - merge(leftValueType, rightValueType), - leftContainsNull || rightContainsNull) - - case (StructType(leftFields), StructType(rightFields)) => - val newFields = ArrayBuffer.empty[StructField] - - leftFields.foreach { - case leftField @ StructField(leftName, leftType, leftNullable, _) => - rightFields - .find(_.name == leftName) - .map { case rightField @ StructField(_, rightType, rightNullable, _) => - leftField.copy( - dataType = merge(leftType, rightType), - nullable = leftNullable || rightNullable) - } - .orElse(Some(leftField)) - .foreach(newFields += _) - } - - rightFields - .filterNot(f => leftFields.map(_.name).contains(f.name)) - .foreach(newFields += _) - - StructType(newFields) - - case (DecimalType.Fixed(leftPrecision, leftScale), - DecimalType.Fixed(rightPrecision, rightScale)) => - DecimalType( - max(leftScale, rightScale) + max(leftPrecision - leftScale, rightPrecision - rightScale), - max(leftScale, rightScale)) - - case (leftUdt: UserDefinedType[_], rightUdt: UserDefinedType[_]) - if leftUdt.userClass == rightUdt.userClass => leftUdt - - case (leftType, rightType) if leftType == rightType => - leftType - - case _ => - throw new SparkException(s"Failed to merge incompatible data types $left and $right") - } -} - - -/** - * :: DeveloperApi :: - * A [[StructType]] object can be constructed by - * {{{ - * StructType(fields: Seq[StructField]) - * }}} - * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. - * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. - * If a provided name does not have a matching field, it will be ignored. For the case - * of extracting a single StructField, a `null` will be returned. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val struct = - * StructType( - * StructField("a", IntegerType, true) :: - * StructField("b", LongType, false) :: - * StructField("c", BooleanType, false) :: Nil) - * - * // Extract a single StructField. - * val singleField = struct("b") - * // singleField: StructField = StructField(b,LongType,false) - * - * // This struct does not have a field called "d". null will be returned. - * val nonExisting = struct("d") - * // nonExisting: StructField = null - * - * // Extract multiple StructFields. Field names are provided in a set. - * // A StructType object will be returned. - * val twoFields = struct(Set("b", "c")) - * // twoFields: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * - * // Any names without matching fields will be ignored. - * // For the case shown below, "d" will be ignored and - * // it is treated as struct(Set("b", "c")). - * val ignoreNonExisting = struct(Set("b", "c", "d")) - * // ignoreNonExisting: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * }}} - * - * A [[org.apache.spark.sql.Row]] object is used as a value of the StructType. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val innerStruct = - * StructType( - * StructField("f1", IntegerType, true) :: - * StructField("f2", LongType, false) :: - * StructField("f3", BooleanType, false) :: Nil) - * - * val struct = StructType( - * StructField("a", innerStruct, true) :: Nil) - * - * // Create a Row with the schema defined by struct - * val row = Row(Row(1, 2, true)) - * // row: Row = [[1,2,true]] - * }}} - * - * @group dataType - */ -@DeveloperApi -case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { - - /** No-arg constructor for kryo. */ - protected def this() = this(null) - - /** Returns all field names in an array. */ - def fieldNames: Array[String] = fields.map(_.name) - - private lazy val fieldNamesSet: Set[String] = fieldNames.toSet - private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap - private lazy val nameToIndex: Map[String, Int] = fieldNames.zipWithIndex.toMap - - /** - * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not - * have a name matching the given name, `null` will be returned. - */ - def apply(name: String): StructField = { - nameToField.getOrElse(name, - throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) - } - - /** - * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the - * original order of fields. Those names which do not have matching fields will be ignored. - */ - def apply(names: Set[String]): StructType = { - val nonExistFields = names -- fieldNamesSet - if (nonExistFields.nonEmpty) { - throw new IllegalArgumentException( - s"Field ${nonExistFields.mkString(",")} does not exist.") - } - // Preserve the original order of fields. - StructType(fields.filter(f => names.contains(f.name))) - } - - /** - * Returns index of a given field - */ - def fieldIndex(name: String): Int = { - nameToIndex.getOrElse(name, - throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) - } - - protected[sql] def toAttributes: Seq[AttributeReference] = - map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) - - def treeString: String = { - val builder = new StringBuilder - builder.append("root\n") - val prefix = " |" - fields.foreach(field => field.buildFormattedString(prefix, builder)) - - builder.toString() - } - - def printTreeString(): Unit = println(treeString) - - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - fields.foreach(field => field.buildFormattedString(prefix, builder)) - } - - override private[sql] def jsonValue = - ("type" -> typeName) ~ - ("fields" -> map(_.jsonValue)) - - override def apply(fieldIndex: Int): StructField = fields(fieldIndex) - - override def length: Int = fields.length - - override def iterator: Iterator[StructField] = fields.iterator - - /** - * The default size of a value of the StructType is the total default sizes of all field types. - */ - override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum - - override def simpleString: String = { - val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") - s"struct<${fieldTypes.mkString(",")}>" - } - - /** - * Merges with another schema (`StructType`). For a struct field A from `this` and a struct field - * B from `that`, - * - * 1. If A and B have the same name and data type, they are merged to a field C with the same name - * and data type. C is nullable if and only if either A or B is nullable. - * 2. If A doesn't exist in `that`, it's included in the result schema. - * 3. If B doesn't exist in `this`, it's also included in the result schema. - * 4. Otherwise, `this` and `that` are considered as conflicting schemas and an exception would be - * thrown. - */ - private[sql] def merge(that: StructType): StructType = - StructType.merge(this, that).asInstanceOf[StructType] - - private[spark] override def asNullable: StructType = { - val newFields = fields.map { - case StructField(name, dataType, nullable, metadata) => - StructField(name, dataType.asNullable, nullable = true, metadata) - } - - StructType(newFields) - } -} - - -object MapType { - /** - * Construct a [[MapType]] object with the given key type and value type. - * The `valueContainsNull` is true. - */ - def apply(keyType: DataType, valueType: DataType): MapType = - MapType(keyType: DataType, valueType: DataType, valueContainsNull = true) -} - - -/** - * :: DeveloperApi :: - * The data type for Maps. Keys in a map are not allowed to have `null` values. - * - * Please use [[DataTypes.createMapType()]] to create a specific instance. - * - * @param keyType The data type of map keys. - * @param valueType The data type of map values. - * @param valueContainsNull Indicates if map values have `null` values. - * - * @group dataType - */ -case class MapType( - keyType: DataType, - valueType: DataType, - valueContainsNull: Boolean) extends DataType { - - /** No-arg constructor for kryo. */ - def this() = this(null, null, false) - - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"$prefix-- key: ${keyType.typeName}\n") - builder.append(s"$prefix-- value: ${valueType.typeName} " + - s"(valueContainsNull = $valueContainsNull)\n") - DataType.buildFormattedString(keyType, s"$prefix |", builder) - DataType.buildFormattedString(valueType, s"$prefix |", builder) - } - - override private[sql] def jsonValue: JValue = - ("type" -> typeName) ~ - ("keyType" -> keyType.jsonValue) ~ - ("valueType" -> valueType.jsonValue) ~ - ("valueContainsNull" -> valueContainsNull) - - /** - * The default size of a value of the MapType is - * 100 * (the default size of the key type + the default size of the value type). - * (We assume that there are 100 elements). - */ - override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize) - - override def simpleString: String = s"map<${keyType.simpleString},${valueType.simpleString}>" - - private[spark] override def asNullable: MapType = - MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) -} - - -/** - * ::DeveloperApi:: - * The data type for User Defined Types (UDTs). - * - * This interface allows a user to make their own classes more interoperable with SparkSQL; - * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create - * a `DataFrame` which has class X in the schema. - * - * For SparkSQL to recognize UDTs, the UDT must be annotated with - * [[SQLUserDefinedType]]. - * - * The conversion via `serialize` occurs when instantiating a `DataFrame` from another RDD. - * The conversion via `deserialize` occurs when reading from a `DataFrame`. - */ -@DeveloperApi -abstract class UserDefinedType[UserType] extends DataType with Serializable { - - /** Underlying storage type for this UDT */ - def sqlType: DataType - - /** Paired Python UDT class, if exists. */ - def pyUDT: String = null - - /** - * Convert the user type to a SQL datum - * - * TODO: Can we make this take obj: UserType? The issue is in - * CatalystTypeConverters.convertToCatalyst, where we need to convert Any to UserType. - */ - def serialize(obj: Any): Any - - /** Convert a SQL datum to the user type */ - def deserialize(datum: Any): UserType - - override private[sql] def jsonValue: JValue = { - ("type" -> "udt") ~ - ("class" -> this.getClass.getName) ~ - ("pyClass" -> pyUDT) ~ - ("sqlType" -> sqlType.jsonValue) - } - - /** - * Class object for the UserType - */ - def userClass: java.lang.Class[UserType] - - /** - * The default size of a value of the UserDefinedType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - /** - * For UDT, asNullable will not change the nullability of its internal sqlType and just returns - * itself. - */ - private[spark] override def asNullable: UserDefinedType[UserType] = this -} From 73db132bf503341c7a5cf9409351c282a8464175 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 23 Apr 2015 16:08:14 -0700 Subject: [PATCH 144/144] [SPARK-6818] [SPARKR] Support column deletion in SparkR DataFrame API. Author: Sun Rui Closes #5655 from sun-rui/SPARK-6818 and squashes the following commits: 7c66570 [Sun Rui] [SPARK-6818][SPARKR] Support column deletion in SparkR DataFrame API. --- R/pkg/R/DataFrame.R | 8 +++++++- R/pkg/inst/tests/test_sparkSQL.R | 5 +++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 861fe1c78b0db..b59b700af5dc9 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -790,9 +790,12 @@ setMethod("$", signature(x = "DataFrame"), setMethod("$<-", signature(x = "DataFrame"), function(x, name, value) { - stopifnot(class(value) == "Column") + stopifnot(class(value) == "Column" || is.null(value)) cols <- columns(x) if (name %in% cols) { + if (is.null(value)) { + cols <- Filter(function(c) { c != name }, cols) + } cols <- lapply(cols, function(c) { if (c == name) { alias(value, name) @@ -802,6 +805,9 @@ setMethod("$<-", signature(x = "DataFrame"), }) nx <- select(x, cols) } else { + if (is.null(value)) { + return(x) + } nx <- withColumn(x, name, value) } x@sdf <- nx@sdf diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 25831ae2d9e18..af7a6c582047a 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -449,6 +449,11 @@ test_that("select operators", { df$age2 <- df$age * 2 expect_equal(columns(df), c("name", "age", "age2")) expect_equal(count(where(df, df$age2 == df$age * 2)), 2) + + df$age2 <- NULL + expect_equal(columns(df), c("name", "age")) + df$age3 <- NULL + expect_equal(columns(df), c("name", "age")) }) test_that("select with column", {