|
@@ -191,6 +191,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
// system files should have 700 permission
|
|
// system files should have 700 permission
|
|
final static FsPermission SYSTEM_FILE_PERMISSION =
|
|
final static FsPermission SYSTEM_FILE_PERMISSION =
|
|
FsPermission.createImmutable((short) 0700); // rwx------
|
|
FsPermission.createImmutable((short) 0700); // rwx------
|
|
|
|
+
|
|
|
|
+ private Clock clock;
|
|
|
|
|
|
private TokenStorage tokenStorage;
|
|
private TokenStorage tokenStorage;
|
|
private final JobTokenSecretManager jobTokenSecretManager
|
|
private final JobTokenSecretManager jobTokenSecretManager
|
|
@@ -225,6 +227,10 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
private int nextJobId = 1;
|
|
private int nextJobId = 1;
|
|
|
|
|
|
public static final Log LOG = LogFactory.getLog(JobTracker.class);
|
|
public static final Log LOG = LogFactory.getLog(JobTracker.class);
|
|
|
|
+
|
|
|
|
+ public Clock getClock() {
|
|
|
|
+ return clock;
|
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
/**
|
|
* Start the JobTracker with given configuration.
|
|
* Start the JobTracker with given configuration.
|
|
@@ -314,7 +320,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
try {
|
|
try {
|
|
// Every 3 minutes check for any tasks that are overdue
|
|
// Every 3 minutes check for any tasks that are overdue
|
|
Thread.sleep(TASKTRACKER_EXPIRY_INTERVAL/3);
|
|
Thread.sleep(TASKTRACKER_EXPIRY_INTERVAL/3);
|
|
- long now = System.currentTimeMillis();
|
|
|
|
|
|
+ long now = clock.getTime();
|
|
LOG.debug("Starting launching task sweep");
|
|
LOG.debug("Starting launching task sweep");
|
|
synchronized (JobTracker.this) {
|
|
synchronized (JobTracker.this) {
|
|
synchronized (launchingTasks) {
|
|
synchronized (launchingTasks) {
|
|
@@ -368,7 +374,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
public void addNewTask(TaskAttemptID taskName) {
|
|
public void addNewTask(TaskAttemptID taskName) {
|
|
synchronized (launchingTasks) {
|
|
synchronized (launchingTasks) {
|
|
launchingTasks.put(taskName,
|
|
launchingTasks.put(taskName,
|
|
- System.currentTimeMillis());
|
|
|
|
|
|
+ clock.getTime());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -412,7 +418,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
synchronized (JobTracker.this) {
|
|
synchronized (JobTracker.this) {
|
|
synchronized (taskTrackers) {
|
|
synchronized (taskTrackers) {
|
|
synchronized (trackerExpiryQueue) {
|
|
synchronized (trackerExpiryQueue) {
|
|
- long now = System.currentTimeMillis();
|
|
|
|
|
|
+ long now = clock.getTime();
|
|
TaskTrackerStatus leastRecent = null;
|
|
TaskTrackerStatus leastRecent = null;
|
|
while ((trackerExpiryQueue.size() > 0) &&
|
|
while ((trackerExpiryQueue.size() > 0) &&
|
|
(leastRecent = trackerExpiryQueue.first()) != null &&
|
|
(leastRecent = trackerExpiryQueue.first()) != null &&
|
|
@@ -548,7 +554,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
try {
|
|
try {
|
|
Thread.sleep(RETIRE_JOB_CHECK_INTERVAL);
|
|
Thread.sleep(RETIRE_JOB_CHECK_INTERVAL);
|
|
List<JobInProgress> retiredJobs = new ArrayList<JobInProgress>();
|
|
List<JobInProgress> retiredJobs = new ArrayList<JobInProgress>();
|
|
- long now = System.currentTimeMillis();
|
|
|
|
|
|
+ long now = clock.getTime();
|
|
long retireBefore = now - RETIRE_JOB_INTERVAL;
|
|
long retireBefore = now - RETIRE_JOB_INTERVAL;
|
|
|
|
|
|
synchronized (jobs) {
|
|
synchronized (jobs) {
|
|
@@ -636,9 +642,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
private boolean isHealthy;
|
|
private boolean isHealthy;
|
|
private HashMap<ReasonForBlackListing, String>rfbMap;
|
|
private HashMap<ReasonForBlackListing, String>rfbMap;
|
|
|
|
|
|
- FaultInfo() {
|
|
|
|
|
|
+ FaultInfo(long time) {
|
|
numFaults = 0;
|
|
numFaults = 0;
|
|
- lastUpdated = System.currentTimeMillis();
|
|
|
|
|
|
+ lastUpdated = time;
|
|
blacklisted = false;
|
|
blacklisted = false;
|
|
rfbMap = new HashMap<ReasonForBlackListing, String>();
|
|
rfbMap = new HashMap<ReasonForBlackListing, String>();
|
|
}
|
|
}
|
|
@@ -730,7 +736,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
int numFaults = fi.getFaultCount();
|
|
int numFaults = fi.getFaultCount();
|
|
++numFaults;
|
|
++numFaults;
|
|
fi.setFaultCount(numFaults);
|
|
fi.setFaultCount(numFaults);
|
|
- fi.setLastUpdated(System.currentTimeMillis());
|
|
|
|
|
|
+ fi.setLastUpdated(clock.getTime());
|
|
if (exceedsFaults(fi)) {
|
|
if (exceedsFaults(fi)) {
|
|
LOG.info("Adding " + hostName + " to the blacklist"
|
|
LOG.info("Adding " + hostName + " to the blacklist"
|
|
+ " across all jobs");
|
|
+ " across all jobs");
|
|
@@ -814,7 +820,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
boolean createIfNeccessary) {
|
|
boolean createIfNeccessary) {
|
|
FaultInfo fi = potentiallyFaultyTrackers.get(hostName);
|
|
FaultInfo fi = potentiallyFaultyTrackers.get(hostName);
|
|
if (fi == null && createIfNeccessary) {
|
|
if (fi == null && createIfNeccessary) {
|
|
- fi = new FaultInfo();
|
|
|
|
|
|
+ fi = new FaultInfo(clock.getTime());
|
|
potentiallyFaultyTrackers.put(hostName, fi);
|
|
potentiallyFaultyTrackers.put(hostName, fi);
|
|
}
|
|
}
|
|
return fi;
|
|
return fi;
|
|
@@ -1362,7 +1368,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
TaskTrackerStatus ttStatus =
|
|
TaskTrackerStatus ttStatus =
|
|
new TaskTrackerStatus(trackerName, trackerHostName, port, ttStatusList,
|
|
new TaskTrackerStatus(trackerName, trackerHostName, port, ttStatusList,
|
|
0 , 0, 0);
|
|
0 , 0, 0);
|
|
- ttStatus.setLastSeen(System.currentTimeMillis());
|
|
|
|
|
|
+ ttStatus.setLastSeen(clock.getTime());
|
|
|
|
|
|
synchronized (JobTracker.this) {
|
|
synchronized (JobTracker.this) {
|
|
synchronized (taskTrackers) {
|
|
synchronized (taskTrackers) {
|
|
@@ -1688,7 +1694,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- long recoveryStartTime = System.currentTimeMillis();
|
|
|
|
|
|
+ long recoveryStartTime = clock.getTime();
|
|
|
|
|
|
// II. Recover each job
|
|
// II. Recover each job
|
|
idIter = jobsToRecover.iterator();
|
|
idIter = jobsToRecover.iterator();
|
|
@@ -1745,14 +1751,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- recoveryDuration = System.currentTimeMillis() - recoveryStartTime;
|
|
|
|
|
|
+ recoveryDuration = clock.getTime() - recoveryStartTime;
|
|
hasRecovered = true;
|
|
hasRecovered = true;
|
|
|
|
|
|
// III. Finalize the recovery
|
|
// III. Finalize the recovery
|
|
synchronized (trackerExpiryQueue) {
|
|
synchronized (trackerExpiryQueue) {
|
|
// Make sure that the tracker statuses in the expiry-tracker queue
|
|
// Make sure that the tracker statuses in the expiry-tracker queue
|
|
// are updated
|
|
// are updated
|
|
- long now = System.currentTimeMillis();
|
|
|
|
|
|
+ long now = clock.getTime();
|
|
int size = trackerExpiryQueue.size();
|
|
int size = trackerExpiryQueue.size();
|
|
for (int i = 0; i < size ; ++i) {
|
|
for (int i = 0; i < size ; ++i) {
|
|
// Get the first tasktracker
|
|
// Get the first tasktracker
|
|
@@ -1947,6 +1953,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
|
|
|
|
JobTracker(final JobConf conf, String identifier)
|
|
JobTracker(final JobConf conf, String identifier)
|
|
throws IOException, InterruptedException {
|
|
throws IOException, InterruptedException {
|
|
|
|
+ this(conf, identifier, new Clock());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ JobTracker(final JobConf conf, String identifier, Clock clock)
|
|
|
|
+ throws IOException, InterruptedException {
|
|
|
|
+ this.clock = clock;
|
|
// find the owner of the process
|
|
// find the owner of the process
|
|
// get the desired principal to load
|
|
// get the desired principal to load
|
|
String keytabFilename = conf.get(JT_KEYTAB_FILE);
|
|
String keytabFilename = conf.get(JT_KEYTAB_FILE);
|
|
@@ -2064,7 +2076,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
|
|
InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
|
|
String infoBindAddress = infoSocAddr.getHostName();
|
|
String infoBindAddress = infoSocAddr.getHostName();
|
|
int tmpInfoPort = infoSocAddr.getPort();
|
|
int tmpInfoPort = infoSocAddr.getPort();
|
|
- this.startTime = System.currentTimeMillis();
|
|
|
|
|
|
+ this.startTime = clock.getTime();
|
|
infoServer = new HttpServer("job", infoBindAddress, tmpInfoPort,
|
|
infoServer = new HttpServer("job", infoBindAddress, tmpInfoPort,
|
|
tmpInfoPort == 0, conf);
|
|
tmpInfoPort == 0, conf);
|
|
infoServer.setAttribute("job.tracker", this);
|
|
infoServer.setAttribute("job.tracker", this);
|
|
@@ -2598,7 +2610,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
final JobTrackerInstrumentation metrics = getInstrumentation();
|
|
final JobTrackerInstrumentation metrics = getInstrumentation();
|
|
metrics.finalizeJob(conf, id);
|
|
metrics.finalizeJob(conf, id);
|
|
|
|
|
|
- long now = System.currentTimeMillis();
|
|
|
|
|
|
+ long now = clock.getTime();
|
|
|
|
|
|
// mark the job for cleanup at all the trackers
|
|
// mark the job for cleanup at all the trackers
|
|
addJobForCleanup(id);
|
|
addJobForCleanup(id);
|
|
@@ -2979,7 +2991,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
|
|
|
|
|
|
// First check if the last heartbeat response got through
|
|
// First check if the last heartbeat response got through
|
|
String trackerName = status.getTrackerName();
|
|
String trackerName = status.getTrackerName();
|
|
- long now = System.currentTimeMillis();
|
|
|
|
|
|
+ long now = clock.getTime();
|
|
boolean isBlacklisted = false;
|
|
boolean isBlacklisted = false;
|
|
if (restarted) {
|
|
if (restarted) {
|
|
faultyTrackers.markTrackerHealthy(status.getHost());
|
|
faultyTrackers.markTrackerHealthy(status.getHost());
|