Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15208037
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
---
    @@ -57,7 +57,9 @@ private[spark] class PythonRDD[T: ClassTag](
       override def compute(split: Partition, context: TaskContext): 
Iterator[Array[Byte]] = {
         val startTime = System.currentTimeMillis
         val env = SparkEnv.get
    -    val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap)
    +    val localdir = env.conf.get("spark.local.dir", 
System.getProperty("java.io.tmpdir"))
    +    val worker: Socket = env.createPythonWorker(pythonExec,
    +      envVars.toMap + ("SPARK_LOCAL_DIR" -> localdir))
    --- End diff --
    
    Instead of passing spark.local.dir, we should figure out which directories 
the DiskBlockManager created (you can get it from `env.blockManager`) and pass 
a comma-separated list of those. This way the data for this Spark application 
is all in one directory, and Java can make sure we clean it all up at the end. 
Otherwise the way you have things set up now, those directories are never 
cleared if the Python worker crashes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to