浏览代码

HUE-3018 [livy] Add support for whitelisting spark config options

Erick Tryzelaar 10 年之前
父节点
当前提交
a4659f3
共有 19 个文件被更改,包括 529 次插入64 次删除
  1. 4 0
      apps/spark/java/README.rst
  2. 8 0
      apps/spark/java/conf/livy-defaults.conf.tmpl
  3. 123 0
      apps/spark/java/conf/spark-user-configurable-options.template
  4. 30 19
      apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/Utils.scala
  5. 2 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionServlet.scala
  6. 1 1
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batch/BatchServletSpec.scala
  7. 122 0
      apps/spark/java/livy-spark/src/main/resources/com/cloudera/hue/livy/spark/default-spark-user-configurable-options.conf
  8. 32 0
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/ConfigOptionNotAllowed.scala
  9. 42 2
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkManager.scala
  10. 30 22
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkProcessBuilder.scala
  11. 22 6
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkProcessBuilderFactory.scala
  12. 42 0
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkUserConfigurableOptions.scala
  13. 1 1
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/batch/BatchSessionFactory.scala
  14. 2 1
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/batch/CreateBatchRequest.scala
  15. 2 2
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/interactive/CreateInteractiveRequest.scala
  16. 10 3
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/interactive/InteractiveSessionFactory.scala
  17. 12 6
      apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/interactive/InteractiveSessionYarnFactory.scala
  18. 43 0
      apps/spark/java/livy-spark/src/test/scala/com/cloudera/hue/livy/spark/SparkProcessBuilderSpec.scala
  19. 1 1
      apps/spark/java/livy-spark/src/test/scala/com/cloudera/hue/livy/spark/batch/BatchProcessSpec.scala

+ 4 - 0
apps/spark/java/README.rst

@@ -376,6 +376,8 @@ Request Body
 +----------------+--------------------------------------------------------------------------------+------------------+
 | name           | Name of the application                                                        | string           |
 +----------------+--------------------------------------------------------------------------------+------------------+
+| conf           | Spark configuration property                                                   | list of key=val  |
++----------------+--------------------------------------------------------------------------------+------------------+
 
 
 Response Body
@@ -522,6 +524,8 @@ Request Body
 +----------------+--------------------------------------------------+-----------------+
 | name           | Name of the application                          | string          |
 +----------------+--------------------------------------------------+-----------------+
+| conf           | Spark configuration property                     | list of key=val |
++----------------+--------------------------------------------------+-----------------+
 
 
 Response Body

+ 8 - 0
apps/spark/java/conf/livy-defaults.conf.tmpl

@@ -33,3 +33,11 @@
 
 # If livy should use proxy users when submitting a job.
 ## livy.impersonation.enabled = true
+
+# Allow spark to reference files found in this directory. This may either be a local directory
+# or an hdfs:// directory. Be careful setting this option, as it may allow users to access
+# any file the Livy process can access, such as Livy's SSL certificate.
+## livy.files.dir = /var/run/livy/uploaded-files
+
+# Optional list of Spark options that can be configured by the user.
+## livy.spark.user-configurable-options = spark.serializer,spark.logConf

+ 123 - 0
apps/spark/java/conf/spark-user-configurable-options.template

