-
Notifications
You must be signed in to change notification settings - Fork 461
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
Merged
Merged
Changes from 7 commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
45c393c
cleanup
ArnavBalyan 752832e
update
ArnavBalyan 0c95def
mend
ArnavBalyan a3bbead
cleanup
ArnavBalyan fff46c1
mend
ArnavBalyan aca9135
update
ArnavBalyan 90f1960
update
ArnavBalyan e3a8ed1
update
ArnavBalyan File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
139 changes: 139 additions & 0 deletions
139
backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarRangeExec.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,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.ArrowBatches.ArrowJavaBatch | ||
import org.apache.gluten.extension.columnar.transition.Convention | ||
import org.apache.gluten.iterator.Iterators | ||
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.vectorized.{ColumnarBatch, ColumnVector} | ||
|
||
/** | ||
* ColumnarRangeExec 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 ColumnarRangeExec( | ||
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 def batchType(): Convention.BatchType = { | ||
ArrowJavaBatch | ||
} | ||
|
||
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 batchSize = 1000 | ||
val safePartitionStart = (partitionIndex) * numElements / numSlices * step + start | ||
val safePartitionEnd = (partitionIndex + 1) * numElements / numSlices * step + start | ||
|
||
def getSafeMargin(value: BigInt): Long = | ||
if (value.isValidLong) value.toLong | ||
else if (value > 0) Long.MaxValue | ||
else Long.MinValue | ||
|
||
val partitionStart = getSafeMargin(safePartitionStart) | ||
val partitionEnd = getSafeMargin(safePartitionEnd) | ||
|
||
/** | ||
* 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, getSafeMargin(value)) | ||
} | ||
vectors.foreach(_.setValueCount(numRows)) | ||
current += numRows * step | ||
|
||
val batch = new ColumnarBatch(vectors.asInstanceOf[Array[ColumnVector]], numRows) | ||
batch | ||
} | ||
} | ||
Iterators | ||
.wrap(iterator) | ||
.recyclePayload( | ||
batch => { | ||
batch.close() | ||
}) | ||
.create() | ||
|
||
} | ||
} | ||
} | ||
|
||
override protected def doExecute(): RDD[org.apache.spark.sql.catalyst.InternalRow] = { | ||
throw new UnsupportedOperationException("doExecute is not supported for this operator") | ||
} | ||
} |
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
69 changes: 69 additions & 0 deletions
69
gluten-substrait/src/main/scala/org/apache/gluten/execution/RangeExecBaseTransformer.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,69 @@ | ||
/* | ||
* 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.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 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 | ||
.genColumnarRangeExec( | ||
rangeExec.start, | ||
rangeExec.end, | ||
rangeExec.step, | ||
rangeExec.numSlices, | ||
rangeExec.numElements, | ||
rangeExec.output, | ||
rangeExec.children | ||
) | ||
} | ||
} |
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
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 rename it to
ColumnarRangeBaseExec
? Given it's not a Substrait operator.It feels a similar issue is in PR #8566 as well.
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.
Updated both