-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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
[SPARK-6194] [SPARK-677] [PySpark] fix memory leak in collect() #4923
Changes from 3 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,26 +19,27 @@ package org.apache.spark.api.python | |
|
||
import java.io._ | ||
import java.net._ | ||
import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, UUID, Collections} | ||
|
||
import org.apache.spark.input.PortableDataStream | ||
import java.util.{Collections, ArrayList => JArrayList, List => JList, Map => JMap} | ||
|
||
import scala.collection.JavaConversions._ | ||
import scala.collection.mutable | ||
import scala.language.existentials | ||
|
||
import com.google.common.base.Charsets.UTF_8 | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.io.compress.CompressionCodec | ||
import org.apache.hadoop.mapred.{InputFormat, OutputFormat, JobConf} | ||
import org.apache.hadoop.mapred.{InputFormat, JobConf, OutputFormat} | ||
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, OutputFormat => NewOutputFormat} | ||
|
||
import org.apache.spark._ | ||
import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} | ||
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} | ||
import org.apache.spark.broadcast.Broadcast | ||
import org.apache.spark.input.PortableDataStream | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.util.Utils | ||
|
||
import scala.util.control.NonFatal | ||
|
||
private[spark] class PythonRDD( | ||
@transient parent: RDD[_], | ||
command: Array[Byte], | ||
|
@@ -341,21 +342,28 @@ private[spark] object PythonRDD extends Logging { | |
/** | ||
* Adapter for calling SparkContext#runJob from Python. | ||
* | ||
* This method will return an iterator of an array that contains all elements in the RDD | ||
* This method will serve an iterator of an array that contains all elements in the RDD | ||
* (effectively a collect()), but allows you to run on a certain subset of partitions, | ||
* or to enable local execution. | ||
*/ | ||
def runJob( | ||
sc: SparkContext, | ||
rdd: JavaRDD[Array[Byte]], | ||
partitions: JArrayList[Int], | ||
allowLocal: Boolean): Iterator[Array[Byte]] = { | ||
allowLocal: Boolean): Int = { | ||
type ByteArray = Array[Byte] | ||
type UnrolledPartition = Array[ByteArray] | ||
val allPartitions: Array[UnrolledPartition] = | ||
sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions, allowLocal) | ||
val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*) | ||
flattenedPartition.iterator | ||
serveIterator(flattenedPartition.iterator) | ||
} | ||
|
||
/** | ||
* A helper function to collect an RDD as an iterator, then serve it via socket | ||
*/ | ||
def collectAndServe[T](rdd: RDD[T]): Int = { | ||
serveIterator(rdd.collect().iterator) | ||
} | ||
|
||
def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): | ||
|
@@ -575,15 +583,32 @@ private[spark] object PythonRDD extends Logging { | |
dataOut.write(bytes) | ||
} | ||
|
||
def writeToFile[T](items: java.util.Iterator[T], filename: String) { | ||
import scala.collection.JavaConverters._ | ||
writeToFile(items.asScala, filename) | ||
} | ||
private def serveIterator[T](items: Iterator[T]): Int = { | ||
val serverSocket = new ServerSocket(0, 1) | ||
serverSocket.setReuseAddress(true) | ||
serverSocket.setSoTimeout(3000) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What's the motivation for this timeout? Might be nice to add a comment. |
||
|
||
new Thread("serve iterator") { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we could add a second |
||
setDaemon(true) | ||
override def run() { | ||
try { | ||
val sock = serverSocket.accept() | ||
val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) | ||
try { | ||
writeIteratorToStream(items, out) | ||
} finally { | ||
out.close() | ||
} | ||
} catch { | ||
case NonFatal(e) => | ||
logError(s"Error while sending iterator: $e") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we change this to
then the full stacktrace will be printed as part of the log message. |
||
} finally { | ||
serverSocket.close() | ||
} | ||
} | ||
}.start() | ||
|
||
def writeToFile[T](items: Iterator[T], filename: String) { | ||
val file = new DataOutputStream(new FileOutputStream(filename)) | ||
writeIteratorToStream(items, file) | ||
file.close() | ||
serverSocket.getLocalPort | ||
} | ||
|
||
private def getMergedConf(confAsMap: java.util.HashMap[String, String], | ||
|
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.
Even with the updated description, this method's return type could be confusing to new readers.
It might help to add an explicit description of the return type, e.g.
We should also document the lifecycle of the server socket created by this method: what happens if a client does not consume the whole iterator? Is it the caller's responsibility to close the socket at that point?