@@ -0,0 +1,123 @@
+// Application properties
+spark.app.name
+spark.driver.cores
+spark.driver.maxResultSize
+spark.driver.memory
+spark.executor.memory
+spark.extraListeners
+// spark.local.dir
+spark.logConf
+// spark.master
+
+// Runtime Environment
+spark.driver.extraClassPath
+// spark.driver.extraJavaOptions
+spark.driver.extraLibraryPath
+spark.driver.userClassPathFirst
+spark.executor.extraClassPath
+// spark.executor.extraJavaOptions
+spark.executor.extraLibraryPath
+spark.executor.instances
+spark.executor.logs.rolling.maxRetainedFiles
+spark.executor.logs.rolling.maxSize
+spark.executor.logs.rolling.strategy
+spark.executor.logs.rolling.time.interval
+spark.executor.userClassPathFirst
+// spark.executorEnv.[EnvironmentVariableName]
+spark.python.profile
+spark.python.profile.dump
+spark.python.worker.memory
+spark.python.worker.reuse
+spark.reducer.maxSizeInFlight
+
+// Shuffle Behavior
+spark.shuffle.blockTransferService
+spark.shuffle.compress
+spark.shuffle.consolidateFiles
+spark.shuffle.file.buffer
+spark.shuffle.io.maxRetries
+spark.shuffle.io.numConnectionsPerPeer
+spark.shuffle.io.preferDirectBufs
+spark.shuffle.io.retryWait
+spark.shuffle.manager
+spark.shuffle.memoryFraction
+spark.shuffle.service.enabled
+spark.shuffle.service.port
+spark.shuffle.sort.bypassMergeThreshold
+spark.shuffle.spill
+spark.shuffle.spill.compress
+
+// Spark UI
+spark.eventLog.compress
+spark.eventLog.dir
+spark.eventLog.enabled
+spark.ui.killEnabled
+spark.ui.port
+spark.ui.retainedJobs
+spark.ui.retainedStages
+spark.worker.ui.retainedExecutors
+spark.worker.ui.retainedDrivers
+
+// Compression and Serialization
+spark.broadcast.compress
+spark.closure.serializer
+spark.io.compression.codec
+spark.io.compression.lz4.blockSize
+spark.io.compression.snappy.blockSize
+spark.kryo.classesToRegister
+spark.kryo.referenceTracking
+spark.kryo.registrationRequired
+spark.kryo.registrator
+spark.kryoserializer.buffer.max
+spark.kryoserializer.buffer
+spark.rdd.compress
+spark.serializer
+spark.serializer.objectStreamReset
+
+// Execution Behavior
+spark.broadcast.blockSize
+spark.broadcast.factory
+spark.cleaner.ttl
+spark.executor.cores
+spark.default.parallelism
+spark.executor.heartbeatInterval
+spark.files.fetchTimeout
+spark.files.useFetchCache
+spark.files.overwrite
+spark.hadoop.cloneConf
+spark.hadoop.validateOutputSpecs
+spark.storage.memoryFraction
+spark.storage.memoryMapThreshold
+spark.storage.unrollFraction
+spark.externalBlockStore.blockManager
+spark.externalBlockStore.baseDir
+spark.externalBlockStore.url
+
+// Networking
+
+// Dynamic Allocation
+spark.dynamicAllocation.enabled
+spark.dynamicAllocation.executorIdleTimeout
+spark.dynamicAllocation.cachedExecutorIdleTimeout
+spark.dynamicAllocation.initialExecutors
+spark.dynamicAllocation.maxExecutors
+spark.dynamicAllocation.minExecutors
+spark.dynamicAllocation.schedulerBacklogTimeout
+spark.dynamicAllocation.sustainedSchedulerBacklogTimeout
+
+// Security
+
+// Encryption
+
+// Spark Streaming
+spark.streaming.backpressure.enabled
+spark.streaming.blockInterval
+spark.streaming.receiver.maxRate
+spark.streaming.receiver.writeAheadLog.enable
+spark.streaming.unpersist
+spark.streaming.kafka.maxRatePerPartition
+spark.streaming.kafka.maxRetries
+spark.streaming.ui.retainedBatches
+
+// SparkR
+spark.r.numRBackendThreads

+ 30 - 19
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/Utils.scala

@@ -18,7 +18,7 @@
 
 package com.cloudera.hue.livy
 
-import java.io.{FileInputStream, InputStreamReader, File}
+import java.io.{File, FileInputStream, InputStreamReader}
 import java.util.Properties
 
 import scala.annotation.tailrec
