You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by "Rosbrook, Andrew J" <an...@jpmchase.com.INVALID> on 2018/08/24 10:37:29 UTC

Slow Query Plan Generation

Hello,

We have a pyspark job which takes around 15 minutes to execute, we are observing that 7-8 minutes of this time is spent inside the driver program generating a query plan.

The delay can be seen in the history server;

[cid:image003.jpg@01D43B9E.CB2777E0]

And the driver log file;

[cid:image004.png@01D43B9B.8E2B9EA0]


I executed jstack a few times and observed "Thread-3" busy generating a query plan. Below is the thread dump from one observation.

The job uses DataFrames and involves groupBy's and pyspark udfs executed over windows.

Any advice on how to reduce this time?

Many Thanks,
Andy



"Thread-3" #30 daemon prio=5 os_prio=0 tid=0x00007faff0008800 nid=0x2e235 runnable [0x00007fb08af62000]
   java.lang.Thread.State: RUNNABLE
                at scala.collection.mutable.FlatHashTable$class.addEntry(FlatHashTable.scala:148)
                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.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:300)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode$$anonfun$getAliasedConstraints$1.apply(LogicalPlan.scala:297)
                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:297)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae189a0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae31fd0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae35ef8> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae35ea0> (a org.apache.spark.sql.catalyst.plans.logical.BroadcastHint)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae2ebd0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Union$$anonfun$validConstraints$1.apply(basicLogicalOperators.scala:257)
                at org.apache.spark.sql.catalyst.plans.logical.Union$$anonfun$validConstraints$1.apply(basicLogicalOperators.scala:257)
                at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
                at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
                at scala.collection.immutable.List.foreach(List.scala:381)
                at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
                at scala.collection.immutable.List.map(List.scala:285)
                at org.apache.spark.sql.catalyst.plans.logical.Union.validConstraints(basicLogicalOperators.scala:257)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14510> (a org.apache.spark.sql.catalyst.plans.logical.Union)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae145d0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14628> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14680> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae146d8> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14730> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14788> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae147e0> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14838> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14890> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae148e8> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14940> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14998> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae149f0> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14a48> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14aa0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14af8> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Filter.validConstraints(basicLogicalOperators.scala:116)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14b50> (a org.apache.spark.sql.catalyst.plans.logical.Filter)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$apply$11.applyOrElse(Optimizer.scala:564)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$apply$11.applyOrElse(Optimizer.scala:562)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:288)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:288)
                at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:287)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:277)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:562)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:561)
                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.IndexedSeqOptimized$class.foldl(IndexedSeqOptimized.scala:57)
                at scala.collection.IndexedSeqOptimized$class.foldLeft(IndexedSeqOptimized.scala:66)
                at scala.collection.mutable.WrappedArray.foldLeft(WrappedArray.scala:35)
                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:73)
                - locked <0x00000000da551b98> (a org.apache.spark.sql.execution.QueryExecution)
                at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:73)
                at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:230)
                at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:230)
                at org.apache.spark.sql.execution.QueryExecution.stringOrError(QueryExecution.scala:107)
                at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:230)
                at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:121)
                at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:101)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
                - locked <0x00000000da551cd8> (a org.apache.spark.sql.execution.command.ExecutedCommandExec)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
                at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
                at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
                at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
                at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:87)
                - locked <0x00000000da551d80> (a org.apache.spark.sql.execution.QueryExecution)
                at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:87)
                at org.apache.spark.sql.execution.datasources.DataSource.writeInFileFormat(DataSource.scala:484)
                at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:520)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:215)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:198)
                at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:494)
                at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
                at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
                at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
                at java.lang.reflect.Method.invoke(Method.java:497)
                at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
                at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
                at py4j.Gateway.invoke(Gateway.java:280)
                at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
                at py4j.commands.CallCommand.execute(CallCommand.java:79)
                at py4j.GatewayConnection.run(GatewayConnection.java:214)
                at java.lang.Thread.run(Thread.java:745)


This message is confidential and subject to terms at: http://www.jpmorgan.com/emaildisclaimer including on confidentiality, legal privilege, viruses and monitoring of electronic messages. If you are not the intended recipient, please delete this message and notify the sender immediately. Any unauthorized use is strictly prohibited.

RE: Slow Query Plan Generation

Posted by "Rosbrook, Andrew J" <an...@jpmchase.com.INVALID>.
Thanks Kazuaki, this has fixed the problem :)


From: Kazuaki Ishizaki [mailto:ISHIZAKI@jp.ibm.com]
Sent: 24 August 2018 18:12
To: Rosbrook, Andrew J <an...@jpmchase.com.INVALID>
Cc: user@spark.apache.org
Subject: Re: Slow Query Plan Generation

Hi
Does setting false to this property "spark.sql.constraintPropagation.enabled" alleviate this issue?

Regards,
Kazuaki Ishizaki



From:        "Rosbrook, Andrew J" <an...@jpmchase.com>.INVALID>
To:        "user@spark.apache.org<ma...@spark.apache.org>" <us...@spark.apache.org>>
Date:        2018/08/24 19:38
Subject:        Slow Query Plan Generation
________________________________



