-
Notifications
You must be signed in to change notification settings - Fork 8.7k
/
JobHistoryEventHandler.java
1748 lines (1626 loc) · 69.6 KB
/
JobHistoryEventHandler.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
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
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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.hadoop.mapreduce.jobhistory;
import java.io.IOException;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.TaskStatus;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.client.api.TimelineClient;
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import com.google.common.annotations.VisibleForTesting;
import com.sun.jersey.api.client.ClientHandlerException;
/**
* The job history events get routed to this class. This class writes the Job
* history events to the DFS directly into a staging dir and then moved to a
* done-dir. JobHistory implementation is in this package to access package
* private classes.
*/
public class JobHistoryEventHandler extends AbstractService
implements EventHandler<JobHistoryEvent> {
private final AppContext context;
private final int startCount;
private int eventCounter;
// Those file systems may differ from the job configuration
// See org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils
// #ensurePathInDefaultFileSystem
private FileSystem stagingDirFS; // log Dir FileSystem
private FileSystem doneDirFS; // done Dir FileSystem
private Path stagingDirPath = null;
private Path doneDirPrefixPath = null; // folder for completed jobs
private int maxUnflushedCompletionEvents;
private int postJobCompletionMultiplier;
private long flushTimeout;
private int minQueueSizeForBatchingFlushes; // TODO: Rename
private int numUnflushedCompletionEvents = 0;
private boolean isTimerActive;
private EventWriter.WriteMode jhistMode =
EventWriter.WriteMode.JSON;
protected BlockingQueue<JobHistoryEvent> eventQueue =
new LinkedBlockingQueue<JobHistoryEvent>();
protected Thread eventHandlingThread;
private volatile boolean stopped;
private final Object lock = new Object();
private static final Log LOG = LogFactory.getLog(
JobHistoryEventHandler.class);
protected static final Map<JobId, MetaInfo> fileMap =
Collections.<JobId,MetaInfo>synchronizedMap(new HashMap<JobId,MetaInfo>());
// should job completion be force when the AM shuts down?
protected volatile boolean forceJobCompletion = false;
@VisibleForTesting
protected TimelineClient timelineClient;
@VisibleForTesting
protected TimelineV2Client timelineV2Client;
private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK";
private static final String MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE =
"MAPREDUCE_TASK_ATTEMPT";
public JobHistoryEventHandler(AppContext context, int startCount) {
super("JobHistoryEventHandler");
this.context = context;
this.startCount = startCount;
}
/* (non-Javadoc)
* @see org.apache.hadoop.yarn.service.AbstractService#init(org.
* apache.hadoop.conf.Configuration)
* Initializes the FileSystem and Path objects for the log and done directories.
* Creates these directories if they do not already exist.
*/
@Override
protected void serviceInit(Configuration conf) throws Exception {
String jobId =
TypeConverter.fromYarn(context.getApplicationID()).toString();
String stagingDirStr = null;
String doneDirStr = null;
String userDoneDirStr = null;
try {
stagingDirStr = JobHistoryUtils.getConfiguredHistoryStagingDirPrefix(conf,
jobId);
doneDirStr =
JobHistoryUtils.getConfiguredHistoryIntermediateDoneDirPrefix(conf);
userDoneDirStr =
JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf);
} catch (IOException e) {
LOG.error("Failed while getting the configured log directories", e);
throw new YarnRuntimeException(e);
}
//Check for the existence of the history staging dir. Maybe create it.
try {
stagingDirPath =
FileContext.getFileContext(conf).makeQualified(new Path(stagingDirStr));
stagingDirFS = FileSystem.get(stagingDirPath.toUri(), conf);
mkdir(stagingDirFS, stagingDirPath, new FsPermission(
JobHistoryUtils.HISTORY_STAGING_DIR_PERMISSIONS));
} catch (IOException e) {
LOG.error("Failed while checking for/creating history staging path: ["
+ stagingDirPath + "]", e);
throw new YarnRuntimeException(e);
}
//Check for the existence of intermediate done dir.
Path doneDirPath = null;
try {
doneDirPath = FileContext.getFileContext(conf).makeQualified(new Path(doneDirStr));
doneDirFS = FileSystem.get(doneDirPath.toUri(), conf);
// This directory will be in a common location, or this may be a cluster
// meant for a single user. Creating based on the conf. Should ideally be
// created by the JobHistoryServer or as part of deployment.
if (!doneDirFS.exists(doneDirPath)) {
if (JobHistoryUtils.shouldCreateNonUserDirectory(conf)) {
LOG.info("Creating intermediate history logDir: ["
+ doneDirPath
+ "] + based on conf. Should ideally be created by the JobHistoryServer: "
+ MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR);
mkdir(
doneDirFS,
doneDirPath,
new FsPermission(
JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS
.toShort()));
// TODO Temporary toShort till new FsPermission(FsPermissions)
// respects
// sticky
} else {
String message = "Not creating intermediate history logDir: ["
+ doneDirPath
+ "] based on conf: "
+ MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR
+ ". Either set to true or pre-create this directory with" +
" appropriate permissions";
LOG.error(message);
throw new YarnRuntimeException(message);
}
}
} catch (IOException e) {
LOG.error("Failed checking for the existance of history intermediate " +
"done directory: [" + doneDirPath + "]");
throw new YarnRuntimeException(e);
}
//Check/create user directory under intermediate done dir.
try {
doneDirPrefixPath =
FileContext.getFileContext(conf).makeQualified(new Path(userDoneDirStr));
mkdir(doneDirFS, doneDirPrefixPath, new FsPermission(
JobHistoryUtils.HISTORY_INTERMEDIATE_USER_DIR_PERMISSIONS));
} catch (IOException e) {
LOG.error("Error creating user intermediate history done directory: [ "
+ doneDirPrefixPath + "]", e);
throw new YarnRuntimeException(e);
}
// Maximum number of unflushed completion-events that can stay in the queue
// before flush kicks in.
maxUnflushedCompletionEvents =
conf.getInt(MRJobConfig.MR_AM_HISTORY_MAX_UNFLUSHED_COMPLETE_EVENTS,
MRJobConfig.DEFAULT_MR_AM_HISTORY_MAX_UNFLUSHED_COMPLETE_EVENTS);
// We want to cut down flushes after job completes so as to write quicker,
// so we increase maxUnflushedEvents post Job completion by using the
// following multiplier.
postJobCompletionMultiplier =
conf.getInt(
MRJobConfig.MR_AM_HISTORY_JOB_COMPLETE_UNFLUSHED_MULTIPLIER,
MRJobConfig.DEFAULT_MR_AM_HISTORY_JOB_COMPLETE_UNFLUSHED_MULTIPLIER);
// Max time until which flush doesn't take place.
flushTimeout =
conf.getLong(MRJobConfig.MR_AM_HISTORY_COMPLETE_EVENT_FLUSH_TIMEOUT_MS,
MRJobConfig.DEFAULT_MR_AM_HISTORY_COMPLETE_EVENT_FLUSH_TIMEOUT_MS);
minQueueSizeForBatchingFlushes =
conf.getInt(
MRJobConfig.MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD,
MRJobConfig.DEFAULT_MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD);
// TODO replace MR specific configurations on timeline service with getting
// configuration from RM through registerApplicationMaster() in
// ApplicationMasterProtocol with return value for timeline service
// configuration status: off, on_with_v1 or on_with_v2.
if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) {
LOG.info("Emitting job history data to the timeline service is enabled");
if (YarnConfiguration.timelineServiceEnabled(conf)) {
boolean timelineServiceV2Enabled =
((int) YarnConfiguration.getTimelineServiceVersion(conf) == 2);
if(timelineServiceV2Enabled) {
timelineV2Client =
((MRAppMaster.RunningAppContext)context).getTimelineV2Client();
timelineV2Client.init(conf);
} else {
timelineClient =
((MRAppMaster.RunningAppContext) context).getTimelineClient();
timelineClient.init(conf);
}
LOG.info("Timeline service is enabled; version: " +
YarnConfiguration.getTimelineServiceVersion(conf));
} else {
LOG.info("Timeline service is not enabled");
}
} else {
LOG.info("Emitting job history data to the timeline server is not " +
"enabled");
}
// Flag for setting
String jhistFormat = conf.get(JHAdminConfig.MR_HS_JHIST_FORMAT,
JHAdminConfig.DEFAULT_MR_HS_JHIST_FORMAT);
if (jhistFormat.equals("json")) {
jhistMode = EventWriter.WriteMode.JSON;
} else if (jhistFormat.equals("binary")) {
jhistMode = EventWriter.WriteMode.BINARY;
} else {
LOG.warn("Unrecognized value '" + jhistFormat + "' for property " +
JHAdminConfig.MR_HS_JHIST_FORMAT + ". Valid values are " +
"'json' or 'binary'. Falling back to default value '" +
JHAdminConfig.DEFAULT_MR_HS_JHIST_FORMAT + "'.");
}
super.serviceInit(conf);
}
private void mkdir(FileSystem fs, Path path, FsPermission fsp)
throws IOException {
if (!fs.exists(path)) {
try {
fs.mkdirs(path, fsp);
FileStatus fsStatus = fs.getFileStatus(path);
LOG.info("Perms after creating " + fsStatus.getPermission().toShort()
+ ", Expected: " + fsp.toShort());
if (fsStatus.getPermission().toShort() != fsp.toShort()) {
LOG.info("Explicitly setting permissions to : " + fsp.toShort()
+ ", " + fsp);
fs.setPermission(path, fsp);
}
} catch (FileAlreadyExistsException e) {
LOG.info("Directory: [" + path + "] already exists.");
}
}
}
@Override
protected void serviceStart() throws Exception {
if (timelineClient != null) {
timelineClient.start();
} else if (timelineV2Client != null) {
timelineV2Client.start();
}
eventHandlingThread = new Thread(new Runnable() {
@Override
public void run() {
JobHistoryEvent event = null;
while (!stopped && !Thread.currentThread().isInterrupted()) {
// Log the size of the history-event-queue every so often.
if (eventCounter != 0 && eventCounter % 1000 == 0) {
eventCounter = 0;
LOG.info("Size of the JobHistory event queue is "
+ eventQueue.size());
} else {
eventCounter++;
}
try {
event = eventQueue.take();
} catch (InterruptedException e) {
LOG.info("EventQueue take interrupted. Returning");
return;
}
// If an event has been removed from the queue. Handle it.
// The rest of the queue is handled via stop()
// Clear the interrupt status if it's set before calling handleEvent
// and set it if it was set before calling handleEvent.
// Interrupts received from other threads during handleEvent cannot be
// dealth with - Shell.runCommand() ignores them.
synchronized (lock) {
boolean isInterrupted = Thread.interrupted();
handleEvent(event);
if (isInterrupted) {
LOG.debug("Event handling interrupted");
Thread.currentThread().interrupt();
}
}
}
}
}, "eventHandlingThread");
eventHandlingThread.start();
super.serviceStart();
}
@Override
protected void serviceStop() throws Exception {
LOG.info("Stopping JobHistoryEventHandler. "
+ "Size of the outstanding queue size is " + eventQueue.size());
stopped = true;
//do not interrupt while event handling is in progress
synchronized(lock) {
if (eventHandlingThread != null) {
LOG.debug("Interrupting Event Handling thread");
eventHandlingThread.interrupt();
} else {
LOG.debug("Null event handling thread");
}
}
try {
if (eventHandlingThread != null) {
LOG.debug("Waiting for Event Handling thread to complete");
eventHandlingThread.join();
}
} catch (InterruptedException ie) {
LOG.info("Interrupted Exception while stopping", ie);
}
// Cancel all timers - so that they aren't invoked during or after
// the metaInfo object is wrapped up.
for (MetaInfo mi : fileMap.values()) {
try {
if (LOG.isDebugEnabled()) {
LOG.debug("Shutting down timer for " + mi);
}
mi.shutDownTimer();
} catch (IOException e) {
LOG.info("Exception while canceling delayed flush timer. "
+ "Likely caused by a failed flush " + e.getMessage());
}
}
//write all the events remaining in queue
Iterator<JobHistoryEvent> it = eventQueue.iterator();
while(it.hasNext()) {
JobHistoryEvent ev = it.next();
LOG.info("In stop, writing event " + ev.getType());
handleEvent(ev);
}
// Process JobUnsuccessfulCompletionEvent for jobIds which still haven't
// closed their event writers
if(forceJobCompletion) {
for (Map.Entry<JobId,MetaInfo> jobIt : fileMap.entrySet()) {
JobId toClose = jobIt.getKey();
MetaInfo mi = jobIt.getValue();
if(mi != null && mi.isWriterActive()) {
LOG.warn("Found jobId " + toClose
+ " to have not been closed. Will close");
//Create a JobFinishEvent so that it is written to the job history
final Job job = context.getJob(toClose);
int successfulMaps = job.getCompletedMaps() - job.getFailedMaps()
- job.getKilledMaps();
int successfulReduces = job.getCompletedReduces()
- job.getFailedReduces() - job.getKilledReduces();
JobUnsuccessfulCompletionEvent jucEvent =
new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(toClose),
System.currentTimeMillis(),
successfulMaps,
successfulReduces,
job.getFailedMaps(), job.getFailedReduces(),
job.getKilledMaps(), job.getKilledReduces(),
createJobStateForJobUnsuccessfulCompletionEvent(
mi.getForcedJobStateOnShutDown()),
job.getDiagnostics());
JobHistoryEvent jfEvent = new JobHistoryEvent(toClose, jucEvent);
//Bypass the queue mechanism which might wait. Call the method directly
handleEvent(jfEvent);
}
}
}
//close all file handles
for (MetaInfo mi : fileMap.values()) {
try {
mi.closeWriter();
} catch (IOException e) {
LOG.info("Exception while closing file " + e.getMessage());
}
}
if (timelineClient != null) {
timelineClient.stop();
} else if (timelineV2Client != null) {
timelineV2Client.stop();
}
LOG.info("Stopped JobHistoryEventHandler. super.stop()");
super.serviceStop();
}
protected EventWriter createEventWriter(Path historyFilePath)
throws IOException {
FSDataOutputStream out = stagingDirFS.create(historyFilePath, true);
return new EventWriter(out, this.jhistMode);
}
/**
* Create an event writer for the Job represented by the jobID.
* Writes out the job configuration to the log directory.
* This should be the first call to history for a job
*
* @param jobId the jobId.
* @param amStartedEvent
* @throws IOException
*/
protected void setupEventWriter(JobId jobId, AMStartedEvent amStartedEvent)
throws IOException {
if (stagingDirPath == null) {
LOG.error("Log Directory is null, returning");
throw new IOException("Missing Log Directory for History");
}
MetaInfo oldFi = fileMap.get(jobId);
Configuration conf = getConfig();
// TODO Ideally this should be written out to the job dir
// (.staging/jobid/files - RecoveryService will need to be patched)
Path historyFile = JobHistoryUtils.getStagingJobHistoryFile(
stagingDirPath, jobId, startCount);
String user = UserGroupInformation.getCurrentUser().getShortUserName();
if (user == null) {
throw new IOException(
"User is null while setting up jobhistory eventwriter");
}
String jobName = context.getJob(jobId).getName();
EventWriter writer = (oldFi == null) ? null : oldFi.writer;
Path logDirConfPath =
JobHistoryUtils.getStagingConfFile(stagingDirPath, jobId, startCount);
if (writer == null) {
try {
writer = createEventWriter(historyFile);
LOG.info("Event Writer setup for JobId: " + jobId + ", File: "
+ historyFile);
} catch (IOException ioe) {
LOG.info("Could not create log file: [" + historyFile + "] + for job "
+ "[" + jobName + "]");
throw ioe;
}
//Write out conf only if the writer isn't already setup.
if (conf != null) {
// TODO Ideally this should be written out to the job dir
// (.staging/jobid/files - RecoveryService will need to be patched)
if (logDirConfPath != null) {
Configuration redactedConf = new Configuration(conf);
MRJobConfUtil.redact(redactedConf);
try (FSDataOutputStream jobFileOut = stagingDirFS
.create(logDirConfPath, true)) {
redactedConf.writeXml(jobFileOut);
} catch (IOException e) {
LOG.info("Failed to write the job configuration file", e);
throw e;
}
}
}
}
String queueName = JobConf.DEFAULT_QUEUE_NAME;
if (conf != null) {
queueName = conf.get(MRJobConfig.QUEUE_NAME, JobConf.DEFAULT_QUEUE_NAME);
}
MetaInfo fi = new MetaInfo(historyFile, logDirConfPath, writer,
user, jobName, jobId, amStartedEvent.getForcedJobStateOnShutDown(),
queueName);
fi.getJobSummary().setJobId(jobId);
fi.getJobSummary().setJobLaunchTime(amStartedEvent.getStartTime());
fi.getJobSummary().setJobSubmitTime(amStartedEvent.getSubmitTime());
fi.getJobIndexInfo().setJobStartTime(amStartedEvent.getStartTime());
fi.getJobIndexInfo().setSubmitTime(amStartedEvent.getSubmitTime());
fileMap.put(jobId, fi);
}
/** Close the event writer for this id
* @throws IOException */
public void closeWriter(JobId id) throws IOException {
try {
final MetaInfo mi = fileMap.get(id);
if (mi != null) {
mi.closeWriter();
}
} catch (IOException e) {
LOG.error("Error closing writer for JobID: " + id);
throw e;
}
}
@Override
public void handle(JobHistoryEvent event) {
try {
if (isJobCompletionEvent(event.getHistoryEvent())) {
// When the job is complete, flush slower but write faster.
maxUnflushedCompletionEvents =
maxUnflushedCompletionEvents * postJobCompletionMultiplier;
}
eventQueue.put(event);
} catch (InterruptedException e) {
throw new YarnRuntimeException(e);
}
}
private boolean isJobCompletionEvent(HistoryEvent historyEvent) {
if (EnumSet.of(EventType.JOB_FINISHED, EventType.JOB_FAILED,
EventType.JOB_KILLED).contains(historyEvent.getEventType())) {
return true;
}
return false;
}
@Private
public void handleEvent(JobHistoryEvent event) {
synchronized (lock) {
// If this is JobSubmitted Event, setup the writer
if (event.getHistoryEvent().getEventType() == EventType.AM_STARTED) {
try {
AMStartedEvent amStartedEvent =
(AMStartedEvent) event.getHistoryEvent();
setupEventWriter(event.getJobID(), amStartedEvent);
} catch (IOException ioe) {
LOG.error("Error JobHistoryEventHandler in handleEvent: " + event,
ioe);
throw new YarnRuntimeException(ioe);
}
}
// For all events
// (1) Write it out
// (2) Process it for JobSummary
// (3) Process it for ATS (if enabled)
MetaInfo mi = fileMap.get(event.getJobID());
try {
HistoryEvent historyEvent = event.getHistoryEvent();
if (! (historyEvent instanceof NormalizedResourceEvent)) {
mi.writeEvent(historyEvent);
}
processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(),
event.getJobID());
if (timelineV2Client != null) {
processEventForNewTimelineService(historyEvent, event.getJobID(),
event.getTimestamp());
} else if (timelineClient != null) {
processEventForTimelineServer(historyEvent, event.getJobID(),
event.getTimestamp());
}
if (LOG.isDebugEnabled()) {
LOG.debug("In HistoryEventHandler "
+ event.getHistoryEvent().getEventType());
}
} catch (IOException e) {
LOG.error("Error writing History Event: " + event.getHistoryEvent(),
e);
throw new YarnRuntimeException(e);
}
if (event.getHistoryEvent().getEventType() == EventType.JOB_SUBMITTED) {
JobSubmittedEvent jobSubmittedEvent =
(JobSubmittedEvent) event.getHistoryEvent();
mi.getJobIndexInfo().setSubmitTime(jobSubmittedEvent.getSubmitTime());
mi.getJobIndexInfo().setQueueName(jobSubmittedEvent.getJobQueueName());
}
//initialize the launchTime in the JobIndexInfo of MetaInfo
if(event.getHistoryEvent().getEventType() == EventType.JOB_INITED ){
JobInitedEvent jie = (JobInitedEvent) event.getHistoryEvent();
mi.getJobIndexInfo().setJobStartTime(jie.getLaunchTime());
}
if (event.getHistoryEvent().getEventType() == EventType.JOB_QUEUE_CHANGED) {
JobQueueChangeEvent jQueueEvent =
(JobQueueChangeEvent) event.getHistoryEvent();
mi.getJobIndexInfo().setQueueName(jQueueEvent.getJobQueueName());
}
// If this is JobFinishedEvent, close the writer and setup the job-index
if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {
try {
JobFinishedEvent jFinishedEvent =
(JobFinishedEvent) event.getHistoryEvent();
mi.getJobIndexInfo().setFinishTime(jFinishedEvent.getFinishTime());
mi.getJobIndexInfo().setNumMaps(jFinishedEvent.getSucceededMaps());
mi.getJobIndexInfo().setNumReduces(
jFinishedEvent.getSucceededReduces());
mi.getJobIndexInfo().setJobStatus(JobState.SUCCEEDED.toString());
closeEventWriter(event.getJobID());
processDoneFiles(event.getJobID());
} catch (IOException e) {
throw new YarnRuntimeException(e);
}
}
// In case of JOB_ERROR, only process all the Done files(e.g. job
// summary, job history file etc.) if it is last AM retry.
if (event.getHistoryEvent().getEventType() == EventType.JOB_ERROR) {
try {
JobUnsuccessfulCompletionEvent jucEvent =
(JobUnsuccessfulCompletionEvent) event.getHistoryEvent();
mi.getJobIndexInfo().setFinishTime(jucEvent.getFinishTime());
mi.getJobIndexInfo().setNumMaps(jucEvent.getSucceededMaps());
mi.getJobIndexInfo().setNumReduces(jucEvent.getSucceededReduces());
mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
closeEventWriter(event.getJobID());
if(context.isLastAMRetry())
processDoneFiles(event.getJobID());
} catch (IOException e) {
throw new YarnRuntimeException(e);
}
}
if (event.getHistoryEvent().getEventType() == EventType.JOB_FAILED
|| event.getHistoryEvent().getEventType() == EventType.JOB_KILLED) {
try {
JobUnsuccessfulCompletionEvent jucEvent =
(JobUnsuccessfulCompletionEvent) event
.getHistoryEvent();
mi.getJobIndexInfo().setFinishTime(jucEvent.getFinishTime());
mi.getJobIndexInfo().setNumMaps(jucEvent.getSucceededMaps());
mi.getJobIndexInfo().setNumReduces(jucEvent.getSucceededReduces());
mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
closeEventWriter(event.getJobID());
processDoneFiles(event.getJobID());
} catch (IOException e) {
throw new YarnRuntimeException(e);
}
}
}
}
public void processEventForJobSummary(HistoryEvent event, JobSummary summary,
JobId jobId) {
// context.getJob could be used for some of this info as well.
switch (event.getEventType()) {
case JOB_SUBMITTED:
JobSubmittedEvent jse = (JobSubmittedEvent) event;
summary.setUser(jse.getUserName());
summary.setQueue(jse.getJobQueueName());
summary.setJobSubmitTime(jse.getSubmitTime());
summary.setJobName(jse.getJobName());
break;
case NORMALIZED_RESOURCE:
NormalizedResourceEvent normalizedResourceEvent =
(NormalizedResourceEvent) event;
if (normalizedResourceEvent.getTaskType() == TaskType.MAP) {
summary.setResourcesPerMap((int) normalizedResourceEvent.getMemory());
} else if (normalizedResourceEvent.getTaskType() == TaskType.REDUCE) {
summary.setResourcesPerReduce((int) normalizedResourceEvent.getMemory());
}
break;
case JOB_INITED:
JobInitedEvent jie = (JobInitedEvent) event;
summary.setJobLaunchTime(jie.getLaunchTime());
break;
case MAP_ATTEMPT_STARTED:
TaskAttemptStartedEvent mtase = (TaskAttemptStartedEvent) event;
if (summary.getFirstMapTaskLaunchTime() == 0)
summary.setFirstMapTaskLaunchTime(mtase.getStartTime());
break;
case REDUCE_ATTEMPT_STARTED:
TaskAttemptStartedEvent rtase = (TaskAttemptStartedEvent) event;
if (summary.getFirstReduceTaskLaunchTime() == 0)
summary.setFirstReduceTaskLaunchTime(rtase.getStartTime());
break;
case JOB_FINISHED:
JobFinishedEvent jfe = (JobFinishedEvent) event;
summary.setJobFinishTime(jfe.getFinishTime());
summary.setNumSucceededMaps(jfe.getSucceededMaps());
summary.setNumFailedMaps(jfe.getFailedMaps());
summary.setNumSucceededReduces(jfe.getSucceededReduces());
summary.setNumFailedReduces(jfe.getFailedReduces());
summary.setNumKilledMaps(jfe.getKilledMaps());
summary.setNumKilledReduces(jfe.getKilledReduces());
if (summary.getJobStatus() == null)
summary
.setJobStatus(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED
.toString());
// TODO JOB_FINISHED does not have state. Effectively job history does not
// have state about the finished job.
setSummarySlotSeconds(summary, jfe.getTotalCounters());
break;
case JOB_FAILED:
case JOB_KILLED:
Job job = context.getJob(jobId);
JobUnsuccessfulCompletionEvent juce = (JobUnsuccessfulCompletionEvent) event;
int successfulMaps = job.getCompletedMaps() - job.getFailedMaps()
- job.getKilledMaps();
int successfulReduces = job.getCompletedReduces()
- job.getFailedReduces() - job.getKilledReduces();
summary.setJobStatus(juce.getStatus());
summary.setNumSucceededMaps(successfulMaps);
summary.setNumSucceededReduces(successfulReduces);
summary.setNumFailedMaps(job.getFailedMaps());
summary.setNumFailedReduces(job.getFailedReduces());
summary.setJobFinishTime(juce.getFinishTime());
summary.setNumKilledMaps(juce.getKilledMaps());
summary.setNumKilledReduces(juce.getKilledReduces());
setSummarySlotSeconds(summary, context.getJob(jobId).getAllCounters());
break;
default:
break;
}
}
private void processEventForTimelineServer(HistoryEvent event, JobId jobId,
long timestamp) {
TimelineEvent tEvent = new TimelineEvent();
tEvent.setEventType(StringUtils.toUpperCase(event.getEventType().name()));
tEvent.setTimestamp(timestamp);
TimelineEntity tEntity = new TimelineEntity();
switch (event.getEventType()) {
case JOB_SUBMITTED:
JobSubmittedEvent jse =
(JobSubmittedEvent) event;
tEvent.addEventInfo("SUBMIT_TIME", jse.getSubmitTime());
tEvent.addEventInfo("QUEUE_NAME", jse.getJobQueueName());
tEvent.addEventInfo("JOB_NAME", jse.getJobName());
tEvent.addEventInfo("USER_NAME", jse.getUserName());
tEvent.addEventInfo("JOB_CONF_PATH", jse.getJobConfPath());
tEvent.addEventInfo("ACLS", jse.getJobAcls());
tEvent.addEventInfo("JOB_QUEUE_NAME", jse.getJobQueueName());
tEvent.addEventInfo("WORKFLOW_ID", jse.getWorkflowId());
tEvent.addEventInfo("WORKFLOW_NAME", jse.getWorkflowName());
tEvent.addEventInfo("WORKFLOW_NAME_NAME", jse.getWorkflowNodeName());
tEvent.addEventInfo("WORKFLOW_ADJACENCIES",
jse.getWorkflowAdjacencies());
tEvent.addEventInfo("WORKFLOW_TAGS", jse.getWorkflowTags());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case JOB_STATUS_CHANGED:
JobStatusChangedEvent jsce = (JobStatusChangedEvent) event;
tEvent.addEventInfo("STATUS", jsce.getStatus());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case JOB_INFO_CHANGED:
JobInfoChangeEvent jice = (JobInfoChangeEvent) event;
tEvent.addEventInfo("SUBMIT_TIME", jice.getSubmitTime());
tEvent.addEventInfo("LAUNCH_TIME", jice.getLaunchTime());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case JOB_INITED:
JobInitedEvent jie = (JobInitedEvent) event;
tEvent.addEventInfo("START_TIME", jie.getLaunchTime());
tEvent.addEventInfo("STATUS", jie.getStatus());
tEvent.addEventInfo("TOTAL_MAPS", jie.getTotalMaps());
tEvent.addEventInfo("TOTAL_REDUCES", jie.getTotalReduces());
tEvent.addEventInfo("UBERIZED", jie.getUberized());
tEntity.setStartTime(jie.getLaunchTime());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case JOB_PRIORITY_CHANGED:
JobPriorityChangeEvent jpce = (JobPriorityChangeEvent) event;
tEvent.addEventInfo("PRIORITY", jpce.getPriority().toString());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case JOB_QUEUE_CHANGED:
JobQueueChangeEvent jqe = (JobQueueChangeEvent) event;
tEvent.addEventInfo("QUEUE_NAMES", jqe.getJobQueueName());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case JOB_FAILED:
case JOB_KILLED:
case JOB_ERROR:
JobUnsuccessfulCompletionEvent juce =
(JobUnsuccessfulCompletionEvent) event;
tEvent.addEventInfo("FINISH_TIME", juce.getFinishTime());
tEvent.addEventInfo("NUM_MAPS",
juce.getSucceededMaps() +
juce.getFailedMaps() +
juce.getKilledMaps());
tEvent.addEventInfo("NUM_REDUCES",
juce.getSucceededReduces() +
juce.getFailedReduces() +
juce.getKilledReduces());
tEvent.addEventInfo("JOB_STATUS", juce.getStatus());
tEvent.addEventInfo("DIAGNOSTICS", juce.getDiagnostics());
tEvent.addEventInfo("SUCCESSFUL_MAPS", juce.getSucceededMaps());
tEvent.addEventInfo("SUCCESSFUL_REDUCES", juce.getSucceededReduces());
tEvent.addEventInfo("FAILED_MAPS", juce.getFailedMaps());
tEvent.addEventInfo("FAILED_REDUCES", juce.getFailedReduces());
tEvent.addEventInfo("KILLED_MAPS", juce.getKilledMaps());
tEvent.addEventInfo("KILLED_REDUCES", juce.getKilledReduces());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case JOB_FINISHED:
JobFinishedEvent jfe = (JobFinishedEvent) event;
tEvent.addEventInfo("FINISH_TIME", jfe.getFinishTime());
tEvent.addEventInfo("NUM_MAPS",
jfe.getSucceededMaps() +
jfe.getFailedMaps() +
jfe.getKilledMaps());
tEvent.addEventInfo("NUM_REDUCES",
jfe.getSucceededReduces() +
jfe.getFailedReduces() +
jfe.getKilledReduces());
tEvent.addEventInfo("FAILED_MAPS", jfe.getFailedMaps());
tEvent.addEventInfo("FAILED_REDUCES", jfe.getFailedReduces());
tEvent.addEventInfo("SUCCESSFUL_MAPS", jfe.getSucceededMaps());
tEvent.addEventInfo("SUCCESSFUL_REDUCES", jfe.getSucceededReduces());
tEvent.addEventInfo("KILLED_MAPS", jfe.getKilledMaps());
tEvent.addEventInfo("KILLED_REDUCES", jfe.getKilledReduces());
tEvent.addEventInfo("MAP_COUNTERS_GROUPS",
JobHistoryEventUtils.countersToJSON(jfe.getMapCounters()));
tEvent.addEventInfo("REDUCE_COUNTERS_GROUPS",
JobHistoryEventUtils.countersToJSON(jfe.getReduceCounters()));
tEvent.addEventInfo("TOTAL_COUNTERS_GROUPS",
JobHistoryEventUtils.countersToJSON(jfe.getTotalCounters()));
tEvent.addEventInfo("JOB_STATUS", JobState.SUCCEEDED.toString());
tEntity.addEvent(tEvent);
tEntity.setEntityId(jobId.toString());
tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);
break;
case TASK_STARTED:
TaskStartedEvent tse = (TaskStartedEvent) event;
tEvent.addEventInfo("TASK_TYPE", tse.getTaskType().toString());
tEvent.addEventInfo("START_TIME", tse.getStartTime());
tEvent.addEventInfo("SPLIT_LOCATIONS", tse.getSplitLocations());
tEntity.addEvent(tEvent);
tEntity.setEntityId(tse.getTaskId().toString());
tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE);
tEntity.addRelatedEntity(MAPREDUCE_JOB_ENTITY_TYPE, jobId.toString());
break;
case TASK_FAILED:
TaskFailedEvent tfe = (TaskFailedEvent) event;
tEvent.addEventInfo("TASK_TYPE", tfe.getTaskType().toString());
tEvent.addEventInfo("STATUS", TaskStatus.State.FAILED.toString());
tEvent.addEventInfo("FINISH_TIME", tfe.getFinishTime());
tEvent.addEventInfo("ERROR", tfe.getError());
tEvent.addEventInfo("FAILED_ATTEMPT_ID",
tfe.getFailedAttemptID() == null ?
"" : tfe.getFailedAttemptID().toString());
tEvent.addEventInfo("COUNTERS_GROUPS",
JobHistoryEventUtils.countersToJSON(tfe.getCounters()));
tEntity.addEvent(tEvent);
tEntity.setEntityId(tfe.getTaskId().toString());
tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE);
tEntity.addRelatedEntity(MAPREDUCE_JOB_ENTITY_TYPE, jobId.toString());
break;
case TASK_UPDATED:
TaskUpdatedEvent tue = (TaskUpdatedEvent) event;
tEvent.addEventInfo("FINISH_TIME", tue.getFinishTime());
tEntity.addEvent(tEvent);
tEntity.setEntityId(tue.getTaskId().toString());
tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE);
tEntity.addRelatedEntity(MAPREDUCE_JOB_ENTITY_TYPE, jobId.toString());
break;
case TASK_FINISHED:
TaskFinishedEvent tfe2 = (TaskFinishedEvent) event;
tEvent.addEventInfo("TASK_TYPE", tfe2.getTaskType().toString());
tEvent.addEventInfo("COUNTERS_GROUPS",
JobHistoryEventUtils.countersToJSON(tfe2.getCounters()));
tEvent.addEventInfo("FINISH_TIME", tfe2.getFinishTime());
tEvent.addEventInfo("STATUS", TaskStatus.State.SUCCEEDED.toString());
tEvent.addEventInfo("SUCCESSFUL_TASK_ATTEMPT_ID",
tfe2.getSuccessfulTaskAttemptId() == null ?
"" : tfe2.getSuccessfulTaskAttemptId().toString());
tEntity.addEvent(tEvent);
tEntity.setEntityId(tfe2.getTaskId().toString());
tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE);
tEntity.addRelatedEntity(MAPREDUCE_JOB_ENTITY_TYPE, jobId.toString());
break;
case MAP_ATTEMPT_STARTED:
case CLEANUP_ATTEMPT_STARTED:
case REDUCE_ATTEMPT_STARTED:
case SETUP_ATTEMPT_STARTED:
TaskAttemptStartedEvent tase = (TaskAttemptStartedEvent) event;
tEvent.addEventInfo("TASK_TYPE", tase.getTaskType().toString());
tEvent.addEventInfo("TASK_ATTEMPT_ID",
tase.getTaskAttemptId().toString());
tEvent.addEventInfo("START_TIME", tase.getStartTime());
tEvent.addEventInfo("HTTP_PORT", tase.getHttpPort());
tEvent.addEventInfo("TRACKER_NAME", tase.getTrackerName());
tEvent.addEventInfo("SHUFFLE_PORT", tase.getShufflePort());
tEvent.addEventInfo("CONTAINER_ID", tase.getContainerId() == null ?
"" : tase.getContainerId().toString());
tEntity.addEvent(tEvent);
tEntity.setEntityId(tase.getTaskId().toString());
tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE);
tEntity.addRelatedEntity(MAPREDUCE_JOB_ENTITY_TYPE, jobId.toString());
break;
case MAP_ATTEMPT_FAILED:
case CLEANUP_ATTEMPT_FAILED:
case REDUCE_ATTEMPT_FAILED:
case SETUP_ATTEMPT_FAILED:
case MAP_ATTEMPT_KILLED:
case CLEANUP_ATTEMPT_KILLED:
case REDUCE_ATTEMPT_KILLED:
case SETUP_ATTEMPT_KILLED:
TaskAttemptUnsuccessfulCompletionEvent tauce =
(TaskAttemptUnsuccessfulCompletionEvent) event;
tEvent.addEventInfo("TASK_TYPE", tauce.getTaskType().toString());
tEvent.addEventInfo("TASK_ATTEMPT_ID",
tauce.getTaskAttemptId() == null ?
"" : tauce.getTaskAttemptId().toString());
tEvent.addEventInfo("FINISH_TIME", tauce.getFinishTime());
tEvent.addEventInfo("ERROR", tauce.getError());
tEvent.addEventInfo("STATUS", tauce.getTaskStatus());
tEvent.addEventInfo("HOSTNAME", tauce.getHostname());
tEvent.addEventInfo("PORT", tauce.getPort());
tEvent.addEventInfo("RACK_NAME", tauce.getRackName());
tEvent.addEventInfo("SHUFFLE_FINISH_TIME", tauce.getFinishTime());
tEvent.addEventInfo("SORT_FINISH_TIME", tauce.getFinishTime());