-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[WIP][SPARK-56661] Introducing logical and physical planning nodes for language-agnostic Spark UDFs #55611
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
Draft
sven-weber-db
wants to merge
2
commits into
apache:master
Choose a base branch
from
sven-weber-db:sven-weber_data/SPARK-56661
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
[WIP][SPARK-56661] Introducing logical and physical planning nodes for language-agnostic Spark UDFs #55611
Changes from all commits
Commits
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 hidden or 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 hidden or 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
41 changes: 41 additions & 0 deletions
41
core/src/main/scala/org/apache/spark/util/SparkUDFWorkerLogger.scala
This file contains hidden or 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,41 @@ | ||
| /* | ||
| * 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.util | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.udf.worker.core.WorkerLogger | ||
|
|
||
| /** | ||
| * Adapts the UDF worker framework's [[WorkerLogger]] to Spark's | ||
| * [[Logging]] trait so that worker log messages go through the | ||
| * standard Spark logging pipeline. | ||
| */ | ||
| private[spark] class SparkUDFWorkerLogger | ||
| extends WorkerLogger with Logging { | ||
|
|
||
| override def info(msg: => String): Unit = { | ||
| logInfo(msg) | ||
| } | ||
| override def info(msg: => String, t: Throwable): Unit = | ||
| logInfo(msg, t) | ||
| override def warn(msg: => String): Unit = logWarning(msg) | ||
| override def warn(msg: => String, t: Throwable): Unit = | ||
| logWarning(msg, t) | ||
| override def debug(msg: => String): Unit = logDebug(msg) | ||
| override def debug(msg: => String, t: Throwable): Unit = | ||
| logDebug(msg, t) | ||
| } | ||
This file contains hidden or 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
76 changes: 76 additions & 0 deletions
76
...rc/main/scala/org/apache/spark/sql/catalyst/expressions/ExternalUserDefinedFunction.scala
This file contains hidden or 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,76 @@ | ||
| /* | ||
| * 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.catalyst.expressions | ||
|
|
||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.sql.catalyst.trees.TreePattern.{EXTERNAL_UDF, TreePattern} | ||
| import org.apache.spark.sql.types.DataType | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * A serialized external UDF that is executed in an external worker process | ||
| * via the language-agnostic UDF worker framework. | ||
| * | ||
| * This is a Catalyst expression analogous to [[PythonUDF]] but | ||
| * language-agnostic. The [[payload]] carries an opaque serialized | ||
| * function definition whose interpretation is left to the worker. | ||
| * The optional [[inputTypes]] declare the expected argument types for | ||
| * validation during analysis; when absent, any input types are accepted. | ||
| * | ||
| * This expression is [[Unevaluable]] and requires a dedicated physical | ||
| * operator (e.g. [[org.apache.spark.sql.execution.externalUDF.MapPartitionExternalUDFExec]]) | ||
| * to execute. | ||
| * | ||
| * @param name Name of the UDF. | ||
| * @param payload Opaque serialized function definition. | ||
| * @param dataType Return type of the UDF. | ||
| * @param children Input argument expressions. | ||
| * @param inputTypes Optional declared input types for validation. | ||
| * @param udfDeterministic Whether this UDF is deterministic. | ||
| * @param udfNullable Whether this UDF can return null. | ||
| * @param resultId Unique expression ID for this invocation. | ||
| */ | ||
| @Experimental | ||
| case class ExternalUserDefinedFunction( | ||
| name: String, | ||
| payload: Array[Byte], | ||
| dataType: DataType, | ||
| children: Seq[Expression], | ||
| inputTypes: Option[Seq[DataType]] = None, | ||
| udfDeterministic: Boolean, | ||
| udfNullable: Boolean, | ||
| resultId: ExprId = NamedExpression.newExprId) | ||
| extends Expression with NonSQLExpression with UserDefinedExpression with Unevaluable { | ||
|
|
||
| override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) | ||
|
|
||
| override def nullable: Boolean = udfNullable | ||
|
|
||
| override lazy val canonicalized: Expression = { | ||
| val canonicalizedChildren = children.map(_.canonicalized) | ||
| // `resultId` can be seen as cosmetic variation in ExternalUserDefinedFunction, | ||
| // as it doesn't affect the result. | ||
| this.copy(resultId = ExprId(-1)).withNewChildren(canonicalizedChildren) | ||
| } | ||
|
|
||
| final override val nodePatterns: Seq[TreePattern] = Seq(EXTERNAL_UDF) | ||
|
|
||
| override protected def withNewChildrenInternal( | ||
| newChildren: IndexedSeq[Expression]): ExternalUserDefinedFunction = | ||
| copy(children = newChildren) | ||
| } |
48 changes: 48 additions & 0 deletions
48
.../src/main/scala/org/apache/spark/sql/catalyst/plans/logical/externalUDF/ExternalUDF.scala
This file contains hidden or 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,48 @@ | ||
| /* | ||
| * 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.catalyst.plans.logical.externalUDF | ||
|
|
||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet} | ||
| import org.apache.spark.sql.catalyst.plans.logical.UnaryNode | ||
| import org.apache.spark.udf.worker.UDFWorkerSpecification | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Base trait for logical plan nodes representing UDFs that are executed | ||
| * in an external worker process. This covers Python UDFs, and any future | ||
| * UDF languages that use the language-agnostic UDF worker framework. | ||
| */ | ||
| @Experimental | ||
| trait ExternalUDF extends UnaryNode { | ||
|
|
||
| /** Specification describing how to create and communicate with the UDF worker. */ | ||
| def workerSpec: UDFWorkerSpecification | ||
|
|
||
| /** Output attributes produced by this UDF node. */ | ||
| def resultAttributes: Seq[Attribute] | ||
|
|
||
| override def output: Seq[Attribute] = resultAttributes | ||
|
|
||
| override val producedAttributes: AttributeSet = AttributeSet(resultAttributes) | ||
|
|
||
| // The UDF may reference any column from the child output. | ||
| // Explicitly include child.outputSet so the optimizer does not | ||
| // prune input columns that the external worker needs. | ||
| override lazy val references: AttributeSet = child.outputSet | ||
| } |
47 changes: 47 additions & 0 deletions
47
...la/org/apache/spark/sql/catalyst/plans/logical/externalUDF/MapPartitionsExternalUDF.scala
This file contains hidden or 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,47 @@ | ||
| /* | ||
| * 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.catalyst.plans.logical.externalUDF | ||
|
|
||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, | ||
| ExternalUserDefinedFunction} | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.udf.worker.UDFWorkerSpecification | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Logical plan node for mapPartitions-style UDF execution in an | ||
| * external worker process. | ||
| * | ||
| * @param workerSpec Specification describing the UDF worker. | ||
| * @param functionExpr The UDF to invoke. | ||
| * @param resultAttributes Output attributes produced by the UDF. | ||
| * @param child Input relation whose partitions are processed. | ||
| */ | ||
| @Experimental | ||
| case class MapPartitionsExternalUDF( | ||
| workerSpec: UDFWorkerSpecification, | ||
| function: ExternalUserDefinedFunction, | ||
| resultAttributes: Seq[Attribute], | ||
| child: LogicalPlan) | ||
| extends ExternalUDF { | ||
|
|
||
| override protected def withNewChildInternal( | ||
| newChild: LogicalPlan): MapPartitionsExternalUDF = | ||
| copy(child = newChild) | ||
| } |
This file contains hidden or 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 hidden or 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 hidden or 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.
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.
why do we need a pass-through interface? Can we just use Spark's logging?
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.
Because it is intended that the
udfpackages does not have any spark dependencies and can be consumed as a standalone package