@@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster
20
20
import java .util .concurrent .atomic .AtomicInteger
21
21
22
22
import com .amazonaws .ClientConfiguration
23
+ import com .amazonaws .auth .{AWSStaticCredentialsProvider , BasicAWSCredentials }
23
24
import com .amazonaws .services .lambda .AWSLambdaClientBuilder
24
25
import com .amazonaws .services .lambda .invoke .{LambdaFunction , LambdaInvokerFactory }
25
26
import com .amazonaws .services .lambda .model .InvokeRequest
@@ -126,8 +127,8 @@ private[spark] class LambdaSchedulerBackend(
126
127
extends CoarseGrainedSchedulerBackend (scheduler, sc.env.rpcEnv)
127
128
with Logging {
128
129
129
- val lambdaFunctionName = sc.conf.get(" spark.qubole. lambda.function.name" , " get_spark_from_s3" )
130
- val s3SparkVersion = sc.conf.get(" spark.qubole. lambda.spark.software.version" , " LATEST" )
130
+ val lambdaFunctionName = sc.conf.get(" spark.lambda.function.name" , " get_spark_from_s3" )
131
+ val s3SparkVersion = sc.conf.get(" spark.lambda.spark.software.version" , " LATEST" )
131
132
var numExecutorsExpected = 0
132
133
var numExecutorsRegistered = new AtomicInteger (0 )
133
134
var executorId = new AtomicInteger (0 )
@@ -137,7 +138,7 @@ private[spark] class LambdaSchedulerBackend(
137
138
// Set of executorIds which are currently alive
138
139
val liveExecutors = new HashSet [String ]
139
140
140
- var lambdaContainerMemoryBytes : Int = 0
141
+ var lambdaContainerMemory : Int = 0
141
142
var lambdaContainerTimeoutSecs : Int = 0
142
143
143
144
val clientConfig = new ClientConfiguration ()
@@ -146,22 +147,32 @@ private[spark] class LambdaSchedulerBackend(
146
147
clientConfig.setRequestTimeout(345680 )
147
148
clientConfig.setSocketTimeout(345681 )
148
149
150
+ val lambdaBucket = Option (sc.getConf.get(" spark.lambda.s3.bucket" ))
151
+
152
+ if (! lambdaBucket.isDefined) {
153
+ throw new Exception (s " spark.lambda.s3.bucket should " +
154
+ s " have a valid S3 bucket name having Spark binaries " )
155
+ }
156
+
157
+ val lambdaClient = AWSLambdaClientBuilder
158
+ .standard()
159
+ .withClientConfiguration(clientConfig)
160
+ .build()
161
+
149
162
final val lambdaExecutorService : LambdaExecutorService =
150
163
LambdaInvokerFactory .builder()
151
- .lambdaClient(AWSLambdaClientBuilder .standard().withClientConfiguration(clientConfig).build() )
164
+ .lambdaClient(lambdaClient )
152
165
.build(classOf [LambdaExecutorService ])
153
166
logInfo(s " Created LambdaExecutorService: $lambdaExecutorService" )
154
167
155
- val maxConcurrentRequests = sc.conf.getInt(" spark.qubole. lambda.concurrent.requests.max" , 100 )
168
+ val maxConcurrentRequests = sc.conf.getInt(" spark.lambda.concurrent.requests.max" , 100 )
156
169
val limiter = RateLimiter .create(maxConcurrentRequests)
157
170
158
171
override def start () {
159
172
super .start()
160
173
logInfo(" start" )
161
174
numExecutorsExpected = getInitialTargetExecutorNumber(conf)
162
175
163
- val lambdaClient = AWSLambdaClientBuilder .defaultClient()
164
-
165
176
val request = new com.amazonaws.services.lambda.model.GetFunctionRequest
166
177
request.setFunctionName(lambdaFunctionName)
167
178
val result = lambdaClient.getFunction(request)
@@ -170,7 +181,7 @@ private[spark] class LambdaSchedulerBackend(
170
181
val request2 = new com.amazonaws.services.lambda.model.GetFunctionConfigurationRequest
171
182
request2.setFunctionName(lambdaFunctionName)
172
183
val result2 = lambdaClient.getFunctionConfiguration(request2)
173
- lambdaContainerMemoryBytes = result2.getMemorySize * 1024 * 1024
184
+ lambdaContainerMemory = result2.getMemorySize
174
185
lambdaContainerTimeoutSecs = result2.getTimeout
175
186
logDebug(s " LAMBDA: 16001: Function configuration: ${result2.toString}" )
176
187
@@ -191,97 +202,11 @@ private[spark] class LambdaSchedulerBackend(
191
202
192
203
override def applicationId (): String = {
193
204
val appId = super .applicationId()
194
- logInfo (s " applicationId: $appId" )
205
+ logDebug (s " applicationId: $appId" )
195
206
return appId
196
207
}
197
208
198
209
private def launchExecutorsOnLambda (newExecutorsNeeded : Int ) : Future [Boolean ] = {
199
- Future {
200
- // TODO: Can we launch in parallel?
201
- // TODO: Can we track each thread separately and audit
202
- (1 to newExecutorsNeeded).foreach { x =>
203
- val request = new Request
204
- request.setSparkS3Bucket(" bharatb" )
205
- request.setSparkS3Key(s " lambda/spark-small- ${s3SparkVersion}.zip " )
206
- request.setHadoop2S3Bucket(" bharatb" )
207
- request.setHadoop2S3Key(s " lambda/hadoop2-small- ${s3SparkVersion}.zip " )
208
- request.setHive12S3Bucket(" bharatb" )
209
- request.setHive12S3Key(s " lambda/hive1.2-small- ${s3SparkVersion}.zip " )
210
- val hostname = sc.env.rpcEnv.address.host
211
- val port = sc.env.rpcEnv.address.port.toString
212
- request.setSparkDriverHostname(hostname)
213
- request.setSparkDriverPort(port)
214
-
215
- val classpathSeq = Seq (" spark/assembly/target/scala-2.11/jars/*" ,
216
- " spark/conf" ,
217
- " hadoop2/share/hadoop/*" ,
218
- " hadoop2/share/hadoop/common/lib/*" ,
219
- " hadoop2/share/hadoop/common/*" ,
220
- " hadoop2/share/hadoop/hdfs" ,
221
- " hadoop2/share/hadoop/hdfs/lib/*" ,
222
- " hadoop2/share/hadoop/hdfs/*" ,
223
- " hadoop2/share/hadoop/yarn/lib/*" ,
224
- " hadoop2/share/hadoop/yarn/*" ,
225
- " hadoop2/share/hadoop/mapreduce/*" ,
226
- " hadoop2/share/hadoop/tools/lib/*" ,
227
- " hadoop2/share/hadoop/tools/*" ,
228
- " hadoop2/share/hadoop/qubole/lib/*" ,
229
- " hadoop2/share/hadoop/qubole/*" ,
230
- " hadoop2/etc/hadoop/*" ,
231
- " hive1.2/lib/*"
232
- )
233
- val classpaths = classpathSeq.map(x => s " /tmp/lambda/ $x" ).mkString(" :" )
234
- val currentExecutorId = executorId.addAndGet(1 )
235
- val containerId = applicationId() + " _%08d" .format(currentExecutorId)
236
- request.setSparkCommandLine(
237
- s " java -cp ${classpaths} " +
238
- " -server -Xmx1400m " +
239
- " -Djava.net.preferIPv4Stack=true " +
240
- s " -Dspark.driver.port= ${port} " +
241
- // "-Dspark.blockManager.port=12345 " +
242
- " -Dspark.dynamicAllocation.enabled=true " +
243
- " -Dspark.shuffle.service.enabled=false " +
244
- " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
245
- s " --driver-url spark://CoarseGrainedScheduler@ ${hostname}: ${port} " +
246
- s " --executor-id ${currentExecutorId} " +
247
- " --hostname LAMBDA " +
248
- " --cores 1 " +
249
- s " --app-id ${applicationId()} " +
250
- s " --container-id ${containerId} " +
251
- s " --container-size ${lambdaContainerMemoryBytes} " +
252
- " --user-class-path file:/tmp/lambda/* "
253
- )
254
-
255
- val lambdaRequesterThread = new Thread () {
256
- override def run () {
257
- val executorId = currentExecutorId.toString
258
- logDebug(s " LAMBDA: 9002: Invoking lambda for $executorId: $request" )
259
- numLambdaCallsPending.addAndGet(1 )
260
- try {
261
- val response = lambdaExecutorService.runExecutor(request)
262
- logDebug(s " LAMBDA: 9003: Returned from lambda $executorId: $response" )
263
- } catch {
264
- case t : Throwable => logError(s " Exception in Lambda invocation: $t" )
265
- } finally {
266
- logDebug(s " LAMBDA: 9003: Returned from lambda $executorId: finally block " )
267
- numLambdaCallsPending.addAndGet(- 1 )
268
- pendingLambdaRequests.remove(executorId)
269
- }
270
- }
271
- }
272
- lambdaRequesterThread.setDaemon(true )
273
- lambdaRequesterThread.setName(s " Lambda Requester Thread for $currentExecutorId" )
274
- pendingLambdaRequests(currentExecutorId.toString) = lambdaRequesterThread
275
- logDebug(s " LAMBDA: 9004: starting lambda requester thread for $currentExecutorId" )
276
- lambdaRequesterThread.start()
277
-
278
- logDebug(s " LAMBDA: 9005: returning from launchExecutorsOnLambda for $currentExecutorId" )
279
- }
280
- true // TODO: Return true/false properly
281
- }
282
- }
283
-
284
- private def launchExecutorsOnLambda2 (newExecutorsNeeded : Int ) : Future [Boolean ] = {
285
210
Future {
286
211
// TODO: Can we launch in parallel?
287
212
// TODO: Can we track each thread separately and audit
@@ -311,29 +236,30 @@ private[spark] class LambdaSchedulerBackend(
311
236
val containerId = applicationId() + " _%08d" .format(currentExecutorId)
312
237
313
238
val javaPartialCommandLine = s " java -cp ${classpaths} " +
314
- " -server -Xmx1400m " +
239
+ s " -server -Xmx ${lambdaContainerMemory} m " +
315
240
" -Djava.net.preferIPv4Stack=true " +
316
241
s " -Dspark.driver.port= ${port} " +
317
- // "-Dspark.blockManager.port=12345 " +
318
242
" -Dspark.dynamicAllocation.enabled=true " +
319
243
" -Dspark.shuffle.service.enabled=false "
244
+
320
245
val executorPartialCommandLine = " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
321
246
s " --driver-url spark://CoarseGrainedScheduler@ ${hostname}: ${port} " +
322
247
s " --executor-id ${currentExecutorId} " +
323
248
" --hostname LAMBDA " +
324
249
" --cores 1 " +
325
250
s " --app-id ${applicationId()} " +
326
251
s " --container-id ${containerId} " +
327
- s " --container-size ${lambdaContainerMemoryBytes } " +
252
+ s " --container-size ${lambdaContainerMemory } " +
328
253
" --user-class-path file:/tmp/lambda/* "
254
+
329
255
val commandLine = javaPartialCommandLine + executorPartialCommandLine
330
256
331
257
val request = new LambdaRequestPayload (
332
- sparkS3Bucket = " bharatb " ,
258
+ sparkS3Bucket = lambdaBucket.get ,
333
259
sparkS3Key = s " lambda/spark-small- ${s3SparkVersion}.zip " ,
334
- hadoop2S3Bucket = " bharatb " ,
260
+ hadoop2S3Bucket = lambdaBucket.get ,
335
261
hadoop2S3Key = s " lambda/hadoop2-small- ${s3SparkVersion}.zip " ,
336
- hive12S3Bucket = " bharatb " ,
262
+ hive12S3Bucket = lambdaBucket.get ,
337
263
hive12S3Key = s " lambda/hive1.2-small- ${s3SparkVersion}.zip " ,
338
264
sparkDriverHostname = hostname,
339
265
sparkDriverPort = port,
@@ -349,9 +275,7 @@ private[spark] class LambdaSchedulerBackend(
349
275
limiter.acquire()
350
276
logDebug(s " LAMBDA: 9050.1: LambdaRequesterThread started $executorId" )
351
277
numLambdaCallsPending.addAndGet(1 )
352
- // TODO: Can we reuse the same client across calls?
353
- val lambdaClient = AWSLambdaClientBuilder .standard()
354
- .withClientConfiguration(clientConfig).build()
278
+
355
279
val invokeRequest = new InvokeRequest
356
280
try {
357
281
invokeRequest.setFunctionName(lambdaFunctionName)
@@ -396,11 +320,11 @@ private[spark] class LambdaSchedulerBackend(
396
320
if (newExecutorsNeeded <= 0 ) {
397
321
return Future { true }
398
322
}
399
- return launchExecutorsOnLambda2 (newExecutorsNeeded)
323
+ return launchExecutorsOnLambda (newExecutorsNeeded)
400
324
}
401
325
402
326
override def doKillExecutors (executorIds : Seq [String ]): Future [Boolean ] = {
403
- // TODO: Fill this function
327
+ // TODO: Right now not implemented
404
328
logDebug(s " LAMBDA: 10200: doKillExecutors: $executorIds" )
405
329
Future {
406
330
executorIds.foreach { x =>
0 commit comments