Skip to content
Draft
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
10 changes: 10 additions & 0 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,16 @@
</properties>

<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-udf-worker-proto_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-udf-worker-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.scala-lang.modules</groupId>
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
Expand Down
31 changes: 31 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,13 @@ import org.apache.spark.security.CryptoStreamUtils
import org.apache.spark.serializer.{JavaSerializer, Serializer, SerializerManager}
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.storage._
import org.apache.spark.udf.worker.UDFWorkerSpecification
import org.apache.spark.udf.worker.core.{UDFWorkerManager, WorkerSecurityScope,
WorkerSession}
import org.apache.spark.udf.worker.core.direct.DirectUDFWorkerManager
import org.apache.spark.util.{RpcUtils, Utils}
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.SparkUDFWorkerLogger

/**
* :: DeveloperApi ::
Expand Down Expand Up @@ -120,6 +125,31 @@ class SparkEnv (
pythonExec: String, workerModule: String, daemonModule: String, envVars: Map[String, String])
private val pythonWorkers = mutable.HashMap[PythonWorkersKey, PythonWorkerFactory]()

/**
* :: Experimental ::
* Session factory to generate UDF worker sessions
* using the new UDF framework proposed in SPARK-55278
*/
private val udfWorkerManager: UDFWorkerManager = createUDFWorkerManager()

private def createUDFWorkerManager(): UDFWorkerManager = {
// TODO [SPARK-55278]: Select the right manager here.
new DirectUDFWorkerManager(new SparkUDFWorkerLogger())
}

/**
* :: Experimental ::
* Creates a [[WorkerSession]] for the given worker specification
* and optional security scope using the registered
* [[UDFWorkerManager]].
*/
private[spark] def createExternalUDFSession(
workerSpec: UDFWorkerSpecification,
securityScope: Option[WorkerSecurityScope] = None
): WorkerSession = {
udfWorkerManager.createSession(workerSpec, securityScope)
}

// A general, soft-reference map for metadata needed during HadoopRDD split computation
// (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
private[spark] val hadoopJobMetadata =
Expand All @@ -134,6 +164,7 @@ class SparkEnv (
if (!isStopped) {
isStopped = true
pythonWorkers.values.foreach(_.stop())
udfWorkerManager.stop()
mapOutputTracker.stop()
if (shuffleManager != null) {
shuffleManager.stop()
Expand Down
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
Copy link
Copy Markdown
Contributor

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?

Copy link
Copy Markdown
Contributor Author

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 udf packages does not have any spark dependencies and can be consumed as a standalone package

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)
}
10 changes: 10 additions & 0 deletions sql/catalyst/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,16 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-tags_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-udf-worker-proto_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-udf-worker-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>

<!--
This spark-tags test-dep is needed even though it isn't used in this module, otherwise testing-cmds that exclude
Expand Down
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)
}
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
}
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)
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ object TreePattern extends Enumeration {
val DYNAMIC_PRUNING_SUBQUERY: Value = Value
val EXISTS_SUBQUERY = Value
val EXPRESSION_WITH_RANDOM_SEED: Value = Value
val EXTERNAL_UDF: Value = Value
val EXTRACT_VALUE: Value = Value
val FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION: Value = Value
val GENERATOR: Value = Value
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4613,6 +4613,17 @@ object SQLConf {
.booleanConf
.createWithDefault(false)

val UNIFIED_UDF_EXECUTION_ENABLED =
buildConf("spark.sql.execution.udf.unified.execution.enabled")
.doc("When true, UDFs that support the language-agnostic " +
"UDF worker protocol are executed via the unified, " +
"external UDF worker framework instead of the " +
"language-specific runners. Experimental.")
.version("4.2.0")
.withBindingPolicy(ConfigBindingPolicy.SESSION)
.booleanConf
.createWithDefault(false)

val PYTHON_UDF_ARROW_ENABLED =
buildConf("spark.sql.execution.pythonUDF.arrow.enabled")
.doc("Enable Arrow optimization in regular Python UDFs. This optimization " +
Expand Down
10 changes: 10 additions & 0 deletions sql/core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -302,6 +302,16 @@
<groupId>org.apache.arrow</groupId>
<artifactId>arrow-compression</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-udf-worker-proto_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-udf-worker-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
Expand Down
Loading