diff options
author | Hiroshi Inoue <inouehrs@jp.ibm.com> | 2016-06-30 21:47:44 -0700 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2016-06-30 21:47:44 -0700 |
commit | 14cf61e909598d9f6b9c3b920de7299e9bc828e0 (patch) | |
tree | 6cda21096caf50bfde8e4de43ae211f49ad14317 /sql/catalyst | |
parent | aa6564f37f1d8de77c3b7bfa885000252efffea6 (diff) | |
download | spark-14cf61e909598d9f6b9c3b920de7299e9bc828e0.tar.gz spark-14cf61e909598d9f6b9c3b920de7299e9bc828e0.tar.bz2 spark-14cf61e909598d9f6b9c3b920de7299e9bc828e0.zip |
[SPARK-16331][SQL] Reduce code generation time
## What changes were proposed in this pull request?
During the code generation, a `LocalRelation` often has a huge `Vector` object as `data`. In the simple example below, a `LocalRelation` has a Vector with 1000000 elements of `UnsafeRow`.
```
val numRows = 1000000
val ds = (1 to numRows).toDS().persist()
benchmark.addCase("filter+reduce") { iter =>
ds.filter(a => (a & 1) == 0).reduce(_ + _)
}
```
At `TreeNode.transformChildren`, all elements of the vector is unnecessarily iterated to check whether any children exist in the vector since `Vector` is Traversable. This part significantly increases code generation time.
This patch avoids this overhead by checking the number of children before iterating all elements; `LocalRelation` does not have children since it extends `LeafNode`.
The performance of the above example
```
without this patch
Java HotSpot(TM) 64-Bit Server VM 1.8.0_91-b14 on Mac OS X 10.11.5
Intel(R) Core(TM) i5-5257U CPU 2.70GHz
compilationTime: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
filter+reduce 4426 / 4533 0.2 4426.0 1.0X
with this patch
compilationTime: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
filter+reduce 3117 / 3391 0.3 3116.6 1.0X
```
## How was this patch tested?
using existing unit tests
Author: Hiroshi Inoue <inouehrs@jp.ibm.com>
Closes #14000 from inouehrs/compilation-time-reduction.
Diffstat (limited to 'sql/catalyst')
-rw-r--r-- | sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 82 |
1 files changed, 43 insertions, 39 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 072445af4f..8bce404735 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -315,25 +315,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { protected def transformChildren( rule: PartialFunction[BaseType, BaseType], nextOperation: (BaseType, PartialFunction[BaseType, BaseType]) => BaseType): BaseType = { - var changed = false - val newArgs = mapProductIterator { - case arg: TreeNode[_] if containsChild(arg) => - val newChild = nextOperation(arg.asInstanceOf[BaseType], rule) - if (!(newChild fastEquals arg)) { - changed = true - newChild - } else { - arg - } - case Some(arg: TreeNode[_]) if containsChild(arg) => - val newChild = nextOperation(arg.asInstanceOf[BaseType], rule) - if (!(newChild fastEquals arg)) { - changed = true - Some(newChild) - } else { - Some(arg) - } - case m: Map[_, _] => m.mapValues { + if (children.nonEmpty) { + var changed = false + val newArgs = mapProductIterator { case arg: TreeNode[_] if containsChild(arg) => val newChild = nextOperation(arg.asInstanceOf[BaseType], rule) if (!(newChild fastEquals arg)) { @@ -342,33 +326,53 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } else { arg } - case other => other - }.view.force // `mapValues` is lazy and we need to force it to materialize - case d: DataType => d // Avoid unpacking Structs - case args: Traversable[_] => args.map { - case arg: TreeNode[_] if containsChild(arg) => + case Some(arg: TreeNode[_]) if containsChild(arg) => val newChild = nextOperation(arg.asInstanceOf[BaseType], rule) if (!(newChild fastEquals arg)) { changed = true - newChild + Some(newChild) } else { - arg + Some(arg) } - case tuple @ (arg1: TreeNode[_], arg2: TreeNode[_]) => - val newChild1 = nextOperation(arg1.asInstanceOf[BaseType], rule) - val newChild2 = nextOperation(arg2.asInstanceOf[BaseType], rule) - if (!(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { - changed = true - (newChild1, newChild2) - } else { - tuple - } - case other => other + case m: Map[_, _] => m.mapValues { + case arg: TreeNode[_] if containsChild(arg) => + val newChild = nextOperation(arg.asInstanceOf[BaseType], rule) + if (!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case other => other + }.view.force // `mapValues` is lazy and we need to force it to materialize + case d: DataType => d // Avoid unpacking Structs + case args: Traversable[_] => args.map { + case arg: TreeNode[_] if containsChild(arg) => + val newChild = nextOperation(arg.asInstanceOf[BaseType], rule) + if (!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case tuple@(arg1: TreeNode[_], arg2: TreeNode[_]) => + val newChild1 = nextOperation(arg1.asInstanceOf[BaseType], rule) + val newChild2 = nextOperation(arg2.asInstanceOf[BaseType], rule) + if (!(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { + changed = true + (newChild1, newChild2) + } else { + tuple + } + case other => other + } + case nonChild: AnyRef => nonChild + case null => null } - case nonChild: AnyRef => nonChild - case null => null + if (changed) makeCopy(newArgs) else this + } else { + this } - if (changed) makeCopy(newArgs) else this } /** |