-
Notifications
You must be signed in to change notification settings - Fork 444
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[GLUTEN-3620][VL] Support Range operator for Velox Backend #8161
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,139 @@ | ||
/* | ||
* 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.gluten.execution | ||
|
||
import org.apache.gluten.columnarbatch.ColumnarBatches | ||
import org.apache.gluten.iterator.Iterators | ||
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators | ||
import org.apache.gluten.vectorized.ArrowWritableColumnVector | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.expressions.Attribute | ||
import org.apache.spark.sql.execution.SparkPlan | ||
import org.apache.spark.sql.internal.SQLConf | ||
import org.apache.spark.sql.utils.SparkArrowUtil | ||
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} | ||
|
||
/** | ||
* RangeExecTransformer is a concrete implementation of RangeExecBaseTransformer that executes the | ||
* Range operation and supports columnar processing. It generates columnar batches for the specified | ||
* range. | ||
* | ||
* @param start | ||
* Starting value of the range. | ||
* @param end | ||
* Ending value of the range. | ||
* @param step | ||
* Step size for the range. | ||
* @param numSlices | ||
* Number of slices for partitioning the range. | ||
* @param numElements | ||
* Total number of elements in the range. | ||
* @param outputAttributes | ||
* Attributes defining the output schema of the operator. | ||
* @param child | ||
* Child SparkPlan nodes for this operator, if any. | ||
*/ | ||
case class RangeExecTransformer( | ||
start: Long, | ||
end: Long, | ||
step: Long, | ||
numSlices: Int, | ||
numElements: BigInt, | ||
outputAttributes: Seq[Attribute], | ||
child: Seq[SparkPlan] | ||
) extends RangeExecBaseTransformer( | ||
start, | ||
end, | ||
step, | ||
numSlices, | ||
numElements, | ||
outputAttributes, | ||
child) { | ||
|
||
override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { | ||
if (start == end || (start < end ^ 0 < step)) { | ||
sparkContext.emptyRDD[ColumnarBatch] | ||
} else { | ||
sparkContext | ||
.parallelize(0 until numSlices, numSlices) | ||
.mapPartitionsWithIndex { | ||
(partitionIndex, _) => | ||
val allocator = ArrowBufferAllocators.contextInstance() | ||
val sessionLocalTimeZone = SQLConf.get.sessionLocalTimeZone | ||
val arrowSchema = SparkArrowUtil.toArrowSchema(schema, SQLConf.get.sessionLocalTimeZone) | ||
|
||
val batchSize = 1000 | ||
val safePartitionStart = | ||
start + step * (partitionIndex * numElements.toLong / numSlices) | ||
val safePartitionEnd = | ||
start + step * ((partitionIndex + 1) * numElements.toLong / numSlices) | ||
|
||
/** | ||
* Generates the columnar batches for the specified range. Each batch contains a subset | ||
* of the range values, managed using Arrow column vectors. | ||
*/ | ||
val iterator = new Iterator[ColumnarBatch] { | ||
var current = safePartitionStart | ||
|
||
override def hasNext: Boolean = { | ||
if (step > 0) { | ||
current < safePartitionEnd | ||
} else { | ||
current > safePartitionEnd | ||
} | ||
} | ||
|
||
override def next(): ColumnarBatch = { | ||
val numRows = math.min( | ||
((safePartitionEnd - current) / step).toInt.max(1), | ||
batchSize | ||
) | ||
|
||
val vectors = ArrowWritableColumnVector.allocateColumns(numRows, schema) | ||
|
||
for (i <- 0 until numRows) { | ||
val value = current + i * step | ||
vectors(0).putLong(i, value) | ||
} | ||
vectors.foreach(_.setValueCount(numRows)) | ||
current += numRows * step | ||
|
||
val batch = new ColumnarBatch(vectors.asInstanceOf[Array[ColumnVector]], numRows) | ||
val offloadedBatch = ColumnarBatches.offload(allocator, batch) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Because of the code, the operator's behaviour acutally align more precisely with
(as Would you see if we can add the code to this class? If yes we can also remove Doing this will make query planner to insert an explicit Futhermore, I will suggest remove line 117 and line 118 then return the
So two explicit c2cs (ArrowJava-to-ArrowNative, ArrowNative-to-Velox) can be inserted to query plan. More details please refer to test code. |
||
offloadedBatch | ||
} | ||
} | ||
Iterators | ||
.wrap(iterator) | ||
.recyclePayload( | ||
batch => { | ||
batch.close() | ||
}) | ||
.recycleIterator { | ||
allocator.close() | ||
} | ||
.create() | ||
|
||
} | ||
} | ||
} | ||
|
||
override protected def doExecute(): RDD[org.apache.spark.sql.catalyst.InternalRow] = { | ||
throw new UnsupportedOperationException("doExecute is not supported for this operator") | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* 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.gluten.execution | ||
|
||
import org.apache.gluten.backendsapi.BackendsApiManager | ||
import org.apache.gluten.extension.ValidationResult | ||
import org.apache.gluten.extension.columnar.transition.Convention | ||
|
||
import org.apache.spark.sql.catalyst.expressions.Attribute | ||
import org.apache.spark.sql.execution.{LeafExecNode, RangeExec, SparkPlan} | ||
|
||
/** | ||
* Base class for RangeExec transformation, can be implemented by the by supported backends. | ||
* Currently velox is supported. | ||
*/ | ||
abstract class RangeExecBaseTransformer( | ||
start: Long, | ||
end: Long, | ||
step: Long, | ||
numSlices: Int, | ||
numElements: BigInt, | ||
outputAttributes: Seq[Attribute], | ||
child: Seq[SparkPlan]) | ||
extends LeafExecNode | ||
with ValidatablePlan { | ||
|
||
override def output: Seq[Attribute] = { | ||
outputAttributes | ||
} | ||
|
||
override protected def doValidateInternal(): ValidationResult = { | ||
val isSupported = BackendsApiManager.getSettings.supportRangeExec() | ||
|
||
if (!isSupported) { | ||
return ValidationResult.failed( | ||
s"RangeExec is not supported by the current backend." | ||
) | ||
} | ||
ValidationResult.succeeded | ||
} | ||
Comment on lines
+45
to
+54
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think usually we have to add some code to this validator to make There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am opening #8177 that may simplify the usage of validators. Perhaps could do a rebase once that PR merged. |
||
|
||
override def batchType(): Convention.BatchType = { | ||
BackendsApiManager.getSettings.primaryBatchType | ||
} | ||
|
||
override def rowType0(): Convention.RowType = Convention.RowType.None | ||
|
||
override protected def doExecute() | ||
: org.apache.spark.rdd.RDD[org.apache.spark.sql.catalyst.InternalRow] = { | ||
throw new UnsupportedOperationException(s"This operator doesn't support doExecute().") | ||
} | ||
} | ||
|
||
/** | ||
* Companion object for RangeExecBaseTransformer, provides factory methods to create instance from | ||
* existing RangeExec plan. | ||
*/ | ||
object RangeExecBaseTransformer { | ||
def from(rangeExec: RangeExec): RangeExecBaseTransformer = { | ||
BackendsApiManager.getSparkPlanExecApiInstance | ||
.genRangeExecTransformer( | ||
rangeExec.start, | ||
rangeExec.end, | ||
rangeExec.step, | ||
rangeExec.numSlices, | ||
rangeExec.numElements, | ||
rangeExec.output, | ||
rangeExec.children | ||
) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,94 @@ | ||
/* | ||
* 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 org.apache.gluten.execution.RangeExecTransformer | ||
|
||
import org.apache.spark.sql.GlutenSQLTestsTrait | ||
import org.apache.spark.sql.Row | ||
|
||
class GlutenSQLRangeExecSuite extends GlutenSQLTestsTrait { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why we need test suites for all Spark versions? |
||
|
||
testGluten("RangeExecTransformer produces correct results") { | ||
val df = spark.range(0, 10, 1) | ||
val expectedData = (0L until 10L).map(Row(_)).toSeq | ||
checkAnswer(df, expectedData) | ||
|
||
assert( | ||
getExecutedPlan(df).exists { | ||
case _: RangeExecTransformer => true | ||
case _ => false | ||
} | ||
) | ||
} | ||
|
||
testGluten("RangeExecTransformer with step") { | ||
val df = spark.range(5, 15, 2) | ||
val expectedData = Seq(523L, 7L, 9L, 11L, 13L).map(Row(_)) | ||
checkAnswer(df, expectedData) | ||
|
||
assert( | ||
getExecutedPlan(df).exists { | ||
case _: RangeExecTransformer => true | ||
case _ => false | ||
} | ||
) | ||
} | ||
|
||
testGluten("RangeExecTransformer with filter") { | ||
val df = spark.range(0, 20, 1).filter("id % 3 == 0") | ||
val expectedData = Seq(0L, 3L, 6L, 9L, 12L, 15L, 18L).map(Row(_)) | ||
checkAnswer(df, expectedData) | ||
|
||
assert( | ||
getExecutedPlan(df).exists { | ||
case _: RangeExecTransformer => true | ||
case _ => false | ||
} | ||
) | ||
} | ||
|
||
testGluten("RangeExecTransformer with aggregation") { | ||
val df = spark.range(1, 6, 1) | ||
val sumDf = df.agg(sum("id")) | ||
val expectedData = Seq(Row(15L)) | ||
checkAnswer(sumDf, expectedData) | ||
|
||
assert( | ||
getExecutedPlan(sumDf).exists { | ||
case _: RangeExecTransformer => true | ||
case _ => false | ||
} | ||
) | ||
} | ||
|
||
testGluten("RangeExecTransformer with join") { | ||
val df1 = spark.range(0, 5, 1).toDF("id1") | ||
val df2 = spark.range(3, 8, 1).toDF("id2") | ||
val joinDf = df1.join(df2, df1("id1") === df2("id2")) | ||
val expectedData = Seq(Row(3L, 3L), Row(4L, 4L)) | ||
checkAnswer(joinDf, expectedData) | ||
|
||
assert( | ||
getExecutedPlan(joinDf).exists { | ||
case _: RangeExecTransformer => true | ||
case _ => false | ||
} | ||
) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we name it "ColumnarRangeExec" (or
VeloxColumnarRangeExec
or something) as it doesn't extend theTransformSupport
trait? Thanks.