[ https://issues.apache.org/jira/browse/SPARK-21807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16149600#comment-16149600 ]
Andrew Ash commented on SPARK-21807: ------------------------------------ For reference, here's a stacktrace I'm seeing on a cluster before this change that I think this PR will improve: {noformat} "spark-task-4" #714 prio=5 os_prio=0 tid=0x00007fa368031000 nid=0x4d91 runnable [0x00007fa24e592000] java.lang.Thread.State: RUNNABLE at org.apache.spark.sql.catalyst.expressions.AttributeReference.equals(namedExpressions.scala:220) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.Add.equals(arithmetic.scala:149) at org.apache.spark.sql.catalyst.expressions.EqualNullSafe.equals(predicates.scala:505) at scala.collection.mutable.FlatHashTable$class.addEntry(FlatHashTable.scala:151) at scala.collection.mutable.HashSet.addEntry(HashSet.scala:40) at scala.collection.mutable.FlatHashTable$class.growTable(FlatHashTable.scala:225) at scala.collection.mutable.FlatHashTable$class.addEntry(FlatHashTable.scala:159) at scala.collection.mutable.HashSet.addEntry(HashSet.scala:40) at scala.collection.mutable.FlatHashTable$class.addElem(FlatHashTable.scala:139) at scala.collection.mutable.HashSet.addElem(HashSet.scala:40) at scala.collection.mutable.HashSet.$plus$eq(HashSet.scala:59) at scala.collection.mutable.HashSet.$plus$eq(HashSet.scala:40) at scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59) at scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59) at scala.collection.mutable.HashSet.foreach(HashSet.scala:78) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.AbstractSet.$plus$plus$eq(Set.scala:46) at scala.collection.mutable.HashSet.clone(HashSet.scala:83) at scala.collection.mutable.HashSet.clone(HashSet.scala:40) at org.apache.spark.sql.catalyst.expressions.ExpressionSet.$plus(ExpressionSet.scala:65) at org.apache.spark.sql.catalyst.expressions.ExpressionSet.$plus(ExpressionSet.scala:50) at scala.collection.SetLike$$anonfun$$plus$plus$1.apply(SetLike.scala:141) at scala.collection.SetLike$$anonfun$$plus$plus$1.apply(SetLike.scala:141) at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) at scala.collection.immutable.HashSet$HashSet1.foreach(HashSet.scala:316) at scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:972) at scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:972) at scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:972) at scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:972) at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) at scala.collection.TraversableOnce$class.$div$colon(TraversableOnce.scala:151) at scala.collection.AbstractTraversable.$div$colon(Traversable.scala:104) at scala.collection.SetLike$class.$plus$plus(SetLike.scala:141) at org.apache.spark.sql.catalyst.expressions.ExpressionSet.$plus$plus(ExpressionSet.scala:50) at org.apache.spark.sql.catalyst.plans.logical.UnaryNode$$anonfun$getAliasedConstraints$1.apply(LogicalPlan.scala:322) at org.apache.spark.sql.catalyst.plans.logical.UnaryNode$$anonfun$getAliasedConstraints$1.apply(LogicalPlan.scala:319) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.getAliasedConstraints(LogicalPlan.scala:319) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x00000004004411e8> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441238> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441288> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x00000004004412d8> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441328> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441378> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x00000004004413c8> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441418> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441468> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x00000004004414b8> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441508> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441558> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x00000004004415a8> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x00000004004415f8> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441648> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441698> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x00000004004416e8> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441738> (a org.apache.spark.sql.catalyst.plans.logical.Project) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.Join.validConstraints(basicLogicalOperators.scala:332) at org.apache.spark.sql.catalyst.plans.logical.QueryPlanConstraints$class.constraints(QueryPlanConstraints.scala:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints$lzycompute(LogicalPlan.scala:29) - locked <0x0000000400441788> (a org.apache.spark.sql.catalyst.plans.logical.Join) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.constraints(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$inferFilters$1.applyOrElse(Optimizer.scala:660) at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$inferFilters$1.applyOrElse(Optimizer.scala:647) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256) at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.inferFilters(Optimizer.scala:647) at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:641) at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:637) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:85) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:82) at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124) at scala.collection.immutable.List.foldLeft(List.scala:84) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:82) at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:74) at scala.collection.immutable.List.foreach(List.scala:381) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:74) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:78) - locked <0x0000000400441ad8> (a org.apache.spark.sql.execution.QueryExecution) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:78) at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:216) at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:216) at org.apache.spark.sql.execution.QueryExecution.stringOrError(QueryExecution.scala:112) at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:216) <snip> {noformat} Note this line: {{org.apache.spark.sql.catalyst.expressions.ExpressionSet.$plus$plus(ExpressionSet.scala:50)}} > The getAliasedConstraints function in LogicalPlan will take a long time when > number of expressions is greater than 100 > ------------------------------------------------------------------------------------------------------------------------ > > Key: SPARK-21807 > URL: https://issues.apache.org/jira/browse/SPARK-21807 > Project: Spark > Issue Type: Improvement > Components: SQL > Affects Versions: 2.2.0 > Reporter: eaton > Assignee: eaton > Fix For: 2.3.0 > > > The getAliasedConstraints fuction in LogicalPlan.scala will clone the > expression set when an element added, > and it will take a long time. > Before modified, the cost of getAliasedConstraints is: > 100 expressions: 41 seconds > 150 expressions: 466 seconds > The test is like this: > test("getAliasedConstraints") { > val expressionNum = 150 > val aggExpression = (1 to expressionNum).map(i => Alias(Count(Literal(1)), > s"cnt$i")()) > val aggPlan = Aggregate(Nil, aggExpression, LocalRelation()) > val beginTime = System.currentTimeMillis() > val expressions = aggPlan.validConstraints > println(s"validConstraints cost: ${System.currentTimeMillis() - beginTime}ms") > // The size of Aliased expression is n * (n - 1) / 2 + n > assert( expressions.size === expressionNum * (expressionNum - 1) / 2 + > expressionNum) > } -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org