-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-13587] [PYSPARK] Support virtualenv in pyspark #13599
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
Changes from all 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 |
|---|---|---|
| @@ -0,0 +1,164 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.api.python | ||
|
|
||
| import java.io.File | ||
| import java.util.{Map => JMap} | ||
| import java.util.Arrays | ||
| import java.util.concurrent.atomic.AtomicInteger | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import com.google.common.io.Files | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.internal.Logging | ||
|
|
||
|
|
||
| class VirtualEnvFactory(pythonExec: String, conf: SparkConf, isDriver: Boolean) | ||
| extends Logging { | ||
|
|
||
| private val virtualEnvType = conf.get("spark.pyspark.virtualenv.type", "native") | ||
| private val virtualEnvBinPath = conf.get("spark.pyspark.virtualenv.bin.path", "") | ||
| private val initPythonPackages = conf.getOption("spark.pyspark.virtualenv.packages") | ||
| private var virtualEnvName: String = _ | ||
| private var virtualPythonExec: String = _ | ||
| private val VIRTUALENV_ID = new AtomicInteger() | ||
| private var isLauncher: Boolean = false | ||
|
|
||
| // used by launcher when user want to use virtualenv in pyspark shell. Launcher need this class | ||
| // to create virtualenv for driver. | ||
| def this(pythonExec: String, properties: JMap[String, String], isDriver: java.lang.Boolean) { | ||
|
||
| this(pythonExec, new SparkConf().setAll(properties.asScala), isDriver) | ||
| this.isLauncher = true | ||
| } | ||
|
|
||
| /* | ||
| * Create virtualenv using native virtualenv or conda | ||
|
||
| * | ||
| */ | ||
| def setupVirtualEnv(): String = { | ||
|
Contributor
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. Might be better to pass args into this function so that is could be properly unit-tested. It seems that there are no unit-tests for this class, so that seems to be a necessary addition. |
||
| /* | ||
| * | ||
| * Native Virtualenv: | ||
| * - Execute command: virtualenv -p <pythonExec> --no-site-packages <virtualenvName> | ||
| * - Execute command: python -m pip --cache-dir <cache-dir> install -r <requirement_file> | ||
| * | ||
| * Conda | ||
| * - Execute command: conda create --prefix <prefix> --file <requirement_file> -y | ||
| * | ||
| */ | ||
| logInfo("Start to setup virtualenv...") | ||
| logDebug("user.dir=" + System.getProperty("user.dir")) | ||
| logDebug("user.home=" + System.getProperty("user.home")) | ||
|
|
||
| require(virtualEnvType == "native" || virtualEnvType == "conda", | ||
| s"VirtualEnvType: $virtualEnvType is not supported." ) | ||
| require(new File(virtualEnvBinPath).exists(), | ||
| s"VirtualEnvBinPath: $virtualEnvBinPath is not defined or doesn't exist.") | ||
|
Contributor
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. In addition how are we handling the case for an existing |
||
| // Two scenarios of creating virtualenv: | ||
| // 1. created in yarn container. Yarn will clean it up after container is exited | ||
| // 2. created outside yarn container. Spark need to create temp directory and clean it after app | ||
| // finish. | ||
| // - driver of PySpark shell | ||
| // - driver of yarn-client mode | ||
|
Contributor
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. In the case of Kubernetes: This will be created in the base spark-py Docker image, which is shared between the driver and executors and the containers will be cleaned up upon termination of the job via owner-labels (for the executor) and the k8s API-Server (for the driver). As such, (hopefully with client-mode support being completed soon), the below logic should hold as well. Is this work going to be cluster-manage agnostic? Or is this supposed to only support Yarn? I would like to see this be applicable to all first-class cluster-management systems. I can help with appending to this PR: k8s Support and the appropriate integration tests. |
||
| if (isLauncher || | ||
| (isDriver && conf.get("spark.submit.deployMode") == "client")) { | ||
| val virtualenvBasedir = Files.createTempDir() | ||
| virtualenvBasedir.deleteOnExit() | ||
|
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. the temporary directory is not being deleted on exit. |
||
| virtualEnvName = virtualenvBasedir.getAbsolutePath | ||
| } else if (isDriver && conf.get("spark.submit.deployMode") == "cluster") { | ||
| virtualEnvName = "virtualenv_driver" | ||
| } else { | ||
| // use the working directory of Executor | ||
| virtualEnvName = "virtualenv_" + conf.getAppId + "_" + VIRTUALENV_ID.getAndIncrement() | ||
| } | ||
|
|
||
| // Use the absolute path of requirement file in the following cases | ||
| // 1. driver of pyspark shell | ||
| // 2. driver of yarn-client mode | ||
| // otherwise just use filename as it would be downloaded to the working directory of Executor | ||
|
Contributor
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. In Kubernetes world, I might want to use a Furthermore, when we introduce a Resource Staging Server that allows us to stage files locally, this setting will be inter-changable between something that is locally baked in vs. staged. |
||
| val pysparkRequirements = | ||
| if (isLauncher || | ||
| (isDriver && conf.get("spark.submit.deployMode") == "client")) { | ||
| conf.getOption("spark.pyspark.virtualenv.requirements") | ||
| } else { | ||
| conf.getOption("spark.pyspark.virtualenv.requirements").map(_.split("/").last) | ||
| } | ||
|
|
||
| val createEnvCommand = | ||
| if (virtualEnvType == "native") { | ||
| List(virtualEnvBinPath, | ||
| "-p", pythonExec, | ||
| "--no-site-packages", virtualEnvName) | ||
| } else { | ||
| // Two cases of conda | ||
| // 1. requirement file is specified. (Batch mode) | ||
| // 2. requirement file is not specified. (Interactive mode). | ||
| // In this case `spark.pyspark.virtualenv.python_version` must be specified. | ||
|
|
||
| if (pysparkRequirements.isDefined) { | ||
|
Contributor
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. Please re-write as a |
||
| List(virtualEnvBinPath, | ||
| "create", "--prefix", virtualEnvName, | ||
| "--file", pysparkRequirements.get, "-y") | ||
| } else { | ||
| require(conf.contains("spark.pyspark.virtualenv.python_version"), | ||
| "spark.pyspark.virtualenv.python_version is not set when using conda " + | ||
| "in interactive mode") | ||
| val pythonVersion = conf.get("spark.pyspark.virtualenv.python_version") | ||
| List(virtualEnvBinPath, | ||
| "create", "--prefix", virtualEnvName, | ||
| "python=" + pythonVersion, "-y") | ||
|
||
| } | ||
| } | ||
| execCommand(createEnvCommand) | ||
|
|
||
| virtualPythonExec = virtualEnvName + "/bin/python" | ||
| if (virtualEnvType == "native" && pysparkRequirements.isDefined) { | ||
|
Contributor
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.
|
||
| // requirement file for native is not mandatory, run this only when requirement file | ||
| // is specified. | ||
| execCommand(List(virtualPythonExec, "-m", "pip", | ||
| "--cache-dir", System.getProperty("user.home"), | ||
|
||
| "install", "-r", pysparkRequirements.get)) | ||
| } | ||
| // install additional packages | ||
| if (initPythonPackages.isDefined) { | ||
|
Contributor
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.
|
||
| execCommand(List(virtualPythonExec, "-m", "pip", | ||
| "install") ::: initPythonPackages.get.split(":").toList); | ||
| } | ||
| logInfo(s"virtualenv is created at to $virtualPythonExec") | ||
| virtualPythonExec | ||
| } | ||
|
|
||
| private def execCommand(commands: List[String]): Unit = { | ||
| logInfo("Running command:" + commands.mkString(" ")) | ||
| val pb = new ProcessBuilder(commands.asJava) | ||
| // don't inheritIO when it is used in launcher, because launcher would capture the standard | ||
| // output to assemble the spark-submit command. | ||
| if(!isLauncher) { | ||
| pb.inheritIO(); | ||
| } | ||
| // pip internally use environment variable `HOME` | ||
| pb.environment().put("HOME", System.getProperty("user.home")) | ||
| val proc = pb.start() | ||
| val exitCode = proc.waitFor() | ||
| if (exitCode != 0) { | ||
| throw new RuntimeException("Fail to run command: " + commands.mkString(" ")) | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer | |
| import scala.util.Try | ||
|
|
||
| import org.apache.spark.{SparkConf, SparkUserAppException} | ||
| import org.apache.spark.api.python.PythonUtils | ||
| import org.apache.spark.api.python.{PythonUtils, VirtualEnvFactory} | ||
| import org.apache.spark.internal.config._ | ||
| import org.apache.spark.util.{RedirectThread, Utils} | ||
|
|
||
|
|
@@ -41,12 +41,17 @@ object PythonRunner { | |
| val otherArgs = args.slice(2, args.length) | ||
| val sparkConf = new SparkConf() | ||
| val secret = Utils.createSecret(sparkConf) | ||
| val pythonExec = sparkConf.get(PYSPARK_DRIVER_PYTHON) | ||
| var pythonExec = sparkConf.get(PYSPARK_DRIVER_PYTHON) | ||
| .orElse(sparkConf.get(PYSPARK_PYTHON)) | ||
| .orElse(sys.env.get("PYSPARK_DRIVER_PYTHON")) | ||
| .orElse(sys.env.get("PYSPARK_PYTHON")) | ||
| .getOrElse("python") | ||
|
|
||
| if (sparkConf.getBoolean("spark.pyspark.virtualenv.enabled", false)) { | ||
| val virtualEnvFactory = new VirtualEnvFactory(pythonExec, sparkConf, true) | ||
| pythonExec = virtualEnvFactory.setupVirtualEnv() | ||
|
||
| } | ||
|
|
||
| // Format python file paths before adding them to the PYTHONPATH | ||
| val formattedPythonFile = formatPath(pythonFile) | ||
| val formattedPyFiles = resolvePyFiles(formatPaths(pyFiles)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -439,6 +439,19 @@ private[spark] class SparkSubmit extends Logging { | |
| sparkConf.set("spark.submit.pyFiles", localPyFiles) | ||
| } | ||
|
|
||
| // for pyspark virtualenv | ||
| if (args.isPython) { | ||
| if (clusterManager != YARN && | ||
| args.sparkProperties.getOrElse("spark.pyspark.virtualenv.enabled", "false") == "true") { | ||
| printErrorAndExit("virtualenv is only supported in yarn mode") | ||
|
||
| } | ||
| if (args.sparkProperties.contains("spark.pyspark.virtualenv.requirements")) { | ||
| // distribute virtualenv requirement file as --files | ||
| args.files = mergeFileLists(args.files, | ||
| args.sparkProperties("spark.pyspark.virtualenv.requirements")) | ||
| } | ||
| } | ||
|
|
||
| // In YARN mode for an R app, add the SparkR package archive and the R package | ||
| // archive containing all of the built R libraries to archives so that they can | ||
| // be distributed with the job | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,7 +24,7 @@ import javax.annotation.concurrent.GuardedBy | |
| import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} | ||
| import scala.concurrent.Future | ||
|
|
||
| import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} | ||
| import org.apache.spark._ | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.rpc._ | ||
| import org.apache.spark.scheduler._ | ||
|
|
@@ -98,7 +98,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| private val reviveThread = | ||
| ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") | ||
|
|
||
| class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) | ||
| class DriverEndpoint(override val rpcEnv: RpcEnv) | ||
|
||
| extends ThreadSafeRpcEndpoint with Logging { | ||
|
|
||
| // Executors that have been lost, but for which we don't yet know the real exit reason. | ||
|
|
@@ -228,6 +228,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| context.reply(true) | ||
|
|
||
| case RetrieveSparkAppConfig => | ||
| val sparkProperties = new ArrayBuffer[(String, String)] | ||
| for ((key, value) <- scheduler.sc.conf.getAll) { | ||
| if (key.startsWith("spark.")) { | ||
| sparkProperties += ((key, value)) | ||
| } | ||
| } | ||
| val reply = SparkAppConfig( | ||
| sparkProperties, | ||
| SparkEnv.get.securityManager.getIOEncryptionKey(), | ||
|
|
@@ -380,24 +386,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| protected def minRegisteredRatio: Double = _minRegisteredRatio | ||
|
|
||
| override def start() { | ||
| val properties = new ArrayBuffer[(String, String)] | ||
| for ((key, value) <- scheduler.sc.conf.getAll) { | ||
| if (key.startsWith("spark.")) { | ||
| properties += ((key, value)) | ||
| } | ||
| } | ||
|
|
||
| // TODO (prashant) send conf instead of properties | ||
| driverEndpoint = createDriverEndpointRef(properties) | ||
| driverEndpoint = createDriverEndpointRef() | ||
| } | ||
|
|
||
| protected def createDriverEndpointRef( | ||
| properties: ArrayBuffer[(String, String)]): RpcEndpointRef = { | ||
| rpcEnv.setupEndpoint(ENDPOINT_NAME, createDriverEndpoint(properties)) | ||
| protected def createDriverEndpointRef(): RpcEndpointRef = { | ||
| rpcEnv.setupEndpoint(ENDPOINT_NAME, createDriverEndpoint()) | ||
| } | ||
|
|
||
| protected def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { | ||
| new DriverEndpoint(rpcEnv, properties) | ||
| protected def createDriverEndpoint(): DriverEndpoint = { | ||
| new DriverEndpoint(rpcEnv) | ||
| } | ||
|
|
||
| def stopExecutors() { | ||
|
|
||
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.
So if the factory is made once then how will these get updated?
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.
For the existing running executor, the only way to install additional packages is via
sc.intall_packages. Andspark.pyspark.virtualenv.packageswill be updated on driver side first whensc.install_packgesis called, then new allocated executor will fetch this property and install all the additional packages properly.