Browse Source

[livy] Fix launching pyspark inside yarn

Livy-on-Yarn now requires that the SPARK_HOME environment variable
be set in order to find and upload the pyspark.zip from the local
machine.
Erick Tryzelaar 10 years ago
parent
commit
dff49eaa7c

+ 1 - 1
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/spark/SparkSubmitProcessBuilder.scala

@@ -46,7 +46,7 @@ class SparkSubmitProcessBuilder(livyConf: LivyConf) extends Logging {
   private[this] var _master: Option[String] = None
   private[this] var _master: Option[String] = None
   private[this] var _deployMode: Option[String] = None
   private[this] var _deployMode: Option[String] = None
   private[this] var _className: Option[String] = None
   private[this] var _className: Option[String] = None
-  private[this] var _name: Option[String] = None
+  private[this] var _name: Option[String] = Some("Livy")
   private[this] var _jars: ArrayBuffer[Path] = ArrayBuffer()
   private[this] var _jars: ArrayBuffer[Path] = ArrayBuffer()
   private[this] var _pyFiles: ArrayBuffer[Path] = ArrayBuffer()
   private[this] var _pyFiles: ArrayBuffer[Path] = ArrayBuffer()
   private[this] var _files: ArrayBuffer[Path] = ArrayBuffer()
   private[this] var _files: ArrayBuffer[Path] = ArrayBuffer()

+ 1 - 0
apps/spark/java/livy-repl/pom.xml

@@ -154,6 +154,7 @@
                 <version>1.0</version>
                 <version>1.0</version>
                 <configuration>
                 <configuration>
                     <systemProperties>
                     <systemProperties>
+                        <spark.app.name>Livy</spark.app.name>
                         <spark.master>local</spark.master>
                         <spark.master>local</spark.master>
                         <spark.driver.allowMultipleContexts>true</spark.driver.allowMultipleContexts>
                         <spark.driver.allowMultipleContexts>true</spark.driver.allowMultipleContexts>
                         <spark.ui.enabled>false</spark.ui.enabled>
                         <spark.ui.enabled>false</spark.ui.enabled>

+ 18 - 12
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/python/PythonInterpreter.scala

@@ -22,10 +22,9 @@ import java.io._
 import java.lang.ProcessBuilder.Redirect
 import java.lang.ProcessBuilder.Redirect
 import java.nio.file.Files
 import java.nio.file.Files
 
 
+import com.cloudera.hue.livy.Logging
 import com.cloudera.hue.livy.repl.Interpreter
 import com.cloudera.hue.livy.repl.Interpreter
 import com.cloudera.hue.livy.repl.process.ProcessInterpreter
 import com.cloudera.hue.livy.repl.process.ProcessInterpreter
-import com.cloudera.hue.livy.{Logging, Utils}
-import org.apache.spark.SparkContext
 import org.json4s.JsonAST.JObject
 import org.json4s.JsonAST.JObject
 import org.json4s.jackson.JsonMethods._
 import org.json4s.jackson.JsonMethods._
 import org.json4s.jackson.Serialization.write
 import org.json4s.jackson.Serialization.write
@@ -34,7 +33,6 @@ import py4j.GatewayServer
 
 
 import scala.annotation.tailrec
 import scala.annotation.tailrec
 import scala.collection.JavaConversions._
 import scala.collection.JavaConversions._
-import scala.collection.mutable.ArrayBuffer
 
 
 object PythonInterpreter extends Logging {
 object PythonInterpreter extends Logging {
   def apply(): Interpreter = {
   def apply(): Interpreter = {
@@ -43,12 +41,13 @@ object PythonInterpreter extends Logging {
     val gatewayServer = new GatewayServer(null, 0)
     val gatewayServer = new GatewayServer(null, 0)
     gatewayServer.start()
     gatewayServer.start()
 
 
-    val pythonPath = buildPythonPath
+    val pySparkArchives = findPySparkArchives()
+
     val builder = new ProcessBuilder(Seq(pythonExec, createFakeShell().toString))
     val builder = new ProcessBuilder(Seq(pythonExec, createFakeShell().toString))
 
 
     val env = builder.environment()
     val env = builder.environment()
 
 
-    env.put("PYTHONPATH", pythonPath)
+    env.put("PYTHONPATH", (sys.env.get("PYTHONPATH") ++ pySparkArchives).mkString(File.pathSeparator))
     env.put("PYTHONUNBUFFERED", "YES")
     env.put("PYTHONUNBUFFERED", "YES")
     env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort)
     env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort)
     env.put("SPARK_HOME", sys.env.getOrElse("SPARK_HOME", "."))
     env.put("SPARK_HOME", sys.env.getOrElse("SPARK_HOME", "."))
@@ -60,14 +59,21 @@ object PythonInterpreter extends Logging {
     new PythonInterpreter(process, gatewayServer)
     new PythonInterpreter(process, gatewayServer)
   }
   }
 
 
-  private def buildPythonPath = {
-    val pythonPath = new ArrayBuffer[String]
-    for (sparkHome <- sys.env.get("SPARK_HOME")) {
-      pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator)
-      pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.2.1-src.zip").mkString(File.separator)
+  private def findPySparkArchives(): Seq[String] = {
+    sys.env.get("PYSPARK_ARCHIVES_PATH")
+      .map(_.split(",").toSeq)
+      .getOrElse {
+      sys.env.get("SPARK_HOME") .map { case sparkHome =>
+        val pyLibPath = Seq(sparkHome, "python", "lib").mkString(File.separator)
+        val pyArchivesFile = new File(pyLibPath, "pyspark.zip")
+        require(pyArchivesFile.exists(),
+          "pyspark.zip not found; cannot run pyspark application in YARN mode.")
+        val py4jFile = new File(pyLibPath, "py4j-0.8.2.1-src.zip")
+        require(py4jFile.exists(),
+          "py4j-0.8.2.1-src.zip not found; cannot run pyspark application in YARN mode.")
+        Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath())
+      }.getOrElse(Seq())
     }
     }
-    pythonPath ++= Utils.jarOfClass(classOf[SparkContext])
-    pythonPath.mkString(File.pathSeparator)
   }
   }
 
 
   private def createFakeShell(): File = {
   private def createFakeShell(): File = {

+ 1 - 1
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/scala/SparkInterpreter.scala

@@ -66,7 +66,7 @@ class SparkInterpreter extends Interpreter {
       .setAppName("Livy Spark shell")
       .setAppName("Livy Spark shell")
       .set("spark.repl.class.uri", sparkIMain.classServerUri)
       .set("spark.repl.class.uri", sparkIMain.classServerUri)
 
 
-    sparkContext = new SparkContext(sparkConf)
+    sparkContext = SparkContext.getOrCreate(sparkConf)
 
 
     sparkIMain.beQuietDuring {
     sparkIMain.beQuietDuring {
       sparkIMain.bind("sc", "org.apache.spark.SparkContext", sparkContext, List("""@transient"""))
       sparkIMain.bind("sc", "org.apache.spark.SparkContext", sparkContext, List("""@transient"""))

+ 12 - 5
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionYarn.scala

@@ -21,11 +21,11 @@ package com.cloudera.hue.livy.server.interactive
 import java.lang.ProcessBuilder.Redirect
 import java.lang.ProcessBuilder.Redirect
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.TimeUnit
 
 
-import com.cloudera.hue.livy.sessions.Error
+import com.cloudera.hue.livy.sessions.{PySpark, Error, Spark}
+import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder.{AbsolutePath, RelativePath}
 import com.cloudera.hue.livy.spark.{SparkProcess, SparkSubmitProcessBuilder}
 import com.cloudera.hue.livy.spark.{SparkProcess, SparkSubmitProcessBuilder}
-import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder.{RelativePath, AbsolutePath}
-import com.cloudera.hue.livy.yarn.{Client, Job}
-import com.cloudera.hue.livy.{LineBufferedProcess, LivyConf, Utils}
+import com.cloudera.hue.livy.yarn.Client
+import com.cloudera.hue.livy.{LivyConf, Utils}
 
 
 import scala.concurrent.duration._
 import scala.concurrent.duration._
 import scala.concurrent.{Await, ExecutionContext, ExecutionContextExecutor, Future}
 import scala.concurrent.{Await, ExecutionContext, ExecutionContextExecutor, Future}
@@ -58,10 +58,17 @@ object InteractiveSessionYarn {
     createInteractiveRequest.proxyUser.foreach(builder.proxyUser)
     createInteractiveRequest.proxyUser.foreach(builder.proxyUser)
     createInteractiveRequest.pyFiles.map(RelativePath).foreach(builder.pyFile)
     createInteractiveRequest.pyFiles.map(RelativePath).foreach(builder.pyFile)
 
 
+    val kind = createInteractiveRequest.kind.toString
+
+    createInteractiveRequest.kind match {
+      case PySpark() => builder.conf("spark.yarn.isPython", "true")
+      case _ =>
+    }
+
     builder.redirectOutput(Redirect.PIPE)
     builder.redirectOutput(Redirect.PIPE)
     builder.redirectErrorStream(true)
     builder.redirectErrorStream(true)
 
 
-    val process = builder.start(AbsolutePath(livyJar(livyConf)), List(createInteractiveRequest.kind.toString))
+    val process = builder.start(AbsolutePath(livyJar(livyConf)), List(kind))
 
 
     new InteractiveSessionYarn(id, client, process, createInteractiveRequest)
     new InteractiveSessionYarn(id, client, process, createInteractiveRequest)
   }
   }

+ 2 - 1
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/BaseSessionSpec.scala

@@ -81,7 +81,8 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
         "evalue" ->
         "evalue" ->
           """<console>:8: error: not found: value x
           """<console>:8: error: not found: value x
             |              x
             |              x
-            |              ^""".stripMargin
+            |              ^""".stripMargin,
+        "traceback" -> List()
       ))
       ))
 
 
       result should equal (expectedResult)
       result should equal (expectedResult)