|
@@ -146,6 +146,7 @@ public class GridmixJobVerification {
|
|
verifyDistributeCache(zombieJob,simuJobConf);
|
|
verifyDistributeCache(zombieJob,simuJobConf);
|
|
setJobDistributedCacheInfo(simuJobId.toString(), simuJobConf,
|
|
setJobDistributedCacheInfo(simuJobId.toString(), simuJobConf,
|
|
zombieJob.getJobConf());
|
|
zombieJob.getJobConf());
|
|
|
|
+ verifyHighRamMemoryJobs(zombieJob, simuJobConf);
|
|
LOG.info("Done.");
|
|
LOG.info("Done.");
|
|
}
|
|
}
|
|
verifyDistributedCacheBetweenJobs(simuAndOrigJobsInfo);
|
|
verifyDistributedCacheBetweenJobs(simuAndOrigJobsInfo);
|
|
@@ -740,6 +741,164 @@ public class GridmixJobVerification {
|
|
return occursList;
|
|
return occursList;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * It verifies the high ram gridmix jobs.
|
|
|
|
+ * @param zombieJob - Original job story.
|
|
|
|
+ * @param simuJobConf - Simulated job configuration.
|
|
|
|
+ */
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
|
+ public void verifyHighRamMemoryJobs(ZombieJob zombieJob,
|
|
|
|
+ JobConf simuJobConf) {
|
|
|
|
+ JobConf origJobConf = zombieJob.getJobConf();
|
|
|
|
+ int origMapFactor = getMapFactor(origJobConf);
|
|
|
|
+ int origReduceFactor = getReduceFactor(origJobConf);
|
|
|
|
+ boolean isHighRamEnable =
|
|
|
|
+ simuJobConf.getBoolean(GridMixConfig.GRIDMIX_HIGH_RAM_JOB_ENABLE,
|
|
|
|
+ false);
|
|
|
|
+ if (isHighRamEnable) {
|
|
|
|
+ if (origMapFactor >= 2 && origReduceFactor >= 2) {
|
|
|
|
+ assertGridMixHighRamJob(simuJobConf, origJobConf, 1);
|
|
|
|
+ } else if(origMapFactor >= 2) {
|
|
|
|
+ assertGridMixHighRamJob(simuJobConf, origJobConf, 2);
|
|
|
|
+ } else if(origReduceFactor >= 2) {
|
|
|
|
+ assertGridMixHighRamJob(simuJobConf, origJobConf, 3);
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ if (origMapFactor >= 2 && origReduceFactor >= 2) {
|
|
|
|
+ assertGridMixHighRamJob(simuJobConf, origJobConf, 4);
|
|
|
|
+ } else if(origMapFactor >= 2) {
|
|
|
|
+ assertGridMixHighRamJob(simuJobConf, origJobConf, 5);
|
|
|
|
+ } else if(origReduceFactor >= 2) {
|
|
|
|
+ assertGridMixHighRamJob(simuJobConf, origJobConf, 6);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the value for identifying the slots used by the map.
|
|
|
|
+ * @param jobConf - job configuration
|
|
|
|
+ * @return - map factor value.
|
|
|
|
+ */
|
|
|
|
+ public static int getMapFactor(Configuration jobConf) {
|
|
|
|
+ long clusterMapMem =
|
|
|
|
+ Long.parseLong(jobConf.get(GridMixConfig.CLUSTER_MAP_MEMORY));
|
|
|
|
+ long jobMapMem =
|
|
|
|
+ Long.parseLong(jobConf.get(GridMixConfig.JOB_MAP_MEMORY_MB));
|
|
|
|
+ return (int)Math.ceil((double)jobMapMem / clusterMapMem);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the value for identifying the slots used by the reduce.
|
|
|
|
+ * @param jobConf - job configuration.
|
|
|
|
+ * @return - reduce factor value.
|
|
|
|
+ */
|
|
|
|
+ public static int getReduceFactor(Configuration jobConf) {
|
|
|
|
+ long clusterReduceMem =
|
|
|
|
+ Long.parseLong(jobConf.get(GridMixConfig.CLUSTER_REDUCE_MEMORY));
|
|
|
|
+ long jobReduceMem =
|
|
|
|
+ Long.parseLong(jobConf.get(GridMixConfig.JOB_REDUCE_MEMORY_MB));
|
|
|
|
+ return (int)Math.ceil((double)jobReduceMem / clusterReduceMem);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
|
+ private void assertGridMixHighRamJob(JobConf simuJobConf,
|
|
|
|
+ Configuration origConf, int option) {
|
|
|
|
+ int simuMapFactor = getMapFactor(simuJobConf);
|
|
|
|
+ int simuReduceFactor = getReduceFactor(simuJobConf);
|
|
|
|
+ /**
|
|
|
|
+ * option 1 : Both map and reduce honors the high ram.
|
|
|
|
+ * option 2 : Map only honors the high ram.
|
|
|
|
+ * option 3 : Reduce only honors the high ram.
|
|
|
|
+ * option 4 : Both map and reduce should not honors the high ram
|
|
|
|
+ * in disable state.
|
|
|
|
+ * option 5 : Map should not honors the high ram in disable state.
|
|
|
|
+ * option 6 : Reduce should not honors the high ram in disable state.
|
|
|
|
+ */
|
|
|
|
+ switch (option) {
|
|
|
|
+ case 1 :
|
|
|
|
+ Assert.assertTrue("Gridmix job has not honored the high "
|
|
|
|
+ + "ram for map.", simuMapFactor >= 2
|
|
|
|
+ && simuMapFactor == getMapFactor(origConf));
|
|
|
|
+ Assert.assertTrue("Gridmix job has not honored the high "
|
|
|
|
+ + "ram for reduce.", simuReduceFactor >= 2
|
|
|
|
+ && simuReduceFactor
|
|
|
|
+ == getReduceFactor(origConf));
|
|
|
|
+ break;
|
|
|
|
+ case 2 :
|
|
|
|
+ Assert.assertTrue("Gridmix job has not honored the high "
|
|
|
|
+ + "ram for map.", simuMapFactor >= 2
|
|
|
|
+ && simuMapFactor == getMapFactor(origConf));
|
|
|
|
+ break;
|
|
|
|
+ case 3 :
|
|
|
|
+ Assert.assertTrue("Girdmix job has not honored the high "
|
|
|
|
+ + "ram for reduce.", simuReduceFactor >= 2
|
|
|
|
+ && simuReduceFactor
|
|
|
|
+ == getReduceFactor(origConf));
|
|
|
|
+ break;
|
|
|
|
+ case 4 :
|
|
|
|
+ Assert.assertTrue("Gridmix job has honored the high "
|
|
|
|
+ + "ram for map in emulation disable state.",
|
|
|
|
+ simuMapFactor < 2
|
|
|
|
+ && simuMapFactor != getMapFactor(origConf));
|
|
|
|
+ Assert.assertTrue("Gridmix job has honored the high "
|
|
|
|
+ + "ram for reduce in emulation disable state.",
|
|
|
|
+ simuReduceFactor < 2
|
|
|
|
+ && simuReduceFactor
|
|
|
|
+ != getReduceFactor(origConf));
|
|
|
|
+ break;
|
|
|
|
+ case 5 :
|
|
|
|
+ Assert.assertTrue("Gridmix job has honored the high "
|
|
|
|
+ + "ram for map in emulation disable state.",
|
|
|
|
+ simuMapFactor < 2
|
|
|
|
+ && simuMapFactor != getMapFactor(origConf));
|
|
|
|
+ break;
|
|
|
|
+ case 6 :
|
|
|
|
+ Assert.assertTrue("Girdmix job has honored the high "
|
|
|
|
+ + "ram for reduce in emulation disable state.",
|
|
|
|
+ simuReduceFactor < 2
|
|
|
|
+ && simuReduceFactor
|
|
|
|
+ != getReduceFactor(origConf));
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get task memory after scaling based on cluster configuration.
|
|
|
|
+ * @param jobTaskKey - Job task key attribute.
|
|
|
|
+ * @param clusterTaskKey - Cluster task key attribute.
|
|
|
|
+ * @param origConf - Original job configuration.
|
|
|
|
+ * @param simuConf - Simulated job configuration.
|
|
|
|
+ * @return scaled task memory value.
|
|
|
|
+ */
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
|
+ public static long getScaledTaskMemInMB(String jobTaskKey,
|
|
|
|
+ String clusterTaskKey,
|
|
|
|
+ Configuration origConf,
|
|
|
|
+ Configuration simuConf) {
|
|
|
|
+ long simuClusterTaskValue =
|
|
|
|
+ simuConf.getLong(clusterTaskKey, JobConf.DISABLED_MEMORY_LIMIT);
|
|
|
|
+ long origClusterTaskValue =
|
|
|
|
+ origConf.getLong(clusterTaskKey, JobConf.DISABLED_MEMORY_LIMIT);
|
|
|
|
+ long origJobTaskValue =
|
|
|
|
+ origConf.getLong(jobTaskKey, JobConf.DISABLED_MEMORY_LIMIT);
|
|
|
|
+ double scaleFactor =
|
|
|
|
+ Math.ceil((double)origJobTaskValue / origClusterTaskValue);
|
|
|
|
+ long simulatedJobValue = (long)(scaleFactor * simuClusterTaskValue);
|
|
|
|
+ return simulatedJobValue;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * It Verifies the memory limit of a task.
|
|
|
|
+ * @param TaskMemInMB - task memory limit.
|
|
|
|
+ * @param taskLimitInMB - task upper limit.
|
|
|
|
+ */
|
|
|
|
+ public static void verifyMemoryLimits(long TaskMemInMB, long taskLimitInMB) {
|
|
|
|
+ if (TaskMemInMB > taskLimitInMB) {
|
|
|
|
+ Assert.fail("Simulated job's task memory exceeds the "
|
|
|
|
+ + "upper limit of task virtual memory.");
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private String convertJobStatus(String jobStatus) {
|
|
private String convertJobStatus(String jobStatus) {
|
|
if (jobStatus.equals("SUCCEEDED")) {
|
|
if (jobStatus.equals("SUCCEEDED")) {
|
|
return "SUCCESS";
|
|
return "SUCCESS";
|