Skip to content
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

support user udf #3

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -95,3 +95,9 @@ dist/

/cpp-ch/local-engine/Parser/*_udf
!/cpp-ch/local-engine/Parser/example_udf


# build arrow
dev/arrow_ep/
ep/_ep/

Original file line number Diff line number Diff line change
Expand Up @@ -114,28 +114,28 @@ object VeloxBackendSettings extends BackendSettingsApi {
case ParquetReadFormat =>
val typeValidator: PartialFunction[StructField, String] = {
// Parquet scan of nested array with struct/array as element type is unsupported in Velox.
case StructField(_, arrayType: ArrayType, _, _)
if arrayType.elementType.isInstanceOf[StructType] =>
"StructType as element in ArrayType"
case StructField(_, arrayType: ArrayType, _, _)
if arrayType.elementType.isInstanceOf[ArrayType] =>
"ArrayType as element in ArrayType"
// case StructField(_, arrayType: ArrayType, _, _)
// if arrayType.elementType.isInstanceOf[StructType] =>
// "StructType as element in ArrayType"
// case StructField(_, arrayType: ArrayType, _, _)
// if arrayType.elementType.isInstanceOf[ArrayType] =>
// "ArrayType as element in ArrayType"
// Parquet scan of nested map with struct as key type,
// or array type as value type is not supported in Velox.
case StructField(_, mapType: MapType, _, _) if mapType.keyType.isInstanceOf[StructType] =>
"StructType as Key in MapType"
case StructField(_, mapType: MapType, _, _)
if mapType.valueType.isInstanceOf[ArrayType] =>
"ArrayType as Value in MapType"
// case StructField(_, mapType: MapType, _, _) if mapType.keyType.isInstanceOf[StructType] =>
// "StructType as Key in MapType"
// case StructField(_, mapType: MapType, _, _)
// if mapType.valueType.isInstanceOf[ArrayType] =>
// "ArrayType as Value in MapType"
case StructField(_, TimestampType, _, _)
if GlutenConfig.getConf.forceParquetTimestampTypeScanFallbackEnabled =>
"TimestampType"
}
if (!GlutenConfig.getConf.forceComplexTypeScanFallbackEnabled) {
validateTypes(typeValidator)
} else {
validateTypes(parquetTypeValidatorWithComplexTypeFallback)
}
// if (!GlutenConfig.getConf.forceComplexTypeScanFallbackEnabled) {
validateTypes(typeValidator)
// } else {
// validateTypes(parquetTypeValidatorWithComplexTypeFallback)
// }
case DwrfReadFormat => ValidationResult.succeeded
case OrcReadFormat =>
if (!GlutenConfig.getConf.veloxOrcScanEnabled) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,14 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
FilterExecTransformer(condition, child)
}

override def genSparkPartialProjectColumnarExec(original: ProjectExec): GlutenPlan = {
SparkPartialProjectColumnarExec.create(original)
}

override def genProjectColumnarExec(original: ProjectExec): GlutenPlan = {
ProjectColumnarExec(original.projectList, original.child)
}

/** Generate HashAggregateExecTransformer. */
override def genHashAggregateExecTransformer(
requiredChildDistributionExpressions: Option[Seq[Expression]],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ case class ArrowConvertorRule(session: SparkSession) extends Rule[LogicalPlan] {
options,
columnPruning = session.sessionState.conf.csvColumnPruning,
session.sessionState.conf.sessionLocalTimeZone)
checkSchema(dataSchema) &&
SparkSchemaUtil.checkSchema(dataSchema) &&
checkCsvOptions(csvOptions, session.sessionState.conf.sessionLocalTimeZone) &&
dataSchema.nonEmpty
}
Expand All @@ -106,13 +106,4 @@ case class ArrowConvertorRule(session: SparkSession) extends Rule[LogicalPlan] {
SparkShimLoader.getSparkShims.dateTimestampFormatInReadIsDefaultValue(csvOptions, timeZone)
}

