Skip to content

Commit 65ef545

Browse files
author
Stephen Carman
committed
Style changes, renamed ContextConfig to GetContextConfig, added readme and application examples
1 parent 2884355 commit 65ef545

File tree

4 files changed

+18
-5
lines changed

4 files changed

+18
-5
lines changed

README.md

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -600,6 +600,14 @@ To pass settings directly to the sparkConf that do not use the "spark." prefix "
600600
}
601601
}
602602
603+
To add to the underlying Hadoop configuration in a Spark context, add the hadoop section to the context settings
604+
605+
spark.context-settings {
606+
hadoop {
607+
mapreduce.framework.name = "Foo"
608+
}
609+
}
610+
603611
For the exact context configuration parameters, see JobManagerActor docs as well as application.conf.
604612
605613
Also see the [yarn doc](doc/yarn.md) for more tips.

job-server/src/main/resources/application.conf

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -127,6 +127,11 @@ spark {
127127
passthrough {
128128
spark.driver.allowMultipleContexts = true # Ignore the Multiple context exception related with SPARK-2243
129129
}
130+
131+
#This adds configuration to the underlying Hadoop configuration in the Spark Context
132+
#hadoop {
133+
# mapreduce.framework.name = "FooFramework"
134+
#}
130135
}
131136
}
132137

job-server/src/spark.jobserver/JobManagerActor.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ object JobManagerActor {
2424
case class StartJob(appName: String, classPath: String, config: Config,
2525
subscribedEvents: Set[Class[_]])
2626
case class KillJob(jobId: String)
27-
case object ContextConfig
27+
case object GetContextConfig
2828
case object SparkContextStatus
2929

3030
// Results/Data
@@ -164,10 +164,10 @@ class JobManagerActor(contextConfig: Config) extends InstrumentedActor {
164164
}
165165
}
166166
}
167-
case ContextConfig => {
168-
if (jobContext.sparkContext == null){
167+
case GetContextConfig => {
168+
if (jobContext.sparkContext == null) {
169169
sender ! SparkContextDead
170-
}else{
170+
} else {
171171
try {
172172
val conf: SparkConf = jobContext.sparkContext.getConf
173173
val hadoopConf: Configuration = jobContext.sparkContext.hadoopConfiguration

job-server/test/spark.jobserver/LocalContextSupervisorSpec.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -111,7 +111,7 @@ class LocalContextSupervisorSpec extends TestKit(LocalContextSupervisorSpec.syst
111111
supervisor ! GetContext("c1")
112112
expectMsgPF(5 seconds, "I can't find that context :'-(") {
113113
case (contextActor: ActorRef, resultActor: ActorRef) => {
114-
contextActor ! ContextConfig
114+
contextActor ! GetContextConfig
115115
val cc = expectMsgClass(classOf[ContextConfig])
116116
cc.contextName shouldBe "c1"
117117
cc.contextConfig.get("spark.ui.enabled") shouldBe "false"

0 commit comments

Comments
 (0)