Skip to content

Commit 6c02bd0

Browse files
beliefercloud-fan
authored andcommitted
[SPARK-44340][SQL] Define the computing logic through PartitionEvaluator API and use it in WindowGroupLimitExec
### What changes were proposed in this pull request? `WindowGroupLimitExec` is updated to use the PartitionEvaluator API to do execution. ### Why are the changes needed? To define the computing logic and requires the caller side to explicitly list what needs to be serialized and sent to executors ### Does this PR introduce _any_ user-facing change? 'No'. Just update the inner implementation. ### How was this patch tested? Test cases updated & running benchmark manually. ``` [info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_311-b11 on Mac OS X 10.16 [info] Intel(R) Core(TM) i7-9750H CPU 2.60GHz [info] Benchmark Top-K: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative [info] ----------------------------------------------------------------------------------------------------------------------------------------------- [info] ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 10622 11266 617 2.0 506.5 1.0X [info] ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1712 1744 19 12.2 81.6 6.2X [info] ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 23679 25107 NaN 0.9 1129.1 0.4X [info] ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 6381 6527 95 3.3 304.3 1.7X [info] RANK (PARTITION: , WindowGroupLimit: false) 11492 11631 106 1.8 548.0 0.9X [info] RANK (PARTITION: , WindowGroupLimit: true) 2675 2920 118 7.8 127.5 4.0X [info] RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 24208 24299 95 0.9 1154.3 0.4X [info] RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 6347 6478 85 3.3 302.6 1.7X [info] DENSE_RANK (PARTITION: , WindowGroupLimit: false) 11288 11959 458 1.9 538.2 0.9X [info] DENSE_RANK (PARTITION: , WindowGroupLimit: true) 2684 2945 144 7.8 128.0 4.0X [info] DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 24316 25130 711 0.9 1159.5 0.4X [info] DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 6589 6925 383 3.2 314.2 1.6X ``` Closes #41899 from beliefer/SPARK-44340. Authored-by: Jiaan Geng <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent daa9844 commit 6c02bd0

File tree

3 files changed

+253
-189
lines changed

3 files changed

+253
-189
lines changed
Lines changed: 63 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,63 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution.window
19+
20+
import org.apache.spark.{PartitionEvaluator, PartitionEvaluatorFactory}
21+
import org.apache.spark.sql.catalyst.InternalRow
22+
import org.apache.spark.sql.catalyst.expressions.{Attribute, DenseRank, Expression, Rank, RowNumber, SortOrder}
23+
import org.apache.spark.sql.execution.metric.SQLMetric
24+
25+
class WindowGroupLimitEvaluatorFactory(
26+
partitionSpec: Seq[Expression],
27+
orderSpec: Seq[SortOrder],
28+
rankLikeFunction: Expression,
29+
limit: Int,
30+
childOutput: Seq[Attribute],
31+
numOutputRows: SQLMetric)
32+
extends PartitionEvaluatorFactory[InternalRow, InternalRow] {
33+
34+
override def createEvaluator(): PartitionEvaluator[InternalRow, InternalRow] = {
35+
val limitFunc = rankLikeFunction match {
36+
case _: RowNumber =>
37+
(iter: Iterator[InternalRow]) => SimpleLimitIterator(iter, limit, numOutputRows)
38+
case _: Rank =>
39+
(iter: Iterator[InternalRow]) =>
40+
RankLimitIterator(childOutput, iter, orderSpec, limit, numOutputRows)
41+
case _: DenseRank =>
42+
(iter: Iterator[InternalRow]) =>
43+
DenseRankLimitIterator(childOutput, iter, orderSpec, limit, numOutputRows)
44+
}
45+
46+
if (partitionSpec.isEmpty) {
47+
new WindowGroupLimitPartitionEvaluator(limitFunc)
48+
} else {
49+
new WindowGroupLimitPartitionEvaluator(
50+
input => new GroupedLimitIterator(input, childOutput, partitionSpec, limitFunc))
51+
}
52+
}
53+
54+
class WindowGroupLimitPartitionEvaluator(f: Iterator[InternalRow] => Iterator[InternalRow])
55+
extends PartitionEvaluator[InternalRow, InternalRow] {
56+
57+
override def eval(
58+
partitionIndex: Int,
59+
inputs: Iterator[InternalRow]*): Iterator[InternalRow] = {
60+
f(inputs.head)
61+
}
62+
}
63+
}

sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala

Lines changed: 18 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.window
1919

2020
import org.apache.spark.rdd.RDD
2121
import org.apache.spark.sql.catalyst.InternalRow
22-
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, DenseRank, Expression, Rank, RowNumber, SortOrder, UnsafeProjection, UnsafeRow}
22+
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeProjection, UnsafeRow}
2323
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering
2424
import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning}
2525
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
@@ -73,26 +73,23 @@ case class WindowGroupLimitExec(
7373

7474
protected override def doExecute(): RDD[InternalRow] = {
7575
val numOutputRows = longMetric("numOutputRows")
76-
rankLikeFunction match {
77-
case _: RowNumber if partitionSpec.isEmpty =>
78-
child.execute().mapPartitionsInternal(SimpleLimitIterator(_, limit, numOutputRows))
79-
case _: RowNumber =>
80-
child.execute().mapPartitionsInternal(new GroupedLimitIterator(_, output, partitionSpec,
81-
(input: Iterator[InternalRow]) => SimpleLimitIterator(input, limit, numOutputRows)))
82-
case _: Rank if partitionSpec.isEmpty =>
83-
child.execute().mapPartitionsInternal(
84-
RankLimitIterator(output, _, orderSpec, limit, numOutputRows))
85-
case _: Rank =>
86-
child.execute().mapPartitionsInternal(new GroupedLimitIterator(_, output, partitionSpec,
87-
(input: Iterator[InternalRow]) =>
88-
RankLimitIterator(output, input, orderSpec, limit, numOutputRows)))
89-
case _: DenseRank if partitionSpec.isEmpty =>
90-
child.execute().mapPartitionsInternal(
91-
DenseRankLimitIterator(output, _, orderSpec, limit, numOutputRows))
92-
case _: DenseRank =>
93-
child.execute().mapPartitionsInternal(new GroupedLimitIterator(_, output, partitionSpec,
94-
(input: Iterator[InternalRow]) =>
95-
DenseRankLimitIterator(output, input, orderSpec, limit, numOutputRows)))
76+
77+
val evaluatorFactory =
78+
new WindowGroupLimitEvaluatorFactory(
79+
partitionSpec,
80+
orderSpec,
81+
rankLikeFunction,
82+
limit,
83+
child.output,
84+
numOutputRows)
85+
86+
if (conf.usePartitionEvaluator) {
87+
child.execute().mapPartitionsWithEvaluator(evaluatorFactory)
88+
} else {
89+
child.execute().mapPartitionsInternal { iter =>
90+
val evaluator = evaluatorFactory.createEvaluator()
91+
evaluator.eval(0, iter)
92+
}
9693
}
9794
}
9895

0 commit comments

Comments
 (0)