YARN-11103. SLS cleanup after previously merged SLS refactor jiras. Contributed by Szilard Nemeth

This commit is contained in:
Szilard Nemeth 2022-03-29 21:05:51 +02:00 committed by Benjamin Teke
parent e044a46f97
commit 94031b729d
15 changed files with 109 additions and 111 deletions

View File

@ -53,7 +53,7 @@ import java.util.concurrent.ConcurrentHashMap;
public class AMRunner {
private static final Logger LOG = LoggerFactory.getLogger(AMRunner.class);
static int REMAINING_APPS = 0;
static int remainingApps = 0;
private final Configuration conf;
private int AM_ID;
@ -63,8 +63,8 @@ public class AMRunner {
private Map<String, Class> amClassMap;
private TraceType inputType;
private String[] inputTraces;
private TaskRunner runner;
private SLSRunner slsRunner;
private final TaskRunner runner;
private final SLSRunner slsRunner;
private int numAMs, numTasks;
private long maxRuntime;
private ResourceManager rm;
@ -81,8 +81,8 @@ public class AMRunner {
amClassMap = new HashMap<>();
appIdAMSim = new ConcurrentHashMap<>();
// <AMType, Class> map
for (Map.Entry e : conf) {
String key = e.getKey().toString();
for (Map.Entry<String, String> e : conf) {
String key = e.getKey();
if (key.startsWith(SLSConfiguration.AM_TYPE_PREFIX)) {
String amType = key.substring(SLSConfiguration.AM_TYPE_PREFIX.length());
amClassMap.put(amType, Class.forName(conf.get(key)));
@ -112,7 +112,7 @@ public class AMRunner {
}
numAMs = amMap.size();
REMAINING_APPS = numAMs;
remainingApps = numAMs;
}
/**

View File

@ -70,9 +70,7 @@ public final class ReservationClientUtil {
deadline, reservationRequests, name);
// outermost request
ReservationSubmissionRequest request = ReservationSubmissionRequest
return ReservationSubmissionRequest
.newInstance(resDef, queueName, reservationId);
return request;
}
}

View File

@ -25,6 +25,7 @@ import java.io.InputStreamReader;
import java.io.OutputStreamWriter;
import java.io.Reader;
import java.io.Writer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.LinkedHashMap;
@ -54,7 +55,7 @@ public class RumenToSLSConverter {
private static Map<String, Set<String>> rackNodeMap =
new TreeMap<String, Set<String>>();
public static void main(String args[]) throws Exception {
public static void main(String[] args) throws Exception {
Options options = new Options();
options.addOption("input", true, "input rumen json file");
options.addOption("outputJobs", true, "output jobs file");
@ -121,9 +122,10 @@ public class RumenToSLSConverter {
private static void generateSLSLoadFile(String inputFile, String outputFile)
throws IOException {
try (Reader input =
new InputStreamReader(new FileInputStream(inputFile), "UTF-8")) {
new InputStreamReader(new FileInputStream(inputFile),
StandardCharsets.UTF_8)) {
try (Writer output =
new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) {
new OutputStreamWriter(new FileOutputStream(outputFile), StandardCharsets.UTF_8)) {
ObjectMapper mapper = new ObjectMapper();
ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
Iterator<Map> i = mapper.readValues(
@ -140,7 +142,7 @@ public class RumenToSLSConverter {
private static void generateSLSNodeFile(String outputFile)
throws IOException {
try (Writer output =
new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) {
new OutputStreamWriter(new FileOutputStream(outputFile), StandardCharsets.UTF_8)) {
ObjectMapper mapper = new ObjectMapper();
ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
for (Map.Entry<String, Set<String>> entry : rackNodeMap.entrySet()) {
@ -218,7 +220,7 @@ public class RumenToSLSConverter {
task.put("container.priority", priority);
task.put("container.type", taskType);
array.add(task);
String rackHost[] = SLSUtils.getRackHostName(hostname);
String[] rackHost = SLSUtils.getRackHostName(hostname);
if (rackNodeMap.containsKey(rackHost[0])) {
rackNodeMap.get(rackHost[0]).add(rackHost[1]);
} else {

View File

@ -18,16 +18,11 @@
package org.apache.hadoop.yarn.sls;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.Reader;
import java.nio.charset.StandardCharsets;
import java.security.Security;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.Objects;
import java.util.Random;
@ -38,10 +33,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.databind.JavaType;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser;
@ -73,20 +64,10 @@ import org.apache.hadoop.yarn.util.resource.Resources;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.security.Security;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@Private
@Unstable
public class SLSRunner extends Configured implements Tool {
private static TaskRunner runner = new TaskRunner();
private static final TaskRunner runner = new TaskRunner();
private String[] inputTraces;
// metrics
@ -103,6 +84,7 @@ public class SLSRunner extends Configured implements Tool {
private RMRunner rmRunner;
private NMRunner nmRunner;
private TraceType inputType;
private SynthTraceJobProducer stjp;
/**
@ -117,7 +99,7 @@ public class SLSRunner extends Configured implements Tool {
"networkaddress.cache.negative.ttl";
public static int getRemainingApps() {
return AMRunner.REMAINING_APPS;
return AMRunner.remainingApps;
}
public SLSRunner() throws ClassNotFoundException, YarnException {
@ -175,31 +157,32 @@ public class SLSRunner extends Configured implements Tool {
/**
* This is invoked before start.
* @param inType
* @param inTraces
* @param nodes
* @param metricsOutputDir
* @param trackApps
* @param printsimulation
* @param inputType The trace type
* @param inTraces Input traces
* @param nodes The node file
* @param metricsOutputDir Output dir for metrics
* @param trackApps Track these applications
* @param printSimulation Whether to print the simulation
*/
public void setSimulationParams(TraceType inType, String[] inTraces,
public void setSimulationParams(TraceType inputType, String[] inTraces,
String nodes, String metricsOutputDir, Set<String> trackApps,
boolean printsimulation) throws YarnException {
boolean printSimulation) throws YarnException {
this.inputType = inputType;
this.inputTraces = inTraces.clone();
this.amRunner.setInputType(inType);
this.amRunner.setInputType(inputType);
this.amRunner.setInputTraces(this.inputTraces);
this.amRunner.setTrackedApps(trackApps);
this.nmRunner.setNodeFile(nodes);
this.nmRunner.setInputType(inType);
this.nmRunner.setInputType(inputType);
this.nmRunner.setInputTraces(this.inputTraces);
this.printSimulation = printsimulation;
this.printSimulation = printSimulation;
this.rmRunner.setMetricsOutputDir(metricsOutputDir);
String tableMapping = metricsOutputDir + "/tableMapping.csv";
this.rmRunner.setTableMapping(tableMapping);
this.nmRunner.setTableMapping(tableMapping);
//We need this.inputTraces to set before creating SynthTraceJobProducer
if (inType == TraceType.SYNTH) {
if (inputType == TraceType.SYNTH) {
this.stjp = getSynthJobTraceProducer();
}
}
@ -319,8 +302,8 @@ public class SLSRunner extends Configured implements Tool {
}
public static void decreaseRemainingApps() {
AMRunner.REMAINING_APPS--;
if (AMRunner.REMAINING_APPS == 0) {
AMRunner.remainingApps--;
if (AMRunner.remainingApps == 0) {
exitSLSRunner();
}
}
@ -359,24 +342,15 @@ public class SLSRunner extends Configured implements Tool {
CommandLineParser parser = new GnuParser();
CommandLine cmd = parser.parse(options, argv);
String traceType = null;
String traceLocation = null;
// compatibility with old commandline
if (cmd.hasOption("inputrumen")) {
traceType = "RUMEN";
traceLocation = cmd.getOptionValue("inputrumen");
}
if (cmd.hasOption("inputsls")) {
traceType = "SLS";
traceLocation = cmd.getOptionValue("inputsls");
}
if (cmd.hasOption("tracetype")) {
traceType = cmd.getOptionValue("tracetype");
traceLocation = cmd.getOptionValue("tracelocation");
}
boolean hasInputRumenOption = cmd.hasOption("inputrumen");
boolean hasInputSlsOption = cmd.hasOption("inputsls");
boolean hasTraceTypeOption = cmd.hasOption("tracetype");
TraceType traceType = determineTraceType(cmd, hasInputRumenOption,
hasInputSlsOption, hasTraceTypeOption);
String traceLocation = determineTraceLocation(cmd, hasInputRumenOption,
hasInputSlsOption, hasTraceTypeOption);
String output = cmd.getOptionValue("output");
File outputFile = new File(output);
@ -396,25 +370,9 @@ public class SLSRunner extends Configured implements Tool {
String tempNodeFile =
cmd.hasOption("nodes") ? cmd.getOptionValue("nodes") : "";
TraceType tempTraceType;
switch (traceType) {
case "SLS":
tempTraceType = TraceType.SLS;
break;
case "RUMEN":
tempTraceType = TraceType.RUMEN;
break;
case "SYNTH":
tempTraceType = TraceType.SYNTH;
break;
default:
printUsage();
throw new YarnException("Misconfigured input");
}
String[] inputFiles = traceLocation.split(",");
setSimulationParams(tempTraceType, inputFiles, tempNodeFile, output,
setSimulationParams(traceType, inputFiles, tempNodeFile, output,
trackedJobSet, cmd.hasOption("printsimulation"));
start();
@ -422,6 +380,48 @@ public class SLSRunner extends Configured implements Tool {
return 0;
}
private TraceType determineTraceType(CommandLine cmd, boolean hasInputRumenOption,
boolean hasInputSlsOption, boolean hasTraceTypeOption) throws YarnException {
String traceType = null;
if (hasInputRumenOption) {
traceType = "RUMEN";
}
if (hasInputSlsOption) {
traceType = "SLS";
}
if (hasTraceTypeOption) {
traceType = cmd.getOptionValue("tracetype");
}
if (traceType == null) {
throw new YarnException("Misconfigured input");
}
switch (traceType) {
case "SLS":
return TraceType.SLS;
case "RUMEN":
return TraceType.RUMEN;
case "SYNTH":
return TraceType.SYNTH;
default:
printUsage();
throw new YarnException("Misconfigured input");
}
}
private String determineTraceLocation(CommandLine cmd, boolean hasInputRumenOption,
boolean hasInputSlsOption, boolean hasTraceTypeOption) throws YarnException {
if (hasInputRumenOption) {
return cmd.getOptionValue("inputrumen");
}
if (hasInputSlsOption) {
return cmd.getOptionValue("inputsls");
}
if (hasTraceTypeOption) {
return cmd.getOptionValue("tracelocation");
}
throw new YarnException("Misconfigured input! ");
}
public static void main(String[] argv) throws Exception {
exitAtTheFinish = true;
ToolRunner.run(new Configuration(), new SLSRunner(), argv);

View File

@ -85,7 +85,7 @@ public class NMSimulator extends TaskRunner.Task {
super.init(dispatchTime, dispatchTime + 1000000L * heartBeatInterval,
heartBeatInterval);
// create resource
String rackHostName[] = SLSUtils.getRackHostName(nodeIdStr);
String[] rackHostName = SLSUtils.getRackHostName(nodeIdStr);
this.node = NodeInfo.newNodeInfo(rackHostName[0], rackHostName[1],
Resources.clone(nodeResource));
this.rm = pRm;
@ -128,7 +128,7 @@ public class NMSimulator extends TaskRunner.Task {
@Override
public void middleStep() throws Exception {
// we check the lifetime for each running containers
ContainerSimulator cs = null;
ContainerSimulator cs;
synchronized(completedContainerList) {
while ((cs = containerQueue.poll()) != null) {
runningContainers.remove(cs.getId());

View File

@ -258,9 +258,8 @@ public class NodeInfo {
final Resource resource, int port) {
final NodeId nodeId = newNodeID(hostName, port);
final String nodeAddr = hostName + ":" + port;
final String httpAddress = hostName;
return new FakeRMNodeImpl(nodeId, nodeAddr, httpAddress,
return new FakeRMNodeImpl(nodeId, nodeAddr, hostName,
resource, rackName, "Me good",
port, hostName, null);
}

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptI
import org.apache.hadoop.yarn.sls.SLSRunner;
import org.apache.hadoop.yarn.sls.appmaster.AMSimulator;
import java.util.Map;
public class MockAMLauncher extends ApplicationMasterLauncher
implements EventHandler<AMLauncherEvent> {

View File

@ -61,7 +61,7 @@ public class SLSSchedulerCommons {
private final Map<ApplicationAttemptId, String> appQueueMap = new ConcurrentHashMap<>();
private final Tracker tracker;
public SLSSchedulerCommons(AbstractYarnScheduler scheduler) {
public SLSSchedulerCommons(AbstractYarnScheduler<?, ?> scheduler) {
this.scheduler = scheduler;
this.tracker = new Tracker();
}
@ -174,7 +174,7 @@ public class SLSSchedulerCommons {
}
}
// containers released/preemption from scheduler
Set<ContainerId> preemptionContainers = new HashSet<ContainerId>();
Set<ContainerId> preemptionContainers = new HashSet<>();
if (allocation.getContainerPreemptions() != null) {
preemptionContainers.addAll(allocation.getContainerPreemptions());
}
@ -277,7 +277,7 @@ public class SLSSchedulerCommons {
AppAttemptAddedSchedulerEvent appAddEvent =
(AppAttemptAddedSchedulerEvent) schedulerEvent;
SchedulerApplication app =
(SchedulerApplication) scheduler.getSchedulerApplications()
scheduler.getSchedulerApplications()
.get(appAddEvent.getApplicationAttemptId().getApplicationId());
appQueueMap.put(appAddEvent.getApplicationAttemptId(), app.getQueue()
.getQueueName());

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.sls.scheduler;
import java.io.BufferedWriter;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
@ -184,7 +185,7 @@ public abstract class SchedulerMetrics {
// application running information
jobRuntimeLogBW =
new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
metricsOutputDir + "/jobruntime.csv"), "UTF-8"));
metricsOutputDir + "/jobruntime.csv"), StandardCharsets.UTF_8));
jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
"simulate_start_time,simulate_end_time" + EOL);
jobRuntimeLogBW.flush();
@ -560,7 +561,7 @@ public abstract class SchedulerMetrics {
try {
metricsLogBW =
new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
metricsOutputDir + "/realtimetrack.json"), "UTF-8"));
metricsOutputDir + "/realtimetrack.json"), StandardCharsets.UTF_8));
metricsLogBW.write("[");
} catch (IOException e) {
LOG.info(e.getMessage());
@ -717,11 +718,10 @@ public abstract class SchedulerMetrics {
long traceEndTimeMS, long simulateStartTimeMS, long simulateEndTimeMS) {
try {
// write job runtime information
StringBuilder sb = new StringBuilder();
sb.append(appId).append(",").append(traceStartTimeMS).append(",")
.append(traceEndTimeMS).append(",").append(simulateStartTimeMS)
.append(",").append(simulateEndTimeMS);
jobRuntimeLogBW.write(sb.toString() + EOL);
String runtimeInfo = appId + "," + traceStartTimeMS + "," +
traceEndTimeMS + "," + simulateStartTimeMS +
"," + simulateEndTimeMS;
jobRuntimeLogBW.write(runtimeInfo + EOL);
jobRuntimeLogBW.flush();
} catch (IOException e) {
LOG.info(e.getMessage());

View File

@ -184,9 +184,9 @@ public class SynthJob implements JobStory {
int num = task.count.getInt();
String taskType = task.type;
long memory = task.max_memory.getLong();
memory = memory < MIN_MEMORY ? MIN_MEMORY: memory;
memory = Math.max(memory, MIN_MEMORY);
long vcores = task.max_vcores.getLong();
vcores = vcores < MIN_VCORES ? MIN_VCORES : vcores;
vcores = Math.max(vcores, MIN_VCORES);
int priority = task.priority;
ExecutionType executionType = task.executionType == null
? ExecutionType.GUARANTEED

View File

@ -310,7 +310,6 @@ public class SynthTraceJobProducer implements JobStoryProducer {
// Initialize job weights
job_weights = new ArrayList<>();
job_weights = new ArrayList<>();
for(JobDefinition j : job_classes){
job_weights.add(j.class_weight);
}
@ -638,7 +637,7 @@ public class SynthTraceJobProducer implements JobStoryProducer {
public String toString(){
switch(mode){
case CONST:
return "value: " + Double.toString(val);
return "value: " + val;
case DIST:
return "value: " + this.val + " std: " + this.std + " dist: "
+ this.dist.name();

View File

@ -85,7 +85,7 @@ public class SLSUtils {
JobTraceReader reader = new JobTraceReader(
new Path(fin.getAbsolutePath()), conf);
try {
LoggedJob job = null;
LoggedJob job;
while ((job = reader.getNext()) != null) {
for(LoggedTask mapTask : job.getMapTasks()) {
// select the last attempt
@ -123,7 +123,7 @@ public class SLSUtils {
JsonFactory jsonF = new JsonFactory();
ObjectMapper mapper = new ObjectMapper();
Reader input =
new InputStreamReader(new FileInputStream(jobTrace), "UTF-8");
new InputStreamReader(new FileInputStream(jobTrace), StandardCharsets.UTF_8);
try {
Iterator<Map> i = mapper.readValues(jsonF.createParser(input), Map.class);
while (i.hasNext()) {
@ -170,7 +170,7 @@ public class SLSUtils {
JsonFactory jsonF = new JsonFactory();
ObjectMapper mapper = new ObjectMapper();
Reader input =
new InputStreamReader(new FileInputStream(nodeFile), "UTF-8");
new InputStreamReader(new FileInputStream(nodeFile), StandardCharsets.UTF_8);
try {
Iterator<Map> i = mapper.readValues(jsonF.createParser(input), Map.class);
while (i.hasNext()) {

View File

@ -103,7 +103,7 @@ public abstract class BaseSLSRunnerTest {
}
if (nodeFile != null) {
args = ArrayUtils.addAll(args, new String[] {"-nodes", nodeFile });
args = ArrayUtils.addAll(args, "-nodes", nodeFile);
}
// enable continuous invariant checks

View File

@ -37,6 +37,7 @@ import java.util.Arrays;
import static com.fasterxml.jackson.databind.DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
/**
@ -88,7 +89,7 @@ public class TestSynthJobGeneration {
}
Assert.assertTrue(bucket0 > 0);
Assert.assertTrue(bucket1 == 0);
assertEquals(0, bucket1);
Assert.assertTrue(bucket2 > 0);
Assert.assertTrue(bucket3 > 0);
Assert.assertTrue(bucket2 > bucket0);
@ -255,7 +256,7 @@ public class TestSynthJobGeneration {
assertTrue(js.getTasks().size() > 0);
for (SynthJob.SynthTask t : js.getTasks()) {
assertTrue(t.getType() != null);
assertNotNull(t.getType());
assertTrue(t.getTime() > 0);
assertTrue(t.getMemory() > 0);
assertTrue(t.getVcores() > 0);

View File

@ -36,7 +36,7 @@ public class TestSLSUtils {
@Test
public void testGetRackHostname() {
String str = "/rack1/node1";
String rackHostname[] = SLSUtils.getRackHostName(str);
String[] rackHostname = SLSUtils.getRackHostName(str);
Assert.assertEquals("rack1", rackHostname[0]);
Assert.assertEquals("node1", rackHostname[1]);
@ -81,7 +81,7 @@ public class TestSLSUtils {
} else if(nodeLabel.getName().equals("label2")) {
Assert.assertFalse(nodeLabel.isExclusive());
} else {
Assert.assertTrue("Unexepected label", false);
Assert.fail("Unexpected label");
}
}
} else if (nodeDetail.getHostname().equals("/rack1/node4")) {