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

[GLUTEN-8668][VL] Support complex type in ColumnarPartialProject #8669

Open
wants to merge 7 commits into
base: main
Choose a base branch
from
Open
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
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.gluten.vectorized.{ArrowColumnarRow, ArrowWritableColumnVector

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CaseWhen, Coalesce, Expression, If, LambdaFunction, NamedExpression, NaNvl, ScalaUDF}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.execution.{ExplainUtils, ProjectExec, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
Expand Down Expand Up @@ -118,6 +118,9 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)(
case _: DecimalType => true
case YearMonthIntervalType.DEFAULT => true
case _: NullType => true
case _: ArrayType => true
case _: MapType => true
case _: StructType => true
case _ => false
}
}
Expand Down Expand Up @@ -256,6 +259,7 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)(
targetRow.rowId = i
proj.target(targetRow).apply(arrowBatch.getRow(i))
}
targetRow.finishWriteRow()
val targetBatch = new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), numRows)
val start2 = System.currentTimeMillis()
val veloxBatch = VeloxColumnarBatches.toVeloxBatch(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.gluten.execution;
package org.apache.gluten.test.udf;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove "test" if there is no reason to keep it.


import org.apache.hadoop.hive.ql.exec.Description;
import org.apache.hadoop.hive.ql.exec.UDF;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@ import org.apache.spark.sql.functions.udf

import java.io.File

case class MyStruct(a: Long, b: Array[Long])

class UDFPartialProjectSuiteRasOff extends UDFPartialProjectSuite {
override protected def sparkConf: SparkConf = {
super.sparkConf
Expand Down Expand Up @@ -147,4 +149,59 @@ abstract class UDFPartialProjectSuite extends WholeStageTransformerSuite {
}
}

test("udf with array") {
spark.udf.register("array_plus_one", udf((arr: Array[Int]) => arr.map(_ + 1)))
runQueryAndCompare("""
|SELECT
| l_partkey,
| sort_array(array_plus_one(array_data)) as orderkey_arr_plus_one
|FROM (
| SELECT l_partkey, collect_list(l_orderkey) as array_data
| FROM lineitem
| GROUP BY l_partkey
|)
|""".stripMargin) {
checkGlutenOperatorMatch[ColumnarPartialProjectExec]
}
}

test("udf with map") {
spark.udf.register(
"map_value_plus_one",
udf((m: Map[String, Long]) => m.map { case (key, value) => key -> (value + 1) }))
runQueryAndCompare("""
|SELECT
| l_partkey,
| map_value_plus_one(map_data)
|FROM (
| SELECT l_partkey,
| map(
| concat('hello', l_orderkey % 2), l_orderkey,
| concat('world', l_orderkey % 2), l_orderkey
| ) as map_data
| FROM lineitem
|)
|""".stripMargin) {
checkGlutenOperatorMatch[ColumnarPartialProjectExec]
}
}

test("udf with struct and array") {
spark.udf.register("struct_plus_one", udf((m: MyStruct) => MyStruct(m.a + 1, m.b.map(_ + 1))))
runQueryAndCompare("""
|SELECT
| l_partkey,
| struct_plus_one(struct_data)
|FROM (
| SELECT l_partkey,
| struct(
| l_orderkey % 2 as a,
| array(l_orderkey % 2, l_orderkey % 2 + 1, l_orderkey % 2 + 2) as b
| ) as struct_data
| FROM lineitem
|)
|""".stripMargin) {
checkGlutenOperatorMatch[ColumnarPartialProjectExec]
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,203 @@
/*
* 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.ColumnarPartialProjectExec
import org.apache.gluten.test.udf.CustomerUDF

import org.apache.spark.SparkConf
import org.apache.spark.internal.config
import org.apache.spark.internal.config.UI.UI_ENABLED
import org.apache.spark.sql.{DataFrame, GlutenQueryTest, Row, SparkSession}
import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode
import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation
import org.apache.spark.sql.hive.HiveUtils
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.test.SQLTestUtils

import java.io.File

import scala.collection.mutable
import scala.reflect.ClassTag

class GlutenHiveUDFSuite extends GlutenQueryTest with SQLTestUtils {
private var _spark: SparkSession = _

override protected def beforeAll(): Unit = {
super.beforeAll()

if (_spark == null) {
_spark = SparkSession.builder().config(sparkConf).enableHiveSupport().getOrCreate()
}

_spark.sparkContext.setLogLevel("info")

createTestTable()
}

override def afterAll(): Unit = {
super.afterAll()
}

override protected def spark: SparkSession = _spark

protected def defaultSparkConf: SparkConf = {
val conf = new SparkConf()
.set("spark.master", "local[1]")
.set("spark.sql.test", "")
.set("spark.sql.testkey", "true")
.set(SQLConf.CODEGEN_FALLBACK.key, "false")
.set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString)
.set(
HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key,
"org.apache.spark.sql.hive.execution.PairSerDe")
// SPARK-8910
.set(UI_ENABLED, false)
.set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true)
// Hive changed the default of hive.metastore.disallow.incompatible.col.type.changes
// from false to true. For details, see the JIRA HIVE-12320 and HIVE-17764.
.set("spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes", "false")
// Disable ConvertToLocalRelation for better test coverage. Test cases built on
// LocalRelation will exercise the optimization rules better by disabling it as
// this rule may potentially block testing of other optimization rules such as
// ConstantPropagation etc.
.set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName)

conf.set(
StaticSQLConf.WAREHOUSE_PATH,
conf.get(StaticSQLConf.WAREHOUSE_PATH) + "/" + getClass.getCanonicalName)
}

protected def sparkConf: SparkConf = {
defaultSparkConf
.set("spark.plugins", "org.apache.gluten.GlutenPlugin")
.set("spark.default.parallelism", "1")
.set("spark.memory.offHeap.enabled", "true")
.set("spark.memory.offHeap.size", "1024MB")
.set("spark.gluten.sql.native.writer.enabled", "true")
}

private def withTempFunction(funcName: String)(f: => Unit): Unit = {
try f
finally sql(s"DROP TEMPORARY FUNCTION IF EXISTS $funcName")
}

private def checkOperatorMatch[T <: SparkPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = {
val executedPlan = getExecutedPlan(df)
assert(executedPlan.exists(plan => plan.getClass == tag.runtimeClass))
}

private def createTestTable(): Unit = {
val table = "lineitem"
val tableDir = getClass.getResource("/tpch-data-parquet").getFile
val tablePath = new File(tableDir, table).getAbsolutePath
val tableDF = spark.read.format("parquet").load(tablePath)
tableDF.createOrReplaceTempView(table)
}

test("customer udf") {
withTempFunction("testUDF") {
sql(s"CREATE TEMPORARY FUNCTION testUDF AS '${classOf[CustomerUDF].getName}'")
val df = sql("select l_partkey, testUDF(l_comment) from lineitem")
df.show()
checkOperatorMatch[ColumnarPartialProjectExec](df)
}
}

test("customer udf wrapped in function") {
withTempFunction("testUDF") {
sql(s"CREATE TEMPORARY FUNCTION testUDF AS '${classOf[CustomerUDF].getName}'")
val df = sql("select l_partkey, hash(testUDF(l_comment)) from lineitem")
df.show()
checkOperatorMatch[ColumnarPartialProjectExec](df)
}
}

test("example") {
withTempFunction("testUDF") {
sql("CREATE TEMPORARY FUNCTION testUDF AS 'org.apache.hadoop.hive.ql.udf.UDFSubstr';")
val df = sql("select testUDF('l_commen', 1, 5)")
df.show()
// It should not be converted to ColumnarPartialProjectExec, since
// the UDF need all the columns in child output.
assert(!getExecutedPlan(df).exists {
case _: ColumnarPartialProjectExec => true
case _ => false
})
}
}

test("udf with array") {
withTempFunction("udf_sort_array") {
sql("""
|CREATE TEMPORARY FUNCTION udf_sort_array AS
|'org.apache.hadoop.hive.ql.udf.generic.GenericUDFSortArray';
|""".stripMargin)

val df = sql("""
|SELECT
| l_orderkey,
| l_partkey,
| udf_sort_array(array(10, l_orderkey, 1)) as udf_result
|FROM lineitem WHERE l_partkey <= 5 and l_orderkey <1000
|""".stripMargin)

checkAnswer(
df,
Seq(
Row(35, 5, mutable.WrappedArray.make(Array(1, 10, 35))),
Row(321, 4, mutable.WrappedArray.make(Array(1, 10, 321))),
Row(548, 2, mutable.WrappedArray.make(Array(1, 10, 548))),
Row(640, 5, mutable.WrappedArray.make(Array(1, 10, 640))),
Row(807, 2, mutable.WrappedArray.make(Array(1, 10, 807)))
)
)
checkOperatorMatch[ColumnarPartialProjectExec](df)
}
}

test("udf with map") {
withTempFunction("udf_str_to_map") {
sql("""
|CREATE TEMPORARY FUNCTION udf_str_to_map AS
|'org.apache.hadoop.hive.ql.udf.generic.GenericUDFStringToMap';
|""".stripMargin)

val df = sql(
"""
|SELECT
| l_orderkey,
| l_partkey,
| udf_str_to_map(
| concat_ws(',', array(concat('hello', l_partkey), 'world')), ',', 'l') as udf_result
|FROM lineitem WHERE l_partkey <= 5 and l_orderkey <1000
|""".stripMargin)

checkAnswer(
df,
Seq(
Row(321, 4, Map("he" -> "lo4", "wor" -> "d")),
Row(35, 5, Map("he" -> "lo5", "wor" -> "d")),
Row(548, 2, Map("he" -> "lo2", "wor" -> "d")),
Row(640, 5, Map("he" -> "lo5", "wor" -> "d")),
Row(807, 2, Map("he" -> "lo2", "wor" -> "d"))
)
)
checkOperatorMatch[ColumnarPartialProjectExec](df)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,14 @@
*/
package org.apache.gluten.vectorized;

import org.apache.gluten.exception.GlutenException;

import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.types.ArrayType;
import org.apache.spark.sql.types.BinaryType;
import org.apache.spark.sql.types.BooleanType;
import org.apache.spark.sql.types.ByteType;
import org.apache.spark.sql.types.CalendarIntervalType;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.DateType;
import org.apache.spark.sql.types.Decimal;
Expand All @@ -42,8 +43,6 @@
import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;

import java.math.BigDecimal;

// Copy from Spark MutableColumnarRow mostly but class member columns`type is
// ArrowWritableColumnVector. And support string and binary type to write, Arrow writer does not
// need to setNotNull before write a value.
Expand Down Expand Up @@ -227,36 +226,7 @@ public Object get(int ordinal, DataType dataType) {

@Override
public void update(int ordinal, Object value) {
if (value == null) {
setNullAt(ordinal);
} else {
DataType dt = columns[ordinal].dataType();
if (dt instanceof BooleanType) {
setBoolean(ordinal, (boolean) value);
} else if (dt instanceof IntegerType) {
setInt(ordinal, (int) value);
} else if (dt instanceof ShortType) {
setShort(ordinal, (short) value);
} else if (dt instanceof LongType) {
setLong(ordinal, (long) value);
} else if (dt instanceof FloatType) {
setFloat(ordinal, (float) value);
} else if (dt instanceof DoubleType) {
setDouble(ordinal, (double) value);
} else if (dt instanceof DecimalType) {
DecimalType t = (DecimalType) dt;
Decimal d = Decimal.apply((BigDecimal) value, t.precision(), t.scale());
setDecimal(ordinal, d, t.precision());
} else if (dt instanceof CalendarIntervalType) {
setInterval(ordinal, (CalendarInterval) value);
} else if (dt instanceof StringType) {
setUTF8String(ordinal, (UTF8String) value);
} else if (dt instanceof BinaryType) {
setBinary(ordinal, (byte[]) value);
} else {
throw new UnsupportedOperationException("Datatype not supported " + dt);
}
}
throw new UnsupportedOperationException();
}

@Override
Expand Down Expand Up @@ -316,4 +286,20 @@ public void setUTF8String(int ordinal, UTF8String value) {
public void setBinary(int ordinal, byte[] value) {
columns[ordinal].putBytes(rowId, value.length, value, 0);
}

public void writeRow(GenericInternalRow input) {
if (input.numFields() != columns.length) {
throw new GlutenException(
"The numFields of input row should be equal to the number of column vector!");
}
for (int i = 0; i < input.numFields(); ++i) {
columns[i].write(input, i);
}
}

public void finishWriteRow() {
for (int i = 0; i < columns.length; ++i) {
columns[i].finishWrite();
}
}
}
Loading
Loading