|
@@ -25,11 +25,15 @@ import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.mapred.ClusterStatus;
|
|
import org.apache.hadoop.mapred.ClusterStatus;
|
|
import org.apache.hadoop.mapred.gridmix.Statistics.ClusterStats;
|
|
import org.apache.hadoop.mapred.gridmix.Statistics.ClusterStats;
|
|
import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
|
|
import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
|
|
|
|
+import org.apache.hadoop.mapreduce.JobID;
|
|
|
|
+import org.apache.hadoop.mapreduce.JobStatus;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.tools.rumen.JobStory;
|
|
import org.apache.hadoop.tools.rumen.JobStory;
|
|
import org.apache.hadoop.tools.rumen.JobStoryProducer;
|
|
import org.apache.hadoop.tools.rumen.JobStoryProducer;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.util.HashSet;
|
|
|
|
+import java.util.Set;
|
|
import java.util.concurrent.CountDownLatch;
|
|
import java.util.concurrent.CountDownLatch;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
@@ -87,6 +91,13 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
"gridmix.throttle.jobs-to-tracker-ratio";
|
|
"gridmix.throttle.jobs-to-tracker-ratio";
|
|
final float maxJobTrackerRatio;
|
|
final float maxJobTrackerRatio;
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Represents a list of blacklisted jobs. Jobs are blacklisted when either
|
|
|
|
+ * they are complete or their status cannot be obtained. Stress mode will
|
|
|
|
+ * ignore blacklisted jobs from its overload computation.
|
|
|
|
+ */
|
|
|
|
+ private Set<JobID> blacklistedJobs = new HashSet<JobID>();
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Creating a new instance does not start the thread.
|
|
* Creating a new instance does not start the thread.
|
|
*
|
|
*
|
|
@@ -145,42 +156,66 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
try {
|
|
try {
|
|
startFlag.await();
|
|
startFlag.await();
|
|
if (Thread.currentThread().isInterrupted()) {
|
|
if (Thread.currentThread().isInterrupted()) {
|
|
|
|
+ LOG.warn("[STRESS] Interrupted before start!. Exiting..");
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
LOG.info("START STRESS @ " + System.currentTimeMillis());
|
|
LOG.info("START STRESS @ " + System.currentTimeMillis());
|
|
while (!Thread.currentThread().isInterrupted()) {
|
|
while (!Thread.currentThread().isInterrupted()) {
|
|
try {
|
|
try {
|
|
while (loadStatus.overloaded()) {
|
|
while (loadStatus.overloaded()) {
|
|
|
|
+ // update the overload status
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
- LOG.debug("Cluster overloaded in run! Sleeping...");
|
|
|
|
|
|
+ LOG.debug("Updating the overload status.");
|
|
}
|
|
}
|
|
- // sleep
|
|
|
|
try {
|
|
try {
|
|
- Thread.sleep(1000);
|
|
|
|
- } catch (InterruptedException ie) {
|
|
|
|
|
|
+ checkLoadAndGetSlotsToBackfill();
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ LOG.warn("[STRESS] Check failed!", ioe);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // if the cluster is still overloaded, then sleep
|
|
|
|
+ if (loadStatus.overloaded()) {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("[STRESS] Cluster overloaded in run! Sleeping...");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // sleep
|
|
|
|
+ try {
|
|
|
|
+ Thread.sleep(1000);
|
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
|
+ LOG.warn("[STRESS] Interrupted while sleeping! Exiting.", ie);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
while (!loadStatus.overloaded()) {
|
|
while (!loadStatus.overloaded()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
- LOG.debug("Cluster underloaded in run! Stressing...");
|
|
|
|
|
|
+ LOG.debug("[STRESS] Cluster underloaded in run! Stressing...");
|
|
}
|
|
}
|
|
try {
|
|
try {
|
|
//TODO This in-line read can block submission for large jobs.
|
|
//TODO This in-line read can block submission for large jobs.
|
|
final JobStory job = getNextJobFiltered();
|
|
final JobStory job = getNextJobFiltered();
|
|
if (null == job) {
|
|
if (null == job) {
|
|
|
|
+ LOG.warn("[STRESS] Finished consuming the input trace. "
|
|
|
|
+ + "Exiting..");
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Job Selected: " + job.getJobID());
|
|
LOG.debug("Job Selected: " + job.getJobID());
|
|
}
|
|
}
|
|
- submitter.add(
|
|
|
|
- jobCreator.createGridmixJob(
|
|
|
|
- conf, 0L, job, scratch,
|
|
|
|
- userResolver.getTargetUgi(
|
|
|
|
- UserGroupInformation.createRemoteUser(job.getUser())),
|
|
|
|
- sequence.getAndIncrement()));
|
|
|
|
|
|
+
|
|
|
|
+ UserGroupInformation ugi =
|
|
|
|
+ UserGroupInformation.createRemoteUser(job.getUser());
|
|
|
|
+ UserGroupInformation tgtUgi = userResolver.getTargetUgi(ugi);
|
|
|
|
+ GridmixJob tJob =
|
|
|
|
+ jobCreator.createGridmixJob(conf, 0L, job, scratch,
|
|
|
|
+ tgtUgi, sequence.getAndIncrement());
|
|
|
|
+
|
|
|
|
+ // submit the job
|
|
|
|
+ submitter.add(tJob);
|
|
|
|
+
|
|
// TODO: We need to take care of scenario when one map/reduce
|
|
// TODO: We need to take care of scenario when one map/reduce
|
|
// takes more than 1 slot.
|
|
// takes more than 1 slot.
|
|
|
|
|
|
@@ -198,7 +233,7 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
|
|
|
|
loadStatus.decrementJobLoad(1);
|
|
loadStatus.decrementJobLoad(1);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
- LOG.error("Error while submitting the job ", e);
|
|
|
|
|
|
+ LOG.error("[STRESS] Error while submitting the job ", e);
|
|
error = e;
|
|
error = e;
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
@@ -209,6 +244,7 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
|
|
+ LOG.error("[STRESS] Interrupted in the main block!", e);
|
|
return;
|
|
return;
|
|
} finally {
|
|
} finally {
|
|
IOUtils.cleanup(null, jobProducer);
|
|
IOUtils.cleanup(null, jobProducer);
|
|
@@ -224,9 +260,17 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void update(Statistics.ClusterStats item) {
|
|
public void update(Statistics.ClusterStats item) {
|
|
- ClusterStatus clusterMetrics = item.getStatus();
|
|
|
|
|
|
+ ClusterStatus clusterStatus = item.getStatus();
|
|
try {
|
|
try {
|
|
- checkLoadAndGetSlotsToBackfill(item, clusterMetrics);
|
|
|
|
|
|
+ // update the max cluster map/reduce task capacity
|
|
|
|
+ loadStatus.updateMapCapacity(clusterStatus.getMaxMapTasks());
|
|
|
|
+
|
|
|
|
+ loadStatus.updateReduceCapacity(clusterStatus.getMaxReduceTasks());
|
|
|
|
+
|
|
|
|
+ int numTrackers = clusterStatus.getTaskTrackers();
|
|
|
|
+ int jobLoad =
|
|
|
|
+ (int) (maxJobTrackerRatio * numTrackers) - item.getNumRunningJob();
|
|
|
|
+ loadStatus.updateJobLoad(jobLoad);
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
LOG.error("Couldn't get the new Status",e);
|
|
LOG.error("Couldn't get the new Status",e);
|
|
}
|
|
}
|
|
@@ -258,22 +302,8 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
* @param clusterStatus Cluster status
|
|
* @param clusterStatus Cluster status
|
|
* @throws java.io.IOException
|
|
* @throws java.io.IOException
|
|
*/
|
|
*/
|
|
- private void checkLoadAndGetSlotsToBackfill(
|
|
|
|
- ClusterStats stats, ClusterStatus clusterStatus) throws IOException, InterruptedException {
|
|
|
|
-
|
|
|
|
- // update the max cluster capacity incase its updated
|
|
|
|
- int mapCapacity = clusterStatus.getMaxMapTasks();
|
|
|
|
- loadStatus.updateMapCapacity(mapCapacity);
|
|
|
|
-
|
|
|
|
- int reduceCapacity = clusterStatus.getMaxReduceTasks();
|
|
|
|
-
|
|
|
|
- loadStatus.updateReduceCapacity(reduceCapacity);
|
|
|
|
-
|
|
|
|
- int numTrackers = clusterStatus.getTaskTrackers();
|
|
|
|
-
|
|
|
|
- int jobLoad =
|
|
|
|
- (int) (maxJobTrackerRatio * numTrackers) - stats.getNumRunningJob();
|
|
|
|
- loadStatus.updateJobLoad(jobLoad);
|
|
|
|
|
|
+ protected void checkLoadAndGetSlotsToBackfill()
|
|
|
|
+ throws IOException, InterruptedException {
|
|
if (loadStatus.getJobLoad() <= 0) {
|
|
if (loadStatus.getJobLoad() <= 0) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug(System.currentTimeMillis() + " [JobLoad] Overloaded is "
|
|
LOG.debug(System.currentTimeMillis() + " [JobLoad] Overloaded is "
|
|
@@ -283,17 +313,143 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
return; // stop calculation because we know it is overloaded.
|
|
return; // stop calculation because we know it is overloaded.
|
|
}
|
|
}
|
|
|
|
|
|
- float incompleteMapTasks = 0; // include pending & running map tasks.
|
|
|
|
- for (JobStats job : ClusterStats.getRunningJobStats()) {
|
|
|
|
- float mapProgress = job.getJob().mapProgress();
|
|
|
|
- int noOfMaps = job.getNoOfMaps();
|
|
|
|
- incompleteMapTasks +=
|
|
|
|
- calcEffectiveIncompleteMapTasks(mapCapacity, noOfMaps, mapProgress);
|
|
|
|
|
|
+ int mapCapacity = loadStatus.getMapCapacity();
|
|
|
|
+ int reduceCapacity = loadStatus.getReduceCapacity();
|
|
|
|
+
|
|
|
|
+ // return if the cluster status is not set
|
|
|
|
+ if (mapCapacity < 0 || reduceCapacity < 0) {
|
|
|
|
+ // note that, by default, the overload status is true
|
|
|
|
+ // missing cluster status will result into blocking of job submission
|
|
|
|
+ return;
|
|
}
|
|
}
|
|
|
|
|
|
- int mapSlotsBackFill =
|
|
|
|
- (int) ((overloadMapTaskMapSlotRatio * mapCapacity) - incompleteMapTasks);
|
|
|
|
- loadStatus.updateMapLoad(mapSlotsBackFill);
|
|
|
|
|
|
+ // Determine the max permissible map & reduce task load
|
|
|
|
+ int maxMapLoad = (int) (overloadMapTaskMapSlotRatio * mapCapacity);
|
|
|
|
+ int maxReduceLoad =
|
|
|
|
+ (int) (overloadReduceTaskReduceSlotRatio * reduceCapacity);
|
|
|
|
+
|
|
|
|
+ // compute the total number of map & reduce tasks submitted
|
|
|
|
+ int totalMapTasks = ClusterStats.getSubmittedMapTasks();
|
|
|
|
+ int totalReduceTasks = ClusterStats.getSubmittedReduceTasks();
|
|
|
|
+
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Total submitted map tasks: " + totalMapTasks);
|
|
|
|
+ LOG.debug("Total submitted reduce tasks: " + totalReduceTasks);
|
|
|
|
+ LOG.debug("Max map load: " + maxMapLoad);
|
|
|
|
+ LOG.debug("Max reduce load: " + maxReduceLoad);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // generate a pessimistic bound on the max running+pending map tasks
|
|
|
|
+ // this check is to avoid the heavy-duty actual map load calculation
|
|
|
|
+ int mapSlotsBackFill = (int) (maxMapLoad - totalMapTasks);
|
|
|
|
+
|
|
|
|
+ // generate a pessimistic bound on the max running+pending reduce tasks
|
|
|
|
+ // this check is to avoid the heavy-duty actual reduce load calculation
|
|
|
|
+ int reduceSlotsBackFill = (int) (maxReduceLoad - totalReduceTasks);
|
|
|
|
+
|
|
|
|
+ // maintain a list of seen job ids
|
|
|
|
+ Set<JobID> seenJobIDs = new HashSet<JobID>();
|
|
|
|
+
|
|
|
|
+ // check if the total number of submitted map/reduce tasks exceeds the
|
|
|
|
+ // permissible limit
|
|
|
|
+ if (totalMapTasks > maxMapLoad || totalReduceTasks > maxReduceLoad) {
|
|
|
|
+ // if yes, calculate the real load
|
|
|
|
+ float incompleteMapTasks = 0; // include pending & running map tasks.
|
|
|
|
+ float incompleteReduceTasks = 0; // include pending & running reduce tasks
|
|
|
|
+
|
|
|
|
+ for (JobStats job : ClusterStats.getRunningJobStats()) {
|
|
|
|
+ JobID id = job.getJob().getJobID();
|
|
|
|
+ seenJobIDs.add(id);
|
|
|
|
+
|
|
|
|
+ // Note that this is a hack! Ideally, ClusterStats.getRunningJobStats()
|
|
|
|
+ // should be smart enough to take care of completed jobs.
|
|
|
|
+ if (blacklistedJobs.contains(id)) {
|
|
|
|
+ LOG.warn("Ignoring blacklisted job: " + id);
|
|
|
|
+ continue;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ int noOfMaps = job.getNoOfMaps();
|
|
|
|
+ int noOfReduces = job.getNoOfReds();
|
|
|
|
+
|
|
|
|
+ // consider polling for jobs where maps>0 and reds>0
|
|
|
|
+ // TODO: What about setup/cleanup tasks for cases where m=0 and r=0
|
|
|
|
+ // What otherwise?
|
|
|
|
+ if (noOfMaps > 0 || noOfReduces > 0) {
|
|
|
|
+ // get the job's status
|
|
|
|
+ JobStatus status = job.getJobStatus();
|
|
|
|
+
|
|
|
|
+ // blacklist completed jobs and continue
|
|
|
|
+ if (status != null && status.isJobComplete()) {
|
|
|
|
+ LOG.warn("Blacklisting completed job: " + id);
|
|
|
|
+ blacklistedJobs.add(id);
|
|
|
|
+ continue;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // get the map and reduce tasks' progress
|
|
|
|
+ float mapProgress = 0f;
|
|
|
|
+ float reduceProgress = 0f;
|
|
|
|
+
|
|
|
|
+ // check if the status is missing (this can happen for unpolled jobs)
|
|
|
|
+ if (status != null) {
|
|
|
|
+ mapProgress = status.getMapProgress();
|
|
|
|
+ reduceProgress = status.getReduceProgress();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ incompleteMapTasks +=
|
|
|
|
+ calcEffectiveIncompleteMapTasks(mapCapacity, noOfMaps, mapProgress);
|
|
|
|
+
|
|
|
|
+ // bail out early
|
|
|
|
+ int currentMapSlotsBackFill = (int) (maxMapLoad - incompleteMapTasks);
|
|
|
|
+ if (currentMapSlotsBackFill <= 0) {
|
|
|
|
+ // reset the reduce task load since we are bailing out
|
|
|
|
+ incompleteReduceTasks = totalReduceTasks;
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Terminating overload check due to high map load.");
|
|
|
|
+ }
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // compute the real reduce load
|
|
|
|
+ if (noOfReduces > 0) {
|
|
|
|
+ incompleteReduceTasks +=
|
|
|
|
+ calcEffectiveIncompleteReduceTasks(reduceCapacity, noOfReduces,
|
|
|
|
+ reduceProgress);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // bail out early
|
|
|
|
+ int currentReduceSlotsBackFill =
|
|
|
|
+ (int) (maxReduceLoad - incompleteReduceTasks);
|
|
|
|
+ if (currentReduceSlotsBackFill <= 0) {
|
|
|
|
+ // reset the map task load since we are bailing out
|
|
|
|
+ incompleteMapTasks = totalMapTasks;
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Terminating overload check due to high reduce load.");
|
|
|
|
+ }
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ LOG.warn("Blacklisting empty job: " + id);
|
|
|
|
+ blacklistedJobs.add(id);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // calculate the real map load on the cluster
|
|
|
|
+ mapSlotsBackFill = (int) (maxMapLoad - incompleteMapTasks);
|
|
|
|
+
|
|
|
|
+ // calculate the real reduce load on the cluster
|
|
|
|
+ reduceSlotsBackFill = (int)(maxReduceLoad - incompleteReduceTasks);
|
|
|
|
+
|
|
|
|
+ // clean up the backlisted set to keep the memory footprint minimal
|
|
|
|
+ // retain only the jobs that are seen in this cycle
|
|
|
|
+ blacklistedJobs.retainAll(seenJobIDs);
|
|
|
|
+ if (LOG.isDebugEnabled() && blacklistedJobs.size() > 0) {
|
|
|
|
+ LOG.debug("Blacklisted jobs count: " + blacklistedJobs.size());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // update
|
|
|
|
+ loadStatus.updateMapLoad(mapSlotsBackFill);
|
|
|
|
+ loadStatus.updateReduceLoad(reduceSlotsBackFill);
|
|
|
|
|
|
if (loadStatus.getMapLoad() <= 0) {
|
|
if (loadStatus.getMapLoad() <= 0) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -303,23 +459,7 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
}
|
|
}
|
|
return; // stop calculation because we know it is overloaded.
|
|
return; // stop calculation because we know it is overloaded.
|
|
}
|
|
}
|
|
-
|
|
|
|
- float incompleteReduceTasks = 0; // include pending & running reduce tasks.
|
|
|
|
- for (JobStats job : ClusterStats.getRunningJobStats()) {
|
|
|
|
- // Cached the num-reds value in JobStats
|
|
|
|
- int noOfReduces = job.getNoOfReds();
|
|
|
|
- if (noOfReduces > 0) {
|
|
|
|
- float reduceProgress = job.getJob().reduceProgress();
|
|
|
|
- incompleteReduceTasks +=
|
|
|
|
- calcEffectiveIncompleteReduceTasks(reduceCapacity, noOfReduces,
|
|
|
|
- reduceProgress);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
- int reduceSlotsBackFill =
|
|
|
|
- (int)((overloadReduceTaskReduceSlotRatio * reduceCapacity)
|
|
|
|
- - incompleteReduceTasks);
|
|
|
|
- loadStatus.updateReduceLoad(reduceSlotsBackFill);
|
|
|
|
if (loadStatus.getReduceLoad() <= 0) {
|
|
if (loadStatus.getReduceLoad() <= 0) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug(System.currentTimeMillis() + " [REDUCE-LOAD] Overloaded is "
|
|
LOG.debug(System.currentTimeMillis() + " [REDUCE-LOAD] Overloaded is "
|
|
@@ -445,7 +585,7 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|
|| (numJobsBackfill <= 0));
|
|
|| (numJobsBackfill <= 0));
|
|
}
|
|
}
|
|
|
|
|
|
- public synchronized boolean overloaded() {
|
|
|
|
|
|
+ public boolean overloaded() {
|
|
return overloaded.get();
|
|
return overloaded.get();
|
|
}
|
|
}
|
|
|
|
|