Hello,

We have a pyspark job which takes around 15 minutes to execute, we are observing that 7-8 minutes of this time is spent inside the driver program generating a query plan.

The delay can be seen in the history server;

[cid:image001.jpg@01D43ECC.CD5AD6C0]

And the driver log file;

[cid:image002.png@01D43ECC.CD5AD6C0]


I executed jstack a few times and observed "Thread-3" busy generating a query plan. Below is the thread dump from one observation.

The job uses DataFrames and involves groupBy's and pyspark udfs executed over windows.

Any advice on how to reduce this time?

Many Thanks,
Andy



"Thread-3" #30 daemon prio=5 os_prio=0 tid=0x00007faff0008800 nid=0x2e235 runnable [0x00007fb08af62000]
   java.lang.Thread.State: RUNNABLE
                at scala.collection.mutable.FlatHashTable$class.addEntry(FlatHashTable.scala:148)
                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.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:300)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode$$anonfun$getAliasedConstraints$1.apply(LogicalPlan.scala:297)
                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:297)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae189a0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae31fd0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae35ef8> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae35ea0> (a org.apache.spark.sql.catalyst.plans.logical.BroadcastHint)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae2ebd0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Union$$anonfun$validConstraints$1.apply(basicLogicalOperators.scala:257)
                at org.apache.spark.sql.catalyst.plans.logical.Union$$anonfun$validConstraints$1.apply(basicLogicalOperators.scala:257)
                at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
                at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
                at scala.collection.immutable.List.foreach(List.scala:381)
                at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
                at scala.collection.immutable.List.map(List.scala:285)
                at org.apache.spark.sql.catalyst.plans.logical.Union.validConstraints(basicLogicalOperators.scala:257)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14510> (a org.apache.spark.sql.catalyst.plans.logical.Union)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae145d0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14628> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14680> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae146d8> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14730> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14788> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae147e0> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14838> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14890> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae148e8> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14940> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14998> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae149f0> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14a48> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14aa0> (a org.apache.spark.sql.catalyst.plans.logical.Project)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14af8> (a org.apache.spark.sql.catalyst.plans.logical.Window)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.plans.logical.Filter.validConstraints(basicLogicalOperators.scala:116)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14b50> (a org.apache.spark.sql.catalyst.plans.logical.Filter)
                at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$apply$11.applyOrElse(Optimizer.scala:564)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$apply$11.applyOrElse(Optimizer.scala:562)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:288)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:288)
                at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:287)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:277)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:562)
                at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:561)
                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.IndexedSeqOptimized$class.foldl(IndexedSeqOptimized.scala:57)
                at scala.collection.IndexedSeqOptimized$class.foldLeft(IndexedSeqOptimized.scala:66)
                at scala.collection.mutable.WrappedArray.foldLeft(WrappedArray.scala:35)
                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:73)
                - locked <0x00000000da551b98> (a org.apache.spark.sql.execution.QueryExecution)
                at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:73)
                at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:230)
                at org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:230)
                at org.apache.spark.sql.execution.QueryExecution.stringOrError(QueryExecution.scala:107)
                at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:230)
                at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
                at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:121)
                at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:101)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
                - locked <0x00000000da551cd8> (a org.apache.spark.sql.execution.command.ExecutedCommandExec)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
                at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
                at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
                at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
                at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
                at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:87)
                - locked <0x00000000da551d80> (a org.apache.spark.sql.execution.QueryExecution)
                at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:87)
                at org.apache.spark.sql.execution.datasources.DataSource.writeInFileFormat(DataSource.scala:484)
                at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:520)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:215)
                at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:198)
                at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:494)
                at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
                at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
                at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
                at java.lang.reflect.Method.invoke(Method.java:497)
                at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
                at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
                at py4j.Gateway.invoke(Gateway.java:280)
                at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
                at py4j.commands.CallCommand.execute(CallCommand.java:79)
                at py4j.GatewayConnection.run(GatewayConnection.java:214)
                at java.lang.Thread.run(Thread.java:745)


This message is confidential and subject to terms at: http://www.jpmorgan.com/emaildisclaimer<http://www.jpmorgan.com/emaildisclaimer>including on confidentiality, legal privilege, viruses and monitoring of electronic messages. If you are not the intended recipient, please delete this message and notify the sender immediately. Any unauthorized use is strictly prohibited.



This message is confidential and subject to terms at: http://www.jpmorgan.com/emaildisclaimer including on confidentiality, legal privilege, viruses and monitoring of electronic messages. If you are not the intended recipient, please delete this message and notify the sender immediately. Any unauthorized use is strictly prohibited.

Re: Slow Query Plan Generation

Posted by Kazuaki Ishizaki <IS...@jp.ibm.com>.
Hi 
Does setting false to this property 
"spark.sql.constraintPropagation.enabled" alleviate this issue?

Regards,
Kazuaki Ishizaki



From:   "Rosbrook, Andrew J" <an...@jpmchase.com.INVALID>
To:     "user@spark.apache.org" <us...@spark.apache.org>
Date:   2018/08/24 19:38
Subject:        Slow Query Plan Generation



Hello,
 
