aboutsummaryrefslogtreecommitdiff
path: root/core
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 /core
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 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala54
1 files changed, 43 insertions, 11 deletions
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 69da180593..3788d18297 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -24,6 +24,7 @@ import java.util.{Collections, ArrayList => JArrayList, List => JList, Map => JM
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.language.existentials
+import scala.util.control.NonFatal
import com.google.common.base.Charsets.UTF_8
import org.apache.hadoop.conf.Configuration
@@ -38,7 +39,6 @@ import org.apache.spark.input.PortableDataStream
import org.apache.spark.rdd.RDD
import org.apache.spark.util.{SerializableConfiguration, Utils}
-import scala.util.control.NonFatal
private[spark] class PythonRDD(
parent: RDD[_],
@@ -61,11 +61,39 @@ private[spark] class PythonRDD(
if (preservePartitoning) firstParent.partitioner else None
}
+ val asJavaRDD: JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
+
override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
+ val runner = new PythonRunner(
+ command, envVars, pythonIncludes, pythonExec, pythonVer, broadcastVars, accumulator,
+ bufferSize, reuse_worker)
+ runner.compute(firstParent.iterator(split, context), split.index, context)
+ }
+}
+
+
+/**
+ * A helper class to run Python UDFs in Spark.
+ */
+private[spark] class PythonRunner(
+ command: Array[Byte],
+ envVars: JMap[String, String],
+ pythonIncludes: JList[String],
+ pythonExec: String,
+ pythonVer: String,
+ broadcastVars: JList[Broadcast[PythonBroadcast]],
+ accumulator: Accumulator[JList[Array[Byte]]],
+ bufferSize: Int,
+ reuse_worker: Boolean)
+ extends Logging {
+
+ def compute(
+ inputIterator: Iterator[_],
+ partitionIndex: Int,
+ context: TaskContext): Iterator[Array[Byte]] = {
val startTime = System.currentTimeMillis
val env = SparkEnv.get
- val localdir = env.blockManager.diskBlockManager.localDirs.map(
- f => f.getPath()).mkString(",")
+ val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",")
envVars.put("SPARK_LOCAL_DIRS", localdir) // it's also used in monitor thread
if (reuse_worker) {
envVars.put("SPARK_REUSE_WORKER", "1")
@@ -75,7 +103,7 @@ private[spark] class PythonRDD(
@volatile var released = false
// Start a thread to feed the process input from our parent's iterator
- val writerThread = new WriterThread(env, worker, split, context)
+ val writerThread = new WriterThread(env, worker, inputIterator, partitionIndex, context)
context.addTaskCompletionListener { context =>
writerThread.shutdownOnTaskCompletion()
@@ -183,13 +211,16 @@ private[spark] class PythonRDD(
new InterruptibleIterator(context, stdoutIterator)
}
- val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
-
/**
* The thread responsible for writing the data from the PythonRDD's parent iterator to the
* Python process.
*/
- class WriterThread(env: SparkEnv, worker: Socket, split: Partition, context: TaskContext)
+ class WriterThread(
+ env: SparkEnv,
+ worker: Socket,
+ inputIterator: Iterator[_],
+ partitionIndex: Int,
+ context: TaskContext)
extends Thread(s"stdout writer for $pythonExec") {
@volatile private var _exception: Exception = null
@@ -211,11 +242,11 @@ private[spark] class PythonRDD(
val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize)
val dataOut = new DataOutputStream(stream)
// Partition index
- dataOut.writeInt(split.index)
+ dataOut.writeInt(partitionIndex)
// Python version of driver
PythonRDD.writeUTF(pythonVer, dataOut)
// sparkFilesDir
- PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut)
+ PythonRDD.writeUTF(SparkFiles.getRootDirectory(), dataOut)
// Python includes (*.zip and *.egg files)
dataOut.writeInt(pythonIncludes.size())
for (include <- pythonIncludes.asScala) {
@@ -246,7 +277,7 @@ private[spark] class PythonRDD(
dataOut.writeInt(command.length)
dataOut.write(command)
// Data values
- PythonRDD.writeIteratorToStream(firstParent.iterator(split, context), dataOut)
+ PythonRDD.writeIteratorToStream(inputIterator, dataOut)
dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION)
dataOut.writeInt(SpecialLengths.END_OF_STREAM)
dataOut.flush()
@@ -327,7 +358,8 @@ private[spark] object PythonRDD extends Logging {
// remember the broadcasts sent to each worker
private val workerBroadcasts = new mutable.WeakHashMap[Socket, mutable.Set[Long]]()
- private def getWorkerBroadcasts(worker: Socket) = {
+
+ def getWorkerBroadcasts(worker: Socket): mutable.Set[Long] = {
synchronized {
workerBroadcasts.getOrElseUpdate(worker, new mutable.HashSet[Long]())
}