-
Notifications
You must be signed in to change notification settings - Fork 13k
/
ApplicationClient.scala
168 lines (139 loc) · 5.54 KB
/
ApplicationClient.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
/*
* 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.yarn
import java.net.InetSocketAddress
import akka.actor._
import akka.pattern.ask
import org.apache.flink.configuration.Configuration
import org.apache.flink.runtime.{ActorSynchronousLogging, ActorLogMessages}
import org.apache.flink.runtime.akka.AkkaUtils
import org.apache.flink.runtime.jobmanager.JobManager
import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus
import org.apache.flink.yarn.Messages._
import scala.collection.mutable
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.{Failure, Success}
class ApplicationClient(flinkConfig: Configuration)
extends Actor
with ActorLogMessages
with ActorSynchronousLogging {
import context._
val INITIAL_POLLING_DELAY = 0 seconds
val WAIT_FOR_YARN_INTERVAL = 2 seconds
val POLLING_INTERVAL = 3 seconds
var yarnJobManager: Option[ActorRef] = None
var pollingTimer: Option[Cancellable] = None
implicit var timeout: FiniteDuration = 0 seconds
var running = false
var messagesQueue : mutable.Queue[YarnMessage] = mutable.Queue[YarnMessage]()
var latestClusterStatus : Option[FlinkYarnClusterStatus] = None
var stopMessageReceiver : Option[ActorRef] = None
override def preStart(): Unit = {
super.preStart()
timeout = AkkaUtils.getTimeout(flinkConfig)
}
override def postStop(): Unit = {
log.info("Stopped Application client.")
pollingTimer foreach {
_.cancel()
}
pollingTimer = None
// Terminate the whole actor system because there is only the application client running
context.system.shutdown()
}
override def receiveWithLogMessages: Receive = {
// ----------------------------- Registration -> Status updates -> shutdown ----------------
case LocalRegisterClient(address: InetSocketAddress) =>
val jmAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(address)
val jobManagerFuture = AkkaUtils.getReference(jmAkkaUrl, system, timeout)
jobManagerFuture.onComplete {
case Success(jm) => self ! JobManagerActorRef(jm)
case Failure(t) =>
log.error("Registration at JobManager/ApplicationMaster failed. Shutting " +
"ApplicationClient down.", t)
// we could not connect to the job manager --> poison ourselves
self ! PoisonPill
}
case JobManagerActorRef(jm) =>
yarnJobManager = Some(jm)
// the message came from the FlinkYarnCluster. We send the message to the JobManager.
// it is important not to forward the message because the JobManager is storing the
// sender as the Application Client (this class).
(jm ? RegisterClient(self))(timeout).onFailure{
case t: Throwable =>
log.error("Could not register at the job manager.", t)
self ! PoisonPill
}
// schedule a periodic status report from the JobManager
// request the number of task managers and slots from the job manager
pollingTimer = Some(context.system.scheduler.schedule(INITIAL_POLLING_DELAY,
WAIT_FOR_YARN_INTERVAL, jm, PollYarnClusterStatus))
case LocalUnregisterClient =>
// unregister client from AM
yarnJobManager foreach {
_ ! UnregisterClient
}
// poison ourselves
self ! PoisonPill
case msg: StopYarnSession =>
log.info("Sending StopYarnSession request to ApplicationMaster.")
stopMessageReceiver = Some(sender)
yarnJobManager foreach {
_ forward msg
}
case JobManagerStopped =>
log.info("Remote JobManager has been stopped successfully. " +
"Stopping local application client")
stopMessageReceiver foreach {
_ ! JobManagerStopped
}
// poison ourselves
self ! PoisonPill
// handle the responses from the PollYarnClusterStatus messages to the yarn job mgr
case status: FlinkYarnClusterStatus =>
latestClusterStatus = Some(status)
// locally get cluster status
case LocalGetYarnClusterStatus =>
sender() ! latestClusterStatus
// Forward message to Application Master
case msg: StopAMAfterJob =>
yarnJobManager foreach {
_ forward msg
}
// ----------------- handle messages from the cluster -------------------
// receive remote messages
case msg: YarnMessage =>
log.debug(s"Received new YarnMessage $msg. Now ${messagesQueue.size} messages in queue")
messagesQueue.enqueue(msg)
// locally forward messages
case LocalGetYarnMessage =>
if(messagesQueue.size > 0) {
sender() ! Option(messagesQueue.dequeue)
} else {
sender() ! None
}
}
/**
* Handle unmatched messages with an exception.
*/
override def unhandled(message: Any): Unit = {
// let the actor crash
throw new RuntimeException("Received unknown message " + message)
}
}