-
Notifications
You must be signed in to change notification settings - Fork 76
/
JobExecutorManager.java
486 lines (414 loc) · 18.6 KB
/
JobExecutorManager.java
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
package azkaban.jobs;
import java.io.File;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.UUID;
import java.util.Vector;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import azkaban.app.JobDescriptor;
import org.apache.log4j.Logger;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.format.PeriodFormat;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
import com.google.common.collect.Multimaps;
import azkaban.app.AppCommon;
import azkaban.app.JobManager;
import azkaban.app.Mailman;
import azkaban.common.utils.Props;
import azkaban.common.utils.Utils;
import azkaban.flow.ExecutableFlow;
import azkaban.flow.FlowCallback;
import azkaban.flow.FlowExecutionHolder;
import azkaban.flow.FlowManager;
import azkaban.monitor.MonitorImpl;
import azkaban.monitor.MonitorInterface.WorkflowState;
import azkaban.monitor.MonitorInternalInterface.WorkflowAction;
import azkaban.util.process.ProcessFailureException;
public class JobExecutorManager {
private static Logger logger = Logger.getLogger(JobExecutorManager.class);
private final Mailman mailman;
private final JobManager jobManager;
private final String jobSuccessEmail;
private final String jobFailureEmail;
private Properties runtimeProps = null;
private final FlowManager allKnownFlows;
private final ThreadPoolExecutor executor;
private final Map<String, ExecutingJobAndInstance> executing;
private final Multimap<String, JobExecution> completed;
@SuppressWarnings("unchecked")
public JobExecutorManager(
FlowManager allKnownFlows,
JobManager jobManager,
Mailman mailman,
String jobSuccessEmail,
String jobFailureEmail,
int maxThreads
) {
this.jobManager = jobManager;
this.mailman = mailman;
this.jobSuccessEmail = jobSuccessEmail;
this.jobFailureEmail = jobFailureEmail;
this.allKnownFlows = allKnownFlows;
Multimap<String, JobExecution> typedMultiMap = HashMultimap.create();
this.completed = Multimaps.synchronizedMultimap(typedMultiMap);
this.executing = new ConcurrentHashMap<String, ExecutingJobAndInstance>();
this.executor = new ThreadPoolExecutor(0, maxThreads, 10, TimeUnit.SECONDS, new LinkedBlockingQueue(), new ExecutorThreadFactory());
}
/**
* Cancels an already running job.
*
* @param name
* @throws Exception
*/
public void cancel(String name) throws Exception {
ExecutingJobAndInstance instance = executing.get(name);
if(instance == null) {
throw new IllegalArgumentException("'" + name + "' is not currently running.");
}
instance.getExecutableFlow().cancel();
}
/**
* Run job file given the id
*
* @param id
* @param ignoreDep
*/
public void execute(String id, boolean ignoreDep) {
final ExecutableFlow flowToRun = allKnownFlows.createNewExecutableFlow(id);
if (isExecuting(id)) {
throw new JobExecutionException("Job " + id + " is already running.");
}
if(ignoreDep) {
for(ExecutableFlow subFlow: flowToRun.getChildren()) {
subFlow.markCompleted();
}
}
execute(flowToRun);
}
/**
* Runs the job immediately
*
* @param holder The execution of the flow to run
*/
public void execute(ExecutableFlow flow) {
if (isExecuting(flow.getName())) {
throw new JobExecutionException("Job " + flow.getName() + " is already running.");
}
final Props parentProps = produceParentProperties(flow);
FlowExecutionHolder holder = new FlowExecutionHolder(flow, parentProps);
logger.info("Executing job '" + flow.getName() + "' now");
final JobExecution executingJob = new JobExecution(flow.getName(),
new DateTime(),
true);
MonitorImpl.getInternalMonitorInterface().workflowEvent(flow.getId(),
System.currentTimeMillis(),
WorkflowAction.START_WORKFLOW,
WorkflowState.NOP,
flow.getName());
executor.execute(new ExecutingFlowRunnable(holder, executingJob));
}
/**
* Schedule this flow to run one time at the specified date
*
* @param holder The execution of the flow to run
*/
public void execute(FlowExecutionHolder holder) {
ExecutableFlow flow = holder.getFlow();
if (isExecuting(flow.getName())) {
throw new JobExecutionException("Job " + flow.getName() + " is already running.");
}
logger.info("Executing job '" + flow.getName() + "' now");
final JobExecution executingJob = new JobExecution(flow.getName(),
new DateTime(),
true);
executor.execute(new ExecutingFlowRunnable(holder, executingJob));
}
/**
* set runtime properties
*
* @param p
*/
public void setRuntimeProperties(Properties p) {
runtimeProps = p;
}
/**
* get runtime property
*
* @param name property name
* @return property value
*/
public String getRuntimeProperty(String name) {
return (runtimeProps == null) ? null : runtimeProps.getProperty(name);
}
/**
* set runtime property
*
* @param name property name
* @param value property value
*/
public void setRuntimeProperty(String name, String value) {
if(runtimeProps == null) {
runtimeProps = new Properties();
}
runtimeProps.setProperty(name, value);
}
/*
* Wrap a single exception with the name of the scheduled job
*/
private void sendErrorEmail(JobExecution job,
Throwable exception,
String senderAddress,
List<String> emailList) {
Map<String, Throwable> map = new HashMap<String, Throwable>();
map.put(job.getId(), exception);
sendErrorEmail(job, map, senderAddress, emailList);
}
/*
* Send error email
*
* @param job scheduled job
*
* @param exceptions exceptions thrown by failed jobs
*
* @param senderAddress email address of sender
*
* @param emailList email addresses of receivers
*/
private void sendErrorEmail(JobExecution job,
Map<String, Throwable> exceptions,
String senderAddress,
List<String> emailList) {
if((emailList == null || emailList.isEmpty()) && jobFailureEmail != null)
emailList = Arrays.asList(jobFailureEmail);
if(emailList != null && mailman != null) {
try {
StringBuffer body = new StringBuffer("The job '"
+ job.getId()
+ "' running on "
+ InetAddress.getLocalHost().getHostName()
+ " has failed with the following errors: \r\n\r\n");
int errorNo = 1;
String logUrlPrefix = runtimeProps != null ? runtimeProps.getProperty(AppCommon.DEFAULT_LOG_URL_PREFIX)
: null;
if(logUrlPrefix == null && runtimeProps != null) {
logUrlPrefix = runtimeProps.getProperty(AppCommon.LOG_URL_PREFIX);
}
final int lastLogLineNum = 60;
for(Map.Entry<String, Throwable> entry: exceptions.entrySet()) {
final String jobId = entry.getKey();
final Throwable exception = entry.getValue();
/* append job exception */
String error = (exception instanceof ProcessFailureException) ? ((ProcessFailureException) exception).getLogSnippet()
: Utils.stackTrace(exception);
body.append(" Job " + errorNo + ". " + jobId + ":\n" + error + "\n");
/* append log file link */
JobExecution jobExec = jobManager.loadMostRecentJobExecution(jobId);
if(jobExec == null) {
body.append("Job execution object is null for jobId:" + jobId + "\n\n");
}
String logPath = jobExec != null ? jobExec.getLog() : null;
if(logPath == null) {
body.append("Log path is null. \n\n");
} else {
body.append("See log in " + logUrlPrefix + logPath + "\n\n" + "The last "
+ lastLogLineNum + " lines in the log are:\n");
/* append last N lines of the log file */
String logFilePath = this.jobManager.getLogDir() + File.separator
+ logPath;
Vector<String> lastNLines = Utils.tail(logFilePath, 60);
if(lastNLines != null) {
for(String line: lastNLines) {
body.append(line + "\n");
}
}
}
errorNo++;
}
// logger.error("\n\n error email body: \n" + body.toString() +
// "\n");
mailman.sendEmailIfPossible(senderAddress,
emailList,
"Job '" + job.getId() + "' has failed!",
body.toString());
} catch(UnknownHostException uhe) {
logger.error(uhe);
}
}
}
private void sendSuccessEmail(JobExecution job,
Duration duration,
String senderAddress,
List<String> emailList) {
if((emailList == null || emailList.isEmpty()) && jobSuccessEmail != null) {
emailList = Arrays.asList(jobSuccessEmail);
}
if(emailList != null && mailman != null) {
try {
mailman.sendEmailIfPossible(senderAddress,
emailList,
"Job '" + job.getId() + "' has completed on "
+ InetAddress.getLocalHost().getHostName()
+ "!",
"The job '"
+ job.getId()
+ "' completed in "
+ PeriodFormat.getDefault()
.print(duration.toPeriod())
+ ".");
} catch(UnknownHostException uhe) {
logger.error(uhe);
}
}
}
private Props produceParentProperties(final ExecutableFlow flow) {
Props parentProps = new Props();
parentProps.put("azkaban.flow.id", flow.getId());
parentProps.put("azkaban.flow.uuid", UUID.randomUUID().toString());
DateTime loadTime = new DateTime();
parentProps.put("azkaban.flow.start.timestamp", loadTime.toString());
parentProps.put("azkaban.flow.start.year", loadTime.toString("yyyy"));
parentProps.put("azkaban.flow.start.month", loadTime.toString("MM"));
parentProps.put("azkaban.flow.start.day", loadTime.toString("dd"));
parentProps.put("azkaban.flow.start.hour", loadTime.toString("HH"));
parentProps.put("azkaban.flow.start.minute", loadTime.toString("mm"));
parentProps.put("azkaban.flow.start.seconds", loadTime.toString("ss"));
parentProps.put("azkaban.flow.start.milliseconds", loadTime.toString("SSS"));
parentProps.put("azkaban.flow.start.timezone", loadTime.toString("ZZZZ"));
return parentProps;
}
/**
* A thread factory that sets the correct classloader for the thread
*/
public class ExecutorThreadFactory implements ThreadFactory {
private final AtomicInteger threadCount = new AtomicInteger(0);
public Thread newThread(Runnable r) {
Thread t = new Thread(r);
t.setDaemon(true);
t.setName("scheduler-thread-" + threadCount.getAndIncrement());
return t;
}
}
public class ExecutingJobAndInstance {
private final ExecutableFlow flow;
private final JobExecution scheduledJob;
private ExecutingJobAndInstance(ExecutableFlow flow, JobExecution scheduledJob) {
this.flow = flow;
this.scheduledJob = scheduledJob;
}
public ExecutableFlow getExecutableFlow() {
return flow;
}
public JobExecution getScheduledJob() {
return scheduledJob;
}
}
/**
* A runnable adapter for a Job
*/
private class ExecutingFlowRunnable implements Runnable {
private final JobExecution runningJob;
private final FlowExecutionHolder holder;
private ExecutingFlowRunnable(FlowExecutionHolder holder, JobExecution runningJob) {
this.holder = holder;
this.runningJob = runningJob;
}
public void run() {
final ExecutableFlow flow = holder.getFlow();
logger.info("Starting run of " + flow.getName());
List<String> emailList = null;
String senderAddress = null;
try {
final JobDescriptor jobDescriptor = jobManager.getJobDescriptor(flow.getName());
emailList = jobDescriptor.getEmailNotificationList();
final List<String> finalEmailList = emailList;
senderAddress = jobDescriptor.getSenderEmail();
final String senderEmail = senderAddress;
// mark the job as executing
runningJob.setStartTime(new DateTime());
executing.put(flow.getName(), new ExecutingJobAndInstance(flow, runningJob));
flow.execute(holder.getParentProps(), new FlowCallback() {
@Override
public void progressMade() {
allKnownFlows.saveExecutableFlow(holder);
}
@Override
public void completed(Status status) {
runningJob.setEndTime(new DateTime());
MonitorImpl.getInternalMonitorInterface().workflowEvent(flow.getId(),
System.currentTimeMillis(),
WorkflowAction.END_WORKFLOW,
(status == Status.SUCCEEDED ? WorkflowState.SUCCESSFUL :
(status == Status.FAILED ? WorkflowState.FAILED : WorkflowState.UNKNOWN)),
flow.getName());
try {
allKnownFlows.saveExecutableFlow(holder);
switch(status) {
case SUCCEEDED:
if (jobDescriptor.getSendSuccessEmail()) {
sendSuccessEmail(
runningJob,
runningJob.getExecutionDuration(),
senderEmail,
finalEmailList
);
}
break;
case FAILED:
sendErrorEmail(runningJob,
flow.getExceptions(),
senderEmail,
finalEmailList);
break;
default:
sendErrorEmail(runningJob,
new RuntimeException(String.format("Got an unknown status[%s]",
status)),
senderEmail,
finalEmailList);
}
} catch(RuntimeException e) {
logger.warn("Exception caught while saving flow/sending emails", e);
executing.remove(runningJob.getId());
throw e;
} finally {
// mark the job as completed
executing.remove(runningJob.getId());
completed.put(runningJob.getId(), runningJob);
}
}
});
allKnownFlows.saveExecutableFlow(holder);
} catch(Throwable t) {
executing.remove(runningJob.getId());
if(emailList != null) {
sendErrorEmail(runningJob, t, senderAddress, emailList);
}
logger.warn(String.format("An exception almost made it back to the ScheduledThreadPool from job[%s]",
runningJob),
t);
}
}
}
public boolean isExecuting(String name) {
return executing.containsKey(name);
}
public Collection<ExecutingJobAndInstance> getExecutingJobs() {
return executing.values();
}
public Multimap<String, JobExecution> getCompleted() {
return completed;
}
}