aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2015-09-22 14:11:46 -0700
committerJosh Rosen <joshrosen@databricks.com>2015-09-22 14:11:46 -0700
commita96ba40f7ee1352288ea676d8844e1c8174202eb (patch)
treec91b14d838ec753d6e703f2259964c8f2d442b0d /sql
parent5aea987c904b281d7952ad8db40a32561b4ec5cf (diff)
downloadspark-a96ba40f7ee1352288ea676d8844e1c8174202eb.tar.gz
spark-a96ba40f7ee1352288ea676d8844e1c8174202eb.tar.bz2
spark-a96ba40f7ee1352288ea676d8844e1c8174202eb.zip
[SPARK-10714] [SPARK-8632] [SPARK-10685] [SQL] Refactor Python UDF handling
This patch refactors Python UDF handling: 1. Extract the per-partition Python UDF calling logic from PythonRDD into a PythonRunner. PythonRunner itself expects iterator as input/output, and thus has no dependency on RDD. This way, we can use PythonRunner directly in a mapPartitions call, or in the future in an environment without RDDs. 2. Use PythonRunner in Spark SQL's BatchPythonEvaluation. 3. Updated BatchPythonEvaluation to only use its input once, rather than twice. This should fix Python UDF performance regression in Spark 1.5. There are a number of small cleanups I wanted to do when I looked at the code, but I kept most of those out so the diff looks small. This basically implements the approach in https://github.com/apache/spark/pull/8833, but with some code moving around so the correctness doesn't depend on the inner workings of Spark serialization and task execution. Author: Reynold Xin <rxin@databricks.com> Closes #8835 from rxin/python-iter-refactor.
Diffstat (limited to 'sql')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala80
1 files changed, 46 insertions, 34 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala
index d0411da6fd..c35c726bfc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala
@@ -25,7 +25,7 @@ import scala.collection.JavaConverters._
import net.razorvine.pickle._
import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.api.python.{PythonBroadcast, PythonRDD, SerDeUtil}
+import org.apache.spark.api.python.{PythonRunner, PythonBroadcast, PythonRDD, SerDeUtil}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
@@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
-import org.apache.spark.{Accumulator, Logging => SparkLogging}
+import org.apache.spark.{Logging => SparkLogging, TaskContext, Accumulator}
/**
* A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]].
@@ -329,7 +329,13 @@ case class EvaluatePython(
/**
* :: DeveloperApi ::
* Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at a time.
- * The input data is zipped with the result of the udf evaluation.
+ *
+ * Python evaluation works by sending the necessary (projected) input data via a socket to an
+ * external Python process, and combine the result from the Python process with the original row.
+ *
+ * For each row we send to Python, we also put it in a queue. For each output row from Python,
+ * we drain the queue to find the original input row. Note that if the Python process is way too
+ * slow, this could lead to the queue growing unbounded and eventually run out of memory.
*/
@DeveloperApi
case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: SparkPlan)
@@ -342,51 +348,57 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child:
override def canProcessSafeRows: Boolean = true
protected override def doExecute(): RDD[InternalRow] = {
- val childResults = child.execute().map(_.copy())
+ val inputRDD = child.execute().map(_.copy())
+ val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536)
+ val reuseWorker = inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true)
- val parent = childResults.mapPartitions { iter =>
+ inputRDD.mapPartitions { iter =>
EvaluatePython.registerPicklers() // register pickler for Row
+
+ // The queue used to buffer input rows so we can drain it to
+ // combine input with output from Python.
+ val queue = new java.util.concurrent.ConcurrentLinkedQueue[InternalRow]()
+
val pickle = new Pickler
val currentRow = newMutableProjection(udf.children, child.output)()
val fields = udf.children.map(_.dataType)
val schema = new StructType(fields.map(t => new StructField("", t, true)).toArray)
- iter.grouped(100).map { inputRows =>
+
+ // Input iterator to Python: input rows are grouped so we send them in batches to Python.
+ // For each row, add it to the queue.
+ val inputIterator = iter.grouped(100).map { inputRows =>
val toBePickled = inputRows.map { row =>
+ queue.add(row)
EvaluatePython.toJava(currentRow(row), schema)
}.toArray
pickle.dumps(toBePickled)
}
- }
- val pyRDD = new PythonRDD(
- parent,
- udf.command,
- udf.envVars,
- udf.pythonIncludes,
- false,
- udf.pythonExec,
- udf.pythonVer,
- udf.broadcastVars,
- udf.accumulator
- ).mapPartitions { iter =>
- val pickle = new Unpickler
- iter.flatMap { pickedResult =>
- val unpickledBatch = pickle.loads(pickedResult)
- unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala
- }
- }.mapPartitions { iter =>
+ val context = TaskContext.get()
+
+ // Output iterator for results from Python.
+ val outputIterator = new PythonRunner(
+ udf.command,
+ udf.envVars,
+ udf.pythonIncludes,
+ udf.pythonExec,
+ udf.pythonVer,
+ udf.broadcastVars,
+ udf.accumulator,
+ bufferSize,
+ reuseWorker
+ ).compute(inputIterator, context.partitionId(), context)
+
+ val unpickle = new Unpickler
val row = new GenericMutableRow(1)
- iter.map { result =>
- row(0) = EvaluatePython.fromJava(result, udf.dataType)
- row: InternalRow
- }
- }
+ val joined = new JoinedRow
- childResults.zip(pyRDD).mapPartitions { iter =>
- val joinedRow = new JoinedRow()
- iter.map {
- case (row, udfResult) =>
- joinedRow(row, udfResult)
+ outputIterator.flatMap { pickedResult =>
+ val unpickledBatch = unpickle.loads(pickedResult)
+ unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala
+ }.map { result =>
+ row(0) = EvaluatePython.fromJava(result, udf.dataType)
+ joined(queue.poll(), row)
}
}
}