We have a pyspark job which takes around 15 minutes to execute, we are 
observing that 7-8 minutes of this time is spent inside the driver program 
generating a query plan.
 
The delay can be seen in the history server;
 

 
And the driver log file;
 

 
 
I executed jstack a few times and observed “Thread-3” busy generating a 
query plan. Below is the thread dump from one observation.
 
The job uses DataFrames and involves groupBy’s and pyspark udfs executed 
over windows. 
 
Any advice on how to reduce this time? 
 
Many Thanks,
Andy
 
 
 
"Thread-3" #30 daemon prio=5 os_prio=0 tid=0x00007faff0008800 nid=0x2e235 
runnable [0x00007fb08af62000]
   java.lang.Thread.State: RUNNABLE
                at 
scala.collection.mutable.FlatHashTable$class.addEntry(FlatHashTable.scala:148)
                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.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:300)
                at 
org.apache.spark.sql.catalyst.plans.logical.UnaryNode$$anonfun$getAliasedConstraints$1.apply(LogicalPlan.scala:297)
                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:297)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae189a0> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae31fd0> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae35ef8> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae35ea0> (a 
org.apache.spark.sql.catalyst.plans.logical.BroadcastHint)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae2ebd0> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Union$$anonfun$validConstraints$1.apply(basicLogicalOperators.scala:257)
                at 
org.apache.spark.sql.catalyst.plans.logical.Union$$anonfun$validConstraints$1.apply(basicLogicalOperators.scala:257)
                at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
                at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
                at scala.collection.immutable.List.foreach(List.scala:381)
                at 
scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
                at scala.collection.immutable.List.map(List.scala:285)
                at 
org.apache.spark.sql.catalyst.plans.logical.Union.validConstraints(basicLogicalOperators.scala:257)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14510> (a 
org.apache.spark.sql.catalyst.plans.logical.Union)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae145d0> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14628> (a 
org.apache.spark.sql.catalyst.plans.logical.Window)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14680> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae146d8> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14730> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14788> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae147e0> (a 
org.apache.spark.sql.catalyst.plans.logical.Window)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14838> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14890> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae148e8> (a 
org.apache.spark.sql.catalyst.plans.logical.Window)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14940> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14998> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae149f0> (a 
org.apache.spark.sql.catalyst.plans.logical.Window)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14a48> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:58)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14aa0> (a 
org.apache.spark.sql.catalyst.plans.logical.Project)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.UnaryNode.validConstraints(LogicalPlan.scala:311)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14af8> (a 
org.apache.spark.sql.catalyst.plans.logical.Window)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.plans.logical.Filter.validConstraints(basicLogicalOperators.scala:116)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
                - locked <0x00000000dae14b50> (a 
org.apache.spark.sql.catalyst.plans.logical.Filter)
                at 
org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
                at 
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$apply$11.applyOrElse(Optimizer.scala:564)
                at 
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$apply$11.applyOrElse(Optimizer.scala:562)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:288)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:288)
                at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:287)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:293)
                at 
org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:277)
                at 
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:562)
                at 
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$.apply(Optimizer.scala:561)
                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.IndexedSeqOptimized$class.foldl(IndexedSeqOptimized.scala:57)
                at 
scala.collection.IndexedSeqOptimized$class.foldLeft(IndexedSeqOptimized.scala:66)
                at 
scala.collection.mutable.WrappedArray.foldLeft(WrappedArray.scala:35)
                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:73)
                - locked <0x00000000da551b98> (a 
org.apache.spark.sql.execution.QueryExecution)
                at 
org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:73)
                at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:230)
                at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:230)
                at 
org.apache.spark.sql.execution.QueryExecution.stringOrError(QueryExecution.scala:107)
                at 
org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:230)
                at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
                at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:121)
                at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:101)
                at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
                - locked <0x00000000da551cd8> (a 
org.apache.spark.sql.execution.command.ExecutedCommandExec)
                at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
                at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
                at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
                at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
                at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
                at 
org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
                at 
org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
                at 
org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:87)
                - locked <0x00000000da551d80> (a 
org.apache.spark.sql.execution.QueryExecution)
                at 
org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:87)
                at 
org.apache.spark.sql.execution.datasources.DataSource.writeInFileFormat(DataSource.scala:484)
                at 
org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:520)
                at 
org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:215)
                at 
org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:198)
                at 
org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:494)
                at sun.reflect.NativeMethodAccessorImpl.invoke0(Native 
Method)
                at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
                at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
                at java.lang.reflect.Method.invoke(Method.java:497)
                at 
py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
                at 
py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
                at py4j.Gateway.invoke(Gateway.java:280)
                at 
py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
                at py4j.commands.CallCommand.execute(CallCommand.java:79)
                at py4j.GatewayConnection.run(GatewayConnection.java:214)
                at java.lang.Thread.run(Thread.java:745)
 
This message is confidential and subject to terms at: http://
www.jpmorgan.com/emaildisclaimer including on confidentiality, legal 
privilege, viruses and monitoring of electronic messages. If you are not 
the intended recipient, please delete this message and notify the sender 
immediately. Any unauthorized use is strictly prohibited.