/
HadoopJobTracker.java
1626 lines (1283 loc) · 52.2 KB
/
HadoopJobTracker.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.ignite.internal.processors.hadoop.jobtracker;
import org.apache.ignite.*;
import org.apache.ignite.events.*;
import org.apache.ignite.events.EventType;
import org.apache.ignite.internal.*;
import org.apache.ignite.internal.managers.eventstorage.*;
import org.apache.ignite.internal.processors.cache.*;
import org.apache.ignite.internal.processors.hadoop.*;
import org.apache.ignite.internal.processors.hadoop.counter.*;
import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
import org.apache.ignite.internal.util.*;
import org.apache.ignite.internal.util.future.*;
import org.apache.ignite.internal.util.typedef.*;
import org.apache.ignite.internal.util.typedef.internal.*;
import org.apache.ignite.lang.*;
import org.jetbrains.annotations.*;
import org.jsr166.*;
import javax.cache.event.*;
import javax.cache.expiry.*;
import javax.cache.processor.*;
import java.io.*;
import java.util.*;
import java.util.concurrent.*;
import java.util.concurrent.atomic.*;
import static java.util.concurrent.TimeUnit.*;
import static org.apache.ignite.internal.processors.hadoop.HadoopJobPhase.*;
import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.*;
import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.*;
/**
* Hadoop job tracker.
*/
public class HadoopJobTracker extends HadoopComponent {
/** */
private final GridMutex mux = new GridMutex();
/** */
private volatile GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> jobMetaPrj;
/** Projection with expiry policy for finished job updates. */
private volatile GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> finishedJobMetaPrj;
/** Map-reduce execution planner. */
@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
private HadoopMapReducePlanner mrPlanner;
/** All the known jobs. */
private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJob>> jobs = new ConcurrentHashMap8<>();
/** Locally active jobs. */
private final ConcurrentMap<HadoopJobId, JobLocalState> activeJobs = new ConcurrentHashMap8<>();
/** Locally requested finish futures. */
private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJobId>> activeFinishFuts =
new ConcurrentHashMap8<>();
/** Event processing service. */
private ExecutorService evtProcSvc;
/** Component busy lock. */
private GridSpinReadWriteLock busyLock;
/** Closure to check result of async transform of system cache. */
private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() {
@Override public void apply(IgniteInternalFuture<?> gridFut) {
try {
gridFut.get();
}
catch (IgniteCheckedException e) {
U.error(log, "Failed to transform system cache.", e);
}
}
};
/** {@inheritDoc} */
@Override public void start(HadoopContext ctx) throws IgniteCheckedException {
super.start(ctx);
busyLock = new GridSpinReadWriteLock();
evtProcSvc = Executors.newFixedThreadPool(1);
}
/**
* @return Job meta projection.
*/
@SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
private GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> jobMetaCache() {
GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> prj = jobMetaPrj;
if (prj == null) {
synchronized (mux) {
if ((prj = jobMetaPrj) == null) {
GridCacheAdapter<HadoopJobId, HadoopJobMetadata> sysCache = ctx.kernalContext().cache()
.internalCache(CU.SYS_CACHE_HADOOP_MR);
assert sysCache != null;
mrPlanner = ctx.planner();
try {
ctx.kernalContext().resource().injectGeneric(mrPlanner);
}
catch (IgniteCheckedException e) { // Must not happen.
U.error(log, "Failed to inject resources.", e);
throw new IllegalStateException(e);
}
jobMetaPrj = prj = (GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata>)
sysCache.projection(HadoopJobId.class, HadoopJobMetadata.class);
if (ctx.configuration().getFinishedJobInfoTtl() > 0) {
ExpiryPolicy finishedJobPlc = new ModifiedExpiryPolicy(
new Duration(MILLISECONDS, ctx.configuration().getFinishedJobInfoTtl()));
finishedJobMetaPrj = prj.withExpiryPolicy(finishedJobPlc);
}
else
finishedJobMetaPrj = jobMetaPrj;
}
}
}
return prj;
}
/**
* @return Projection with expiry policy for finished job updates.
*/
private GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> finishedJobMetaCache() {
GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> prj = finishedJobMetaPrj;
if (prj == null) {
jobMetaCache();
prj = finishedJobMetaPrj;
assert prj != null;
}
return prj;
}
/** {@inheritDoc} */
@SuppressWarnings("deprecation")
@Override public void onKernalStart() throws IgniteCheckedException {
super.onKernalStart();
jobMetaCache().context().continuousQueries().executeInternalQuery(
new CacheEntryUpdatedListener<HadoopJobId, HadoopJobMetadata>() {
@Override public void onUpdated(final Iterable<CacheEntryEvent<? extends HadoopJobId,
? extends HadoopJobMetadata>> evts) {
if (!busyLock.tryReadLock())
return;
try {
// Must process query callback in a separate thread to avoid deadlocks.
evtProcSvc.submit(new EventHandler() {
@Override protected void body() throws IgniteCheckedException {
processJobMetadataUpdates(evts);
}
});
}
finally {
busyLock.readUnlock();
}
}
},
null,
true,
true
);
ctx.kernalContext().event().addLocalEventListener(new GridLocalEventListener() {
@Override public void onEvent(final Event evt) {
if (!busyLock.tryReadLock())
return;
try {
// Must process discovery callback in a separate thread to avoid deadlock.
evtProcSvc.submit(new EventHandler() {
@Override protected void body() {
processNodeLeft((DiscoveryEvent)evt);
}
});
}
finally {
busyLock.readUnlock();
}
}
}, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
}
/** {@inheritDoc} */
@Override public void onKernalStop(boolean cancel) {
super.onKernalStop(cancel);
busyLock.writeLock();
evtProcSvc.shutdown();
// Fail all pending futures.
for (GridFutureAdapter<HadoopJobId> fut : activeFinishFuts.values())
fut.onDone(new IgniteCheckedException("Failed to execute Hadoop map-reduce job (grid is stopping)."));
}
/**
* Submits execution of Hadoop job to grid.
*
* @param jobId Job ID.
* @param info Job info.
* @return Job completion future.
*/
@SuppressWarnings("unchecked")
public IgniteInternalFuture<HadoopJobId> submit(HadoopJobId jobId, HadoopJobInfo info) {
if (!busyLock.tryReadLock()) {
return new GridFinishedFuture<>(new IgniteCheckedException("Failed to execute map-reduce job " +
"(grid is stopping): " + info));
}
try {
long jobPrepare = U.currentTimeMillis();
if (jobs.containsKey(jobId) || jobMetaCache().containsKey(jobId))
throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId);
HadoopJob job = job(jobId, info);
HadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null);
HadoopJobMetadata meta = new HadoopJobMetadata(ctx.localNodeId(), jobId, info);
meta.mapReducePlan(mrPlan);
meta.pendingSplits(allSplits(mrPlan));
meta.pendingReducers(allReducers(mrPlan));
GridFutureAdapter<HadoopJobId> completeFut = new GridFutureAdapter<>();
GridFutureAdapter<HadoopJobId> old = activeFinishFuts.put(jobId, completeFut);
assert old == null : "Duplicate completion future [jobId=" + jobId + ", old=" + old + ']';
if (log.isDebugEnabled())
log.debug("Submitting job metadata [jobId=" + jobId + ", meta=" + meta + ']');
long jobStart = U.currentTimeMillis();
HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(meta.counters(),
ctx.localNodeId());
perfCntr.clientSubmissionEvents(info);
perfCntr.onJobPrepare(jobPrepare);
perfCntr.onJobStart(jobStart);
if (jobMetaCache().putIfAbsent(jobId, meta) != null)
throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId);
return completeFut;
}
catch (IgniteCheckedException e) {
U.error(log, "Failed to submit job: " + jobId, e);
return new GridFinishedFuture<>(e);
}
finally {
busyLock.readUnlock();
}
}
/**
* Convert Hadoop job metadata to job status.
*
* @param meta Metadata.
* @return Status.
*/
@SuppressWarnings("ThrowableResultOfMethodCallIgnored")
public static HadoopJobStatus status(HadoopJobMetadata meta) {
HadoopJobInfo jobInfo = meta.jobInfo();
return new HadoopJobStatus(
meta.jobId(),
jobInfo.jobName(),
jobInfo.user(),
meta.pendingSplits() != null ? meta.pendingSplits().size() : 0,
meta.pendingReducers() != null ? meta.pendingReducers().size() : 0,
meta.mapReducePlan().mappers(),
meta.mapReducePlan().reducers(),
meta.phase(),
meta.failCause() != null,
meta.version()
);
}
/**
* Gets hadoop job status for given job ID.
*
* @param jobId Job ID to get status for.
* @return Job status for given job ID or {@code null} if job was not found.
*/
@Nullable public HadoopJobStatus status(HadoopJobId jobId) throws IgniteCheckedException {
if (!busyLock.tryReadLock())
return null; // Grid is stopping.
try {
HadoopJobMetadata meta = jobMetaCache().get(jobId);
return meta != null ? status(meta) : null;
}
finally {
busyLock.readUnlock();
}
}
/**
* Gets job finish future.
*
* @param jobId Job ID.
* @return Finish future or {@code null}.
* @throws IgniteCheckedException If failed.
*/
@Nullable public IgniteInternalFuture<?> finishFuture(HadoopJobId jobId) throws IgniteCheckedException {
if (!busyLock.tryReadLock())
return null; // Grid is stopping.
try {
HadoopJobMetadata meta = jobMetaCache().get(jobId);
if (meta == null)
return null;
if (log.isTraceEnabled())
log.trace("Got job metadata for status check [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');
if (meta.phase() == PHASE_COMPLETE) {
if (log.isTraceEnabled())
log.trace("Job is complete, returning finished future: " + jobId);
return new GridFinishedFuture<>(jobId);
}
GridFutureAdapter<HadoopJobId> fut = F.addIfAbsent(activeFinishFuts, jobId,
new GridFutureAdapter<HadoopJobId>());
// Get meta from cache one more time to close the window.
meta = jobMetaCache().get(jobId);
if (log.isTraceEnabled())
log.trace("Re-checking job metadata [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');
if (meta == null) {
fut.onDone();
activeFinishFuts.remove(jobId , fut);
}
else if (meta.phase() == PHASE_COMPLETE) {
fut.onDone(jobId, meta.failCause());
activeFinishFuts.remove(jobId , fut);
}
return fut;
}
finally {
busyLock.readUnlock();
}
}
/**
* Gets job plan by job ID.
*
* @param jobId Job ID.
* @return Job plan.
* @throws IgniteCheckedException If failed.
*/
public HadoopMapReducePlan plan(HadoopJobId jobId) throws IgniteCheckedException {
if (!busyLock.tryReadLock())
return null;
try {
HadoopJobMetadata meta = jobMetaCache().get(jobId);
if (meta != null)
return meta.mapReducePlan();
return null;
}
finally {
busyLock.readUnlock();
}
}
/**
* Callback from task executor invoked when a task has been finished.
*
* @param info Task info.
* @param status Task status.
*/
@SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
public void onTaskFinished(HadoopTaskInfo info, HadoopTaskStatus status) {
if (!busyLock.tryReadLock())
return;
try {
assert status.state() != RUNNING;
if (log.isDebugEnabled())
log.debug("Received task finished callback [info=" + info + ", status=" + status + ']');
JobLocalState state = activeJobs.get(info.jobId());
// Task CRASHes with null fail cause.
assert (status.state() != FAILED) || status.failCause() != null :
"Invalid task status [info=" + info + ", status=" + status + ']';
assert state != null || (ctx.jobUpdateLeader() && (info.type() == COMMIT || info.type() == ABORT)):
"Missing local state for finished task [info=" + info + ", status=" + status + ']';
StackedProcessor incrCntrs = null;
if (status.state() == COMPLETED)
incrCntrs = new IncrementCountersProcessor(null, status.counters());
switch (info.type()) {
case SETUP: {
state.onSetupFinished(info, status, incrCntrs);
break;
}
case MAP: {
state.onMapFinished(info, status, incrCntrs);
break;
}
case REDUCE: {
state.onReduceFinished(info, status, incrCntrs);
break;
}
case COMBINE: {
state.onCombineFinished(info, status, incrCntrs);
break;
}
case COMMIT:
case ABORT: {
GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> cache = finishedJobMetaCache();
cache.invokeAsync(info.jobId(), new UpdatePhaseProcessor(incrCntrs, PHASE_COMPLETE)).
listen(failsLog);
break;
}
}
}
finally {
busyLock.readUnlock();
}
}
/**
* @param jobId Job id.
* @param c Closure of operation.
*/
private void transform(HadoopJobId jobId, EntryProcessor<HadoopJobId, HadoopJobMetadata, Void> c) {
jobMetaCache().invokeAsync(jobId, c).listen(failsLog);
}
/**
* Callback from task executor called when process is ready to received shuffle messages.
*
* @param jobId Job ID.
* @param reducers Reducers.
* @param desc Process descriptor.
*/
public void onExternalMappersInitialized(HadoopJobId jobId, Collection<Integer> reducers,
HadoopProcessDescriptor desc) {
transform(jobId, new InitializeReducersProcessor(null, reducers, desc));
}
/**
* Gets all input splits for given hadoop map-reduce plan.
*
* @param plan Map-reduce plan.
* @return Collection of all input splits that should be processed.
*/
@SuppressWarnings("ConstantConditions")
private Map<HadoopInputSplit, Integer> allSplits(HadoopMapReducePlan plan) {
Map<HadoopInputSplit, Integer> res = new HashMap<>();
int taskNum = 0;
for (UUID nodeId : plan.mapperNodeIds()) {
for (HadoopInputSplit split : plan.mappers(nodeId)) {
if (res.put(split, taskNum++) != null)
throw new IllegalStateException("Split duplicate.");
}
}
return res;
}
/**
* Gets all reducers for this job.
*
* @param plan Map-reduce plan.
* @return Collection of reducers.
*/
private Collection<Integer> allReducers(HadoopMapReducePlan plan) {
Collection<Integer> res = new HashSet<>();
for (int i = 0; i < plan.reducers(); i++)
res.add(i);
return res;
}
/**
* Processes node leave (or fail) event.
*
* @param evt Discovery event.
*/
@SuppressWarnings("ConstantConditions")
private void processNodeLeft(DiscoveryEvent evt) {
if (log.isDebugEnabled())
log.debug("Processing discovery event [locNodeId=" + ctx.localNodeId() + ", evt=" + evt + ']');
// Check only if this node is responsible for job status updates.
if (ctx.jobUpdateLeader()) {
boolean checkSetup = evt.eventNode().order() < ctx.localNodeOrder();
// Iteration over all local entries is correct since system cache is REPLICATED.
for (Object metaObj : jobMetaCache().values()) {
HadoopJobMetadata meta = (HadoopJobMetadata)metaObj;
HadoopJobId jobId = meta.jobId();
HadoopMapReducePlan plan = meta.mapReducePlan();
HadoopJobPhase phase = meta.phase();
try {
if (checkSetup && phase == PHASE_SETUP && !activeJobs.containsKey(jobId)) {
// Failover setup task.
HadoopJob job = job(jobId, meta.jobInfo());
Collection<HadoopTaskInfo> setupTask = setupTask(jobId);
assert setupTask != null;
ctx.taskExecutor().run(job, setupTask);
}
else if (phase == PHASE_MAP || phase == PHASE_REDUCE) {
// Must check all nodes, even that are not event node ID due to
// multiple node failure possibility.
Collection<HadoopInputSplit> cancelSplits = null;
for (UUID nodeId : plan.mapperNodeIds()) {
if (ctx.kernalContext().discovery().node(nodeId) == null) {
// Node has left the grid.
Collection<HadoopInputSplit> mappers = plan.mappers(nodeId);
if (cancelSplits == null)
cancelSplits = new HashSet<>();
cancelSplits.addAll(mappers);
}
}
Collection<Integer> cancelReducers = null;
for (UUID nodeId : plan.reducerNodeIds()) {
if (ctx.kernalContext().discovery().node(nodeId) == null) {
// Node has left the grid.
int[] reducers = plan.reducers(nodeId);
if (cancelReducers == null)
cancelReducers = new HashSet<>();
for (int rdc : reducers)
cancelReducers.add(rdc);
}
}
if (cancelSplits != null || cancelReducers != null)
jobMetaCache().invoke(meta.jobId(), new CancelJobProcessor(null, new IgniteCheckedException(
"One or more nodes participating in map-reduce job execution failed."), cancelSplits,
cancelReducers));
}
}
catch (IgniteCheckedException e) {
U.error(log, "Failed to cancel job: " + meta, e);
}
}
}
}
/**
* @param updated Updated cache entries.
* @throws IgniteCheckedException If failed.
*/
private void processJobMetadataUpdates(
Iterable<CacheEntryEvent<? extends HadoopJobId, ? extends HadoopJobMetadata>> updated)
throws IgniteCheckedException {
UUID locNodeId = ctx.localNodeId();
for (CacheEntryEvent<? extends HadoopJobId, ? extends HadoopJobMetadata> entry : updated) {
HadoopJobId jobId = entry.getKey();
HadoopJobMetadata meta = entry.getValue();
if (meta == null || !ctx.isParticipating(meta))
continue;
if (log.isDebugEnabled())
log.debug("Processing job metadata update callback [locNodeId=" + locNodeId +
", meta=" + meta + ']');
try {
ctx.taskExecutor().onJobStateChanged(meta);
}
catch (IgniteCheckedException e) {
U.error(log, "Failed to process job state changed callback (will fail the job) " +
"[locNodeId=" + locNodeId + ", jobId=" + jobId + ", meta=" + meta + ']', e);
transform(jobId, new CancelJobProcessor(null, e));
continue;
}
processJobMetaUpdate(jobId, meta, locNodeId);
}
}
/**
* @param jobId Job ID.
* @param plan Map-reduce plan.
*/
private void printPlan(HadoopJobId jobId, HadoopMapReducePlan plan) {
log.info("Plan for " + jobId);
SB b = new SB();
b.a(" Map: ");
for (UUID nodeId : plan.mapperNodeIds())
b.a(nodeId).a("=").a(plan.mappers(nodeId).size()).a(' ');
log.info(b.toString());
b = new SB();
b.a(" Reduce: ");
for (UUID nodeId : plan.reducerNodeIds())
b.a(nodeId).a("=").a(Arrays.toString(plan.reducers(nodeId))).a(' ');
log.info(b.toString());
}
/**
* @param jobId Job ID.
* @param meta Job metadata.
* @param locNodeId Local node ID.
* @throws IgniteCheckedException If failed.
*/
private void processJobMetaUpdate(HadoopJobId jobId, HadoopJobMetadata meta, UUID locNodeId)
throws IgniteCheckedException {
JobLocalState state = activeJobs.get(jobId);
HadoopJob job = job(jobId, meta.jobInfo());
HadoopMapReducePlan plan = meta.mapReducePlan();
switch (meta.phase()) {
case PHASE_SETUP: {
if (ctx.jobUpdateLeader()) {
Collection<HadoopTaskInfo> setupTask = setupTask(jobId);
if (setupTask != null)
ctx.taskExecutor().run(job, setupTask);
}
break;
}
case PHASE_MAP: {
// Check if we should initiate new task on local node.
Collection<HadoopTaskInfo> tasks = mapperTasks(plan.mappers(locNodeId), meta);
if (tasks != null)
ctx.taskExecutor().run(job, tasks);
break;
}
case PHASE_REDUCE: {
if (meta.pendingReducers().isEmpty() && ctx.jobUpdateLeader()) {
HadoopTaskInfo info = new HadoopTaskInfo(COMMIT, jobId, 0, 0, null);
if (log.isDebugEnabled())
log.debug("Submitting COMMIT task for execution [locNodeId=" + locNodeId +
", jobId=" + jobId + ']');
ctx.taskExecutor().run(job, Collections.singletonList(info));
break;
}
Collection<HadoopTaskInfo> tasks = reducerTasks(plan.reducers(locNodeId), job);
if (tasks != null)
ctx.taskExecutor().run(job, tasks);
break;
}
case PHASE_CANCELLING: {
// Prevent multiple task executor notification.
if (state != null && state.onCancel()) {
if (log.isDebugEnabled())
log.debug("Cancelling local task execution for job: " + meta);
ctx.taskExecutor().cancelTasks(jobId);
}
if (meta.pendingSplits().isEmpty() && meta.pendingReducers().isEmpty()) {
if (ctx.jobUpdateLeader()) {
if (state == null)
state = initState(jobId);
// Prevent running multiple abort tasks.
if (state.onAborted()) {
HadoopTaskInfo info = new HadoopTaskInfo(ABORT, jobId, 0, 0, null);
if (log.isDebugEnabled())
log.debug("Submitting ABORT task for execution [locNodeId=" + locNodeId +
", jobId=" + jobId + ']');
ctx.taskExecutor().run(job, Collections.singletonList(info));
}
}
break;
}
else {
// Check if there are unscheduled mappers or reducers.
Collection<HadoopInputSplit> cancelMappers = new ArrayList<>();
Collection<Integer> cancelReducers = new ArrayList<>();
Collection<HadoopInputSplit> mappers = plan.mappers(ctx.localNodeId());
if (mappers != null) {
for (HadoopInputSplit b : mappers) {
if (state == null || !state.mapperScheduled(b))
cancelMappers.add(b);
}
}
int[] rdc = plan.reducers(ctx.localNodeId());
if (rdc != null) {
for (int r : rdc) {
if (state == null || !state.reducerScheduled(r))
cancelReducers.add(r);
}
}
if (!cancelMappers.isEmpty() || !cancelReducers.isEmpty())
transform(jobId, new CancelJobProcessor(null, cancelMappers, cancelReducers));
}
break;
}
case PHASE_COMPLETE: {
if (log.isDebugEnabled())
log.debug("Job execution is complete, will remove local state from active jobs " +
"[jobId=" + jobId + ", meta=" + meta + ']');
if (state != null) {
state = activeJobs.remove(jobId);
assert state != null;
ctx.shuffle().jobFinished(jobId);
}
GridFutureAdapter<HadoopJobId> finishFut = activeFinishFuts.remove(jobId);
if (finishFut != null) {
if (log.isDebugEnabled())
log.debug("Completing job future [locNodeId=" + locNodeId + ", meta=" + meta + ']');
finishFut.onDone(jobId, meta.failCause());
}
if (ctx.jobUpdateLeader())
job.cleanupStagingDirectory();
jobs.remove(jobId);
job.dispose(false);
if (ctx.jobUpdateLeader()) {
ClassLoader ldr = job.getClass().getClassLoader();
try {
String statWriterClsName = job.info().property(HadoopUtils.JOB_COUNTER_WRITER_PROPERTY);
if (statWriterClsName != null) {
Class<?> cls = ldr.loadClass(statWriterClsName);
HadoopCounterWriter writer = (HadoopCounterWriter)cls.newInstance();
HadoopCounters cntrs = meta.counters();
writer.write(job.info(), jobId, cntrs);
}
}
catch (Exception e) {
log.error("Can't write statistic due to: ", e);
}
}
break;
}
default:
throw new IllegalStateException("Unknown phase: " + meta.phase());
}
}
/**
* Creates setup task based on job information.
*
* @param jobId Job ID.
* @return Setup task wrapped in collection.
*/
@Nullable private Collection<HadoopTaskInfo> setupTask(HadoopJobId jobId) {
if (activeJobs.containsKey(jobId))
return null;
else {
initState(jobId);
return Collections.singleton(new HadoopTaskInfo(SETUP, jobId, 0, 0, null));
}
}
/**
* Creates mapper tasks based on job information.
*
* @param mappers Mapper blocks.
* @param meta Job metadata.
* @return Collection of created task infos or {@code null} if no mapper tasks scheduled for local node.
*/
private Collection<HadoopTaskInfo> mapperTasks(Iterable<HadoopInputSplit> mappers, HadoopJobMetadata meta) {
UUID locNodeId = ctx.localNodeId();
HadoopJobId jobId = meta.jobId();
JobLocalState state = activeJobs.get(jobId);
Collection<HadoopTaskInfo> tasks = null;
if (mappers != null) {
if (state == null)
state = initState(jobId);
for (HadoopInputSplit split : mappers) {
if (state.addMapper(split)) {
if (log.isDebugEnabled())
log.debug("Submitting MAP task for execution [locNodeId=" + locNodeId +
", split=" + split + ']');
HadoopTaskInfo taskInfo = new HadoopTaskInfo(MAP, jobId, meta.taskNumber(split), 0, split);
if (tasks == null)
tasks = new ArrayList<>();
tasks.add(taskInfo);
}
}
}
return tasks;
}
/**
* Creates reducer tasks based on job information.
*
* @param reducers Reducers (may be {@code null}).
* @param job Job instance.
* @return Collection of task infos.
*/
private Collection<HadoopTaskInfo> reducerTasks(int[] reducers, HadoopJob job) {
UUID locNodeId = ctx.localNodeId();
HadoopJobId jobId = job.id();
JobLocalState state = activeJobs.get(jobId);
Collection<HadoopTaskInfo> tasks = null;
if (reducers != null) {
if (state == null)
state = initState(job.id());
for (int rdc : reducers) {
if (state.addReducer(rdc)) {
if (log.isDebugEnabled())
log.debug("Submitting REDUCE task for execution [locNodeId=" + locNodeId +
", rdc=" + rdc + ']');
HadoopTaskInfo taskInfo = new HadoopTaskInfo(REDUCE, jobId, rdc, 0, null);
if (tasks == null)
tasks = new ArrayList<>();
tasks.add(taskInfo);
}
}
}
return tasks;
}
/**
* Initializes local state for given job metadata.
*
* @param jobId Job ID.
* @return Local state.
*/
private JobLocalState initState(HadoopJobId jobId) {
return F.addIfAbsent(activeJobs, jobId, new JobLocalState());
}
/**
* Gets or creates job instance.
*
* @param jobId Job ID.
* @param jobInfo Job info.
* @return Job.
* @throws IgniteCheckedException If failed.
*/
@Nullable public HadoopJob job(HadoopJobId jobId, @Nullable HadoopJobInfo jobInfo) throws IgniteCheckedException {
GridFutureAdapter<HadoopJob> fut = jobs.get(jobId);
if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapter<HadoopJob>())) != null)
return fut.get();
fut = jobs.get(jobId);
HadoopJob job = null;
try {
if (jobInfo == null) {
HadoopJobMetadata meta = jobMetaCache().get(jobId);
if (meta == null)
throw new IgniteCheckedException("Failed to find job metadata for ID: " + jobId);
jobInfo = meta.jobInfo();
}
job = jobInfo.createJob(jobId, log);
job.initialize(false, ctx.localNodeId());
fut.onDone(job);
return job;
}
catch (IgniteCheckedException e) {
fut.onDone(e);