-
Notifications
You must be signed in to change notification settings - Fork 28.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-44287][SQL] Use PartitionEvaluator API in RowToColumnarExec & …
…ColumnarToRowExec SQL operators ### What changes were proposed in this pull request? SQL operators `RowToColumnarExec` & `ColumnarToRowExec` are updated to use the `PartitionEvaluator` API to do execution. ### Why are the changes needed? To avoid the use of lambda during distributed execution. Ref: SPARK-43061 for more details. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Updated 2 test cases, once all the SQL operators are migrated, the flag `spark.sql.execution.useTaskEvaluator` will be enabled by default to avoid running the tests with and without this TaskEvaluator Closes #41839 from vinodkc/br_refactorToEvaluatorFactory1. Authored-by: Vinod KC <vinod.kc.in@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
- Loading branch information
Showing
4 changed files
with
190 additions
and
95 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
112 changes: 112 additions & 0 deletions
112
sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarEvaluatorFactory.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,112 @@ | ||
/* | ||
* 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.execution | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.spark.{PartitionEvaluator, PartitionEvaluatorFactory, TaskContext} | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} | ||
import org.apache.spark.sql.execution.metric.SQLMetric | ||
import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.vectorized.ColumnarBatch | ||
|
||
class ColumnarToRowEvaluatorFactory( | ||
childOutput: Seq[Attribute], | ||
numOutputRows: SQLMetric, | ||
numInputBatches: SQLMetric) | ||
extends PartitionEvaluatorFactory[ColumnarBatch, InternalRow] { | ||
|
||
override def createEvaluator(): PartitionEvaluator[ColumnarBatch, InternalRow] = { | ||
new ColumnarToRowEvaluator | ||
} | ||
|
||
private class ColumnarToRowEvaluator extends PartitionEvaluator[ColumnarBatch, InternalRow] { | ||
override def eval( | ||
partitionIndex: Int, | ||
inputs: Iterator[ColumnarBatch]*): Iterator[InternalRow] = { | ||
assert(inputs.length == 1) | ||
val toUnsafe = UnsafeProjection.create(childOutput, childOutput) | ||
inputs.head.flatMap { input => | ||
numInputBatches += 1 | ||
numOutputRows += input.numRows() | ||
input.rowIterator().asScala.map(toUnsafe) | ||
} | ||
} | ||
} | ||
} | ||
|
||
class RowToColumnarEvaluatorFactory( | ||
enableOffHeapColumnVector: Boolean, | ||
numRows: Int, | ||
schema: StructType, | ||
numInputRows: SQLMetric, | ||
numOutputBatches: SQLMetric) | ||
extends PartitionEvaluatorFactory[InternalRow, ColumnarBatch] { | ||
|
||
override def createEvaluator(): PartitionEvaluator[InternalRow, ColumnarBatch] = { | ||
new RowToColumnarEvaluator | ||
} | ||
|
||
private class RowToColumnarEvaluator extends PartitionEvaluator[InternalRow, ColumnarBatch] { | ||
override def eval( | ||
partitionIndex: Int, | ||
inputs: Iterator[InternalRow]*): Iterator[ColumnarBatch] = { | ||
assert(inputs.length == 1) | ||
val rowIterator = inputs.head | ||
|
||
if (rowIterator.hasNext) { | ||
new Iterator[ColumnarBatch] { | ||
private val converters = new RowToColumnConverter(schema) | ||
private val vectors: Seq[WritableColumnVector] = if (enableOffHeapColumnVector) { | ||
OffHeapColumnVector.allocateColumns(numRows, schema) | ||
} else { | ||
OnHeapColumnVector.allocateColumns(numRows, schema) | ||
} | ||
private val cb: ColumnarBatch = new ColumnarBatch(vectors.toArray) | ||
|
||
TaskContext.get().addTaskCompletionListener[Unit] { _ => | ||
cb.close() | ||
} | ||
|
||
override def hasNext: Boolean = { | ||
rowIterator.hasNext | ||
} | ||
|
||
override def next(): ColumnarBatch = { | ||
cb.setNumRows(0) | ||
vectors.foreach(_.reset()) | ||
var rowCount = 0 | ||
while (rowCount < numRows && rowIterator.hasNext) { | ||
val row = rowIterator.next() | ||
converters.convert(row, vectors.toArray) | ||
rowCount += 1 | ||
} | ||
cb.setNumRows(rowCount) | ||
numInputRows += rowCount | ||
numOutputBatches += 1 | ||
cb | ||
} | ||
} | ||
} else { | ||
Iterator.empty | ||
} | ||
} | ||
} | ||
} |
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