@@ -27,11 +27,7 @@ import scala.concurrent.TimeoutException
 import scala.concurrent.duration.Duration
 
 object Utils {
-  def getPropertiesFromFile(filename: String): Map[String, String] = {
-    val file = new File(filename)
-    require(file.exists(), s"Properties file $file does not exist")
-    require(file.isFile, s"Properties file $file is not a normal file")
-
+  def getPropertiesFromFile(file: File): Map[String, String] = {
     val inReader = new InputStreamReader(new FileInputStream(file), "UTF-8")
     try {
       val properties = new Properties()
@@ -42,24 +38,39 @@ object Utils {
     }
   }
 
-  def getDefaultPropertiesFile(env: Map[String, String] = sys.env): String = {
+  def getLivyConfDir(env: Map[String, String] = sys.env): Option[File] = {
     env.get("LIVY_CONF_DIR")
       .orElse(env.get("LIVY_HOME").map(path => s"$path${File.separator}conf"))
-      .map(path => new File(s"$path${File.separator}livy-defaults.conf"))
-      .filter(_.isFile)
-      .map(_.getAbsolutePath)
-      .orNull
+      .map(new File(_))
+      .filter(_.exists())
+  }
+
+  def getLivyConfigFile(name: String): Option[File] = {
+    getLivyConfDir().map(new File(_, name)).filter(_.exists())
+  }
+
+  def getLivyConfigFileOrError(name: String): File = {
+    getLivyConfigFile(name).getOrElse {
+      throw new Exception(s"$name does not exist")
+    }
+  }
+
+  def getDefaultPropertiesFile: Option[File] = {
+    getLivyConfigFile("livy-defaults.conf")
   }
 
   def loadDefaultLivyProperties(conf: LivyConf, filePath: String = null) = {
-    val path = Option(filePath).getOrElse(getDefaultPropertiesFile())
-    Option(path).foreach { path =>
-      getPropertiesFromFile(path).filter { case (k, v) =>
-        k.startsWith("livy.")
-      }.foreach { case (k, v) =>
-        conf.setIfMissing(k, v)
-        sys.props.getOrElseUpdate(k, v)
-      }
+    val file: Option[File] = Option(filePath)
+      .map(new File(_))
+      .orElse(getDefaultPropertiesFile)
+
+    file.foreach { f =>
+      getPropertiesFromFile(f)
+        .filterKeys(_.startsWith("livy."))
+        .foreach { case (k, v) =>
+          conf.setIfMissing(k, v)
+          sys.props.getOrElseUpdate(k, v)
+        }
     }
   }
 

+ 2 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionServlet.scala

@@ -21,6 +21,7 @@ package com.cloudera.hue.livy.server
 import com.cloudera.hue.livy.Logging
 import com.cloudera.hue.livy.sessions.{SessionManager, Session}
 import com.cloudera.hue.livy.sessions.interactive.InteractiveSession.SessionFailedToStart
+import com.cloudera.hue.livy.spark.ConfigOptionNotAllowed
 import com.fasterxml.jackson.core.JsonParseException
 import org.json4s.JsonDSL._
 import org.json4s.{DefaultFormats, Formats, JValue, MappingException}
@@ -122,6 +123,7 @@ abstract class SessionServlet[S <: Session](sessionManager: SessionManager[S])
   error {
     case e: JsonParseException => BadRequest(e.getMessage)
     case e: MappingException => BadRequest(e.getMessage)
+    case e: ConfigOptionNotAllowed => BadRequest(e.getMessage)
     case e: SessionFailedToStart => InternalServerError(e.getMessage)
     case e: dispatch.StatusCode => ActionResult(ResponseStatus(e.code), e.getMessage, Map.empty)
     case e =>

+ 1 - 1
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batch/BatchServletSpec.scala

@@ -54,7 +54,7 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
   }
 
   val livyConf = new LivyConf()
-  val batchFactory = new BatchSessionProcessFactory(SparkProcessBuilderFactory(livyConf))
+  val batchFactory = new BatchSessionProcessFactory(new SparkProcessBuilderFactory(livyConf))
   val batchManager = new SessionManager(livyConf, batchFactory)
   val servlet = new BatchSessionServlet(batchManager)
 

+ 122 - 0
apps/spark/java/livy-spark/src/main/resources/com/cloudera/hue/livy/spark/default-spark-user-configurable-options.conf

@@ -0,0 +1,122 @@
+// Application properties
+spark.app.name
+spark.driver.cores
+spark.driver.maxResultSize
+spark.driver.memory
+spark.executor.memory
+spark.extraListeners
+// spark.local.dir
+spark.logConf
+// spark.master
+
+// Runtime Environment
+spark.driver.extraClassPath
+// spark.driver.extraJavaOptions
+spark.driver.extraLibraryPath
+spark.driver.userClassPathFirst
+spark.executor.extraClassPath
+// spark.executor.extraJavaOptions
+spark.executor.extraLibraryPath
+spark.executor.logs.rolling.maxRetainedFiles
+spark.executor.logs.rolling.maxSize
+spark.executor.logs.rolling.strategy
+spark.executor.logs.rolling.time.interval
+spark.executor.userClassPathFirst
+// spark.executorEnv.[EnvironmentVariableName]
+spark.python.profile
+spark.python.profile.dump
+spark.python.worker.memory
+spark.python.worker.reuse
+spark.reducer.maxSizeInFlight
+
+// Shuffle Behavior
+spark.shuffle.blockTransferService
+spark.shuffle.compress
+spark.shuffle.consolidateFiles
+spark.shuffle.file.buffer
+spark.shuffle.io.maxRetries
+spark.shuffle.io.numConnectionsPerPeer
+spark.shuffle.io.preferDirectBufs
+spark.shuffle.io.retryWait
+spark.shuffle.manager
+spark.shuffle.memoryFraction
+spark.shuffle.service.enabled
+spark.shuffle.service.port
+spark.shuffle.sort.bypassMergeThreshold
+spark.shuffle.spill
+spark.shuffle.spill.compress
+
+// Spark UI
+spark.eventLog.compress
+spark.eventLog.dir
+spark.eventLog.enabled
+spark.ui.killEnabled
+spark.ui.port
+spark.ui.retainedJobs
+spark.ui.retainedStages
+spark.worker.ui.retainedExecutors
+spark.worker.ui.retainedDrivers
+
+// Compression and Serialization
+spark.broadcast.compress
+spark.closure.serializer
+spark.io.compression.codec
+spark.io.compression.lz4.blockSize
+spark.io.compression.snappy.blockSize
+spark.kryo.classesToRegister
+spark.kryo.referenceTracking
+spark.kryo.registrationRequired
+spark.kryo.registrator
+spark.kryoserializer.buffer.max
+spark.kryoserializer.buffer
+spark.rdd.compress
+spark.serializer
+spark.serializer.objectStreamReset
+
+// Execution Behavior
+spark.broadcast.blockSize
+spark.broadcast.factory
+spark.cleaner.ttl
+spark.executor.cores
+spark.default.parallelism
+spark.executor.heartbeatInterval
+spark.files.fetchTimeout
+spark.files.useFetchCache
+spark.files.overwrite
+spark.hadoop.cloneConf
+spark.hadoop.validateOutputSpecs
+spark.storage.memoryFraction
+spark.storage.memoryMapThreshold
+spark.storage.unrollFraction
+spark.externalBlockStore.blockManager
+spark.externalBlockStore.baseDir
+spark.externalBlockStore.url
+
+// Networking
+
+// Dynamic Allocation
+spark.dynamicAllocation.enabled
+spark.dynamicAllocation.executorIdleTimeout
+spark.dynamicAllocation.cachedExecutorIdleTimeout
+spark.dynamicAllocation.initialExecutors
+spark.dynamicAllocation.maxExecutors
+spark.dynamicAllocation.minExecutors
+spark.dynamicAllocation.schedulerBacklogTimeout
+spark.dynamicAllocation.sustainedSchedulerBacklogTimeout
+
+// Security
+
+// Encryption
+
+// Spark Streaming
+spark.streaming.backpressure.enabled
+spark.streaming.blockInterval
+spark.streaming.receiver.maxRate
+spark.streaming.receiver.writeAheadLog.enable
+spark.streaming.unpersist
+spark.streaming.kafka.maxRatePerPartition
+spark.streaming.kafka.maxRetries
+spark.streaming.ui.retainedBatches
+
+// SparkR
+spark.r.numRBackendThreads

+ 32 - 0
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/ConfigOptionNotAllowed.scala

@@ -0,0 +1,32 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. 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 com.cloudera.hue.livy.spark
+
+/**
+ * This exception is raised if this configuration option is not allowed to be set by a user.
+ *
+ * @param key The configuration key
+ * @param value The configuration value
+ *
+ */
+case class ConfigOptionNotAllowed(key: String, value: String) extends Exception {
+  override def getMessage(): String = {
+    s"Not allowed to specify the '$key' configuration variable"
+  }
+}

+ 42 - 2
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkManager.scala

@@ -1,6 +1,24 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. 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 com.cloudera.hue.livy.spark
 
-import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.{Utils, LivyConf}
 import com.cloudera.hue.livy.LivyConf.{Process, Yarn}
 import com.cloudera.hue.livy.sessions.SessionManager
 import com.cloudera.hue.livy.sessions.batch.BatchSession
@@ -9,15 +27,37 @@ import com.cloudera.hue.livy.spark.batch.{BatchSessionProcessFactory, BatchSessi
 import com.cloudera.hue.livy.spark.interactive.{InteractiveSessionProcessFactory, InteractiveSessionYarnFactory}
 import com.cloudera.hue.livy.yarn.Client
 
+import scala.io.Source
+
 object SparkManager {
   def apply(livyConf: LivyConf): SparkManager = {
-    val processFactory = SparkProcessBuilderFactory(livyConf)
+    val userConfigurableOptions = loadSparkUserConfigurableOptions()
+    val processFactory = new SparkProcessBuilderFactory(livyConf, userConfigurableOptions)
 
     livyConf.sessionKind() match {
       case Process() => new SparkProcessManager(processFactory)
       case Yarn() => new SparkYarnManager(processFactory)
     }
   }
+
+  private val SparkUserConfig = "spark-user-configurable-options.conf"
+  private val DefaultSparkUserConfig = "default-spark-user-configurable-options.conf"
+
+  private def loadSparkUserConfigurableOptions(): Set[String] = {
+    Utils.getLivyConfigFile(SparkUserConfig)
+      .map(Source.fromFile)
+      .orElse {
+        Option(getClass.getResourceAsStream(DefaultSparkUserConfig))
+          .map(Source.fromInputStream)
+      }
+      .map { source =>
+        source.getLines()
+          .map(_.trim)
+          .filter(!_.startsWith("//"))
+          .toSet
+      }
+      .getOrElse(Set())
+  }
 }
 
 trait SparkManager {

+ 30 - 22
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkProcessBuilder.scala

@@ -18,17 +18,13 @@
 
 package com.cloudera.hue.livy.spark
 
-import com.cloudera.hue.livy.spark.SparkProcessBuilder.{AbsolutePath, Path, RelativePath}
 import com.cloudera.hue.livy.{LivyConf, Logging}
 
 import scala.collection.JavaConversions._
+import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 
 object SparkProcessBuilder {
-  def apply(livyConf: LivyConf): SparkProcessBuilder = {
-    new SparkProcessBuilder(livyConf)
-  }
-
   /**
    * Represents a path that is either allowed to reference a local file, or must exist in our
    * cache directory or on hdfs.
@@ -38,7 +34,9 @@ object SparkProcessBuilder {
   case class RelativePath(path: String) extends Path
 }
 
-class SparkProcessBuilder(livyConf: LivyConf) extends Logging {
+class SparkProcessBuilder(livyConf: LivyConf, userConfigurableOptions: Set[String]) extends Logging {
+  import SparkProcessBuilder._
+
   private[this] val fsRoot = livyConf.filesystemRoot()
 
   private[this] var _executable: Path = AbsolutePath(livyConf.sparkSubmit())
@@ -49,8 +47,7 @@ class SparkProcessBuilder(livyConf: LivyConf) extends Logging {
   private[this] var _jars: ArrayBuffer[Path] = ArrayBuffer()
   private[this] var _pyFiles: ArrayBuffer[Path] = ArrayBuffer()
   private[this] var _files: ArrayBuffer[Path] = ArrayBuffer()
-  private[this] var _conf: ArrayBuffer[(String, String)] = ArrayBuffer()
-  private[this] var _driverJavaOptions: Option[String] = None
+  private[this] val _conf = mutable.HashMap[String, String]()
   private[this] var _driverClassPath: ArrayBuffer[String] = ArrayBuffer()
   private[this] var _proxyUser: Option[String] = None
 
@@ -117,13 +114,22 @@ class SparkProcessBuilder(livyConf: LivyConf) extends Logging {
     this
   }
 
-  def conf(key: String, value: String): SparkProcessBuilder = {
-    this._conf += ((key, value))
+  def conf(key: String): Option[String] = {
+    _conf.get(key)
+  }
+
+  def conf(key: String, value: String, admin: Boolean = false): SparkProcessBuilder = {
+    if (admin || userConfigurableOptions.contains(key)) {
+      this._conf(key) = value
+    } else {
+      throw new ConfigOptionNotAllowed(key, value)
+    }
+
     this
   }
 
   def conf(conf: Traversable[(String, String)]): SparkProcessBuilder = {
-    this._conf ++= conf
+    conf.foreach { case (key, value) => this.conf(key, value) }
     this
   }
 
@@ -214,19 +220,19 @@ class SparkProcessBuilder(livyConf: LivyConf) extends Logging {
   }
 
   def start(file: Path, args: Traversable[String]): SparkProcess = {
-    var args_ = ArrayBuffer(fromPath(_executable))
+    var arguments = ArrayBuffer(fromPath(_executable))
 
     def addOpt(option: String, value: Option[String]): Unit = {
       value.foreach { v =>
-        args_ += option
-        args_ += v
+        arguments += option
+        arguments += v
       }
     }
 
     def addList(option: String, values: Traversable[String]): Unit = {
       if (values.nonEmpty) {
-        args_ += option
-        args_ += values.mkString(",")
+        arguments += option
+        arguments += values.mkString(",")
       }
     }
 
@@ -237,8 +243,10 @@ class SparkProcessBuilder(livyConf: LivyConf) extends Logging {
     addList("--py-files", _pyFiles.map(fromPath))
     addList("--files", _files.map(fromPath))
     addOpt("--class", _className)
-    addList("--conf", _conf.map { case (key, value) => f"$key=$value" })
-    addOpt("--driver-java-options", _driverJavaOptions)
+    _conf.foreach { case (key, value) =>
+      arguments += "--conf"
+      arguments += f"$key=$value"
+    }
     addList("--driver-class-path", _driverClassPath)
 
     if (livyConf.getBoolean(LivyConf.IMPERSONATION_ENABLED_KEY, true)) {
@@ -248,16 +256,16 @@ class SparkProcessBuilder(livyConf: LivyConf) extends Logging {
     addOpt("--queue", _queue)
     addList("--archives", _archives.map(fromPath))
 
-    args_ += fromPath(file)
-    args_ ++= args
+    arguments += fromPath(file)
+    arguments ++= args
 
-    val argsString = args_
+    val argsString = arguments
       .map("'" + _.replace("'", "\\'") + "'")
       .mkString(" ")
 
     info(s"Running $argsString")
 
-    val pb = new ProcessBuilder(args_)
+    val pb = new ProcessBuilder(arguments)
     val env = pb.environment()
 
     for ((key, value) <- _env) {

+ 22 - 6
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkProcessBuilderFactory.scala

@@ -1,15 +1,31 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. 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 com.cloudera.hue.livy.spark
 
 import com.cloudera.hue.livy.LivyConf
 
-object SparkProcessBuilderFactory {
-  def apply(livyConf: LivyConf): SparkProcessBuilderFactory = {
-    new SparkProcessBuilderFactory(livyConf)
+class SparkProcessBuilderFactory(val livyConf: LivyConf, userConfigurableOptions: Set[String]) {
+  def this(livyConf: LivyConf) = {
+    this(livyConf, Set())
   }
-}
 
-class SparkProcessBuilderFactory(val livyConf: LivyConf) {
   def builder() = {
-    SparkProcessBuilder(livyConf)
+    new SparkProcessBuilder(livyConf, userConfigurableOptions)
   }
 }

+ 42 - 0
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/SparkUserConfigurableOptions.scala

@@ -0,0 +1,42 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. 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 com.cloudera.hue.livy.spark
+
+import java.io.File
+
+import com.cloudera.hue.livy.Utils
+
+import scala.io.Source
+
+object SparkUserConfigurableOptions {
+  def apply(): Set[String] = {
+    val file = Utils.getLivyConfDir()
+      .map(path => new File(path, "spark-user-configurable-options.conf"))
+      .get
+
+    apply(file)
+  }
+
+  def apply(configFile: File): Set[String] = {
+    Source.fromFile(configFile).getLines()
+      .map(_.trim)
+      .filter(_.startsWith("//"))
+      .toSet
+  }
+}

+ 1 - 1
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/batch/BatchSessionFactory.scala

@@ -40,7 +40,7 @@ abstract class BatchSessionFactory(factory: SparkProcessBuilderFactory) extends
 
   protected def sparkBuilder(request: CreateBatchRequest): SparkProcessBuilder = {
     val builder = factory.builder()
-
+    builder.conf(request.conf)
     request.proxyUser.foreach(builder.proxyUser)
     request.className.foreach(builder.className)
     request.jars.map(RelativePath).foreach(builder.jar)

+ 2 - 1
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/batch/CreateBatchRequest.scala

@@ -33,4 +33,5 @@ case class CreateBatchRequest(
     numExecutors: Option[Int] = None,
     archives: List[String] = List(),
     queue: Option[String] = None,
-    name: Option[String] = None)
+    name: Option[String] = None,
+    conf: Map[String, String] = Map())

+ 2 - 2
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/interactive/CreateInteractiveRequest.scala

@@ -33,5 +33,5 @@ case class CreateInteractiveRequest(
     numExecutors: Option[Int] = None,
     archives: List[String] = List(),
     queue: Option[String] = None,
-    name: Option[String] = None)
-
+    name: Option[String] = None,
+    conf: Map[String, String] = Map())

+ 10 - 3
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/interactive/InteractiveSessionFactory.scala

@@ -53,6 +53,7 @@ abstract class InteractiveSessionFactory(processFactory: SparkProcessBuilderFact
     val builder = processFactory.builder()
 
     builder.className("com.cloudera.hue.livy.repl.Main")
+    builder.conf(request.conf)
     request.archives.map(RelativePath).foreach(builder.archive)
     request.driverCores.foreach(builder.driverCores)
     request.driverMemory.foreach(builder.driverMemory)
@@ -67,12 +68,18 @@ abstract class InteractiveSessionFactory(processFactory: SparkProcessBuilderFact
     request.name.foreach(builder.name)
 
     val callbackUrl = System.getProperty("livy.server.callback-url")
-    val url = f"$callbackUrl/sessions/$id/callback"
+    val driverOptions =
+      f"-Dlivy.repl.callback-url=$callbackUrl/sessions/$id/callback " +
+      "-Dlivy.repl.port=0"
 
-    builder.driverJavaOptions(f"-Dlivy.repl.callback-url=$url -Dlivy.repl.port=0")
+    builder.conf(
+      "spark.driver.extraJavaOptions",
+      builder.conf("spark.driver.extraJavaOptions")
+        .getOrElse("") + driverOptions,
+        admin = true)
 
     request.kind match {
-      case PySpark() => builder.conf("spark.yarn.isPython", "true")
+      case PySpark() => builder.conf("spark.yarn.isPython", "true", admin = true)
       case _ =>
     }
 

+ 12 - 6
apps/spark/java/livy-spark/src/main/scala/com/cloudera/hue/livy/spark/interactive/InteractiveSessionYarnFactory.scala

@@ -27,11 +27,17 @@ import scala.concurrent.ExecutionContext
 class InteractiveSessionYarnFactory(client: Client, processFactory: SparkProcessBuilderFactory)
   extends InteractiveSessionFactory(processFactory) {
 
-   implicit def executor: ExecutionContext = ExecutionContext.global
+  implicit def executor: ExecutionContext = ExecutionContext.global
 
-   protected override def create(id: Int,
-                                 process: SparkProcess,
-                                 request: CreateInteractiveRequest): InteractiveSession = {
-     InteractiveSessionYarn(client, id, process, request)
-   }
+  protected override def create(id: Int,
+                                process: SparkProcess,
+                                request: CreateInteractiveRequest): InteractiveSession = {
+    InteractiveSessionYarn(client, id, process, request)
+  }
+
+  override def sparkBuilder(id: Int, request: CreateInteractiveRequest) = {
+    val builder = super.sparkBuilder(id, request)
+    builder.master("yarn-cluster")
+    builder
+  }
  }

+ 43 - 0
apps/spark/java/livy-spark/src/test/scala/com/cloudera/hue/livy/spark/SparkProcessBuilderSpec.scala

@@ -0,0 +1,43 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. 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 com.cloudera.hue.livy.spark
+
+import com.cloudera.hue.livy.LivyConf
+import org.scalatest.{FlatSpec, Matchers}
+
+class SparkProcessBuilderSpec
+  extends FlatSpec
+  with Matchers {
+
+  it should "not error if passed an allowed config option" in {
+    val allowed = Set("spark.app.name")
+    val builder = new SparkProcessBuilder(new LivyConf(), allowed)
+
+    builder.conf("spark.app.name", "hello")
+  }
+
+  it should "raise an exception if passed an illegal config option" in {
+    val allowed = Set("spark.app.name")
+    val builder = new SparkProcessBuilder(new LivyConf(), allowed)
+
+    intercept[ConfigOptionNotAllowed] {
+      builder.conf("spark.master", "local")
+    }
+  }
+}

+ 1 - 1
apps/spark/java/livy-spark/src/test/scala/com/cloudera/hue/livy/spark/batch/BatchProcessSpec.scala

@@ -56,7 +56,7 @@ class BatchProcessSpec
       )
 
       val livyConf = new LivyConf()
-      val builder = new BatchSessionProcessFactory(SparkProcessBuilderFactory(livyConf))
+      val builder = new BatchSessionProcessFactory(new SparkProcessBuilderFactory(livyConf))
       val batch = builder.create(0, req)
 
       Utils.waitUntil({ () => !batch.state.isActive }, Duration(10, TimeUnit.SECONDS))