/
JobManagerMessages.scala
590 lines (503 loc) · 18.8 KB
/
JobManagerMessages.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
/*
* 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.flink.runtime.messages
import java.net.URL
import java.util.UUID
import akka.actor.ActorRef
import org.apache.flink.api.common.JobID
import org.apache.flink.runtime.akka.ListeningBehaviour
import org.apache.flink.runtime.blob.BlobKey
import org.apache.flink.runtime.client.{JobStatusMessage, SerializedJobExecutionResult}
import org.apache.flink.runtime.executiongraph.{AccessExecutionGraph, ExecutionAttemptID, ExecutionGraph}
import org.apache.flink.runtime.instance.{Instance, InstanceID}
import org.apache.flink.runtime.io.network.partition.ResultPartitionID
import org.apache.flink.runtime.jobgraph.{IntermediateDataSetID, JobGraph, JobStatus, JobVertexID}
import org.apache.flink.runtime.jobmanager.SubmittedJobGraph
import org.apache.flink.runtime.messages.checkpoint.AbstractCheckpointMessage
import org.apache.flink.runtime.util.SerializedThrowable
import scala.collection.JavaConverters._
/**
* The job manager specific actor messages
*/
object JobManagerMessages {
/** Wrapper class for leader session messages. Leader session messages implement the
* [[RequiresLeaderSessionID]] interface and have to be wrapped in a [[LeaderSessionMessage]],
* which also contains the current leader session ID.
*
* @param leaderSessionID Current leader session ID or null, if no leader session ID was set
* @param message [[RequiresLeaderSessionID]] message to be wrapped in a [[LeaderSessionMessage]]
*/
case class LeaderSessionMessage(leaderSessionID: UUID, message: Any)
/**
* Submits a job to the job manager. Depending on the [[listeningBehaviour]],
* the sender registers for different messages. If [[ListeningBehaviour.DETACHED]], then
* it will only be informed whether the submission was successful or not. If
* [[ListeningBehaviour.EXECUTION_RESULT]], then it will additionally receive the execution
* result. If [[ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES]], then it will additionally
* receive the job status change notifications.
*
* The submission result will be sent back to the sender as a success message.
*
* @param jobGraph The job to be submitted to the JobManager
* @param listeningBehaviour Specifies to what the sender wants to listen (detached, execution
* result, execution result and state changes)
*/
case class SubmitJob(
jobGraph: JobGraph,
listeningBehaviour: ListeningBehaviour)
extends RequiresLeaderSessionID
/**
* Registers the sender of the message as the client for the provided job identifier.
* This message is acknowledged by the JobManager with [[RegisterJobClientSuccess]]
* or [[JobNotFound]] if the job was not running.
* @param jobID The job id of the job
* @param listeningBehaviour The types of updates which will be sent to the sender
* after registration
*/
case class RegisterJobClient(
jobID: JobID,
listeningBehaviour: ListeningBehaviour)
extends RequiresLeaderSessionID
/**
* Triggers the recovery of the job with the given ID.
*
* @param jobId ID of the job to recover
*/
case class RecoverJob(jobId: JobID) extends RequiresLeaderSessionID
/**
* Triggers the submission of the recovered job
*
* @param submittedJobGraph Contains the submitted JobGraph and the associated JobInfo
*/
case class RecoverSubmittedJob(submittedJobGraph: SubmittedJobGraph)
extends RequiresLeaderSessionID
/**
* Triggers recovery of all available jobs.
*/
case object RecoverAllJobs extends RequiresLeaderSessionID
/**
* Cancels a job with the given [[jobID]] at the JobManager. The result of the cancellation is
* sent back to the sender as a [[CancellationResponse]] message.
*
* @param jobID
*/
case class CancelJob(jobID: JobID) extends RequiresLeaderSessionID
/**
* Cancels the job with the given [[jobID]] at the JobManager. Before cancellation a savepoint
* is triggered without any other checkpoints in between. The result of the cancellation is
* the path of the triggered savepoint on success or an exception.
*
* @param jobID ID of the job to cancel
* @param savepointDirectory Optional target directory for the savepoint.
* If no target directory is specified here, the
* cluster default is used.
*/
case class CancelJobWithSavepoint(
jobID: JobID,
savepointDirectory: String = null)
extends RequiresLeaderSessionID
/**
* Stops a (streaming) job with the given [[jobID]] at the JobManager. The result of
* stopping is sent back to the sender as a [[StoppingResponse]] message.
*
* @param jobID
*/
case class StopJob(jobID: JobID) extends RequiresLeaderSessionID
/**
* Requesting next input split for the
* [[org.apache.flink.runtime.executiongraph.ExecutionJobVertex]]
* of the job specified by [[jobID]]. The next input split is sent back to the sender as a
* [[NextInputSplit]] message.
*
* @param jobID
* @param vertexID
*/
case class RequestNextInputSplit(
jobID: JobID,
vertexID: JobVertexID,
executionAttempt: ExecutionAttemptID)
extends RequiresLeaderSessionID
/**
* Contains the next input split for a task. This message is a response to
* [[org.apache.flink.runtime.messages.JobManagerMessages.RequestNextInputSplit]].
*
* @param splitData
*/
case class NextInputSplit(splitData: Array[Byte])
/**
* Requests the execution state of the execution producing a result partition.
*
* @param jobId ID of the job the partition belongs to.
* @param intermediateDataSetId ID of the parent intermediate data set.
* @param resultPartitionId ID of the result partition to check. This
* identifies the producing execution and
* partition.
*/
case class RequestPartitionProducerState(
jobId: JobID,
intermediateDataSetId: IntermediateDataSetID,
resultPartitionId: ResultPartitionID)
extends RequiresLeaderSessionID
/**
* Notifies the [[org.apache.flink.runtime.jobmanager.JobManager]] about available data for a
* produced partition.
* <p>
* There is a call to this method for each
* [[org.apache.flink.runtime.executiongraph.ExecutionVertex]] instance once per produced
* [[org.apache.flink.runtime.io.network.partition.ResultPartition]] instance,
* either when first producing data (for pipelined executions) or when all data has been produced
* (for staged executions).
* <p>
* The [[org.apache.flink.runtime.jobmanager.JobManager]] then can decide when to schedule the
* partition consumers of the given session.
*
* @see [[org.apache.flink.runtime.io.network.partition.ResultPartition]]
*/
case class ScheduleOrUpdateConsumers(jobId: JobID, partitionId: ResultPartitionID)
extends RequiresLeaderSessionID
/**
* Requests the current [[JobStatus]] of the job identified by [[jobID]]. This message triggers
* as response a [[JobStatusResponse]] message.
*
* @param jobID
*/
case class RequestJobStatus(jobID: JobID)
sealed trait JobStatusResponse {
def jobID: JobID
}
/**
* Denotes the current [[JobStatus]] of the job with [[jobID]].
*
* @param jobID
* @param status
*/
case class CurrentJobStatus(jobID: JobID, status: JobStatus) extends JobStatusResponse
/**
* Requests the number of currently registered task manager at the job manager. The result is
* sent back to the sender as an [[Int]].
*/
case object RequestNumberRegisteredTaskManager
/**
* Requests the maximum number of slots available to the job manager. The result is sent back
* to the sender as an [[Int]].
*/
case object RequestTotalNumberOfSlots
/**
* Requests all entities necessary for reconstructing a job class loader
* May respond with [[ClassloadingProps]] or [[JobNotFound]]
* @param jobId The job id of the registered job
*/
case class RequestClassloadingProps(jobId: JobID)
/**
* Response to [[RequestClassloadingProps]]
* @param blobManagerPort The port of the blobManager
* @param requiredJarFiles The blob keys of the required jar files
* @param requiredClasspaths The urls of the required classpaths
*/
case class ClassloadingProps(blobManagerPort: Integer,
requiredJarFiles: java.util.Collection[BlobKey],
requiredClasspaths: java.util.Collection[URL])
/**
* Requests the port of the blob manager from the job manager. The result is sent back to the
* sender as an [[Int]].
*/
case object RequestBlobManagerPort
/** Requests the current leader session ID of the job manager. The result is sent back to the
* sender as an [[ResponseLeaderSessionID]]
*/
case object RequestLeaderSessionID
/** Response to the [[RequestLeaderSessionID]] message.
*
* @param leaderSessionID
*/
case class ResponseLeaderSessionID(leaderSessionID: UUID)
/**
* Denotes a successful job submission.
* @param jobId Ths job's ID.
*/
case class JobSubmitSuccess(jobId: JobID)
/**
* Denotes a successful registration of a JobClientActor for a running job
* @param jobId The job id of the registered job
*/
case class RegisterJobClientSuccess(jobId: JobID)
/**
* Denotes messages which contain the result of a completed job execution
*/
sealed trait JobResultMessage
/**
* Denotes a successful job execution.
* @param result The result of the job execution, in serialized form.
*/
case class JobResultSuccess(result: SerializedJobExecutionResult) extends JobResultMessage
/**
* Denotes an unsuccessful job execution.
* @param cause The exception that caused the job to fail, in serialized form.
*/
case class JobResultFailure(cause: SerializedThrowable) extends JobResultMessage
sealed trait CancellationResponse{
def jobID: JobID
}
/**
* Denotes a successful job cancellation
* @param jobID
*/
case class CancellationSuccess(
jobID: JobID,
savepointPath: String = null) extends CancellationResponse
/**
* Denotes a failed job cancellation
* @param jobID
* @param cause
*/
case class CancellationFailure(jobID: JobID, cause: Throwable) extends CancellationResponse
sealed trait StoppingResponse {
def jobID: JobID
}
/**
* Denotes a successful (streaming) job stopping
* @param jobID
*/
case class StoppingSuccess(jobID: JobID) extends StoppingResponse
/**
* Denotes a failed (streaming) job stopping
* @param jobID
* @param cause
*/
case class StoppingFailure(jobID: JobID, cause: Throwable) extends StoppingResponse
/**
* Requests all currently running jobs from the job manager. This message triggers a
* [[RunningJobs]] response.
*/
case object RequestRunningJobs
/**
* This message is the response to the [[RequestRunningJobs]] message. It contains all
* execution graphs of the currently running jobs.
*/
case class RunningJobs(runningJobs: Iterable[ExecutionGraph]) {
def this() = this(Seq())
def asJavaIterable: java.lang.Iterable[ExecutionGraph] = {
runningJobs.asJava
}
}
/**
* Requests the status of all currently running jobs from the job manager.
* This message triggers a [[RunningJobsStatus]] response.
*/
case object RequestRunningJobsStatus
case class RunningJobsStatus(runningJobs: Iterable[JobStatusMessage]) {
def this() = this(Seq())
def getStatusMessages(): java.util.List[JobStatusMessage] = {
new java.util.ArrayList[JobStatusMessage](runningJobs.asJavaCollection)
}
}
/**
* Requests the execution graph of a specific job identified by [[jobID]].
* The result is sent back to the sender as a [[JobResponse]].
*/
case class RequestJob(jobID: JobID)
sealed trait JobResponse{
def jobID: JobID
}
/**
* Contains the [[executionGraph]] of a job with [[jobID]]. This is the response to
* [[RequestJob]] if the job runs or is archived.
*
* @param jobID
* @param executionGraph
*/
case class JobFound(jobID: JobID, executionGraph: AccessExecutionGraph) extends JobResponse
/**
* Denotes that there is no job with [[jobID]] retrievable. This message can be the response of
* [[RequestJob]], [[RequestJobStatus]] or [[RegisterJobClient]].
*
* @param jobID
*/
case class JobNotFound(jobID: JobID) extends JobResponse with JobStatusResponse
/** Triggers the removal of the job with the given job ID
*
* @param jobID
* @param removeJobFromStateBackend true if the job has properly finished
*/
case class RemoveJob(jobID: JobID, removeJobFromStateBackend: Boolean = true)
extends RequiresLeaderSessionID
/**
* Removes the job belonging to the job identifier from the job manager and archives it.
* @param jobID The job identifier
*/
case class RemoveCachedJob(jobID: JobID)
/**
* Requests the instances of all registered task managers.
*/
case object RequestRegisteredTaskManagers
/**
* Contains the [[Instance]] objects of all registered task managers. It is the response to the
* message [[RequestRegisteredTaskManagers]].
*
* @param taskManagers
*/
case class RegisteredTaskManagers(taskManagers: Iterable[Instance]){
def asJavaIterable: java.lang.Iterable[Instance] = {
import scala.collection.JavaConverters._
taskManagers.asJava
}
def asJavaCollection: java.util.Collection[Instance] = {
import scala.collection.JavaConverters._
taskManagers.asJavaCollection
}
}
/**
* Requests the [[Instance]] object of the task manager with the given instance ID
*
* @param instanceID Instance ID of the task manager
*/
case class RequestTaskManagerInstance(instanceID: InstanceID)
/**
* Returns the [[Instance]] object of the requested task manager. This is in response to
* [[RequestTaskManagerInstance]]
*/
case class TaskManagerInstance(instance: Option[Instance])
/**
* Requests stack trace messages of the task manager
*
* @param instanceID Instance ID of the task manager
*/
case class RequestStackTrace(instanceID: InstanceID)
/**
* Requests the current state of the job manager
*/
case object RequestJobManagerStatus
/**
* Response to RequestJobManagerStatus
*/
sealed trait JobManagerStatus
case object JobManagerStatusAlive extends JobManagerStatus
/** Grants leadership to the receiver. The message contains the new leader session id.
*
* @param leaderSessionID
*/
case class GrantLeadership(leaderSessionID: Option[UUID])
/** Revokes leadership of the receiver.
*/
case object RevokeLeadership
/** Requests the ActorRef of the archiver */
case object RequestArchive
/** Response containing the ActorRef of the archiver */
case class ResponseArchive(actor: ActorRef)
/** Request for the [[org.apache.flink.runtime.webmonitor.WebMonitor]] port. */
case object RequestWebMonitorPort
/**
* Response containing the [[org.apache.flink.runtime.webmonitor.WebMonitor]] port.
*
* -1 indicates that there is no web monitor running.
*/
case class ResponseWebMonitorPort(port: Integer)
/**
* Triggers a savepoint for the specified job.
*
* This is not a subtype of [[AbstractCheckpointMessage]], because it is a
* control-flow message, which is *not* part of the checkpointing mechanism
* of triggering and acknowledging checkpoints.
*
* @param jobId The JobID of the job to trigger the savepoint for.
* @param savepointDirectory Optional target directory
*/
case class TriggerSavepoint(
jobId: JobID,
savepointDirectory : Option[String] = Option.empty) extends RequiresLeaderSessionID
/**
* Response after a successful savepoint trigger containing the savepoint path.
*
* @param jobId The job ID for which the savepoint was triggered.
* @param savepointPath The path of the savepoint.
*/
case class TriggerSavepointSuccess(
jobId: JobID,
checkpointId: Long,
savepointPath: String,
triggerTime: Long
)
/**
* Response after a failed savepoint trigger containing the failure cause.
*
* @param jobId The job ID for which the savepoint was triggered.
* @param cause The cause of the failure.
*/
case class TriggerSavepointFailure(jobId: JobID, cause: Throwable)
/**
* Disposes a savepoint.
*
* @param savepointPath The path of the savepoint to dispose.
*/
case class DisposeSavepoint(
savepointPath: String)
extends RequiresLeaderSessionID
/** Response after a successful savepoint dispose. */
case object DisposeSavepointSuccess
/**
* Response after a failed savepoint dispose containing the failure cause.
*
* @param cause The cause of the failure.
*/
case class DisposeSavepointFailure(cause: Throwable)
// --------------------------------------------------------------------------
// Utility methods to allow simpler case object access from Java
// --------------------------------------------------------------------------
def getRequestJobStatus(jobId : JobID) : AnyRef = {
RequestJobStatus(jobId)
}
def getRequestNumberRegisteredTaskManager : AnyRef = {
RequestNumberRegisteredTaskManager
}
def getRequestTotalNumberOfSlots : AnyRef = {
RequestTotalNumberOfSlots
}
def getRequestBlobManagerPort : AnyRef = {
RequestBlobManagerPort
}
def getRequestRunningJobs : AnyRef = {
RequestRunningJobs
}
def getRequestRunningJobsStatus : AnyRef = {
RequestRunningJobsStatus
}
def getRequestRegisteredTaskManagers : AnyRef = {
RequestRegisteredTaskManagers
}
def getRequestJobManagerStatus : AnyRef = {
RequestJobManagerStatus
}
def getJobManagerStatusAlive : AnyRef = {
JobManagerStatusAlive
}
def getRequestLeaderSessionID: AnyRef = {
RequestLeaderSessionID
}
def getRequestArchive: AnyRef = {
RequestArchive
}
def getRecoverAllJobs: AnyRef = {
RecoverAllJobs
}
def getRequestWebMonitorPort: AnyRef = {
RequestWebMonitorPort
}
def getDisposeSavepointSuccess: AnyRef = {
DisposeSavepointSuccess
}
}