This repository has been archived by the owner on Sep 18, 2023. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 75
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[NSE-947] Add a whole stage fallback strategy (#948)
* Initial commit * Move to add the strategy in ColumnarOverrideRules * Revert "Initial commit" This reverts commit c4210ccfe39fac9b144117ec403bd9b00d8b5703. * Consider stage boundary * Differentiate the handling for spark raw BatchScanExec & arrow datasource BatchScanExec * Handle InMemoryTableScanExec * Use spark's transition code * Support fallback for LocalWindowExec * Enable this feature when AQE is on * Add a config * Check whether AQE is supported * Set default value to -1
- Loading branch information
Showing
8 changed files
with
285 additions
and
60 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
154 changes: 154 additions & 0 deletions
154
native-sql-engine/core/src/main/scala/org/apache/spark/sql/LocalWindowExec.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,154 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, JoinedRow, NamedExpression, SortOrder, SpecificInternalRow, UnsafeProjection, UnsafeRow} | ||
import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning} | ||
import org.apache.spark.sql.execution.{ExternalAppendOnlyUnsafeRowArray, SparkPlan} | ||
import org.apache.spark.sql.execution.window.WindowExecBase | ||
|
||
case class LocalWindowExec( | ||
windowExpression: Seq[NamedExpression], | ||
partitionSpec: Seq[Expression], | ||
orderSpec: Seq[SortOrder], | ||
child: SparkPlan) | ||
extends WindowExecBase { | ||
|
||
override def output: Seq[Attribute] = | ||
child.output ++ windowExpression.map(_.toAttribute) | ||
|
||
override def requiredChildDistribution: Seq[Distribution] = { | ||
super.requiredChildDistribution | ||
} | ||
|
||
override def requiredChildOrdering: Seq[Seq[SortOrder]] = | ||
Seq(partitionSpec.map(SortOrder(_, Ascending)) ++ orderSpec) | ||
|
||
override def outputOrdering: Seq[SortOrder] = child.outputOrdering | ||
|
||
override def outputPartitioning: Partitioning = child.outputPartitioning | ||
|
||
// This function is copied from Spark's WindowExec. | ||
protected override def doExecute(): RDD[InternalRow] = { | ||
// Unwrap the window expressions and window frame factories from the map. | ||
val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1) | ||
val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray | ||
val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold | ||
val spillThreshold = conf.windowExecBufferSpillThreshold | ||
|
||
// Start processing. | ||
child.execute().mapPartitions { stream => | ||
new Iterator[InternalRow] { | ||
|
||
// Get all relevant projections. | ||
val result = createResultProjection(expressions) | ||
val grouping = UnsafeProjection.create(partitionSpec, child.output) | ||
|
||
// Manage the stream and the grouping. | ||
var nextRow: UnsafeRow = null | ||
var nextGroup: UnsafeRow = null | ||
var nextRowAvailable: Boolean = false | ||
private[this] def fetchNextRow(): Unit = { | ||
nextRowAvailable = stream.hasNext | ||
if (nextRowAvailable) { | ||
nextRow = stream.next().asInstanceOf[UnsafeRow] | ||
nextGroup = grouping(nextRow) | ||
} else { | ||
nextRow = null | ||
nextGroup = null | ||
} | ||
} | ||
fetchNextRow() | ||
|
||
// Manage the current partition. | ||
val buffer: ExternalAppendOnlyUnsafeRowArray = | ||
new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold) | ||
|
||
var bufferIterator: Iterator[UnsafeRow] = _ | ||
|
||
val windowFunctionResult = new SpecificInternalRow(expressions.map(_.dataType)) | ||
val frames = factories.map(_(windowFunctionResult)) | ||
val numFrames = frames.length | ||
private[this] def fetchNextPartition(): Unit = { | ||
// Collect all the rows in the current partition. | ||
// Before we start to fetch new input rows, make a copy of nextGroup. | ||
val currentGroup = nextGroup.copy() | ||
|
||
// clear last partition | ||
buffer.clear() | ||
|
||
while (nextRowAvailable && nextGroup == currentGroup) { | ||
buffer.add(nextRow) | ||
fetchNextRow() | ||
} | ||
|
||
// Setup the frames. | ||
var i = 0 | ||
while (i < numFrames) { | ||
frames(i).prepare(buffer) | ||
i += 1 | ||
} | ||
|
||
// Setup iteration | ||
rowIndex = 0 | ||
bufferIterator = buffer.generateIterator() | ||
} | ||
|
||
// Iteration | ||
var rowIndex = 0 | ||
|
||
override final def hasNext: Boolean = | ||
(bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable | ||
|
||
val join = new JoinedRow | ||
override final def next(): InternalRow = { | ||
// Load the next partition if we need to. | ||
if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) { | ||
fetchNextPartition() | ||
} | ||
|
||
if (bufferIterator.hasNext) { | ||
val current = bufferIterator.next() | ||
|
||
// Get the results for the window frames. | ||
var i = 0 | ||
while (i < numFrames) { | ||
frames(i).write(rowIndex, current) | ||
i += 1 | ||
} | ||
|
||
// 'Merge' the input row with the window function result | ||
join(current, windowFunctionResult) | ||
rowIndex += 1 | ||
|
||
// Return the projection. | ||
result(join) | ||
} else { | ||
throw new NoSuchElementException | ||
} | ||
} | ||
} | ||
} | ||
} | ||
|
||
protected def withNewChildInternal(newChild: SparkPlan): | ||
LocalWindowExec = | ||
copy(child = newChild) | ||
} |
Oops, something went wrong.