private def checkSchema(schema: StructType): Boolean = {
try {
SparkSchemaUtil.toArrowSchema(schema)
true
} catch {
case _: Exception =>
false
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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.GlutenConfig
import org.apache.gluten.columnarbatch.ColumnarBatches
import org.apache.gluten.extension.{GlutenPlan, ValidationResult}
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.gluten.utils.iterator.Iterators
import org.apache.gluten.vectorized.ArrowWritableColumnVector

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, MutableProjection, NamedExpression, SortOrder}
import org.apache.spark.sql.execution.{OrderPreservingNodeShim, PartitioningPreservingNodeShim, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.vectorized.{MutableColumnarRow, WritableColumnVector}
import org.apache.spark.sql.utils.SparkSchemaUtil
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}

case class ProjectColumnarExec(projectList: Seq[NamedExpression], child: SparkPlan)
extends UnaryExecNode
with PartitioningPreservingNodeShim
with OrderPreservingNodeShim
with GlutenPlan {

override protected def orderingExpressions: Seq[SortOrder] = child.outputOrdering

override protected def outputExpressions: Seq[NamedExpression] = projectList

override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()

override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
copy(child = newChild)

override def output: Seq[Attribute] = projectList.map(_.toAttribute)

override def supportsColumnar: Boolean = true

override def doExecuteColumnar(): RDD[ColumnarBatch] = {
child.executeColumnar().mapPartitions {
batches =>
val res: Iterator[Iterator[ColumnarBatch]] = new Iterator[Iterator[ColumnarBatch]] {
override def hasNext: Boolean = batches.hasNext

override def next(): Iterator[ColumnarBatch] = {
val batch = batches.next()
if (batch.numRows == 0) {
Iterator.empty
} else {
val proj = MutableProjection.create(projectList, child.output)
val numRows = batch.numRows()
val arrowBatch =
ColumnarBatches.ensureLoaded(ArrowBufferAllocators.contextInstance(), batch)

val schema =
SparkShimLoader.getSparkShims.structFromAttributes(child.output.map(_.toAttribute))
val vectors: Array[WritableColumnVector] = ArrowWritableColumnVector
.allocateColumns(numRows, schema)
.map {
vector =>
vector.setValueCount(numRows)
vector.asInstanceOf[WritableColumnVector]
}
val targetRow = new MutableColumnarRow(vectors)
for (i <- 0 until numRows) {
targetRow.rowId = i
proj.target(targetRow).apply(arrowBatch.getRow(i))
}
val targetBatch =
new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), numRows)
val veloxBatch = ColumnarBatches
.ensureOffloaded(ArrowBufferAllocators.contextInstance(), targetBatch)
Iterators
.wrap(Iterator.single(veloxBatch))
.recycleIterator({
arrowBatch.close()
targetBatch.close()
})
.create()

}
}
}
Iterators
.wrap(res.flatten)
.protectInvocationFlow() // Spark may call `hasNext()` again after a false output which
// is not allowed by Gluten iterators. E.g. GroupedIterator#fetchNextGroupIterator
.recyclePayload(_.close())
.create()
}
}

override protected def doValidateInternal(): ValidationResult = {
if (!GlutenConfig.getConf.enableProjectColumnarExec) {
return ValidationResult.failed("Config disable this feature")
}
if (!(SparkSchemaUtil.checkSchema(schema) && SparkSchemaUtil.checkSchema(child.schema))) {
return ValidationResult.failed("Input type or output type cannot convert to arrow")
}
ValidationResult.succeeded
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,9 @@ case class RowToVeloxColumnarExec(child: SparkPlan) extends RowToColumnarExecBas
val numInputRows = longMetric("numInputRows")
val numOutputBatches = longMetric("numOutputBatches")
val convertTime = longMetric("convertTime")

// Instead of creating a new config we are reusing columnBatchSize. In the future if we do
// combine with some of the Arrow conversion tools we will need to unify some of the configs.
val numRows = GlutenConfig.getConf.maxBatchSize
// This avoids calling `schema` in the RDD closure, so that we don't need to include the entire
// plan (this) in the closure.
Expand Down Expand Up @@ -92,6 +95,23 @@ case class RowToVeloxColumnarExec(child: SparkPlan) extends RowToColumnarExecBas
}

object RowToVeloxColumnarExec {

def toColumnarBatchIterator(
in: Iterator[InternalRow],
schema: StructType,
columnBatchSize: Int): Iterator[ColumnarBatch] = {
val numInputRows = new SQLMetric("numInputRows")
val numOutputBatches = new SQLMetric("numOutputBatches")
val convertTime = new SQLMetric("convertTime")
RowToVeloxColumnarExec.toColumnarBatchIterator(
in,
schema,
numInputRows,
numOutputBatches,
convertTime,
columnBatchSize)
}

def toColumnarBatchIterator(
it: Iterator[InternalRow],
schema: StructType,
Expand Down
Loading
Loading