parthchandra commented on code in PR #4234: URL: https://github.com/apache/datafusion-comet/pull/4234#discussion_r3198079384
########## .github/workflows/pyarrow_udf_test.yml: ########## @@ -0,0 +1,115 @@ +# 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. + +name: PyArrow UDF Tests + +concurrency: + group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} + cancel-in-progress: true + +on: + push: + branches: + - main + paths-ignore: + - "benchmarks/**" + - "doc/**" + - "docs/**" + - "**.md" + - "dev/changelog/*.md" + - "native/core/benches/**" + - "native/spark-expr/benches/**" + - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" + - "spark/src/main/scala/org/apache/comet/GenerateDocs.scala" + pull_request: + paths-ignore: + - "benchmarks/**" + - "doc/**" + - "docs/**" + - "**.md" + - "dev/changelog/*.md" + - "native/core/benches/**" + - "native/spark-expr/benches/**" + - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" + - "spark/src/main/scala/org/apache/comet/GenerateDocs.scala" + workflow_dispatch: + +permissions: + contents: read + +env: + RUST_VERSION: stable + RUST_BACKTRACE: 1 + RUSTFLAGS: "-Clink-arg=-fuse-ld=bfd" + +jobs: + pyarrow-udf: + name: PyArrow UDF (Spark 3.5, JDK 17, Python 3.11) Review Comment: Should this be Spark 4 now? I'm assuming that this is enabled for only one version of Spark because it is experimental? ########## spark/src/main/scala/org/apache/spark/sql/comet/CometPythonMapInArrowExec.scala: ########## @@ -0,0 +1,153 @@ +/* + * 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.comet + +import scala.collection.JavaConverters._ + +import org.apache.spark.{ContextAwareIterator, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.PythonUDF +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.comet.shims.ShimCometPythonMapInArrow +import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.python.{BatchIterator, PythonSQLMetrics} +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + +/** + * An optimized version of Spark's MapInBatchExec (PythonMapInArrowExec / MapInPandasExec) that + * accepts columnar input directly from Comet operators, avoiding unnecessary Arrow -> Row -> + * Arrow conversions. + * + * Normal Spark flow: CometNativeExec (Arrow) -> ColumnarToRow -> PythonMapInArrowExec + * (internally: rows -> Arrow -> Python -> Arrow -> rows) + * + * Optimized flow: CometNativeExec (Arrow) -> CometPythonMapInArrowExec (batch.rowIterator() -> + * Arrow -> Python -> Arrow columnar output) + * + * This eliminates: Review Comment: +1 ! ########## spark/src/main/scala/org/apache/spark/sql/comet/CometPythonMapInArrowExec.scala: ########## @@ -0,0 +1,153 @@ +/* + * 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.comet + +import scala.collection.JavaConverters._ + +import org.apache.spark.{ContextAwareIterator, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.PythonUDF +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.comet.shims.ShimCometPythonMapInArrow +import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.python.{BatchIterator, PythonSQLMetrics} +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + +/** + * An optimized version of Spark's MapInBatchExec (PythonMapInArrowExec / MapInPandasExec) that + * accepts columnar input directly from Comet operators, avoiding unnecessary Arrow -> Row -> + * Arrow conversions. + * + * Normal Spark flow: CometNativeExec (Arrow) -> ColumnarToRow -> PythonMapInArrowExec + * (internally: rows -> Arrow -> Python -> Arrow -> rows) + * + * Optimized flow: CometNativeExec (Arrow) -> CometPythonMapInArrowExec (batch.rowIterator() -> + * Arrow -> Python -> Arrow columnar output) + * + * This eliminates: + * 1. The UnsafeProjection in ColumnarToRow (expensive copy) 2. The output Arrow->Row conversion + * (keeps Python output as ColumnarBatch) + */ +case class CometPythonMapInArrowExec( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + isBarrier: Boolean, + pythonEvalType: Int) + extends UnaryExecNode Review Comment: Other Comet operators extend `CometPlan`. The idea was that all Comet specific common behavior (say some Comet specific metrics, for example) can be in a single place. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
