Skip to content

Commit a5a73e0

Browse files
mnazbroRobert Kruszewski
authored andcommitted
Update based on review
1 parent cd6073c commit a5a73e0

File tree

3 files changed

+11
-12
lines changed

3 files changed

+11
-12
lines changed

core/src/main/scala/org/apache/spark/api/conda/CondaEnvironmentManager.scala

Lines changed: 9 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ import java.nio.file.Files
2020
import java.nio.file.Path
2121
import java.nio.file.Paths
2222
import java.nio.file.attribute.PosixFilePermission
23-
import java.util.concurrent.locks.{Lock, ReentrantLock}
23+
import java.util.concurrent.locks.ReentrantLock
2424
import java.util.regex.Pattern
2525

2626
import scala.collection.JavaConverters._
@@ -213,8 +213,9 @@ object CondaEnvironmentManager extends Logging {
213213
private[this] val httpUrlToken =
214214
Pattern.compile("(\\b\\w+://[^:/@]*:)([^/@]+)(?=@([\\w-.]+)(:\\d+)?\\b)")
215215

216-
private val initializedEnvironmentsLock = new ReentrantLock()
217-
private val initializedEnvironments = mutable.HashMap[CondaSetupInstructions, CondaEnvironment]()
216+
private[this] val initializedEnvironmentsLock = new ReentrantLock()
217+
private[this] val initializedEnvironments =
218+
mutable.HashMap[CondaSetupInstructions, CondaEnvironment]()
218219

219220
private[conda] def redactCredentials(line: String): String = {
220221
httpUrlToken.matcher(line).replaceAll("$1<password>")
@@ -228,11 +229,8 @@ object CondaEnvironmentManager extends Logging {
228229
new CondaEnvironmentManager(condaBinaryPath, verbosity, packageDirs)
229230
}
230231

231-
/**
232-
* Helper method to create a conda environment from [[CondaEnvironment.CondaSetupInstructions]].
233-
* This is intended to be called on the executor with serialized instructions.
234-
*/
235-
def createCondaEnvironment(instructions: CondaSetupInstructions): CondaEnvironment = {
232+
private[this] def createCondaEnvironment(
233+
instructions: CondaSetupInstructions): CondaEnvironment = {
236234
val condaPackages = instructions.packages
237235
val env = SparkEnv.get
238236
val condaEnvManager = CondaEnvironmentManager.fromConf(env.conf)
@@ -247,9 +245,10 @@ object CondaEnvironmentManager extends Logging {
247245
}
248246

249247
/**
250-
* Helper method that will cache precreated conda environments.
248+
* Gets the conda environment for [[CondaEnvironment.CondaSetupInstructions]]. This will create
249+
* a new one if none exists.
251250
*/
252-
def createOrGetCondaEnvironment(instructions: CondaSetupInstructions): CondaEnvironment = {
251+
def getOrCreateCondaEnvironment(instructions: CondaSetupInstructions): CondaEnvironment = {
253252
initializedEnvironmentsLock.lock()
254253
try {
255254
initializedEnvironments.get(instructions) match {

core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -94,7 +94,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String],
9494

9595
private[this] val condaEnv = {
9696
// Set up conda environment if there are any conda packages requested
97-
condaInstructions.map(CondaEnvironmentManager.createOrGetCondaEnvironment)
97+
condaInstructions.map(CondaEnvironmentManager.getOrCreateCondaEnvironment)
9898
}
9999

100100
private[this] val envVars: Map[String, String] = {

core/src/main/scala/org/apache/spark/api/r/RRunner.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -348,7 +348,7 @@ private[r] object RRunner {
348348
val sparkConf = SparkEnv.get.conf
349349
val requestedRCommand = Provenance.fromConf("spark.r.command")
350350
.orElse(Provenance.fromConf("spark.sparkr.r.command"))
351-
val condaEnv = condaSetupInstructions.map(CondaEnvironmentManager.createOrGetCondaEnvironment)
351+
val condaEnv = condaSetupInstructions.map(CondaEnvironmentManager.getOrCreateCondaEnvironment)
352352
val rCommand = condaEnv.map { conda =>
353353
requestedRCommand.foreach(exec => sys.error(s"It's forbidden to set the r executable " +
354354
s"when using conda, but found: $exec"))

0 commit comments

Comments
 (0)