From 2d762b4f10a54b5abe2103e802c5cdcc73291daf Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 8 Nov 2019 18:26:49 +0800 Subject: [PATCH 01/36] plan exists subquery --- .../sql/catalyst/expressions/predicates.scala | 44 +++++++++++ .../apache/spark/sql/execution/subquery.scala | 77 ++++++++++++++++++- 2 files changed, 119 insertions(+), 2 deletions(-) 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 4c0998412f72..658944a2817d 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 @@ -487,6 +487,50 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } } +case class ExistsSubquery(child: Expression, + subQuery: String, + hset: Set[Any]) extends UnaryExpression with Predicate { + + require(hset != null, "hset could not be null") + + override def toString: String = s"Exists ${subQuery}" + + override def nullable: Boolean = child.nullable + + protected override def nullSafeEval(value: Any): Any = { + if (set.contains(value)) { + true + } else { + false + } + } + + @transient lazy val set: Set[Any] = child.dataType match { + case t: AtomicType if !t.isInstanceOf[BinaryType] => hset + case _: NullType => hset + case _ => + // for structs use interpreted ordering to be able to compare UnsafeRows with non-UnsafeRows + TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ (hset - null) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + genCodeWithSet(ctx, ev) + } + + private def genCodeWithSet(ctx: CodegenContext, ev: ExprCode): ExprCode = { + nullSafeCodeGen(ctx, ev, c => { + val setTerm = ctx.addReferenceObj("set", set) + s""" + |${ev.value} = $setTerm.size() > 0; + """.stripMargin + }) + } + + override def sql: String = { + s"(EXISTS (${subQuery}))" + } +} + @ExpressionDescription( usage = "expr1 _FUNC_ expr2 - Logical AND.") case class And(left: Expression, right: Expression) extends BinaryOperator with Predicate { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index adb4956c6d3c..0a993148c749 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution import scala.collection.mutable import scala.collection.mutable.ArrayBuffer - import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, CreateNamedStruct, Exists, ExistsSubquery, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, DataType, StructType} @@ -171,6 +171,66 @@ case class InSubqueryExec( } } + +case class ExistsExec(child: Expression, + subQuery: String, + plan: BaseSubqueryExec, + exprId: ExprId, + private var resultBroadcast: Broadcast[Array[Any]] = null) + extends ExecSubqueryExpression { + + @transient private var result: Array[Any] = _ + + override def dataType: DataType = BooleanType + override def children: Seq[Expression] = child :: Nil + override def nullable: Boolean = child.nullable + override def toString: String = s"EXISTS ${plan.name}" + override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = plan) + + override def semanticEquals(other: Expression): Boolean = other match { + case in: ExistsExec => child.semanticEquals(in.child) && plan.sameResult(in.plan) + case _ => false + } + + + def updateResult(): Unit = { + val rows = plan.executeCollect() + result = child.dataType match { + case _: StructType => rows.toArray + case _ => rows.map(_.get(0, child.dataType)) + } + resultBroadcast = plan.sqlContext.sparkContext.broadcast(result) + } + + def values(): Option[Array[Any]] = Option(resultBroadcast).map(_.value) + + private def prepareResult(): Unit = { + require(resultBroadcast != null, s"$this has not finished") + if (result == null) { + result = resultBroadcast.value + } + } + + override def eval(input: InternalRow): Any = { + prepareResult() + !result.isEmpty + } + + override lazy val canonicalized: ExistsExec = { + copy( + child = child.canonicalized, + subQuery = subQuery, + plan = plan.canonicalized.asInstanceOf[BaseSubqueryExec], + exprId = ExprId(0), + resultBroadcast = null) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + prepareResult() + ExistsSubquery(child, subQuery, result.toSet).doGenCode(ctx, ev) + } +} + /** * Plans subqueries that are present in the given [[SparkPlan]]. */ @@ -194,6 +254,19 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { } val executedPlan = new QueryExecution(sparkSession, query).executedPlan InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) + case expressions.Exists(sub, children, exprId) => + val expr = if (children.length == 1) { + children.head + } else { + CreateNamedStruct( + children.zipWithIndex.flatMap { case (v, index) => + Seq(Literal(s"col_$index"), v) + } + ) + } + val executedPlan = new QueryExecution(sparkSession, sub).executedPlan + ExistsExec(expr, sub.treeString, + SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) } } } From 1c577bc9fa224f7d0df0b4374b25a4eafe4edb2d Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 8 Nov 2019 18:31:24 +0800 Subject: [PATCH 02/36] Update subquery.scala --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 0a993148c749..27cbfb5665ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -171,7 +171,10 @@ case class InSubqueryExec( } } - +/** + * The physical node of exists-subquery. This is for support use exists in join's on condition, + * since some join type we can't pushdown exists condition, we plan it here + */ case class ExistsExec(child: Expression, subQuery: String, plan: BaseSubqueryExec, From 5fa971b50525523c5dd85f5af3a8b14051592816 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 8 Nov 2019 18:35:22 +0800 Subject: [PATCH 03/36] format import --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 27cbfb5665ee..ad8f4e923764 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution import scala.collection.mutable import scala.collection.mutable.ArrayBuffer + import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, CreateNamedStruct, Exists, ExistsSubquery, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, ExistsSubquery, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, DataType, StructType} From 14013492da3863b36cc7b58b324136f6d6a33e1c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 8 Nov 2019 22:29:46 +0800 Subject: [PATCH 04/36] don;t collect executed rdd --- .../sql/catalyst/expressions/predicates.scala | 21 +++++------------- .../apache/spark/sql/execution/subquery.scala | 22 +++++++------------ 2 files changed, 13 insertions(+), 30 deletions(-) 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 658944a2817d..78f88ae5e393 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 @@ -489,29 +489,18 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with case class ExistsSubquery(child: Expression, subQuery: String, - hset: Set[Any]) extends UnaryExpression with Predicate { + result: Boolean) extends UnaryExpression with Predicate { - require(hset != null, "hset could not be null") + require(result != null, "hset could not be null") override def toString: String = s"Exists ${subQuery}" override def nullable: Boolean = child.nullable protected override def nullSafeEval(value: Any): Any = { - if (set.contains(value)) { - true - } else { - false - } + true } - @transient lazy val set: Set[Any] = child.dataType match { - case t: AtomicType if !t.isInstanceOf[BinaryType] => hset - case _: NullType => hset - case _ => - // for structs use interpreted ordering to be able to compare UnsafeRows with non-UnsafeRows - TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ (hset - null) - } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { genCodeWithSet(ctx, ev) @@ -519,9 +508,9 @@ case class ExistsSubquery(child: Expression, private def genCodeWithSet(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, c => { - val setTerm = ctx.addReferenceObj("set", set) + val setTerm = ctx.addReferenceObj("result", result) s""" - |${ev.value} = $setTerm.size() > 0; + |${ev.value} = $setTerm; """.stripMargin }) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index ad8f4e923764..ea40221a2cff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -179,10 +179,10 @@ case class ExistsExec(child: Expression, subQuery: String, plan: BaseSubqueryExec, exprId: ExprId, - private var resultBroadcast: Broadcast[Array[Any]] = null) + private var resultBroadcast: Broadcast[Boolean] = null) extends ExecSubqueryExpression { - @transient private var result: Array[Any] = _ + @transient private var result: Boolean = _ override def dataType: DataType = BooleanType override def children: Seq[Expression] = child :: Nil @@ -197,26 +197,20 @@ case class ExistsExec(child: Expression, def updateResult(): Unit = { - val rows = plan.executeCollect() - result = child.dataType match { - case _: StructType => rows.toArray - case _ => rows.map(_.get(0, child.dataType)) - } - resultBroadcast = plan.sqlContext.sparkContext.broadcast(result) + result = !plan.execute().isEmpty() + resultBroadcast = plan.sqlContext.sparkContext.broadcast[Boolean](result) } - def values(): Option[Array[Any]] = Option(resultBroadcast).map(_.value) + def values(): Option[Boolean] = Option(resultBroadcast).map(_.value) private def prepareResult(): Unit = { require(resultBroadcast != null, s"$this has not finished") - if (result == null) { - result = resultBroadcast.value - } + result = resultBroadcast.value } override def eval(input: InternalRow): Any = { prepareResult() - !result.isEmpty + result } override lazy val canonicalized: ExistsExec = { @@ -230,7 +224,7 @@ case class ExistsExec(child: Expression, override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { prepareResult() - ExistsSubquery(child, subQuery, result.toSet).doGenCode(ctx, ev) + ExistsSubquery(child, subQuery, result).doGenCode(ctx, ev) } } From 7b943aa643017d6815d64e1855a3be42254e6db2 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 8 Nov 2019 22:30:55 +0800 Subject: [PATCH 05/36] format code --- .../org/apache/spark/sql/catalyst/expressions/predicates.scala | 1 - 1 file changed, 1 deletion(-) 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 78f88ae5e393..303a578c1156 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 @@ -501,7 +501,6 @@ case class ExistsSubquery(child: Expression, true } - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { genCodeWithSet(ctx, ev) } From 95e446d2cd4d776dcf99aab7cda2769f357b05a9 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 10 Nov 2019 16:26:09 +0800 Subject: [PATCH 06/36] Update predicates.scala --- .../org/apache/spark/sql/catalyst/expressions/predicates.scala | 2 -- 1 file changed, 2 deletions(-) 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 303a578c1156..808cdc7f1332 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 @@ -491,8 +491,6 @@ case class ExistsSubquery(child: Expression, subQuery: String, result: Boolean) extends UnaryExpression with Predicate { - require(result != null, "hset could not be null") - override def toString: String = s"Exists ${subQuery}" override def nullable: Boolean = child.nullable From 20cda4212e98ece033cbb7734d95e3433e04c9be Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 10 Nov 2019 17:17:43 +0800 Subject: [PATCH 07/36] Update subquery.scala --- .../sql/catalyst/optimizer/subquery.scala | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 32dbd389afd9..ef09d205a235 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -106,12 +106,20 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Filter the plan by applying left semi and left anti joins. withSubquery.foldLeft(newFilter) { - case (p, Exists(sub, conditions, _)) => - val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - buildJoin(outerPlan, sub, LeftSemi, joinCond) - case (p, Not(Exists(sub, conditions, _))) => - val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - buildJoin(outerPlan, sub, LeftAnti, joinCond) + case (p, exists @ Exists(sub, conditions, _)) => + if (SubqueryExpression.hasCorrelatedSubquery(exists)) { + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) + buildJoin(outerPlan, sub, LeftSemi, joinCond) + } else { + Filter(exists, newFilter) + } + case (p, Not(exists @ Exists(sub, conditions, _))) => + if (SubqueryExpression.hasCorrelatedSubquery(exists)) { + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) + buildJoin(outerPlan, sub, LeftAnti, joinCond) + } else { + Filter(Not(exists), newFilter) + } case (p, InSubquery(values, ListQuery(sub, conditions, _, _))) => // Deduplicate conflicting attributes if any. val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) From 8e3ce4f48f9e9230c2f107a21b27f62eb15d9911 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 11 Nov 2019 15:51:56 +0800 Subject: [PATCH 08/36] remove ExistsSubquery --- .../sql/catalyst/expressions/predicates.scala | 30 ------------------- .../apache/spark/sql/execution/subquery.scala | 28 ++++++++++++++--- 2 files changed, 24 insertions(+), 34 deletions(-) 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 808cdc7f1332..4c0998412f72 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 @@ -487,36 +487,6 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } } -case class ExistsSubquery(child: Expression, - subQuery: String, - result: Boolean) extends UnaryExpression with Predicate { - - override def toString: String = s"Exists ${subQuery}" - - override def nullable: Boolean = child.nullable - - protected override def nullSafeEval(value: Any): Any = { - true - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - genCodeWithSet(ctx, ev) - } - - private def genCodeWithSet(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, c => { - val setTerm = ctx.addReferenceObj("result", result) - s""" - |${ev.value} = $setTerm; - """.stripMargin - }) - } - - override def sql: String = { - s"(EXISTS (${subQuery}))" - } -} - @ExpressionDescription( usage = "expr1 _FUNC_ expr2 - Logical AND.") case class And(left: Expression, right: Expression) extends BinaryOperator with Predicate { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index ea40221a2cff..ada6fbd9ddaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -23,8 +23,9 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, ExistsSubquery, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral} +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, DataType, StructType} @@ -195,7 +196,6 @@ case class ExistsExec(child: Expression, case _ => false } - def updateResult(): Unit = { result = !plan.execute().isEmpty() resultBroadcast = plan.sqlContext.sparkContext.broadcast[Boolean](result) @@ -224,7 +224,27 @@ case class ExistsExec(child: Expression, override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { prepareResult() - ExistsSubquery(child, subQuery, result).doGenCode(ctx, ev) + val childGen = child.genCode(ctx) + val setTerm = ctx.addReferenceObj("result", result) + val resultCode = + s""" + |${ev.value} = $setTerm; + """.stripMargin + + if (nullable) { + val nullSafeEval = ctx.nullSafeExec(child.nullable, childGen.isNull)(resultCode) + ev.copy(code = code""" + ${childGen.code} + boolean ${ev.isNull} = ${childGen.isNull}; + ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + $nullSafeEval + """) + } else { + ev.copy(code = code""" + ${childGen.code} + ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + $resultCode""", isNull = FalseLiteral) + } } } From c29041183687d8f9ed9b89727fe66468a665ae98 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 11 Nov 2019 16:38:41 +0800 Subject: [PATCH 09/36] minimize cost --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index ada6fbd9ddaf..4b95a22e4a65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.plans.logical.{Limit, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, DataType, StructType} @@ -197,7 +198,7 @@ case class ExistsExec(child: Expression, } def updateResult(): Unit = { - result = !plan.execute().isEmpty() + result = plan.executeTake(1).length == 1 resultBroadcast = plan.sqlContext.sparkContext.broadcast[Boolean](result) } @@ -281,7 +282,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { } ) } - val executedPlan = new QueryExecution(sparkSession, sub).executedPlan + val executedPlan = new QueryExecution(sparkSession, Project(Nil, sub)).executedPlan ExistsExec(expr, sub.treeString, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) } From 866ddc71ddaec291c9df5ebdbac6bee7035f3621 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 11 Nov 2019 18:53:12 +0800 Subject: [PATCH 10/36] follow comment --- .../sql/catalyst/expressions/subquery.scala | 4 +- .../sql/catalyst/optimizer/subquery.scala | 23 ++++------ .../apache/spark/sql/execution/subquery.scala | 44 +++++-------------- 3 files changed, 20 insertions(+), 51 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 48f5136d3399..9586a28e5d8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -64,9 +64,9 @@ object SubqueryExpression { /** * Returns true when an expression contains an IN or EXISTS subquery and false otherwise. */ - def hasInOrExistsSubquery(e: Expression): Boolean = { + def hasInOrCorrelatedExistsSubquery(e: Expression): Boolean = { e.find { - case _: ListQuery | _: Exists => true + case _: ListQuery | _: Exists if e.children.nonEmpty => true case _ => false }.isDefined } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index ef09d205a235..56fb9f82c47a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -96,7 +96,8 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Filter(condition, child) => val (withSubquery, withoutSubquery) = - splitConjunctivePredicates(condition).partition(SubqueryExpression.hasInOrExistsSubquery) + splitConjunctivePredicates(condition) + .partition(SubqueryExpression.hasInOrCorrelatedExistsSubquery) // Construct the pruned filter condition. val newFilter: LogicalPlan = withoutSubquery match { @@ -106,20 +107,12 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Filter the plan by applying left semi and left anti joins. withSubquery.foldLeft(newFilter) { - case (p, exists @ Exists(sub, conditions, _)) => - if (SubqueryExpression.hasCorrelatedSubquery(exists)) { - val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - buildJoin(outerPlan, sub, LeftSemi, joinCond) - } else { - Filter(exists, newFilter) - } - case (p, Not(exists @ Exists(sub, conditions, _))) => - if (SubqueryExpression.hasCorrelatedSubquery(exists)) { - val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - buildJoin(outerPlan, sub, LeftAnti, joinCond) - } else { - Filter(Not(exists), newFilter) - } + case (p, Exists(sub, conditions, _)) => + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) + buildJoin(outerPlan, sub, LeftSemi, joinCond) + case (p, Not(Exists(sub, conditions, _))) => + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) + buildJoin(outerPlan, sub, LeftAnti, joinCond) case (p, InSubquery(values, ListQuery(sub, conditions, _, _))) => // Deduplicate conflicting attributes if any. val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 4b95a22e4a65..cb4fe9a6b489 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -177,23 +177,22 @@ case class InSubqueryExec( * The physical node of exists-subquery. This is for support use exists in join's on condition, * since some join type we can't pushdown exists condition, we plan it here */ -case class ExistsExec(child: Expression, - subQuery: String, - plan: BaseSubqueryExec, - exprId: ExprId, - private var resultBroadcast: Broadcast[Boolean] = null) +case class ExistsExec( + plan: BaseSubqueryExec, + exprId: ExprId, + private var resultBroadcast: Broadcast[Boolean] = null) extends ExecSubqueryExpression { @transient private var result: Boolean = _ override def dataType: DataType = BooleanType - override def children: Seq[Expression] = child :: Nil - override def nullable: Boolean = child.nullable + override def children: Seq[Expression] = Nil + override def nullable: Boolean = false override def toString: String = s"EXISTS ${plan.name}" override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = plan) override def semanticEquals(other: Expression): Boolean = other match { - case in: ExistsExec => child.semanticEquals(in.child) && plan.sameResult(in.plan) + case in: ExistsExec => plan.sameResult(in.plan) case _ => false } @@ -216,8 +215,6 @@ case class ExistsExec(child: Expression, override lazy val canonicalized: ExistsExec = { copy( - child = child.canonicalized, - subQuery = subQuery, plan = plan.canonicalized.asInstanceOf[BaseSubqueryExec], exprId = ExprId(0), resultBroadcast = null) @@ -225,27 +222,16 @@ case class ExistsExec(child: Expression, override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { prepareResult() - val childGen = child.genCode(ctx) val setTerm = ctx.addReferenceObj("result", result) val resultCode = s""" |${ev.value} = $setTerm; """.stripMargin - if (nullable) { - val nullSafeEval = ctx.nullSafeExec(child.nullable, childGen.isNull)(resultCode) - ev.copy(code = code""" - ${childGen.code} - boolean ${ev.isNull} = ${childGen.isNull}; - ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; - $nullSafeEval - """) - } else { - ev.copy(code = code""" - ${childGen.code} + ev.copy(code = + code""" ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; $resultCode""", isNull = FalseLiteral) - } } } @@ -273,18 +259,8 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { val executedPlan = new QueryExecution(sparkSession, query).executedPlan InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) case expressions.Exists(sub, children, exprId) => - val expr = if (children.length == 1) { - children.head - } else { - CreateNamedStruct( - children.zipWithIndex.flatMap { case (v, index) => - Seq(Literal(s"col_$index"), v) - } - ) - } val executedPlan = new QueryExecution(sparkSession, Project(Nil, sub)).executedPlan - ExistsExec(expr, sub.treeString, - SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) + ExistsExec(SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) } } } From 3de0eccac5345f1b0037d9653d3e85f743071d18 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 11 Nov 2019 18:54:29 +0800 Subject: [PATCH 11/36] update import --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index cb4fe9a6b489..fc3eab284c77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.plans.logical.{Limit, Project} +import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, DataType, StructType} From 32f85c3e9fb6562f7b7b0f319f43553afa942df5 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 12 Nov 2019 16:03:09 +0800 Subject: [PATCH 12/36] follow comment --- .../org/apache/spark/sql/catalyst/expressions/subquery.scala | 3 ++- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 5 ++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 9586a28e5d8b..e6c4be9d8173 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -66,7 +66,8 @@ object SubqueryExpression { */ def hasInOrCorrelatedExistsSubquery(e: Expression): Boolean = { e.find { - case _: ListQuery | _: Exists if e.children.nonEmpty => true + case _: ListQuery => true + case _: Exists if e.children.nonEmpty => true case _ => false }.isDefined } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index fc3eab284c77..5b9e156cec8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -174,8 +174,7 @@ case class InSubqueryExec( } /** - * The physical node of exists-subquery. This is for support use exists in join's on condition, - * since some join type we can't pushdown exists condition, we plan it here + * The physical node of non-correlated EXISTS subquery. */ case class ExistsExec( plan: BaseSubqueryExec, @@ -188,7 +187,7 @@ case class ExistsExec( override def dataType: DataType = BooleanType override def children: Seq[Expression] = Nil override def nullable: Boolean = false - override def toString: String = s"EXISTS ${plan.name}" + override def toString: String = s"EXISTS (${plan.simpleString(SQLConf.get.maxToStringFields)})" override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = plan) override def semanticEquals(other: Expression): Boolean = other match { From 4c86605a1012f72cf4346ae9e98a4dcc51ba81b5 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 12 Nov 2019 16:09:03 +0800 Subject: [PATCH 13/36] remove broadcaset --- .../apache/spark/sql/execution/subquery.scala | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 5b9e156cec8d..9dfbdc5bd2e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -178,8 +178,7 @@ case class InSubqueryExec( */ case class ExistsExec( plan: BaseSubqueryExec, - exprId: ExprId, - private var resultBroadcast: Broadcast[Boolean] = null) + exprId: ExprId) extends ExecSubqueryExpression { @transient private var result: Boolean = _ @@ -197,30 +196,21 @@ case class ExistsExec( def updateResult(): Unit = { result = plan.executeTake(1).length == 1 - resultBroadcast = plan.sqlContext.sparkContext.broadcast[Boolean](result) } - def values(): Option[Boolean] = Option(resultBroadcast).map(_.value) - - private def prepareResult(): Unit = { - require(resultBroadcast != null, s"$this has not finished") - result = resultBroadcast.value - } + def values(): Option[Boolean] = Option(result) override def eval(input: InternalRow): Any = { - prepareResult() result } override lazy val canonicalized: ExistsExec = { copy( plan = plan.canonicalized.asInstanceOf[BaseSubqueryExec], - exprId = ExprId(0), - resultBroadcast = null) + exprId = ExprId(0)) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - prepareResult() val setTerm = ctx.addReferenceObj("result", result) val resultCode = s""" From 626e41f4a50fba9cb1c12b76cb579f9b272b4cdb Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 13 Nov 2019 10:02:15 +0800 Subject: [PATCH 14/36] Update subquery.scala --- .../apache/spark/sql/execution/subquery.scala | 25 +++++-------------- 1 file changed, 6 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 9dfbdc5bd2e3..51812a4fcc62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -181,7 +181,7 @@ case class ExistsExec( exprId: ExprId) extends ExecSubqueryExpression { - @transient private var result: Boolean = _ + @volatile private var result: Option[Boolean] = None override def dataType: DataType = BooleanType override def children: Seq[Expression] = Nil @@ -195,32 +195,19 @@ case class ExistsExec( } def updateResult(): Unit = { - result = plan.executeTake(1).length == 1 + result = Some(plan.executeTake(1).length == 1) } - def values(): Option[Boolean] = Option(result) + def values(): Option[Boolean] = result override def eval(input: InternalRow): Any = { + require(result.isDefined, s"$this has not finished") result } - override lazy val canonicalized: ExistsExec = { - copy( - plan = plan.canonicalized.asInstanceOf[BaseSubqueryExec], - exprId = ExprId(0)) - } - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val setTerm = ctx.addReferenceObj("result", result) - val resultCode = - s""" - |${ev.value} = $setTerm; - """.stripMargin - - ev.copy(code = - code""" - ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; - $resultCode""", isNull = FalseLiteral) + require(result.isDefined, s"$this has not finished") + Literal.create(result.get, dataType).doGenCode(ctx, ev) } } From ce76e0c6c65c0ce029e01ab709e2dbeb18a01ab2 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 13 Nov 2019 14:18:41 +0800 Subject: [PATCH 15/36] remove unused import --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 51812a4fcc62..310c4c0380f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -24,8 +24,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral} -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf From 4a4ca9b166203e37e5b08ea9454591b17decfd18 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 13 Nov 2019 16:06:15 +0800 Subject: [PATCH 16/36] Update subquery.scala --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 310c4c0380f4..7b7857360751 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -201,7 +201,7 @@ case class ExistsExec( override def eval(input: InternalRow): Any = { require(result.isDefined, s"$this has not finished") - result + result.get } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { From 88f804df753ea8b0c7a61002ff33922f2edbe62e Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 21 Nov 2019 23:33:09 +0800 Subject: [PATCH 17/36] Merge branch 'master' into SPARK-29800 --- .github/workflows/master.yml | 25 +- LICENSE-binary | 2 +- R/pkg/.lintr | 2 +- R/pkg/R/DataFrame.R | 8 +- R/pkg/R/SQLContext.R | 8 +- R/pkg/R/context.R | 2 +- R/pkg/R/deserialize.R | 2 +- R/pkg/R/group.R | 2 +- R/pkg/R/utils.R | 14 +- R/pkg/inst/worker/worker.R | 2 +- R/pkg/tests/fulltests/test_sparkSQL.R | 11 +- R/pkg/tests/fulltests/test_utils.R | 9 + R/run-tests.sh | 2 +- appveyor.yml | 5 +- .../unsafe/types/CalendarIntervalSuite.java | 30 - .../java/org/apache/spark/ExecutorPlugin.java | 60 - .../apache/spark/api/plugin/SparkPlugin.java | 2 +- .../unsafe/sort/UnsafeInMemorySorter.java | 4 + .../ui/static/executorspage-template.html | 15 +- .../apache/spark/ui/static/executorspage.js | 3 +- .../org/apache/spark/ui/static/historypage.js | 1 + .../apache/spark/ui/static/spark-dag-viz.js | 2 +- .../org/apache/spark/ui/static/stagepage.js | 17 +- .../scala/org/apache/spark/SparkConf.scala | 4 +- .../scala/org/apache/spark/SparkEnv.scala | 48 +- ...TaskOutputFileAlreadyExistException.scala} | 17 +- .../org/apache/spark/api/r/BaseRRunner.scala | 17 +- .../org/apache/spark/api/r/RRunner.scala | 5 +- .../deploy/history/FsHistoryProvider.scala | 4 + .../deploy/master/ui/ApplicationPage.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 22 +- .../org/apache/spark/executor/Executor.scala | 37 - .../spark/internal/config/package.scala | 27 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 9 + .../cluster/CoarseGrainedClusterMessage.scala | 2 + .../CoarseGrainedSchedulerBackend.scala | 9 +- .../spark/storage/BlockManagerMaster.scala | 7 + .../storage/BlockManagerMasterEndpoint.scala | 24 +- .../BlockManagerMasterHeartbeatEndpoint.scala | 58 + .../scala/org/apache/spark/ui/UIUtils.scala | 4 +- .../ui/exec/ExecutorThreadDumpPage.scala | 7 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 6 +- .../org/apache/spark/ui/jobs/JobPage.scala | 6 +- .../org/apache/spark/ui/jobs/StageTable.scala | 37 +- .../org/apache/spark/util/ListenerBus.scala | 29 +- .../org/apache/spark/ExecutorPluginSuite.java | 179 --- .../StandaloneDynamicAllocationSuite.scala | 3 +- .../history/FsHistoryProviderSuite.scala | 50 + .../CoarseGrainedExecutorBackendSuite.scala | 41 +- .../plugin/PluginContainerSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 20 + .../BlockManagerReplicationSuite.scala | 6 +- .../spark/storage/BlockManagerSuite.scala | 14 +- dev/create-release/spark-rm/Dockerfile | 4 +- dev/deps/spark-deps-hadoop-2.7 | 7 +- dev/deps/spark-deps-hadoop-3.2 | 7 +- dev/lint-r | 5 +- dev/lint-r.R | 2 +- dev/pip-sanity-check.py | 2 - dev/run-pip-tests | 23 +- dev/run-tests | 6 +- dev/run-tests-jenkins | 8 +- dev/run-tests-jenkins.py | 3 +- dev/run-tests.py | 12 +- dev/sparktestsupport/shellutils.py | 6 +- docs/README.md | 3 +- docs/_config.yml | 2 +- docs/_plugins/include_example.rb | 6 +- docs/configuration.md | 45 + docs/core-migration-guide.md | 17 +- docs/css/pygments-default.css | 6 +- docs/ml-classification-regression.md | 9 +- docs/sql-keywords.md | 8 +- docs/sql-migration-guide.md | 2 - docs/sql-ref-syntax-aux-show-databases.md | 2 +- docs/sql-ref-syntax-aux-show-tblproperties.md | 3 +- docs/sql-ref-syntax-ddl-alter-view.md | 215 ++- docs/sql-ref-syntax-ddl-create-view.md | 6 +- docs/sql-ref-syntax-ddl-drop-view.md | 1 + docs/sql-ref-syntax-ddl-truncate-table.md | 2 +- docs/ss-migration-guide.md | 1 + .../DecisionTreeClassifier.scala | 3 +- .../ml/classification/GBTClassifier.scala | 3 +- .../spark/ml/classification/LinearSVC.scala | 4 + .../classification/LogisticRegression.scala | 3 +- .../MultilayerPerceptronClassifier.scala | 6 + .../spark/ml/classification/NaiveBayes.scala | 302 +++- .../spark/ml/classification/OneVsRest.scala | 6 + .../RandomForestClassifier.scala | 3 +- .../spark/ml/clustering/BisectingKMeans.scala | 9 + .../spark/ml/clustering/GaussianMixture.scala | 8 + .../apache/spark/ml/clustering/KMeans.scala | 9 + .../org/apache/spark/ml/clustering/LDA.scala | 10 + .../BinaryClassificationEvaluator.scala | 6 + .../ml/evaluation/ClusteringEvaluator.scala | 6 + .../MulticlassClassificationEvaluator.scala | 6 + .../MultilabelClassificationEvaluator.scala | 6 + .../ml/evaluation/RankingEvaluator.scala | 5 + .../ml/evaluation/RegressionEvaluator.scala | 6 + .../apache/spark/ml/feature/Binarizer.scala | 7 + .../feature/BucketedRandomProjectionLSH.scala | 5 + .../apache/spark/ml/feature/Bucketizer.scala | 7 + .../spark/ml/feature/ChiSqSelector.scala | 5 + .../spark/ml/feature/CountVectorizer.scala | 7 +- .../org/apache/spark/ml/feature/DCT.scala | 5 + .../spark/ml/feature/ElementwiseProduct.scala | 6 + .../spark/ml/feature/FeatureHasher.scala | 9 +- .../apache/spark/ml/feature/HashingTF.scala | 5 + .../org/apache/spark/ml/feature/IDF.scala | 6 +- .../org/apache/spark/ml/feature/Imputer.scala | 7 + .../apache/spark/ml/feature/Interaction.scala | 5 + .../spark/ml/feature/MaxAbsScaler.scala | 5 + .../apache/spark/ml/feature/MinHashLSH.scala | 5 + .../spark/ml/feature/MinMaxScaler.scala | 6 + .../org/apache/spark/ml/feature/NGram.scala | 5 + .../apache/spark/ml/feature/Normalizer.scala | 5 + .../spark/ml/feature/OneHotEncoder.scala | 7 + .../org/apache/spark/ml/feature/PCA.scala | 5 + .../ml/feature/PolynomialExpansion.scala | 5 + .../apache/spark/ml/feature/RFormula.scala | 9 +- .../spark/ml/feature/RobustScaler.scala | 6 + .../spark/ml/feature/SQLTransformer.scala | 5 + .../spark/ml/feature/StandardScaler.scala | 6 + .../spark/ml/feature/StopWordsRemover.scala | 68 +- .../spark/ml/feature/StringIndexer.scala | 10 +- .../spark/ml/feature/VectorAssembler.scala | 6 + .../spark/ml/feature/VectorIndexer.scala | 7 +- .../spark/ml/feature/VectorSizeHint.scala | 5 + .../spark/ml/feature/VectorSlicer.scala | 6 + .../apache/spark/ml/feature/Word2Vec.scala | 6 + .../org/apache/spark/ml/fpm/FPGrowth.scala | 5 + .../apache/spark/ml/recommendation/ALS.scala | 11 +- .../ml/regression/AFTSurvivalRegression.scala | 8 + .../ml/regression/DecisionTreeRegressor.scala | 3 +- .../spark/ml/regression/GBTRegressor.scala | 2 +- .../GeneralizedLinearRegression.scala | 6 + .../ml/regression/IsotonicRegression.scala | 8 + .../ml/regression/LinearRegression.scala | 5 + .../ml/regression/RandomForestRegressor.scala | 2 +- .../spark/ml/tuning/CrossValidator.scala | 5 + .../ml/tuning/TrainValidationSplit.scala | 7 +- .../spark/mllib/clustering/KMeans.scala | 4 +- .../LogisticRegressionSuite.scala | 2 +- .../ml/classification/NaiveBayesSuite.scala | 193 ++- .../ml/feature/StopWordsRemoverSuite.scala | 133 +- pom.xml | 8 +- project/MimaExcludes.scala | 11 +- project/SparkBuild.scala | 1 + python/pyspark/context.py | 2 - python/pyspark/ml/classification.py | 47 +- python/pyspark/ml/clustering.py | 8 +- python/pyspark/ml/feature.py | 69 +- python/pyspark/ml/fpm.py | 2 +- python/pyspark/ml/recommendation.py | 2 + python/pyspark/ml/regression.py | 16 +- python/pyspark/ml/tree.py | 6 - python/pyspark/ml/wrapper.py | 3 + python/pyspark/sql/utils.py | 6 +- python/pyspark/version.py | 2 +- python/run-tests | 8 +- python/run-tests.py | 22 +- python/setup.py | 9 +- .../k8s/KubernetesClusterManager.scala | 13 +- .../ClientModeTestsSuite.scala | 27 +- .../backend/minikube/Minikube.scala | 18 +- .../org/apache/spark/deploy/yarn/Client.scala | 6 +- .../YarnCoarseGrainedExecutorBackend.scala | 4 +- .../spark/deploy/yarn/ClientSuite.scala | 14 +- sql/catalyst/pom.xml | 2 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 45 +- .../connector/catalog/SupportsNamespaces.java | 8 +- .../connector/expressions/Expressions.java | 29 +- .../sql/catalyst/analysis/Analyzer.scala | 17 + .../catalyst/analysis/FunctionRegistry.scala | 13 +- .../catalyst/analysis/PostgreSQLDialect.scala | 12 +- .../catalyst/analysis/ResolveCatalogs.scala | 32 +- .../sql/catalyst/analysis/TypeCoercion.scala | 7 +- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../spark/sql/catalyst/expressions/Cast.scala | 36 +- .../sql/catalyst/expressions/TimeWindow.scala | 3 +- .../aggregate/UnevaluableAggs.scala | 8 +- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../expressions/collectionOperations.scala | 48 - .../expressions/complexTypeCreator.scala | 2 +- .../expressions/higherOrderFunctions.scala | 108 ++ .../expressions/intervalExpressions.scala | 68 + .../expressions/jsonExpressions.scala | 4 +- .../PostgreCastStringToBoolean.scala | 80 - .../postgreSQL/PostgreCastToBoolean.scala | 83 ++ .../sql/catalyst/optimizer/Optimizer.scala | 11 +- .../catalyst/optimizer/finishAnalysis.scala | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 171 ++- .../sql/catalyst/parser/ParseDriver.scala | 13 +- .../plans/logical/basicLogicalOperators.scala | 2 + .../catalyst/plans/logical/statements.scala | 63 +- .../catalyst/plans/logical/v2Commands.scala | 52 +- .../sql/catalyst/util/IntervalUtils.scala | 447 +++--- .../connector/catalog/CatalogManager.scala | 16 +- .../catalog/CatalogV2Implicits.scala | 9 +- .../spark/sql/connector/catalog/V1Table.scala | 7 +- .../connector/expressions/expressions.scala | 20 +- .../apache/spark/sql/internal/SQLConf.scala | 103 +- .../spark/sql/internal/StaticSQLConf.scala | 7 + ...eateTablePartitioningValidationSuite.scala | 16 +- .../ExpressionTypeCheckingSuite.scala | 4 +- .../catalyst/analysis/TypeCoercionSuite.scala | 29 +- .../encoders/ExpressionEncoderSuite.scala | 2 +- .../catalyst/encoders/RowEncoderSuite.scala | 4 +- .../ArithmeticExpressionSuite.scala | 24 +- .../sql/catalyst/expressions/CastSuite.scala | 17 +- .../CollectionExpressionsSuite.scala | 47 +- .../expressions/DateExpressionsSuite.scala | 22 +- .../expressions/DecimalExpressionSuite.scala | 4 +- .../expressions/HashExpressionsSuite.scala | 4 +- .../HigherOrderFunctionsSuite.scala | 50 + .../IntervalExpressionsSuite.scala | 16 +- .../expressions/JsonExpressionsSuite.scala | 36 +- .../expressions/MutableProjectionSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 3 +- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +- .../expressions/UnsafeRowConverterSuite.scala | 3 +- .../expressions/postgreSQL/CastSuite.scala | 74 +- .../sql/catalyst/parser/DDLParserSuite.scala | 127 +- .../catalyst/parser/DataTypeParserSuite.scala | 3 + .../parser/ExpressionParserSuite.scala | 23 +- .../parser/TableIdentifierParserSuite.scala | 2 +- .../catalyst/util/IntervalUtilsSuite.scala | 130 +- .../sql/connector/InMemoryTableCatalog.scala | 14 +- .../catalog/CatalogManagerSuite.scala | 4 +- .../connector/catalog/TableCatalogSuite.scala | 11 +- .../sql/util/TimestampFormatterSuite.scala | 39 + .../IntervalBenchmark-jdk11-results.txt | 48 +- .../benchmarks/IntervalBenchmark-results.txt | 48 +- sql/core/pom.xml | 2 +- .../sql/execution/ui/static/spark-sql-viz.js | 2 +- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../apache/spark/sql/DataFrameWriterV2.scala | 16 +- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../analysis/ResolveSessionCatalog.scala | 45 +- .../spark/sql/execution/HiveResult.scala | 16 +- .../spark/sql/execution/SparkSqlParser.scala | 92 -- .../spark/sql/execution/SparkStrategies.scala | 5 +- .../aggregate/HashAggregateExec.scala | 34 +- .../sql/execution/arrow/ArrowConverters.scala | 8 +- .../datasources/FileFormatWriter.scala | 6 +- .../datasources/jdbc/JdbcUtils.scala | 2 +- .../v2/AlterNamespaceSetPropertiesExec.scala | 40 + .../datasources/v2/DataSourceV2Strategy.scala | 44 +- .../v2/DescribeNamespaceExec.scala | 62 + .../datasources/v2/DropNamespaceExec.scala | 26 +- .../datasources/v2/RenameTableExec.scala | 40 +- .../v2/ShowTablePropertiesExec.scala | 48 + .../exchange/EnsureRequirements.scala | 19 +- .../spark/sql/execution/r/ArrowRRunner.scala | 6 +- .../execution/streaming/GroupStateImpl.scala | 4 +- .../sql/execution/streaming/OffsetSeq.scala | 9 +- .../StreamingSymmetricHashJoinExec.scala | 84 +- .../sql/execution/streaming/Triggers.scala | 3 +- .../state/SymmetricHashJoinStateManager.scala | 251 +++- .../execution/ui/SQLAppStatusListener.scala | 21 +- .../org/apache/spark/sql/functions.scala | 2 +- .../spark/sql/internal/SharedState.scala | 33 +- .../sql/streaming/StreamingQueryManager.scala | 82 +- .../ansi/decimalArithmeticOperations.sql | 32 + .../inputs/ansi/higher-order-functions.sql | 1 + .../sql-tests/inputs/ansi/interval.sql | 201 +-- .../sql-tests/inputs/ansi/literals.sql | 2 + .../inputs/decimalArithmeticOperations.sql | 24 - .../resources/sql-tests/inputs/group-by.sql | 64 - .../inputs/higher-order-functions.sql | 5 - .../resources/sql-tests/inputs/inner-join.sql | 5 + .../inputs/interval-display-iso_8601.sql | 3 + .../inputs/interval-display-sql_standard.sql | 3 + .../sql-tests/inputs/interval-display.sql | 14 + .../resources/sql-tests/inputs/interval.sql | 194 ++- .../resources/sql-tests/inputs/literals.sql | 54 +- .../sql-tests/inputs/postgreSQL/interval.sql | 18 +- .../sql-tests/inputs/postgreSQL/join.sql | 6 + .../sql-tests/inputs/postgreSQL/text.sql | 4 +- .../inputs/subquery/in-subquery/in-joins.sql | 110 ++ .../ansi/decimalArithmeticOperations.sql.out | 138 ++ .../ansi/higher-order-functions.sql.out | 284 ++++ .../sql-tests/results/ansi/interval.sql.out | 1326 +++++++++++++---- .../sql-tests/results/ansi/literals.sql.out | 474 ++++++ .../decimalArithmeticOperations.sql.out | 129 +- .../sql-tests/results/group-by.sql.out | 210 +-- .../results/higher-order-functions.sql.out | 46 +- .../results/interval-display-iso_8601.sql.out | 21 + .../interval-display-sql_standard.sql.out | 21 + .../results/interval-display.sql.out | 21 + .../sql-tests/results/interval.sql.out | 833 ++++++++++- .../sql-tests/results/literals.sql.out | 488 +----- .../results/postgreSQL/boolean.sql.out | 60 +- .../results/postgreSQL/interval.sql.out | 187 ++- .../sql-tests/results/postgreSQL/text.sql.out | 8 +- .../subquery/in-subquery/in-joins.sql.out | 300 +++- .../results/udf/udf-group-by.sql.out | 26 +- .../commits/0 | 2 + .../metadata | 1 + .../offsets/0 | 4 + .../state/0/0/left-keyToNumValues/1.delta | Bin 0 -> 46 bytes .../0/0/left-keyWithIndexToValue/1.delta | Bin 0 -> 46 bytes .../state/0/0/right-keyToNumValues/1.delta | Bin 0 -> 46 bytes .../0/0/right-keyWithIndexToValue/1.delta | Bin 0 -> 46 bytes .../state/0/1/left-keyToNumValues/1.delta | Bin 0 -> 86 bytes .../0/1/left-keyWithIndexToValue/1.delta | Bin 0 -> 92 bytes .../state/0/1/right-keyToNumValues/1.delta | Bin 0 -> 46 bytes .../0/1/right-keyWithIndexToValue/1.delta | Bin 0 -> 46 bytes .../state/0/2/left-keyToNumValues/1.delta | Bin 0 -> 70 bytes .../0/2/left-keyWithIndexToValue/1.delta | Bin 0 -> 72 bytes .../state/0/2/right-keyToNumValues/1.delta | Bin 0 -> 70 bytes .../0/2/right-keyWithIndexToValue/1.delta | Bin 0 -> 72 bytes .../state/0/3/left-keyToNumValues/1.delta | Bin 0 -> 64 bytes .../0/3/left-keyWithIndexToValue/1.delta | Bin 0 -> 73 bytes .../state/0/3/right-keyToNumValues/1.delta | Bin 0 -> 46 bytes .../0/3/right-keyWithIndexToValue/1.delta | Bin 0 -> 46 bytes .../state/0/4/left-keyToNumValues/1.delta | Bin 0 -> 70 bytes .../0/4/left-keyWithIndexToValue/1.delta | Bin 0 -> 73 bytes .../state/0/4/right-keyToNumValues/1.delta | Bin 0 -> 70 bytes .../0/4/right-keyWithIndexToValue/1.delta | Bin 0 -> 73 bytes .../spark/sql/DataFrameFunctionsSuite.scala | 94 +- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../apache/spark/sql/DateFunctionsSuite.scala | 10 +- .../org/apache/spark/sql/ExplainSuite.scala | 4 +- .../spark/sql/IntegratedUDFTestUtils.scala | 20 +- .../org/apache/spark/sql/JoinSuite.scala | 9 + .../apache/spark/sql/JsonFunctionsSuite.scala | 10 + .../sql/PostgreSQLDialectQuerySuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 9 + .../apache/spark/sql/SQLQueryTestSuite.scala | 66 +- .../org/apache/spark/sql/SubquerySuite.scala | 254 ++-- .../DataSourceV2DataFrameSuite.scala | 38 + .../sql/connector/DataSourceV2SQLSuite.scala | 256 +++- .../ReduceNumShufflePartitionsSuite.scala | 3 +- .../benchmark/IntervalBenchmark.scala | 3 +- .../execution/command/DDLParserSuite.scala | 84 +- .../command/PlanResolutionSuite.scala | 168 ++- .../execution/datasources/csv/CSVSuite.scala | 12 + .../SymmetricHashJoinStateManagerSuite.scala | 18 +- .../spark/sql/sources/BucketedReadSuite.scala | 31 +- .../spark/sql/sources/InsertSuite.scala | 33 + .../sql/streaming/StreamingJoinSuite.scala | 225 ++- .../StreamingQueryManagerSuite.scala | 134 +- .../sql/streaming/StreamingQuerySuite.scala | 8 +- .../thriftserver/ui/ThriftServerPage.scala | 43 +- .../sql/hive/thriftserver/ui/ToolTips.scala | 7 + .../ThriftServerQueryTestSuite.scala | 30 +- sql/hive/pom.xml | 2 +- .../spark/streaming/ui/StreamingPage.scala | 8 +- .../streaming/ReceivedBlockHandlerSuite.scala | 6 +- 352 files changed, 9176 insertions(+), 3778 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/ExecutorPlugin.java rename core/src/main/scala/org/apache/spark/{executor/ExecutorPluginSource.scala => TaskOutputFileAlreadyExistException.scala} (73%) create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala delete mode 100644 core/src/test/java/org/apache/spark/ExecutorPluginSuite.java mode change 100644 => 100755 python/setup.py delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala rename core/src/main/java/org/apache/spark/ExecutorPluginContext.java => sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala (51%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala create mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display.sql.out create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/commits/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/metadata create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/offsets/0 create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/left-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/left-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/right-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/right-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/left-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/left-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/right-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/right-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/left-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/left-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/right-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/right-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/left-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/left-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/right-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/right-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/left-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/left-keyWithIndexToValue/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/right-keyToNumValues/1.delta create mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/right-keyWithIndexToValue/1.delta diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 5eb00c4aba0f..d2b7dca3684f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -50,7 +50,7 @@ jobs: lint: runs-on: ubuntu-latest - name: Linters + name: Linters (Java/Scala/Python), licenses, dependencies steps: - uses: actions/checkout@master - uses: actions/setup-java@v1 @@ -72,3 +72,26 @@ jobs: run: ./dev/check-license - name: Dependencies run: ./dev/test-dependencies.sh + + lintr: + runs-on: ubuntu-latest + name: Linter (R) + steps: + - uses: actions/checkout@master + - uses: actions/setup-java@v1 + with: + java-version: '11' + - name: install R + run: | + echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' | sudo tee -a /etc/apt/sources.list + sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev + - name: install R packages + run: | + sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" + sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')" + - name: package and install SparkR + run: ./R/install-dev.sh + - name: lint-r + run: ./dev/lint-r diff --git a/LICENSE-binary b/LICENSE-binary index 6858193515a8..0b45686f9ba3 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -502,7 +502,7 @@ com.github.scopt:scopt_2.12 core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js core/src/main/resources/org/apache/spark/ui/static/*dataTables* core/src/main/resources/org/apache/spark/ui/static/graphlib-dot.min.js -ore/src/main/resources/org/apache/spark/ui/static/jquery* +core/src/main/resources/org/apache/spark/ui/static/jquery* core/src/main/resources/org/apache/spark/ui/static/sorttable.js docs/js/vendor/anchor.min.js docs/js/vendor/jquery* diff --git a/R/pkg/.lintr b/R/pkg/.lintr index c83ad2adfe0e..67dc1218ea55 100644 --- a/R/pkg/.lintr +++ b/R/pkg/.lintr @@ -1,2 +1,2 @@ -linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE)) +linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE), object_usage_linter = NULL, cyclocomp_linter = NULL) exclusions: list("inst/profile/general.R" = 1, "inst/profile/shell.R") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 6f3c7c120ba3..593d3ca16220 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2252,7 +2252,7 @@ setMethod("mutate", # The last column of the same name in the specific columns takes effect deDupCols <- list() - for (i in 1:length(cols)) { + for (i in seq_len(length(cols))) { deDupCols[[ns[[i]]]] <- alias(cols[[i]], ns[[i]]) } @@ -2416,7 +2416,7 @@ setMethod("arrange", # builds a list of columns of type Column # example: [[1]] Column Species ASC # [[2]] Column Petal_Length DESC - jcols <- lapply(seq_len(length(decreasing)), function(i){ + jcols <- lapply(seq_len(length(decreasing)), function(i) { if (decreasing[[i]]) { desc(getColumn(x, by[[i]])) } else { @@ -2749,7 +2749,7 @@ genAliasesForIntersectedCols <- function(x, intersectedColNames, suffix) { col <- getColumn(x, colName) if (colName %in% intersectedColNames) { newJoin <- paste(colName, suffix, sep = "") - if (newJoin %in% allColNames){ + if (newJoin %in% allColNames) { stop("The following column name: ", newJoin, " occurs more than once in the 'DataFrame'.", "Please use different suffixes for the intersected columns.") } @@ -3475,7 +3475,7 @@ setMethod("str", cat(paste0("'", class(object), "': ", length(names), " variables:\n")) if (nrow(localDF) > 0) { - for (i in 1 : ncol(localDF)) { + for (i in seq_len(ncol(localDF))) { # Get the first elements for each column firstElements <- if (types[i] == "character") { diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index f27ef4ee28f1..f48a334ed676 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -166,9 +166,9 @@ writeToFileInArrow <- function(fileName, rdf, numPartitions) { for (rdf_slice in rdf_slices) { batch <- arrow::record_batch(rdf_slice) if (is.null(stream_writer)) { - stream <- arrow::FileOutputStream(fileName) + stream <- arrow::FileOutputStream$create(fileName) schema <- batch$schema - stream_writer <- arrow::RecordBatchStreamWriter(stream, schema) + stream_writer <- arrow::RecordBatchStreamWriter$create(stream, schema) } stream_writer$write_batch(batch) @@ -197,7 +197,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { as.list(schema) } if (is.null(names)) { - names <- lapply(1:length(firstRow), function(x) { + names <- lapply(seq_len(length(firstRow)), function(x) { paste0("_", as.character(x)) }) } @@ -213,7 +213,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { }) types <- lapply(firstRow, infer_type) - fields <- lapply(1:length(firstRow), function(i) { + fields <- lapply(seq_len(length(firstRow)), function(i) { structField(names[[i]], types[[i]], TRUE) }) schema <- do.call(structType, fields) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 93ba1307043a..d96a287f818a 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -416,7 +416,7 @@ spark.getSparkFiles <- function(fileName) { #' @examples #'\dontrun{ #' sparkR.session() -#' doubled <- spark.lapply(1:10, function(x){2 * x}) +#' doubled <- spark.lapply(1:10, function(x) {2 * x}) #'} #' @note spark.lapply since 2.0.0 spark.lapply <- function(list, func) { diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index a6febb1cbd13..ca4a6e342d77 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -242,7 +242,7 @@ readDeserializeInArrow <- function(inputCon) { # for now. dataLen <- readInt(inputCon) arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") - batches <- arrow::RecordBatchStreamReader(arrowData)$batches() + batches <- arrow::RecordBatchStreamReader$create(arrowData)$batches() if (useAsTibble) { as_tibble <- get("as_tibble", envir = asNamespace("arrow")) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 6e8f4dc3a790..2b7995e1e37f 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -162,7 +162,7 @@ methods <- c("avg", "max", "mean", "min", "sum") #' @note pivot since 2.0.0 setMethod("pivot", signature(x = "GroupedData", colname = "character"), - function(x, colname, values = list()){ + function(x, colname, values = list()) { stopifnot(length(colname) == 1) if (length(values) == 0) { result <- callJMethod(x@sgd, "pivot", colname) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index c3501977e64b..a8c1ddb3dd20 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -131,7 +131,7 @@ hashCode <- function(key) { } else { asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) hashC <- 0 - for (k in 1:length(asciiVals)) { + for (k in seq_len(length(asciiVals))) { hashC <- mult31AndAdd(hashC, asciiVals[k]) } as.integer(hashC) @@ -543,10 +543,14 @@ processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F, ifnotfound = list(list(NULL)))[[1]] found <- sapply(funcList, function(func) { - ifelse(identical(func, obj), TRUE, FALSE) + ifelse( + identical(func, obj) && + # Also check if the parent environment is identical to current parent + identical(parent.env(environment(func)), func.env), + TRUE, FALSE) }) if (sum(found) > 0) { - # If function has been examined, ignore. + # If function has been examined ignore break } # Function has not been examined, record it and recursively clean its closure. @@ -724,7 +728,7 @@ assignNewEnv <- function(data) { stopifnot(length(cols) > 0) env <- new.env() - for (i in 1:length(cols)) { + for (i in seq_len(length(cols))) { assign(x = cols[i], value = data[, cols[i], drop = F], envir = env) } env @@ -750,7 +754,7 @@ launchScript <- function(script, combinedArgs, wait = FALSE, stdout = "", stderr if (.Platform$OS.type == "windows") { scriptWithArgs <- paste(script, combinedArgs, sep = " ") # on Windows, intern = F seems to mean output to the console. (documentation on this is missing) - shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) # nolint + shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) } else { # http://stat.ethz.ch/R-manual/R-devel/library/base/html/system2.html # stdout = F means discard output diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index dfe69b7f4f1f..1ef05ea621e8 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -194,7 +194,7 @@ if (isEmpty != 0) { } else { # gapply mode outputs <- list() - for (i in 1:length(data)) { + for (i in seq_len(length(data))) { # Timing reading input data for execution inputElap <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 035525a7a849..cb47353d600d 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -172,7 +172,7 @@ test_that("structField type strings", { typeList <- c(primitiveTypes, complexTypes) typeStrings <- names(typeList) - for (i in seq_along(typeStrings)){ + for (i in seq_along(typeStrings)) { typeString <- typeStrings[i] expected <- typeList[[i]] testField <- structField("_col", typeString) @@ -203,7 +203,7 @@ test_that("structField type strings", { errorList <- c(primitiveErrors, complexErrors) typeStrings <- names(errorList) - for (i in seq_along(typeStrings)){ + for (i in seq_along(typeStrings)) { typeString <- typeStrings[i] expected <- paste0("Unsupported type for SparkDataframe: ", errorList[[i]]) expect_error(structField("_col", typeString), expected) @@ -3238,6 +3238,13 @@ test_that("Histogram", { expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) }) +test_that("dapply() should show error message from R worker", { + df <- createDataFrame(list(list(n = 1))) + expect_error({ + collect(dapply(df, function(x) stop("custom error message"), structType("a double"))) + }, "custom error message") +}) + test_that("dapply() and dapplyCollect() on a DataFrame", { df <- createDataFrame( list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), diff --git a/R/pkg/tests/fulltests/test_utils.R b/R/pkg/tests/fulltests/test_utils.R index b2b6f34aaa08..c4fcbecee18e 100644 --- a/R/pkg/tests/fulltests/test_utils.R +++ b/R/pkg/tests/fulltests/test_utils.R @@ -110,6 +110,15 @@ test_that("cleanClosure on R functions", { actual <- get("y", envir = env, inherits = FALSE) expect_equal(actual, y) + # Test for combination for nested and sequenctial functions in a closure + f1 <- function(x) x + 1 + f2 <- function(x) f1(x) + 2 + userFunc <- function(x) { f1(x); f2(x) } + cUserFuncEnv <- environment(cleanClosure(userFunc)) + expect_equal(length(cUserFuncEnv), 2) + innerCUserFuncEnv <- environment(cUserFuncEnv$f2) + expect_equal(length(innerCUserFuncEnv), 1) + # Test for function (and variable) definitions. f <- function(x) { g <- function(y) { y * 2 } diff --git a/R/run-tests.sh b/R/run-tests.sh index 86bd8aad5f11..51ca7d600caf 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -23,7 +23,7 @@ FAILED=0 LOGFILE=$FWDIR/unit-tests.out rm -f $LOGFILE -SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" --conf spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" --conf spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE FAILED=$((PIPESTATUS[0]||$FAILED)) NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" diff --git a/appveyor.yml b/appveyor.yml index b36175a787ae..00c688ba18eb 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -42,10 +42,7 @@ install: # Install maven and dependencies - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" - # Use Arrow R 0.14.1 for now. 0.15.0 seems not working for now. See SPARK-29378. - - cmd: R -e "install.packages(c('assertthat', 'bit64', 'fs', 'purrr', 'R6', 'tidyselect'), repos='https://cloud.r-project.org/')" - - cmd: R -e "install.packages('https://cran.r-project.org/src/contrib/Archive/arrow/arrow_0.14.1.tar.gz', repos=NULL, type='source')" + - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 6397f26c02f3..01bf7eb2438a 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -46,36 +46,6 @@ public void equalsTest() { assertEquals(i1, i6); } - @Test - public void toStringTest() { - CalendarInterval i; - - i = new CalendarInterval(0, 0, 0); - assertEquals("0 seconds", i.toString()); - - i = new CalendarInterval(34, 0, 0); - assertEquals("2 years 10 months", i.toString()); - - i = new CalendarInterval(-34, 0, 0); - assertEquals("-2 years -10 months", i.toString()); - - i = new CalendarInterval(0, 31, 0); - assertEquals("31 days", i.toString()); - - i = new CalendarInterval(0, -31, 0); - assertEquals("-31 days", i.toString()); - - i = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); - assertEquals("3 hours 13 minutes 0.000123 seconds", i.toString()); - - i = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123); - assertEquals("-3 hours -13 minutes -0.000123 seconds", i.toString()); - - i = new CalendarInterval(34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); - assertEquals("2 years 10 months 31 days 3 hours 13 minutes 0.000123 seconds", - i.toString()); - } - @Test public void periodAndDurationTest() { CalendarInterval interval = new CalendarInterval(120, -40, 123456); diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java deleted file mode 100644 index b25c46266247..000000000000 --- a/core/src/main/java/org/apache/spark/ExecutorPlugin.java +++ /dev/null @@ -1,60 +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; - -import org.apache.spark.annotation.DeveloperApi; - -/** - * A plugin which can be automatically instantiated within each Spark executor. Users can specify - * plugins which should be created with the "spark.executor.plugins" configuration. An instance - * of each plugin will be created for every executor, including those created by dynamic allocation, - * before the executor starts running any tasks. - * - * The specific api exposed to the end users still considered to be very unstable. We will - * hopefully be able to keep compatibility by providing default implementations for any methods - * added, but make no guarantees this will always be possible across all Spark releases. - * - * Spark does nothing to verify the plugin is doing legitimate things, or to manage the resources - * it uses. A plugin acquires the same privileges as the user running the task. A bad plugin - * could also interfere with task execution and make the executor fail in unexpected ways. - */ -@DeveloperApi -public interface ExecutorPlugin { - - /** - * Initialize the executor plugin. - * - *

Each executor will, during its initialization, invoke this method on each - * plugin provided in the spark.executor.plugins configuration. The Spark executor - * will wait on the completion of the execution of the init method.

- * - *

Plugins should create threads in their implementation of this method for - * any polling, blocking, or intensive computation.

- * - * @param pluginContext Context information for the executor where the plugin is running. - */ - default void init(ExecutorPluginContext pluginContext) {} - - /** - * Clean up and terminate this plugin. - * - *

This function is called during the executor shutdown phase. The executor - * will wait for the plugin to terminate before continuing its own shutdown.

- */ - default void shutdown() {} -} diff --git a/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java b/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java index a500f5d2188f..21ddae37d8a0 100644 --- a/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java +++ b/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java @@ -24,7 +24,7 @@ * A plugin that can be dynamically loaded into a Spark application. *

* Plugins can be loaded by adding the plugin's class name to the appropriate Spark configuration. - * Check the Spark configuration documentation for details. + * Check the Spark monitoring guide for details. *

* Plugins have two optional components: a driver-side component, of which a single instance is * created per application, inside the Spark driver. And an executor-side component, of which one diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 1a9453a8b3e8..e14964d68119 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -205,6 +205,10 @@ public long getSortTimeNanos() { } public long getMemoryUsage() { + if (array == null) { + return 0L; + } + return array.size() * 8; } diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 4c06ddf3b31f..27a4d2cdb5b5 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -50,12 +50,12 @@

Summary

title="Total shuffle bytes and records read (includes both data read locally and data read from remote executors)."> Shuffle Read - Shuffle Write - Blacklisted @@ -72,7 +72,7 @@

Executors

- Executor ID + Executor ID Address Status @@ -110,14 +110,11 @@

Executors

title="Total shuffle bytes and records read (includes both data read locally and data read from remote executors)."> Shuffle Read - - Shuffle Write - Logs - Thread Dump + Logs + Thread Dump diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 11d7c77d0c66..090bc72dc927 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -462,7 +462,8 @@ $(document).ready(function () { {"visible": false, "targets": 5}, {"visible": false, "targets": 6}, {"visible": false, "targets": 9} - ] + ], + "deferRender": true }; execDataTable = $(selector).DataTable(conf); diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index 5f291620e0e9..4df5f07f077d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -177,6 +177,7 @@ $(document).ready(function() { {name: 'eventLog'}, ], "autoWidth": false, + "deferRender": true }; if (hasMultipleAttempts) { diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index 035d72f8956f..25dec9d3788b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -513,7 +513,7 @@ function addTooltipsForRDDs(svgContainer) { if (tooltipText) { node.select("circle") .attr("data-toggle", "tooltip") - .attr("data-placement", "bottom") + .attr("data-placement", "top") .attr("data-html", "true") // to interpret line break, tooltipText is showing title .attr("title", tooltipText); } diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index f01d030c73a4..4b2f007f866c 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -297,26 +297,26 @@ $(document).ready(function () { ""); $('#scheduler_delay').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Scheduler delay includes time to ship the task from the scheduler to the executor, and time to send " + "the task result from the executor to the scheduler. If scheduler delay is large, consider decreasing the size of tasks or decreasing the size of task results."); $('#task_deserialization_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time spent deserializing the task closure on the executor, including the time to read the broadcasted task."); $('#shuffle_read_blocked_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time that the task spent blocked waiting for shuffle data to be read from remote machines."); $('#shuffle_remote_reads').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Total shuffle bytes read from remote executors. This is a subset of the shuffle read bytes; the remaining shuffle data is read locally. "); $('#result_serialization_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time spent serializing the task result on the executor before sending it back to the driver."); $('#getting_result_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time that the driver spends fetching task results from workers. If this is large, consider decreasing the amount of data returned from each task."); $('#peak_execution_memory').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Execution memory refers to the memory used by internal data structures created during " + "shuffles, aggregations and joins when Tungsten is enabled. The value of this accumulator " + "should be approximately the sum of the peak sizes across all such data structures created " + @@ -880,7 +880,8 @@ $(document).ready(function () { { "visible": false, "targets": 16 }, { "visible": false, "targets": 17 }, { "visible": false, "targets": 18 } - ] + ], + "deferRender": true }; taskTableSelector = $(taskTable).DataTable(taskConf); $('#active-tasks-table_filter input').unbind(); diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 3a2eaae092e8..0e0291d2407d 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -619,7 +619,9 @@ private[spark] object SparkConf extends Logging { "Not used anymore. Please use spark.shuffle.service.index.cache.size"), DeprecatedConfig("spark.yarn.credentials.file.retention.count", "2.4.0", "Not used anymore."), DeprecatedConfig("spark.yarn.credentials.file.retention.days", "2.4.0", "Not used anymore."), - DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available.") + DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available."), + DeprecatedConfig("spark.executor.plugins", "3.0.0", + "Feature replaced with new plugin API. See Monitoring documentation.") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 9232938464e0..5fe1c663affa 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -22,6 +22,7 @@ import java.net.Socket import java.util.Locale import scala.collection.JavaConverters._ +import scala.collection.concurrent import scala.collection.mutable import scala.util.Properties @@ -195,6 +196,7 @@ object SparkEnv extends Logging { private[spark] def createExecutorEnv( conf: SparkConf, executorId: String, + bindAddress: String, hostname: String, numCores: Int, ioEncryptionKey: Option[Array[Byte]], @@ -202,7 +204,7 @@ object SparkEnv extends Logging { val env = create( conf, executorId, - hostname, + bindAddress, hostname, None, isLocal, @@ -213,6 +215,17 @@ object SparkEnv extends Logging { env } + private[spark] def createExecutorEnv( + conf: SparkConf, + executorId: String, + hostname: String, + numCores: Int, + ioEncryptionKey: Option[Array[Byte]], + isLocal: Boolean): SparkEnv = { + createExecutorEnv(conf, executorId, hostname, + hostname, numCores, ioEncryptionKey, isLocal) + } + /** * Helper method to create a SparkEnv for a driver or an executor. */ @@ -339,19 +352,26 @@ object SparkEnv extends Logging { None } - val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint( - BlockManagerMaster.DRIVER_ENDPOINT_NAME, - new BlockManagerMasterEndpoint( - rpcEnv, - isLocal, - conf, - listenerBus, - if (conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)) { - externalShuffleClient - } else { - None - })), - conf, isDriver) + // Mapping from block manager id to the block manager's information. + val blockManagerInfo = new concurrent.TrieMap[BlockManagerId, BlockManagerInfo]() + val blockManagerMaster = new BlockManagerMaster( + registerOrLookupEndpoint( + BlockManagerMaster.DRIVER_ENDPOINT_NAME, + new BlockManagerMasterEndpoint( + rpcEnv, + isLocal, + conf, + listenerBus, + if (conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)) { + externalShuffleClient + } else { + None + }, blockManagerInfo)), + registerOrLookupEndpoint( + BlockManagerMaster.DRIVER_HEARTBEAT_ENDPOINT_NAME, + new BlockManagerMasterHeartbeatEndpoint(rpcEnv, isLocal, blockManagerInfo)), + conf, + isDriver) val blockTransferService = new NettyBlockTransferService(conf, securityManager, bindAddress, advertiseAddress, diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala b/core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala similarity index 73% rename from core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala rename to core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala index 5625e953c5e6..68054625bac2 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala +++ b/core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala @@ -15,16 +15,9 @@ * limitations under the License. */ -package org.apache.spark.executor +package org.apache.spark -import com.codahale.metrics.MetricRegistry - -import org.apache.spark.metrics.source.Source - -private[spark] -class ExecutorPluginSource(name: String) extends Source { - - override val metricRegistry = new MetricRegistry() - - override val sourceName = name -} +/** + * Exception thrown when a task cannot write to output file due to the file already exists. + */ +private[spark] class TaskOutputFileAlreadyExistException(error: Throwable) extends Exception(error) diff --git a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala index d8f9d1f1729b..fdfe5f5b41d0 100644 --- a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala @@ -82,12 +82,7 @@ private[spark] abstract class BaseRRunner[IN, OUT]( serverSocket.close() } - try { - newReaderIterator(dataStream, errThread) - } catch { - case e: Exception => - throw new SparkException("R computation failed with\n " + errThread.getLines(), e) - } + newReaderIterator(dataStream, errThread) } /** @@ -138,6 +133,16 @@ private[spark] abstract class BaseRRunner[IN, OUT]( * and then returns null. */ protected def read(): OUT + + protected val handleException: PartialFunction[Throwable, OUT] = { + case e: Exception => + var msg = "R unexpectedly exited." + val lines = errThread.getLines() + if (lines.trim().nonEmpty) { + msg += s"\nR worker produced errors: $lines\n" + } + throw new SparkException(msg, e) + } } /** diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 0327386b45ed..20ab6fc2f348 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -125,10 +125,7 @@ private[spark] class RRunner[IN, OUT]( eos = true null.asInstanceOf[OUT] } - } catch { - case eof: EOFException => - throw new SparkException("R worker exited unexpectedly (cranshed)", eof) - } + } catch handleException } } } 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 70864d590988..e2f3314bc859 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 @@ -18,6 +18,7 @@ package org.apache.spark.deploy.history import java.io.{File, FileNotFoundException, IOException} +import java.lang.{Long => JLong} import java.nio.file.Files import java.util.{Date, ServiceLoader} import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Future, TimeUnit} @@ -30,6 +31,7 @@ import scala.io.Source import scala.xml.Node import com.fasterxml.jackson.annotation.JsonIgnore +import com.fasterxml.jackson.databind.annotation.JsonDeserialize import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.hdfs.DistributedFileSystem @@ -1167,6 +1169,7 @@ private[history] case class LogInfo( appId: Option[String], attemptId: Option[String], fileSize: Long, + @JsonDeserialize(contentAs = classOf[JLong]) lastIndex: Option[Long], isComplete: Boolean) @@ -1174,6 +1177,7 @@ private[history] class AttemptInfoWrapper( val info: ApplicationAttemptInfo, val logPath: String, val fileSize: Long, + @JsonDeserialize(contentAs = classOf[JLong]) val lastIndex: Option[Long], val adminAcls: Option[String], val viewAcls: Option[String], 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 c7e73bcc13c5..071b79135d64 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 @@ -73,7 +73,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
  • + data-placement="top"> Executor Limit: { if (app.executorLimit == Int.MaxValue) "Unlimited" else app.executorLimit 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 b4bca1e9401e..b1837c9c0c9e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -46,6 +46,7 @@ private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, driverUrl: String, executorId: String, + bindAddress: String, hostname: String, cores: Int, userClassPath: Seq[URL], @@ -126,6 +127,7 @@ private[spark] class CoarseGrainedExecutorBackend( logInfo("Successfully registered with driver") try { executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + driver.get.send(LaunchedExecutor(executorId)) } catch { case NonFatal(e) => exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) @@ -227,6 +229,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case class Arguments( driverUrl: String, executorId: String, + bindAddress: String, hostname: String, cores: Int, appId: String, @@ -238,7 +241,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, Arguments, SparkEnv) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => new CoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.hostname, arguments.cores, arguments.userClassPath, env, + arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourcesFileOpt) } run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) @@ -259,10 +262,12 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val executorConf = new SparkConf val fetcher = RpcEnv.create( "driverPropsFetcher", + arguments.bindAddress, arguments.hostname, -1, executorConf, new SecurityManager(executorConf), + numUsableCores = 0, clientMode = true) var driver: RpcEndpointRef = null @@ -297,8 +302,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } driverConf.set(EXECUTOR_ID, arguments.executorId) - val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.hostname, - arguments.cores, cfg.ioEncryptionKey, isLocal = false) + val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.bindAddress, + arguments.hostname, arguments.cores, cfg.ioEncryptionKey, isLocal = false) env.rpcEnv.setupEndpoint("Executor", backendCreateFn(env.rpcEnv, arguments, env)) arguments.workerUrl.foreach { url => @@ -311,6 +316,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { def parseArguments(args: Array[String], classNameForEntry: String): Arguments = { var driverUrl: String = null var executorId: String = null + var bindAddress: String = null var hostname: String = null var cores: Int = 0 var resourcesFileOpt: Option[String] = None @@ -327,6 +333,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case ("--executor-id") :: value :: tail => executorId = value argv = tail + case ("--bind-address") :: value :: tail => + bindAddress = value + argv = tail case ("--hostname") :: value :: tail => hostname = value argv = tail @@ -364,7 +373,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { printUsageAndExit(classNameForEntry) } - Arguments(driverUrl, executorId, hostname, cores, appId, workerUrl, + if (bindAddress == null) { + bindAddress = hostname + } + + Arguments(driverUrl, executorId, bindAddress, hostname, cores, appId, workerUrl, userClassPath, resourcesFileOpt) } @@ -377,6 +390,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | Options are: | --driver-url | --executor-id + | --bind-address | --hostname | --cores | --resourcesFile 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 0f595d095a22..8cd98e47b8a4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -137,35 +137,6 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) - private val executorPlugins: Seq[ExecutorPlugin] = { - val pluginNames = conf.get(EXECUTOR_PLUGINS) - if (pluginNames.nonEmpty) { - logInfo(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") - - // Plugins need to load using a class loader that includes the executor's user classpath - val pluginList: Seq[ExecutorPlugin] = - Utils.withContextClassLoader(replClassLoader) { - val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) - plugins.foreach { plugin => - val pluginSource = new ExecutorPluginSource(plugin.getClass().getSimpleName()) - val pluginContext = new ExecutorPluginContext(pluginSource.metricRegistry, conf, - executorId, executorHostname, isLocal) - plugin.init(pluginContext) - logInfo("Successfully loaded plugin " + plugin.getClass().getCanonicalName()) - if (pluginSource.metricRegistry.getNames.size() > 0) { - env.metricsSystem.registerSource(pluginSource) - } - } - plugins - } - - logInfo("Finished initializing plugins") - pluginList - } else { - Nil - } - } - // Plugins need to load using a class loader that includes the executor's user classpath private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { PluginContainer(env) @@ -295,14 +266,6 @@ private[spark] class Executor( // Notify plugins that executor is shutting down so they can terminate cleanly Utils.withContextClassLoader(replClassLoader) { - executorPlugins.foreach { plugin => - try { - plugin.shutdown() - } catch { - case e: Exception => - logWarning("Plugin " + plugin.getClass().getCanonicalName() + " shutdown failed", e) - } - } plugins.foreach(_.shutdown()) } if (!isLocal) { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 00acb1ff115f..8e8e36dbda94 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -606,6 +606,23 @@ package object config { .intConf .createWithDefault(128) + private[spark] val LISTENER_BUS_LOG_SLOW_EVENT_ENABLED = + ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent.enabled") + .internal() + .doc("When enabled, log the event that takes too much time to process. This helps us " + + "discover the event types that cause performance bottlenecks. The time threshold is " + + "controlled by spark.scheduler.listenerbus.logSlowEvent.threshold.") + .booleanConf + .createWithDefault(true) + + private[spark] val LISTENER_BUS_LOG_SLOW_EVENT_TIME_THRESHOLD = + ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent.threshold") + .internal() + .doc("The time threshold of whether a event is considered to be taking too much time to " + + "process. Log the event if spark.scheduler.listenerbus.logSlowEvent.enabled is true.") + .timeConf(TimeUnit.NANOSECONDS) + .createWithDefaultString("1s") + // This property sets the root namespace for metrics reporting private[spark] val METRICS_NAMESPACE = ConfigBuilder("spark.metrics.namespace") .stringConf @@ -1176,16 +1193,6 @@ package object config { .toSequence .createWithDefault(Nil) - private[spark] val EXECUTOR_PLUGINS = - ConfigBuilder("spark.executor.plugins") - .doc("Comma-separated list of class names for \"plugins\" implementing " + - "org.apache.spark.ExecutorPlugin. Plugins have the same privileges as any task " + - "in a Spark executor. They can also interfere with task execution and fail in " + - "unexpected ways. So be sure to only use this for trusted plugins.") - .stringConf - .toSequence - .createWithDefault(Nil) - private[spark] val CLEANER_PERIODIC_GC_INTERVAL = ConfigBuilder("spark.cleaner.periodicGC.interval") .timeConf(TimeUnit.SECONDS) 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 c3e1cd8b23f1..fe3a48440991 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -267,7 +267,7 @@ private[spark] class DAGScheduler( executorUpdates: mutable.Map[(Int, Int), ExecutorMetrics]): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, executorUpdates)) - blockManagerMaster.driverEndpoint.askSync[Boolean]( + blockManagerMaster.driverHeartbeatEndPoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(10.minutes, "BlockManagerHeartbeat")) } 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 9defbefabb86..5c0bc497dd1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -799,6 +799,15 @@ private[spark] class TaskSetManager( info.id, taskSet.id, tid, ef.description)) return } + if (ef.className == classOf[TaskOutputFileAlreadyExistException].getName) { + // If we can not write to output file in the task, there's no point in trying to + // re-execute it. + logError("Task %s in stage %s (TID %d) can not write to output file: %s; not retrying" + .format(info.id, taskSet.id, tid, ef.description)) + abort("Task %s in stage %s (TID %d) can not write to output file: %s".format( + info.id, taskSet.id, tid, ef.description)) + return + } val key = ef.description val now = clock.getTimeMillis() val (printFull, dupCount) = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index a90fff02ac73..9ce23683245e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -69,6 +69,8 @@ private[spark] object CoarseGrainedClusterMessages { resources: Map[String, ResourceInformation]) extends CoarseGrainedClusterMessage + case class LaunchedExecutor(executorId: String) extends CoarseGrainedClusterMessage + case class StatusUpdate( executorId: String, taskId: Long, 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 ea045e6280e4..7d9c1c6f96f6 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 @@ -194,6 +194,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // automatically, so try to tell the executor to stop itself. See SPARK-13519. executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) + + case LaunchedExecutor(executorId) => + executorDataMap.get(executorId).foreach { data => + data.freeCores = data.totalCores + } + makeOffers(executorId) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -230,7 +236,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp taskResourceNumParts.getOrElse(v.name, 1))) } val data = new ExecutorData(executorRef, executorAddress, hostname, - cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, + 0, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resourcesInfo) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -249,7 +255,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(true) listenerBus.post( SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) - makeOffers() } case StopDriver => 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 525304fe3c9d..9678c917882c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -30,6 +30,7 @@ import org.apache.spark.util.{RpcUtils, ThreadUtils} private[spark] class BlockManagerMaster( var driverEndpoint: RpcEndpointRef, + var driverHeartbeatEndPoint: RpcEndpointRef, conf: SparkConf, isDriver: Boolean) extends Logging { @@ -230,6 +231,11 @@ class BlockManagerMaster( if (driverEndpoint != null && isDriver) { tell(StopBlockManagerMaster) driverEndpoint = null + if (driverHeartbeatEndPoint.askSync[Boolean](StopBlockManagerMaster)) { + driverHeartbeatEndPoint = null + } else { + logWarning("Failed to stop BlockManagerMasterHeartbeatEndpoint") + } logInfo("BlockManagerMaster stopped") } } @@ -245,4 +251,5 @@ class BlockManagerMaster( private[spark] object BlockManagerMaster { val DRIVER_ENDPOINT_NAME = "BlockManagerMaster" + val DRIVER_HEARTBEAT_ENDPOINT_NAME = "BlockManagerMasterHeartbeat" } 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 02d0e1a83490..7e2027701c33 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -36,7 +36,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} /** - * BlockManagerMasterEndpoint is an [[ThreadSafeRpcEndpoint]] on the master node to track statuses + * BlockManagerMasterEndpoint is an [[IsolatedRpcEndpoint]] on the master node to track statuses * of all slaves' block managers. */ private[spark] @@ -45,12 +45,10 @@ class BlockManagerMasterEndpoint( val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus, - externalBlockStoreClient: Option[ExternalBlockStoreClient]) + externalBlockStoreClient: Option[ExternalBlockStoreClient], + blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) extends IsolatedRpcEndpoint with Logging { - // Mapping from block manager id to the block manager's information. - private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] - // Mapping from external shuffle service block manager id to the block statuses. private val blockStatusByShuffleService = new mutable.HashMap[BlockManagerId, JHashMap[BlockId, BlockStatus]] @@ -144,9 +142,6 @@ class BlockManagerMasterEndpoint( case StopBlockManagerMaster => context.reply(true) stop() - - case BlockManagerHeartbeat(blockManagerId) => - context.reply(heartbeatReceived(blockManagerId)) } private def removeRdd(rddId: Int): Future[Seq[Int]] = { @@ -290,19 +285,6 @@ class BlockManagerMasterEndpoint( blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) } - /** - * Return true if the driver knows about the given block manager. Otherwise, return false, - * indicating that the block manager should re-register. - */ - private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { - if (!blockManagerInfo.contains(blockManagerId)) { - blockManagerId.isDriver && !isLocal - } else { - blockManagerInfo(blockManagerId).updateLastSeenMs() - true - } - } - // Remove a block from the slaves that have it. This can only be used to remove // blocks that the master knows about. private def removeBlockFromWorkers(blockId: BlockId): Unit = { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala new file mode 100644 index 000000000000..b06002123d80 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable + +import org.apache.spark.internal.Logging +import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} +import org.apache.spark.storage.BlockManagerMessages.{BlockManagerHeartbeat, StopBlockManagerMaster} + +/** + * Separate heartbeat out of BlockManagerMasterEndpoint due to performance consideration. + */ +private[spark] class BlockManagerMasterHeartbeatEndpoint( + override val rpcEnv: RpcEnv, + isLocal: Boolean, + blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) + extends ThreadSafeRpcEndpoint with Logging { + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case BlockManagerHeartbeat(blockManagerId) => + context.reply(heartbeatReceived(blockManagerId)) + + case StopBlockManagerMaster => + stop() + context.reply(true) + + case _ => // do nothing for unexpected events + } + + /** + * Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { + if (!blockManagerInfo.contains(blockManagerId)) { + blockManagerId.isDriver && !isLocal + } else { + blockManagerInfo(blockManagerId).updateLastSeenMs() + true + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 6dbe63b564e6..143303df0d10 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -227,7 +227,7 @@ private[spark] object UIUtils extends Logging { {tab.name}
  • } - val helpButton: Seq[Node] = helpText.map(tooltip(_, "bottom")).getOrElse(Seq.empty) + val helpButton: Seq[Node] = helpText.map(tooltip(_, "top")).getOrElse(Seq.empty) @@ -428,7 +428,7 @@ private[spark] object UIUtils extends Logging { class="expand-dag-viz" onclick={s"toggleDagViz($forJob);"}> + data-placement="top"> DAG Visualization diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index a13037b5e24d..77564f48015f 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -89,7 +89,12 @@ private[ui] class ExecutorThreadDumpPage( Thread ID Thread Name Thread State - Thread Locks + + + Thread Locks + + {dumpRows} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 11fcbf1c29c0..91e9caeae05d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -123,7 +123,7 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We | 'group': 'executors', | 'start': new Date(${e.addTime.getTime()}), | 'content': '
    Executor ${e.id} added
    ' @@ -139,7 +139,7 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We | 'group': 'executors', | 'start': new Date(${removeTime.getTime()}), | 'content': '
    - + Event Timeline ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index f7aca507d6f9..12f1aa25e8d2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -104,7 +104,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP | 'group': 'executors', | 'start': new Date(${e.addTime.getTime()}), | 'content': '
    Executor ${e.id} added
    ' @@ -120,7 +120,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP | 'group': 'executors', | 'start': new Date(${removeTime.getTime()}), | 'content': '
    - + Event Timeline ++ 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 e24b2f2ec36d..09a215ba9f03 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 @@ -179,18 +179,18 @@ private[ui] class StagePagedTable( // stageHeadersAndCssClasses has three parts: header title, tooltip information, and sortable. // The tooltip information could be None, which indicates it does not have a tooltip. // Otherwise, it has two parts: tooltip text, and position (true for left, false for default). - val stageHeadersAndCssClasses: Seq[(String, Option[(String, Boolean)], Boolean)] = - Seq(("Stage Id", None, true)) ++ - {if (isFairScheduler) {Seq(("Pool Name", None, true))} else Seq.empty} ++ + val stageHeadersAndCssClasses: Seq[(String, String, Boolean)] = + Seq(("Stage Id", null, true)) ++ + {if (isFairScheduler) {Seq(("Pool Name", null, true))} else Seq.empty} ++ Seq( - ("Description", None, true), ("Submitted", None, true), ("Duration", None, true), - ("Tasks: Succeeded/Total", None, false), - ("Input", Some((ToolTips.INPUT, false)), true), - ("Output", Some((ToolTips.OUTPUT, false)), true), - ("Shuffle Read", Some((ToolTips.SHUFFLE_READ, false)), true), - ("Shuffle Write", Some((ToolTips.SHUFFLE_WRITE, true)), true) + ("Description", null, true), ("Submitted", null, true), ("Duration", null, true), + ("Tasks: Succeeded/Total", null, false), + ("Input", ToolTips.INPUT, true), + ("Output", ToolTips.OUTPUT, true), + ("Shuffle Read", ToolTips.SHUFFLE_READ, true), + ("Shuffle Write", ToolTips.SHUFFLE_WRITE, true) ) ++ - {if (isFailedStage) {Seq(("Failure Reason", None, false))} else Seq.empty} + {if (isFailedStage) {Seq(("Failure Reason", null, false))} else Seq.empty} if (!stageHeadersAndCssClasses.filter(_._3).map(_._1).contains(sortColumn)) { throw new IllegalArgumentException(s"Unknown column: $sortColumn") @@ -198,22 +198,13 @@ private[ui] class StagePagedTable( val headerRow: Seq[Node] = { stageHeadersAndCssClasses.map { case (header, tooltip, sortable) => - val headerSpan = tooltip.map { case (title, left) => - if (left) { - /* Place the shuffle write tooltip on the left (rather than the default position - of on top) because the shuffle write column is the last column on the right side and - the tooltip is wider than the column, so it doesn't fit on top. */ - + val headerSpan = if (null != tooltip && !tooltip.isEmpty) { + {header} - } else { - - {header} - - } - }.getOrElse( + } else { {header} - ) + } if (header == sortColumn) { val headerLink = Unparsed( 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 2e517707ff77..51cd7d1284ff 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -25,7 +25,8 @@ import scala.util.control.NonFatal import com.codahale.metrics.Timer -import org.apache.spark.internal.Logging +import org.apache.spark.SparkEnv +import org.apache.spark.internal.{config, Logging} /** * An event bus which posts events to its listeners. @@ -37,6 +38,20 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { // Marked `private[spark]` for access in tests. private[spark] def listeners = listenersPlusTimers.asScala.map(_._1).asJava + private lazy val env = SparkEnv.get + + private lazy val logSlowEventEnabled = if (env != null) { + env.conf.get(config.LISTENER_BUS_LOG_SLOW_EVENT_ENABLED) + } else { + false + } + + private lazy val logSlowEventThreshold = if (env != null) { + env.conf.get(config.LISTENER_BUS_LOG_SLOW_EVENT_TIME_THRESHOLD) + } else { + Long.MaxValue + } + /** * Returns a CodaHale metrics Timer for measuring the listener's event processing time. * This method is intended to be overridden by subclasses. @@ -95,6 +110,7 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { } else { null } + lazy val listenerName = Utils.getFormattedClassName(listener) try { doPostEvent(listener, event) if (Thread.interrupted()) { @@ -104,14 +120,17 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { } } catch { case ie: InterruptedException => - logError(s"Interrupted while posting to ${Utils.getFormattedClassName(listener)}. " + - s"Removing that listener.", ie) + logError(s"Interrupted while posting to ${listenerName}. Removing that listener.", ie) removeListenerOnError(listener) case NonFatal(e) if !isIgnorableException(e) => - logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e) + logError(s"Listener ${listenerName} threw an exception", e) } finally { if (maybeTimerContext != null) { - maybeTimerContext.stop() + val elapsed = maybeTimerContext.stop() + if (logSlowEventEnabled && elapsed > logSlowEventThreshold) { + logInfo(s"Process of event ${event} by listener ${listenerName} took " + + s"${elapsed / 1000000000d}s.") + } } } } diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java deleted file mode 100644 index ade13f02bde7..000000000000 --- a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java +++ /dev/null @@ -1,179 +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; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.MetricRegistry; -import org.apache.spark.api.java.JavaSparkContext; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.*; - -public class ExecutorPluginSuite { - private static final String EXECUTOR_PLUGIN_CONF_NAME = "spark.executor.plugins"; - private static final String testBadPluginName = TestBadShutdownPlugin.class.getName(); - private static final String testPluginName = TestExecutorPlugin.class.getName(); - private static final String testSecondPluginName = TestSecondPlugin.class.getName(); - private static final String testMetricsPluginName = TestMetricsPlugin.class.getName(); - - // Static value modified by testing plugins to ensure plugins loaded correctly. - public static int numSuccessfulPlugins = 0; - - // Static value modified by testing plugins to verify plugins shut down properly. - public static int numSuccessfulTerminations = 0; - - // Static values modified by testing plugins to ensure metrics have been registered correctly. - public static MetricRegistry testMetricRegistry; - public static String gaugeName; - - private JavaSparkContext sc; - - @Before - public void setUp() { - sc = null; - numSuccessfulPlugins = 0; - numSuccessfulTerminations = 0; - } - - @After - public void tearDown() { - if (sc != null) { - sc.stop(); - sc = null; - } - } - - private SparkConf initializeSparkConf(String pluginNames) { - return new SparkConf() - .setMaster("local") - .setAppName("test") - .set(EXECUTOR_PLUGIN_CONF_NAME, pluginNames); - } - - @Test - public void testPluginClassDoesNotExist() { - SparkConf conf = initializeSparkConf("nonexistent.plugin"); - try { - sc = new JavaSparkContext(conf); - fail("No exception thrown for nonexistent plugin"); - } catch (Exception e) { - // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown - assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException")); - } - } - - @Test - public void testAddPlugin() throws InterruptedException { - // Load the sample TestExecutorPlugin, which will change the value of numSuccessfulPlugins - SparkConf conf = initializeSparkConf(testPluginName); - sc = new JavaSparkContext(conf); - assertEquals(1, numSuccessfulPlugins); - sc.stop(); - sc = null; - assertEquals(1, numSuccessfulTerminations); - } - - @Test - public void testAddMultiplePlugins() throws InterruptedException { - // Load two plugins and verify they both execute. - SparkConf conf = initializeSparkConf(testPluginName + "," + testSecondPluginName); - sc = new JavaSparkContext(conf); - assertEquals(2, numSuccessfulPlugins); - sc.stop(); - sc = null; - assertEquals(2, numSuccessfulTerminations); - } - - @Test - public void testPluginShutdownWithException() { - // Verify an exception in one plugin shutdown does not affect the others - String pluginNames = testPluginName + "," + testBadPluginName + "," + testPluginName; - SparkConf conf = initializeSparkConf(pluginNames); - sc = new JavaSparkContext(conf); - assertEquals(3, numSuccessfulPlugins); - sc.stop(); - sc = null; - assertEquals(2, numSuccessfulTerminations); - } - - @Test - public void testPluginMetrics() { - // Verify that a custom metric is registered with the Spark metrics system - gaugeName = "test42"; - SparkConf conf = initializeSparkConf(testMetricsPluginName); - sc = new JavaSparkContext(conf); - assertEquals(1, numSuccessfulPlugins); - assertEquals(gaugeName, testMetricRegistry.getGauges().firstKey()); - sc.stop(); - sc = null; - assertEquals(1, numSuccessfulTerminations); - } - - public static class TestExecutorPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext pluginContext) { - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - ExecutorPluginSuite.numSuccessfulTerminations++; - } - } - - public static class TestSecondPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext pluginContext) { - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - ExecutorPluginSuite.numSuccessfulTerminations++; - } - } - - public static class TestBadShutdownPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext pluginContext) { - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - throw new RuntimeException("This plugin will fail to cleanly shut down"); - } - } - - public static class TestMetricsPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext myContext) { - MetricRegistry metricRegistry = myContext.metricRegistry; - // Registers a dummy metrics gauge for testing - String gaugeName = ExecutorPluginSuite.gaugeName; - metricRegistry.register(MetricRegistry.name(gaugeName), new Gauge() { - @Override - public Integer getValue() { - return 42; - } - }); - ExecutorPluginSuite.testMetricRegistry = metricRegistry; - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - ExecutorPluginSuite.numSuccessfulTerminations++; - } - } -} diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 17758783d259..dd790b8dbb85 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.config import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster._ -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RegisterExecutorFailed} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{LaunchedExecutor, RegisterExecutor, RegisterExecutorFailed} /** * End-to-end tests for dynamic allocation in standalone mode. @@ -634,6 +634,7 @@ class StandaloneDynamicAllocationSuite Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) + backend.driverEndpoint.send(LaunchedExecutor(id)) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 281e6935de37..ed195dd44e91 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1283,6 +1283,56 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { assert(deserializedOldObj.isComplete === false) } + test("SPARK-29755 LogInfo should be serialized/deserialized by jackson properly") { + def assertSerDe(serializer: KVStoreScalaSerializer, info: LogInfo): Unit = { + val infoAfterSerDe = serializer.deserialize(serializer.serialize(info), classOf[LogInfo]) + assert(infoAfterSerDe === info) + assertOptionAfterSerde(infoAfterSerDe.lastIndex, info.lastIndex) + } + + val serializer = new KVStoreScalaSerializer() + val logInfoWithIndexAsNone = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), + Some("attemptId"), 100, None, false) + assertSerDe(serializer, logInfoWithIndexAsNone) + + val logInfoWithIndex = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), + Some("attemptId"), 100, Some(3), false) + assertSerDe(serializer, logInfoWithIndex) + } + + test("SPARK-29755 AttemptInfoWrapper should be serialized/deserialized by jackson properly") { + def assertSerDe(serializer: KVStoreScalaSerializer, attempt: AttemptInfoWrapper): Unit = { + val attemptAfterSerDe = serializer.deserialize(serializer.serialize(attempt), + classOf[AttemptInfoWrapper]) + assert(attemptAfterSerDe.info === attempt.info) + // skip comparing some fields, as they've not triggered SPARK-29755 + assertOptionAfterSerde(attemptAfterSerDe.lastIndex, attempt.lastIndex) + } + + val serializer = new KVStoreScalaSerializer() + val appInfo = new ApplicationAttemptInfo(None, new Date(1), new Date(1), new Date(1), + 10, "spark", false, "dummy") + val attemptInfoWithIndexAsNone = new AttemptInfoWrapper(appInfo, "dummyPath", 10, None, + None, None, None, None) + assertSerDe(serializer, attemptInfoWithIndexAsNone) + + val attemptInfoWithIndex = new AttemptInfoWrapper(appInfo, "dummyPath", 10, Some(1), + None, None, None, None) + assertSerDe(serializer, attemptInfoWithIndex) + } + + private def assertOptionAfterSerde(opt: Option[Long], expected: Option[Long]): Unit = { + if (expected.isEmpty) { + assert(opt.isEmpty) + } else { + // The issue happens only when the value in Option is being unboxed. Here we ensure unboxing + // to Long succeeds: even though IDE suggests `.toLong` is redundant, direct comparison + // doesn't trigger unboxing and passes even without SPARK-29755, so don't remove + // `.toLong` below. Please refer SPARK-29755 for more details. + assert(opt.get.toLong === expected.get.toLong) + } + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index e40cf0d66c19..7e96039ca924 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -54,7 +54,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => val testResourceArgs: JObject = ("" -> "") @@ -76,7 +76,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => val ra = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) @@ -101,7 +101,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => @@ -129,7 +129,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) // not enough gpu's on the executor @@ -168,7 +168,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) // executor resources < required @@ -200,7 +200,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) val parsedResources = backend.parseOrFindResources(None) @@ -226,7 +226,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) val ja = Extraction.decompose(Seq(gpuArgs)) @@ -254,7 +254,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val rpcEnv = RpcEnv.create("1", "localhost", 0, conf, securityMgr) val env = createMockEnv(conf, serializer, Some(rpcEnv)) backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", - "host1", 4, Seq.empty[URL], env, None) + "host1", "host1", 4, Seq.empty[URL], env, None) assert(backend.taskResources.isEmpty) val taskId = 1000000 @@ -289,6 +289,31 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite } } + test("SPARK-24203 when bindAddress is not set, it defaults to hostname") { + val args1 = Array( + "--driver-url", "driverurl", + "--executor-id", "1", + "--hostname", "host1", + "--cores", "1", + "--app-id", "app1") + + val arg = CoarseGrainedExecutorBackend.parseArguments(args1, "") + assert(arg.bindAddress == "host1") + } + + test("SPARK-24203 when bindAddress is different, it does not default to hostname") { + val args1 = Array( + "--driver-url", "driverurl", + "--executor-id", "1", + "--hostname", "host1", + "--bind-address", "bindaddress1", + "--cores", "1", + "--app-id", "app1") + + val arg = CoarseGrainedExecutorBackend.parseArguments(args1, "") + assert(arg.bindAddress == "bindaddress1") + } + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer, rpcEnv: Option[RpcEnv] = None): SparkEnv = { val mockEnv = mock[SparkEnv] diff --git a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala index 24fa01736365..b432253ad80d 100644 --- a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala @@ -31,7 +31,7 @@ import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} -import org.apache.spark.{ExecutorPlugin => _, _} +import org.apache.spark._ import org.apache.spark.api.plugin._ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index c27d50ab66e6..1e3b59f7e97d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -245,7 +245,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null, conf, true) { + val blockManagerMaster = new BlockManagerMaster(null, null, conf, true) { override def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { blockIds.map { _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). 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 441ec6ab6e18..89df5de97c44 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,6 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.fs.FileAlreadyExistsException import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -1775,4 +1776,23 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.checkSpeculatableTasks(0)) assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) } + + test("TaskOutputFileAlreadyExistException lead to task set abortion") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = FakeTask.createTaskSet(1) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + assert(sched.taskSetsFailed.isEmpty) + + val offerResult = manager.resourceOffer("exec1", "host1", ANY) + assert(offerResult.isDefined, + "Expect resource offer on iteration 0 to return a task") + assert(offerResult.get.index === 0) + val reason = new ExceptionFailure( + new TaskOutputFileAlreadyExistException( + new FileAlreadyExistsException("file already exists")), + Seq.empty[AccumulableInfo]) + manager.handleFailedTask(offerResult.get.taskId, TaskState.FAILED, reason) + assert(sched.taskSetsFailed.contains(taskSet.id)) + } } 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 d8f42ea9557d..59ace850d0bd 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.util.Locale +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.implicitConversions @@ -97,9 +98,12 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite conf.set(STORAGE_CACHED_PEERS_TTL, 10) sc = new SparkContext("local", "test", conf) + val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None)), conf, true) + new LiveListenerBus(conf), None, blockManagerInfo)), + rpcEnv.setupEndpoint("blockmanagerHeartbeat", + new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) allStores.clear() } 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 484d246959ec..8595f73fe5dd 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,6 +21,7 @@ import java.io.File import java.nio.ByteBuffer import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import scala.concurrent.duration._ @@ -142,10 +143,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // need to create a SparkContext is to initialize LiveListenerBus. sc = mock(classOf[SparkContext]) when(sc.conf).thenReturn(conf) - master = spy(new BlockManagerMaster( - rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None)), conf, true)) + + val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() + master = spy(new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(conf), None, blockManagerInfo)), + rpcEnv.setupEndpoint("blockmanagerHeartbeat", + new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true)) val initialize = PrivateMethod[Unit](Symbol("initialize")) SizeEstimator invokePrivate initialize() @@ -468,7 +472,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - val reregister = !master.driverEndpoint.askSync[Boolean]( + val reregister = !master.driverHeartbeatEndPoint.askSync[Boolean]( BlockManagerHeartbeat(store.blockManagerId)) assert(reregister) } diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 4bfecedbf040..fde75495b442 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -34,7 +34,7 @@ ENV DEBCONF_NONINTERACTIVE_SEEN true ARG APT_INSTALL="apt-get install --no-install-recommends -y" ARG BASE_PIP_PKGS="setuptools wheel virtualenv" -ARG PIP_PKGS="pyopenssl pypandoc numpy pygments sphinx" +ARG PIP_PKGS="pyopenssl pypandoc numpy sphinx" # Install extra needed repos and refresh. # - CRAN repo @@ -80,7 +80,7 @@ RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ $APT_INSTALL ruby2.3 ruby2.3-dev mkdocs && \ gem install jekyll --no-rdoc --no-ri -v 3.8.6 && \ gem install jekyll-redirect-from && \ - gem install pygments.rb + gem install rogue WORKDIR /opt/spark-rm/output diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index e6d29d04acbf..54608d203133 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -17,9 +17,9 @@ apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -arrow-format-0.12.0.jar -arrow-memory-0.12.0.jar -arrow-vector-0.12.0.jar +arrow-format-0.15.1.jar +arrow-memory-0.15.1.jar +arrow-vector-0.15.1.jar audience-annotations-0.5.0.jar automaton-1.11-8.jar avro-1.8.2.jar @@ -83,7 +83,6 @@ hadoop-yarn-server-web-proxy-2.7.4.jar hk2-api-2.5.0.jar hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar -hppc-0.7.2.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.10.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 8f1e7fe125b9..917fde61fad1 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -12,9 +12,9 @@ antlr4-runtime-4.7.1.jar aopalliance-1.0.jar aopalliance-repackaged-2.5.0.jar arpack_combined_all-0.1.jar -arrow-format-0.12.0.jar -arrow-memory-0.12.0.jar -arrow-vector-0.12.0.jar +arrow-format-0.15.1.jar +arrow-memory-0.15.1.jar +arrow-vector-0.15.1.jar audience-annotations-0.5.0.jar automaton-1.11-8.jar avro-1.8.2.jar @@ -96,7 +96,6 @@ hive-vector-code-gen-2.3.6.jar hk2-api-2.5.0.jar hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar -hppc-0.7.2.jar htrace-core4-4.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.10.jar diff --git a/dev/lint-r b/dev/lint-r index bfda0bca15eb..b08f5efecd5d 100755 --- a/dev/lint-r +++ b/dev/lint-r @@ -17,6 +17,9 @@ # limitations under the License. # +set -o pipefail +set -e + SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" @@ -24,7 +27,7 @@ LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" if ! type "Rscript" > /dev/null; then echo "ERROR: You should install R" - exit + exit 1 fi `which Rscript` --vanilla "$SPARK_ROOT_DIR/dev/lint-r.R" "$SPARK_ROOT_DIR" | tee "$LINT_R_REPORT_FILE_NAME" diff --git a/dev/lint-r.R b/dev/lint-r.R index a4261d266bbc..7e165319e316 100644 --- a/dev/lint-r.R +++ b/dev/lint-r.R @@ -27,7 +27,7 @@ if (! library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE)) { # Installs lintr from Github in a local directory. # NOTE: The CRAN's version is too old to adapt to our rules. if ("lintr" %in% row.names(installed.packages()) == FALSE) { - devtools::install_github("jimhester/lintr@5431140") + devtools::install_github("jimhester/lintr@v2.0.0") } library(lintr) diff --git a/dev/pip-sanity-check.py b/dev/pip-sanity-check.py index 4171f28684d5..e9f10233b12b 100644 --- a/dev/pip-sanity-check.py +++ b/dev/pip-sanity-check.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark.sql import SparkSession from pyspark.mllib.linalg import * import sys diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 60cf4d820941..1294a9096fb9 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -39,21 +39,16 @@ PYTHON_EXECS=() # Some systems don't have pip or virtualenv - in those cases our tests won't work. if hash virtualenv 2>/dev/null && [ ! -n "$USE_CONDA" ]; then echo "virtualenv installed - using. Note if this is a conda virtual env you may wish to set USE_CONDA" - # Figure out which Python execs we should test pip installation with - if hash python2 2>/dev/null; then - # We do this since we are testing with virtualenv and the default virtual env python - # is in /usr/bin/python - PYTHON_EXECS+=('python2') - elif hash python 2>/dev/null; then - # If python2 isn't installed fallback to python if available - PYTHON_EXECS+=('python') - fi + # test only against python3 if hash python3 2>/dev/null; then - PYTHON_EXECS+=('python3') + PYTHON_EXECS=('python3') + else + echo "Python3 not installed on system, skipping pip installability tests" + exit 0 fi elif hash conda 2>/dev/null; then echo "Using conda virtual environments" - PYTHON_EXECS=('3.5') + PYTHON_EXECS=('3.6') USE_CONDA=1 else echo "Missing virtualenv & conda, skipping pip installability tests" @@ -97,7 +92,7 @@ for python in "${PYTHON_EXECS[@]}"; do cd "$FWDIR"/python # Delete the egg info file if it exists, this can cache the setup file. rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" - python setup.py sdist + python3 setup.py sdist echo "Installing dist into virtual env" @@ -117,9 +112,9 @@ for python in "${PYTHON_EXECS[@]}"; do echo "Run basic sanity check on pip installed version with spark-submit" spark-submit "$FWDIR"/dev/pip-sanity-check.py echo "Run basic sanity check with import based" - python "$FWDIR"/dev/pip-sanity-check.py + python3 "$FWDIR"/dev/pip-sanity-check.py echo "Run the tests for context.py" - python "$FWDIR"/python/pyspark/context.py + python3 "$FWDIR"/python/pyspark/context.py cd "$FWDIR" diff --git a/dev/run-tests b/dev/run-tests index 9cf93d000d0e..143d78ec6373 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -20,10 +20,10 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -PYTHON_VERSION_CHECK=$(python -c 'import sys; print(sys.version_info < (2, 7, 0))') +PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 6, 0))') if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then - echo "Python versions prior to 2.7 are not supported." + echo "Python versions prior to 3.6 are not supported." exit -1 fi -exec python -u ./dev/run-tests.py "$@" +exec python3 -u ./dev/run-tests.py "$@" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 5bc03e41d1f2..c3adc696a512 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -25,10 +25,12 @@ FWDIR="$( cd "$( dirname "$0" )/.." && pwd )" cd "$FWDIR" -PYTHON_VERSION_CHECK=$(python -c 'import sys; print(sys.version_info < (2, 7, 0))') +export PATH=/home/anaconda/envs/py36/bin:$PATH + +PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 6, 0))') if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then - echo "Python versions prior to 2.7 are not supported." + echo "Python versions prior to 3.6 are not supported." exit -1 fi -exec python -u ./dev/run-tests-jenkins.py "$@" +exec python3 -u ./dev/run-tests-jenkins.py "$@" diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index e9b0b327603b..5429aeba8ea1 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -17,7 +17,6 @@ # limitations under the License. # -from __future__ import print_function import os import sys import json diff --git a/dev/run-tests.py b/dev/run-tests.py index ea515708124d..fc8b7251a85f 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -17,7 +17,6 @@ # limitations under the License. # -from __future__ import print_function import itertools from argparse import ArgumentParser import os @@ -44,15 +43,20 @@ def determine_modules_for_files(filenames): """ Given a list of filenames, return the set of modules that contain those files. If a file is not associated with a more specific submodule, then this method will consider that - file to belong to the 'root' module. + file to belong to the 'root' module. GitHub Action and Appveyor files are ignored. >>> sorted(x.name for x in determine_modules_for_files(["python/pyspark/a.py", "sql/core/foo"])) ['pyspark-core', 'sql'] >>> [x.name for x in determine_modules_for_files(["file_not_matched_by_any_subproject"])] ['root'] + >>> [x.name for x in determine_modules_for_files( \ + [".github/workflows/master.yml", "appveyor.yml"])] + [] """ changed_modules = set() for filename in filenames: + if filename in (".github/workflows/master.yml", "appveyor.yml"): + continue matched_at_least_one_module = False for module in modules.all_modules: if module.contains_file(filename): @@ -265,7 +269,7 @@ def exec_sbt(sbt_args=()): echo_proc.wait() for line in iter(sbt_proc.stdout.readline, b''): if not sbt_output_filter.match(line): - print(line, end='') + print(line.decode('utf-8'), end='') retcode = sbt_proc.wait() if retcode != 0: diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index ec6ea86269f5..d9cb8aa45c8d 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -15,14 +15,12 @@ # limitations under the License. # -from __future__ import print_function import os import shutil import subprocess import sys subprocess_check_output = subprocess.check_output -subprocess_check_call = subprocess.check_call def exit_from_command_with_retcode(cmd, retcode): @@ -55,9 +53,9 @@ def run_cmd(cmd, return_output=False): cmd = cmd.split() try: if return_output: - return subprocess_check_output(cmd).decode(sys.getdefaultencoding()) + return subprocess_check_output(cmd).decode('utf-8') else: - return subprocess_check_call(cmd) + return subprocess.run(cmd, universal_newlines=True, check=True) except subprocess.CalledProcessError as e: exit_from_command_with_retcode(e.cmd, e.returncode) diff --git a/docs/README.md b/docs/README.md index da531321aa5d..0bb1ada7167d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -36,8 +36,7 @@ You need to have [Ruby](https://www.ruby-lang.org/en/documentation/installation/ installed. Also install the following libraries: ```sh -$ sudo gem install jekyll jekyll-redirect-from pygments.rb -$ sudo pip install Pygments +$ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' diff --git a/docs/_config.yml b/docs/_config.yml index 57b8d716ee55..a88862013920 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,4 +1,4 @@ -highlighter: pygments +highlighter: rouge markdown: kramdown gems: - jekyll-redirect-from diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb index 1e91f12518e0..8a3af36e9230 100644 --- a/docs/_plugins/include_example.rb +++ b/docs/_plugins/include_example.rb @@ -16,7 +16,7 @@ # require 'liquid' -require 'pygments' +require 'rouge' module Jekyll class IncludeExampleTag < Liquid::Tag @@ -56,7 +56,9 @@ def render(context) end code = select_lines(code) - rendered_code = Pygments.highlight(code, :lexer => @lang) + formatter = Rouge::Formatters::HTML.new + lexer = Rouge::Lexer.find(@lang) + rendered_code = formatter.format(lexer.lex(code)) hint = "
    Find full example code at " \ "\"examples/src/main/#{snippet_file}\" in the Spark repo.
    " diff --git a/docs/configuration.md b/docs/configuration.md index 97ea1fb4ba04..0c7cc6022eb0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1857,6 +1857,51 @@ Apart from these, the following properties are also available, and may be useful driver using more memory. + + spark.scheduler.listenerbus.eventqueue.shared.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for shared event queue in Spark listener bus, which hold events for external listener(s) + that register to the listener bus. Consider increasing value, if the listener events corresponding + to shared queue are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.appStatus.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for appStatus event queue, which hold events for internal application status listeners. + Consider increasing value, if the listener events corresponding to appStatus queue are dropped. + Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.executorManagement.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for executorManagement event queue in Spark listener bus, which hold events for internal + executor management listeners. Consider increasing value if the listener events corresponding to + executorManagement queue are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.eventLog.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for eventLog queue in Spark listener bus, which hold events for Event logging listeners + that write events to eventLogs. Consider increasing value if the listener events corresponding to eventLog queue + are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.streams.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for streams queue in Spark listener bus, which hold events for internal streaming listener. + Consider increasing value if the listener events corresponding to streams queue are dropped. Increasing + this value may result in the driver using more memory. + + spark.scheduler.blacklist.unschedulableTaskSetTimeout 120s diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 2d4d91dab075..17d071d0779b 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -9,9 +9,9 @@ license: | 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. @@ -24,10 +24,15 @@ license: | ## Upgrading from Core 2.4 to 3.0 -- In Spark 3.0, deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. +- The `org.apache.spark.ExecutorPlugin` interface and related configuration has been replaced with + `org.apache.spark.plugin.SparkPlugin`, which adds new functionality. Plugins using the old + interface need to be modified to extend the new interfaces. Check the + [Monitoring](monitoring.html) guide for more details. + +- Deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. -- In Spark 3.0, deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. +- Deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. -- In Spark 3.0, deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. +- Deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. -- In Spark 3.0, event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. \ No newline at end of file +- Event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. diff --git a/docs/css/pygments-default.css b/docs/css/pygments-default.css index a4d583b36660..0d62ad1a018a 100644 --- a/docs/css/pygments-default.css +++ b/docs/css/pygments-default.css @@ -11,6 +11,10 @@ Also, I was thrown off for a while at first when I was using markdown code block inside my {% highlight scala %} ... {% endhighlight %} tags (I was using 4 spaces for this), when it turns out that pygments will insert the code (or pre?) tags for you. + +Note that due to Python 3 compatibility in the project, now we use +Rouge which claims Pygments compatibility, instead of pygments.rb which +does not support Python 3. See SPARK-28752. */ .hll { background-color: #ffffcc } @@ -73,4 +77,4 @@ insert the code (or pre?) tags for you. .vc { color: #bb60d5 } /* Name.Variable.Class */ .vg { color: #bb60d5 } /* Name.Variable.Global */ .vi { color: #bb60d5 } /* Name.Variable.Instance */ -.il { color: #40a070 } /* Literal.Number.Integer.Long */ \ No newline at end of file +.il { color: #40a070 } /* Literal.Number.Integer.Long */ diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index b83b4ba08a5f..d8c7d8a72962 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -478,15 +478,16 @@ it computes the conditional probability distribution of each feature given each For prediction, it applies Bayes' theorem to compute the conditional probability distribution of each label given an observation. -MLlib supports both [multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) -and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). +MLlib supports [Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), +[Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html) +and [Gaussian naive Bayes](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Gaussian_naive_Bayes). *Input data*: -These models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +These Multinomial and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each feature represents a term. A feature's value is the frequency of the term (in multinomial Naive Bayes) or a zero or one indicating whether the term was found in the document (in Bernoulli Naive Bayes). -Feature values must be *non-negative*. The model type is selected with an optional parameter +Feature values for Multinomial and Bernoulli models must be *non-negative*. The model type is selected with an optional parameter "multinomial" or "bernoulli" with "multinomial" as the default. For document classification, the input feature vectors should usually be sparse vectors. Since the training data is only used once, it is not necessary to cache it. diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 81d7ce37af17..79bc13459623 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,15 +19,15 @@ license: | limitations under the License. --- -When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.ansi.enabled=true`. +When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. -By default `spark.sql.ansi.enabled` is false. +By default `spark.sql.dialect.spark.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 153e68b58e74..2d5afa919e66 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -217,8 +217,6 @@ license: | For example `SELECT timestamp 'tomorrow';`. - Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`. - - - Since Spark 3.0, when `array` function is called without parameters, it returns an empty array with `NullType` data type. In Spark version 2.4 and earlier, the data type of the result is `StringType`. - Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. diff --git a/docs/sql-ref-syntax-aux-show-databases.md b/docs/sql-ref-syntax-aux-show-databases.md index 39121f764de2..efa32f291514 100644 --- a/docs/sql-ref-syntax-aux-show-databases.md +++ b/docs/sql-ref-syntax-aux-show-databases.md @@ -74,6 +74,6 @@ SHOW SCHEMAS; +------------+ {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-databases.html) +- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) - [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) - [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) diff --git a/docs/sql-ref-syntax-aux-show-tblproperties.md b/docs/sql-ref-syntax-aux-show-tblproperties.md index e0738316e97b..212db05017ad 100644 --- a/docs/sql-ref-syntax-aux-show-tblproperties.md +++ b/docs/sql-ref-syntax-aux-show-tblproperties.md @@ -111,4 +111,5 @@ SHOW TBLPROPERTIES customer ('created.date'); ### Related Statements - [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) - [ALTER TABLE SET TBLPROPERTIES](sql-ref-syntax-ddl-alter-table.html) -- [SHOW TABLE](sql-ref-syntax-aux-show-table.html) +- [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) +- [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) diff --git a/docs/sql-ref-syntax-ddl-alter-view.md b/docs/sql-ref-syntax-ddl-alter-view.md index 7a7947a3e0ac..d5cbb86d4882 100644 --- a/docs/sql-ref-syntax-ddl-alter-view.md +++ b/docs/sql-ref-syntax-ddl-alter-view.md @@ -19,4 +19,217 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +The `ALTER VIEW` statement can alter metadata associated with the view. It can change the definition of the view, change +the name of a view to a different name, set and unset the metadata of the view by setting `TBLPROPERTIES`. + +#### RENAME View +Renames the existing view. If the new view name already exists in the source database, a `TableAlreadyExistsException` is thrown. This operation +does not support moving the views across databases. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier RENAME TO view_identifier +{% endhighlight %} + +#### Parameters +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    + +#### SET View Properties +Set one or more properties of an existing view. The properties are the key value pairs. If the properties' keys exist, +the values are replaced with the new values. If the properties' keys do not exist, the key value pairs are added into +the properties. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier SET TBLPROPERTIES (property_key=property_val [, ...]) +{% endhighlight %} + +#### Parameters +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    property_key
    +
    + Specifies the property key. The key may consists of multiple parts separated by dot.

    + Syntax: + + [key_part1][.key_part2][...] + +
    +
    + +#### UNSET View Properties +Drop one or more properties of an existing view. If the specified keys do not exist, an exception is thrown. Use +`IF EXISTS` to avoid the exception. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier UNSET TBLPROPERTIES [IF EXISTS] (property_key [, ...]) +{% endhighlight %} + +#### Parameters +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    property_key
    +
    + Specifies the property key. The key may consists of multiple parts separated by dot.

    + Syntax: + + [key_part1][.key_part2][...] + +
    +
    + +#### ALTER View AS SELECT +`ALTER VIEW view_identifier AS SELECT` statement changes the definition of a view, the `SELECT` statement must be valid, +and the `view_identifier` must exist. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier AS select_statement +{% endhighlight %} + +#### Parameters +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    select_statement
    +
    + Specifies the definition of the view, detail check select_statement +
    +
    + +### Examples + +{% highlight sql %} +-- Rename only changes the view name. +-- The source and target databases of the view have to be the same. +-- Use qualified or unqualified name for the source and target view +ALTER VIEW tempdb1.v1 RENAME TO tempdb1.v2; + +-- Verify that the new view is created. +DESCRIBE TABLE EXTENDED tempdb1.v2; + ++----------------------------+----------+-------+ +|col_name |data_type |comment| ++----------------------------+----------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | ++----------------------------+----------+-------+ + +-- Before ALTER VIEW SET TBLPROPERTIES +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+----------+-------+ +|col_name |data_type |comment| ++----------------------------+----------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Table Properties |[....] | | ++----------------------------+----------+-------+ + +-- Set properties in TBLPROPERTIES +ALTER VIEW tempdb1.v2 SET TBLPROPERTIES ('created.by.user' = "John", 'created.date' = '01-01-2001' ); + +-- Use `DESCRIBE TABLE EXTENDED tempdb1.v2` to verify +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+-----------------------------------------------------+-------+ +|col_name |data_type |comment| ++----------------------------+-----------------------------------------------------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Table Properties |[created.by.user=John, created.date=01-01-2001, ....]| | ++----------------------------+-----------------------------------------------------+-------+ + +-- Remove the key `created.by.user` and `created.date` from `TBLPROPERTIES` +ALTER VIEW tempdb1.v2 UNSET TBLPROPERTIES ('created.by.user', 'created.date'); + +--Use `DESC TABLE EXTENDED tempdb1.v2` to verify the changes +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+----------+-------+ +|col_name |data_type |comment| ++----------------------------+----------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Table Properties |[....] | | ++----------------------------+----------+-------+ + +-- Change the view definition +ALTER VIEW tempdb1.v2 AS SELECT * FROM tempdb1.v1; + +-- Use `DESC TABLE EXTENDED` to verify +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+---------------------------+-------+ +|col_name |data_type |comment| ++----------------------------+---------------------------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Type |VIEW | | +|View Text |select * from tempdb1.v1 | | +|View Original Text |select * from tempdb1.v1 | | ++----------------------------+---------------------------+-------+ +{% endhighlight %} + +### Related Statements + +- [describe-table](sql-ref-syntax-aux-describe-table.html) +- [create-view](sql-ref-syntax-ddl-create-view.html) +- [drop-view](sql-ref-syntax-ddl-drop-view.html) + +#### Note: + +`ALTER VIEW` statement does not support `SET SERDE` or `SET SERDEPROPERTIES` properties + diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index c7ca28ea5b62..16310caca0bd 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -58,7 +58,7 @@ CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_nam
    query
    -
    A SELECT statement that constructs the view from base tables or other views.
    +
    A SELECT statement that constructs the view from base tables or other views.
    ### Examples @@ -78,5 +78,5 @@ CREATE GLOBAL TEMPORARY VIEW IF NOT EXISTS subscribed_movies {% endhighlight %} ### Related Statements -- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.md) -- [DROP VIEW](sql-ref-syntax-ddl-drop-view.md) +- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) +- [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) diff --git a/docs/sql-ref-syntax-ddl-drop-view.md b/docs/sql-ref-syntax-ddl-drop-view.md index f095a3456772..bf99150048d4 100644 --- a/docs/sql-ref-syntax-ddl-drop-view.md +++ b/docs/sql-ref-syntax-ddl-drop-view.md @@ -77,5 +77,6 @@ DROP VIEW IF EXISTS employeeView; ### Related Statements - [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) +- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) - [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) - [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md index 58d1b9618f0a..7baaefe317ba 100644 --- a/docs/sql-ref-syntax-ddl-truncate-table.md +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -82,5 +82,5 @@ No rows selected ### Related Statements - [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) -- [ALTER TABLE](sql-ref-syntax-ddl-alter-tabley.html) +- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md index b0fd8a8325df..db8fdff8b2ac 100644 --- a/docs/ss-migration-guide.md +++ b/docs/ss-migration-guide.md @@ -30,3 +30,4 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide. - In Spark 3.0, Structured Streaming forces the source schema into nullable when file-based datasources such as text, json, csv, parquet and orc are used via `spark.readStream(...)`. Previously, it respected the nullability in source schema; however, it caused issues tricky to debug with NPE. To restore the previous behavior, set `spark.sql.streaming.fileSource.schema.forceNullable` to `false`. +- Spark 3.0 fixes the correctness issue on Stream-stream outer join, which changes the schema of state. (SPARK-26154 for more details) Spark 3.0 will fail the query if you start your query from checkpoint constructed from Spark 2.x which uses stream-stream outer join. Please discard the checkpoint and replay previous inputs to recalculate outputs. \ No newline at end of file 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 index 2d0212f36fad..e02109375373 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -237,7 +237,8 @@ class DecisionTreeClassificationModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"DecisionTreeClassificationModel (uid=$uid) of depth $depth with $numNodes nodes" + s"DecisionTreeClassificationModel: uid=$uid, depth=$depth, numNodes=$numNodes, " + + s"numClasses=$numClasses, numFeatures=$numFeatures" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 5bc45f2b02a4..e1f5338f3489 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -340,7 +340,8 @@ class GBTClassificationModel private[ml]( @Since("1.4.0") override def toString: String = { - s"GBTClassificationModel (uid=$uid) with $numTrees trees" + s"GBTClassificationModel: uid = $uid, numTrees=$numTrees, numClasses=$numClasses, " + + s"numFeatures=$numFeatures" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 0dc1c24570b3..45114f6ee80b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -326,6 +326,10 @@ class LinearSVCModel private[classification] ( @Since("2.2.0") override def write: MLWriter = new LinearSVCModel.LinearSVCWriter(this) + @Since("3.0.0") + override def toString: String = { + s"LinearSVCModel: uid=$uid, numClasses=$numClasses, numFeatures=$numFeatures" + } } 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 1d1d139ba054..1cb591523968 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 @@ -1181,8 +1181,7 @@ class LogisticRegressionModel private[spark] ( override def write: MLWriter = new LogisticRegressionModel.LogisticRegressionModelWriter(this) override def toString: String = { - s"LogisticRegressionModel: " + - s"uid = ${super.toString}, numClasses = $numClasses, numFeatures = $numFeatures" + s"LogisticRegressionModel: uid=$uid, numClasses=$numClasses, numFeatures=$numFeatures" } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 41db6f3f4434..8c5d768044ac 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -323,6 +323,12 @@ class MultilayerPerceptronClassificationModel private[ml] ( override protected def predictRaw(features: Vector): Vector = mlpModel.predictRaw(features) override def numClasses: Int = layers.last + + @Since("3.0.0") + override def toString: String = { + s"MultilayerPerceptronClassificationModel: uid=$uid, numLayers=${layers.length}, " + + s"numClasses=$numClasses, numFeatures=$numFeatures" + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 205f565aa268..806287079441 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -18,18 +18,22 @@ package org.apache.spark.ml.classification import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams -import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.HasWeightCol +import org.apache.spark.ml.stat.Summarizer import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.VersionUtils /** * Params for Naive Bayes Classifiers. @@ -49,12 +53,13 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** * The model type which is a string (case-sensitive). - * Supported options: "multinomial" and "bernoulli". + * Supported options: "multinomial", "bernoulli", "gaussian". * (default = multinomial) * @group param */ final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + - "which is a string (case-sensitive). Supported options: multinomial (default) and bernoulli.", + "which is a string (case-sensitive). Supported options: multinomial (default), bernoulli" + + " and gaussian.", ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) /** @group getParam */ @@ -72,7 +77,11 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW * binary (0/1) data, it can also be used as Bernoulli NB * (see * here). - * The input feature values must be nonnegative. + * The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + * Since 3.0.0, it also supports Gaussian NB + * (see + * here) + * which can handle continuous data. */ // scalastyle:on line.size.limit @Since("1.5.0") @@ -103,7 +112,7 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setModelType(value: String): this.type = set(modelType, value) - setDefault(modelType -> NaiveBayes.Multinomial) + setDefault(modelType -> Multinomial) /** * Sets the value of param [[weightCol]]. @@ -130,6 +139,9 @@ class NaiveBayes @Since("1.5.0") ( positiveLabel: Boolean): NaiveBayesModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, + probabilityCol, modelType, smoothing, thresholds) + if (positiveLabel && isDefined(thresholds)) { val numClasses = getNumClasses(dataset) instr.logNumClasses(numClasses) @@ -138,44 +150,55 @@ class NaiveBayes @Since("1.5.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val validateInstance = $(modelType) match { - case Multinomial => - (instance: Instance) => requireNonnegativeValues(instance.features) - case Bernoulli => - (instance: Instance) => requireZeroOneBernoulliValues(instance.features) + $(modelType) match { + case Bernoulli | Multinomial => + trainDiscreteImpl(dataset, instr) + case Gaussian => + trainGaussianImpl(dataset, instr) case _ => // This should never happen. throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } + } - instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, - probabilityCol, modelType, smoothing, thresholds) + private def trainDiscreteImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ - val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size - instr.logNumFeatures(numFeatures) + val validateUDF = $(modelType) match { + case Multinomial => + udf { vector: Vector => requireNonnegativeValues(vector); vector } + case Bernoulli => + udf { vector: Vector => requireZeroOneBernoulliValues(vector); vector } + } + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } // Aggregates term frequencies per label. - // TODO: Calling aggregateByKey and collect creates two stages, we can implement something - // TODO: similar to reduceByKeyLocally to save one stage. - val aggregated = extractInstances(dataset, validateInstance).map { instance => - (instance.label, (instance.weight, instance.features)) - }.aggregateByKey[(Double, DenseVector, Long)]((0.0, Vectors.zeros(numFeatures).toDense, 0L))( - seqOp = { - case ((weightSum, featureSum, count), (weight, features)) => - BLAS.axpy(weight, features, featureSum) - (weightSum + weight, featureSum, count + 1) - }, - combOp = { - case ((weightSum1, featureSum1, count1), (weightSum2, featureSum2, count2)) => - BLAS.axpy(1.0, featureSum2, featureSum1) - (weightSum1 + weightSum2, featureSum1, count1 + count2) - }).collect().sortBy(_._1) - - val numSamples = aggregated.map(_._2._3).sum + // TODO: Summarizer directly returns sum vector. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "count") + .summary(validateUDF(col($(featuresCol))), w).as("summary")) + .select($(labelCol), "weightSum", "summary.mean", "summary.count") + .as[(Double, Double, Vector, Long)] + .map { case (label, weightSum, mean, count) => + BLAS.scal(weightSum, mean) + (label, weightSum, mean, count) + }.collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + val numSamples = aggregated.map(_._4).sum instr.logNumExamples(numSamples) val numLabels = aggregated.length instr.logNumClasses(numLabels) - val numDocuments = aggregated.map(_._2._1).sum + val numDocuments = aggregated.map(_._2).sum val labelArray = new Array[Double](numLabels) val piArray = new Array[Double](numLabels) @@ -184,19 +207,17 @@ class NaiveBayes @Since("1.5.0") ( val lambda = $(smoothing) val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 - aggregated.foreach { case (label, (n, sumTermFreqs, _)) => + aggregated.foreach { case (label, n, sumTermFreqs, _) => labelArray(i) = label piArray(i) = math.log(n + lambda) - piLogDenom val thetaLogDenom = $(modelType) match { - case Multinomial => math.log(sumTermFreqs.values.sum + numFeatures * lambda) + case Multinomial => math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) case Bernoulli => math.log(n + 2.0 * lambda) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } var j = 0 + val offset = i * numFeatures while (j < numFeatures) { - thetaArray(i * numFeatures + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + thetaArray(offset + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom j += 1 } i += 1 @@ -204,7 +225,86 @@ class NaiveBayes @Since("1.5.0") ( val pi = Vectors.dense(piArray) val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + .setOldLabels(labelArray) + } + + private def trainGaussianImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + // Aggregates mean vector and square-sum vector per label. + // TODO: Summarizer directly returns square-sum vector. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "normL2") + .summary(col($(featuresCol)), w).as("summary")) + .select($(labelCol), "weightSum", "summary.mean", "summary.normL2") + .as[(Double, Double, Vector, Vector)] + .map { case (label, weightSum, mean, normL2) => + (label, weightSum, mean, Vectors.dense(normL2.toArray.map(v => v * v))) + }.collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + + val numLabels = aggregated.length + instr.logNumClasses(numLabels) + + val numInstances = aggregated.map(_._2).sum + + // If the ratio of data variance between dimensions is too small, it + // will cause numerical errors. To address this, we artificially + // boost the variance by epsilon, a small fraction of the standard + // deviation of the largest dimension. + // Refer to scikit-learn's implementation + // [https://github.com/scikit-learn/scikit-learn/blob/0.21.X/sklearn/naive_bayes.py#L348] + // and discussion [https://github.com/scikit-learn/scikit-learn/pull/5349] for detail. + val epsilon = Iterator.range(0, numFeatures).map { j => + var globalSum = 0.0 + var globalSqrSum = 0.0 + aggregated.foreach { case (_, weightSum, mean, squareSum) => + globalSum += mean(j) * weightSum + globalSqrSum += squareSum(j) + } + globalSqrSum / numInstances - + globalSum * globalSum / numInstances / numInstances + }.max * 1e-9 + + val piArray = new Array[Double](numLabels) + + // thetaArray in Gaussian NB store the means of features per label + val thetaArray = new Array[Double](numLabels * numFeatures) + + // thetaArray in Gaussian NB store the variances of features per label + val sigmaArray = new Array[Double](numLabels * numFeatures) + + var i = 0 + val logNumInstances = math.log(numInstances) + aggregated.foreach { case (_, weightSum, mean, squareSum) => + piArray(i) = math.log(weightSum) - logNumInstances + var j = 0 + val offset = i * numFeatures + while (j < numFeatures) { + val m = mean(j) + thetaArray(offset + j) = m + sigmaArray(offset + j) = epsilon + squareSum(j) / weightSum - m * m + j += 1 + } + i += 1 + } + + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + val sigma = new DenseMatrix(numLabels, numFeatures, sigmaArray, true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, sigma.compressed) } @Since("1.5.0") @@ -219,8 +319,11 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** String name for Bernoulli model type. */ private[classification] val Bernoulli: String = "bernoulli" + /** String name for Gaussian model type. */ + private[classification] val Gaussian: String = "gaussian" + /* Set of modelTypes that NaiveBayes supports */ - private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli) + private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli, Gaussian) private[NaiveBayes] def requireNonnegativeValues(v: Vector): Unit = { val values = v match { @@ -248,19 +351,24 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** * Model produced by [[NaiveBayes]] - * @param pi log of class priors, whose dimension is C (number of classes) + * + * @param pi log of class priors, whose dimension is C (number of classes) * @param theta log of class conditional probabilities, whose dimension is C (number of classes) * by D (number of features) + * @param sigma variance of each feature, whose dimension is C (number of classes) + * by D (number of features). This matrix is only available when modelType + * is set Gaussian. */ @Since("1.5.0") class NaiveBayesModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("2.0.0") val pi: Vector, - @Since("2.0.0") val theta: Matrix) + @Since("2.0.0") val theta: Matrix, + @Since("3.0.0") val sigma: Matrix) extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams with MLWritable { - import NaiveBayes.{Bernoulli, Multinomial} + import NaiveBayes.{Bernoulli, Multinomial, Gaussian} /** * mllib NaiveBayes is a wrapper of ml implementation currently. @@ -280,18 +388,36 @@ class NaiveBayesModel private[ml] ( * This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra * application of this condition (in predict function). */ - private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { - case Multinomial => (None, None) + @transient private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { case Bernoulli => val negTheta = theta.map(value => math.log1p(-math.exp(value))) val ones = new DenseVector(Array.fill(theta.numCols) {1.0}) val thetaMinusNegTheta = theta.map { value => value - math.log1p(-math.exp(value)) } - (Option(thetaMinusNegTheta), Option(negTheta.multiply(ones))) + (thetaMinusNegTheta, negTheta.multiply(ones)) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables thetaMinusNegTheta and negThetaSum should only be precomputed in Bernoulli NB.") + } + + /** + * Gaussian scoring requires sum of log(Variance). + * This precomputes sum of log(Variance) which are used for the linear algebra + * application of this condition (in predict function). + */ + @transient private lazy val logVarSum = $(modelType) match { + case Gaussian => + Array.tabulate(numClasses) { i => + Iterator.range(0, numFeatures).map { j => + math.log(sigma(i, j)) + }.sum + } case _ => // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables logVarSum should only be precomputed in Gaussian NB.") } @Since("1.6.0") @@ -311,24 +437,42 @@ class NaiveBayesModel private[ml] ( require(value == 0.0 || value == 1.0, s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") ) - val prob = thetaMinusNegTheta.get.multiply(features) + val prob = thetaMinusNegTheta.multiply(features) BLAS.axpy(1.0, pi, prob) - BLAS.axpy(1.0, negThetaSum.get, prob) + BLAS.axpy(1.0, negThetaSum, prob) prob } - override protected def predictRaw(features: Vector): Vector = { + private def gaussianCalculation(features: Vector) = { + val prob = Array.ofDim[Double](numClasses) + var i = 0 + while (i < numClasses) { + var s = 0.0 + var j = 0 + while (j < numFeatures) { + val d = features(j) - theta(i, j) + s += d * d / sigma(i, j) + j += 1 + } + prob(i) = pi(i) - (s + logVarSum(i)) / 2 + i += 1 + } + Vectors.dense(prob) + } + + @transient private lazy val predictRawFunc = { $(modelType) match { case Multinomial => - multinomialCalculation(features) + features: Vector => multinomialCalculation(features) case Bernoulli => - bernoulliCalculation(features) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") + features: Vector => bernoulliCalculation(features) + case Gaussian => + features: Vector => gaussianCalculation(features) } } + override protected def predictRaw(features: Vector): Vector = predictRawFunc(features) + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { rawPrediction match { case dv: DenseVector => @@ -354,12 +498,13 @@ class NaiveBayesModel private[ml] ( @Since("1.5.0") override def copy(extra: ParamMap): NaiveBayesModel = { - copyValues(new NaiveBayesModel(uid, pi, theta).setParent(this.parent), extra) + copyValues(new NaiveBayesModel(uid, pi, theta, sigma).setParent(this.parent), extra) } @Since("1.5.0") override def toString: String = { - s"NaiveBayesModel (uid=$uid) with ${pi.size} classes" + s"NaiveBayesModel: uid=$uid, modelType=${$(modelType)}, numClasses=$numClasses, " + + s"numFeatures=$numFeatures" } @Since("1.6.0") @@ -377,34 +522,61 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { /** [[MLWriter]] instance for [[NaiveBayesModel]] */ private[NaiveBayesModel] class NaiveBayesModelWriter(instance: NaiveBayesModel) extends MLWriter { + import NaiveBayes._ private case class Data(pi: Vector, theta: Matrix) + private case class GaussianData(pi: Vector, theta: Matrix, sigma: Matrix) override protected def saveImpl(path: String): Unit = { // Save metadata and Params DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: pi, theta - val data = Data(instance.pi, instance.theta) val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + + instance.getModelType match { + case Multinomial | Bernoulli => + // Save model data: pi, theta + require(instance.sigma == null) + val data = Data(instance.pi, instance.theta) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + + case Gaussian => + require(instance.sigma != null) + val data = GaussianData(instance.pi, instance.theta, instance.sigma) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } } } private class NaiveBayesModelReader extends MLReader[NaiveBayesModel] { + import NaiveBayes._ /** Checked against metadata when loading model */ private val className = classOf[NaiveBayesModel].getName override def load(path: String): NaiveBayesModel = { + implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) + val modelTypeJson = metadata.getParamValue("modelType") + val modelType = Param.jsonDecode[String](compact(render(modelTypeJson))) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") - val Row(pi: Vector, theta: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta") - .select("pi", "theta") - .head() - val model = new NaiveBayesModel(metadata.uid, pi, theta) + + val model = if (major.toInt < 3 || modelType != Gaussian) { + val Row(pi: Vector, theta: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta") + .select("pi", "theta") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, null) + } else { + val Row(pi: Vector, theta: Matrix, sigma: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta", "sigma") + .select("pi", "theta", "sigma") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, sigma) + } metadata.getAndSetParams(model) model diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 675315e3bb07..51a624795cdd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -257,6 +257,12 @@ final class OneVsRestModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new OneVsRestModel.OneVsRestModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"OneVsRestModel: uid=$uid, classifier=${$(classifier)}, numClasses=$numClasses, " + + s"numFeatures=$numFeatures" + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 245cda35d8ad..bc28d783ed96 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -260,7 +260,8 @@ class RandomForestClassificationModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"RandomForestClassificationModel (uid=$uid) with $getNumTrees trees" + s"RandomForestClassificationModel: uid=$uid, numTrees=$getNumTrees, numClasses=$numClasses, " + + s"numFeatures=$numFeatures" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 4ad0cb55b007..5f2316fa7ce1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -91,6 +91,9 @@ class BisectingKMeansModel private[ml] ( extends Model[BisectingKMeansModel] with BisectingKMeansParams with MLWritable with HasTrainingSummary[BisectingKMeansSummary] { + @Since("3.0.0") + lazy val numFeatures: Int = parentModel.clusterCenters.head.size + @Since("2.0.0") override def copy(extra: ParamMap): BisectingKMeansModel = { val copied = copyValues(new BisectingKMeansModel(uid, parentModel), extra) @@ -145,6 +148,12 @@ class BisectingKMeansModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new BisectingKMeansModel.BisectingKMeansModelWriter(this) + @Since("3.0.0") + override def toString: String = { + s"BisectingKMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + + s"numFeatures=$numFeatures" + } + /** * Gets summary of model on training set. An exception is * thrown if `hasSummary` is false. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index b4d9a9f88246..916f326ab561 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -89,6 +89,9 @@ class GaussianMixtureModel private[ml] ( extends Model[GaussianMixtureModel] with GaussianMixtureParams with MLWritable with HasTrainingSummary[GaussianMixtureSummary] { + @Since("3.0.0") + lazy val numFeatures: Int = gaussians.head.mean.size + /** @group setParam */ @Since("2.1.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) @@ -186,6 +189,11 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) + @Since("3.0.0") + override def toString: String = { + s"GaussianMixtureModel: uid=$uid, k=${weights.length}, numFeatures=$numFeatures" + } + /** * Gets summary of model on training set. An exception is * thrown if `hasSummary` is false. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 5cc0f38c67e7..caeded400f9a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -108,6 +108,9 @@ class KMeansModel private[ml] ( extends Model[KMeansModel] with KMeansParams with GeneralMLWritable with HasTrainingSummary[KMeansSummary] { + @Since("3.0.0") + lazy val numFeatures: Int = parentModel.clusterCenters.head.size + @Since("1.5.0") override def copy(extra: ParamMap): KMeansModel = { val copied = copyValues(new KMeansModel(uid, parentModel), extra) @@ -153,6 +156,12 @@ class KMeansModel private[ml] ( @Since("1.6.0") override def write: GeneralMLWriter = new GeneralMLWriter(this) + @Since("3.0.0") + override def toString: String = { + s"KMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + + s"numFeatures=$numFeatures" + } + /** * Gets summary of model on training set. An exception is * thrown if `hasSummary` is false. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 91201e7bd03f..9b0005b3747d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -620,6 +620,11 @@ class LocalLDAModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new LocalLDAModel.LocalLDAModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"LocalLDAModel: uid=$uid, k=${$(k)}, numFeatures=$vocabSize" + } } @@ -783,6 +788,11 @@ class DistributedLDAModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new DistributedLDAModel.DistributedWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"DistributedLDAModel: uid=$uid, k=${$(k)}, numFeatures=$vocabSize" + } } 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 09e8e7b232f3..55b910e98d40 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 @@ -130,6 +130,12 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("1.4.1") override def copy(extra: ParamMap): BinaryClassificationEvaluator = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"BinaryClassificationEvaluator: uid=$uid, metricName=${$(metricName)}, " + + s"numBins=${$(numBins)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala index 868bd2a763f5..157bed2a6b11 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala @@ -120,6 +120,12 @@ class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val uid: Str throw new IllegalArgumentException(s"No support for metric $mn, distance $dm") } } + + @Since("3.0.0") + override def toString: String = { + s"ClusteringEvaluator: uid=$uid, metricName=${$(metricName)}, " + + s"distanceMeasure=${$(distanceMeasure)}" + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index bac3e2377439..ab14227f06be 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -184,6 +184,12 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("1.5.0") override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"MulticlassClassificationEvaluator: uid=$uid, metricName=${$(metricName)}, " + + s"metricLabel=${$(metricLabel)}, beta=${$(beta)}, eps=${$(eps)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala index f12c6700be04..5216c40819b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala @@ -121,6 +121,12 @@ class MultilabelClassificationEvaluator (override val uid: String) } override def copy(extra: ParamMap): MultilabelClassificationEvaluator = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"MultilabelClassificationEvaluator: uid=$uid, metricName=${$(metricName)}, " + + s"metricLabel=${$(metricLabel)}" + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala index 64ab3c3f7fdd..ca3a8ebc1659 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala @@ -105,6 +105,11 @@ class RankingEvaluator (override val uid: String) override def isLargerBetter: Boolean = true override def copy(extra: ParamMap): RankingEvaluator = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"RankingEvaluator: uid=$uid, metricName=${$(metricName)}, k=${$(k)}" + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index b0cafefe420a..9f32d40d166b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -124,6 +124,12 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("1.5.0") override def copy(extra: ParamMap): RegressionEvaluator = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"RegressionEvaluator: uid=$uid, metricName=${$(metricName)}, " + + s"throughOrigin=${$(throughOrigin)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index ec4d45b65317..07a4f91443bc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -204,6 +204,13 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.1") override def copy(extra: ParamMap): Binarizer = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"Binarizer: uid=$uid" + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + + get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala index c074830ec923..4e266fbc1ec1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala @@ -106,6 +106,11 @@ class BucketedRandomProjectionLSHModel private[ml]( override def write: MLWriter = { new BucketedRandomProjectionLSHModel.BucketedRandomProjectionLSHModelWriter(this) } + + @Since("3.0.0") + override def toString: String = { + s"BucketedRandomProjectionLSHModel: uid=$uid, numHashTables=${$(numHashTables)}" + } } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index 8533ed5ce572..9aeddae78ed7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -215,6 +215,13 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String override def copy(extra: ParamMap): Bucketizer = { defaultCopy[Bucketizer](extra).setParent(parent) } + + @Since("3.0.0") + override def toString: String = { + s"Bucketizer: uid=$uid" + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + + get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 2a3656c49584..9103e4feac45 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -316,6 +316,11 @@ final class ChiSqSelectorModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new ChiSqSelectorModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"ChiSqSelectorModel: uid=$uid, numSelectedFeatures=${selectedFeatures.length}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index e78e6d4f76d8..c58d44d49234 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -307,7 +307,7 @@ class CountVectorizerModel( } val dictBr = broadcastDict.get val minTf = $(minTF) - val vectorizer = udf { (document: Seq[String]) => + val vectorizer = udf { document: Seq[String] => val termCounts = new OpenHashMap[Int, Double] var tokenCount = 0L document.foreach { term => @@ -344,6 +344,11 @@ class CountVectorizerModel( @Since("1.6.0") override def write: MLWriter = new CountVectorizerModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"CountVectorizerModel: uid=$uid, vocabularySize=${vocabulary.length}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 84d6a536ccca..e2167f01281d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -74,6 +74,11 @@ class DCT @Since("1.5.0") (@Since("1.5.0") override val uid: String) } override protected def outputDataType: DataType = new VectorUDT + + @Since("3.0.0") + override def toString: String = { + s"DCT: uid=$uid, inverse=$inverse" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 2f329235431f..227c13d60fd8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -81,6 +81,12 @@ class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: Stri } override protected def outputDataType: DataType = new VectorUDT() + + @Since("3.0.0") + override def toString: String = { + s"ElementwiseProduct: uid=$uid" + + get(scalingVec).map(v => s", vectorSize=${v.size}").getOrElse("") + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala index 61b4d5d54aca..39862554c5d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala @@ -22,7 +22,7 @@ import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.Vectors -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators, StringArrayParam} +import org.apache.spark.ml.param.{ParamMap, StringArrayParam} import org.apache.spark.ml.param.shared.{HasInputCols, HasNumFeatures, HasOutputCol} import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} @@ -199,6 +199,13 @@ class FeatureHasher(@Since("2.3.0") override val uid: String) extends Transforme val attrGroup = new AttributeGroup($(outputCol), $(numFeatures)) SchemaUtils.appendColumn(schema, attrGroup.toStructField()) } + + @Since("3.0.0") + override def toString: String = { + s"FeatureHasher: uid=$uid, numFeatures=${$(numFeatures)}" + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + + get(categoricalCols).map(c => s", numCategoricalCols=${c.length}").getOrElse("") + } } @Since("2.3.0") 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 fe9f4f2123da..80bf85936aac 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 @@ -127,6 +127,11 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.1") override def copy(extra: ParamMap): HashingTF = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"HashingTF: uid=$uid, binary=${$(binary)}, numFeatures=${$(numFeatures)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 4338421bf8bc..5f4103abcf50 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -175,9 +175,13 @@ class IDFModel private[ml] ( @Since("3.0.0") def numDocs: Long = idfModel.numDocs - @Since("1.6.0") override def write: MLWriter = new IDFModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"IDFModel: uid=$uid, numDocs=$numDocs" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index fbccfb1041d1..64f1722f5fcb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -274,6 +274,13 @@ class ImputerModel private[ml] ( @Since("2.2.0") override def write: MLWriter = new ImputerModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"ImputerModel: uid=$uid, strategy=${$(strategy)}, missingValue=${$(missingValue)}" + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + + get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index 611f1b691b78..9a4f1d97c907 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -218,6 +218,11 @@ class Interaction @Since("1.6.0") (@Since("1.6.0") override val uid: String) ext @Since("1.6.0") override def copy(extra: ParamMap): Interaction = defaultCopy(extra) + @Since("3.0.0") + override def toString: String = { + s"Interaction: uid=$uid" + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index 88d09d760b51..6bab70e502ed 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -140,6 +140,11 @@ class MaxAbsScalerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new MaxAbsScalerModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"MaxAbsScalerModel: uid=$uid, numFeatures=${maxAbs.size}" + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala index da0eaad667cc..de7fe91c417b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala @@ -96,6 +96,11 @@ class MinHashLSHModel private[ml]( @Since("2.1.0") override def write: MLWriter = new MinHashLSHModel.MinHashLSHModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"MinHashLSHModel: uid=$uid, numHashTables=${$(numHashTables)}" + } } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index b0d7b637caa5..e381a0435e9e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -226,6 +226,12 @@ class MinMaxScalerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new MinMaxScalerModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"MinMaxScalerModel: uid=$uid, numFeatures=${originalMin.size}, min=${$(min)}, " + + s"max=${$(max)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index e0772d5af20a..fd6fde0744d0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -70,6 +70,11 @@ class NGram @Since("1.5.0") (@Since("1.5.0") override val uid: String) } override protected def outputDataType: DataType = new ArrayType(StringType, false) + + @Since("3.0.0") + override def toString: String = { + s"NGram: uid=$uid, n=${$(n)}" + } } @Since("1.6.0") 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 5db7f5da49c6..d129c2b2c2dc 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 @@ -65,6 +65,11 @@ class Normalizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) } override protected def outputDataType: DataType = new VectorUDT() + + @Since("3.0.0") + override def toString: String = { + s"Normalizer: uid=$uid, p=${$(p)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 459994c352da..fd58043cda07 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -376,6 +376,13 @@ class OneHotEncoderModel private[ml] ( @Since("3.0.0") override def write: MLWriter = new OneHotEncoderModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"OneHotEncoderModel: uid=$uid, dropLast=${$(dropLast)}, handleInvalid=${$(handleInvalid)}" + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + + get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") + } } @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index aa5a171d4fec..69dcacbb0c18 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -179,6 +179,11 @@ class PCAModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new PCAModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"PCAModel: uid=$uid, k=${$(k)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 5734b06ee963..592ca001a246 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -77,6 +77,11 @@ class PolynomialExpansion @Since("1.4.0") (@Since("1.4.0") override val uid: Str @Since("1.4.1") override def copy(extra: ParamMap): PolynomialExpansion = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"PolynomialExpansion: uid=$uid, degree=${$(degree)}" + } } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 9e95762f1291..7ccfafa4ac81 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -320,7 +320,10 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) override def copy(extra: ParamMap): RFormula = defaultCopy(extra) @Since("2.0.0") - override def toString: String = s"RFormula(${get(formula).getOrElse("")}) (uid=$uid)" + override def toString: String = { + s"RFormula: uid=$uid" + + get(formula).map(f => s", formula = $f").getOrElse("") + } } @Since("2.0.0") @@ -376,7 +379,9 @@ class RFormulaModel private[feature]( } @Since("2.0.0") - override def toString: String = s"RFormulaModel($resolvedFormula) (uid=$uid)" + override def toString: String = { + s"RFormulaModel: uid=$uid, resolvedFormula=$resolvedFormula" + } private def transformLabel(dataset: Dataset[_]): DataFrame = { val labelName = resolvedFormula.label diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index 1d609ef3190d..1b9b8082931a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -251,6 +251,12 @@ class RobustScalerModel private[ml] ( } override def write: MLWriter = new RobustScalerModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"RobustScalerModel: uid=$uid, numFeatures=${median.size}, " + + s"withCentering=${$(withCentering)}, withScaling=${$(withScaling)}" + } } @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 0fb1d8c5dc57..9b99b8177a26 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -90,6 +90,11 @@ class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) @Since("1.6.0") override def copy(extra: ParamMap): SQLTransformer = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"SQLTransformer: uid=$uid, statement=${$(statement)}" + } } @Since("1.6.0") 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 834c21e16dd2..41e319c1fb1f 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 @@ -184,6 +184,12 @@ class StandardScalerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new StandardScalerModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"StandardScalerModel: uid=$uid, numFeatures=${mean.size}, withMean=${$(withMean)}, " + + s"withStd=${$(withStd)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala index f95e03ae6c82..b6ed4f2b000c 100755 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -22,15 +22,19 @@ import java.util.Locale import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared.{HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} -import org.apache.spark.sql.types.{ArrayType, StringType, StructType} +import org.apache.spark.sql.types.{ArrayType, StringType, StructField, StructType} /** * A feature transformer that filters out stop words from input. * + * Since 3.0.0, `StopWordsRemover` can filter out multiple columns at once by setting the + * `inputCols` parameter. Note that when both the `inputCol` and `inputCols` parameters are set, + * an Exception will be thrown. + * * @note null values from input array are preserved unless adding null to stopWords * explicitly. * @@ -38,7 +42,8 @@ import org.apache.spark.sql.types.{ArrayType, StringType, StructType} */ @Since("1.5.0") class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + extends Transformer with HasInputCol with HasOutputCol with HasInputCols with HasOutputCols + with DefaultParamsWritable { @Since("1.5.0") def this() = this(Identifiable.randomUID("stopWords")) @@ -51,6 +56,14 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ + @Since("3.0.0") + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + /** * The words to be filtered out. * Default: English stop words @@ -121,6 +134,15 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String } } + /** Returns the input and output column names corresponding in pair. */ + private[feature] def getInOutCols(): (Array[String], Array[String]) = { + if (isSet(inputCol)) { + (Array($(inputCol)), Array($(outputCol))) + } else { + ($(inputCols), $(outputCols)) + } + } + setDefault(stopWords -> StopWordsRemover.loadDefaultStopWords("english"), caseSensitive -> false, locale -> getDefaultOrUS.toString) @@ -142,20 +164,48 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String terms.filter(s => !lowerStopWords.contains(toLower(s))) } } - val metadata = outputSchema($(outputCol)).metadata - dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) + + val (inputColNames, outputColNames) = getInOutCols() + val ouputCols = inputColNames.map { inputColName => + t(col(inputColName)) + } + val ouputMetadata = outputColNames.map(outputSchema(_).metadata) + dataset.withColumns(outputColNames, ouputCols, ouputMetadata) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.sameType(ArrayType(StringType)), "Input type must be " + - s"${ArrayType(StringType).catalogString} but got ${inputType.catalogString}.") - SchemaUtils.appendColumn(schema, $(outputCol), inputType, schema($(inputCol)).nullable) + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), + Seq(outputCols)) + + if (isSet(inputCols)) { + require(getInputCols.length == getOutputCols.length, + s"StopWordsRemover $this has mismatched Params " + + s"for multi-column transform. Params ($inputCols, $outputCols) should have " + + "equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}).") + } + + val (inputColNames, outputColNames) = getInOutCols() + val newCols = inputColNames.zip(outputColNames).map { case (inputColName, outputColName) => + require(!schema.fieldNames.contains(outputColName), + s"Output Column $outputColName already exists.") + val inputType = schema(inputColName).dataType + require(inputType.sameType(ArrayType(StringType)), "Input type must be " + + s"${ArrayType(StringType).catalogString} but got ${inputType.catalogString}.") + StructField(outputColName, inputType, schema(inputColName).nullable) + } + StructType(schema.fields ++ newCols) } @Since("1.5.0") override def copy(extra: ParamMap): StopWordsRemover = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"StopWordsRemover: uid=$uid, numStopWords=${$(stopWords).length}, locale=${$(locale)}, " + + s"caseSensitive=${$(caseSensitive)}" + } } @Since("1.6.0") 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 2ce5acf1fea1..9f9f097a26ea 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 @@ -412,7 +412,7 @@ class StringIndexerModel ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - var (inputColNames, outputColNames) = getInOutCols() + val (inputColNames, outputColNames) = getInOutCols() val outputColumns = new Array[Column](outputColNames.length) // Skips invalid rows if `handleInvalid` is set to `StringIndexer.SKIP_INVALID`. @@ -473,6 +473,14 @@ class StringIndexerModel ( @Since("1.6.0") override def write: StringIndexModelWriter = new StringIndexModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"StringIndexerModel: uid=$uid, handleInvalid=${$(handleInvalid)}" + + get(stringOrderType).map(t => s", stringOrderType=$t").getOrElse("") + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + + get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") + } } @Since("1.6.0") 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 e6e9bdfd292b..5ec5b7732564 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 @@ -175,6 +175,12 @@ class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.1") override def copy(extra: ParamMap): VectorAssembler = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"VectorAssembler: uid=$uid, handleInvalid=${$(handleInvalid)}" + + get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + } } @Since("1.6.0") 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 6a3305aad8f3..18a82e5fe945 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 @@ -428,7 +428,7 @@ class VectorIndexerModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) val newField = prepOutputField(dataset.schema) - val transformUDF = udf { (vector: Vector) => transformFunc(vector) } + val transformUDF = udf { vector: Vector => transformFunc(vector) } val newCol = transformUDF(dataset($(inputCol))) val ds = dataset.withColumn($(outputCol), newCol, newField.metadata) if (getHandleInvalid == VectorIndexer.SKIP_INVALID) { @@ -506,6 +506,11 @@ class VectorIndexerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new VectorIndexerModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"VectorIndexerModel: uid=$uid, numFeatures=$numFeatures, handleInvalid=${$(handleInvalid)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala index 5d787f263a12..0f778e09a80b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala @@ -176,6 +176,11 @@ class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String) @Since("2.3.0") override def copy(extra: ParamMap): this.type = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"VectorSizeHint: uid=$uid, size=${$(size)}, handleInvalid=${$(handleInvalid)}" + } } @Since("2.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index e3e462d07e10..b84b8af4e8a9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -159,6 +159,12 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri @Since("1.5.0") override def copy(extra: ParamMap): VectorSlicer = defaultCopy(extra) + + @Since("3.0.0") + override def toString: String = { + s"VectorSlicer: uid=$uid" + + get(indices).map(i => s", numSelectedFeatures=${i.length}").getOrElse("") + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 6ae90b805093..81dde0315c19 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -324,6 +324,12 @@ class Word2VecModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new Word2VecModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"Word2VecModel: uid=$uid, numWords=${wordVectors.wordIndex.size}, " + + s"vectorSize=${$(vectorSize)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index e1c9b927a28c..a9592dbfca06 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -310,6 +310,11 @@ class FPGrowthModel private[ml] ( @Since("2.2.0") override def write: MLWriter = new FPGrowthModel.FPGrowthModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"FPGrowthModel: uid=$uid, numTrainingRecords=$numTrainingRecords" + } } @Since("2.2.0") 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 5049ef924561..2fb9a276be88 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 @@ -338,6 +338,11 @@ class ALSModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new ALSModel.ALSModelWriter(this) + @Since("3.0.0") + override def toString: String = { + s"ALSModel: uid=$uid, rank=$rank" + } + /** * Returns top `numItems` items recommended for each user, for all users. * @param numItems max number of recommendations for each user @@ -1041,13 +1046,13 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { .persist(finalRDDStorageLevel) if (finalRDDStorageLevel != StorageLevel.NONE) { userIdAndFactors.count() - itemIdAndFactors.count() - itemFactors.unpersist() userInBlocks.unpersist() userOutBlocks.unpersist() - itemInBlocks.unpersist() itemOutBlocks.unpersist() blockRatings.unpersist() + itemIdAndFactors.count() + itemFactors.unpersist() + itemInBlocks.unpersist() } (userIdAndFactors, itemIdAndFactors) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 1ce5b5b02b50..fe89aed9a928 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -311,6 +311,9 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") val scale: Double) extends Model[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams with MLWritable { + @Since("3.0.0") + lazy val numFeatures: Int = coefficients.size + /** @group setParam */ @Since("1.6.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) @@ -386,6 +389,11 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new AFTSurvivalRegressionModel.AFTSurvivalRegressionModelWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"AFTSurvivalRegressionModel: uid=$uid, numFeatures=$numFeatures" + } } @Since("1.6.0") 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 index 05851d511675..4a97997a1deb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -243,7 +243,8 @@ class DecisionTreeRegressionModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"DecisionTreeRegressionModel (uid=$uid) of depth $depth with $numNodes nodes" + s"DecisionTreeRegressionModel: uid=$uid, depth=$depth, numNodes=$numNodes, " + + s"numFeatures=$numFeatures" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 9c38647642a6..700f7a2075a9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -302,7 +302,7 @@ class GBTRegressionModel private[ml]( @Since("1.4.0") override def toString: String = { - s"GBTRegressionModel (uid=$uid) with $numTrees trees" + s"GBTRegressionModel: uid=$uid, numTrees=$numTrees, numFeatures=$numFeatures" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index c504a5462900..53b29102f01b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1106,6 +1106,12 @@ class GeneralizedLinearRegressionModel private[ml] ( new GeneralizedLinearRegressionModel.GeneralizedLinearRegressionModelWriter(this) override val numFeatures: Int = coefficients.size + + @Since("3.0.0") + override def toString: String = { + s"GeneralizedLinearRegressionModel: uid=$uid, family=${$(family)}, link=${$(link)}, " + + s"numFeatures=$numFeatures" + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 8b9233dcdc4d..47f9e4bfb833 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -259,6 +259,14 @@ class IsotonicRegressionModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new IsotonicRegressionModelWriter(this) + + @Since("3.0.0") + val numFeatures: Int = 1 + + @Since("3.0.0") + override def toString: String = { + s"IsotonicRegressionModel: uid=$uid, numFeatures=$numFeatures" + } } @Since("1.6.0") 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 25c6f4d980b9..dad1080cea39 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 @@ -702,6 +702,11 @@ class LinearRegressionModel private[ml] ( */ @Since("1.6.0") override def write: GeneralMLWriter = new GeneralMLWriter(this) + + @Since("3.0.0") + override def toString: String = { + s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" + } } /** A writer for LinearRegression that handles the "internal" (or default) format */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 8f78fc1da18c..c3afab57a49c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -235,7 +235,7 @@ class RandomForestRegressionModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"RandomForestRegressionModel (uid=$uid) with $getNumTrees trees" + s"RandomForestRegressionModel: uid=$uid, numTrees=$getNumTrees, numFeatures=$numFeatures" } /** 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 e60a14f976a5..fff18bcbec65 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 @@ -323,6 +323,11 @@ class CrossValidatorModel private[ml] ( override def write: CrossValidatorModel.CrossValidatorModelWriter = { new CrossValidatorModel.CrossValidatorModelWriter(this) } + + @Since("3.0.0") + override def toString: String = { + s"CrossValidatorModel: uid=$uid, bestModel=$bestModel, numFolds=${$(numFolds)}" + } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index 8b251197afbe..ecf9b846eedd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -140,7 +140,7 @@ class TrainValidationSplit @Since("1.5.0") (@Since("1.5.0") override val uid: St val collectSubModelsParam = $(collectSubModels) - var subModels: Option[Array[Model[_]]] = if (collectSubModelsParam) { + val subModels: Option[Array[Model[_]]] = if (collectSubModelsParam) { Some(Array.fill[Model[_]](epm.length)(null)) } else None @@ -314,6 +314,11 @@ class TrainValidationSplitModel private[ml] ( override def write: TrainValidationSplitModel.TrainValidationSplitModelWriter = { new TrainValidationSplitModel.TrainValidationSplitModelWriter(this) } + + @Since("3.0.0") + override def toString: String = { + s"TrainValidationSplitModel: uid=$uid, bestModel=$bestModel, trainRatio=${$(trainRatio)}" + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 278d61d91673..ac85fbc235c9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -223,12 +223,12 @@ class KMeans private ( // Compute squared norms and cache them. val norms = data.map(Vectors.norm(_, 2.0)) - norms.persist() val zippedData = data.zip(norms).map { case (v, norm) => new VectorWithNorm(v, norm) } + zippedData.persist() val model = runAlgorithm(zippedData, instr) - norms.unpersist() + zippedData.unpersist() // Warn at the end of the run as well, for increased visibility. if (data.getStorageLevel == StorageLevel.NONE) { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index d2b8751360e9..07116606dfb5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -2767,7 +2767,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("toString") { val model = new LogisticRegressionModel("logReg", Vectors.dense(0.1, 0.2, 0.3), 0.0) - val expected = "LogisticRegressionModel: uid = logReg, numClasses = 2, numFeatures = 3" + val expected = "LogisticRegressionModel: uid=logReg, numClasses=2, numFeatures=3" assert(model.toString === expected) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 9100ef1db6e1..9e4844ff8907 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -22,15 +22,15 @@ import scala.util.Random import breeze.linalg.{DenseVector => BDV, Vector => BV} import breeze.stats.distributions.{Multinomial => BrzMultinomial, RandBasis => BrzRandBasis} -import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.ml.classification.NaiveBayes.{Bernoulli, Multinomial} +import org.apache.spark.SparkException +import org.apache.spark.ml.classification.NaiveBayes._ import org.apache.spark.ml.classification.NaiveBayesSuite._ import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Dataset, Row} class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @@ -38,6 +38,8 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @transient var dataset: Dataset[_] = _ @transient var bernoulliDataset: Dataset[_] = _ + @transient var gaussianDataset: Dataset[_] = _ + @transient var gaussianDataset2: Dataset[_] = _ private val seed = 42 @@ -53,6 +55,23 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { dataset = generateNaiveBayesInput(pi, theta, 100, seed).toDF() bernoulliDataset = generateNaiveBayesInput(pi, theta, 100, seed, "bernoulli").toDF() + + // theta for gaussian nb + val theta2 = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0: mean + Array(0.10, 0.70, 0.10, 0.10), // label 1: mean + Array(0.10, 0.10, 0.70, 0.10) // label 2: mean + ) + + // sigma for gaussian nb + val sigma = Array( + Array(0.10, 0.10, 0.50, 0.10), // label 0: variance + Array(0.50, 0.10, 0.10, 0.10), // label 1: variance + Array(0.10, 0.10, 0.10, 0.50) // label 2: variance + ) + gaussianDataset = generateGaussianNaiveBayesInput(pi, theta2, sigma, 1000, seed).toDF() + gaussianDataset2 = spark.read.format("libsvm") + .load("../data/mllib/sample_multiclass_classification_data.txt") } def validatePrediction(predictionAndLabels: Seq[Row]): Unit = { @@ -67,10 +86,17 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { def validateModelFit( piData: Vector, thetaData: Matrix, + sigmaData: Matrix, model: NaiveBayesModel): Unit = { assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~== Vectors.dense(piData.toArray.map(math.exp)) absTol 0.05, "pi mismatch") assert(model.theta.map(math.exp) ~== thetaData.map(math.exp) absTol 0.05, "theta mismatch") + if (sigmaData == null) { + assert(model.sigma == null, "sigma mismatch") + } else { + assert(model.sigma.map(math.exp) ~== sigmaData.map(math.exp) absTol 0.05, + "sigma mismatch") + } } def expectedMultinomialProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { @@ -90,6 +116,19 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Vectors.dense(classProbs.map(_ / classProbsSum)) } + def expectedGaussianProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { + val pi = model.pi.toArray.map(math.exp) + val classProbs = pi.indices.map { i => + feature.toArray.zipWithIndex.map { case (v, j) => + val mean = model.theta(i, j) + val variance = model.sigma(i, j) + math.exp(- (v - mean) * (v - mean) / variance / 2) / math.sqrt(variance * math.Pi * 2) + }.product * pi(i) + }.toArray + val classProbsSum = classProbs.sum + Vectors.dense(classProbs.map(_ / classProbsSum)) + } + def validateProbabilities( featureAndProbabilities: Seq[Row], model: NaiveBayesModel, @@ -102,6 +141,8 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { expectedMultinomialProbabilities(model, features) case Bernoulli => expectedBernoulliProbabilities(model, features) + case Gaussian => + expectedGaussianProbabilities(model, features) case _ => throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } @@ -112,12 +153,14 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("model types") { assert(Multinomial === "multinomial") assert(Bernoulli === "bernoulli") + assert(Gaussian === "gaussian") } test("params") { ParamsSuite.checkParams(new NaiveBayes) val model = new NaiveBayesModel("nb", pi = Vectors.dense(Array(0.2, 0.8)), - theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4))) + theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4)), + sigma = null) ParamsSuite.checkParams(model) } @@ -146,7 +189,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("multinomial") val model = nb.fit(testDataset) - validateModelFit(pi, theta, model) + validateModelFit(pi, theta, null, model) assert(model.hasParent) MLTestingUtils.checkCopyAndUids(nb, model) @@ -192,12 +235,17 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("Naive Bayes with weighted samples") { val numClasses = 3 def modelEquals(m1: NaiveBayesModel, m2: NaiveBayesModel): Unit = { + assert(m1.getModelType === m2.getModelType) assert(m1.pi ~== m2.pi relTol 0.01) assert(m1.theta ~== m2.theta relTol 0.01) + if (m1.getModelType == Gaussian) { + assert(m1.sigma ~== m2.sigma relTol 0.01) + } } val testParams = Seq[(String, Dataset[_])]( ("bernoulli", bernoulliDataset), - ("multinomial", dataset) + ("multinomial", dataset), + ("gaussian", gaussianDataset) ) testParams.foreach { case (family, dataset) => // NaiveBayes is sensitive to constant scaling of the weights unless smoothing is set to 0 @@ -228,7 +276,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("bernoulli") val model = nb.fit(testDataset) - validateModelFit(pi, theta, model) + validateModelFit(pi, theta, null, model) assert(model.hasParent) val validationDataset = @@ -308,14 +356,112 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { } } + test("Naive Bayes Gaussian") { + val piArray = Array(0.5, 0.1, 0.4).map(math.log) + + val thetaArray = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0: mean + Array(0.10, 0.70, 0.10, 0.10), // label 1: mean + Array(0.10, 0.10, 0.70, 0.10) // label 2: mean + ) + + val sigmaArray = Array( + Array(0.10, 0.10, 0.50, 0.10), // label 0: variance + Array(0.50, 0.10, 0.10, 0.10), // label 1: variance + Array(0.10, 0.10, 0.10, 0.50) // label 2: variance + ) + + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) + val sigma = new DenseMatrix(3, 4, sigmaArray.flatten, true) + + val nPoints = 10000 + val testDataset = + generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 42).toDF() + val gnb = new NaiveBayes().setModelType("gaussian") + val model = gnb.fit(testDataset) + + validateModelFit(pi, theta, sigma, model) + assert(model.hasParent) + + val validationDataset = + generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 17).toDF() + + val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") + validatePrediction(predictionAndLabels.collect()) + + val featureAndProbabilities = model.transform(validationDataset) + .select("features", "probability") + validateProbabilities(featureAndProbabilities.collect(), model, "gaussian") + } + + test("Naive Bayes Gaussian - Model Coefficients") { + /* + Using the following Python code to verify the correctness. + + import numpy as np + from sklearn.naive_bayes import GaussianNB + from sklearn.datasets import load_svmlight_file + + path = "./data/mllib/sample_multiclass_classification_data.txt" + X, y = load_svmlight_file(path) + X = X.toarray() + clf = GaussianNB() + clf.fit(X, y) + + >>> clf.class_prior_ + array([0.33333333, 0.33333333, 0.33333333]) + >>> clf.theta_ + array([[ 0.27111101, -0.18833335, 0.54305072, 0.60500005], + [-0.60777778, 0.18166667, -0.84271174, -0.88000014], + [-0.09111114, -0.35833336, 0.10508474, 0.0216667 ]]) + >>> clf.sigma_ + array([[0.12230125, 0.07078052, 0.03430001, 0.05133607], + [0.03758145, 0.0988028 , 0.0033903 , 0.00782224], + [0.08058764, 0.06701387, 0.02486641, 0.02661392]]) + */ + + val gnb = new NaiveBayes().setModelType(Gaussian) + val model = gnb.fit(gaussianDataset2) + assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~= + Vectors.dense(0.33333333, 0.33333333, 0.33333333) relTol 1E-5) + + val thetaRows = model.theta.rowIter.toArray + assert(thetaRows(0) ~= + Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005)relTol 1E-5) + assert(thetaRows(1) ~= + Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014)relTol 1E-5) + assert(thetaRows(2) ~= + Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667)relTol 1E-5) + + val sigmaRows = model.sigma.rowIter.toArray + assert(sigmaRows(0) ~= + Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607)relTol 1E-5) + assert(sigmaRows(1) ~= + Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224)relTol 1E-5) + assert(sigmaRows(2) ~= + Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392)relTol 1E-5) + } + test("read/write") { def checkModelData(model: NaiveBayesModel, model2: NaiveBayesModel): Unit = { + assert(model.getModelType === model2.getModelType) assert(model.pi === model2.pi) assert(model.theta === model2.theta) + if (model.getModelType == "gaussian") { + assert(model.sigma === model2.sigma) + } else { + assert(model.sigma === null && model2.sigma === null) + } } val nb = new NaiveBayes() testEstimatorAndModelReadWrite(nb, dataset, NaiveBayesSuite.allParamSettings, NaiveBayesSuite.allParamSettings, checkModelData) + + val gnb = new NaiveBayes().setModelType("gaussian") + testEstimatorAndModelReadWrite(gnb, gaussianDataset, + NaiveBayesSuite.allParamSettingsForGaussian, + NaiveBayesSuite.allParamSettingsForGaussian, checkModelData) } test("should support all NumericType labels and weights, and not support other types") { @@ -324,6 +470,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) + assert(expected.sigma === null && actual.sigma === null) } } } @@ -340,6 +487,16 @@ object NaiveBayesSuite { "smoothing" -> 0.1 ) + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettingsForGaussian: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "modelType" -> "gaussian" + ) + private def calcLabel(p: Double, pi: Array[Double]): Int = { var sum = 0.0 for (j <- 0 until pi.length) { @@ -384,4 +541,26 @@ object NaiveBayesSuite { LabeledPoint(y, Vectors.dense(xi)) } } + + // Generate input + def generateGaussianNaiveBayesInput( + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + sigma: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val D = theta(0).length + val rnd = new Random(seed) + val _pi = pi.map(math.exp) + + for (i <- 0 until nPoints) yield { + val y = calcLabel(rnd.nextDouble(), _pi) + val xi = Array.tabulate[Double] (D) { j => + val mean = theta(y)(j) + val variance = sigma(y)(j) + mean + rnd.nextGaussian() * math.sqrt(variance) + } + LabeledPoint(y, Vectors.dense(xi)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala index 6d0b83e85733..c142f83e0595 100755 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.feature import java.util.Locale +import org.apache.spark.ml.Pipeline import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.sql.{DataFrame, Row} @@ -181,12 +182,19 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { } test("read/write") { - val t = new StopWordsRemover() + val t1 = new StopWordsRemover() .setInputCol("myInputCol") .setOutputCol("myOutputCol") .setStopWords(Array("the", "a")) .setCaseSensitive(true) - testDefaultReadWrite(t) + testDefaultReadWrite(t1) + + val t2 = new StopWordsRemover() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setStopWords(Array("the", "a")) + .setCaseSensitive(true) + testDefaultReadWrite(t2) } test("StopWordsRemover output column already exists") { @@ -199,7 +207,7 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { testTransformerByInterceptingException[(Array[String], Array[String])]( dataSet, remover, - s"requirement failed: Column $outputCol already exists.", + s"requirement failed: Output Column $outputCol already exists.", "expected") } @@ -217,4 +225,123 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { Locale.setDefault(oldDefault) } } + + test("Multiple Columns: StopWordsRemover default") { + val remover = new StopWordsRemover() + .setInputCols(Array("raw1", "raw2")) + .setOutputCols(Array("filtered1", "filtered2")) + val df = Seq( + (Seq("test", "test"), Seq("test1", "test2"), Seq("test", "test"), Seq("test1", "test2")), + (Seq("a", "b", "c", "d"), Seq("a", "b"), Seq("b", "c", "d"), Seq("b")), + (Seq("a", "the", "an"), Seq("the", "an"), Seq(), Seq()), + (Seq("A", "The", "AN"), Seq("A", "The"), Seq(), Seq()), + (Seq(null), Seq(null), Seq(null), Seq(null)), + (Seq(), Seq(), Seq(), Seq()) + ).toDF("raw1", "raw2", "expected1", "expected2") + + remover.transform(df) + .select("filtered1", "expected1", "filtered2", "expected2") + .collect().foreach { + case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + } + } + + test("Multiple Columns: StopWordsRemover with particular stop words list") { + val stopWords = Array("test", "a", "an", "the") + val remover = new StopWordsRemover() + .setInputCols(Array("raw1", "raw2")) + .setOutputCols(Array("filtered1", "filtered2")) + .setStopWords(stopWords) + val df = Seq( + (Seq("test", "test"), Seq("test1", "test2"), Seq(), Seq("test1", "test2")), + (Seq("a", "b", "c", "d"), Seq("a", "b"), Seq("b", "c", "d"), Seq("b")), + (Seq("a", "the", "an"), Seq("a", "the", "test1"), Seq(), Seq("test1")), + (Seq("A", "The", "AN"), Seq("A", "The", "AN"), Seq(), Seq()), + (Seq(null), Seq(null), Seq(null), Seq(null)), + (Seq(), Seq(), Seq(), Seq()) + ).toDF("raw1", "raw2", "expected1", "expected2") + + remover.transform(df) + .select("filtered1", "expected1", "filtered2", "expected2") + .collect().foreach { + case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + } + } + + test("Compare single/multiple column(s) StopWordsRemover in pipeline") { + val df = Seq( + (Seq("test", "test"), Seq("test1", "test2")), + (Seq("a", "b", "c", "d"), Seq("a", "b")), + (Seq("a", "the", "an"), Seq("a", "the", "test1")), + (Seq("A", "The", "AN"), Seq("A", "The", "AN")), + (Seq(null), Seq(null)), + (Seq(), Seq()) + ).toDF("input1", "input2") + + val multiColsRemover = new StopWordsRemover() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("output1", "output2")) + + val plForMultiCols = new Pipeline() + .setStages(Array(multiColsRemover)) + .fit(df) + + val removerForCol1 = new StopWordsRemover() + .setInputCol("input1") + .setOutputCol("output1") + val removerForCol2 = new StopWordsRemover() + .setInputCol("input2") + .setOutputCol("output2") + + val plForSingleCol = new Pipeline() + .setStages(Array(removerForCol1, removerForCol2)) + .fit(df) + + val resultForSingleCol = plForSingleCol.transform(df) + .select("output1", "output2") + .collect() + val resultForMultiCols = plForMultiCols.transform(df) + .select("output1", "output2") + .collect() + + resultForSingleCol.zip(resultForMultiCols).foreach { + case (rowForSingle, rowForMultiCols) => + assert(rowForSingle === rowForMultiCols) + } + } + + test("Multiple Columns: Mismatched sizes of inputCols/outputCols") { + val remover = new StopWordsRemover() + .setInputCols(Array("input1")) + .setOutputCols(Array("result1", "result2")) + val df = Seq( + (Seq("A"), Seq("A")), + (Seq("The", "the"), Seq("The")) + ).toDF("input1", "input2") + intercept[IllegalArgumentException] { + remover.transform(df).count() + } + } + + test("Multiple Columns: Set both of inputCol/inputCols") { + val remover = new StopWordsRemover() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("result1", "result2")) + .setInputCol("input1") + val df = Seq( + (Seq("A"), Seq("A")), + (Seq("The", "the"), Seq("The")) + ).toDF("input1", "input2") + intercept[IllegalArgumentException] { + remover.transform(df).count() + } + } } diff --git a/pom.xml b/pom.xml index 5110285547ab..44593b78c9a0 100644 --- a/pom.xml +++ b/pom.xml @@ -200,9 +200,9 @@ 1.0.0 - 0.12.0 + 0.15.1 ${java.home} @@ -2326,7 +2326,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports - -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true - -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} + -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true 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 d16611f41203..f28fc2153806 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 @@ -321,7 +321,7 @@ private[ui] class StreamingPage(parent: StreamingTab) if (hasStream) { - + Input Rate @@ -351,7 +351,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Scheduling Delay {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "right")}
    +
    Scheduling Delay {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "top")}
    Avg: {schedulingDelay.formattedAvg}
    @@ -361,7 +361,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Processing Time {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "right")}
    +
    Processing Time {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "top")}
    Avg: {processingTime.formattedAvg}
    @@ -371,7 +371,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "right")}
    +
    Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "top")}
    Avg: {totalDelay.formattedAvg}
    diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 286095e4ee0d..0976494b6d09 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming import java.io.File import java.nio.ByteBuffer +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.reflect.ClassTag @@ -87,9 +88,12 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) + val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None)), conf, true) + new LiveListenerBus(conf), None, blockManagerInfo)), + rpcEnv.setupEndpoint("blockmanagerHeartbeat", + new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) storageLevel = StorageLevel.MEMORY_ONLY_SER blockManager = createBlockManager(blockManagerSize, conf) From 7668bd6989e4d2cc5921adf1e9f4765598b2432a Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 25 Nov 2019 23:26:54 +0800 Subject: [PATCH 18/36] ExistsSExec -> ExistsSubqueryExec --- .../scala/org/apache/spark/sql/execution/subquery.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 7b7857360751..637a95b40a7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -175,7 +175,7 @@ case class InSubqueryExec( /** * The physical node of non-correlated EXISTS subquery. */ -case class ExistsExec( +case class ExistsSubqueryExec( plan: BaseSubqueryExec, exprId: ExprId) extends ExecSubqueryExpression { @@ -186,10 +186,10 @@ case class ExistsExec( override def children: Seq[Expression] = Nil override def nullable: Boolean = false override def toString: String = s"EXISTS (${plan.simpleString(SQLConf.get.maxToStringFields)})" - override def withNewPlan(plan: BaseSubqueryExec): ExistsExec = copy(plan = plan) + override def withNewPlan(plan: BaseSubqueryExec): ExistsSubqueryExec = copy(plan = plan) override def semanticEquals(other: Expression): Boolean = other match { - case in: ExistsExec => plan.sameResult(in.plan) + case in: ExistsSubqueryExec => plan.sameResult(in.plan) case _ => false } @@ -235,7 +235,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) case expressions.Exists(sub, children, exprId) => val executedPlan = new QueryExecution(sparkSession, Project(Nil, sub)).executedPlan - ExistsExec(SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) + ExistsSubqueryExec(SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) } } } From a6b8485eb52162faa35f5bba3ea7c85f7cdf9b5c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 26 Nov 2019 07:17:41 +0800 Subject: [PATCH 19/36] Revert "Merge branch 'master' into SPARK-29800" This reverts commit 88f804df753ea8b0c7a61002ff33922f2edbe62e. --- .github/workflows/master.yml | 25 +- LICENSE-binary | 2 +- R/pkg/.lintr | 2 +- R/pkg/R/DataFrame.R | 8 +- R/pkg/R/SQLContext.R | 8 +- R/pkg/R/context.R | 2 +- R/pkg/R/deserialize.R | 2 +- R/pkg/R/group.R | 2 +- R/pkg/R/utils.R | 14 +- R/pkg/inst/worker/worker.R | 2 +- R/pkg/tests/fulltests/test_sparkSQL.R | 11 +- R/pkg/tests/fulltests/test_utils.R | 9 - R/run-tests.sh | 2 +- appveyor.yml | 5 +- .../unsafe/types/CalendarIntervalSuite.java | 30 + .../java/org/apache/spark/ExecutorPlugin.java | 60 + .../apache/spark/ExecutorPluginContext.java | 40 +- .../apache/spark/api/plugin/SparkPlugin.java | 2 +- .../unsafe/sort/UnsafeInMemorySorter.java | 4 - .../ui/static/executorspage-template.html | 15 +- .../apache/spark/ui/static/executorspage.js | 3 +- .../org/apache/spark/ui/static/historypage.js | 1 - .../apache/spark/ui/static/spark-dag-viz.js | 2 +- .../org/apache/spark/ui/static/stagepage.js | 17 +- .../scala/org/apache/spark/SparkConf.scala | 4 +- .../scala/org/apache/spark/SparkEnv.scala | 48 +- .../org/apache/spark/api/r/BaseRRunner.scala | 17 +- .../org/apache/spark/api/r/RRunner.scala | 5 +- .../deploy/history/FsHistoryProvider.scala | 4 - .../deploy/master/ui/ApplicationPage.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 22 +- .../org/apache/spark/executor/Executor.scala | 37 + .../ExecutorPluginSource.scala} | 17 +- .../spark/internal/config/package.scala | 27 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 9 - .../cluster/CoarseGrainedClusterMessage.scala | 2 - .../CoarseGrainedSchedulerBackend.scala | 9 +- .../spark/storage/BlockManagerMaster.scala | 7 - .../storage/BlockManagerMasterEndpoint.scala | 24 +- .../BlockManagerMasterHeartbeatEndpoint.scala | 58 - .../scala/org/apache/spark/ui/UIUtils.scala | 4 +- .../ui/exec/ExecutorThreadDumpPage.scala | 7 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 6 +- .../org/apache/spark/ui/jobs/JobPage.scala | 6 +- .../org/apache/spark/ui/jobs/StageTable.scala | 37 +- .../org/apache/spark/util/ListenerBus.scala | 29 +- .../org/apache/spark/ExecutorPluginSuite.java | 179 +++ .../StandaloneDynamicAllocationSuite.scala | 3 +- .../history/FsHistoryProviderSuite.scala | 50 - .../CoarseGrainedExecutorBackendSuite.scala | 41 +- .../plugin/PluginContainerSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 20 - .../BlockManagerReplicationSuite.scala | 6 +- .../spark/storage/BlockManagerSuite.scala | 14 +- dev/create-release/spark-rm/Dockerfile | 4 +- dev/deps/spark-deps-hadoop-2.7 | 7 +- dev/deps/spark-deps-hadoop-3.2 | 7 +- dev/lint-r | 5 +- dev/lint-r.R | 2 +- dev/pip-sanity-check.py | 2 + dev/run-pip-tests | 23 +- dev/run-tests | 6 +- dev/run-tests-jenkins | 8 +- dev/run-tests-jenkins.py | 3 +- dev/run-tests.py | 12 +- dev/sparktestsupport/shellutils.py | 6 +- docs/README.md | 3 +- docs/_config.yml | 2 +- docs/_plugins/include_example.rb | 6 +- docs/configuration.md | 45 - docs/core-migration-guide.md | 17 +- docs/css/pygments-default.css | 6 +- docs/ml-classification-regression.md | 9 +- docs/sql-keywords.md | 8 +- docs/sql-migration-guide.md | 2 + docs/sql-ref-syntax-aux-show-databases.md | 2 +- docs/sql-ref-syntax-aux-show-tblproperties.md | 3 +- docs/sql-ref-syntax-ddl-alter-view.md | 215 +-- docs/sql-ref-syntax-ddl-create-view.md | 6 +- docs/sql-ref-syntax-ddl-drop-view.md | 1 - docs/sql-ref-syntax-ddl-truncate-table.md | 2 +- docs/ss-migration-guide.md | 1 - .../DecisionTreeClassifier.scala | 3 +- .../ml/classification/GBTClassifier.scala | 3 +- .../spark/ml/classification/LinearSVC.scala | 4 - .../classification/LogisticRegression.scala | 3 +- .../MultilayerPerceptronClassifier.scala | 6 - .../spark/ml/classification/NaiveBayes.scala | 302 +--- .../spark/ml/classification/OneVsRest.scala | 6 - .../RandomForestClassifier.scala | 3 +- .../spark/ml/clustering/BisectingKMeans.scala | 9 - .../spark/ml/clustering/GaussianMixture.scala | 8 - .../apache/spark/ml/clustering/KMeans.scala | 9 - .../org/apache/spark/ml/clustering/LDA.scala | 10 - .../BinaryClassificationEvaluator.scala | 6 - .../ml/evaluation/ClusteringEvaluator.scala | 6 - .../MulticlassClassificationEvaluator.scala | 6 - .../MultilabelClassificationEvaluator.scala | 6 - .../ml/evaluation/RankingEvaluator.scala | 5 - .../ml/evaluation/RegressionEvaluator.scala | 6 - .../apache/spark/ml/feature/Binarizer.scala | 7 - .../feature/BucketedRandomProjectionLSH.scala | 5 - .../apache/spark/ml/feature/Bucketizer.scala | 7 - .../spark/ml/feature/ChiSqSelector.scala | 5 - .../spark/ml/feature/CountVectorizer.scala | 7 +- .../org/apache/spark/ml/feature/DCT.scala | 5 - .../spark/ml/feature/ElementwiseProduct.scala | 6 - .../spark/ml/feature/FeatureHasher.scala | 9 +- .../apache/spark/ml/feature/HashingTF.scala | 5 - .../org/apache/spark/ml/feature/IDF.scala | 6 +- .../org/apache/spark/ml/feature/Imputer.scala | 7 - .../apache/spark/ml/feature/Interaction.scala | 5 - .../spark/ml/feature/MaxAbsScaler.scala | 5 - .../apache/spark/ml/feature/MinHashLSH.scala | 5 - .../spark/ml/feature/MinMaxScaler.scala | 6 - .../org/apache/spark/ml/feature/NGram.scala | 5 - .../apache/spark/ml/feature/Normalizer.scala | 5 - .../spark/ml/feature/OneHotEncoder.scala | 7 - .../org/apache/spark/ml/feature/PCA.scala | 5 - .../ml/feature/PolynomialExpansion.scala | 5 - .../apache/spark/ml/feature/RFormula.scala | 9 +- .../spark/ml/feature/RobustScaler.scala | 6 - .../spark/ml/feature/SQLTransformer.scala | 5 - .../spark/ml/feature/StandardScaler.scala | 6 - .../spark/ml/feature/StopWordsRemover.scala | 68 +- .../spark/ml/feature/StringIndexer.scala | 10 +- .../spark/ml/feature/VectorAssembler.scala | 6 - .../spark/ml/feature/VectorIndexer.scala | 7 +- .../spark/ml/feature/VectorSizeHint.scala | 5 - .../spark/ml/feature/VectorSlicer.scala | 6 - .../apache/spark/ml/feature/Word2Vec.scala | 6 - .../org/apache/spark/ml/fpm/FPGrowth.scala | 5 - .../apache/spark/ml/recommendation/ALS.scala | 11 +- .../ml/regression/AFTSurvivalRegression.scala | 8 - .../ml/regression/DecisionTreeRegressor.scala | 3 +- .../spark/ml/regression/GBTRegressor.scala | 2 +- .../GeneralizedLinearRegression.scala | 6 - .../ml/regression/IsotonicRegression.scala | 8 - .../ml/regression/LinearRegression.scala | 5 - .../ml/regression/RandomForestRegressor.scala | 2 +- .../spark/ml/tuning/CrossValidator.scala | 5 - .../ml/tuning/TrainValidationSplit.scala | 7 +- .../spark/mllib/clustering/KMeans.scala | 4 +- .../LogisticRegressionSuite.scala | 2 +- .../ml/classification/NaiveBayesSuite.scala | 193 +-- .../ml/feature/StopWordsRemoverSuite.scala | 133 +- pom.xml | 8 +- project/MimaExcludes.scala | 11 +- project/SparkBuild.scala | 1 - python/pyspark/context.py | 2 + python/pyspark/ml/classification.py | 47 +- python/pyspark/ml/clustering.py | 8 +- python/pyspark/ml/feature.py | 69 +- python/pyspark/ml/fpm.py | 2 +- python/pyspark/ml/recommendation.py | 2 - python/pyspark/ml/regression.py | 16 +- python/pyspark/ml/tree.py | 6 + python/pyspark/ml/wrapper.py | 3 - python/pyspark/sql/utils.py | 6 +- python/pyspark/version.py | 2 +- python/run-tests | 8 +- python/run-tests.py | 22 +- python/setup.py | 9 +- .../k8s/KubernetesClusterManager.scala | 13 +- .../ClientModeTestsSuite.scala | 27 +- .../backend/minikube/Minikube.scala | 18 +- .../org/apache/spark/deploy/yarn/Client.scala | 6 +- .../YarnCoarseGrainedExecutorBackend.scala | 4 +- .../spark/deploy/yarn/ClientSuite.scala | 14 +- sql/catalyst/pom.xml | 2 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 45 +- .../connector/catalog/SupportsNamespaces.java | 8 +- .../connector/expressions/Expressions.java | 29 +- .../sql/catalyst/analysis/Analyzer.scala | 17 - .../catalyst/analysis/FunctionRegistry.scala | 13 +- .../catalyst/analysis/PostgreSQLDialect.scala | 12 +- .../catalyst/analysis/ResolveCatalogs.scala | 32 +- .../sql/catalyst/analysis/TypeCoercion.scala | 7 +- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../spark/sql/catalyst/expressions/Cast.scala | 36 +- .../sql/catalyst/expressions/TimeWindow.scala | 3 +- .../aggregate/UnevaluableAggs.scala | 8 +- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../expressions/collectionOperations.scala | 48 + .../expressions/complexTypeCreator.scala | 2 +- .../expressions/higherOrderFunctions.scala | 108 -- .../expressions/intervalExpressions.scala | 68 - .../expressions/jsonExpressions.scala | 4 +- .../PostgreCastStringToBoolean.scala | 80 + .../postgreSQL/PostgreCastToBoolean.scala | 83 -- .../sql/catalyst/optimizer/Optimizer.scala | 11 +- .../catalyst/optimizer/finishAnalysis.scala | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 171 +-- .../sql/catalyst/parser/ParseDriver.scala | 13 +- .../plans/logical/basicLogicalOperators.scala | 2 - .../catalyst/plans/logical/statements.scala | 63 +- .../catalyst/plans/logical/v2Commands.scala | 52 +- .../sql/catalyst/util/IntervalUtils.scala | 447 +++--- .../connector/catalog/CatalogManager.scala | 16 +- .../catalog/CatalogV2Implicits.scala | 9 +- .../spark/sql/connector/catalog/V1Table.scala | 7 +- .../connector/expressions/expressions.scala | 20 +- .../apache/spark/sql/internal/SQLConf.scala | 103 +- .../spark/sql/internal/StaticSQLConf.scala | 7 - ...eateTablePartitioningValidationSuite.scala | 16 +- .../ExpressionTypeCheckingSuite.scala | 4 +- .../catalyst/analysis/TypeCoercionSuite.scala | 29 +- .../encoders/ExpressionEncoderSuite.scala | 2 +- .../catalyst/encoders/RowEncoderSuite.scala | 4 +- .../ArithmeticExpressionSuite.scala | 24 +- .../sql/catalyst/expressions/CastSuite.scala | 17 +- .../CollectionExpressionsSuite.scala | 47 +- .../expressions/DateExpressionsSuite.scala | 22 +- .../expressions/DecimalExpressionSuite.scala | 4 +- .../expressions/HashExpressionsSuite.scala | 4 +- .../HigherOrderFunctionsSuite.scala | 50 - .../IntervalExpressionsSuite.scala | 16 +- .../expressions/JsonExpressionsSuite.scala | 36 +- .../expressions/MutableProjectionSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 3 +- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +- .../expressions/UnsafeRowConverterSuite.scala | 3 +- .../expressions/postgreSQL/CastSuite.scala | 74 +- .../sql/catalyst/parser/DDLParserSuite.scala | 127 +- .../catalyst/parser/DataTypeParserSuite.scala | 3 - .../parser/ExpressionParserSuite.scala | 23 +- .../parser/TableIdentifierParserSuite.scala | 2 +- .../catalyst/util/IntervalUtilsSuite.scala | 130 +- .../sql/connector/InMemoryTableCatalog.scala | 14 +- .../catalog/CatalogManagerSuite.scala | 4 +- .../connector/catalog/TableCatalogSuite.scala | 11 +- .../sql/util/TimestampFormatterSuite.scala | 39 - .../IntervalBenchmark-jdk11-results.txt | 48 +- .../benchmarks/IntervalBenchmark-results.txt | 48 +- sql/core/pom.xml | 2 +- .../sql/execution/ui/static/spark-sql-viz.js | 2 +- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../apache/spark/sql/DataFrameWriterV2.scala | 16 +- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../analysis/ResolveSessionCatalog.scala | 45 +- .../spark/sql/execution/HiveResult.scala | 16 +- .../spark/sql/execution/SparkSqlParser.scala | 92 ++ .../spark/sql/execution/SparkStrategies.scala | 5 +- .../aggregate/HashAggregateExec.scala | 34 +- .../sql/execution/arrow/ArrowConverters.scala | 8 +- .../datasources/FileFormatWriter.scala | 6 +- .../datasources/jdbc/JdbcUtils.scala | 2 +- .../v2/AlterNamespaceSetPropertiesExec.scala | 40 - .../datasources/v2/DataSourceV2Strategy.scala | 44 +- .../v2/DescribeNamespaceExec.scala | 62 - .../datasources/v2/DropNamespaceExec.scala | 26 +- .../v2/ShowTablePropertiesExec.scala | 48 - .../exchange/EnsureRequirements.scala | 19 +- .../spark/sql/execution/r/ArrowRRunner.scala | 6 +- .../execution/streaming/GroupStateImpl.scala | 4 +- .../sql/execution/streaming/OffsetSeq.scala | 9 +- .../StreamingSymmetricHashJoinExec.scala | 84 +- .../sql/execution/streaming/Triggers.scala | 3 +- .../state/SymmetricHashJoinStateManager.scala | 251 +--- .../execution/ui/SQLAppStatusListener.scala | 21 +- .../org/apache/spark/sql/functions.scala | 2 +- .../spark/sql/internal/SharedState.scala | 33 +- .../sql/streaming/StreamingQueryManager.scala | 82 +- .../ansi/decimalArithmeticOperations.sql | 32 - .../inputs/ansi/higher-order-functions.sql | 1 - .../sql-tests/inputs/ansi/interval.sql | 201 ++- .../sql-tests/inputs/ansi/literals.sql | 2 - .../inputs/decimalArithmeticOperations.sql | 24 + .../resources/sql-tests/inputs/group-by.sql | 64 + .../inputs/higher-order-functions.sql | 5 + .../resources/sql-tests/inputs/inner-join.sql | 5 - .../inputs/interval-display-iso_8601.sql | 3 - .../inputs/interval-display-sql_standard.sql | 3 - .../sql-tests/inputs/interval-display.sql | 14 - .../resources/sql-tests/inputs/interval.sql | 194 +-- .../resources/sql-tests/inputs/literals.sql | 54 +- .../sql-tests/inputs/postgreSQL/interval.sql | 18 +- .../sql-tests/inputs/postgreSQL/join.sql | 6 - .../sql-tests/inputs/postgreSQL/text.sql | 4 +- .../inputs/subquery/in-subquery/in-joins.sql | 110 -- .../ansi/decimalArithmeticOperations.sql.out | 138 -- .../ansi/higher-order-functions.sql.out | 284 ---- .../sql-tests/results/ansi/interval.sql.out | 1326 ++++------------- .../sql-tests/results/ansi/literals.sql.out | 474 ------ .../decimalArithmeticOperations.sql.out | 129 +- .../sql-tests/results/group-by.sql.out | 210 ++- .../results/higher-order-functions.sql.out | 46 +- .../results/interval-display-iso_8601.sql.out | 21 - .../interval-display-sql_standard.sql.out | 21 - .../results/interval-display.sql.out | 21 - .../sql-tests/results/interval.sql.out | 833 +---------- .../sql-tests/results/literals.sql.out | 488 +++++- .../results/postgreSQL/boolean.sql.out | 60 +- .../results/postgreSQL/interval.sql.out | 187 +-- .../sql-tests/results/postgreSQL/text.sql.out | 8 +- .../subquery/in-subquery/in-joins.sql.out | 300 +--- .../results/udf/udf-group-by.sql.out | 26 +- .../commits/0 | 2 - .../metadata | 1 - .../offsets/0 | 4 - .../state/0/0/left-keyToNumValues/1.delta | Bin 46 -> 0 bytes .../0/0/left-keyWithIndexToValue/1.delta | Bin 46 -> 0 bytes .../state/0/0/right-keyToNumValues/1.delta | Bin 46 -> 0 bytes .../0/0/right-keyWithIndexToValue/1.delta | Bin 46 -> 0 bytes .../state/0/1/left-keyToNumValues/1.delta | Bin 86 -> 0 bytes .../0/1/left-keyWithIndexToValue/1.delta | Bin 92 -> 0 bytes .../state/0/1/right-keyToNumValues/1.delta | Bin 46 -> 0 bytes .../0/1/right-keyWithIndexToValue/1.delta | Bin 46 -> 0 bytes .../state/0/2/left-keyToNumValues/1.delta | Bin 70 -> 0 bytes .../0/2/left-keyWithIndexToValue/1.delta | Bin 72 -> 0 bytes .../state/0/2/right-keyToNumValues/1.delta | Bin 70 -> 0 bytes .../0/2/right-keyWithIndexToValue/1.delta | Bin 72 -> 0 bytes .../state/0/3/left-keyToNumValues/1.delta | Bin 64 -> 0 bytes .../0/3/left-keyWithIndexToValue/1.delta | Bin 73 -> 0 bytes .../state/0/3/right-keyToNumValues/1.delta | Bin 46 -> 0 bytes .../0/3/right-keyWithIndexToValue/1.delta | Bin 46 -> 0 bytes .../state/0/4/left-keyToNumValues/1.delta | Bin 70 -> 0 bytes .../0/4/left-keyWithIndexToValue/1.delta | Bin 73 -> 0 bytes .../state/0/4/right-keyToNumValues/1.delta | Bin 70 -> 0 bytes .../0/4/right-keyWithIndexToValue/1.delta | Bin 73 -> 0 bytes .../spark/sql/DataFrameFunctionsSuite.scala | 94 +- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../apache/spark/sql/DateFunctionsSuite.scala | 10 +- .../org/apache/spark/sql/ExplainSuite.scala | 4 +- .../spark/sql/IntegratedUDFTestUtils.scala | 20 +- .../org/apache/spark/sql/JoinSuite.scala | 9 - .../apache/spark/sql/JsonFunctionsSuite.scala | 10 - .../sql/PostgreSQLDialectQuerySuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 9 - .../apache/spark/sql/SQLQueryTestSuite.scala | 66 +- .../org/apache/spark/sql/SubquerySuite.scala | 254 ++-- .../DataSourceV2DataFrameSuite.scala | 38 - .../sql/connector/DataSourceV2SQLSuite.scala | 256 +--- .../ReduceNumShufflePartitionsSuite.scala | 3 +- .../benchmark/IntervalBenchmark.scala | 3 +- .../execution/command/DDLParserSuite.scala | 84 +- .../command/PlanResolutionSuite.scala | 168 +-- .../execution/datasources/csv/CSVSuite.scala | 12 - .../SymmetricHashJoinStateManagerSuite.scala | 18 +- .../spark/sql/sources/BucketedReadSuite.scala | 31 +- .../spark/sql/sources/InsertSuite.scala | 33 - .../sql/streaming/StreamingJoinSuite.scala | 225 +-- .../StreamingQueryManagerSuite.scala | 134 +- .../sql/streaming/StreamingQuerySuite.scala | 8 +- .../thriftserver/ui/ThriftServerPage.scala | 43 +- .../sql/hive/thriftserver/ui/ToolTips.scala | 7 - .../ThriftServerQueryTestSuite.scala | 30 +- sql/hive/pom.xml | 2 +- .../spark/streaming/ui/StreamingPage.scala | 8 +- .../streaming/ReceivedBlockHandlerSuite.scala | 6 +- 352 files changed, 3778 insertions(+), 9176 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/ExecutorPlugin.java rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala => core/src/main/java/org/apache/spark/ExecutorPluginContext.java (51%) rename core/src/main/scala/org/apache/spark/{TaskOutputFileAlreadyExistException.scala => executor/ExecutorPluginSource.scala} (73%) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala create mode 100644 core/src/test/java/org/apache/spark/ExecutorPluginSuite.java mode change 100755 => 100644 python/setup.py create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display.sql delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display.sql.out delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/commits/0 delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/metadata delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/offsets/0 delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/left-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/left-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/right-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/0/right-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/left-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/left-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/right-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/1/right-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/left-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/left-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/right-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/2/right-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/left-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/left-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/right-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/3/right-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/left-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/left-keyWithIndexToValue/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/right-keyToNumValues/1.delta delete mode 100644 sql/core/src/test/resources/structured-streaming/checkpoint-version-2.4.0-streaming-join/state/0/4/right-keyWithIndexToValue/1.delta diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d2b7dca3684f..5eb00c4aba0f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -50,7 +50,7 @@ jobs: lint: runs-on: ubuntu-latest - name: Linters (Java/Scala/Python), licenses, dependencies + name: Linters steps: - uses: actions/checkout@master - uses: actions/setup-java@v1 @@ -72,26 +72,3 @@ jobs: run: ./dev/check-license - name: Dependencies run: ./dev/test-dependencies.sh - - lintr: - runs-on: ubuntu-latest - name: Linter (R) - steps: - - uses: actions/checkout@master - - uses: actions/setup-java@v1 - with: - java-version: '11' - - name: install R - run: | - echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' | sudo tee -a /etc/apt/sources.list - sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 - sudo apt-get update - sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev - - name: install R packages - run: | - sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" - sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')" - - name: package and install SparkR - run: ./R/install-dev.sh - - name: lint-r - run: ./dev/lint-r diff --git a/LICENSE-binary b/LICENSE-binary index 0b45686f9ba3..6858193515a8 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -502,7 +502,7 @@ com.github.scopt:scopt_2.12 core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js core/src/main/resources/org/apache/spark/ui/static/*dataTables* core/src/main/resources/org/apache/spark/ui/static/graphlib-dot.min.js -core/src/main/resources/org/apache/spark/ui/static/jquery* +ore/src/main/resources/org/apache/spark/ui/static/jquery* core/src/main/resources/org/apache/spark/ui/static/sorttable.js docs/js/vendor/anchor.min.js docs/js/vendor/jquery* diff --git a/R/pkg/.lintr b/R/pkg/.lintr index 67dc1218ea55..c83ad2adfe0e 100644 --- a/R/pkg/.lintr +++ b/R/pkg/.lintr @@ -1,2 +1,2 @@ -linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE), object_usage_linter = NULL, cyclocomp_linter = NULL) +linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE)) exclusions: list("inst/profile/general.R" = 1, "inst/profile/shell.R") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 593d3ca16220..6f3c7c120ba3 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2252,7 +2252,7 @@ setMethod("mutate", # The last column of the same name in the specific columns takes effect deDupCols <- list() - for (i in seq_len(length(cols))) { + for (i in 1:length(cols)) { deDupCols[[ns[[i]]]] <- alias(cols[[i]], ns[[i]]) } @@ -2416,7 +2416,7 @@ setMethod("arrange", # builds a list of columns of type Column # example: [[1]] Column Species ASC # [[2]] Column Petal_Length DESC - jcols <- lapply(seq_len(length(decreasing)), function(i) { + jcols <- lapply(seq_len(length(decreasing)), function(i){ if (decreasing[[i]]) { desc(getColumn(x, by[[i]])) } else { @@ -2749,7 +2749,7 @@ genAliasesForIntersectedCols <- function(x, intersectedColNames, suffix) { col <- getColumn(x, colName) if (colName %in% intersectedColNames) { newJoin <- paste(colName, suffix, sep = "") - if (newJoin %in% allColNames) { + if (newJoin %in% allColNames){ stop("The following column name: ", newJoin, " occurs more than once in the 'DataFrame'.", "Please use different suffixes for the intersected columns.") } @@ -3475,7 +3475,7 @@ setMethod("str", cat(paste0("'", class(object), "': ", length(names), " variables:\n")) if (nrow(localDF) > 0) { - for (i in seq_len(ncol(localDF))) { + for (i in 1 : ncol(localDF)) { # Get the first elements for each column firstElements <- if (types[i] == "character") { diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index f48a334ed676..f27ef4ee28f1 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -166,9 +166,9 @@ writeToFileInArrow <- function(fileName, rdf, numPartitions) { for (rdf_slice in rdf_slices) { batch <- arrow::record_batch(rdf_slice) if (is.null(stream_writer)) { - stream <- arrow::FileOutputStream$create(fileName) + stream <- arrow::FileOutputStream(fileName) schema <- batch$schema - stream_writer <- arrow::RecordBatchStreamWriter$create(stream, schema) + stream_writer <- arrow::RecordBatchStreamWriter(stream, schema) } stream_writer$write_batch(batch) @@ -197,7 +197,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { as.list(schema) } if (is.null(names)) { - names <- lapply(seq_len(length(firstRow)), function(x) { + names <- lapply(1:length(firstRow), function(x) { paste0("_", as.character(x)) }) } @@ -213,7 +213,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { }) types <- lapply(firstRow, infer_type) - fields <- lapply(seq_len(length(firstRow)), function(i) { + fields <- lapply(1:length(firstRow), function(i) { structField(names[[i]], types[[i]], TRUE) }) schema <- do.call(structType, fields) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index d96a287f818a..93ba1307043a 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -416,7 +416,7 @@ spark.getSparkFiles <- function(fileName) { #' @examples #'\dontrun{ #' sparkR.session() -#' doubled <- spark.lapply(1:10, function(x) {2 * x}) +#' doubled <- spark.lapply(1:10, function(x){2 * x}) #'} #' @note spark.lapply since 2.0.0 spark.lapply <- function(list, func) { diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index ca4a6e342d77..a6febb1cbd13 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -242,7 +242,7 @@ readDeserializeInArrow <- function(inputCon) { # for now. dataLen <- readInt(inputCon) arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") - batches <- arrow::RecordBatchStreamReader$create(arrowData)$batches() + batches <- arrow::RecordBatchStreamReader(arrowData)$batches() if (useAsTibble) { as_tibble <- get("as_tibble", envir = asNamespace("arrow")) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 2b7995e1e37f..6e8f4dc3a790 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -162,7 +162,7 @@ methods <- c("avg", "max", "mean", "min", "sum") #' @note pivot since 2.0.0 setMethod("pivot", signature(x = "GroupedData", colname = "character"), - function(x, colname, values = list()) { + function(x, colname, values = list()){ stopifnot(length(colname) == 1) if (length(values) == 0) { result <- callJMethod(x@sgd, "pivot", colname) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index a8c1ddb3dd20..c3501977e64b 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -131,7 +131,7 @@ hashCode <- function(key) { } else { asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) hashC <- 0 - for (k in seq_len(length(asciiVals))) { + for (k in 1:length(asciiVals)) { hashC <- mult31AndAdd(hashC, asciiVals[k]) } as.integer(hashC) @@ -543,14 +543,10 @@ processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F, ifnotfound = list(list(NULL)))[[1]] found <- sapply(funcList, function(func) { - ifelse( - identical(func, obj) && - # Also check if the parent environment is identical to current parent - identical(parent.env(environment(func)), func.env), - TRUE, FALSE) + ifelse(identical(func, obj), TRUE, FALSE) }) if (sum(found) > 0) { - # If function has been examined ignore + # If function has been examined, ignore. break } # Function has not been examined, record it and recursively clean its closure. @@ -728,7 +724,7 @@ assignNewEnv <- function(data) { stopifnot(length(cols) > 0) env <- new.env() - for (i in seq_len(length(cols))) { + for (i in 1:length(cols)) { assign(x = cols[i], value = data[, cols[i], drop = F], envir = env) } env @@ -754,7 +750,7 @@ launchScript <- function(script, combinedArgs, wait = FALSE, stdout = "", stderr if (.Platform$OS.type == "windows") { scriptWithArgs <- paste(script, combinedArgs, sep = " ") # on Windows, intern = F seems to mean output to the console. (documentation on this is missing) - shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) + shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) # nolint } else { # http://stat.ethz.ch/R-manual/R-devel/library/base/html/system2.html # stdout = F means discard output diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 1ef05ea621e8..dfe69b7f4f1f 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -194,7 +194,7 @@ if (isEmpty != 0) { } else { # gapply mode outputs <- list() - for (i in seq_len(length(data))) { + for (i in 1:length(data)) { # Timing reading input data for execution inputElap <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index cb47353d600d..035525a7a849 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -172,7 +172,7 @@ test_that("structField type strings", { typeList <- c(primitiveTypes, complexTypes) typeStrings <- names(typeList) - for (i in seq_along(typeStrings)) { + for (i in seq_along(typeStrings)){ typeString <- typeStrings[i] expected <- typeList[[i]] testField <- structField("_col", typeString) @@ -203,7 +203,7 @@ test_that("structField type strings", { errorList <- c(primitiveErrors, complexErrors) typeStrings <- names(errorList) - for (i in seq_along(typeStrings)) { + for (i in seq_along(typeStrings)){ typeString <- typeStrings[i] expected <- paste0("Unsupported type for SparkDataframe: ", errorList[[i]]) expect_error(structField("_col", typeString), expected) @@ -3238,13 +3238,6 @@ test_that("Histogram", { expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) }) -test_that("dapply() should show error message from R worker", { - df <- createDataFrame(list(list(n = 1))) - expect_error({ - collect(dapply(df, function(x) stop("custom error message"), structType("a double"))) - }, "custom error message") -}) - test_that("dapply() and dapplyCollect() on a DataFrame", { df <- createDataFrame( list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), diff --git a/R/pkg/tests/fulltests/test_utils.R b/R/pkg/tests/fulltests/test_utils.R index c4fcbecee18e..b2b6f34aaa08 100644 --- a/R/pkg/tests/fulltests/test_utils.R +++ b/R/pkg/tests/fulltests/test_utils.R @@ -110,15 +110,6 @@ test_that("cleanClosure on R functions", { actual <- get("y", envir = env, inherits = FALSE) expect_equal(actual, y) - # Test for combination for nested and sequenctial functions in a closure - f1 <- function(x) x + 1 - f2 <- function(x) f1(x) + 2 - userFunc <- function(x) { f1(x); f2(x) } - cUserFuncEnv <- environment(cleanClosure(userFunc)) - expect_equal(length(cUserFuncEnv), 2) - innerCUserFuncEnv <- environment(cUserFuncEnv$f2) - expect_equal(length(innerCUserFuncEnv), 1) - # Test for function (and variable) definitions. f <- function(x) { g <- function(y) { y * 2 } diff --git a/R/run-tests.sh b/R/run-tests.sh index 51ca7d600caf..86bd8aad5f11 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -23,7 +23,7 @@ FAILED=0 LOGFILE=$FWDIR/unit-tests.out rm -f $LOGFILE -SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" --conf spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" --conf spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE FAILED=$((PIPESTATUS[0]||$FAILED)) NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" diff --git a/appveyor.yml b/appveyor.yml index 00c688ba18eb..b36175a787ae 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -42,7 +42,10 @@ install: # Install maven and dependencies - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" + - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" + # Use Arrow R 0.14.1 for now. 0.15.0 seems not working for now. See SPARK-29378. + - cmd: R -e "install.packages(c('assertthat', 'bit64', 'fs', 'purrr', 'R6', 'tidyselect'), repos='https://cloud.r-project.org/')" + - cmd: R -e "install.packages('https://cran.r-project.org/src/contrib/Archive/arrow/arrow_0.14.1.tar.gz', repos=NULL, type='source')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 01bf7eb2438a..6397f26c02f3 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -46,6 +46,36 @@ public void equalsTest() { assertEquals(i1, i6); } + @Test + public void toStringTest() { + CalendarInterval i; + + i = new CalendarInterval(0, 0, 0); + assertEquals("0 seconds", i.toString()); + + i = new CalendarInterval(34, 0, 0); + assertEquals("2 years 10 months", i.toString()); + + i = new CalendarInterval(-34, 0, 0); + assertEquals("-2 years -10 months", i.toString()); + + i = new CalendarInterval(0, 31, 0); + assertEquals("31 days", i.toString()); + + i = new CalendarInterval(0, -31, 0); + assertEquals("-31 days", i.toString()); + + i = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); + assertEquals("3 hours 13 minutes 0.000123 seconds", i.toString()); + + i = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123); + assertEquals("-3 hours -13 minutes -0.000123 seconds", i.toString()); + + i = new CalendarInterval(34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); + assertEquals("2 years 10 months 31 days 3 hours 13 minutes 0.000123 seconds", + i.toString()); + } + @Test public void periodAndDurationTest() { CalendarInterval interval = new CalendarInterval(120, -40, 123456); diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java new file mode 100644 index 000000000000..b25c46266247 --- /dev/null +++ b/core/src/main/java/org/apache/spark/ExecutorPlugin.java @@ -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; + +import org.apache.spark.annotation.DeveloperApi; + +/** + * A plugin which can be automatically instantiated within each Spark executor. Users can specify + * plugins which should be created with the "spark.executor.plugins" configuration. An instance + * of each plugin will be created for every executor, including those created by dynamic allocation, + * before the executor starts running any tasks. + * + * The specific api exposed to the end users still considered to be very unstable. We will + * hopefully be able to keep compatibility by providing default implementations for any methods + * added, but make no guarantees this will always be possible across all Spark releases. + * + * Spark does nothing to verify the plugin is doing legitimate things, or to manage the resources + * it uses. A plugin acquires the same privileges as the user running the task. A bad plugin + * could also interfere with task execution and make the executor fail in unexpected ways. + */ +@DeveloperApi +public interface ExecutorPlugin { + + /** + * Initialize the executor plugin. + * + *

    Each executor will, during its initialization, invoke this method on each + * plugin provided in the spark.executor.plugins configuration. The Spark executor + * will wait on the completion of the execution of the init method.

    + * + *

    Plugins should create threads in their implementation of this method for + * any polling, blocking, or intensive computation.

    + * + * @param pluginContext Context information for the executor where the plugin is running. + */ + default void init(ExecutorPluginContext pluginContext) {} + + /** + * Clean up and terminate this plugin. + * + *

    This function is called during the executor shutdown phase. The executor + * will wait for the plugin to terminate before continuing its own shutdown.

    + */ + default void shutdown() {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala b/core/src/main/java/org/apache/spark/ExecutorPluginContext.java similarity index 51% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala rename to core/src/main/java/org/apache/spark/ExecutorPluginContext.java index a650607d5f12..8f018732b821 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala +++ b/core/src/main/java/org/apache/spark/ExecutorPluginContext.java @@ -15,26 +15,36 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.v2 +package org.apache.spark; -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import com.codahale.metrics.MetricRegistry; +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.annotation.Private; /** - * Physical plan node for renaming a table. + * Encapsulates information about the executor when initializing {@link ExecutorPlugin} instances. */ -case class RenameTableExec( - catalog: TableCatalog, - oldIdent: Identifier, - newIdent: Identifier) extends V2CommandExec { +@DeveloperApi +public class ExecutorPluginContext { - override def output: Seq[Attribute] = Seq.empty + public final MetricRegistry metricRegistry; + public final SparkConf sparkConf; + public final String executorId; + public final String executorHostName; + public final boolean isLocal; - override protected def run(): Seq[InternalRow] = { - catalog.invalidateTable(oldIdent) - catalog.renameTable(oldIdent, newIdent) - - Seq.empty + @Private + public ExecutorPluginContext( + MetricRegistry registry, + SparkConf conf, + String id, + String hostName, + boolean local) { + metricRegistry = registry; + sparkConf = conf; + executorId = id; + executorHostName = hostName; + isLocal = local; } + } diff --git a/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java b/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java index 21ddae37d8a0..a500f5d2188f 100644 --- a/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java +++ b/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java @@ -24,7 +24,7 @@ * A plugin that can be dynamically loaded into a Spark application. *

    * Plugins can be loaded by adding the plugin's class name to the appropriate Spark configuration. - * Check the Spark monitoring guide for details. + * Check the Spark configuration documentation for details. *

    * Plugins have two optional components: a driver-side component, of which a single instance is * created per application, inside the Spark driver. And an executor-side component, of which one diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index e14964d68119..1a9453a8b3e8 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -205,10 +205,6 @@ public long getSortTimeNanos() { } public long getMemoryUsage() { - if (array == null) { - return 0L; - } - return array.size() * 8; } diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 27a4d2cdb5b5..4c06ddf3b31f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -50,12 +50,12 @@

    Summary

    title="Total shuffle bytes and records read (includes both data read locally and data read from remote executors)."> Shuffle Read
    - Shuffle Write - Blacklisted @@ -72,7 +72,7 @@

    Executors

    - Executor ID + Executor ID Address Status @@ -110,11 +110,14 @@

    Executors

    title="Total shuffle bytes and records read (includes both data read locally and data read from remote executors)."> Shuffle Read - + Shuffle Write - Logs - Thread Dump + Logs + Thread Dump diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 090bc72dc927..11d7c77d0c66 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -462,8 +462,7 @@ $(document).ready(function () { {"visible": false, "targets": 5}, {"visible": false, "targets": 6}, {"visible": false, "targets": 9} - ], - "deferRender": true + ] }; execDataTable = $(selector).DataTable(conf); diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index 4df5f07f077d..5f291620e0e9 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -177,7 +177,6 @@ $(document).ready(function() { {name: 'eventLog'}, ], "autoWidth": false, - "deferRender": true }; if (hasMultipleAttempts) { diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index 25dec9d3788b..035d72f8956f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -513,7 +513,7 @@ function addTooltipsForRDDs(svgContainer) { if (tooltipText) { node.select("circle") .attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "bottom") .attr("data-html", "true") // to interpret line break, tooltipText is showing title .attr("title", tooltipText); } diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 4b2f007f866c..f01d030c73a4 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -297,26 +297,26 @@ $(document).ready(function () { "
    "); $('#scheduler_delay').attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "right") .attr("title", "Scheduler delay includes time to ship the task from the scheduler to the executor, and time to send " + "the task result from the executor to the scheduler. If scheduler delay is large, consider decreasing the size of tasks or decreasing the size of task results."); $('#task_deserialization_time').attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "right") .attr("title", "Time spent deserializing the task closure on the executor, including the time to read the broadcasted task."); $('#shuffle_read_blocked_time').attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "right") .attr("title", "Time that the task spent blocked waiting for shuffle data to be read from remote machines."); $('#shuffle_remote_reads').attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "right") .attr("title", "Total shuffle bytes read from remote executors. This is a subset of the shuffle read bytes; the remaining shuffle data is read locally. "); $('#result_serialization_time').attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "right") .attr("title", "Time spent serializing the task result on the executor before sending it back to the driver."); $('#getting_result_time').attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "right") .attr("title", "Time that the driver spends fetching task results from workers. If this is large, consider decreasing the amount of data returned from each task."); $('#peak_execution_memory').attr("data-toggle", "tooltip") - .attr("data-placement", "top") + .attr("data-placement", "right") .attr("title", "Execution memory refers to the memory used by internal data structures created during " + "shuffles, aggregations and joins when Tungsten is enabled. The value of this accumulator " + "should be approximately the sum of the peak sizes across all such data structures created " + @@ -880,8 +880,7 @@ $(document).ready(function () { { "visible": false, "targets": 16 }, { "visible": false, "targets": 17 }, { "visible": false, "targets": 18 } - ], - "deferRender": true + ] }; taskTableSelector = $(taskTable).DataTable(taskConf); $('#active-tasks-table_filter input').unbind(); diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0e0291d2407d..3a2eaae092e8 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -619,9 +619,7 @@ private[spark] object SparkConf extends Logging { "Not used anymore. Please use spark.shuffle.service.index.cache.size"), DeprecatedConfig("spark.yarn.credentials.file.retention.count", "2.4.0", "Not used anymore."), DeprecatedConfig("spark.yarn.credentials.file.retention.days", "2.4.0", "Not used anymore."), - DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available."), - DeprecatedConfig("spark.executor.plugins", "3.0.0", - "Feature replaced with new plugin API. See Monitoring documentation.") + DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available.") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5fe1c663affa..9232938464e0 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -22,7 +22,6 @@ import java.net.Socket import java.util.Locale import scala.collection.JavaConverters._ -import scala.collection.concurrent import scala.collection.mutable import scala.util.Properties @@ -196,7 +195,6 @@ object SparkEnv extends Logging { private[spark] def createExecutorEnv( conf: SparkConf, executorId: String, - bindAddress: String, hostname: String, numCores: Int, ioEncryptionKey: Option[Array[Byte]], @@ -204,7 +202,7 @@ object SparkEnv extends Logging { val env = create( conf, executorId, - bindAddress, + hostname, hostname, None, isLocal, @@ -215,17 +213,6 @@ object SparkEnv extends Logging { env } - private[spark] def createExecutorEnv( - conf: SparkConf, - executorId: String, - hostname: String, - numCores: Int, - ioEncryptionKey: Option[Array[Byte]], - isLocal: Boolean): SparkEnv = { - createExecutorEnv(conf, executorId, hostname, - hostname, numCores, ioEncryptionKey, isLocal) - } - /** * Helper method to create a SparkEnv for a driver or an executor. */ @@ -352,26 +339,19 @@ object SparkEnv extends Logging { None } - // Mapping from block manager id to the block manager's information. - val blockManagerInfo = new concurrent.TrieMap[BlockManagerId, BlockManagerInfo]() - val blockManagerMaster = new BlockManagerMaster( - registerOrLookupEndpoint( - BlockManagerMaster.DRIVER_ENDPOINT_NAME, - new BlockManagerMasterEndpoint( - rpcEnv, - isLocal, - conf, - listenerBus, - if (conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)) { - externalShuffleClient - } else { - None - }, blockManagerInfo)), - registerOrLookupEndpoint( - BlockManagerMaster.DRIVER_HEARTBEAT_ENDPOINT_NAME, - new BlockManagerMasterHeartbeatEndpoint(rpcEnv, isLocal, blockManagerInfo)), - conf, - isDriver) + val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint( + BlockManagerMaster.DRIVER_ENDPOINT_NAME, + new BlockManagerMasterEndpoint( + rpcEnv, + isLocal, + conf, + listenerBus, + if (conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)) { + externalShuffleClient + } else { + None + })), + conf, isDriver) val blockTransferService = new NettyBlockTransferService(conf, securityManager, bindAddress, advertiseAddress, diff --git a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala index fdfe5f5b41d0..d8f9d1f1729b 100644 --- a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala @@ -82,7 +82,12 @@ private[spark] abstract class BaseRRunner[IN, OUT]( serverSocket.close() } - newReaderIterator(dataStream, errThread) + try { + newReaderIterator(dataStream, errThread) + } catch { + case e: Exception => + throw new SparkException("R computation failed with\n " + errThread.getLines(), e) + } } /** @@ -133,16 +138,6 @@ private[spark] abstract class BaseRRunner[IN, OUT]( * and then returns null. */ protected def read(): OUT - - protected val handleException: PartialFunction[Throwable, OUT] = { - case e: Exception => - var msg = "R unexpectedly exited." - val lines = errThread.getLines() - if (lines.trim().nonEmpty) { - msg += s"\nR worker produced errors: $lines\n" - } - throw new SparkException(msg, e) - } } /** diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 20ab6fc2f348..0327386b45ed 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -125,7 +125,10 @@ private[spark] class RRunner[IN, OUT]( eos = true null.asInstanceOf[OUT] } - } catch handleException + } catch { + case eof: EOFException => + throw new SparkException("R worker exited unexpectedly (cranshed)", eof) + } } } } 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 e2f3314bc859..70864d590988 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 @@ -18,7 +18,6 @@ package org.apache.spark.deploy.history import java.io.{File, FileNotFoundException, IOException} -import java.lang.{Long => JLong} import java.nio.file.Files import java.util.{Date, ServiceLoader} import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Future, TimeUnit} @@ -31,7 +30,6 @@ import scala.io.Source import scala.xml.Node import com.fasterxml.jackson.annotation.JsonIgnore -import com.fasterxml.jackson.databind.annotation.JsonDeserialize import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.hdfs.DistributedFileSystem @@ -1169,7 +1167,6 @@ private[history] case class LogInfo( appId: Option[String], attemptId: Option[String], fileSize: Long, - @JsonDeserialize(contentAs = classOf[JLong]) lastIndex: Option[Long], isComplete: Boolean) @@ -1177,7 +1174,6 @@ private[history] class AttemptInfoWrapper( val info: ApplicationAttemptInfo, val logPath: String, val fileSize: Long, - @JsonDeserialize(contentAs = classOf[JLong]) val lastIndex: Option[Long], val adminAcls: Option[String], val viewAcls: Option[String], 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 071b79135d64..c7e73bcc13c5 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 @@ -73,7 +73,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
  • + data-placement="right"> Executor Limit: { if (app.executorLimit == Int.MaxValue) "Unlimited" else app.executorLimit 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 b1837c9c0c9e..b4bca1e9401e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -46,7 +46,6 @@ private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, driverUrl: String, executorId: String, - bindAddress: String, hostname: String, cores: Int, userClassPath: Seq[URL], @@ -127,7 +126,6 @@ private[spark] class CoarseGrainedExecutorBackend( logInfo("Successfully registered with driver") try { executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) - driver.get.send(LaunchedExecutor(executorId)) } catch { case NonFatal(e) => exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) @@ -229,7 +227,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case class Arguments( driverUrl: String, executorId: String, - bindAddress: String, hostname: String, cores: Int, appId: String, @@ -241,7 +238,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, Arguments, SparkEnv) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => new CoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, + arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourcesFileOpt) } run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) @@ -262,12 +259,10 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val executorConf = new SparkConf val fetcher = RpcEnv.create( "driverPropsFetcher", - arguments.bindAddress, arguments.hostname, -1, executorConf, new SecurityManager(executorConf), - numUsableCores = 0, clientMode = true) var driver: RpcEndpointRef = null @@ -302,8 +297,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } driverConf.set(EXECUTOR_ID, arguments.executorId) - val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.bindAddress, - arguments.hostname, arguments.cores, cfg.ioEncryptionKey, isLocal = false) + val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.hostname, + arguments.cores, cfg.ioEncryptionKey, isLocal = false) env.rpcEnv.setupEndpoint("Executor", backendCreateFn(env.rpcEnv, arguments, env)) arguments.workerUrl.foreach { url => @@ -316,7 +311,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { def parseArguments(args: Array[String], classNameForEntry: String): Arguments = { var driverUrl: String = null var executorId: String = null - var bindAddress: String = null var hostname: String = null var cores: Int = 0 var resourcesFileOpt: Option[String] = None @@ -333,9 +327,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case ("--executor-id") :: value :: tail => executorId = value argv = tail - case ("--bind-address") :: value :: tail => - bindAddress = value - argv = tail case ("--hostname") :: value :: tail => hostname = value argv = tail @@ -373,11 +364,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { printUsageAndExit(classNameForEntry) } - if (bindAddress == null) { - bindAddress = hostname - } - - Arguments(driverUrl, executorId, bindAddress, hostname, cores, appId, workerUrl, + Arguments(driverUrl, executorId, hostname, cores, appId, workerUrl, userClassPath, resourcesFileOpt) } @@ -390,7 +377,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | Options are: | --driver-url | --executor-id - | --bind-address | --hostname | --cores | --resourcesFile 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 8cd98e47b8a4..0f595d095a22 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -137,6 +137,35 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) + private val executorPlugins: Seq[ExecutorPlugin] = { + val pluginNames = conf.get(EXECUTOR_PLUGINS) + if (pluginNames.nonEmpty) { + logInfo(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") + + // Plugins need to load using a class loader that includes the executor's user classpath + val pluginList: Seq[ExecutorPlugin] = + Utils.withContextClassLoader(replClassLoader) { + val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) + plugins.foreach { plugin => + val pluginSource = new ExecutorPluginSource(plugin.getClass().getSimpleName()) + val pluginContext = new ExecutorPluginContext(pluginSource.metricRegistry, conf, + executorId, executorHostname, isLocal) + plugin.init(pluginContext) + logInfo("Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + if (pluginSource.metricRegistry.getNames.size() > 0) { + env.metricsSystem.registerSource(pluginSource) + } + } + plugins + } + + logInfo("Finished initializing plugins") + pluginList + } else { + Nil + } + } + // Plugins need to load using a class loader that includes the executor's user classpath private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { PluginContainer(env) @@ -266,6 +295,14 @@ private[spark] class Executor( // Notify plugins that executor is shutting down so they can terminate cleanly Utils.withContextClassLoader(replClassLoader) { + executorPlugins.foreach { plugin => + try { + plugin.shutdown() + } catch { + case e: Exception => + logWarning("Plugin " + plugin.getClass().getCanonicalName() + " shutdown failed", e) + } + } plugins.foreach(_.shutdown()) } if (!isLocal) { diff --git a/core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala similarity index 73% rename from core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala rename to core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala index 68054625bac2..5625e953c5e6 100644 --- a/core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala @@ -15,9 +15,16 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.executor -/** - * Exception thrown when a task cannot write to output file due to the file already exists. - */ -private[spark] class TaskOutputFileAlreadyExistException(error: Throwable) extends Exception(error) +import com.codahale.metrics.MetricRegistry + +import org.apache.spark.metrics.source.Source + +private[spark] +class ExecutorPluginSource(name: String) extends Source { + + override val metricRegistry = new MetricRegistry() + + override val sourceName = name +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 8e8e36dbda94..00acb1ff115f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -606,23 +606,6 @@ package object config { .intConf .createWithDefault(128) - private[spark] val LISTENER_BUS_LOG_SLOW_EVENT_ENABLED = - ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent.enabled") - .internal() - .doc("When enabled, log the event that takes too much time to process. This helps us " + - "discover the event types that cause performance bottlenecks. The time threshold is " + - "controlled by spark.scheduler.listenerbus.logSlowEvent.threshold.") - .booleanConf - .createWithDefault(true) - - private[spark] val LISTENER_BUS_LOG_SLOW_EVENT_TIME_THRESHOLD = - ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent.threshold") - .internal() - .doc("The time threshold of whether a event is considered to be taking too much time to " + - "process. Log the event if spark.scheduler.listenerbus.logSlowEvent.enabled is true.") - .timeConf(TimeUnit.NANOSECONDS) - .createWithDefaultString("1s") - // This property sets the root namespace for metrics reporting private[spark] val METRICS_NAMESPACE = ConfigBuilder("spark.metrics.namespace") .stringConf @@ -1193,6 +1176,16 @@ package object config { .toSequence .createWithDefault(Nil) + private[spark] val EXECUTOR_PLUGINS = + ConfigBuilder("spark.executor.plugins") + .doc("Comma-separated list of class names for \"plugins\" implementing " + + "org.apache.spark.ExecutorPlugin. Plugins have the same privileges as any task " + + "in a Spark executor. They can also interfere with task execution and fail in " + + "unexpected ways. So be sure to only use this for trusted plugins.") + .stringConf + .toSequence + .createWithDefault(Nil) + private[spark] val CLEANER_PERIODIC_GC_INTERVAL = ConfigBuilder("spark.cleaner.periodicGC.interval") .timeConf(TimeUnit.SECONDS) 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 fe3a48440991..c3e1cd8b23f1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -267,7 +267,7 @@ private[spark] class DAGScheduler( executorUpdates: mutable.Map[(Int, Int), ExecutorMetrics]): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, executorUpdates)) - blockManagerMaster.driverHeartbeatEndPoint.askSync[Boolean]( + blockManagerMaster.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(10.minutes, "BlockManagerHeartbeat")) } 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 5c0bc497dd1b..9defbefabb86 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -799,15 +799,6 @@ private[spark] class TaskSetManager( info.id, taskSet.id, tid, ef.description)) return } - if (ef.className == classOf[TaskOutputFileAlreadyExistException].getName) { - // If we can not write to output file in the task, there's no point in trying to - // re-execute it. - logError("Task %s in stage %s (TID %d) can not write to output file: %s; not retrying" - .format(info.id, taskSet.id, tid, ef.description)) - abort("Task %s in stage %s (TID %d) can not write to output file: %s".format( - info.id, taskSet.id, tid, ef.description)) - return - } val key = ef.description val now = clock.getTimeMillis() val (printFull, dupCount) = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 9ce23683245e..a90fff02ac73 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -69,8 +69,6 @@ private[spark] object CoarseGrainedClusterMessages { resources: Map[String, ResourceInformation]) extends CoarseGrainedClusterMessage - case class LaunchedExecutor(executorId: String) extends CoarseGrainedClusterMessage - case class StatusUpdate( executorId: String, taskId: Long, 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 7d9c1c6f96f6..ea045e6280e4 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 @@ -194,12 +194,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // automatically, so try to tell the executor to stop itself. See SPARK-13519. executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) - - case LaunchedExecutor(executorId) => - executorDataMap.get(executorId).foreach { data => - data.freeCores = data.totalCores - } - makeOffers(executorId) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -236,7 +230,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp taskResourceNumParts.getOrElse(v.name, 1))) } val data = new ExecutorData(executorRef, executorAddress, hostname, - 0, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, + cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resourcesInfo) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -255,6 +249,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(true) listenerBus.post( SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) + makeOffers() } case StopDriver => 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 9678c917882c..525304fe3c9d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -30,7 +30,6 @@ import org.apache.spark.util.{RpcUtils, ThreadUtils} private[spark] class BlockManagerMaster( var driverEndpoint: RpcEndpointRef, - var driverHeartbeatEndPoint: RpcEndpointRef, conf: SparkConf, isDriver: Boolean) extends Logging { @@ -231,11 +230,6 @@ class BlockManagerMaster( if (driverEndpoint != null && isDriver) { tell(StopBlockManagerMaster) driverEndpoint = null - if (driverHeartbeatEndPoint.askSync[Boolean](StopBlockManagerMaster)) { - driverHeartbeatEndPoint = null - } else { - logWarning("Failed to stop BlockManagerMasterHeartbeatEndpoint") - } logInfo("BlockManagerMaster stopped") } } @@ -251,5 +245,4 @@ class BlockManagerMaster( private[spark] object BlockManagerMaster { val DRIVER_ENDPOINT_NAME = "BlockManagerMaster" - val DRIVER_HEARTBEAT_ENDPOINT_NAME = "BlockManagerMasterHeartbeat" } 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 7e2027701c33..02d0e1a83490 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -36,7 +36,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} /** - * BlockManagerMasterEndpoint is an [[IsolatedRpcEndpoint]] on the master node to track statuses + * BlockManagerMasterEndpoint is an [[ThreadSafeRpcEndpoint]] on the master node to track statuses * of all slaves' block managers. */ private[spark] @@ -45,10 +45,12 @@ class BlockManagerMasterEndpoint( val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus, - externalBlockStoreClient: Option[ExternalBlockStoreClient], - blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) + externalBlockStoreClient: Option[ExternalBlockStoreClient]) extends IsolatedRpcEndpoint with Logging { + // Mapping from block manager id to the block manager's information. + private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] + // Mapping from external shuffle service block manager id to the block statuses. private val blockStatusByShuffleService = new mutable.HashMap[BlockManagerId, JHashMap[BlockId, BlockStatus]] @@ -142,6 +144,9 @@ class BlockManagerMasterEndpoint( case StopBlockManagerMaster => context.reply(true) stop() + + case BlockManagerHeartbeat(blockManagerId) => + context.reply(heartbeatReceived(blockManagerId)) } private def removeRdd(rddId: Int): Future[Seq[Int]] = { @@ -285,6 +290,19 @@ class BlockManagerMasterEndpoint( blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) } + /** + * Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { + if (!blockManagerInfo.contains(blockManagerId)) { + blockManagerId.isDriver && !isLocal + } else { + blockManagerInfo(blockManagerId).updateLastSeenMs() + true + } + } + // Remove a block from the slaves that have it. This can only be used to remove // blocks that the master knows about. private def removeBlockFromWorkers(blockId: BlockId): Unit = { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala deleted file mode 100644 index b06002123d80..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterHeartbeatEndpoint.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import scala.collection.mutable - -import org.apache.spark.internal.Logging -import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} -import org.apache.spark.storage.BlockManagerMessages.{BlockManagerHeartbeat, StopBlockManagerMaster} - -/** - * Separate heartbeat out of BlockManagerMasterEndpoint due to performance consideration. - */ -private[spark] class BlockManagerMasterHeartbeatEndpoint( - override val rpcEnv: RpcEnv, - isLocal: Boolean, - blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) - extends ThreadSafeRpcEndpoint with Logging { - - override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case BlockManagerHeartbeat(blockManagerId) => - context.reply(heartbeatReceived(blockManagerId)) - - case StopBlockManagerMaster => - stop() - context.reply(true) - - case _ => // do nothing for unexpected events - } - - /** - * Return true if the driver knows about the given block manager. Otherwise, return false, - * indicating that the block manager should re-register. - */ - private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { - if (!blockManagerInfo.contains(blockManagerId)) { - blockManagerId.isDriver && !isLocal - } else { - blockManagerInfo(blockManagerId).updateLastSeenMs() - true - } - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 143303df0d10..6dbe63b564e6 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -227,7 +227,7 @@ private[spark] object UIUtils extends Logging { {tab.name}
  • } - val helpButton: Seq[Node] = helpText.map(tooltip(_, "top")).getOrElse(Seq.empty) + val helpButton: Seq[Node] = helpText.map(tooltip(_, "bottom")).getOrElse(Seq.empty) @@ -428,7 +428,7 @@ private[spark] object UIUtils extends Logging { class="expand-dag-viz" onclick={s"toggleDagViz($forJob);"}> + data-placement="right"> DAG Visualization diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index 77564f48015f..a13037b5e24d 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -89,12 +89,7 @@ private[ui] class ExecutorThreadDumpPage( Thread ID Thread Name Thread State - - - Thread Locks - - + Thread Locks {dumpRows} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 91e9caeae05d..11fcbf1c29c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -123,7 +123,7 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We | 'group': 'executors', | 'start': new Date(${e.addTime.getTime()}), | 'content': '
    Executor ${e.id} added
    ' @@ -139,7 +139,7 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We | 'group': 'executors', | 'start': new Date(${removeTime.getTime()}), | 'content': '
    - + Event Timeline ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 12f1aa25e8d2..f7aca507d6f9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -104,7 +104,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP | 'group': 'executors', | 'start': new Date(${e.addTime.getTime()}), | 'content': '
    Executor ${e.id} added
    ' @@ -120,7 +120,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP | 'group': 'executors', | 'start': new Date(${removeTime.getTime()}), | 'content': '
    - + Event Timeline ++ 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 09a215ba9f03..e24b2f2ec36d 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 @@ -179,18 +179,18 @@ private[ui] class StagePagedTable( // stageHeadersAndCssClasses has three parts: header title, tooltip information, and sortable. // The tooltip information could be None, which indicates it does not have a tooltip. // Otherwise, it has two parts: tooltip text, and position (true for left, false for default). - val stageHeadersAndCssClasses: Seq[(String, String, Boolean)] = - Seq(("Stage Id", null, true)) ++ - {if (isFairScheduler) {Seq(("Pool Name", null, true))} else Seq.empty} ++ + val stageHeadersAndCssClasses: Seq[(String, Option[(String, Boolean)], Boolean)] = + Seq(("Stage Id", None, true)) ++ + {if (isFairScheduler) {Seq(("Pool Name", None, true))} else Seq.empty} ++ Seq( - ("Description", null, true), ("Submitted", null, true), ("Duration", null, true), - ("Tasks: Succeeded/Total", null, false), - ("Input", ToolTips.INPUT, true), - ("Output", ToolTips.OUTPUT, true), - ("Shuffle Read", ToolTips.SHUFFLE_READ, true), - ("Shuffle Write", ToolTips.SHUFFLE_WRITE, true) + ("Description", None, true), ("Submitted", None, true), ("Duration", None, true), + ("Tasks: Succeeded/Total", None, false), + ("Input", Some((ToolTips.INPUT, false)), true), + ("Output", Some((ToolTips.OUTPUT, false)), true), + ("Shuffle Read", Some((ToolTips.SHUFFLE_READ, false)), true), + ("Shuffle Write", Some((ToolTips.SHUFFLE_WRITE, true)), true) ) ++ - {if (isFailedStage) {Seq(("Failure Reason", null, false))} else Seq.empty} + {if (isFailedStage) {Seq(("Failure Reason", None, false))} else Seq.empty} if (!stageHeadersAndCssClasses.filter(_._3).map(_._1).contains(sortColumn)) { throw new IllegalArgumentException(s"Unknown column: $sortColumn") @@ -198,13 +198,22 @@ private[ui] class StagePagedTable( val headerRow: Seq[Node] = { stageHeadersAndCssClasses.map { case (header, tooltip, sortable) => - val headerSpan = if (null != tooltip && !tooltip.isEmpty) { - + val headerSpan = tooltip.map { case (title, left) => + if (left) { + /* Place the shuffle write tooltip on the left (rather than the default position + of on top) because the shuffle write column is the last column on the right side and + the tooltip is wider than the column, so it doesn't fit on top. */ + {header} - } else { + } else { + + {header} + + } + }.getOrElse( {header} - } + ) if (header == sortColumn) { val headerLink = Unparsed( 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 51cd7d1284ff..2e517707ff77 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -25,8 +25,7 @@ import scala.util.control.NonFatal import com.codahale.metrics.Timer -import org.apache.spark.SparkEnv -import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.Logging /** * An event bus which posts events to its listeners. @@ -38,20 +37,6 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { // Marked `private[spark]` for access in tests. private[spark] def listeners = listenersPlusTimers.asScala.map(_._1).asJava - private lazy val env = SparkEnv.get - - private lazy val logSlowEventEnabled = if (env != null) { - env.conf.get(config.LISTENER_BUS_LOG_SLOW_EVENT_ENABLED) - } else { - false - } - - private lazy val logSlowEventThreshold = if (env != null) { - env.conf.get(config.LISTENER_BUS_LOG_SLOW_EVENT_TIME_THRESHOLD) - } else { - Long.MaxValue - } - /** * Returns a CodaHale metrics Timer for measuring the listener's event processing time. * This method is intended to be overridden by subclasses. @@ -110,7 +95,6 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { } else { null } - lazy val listenerName = Utils.getFormattedClassName(listener) try { doPostEvent(listener, event) if (Thread.interrupted()) { @@ -120,17 +104,14 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { } } catch { case ie: InterruptedException => - logError(s"Interrupted while posting to ${listenerName}. Removing that listener.", ie) + logError(s"Interrupted while posting to ${Utils.getFormattedClassName(listener)}. " + + s"Removing that listener.", ie) removeListenerOnError(listener) case NonFatal(e) if !isIgnorableException(e) => - logError(s"Listener ${listenerName} threw an exception", e) + logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e) } finally { if (maybeTimerContext != null) { - val elapsed = maybeTimerContext.stop() - if (logSlowEventEnabled && elapsed > logSlowEventThreshold) { - logInfo(s"Process of event ${event} by listener ${listenerName} took " + - s"${elapsed / 1000000000d}s.") - } + maybeTimerContext.stop() } } } diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java new file mode 100644 index 000000000000..ade13f02bde7 --- /dev/null +++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.MetricRegistry; +import org.apache.spark.api.java.JavaSparkContext; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class ExecutorPluginSuite { + private static final String EXECUTOR_PLUGIN_CONF_NAME = "spark.executor.plugins"; + private static final String testBadPluginName = TestBadShutdownPlugin.class.getName(); + private static final String testPluginName = TestExecutorPlugin.class.getName(); + private static final String testSecondPluginName = TestSecondPlugin.class.getName(); + private static final String testMetricsPluginName = TestMetricsPlugin.class.getName(); + + // Static value modified by testing plugins to ensure plugins loaded correctly. + public static int numSuccessfulPlugins = 0; + + // Static value modified by testing plugins to verify plugins shut down properly. + public static int numSuccessfulTerminations = 0; + + // Static values modified by testing plugins to ensure metrics have been registered correctly. + public static MetricRegistry testMetricRegistry; + public static String gaugeName; + + private JavaSparkContext sc; + + @Before + public void setUp() { + sc = null; + numSuccessfulPlugins = 0; + numSuccessfulTerminations = 0; + } + + @After + public void tearDown() { + if (sc != null) { + sc.stop(); + sc = null; + } + } + + private SparkConf initializeSparkConf(String pluginNames) { + return new SparkConf() + .setMaster("local") + .setAppName("test") + .set(EXECUTOR_PLUGIN_CONF_NAME, pluginNames); + } + + @Test + public void testPluginClassDoesNotExist() { + SparkConf conf = initializeSparkConf("nonexistent.plugin"); + try { + sc = new JavaSparkContext(conf); + fail("No exception thrown for nonexistent plugin"); + } catch (Exception e) { + // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown + assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException")); + } + } + + @Test + public void testAddPlugin() throws InterruptedException { + // Load the sample TestExecutorPlugin, which will change the value of numSuccessfulPlugins + SparkConf conf = initializeSparkConf(testPluginName); + sc = new JavaSparkContext(conf); + assertEquals(1, numSuccessfulPlugins); + sc.stop(); + sc = null; + assertEquals(1, numSuccessfulTerminations); + } + + @Test + public void testAddMultiplePlugins() throws InterruptedException { + // Load two plugins and verify they both execute. + SparkConf conf = initializeSparkConf(testPluginName + "," + testSecondPluginName); + sc = new JavaSparkContext(conf); + assertEquals(2, numSuccessfulPlugins); + sc.stop(); + sc = null; + assertEquals(2, numSuccessfulTerminations); + } + + @Test + public void testPluginShutdownWithException() { + // Verify an exception in one plugin shutdown does not affect the others + String pluginNames = testPluginName + "," + testBadPluginName + "," + testPluginName; + SparkConf conf = initializeSparkConf(pluginNames); + sc = new JavaSparkContext(conf); + assertEquals(3, numSuccessfulPlugins); + sc.stop(); + sc = null; + assertEquals(2, numSuccessfulTerminations); + } + + @Test + public void testPluginMetrics() { + // Verify that a custom metric is registered with the Spark metrics system + gaugeName = "test42"; + SparkConf conf = initializeSparkConf(testMetricsPluginName); + sc = new JavaSparkContext(conf); + assertEquals(1, numSuccessfulPlugins); + assertEquals(gaugeName, testMetricRegistry.getGauges().firstKey()); + sc.stop(); + sc = null; + assertEquals(1, numSuccessfulTerminations); + } + + public static class TestExecutorPlugin implements ExecutorPlugin { + public void init(ExecutorPluginContext pluginContext) { + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + ExecutorPluginSuite.numSuccessfulTerminations++; + } + } + + public static class TestSecondPlugin implements ExecutorPlugin { + public void init(ExecutorPluginContext pluginContext) { + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + ExecutorPluginSuite.numSuccessfulTerminations++; + } + } + + public static class TestBadShutdownPlugin implements ExecutorPlugin { + public void init(ExecutorPluginContext pluginContext) { + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + throw new RuntimeException("This plugin will fail to cleanly shut down"); + } + } + + public static class TestMetricsPlugin implements ExecutorPlugin { + public void init(ExecutorPluginContext myContext) { + MetricRegistry metricRegistry = myContext.metricRegistry; + // Registers a dummy metrics gauge for testing + String gaugeName = ExecutorPluginSuite.gaugeName; + metricRegistry.register(MetricRegistry.name(gaugeName), new Gauge() { + @Override + public Integer getValue() { + return 42; + } + }); + ExecutorPluginSuite.testMetricRegistry = metricRegistry; + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + ExecutorPluginSuite.numSuccessfulTerminations++; + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index dd790b8dbb85..17758783d259 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.config import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster._ -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{LaunchedExecutor, RegisterExecutor, RegisterExecutorFailed} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RegisterExecutorFailed} /** * End-to-end tests for dynamic allocation in standalone mode. @@ -634,7 +634,6 @@ class StandaloneDynamicAllocationSuite Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) - backend.driverEndpoint.send(LaunchedExecutor(id)) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index ed195dd44e91..281e6935de37 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1283,56 +1283,6 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { assert(deserializedOldObj.isComplete === false) } - test("SPARK-29755 LogInfo should be serialized/deserialized by jackson properly") { - def assertSerDe(serializer: KVStoreScalaSerializer, info: LogInfo): Unit = { - val infoAfterSerDe = serializer.deserialize(serializer.serialize(info), classOf[LogInfo]) - assert(infoAfterSerDe === info) - assertOptionAfterSerde(infoAfterSerDe.lastIndex, info.lastIndex) - } - - val serializer = new KVStoreScalaSerializer() - val logInfoWithIndexAsNone = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, None, false) - assertSerDe(serializer, logInfoWithIndexAsNone) - - val logInfoWithIndex = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, Some(3), false) - assertSerDe(serializer, logInfoWithIndex) - } - - test("SPARK-29755 AttemptInfoWrapper should be serialized/deserialized by jackson properly") { - def assertSerDe(serializer: KVStoreScalaSerializer, attempt: AttemptInfoWrapper): Unit = { - val attemptAfterSerDe = serializer.deserialize(serializer.serialize(attempt), - classOf[AttemptInfoWrapper]) - assert(attemptAfterSerDe.info === attempt.info) - // skip comparing some fields, as they've not triggered SPARK-29755 - assertOptionAfterSerde(attemptAfterSerDe.lastIndex, attempt.lastIndex) - } - - val serializer = new KVStoreScalaSerializer() - val appInfo = new ApplicationAttemptInfo(None, new Date(1), new Date(1), new Date(1), - 10, "spark", false, "dummy") - val attemptInfoWithIndexAsNone = new AttemptInfoWrapper(appInfo, "dummyPath", 10, None, - None, None, None, None) - assertSerDe(serializer, attemptInfoWithIndexAsNone) - - val attemptInfoWithIndex = new AttemptInfoWrapper(appInfo, "dummyPath", 10, Some(1), - None, None, None, None) - assertSerDe(serializer, attemptInfoWithIndex) - } - - private def assertOptionAfterSerde(opt: Option[Long], expected: Option[Long]): Unit = { - if (expected.isEmpty) { - assert(opt.isEmpty) - } else { - // The issue happens only when the value in Option is being unboxed. Here we ensure unboxing - // to Long succeeds: even though IDE suggests `.toLong` is redundant, direct comparison - // doesn't trigger unboxing and passes even without SPARK-29755, so don't remove - // `.toLong` below. Please refer SPARK-29755 for more details. - assert(opt.get.toLong === expected.get.toLong) - } - } - /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 7e96039ca924..e40cf0d66c19 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -54,7 +54,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => val testResourceArgs: JObject = ("" -> "") @@ -76,7 +76,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => val ra = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) @@ -101,7 +101,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => @@ -129,7 +129,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) // not enough gpu's on the executor @@ -168,7 +168,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) // executor resources < required @@ -200,7 +200,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) val parsedResources = backend.parseOrFindResources(None) @@ -226,7 +226,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) val ja = Extraction.decompose(Seq(gpuArgs)) @@ -254,7 +254,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val rpcEnv = RpcEnv.create("1", "localhost", 0, conf, securityMgr) val env = createMockEnv(conf, serializer, Some(rpcEnv)) backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", - "host1", "host1", 4, Seq.empty[URL], env, None) + "host1", 4, Seq.empty[URL], env, None) assert(backend.taskResources.isEmpty) val taskId = 1000000 @@ -289,31 +289,6 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite } } - test("SPARK-24203 when bindAddress is not set, it defaults to hostname") { - val args1 = Array( - "--driver-url", "driverurl", - "--executor-id", "1", - "--hostname", "host1", - "--cores", "1", - "--app-id", "app1") - - val arg = CoarseGrainedExecutorBackend.parseArguments(args1, "") - assert(arg.bindAddress == "host1") - } - - test("SPARK-24203 when bindAddress is different, it does not default to hostname") { - val args1 = Array( - "--driver-url", "driverurl", - "--executor-id", "1", - "--hostname", "host1", - "--bind-address", "bindaddress1", - "--cores", "1", - "--app-id", "app1") - - val arg = CoarseGrainedExecutorBackend.parseArguments(args1, "") - assert(arg.bindAddress == "bindaddress1") - } - private def createMockEnv(conf: SparkConf, serializer: JavaSerializer, rpcEnv: Option[RpcEnv] = None): SparkEnv = { val mockEnv = mock[SparkEnv] diff --git a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala index b432253ad80d..24fa01736365 100644 --- a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala @@ -31,7 +31,7 @@ import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} -import org.apache.spark._ +import org.apache.spark.{ExecutorPlugin => _, _} import org.apache.spark.api.plugin._ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 1e3b59f7e97d..c27d50ab66e6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -245,7 +245,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null, null, conf, true) { + val blockManagerMaster = new BlockManagerMaster(null, conf, true) { override def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { blockIds.map { _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). 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 89df5de97c44..441ec6ab6e18 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,6 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.fs.FileAlreadyExistsException import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -1776,23 +1775,4 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.checkSpeculatableTasks(0)) assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) } - - test("TaskOutputFileAlreadyExistException lead to task set abortion") { - sc = new SparkContext("local", "test") - sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = FakeTask.createTaskSet(1) - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - assert(sched.taskSetsFailed.isEmpty) - - val offerResult = manager.resourceOffer("exec1", "host1", ANY) - assert(offerResult.isDefined, - "Expect resource offer on iteration 0 to return a task") - assert(offerResult.get.index === 0) - val reason = new ExceptionFailure( - new TaskOutputFileAlreadyExistException( - new FileAlreadyExistsException("file already exists")), - Seq.empty[AccumulableInfo]) - manager.handleFailedTask(offerResult.get.taskId, TaskState.FAILED, reason) - assert(sched.taskSetsFailed.contains(taskSet.id)) - } } 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 59ace850d0bd..d8f42ea9557d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.storage import java.util.Locale -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.implicitConversions @@ -98,12 +97,9 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite conf.set(STORAGE_CACHED_PEERS_TTL, 10) sc = new SparkContext("local", "test", conf) - val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), - rpcEnv.setupEndpoint("blockmanagerHeartbeat", - new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) + new LiveListenerBus(conf), None)), conf, true) allStores.clear() } 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 8595f73fe5dd..484d246959ec 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,7 +21,6 @@ import java.io.File import java.nio.ByteBuffer import scala.collection.JavaConverters._ -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import scala.concurrent.duration._ @@ -143,13 +142,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // need to create a SparkContext is to initialize LiveListenerBus. sc = mock(classOf[SparkContext]) when(sc.conf).thenReturn(conf) - - val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() - master = spy(new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), - rpcEnv.setupEndpoint("blockmanagerHeartbeat", - new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true)) + master = spy(new BlockManagerMaster( + rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(conf), None)), conf, true)) val initialize = PrivateMethod[Unit](Symbol("initialize")) SizeEstimator invokePrivate initialize() @@ -472,7 +468,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - val reregister = !master.driverHeartbeatEndPoint.askSync[Boolean]( + val reregister = !master.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(store.blockManagerId)) assert(reregister) } diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index fde75495b442..4bfecedbf040 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -34,7 +34,7 @@ ENV DEBCONF_NONINTERACTIVE_SEEN true ARG APT_INSTALL="apt-get install --no-install-recommends -y" ARG BASE_PIP_PKGS="setuptools wheel virtualenv" -ARG PIP_PKGS="pyopenssl pypandoc numpy sphinx" +ARG PIP_PKGS="pyopenssl pypandoc numpy pygments sphinx" # Install extra needed repos and refresh. # - CRAN repo @@ -80,7 +80,7 @@ RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ $APT_INSTALL ruby2.3 ruby2.3-dev mkdocs && \ gem install jekyll --no-rdoc --no-ri -v 3.8.6 && \ gem install jekyll-redirect-from && \ - gem install rogue + gem install pygments.rb WORKDIR /opt/spark-rm/output diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 54608d203133..e6d29d04acbf 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -17,9 +17,9 @@ apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -arrow-format-0.15.1.jar -arrow-memory-0.15.1.jar -arrow-vector-0.15.1.jar +arrow-format-0.12.0.jar +arrow-memory-0.12.0.jar +arrow-vector-0.12.0.jar audience-annotations-0.5.0.jar automaton-1.11-8.jar avro-1.8.2.jar @@ -83,6 +83,7 @@ hadoop-yarn-server-web-proxy-2.7.4.jar hk2-api-2.5.0.jar hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar +hppc-0.7.2.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.10.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 917fde61fad1..8f1e7fe125b9 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -12,9 +12,9 @@ antlr4-runtime-4.7.1.jar aopalliance-1.0.jar aopalliance-repackaged-2.5.0.jar arpack_combined_all-0.1.jar -arrow-format-0.15.1.jar -arrow-memory-0.15.1.jar -arrow-vector-0.15.1.jar +arrow-format-0.12.0.jar +arrow-memory-0.12.0.jar +arrow-vector-0.12.0.jar audience-annotations-0.5.0.jar automaton-1.11-8.jar avro-1.8.2.jar @@ -96,6 +96,7 @@ hive-vector-code-gen-2.3.6.jar hk2-api-2.5.0.jar hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar +hppc-0.7.2.jar htrace-core4-4.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.10.jar diff --git a/dev/lint-r b/dev/lint-r index b08f5efecd5d..bfda0bca15eb 100755 --- a/dev/lint-r +++ b/dev/lint-r @@ -17,9 +17,6 @@ # limitations under the License. # -set -o pipefail -set -e - SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" @@ -27,7 +24,7 @@ LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" if ! type "Rscript" > /dev/null; then echo "ERROR: You should install R" - exit 1 + exit fi `which Rscript` --vanilla "$SPARK_ROOT_DIR/dev/lint-r.R" "$SPARK_ROOT_DIR" | tee "$LINT_R_REPORT_FILE_NAME" diff --git a/dev/lint-r.R b/dev/lint-r.R index 7e165319e316..a4261d266bbc 100644 --- a/dev/lint-r.R +++ b/dev/lint-r.R @@ -27,7 +27,7 @@ if (! library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE)) { # Installs lintr from Github in a local directory. # NOTE: The CRAN's version is too old to adapt to our rules. if ("lintr" %in% row.names(installed.packages()) == FALSE) { - devtools::install_github("jimhester/lintr@v2.0.0") + devtools::install_github("jimhester/lintr@5431140") } library(lintr) diff --git a/dev/pip-sanity-check.py b/dev/pip-sanity-check.py index e9f10233b12b..4171f28684d5 100644 --- a/dev/pip-sanity-check.py +++ b/dev/pip-sanity-check.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + from pyspark.sql import SparkSession from pyspark.mllib.linalg import * import sys diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 1294a9096fb9..60cf4d820941 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -39,16 +39,21 @@ PYTHON_EXECS=() # Some systems don't have pip or virtualenv - in those cases our tests won't work. if hash virtualenv 2>/dev/null && [ ! -n "$USE_CONDA" ]; then echo "virtualenv installed - using. Note if this is a conda virtual env you may wish to set USE_CONDA" - # test only against python3 + # Figure out which Python execs we should test pip installation with + if hash python2 2>/dev/null; then + # We do this since we are testing with virtualenv and the default virtual env python + # is in /usr/bin/python + PYTHON_EXECS+=('python2') + elif hash python 2>/dev/null; then + # If python2 isn't installed fallback to python if available + PYTHON_EXECS+=('python') + fi if hash python3 2>/dev/null; then - PYTHON_EXECS=('python3') - else - echo "Python3 not installed on system, skipping pip installability tests" - exit 0 + PYTHON_EXECS+=('python3') fi elif hash conda 2>/dev/null; then echo "Using conda virtual environments" - PYTHON_EXECS=('3.6') + PYTHON_EXECS=('3.5') USE_CONDA=1 else echo "Missing virtualenv & conda, skipping pip installability tests" @@ -92,7 +97,7 @@ for python in "${PYTHON_EXECS[@]}"; do cd "$FWDIR"/python # Delete the egg info file if it exists, this can cache the setup file. rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" - python3 setup.py sdist + python setup.py sdist echo "Installing dist into virtual env" @@ -112,9 +117,9 @@ for python in "${PYTHON_EXECS[@]}"; do echo "Run basic sanity check on pip installed version with spark-submit" spark-submit "$FWDIR"/dev/pip-sanity-check.py echo "Run basic sanity check with import based" - python3 "$FWDIR"/dev/pip-sanity-check.py + python "$FWDIR"/dev/pip-sanity-check.py echo "Run the tests for context.py" - python3 "$FWDIR"/python/pyspark/context.py + python "$FWDIR"/python/pyspark/context.py cd "$FWDIR" diff --git a/dev/run-tests b/dev/run-tests index 143d78ec6373..9cf93d000d0e 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -20,10 +20,10 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 6, 0))') +PYTHON_VERSION_CHECK=$(python -c 'import sys; print(sys.version_info < (2, 7, 0))') if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then - echo "Python versions prior to 3.6 are not supported." + echo "Python versions prior to 2.7 are not supported." exit -1 fi -exec python3 -u ./dev/run-tests.py "$@" +exec python -u ./dev/run-tests.py "$@" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index c3adc696a512..5bc03e41d1f2 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -25,12 +25,10 @@ FWDIR="$( cd "$( dirname "$0" )/.." && pwd )" cd "$FWDIR" -export PATH=/home/anaconda/envs/py36/bin:$PATH - -PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 6, 0))') +PYTHON_VERSION_CHECK=$(python -c 'import sys; print(sys.version_info < (2, 7, 0))') if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then - echo "Python versions prior to 3.6 are not supported." + echo "Python versions prior to 2.7 are not supported." exit -1 fi -exec python3 -u ./dev/run-tests-jenkins.py "$@" +exec python -u ./dev/run-tests-jenkins.py "$@" diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 5429aeba8ea1..e9b0b327603b 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python3 +#!/usr/bin/env python # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -17,6 +17,7 @@ # limitations under the License. # +from __future__ import print_function import os import sys import json diff --git a/dev/run-tests.py b/dev/run-tests.py index fc8b7251a85f..ea515708124d 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python3 +#!/usr/bin/env python # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -17,6 +17,7 @@ # limitations under the License. # +from __future__ import print_function import itertools from argparse import ArgumentParser import os @@ -43,20 +44,15 @@ def determine_modules_for_files(filenames): """ Given a list of filenames, return the set of modules that contain those files. If a file is not associated with a more specific submodule, then this method will consider that - file to belong to the 'root' module. GitHub Action and Appveyor files are ignored. + file to belong to the 'root' module. >>> sorted(x.name for x in determine_modules_for_files(["python/pyspark/a.py", "sql/core/foo"])) ['pyspark-core', 'sql'] >>> [x.name for x in determine_modules_for_files(["file_not_matched_by_any_subproject"])] ['root'] - >>> [x.name for x in determine_modules_for_files( \ - [".github/workflows/master.yml", "appveyor.yml"])] - [] """ changed_modules = set() for filename in filenames: - if filename in (".github/workflows/master.yml", "appveyor.yml"): - continue matched_at_least_one_module = False for module in modules.all_modules: if module.contains_file(filename): @@ -269,7 +265,7 @@ def exec_sbt(sbt_args=()): echo_proc.wait() for line in iter(sbt_proc.stdout.readline, b''): if not sbt_output_filter.match(line): - print(line.decode('utf-8'), end='') + print(line, end='') retcode = sbt_proc.wait() if retcode != 0: diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index d9cb8aa45c8d..ec6ea86269f5 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -15,12 +15,14 @@ # limitations under the License. # +from __future__ import print_function import os import shutil import subprocess import sys subprocess_check_output = subprocess.check_output +subprocess_check_call = subprocess.check_call def exit_from_command_with_retcode(cmd, retcode): @@ -53,9 +55,9 @@ def run_cmd(cmd, return_output=False): cmd = cmd.split() try: if return_output: - return subprocess_check_output(cmd).decode('utf-8') + return subprocess_check_output(cmd).decode(sys.getdefaultencoding()) else: - return subprocess.run(cmd, universal_newlines=True, check=True) + return subprocess_check_call(cmd) except subprocess.CalledProcessError as e: exit_from_command_with_retcode(e.cmd, e.returncode) diff --git a/docs/README.md b/docs/README.md index 0bb1ada7167d..da531321aa5d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -36,7 +36,8 @@ You need to have [Ruby](https://www.ruby-lang.org/en/documentation/installation/ installed. Also install the following libraries: ```sh -$ sudo gem install jekyll jekyll-redirect-from rouge +$ sudo gem install jekyll jekyll-redirect-from pygments.rb +$ sudo pip install Pygments # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' diff --git a/docs/_config.yml b/docs/_config.yml index a88862013920..57b8d716ee55 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,4 +1,4 @@ -highlighter: rouge +highlighter: pygments markdown: kramdown gems: - jekyll-redirect-from diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb index 8a3af36e9230..1e91f12518e0 100644 --- a/docs/_plugins/include_example.rb +++ b/docs/_plugins/include_example.rb @@ -16,7 +16,7 @@ # require 'liquid' -require 'rouge' +require 'pygments' module Jekyll class IncludeExampleTag < Liquid::Tag @@ -56,9 +56,7 @@ def render(context) end code = select_lines(code) - formatter = Rouge::Formatters::HTML.new - lexer = Rouge::Lexer.find(@lang) - rendered_code = formatter.format(lexer.lex(code)) + rendered_code = Pygments.highlight(code, :lexer => @lang) hint = "
    Find full example code at " \ "\"examples/src/main/#{snippet_file}\" in the Spark repo.
    " diff --git a/docs/configuration.md b/docs/configuration.md index 0c7cc6022eb0..97ea1fb4ba04 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1857,51 +1857,6 @@ Apart from these, the following properties are also available, and may be useful driver using more memory. - - spark.scheduler.listenerbus.eventqueue.shared.capacity - spark.scheduler.listenerbus.eventqueue.capacity - - Capacity for shared event queue in Spark listener bus, which hold events for external listener(s) - that register to the listener bus. Consider increasing value, if the listener events corresponding - to shared queue are dropped. Increasing this value may result in the driver using more memory. - - - - spark.scheduler.listenerbus.eventqueue.appStatus.capacity - spark.scheduler.listenerbus.eventqueue.capacity - - Capacity for appStatus event queue, which hold events for internal application status listeners. - Consider increasing value, if the listener events corresponding to appStatus queue are dropped. - Increasing this value may result in the driver using more memory. - - - - spark.scheduler.listenerbus.eventqueue.executorManagement.capacity - spark.scheduler.listenerbus.eventqueue.capacity - - Capacity for executorManagement event queue in Spark listener bus, which hold events for internal - executor management listeners. Consider increasing value if the listener events corresponding to - executorManagement queue are dropped. Increasing this value may result in the driver using more memory. - - - - spark.scheduler.listenerbus.eventqueue.eventLog.capacity - spark.scheduler.listenerbus.eventqueue.capacity - - Capacity for eventLog queue in Spark listener bus, which hold events for Event logging listeners - that write events to eventLogs. Consider increasing value if the listener events corresponding to eventLog queue - are dropped. Increasing this value may result in the driver using more memory. - - - - spark.scheduler.listenerbus.eventqueue.streams.capacity - spark.scheduler.listenerbus.eventqueue.capacity - - Capacity for streams queue in Spark listener bus, which hold events for internal streaming listener. - Consider increasing value if the listener events corresponding to streams queue are dropped. Increasing - this value may result in the driver using more memory. - - spark.scheduler.blacklist.unschedulableTaskSetTimeout 120s diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 17d071d0779b..2d4d91dab075 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -9,9 +9,9 @@ license: | 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. @@ -24,15 +24,10 @@ license: | ## Upgrading from Core 2.4 to 3.0 -- The `org.apache.spark.ExecutorPlugin` interface and related configuration has been replaced with - `org.apache.spark.plugin.SparkPlugin`, which adds new functionality. Plugins using the old - interface need to be modified to extend the new interfaces. Check the - [Monitoring](monitoring.html) guide for more details. - -- Deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. +- In Spark 3.0, deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. -- Deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. +- In Spark 3.0, deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. -- Deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. +- In Spark 3.0, deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. -- Event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. +- In Spark 3.0, event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. \ No newline at end of file diff --git a/docs/css/pygments-default.css b/docs/css/pygments-default.css index 0d62ad1a018a..a4d583b36660 100644 --- a/docs/css/pygments-default.css +++ b/docs/css/pygments-default.css @@ -11,10 +11,6 @@ Also, I was thrown off for a while at first when I was using markdown code block inside my {% highlight scala %} ... {% endhighlight %} tags (I was using 4 spaces for this), when it turns out that pygments will insert the code (or pre?) tags for you. - -Note that due to Python 3 compatibility in the project, now we use -Rouge which claims Pygments compatibility, instead of pygments.rb which -does not support Python 3. See SPARK-28752. */ .hll { background-color: #ffffcc } @@ -77,4 +73,4 @@ does not support Python 3. See SPARK-28752. .vc { color: #bb60d5 } /* Name.Variable.Class */ .vg { color: #bb60d5 } /* Name.Variable.Global */ .vi { color: #bb60d5 } /* Name.Variable.Instance */ -.il { color: #40a070 } /* Literal.Number.Integer.Long */ +.il { color: #40a070 } /* Literal.Number.Integer.Long */ \ No newline at end of file diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index d8c7d8a72962..b83b4ba08a5f 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -478,16 +478,15 @@ it computes the conditional probability distribution of each feature given each For prediction, it applies Bayes' theorem to compute the conditional probability distribution of each label given an observation. -MLlib supports [Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), -[Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html) -and [Gaussian naive Bayes](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Gaussian_naive_Bayes). +MLlib supports both [multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). *Input data*: -These Multinomial and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +These models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each feature represents a term. A feature's value is the frequency of the term (in multinomial Naive Bayes) or a zero or one indicating whether the term was found in the document (in Bernoulli Naive Bayes). -Feature values for Multinomial and Bernoulli models must be *non-negative*. The model type is selected with an optional parameter +Feature values must be *non-negative*. The model type is selected with an optional parameter "multinomial" or "bernoulli" with "multinomial" as the default. For document classification, the input feature vectors should usually be sparse vectors. Since the training data is only used once, it is not necessary to cache it. diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 79bc13459623..81d7ce37af17 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,15 +19,15 @@ license: | limitations under the License. --- -When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. +When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.ansi.enabled=true`. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. -By default `spark.sql.dialect.spark.ansi.enabled` is false. +By default `spark.sql.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 2d5afa919e66..153e68b58e74 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -217,6 +217,8 @@ license: | For example `SELECT timestamp 'tomorrow';`. - Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`. + + - Since Spark 3.0, when `array` function is called without parameters, it returns an empty array with `NullType` data type. In Spark version 2.4 and earlier, the data type of the result is `StringType`. - Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. diff --git a/docs/sql-ref-syntax-aux-show-databases.md b/docs/sql-ref-syntax-aux-show-databases.md index efa32f291514..39121f764de2 100644 --- a/docs/sql-ref-syntax-aux-show-databases.md +++ b/docs/sql-ref-syntax-aux-show-databases.md @@ -74,6 +74,6 @@ SHOW SCHEMAS; +------------+ {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) +- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-databases.html) - [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) - [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) diff --git a/docs/sql-ref-syntax-aux-show-tblproperties.md b/docs/sql-ref-syntax-aux-show-tblproperties.md index 212db05017ad..e0738316e97b 100644 --- a/docs/sql-ref-syntax-aux-show-tblproperties.md +++ b/docs/sql-ref-syntax-aux-show-tblproperties.md @@ -111,5 +111,4 @@ SHOW TBLPROPERTIES customer ('created.date'); ### Related Statements - [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) - [ALTER TABLE SET TBLPROPERTIES](sql-ref-syntax-ddl-alter-table.html) -- [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) -- [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) +- [SHOW TABLE](sql-ref-syntax-aux-show-table.html) diff --git a/docs/sql-ref-syntax-ddl-alter-view.md b/docs/sql-ref-syntax-ddl-alter-view.md index d5cbb86d4882..7a7947a3e0ac 100644 --- a/docs/sql-ref-syntax-ddl-alter-view.md +++ b/docs/sql-ref-syntax-ddl-alter-view.md @@ -19,217 +19,4 @@ license: | limitations under the License. --- -### Description - -The `ALTER VIEW` statement can alter metadata associated with the view. It can change the definition of the view, change -the name of a view to a different name, set and unset the metadata of the view by setting `TBLPROPERTIES`. - -#### RENAME View -Renames the existing view. If the new view name already exists in the source database, a `TableAlreadyExistsException` is thrown. This operation -does not support moving the views across databases. - -#### Syntax -{% highlight sql %} -ALTER VIEW view_identifier RENAME TO view_identifier -{% endhighlight %} - -#### Parameters -
    -
    view_identifier
    -
    - Specifies a view name, which may be optionally qualified with a database name.

    - Syntax: - - [database_name.]view_name - -
    -
    - -#### SET View Properties -Set one or more properties of an existing view. The properties are the key value pairs. If the properties' keys exist, -the values are replaced with the new values. If the properties' keys do not exist, the key value pairs are added into -the properties. - -#### Syntax -{% highlight sql %} -ALTER VIEW view_identifier SET TBLPROPERTIES (property_key=property_val [, ...]) -{% endhighlight %} - -#### Parameters -
    -
    view_identifier
    -
    - Specifies a view name, which may be optionally qualified with a database name.

    - Syntax: - - [database_name.]view_name - -
    -
    property_key
    -
    - Specifies the property key. The key may consists of multiple parts separated by dot.

    - Syntax: - - [key_part1][.key_part2][...] - -
    -
    - -#### UNSET View Properties -Drop one or more properties of an existing view. If the specified keys do not exist, an exception is thrown. Use -`IF EXISTS` to avoid the exception. - -#### Syntax -{% highlight sql %} -ALTER VIEW view_identifier UNSET TBLPROPERTIES [IF EXISTS] (property_key [, ...]) -{% endhighlight %} - -#### Parameters -
    -
    view_identifier
    -
    - Specifies a view name, which may be optionally qualified with a database name.

    - Syntax: - - [database_name.]view_name - -
    -
    property_key
    -
    - Specifies the property key. The key may consists of multiple parts separated by dot.

    - Syntax: - - [key_part1][.key_part2][...] - -
    -
    - -#### ALTER View AS SELECT -`ALTER VIEW view_identifier AS SELECT` statement changes the definition of a view, the `SELECT` statement must be valid, -and the `view_identifier` must exist. - -#### Syntax -{% highlight sql %} -ALTER VIEW view_identifier AS select_statement -{% endhighlight %} - -#### Parameters -
    -
    view_identifier
    -
    - Specifies a view name, which may be optionally qualified with a database name.

    - Syntax: - - [database_name.]view_name - -
    -
    select_statement
    -
    - Specifies the definition of the view, detail check select_statement -
    -
    - -### Examples - -{% highlight sql %} --- Rename only changes the view name. --- The source and target databases of the view have to be the same. --- Use qualified or unqualified name for the source and target view -ALTER VIEW tempdb1.v1 RENAME TO tempdb1.v2; - --- Verify that the new view is created. -DESCRIBE TABLE EXTENDED tempdb1.v2; - -+----------------------------+----------+-------+ -|col_name |data_type |comment| -+----------------------------+----------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -+----------------------------+----------+-------+ - --- Before ALTER VIEW SET TBLPROPERTIES -DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+----------+-------+ -|col_name |data_type |comment| -+----------------------------+----------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Table Properties |[....] | | -+----------------------------+----------+-------+ - --- Set properties in TBLPROPERTIES -ALTER VIEW tempdb1.v2 SET TBLPROPERTIES ('created.by.user' = "John", 'created.date' = '01-01-2001' ); - --- Use `DESCRIBE TABLE EXTENDED tempdb1.v2` to verify -DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+-----------------------------------------------------+-------+ -|col_name |data_type |comment| -+----------------------------+-----------------------------------------------------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Table Properties |[created.by.user=John, created.date=01-01-2001, ....]| | -+----------------------------+-----------------------------------------------------+-------+ - --- Remove the key `created.by.user` and `created.date` from `TBLPROPERTIES` -ALTER VIEW tempdb1.v2 UNSET TBLPROPERTIES ('created.by.user', 'created.date'); - ---Use `DESC TABLE EXTENDED tempdb1.v2` to verify the changes -DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+----------+-------+ -|col_name |data_type |comment| -+----------------------------+----------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Table Properties |[....] | | -+----------------------------+----------+-------+ - --- Change the view definition -ALTER VIEW tempdb1.v2 AS SELECT * FROM tempdb1.v1; - --- Use `DESC TABLE EXTENDED` to verify -DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+---------------------------+-------+ -|col_name |data_type |comment| -+----------------------------+---------------------------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Type |VIEW | | -|View Text |select * from tempdb1.v1 | | -|View Original Text |select * from tempdb1.v1 | | -+----------------------------+---------------------------+-------+ -{% endhighlight %} - -### Related Statements - -- [describe-table](sql-ref-syntax-aux-describe-table.html) -- [create-view](sql-ref-syntax-ddl-create-view.html) -- [drop-view](sql-ref-syntax-ddl-drop-view.html) - -#### Note: - -`ALTER VIEW` statement does not support `SET SERDE` or `SET SERDEPROPERTIES` properties - +**This page is under construction** diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index 16310caca0bd..c7ca28ea5b62 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -58,7 +58,7 @@ CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_nam
    query
    -
    A SELECT statement that constructs the view from base tables or other views.
    +
    A SELECT statement that constructs the view from base tables or other views.
    ### Examples @@ -78,5 +78,5 @@ CREATE GLOBAL TEMPORARY VIEW IF NOT EXISTS subscribed_movies {% endhighlight %} ### Related Statements -- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) -- [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) +- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.md) +- [DROP VIEW](sql-ref-syntax-ddl-drop-view.md) diff --git a/docs/sql-ref-syntax-ddl-drop-view.md b/docs/sql-ref-syntax-ddl-drop-view.md index bf99150048d4..f095a3456772 100644 --- a/docs/sql-ref-syntax-ddl-drop-view.md +++ b/docs/sql-ref-syntax-ddl-drop-view.md @@ -77,6 +77,5 @@ DROP VIEW IF EXISTS employeeView; ### Related Statements - [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) -- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) - [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) - [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md index 7baaefe317ba..58d1b9618f0a 100644 --- a/docs/sql-ref-syntax-ddl-truncate-table.md +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -82,5 +82,5 @@ No rows selected ### Related Statements - [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) -- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) +- [ALTER TABLE](sql-ref-syntax-ddl-alter-tabley.html) diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md index db8fdff8b2ac..b0fd8a8325df 100644 --- a/docs/ss-migration-guide.md +++ b/docs/ss-migration-guide.md @@ -30,4 +30,3 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide. - In Spark 3.0, Structured Streaming forces the source schema into nullable when file-based datasources such as text, json, csv, parquet and orc are used via `spark.readStream(...)`. Previously, it respected the nullability in source schema; however, it caused issues tricky to debug with NPE. To restore the previous behavior, set `spark.sql.streaming.fileSource.schema.forceNullable` to `false`. -- Spark 3.0 fixes the correctness issue on Stream-stream outer join, which changes the schema of state. (SPARK-26154 for more details) Spark 3.0 will fail the query if you start your query from checkpoint constructed from Spark 2.x which uses stream-stream outer join. Please discard the checkpoint and replay previous inputs to recalculate outputs. \ No newline at end of file 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 index e02109375373..2d0212f36fad 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -237,8 +237,7 @@ class DecisionTreeClassificationModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"DecisionTreeClassificationModel: uid=$uid, depth=$depth, numNodes=$numNodes, " + - s"numClasses=$numClasses, numFeatures=$numFeatures" + s"DecisionTreeClassificationModel (uid=$uid) of depth $depth with $numNodes nodes" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index e1f5338f3489..5bc45f2b02a4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -340,8 +340,7 @@ class GBTClassificationModel private[ml]( @Since("1.4.0") override def toString: String = { - s"GBTClassificationModel: uid = $uid, numTrees=$numTrees, numClasses=$numClasses, " + - s"numFeatures=$numFeatures" + s"GBTClassificationModel (uid=$uid) with $numTrees trees" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 45114f6ee80b..0dc1c24570b3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -326,10 +326,6 @@ class LinearSVCModel private[classification] ( @Since("2.2.0") override def write: MLWriter = new LinearSVCModel.LinearSVCWriter(this) - @Since("3.0.0") - override def toString: String = { - s"LinearSVCModel: uid=$uid, numClasses=$numClasses, numFeatures=$numFeatures" - } } 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 1cb591523968..1d1d139ba054 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 @@ -1181,7 +1181,8 @@ class LogisticRegressionModel private[spark] ( override def write: MLWriter = new LogisticRegressionModel.LogisticRegressionModelWriter(this) override def toString: String = { - s"LogisticRegressionModel: uid=$uid, numClasses=$numClasses, numFeatures=$numFeatures" + s"LogisticRegressionModel: " + + s"uid = ${super.toString}, numClasses = $numClasses, numFeatures = $numFeatures" } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 8c5d768044ac..41db6f3f4434 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -323,12 +323,6 @@ class MultilayerPerceptronClassificationModel private[ml] ( override protected def predictRaw(features: Vector): Vector = mlpModel.predictRaw(features) override def numClasses: Int = layers.last - - @Since("3.0.0") - override def toString: String = { - s"MultilayerPerceptronClassificationModel: uid=$uid, numLayers=${layers.length}, " + - s"numClasses=$numClasses, numFeatures=$numFeatures" - } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 806287079441..205f565aa268 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -18,22 +18,18 @@ package org.apache.spark.ml.classification import org.apache.hadoop.fs.Path -import org.json4s.DefaultFormats -import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.HasWeightCol -import org.apache.spark.ml.stat.Summarizer import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types._ -import org.apache.spark.util.VersionUtils +import org.apache.spark.sql.functions.col /** * Params for Naive Bayes Classifiers. @@ -53,13 +49,12 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** * The model type which is a string (case-sensitive). - * Supported options: "multinomial", "bernoulli", "gaussian". + * Supported options: "multinomial" and "bernoulli". * (default = multinomial) * @group param */ final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + - "which is a string (case-sensitive). Supported options: multinomial (default), bernoulli" + - " and gaussian.", + "which is a string (case-sensitive). Supported options: multinomial (default) and bernoulli.", ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) /** @group getParam */ @@ -77,11 +72,7 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW * binary (0/1) data, it can also be used as Bernoulli NB * (see * here). - * The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. - * Since 3.0.0, it also supports Gaussian NB - * (see - * here) - * which can handle continuous data. + * The input feature values must be nonnegative. */ // scalastyle:on line.size.limit @Since("1.5.0") @@ -112,7 +103,7 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setModelType(value: String): this.type = set(modelType, value) - setDefault(modelType -> Multinomial) + setDefault(modelType -> NaiveBayes.Multinomial) /** * Sets the value of param [[weightCol]]. @@ -139,9 +130,6 @@ class NaiveBayes @Since("1.5.0") ( positiveLabel: Boolean): NaiveBayesModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, - probabilityCol, modelType, smoothing, thresholds) - if (positiveLabel && isDefined(thresholds)) { val numClasses = getNumClasses(dataset) instr.logNumClasses(numClasses) @@ -150,55 +138,44 @@ class NaiveBayes @Since("1.5.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - $(modelType) match { - case Bernoulli | Multinomial => - trainDiscreteImpl(dataset, instr) - case Gaussian => - trainGaussianImpl(dataset, instr) + val validateInstance = $(modelType) match { + case Multinomial => + (instance: Instance) => requireNonnegativeValues(instance.features) + case Bernoulli => + (instance: Instance) => requireZeroOneBernoulliValues(instance.features) case _ => // This should never happen. throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } - } - private def trainDiscreteImpl( - dataset: Dataset[_], - instr: Instrumentation): NaiveBayesModel = { - val spark = dataset.sparkSession - import spark.implicits._ - - val validateUDF = $(modelType) match { - case Multinomial => - udf { vector: Vector => requireNonnegativeValues(vector); vector } - case Bernoulli => - udf { vector: Vector => requireZeroOneBernoulliValues(vector); vector } - } + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, + probabilityCol, modelType, smoothing, thresholds) - val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { - col($(weightCol)).cast(DoubleType) - } else { - lit(1.0) - } + val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size + instr.logNumFeatures(numFeatures) // Aggregates term frequencies per label. - // TODO: Summarizer directly returns sum vector. - val aggregated = dataset.groupBy(col($(labelCol))) - .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "count") - .summary(validateUDF(col($(featuresCol))), w).as("summary")) - .select($(labelCol), "weightSum", "summary.mean", "summary.count") - .as[(Double, Double, Vector, Long)] - .map { case (label, weightSum, mean, count) => - BLAS.scal(weightSum, mean) - (label, weightSum, mean, count) - }.collect().sortBy(_._1) - - val numFeatures = aggregated.head._3.size - instr.logNumFeatures(numFeatures) - val numSamples = aggregated.map(_._4).sum + // TODO: Calling aggregateByKey and collect creates two stages, we can implement something + // TODO: similar to reduceByKeyLocally to save one stage. + val aggregated = extractInstances(dataset, validateInstance).map { instance => + (instance.label, (instance.weight, instance.features)) + }.aggregateByKey[(Double, DenseVector, Long)]((0.0, Vectors.zeros(numFeatures).toDense, 0L))( + seqOp = { + case ((weightSum, featureSum, count), (weight, features)) => + BLAS.axpy(weight, features, featureSum) + (weightSum + weight, featureSum, count + 1) + }, + combOp = { + case ((weightSum1, featureSum1, count1), (weightSum2, featureSum2, count2)) => + BLAS.axpy(1.0, featureSum2, featureSum1) + (weightSum1 + weightSum2, featureSum1, count1 + count2) + }).collect().sortBy(_._1) + + val numSamples = aggregated.map(_._2._3).sum instr.logNumExamples(numSamples) val numLabels = aggregated.length instr.logNumClasses(numLabels) - val numDocuments = aggregated.map(_._2).sum + val numDocuments = aggregated.map(_._2._1).sum val labelArray = new Array[Double](numLabels) val piArray = new Array[Double](numLabels) @@ -207,17 +184,19 @@ class NaiveBayes @Since("1.5.0") ( val lambda = $(smoothing) val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 - aggregated.foreach { case (label, n, sumTermFreqs, _) => + aggregated.foreach { case (label, (n, sumTermFreqs, _)) => labelArray(i) = label piArray(i) = math.log(n + lambda) - piLogDenom val thetaLogDenom = $(modelType) match { - case Multinomial => math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) + case Multinomial => math.log(sumTermFreqs.values.sum + numFeatures * lambda) case Bernoulli => math.log(n + 2.0 * lambda) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } var j = 0 - val offset = i * numFeatures while (j < numFeatures) { - thetaArray(offset + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + thetaArray(i * numFeatures + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom j += 1 } i += 1 @@ -225,86 +204,7 @@ class NaiveBayes @Since("1.5.0") ( val pi = Vectors.dense(piArray) val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) - .setOldLabels(labelArray) - } - - private def trainGaussianImpl( - dataset: Dataset[_], - instr: Instrumentation): NaiveBayesModel = { - val spark = dataset.sparkSession - import spark.implicits._ - - val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { - col($(weightCol)).cast(DoubleType) - } else { - lit(1.0) - } - - // Aggregates mean vector and square-sum vector per label. - // TODO: Summarizer directly returns square-sum vector. - val aggregated = dataset.groupBy(col($(labelCol))) - .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "normL2") - .summary(col($(featuresCol)), w).as("summary")) - .select($(labelCol), "weightSum", "summary.mean", "summary.normL2") - .as[(Double, Double, Vector, Vector)] - .map { case (label, weightSum, mean, normL2) => - (label, weightSum, mean, Vectors.dense(normL2.toArray.map(v => v * v))) - }.collect().sortBy(_._1) - - val numFeatures = aggregated.head._3.size - instr.logNumFeatures(numFeatures) - - val numLabels = aggregated.length - instr.logNumClasses(numLabels) - - val numInstances = aggregated.map(_._2).sum - - // If the ratio of data variance between dimensions is too small, it - // will cause numerical errors. To address this, we artificially - // boost the variance by epsilon, a small fraction of the standard - // deviation of the largest dimension. - // Refer to scikit-learn's implementation - // [https://github.com/scikit-learn/scikit-learn/blob/0.21.X/sklearn/naive_bayes.py#L348] - // and discussion [https://github.com/scikit-learn/scikit-learn/pull/5349] for detail. - val epsilon = Iterator.range(0, numFeatures).map { j => - var globalSum = 0.0 - var globalSqrSum = 0.0 - aggregated.foreach { case (_, weightSum, mean, squareSum) => - globalSum += mean(j) * weightSum - globalSqrSum += squareSum(j) - } - globalSqrSum / numInstances - - globalSum * globalSum / numInstances / numInstances - }.max * 1e-9 - - val piArray = new Array[Double](numLabels) - - // thetaArray in Gaussian NB store the means of features per label - val thetaArray = new Array[Double](numLabels * numFeatures) - - // thetaArray in Gaussian NB store the variances of features per label - val sigmaArray = new Array[Double](numLabels * numFeatures) - - var i = 0 - val logNumInstances = math.log(numInstances) - aggregated.foreach { case (_, weightSum, mean, squareSum) => - piArray(i) = math.log(weightSum) - logNumInstances - var j = 0 - val offset = i * numFeatures - while (j < numFeatures) { - val m = mean(j) - thetaArray(offset + j) = m - sigmaArray(offset + j) = epsilon + squareSum(j) / weightSum - m * m - j += 1 - } - i += 1 - } - - val pi = Vectors.dense(piArray) - val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - val sigma = new DenseMatrix(numLabels, numFeatures, sigmaArray, true) - new NaiveBayesModel(uid, pi.compressed, theta.compressed, sigma.compressed) + new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) } @Since("1.5.0") @@ -319,11 +219,8 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** String name for Bernoulli model type. */ private[classification] val Bernoulli: String = "bernoulli" - /** String name for Gaussian model type. */ - private[classification] val Gaussian: String = "gaussian" - /* Set of modelTypes that NaiveBayes supports */ - private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli, Gaussian) + private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli) private[NaiveBayes] def requireNonnegativeValues(v: Vector): Unit = { val values = v match { @@ -351,24 +248,19 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** * Model produced by [[NaiveBayes]] - * - * @param pi log of class priors, whose dimension is C (number of classes) + * @param pi log of class priors, whose dimension is C (number of classes) * @param theta log of class conditional probabilities, whose dimension is C (number of classes) * by D (number of features) - * @param sigma variance of each feature, whose dimension is C (number of classes) - * by D (number of features). This matrix is only available when modelType - * is set Gaussian. */ @Since("1.5.0") class NaiveBayesModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("2.0.0") val pi: Vector, - @Since("2.0.0") val theta: Matrix, - @Since("3.0.0") val sigma: Matrix) + @Since("2.0.0") val theta: Matrix) extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams with MLWritable { - import NaiveBayes.{Bernoulli, Multinomial, Gaussian} + import NaiveBayes.{Bernoulli, Multinomial} /** * mllib NaiveBayes is a wrapper of ml implementation currently. @@ -388,36 +280,18 @@ class NaiveBayesModel private[ml] ( * This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra * application of this condition (in predict function). */ - @transient private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { + private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { + case Multinomial => (None, None) case Bernoulli => val negTheta = theta.map(value => math.log1p(-math.exp(value))) val ones = new DenseVector(Array.fill(theta.numCols) {1.0}) val thetaMinusNegTheta = theta.map { value => value - math.log1p(-math.exp(value)) } - (thetaMinusNegTheta, negTheta.multiply(ones)) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + - "Variables thetaMinusNegTheta and negThetaSum should only be precomputed in Bernoulli NB.") - } - - /** - * Gaussian scoring requires sum of log(Variance). - * This precomputes sum of log(Variance) which are used for the linear algebra - * application of this condition (in predict function). - */ - @transient private lazy val logVarSum = $(modelType) match { - case Gaussian => - Array.tabulate(numClasses) { i => - Iterator.range(0, numFeatures).map { j => - math.log(sigma(i, j)) - }.sum - } + (Option(thetaMinusNegTheta), Option(negTheta.multiply(ones))) case _ => // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + - "Variables logVarSum should only be precomputed in Gaussian NB.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } @Since("1.6.0") @@ -437,42 +311,24 @@ class NaiveBayesModel private[ml] ( require(value == 0.0 || value == 1.0, s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") ) - val prob = thetaMinusNegTheta.multiply(features) + val prob = thetaMinusNegTheta.get.multiply(features) BLAS.axpy(1.0, pi, prob) - BLAS.axpy(1.0, negThetaSum, prob) + BLAS.axpy(1.0, negThetaSum.get, prob) prob } - private def gaussianCalculation(features: Vector) = { - val prob = Array.ofDim[Double](numClasses) - var i = 0 - while (i < numClasses) { - var s = 0.0 - var j = 0 - while (j < numFeatures) { - val d = features(j) - theta(i, j) - s += d * d / sigma(i, j) - j += 1 - } - prob(i) = pi(i) - (s + logVarSum(i)) / 2 - i += 1 - } - Vectors.dense(prob) - } - - @transient private lazy val predictRawFunc = { + override protected def predictRaw(features: Vector): Vector = { $(modelType) match { case Multinomial => - features: Vector => multinomialCalculation(features) + multinomialCalculation(features) case Bernoulli => - features: Vector => bernoulliCalculation(features) - case Gaussian => - features: Vector => gaussianCalculation(features) + bernoulliCalculation(features) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } } - override protected def predictRaw(features: Vector): Vector = predictRawFunc(features) - override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { rawPrediction match { case dv: DenseVector => @@ -498,13 +354,12 @@ class NaiveBayesModel private[ml] ( @Since("1.5.0") override def copy(extra: ParamMap): NaiveBayesModel = { - copyValues(new NaiveBayesModel(uid, pi, theta, sigma).setParent(this.parent), extra) + copyValues(new NaiveBayesModel(uid, pi, theta).setParent(this.parent), extra) } @Since("1.5.0") override def toString: String = { - s"NaiveBayesModel: uid=$uid, modelType=${$(modelType)}, numClasses=$numClasses, " + - s"numFeatures=$numFeatures" + s"NaiveBayesModel (uid=$uid) with ${pi.size} classes" } @Since("1.6.0") @@ -522,61 +377,34 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { /** [[MLWriter]] instance for [[NaiveBayesModel]] */ private[NaiveBayesModel] class NaiveBayesModelWriter(instance: NaiveBayesModel) extends MLWriter { - import NaiveBayes._ private case class Data(pi: Vector, theta: Matrix) - private case class GaussianData(pi: Vector, theta: Matrix, sigma: Matrix) override protected def saveImpl(path: String): Unit = { // Save metadata and Params DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: pi, theta + val data = Data(instance.pi, instance.theta) val dataPath = new Path(path, "data").toString - - instance.getModelType match { - case Multinomial | Bernoulli => - // Save model data: pi, theta - require(instance.sigma == null) - val data = Data(instance.pi, instance.theta) - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - - case Gaussian => - require(instance.sigma != null) - val data = GaussianData(instance.pi, instance.theta, instance.sigma) - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } private class NaiveBayesModelReader extends MLReader[NaiveBayesModel] { - import NaiveBayes._ /** Checked against metadata when loading model */ private val className = classOf[NaiveBayesModel].getName override def load(path: String): NaiveBayesModel = { - implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) - val modelTypeJson = metadata.getParamValue("modelType") - val modelType = Param.jsonDecode[String](compact(render(modelTypeJson))) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") - - val model = if (major.toInt < 3 || modelType != Gaussian) { - val Row(pi: Vector, theta: Matrix) = - MLUtils.convertMatrixColumnsToML(vecConverted, "theta") - .select("pi", "theta") - .head() - new NaiveBayesModel(metadata.uid, pi, theta, null) - } else { - val Row(pi: Vector, theta: Matrix, sigma: Matrix) = - MLUtils.convertMatrixColumnsToML(vecConverted, "theta", "sigma") - .select("pi", "theta", "sigma") - .head() - new NaiveBayesModel(metadata.uid, pi, theta, sigma) - } + val Row(pi: Vector, theta: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta") + .select("pi", "theta") + .head() + val model = new NaiveBayesModel(metadata.uid, pi, theta) metadata.getAndSetParams(model) model diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 51a624795cdd..675315e3bb07 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -257,12 +257,6 @@ final class OneVsRestModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new OneVsRestModel.OneVsRestModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"OneVsRestModel: uid=$uid, classifier=${$(classifier)}, numClasses=$numClasses, " + - s"numFeatures=$numFeatures" - } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index bc28d783ed96..245cda35d8ad 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -260,8 +260,7 @@ class RandomForestClassificationModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"RandomForestClassificationModel: uid=$uid, numTrees=$getNumTrees, numClasses=$numClasses, " + - s"numFeatures=$numFeatures" + s"RandomForestClassificationModel (uid=$uid) with $getNumTrees trees" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 5f2316fa7ce1..4ad0cb55b007 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -91,9 +91,6 @@ class BisectingKMeansModel private[ml] ( extends Model[BisectingKMeansModel] with BisectingKMeansParams with MLWritable with HasTrainingSummary[BisectingKMeansSummary] { - @Since("3.0.0") - lazy val numFeatures: Int = parentModel.clusterCenters.head.size - @Since("2.0.0") override def copy(extra: ParamMap): BisectingKMeansModel = { val copied = copyValues(new BisectingKMeansModel(uid, parentModel), extra) @@ -148,12 +145,6 @@ class BisectingKMeansModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new BisectingKMeansModel.BisectingKMeansModelWriter(this) - @Since("3.0.0") - override def toString: String = { - s"BisectingKMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + - s"numFeatures=$numFeatures" - } - /** * Gets summary of model on training set. An exception is * thrown if `hasSummary` is false. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 916f326ab561..b4d9a9f88246 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -89,9 +89,6 @@ class GaussianMixtureModel private[ml] ( extends Model[GaussianMixtureModel] with GaussianMixtureParams with MLWritable with HasTrainingSummary[GaussianMixtureSummary] { - @Since("3.0.0") - lazy val numFeatures: Int = gaussians.head.mean.size - /** @group setParam */ @Since("2.1.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) @@ -189,11 +186,6 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) - @Since("3.0.0") - override def toString: String = { - s"GaussianMixtureModel: uid=$uid, k=${weights.length}, numFeatures=$numFeatures" - } - /** * Gets summary of model on training set. An exception is * thrown if `hasSummary` is false. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index caeded400f9a..5cc0f38c67e7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -108,9 +108,6 @@ class KMeansModel private[ml] ( extends Model[KMeansModel] with KMeansParams with GeneralMLWritable with HasTrainingSummary[KMeansSummary] { - @Since("3.0.0") - lazy val numFeatures: Int = parentModel.clusterCenters.head.size - @Since("1.5.0") override def copy(extra: ParamMap): KMeansModel = { val copied = copyValues(new KMeansModel(uid, parentModel), extra) @@ -156,12 +153,6 @@ class KMeansModel private[ml] ( @Since("1.6.0") override def write: GeneralMLWriter = new GeneralMLWriter(this) - @Since("3.0.0") - override def toString: String = { - s"KMeansModel: uid=$uid, k=${parentModel.k}, distanceMeasure=${$(distanceMeasure)}, " + - s"numFeatures=$numFeatures" - } - /** * Gets summary of model on training set. An exception is * thrown if `hasSummary` is false. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 9b0005b3747d..91201e7bd03f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -620,11 +620,6 @@ class LocalLDAModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new LocalLDAModel.LocalLDAModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"LocalLDAModel: uid=$uid, k=${$(k)}, numFeatures=$vocabSize" - } } @@ -788,11 +783,6 @@ class DistributedLDAModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new DistributedLDAModel.DistributedWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"DistributedLDAModel: uid=$uid, k=${$(k)}, numFeatures=$vocabSize" - } } 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 55b910e98d40..09e8e7b232f3 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 @@ -130,12 +130,6 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("1.4.1") override def copy(extra: ParamMap): BinaryClassificationEvaluator = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"BinaryClassificationEvaluator: uid=$uid, metricName=${$(metricName)}, " + - s"numBins=${$(numBins)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala index 157bed2a6b11..868bd2a763f5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala @@ -120,12 +120,6 @@ class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val uid: Str throw new IllegalArgumentException(s"No support for metric $mn, distance $dm") } } - - @Since("3.0.0") - override def toString: String = { - s"ClusteringEvaluator: uid=$uid, metricName=${$(metricName)}, " + - s"distanceMeasure=${$(distanceMeasure)}" - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index ab14227f06be..bac3e2377439 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -184,12 +184,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("1.5.0") override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"MulticlassClassificationEvaluator: uid=$uid, metricName=${$(metricName)}, " + - s"metricLabel=${$(metricLabel)}, beta=${$(beta)}, eps=${$(eps)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala index 5216c40819b0..f12c6700be04 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala @@ -121,12 +121,6 @@ class MultilabelClassificationEvaluator (override val uid: String) } override def copy(extra: ParamMap): MultilabelClassificationEvaluator = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"MultilabelClassificationEvaluator: uid=$uid, metricName=${$(metricName)}, " + - s"metricLabel=${$(metricLabel)}" - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala index ca3a8ebc1659..64ab3c3f7fdd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala @@ -105,11 +105,6 @@ class RankingEvaluator (override val uid: String) override def isLargerBetter: Boolean = true override def copy(extra: ParamMap): RankingEvaluator = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"RankingEvaluator: uid=$uid, metricName=${$(metricName)}, k=${$(k)}" - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index 9f32d40d166b..b0cafefe420a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -124,12 +124,6 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("1.5.0") override def copy(extra: ParamMap): RegressionEvaluator = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"RegressionEvaluator: uid=$uid, metricName=${$(metricName)}, " + - s"throughOrigin=${$(throughOrigin)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 07a4f91443bc..ec4d45b65317 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -204,13 +204,6 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.1") override def copy(extra: ParamMap): Binarizer = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"Binarizer: uid=$uid" + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + - get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala index 4e266fbc1ec1..c074830ec923 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala @@ -106,11 +106,6 @@ class BucketedRandomProjectionLSHModel private[ml]( override def write: MLWriter = { new BucketedRandomProjectionLSHModel.BucketedRandomProjectionLSHModelWriter(this) } - - @Since("3.0.0") - override def toString: String = { - s"BucketedRandomProjectionLSHModel: uid=$uid, numHashTables=${$(numHashTables)}" - } } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index 9aeddae78ed7..8533ed5ce572 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -215,13 +215,6 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String override def copy(extra: ParamMap): Bucketizer = { defaultCopy[Bucketizer](extra).setParent(parent) } - - @Since("3.0.0") - override def toString: String = { - s"Bucketizer: uid=$uid" + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + - get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 9103e4feac45..2a3656c49584 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -316,11 +316,6 @@ final class ChiSqSelectorModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new ChiSqSelectorModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"ChiSqSelectorModel: uid=$uid, numSelectedFeatures=${selectedFeatures.length}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index c58d44d49234..e78e6d4f76d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -307,7 +307,7 @@ class CountVectorizerModel( } val dictBr = broadcastDict.get val minTf = $(minTF) - val vectorizer = udf { document: Seq[String] => + val vectorizer = udf { (document: Seq[String]) => val termCounts = new OpenHashMap[Int, Double] var tokenCount = 0L document.foreach { term => @@ -344,11 +344,6 @@ class CountVectorizerModel( @Since("1.6.0") override def write: MLWriter = new CountVectorizerModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"CountVectorizerModel: uid=$uid, vocabularySize=${vocabulary.length}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index e2167f01281d..84d6a536ccca 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -74,11 +74,6 @@ class DCT @Since("1.5.0") (@Since("1.5.0") override val uid: String) } override protected def outputDataType: DataType = new VectorUDT - - @Since("3.0.0") - override def toString: String = { - s"DCT: uid=$uid, inverse=$inverse" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 227c13d60fd8..2f329235431f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -81,12 +81,6 @@ class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: Stri } override protected def outputDataType: DataType = new VectorUDT() - - @Since("3.0.0") - override def toString: String = { - s"ElementwiseProduct: uid=$uid" + - get(scalingVec).map(v => s", vectorSize=${v.size}").getOrElse("") - } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala index 39862554c5d8..61b4d5d54aca 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala @@ -22,7 +22,7 @@ import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.Vectors -import org.apache.spark.ml.param.{ParamMap, StringArrayParam} +import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators, StringArrayParam} import org.apache.spark.ml.param.shared.{HasInputCols, HasNumFeatures, HasOutputCol} import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} @@ -199,13 +199,6 @@ class FeatureHasher(@Since("2.3.0") override val uid: String) extends Transforme val attrGroup = new AttributeGroup($(outputCol), $(numFeatures)) SchemaUtils.appendColumn(schema, attrGroup.toStructField()) } - - @Since("3.0.0") - override def toString: String = { - s"FeatureHasher: uid=$uid, numFeatures=${$(numFeatures)}" + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + - get(categoricalCols).map(c => s", numCategoricalCols=${c.length}").getOrElse("") - } } @Since("2.3.0") 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 80bf85936aac..fe9f4f2123da 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 @@ -127,11 +127,6 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.1") override def copy(extra: ParamMap): HashingTF = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"HashingTF: uid=$uid, binary=${$(binary)}, numFeatures=${$(numFeatures)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 5f4103abcf50..4338421bf8bc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -175,13 +175,9 @@ class IDFModel private[ml] ( @Since("3.0.0") def numDocs: Long = idfModel.numDocs + @Since("1.6.0") override def write: MLWriter = new IDFModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"IDFModel: uid=$uid, numDocs=$numDocs" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index 64f1722f5fcb..fbccfb1041d1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -274,13 +274,6 @@ class ImputerModel private[ml] ( @Since("2.2.0") override def write: MLWriter = new ImputerModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"ImputerModel: uid=$uid, strategy=${$(strategy)}, missingValue=${$(missingValue)}" + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + - get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index 9a4f1d97c907..611f1b691b78 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -218,11 +218,6 @@ class Interaction @Since("1.6.0") (@Since("1.6.0") override val uid: String) ext @Since("1.6.0") override def copy(extra: ParamMap): Interaction = defaultCopy(extra) - @Since("3.0.0") - override def toString: String = { - s"Interaction: uid=$uid" + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index 6bab70e502ed..88d09d760b51 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -140,11 +140,6 @@ class MaxAbsScalerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new MaxAbsScalerModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"MaxAbsScalerModel: uid=$uid, numFeatures=${maxAbs.size}" - } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala index de7fe91c417b..da0eaad667cc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala @@ -96,11 +96,6 @@ class MinHashLSHModel private[ml]( @Since("2.1.0") override def write: MLWriter = new MinHashLSHModel.MinHashLSHModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"MinHashLSHModel: uid=$uid, numHashTables=${$(numHashTables)}" - } } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index e381a0435e9e..b0d7b637caa5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -226,12 +226,6 @@ class MinMaxScalerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new MinMaxScalerModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"MinMaxScalerModel: uid=$uid, numFeatures=${originalMin.size}, min=${$(min)}, " + - s"max=${$(max)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index fd6fde0744d0..e0772d5af20a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -70,11 +70,6 @@ class NGram @Since("1.5.0") (@Since("1.5.0") override val uid: String) } override protected def outputDataType: DataType = new ArrayType(StringType, false) - - @Since("3.0.0") - override def toString: String = { - s"NGram: uid=$uid, n=${$(n)}" - } } @Since("1.6.0") 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 d129c2b2c2dc..5db7f5da49c6 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 @@ -65,11 +65,6 @@ class Normalizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) } override protected def outputDataType: DataType = new VectorUDT() - - @Since("3.0.0") - override def toString: String = { - s"Normalizer: uid=$uid, p=${$(p)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index fd58043cda07..459994c352da 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -376,13 +376,6 @@ class OneHotEncoderModel private[ml] ( @Since("3.0.0") override def write: MLWriter = new OneHotEncoderModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"OneHotEncoderModel: uid=$uid, dropLast=${$(dropLast)}, handleInvalid=${$(handleInvalid)}" + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + - get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") - } } @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 69dcacbb0c18..aa5a171d4fec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -179,11 +179,6 @@ class PCAModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new PCAModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"PCAModel: uid=$uid, k=${$(k)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 592ca001a246..5734b06ee963 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -77,11 +77,6 @@ class PolynomialExpansion @Since("1.4.0") (@Since("1.4.0") override val uid: Str @Since("1.4.1") override def copy(extra: ParamMap): PolynomialExpansion = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"PolynomialExpansion: uid=$uid, degree=${$(degree)}" - } } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 7ccfafa4ac81..9e95762f1291 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -320,10 +320,7 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) override def copy(extra: ParamMap): RFormula = defaultCopy(extra) @Since("2.0.0") - override def toString: String = { - s"RFormula: uid=$uid" + - get(formula).map(f => s", formula = $f").getOrElse("") - } + override def toString: String = s"RFormula(${get(formula).getOrElse("")}) (uid=$uid)" } @Since("2.0.0") @@ -379,9 +376,7 @@ class RFormulaModel private[feature]( } @Since("2.0.0") - override def toString: String = { - s"RFormulaModel: uid=$uid, resolvedFormula=$resolvedFormula" - } + override def toString: String = s"RFormulaModel($resolvedFormula) (uid=$uid)" private def transformLabel(dataset: Dataset[_]): DataFrame = { val labelName = resolvedFormula.label diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index 1b9b8082931a..1d609ef3190d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -251,12 +251,6 @@ class RobustScalerModel private[ml] ( } override def write: MLWriter = new RobustScalerModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"RobustScalerModel: uid=$uid, numFeatures=${median.size}, " + - s"withCentering=${$(withCentering)}, withScaling=${$(withScaling)}" - } } @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 9b99b8177a26..0fb1d8c5dc57 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -90,11 +90,6 @@ class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) @Since("1.6.0") override def copy(extra: ParamMap): SQLTransformer = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"SQLTransformer: uid=$uid, statement=${$(statement)}" - } } @Since("1.6.0") 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 41e319c1fb1f..834c21e16dd2 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 @@ -184,12 +184,6 @@ class StandardScalerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new StandardScalerModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"StandardScalerModel: uid=$uid, numFeatures=${mean.size}, withMean=${$(withMean)}, " + - s"withStd=${$(withStd)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala index b6ed4f2b000c..f95e03ae6c82 100755 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -22,19 +22,15 @@ import java.util.Locale import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} -import org.apache.spark.sql.types.{ArrayType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, StringType, StructType} /** * A feature transformer that filters out stop words from input. * - * Since 3.0.0, `StopWordsRemover` can filter out multiple columns at once by setting the - * `inputCols` parameter. Note that when both the `inputCol` and `inputCols` parameters are set, - * an Exception will be thrown. - * * @note null values from input array are preserved unless adding null to stopWords * explicitly. * @@ -42,8 +38,7 @@ import org.apache.spark.sql.types.{ArrayType, StringType, StructField, StructTyp */ @Since("1.5.0") class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Transformer with HasInputCol with HasOutputCol with HasInputCols with HasOutputCols - with DefaultParamsWritable { + extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { @Since("1.5.0") def this() = this(Identifiable.randomUID("stopWords")) @@ -56,14 +51,6 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) - /** @group setParam */ - @Since("3.0.0") - def setInputCols(value: Array[String]): this.type = set(inputCols, value) - - /** @group setParam */ - @Since("3.0.0") - def setOutputCols(value: Array[String]): this.type = set(outputCols, value) - /** * The words to be filtered out. * Default: English stop words @@ -134,15 +121,6 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String } } - /** Returns the input and output column names corresponding in pair. */ - private[feature] def getInOutCols(): (Array[String], Array[String]) = { - if (isSet(inputCol)) { - (Array($(inputCol)), Array($(outputCol))) - } else { - ($(inputCols), $(outputCols)) - } - } - setDefault(stopWords -> StopWordsRemover.loadDefaultStopWords("english"), caseSensitive -> false, locale -> getDefaultOrUS.toString) @@ -164,48 +142,20 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String terms.filter(s => !lowerStopWords.contains(toLower(s))) } } - - val (inputColNames, outputColNames) = getInOutCols() - val ouputCols = inputColNames.map { inputColName => - t(col(inputColName)) - } - val ouputMetadata = outputColNames.map(outputSchema(_).metadata) - dataset.withColumns(outputColNames, ouputCols, ouputMetadata) + val metadata = outputSchema($(outputCol)).metadata + dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), - Seq(outputCols)) - - if (isSet(inputCols)) { - require(getInputCols.length == getOutputCols.length, - s"StopWordsRemover $this has mismatched Params " + - s"for multi-column transform. Params ($inputCols, $outputCols) should have " + - "equal lengths, but they have different lengths: " + - s"(${getInputCols.length}, ${getOutputCols.length}).") - } - - val (inputColNames, outputColNames) = getInOutCols() - val newCols = inputColNames.zip(outputColNames).map { case (inputColName, outputColName) => - require(!schema.fieldNames.contains(outputColName), - s"Output Column $outputColName already exists.") - val inputType = schema(inputColName).dataType - require(inputType.sameType(ArrayType(StringType)), "Input type must be " + - s"${ArrayType(StringType).catalogString} but got ${inputType.catalogString}.") - StructField(outputColName, inputType, schema(inputColName).nullable) - } - StructType(schema.fields ++ newCols) + val inputType = schema($(inputCol)).dataType + require(inputType.sameType(ArrayType(StringType)), "Input type must be " + + s"${ArrayType(StringType).catalogString} but got ${inputType.catalogString}.") + SchemaUtils.appendColumn(schema, $(outputCol), inputType, schema($(inputCol)).nullable) } @Since("1.5.0") override def copy(extra: ParamMap): StopWordsRemover = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"StopWordsRemover: uid=$uid, numStopWords=${$(stopWords).length}, locale=${$(locale)}, " + - s"caseSensitive=${$(caseSensitive)}" - } } @Since("1.6.0") 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 9f9f097a26ea..2ce5acf1fea1 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 @@ -412,7 +412,7 @@ class StringIndexerModel ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val (inputColNames, outputColNames) = getInOutCols() + var (inputColNames, outputColNames) = getInOutCols() val outputColumns = new Array[Column](outputColNames.length) // Skips invalid rows if `handleInvalid` is set to `StringIndexer.SKIP_INVALID`. @@ -473,14 +473,6 @@ class StringIndexerModel ( @Since("1.6.0") override def write: StringIndexModelWriter = new StringIndexModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"StringIndexerModel: uid=$uid, handleInvalid=${$(handleInvalid)}" + - get(stringOrderType).map(t => s", stringOrderType=$t").getOrElse("") + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") + - get(outputCols).map(c => s", numOutputCols=${c.length}").getOrElse("") - } } @Since("1.6.0") 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 5ec5b7732564..e6e9bdfd292b 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 @@ -175,12 +175,6 @@ class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.1") override def copy(extra: ParamMap): VectorAssembler = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"VectorAssembler: uid=$uid, handleInvalid=${$(handleInvalid)}" + - get(inputCols).map(c => s", numInputCols=${c.length}").getOrElse("") - } } @Since("1.6.0") 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 18a82e5fe945..6a3305aad8f3 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 @@ -428,7 +428,7 @@ class VectorIndexerModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) val newField = prepOutputField(dataset.schema) - val transformUDF = udf { vector: Vector => transformFunc(vector) } + val transformUDF = udf { (vector: Vector) => transformFunc(vector) } val newCol = transformUDF(dataset($(inputCol))) val ds = dataset.withColumn($(outputCol), newCol, newField.metadata) if (getHandleInvalid == VectorIndexer.SKIP_INVALID) { @@ -506,11 +506,6 @@ class VectorIndexerModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new VectorIndexerModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"VectorIndexerModel: uid=$uid, numFeatures=$numFeatures, handleInvalid=${$(handleInvalid)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala index 0f778e09a80b..5d787f263a12 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala @@ -176,11 +176,6 @@ class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String) @Since("2.3.0") override def copy(extra: ParamMap): this.type = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"VectorSizeHint: uid=$uid, size=${$(size)}, handleInvalid=${$(handleInvalid)}" - } } @Since("2.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index b84b8af4e8a9..e3e462d07e10 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -159,12 +159,6 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri @Since("1.5.0") override def copy(extra: ParamMap): VectorSlicer = defaultCopy(extra) - - @Since("3.0.0") - override def toString: String = { - s"VectorSlicer: uid=$uid" + - get(indices).map(i => s", numSelectedFeatures=${i.length}").getOrElse("") - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 81dde0315c19..6ae90b805093 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -324,12 +324,6 @@ class Word2VecModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new Word2VecModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"Word2VecModel: uid=$uid, numWords=${wordVectors.wordIndex.size}, " + - s"vectorSize=${$(vectorSize)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index a9592dbfca06..e1c9b927a28c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -310,11 +310,6 @@ class FPGrowthModel private[ml] ( @Since("2.2.0") override def write: MLWriter = new FPGrowthModel.FPGrowthModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"FPGrowthModel: uid=$uid, numTrainingRecords=$numTrainingRecords" - } } @Since("2.2.0") 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 2fb9a276be88..5049ef924561 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 @@ -338,11 +338,6 @@ class ALSModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new ALSModel.ALSModelWriter(this) - @Since("3.0.0") - override def toString: String = { - s"ALSModel: uid=$uid, rank=$rank" - } - /** * Returns top `numItems` items recommended for each user, for all users. * @param numItems max number of recommendations for each user @@ -1046,13 +1041,13 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { .persist(finalRDDStorageLevel) if (finalRDDStorageLevel != StorageLevel.NONE) { userIdAndFactors.count() + itemIdAndFactors.count() + itemFactors.unpersist() userInBlocks.unpersist() userOutBlocks.unpersist() + itemInBlocks.unpersist() itemOutBlocks.unpersist() blockRatings.unpersist() - itemIdAndFactors.count() - itemFactors.unpersist() - itemInBlocks.unpersist() } (userIdAndFactors, itemIdAndFactors) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index fe89aed9a928..1ce5b5b02b50 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -311,9 +311,6 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") val scale: Double) extends Model[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams with MLWritable { - @Since("3.0.0") - lazy val numFeatures: Int = coefficients.size - /** @group setParam */ @Since("1.6.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) @@ -389,11 +386,6 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new AFTSurvivalRegressionModel.AFTSurvivalRegressionModelWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"AFTSurvivalRegressionModel: uid=$uid, numFeatures=$numFeatures" - } } @Since("1.6.0") 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 index 4a97997a1deb..05851d511675 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -243,8 +243,7 @@ class DecisionTreeRegressionModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"DecisionTreeRegressionModel: uid=$uid, depth=$depth, numNodes=$numNodes, " + - s"numFeatures=$numFeatures" + s"DecisionTreeRegressionModel (uid=$uid) of depth $depth with $numNodes nodes" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 700f7a2075a9..9c38647642a6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -302,7 +302,7 @@ class GBTRegressionModel private[ml]( @Since("1.4.0") override def toString: String = { - s"GBTRegressionModel: uid=$uid, numTrees=$numTrees, numFeatures=$numFeatures" + s"GBTRegressionModel (uid=$uid) with $numTrees trees" } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 53b29102f01b..c504a5462900 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1106,12 +1106,6 @@ class GeneralizedLinearRegressionModel private[ml] ( new GeneralizedLinearRegressionModel.GeneralizedLinearRegressionModelWriter(this) override val numFeatures: Int = coefficients.size - - @Since("3.0.0") - override def toString: String = { - s"GeneralizedLinearRegressionModel: uid=$uid, family=${$(family)}, link=${$(link)}, " + - s"numFeatures=$numFeatures" - } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 47f9e4bfb833..8b9233dcdc4d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -259,14 +259,6 @@ class IsotonicRegressionModel private[ml] ( @Since("1.6.0") override def write: MLWriter = new IsotonicRegressionModelWriter(this) - - @Since("3.0.0") - val numFeatures: Int = 1 - - @Since("3.0.0") - override def toString: String = { - s"IsotonicRegressionModel: uid=$uid, numFeatures=$numFeatures" - } } @Since("1.6.0") 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 dad1080cea39..25c6f4d980b9 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 @@ -702,11 +702,6 @@ class LinearRegressionModel private[ml] ( */ @Since("1.6.0") override def write: GeneralMLWriter = new GeneralMLWriter(this) - - @Since("3.0.0") - override def toString: String = { - s"LinearRegressionModel: uid=$uid, numFeatures=$numFeatures" - } } /** A writer for LinearRegression that handles the "internal" (or default) format */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index c3afab57a49c..8f78fc1da18c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -235,7 +235,7 @@ class RandomForestRegressionModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"RandomForestRegressionModel: uid=$uid, numTrees=$getNumTrees, numFeatures=$numFeatures" + s"RandomForestRegressionModel (uid=$uid) with $getNumTrees trees" } /** 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 fff18bcbec65..e60a14f976a5 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 @@ -323,11 +323,6 @@ class CrossValidatorModel private[ml] ( override def write: CrossValidatorModel.CrossValidatorModelWriter = { new CrossValidatorModel.CrossValidatorModelWriter(this) } - - @Since("3.0.0") - override def toString: String = { - s"CrossValidatorModel: uid=$uid, bestModel=$bestModel, numFolds=${$(numFolds)}" - } } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index ecf9b846eedd..8b251197afbe 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -140,7 +140,7 @@ class TrainValidationSplit @Since("1.5.0") (@Since("1.5.0") override val uid: St val collectSubModelsParam = $(collectSubModels) - val subModels: Option[Array[Model[_]]] = if (collectSubModelsParam) { + var subModels: Option[Array[Model[_]]] = if (collectSubModelsParam) { Some(Array.fill[Model[_]](epm.length)(null)) } else None @@ -314,11 +314,6 @@ class TrainValidationSplitModel private[ml] ( override def write: TrainValidationSplitModel.TrainValidationSplitModelWriter = { new TrainValidationSplitModel.TrainValidationSplitModelWriter(this) } - - @Since("3.0.0") - override def toString: String = { - s"TrainValidationSplitModel: uid=$uid, bestModel=$bestModel, trainRatio=${$(trainRatio)}" - } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index ac85fbc235c9..278d61d91673 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -223,12 +223,12 @@ class KMeans private ( // Compute squared norms and cache them. val norms = data.map(Vectors.norm(_, 2.0)) + norms.persist() val zippedData = data.zip(norms).map { case (v, norm) => new VectorWithNorm(v, norm) } - zippedData.persist() val model = runAlgorithm(zippedData, instr) - zippedData.unpersist() + norms.unpersist() // Warn at the end of the run as well, for increased visibility. if (data.getStorageLevel == StorageLevel.NONE) { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 07116606dfb5..d2b8751360e9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -2767,7 +2767,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("toString") { val model = new LogisticRegressionModel("logReg", Vectors.dense(0.1, 0.2, 0.3), 0.0) - val expected = "LogisticRegressionModel: uid=logReg, numClasses=2, numFeatures=3" + val expected = "LogisticRegressionModel: uid = logReg, numClasses = 2, numFeatures = 3" assert(model.toString === expected) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 9e4844ff8907..9100ef1db6e1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -22,15 +22,15 @@ import scala.util.Random import breeze.linalg.{DenseVector => BDV, Vector => BV} import breeze.stats.distributions.{Multinomial => BrzMultinomial, RandBasis => BrzRandBasis} -import org.apache.spark.SparkException -import org.apache.spark.ml.classification.NaiveBayes._ +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.classification.NaiveBayes.{Bernoulli, Multinomial} import org.apache.spark.ml.classification.NaiveBayesSuite._ import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row} class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @@ -38,8 +38,6 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @transient var dataset: Dataset[_] = _ @transient var bernoulliDataset: Dataset[_] = _ - @transient var gaussianDataset: Dataset[_] = _ - @transient var gaussianDataset2: Dataset[_] = _ private val seed = 42 @@ -55,23 +53,6 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { dataset = generateNaiveBayesInput(pi, theta, 100, seed).toDF() bernoulliDataset = generateNaiveBayesInput(pi, theta, 100, seed, "bernoulli").toDF() - - // theta for gaussian nb - val theta2 = Array( - Array(0.70, 0.10, 0.10, 0.10), // label 0: mean - Array(0.10, 0.70, 0.10, 0.10), // label 1: mean - Array(0.10, 0.10, 0.70, 0.10) // label 2: mean - ) - - // sigma for gaussian nb - val sigma = Array( - Array(0.10, 0.10, 0.50, 0.10), // label 0: variance - Array(0.50, 0.10, 0.10, 0.10), // label 1: variance - Array(0.10, 0.10, 0.10, 0.50) // label 2: variance - ) - gaussianDataset = generateGaussianNaiveBayesInput(pi, theta2, sigma, 1000, seed).toDF() - gaussianDataset2 = spark.read.format("libsvm") - .load("../data/mllib/sample_multiclass_classification_data.txt") } def validatePrediction(predictionAndLabels: Seq[Row]): Unit = { @@ -86,17 +67,10 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { def validateModelFit( piData: Vector, thetaData: Matrix, - sigmaData: Matrix, model: NaiveBayesModel): Unit = { assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~== Vectors.dense(piData.toArray.map(math.exp)) absTol 0.05, "pi mismatch") assert(model.theta.map(math.exp) ~== thetaData.map(math.exp) absTol 0.05, "theta mismatch") - if (sigmaData == null) { - assert(model.sigma == null, "sigma mismatch") - } else { - assert(model.sigma.map(math.exp) ~== sigmaData.map(math.exp) absTol 0.05, - "sigma mismatch") - } } def expectedMultinomialProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { @@ -116,19 +90,6 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Vectors.dense(classProbs.map(_ / classProbsSum)) } - def expectedGaussianProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { - val pi = model.pi.toArray.map(math.exp) - val classProbs = pi.indices.map { i => - feature.toArray.zipWithIndex.map { case (v, j) => - val mean = model.theta(i, j) - val variance = model.sigma(i, j) - math.exp(- (v - mean) * (v - mean) / variance / 2) / math.sqrt(variance * math.Pi * 2) - }.product * pi(i) - }.toArray - val classProbsSum = classProbs.sum - Vectors.dense(classProbs.map(_ / classProbsSum)) - } - def validateProbabilities( featureAndProbabilities: Seq[Row], model: NaiveBayesModel, @@ -141,8 +102,6 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { expectedMultinomialProbabilities(model, features) case Bernoulli => expectedBernoulliProbabilities(model, features) - case Gaussian => - expectedGaussianProbabilities(model, features) case _ => throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } @@ -153,14 +112,12 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("model types") { assert(Multinomial === "multinomial") assert(Bernoulli === "bernoulli") - assert(Gaussian === "gaussian") } test("params") { ParamsSuite.checkParams(new NaiveBayes) val model = new NaiveBayesModel("nb", pi = Vectors.dense(Array(0.2, 0.8)), - theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4)), - sigma = null) + theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4))) ParamsSuite.checkParams(model) } @@ -189,7 +146,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("multinomial") val model = nb.fit(testDataset) - validateModelFit(pi, theta, null, model) + validateModelFit(pi, theta, model) assert(model.hasParent) MLTestingUtils.checkCopyAndUids(nb, model) @@ -235,17 +192,12 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("Naive Bayes with weighted samples") { val numClasses = 3 def modelEquals(m1: NaiveBayesModel, m2: NaiveBayesModel): Unit = { - assert(m1.getModelType === m2.getModelType) assert(m1.pi ~== m2.pi relTol 0.01) assert(m1.theta ~== m2.theta relTol 0.01) - if (m1.getModelType == Gaussian) { - assert(m1.sigma ~== m2.sigma relTol 0.01) - } } val testParams = Seq[(String, Dataset[_])]( ("bernoulli", bernoulliDataset), - ("multinomial", dataset), - ("gaussian", gaussianDataset) + ("multinomial", dataset) ) testParams.foreach { case (family, dataset) => // NaiveBayes is sensitive to constant scaling of the weights unless smoothing is set to 0 @@ -276,7 +228,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("bernoulli") val model = nb.fit(testDataset) - validateModelFit(pi, theta, null, model) + validateModelFit(pi, theta, model) assert(model.hasParent) val validationDataset = @@ -356,112 +308,14 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { } } - test("Naive Bayes Gaussian") { - val piArray = Array(0.5, 0.1, 0.4).map(math.log) - - val thetaArray = Array( - Array(0.70, 0.10, 0.10, 0.10), // label 0: mean - Array(0.10, 0.70, 0.10, 0.10), // label 1: mean - Array(0.10, 0.10, 0.70, 0.10) // label 2: mean - ) - - val sigmaArray = Array( - Array(0.10, 0.10, 0.50, 0.10), // label 0: variance - Array(0.50, 0.10, 0.10, 0.10), // label 1: variance - Array(0.10, 0.10, 0.10, 0.50) // label 2: variance - ) - - val pi = Vectors.dense(piArray) - val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) - val sigma = new DenseMatrix(3, 4, sigmaArray.flatten, true) - - val nPoints = 10000 - val testDataset = - generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 42).toDF() - val gnb = new NaiveBayes().setModelType("gaussian") - val model = gnb.fit(testDataset) - - validateModelFit(pi, theta, sigma, model) - assert(model.hasParent) - - val validationDataset = - generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 17).toDF() - - val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") - validatePrediction(predictionAndLabels.collect()) - - val featureAndProbabilities = model.transform(validationDataset) - .select("features", "probability") - validateProbabilities(featureAndProbabilities.collect(), model, "gaussian") - } - - test("Naive Bayes Gaussian - Model Coefficients") { - /* - Using the following Python code to verify the correctness. - - import numpy as np - from sklearn.naive_bayes import GaussianNB - from sklearn.datasets import load_svmlight_file - - path = "./data/mllib/sample_multiclass_classification_data.txt" - X, y = load_svmlight_file(path) - X = X.toarray() - clf = GaussianNB() - clf.fit(X, y) - - >>> clf.class_prior_ - array([0.33333333, 0.33333333, 0.33333333]) - >>> clf.theta_ - array([[ 0.27111101, -0.18833335, 0.54305072, 0.60500005], - [-0.60777778, 0.18166667, -0.84271174, -0.88000014], - [-0.09111114, -0.35833336, 0.10508474, 0.0216667 ]]) - >>> clf.sigma_ - array([[0.12230125, 0.07078052, 0.03430001, 0.05133607], - [0.03758145, 0.0988028 , 0.0033903 , 0.00782224], - [0.08058764, 0.06701387, 0.02486641, 0.02661392]]) - */ - - val gnb = new NaiveBayes().setModelType(Gaussian) - val model = gnb.fit(gaussianDataset2) - assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~= - Vectors.dense(0.33333333, 0.33333333, 0.33333333) relTol 1E-5) - - val thetaRows = model.theta.rowIter.toArray - assert(thetaRows(0) ~= - Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005)relTol 1E-5) - assert(thetaRows(1) ~= - Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014)relTol 1E-5) - assert(thetaRows(2) ~= - Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667)relTol 1E-5) - - val sigmaRows = model.sigma.rowIter.toArray - assert(sigmaRows(0) ~= - Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607)relTol 1E-5) - assert(sigmaRows(1) ~= - Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224)relTol 1E-5) - assert(sigmaRows(2) ~= - Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392)relTol 1E-5) - } - test("read/write") { def checkModelData(model: NaiveBayesModel, model2: NaiveBayesModel): Unit = { - assert(model.getModelType === model2.getModelType) assert(model.pi === model2.pi) assert(model.theta === model2.theta) - if (model.getModelType == "gaussian") { - assert(model.sigma === model2.sigma) - } else { - assert(model.sigma === null && model2.sigma === null) - } } val nb = new NaiveBayes() testEstimatorAndModelReadWrite(nb, dataset, NaiveBayesSuite.allParamSettings, NaiveBayesSuite.allParamSettings, checkModelData) - - val gnb = new NaiveBayes().setModelType("gaussian") - testEstimatorAndModelReadWrite(gnb, gaussianDataset, - NaiveBayesSuite.allParamSettingsForGaussian, - NaiveBayesSuite.allParamSettingsForGaussian, checkModelData) } test("should support all NumericType labels and weights, and not support other types") { @@ -470,7 +324,6 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) - assert(expected.sigma === null && actual.sigma === null) } } } @@ -487,16 +340,6 @@ object NaiveBayesSuite { "smoothing" -> 0.1 ) - /** - * Mapping from all Params to valid settings which differ from the defaults. - * This is useful for tests which need to exercise all Params, such as save/load. - * This excludes input columns to simplify some tests. - */ - val allParamSettingsForGaussian: Map[String, Any] = Map( - "predictionCol" -> "myPrediction", - "modelType" -> "gaussian" - ) - private def calcLabel(p: Double, pi: Array[Double]): Int = { var sum = 0.0 for (j <- 0 until pi.length) { @@ -541,26 +384,4 @@ object NaiveBayesSuite { LabeledPoint(y, Vectors.dense(xi)) } } - - // Generate input - def generateGaussianNaiveBayesInput( - pi: Array[Double], // 1XC - theta: Array[Array[Double]], // CXD - sigma: Array[Array[Double]], // CXD - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { - val D = theta(0).length - val rnd = new Random(seed) - val _pi = pi.map(math.exp) - - for (i <- 0 until nPoints) yield { - val y = calcLabel(rnd.nextDouble(), _pi) - val xi = Array.tabulate[Double] (D) { j => - val mean = theta(y)(j) - val variance = sigma(y)(j) - mean + rnd.nextGaussian() * math.sqrt(variance) - } - LabeledPoint(y, Vectors.dense(xi)) - } - } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala index c142f83e0595..6d0b83e85733 100755 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.ml.feature import java.util.Locale -import org.apache.spark.ml.Pipeline import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.sql.{DataFrame, Row} @@ -182,19 +181,12 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { } test("read/write") { - val t1 = new StopWordsRemover() + val t = new StopWordsRemover() .setInputCol("myInputCol") .setOutputCol("myOutputCol") .setStopWords(Array("the", "a")) .setCaseSensitive(true) - testDefaultReadWrite(t1) - - val t2 = new StopWordsRemover() - .setInputCols(Array("input1", "input2", "input3")) - .setOutputCols(Array("result1", "result2", "result3")) - .setStopWords(Array("the", "a")) - .setCaseSensitive(true) - testDefaultReadWrite(t2) + testDefaultReadWrite(t) } test("StopWordsRemover output column already exists") { @@ -207,7 +199,7 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { testTransformerByInterceptingException[(Array[String], Array[String])]( dataSet, remover, - s"requirement failed: Output Column $outputCol already exists.", + s"requirement failed: Column $outputCol already exists.", "expected") } @@ -225,123 +217,4 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { Locale.setDefault(oldDefault) } } - - test("Multiple Columns: StopWordsRemover default") { - val remover = new StopWordsRemover() - .setInputCols(Array("raw1", "raw2")) - .setOutputCols(Array("filtered1", "filtered2")) - val df = Seq( - (Seq("test", "test"), Seq("test1", "test2"), Seq("test", "test"), Seq("test1", "test2")), - (Seq("a", "b", "c", "d"), Seq("a", "b"), Seq("b", "c", "d"), Seq("b")), - (Seq("a", "the", "an"), Seq("the", "an"), Seq(), Seq()), - (Seq("A", "The", "AN"), Seq("A", "The"), Seq(), Seq()), - (Seq(null), Seq(null), Seq(null), Seq(null)), - (Seq(), Seq(), Seq(), Seq()) - ).toDF("raw1", "raw2", "expected1", "expected2") - - remover.transform(df) - .select("filtered1", "expected1", "filtered2", "expected2") - .collect().foreach { - case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => - assert(r1 === e1, - s"The result value is not correct after bucketing. Expected $e1 but found $r1") - assert(r2 === e2, - s"The result value is not correct after bucketing. Expected $e2 but found $r2") - } - } - - test("Multiple Columns: StopWordsRemover with particular stop words list") { - val stopWords = Array("test", "a", "an", "the") - val remover = new StopWordsRemover() - .setInputCols(Array("raw1", "raw2")) - .setOutputCols(Array("filtered1", "filtered2")) - .setStopWords(stopWords) - val df = Seq( - (Seq("test", "test"), Seq("test1", "test2"), Seq(), Seq("test1", "test2")), - (Seq("a", "b", "c", "d"), Seq("a", "b"), Seq("b", "c", "d"), Seq("b")), - (Seq("a", "the", "an"), Seq("a", "the", "test1"), Seq(), Seq("test1")), - (Seq("A", "The", "AN"), Seq("A", "The", "AN"), Seq(), Seq()), - (Seq(null), Seq(null), Seq(null), Seq(null)), - (Seq(), Seq(), Seq(), Seq()) - ).toDF("raw1", "raw2", "expected1", "expected2") - - remover.transform(df) - .select("filtered1", "expected1", "filtered2", "expected2") - .collect().foreach { - case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => - assert(r1 === e1, - s"The result value is not correct after bucketing. Expected $e1 but found $r1") - assert(r2 === e2, - s"The result value is not correct after bucketing. Expected $e2 but found $r2") - } - } - - test("Compare single/multiple column(s) StopWordsRemover in pipeline") { - val df = Seq( - (Seq("test", "test"), Seq("test1", "test2")), - (Seq("a", "b", "c", "d"), Seq("a", "b")), - (Seq("a", "the", "an"), Seq("a", "the", "test1")), - (Seq("A", "The", "AN"), Seq("A", "The", "AN")), - (Seq(null), Seq(null)), - (Seq(), Seq()) - ).toDF("input1", "input2") - - val multiColsRemover = new StopWordsRemover() - .setInputCols(Array("input1", "input2")) - .setOutputCols(Array("output1", "output2")) - - val plForMultiCols = new Pipeline() - .setStages(Array(multiColsRemover)) - .fit(df) - - val removerForCol1 = new StopWordsRemover() - .setInputCol("input1") - .setOutputCol("output1") - val removerForCol2 = new StopWordsRemover() - .setInputCol("input2") - .setOutputCol("output2") - - val plForSingleCol = new Pipeline() - .setStages(Array(removerForCol1, removerForCol2)) - .fit(df) - - val resultForSingleCol = plForSingleCol.transform(df) - .select("output1", "output2") - .collect() - val resultForMultiCols = plForMultiCols.transform(df) - .select("output1", "output2") - .collect() - - resultForSingleCol.zip(resultForMultiCols).foreach { - case (rowForSingle, rowForMultiCols) => - assert(rowForSingle === rowForMultiCols) - } - } - - test("Multiple Columns: Mismatched sizes of inputCols/outputCols") { - val remover = new StopWordsRemover() - .setInputCols(Array("input1")) - .setOutputCols(Array("result1", "result2")) - val df = Seq( - (Seq("A"), Seq("A")), - (Seq("The", "the"), Seq("The")) - ).toDF("input1", "input2") - intercept[IllegalArgumentException] { - remover.transform(df).count() - } - } - - test("Multiple Columns: Set both of inputCol/inputCols") { - val remover = new StopWordsRemover() - .setInputCols(Array("input1", "input2")) - .setOutputCols(Array("result1", "result2")) - .setInputCol("input1") - val df = Seq( - (Seq("A"), Seq("A")), - (Seq("The", "the"), Seq("The")) - ).toDF("input1", "input2") - intercept[IllegalArgumentException] { - remover.transform(df).count() - } - } } diff --git a/pom.xml b/pom.xml index 44593b78c9a0..5110285547ab 100644 --- a/pom.xml +++ b/pom.xml @@ -200,9 +200,9 @@ 1.0.0 - 0.15.1 + 0.12.0 ${java.home} @@ -2326,7 +2326,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports - -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true + -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} - -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true + -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} 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 f28fc2153806..d16611f41203 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 @@ -321,7 +321,7 @@ private[ui] class StreamingPage(parent: StreamingTab) if (hasStream) { - + Input Rate @@ -351,7 +351,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Scheduling Delay {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "top")}
    +
    Scheduling Delay {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "right")}
    Avg: {schedulingDelay.formattedAvg}
    @@ -361,7 +361,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Processing Time {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "top")}
    +
    Processing Time {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "right")}
    Avg: {processingTime.formattedAvg}
    @@ -371,7 +371,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "top")}
    +
    Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "right")}
    Avg: {totalDelay.formattedAvg}
    diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 0976494b6d09..286095e4ee0d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import java.io.File import java.nio.ByteBuffer -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.reflect.ClassTag @@ -88,12 +87,9 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) - val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), - rpcEnv.setupEndpoint("blockmanagerHeartbeat", - new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) + new LiveListenerBus(conf), None)), conf, true) storageLevel = StorageLevel.MEMORY_ONLY_SER blockManager = createBlockManager(blockManagerSize, conf) From 34046be0ddab0385178a5ccd369807469051b816 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 2 Jan 2020 14:25:20 +0800 Subject: [PATCH 20/36] follow comment --- .../apache/spark/sql/execution/subquery.scala | 49 +++---------------- 1 file changed, 6 insertions(+), 43 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 637a95b40a7e..7911d3d72a6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -23,9 +23,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, IsNotNull, ListQuery, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, DataType, StructType} @@ -172,44 +171,6 @@ case class InSubqueryExec( } } -/** - * The physical node of non-correlated EXISTS subquery. - */ -case class ExistsSubqueryExec( - plan: BaseSubqueryExec, - exprId: ExprId) - extends ExecSubqueryExpression { - - @volatile private var result: Option[Boolean] = None - - override def dataType: DataType = BooleanType - override def children: Seq[Expression] = Nil - override def nullable: Boolean = false - override def toString: String = s"EXISTS (${plan.simpleString(SQLConf.get.maxToStringFields)})" - override def withNewPlan(plan: BaseSubqueryExec): ExistsSubqueryExec = copy(plan = plan) - - override def semanticEquals(other: Expression): Boolean = other match { - case in: ExistsSubqueryExec => plan.sameResult(in.plan) - case _ => false - } - - def updateResult(): Unit = { - result = Some(plan.executeTake(1).length == 1) - } - - def values(): Option[Boolean] = result - - override def eval(input: InternalRow): Any = { - require(result.isDefined, s"$this has not finished") - result.get - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - require(result.isDefined, s"$this has not finished") - Literal.create(result.get, dataType).doGenCode(ctx, ev) - } -} - /** * Plans subqueries that are present in the given [[SparkPlan]]. */ @@ -233,9 +194,11 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { } val executedPlan = new QueryExecution(sparkSession, query).executedPlan InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) - case expressions.Exists(sub, children, exprId) => - val executedPlan = new QueryExecution(sparkSession, Project(Nil, sub)).executedPlan - ExistsSubqueryExec(SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) + case exists: expressions.Exists => + val executedPlan = new QueryExecution(sparkSession, exists.plan).executedPlan + IsNotNull(ScalarSubquery( + SubqueryExec(s"scalar-subquery#${exists.exprId.id}", CollectLimitExec(1, executedPlan)), + exists.exprId)) } } } From 4c6c04ddbd75b158b733f6ff3c3e142025fffe2c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 2 Jan 2020 18:15:52 +0800 Subject: [PATCH 21/36] follow comment --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 1 + .../sql/catalyst/optimizer/finishAnalysis.scala | 12 ++++++++++++ .../org/apache/spark/sql/execution/subquery.scala | 5 ----- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b78bdf082f33..589634f45acc 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 @@ -128,6 +128,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateSubqueryAliases, EliminateView, ReplaceExpressions, + ReplaceNonCorrelatedExists, ComputeCurrentTime, GetCurrentDatabase(catalogManager), RewriteDistinctAggregates, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 70277526cba8..44b15e12533b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -21,8 +21,10 @@ import java.time.LocalDate import scala.collection.mutable +import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery.splitConjunctivePredicates import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -52,6 +54,16 @@ object ReplaceExpressions extends Rule[LogicalPlan] { } } +object ReplaceNonCorrelatedExists extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case exists: expressions.Exists + if !SubqueryExpression.hasInOrCorrelatedExistsSubquery(exists) => + IsNotNull( + ScalarSubquery( + plan = Limit(Literal(1, IntegerType), exists.plan), + exprId = exists.exprId)) + } +} /** * Computes the current date and time to make sure we return the same result in a single query. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 7911d3d72a6f..6092399b5545 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -194,11 +194,6 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { } val executedPlan = new QueryExecution(sparkSession, query).executedPlan InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) - case exists: expressions.Exists => - val executedPlan = new QueryExecution(sparkSession, exists.plan).executedPlan - IsNotNull(ScalarSubquery( - SubqueryExec(s"scalar-subquery#${exists.exprId.id}", CollectLimitExec(1, executedPlan)), - exists.exprId)) } } } From ac6a4d244c70f12dba4163b0c06e6f9893f9930a Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 2 Jan 2020 18:17:17 +0800 Subject: [PATCH 22/36] Update subquery.scala --- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 6092399b5545..adb4956c6d3c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, ExprId, InSet, IsNotNull, ListQuery, Literal, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, CreateNamedStruct, Expression, ExprId, InSet, ListQuery, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf From 59162c65ee295fe06913dd93f93644264273f757 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 2 Jan 2020 18:18:38 +0800 Subject: [PATCH 23/36] Update finishAnalysis.scala --- .../org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 44b15e12533b..cff206d3aa50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -24,7 +24,6 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery.splitConjunctivePredicates import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.util.DateTimeUtils From 89a172187098aa423f74377ff26e75df41165bc4 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 2 Jan 2020 18:19:54 +0800 Subject: [PATCH 24/36] Update finishAnalysis.scala --- .../apache/spark/sql/catalyst/optimizer/finishAnalysis.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index cff206d3aa50..4cf8aa07f7cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -21,7 +21,6 @@ import java.time.LocalDate import scala.collection.mutable -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -55,7 +54,7 @@ object ReplaceExpressions extends Rule[LogicalPlan] { object ReplaceNonCorrelatedExists extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case exists: expressions.Exists + case exists: Exists if !SubqueryExpression.hasInOrCorrelatedExistsSubquery(exists) => IsNotNull( ScalarSubquery( From fb98b540c5b5cf2ce96a176035afe5e21746d7c2 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 2 Jan 2020 20:03:06 +0800 Subject: [PATCH 25/36] update --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- .../sql/catalyst/optimizer/finishAnalysis.scala | 16 ++++++++++------ 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 589634f45acc..9b61a0a02127 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 @@ -128,7 +128,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateSubqueryAliases, EliminateView, ReplaceExpressions, - ReplaceNonCorrelatedExists, + RewriteNonCorrelatedExists, ComputeCurrentTime, GetCurrentDatabase(catalogManager), RewriteDistinctAggregates, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 4cf8aa07f7cd..df62f362749d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -52,14 +52,18 @@ object ReplaceExpressions extends Rule[LogicalPlan] { } } -object ReplaceNonCorrelatedExists extends Rule[LogicalPlan] { +/** + * Rewrite non correlated exists subquery to use ScalarSubquery + * WHERE EXISTS ( SELECT A FROM TABLE B WHERE COL1 > 10) + * will be rewrite to + * WHERE (SELECT A FROM TABLE B WHERE COL1 > 1 LIMIT 1) IS NOT NULL + */ +object RewriteNonCorrelatedExists extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case exists: Exists - if !SubqueryExpression.hasInOrCorrelatedExistsSubquery(exists) => + case exists: Exists if exists.children.isEmpty && + !SubqueryExpression.hasInOrCorrelatedExistsSubquery(exists) => IsNotNull( - ScalarSubquery( - plan = Limit(Literal(1, IntegerType), exists.plan), - exprId = exists.exprId)) + ScalarSubquery(plan = Limit(Literal(1), exists.plan), exprId = exists.exprId)) } } From 67b42815ebfdda612106307a586f5c763626c02d Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 3 Jan 2020 09:59:49 +0800 Subject: [PATCH 26/36] Update finishAnalysis.scala --- .../spark/sql/catalyst/optimizer/finishAnalysis.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index df62f362749d..b7e20d3424d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -60,10 +60,11 @@ object ReplaceExpressions extends Rule[LogicalPlan] { */ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case exists: Exists if exists.children.isEmpty && - !SubqueryExpression.hasInOrCorrelatedExistsSubquery(exists) => + case exists: Exists if exists.children.isEmpty => IsNotNull( - ScalarSubquery(plan = Limit(Literal(1), exists.plan), exprId = exists.exprId)) + ScalarSubquery( + plan = Limit(Literal(1), Project(Seq(Alias(Literal(1), "col")()), exists.plan)), + exprId = exists.exprId)) } } From 821ed40ff855260b15fb55e0c3ed893d2f8ce505 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 3 Jan 2020 11:12:02 +0800 Subject: [PATCH 27/36] Update finishAnalysis.scala --- .../apache/spark/sql/catalyst/optimizer/finishAnalysis.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index b7e20d3424d5..6a9726da94cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -54,9 +54,9 @@ object ReplaceExpressions extends Rule[LogicalPlan] { /** * Rewrite non correlated exists subquery to use ScalarSubquery - * WHERE EXISTS ( SELECT A FROM TABLE B WHERE COL1 > 10) + * WHERE EXISTS (SELECT A FROM TABLE B WHERE COL1 > 10) * will be rewrite to - * WHERE (SELECT A FROM TABLE B WHERE COL1 > 1 LIMIT 1) IS NOT NULL + * WHERE (SELECT 1 FROM (SELECT A FROM TABLE B WHERE COL1 > 1) LIMIT 1) IS NOT NULL */ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { From e319fee5a935b86aef64b855f7d26d956053fb36 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 3 Jan 2020 15:41:46 +0800 Subject: [PATCH 28/36] fix ut --- .../src/test/scala/org/apache/spark/sql/CachedTableSuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/SubquerySuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 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 edafdaa54945..fcfc3e3dd60b 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 @@ -785,7 +785,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi assert(getNumInMemoryRelations(ds) == 2) val cachedDs = sql(sqlText).cache() - assert(getNumInMemoryTablesRecursively(cachedDs.queryExecution.sparkPlan) == 3) + assert(getNumInMemoryTablesRecursively(cachedDs.queryExecution.sparkPlan) == 2) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index e05af08dfb74..1534f0a5f60b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1031,7 +1031,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession { } } - test("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1") { + ignore("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1") { withTable("t1") { withTempPath { path => Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath) From 2c387f238996588809efa038b3347376b9aeba3f Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 3 Jan 2020 17:10:29 +0800 Subject: [PATCH 29/36] Update SubquerySuite.scala --- .../scala/org/apache/spark/sql/SubquerySuite.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 1534f0a5f60b..ef6dcb812a8a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1031,17 +1031,19 @@ class SubquerySuite extends QueryTest with SharedSparkSession { } } - ignore("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1") { + test("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1") { withTable("t1") { withTempPath { path => - Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath) - sql(s"CREATE TABLE t1 USING parquet LOCATION '${path.toURI}'") + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath + "/t1") + Seq(2 -> "b").toDF("i", "j").write.parquet(path.getCanonicalPath + "/t2") + sql(s"CREATE TABLE t1 USING parquet LOCATION '${path.toURI}/t1'") + sql(s"CREATE TABLE t2 USING parquet LOCATION '${path.toURI}/t2'") val sqlText = """ |SELECT * FROM t1 |WHERE - |NOT EXISTS (SELECT * FROM t1) + |NOT EXISTS (SELECT * FROM t2 WHERE t1.i = t2.i) """.stripMargin val optimizedPlan = sql(sqlText).queryExecution.optimizedPlan val join = optimizedPlan.collectFirst { case j: Join => j }.get From 2aff8eb09069c5daf853388ad92b975a6f4d2cd1 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Fri, 3 Jan 2020 17:17:19 +0800 Subject: [PATCH 30/36] Update SubquerySuite.scala --- .../scala/org/apache/spark/sql/SubquerySuite.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index ef6dcb812a8a..e24219ac33b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1034,16 +1034,14 @@ class SubquerySuite extends QueryTest with SharedSparkSession { test("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1") { withTable("t1") { withTempPath { path => - Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath + "/t1") - Seq(2 -> "b").toDF("i", "j").write.parquet(path.getCanonicalPath + "/t2") - sql(s"CREATE TABLE t1 USING parquet LOCATION '${path.toURI}/t1'") - sql(s"CREATE TABLE t2 USING parquet LOCATION '${path.toURI}/t2'") + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath) + sql(s"CREATE TABLE t1 USING parquet LOCATION '${path.toURI}'") val sqlText = """ - |SELECT * FROM t1 + |SELECT * FROM t1 a |WHERE - |NOT EXISTS (SELECT * FROM t2 WHERE t1.i = t2.i) + |NOT EXISTS (SELECT * FROM t1 b WHERE a.i = b.i) """.stripMargin val optimizedPlan = sql(sqlText).queryExecution.optimizedPlan val join = optimizedPlan.collectFirst { case j: Join => j }.get From 2b7b4171021f391d4b718cc38748a534b0f7dead Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 4 Jan 2020 10:28:13 +0800 Subject: [PATCH 31/36] Update CachedTableSuite.scala --- .../apache/spark/sql/CachedTableSuite.scala | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 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 fcfc3e3dd60b..442a7f8f84d9 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 @@ -25,9 +25,9 @@ import org.apache.spark.executor.DataReadMethod._ import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} -import org.apache.spark.sql.execution.{RDDScanExec, SparkPlan} +import org.apache.spark.sql.execution.{RDDScanExec, ScalarSubquery => ExecScalarSubquery, SparkPlan} import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.functions._ @@ -87,10 +87,24 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi sum } + private def getExpressionSubqueryInMemoryTables(plan: SparkPlan): Int = { + var inMemoryTableNum = 0 + plan.transformExpressions { + case sub: ExecScalarSubquery => + inMemoryTableNum = inMemoryTableNum + getNumInMemoryTablesRecursively(sub.plan) + sub + case e => e + } + inMemoryTableNum + } + private def getNumInMemoryTablesRecursively(plan: SparkPlan): Int = { plan.collect { - case InMemoryTableScanExec(_, _, relation) => - getNumInMemoryTablesRecursively(relation.cachedPlan) + 1 + case inMemoryTable @ InMemoryTableScanExec(_, _, relation) => + getNumInMemoryTablesRecursively(relation.cachedPlan) + + getExpressionSubqueryInMemoryTables(inMemoryTable) + 1 + case p => + getExpressionSubqueryInMemoryTables(p) }.sum } @@ -785,7 +799,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi assert(getNumInMemoryRelations(ds) == 2) val cachedDs = sql(sqlText).cache() - assert(getNumInMemoryTablesRecursively(cachedDs.queryExecution.sparkPlan) == 2) + assert(getNumInMemoryTablesRecursively(cachedDs.queryExecution.sparkPlan) == 3) } } From 88fcdbf7aa8f14679b84863307a797bb04f07048 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 4 Jan 2020 10:30:12 +0800 Subject: [PATCH 32/36] Update CachedTableSuite.scala --- .../test/scala/org/apache/spark/sql/CachedTableSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 442a7f8f84d9..c615c08860df 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 @@ -25,9 +25,9 @@ import org.apache.spark.executor.DataReadMethod._ import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} -import org.apache.spark.sql.execution.{RDDScanExec, ScalarSubquery => ExecScalarSubquery, SparkPlan} +import org.apache.spark.sql.execution.{RDDScanExec, ScalarSubquery, SparkPlan} import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.functions._ @@ -90,7 +90,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi private def getExpressionSubqueryInMemoryTables(plan: SparkPlan): Int = { var inMemoryTableNum = 0 plan.transformExpressions { - case sub: ExecScalarSubquery => + case sub: ScalarSubquery => inMemoryTableNum = inMemoryTableNum + getNumInMemoryTablesRecursively(sub.plan) sub case e => e From 8c6060a1a395c81cbd08d0afc25490b533493b69 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 4 Jan 2020 13:31:28 +0800 Subject: [PATCH 33/36] Update CachedTableSuite.scala --- .../apache/spark/sql/CachedTableSuite.scala | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 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 b9ab34afc3c7..e78d0e850e44 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} import org.apache.spark.sql.catalyst.util.DateTimeConstants -import org.apache.spark.sql.execution.{RDDScanExec, ScalarSubquery, SparkPlan} +import org.apache.spark.sql.execution.{ExecSubqueryExpression, RDDScanExec, SparkPlan} import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.functions._ @@ -89,24 +89,19 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi sum } - private def getExpressionSubqueryInMemoryTables(plan: SparkPlan): Int = { - var inMemoryTableNum = 0 - plan.transformExpressions { - case sub: ScalarSubquery => - inMemoryTableNum = inMemoryTableNum + getNumInMemoryTablesRecursively(sub.plan) - sub - case e => e - } - inMemoryTableNum + private def getNumInMemoryTablesInSubquery(plan: SparkPlan): Int = { + plan.expressions.map(_.collect { + case sub: ExecSubqueryExpression => getNumInMemoryTablesRecursively(sub.plan) + }.sum).sum } private def getNumInMemoryTablesRecursively(plan: SparkPlan): Int = { plan.collect { case inMemoryTable @ InMemoryTableScanExec(_, _, relation) => getNumInMemoryTablesRecursively(relation.cachedPlan) + - getExpressionSubqueryInMemoryTables(inMemoryTable) + 1 + getNumInMemoryTablesInSubquery(inMemoryTable) + 1 case p => - getExpressionSubqueryInMemoryTables(p) + getNumInMemoryTablesInSubquery(p) }.sum } From 9a9d9d1ffe8a47736e16252faabfd65a47a18807 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 5 Jan 2020 20:42:53 +0800 Subject: [PATCH 34/36] fix comment error --- .../apache/spark/sql/catalyst/optimizer/finishAnalysis.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 0a31cef974f4..0886f3f859fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -56,7 +56,7 @@ object ReplaceExpressions extends Rule[LogicalPlan] { * Rewrite non correlated exists subquery to use ScalarSubquery * WHERE EXISTS (SELECT A FROM TABLE B WHERE COL1 > 10) * will be rewrite to - * WHERE (SELECT 1 FROM (SELECT A FROM TABLE B WHERE COL1 > 1) LIMIT 1) IS NOT NULL + * WHERE (SELECT 1 FROM (SELECT A FROM TABLE B WHERE COL1 > 10) LIMIT 1) IS NOT NULL */ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { From 173942dc8006e24a3e775877be11b0a492f35968 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 6 Jan 2020 09:53:04 +0800 Subject: [PATCH 35/36] follow comment --- .../sql/catalyst/expressions/subquery.scala | 19 +++++++++++++++++-- .../catalyst/optimizer/finishAnalysis.scala | 2 +- .../apache/spark/sql/CachedTableSuite.scala | 4 ++-- 3 files changed, 20 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index e6c4be9d8173..e33cff2f14e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -62,7 +62,8 @@ abstract class SubqueryExpression( object SubqueryExpression { /** - * Returns true when an expression contains an IN or EXISTS subquery and false otherwise. + * Returns true when an expression contains an IN or correlated EXISTS subquery + * and false otherwise. */ def hasInOrCorrelatedExistsSubquery(e: Expression): Boolean = { e.find { @@ -303,7 +304,10 @@ case class ListQuery( } /** - * The [[Exists]] expression checks if a row exists in a subquery given some correlated condition. + * The [[Exists]] expression checks if a row exists in a subquery given some correlated condition + * or some uncorrelated condition. + * + * 1. correlated condition: * * For example (SQL): * {{{ @@ -313,6 +317,17 @@ case class ListQuery( * FROM b * WHERE b.id = a.id) * }}} + * + * 2. uncorrelated condition example: + * + * For example (SQL): + * {{{ + * SELECT * + * FROM a + * WHERE EXISTS (SELECT * + * FROM b + * WHERE b.id > 10) + * }}} */ case class Exists( plan: LogicalPlan, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 0886f3f859fe..24112064a873 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -53,7 +53,7 @@ object ReplaceExpressions extends Rule[LogicalPlan] { } /** - * Rewrite non correlated exists subquery to use ScalarSubquery + * Rewritten uncorrelated exists subquery to use ScalarSubquery * WHERE EXISTS (SELECT A FROM TABLE B WHERE COL1 > 10) * will be rewrite to * WHERE (SELECT 1 FROM (SELECT A FROM TABLE B WHERE COL1 > 10) LIMIT 1) IS NOT NULL 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 e78d0e850e44..85619beee0c9 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 @@ -90,9 +90,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi } private def getNumInMemoryTablesInSubquery(plan: SparkPlan): Int = { - plan.expressions.map(_.collect { + plan.expressions.flatMap(_.collect { case sub: ExecSubqueryExpression => getNumInMemoryTablesRecursively(sub.plan) - }.sum).sum + }).sum } private def getNumInMemoryTablesRecursively(plan: SparkPlan): Int = { From 26258b0bb285644ea9d1b43f7ac20a7e02c5d6f4 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 6 Jan 2020 11:57:10 +0800 Subject: [PATCH 36/36] Update finishAnalysis.scala --- .../apache/spark/sql/catalyst/optimizer/finishAnalysis.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 24112064a873..c79bf3e20b77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -53,9 +53,9 @@ object ReplaceExpressions extends Rule[LogicalPlan] { } /** - * Rewritten uncorrelated exists subquery to use ScalarSubquery + * Rewrite non correlated exists subquery to use ScalarSubquery * WHERE EXISTS (SELECT A FROM TABLE B WHERE COL1 > 10) - * will be rewrite to + * will be rewritten to * WHERE (SELECT 1 FROM (SELECT A FROM TABLE B WHERE COL1 > 10) LIMIT 1) IS NOT NULL */ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] {