YARN-9694. UI always show default-rack for all the nodes while running SLS.

This commit is contained in:
Abhishek Modi 2019-08-09 11:41:16 +05:30
parent 88ed1e0bfd
commit a92b7a5491
3 changed files with 70 additions and 2 deletions

View File

@ -51,8 +51,11 @@
import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.metrics2.source.JvmMetrics; import org.apache.hadoop.metrics2.source.JvmMetrics;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.TableMapping;
import org.apache.hadoop.tools.rumen.JobTraceReader; import org.apache.hadoop.tools.rumen.JobTraceReader;
import org.apache.hadoop.tools.rumen.LoggedJob; import org.apache.hadoop.tools.rumen.LoggedJob;
import org.apache.hadoop.tools.rumen.LoggedTask; import org.apache.hadoop.tools.rumen.LoggedTask;
@ -126,6 +129,7 @@ public class SLSRunner extends Configured implements Tool {
// other simulation information // other simulation information
private int numNMs, numRacks, numAMs, numTasks; private int numNMs, numRacks, numAMs, numTasks;
private long maxRuntime; private long maxRuntime;
private String tableMapping;
private final static Map<String, Object> simulateInfoMap = private final static Map<String, Object> simulateInfoMap =
new HashMap<String, Object>(); new HashMap<String, Object>();
@ -231,7 +235,7 @@ public void setSimulationParams(TraceType inType, String[] inTraces,
this.trackedApps = trackApps; this.trackedApps = trackApps;
this.printSimulation = printsimulation; this.printSimulation = printsimulation;
metricsOutputDir = outDir; metricsOutputDir = outDir;
tableMapping = outDir + "/tableMapping.csv";
} }
public void start() throws IOException, ClassNotFoundException, YarnException, public void start() throws IOException, ClassNotFoundException, YarnException,
@ -272,7 +276,12 @@ private void startRM() throws ClassNotFoundException, YarnException {
// TODO add support for FifoScheduler // TODO add support for FifoScheduler
throw new YarnException("Fifo Scheduler is not supported yet."); throw new YarnException("Fifo Scheduler is not supported yet.");
} }
rmConf.setClass(
CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
TableMapping.class, DNSToSwitchMapping.class);
rmConf.set(
CommonConfigurationKeysPublic.NET_TOPOLOGY_TABLE_MAPPING_FILE_KEY,
tableMapping);
rmConf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricsOutputDir); rmConf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricsOutputDir);
final SLSRunner se = this; final SLSRunner se = this;
@ -332,6 +341,8 @@ private void startNM() throws YarnException, IOException,
throw new YarnException("No node! Please configure nodes."); throw new YarnException("No node! Please configure nodes.");
} }
SLSUtils.generateNodeTableMapping(nodeSet, tableMapping);
// create NM simulators // create NM simulators
Random random = new Random(); Random random = new Random();
Set<String> rackSet = new ConcurrentHashSet<>(); Set<String> rackSet = new ConcurrentHashSet<>();

View File

@ -23,6 +23,11 @@
import java.io.InputStreamReader; import java.io.InputStreamReader;
import java.io.Reader; import java.io.Reader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -219,4 +224,32 @@ public static Set<NodeDetails> generateNodes(int numNodes,
} }
return nodeSet; return nodeSet;
} }
/**
* Generates a node to rack mapping file based on node details.
* This file is then being used by TableMapping to resolve rack names.
* The format required by TableMapping is a two column text file
* where first column specifies node name
* and second column specifies rack name.
* @param nodeDetails Set of node details.
* @param filePath File path where to write table mapping.
* @throws IOException
*/
public static void generateNodeTableMapping(Set<NodeDetails> nodeDetails,
String filePath) throws IOException {
List<String> entries = new ArrayList<>();
for (NodeDetails nodeDetail : nodeDetails) {
if (nodeDetail.getHostname().contains("/")) {
String hostname = nodeDetail.getHostname();
int lIndex = hostname.lastIndexOf("/");
String node = hostname.substring(lIndex + 1);
String rack = hostname.substring(0, lIndex);
entries.add(node + " " + rack);
}
}
Files.write(Paths.get(filePath),
entries,
StandardCharsets.UTF_8,
StandardOpenOption.CREATE);
}
} }

View File

@ -24,7 +24,11 @@
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import java.io.File;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.HashSet; import java.util.HashSet;
import java.util.List;
import java.util.Set; import java.util.Set;
public class TestSLSUtils { public class TestSLSUtils {
@ -109,6 +113,26 @@ public void testGenerateNodes() {
Assert.assertEquals("Number of racks is wrong.", 1, getNumRack(nodes)); Assert.assertEquals("Number of racks is wrong.", 1, getNumRack(nodes));
} }
/**
* Tests creation of table mapping based on given node details.
* @throws Exception
*/
@Test
public void testGenerateNodeTableMapping() throws Exception {
Set<NodeDetails> nodes = SLSUtils.generateNodes(3, 3);
File tempFile = File.createTempFile("testslsutils", ".tmp");
tempFile.deleteOnExit();
String fileName = tempFile.getAbsolutePath();
SLSUtils.generateNodeTableMapping(nodes, fileName);
List<String> lines = Files.readAllLines(Paths.get(fileName));
Assert.assertEquals(3, lines.size());
for (String line : lines) {
Assert.assertTrue(line.contains("node"));
Assert.assertTrue(line.contains("/rack"));
}
}
private int getNumRack(Set<NodeDetails> nodes) { private int getNumRack(Set<NodeDetails> nodes) {
Set<String> racks = new HashSet<>(); Set<String> racks = new HashSet<>();
for (NodeDetails node : nodes) { for (NodeDetails node : nodes) {