Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1557294 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Chris Nauroth 2014-01-11 00:37:23 +00:00
commit c90211af7a
66 changed files with 1636 additions and 871 deletions

View File

@ -415,6 +415,9 @@ Release 2.4.0 - UNRELEASED
HADOOP-10208. Remove duplicate initialization in StringUtils.getStringCollection. HADOOP-10208. Remove duplicate initialization in StringUtils.getStringCollection.
(Benoy Antony via jing9) (Benoy Antony via jing9)
HADOOP-9420. Add percentile or max metric for rpcQueueTime, processing time.
(Liang Xie via wang)
OPTIMIZATIONS OPTIMIZATIONS
HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn) HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@ -503,6 +506,11 @@ Release 2.4.0 - UNRELEASED
HADOOP-10147 HDFS-5678 Upgrade to commons-logging 1.1.3 to avoid potential HADOOP-10147 HDFS-5678 Upgrade to commons-logging 1.1.3 to avoid potential
deadlock in MiniDFSCluster (stevel) deadlock in MiniDFSCluster (stevel)
HADOOP-10207. TestUserGroupInformation#testLogin is flaky (jxiang via cmccabe)
HADOOP-10214. Fix multithreaded correctness warnings in ActiveStandbyElector
(Liang Xie via kasha)
Release 2.3.0 - UNRELEASED Release 2.3.0 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES
@ -588,6 +596,9 @@ Release 2.3.0 - UNRELEASED
HADOOP-10193. hadoop-auth's PseudoAuthenticationHandler can consume getInputStream. HADOOP-10193. hadoop-auth's PseudoAuthenticationHandler can consume getInputStream.
(gchanan via tucu) (gchanan via tucu)
HADOOP-10178. Configuration deprecation always emit "deprecated" warnings
when a new key is used. (Shanyu Zhao via cnauroth)
Release 2.2.0 - 2013-10-13 Release 2.2.0 - 2013-10-13
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -552,36 +552,6 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
return deprecationContext.get().getDeprecatedKeyMap().containsKey(key); return deprecationContext.get().getDeprecatedKeyMap().containsKey(key);
} }
/**
* Returns the alternate name for a key if the property name is deprecated
* or if deprecates a property name.
*
* @param name property name.
* @return alternate name.
*/
private String[] getAlternateNames(String name) {
String altNames[] = null;
DeprecationContext cur = deprecationContext.get();
DeprecatedKeyInfo keyInfo = cur.getDeprecatedKeyMap().get(name);
if (keyInfo == null) {
altNames = (cur.getReverseDeprecatedKeyMap().get(name) != null ) ?
new String [] {cur.getReverseDeprecatedKeyMap().get(name)} : null;
if(altNames != null && altNames.length > 0) {
//To help look for other new configs for this deprecated config
keyInfo = cur.getDeprecatedKeyMap().get(altNames[0]);
}
}
if(keyInfo != null && keyInfo.newKeys.length > 0) {
List<String> list = new ArrayList<String>();
if(altNames != null) {
list.addAll(Arrays.asList(altNames));
}
list.addAll(Arrays.asList(keyInfo.newKeys));
altNames = list.toArray(new String[list.size()]);
}
return altNames;
}
/** /**
* Checks for the presence of the property <code>name</code> in the * Checks for the presence of the property <code>name</code> in the
* deprecation map. Returns the first of the list of new keys if present * deprecation map. Returns the first of the list of new keys if present
@ -933,6 +903,37 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
return result; return result;
} }
/**
* Returns alternative names (non-deprecated keys or previously-set deprecated keys)
* for a given non-deprecated key.
* If the given key is deprecated, return null.
*
* @param name property name.
* @return alternative names.
*/
private String[] getAlternativeNames(String name) {
String altNames[] = null;
DeprecatedKeyInfo keyInfo = null;
DeprecationContext cur = deprecationContext.get();
String depKey = cur.getReverseDeprecatedKeyMap().get(name);
if(depKey != null) {
keyInfo = cur.getDeprecatedKeyMap().get(depKey);
if(keyInfo.newKeys.length > 0) {
if(getProps().containsKey(depKey)) {
//if deprecated key is previously set explicitly
List<String> list = new ArrayList<String>();
list.addAll(Arrays.asList(keyInfo.newKeys));
list.add(depKey);
altNames = list.toArray(new String[list.size()]);
}
else {
altNames = keyInfo.newKeys;
}
}
}
return altNames;
}
/** /**
* Set the <code>value</code> of the <code>name</code> property. If * Set the <code>value</code> of the <code>name</code> property. If
* <code>name</code> is deprecated or there is a deprecated name associated to it, * <code>name</code> is deprecated or there is a deprecated name associated to it,
@ -947,8 +948,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
/** /**
* Set the <code>value</code> of the <code>name</code> property. If * Set the <code>value</code> of the <code>name</code> property. If
* <code>name</code> is deprecated or there is a deprecated name associated to it, * <code>name</code> is deprecated, it also sets the <code>value</code> to
* it sets the value to both names. * the keys that replace the deprecated key.
* *
* @param name property name. * @param name property name.
* @param value property value. * @param value property value.
@ -969,23 +970,30 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
} }
getOverlay().setProperty(name, value); getOverlay().setProperty(name, value);
getProps().setProperty(name, value); getProps().setProperty(name, value);
if(source == null) { String newSource = (source == null ? "programatically" : source);
updatingResource.put(name, new String[] {"programatically"});
} else { if (!isDeprecated(name)) {
updatingResource.put(name, new String[] {source}); updatingResource.put(name, new String[] {newSource});
} String[] altNames = getAlternativeNames(name);
String[] altNames = getAlternateNames(name); if(altNames != null) {
if (altNames != null && altNames.length > 0) { for(String n: altNames) {
String altSource = "because " + name + " is deprecated"; if(!n.equals(name)) {
for(String altName : altNames) { getOverlay().setProperty(n, value);
if(!altName.equals(name)) { getProps().setProperty(n, value);
getOverlay().setProperty(altName, value); updatingResource.put(n, new String[] {newSource});
getProps().setProperty(altName, value); }
updatingResource.put(altName, new String[] {altSource});
} }
} }
} }
warnOnceIfDeprecated(deprecations, name); else {
String[] names = handleDeprecation(deprecationContext.get(), name);
String altSource = "because " + name + " is deprecated";
for(String n : names) {
getOverlay().setProperty(n, value);
getProps().setProperty(n, value);
updatingResource.put(n, new String[] {altSource});
}
}
} }
private void warnOnceIfDeprecated(DeprecationContext deprecations, String name) { private void warnOnceIfDeprecated(DeprecationContext deprecations, String name) {
@ -999,15 +1007,21 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
* Unset a previously set property. * Unset a previously set property.
*/ */
public synchronized void unset(String name) { public synchronized void unset(String name) {
String[] altNames = getAlternateNames(name); String[] names = null;
getOverlay().remove(name); if (!isDeprecated(name)) {
getProps().remove(name); names = getAlternativeNames(name);
if (altNames !=null && altNames.length > 0) { if(names == null) {
for(String altName : altNames) { names = new String[]{name};
getOverlay().remove(altName);
getProps().remove(altName);
} }
} }
else {
names = handleDeprecation(deprecationContext.get(), name);
}
for(String n: names) {
getOverlay().remove(n);
getProps().remove(n);
}
} }
/** /**
@ -2600,4 +2614,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
System.out.println(entry.getKey() + "\t" + newKeys.toString()); System.out.println(entry.getKey() + "\t" + newKeys.toString());
} }
} }
/**
* Returns whether or not a deprecated name has been warned. If the name is not
* deprecated then always return false
*/
public static boolean hasWarnedDeprecation(String name) {
DeprecationContext deprecations = deprecationContext.get();
if(deprecations.getDeprecatedKeyMap().containsKey(name)) {
if(deprecations.getDeprecatedKeyMap().get(name).accessed.get()) {
return true;
}
}
return false;
}
} }

View File

@ -242,4 +242,9 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
public static final String HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS = public static final String HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS =
"hadoop.user.group.metrics.percentiles.intervals"; "hadoop.user.group.metrics.percentiles.intervals";
public static final String RPC_METRICS_QUANTILE_ENABLE =
"rpc.metrics.quantile.enable";
public static final String RPC_METRICS_PERCENTILES_INTERVALS_KEY =
"rpc.metrics.percentiles.intervals";
} }

View File

@ -768,7 +768,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
} }
@InterfaceAudience.Private @InterfaceAudience.Private
public void terminateConnection() { public synchronized void terminateConnection() {
if (zkClient == null) { if (zkClient == null) {
return; return;
} }

View File

@ -2193,7 +2193,7 @@ public abstract class Server {
listener = new Listener(); listener = new Listener();
this.port = listener.getAddress().getPort(); this.port = listener.getAddress().getPort();
connectionManager = new ConnectionManager(); connectionManager = new ConnectionManager();
this.rpcMetrics = RpcMetrics.create(this); this.rpcMetrics = RpcMetrics.create(this, conf);
this.rpcDetailedMetrics = RpcDetailedMetrics.create(this.port); this.rpcDetailedMetrics = RpcDetailedMetrics.create(this.port);
this.tcpNoDelay = conf.getBoolean( this.tcpNoDelay = conf.getBoolean(
CommonConfigurationKeysPublic.IPC_SERVER_TCPNODELAY_KEY, CommonConfigurationKeysPublic.IPC_SERVER_TCPNODELAY_KEY,

View File

@ -19,14 +19,17 @@ package org.apache.hadoop.ipc.metrics;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.metrics2.annotation.Metric; import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.annotation.Metrics; import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.MetricsRegistry; import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import org.apache.hadoop.metrics2.lib.MutableCounterInt; import org.apache.hadoop.metrics2.lib.MutableCounterInt;
import org.apache.hadoop.metrics2.lib.MutableCounterLong; import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableQuantiles;
import org.apache.hadoop.metrics2.lib.MutableRate; import org.apache.hadoop.metrics2.lib.MutableRate;
/** /**
@ -41,26 +44,48 @@ public class RpcMetrics {
final Server server; final Server server;
final MetricsRegistry registry; final MetricsRegistry registry;
final String name; final String name;
final boolean rpcQuantileEnable;
RpcMetrics(Server server) { RpcMetrics(Server server, Configuration conf) {
String port = String.valueOf(server.getListenerAddress().getPort()); String port = String.valueOf(server.getListenerAddress().getPort());
name = "RpcActivityForPort"+ port; name = "RpcActivityForPort" + port;
this.server = server; this.server = server;
registry = new MetricsRegistry("rpc").tag("port", "RPC port", port); registry = new MetricsRegistry("rpc").tag("port", "RPC port", port);
LOG.debug("Initialized "+ registry); int[] intervals = conf.getInts(
CommonConfigurationKeys.RPC_METRICS_PERCENTILES_INTERVALS_KEY);
rpcQuantileEnable = (intervals.length > 0) && conf.getBoolean(
CommonConfigurationKeys.RPC_METRICS_QUANTILE_ENABLE, false);
if (rpcQuantileEnable) {
rpcQueueTimeMillisQuantiles =
new MutableQuantiles[intervals.length];
rpcProcessingTimeMillisQuantiles =
new MutableQuantiles[intervals.length];
for (int i = 0; i < intervals.length; i++) {
int interval = intervals[i];
rpcQueueTimeMillisQuantiles[i] = registry.newQuantiles("rpcQueueTime"
+ interval + "s", "rpc queue time in milli second", "ops",
"latency", interval);
rpcProcessingTimeMillisQuantiles[i] = registry.newQuantiles(
"rpcProcessingTime" + interval + "s",
"rpc processing time in milli second", "ops", "latency", interval);
}
}
LOG.debug("Initialized " + registry);
} }
public String name() { return name; } public String name() { return name; }
public static RpcMetrics create(Server server) { public static RpcMetrics create(Server server, Configuration conf) {
RpcMetrics m = new RpcMetrics(server); RpcMetrics m = new RpcMetrics(server, conf);
return DefaultMetricsSystem.instance().register(m.name, null, m); return DefaultMetricsSystem.instance().register(m.name, null, m);
} }
@Metric("Number of received bytes") MutableCounterLong receivedBytes; @Metric("Number of received bytes") MutableCounterLong receivedBytes;
@Metric("Number of sent bytes") MutableCounterLong sentBytes; @Metric("Number of sent bytes") MutableCounterLong sentBytes;
@Metric("Queue time") MutableRate rpcQueueTime; @Metric("Queue time") MutableRate rpcQueueTime;
MutableQuantiles[] rpcQueueTimeMillisQuantiles;
@Metric("Processsing time") MutableRate rpcProcessingTime; @Metric("Processsing time") MutableRate rpcProcessingTime;
MutableQuantiles[] rpcProcessingTimeMillisQuantiles;
@Metric("Number of authentication failures") @Metric("Number of authentication failures")
MutableCounterInt rpcAuthenticationFailures; MutableCounterInt rpcAuthenticationFailures;
@Metric("Number of authentication successes") @Metric("Number of authentication successes")
@ -146,6 +171,11 @@ public class RpcMetrics {
//@Override //@Override
public void addRpcQueueTime(int qTime) { public void addRpcQueueTime(int qTime) {
rpcQueueTime.add(qTime); rpcQueueTime.add(qTime);
if (rpcQuantileEnable) {
for (MutableQuantiles q : rpcQueueTimeMillisQuantiles) {
q.add(qTime);
}
}
} }
/** /**
@ -155,5 +185,10 @@ public class RpcMetrics {
//@Override //@Override
public void addRpcProcessingTime(int processingTime) { public void addRpcProcessingTime(int processingTime) {
rpcProcessingTime.add(processingTime); rpcProcessingTime.add(processingTime);
if (rpcQuantileEnable) {
for (MutableQuantiles q : rpcProcessingTimeMillisQuantiles) {
q.add(processingTime);
}
}
} }
} }

View File

@ -95,7 +95,7 @@ public class CompositeService extends AbstractService {
protected synchronized boolean removeService(Service service) { protected synchronized boolean removeService(Service service) {
synchronized (serviceList) { synchronized (serviceList) {
return serviceList.add(service); return serviceList.remove(service);
} }
} }

View File

@ -26,6 +26,7 @@ import java.io.BufferedWriter;
import java.io.File; import java.io.File;
import java.io.FileWriter; import java.io.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.io.ByteArrayOutputStream;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -399,4 +400,30 @@ public class TestConfigurationDeprecation {
Uninterruptibles.getUninterruptibly(future); Uninterruptibles.getUninterruptibly(future);
} }
} }
@Test
public void testNoFalseDeprecationWarning() throws IOException {
Configuration conf = new Configuration();
Configuration.addDeprecation("AA", "BB");
conf.set("BB", "bb");
conf.get("BB");
conf.writeXml(new ByteArrayOutputStream());
assertEquals(false, Configuration.hasWarnedDeprecation("AA"));
conf.set("AA", "aa");
assertEquals(true, Configuration.hasWarnedDeprecation("AA"));
}
@Test
public void testDeprecationSetUnset() throws IOException {
addDeprecationToConfiguration();
Configuration conf = new Configuration();
//"X" is deprecated by "Y" and "Z"
conf.set("Y", "y");
assertEquals("y", conf.get("Z"));
conf.set("X", "x");
assertEquals("x", conf.get("Z"));
conf.unset("Y");
assertEquals(null, conf.get("Z"));
assertEquals(null, conf.get("X"));
}
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.ipc;
import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt; import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertNotSame;
@ -67,6 +68,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.Service; import org.apache.hadoop.security.authorize.Service;
import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.test.MetricsAsserts;
import org.apache.hadoop.test.MockitoUtil; import org.apache.hadoop.test.MockitoUtil;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -961,6 +963,44 @@ public class TestRPC {
} }
} }
@Test
public void testRpcMetrics() throws Exception {
Configuration configuration = new Configuration();
final int interval = 1;
configuration.setBoolean(CommonConfigurationKeys.
RPC_METRICS_QUANTILE_ENABLE, true);
configuration.set(CommonConfigurationKeys.
RPC_METRICS_PERCENTILES_INTERVALS_KEY, "" + interval);
final Server server = new RPC.Builder(configuration)
.setProtocol(TestProtocol.class).setInstance(new TestImpl())
.setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true)
.build();
server.start();
final TestProtocol proxy = RPC.getProxy(TestProtocol.class,
TestProtocol.versionID, server.getListenerAddress(), configuration);
try {
for (int i=0; i<1000; i++) {
proxy.ping();
proxy.echo("" + i);
}
MetricsRecordBuilder rpcMetrics =
getMetrics(server.getRpcMetrics().name());
assertTrue("Expected non-zero rpc queue time",
getLongCounter("RpcQueueTimeNumOps", rpcMetrics) > 0);
assertTrue("Expected non-zero rpc processing time",
getLongCounter("RpcProcessingTimeNumOps", rpcMetrics) > 0);
MetricsAsserts.assertQuantileGauges("RpcQueueTime" + interval + "s",
rpcMetrics);
MetricsAsserts.assertQuantileGauges("RpcProcessingTime" + interval + "s",
rpcMetrics);
} finally {
if (proxy != null) {
RPC.stopProxy(proxy);
}
server.stop();
}
}
public static void main(String[] args) throws IOException { public static void main(String[] args) throws IOException {
new TestRPC().testCallsInternal(conf); new TestRPC().testCallsInternal(conf);

View File

@ -738,7 +738,7 @@ public class TestUserGroupInformation {
long groups) throws InterruptedException { long groups) throws InterruptedException {
MetricsRecordBuilder rb = getMetrics("UgiMetrics"); MetricsRecordBuilder rb = getMetrics("UgiMetrics");
if (groups > 0) { if (groups > 0) {
assertCounter("GetGroupsNumOps", groups, rb); assertCounterGt("GetGroupsNumOps", groups-1, rb);
double avg = getDoubleGauge("GetGroupsAvgTime", rb); double avg = getDoubleGauge("GetGroupsAvgTime", rb);
assertTrue(avg >= 0.0); assertTrue(avg >= 0.0);

View File

@ -245,6 +245,9 @@ Trunk (Unreleased)
HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
FileDiff/DirectoryDiff. (jing9) FileDiff/DirectoryDiff. (jing9)
HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be
closed before method returns. (Ted Yu via junping_du)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe) HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
@ -736,6 +739,13 @@ Release 2.4.0 - UNRELEASED
HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to
HTTP_ONLY. (Haohui Mai via jing9) HTTP_ONLY. (Haohui Mai via jing9)
HDFS-5449. WebHdfs compatibility broken between 2.2 and 1.x / 23.x (kihwal)
HDFS-5756. hadoopRzOptionsSetByteBufferPool does not accept NULL argument,
contrary to docs. (cmccabe via wang)
HDFS-5747. Fix NPEs in BlockManager. (Arpit Agarwal)
BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
HDFS-4985. Add storage type to the protocol and expose it in block report HDFS-4985. Add storage type to the protocol and expose it in block report

View File

@ -324,12 +324,14 @@ public class BlockInfoUnderConstruction extends BlockInfo {
Iterator<ReplicaUnderConstruction> it = replicas.iterator(); Iterator<ReplicaUnderConstruction> it = replicas.iterator();
while (it.hasNext()) { while (it.hasNext()) {
ReplicaUnderConstruction r = it.next(); ReplicaUnderConstruction r = it.next();
if(r.getExpectedStorageLocation() == storage) { DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
if(expectedLocation == storage) {
// Record the gen stamp from the report // Record the gen stamp from the report
r.setGenerationStamp(block.getGenerationStamp()); r.setGenerationStamp(block.getGenerationStamp());
return; return;
} else if (r.getExpectedStorageLocation().getDatanodeDescriptor() == } else if (expectedLocation != null &&
storage.getDatanodeDescriptor()) { expectedLocation.getDatanodeDescriptor() ==
storage.getDatanodeDescriptor()) {
// The Datanode reported that the block is on a different storage // The Datanode reported that the block is on a different storage
// than the one chosen by BlockPlacementPolicy. This can occur as // than the one chosen by BlockPlacementPolicy. This can occur as

View File

@ -621,8 +621,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
long loadStart = now(); long loadStart = now();
String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf); String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
namesystem.loadFSImage(startOpt, fsImage, try {
HAUtil.isHAEnabled(conf, nameserviceId)); namesystem.loadFSImage(startOpt, fsImage,
HAUtil.isHAEnabled(conf, nameserviceId));
} catch (IOException ioe) {
LOG.warn("Encountered exception loading fsimage", ioe);
fsImage.close();
throw ioe;
}
long timeTakenToLoadFSImage = now() - loadStart; long timeTakenToLoadFSImage = now() - loadStart;
LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs"); LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
NameNodeMetrics nnMetrics = NameNode.getNameNodeMetrics(); NameNodeMetrics nnMetrics = NameNode.getNameNodeMetrics();

View File

@ -547,8 +547,8 @@ public class NameNode implements NameNodeStatusMXBean {
} }
private void stopCommonServices() { private void stopCommonServices() {
if(namesystem != null) namesystem.close();
if(rpcServer != null) rpcServer.stop(); if(rpcServer != null) rpcServer.stop();
if(namesystem != null) namesystem.close();
if (pauseMonitor != null) pauseMonitor.stop(); if (pauseMonitor != null) pauseMonitor.stop();
if (plugins != null) { if (plugins != null) {
for (ServicePlugin p : plugins) { for (ServicePlugin p : plugins) {
@ -816,14 +816,20 @@ public class NameNode implements NameNodeStatusMXBean {
System.out.println("Formatting using clusterid: " + clusterId); System.out.println("Formatting using clusterid: " + clusterId);
FSImage fsImage = new FSImage(conf, nameDirsToFormat, editDirsToFormat); FSImage fsImage = new FSImage(conf, nameDirsToFormat, editDirsToFormat);
FSNamesystem fsn = new FSNamesystem(conf, fsImage); try {
fsImage.getEditLog().initJournalsForWrite(); FSNamesystem fsn = new FSNamesystem(conf, fsImage);
fsImage.getEditLog().initJournalsForWrite();
if (!fsImage.confirmFormat(force, isInteractive)) { if (!fsImage.confirmFormat(force, isInteractive)) {
return true; // aborted return true; // aborted
}
fsImage.format(fsn, clusterId);
} catch (IOException ioe) {
LOG.warn("Encountered exception during format: ", ioe);
fsImage.close();
throw ioe;
} }
fsImage.format(fsn, clusterId);
return false; return false;
} }
@ -897,6 +903,7 @@ public class NameNode implements NameNodeStatusMXBean {
} }
NNStorage existingStorage = null; NNStorage existingStorage = null;
FSImage sharedEditsImage = null;
try { try {
FSNamesystem fsns = FSNamesystem fsns =
FSNamesystem.loadFromDisk(getConfigurationWithoutSharedEdits(conf)); FSNamesystem.loadFromDisk(getConfigurationWithoutSharedEdits(conf));
@ -906,7 +913,7 @@ public class NameNode implements NameNodeStatusMXBean {
List<URI> sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf); List<URI> sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf);
FSImage sharedEditsImage = new FSImage(conf, sharedEditsImage = new FSImage(conf,
Lists.<URI>newArrayList(), Lists.<URI>newArrayList(),
sharedEditsDirs); sharedEditsDirs);
sharedEditsImage.getEditLog().initJournalsForWrite(); sharedEditsImage.getEditLog().initJournalsForWrite();
@ -934,6 +941,13 @@ public class NameNode implements NameNodeStatusMXBean {
LOG.error("Could not initialize shared edits dir", ioe); LOG.error("Could not initialize shared edits dir", ioe);
return true; // aborted return true; // aborted
} finally { } finally {
if (sharedEditsImage != null) {
try {
sharedEditsImage.close();
} catch (IOException ioe) {
LOG.warn("Could not close sharedEditsImage", ioe);
}
}
// Have to unlock storage explicitly for the case when we're running in a // Have to unlock storage explicitly for the case when we're running in a
// unit test, which runs in the same JVM as NNs. // unit test, which runs in the same JVM as NNs.
if (existingStorage != null) { if (existingStorage != null) {

View File

@ -190,24 +190,29 @@ public class BootstrapStandby implements Tool, Configurable {
// Load the newly formatted image, using all of the directories (including shared // Load the newly formatted image, using all of the directories (including shared
// edits) // edits)
FSImage image = new FSImage(conf); FSImage image = new FSImage(conf);
image.getStorage().setStorageInfo(storage); try {
image.initEditLog(); image.getStorage().setStorageInfo(storage);
assert image.getEditLog().isOpenForRead() : image.initEditLog();
assert image.getEditLog().isOpenForRead() :
"Expected edit log to be open for read"; "Expected edit log to be open for read";
// Ensure that we have enough edits already in the shared directory to // Ensure that we have enough edits already in the shared directory to
// start up from the last checkpoint on the active. // start up from the last checkpoint on the active.
if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) { if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) {
return ERR_CODE_LOGS_UNAVAILABLE; return ERR_CODE_LOGS_UNAVAILABLE;
} }
image.getStorage().writeTransactionIdFileToStorage(curTxId); image.getStorage().writeTransactionIdFileToStorage(curTxId);
// Download that checkpoint into our storage directories. // Download that checkpoint into our storage directories.
MD5Hash hash = TransferFsImage.downloadImageToStorage( MD5Hash hash = TransferFsImage.downloadImageToStorage(
otherHttpAddr, imageTxId, otherHttpAddr, imageTxId,
storage, true); storage, true);
image.saveDigestAndRenameCheckpointImage(imageTxId, hash); image.saveDigestAndRenameCheckpointImage(imageTxId, hash);
} catch (IOException ioe) {
image.close();
throw ioe;
}
return 0; return 0;
} }

View File

@ -271,7 +271,7 @@ public class JsonUtil {
} }
/** Convert a DatanodeInfo to a Json map. */ /** Convert a DatanodeInfo to a Json map. */
private static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) { static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
if (datanodeinfo == null) { if (datanodeinfo == null) {
return null; return null;
} }
@ -279,6 +279,9 @@ public class JsonUtil {
// TODO: Fix storageID // TODO: Fix storageID
final Map<String, Object> m = new TreeMap<String, Object>(); final Map<String, Object> m = new TreeMap<String, Object>();
m.put("ipAddr", datanodeinfo.getIpAddr()); m.put("ipAddr", datanodeinfo.getIpAddr());
// 'name' is equivalent to ipAddr:xferPort. Older clients (1.x, 0.23.x)
// expects this instead of the two fields.
m.put("name", datanodeinfo.getXferAddr());
m.put("hostName", datanodeinfo.getHostName()); m.put("hostName", datanodeinfo.getHostName());
m.put("storageID", datanodeinfo.getDatanodeUuid()); m.put("storageID", datanodeinfo.getDatanodeUuid());
m.put("xferPort", datanodeinfo.getXferPort()); m.put("xferPort", datanodeinfo.getXferPort());
@ -325,17 +328,49 @@ public class JsonUtil {
} }
/** Convert a Json map to an DatanodeInfo object. */ /** Convert a Json map to an DatanodeInfo object. */
static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) { static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
throws IOException {
if (m == null) { if (m == null) {
return null; return null;
} }
// ipAddr and xferPort are the critical fields for accessing data.
// If any one of the two is missing, an exception needs to be thrown.
// Handle the case of old servers (1.x, 0.23.x) sending 'name' instead
// of ipAddr and xferPort.
String ipAddr = getString(m, "ipAddr", null);
int xferPort = getInt(m, "xferPort", -1);
if (ipAddr == null) {
String name = getString(m, "name", null);
if (name != null) {
int colonIdx = name.indexOf(':');
if (colonIdx > 0) {
ipAddr = name.substring(0, colonIdx);
xferPort = Integer.parseInt(name.substring(colonIdx +1));
} else {
throw new IOException(
"Invalid value in server response: name=[" + name + "]");
}
} else {
throw new IOException(
"Missing both 'ipAddr' and 'name' in server response.");
}
// ipAddr is non-null & non-empty string at this point.
}
// Check the validity of xferPort.
if (xferPort == -1) {
throw new IOException(
"Invalid or missing 'xferPort' in server response.");
}
// TODO: Fix storageID // TODO: Fix storageID
return new DatanodeInfo( return new DatanodeInfo(
(String)m.get("ipAddr"), ipAddr,
(String)m.get("hostName"), (String)m.get("hostName"),
(String)m.get("storageID"), (String)m.get("storageID"),
(int)(long)(Long)m.get("xferPort"), xferPort,
(int)(long)(Long)m.get("infoPort"), (int)(long)(Long)m.get("infoPort"),
getInt(m, "infoSecurePort", 0), getInt(m, "infoSecurePort", 0),
(int)(long)(Long)m.get("ipcPort"), (int)(long)(Long)m.get("ipcPort"),
@ -368,7 +403,8 @@ public class JsonUtil {
} }
/** Convert an Object[] to a DatanodeInfo[]. */ /** Convert an Object[] to a DatanodeInfo[]. */
private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) { private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects)
throws IOException {
if (objects == null) { if (objects == null) {
return null; return null;
} else if (objects.length == 0) { } else if (objects.length == 0) {

View File

@ -2174,16 +2174,18 @@ int hadoopRzOptionsSetByteBufferPool(
return -1; return -1;
} }
// Note: we don't have to call hadoopRzOptionsClearCached in this if (className) {
// function, since the ByteBufferPool is passed separately from the // Note: we don't have to call hadoopRzOptionsClearCached in this
// EnumSet of ReadOptions. // function, since the ByteBufferPool is passed separately from the
// EnumSet of ReadOptions.
jthr = constructNewObjectOfClass(env, &byteBufferPool, className, "()V"); jthr = constructNewObjectOfClass(env, &byteBufferPool, className, "()V");
if (jthr) { if (jthr) {
printExceptionAndFree(env, jthr, PRINT_EXC_ALL, printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
"hadoopRzOptionsSetByteBufferPool(className=%s): ", className); "hadoopRzOptionsSetByteBufferPool(className=%s): ", className);
errno = EINVAL; errno = EINVAL;
return -1; return -1;
}
} }
if (opts->byteBufferPool) { if (opts->byteBufferPool) {
// Delete any previous ByteBufferPool we had. // Delete any previous ByteBufferPool we had.

View File

@ -140,6 +140,12 @@ static int doTestZeroCopyReads(hdfsFS fs, const char *fileName)
EXPECT_NULL(hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE)); EXPECT_NULL(hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE));
EXPECT_INT_EQ(EPROTONOSUPPORT, errno); EXPECT_INT_EQ(EPROTONOSUPPORT, errno);
/* Verify that setting a NULL ByteBufferPool class works. */
EXPECT_ZERO(hadoopRzOptionsSetByteBufferPool(opts, NULL));
EXPECT_ZERO(hadoopRzOptionsSetSkipChecksum(opts, 0));
EXPECT_NULL(hadoopReadZero(file, opts, TEST_ZEROCOPY_FULL_BLOCK_SIZE));
EXPECT_INT_EQ(EPROTONOSUPPORT, errno);
/* Now set a ByteBufferPool and try again. It should succeed this time. */ /* Now set a ByteBufferPool and try again. It should succeed this time. */
EXPECT_ZERO(hadoopRzOptionsSetByteBufferPool(opts, EXPECT_ZERO(hadoopRzOptionsSetByteBufferPool(opts,
ELASTIC_BYTE_BUFFER_POOL_CLASS)); ELASTIC_BYTE_BUFFER_POOL_CLASS));

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.namenode.INodeId; import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
@ -61,7 +62,7 @@ public class TestJsonUtil {
} }
@Test @Test
public void testToDatanodeInfoWithoutSecurePort() { public void testToDatanodeInfoWithoutSecurePort() throws Exception {
Map<String, Object> response = new HashMap<String, Object>(); Map<String, Object> response = new HashMap<String, Object>();
response.put("ipAddr", "127.0.0.1"); response.put("ipAddr", "127.0.0.1");
@ -84,4 +85,63 @@ public class TestJsonUtil {
JsonUtil.toDatanodeInfo(response); JsonUtil.toDatanodeInfo(response);
} }
@Test
public void testToDatanodeInfoWithName() throws Exception {
Map<String, Object> response = new HashMap<String, Object>();
// Older servers (1.x, 0.23, etc.) sends 'name' instead of ipAddr
// and xferPort.
String name = "127.0.0.1:1004";
response.put("name", name);
response.put("hostName", "localhost");
response.put("storageID", "fake-id");
response.put("infoPort", 1338l);
response.put("ipcPort", 1339l);
response.put("capacity", 1024l);
response.put("dfsUsed", 512l);
response.put("remaining", 512l);
response.put("blockPoolUsed", 512l);
response.put("lastUpdate", 0l);
response.put("xceiverCount", 4096l);
response.put("networkLocation", "foo.bar.baz");
response.put("adminState", "NORMAL");
response.put("cacheCapacity", 123l);
response.put("cacheUsed", 321l);
DatanodeInfo di = JsonUtil.toDatanodeInfo(response);
Assert.assertEquals(name, di.getXferAddr());
// The encoded result should contain name, ipAddr and xferPort.
Map<String, Object> r = JsonUtil.toJsonMap(di);
Assert.assertEquals(name, (String)r.get("name"));
Assert.assertEquals("127.0.0.1", (String)r.get("ipAddr"));
// In this test, it is Integer instead of Long since json was not actually
// involved in constructing the map.
Assert.assertEquals(1004, (int)(Integer)r.get("xferPort"));
// Invalid names
String[] badNames = {"127.0.0.1", "127.0.0.1:", ":", "127.0.0.1:sweet", ":123"};
for (String badName : badNames) {
response.put("name", badName);
checkDecodeFailure(response);
}
// Missing both name and ipAddr
response.remove("name");
checkDecodeFailure(response);
// Only missing xferPort
response.put("ipAddr", "127.0.0.1");
checkDecodeFailure(response);
}
private void checkDecodeFailure(Map<String, Object> map) {
try {
JsonUtil.toDatanodeInfo(map);
Assert.fail("Exception not thrown against bad input.");
} catch (Exception e) {
// expected
}
}
} }

View File

@ -182,9 +182,10 @@ public class TestMRApps {
MRApps.setClasspath(environment, job.getConfiguration()); MRApps.setClasspath(environment, job.getConfiguration());
assertTrue(environment.get("CLASSPATH").startsWith( assertTrue(environment.get("CLASSPATH").startsWith(
ApplicationConstants.Environment.PWD.$() + File.pathSeparator)); ApplicationConstants.Environment.PWD.$() + File.pathSeparator));
String yarnAppClasspath = String yarnAppClasspath = job.getConfiguration().get(
job.getConfiguration().get( YarnConfiguration.YARN_APPLICATION_CLASSPATH,
YarnConfiguration.YARN_APPLICATION_CLASSPATH); StringUtils.join(",",
YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH));
if (yarnAppClasspath != null) { if (yarnAppClasspath != null) {
yarnAppClasspath = yarnAppClasspath.replaceAll(",\\s*", File.pathSeparator) yarnAppClasspath = yarnAppClasspath.replaceAll(",\\s*", File.pathSeparator)
.trim(); .trim();
@ -217,7 +218,10 @@ public class TestMRApps {
MRApps.setClasspath(environment, conf); MRApps.setClasspath(environment, conf);
assertTrue(environment.get("CLASSPATH").startsWith( assertTrue(environment.get("CLASSPATH").startsWith(
ApplicationConstants.Environment.PWD.$() + File.pathSeparator)); ApplicationConstants.Environment.PWD.$() + File.pathSeparator));
String confClasspath = job.getConfiguration().get(YarnConfiguration.YARN_APPLICATION_CLASSPATH); String confClasspath = job.getConfiguration().get(
YarnConfiguration.YARN_APPLICATION_CLASSPATH,
StringUtils.join(",",
YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH));
if (confClasspath != null) { if (confClasspath != null) {
confClasspath = confClasspath.replaceAll(",\\s*", File.pathSeparator) confClasspath = confClasspath.replaceAll(",\\s*", File.pathSeparator)
.trim(); .trim();

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.RunningJob;
import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -86,8 +87,10 @@ public class TestEncryptedShuffle {
conf.set("dfs.block.access.token.enable", "false"); conf.set("dfs.block.access.token.enable", "false");
conf.set("dfs.permissions", "true"); conf.set("dfs.permissions", "true");
conf.set("hadoop.security.authentication", "simple"); conf.set("hadoop.security.authentication", "simple");
String cp = conf.get(YarnConfiguration.YARN_APPLICATION_CLASSPATH) + String cp = conf.get(YarnConfiguration.YARN_APPLICATION_CLASSPATH,
File.pathSeparator + classpathDir; StringUtils.join(",",
YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH))
+ File.pathSeparator + classpathDir;
conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, cp); conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, cp);
dfsCluster = new MiniDFSCluster(conf, 1, true, null); dfsCluster = new MiniDFSCluster(conf, 1, true, null);
FileSystem fileSystem = dfsCluster.getFileSystem(); FileSystem fileSystem = dfsCluster.getFileSystem();

View File

@ -860,4 +860,9 @@ public class ResourceSchedulerWrapper implements
public List<ApplicationAttemptId> getAppsInQueue(String queue) { public List<ApplicationAttemptId> getAppsInQueue(String queue) {
return scheduler.getAppsInQueue(queue); return scheduler.getAppsInQueue(queue);
} }
@Override
public RMContainer getRMContainer(ContainerId containerId) {
return null;
}
} }

View File

@ -58,6 +58,11 @@ Release 2.4.0 - UNRELEASED
YARN-1029. Added embedded leader election in the ResourceManager. (Karthik YARN-1029. Added embedded leader election in the ResourceManager. (Karthik
Kambatla via vinodkv) Kambatla via vinodkv)
YARN-1490. Introduced the ability to make ResourceManager optionally not kill
all containers when an ApplicationMaster exits. (Jian He via vinodkv)
YARN-1033. Expose RM active/standby state to Web UI and REST API (kasha)
IMPROVEMENTS IMPROVEMENTS
YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu) YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
@ -205,6 +210,8 @@ Release 2.4.0 - UNRELEASED
YARN-1568. Rename clusterid to clusterId in ActiveRMInfoProto (kasha) YARN-1568. Rename clusterid to clusterId in ActiveRMInfoProto (kasha)
YARN-1579. ActiveRMInfoProto fields should be optional (kasha)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES
@ -306,6 +313,9 @@ Release 2.4.0 - UNRELEASED
YARN-1293. Fixed TestContainerLaunch#testInvalidEnvSyntaxDiagnostics failure YARN-1293. Fixed TestContainerLaunch#testInvalidEnvSyntaxDiagnostics failure
caused by non-English system locale. (Tsuyoshi OZAWA via jianhe) caused by non-English system locale. (Tsuyoshi OZAWA via jianhe)
YARN-1574. RMDispatcher should be reset on transition to standby. (Xuan Gong
via kasha)
Release 2.3.0 - UNRELEASED Release 2.3.0 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES
@ -395,6 +405,9 @@ Release 2.3.0 - UNRELEASED
YARN-1438. Ensure container diagnostics includes exception from container YARN-1438. Ensure container diagnostics includes exception from container
launch. (stevel via acmurthy) launch. (stevel via acmurthy)
YARN-1138. yarn.application.classpath is set to point to $HADOOP_CONF_DIR
etc., which does not work on Windows. (Chuan Liu via cnauroth)
Release 2.2.0 - 2013-10-13 Release 2.2.0 - 2013-10-13
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
/** /**
@ -57,7 +58,8 @@ public abstract class ApplicationSubmissionContext {
ApplicationId applicationId, String applicationName, String queue, ApplicationId applicationId, String applicationName, String queue,
Priority priority, ContainerLaunchContext amContainer, Priority priority, ContainerLaunchContext amContainer,
boolean isUnmanagedAM, boolean cancelTokensWhenComplete, boolean isUnmanagedAM, boolean cancelTokensWhenComplete,
int maxAppAttempts, Resource resource, String applicationType) { int maxAppAttempts, Resource resource, String applicationType,
boolean keepContainers) {
ApplicationSubmissionContext context = ApplicationSubmissionContext context =
Records.newRecord(ApplicationSubmissionContext.class); Records.newRecord(ApplicationSubmissionContext.class);
context.setApplicationId(applicationId); context.setApplicationId(applicationId);
@ -70,9 +72,22 @@ public abstract class ApplicationSubmissionContext {
context.setMaxAppAttempts(maxAppAttempts); context.setMaxAppAttempts(maxAppAttempts);
context.setResource(resource); context.setResource(resource);
context.setApplicationType(applicationType); context.setApplicationType(applicationType);
context.setKeepContainersAcrossApplicationAttempts(keepContainers);
return context; return context;
} }
@Public
@Stable
public static ApplicationSubmissionContext newInstance(
ApplicationId applicationId, String applicationName, String queue,
Priority priority, ContainerLaunchContext amContainer,
boolean isUnmanagedAM, boolean cancelTokensWhenComplete,
int maxAppAttempts, Resource resource, String applicationType) {
return newInstance(applicationId, applicationName, queue, priority,
amContainer, isUnmanagedAM, cancelTokensWhenComplete, maxAppAttempts,
resource, null, false);
}
@Public @Public
@Stable @Stable
public static ApplicationSubmissionContext newInstance( public static ApplicationSubmissionContext newInstance(
@ -268,4 +283,35 @@ public abstract class ApplicationSubmissionContext {
@Public @Public
@Stable @Stable
public abstract void setApplicationType(String applicationType); public abstract void setApplicationType(String applicationType);
/**
* Get the flag which indicates whether to keep containers across application
* attempts or not.
*
* @return the flag which indicates whether to keep containers across
* application attempts or not.
*/
@Public
@Stable
public abstract boolean getKeepContainersAcrossApplicationAttempts();
/**
* Set the flag which indicates whether to keep containers across application
* attempts.
* <p>
* If the flag is true, running containers will not be killed when application
* attempt fails and these containers will be retrieved by the new application
* attempt on registration via
* {@link ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)}.
* </p>
*
* @param keepContainers
* the flag which indicates whether to keep containers across
* application attempts.
*/
@Public
@Stable
public abstract void setKeepContainersAcrossApplicationAttempts(
boolean keepContainers);
} }

View File

@ -46,10 +46,20 @@ public abstract class ContainerId implements Comparable<ContainerId>{
} }
/** /**
* Get the <code>ApplicationAttemptId</code> of the application to which * Get the <code>ApplicationAttemptId</code> of the application to which the
* the <code>Container</code> was assigned. * <code>Container</code> was assigned.
* @return <code>ApplicationAttemptId</code> of the application to which * <p>
* the <code>Container</code> was assigned * Note: If containers are kept alive across application attempts via
* {@link ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)}
* the <code>ContainerId</code> does not necessarily contain the current
* running application attempt's <code>ApplicationAttemptId</code> This
* container can be allocated by previously exited application attempt and
* managed by the current running attempt thus have the previous application
* attempt's <code>ApplicationAttemptId</code>.
* </p>
*
* @return <code>ApplicationAttemptId</code> of the application to which the
* <code>Container</code> was assigned
*/ */
@Public @Public
@Stable @Stable

View File

@ -138,6 +138,6 @@ message RMStateVersionProto {
///////////// RM Failover related records //////////////////////// ///////////// RM Failover related records ////////////////////////
////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////
message ActiveRMInfoProto { message ActiveRMInfoProto {
required string clusterId = 1; optional string clusterId = 1;
required string rmId = 2; optional string rmId = 2;
} }

View File

@ -248,6 +248,7 @@ message ApplicationSubmissionContextProto {
optional int32 maxAppAttempts = 8 [default = 0]; optional int32 maxAppAttempts = 8 [default = 0];
optional ResourceProto resource = 9; optional ResourceProto resource = 9;
optional string applicationType = 10 [default = "YARN"]; optional string applicationType = 10 [default = "YARN"];
optional bool keep_containers_across_application_attempts = 11 [default = false];
} }
enum ApplicationAccessTypeProto { enum ApplicationAccessTypeProto {

View File

@ -298,6 +298,19 @@ extends ApplicationSubmissionContext {
this.resource = resource; this.resource = resource;
} }
@Override
public void
setKeepContainersAcrossApplicationAttempts(boolean keepContainers) {
maybeInitBuilder();
builder.setKeepContainersAcrossApplicationAttempts(keepContainers);
}
@Override
public boolean getKeepContainersAcrossApplicationAttempts() {
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
return p.getKeepContainersAcrossApplicationAttempts();
}
private PriorityPBImpl convertFromProtoFormat(PriorityProto p) { private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
return new PriorityPBImpl(p); return new PriorityPBImpl(p);
} }

View File

@ -1016,10 +1016,29 @@
<!-- Applications' Configuration--> <!-- Applications' Configuration-->
<property> <property>
<description>CLASSPATH for YARN applications. A comma-separated list <description>
of CLASSPATH entries</description> CLASSPATH for YARN applications. A comma-separated list
<name>yarn.application.classpath</name> of CLASSPATH entries. When this value is empty, the following default
<value>$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*</value> CLASSPATH for YARN applications would be used.
For Linux:
$HADOOP_CONF_DIR,
$HADOOP_COMMON_HOME/share/hadoop/common/*,
$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,
$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,
$HADOOP_YARN_HOME/share/hadoop/yarn/*,
$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*
For Windows:
%HADOOP_CONF_DIR%,
%HADOOP_COMMON_HOME%/share/hadoop/common/*,
%HADOOP_COMMON_HOME%/share/hadoop/common/lib/*,
%HADOOP_HDFS_HOME%/share/hadoop/hdfs/*,
%HADOOP_HDFS_HOME%/share/hadoop/hdfs/lib/*,
%HADOOP_YARN_HOME%/share/hadoop/yarn/*,
%HADOOP_YARN_HOME%/share/hadoop/yarn/lib/*
</description>
<name>yarn.application.classpath</name>
<value></value>
</property> </property>
<!-- Other configuration --> <!-- Other configuration -->

View File

@ -338,6 +338,33 @@ public class TestCompositeService {
1, testService.getServices().size()); 1, testService.getServices().size());
} }
@Test
public void testRemoveService() {
CompositeService testService = new CompositeService("TestService") {
@Override
public void serviceInit(Configuration conf) {
Integer notAService = new Integer(0);
assertFalse("Added an integer as a service",
addIfService(notAService));
Service service1 = new AbstractService("Service1") {};
addIfService(service1);
Service service2 = new AbstractService("Service2") {};
addIfService(service2);
Service service3 = new AbstractService("Service3") {};
addIfService(service3);
removeService(service1);
}
};
testService.init(new Configuration());
assertEquals("Incorrect number of services",
2, testService.getServices().size());
}
public static class CompositeServiceAddingAChild extends CompositeService{ public static class CompositeServiceAddingAChild extends CompositeService{
Service child; Service child;

View File

@ -422,11 +422,18 @@ public class ApplicationMasterService extends AbstractService implements
throw e; throw e;
} }
try { RMApp app =
RMServerUtils.validateContainerReleaseRequest(release, appAttemptId); this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
} catch (InvalidContainerReleaseException e) { // In the case of work-preserving AM restart, it's possible for the
LOG.warn("Invalid container release by application " + appAttemptId, e); // AM to release containers from the earlier attempt.
throw e; if (!app.getApplicationSubmissionContext()
.getKeepContainersAcrossApplicationAttempts()) {
try {
RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
} catch (InvalidContainerReleaseException e) {
LOG.warn("Invalid container release by application " + appAttemptId, e);
throw e;
}
} }
// Send new requests to appAttempt. // Send new requests to appAttempt.
@ -434,8 +441,6 @@ public class ApplicationMasterService extends AbstractService implements
this.rScheduler.allocate(appAttemptId, ask, release, this.rScheduler.allocate(appAttemptId, ask, release,
blacklistAdditions, blacklistRemovals); blacklistAdditions, blacklistRemovals);
RMApp app = this.rmContext.getRMApps().get(
appAttemptId.getApplicationId());
RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId); RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
AllocateResponse allocateResponse = AllocateResponse allocateResponse =

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.service.Service;
import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.ShutdownHookManager;
@ -180,13 +181,11 @@ public class ResourceManager extends CompositeService implements Recoverable {
this.conf = conf; this.conf = conf;
this.rmContext = new RMContextImpl(); this.rmContext = new RMContextImpl();
rmDispatcher = createDispatcher(); // register the handlers for all AlwaysOn services using setupDispatcher().
rmDispatcher = setupDispatcher();
addIfService(rmDispatcher); addIfService(rmDispatcher);
rmContext.setDispatcher(rmDispatcher); rmContext.setDispatcher(rmDispatcher);
rmDispatcher.register(RMFatalEventType.class,
new ResourceManager.RMFatalEventDispatcher(this.rmContext, this));
adminService = createAdminService(); adminService = createAdminService();
addService(adminService); addService(adminService);
rmContext.setRMAdminService(adminService); rmContext.setRMAdminService(adminService);
@ -832,6 +831,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
HAServiceProtocol.HAServiceState.ACTIVE) { HAServiceProtocol.HAServiceState.ACTIVE) {
stopActiveServices(); stopActiveServices();
if (initialize) { if (initialize) {
resetDispatcher();
createAndInitActiveServices(); createAndInitActiveServices();
} }
} }
@ -994,4 +994,24 @@ public class ResourceManager extends CompositeService implements Recoverable {
YarnConfiguration.YARN_HTTP_POLICY_KEY, YarnConfiguration.YARN_HTTP_POLICY_KEY,
YarnConfiguration.YARN_HTTP_POLICY_DEFAULT))); YarnConfiguration.YARN_HTTP_POLICY_DEFAULT)));
} }
/**
* Register the handlers for alwaysOn services
*/
private Dispatcher setupDispatcher() {
Dispatcher dispatcher = createDispatcher();
dispatcher.register(RMFatalEventType.class,
new ResourceManager.RMFatalEventDispatcher(this.rmContext, this));
return dispatcher;
}
private void resetDispatcher() {
Dispatcher dispatcher = setupDispatcher();
((Service)dispatcher).init(this.conf);
((Service)dispatcher).start();
removeService((Service)rmDispatcher);
rmDispatcher = dispatcher;
addIfService(rmDispatcher);
rmContext.setDispatcher(rmDispatcher);
}
} }

View File

@ -23,14 +23,20 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
public class RMAppFailedAttemptEvent extends RMAppEvent { public class RMAppFailedAttemptEvent extends RMAppEvent {
private final String diagnostics; private final String diagnostics;
private final boolean transferStateFromPreviousAttempt;
public RMAppFailedAttemptEvent(ApplicationId appId, RMAppEventType event, public RMAppFailedAttemptEvent(ApplicationId appId, RMAppEventType event,
String diagnostics) { String diagnostics, boolean transferStateFromPreviousAttempt) {
super(appId, event); super(appId, event);
this.diagnostics = diagnostics; this.diagnostics = diagnostics;
this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
} }
public String getDiagnostics() { public String getDiagnostics() {
return this.diagnostics; return this.diagnostics;
} }
public boolean getTransferStateFromPreviousAttempt() {
return transferStateFromPreviousAttempt;
}
} }

View File

@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppStartAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@ -76,6 +77,7 @@ import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
@SuppressWarnings({ "rawtypes", "unchecked" })
public class RMAppImpl implements RMApp, Recoverable { public class RMAppImpl implements RMApp, Recoverable {
private static final Log LOG = LogFactory.getLog(RMAppImpl.class); private static final Log LOG = LogFactory.getLog(RMAppImpl.class);
@ -646,24 +648,26 @@ public class RMAppImpl implements RMApp, Recoverable {
for(int i=0; i<appState.getAttemptCount(); ++i) { for(int i=0; i<appState.getAttemptCount(); ++i) {
// create attempt // create attempt
createNewAttempt(false); createNewAttempt();
((RMAppAttemptImpl)this.currentAttempt).recover(state); ((RMAppAttemptImpl)this.currentAttempt).recover(state);
} }
} }
@SuppressWarnings("unchecked") private void createNewAttempt() {
private void createNewAttempt(boolean startAttempt) {
ApplicationAttemptId appAttemptId = ApplicationAttemptId appAttemptId =
ApplicationAttemptId.newInstance(applicationId, attempts.size() + 1); ApplicationAttemptId.newInstance(applicationId, attempts.size() + 1);
RMAppAttempt attempt = RMAppAttempt attempt =
new RMAppAttemptImpl(appAttemptId, rmContext, scheduler, masterService, new RMAppAttemptImpl(appAttemptId, rmContext, scheduler, masterService,
submissionContext, conf); submissionContext, conf, maxAppAttempts == attempts.size());
attempts.put(appAttemptId, attempt); attempts.put(appAttemptId, attempt);
currentAttempt = attempt; currentAttempt = attempt;
if(startAttempt) { }
handler.handle(
new RMAppAttemptEvent(appAttemptId, RMAppAttemptEventType.START)); private void
} createAndStartNewAttempt(boolean transferStateFromPreviousAttempt) {
createNewAttempt();
handler.handle(new RMAppStartAttemptEvent(currentAttempt.getAppAttemptId(),
transferStateFromPreviousAttempt));
} }
private void processNodeUpdate(RMAppNodeUpdateType type, RMNode node) { private void processNodeUpdate(RMAppNodeUpdateType type, RMNode node) {
@ -688,7 +692,6 @@ public class RMAppImpl implements RMApp, Recoverable {
}; };
} }
@SuppressWarnings("unchecked")
private static final class RMAppRecoveredTransition implements private static final class RMAppRecoveredTransition implements
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> { MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
@ -729,7 +732,6 @@ public class RMAppImpl implements RMApp, Recoverable {
private static final class AddApplicationToSchedulerTransition extends private static final class AddApplicationToSchedulerTransition extends
RMAppTransition { RMAppTransition {
@SuppressWarnings("unchecked")
@Override @Override
public void transition(RMAppImpl app, RMAppEvent event) { public void transition(RMAppImpl app, RMAppEvent event) {
if (event instanceof RMAppNewSavedEvent) { if (event instanceof RMAppNewSavedEvent) {
@ -751,14 +753,13 @@ public class RMAppImpl implements RMApp, Recoverable {
private static final class StartAppAttemptTransition extends RMAppTransition { private static final class StartAppAttemptTransition extends RMAppTransition {
@Override @Override
public void transition(RMAppImpl app, RMAppEvent event) { public void transition(RMAppImpl app, RMAppEvent event) {
app.createNewAttempt(true); app.createAndStartNewAttempt(false);
}; };
} }
private static final class FinalStateSavedTransition implements private static final class FinalStateSavedTransition implements
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> { MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
@SuppressWarnings({ "rawtypes", "unchecked" })
@Override @Override
public RMAppState transition(RMAppImpl app, RMAppEvent event) { public RMAppState transition(RMAppImpl app, RMAppEvent event) {
RMAppUpdateSavedEvent storeEvent = (RMAppUpdateSavedEvent) event; RMAppUpdateSavedEvent storeEvent = (RMAppUpdateSavedEvent) event;
@ -959,7 +960,6 @@ public class RMAppImpl implements RMApp, Recoverable {
} }
private static class KillAttemptTransition extends RMAppTransition { private static class KillAttemptTransition extends RMAppTransition {
@SuppressWarnings("unchecked")
@Override @Override
public void transition(RMAppImpl app, RMAppEvent event) { public void transition(RMAppImpl app, RMAppEvent event) {
app.stateBeforeKilling = app.getState(); app.stateBeforeKilling = app.getState();
@ -987,7 +987,6 @@ public class RMAppImpl implements RMApp, Recoverable {
return nodes; return nodes;
} }
@SuppressWarnings("unchecked")
public void transition(RMAppImpl app, RMAppEvent event) { public void transition(RMAppImpl app, RMAppEvent event) {
Set<NodeId> nodes = getNodesOnWhichAttemptRan(app); Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
for (NodeId nodeId : nodes) { for (NodeId nodeId : nodes) {
@ -1019,7 +1018,21 @@ public class RMAppImpl implements RMApp, Recoverable {
public RMAppState transition(RMAppImpl app, RMAppEvent event) { public RMAppState transition(RMAppImpl app, RMAppEvent event) {
if (!app.submissionContext.getUnmanagedAM() if (!app.submissionContext.getUnmanagedAM()
&& app.attempts.size() < app.maxAppAttempts) { && app.attempts.size() < app.maxAppAttempts) {
app.createNewAttempt(true); boolean transferStateFromPreviousAttempt = false;
RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event;
transferStateFromPreviousAttempt =
failedEvent.getTransferStateFromPreviousAttempt();
RMAppAttempt oldAttempt = app.currentAttempt;
app.createAndStartNewAttempt(transferStateFromPreviousAttempt);
// Transfer the state from the previous attempt to the current attempt.
// Note that the previous failed attempt may still be collecting the
// container events from the scheduler and update its data structures
// before the new attempt is created.
if (transferStateFromPreviousAttempt) {
((RMAppAttemptImpl) app.currentAttempt)
.transferStateFromPreviousAttempt(oldAttempt);
}
return initialState; return initialState;
} else { } else {
app.rememberTargetTransitionsAndStoreState(event, app.rememberTargetTransitionsAndStoreState(event,

View File

@ -129,9 +129,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private SecretKey clientTokenMasterKey = null; private SecretKey clientTokenMasterKey = null;
//nodes on while this attempt's containers ran //nodes on while this attempt's containers ran
private final Set<NodeId> ranNodes = private Set<NodeId> ranNodes =
new HashSet<NodeId>(); new HashSet<NodeId>();
private final List<ContainerStatus> justFinishedContainers = private List<ContainerStatus> justFinishedContainers =
new ArrayList<ContainerStatus>(); new ArrayList<ContainerStatus>();
private Container masterContainer; private Container masterContainer;
@ -148,7 +148,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private final StringBuilder diagnostics = new StringBuilder(); private final StringBuilder diagnostics = new StringBuilder();
private Configuration conf; private Configuration conf;
private final boolean isLastAttempt;
private static final ExpiredTransition EXPIRED_TRANSITION = private static final ExpiredTransition EXPIRED_TRANSITION =
new ExpiredTransition(); new ExpiredTransition();
@ -330,6 +330,12 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMAppAttemptEventType.KILL)) RMAppAttemptEventType.KILL))
// Transitions from FAILED State // Transitions from FAILED State
// For work-preserving AM restart, failed attempt are still capturing
// CONTAINER_FINISHED event and record the finished containers for the
// use by the next new attempt.
.addTransition(RMAppAttemptState.FAILED, RMAppAttemptState.FAILED,
RMAppAttemptEventType.CONTAINER_FINISHED,
new ContainerFinishedAtFailedTransition())
.addTransition( .addTransition(
RMAppAttemptState.FAILED, RMAppAttemptState.FAILED,
RMAppAttemptState.FAILED, RMAppAttemptState.FAILED,
@ -338,8 +344,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMAppAttemptEventType.KILL, RMAppAttemptEventType.KILL,
RMAppAttemptEventType.UNREGISTERED, RMAppAttemptEventType.UNREGISTERED,
RMAppAttemptEventType.STATUS_UPDATE, RMAppAttemptEventType.STATUS_UPDATE,
RMAppAttemptEventType.CONTAINER_ALLOCATED, RMAppAttemptEventType.CONTAINER_ALLOCATED))
RMAppAttemptEventType.CONTAINER_FINISHED))
// Transitions from FINISHING State // Transitions from FINISHING State
.addTransition(RMAppAttemptState.FINISHING, .addTransition(RMAppAttemptState.FINISHING,
@ -390,7 +395,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMContext rmContext, YarnScheduler scheduler, RMContext rmContext, YarnScheduler scheduler,
ApplicationMasterService masterService, ApplicationMasterService masterService,
ApplicationSubmissionContext submissionContext, ApplicationSubmissionContext submissionContext,
Configuration conf) { Configuration conf, boolean isLastAttempt) {
this.conf = conf; this.conf = conf;
this.applicationAttemptId = appAttemptId; this.applicationAttemptId = appAttemptId;
this.rmContext = rmContext; this.rmContext = rmContext;
@ -404,7 +409,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
this.writeLock = lock.writeLock(); this.writeLock = lock.writeLock();
this.proxiedTrackingUrl = generateProxyUriWithScheme(null); this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
this.isLastAttempt = isLastAttempt;
this.stateMachine = stateMachineFactory.make(this); this.stateMachine = stateMachineFactory.make(this);
} }
@ -685,6 +690,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
this.startTime = attemptState.getStartTime(); this.startTime = attemptState.getStartTime();
} }
public void transferStateFromPreviousAttempt(RMAppAttempt attempt) {
this.justFinishedContainers = attempt.getJustFinishedContainers();
this.ranNodes = attempt.getRanNodes();
}
private void recoverAppAttemptCredentials(Credentials appAttemptTokens) private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
throws IOException { throws IOException {
if (appAttemptTokens == null) { if (appAttemptTokens == null) {
@ -721,6 +731,12 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
public void transition(RMAppAttemptImpl appAttempt, public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) { RMAppAttemptEvent event) {
boolean transferStateFromPreviousAttempt = false;
if (event instanceof RMAppStartAttemptEvent) {
transferStateFromPreviousAttempt =
((RMAppStartAttemptEvent) event)
.getTransferStateFromPreviousAttempt();
}
appAttempt.startTime = System.currentTimeMillis(); appAttempt.startTime = System.currentTimeMillis();
// Register with the ApplicationMasterService // Register with the ApplicationMasterService
@ -740,9 +756,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
new Token<AMRMTokenIdentifier>(id, new Token<AMRMTokenIdentifier>(id,
appAttempt.rmContext.getAMRMTokenSecretManager()); appAttempt.rmContext.getAMRMTokenSecretManager());
// Add the applicationAttempt to the scheduler // Add the applicationAttempt to the scheduler and inform the scheduler
// whether to transfer the state from previous attempt.
appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent( appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent(
appAttempt.applicationAttemptId)); appAttempt.applicationAttemptId, transferStateFromPreviousAttempt));
} }
} }
@ -981,6 +998,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
// Tell the application and the scheduler // Tell the application and the scheduler
ApplicationId applicationId = appAttemptId.getApplicationId(); ApplicationId applicationId = appAttemptId.getApplicationId();
RMAppEvent appEvent = null; RMAppEvent appEvent = null;
boolean keepContainersAcrossAppAttempts = false;
switch (finalAttemptState) { switch (finalAttemptState) {
case FINISHED: case FINISHED:
{ {
@ -996,7 +1014,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
appEvent = appEvent =
new RMAppFailedAttemptEvent(applicationId, new RMAppFailedAttemptEvent(applicationId,
RMAppEventType.ATTEMPT_KILLED, RMAppEventType.ATTEMPT_KILLED,
"Application killed by user."); "Application killed by user.", false);
} }
break; break;
case FAILED: case FAILED:
@ -1004,10 +1022,17 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
// don't leave the tracking URL pointing to a non-existent AM // don't leave the tracking URL pointing to a non-existent AM
appAttempt.setTrackingUrlToRMAppPage(); appAttempt.setTrackingUrlToRMAppPage();
appAttempt.invalidateAMHostAndPort(); appAttempt.invalidateAMHostAndPort();
if (appAttempt.submissionContext
.getKeepContainersAcrossApplicationAttempts()
&& !appAttempt.isLastAttempt
&& !appAttempt.submissionContext.getUnmanagedAM()) {
keepContainersAcrossAppAttempts = true;
}
appEvent = appEvent =
new RMAppFailedAttemptEvent(applicationId, new RMAppFailedAttemptEvent(applicationId,
RMAppEventType.ATTEMPT_FAILED, RMAppEventType.ATTEMPT_FAILED, appAttempt.getDiagnostics(),
appAttempt.getDiagnostics()); keepContainersAcrossAppAttempts);
} }
break; break;
default: default:
@ -1019,7 +1044,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
appAttempt.eventHandler.handle(appEvent); appAttempt.eventHandler.handle(appEvent);
appAttempt.eventHandler.handle(new AppAttemptRemovedSchedulerEvent( appAttempt.eventHandler.handle(new AppAttemptRemovedSchedulerEvent(
appAttemptId, finalAttemptState)); appAttemptId, finalAttemptState, keepContainersAcrossAppAttempts));
appAttempt.removeCredentials(appAttempt); appAttempt.removeCredentials(appAttempt);
} }
} }
@ -1045,6 +1070,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
public void transition(RMAppAttemptImpl appAttempt, public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) { RMAppAttemptEvent event) {
appAttempt.checkAttemptStoreError(event); appAttempt.checkAttemptStoreError(event);
// TODO Today unmanaged AM client is waiting for app state to be Accepted to
// launch the AM. This is broken since we changed to start the attempt
// after the application is Accepted. We may need to introduce an attempt
// report that client can rely on to query the attempt state and choose to
// launch the unmanaged AM.
super.transition(appAttempt, event); super.transition(appAttempt, event);
} }
} }
@ -1346,6 +1376,20 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
} }
} }
private static final class ContainerFinishedAtFailedTransition
extends BaseTransition {
@Override
public void
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
RMAppAttemptContainerFinishedEvent containerFinishedEvent =
(RMAppAttemptContainerFinishedEvent) event;
ContainerStatus containerStatus =
containerFinishedEvent.getContainerStatus();
// Normal container. Add it in completed containers list
appAttempt.justFinishedContainers.add(containerStatus);
}
}
private static class ContainerFinishedFinalStateSavedTransition extends private static class ContainerFinishedFinalStateSavedTransition extends
BaseTransition { BaseTransition {
@Override @Override

View File

@ -0,0 +1,36 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
public class RMAppStartAttemptEvent extends RMAppAttemptEvent {
private final boolean transferStateFromPreviousAttempt;
public RMAppStartAttemptEvent(ApplicationAttemptId appAttemptId,
boolean transferStateFromPreviousAttempt) {
super(appAttemptId, RMAppAttemptEventType.START);
this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
}
public boolean getTransferStateFromPreviousAttempt() {
return transferStateFromPreviousAttempt;
}
}

View File

@ -61,7 +61,7 @@ public class AppSchedulingInfo {
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator()); new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
final Map<Priority, Map<String, ResourceRequest>> requests = final Map<Priority, Map<String, ResourceRequest>> requests =
new HashMap<Priority, Map<String, ResourceRequest>>(); new HashMap<Priority, Map<String, ResourceRequest>>();
final Set<String> blacklist = new HashSet<String>(); private Set<String> blacklist = new HashSet<String>();
//private final ApplicationStore store; //private final ApplicationStore store;
private final ActiveUsersManager activeUsersManager; private final ActiveUsersManager activeUsersManager;
@ -399,4 +399,15 @@ public class AppSchedulingInfo {
public synchronized void setQueue(Queue queue) { public synchronized void setQueue(Queue queue) {
this.queue = queue; this.queue = queue;
} }
public synchronized Set<String> getBlackList() {
return this.blacklist;
}
public synchronized void transferStateFromPreviousAppSchedulingInfo(
AppSchedulingInfo appInfo) {
// this.priorities = appInfo.getPriorities();
// this.requests = appInfo.getRequests();
this.blacklist = appInfo.getBlackList();
}
} }

View File

@ -26,6 +26,7 @@ public class SchedulerApplication {
private final Queue queue; private final Queue queue;
private final String user; private final String user;
private SchedulerApplicationAttempt currentAttempt;
public SchedulerApplication(Queue queue, String user) { public SchedulerApplication(Queue queue, String user) {
this.queue = queue; this.queue = queue;
@ -39,4 +40,12 @@ public class SchedulerApplication {
public String getUser() { public String getUser() {
return user; return user;
} }
public SchedulerApplicationAttempt getCurrentAppAttempt() {
return currentAttempt;
}
public void setCurrentAppAttempt(SchedulerApplicationAttempt currentAttempt) {
this.currentAttempt = currentAttempt;
}
} }

View File

@ -64,7 +64,7 @@ public abstract class SchedulerApplicationAttempt {
protected final AppSchedulingInfo appSchedulingInfo; protected final AppSchedulingInfo appSchedulingInfo;
protected final Map<ContainerId, RMContainer> liveContainers = protected Map<ContainerId, RMContainer> liveContainers =
new HashMap<ContainerId, RMContainer>(); new HashMap<ContainerId, RMContainer>();
protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers = protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers =
new HashMap<Priority, Map<NodeId, RMContainer>>(); new HashMap<Priority, Map<NodeId, RMContainer>>();
@ -73,7 +73,7 @@ public abstract class SchedulerApplicationAttempt {
protected final Resource currentReservation = Resource.newInstance(0, 0); protected final Resource currentReservation = Resource.newInstance(0, 0);
private Resource resourceLimit = Resource.newInstance(0, 0); private Resource resourceLimit = Resource.newInstance(0, 0);
protected final Resource currentConsumption = Resource.newInstance(0, 0); protected Resource currentConsumption = Resource.newInstance(0, 0);
protected List<RMContainer> newlyAllocatedContainers = protected List<RMContainer> newlyAllocatedContainers =
new ArrayList<RMContainer>(); new ArrayList<RMContainer>();
@ -407,4 +407,29 @@ public abstract class SchedulerApplicationAttempt {
Resources.add(currentConsumption, currentReservation)); Resources.add(currentConsumption, currentReservation));
} }
public synchronized Map<ContainerId, RMContainer> getLiveContainersMap() {
return this.liveContainers;
}
public synchronized Resource getResourceLimit() {
return this.resourceLimit;
}
public synchronized Map<Priority, Long> getLastScheduledContainer() {
return this.lastScheduledContainer;
}
public synchronized void transferStateFromPreviousAttempt(
SchedulerApplicationAttempt appAttempt) {
this.liveContainers = appAttempt.getLiveContainersMap();
// this.reReservations = appAttempt.reReservations;
this.currentConsumption = appAttempt.getCurrentConsumption();
this.resourceLimit = appAttempt.getResourceLimit();
// this.currentReservation = appAttempt.currentReservation;
// this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
// this.schedulingOpportunities = appAttempt.schedulingOpportunities;
this.lastScheduledContainer = appAttempt.getLastScheduledContainer();
this.appSchedulingInfo
.transferStateFromPreviousAppSchedulingInfo(appAttempt.appSchedulingInfo);
}
} }

View File

@ -19,13 +19,13 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler; package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.List; import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
/** /**
@ -170,4 +171,13 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
@LimitedPrivate("yarn") @LimitedPrivate("yarn")
@Stable @Stable
public List<ApplicationAttemptId> getAppsInQueue(String queueName); public List<ApplicationAttemptId> getAppsInQueue(String queueName);
/**
* Get the container for the given containerId.
* @param containerId
* @return the container for the given containerId.
*/
@LimitedPrivate("yarn")
@Unstable
public RMContainer getRMContainer(ContainerId containerId);
} }

View File

@ -63,14 +63,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@ -194,10 +195,6 @@ public class CapacityScheduler
protected Map<ApplicationId, SchedulerApplication> applications = protected Map<ApplicationId, SchedulerApplication> applications =
new ConcurrentHashMap<ApplicationId, SchedulerApplication>(); new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
@VisibleForTesting
protected Map<ApplicationAttemptId, FiCaSchedulerApp> appAttempts =
new ConcurrentHashMap<ApplicationAttemptId, FiCaSchedulerApp>();
private boolean initialized = false; private boolean initialized = false;
private ResourceCalculator calculator; private ResourceCalculator calculator;
@ -464,21 +461,27 @@ public class CapacityScheduler
} }
private synchronized void addApplicationAttempt( private synchronized void addApplicationAttempt(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId,
boolean transferStateFromPreviousAttempt) {
SchedulerApplication application = SchedulerApplication application =
applications.get(applicationAttemptId.getApplicationId()); applications.get(applicationAttemptId.getApplicationId());
CSQueue queue = (CSQueue) application.getQueue(); CSQueue queue = (CSQueue) application.getQueue();
FiCaSchedulerApp SchedulerApp = FiCaSchedulerApp attempt =
new FiCaSchedulerApp(applicationAttemptId, application.getUser(), new FiCaSchedulerApp(applicationAttemptId, application.getUser(),
queue, queue.getActiveUsersManager(), rmContext); queue, queue.getActiveUsersManager(), rmContext);
appAttempts.put(applicationAttemptId, SchedulerApp); if (transferStateFromPreviousAttempt) {
queue.submitApplicationAttempt(SchedulerApp, application.getUser()); attempt.transferStateFromPreviousAttempt(application
.getCurrentAppAttempt());
}
application.setCurrentAppAttempt(attempt);
queue.submitApplicationAttempt(attempt, application.getUser());
LOG.info("Added Application Attempt " + applicationAttemptId LOG.info("Added Application Attempt " + applicationAttemptId
+ " to scheduler from user " + application.getUser() + " in queue " + " to scheduler from user " + application.getUser() + " in queue "
+ queue.getQueueName()); + queue.getQueueName());
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler() .handle(
new RMAppAttemptEvent(applicationAttemptId, new RMAppAttemptEvent(applicationAttemptId,
RMAppAttemptEventType.ATTEMPT_ADDED)); RMAppAttemptEventType.ATTEMPT_ADDED));
} }
@ -486,7 +489,8 @@ public class CapacityScheduler
RMAppState finalState) { RMAppState finalState) {
SchedulerApplication application = applications.get(applicationId); SchedulerApplication application = applications.get(applicationId);
if (application == null){ if (application == null){
// The AppRemovedSchedulerEvent maybe sent on recovery for completed apps. // The AppRemovedSchedulerEvent maybe sent on recovery for completed apps,
// ignore it.
return; return;
} }
CSQueue queue = (CSQueue) application.getQueue(); CSQueue queue = (CSQueue) application.getQueue();
@ -501,52 +505,56 @@ public class CapacityScheduler
private synchronized void doneApplicationAttempt( private synchronized void doneApplicationAttempt(
ApplicationAttemptId applicationAttemptId, ApplicationAttemptId applicationAttemptId,
RMAppAttemptState rmAppAttemptFinalState) { RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
LOG.info("Application Attempt " + applicationAttemptId + " is done." + LOG.info("Application Attempt " + applicationAttemptId + " is done." +
" finalState=" + rmAppAttemptFinalState); " finalState=" + rmAppAttemptFinalState);
FiCaSchedulerApp application = getApplication(applicationAttemptId); FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
SchedulerApplication application =
applications.get(applicationAttemptId.getApplicationId());
if (application == null) { if (application == null || attempt == null) {
// throw new IOException("Unknown application " + applicationId +
// " has completed!");
LOG.info("Unknown application " + applicationAttemptId + " has completed!"); LOG.info("Unknown application " + applicationAttemptId + " has completed!");
return; return;
} }
// Release all the running containers // Release all the allocated, acquired, running containers
for (RMContainer rmContainer : application.getLiveContainers()) { for (RMContainer rmContainer : attempt.getLiveContainers()) {
completedContainer(rmContainer, if (keepContainers
SchedulerUtils.createAbnormalContainerStatus( && rmContainer.getState().equals(RMContainerState.RUNNING)) {
rmContainer.getContainerId(), // do not kill the running container in the case of work-preserving AM
SchedulerUtils.COMPLETED_APPLICATION), // restart.
RMContainerEventType.KILL); LOG.info("Skip killing " + rmContainer.getContainerId());
continue;
}
completedContainer(
rmContainer,
SchedulerUtils.createAbnormalContainerStatus(
rmContainer.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
RMContainerEventType.KILL);
} }
// Release all reserved containers // Release all reserved containers
for (RMContainer rmContainer : application.getReservedContainers()) { for (RMContainer rmContainer : attempt.getReservedContainers()) {
completedContainer(rmContainer, completedContainer(
SchedulerUtils.createAbnormalContainerStatus( rmContainer,
rmContainer.getContainerId(), SchedulerUtils.createAbnormalContainerStatus(
"Application Complete"), rmContainer.getContainerId(), "Application Complete"),
RMContainerEventType.KILL); RMContainerEventType.KILL);
} }
// Clean up pending requests, metrics etc. // Clean up pending requests, metrics etc.
application.stop(rmAppAttemptFinalState); attempt.stop(rmAppAttemptFinalState);
// Inform the queue // Inform the queue
String queueName = application.getQueue().getQueueName(); String queueName = attempt.getQueue().getQueueName();
CSQueue queue = queues.get(queueName); CSQueue queue = queues.get(queueName);
if (!(queue instanceof LeafQueue)) { if (!(queue instanceof LeafQueue)) {
LOG.error("Cannot finish application " + "from non-leaf queue: " LOG.error("Cannot finish application " + "from non-leaf queue: "
+ queueName); + queueName);
} else { } else {
queue.finishApplicationAttempt(application, queue.getQueueName()); queue.finishApplicationAttempt(attempt, queue.getQueueName());
} }
// Remove from our data-structure
appAttempts.remove(applicationAttemptId);
} }
private static final Allocation EMPTY_ALLOCATION = private static final Allocation EMPTY_ALLOCATION =
@ -558,7 +566,7 @@ public class CapacityScheduler
List<ResourceRequest> ask, List<ContainerId> release, List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals) { List<String> blacklistAdditions, List<String> blacklistRemovals) {
FiCaSchedulerApp application = getApplication(applicationAttemptId); FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.info("Calling allocate on removed " + LOG.info("Calling allocate on removed " +
"or non existant application " + applicationAttemptId); "or non existant application " + applicationAttemptId);
@ -701,7 +709,7 @@ public class CapacityScheduler
RMContainer reservedContainer = node.getReservedContainer(); RMContainer reservedContainer = node.getReservedContainer();
if (reservedContainer != null) { if (reservedContainer != null) {
FiCaSchedulerApp reservedApplication = FiCaSchedulerApp reservedApplication =
getApplication(reservedContainer.getApplicationAttemptId()); getCurrentAttemptForContainer(reservedContainer.getContainerId());
// Try to fulfill the reservation // Try to fulfill the reservation
LOG.info("Trying to fulfill reservation for application " + LOG.info("Trying to fulfill reservation for application " +
@ -738,12 +746,11 @@ public class CapacityScheduler
private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
// Get the application for the finished container // Get the application for the finished container
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId);
FiCaSchedulerApp application = getApplication(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.info("Unknown application: " + applicationAttemptId + LOG.info("Unknown application "
" launched container " + containerId + + containerId.getApplicationAttemptId().getApplicationId()
" on node: " + node); + " launched container " + containerId + " on node: " + node);
this.rmContext.getDispatcher().getEventHandler() this.rmContext.getDispatcher().getEventHandler()
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
return; return;
@ -791,7 +798,8 @@ public class CapacityScheduler
{ {
AppAttemptAddedSchedulerEvent appAttemptAddedEvent = AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
(AppAttemptAddedSchedulerEvent) event; (AppAttemptAddedSchedulerEvent) event;
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId()); addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
} }
break; break;
case APP_ATTEMPT_REMOVED: case APP_ATTEMPT_REMOVED:
@ -799,7 +807,8 @@ public class CapacityScheduler
AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent = AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
(AppAttemptRemovedSchedulerEvent) event; (AppAttemptRemovedSchedulerEvent) event;
doneApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(), doneApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(),
appAttemptRemovedEvent.getFinalAttemptState()); appAttemptRemovedEvent.getFinalAttemptState(),
appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
} }
break; break;
case CONTAINER_EXPIRED: case CONTAINER_EXPIRED:
@ -874,13 +883,13 @@ public class CapacityScheduler
Container container = rmContainer.getContainer(); Container container = rmContainer.getContainer();
// Get the application for the finished container // Get the application for the finished container
ApplicationAttemptId applicationAttemptId = FiCaSchedulerApp application =
container.getId().getApplicationAttemptId(); getCurrentAttemptForContainer(container.getId());
FiCaSchedulerApp application = getApplication(applicationAttemptId); ApplicationId appId =
container.getId().getApplicationAttemptId().getApplicationId();
if (application == null) { if (application == null) {
LOG.info("Container " + container + " of" + LOG.info("Container " + container + " of" + " unknown application "
" unknown application " + applicationAttemptId + + appId + " completed with event " + event);
" completed with event " + event);
return; return;
} }
@ -892,28 +901,33 @@ public class CapacityScheduler
queue.completedContainer(clusterResource, application, node, queue.completedContainer(clusterResource, application, node,
rmContainer, containerStatus, event, null); rmContainer, containerStatus, event, null);
LOG.info("Application " + applicationAttemptId + LOG.info("Application attempt " + application.getApplicationAttemptId()
" released container " + container.getId() + + " released container " + container.getId() + " on node: " + node
" on node: " + node + + " with event: " + event);
" with event: " + event);
} }
@Lock(Lock.NoLock.class) @Lock(Lock.NoLock.class)
FiCaSchedulerApp getApplication(ApplicationAttemptId applicationAttemptId) { FiCaSchedulerApp getApplicationAttempt(
return appAttempts.get(applicationAttemptId); ApplicationAttemptId applicationAttemptId) {
SchedulerApplication app =
applications.get(applicationAttemptId.getApplicationId());
if (app != null) {
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
}
return null;
} }
@Override @Override
public SchedulerAppReport getSchedulerAppInfo( public SchedulerAppReport getSchedulerAppInfo(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId) {
FiCaSchedulerApp app = getApplication(applicationAttemptId); FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
return app == null ? null : new SchedulerAppReport(app); return app == null ? null : new SchedulerAppReport(app);
} }
@Override @Override
public ApplicationResourceUsageReport getAppResourceUsageReport( public ApplicationResourceUsageReport getAppResourceUsageReport(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId) {
FiCaSchedulerApp app = getApplication(applicationAttemptId); FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
return app == null ? null : app.getResourceUsageReport(); return app == null ? null : app.getResourceUsageReport();
} }
@ -922,10 +936,22 @@ public class CapacityScheduler
return nodes.get(nodeId); return nodes.get(nodeId);
} }
private RMContainer getRMContainer(ContainerId containerId) { @Override
FiCaSchedulerApp application = public RMContainer getRMContainer(ContainerId containerId) {
getApplication(containerId.getApplicationAttemptId()); FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
return (application == null) ? null : application.getRMContainer(containerId); return (attempt == null) ? null : attempt.getRMContainer(containerId);
}
@VisibleForTesting
public FiCaSchedulerApp getCurrentAttemptForContainer(
ContainerId containerId) {
SchedulerApplication app =
applications.get(containerId.getApplicationAttemptId()
.getApplicationId());
if (app != null) {
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
}
return null;
} }
@Override @Override
@ -958,7 +984,7 @@ public class CapacityScheduler
LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() + LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() +
" container: " + cont.toString()); " container: " + cont.toString());
} }
FiCaSchedulerApp app = appAttempts.get(aid); FiCaSchedulerApp app = getApplicationAttempt(aid);
if (app != null) { if (app != null) {
app.addPreemptContainer(cont.getContainerId()); app.addPreemptContainer(cont.getContainerId());
} }

View File

@ -219,7 +219,8 @@ public class FiCaSchedulerNode extends SchedulerNode {
" on node " + this.reservedContainer.getReservedNode()); " on node " + this.reservedContainer.getReservedNode());
} }
// Cannot reserve more than one application on a given node! // Cannot reserve more than one application attempt on a given node!
// Reservation is still against attempt.
if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals( if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals(
reservedContainer.getContainer().getId().getApplicationAttemptId())) { reservedContainer.getContainer().getId().getApplicationAttemptId())) {
throw new IllegalStateException("Trying to reserve" + throw new IllegalStateException("Trying to reserve" +

View File

@ -23,14 +23,21 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
public class AppAttemptAddedSchedulerEvent extends SchedulerEvent { public class AppAttemptAddedSchedulerEvent extends SchedulerEvent {
private final ApplicationAttemptId applicationAttemptId; private final ApplicationAttemptId applicationAttemptId;
private final boolean transferStateFromPreviousAttempt;
public AppAttemptAddedSchedulerEvent( public AppAttemptAddedSchedulerEvent(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId,
boolean transferStateFromPreviousAttempt) {
super(SchedulerEventType.APP_ATTEMPT_ADDED); super(SchedulerEventType.APP_ATTEMPT_ADDED);
this.applicationAttemptId = applicationAttemptId; this.applicationAttemptId = applicationAttemptId;
this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
} }
public ApplicationAttemptId getApplicationAttemptId() { public ApplicationAttemptId getApplicationAttemptId() {
return applicationAttemptId; return applicationAttemptId;
} }
public boolean getTransferStateFromPreviousAttempt() {
return transferStateFromPreviousAttempt;
}
} }

View File

@ -25,13 +25,15 @@ public class AppAttemptRemovedSchedulerEvent extends SchedulerEvent {
private final ApplicationAttemptId applicationAttemptId; private final ApplicationAttemptId applicationAttemptId;
private final RMAppAttemptState finalAttemptState; private final RMAppAttemptState finalAttemptState;
private final boolean keepContainersAcrossAppAttempts;
public AppAttemptRemovedSchedulerEvent( public AppAttemptRemovedSchedulerEvent(
ApplicationAttemptId applicationAttemptId, ApplicationAttemptId applicationAttemptId,
RMAppAttemptState finalAttemptState) { RMAppAttemptState finalAttemptState, boolean keepContainers) {
super(SchedulerEventType.APP_ATTEMPT_REMOVED); super(SchedulerEventType.APP_ATTEMPT_REMOVED);
this.applicationAttemptId = applicationAttemptId; this.applicationAttemptId = applicationAttemptId;
this.finalAttemptState = finalAttemptState; this.finalAttemptState = finalAttemptState;
this.keepContainersAcrossAppAttempts = keepContainers;
} }
public ApplicationAttemptId getApplicationAttemptID() { public ApplicationAttemptId getApplicationAttemptID() {
@ -41,4 +43,8 @@ public class AppAttemptRemovedSchedulerEvent extends SchedulerEvent {
public RMAppAttemptState getFinalAttemptState() { public RMAppAttemptState getFinalAttemptState() {
return this.finalAttemptState; return this.finalAttemptState;
} }
public boolean getKeepContainersAcrossAppAttempts() {
return this.keepContainersAcrossAppAttempts;
}
} }

View File

@ -162,12 +162,6 @@ public class FairScheduler implements ResourceScheduler {
protected Map<ApplicationId, SchedulerApplication> applications = protected Map<ApplicationId, SchedulerApplication> applications =
new ConcurrentHashMap<ApplicationId, SchedulerApplication>(); new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
// This stores per-application-attempt scheduling information, indexed by
// attempt ID's for fast lookup.
@VisibleForTesting
protected Map<ApplicationAttemptId, FSSchedulerApp> appAttempts =
new ConcurrentHashMap<ApplicationAttemptId, FSSchedulerApp>();
// Nodes in the cluster, indexed by NodeId // Nodes in the cluster, indexed by NodeId
private Map<NodeId, FSSchedulerNode> nodes = private Map<NodeId, FSSchedulerNode> nodes =
new ConcurrentHashMap<NodeId, FSSchedulerNode>(); new ConcurrentHashMap<NodeId, FSSchedulerNode>();
@ -262,10 +256,21 @@ public class FairScheduler implements ResourceScheduler {
return queueMgr; return queueMgr;
} }
private RMContainer getRMContainer(ContainerId containerId) { @Override
FSSchedulerApp application = public RMContainer getRMContainer(ContainerId containerId) {
appAttempts.get(containerId.getApplicationAttemptId()); FSSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
return (application == null) ? null : application.getRMContainer(containerId); return (attempt == null) ? null : attempt.getRMContainer(containerId);
}
private FSSchedulerApp getCurrentAttemptForContainer(
ContainerId containerId) {
SchedulerApplication app =
applications.get(containerId.getApplicationAttemptId()
.getApplicationId());
if (app != null) {
return (FSSchedulerApp) app.getCurrentAppAttempt();
}
return null;
} }
/** /**
@ -640,7 +645,8 @@ public class FairScheduler implements ResourceScheduler {
applications.put(applicationId, application); applications.put(applicationId, application);
LOG.info("Accepted application " + applicationId + " from user: " + user LOG.info("Accepted application " + applicationId + " from user: " + user
+ ", in queue: " + queueName); + ", in queue: " + queueName + ", currently num of applications: "
+ applications.size());
rmContext.getDispatcher().getEventHandler() rmContext.getDispatcher().getEventHandler()
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED)); .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
} }
@ -649,31 +655,35 @@ public class FairScheduler implements ResourceScheduler {
* Add a new application attempt to the scheduler. * Add a new application attempt to the scheduler.
*/ */
protected synchronized void addApplicationAttempt( protected synchronized void addApplicationAttempt(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId,
boolean transferStateFromPreviousAttempt) {
SchedulerApplication application = SchedulerApplication application =
applications.get(applicationAttemptId.getApplicationId()); applications.get(applicationAttemptId.getApplicationId());
String user = application.getUser(); String user = application.getUser();
FSLeafQueue queue = (FSLeafQueue) application.getQueue(); FSLeafQueue queue = (FSLeafQueue) application.getQueue();
FSSchedulerApp schedulerApp = FSSchedulerApp attempt =
new FSSchedulerApp(applicationAttemptId, user, new FSSchedulerApp(applicationAttemptId, user,
queue, new ActiveUsersManager(getRootQueueMetrics()), queue, new ActiveUsersManager(getRootQueueMetrics()),
rmContext); rmContext);
if (transferStateFromPreviousAttempt) {
attempt.transferStateFromPreviousAttempt(application
.getCurrentAppAttempt());
}
application.setCurrentAppAttempt(attempt);
boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user); boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user);
queue.addApp(schedulerApp, runnable); queue.addApp(attempt, runnable);
if (runnable) { if (runnable) {
maxRunningEnforcer.trackRunnableApp(schedulerApp); maxRunningEnforcer.trackRunnableApp(attempt);
} else { } else {
maxRunningEnforcer.trackNonRunnableApp(schedulerApp); maxRunningEnforcer.trackNonRunnableApp(attempt);
} }
queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId()); queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId());
appAttempts.put(applicationAttemptId, schedulerApp);
LOG.info("Added Application Attempt " + applicationAttemptId LOG.info("Added Application Attempt " + applicationAttemptId
+ " to scheduler from user: " + user + ", currently active: " + " to scheduler from user: " + user);
+ appAttempts.size());
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptEvent(applicationAttemptId, new RMAppAttemptEvent(applicationAttemptId,
RMAppAttemptEventType.ATTEMPT_ADDED)); RMAppAttemptEventType.ATTEMPT_ADDED));
@ -709,19 +719,27 @@ public class FairScheduler implements ResourceScheduler {
private synchronized void removeApplicationAttempt( private synchronized void removeApplicationAttempt(
ApplicationAttemptId applicationAttemptId, ApplicationAttemptId applicationAttemptId,
RMAppAttemptState rmAppAttemptFinalState) { RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
LOG.info("Application " + applicationAttemptId + " is done." + LOG.info("Application " + applicationAttemptId + " is done." +
" finalState=" + rmAppAttemptFinalState); " finalState=" + rmAppAttemptFinalState);
SchedulerApplication application =
applications.get(applicationAttemptId.getApplicationId());
FSSchedulerApp attempt = getSchedulerApp(applicationAttemptId);
FSSchedulerApp application = appAttempts.get(applicationAttemptId); if (attempt == null || application == null) {
if (application == null) {
LOG.info("Unknown application " + applicationAttemptId + " has completed!"); LOG.info("Unknown application " + applicationAttemptId + " has completed!");
return; return;
} }
// Release all the running containers // Release all the running containers
for (RMContainer rmContainer : application.getLiveContainers()) { for (RMContainer rmContainer : attempt.getLiveContainers()) {
if (keepContainers
&& rmContainer.getState().equals(RMContainerState.RUNNING)) {
// do not kill the running container in the case of work-preserving AM
// restart.
LOG.info("Skip killing " + rmContainer.getContainerId());
continue;
}
completedContainer(rmContainer, completedContainer(rmContainer,
SchedulerUtils.createAbnormalContainerStatus( SchedulerUtils.createAbnormalContainerStatus(
rmContainer.getContainerId(), rmContainer.getContainerId(),
@ -730,30 +748,26 @@ public class FairScheduler implements ResourceScheduler {
} }
// Release all reserved containers // Release all reserved containers
for (RMContainer rmContainer : application.getReservedContainers()) { for (RMContainer rmContainer : attempt.getReservedContainers()) {
completedContainer(rmContainer, completedContainer(rmContainer,
SchedulerUtils.createAbnormalContainerStatus( SchedulerUtils.createAbnormalContainerStatus(
rmContainer.getContainerId(), rmContainer.getContainerId(),
"Application Complete"), "Application Complete"),
RMContainerEventType.KILL); RMContainerEventType.KILL);
} }
// Clean up pending requests, metrics etc. // Clean up pending requests, metrics etc.
application.stop(rmAppAttemptFinalState); attempt.stop(rmAppAttemptFinalState);
// Inform the queue // Inform the queue
FSLeafQueue queue = queueMgr.getLeafQueue(application.getQueue() FSLeafQueue queue = queueMgr.getLeafQueue(attempt.getQueue()
.getQueueName(), false); .getQueueName(), false);
boolean wasRunnable = queue.removeApp(application); boolean wasRunnable = queue.removeApp(attempt);
if (wasRunnable) { if (wasRunnable) {
maxRunningEnforcer.updateRunnabilityOnAppRemoval(application); maxRunningEnforcer.updateRunnabilityOnAppRemoval(attempt);
} else { } else {
maxRunningEnforcer.untrackNonRunnableApp(application); maxRunningEnforcer.untrackNonRunnableApp(attempt);
} }
// Remove from our data-structure
appAttempts.remove(applicationAttemptId);
} }
/** /**
@ -769,11 +783,13 @@ public class FairScheduler implements ResourceScheduler {
Container container = rmContainer.getContainer(); Container container = rmContainer.getContainer();
// Get the application for the finished container // Get the application for the finished container
ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); FSSchedulerApp application =
FSSchedulerApp application = appAttempts.get(applicationAttemptId); getCurrentAttemptForContainer(container.getId());
ApplicationId appId =
container.getId().getApplicationAttemptId().getApplicationId();
if (application == null) { if (application == null) {
LOG.info("Container " + container + " of" + LOG.info("Container " + container + " of" +
" unknown application " + applicationAttemptId + " unknown application attempt " + appId +
" completed with event " + event); " completed with event " + event);
return; return;
} }
@ -790,10 +806,9 @@ public class FairScheduler implements ResourceScheduler {
updateRootQueueMetrics(); updateRootQueueMetrics();
} }
LOG.info("Application " + applicationAttemptId + LOG.info("Application attempt " + application.getApplicationAttemptId()
" released container " + container.getId() + + " released container " + container.getId() + " on node: " + node
" on node: " + node + + " with event: " + event);
" with event: " + event);
} }
private synchronized void addNode(RMNode node) { private synchronized void addNode(RMNode node) {
@ -844,7 +859,7 @@ public class FairScheduler implements ResourceScheduler {
List<ResourceRequest> ask, List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) { List<ResourceRequest> ask, List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
// Make sure this application exists // Make sure this application exists
FSSchedulerApp application = appAttempts.get(appAttemptId); FSSchedulerApp application = getSchedulerApp(appAttemptId);
if (application == null) { if (application == null) {
LOG.info("Calling allocate on removed " + LOG.info("Calling allocate on removed " +
"or non existant application " + appAttemptId); "or non existant application " + appAttemptId);
@ -914,12 +929,11 @@ public class FairScheduler implements ResourceScheduler {
*/ */
private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) { private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) {
// Get the application for the finished container // Get the application for the finished container
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FSSchedulerApp application = getCurrentAttemptForContainer(containerId);
FSSchedulerApp application = appAttempts.get(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.info("Unknown application: " + applicationAttemptId + LOG.info("Unknown application "
" launched container " + containerId + + containerId.getApplicationAttemptId().getApplicationId()
" on node: " + node); + " launched container " + containerId + " on node: " + node);
return; return;
} }
@ -1058,28 +1072,34 @@ public class FairScheduler implements ResourceScheduler {
} }
public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) { public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
return appAttempts.get(appAttemptId); SchedulerApplication app =
applications.get(appAttemptId.getApplicationId());
if (app != null) {
return (FSSchedulerApp) app.getCurrentAppAttempt();
}
return null;
} }
@Override @Override
public SchedulerAppReport getSchedulerAppInfo( public SchedulerAppReport getSchedulerAppInfo(
ApplicationAttemptId appAttemptId) { ApplicationAttemptId appAttemptId) {
if (!appAttempts.containsKey(appAttemptId)) { FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
if (attempt == null) {
LOG.error("Request for appInfo of unknown attempt" + appAttemptId); LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
return null; return null;
} }
return new SchedulerAppReport(appAttempts.get(appAttemptId)); return new SchedulerAppReport(attempt);
} }
@Override @Override
public ApplicationResourceUsageReport getAppResourceUsageReport( public ApplicationResourceUsageReport getAppResourceUsageReport(
ApplicationAttemptId appAttemptId) { ApplicationAttemptId appAttemptId) {
FSSchedulerApp app = appAttempts.get(appAttemptId); FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
if (app == null) { if (attempt == null) {
LOG.error("Request for appInfo of unknown attempt" + appAttemptId); LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
return null; return null;
} }
return app.getResourceUsageReport(); return attempt.getResourceUsageReport();
} }
/** /**
@ -1145,7 +1165,8 @@ public class FairScheduler implements ResourceScheduler {
} }
AppAttemptAddedSchedulerEvent appAttemptAddedEvent = AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
(AppAttemptAddedSchedulerEvent) event; (AppAttemptAddedSchedulerEvent) event;
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId()); addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
break; break;
case APP_ATTEMPT_REMOVED: case APP_ATTEMPT_REMOVED:
if (!(event instanceof AppAttemptRemovedSchedulerEvent)) { if (!(event instanceof AppAttemptRemovedSchedulerEvent)) {
@ -1153,8 +1174,10 @@ public class FairScheduler implements ResourceScheduler {
} }
AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent = AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
(AppAttemptRemovedSchedulerEvent) event; (AppAttemptRemovedSchedulerEvent) event;
removeApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(), removeApplicationAttempt(
appAttemptRemovedEvent.getFinalAttemptState()); appAttemptRemovedEvent.getApplicationAttemptID(),
appAttemptRemovedEvent.getFinalAttemptState(),
appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
break; break;
case CONTAINER_EXPIRED: case CONTAINER_EXPIRED:
if (!(event instanceof ContainerExpiredSchedulerEvent)) { if (!(event instanceof ContainerExpiredSchedulerEvent)) {

View File

@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
@ -123,15 +124,11 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
private Resource maximumAllocation; private Resource maximumAllocation;
private boolean usePortForNodeName; private boolean usePortForNodeName;
// Use ConcurrentSkipListMap because applications need to be ordered
@VisibleForTesting @VisibleForTesting
protected Map<ApplicationId, SchedulerApplication> applications = protected Map<ApplicationId, SchedulerApplication> applications =
new ConcurrentSkipListMap<ApplicationId, SchedulerApplication>(); new ConcurrentSkipListMap<ApplicationId, SchedulerApplication>();
// Use ConcurrentSkipListMap because applications need to be ordered
@VisibleForTesting
protected Map<ApplicationAttemptId, FiCaSchedulerApp> appAttempts
= new ConcurrentSkipListMap<ApplicationAttemptId, FiCaSchedulerApp>();
private ActiveUsersManager activeUsersManager; private ActiveUsersManager activeUsersManager;
private static final String DEFAULT_QUEUE_NAME = "default"; private static final String DEFAULT_QUEUE_NAME = "default";
@ -270,7 +267,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
public Allocation allocate( public Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask, ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) { List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
FiCaSchedulerApp application = getApplication(applicationAttemptId); FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.error("Calling allocate on removed " + LOG.error("Calling allocate on removed " +
"or non existant application " + applicationAttemptId); "or non existant application " + applicationAttemptId);
@ -336,22 +333,26 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
} }
@VisibleForTesting @VisibleForTesting
FiCaSchedulerApp getApplication( FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
ApplicationAttemptId applicationAttemptId) { SchedulerApplication app =
return appAttempts.get(applicationAttemptId); applications.get(applicationAttemptId.getApplicationId());
if (app != null) {
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
}
return null;
} }
@Override @Override
public SchedulerAppReport getSchedulerAppInfo( public SchedulerAppReport getSchedulerAppInfo(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId) {
FiCaSchedulerApp app = getApplication(applicationAttemptId); FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
return app == null ? null : new SchedulerAppReport(app); return app == null ? null : new SchedulerAppReport(app);
} }
@Override @Override
public ApplicationResourceUsageReport getAppResourceUsageReport( public ApplicationResourceUsageReport getAppResourceUsageReport(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId) {
FiCaSchedulerApp app = getApplication(applicationAttemptId); FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
return app == null ? null : app.getResourceUsageReport(); return app == null ? null : app.getResourceUsageReport();
} }
@ -364,13 +365,15 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
SchedulerApplication application = SchedulerApplication application =
new SchedulerApplication(null, user); new SchedulerApplication(null, user);
applications.put(applicationId, application); applications.put(applicationId, application);
LOG.info("Accepted application " + applicationId + " from user: " + user); LOG.info("Accepted application " + applicationId + " from user: " + user
+ ", currently num of applications: " + applications.size());
rmContext.getDispatcher().getEventHandler() rmContext.getDispatcher().getEventHandler()
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED)); .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
} }
private synchronized void addApplicationAttempt( private synchronized void
ApplicationAttemptId appAttemptId) { addApplicationAttempt(ApplicationAttemptId appAttemptId,
boolean transferStateFromPreviousAttempt) {
SchedulerApplication application = SchedulerApplication application =
applications.get(appAttemptId.getApplicationId()); applications.get(appAttemptId.getApplicationId());
String user = application.getUser(); String user = application.getUser();
@ -378,11 +381,16 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
FiCaSchedulerApp schedulerApp = FiCaSchedulerApp schedulerApp =
new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE, new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE,
activeUsersManager, this.rmContext); activeUsersManager, this.rmContext);
appAttempts.put(appAttemptId, schedulerApp);
if (transferStateFromPreviousAttempt) {
schedulerApp.transferStateFromPreviousAttempt(application
.getCurrentAppAttempt());
}
application.setCurrentAppAttempt(schedulerApp);
metrics.submitApp(user, appAttemptId.getAttemptId()); metrics.submitApp(user, appAttemptId.getAttemptId());
LOG.info("Added Application Attempt " + appAttemptId LOG.info("Added Application Attempt " + appAttemptId
+ " to scheduler from user " + application.getUser() + " to scheduler from user " + application.getUser());
+ ", currently active: " + appAttempts.size());
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptEvent(appAttemptId, new RMAppAttemptEvent(appAttemptId,
RMAppAttemptEventType.ATTEMPT_ADDED)); RMAppAttemptEventType.ATTEMPT_ADDED));
@ -400,28 +408,33 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
private synchronized void doneApplicationAttempt( private synchronized void doneApplicationAttempt(
ApplicationAttemptId applicationAttemptId, ApplicationAttemptId applicationAttemptId,
RMAppAttemptState rmAppAttemptFinalState) RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers)
throws IOException { throws IOException {
FiCaSchedulerApp application = getApplication(applicationAttemptId); FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
if (application == null) { SchedulerApplication application =
applications.get(applicationAttemptId.getApplicationId());
if (application == null || attempt == null) {
throw new IOException("Unknown application " + applicationAttemptId + throw new IOException("Unknown application " + applicationAttemptId +
" has completed!"); " has completed!");
} }
// Kill all 'live' containers // Kill all 'live' containers
for (RMContainer container : application.getLiveContainers()) { for (RMContainer container : attempt.getLiveContainers()) {
if (keepContainers
&& container.getState().equals(RMContainerState.RUNNING)) {
// do not kill the running container in the case of work-preserving AM
// restart.
LOG.info("Skip killing " + container.getContainerId());
continue;
}
containerCompleted(container, containerCompleted(container,
SchedulerUtils.createAbnormalContainerStatus( SchedulerUtils.createAbnormalContainerStatus(
container.getContainerId(), container.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
SchedulerUtils.COMPLETED_APPLICATION), RMContainerEventType.KILL);
RMContainerEventType.KILL);
} }
// Clean up pending requests, metrics etc. // Clean up pending requests, metrics etc.
application.stop(rmAppAttemptFinalState); attempt.stop(rmAppAttemptFinalState);
// Remove the application
appAttempts.remove(applicationAttemptId);
} }
/** /**
@ -432,12 +445,13 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
private void assignContainers(FiCaSchedulerNode node) { private void assignContainers(FiCaSchedulerNode node) {
LOG.debug("assignContainers:" + LOG.debug("assignContainers:" +
" node=" + node.getRMNode().getNodeAddress() + " node=" + node.getRMNode().getNodeAddress() +
" #applications=" + appAttempts.size()); " #applications=" + applications.size());
// Try to assign containers to applications in fifo order // Try to assign containers to applications in fifo order
for (Map.Entry<ApplicationAttemptId, FiCaSchedulerApp> e : appAttempts for (Map.Entry<ApplicationId, SchedulerApplication> e : applications
.entrySet()) { .entrySet()) {
FiCaSchedulerApp application = e.getValue(); FiCaSchedulerApp application =
(FiCaSchedulerApp) e.getValue().getCurrentAppAttempt();
LOG.debug("pre-assignContainers"); LOG.debug("pre-assignContainers");
application.showRequests(); application.showRequests();
synchronized (application) { synchronized (application) {
@ -474,8 +488,10 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
// Update the applications' headroom to correctly take into // Update the applications' headroom to correctly take into
// account the containers assigned in this update. // account the containers assigned in this update.
for (FiCaSchedulerApp application : appAttempts.values()) { for (SchedulerApplication application : applications.values()) {
application.setHeadroom(Resources.subtract(clusterResource, usedResource)); FiCaSchedulerApp attempt =
(FiCaSchedulerApp) application.getCurrentAppAttempt();
attempt.setHeadroom(Resources.subtract(clusterResource, usedResource));
} }
} }
@ -744,7 +760,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
{ {
AppAttemptAddedSchedulerEvent appAttemptAddedEvent = AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
(AppAttemptAddedSchedulerEvent) event; (AppAttemptAddedSchedulerEvent) event;
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId()); addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
} }
break; break;
case APP_ATTEMPT_REMOVED: case APP_ATTEMPT_REMOVED:
@ -754,7 +771,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
try { try {
doneApplicationAttempt( doneApplicationAttempt(
appAttemptRemovedEvent.getApplicationAttemptID(), appAttemptRemovedEvent.getApplicationAttemptID(),
appAttemptRemovedEvent.getFinalAttemptState()); appAttemptRemovedEvent.getFinalAttemptState(),
appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
} catch(IOException ie) { } catch(IOException ie) {
LOG.error("Unable to remove application " LOG.error("Unable to remove application "
+ appAttemptRemovedEvent.getApplicationAttemptID(), ie); + appAttemptRemovedEvent.getApplicationAttemptID(), ie);
@ -780,12 +798,11 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
// Get the application for the finished container // Get the application for the finished container
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId);
FiCaSchedulerApp application = getApplication(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.info("Unknown application: " + applicationAttemptId + LOG.info("Unknown application "
" launched container " + containerId + + containerId.getApplicationAttemptId().getApplicationId()
" on node: " + node); + " launched container " + containerId + " on node: " + node);
// Some unknown container sneaked into the system. Kill it. // Some unknown container sneaked into the system. Kill it.
this.rmContext.getDispatcher().getEventHandler() this.rmContext.getDispatcher().getEventHandler()
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
@ -806,14 +823,16 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
// Get the application for the finished container // Get the application for the finished container
Container container = rmContainer.getContainer(); Container container = rmContainer.getContainer();
ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); FiCaSchedulerApp application =
FiCaSchedulerApp application = getApplication(applicationAttemptId); getCurrentAttemptForContainer(container.getId());
ApplicationId appId =
container.getId().getApplicationAttemptId().getApplicationId();
// Get the node on which the container was allocated // Get the node on which the container was allocated
FiCaSchedulerNode node = getNode(container.getNodeId()); FiCaSchedulerNode node = getNode(container.getNodeId());
if (application == null) { if (application == null) {
LOG.info("Unknown application: " + applicationAttemptId + LOG.info("Unknown application: " + appId +
" released container " + container.getId() + " released container " + container.getId() +
" on node: " + node + " on node: " + node +
" with event: " + event); " with event: " + event);
@ -829,7 +848,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
// Update total usage // Update total usage
Resources.subtractFrom(usedResource, container.getResource()); Resources.subtractFrom(usedResource, container.getResource());
LOG.info("Application " + applicationAttemptId + LOG.info("Application attempt " + application.getApplicationAttemptId() +
" released container " + container.getId() + " released container " + container.getId() +
" on node: " + node + " on node: " + node +
" with event: " + event); " with event: " + event);
@ -888,10 +907,21 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
return node == null ? null : new SchedulerNodeReport(node); return node == null ? null : new SchedulerNodeReport(node);
} }
private RMContainer getRMContainer(ContainerId containerId) { @Override
FiCaSchedulerApp application = public RMContainer getRMContainer(ContainerId containerId) {
getApplication(containerId.getApplicationAttemptId()); FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
return (application == null) ? null : application.getRMContainer(containerId); return (attempt == null) ? null : attempt.getRMContainer(containerId);
}
private FiCaSchedulerApp getCurrentAttemptForContainer(
ContainerId containerId) {
SchedulerApplication app =
applications.get(containerId.getApplicationAttemptId()
.getApplicationId());
if (app != null) {
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
}
return null;
} }
@Override @Override
@ -908,12 +938,12 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
@Override @Override
public synchronized List<ApplicationAttemptId> getAppsInQueue(String queueName) { public synchronized List<ApplicationAttemptId> getAppsInQueue(String queueName) {
if (queueName.equals(DEFAULT_QUEUE.getQueueName())) { if (queueName.equals(DEFAULT_QUEUE.getQueueName())) {
List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>( List<ApplicationAttemptId> attempts = new ArrayList<ApplicationAttemptId>(
appAttempts.size()); applications.size());
for (FiCaSchedulerApp app : appAttempts.values()) { for (SchedulerApplication app : applications.values()) {
apps.add(app.getApplicationAttemptId()); attempts.add(app.getCurrentAppAttempt().getApplicationAttemptId());
} }
return apps; return attempts;
} else { } else {
return null; return null;
} }

View File

@ -43,6 +43,7 @@ public class AboutBlock extends HtmlBlock {
info("Cluster overview"). info("Cluster overview").
_("Cluster ID:", cinfo.getClusterId()). _("Cluster ID:", cinfo.getClusterId()).
_("ResourceManager state:", cinfo.getState()). _("ResourceManager state:", cinfo.getState()).
_("ResourceManager HA state:", cinfo.getHAState()).
_("ResourceManager started on:", Times.format(cinfo.getStartedOn())). _("ResourceManager started on:", Times.format(cinfo.getStartedOn())).
_("ResourceManager version:", cinfo.getRMBuildVersion() + _("ResourceManager version:", cinfo.getRMBuildVersion() +
" on " + cinfo.getRMVersionBuiltOn()). " on " + cinfo.getRMVersionBuiltOn()).

View File

@ -21,6 +21,7 @@ import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType; import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.service.Service.STATE; import org.apache.hadoop.service.Service.STATE;
import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@ -33,6 +34,7 @@ public class ClusterInfo {
protected long id; protected long id;
protected long startedOn; protected long startedOn;
protected STATE state; protected STATE state;
protected HAServiceProtocol.HAServiceState haState;
protected String resourceManagerVersion; protected String resourceManagerVersion;
protected String resourceManagerBuildVersion; protected String resourceManagerBuildVersion;
protected String resourceManagerVersionBuiltOn; protected String resourceManagerVersionBuiltOn;
@ -48,6 +50,7 @@ public class ClusterInfo {
this.id = ts; this.id = ts;
this.state = rm.getServiceState(); this.state = rm.getServiceState();
this.haState = rm.getRMContext().getHAServiceState();
this.startedOn = ts; this.startedOn = ts;
this.resourceManagerVersion = YarnVersionInfo.getVersion(); this.resourceManagerVersion = YarnVersionInfo.getVersion();
this.resourceManagerBuildVersion = YarnVersionInfo.getBuildVersion(); this.resourceManagerBuildVersion = YarnVersionInfo.getBuildVersion();
@ -61,6 +64,10 @@ public class ClusterInfo {
return this.state.toString(); return this.state.toString();
} }
public String getHAState() {
return this.haState.toString();
}
public String getRMVersion() { public String getRMVersion() {
return this.resourceManagerVersion; return this.resourceManagerVersion;
} }

View File

@ -171,7 +171,7 @@ public class Application {
new AppAddedSchedulerEvent(this.applicationId, this.queue, "user"); new AppAddedSchedulerEvent(this.applicationId, this.queue, "user");
scheduler.handle(addAppEvent); scheduler.handle(addAppEvent);
AppAttemptAddedSchedulerEvent addAttemptEvent = AppAttemptAddedSchedulerEvent addAttemptEvent =
new AppAttemptAddedSchedulerEvent(this.applicationAttemptId); new AppAttemptAddedSchedulerEvent(this.applicationAttemptId, false);
scheduler.handle(addAttemptEvent); scheduler.handle(addAttemptEvent);
} }

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.YarnVersionInfo; import org.apache.hadoop.yarn.util.YarnVersionInfo;
import org.mortbay.log.Log;
public class MockNM { public class MockNM {
@ -130,12 +131,13 @@ public class MockNM {
int containerId, ContainerState containerState) throws Exception { int containerId, ContainerState containerState) throws Exception {
HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate = HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate =
new HashMap<ApplicationId, List<ContainerStatus>>(1); new HashMap<ApplicationId, List<ContainerStatus>>(1);
ContainerStatus amContainerStatus = BuilderUtils.newContainerStatus( ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
BuilderUtils.newContainerId(attemptId, 1), BuilderUtils.newContainerId(attemptId, containerId), containerState,
ContainerState.COMPLETE, "Success", 0); "Success", 0);
ArrayList<ContainerStatus> containerStatusList = ArrayList<ContainerStatus> containerStatusList =
new ArrayList<ContainerStatus>(1); new ArrayList<ContainerStatus>(1);
containerStatusList.add(amContainerStatus); containerStatusList.add(containerStatus);
Log.info("ContainerStatus: " + containerStatus);
nodeUpdate.put(attemptId.getApplicationId(), containerStatusList); nodeUpdate.put(attemptId.getApplicationId(), containerStatusList);
return nodeHeartbeat(nodeUpdate, true); return nodeHeartbeat(nodeUpdate, true);
} }
@ -152,6 +154,7 @@ public class MockNM {
status.setResponseId(resId); status.setResponseId(resId);
status.setNodeId(nodeId); status.setNodeId(nodeId);
for (Map.Entry<ApplicationId, List<ContainerStatus>> entry : conts.entrySet()) { for (Map.Entry<ApplicationId, List<ContainerStatus>> entry : conts.entrySet()) {
Log.info("entry.getValue() " + entry.getValue());
status.setContainersStatuses(entry.getValue()); status.setContainersStatuses(entry.getValue());
} }
NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class); NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
@ -40,7 +41,10 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
@ -56,6 +60,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
@ -122,6 +128,33 @@ public class MockRM extends ResourceManager {
attempt.getAppAttemptState()); attempt.getAppAttemptState());
} }
public void waitForContainerAllocated(MockNM nm, ContainerId containerId)
throws Exception {
int timeoutSecs = 0;
while (getResourceScheduler().getRMContainer(containerId) == null
&& timeoutSecs++ < 40) {
System.out.println("Waiting for" + containerId + " to be allocated.");
nm.nodeHeartbeat(true);
Thread.sleep(200);
}
}
public void waitForState(MockNM nm, ContainerId containerId,
RMContainerState containerState) throws Exception {
RMContainer container = getResourceScheduler().getRMContainer(containerId);
Assert.assertNotNull("Container shouldn't be null", container);
int timeoutSecs = 0;
while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) {
System.out.println("Container : " + containerId + " State is : "
+ container.getState() + " Waiting for state : " + containerState);
nm.nodeHeartbeat(true);
Thread.sleep(300);
}
System.out.println("Container State is : " + container.getState());
Assert.assertEquals("Container state is not correct (timedout)",
containerState, container.getState());
}
// get new application id // get new application id
public GetNewApplicationResponse getNewAppId() throws Exception { public GetNewApplicationResponse getNewAppId() throws Exception {
ApplicationClientProtocol client = getClientRMService(); ApplicationClientProtocol client = getClientRMService();
@ -172,7 +205,17 @@ public class MockRM extends ResourceManager {
public RMApp submitApp(int masterMemory, String name, String user, public RMApp submitApp(int masterMemory, String name, String user,
Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue, Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
int maxAppAttempts, Credentials ts, String appType, int maxAppAttempts, Credentials ts, String appType,
boolean waitForAccepted) throws Exception { boolean waitForAccepted)
throws Exception {
return submitApp(masterMemory, name, user, acls, unmanaged, queue,
maxAppAttempts, ts, appType, waitForAccepted, false);
}
public RMApp submitApp(int masterMemory, String name, String user,
Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
int maxAppAttempts, Credentials ts, String appType,
boolean waitForAccepted, boolean keepContainers)
throws Exception {
ApplicationClientProtocol client = getClientRMService(); ApplicationClientProtocol client = getClientRMService();
GetNewApplicationResponse resp = client.getNewApplication(Records GetNewApplicationResponse resp = client.getNewApplication(Records
.newRecord(GetNewApplicationRequest.class)); .newRecord(GetNewApplicationRequest.class));
@ -182,6 +225,7 @@ public class MockRM extends ResourceManager {
.newRecord(SubmitApplicationRequest.class); .newRecord(SubmitApplicationRequest.class);
ApplicationSubmissionContext sub = Records ApplicationSubmissionContext sub = Records
.newRecord(ApplicationSubmissionContext.class); .newRecord(ApplicationSubmissionContext.class);
sub.setKeepContainersAcrossApplicationAttempts(keepContainers);
sub.setApplicationId(appId); sub.setApplicationId(appId);
sub.setApplicationName(name); sub.setApplicationName(name);
sub.setMaxAppAttempts(maxAppAttempts); sub.setMaxAppAttempts(maxAppAttempts);
@ -421,4 +465,26 @@ public class MockRM extends ResourceManager {
// override to disable webapp // override to disable webapp
} }
public static void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
MockAM am) throws Exception {
FinishApplicationMasterRequest req =
FinishApplicationMasterRequest.newInstance(
FinalApplicationStatus.SUCCEEDED, "", "");
am.unregisterAppAttempt(req);
am.waitForState(RMAppAttemptState.FINISHING);
nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am.waitForState(RMAppAttemptState.FINISHED);
rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
}
public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
throws Exception {
RMAppAttempt attempt = app.getCurrentAppAttempt();
nm.nodeHeartbeat(true);
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
am.registerAppAttempt();
rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
return am;
}
} }

View File

@ -649,7 +649,7 @@ public class TestClientRMService {
.currentTimeMillis(), "YARN")); .currentTimeMillis(), "YARN"));
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(applicationId3, 1); ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(applicationId3, 1);
RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId, RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
rmContext, yarnScheduler, null, asContext, config); rmContext, yarnScheduler, null, asContext, config, false);
when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl); when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl);
return app; return app;
} }

View File

@ -302,7 +302,7 @@ public class TestFifoScheduler {
new AppAddedSchedulerEvent(appId1, "queue", "user"); new AppAddedSchedulerEvent(appId1, "queue", "user");
fs.handle(appEvent); fs.handle(appEvent);
SchedulerEvent attemptEvent = SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId1); new AppAttemptAddedSchedulerEvent(appAttemptId1, false);
fs.handle(attemptEvent); fs.handle(attemptEvent);
List<ContainerId> emptyId = new ArrayList<ContainerId>(); List<ContainerId> emptyId = new ArrayList<ContainerId>();
@ -396,7 +396,7 @@ public class TestFifoScheduler {
new AppAddedSchedulerEvent(appId1, "queue", "user"); new AppAddedSchedulerEvent(appId1, "queue", "user");
fs.handle(appEvent); fs.handle(appEvent);
SchedulerEvent attemptEvent = SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId1); new AppAttemptAddedSchedulerEvent(appAttemptId1, false);
fs.handle(attemptEvent); fs.handle(attemptEvent);
ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2); ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2);
@ -406,7 +406,7 @@ public class TestFifoScheduler {
new AppAddedSchedulerEvent(appId2, "queue", "user"); new AppAddedSchedulerEvent(appId2, "queue", "user");
fs.handle(appEvent2); fs.handle(appEvent2);
SchedulerEvent attemptEvent2 = SchedulerEvent attemptEvent2 =
new AppAttemptAddedSchedulerEvent(appAttemptId2); new AppAttemptAddedSchedulerEvent(appAttemptId2, false);
fs.handle(attemptEvent2); fs.handle(attemptEvent2);
List<ContainerId> emptyId = new ArrayList<ContainerId>(); List<ContainerId> emptyId = new ArrayList<ContainerId>();

View File

@ -28,7 +28,6 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
@ -38,7 +37,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
@ -295,6 +293,8 @@ public class TestRM {
nm2.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(), nm2.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(),
ContainerState.COMPLETE); ContainerState.COMPLETE);
} }
nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1,
ContainerState.COMPLETE);
am.waitForState(RMAppAttemptState.FINISHED); am.waitForState(RMAppAttemptState.FINISHED);
Assert.assertFalse(nmTokenSecretManager Assert.assertFalse(nmTokenSecretManager
.isApplicationAttemptRegistered(attempt.getAppAttemptId())); .isApplicationAttemptRegistered(attempt.getAppAttemptId()));
@ -389,19 +389,19 @@ public class TestRM {
MockNM nm1 = MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService()); new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode(); nm1.registerNode();
MockAM am1 = launchAM(app1, rm1, nm1); MockAM am1 = MockRM.launchAM(app1, rm1, nm1);
finishApplicationMaster(app1, rm1, nm1, am1); MockRM.finishApplicationMaster(app1, rm1, nm1, am1);
// a failed app // a failed app
RMApp app2 = rm1.submitApp(200); RMApp app2 = rm1.submitApp(200);
MockAM am2 = launchAM(app2, rm1, nm1); MockAM am2 = MockRM.launchAM(app2, rm1, nm1);
nm1.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE); nm1.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am2.waitForState(RMAppAttemptState.FAILED); am2.waitForState(RMAppAttemptState.FAILED);
rm1.waitForState(app2.getApplicationId(), RMAppState.FAILED); rm1.waitForState(app2.getApplicationId(), RMAppState.FAILED);
// a killed app // a killed app
RMApp app3 = rm1.submitApp(200); RMApp app3 = rm1.submitApp(200);
MockAM am3 = launchAM(app3, rm1, nm1); MockAM am3 = MockRM.launchAM(app3, rm1, nm1);
rm1.killApp(app3.getApplicationId()); rm1.killApp(app3.getApplicationId());
rm1.waitForState(app3.getApplicationId(), RMAppState.KILLED); rm1.waitForState(app3.getApplicationId(), RMAppState.KILLED);
rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.KILLED); rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.KILLED);
@ -441,7 +441,7 @@ public class TestRM {
// a failed app // a failed app
RMApp app2 = rm1.submitApp(200); RMApp app2 = rm1.submitApp(200);
MockAM am2 = launchAM(app2, rm1, nm1); MockAM am2 = MockRM.launchAM(app2, rm1, nm1);
nm1 nm1
.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE); .nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am2.waitForState(RMAppAttemptState.FAILED); am2.waitForState(RMAppAttemptState.FAILED);
@ -458,28 +458,6 @@ public class TestRM {
Assert.assertEquals(-1, report1.getRpcPort()); Assert.assertEquals(-1, report1.getRpcPort());
} }
private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
throws Exception {
RMAppAttempt attempt = app.getCurrentAppAttempt();
nm.nodeHeartbeat(true);
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
am.registerAppAttempt();
rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
return am;
}
private void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
MockAM am) throws Exception {
FinishApplicationMasterRequest req =
FinishApplicationMasterRequest.newInstance(
FinalApplicationStatus.SUCCEEDED, "", "");
am.unregisterAppAttempt(req);
am.waitForState(RMAppAttemptState.FINISHING);
nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am.waitForState(RMAppAttemptState.FINISHED);
rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
}
public static void main(String[] args) throws Exception { public static void main(String[] args) throws Exception {
TestRM t = new TestRM(); TestRM t = new TestRM();
t.testGetNewAppId(); t.testGetNewAppId();

View File

@ -26,8 +26,11 @@ import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo; import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.ha.HealthCheckFailedException; import org.apache.hadoop.ha.HealthCheckFailedException;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.conf.HAUtil; import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -222,4 +225,81 @@ public class TestRMHA {
checkMonitorHealth(); checkMonitorHealth();
checkActiveRMFunctionality(); checkActiveRMFunctionality();
} }
@Test
public void testRMDispatcherForHA() throws IOException {
String errorMessageForEventHandler =
"Expect to get the same number of handlers";
String errorMessageForService = "Expect to get the same number of services";
Configuration conf = new YarnConfiguration(configuration);
rm = new MockRM(conf) {
@Override
protected Dispatcher createDispatcher() {
return new MyCountingDispatcher();
}
};
rm.init(conf);
int expectedEventHandlerCount =
((MyCountingDispatcher) rm.getRMContext().getDispatcher())
.getEventHandlerCount();
int expectedServiceCount = rm.getServices().size();
assertTrue(expectedEventHandlerCount != 0);
StateChangeRequestInfo requestInfo = new StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_USER);
assertEquals(STATE_ERR, HAServiceState.INITIALIZING,
rm.adminService.getServiceStatus().getState());
assertFalse("RM is ready to become active before being started",
rm.adminService.getServiceStatus().isReadyToBecomeActive());
rm.start();
//call transitions to standby and active a couple of times
rm.adminService.transitionToStandby(requestInfo);
rm.adminService.transitionToActive(requestInfo);
rm.adminService.transitionToStandby(requestInfo);
rm.adminService.transitionToActive(requestInfo);
rm.adminService.transitionToStandby(requestInfo);
rm.adminService.transitionToActive(requestInfo);
assertEquals(errorMessageForEventHandler, expectedEventHandlerCount,
((MyCountingDispatcher) rm.getRMContext().getDispatcher())
.getEventHandlerCount());
assertEquals(errorMessageForService, expectedServiceCount,
rm.getServices().size());
rm.adminService.transitionToStandby(requestInfo);
assertEquals(errorMessageForEventHandler, expectedEventHandlerCount,
((MyCountingDispatcher) rm.getRMContext().getDispatcher())
.getEventHandlerCount());
assertEquals(errorMessageForService, expectedServiceCount,
rm.getServices().size());
rm.stop();
}
@SuppressWarnings("rawtypes")
class MyCountingDispatcher extends AbstractService implements Dispatcher {
private int eventHandlerCount;
public MyCountingDispatcher() {
super("MyCountingDispatcher");
this.eventHandlerCount = 0;
}
@Override
public EventHandler getEventHandler() {
return null;
}
@Override
public void register(Class<? extends Enum> eventType, EventHandler handler) {
this.eventHandlerCount ++;
}
public int getEventHandlerCount() {
return this.eventHandlerCount;
}
}
} }

View File

@ -164,7 +164,7 @@ public class TestResourceManager {
// Notify scheduler application is finished. // Notify scheduler application is finished.
AppAttemptRemovedSchedulerEvent appRemovedEvent1 = AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
new AppAttemptRemovedSchedulerEvent( new AppAttemptRemovedSchedulerEvent(
application.getApplicationAttemptId(), RMAppAttemptState.FINISHED); application.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
resourceManager.getResourceScheduler().handle(appRemovedEvent1); resourceManager.getResourceScheduler().handle(appRemovedEvent1);
checkResourceUsage(nm1, nm2); checkResourceUsage(nm1, nm2);

View File

@ -18,49 +18,30 @@
package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager; package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
import java.io.IOException; import java.util.ArrayList;
import java.nio.ByteBuffer; import java.util.HashMap;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import junit.framework.Assert; import junit.framework.Assert;
import org.apache.commons.logging.Log; import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.After;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
/** /**
@ -68,238 +49,164 @@ import org.junit.Test;
* *
*/ */
public class TestAMRestart { public class TestAMRestart {
// private static final Log LOG = LogFactory.getLog(TestAMRestart.class);
// ApplicationsManagerImpl appImpl; @Test
// RMContext asmContext = new RMContextImpl(new MemStore()); public void testAMRestartWithExistingContainers() throws Exception {
// ApplicationTokenSecretManager appTokenSecretManager = YarnConfiguration conf = new YarnConfiguration();
// new ApplicationTokenSecretManager(); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
// DummyResourceScheduler scheduler;
// private ClientRMService clientRMService; MockRM rm1 = new MockRM(conf);
// int count = 0; rm1.start();
// ApplicationId appID; RMApp app1 =
// final int maxFailures = 3; rm1.submitApp(200, "name", "user",
// AtomicInteger launchNotify = new AtomicInteger(); new HashMap<ApplicationAccessType, String>(), false, "default", -1,
// AtomicInteger schedulerNotify = new AtomicInteger(); null, "MAPREDUCE", false, true);
// volatile boolean stop = false; MockNM nm1 =
// int schedulerAddApplication = 0; new MockNM("127.0.0.1:1234", 10240, rm1.getResourceTrackerService());
// int schedulerRemoveApplication = 0; nm1.registerNode();
// int launcherLaunchCalled = 0; MockNM nm2 =
// int launcherCleanupCalled = 0; new MockNM("127.0.0.1:2351", 4089, rm1.getResourceTrackerService());
// private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); nm2.registerNode();
//
// private class ExtApplicationsManagerImpl extends ApplicationsManagerImpl { MockAM am1 = MockRM.launchAM(app1, rm1, nm1);
// public ExtApplicationsManagerImpl( int NUM_CONTAINERS = 3;
// ApplicationTokenSecretManager applicationTokenSecretManager, // allocate NUM_CONTAINERS containers
// YarnScheduler scheduler, RMContext asmContext) { am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
// super(applicationTokenSecretManager, scheduler, asmContext); new ArrayList<ContainerId>());
// } nm1.nodeHeartbeat(true);
//
// @Override // wait for containers to be allocated.
// public EventHandler<ASMEvent<AMLauncherEventType>> createNewApplicationMasterLauncher( List<Container> containers =
// ApplicationTokenSecretManager tokenSecretManager) { am1.allocate(new ArrayList<ResourceRequest>(),
// return new DummyAMLauncher(); new ArrayList<ContainerId>()).getAllocatedContainers();
// } while (containers.size() != NUM_CONTAINERS) {
// } nm1.nodeHeartbeat(true);
// containers.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
// private class DummyAMLauncher implements EventHandler<ASMEvent<AMLauncherEventType>> { new ArrayList<ContainerId>()).getAllocatedContainers());
// Thread.sleep(200);
// public DummyAMLauncher() { }
// asmContext.getDispatcher().register(AMLauncherEventType.class, this);
// new Thread() { // launch the 2nd container, for testing running container transferred.
// public void run() { nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
// while (!stop) { ContainerId containerId2 =
// LOG.info("DEBUG -- waiting for launch"); ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
// synchronized(launchNotify) { rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
// while (launchNotify.get() == 0) {
// try { // launch the 3rd container, for testing container allocated by previous
// launchNotify.wait(); // attempt is completed by the next new attempt/
// } catch (InterruptedException e) { nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING);
// } ContainerId containerId3 =
// } ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
// asmContext.getDispatcher().getEventHandler().handle( rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
// new ApplicationEvent(
// ApplicationEventType.LAUNCHED, appID)); // 4th container still in AQUIRED state. for testing Acquired container is
// launchNotify.addAndGet(-1); // always killed.
// } ContainerId containerId4 =
// } ContainerId.newInstance(am1.getApplicationAttemptId(), 4);
// } rm1.waitForState(nm1, containerId4, RMContainerState.ACQUIRED);
// }.start();
// } // 5th container is in Allocated state. for testing allocated container is
// // always killed.
// @Override am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
// public void handle(ASMEvent<AMLauncherEventType> event) { nm1.nodeHeartbeat(true);
// switch (event.getType()) { ContainerId containerId5 =
// case CLEANUP: ContainerId.newInstance(am1.getApplicationAttemptId(), 5);
// launcherCleanupCalled++; rm1.waitForContainerAllocated(nm1, containerId5);
// break; rm1.waitForState(nm1, containerId5, RMContainerState.ALLOCATED);
// case LAUNCH:
// LOG.info("DEBUG -- launching"); // 6th container is in Reserved state.
// launcherLaunchCalled++; am1.allocate("127.0.0.1", 6000, 1, new ArrayList<ContainerId>());
// synchronized (launchNotify) { ContainerId containerId6 =
// launchNotify.addAndGet(1); ContainerId.newInstance(am1.getApplicationAttemptId(), 6);
// launchNotify.notify(); nm1.nodeHeartbeat(true);
// } SchedulerApplicationAttempt schedulerAttempt =
// break; ((CapacityScheduler) rm1.getResourceScheduler())
// default: .getCurrentAttemptForContainer(containerId6);
// break; while (schedulerAttempt.getReservedContainers().size() == 0) {
// } System.out.println("Waiting for container " + containerId6
// } + " to be reserved.");
// } nm1.nodeHeartbeat(true);
// Thread.sleep(200);
// private class DummyResourceScheduler implements ResourceScheduler { }
// // assert containerId6 is reserved.
// @Override Assert.assertEquals(containerId6, schedulerAttempt.getReservedContainers()
// public void removeNode(RMNode node) { .get(0).getContainerId());
// }
// // fail the AM by sending CONTAINER_FINISHED event without registering.
// @Override nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
// public Allocation allocate(ApplicationId applicationId, am1.waitForState(RMAppAttemptState.FAILED);
// List<ResourceRequest> ask, List<Container> release) throws IOException {
// Container container = recordFactory.newRecordInstance(Container.class); // wait for some time. previous AM's running containers should still remain
// container.setContainerToken(recordFactory.newRecordInstance(ContainerToken.class)); // in scheduler even though am failed
// container.setNodeId(recordFactory.newRecordInstance(NodeId.class)); Thread.sleep(3000);
// container.setContainerManagerAddress("localhost"); rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
// container.setNodeHttpAddress("localhost:8042"); // acquired/allocated containers are cleaned up.
// container.setId(recordFactory.newRecordInstance(ContainerId.class)); Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4));
// container.getId().setAppId(appID); Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5));
// container.getId().setId(count);
// count++; // wait for app to start a new attempt.
// return new Allocation(Arrays.asList(container), Resources.none()); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
// } // assert this is a new AM.
// ApplicationAttemptId newAttemptId =
// @Override app1.getCurrentAppAttempt().getAppAttemptId();
// public void handle(ASMEvent<ApplicationTrackerEventType> event) { Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId()));
// switch (event.getType()) { MockAM am2 = MockRM.launchAM(app1, rm1, nm1);
// case ADD:
// schedulerAddApplication++; // complete container by sending the container complete event which has earlier
// break; // attempt's attemptId
// case EXPIRE: nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.COMPLETE);
// schedulerRemoveApplication++; rm1.waitForState(nm1, containerId3, RMContainerState.COMPLETED);
// LOG.info("REMOVING app : " + schedulerRemoveApplication);
// if (schedulerRemoveApplication == maxFailures) { // Even though the completed container containerId3 event was sent to the
// synchronized (schedulerNotify) { // earlier failed attempt, new RMAppAttempt can also capture this container
// schedulerNotify.addAndGet(1); // info.
// schedulerNotify.notify(); // completed containerId4 is also transferred to the new attempt.
// } RMAppAttempt newAttempt =
// } app1.getRMAppAttempt(am2.getApplicationAttemptId());
// break; // 4 containers finished, acquired/allocated/reserved/completed.
// default: Assert.assertEquals(4, newAttempt.getJustFinishedContainers().size());
// break; boolean container3Exists = false, container4Exists = false, container5Exists =
// } false, container6Exists = false;
// } for(ContainerStatus status : newAttempt.getJustFinishedContainers()) {
// if(status.getContainerId().equals(containerId3)) {
// @Override // containerId3 is the container ran by previous attempt but finished by the
// public QueueInfo getQueueInfo(String queueName, // new attempt.
// boolean includeChildQueues, container3Exists = true;
// boolean recursive) throws IOException { }
// return null; if (status.getContainerId().equals(containerId4)) {
// } // containerId4 is the Acquired Container killed by the previous attempt,
// @Override // it's now inside new attempt's finished container list.
// public List<QueueUserACLInfo> getQueueUserAclInfo() { container4Exists = true;
// return null; }
// } if (status.getContainerId().equals(containerId5)) {
// @Override // containerId5 is the Allocated container killed by previous failed attempt.
// public void addApplication(ApplicationId applicationId, container5Exists = true;
// ApplicationMaster master, String user, String queue, Priority priority, }
// ApplicationStore store) if (status.getContainerId().equals(containerId6)) {
// throws IOException { // containerId6 is the reserved container killed by previous failed attempt.
// } container6Exists = true;
// @Override }
// public void addNode(RMNode nodeInfo) { }
// } Assert.assertTrue(container3Exists && container4Exists && container5Exists
// @Override && container6Exists);
// public void recover(RMState state) throws Exception {
// } // New SchedulerApplicationAttempt also has the containers info.
// @Override rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
// public void reinitialize(Configuration conf,
// ContainerTokenSecretManager secretManager, RMContext rmContext) // record the scheduler attempt for testing.
// throws IOException { SchedulerApplicationAttempt schedulerNewAttempt =
// } ((CapacityScheduler) rm1.getResourceScheduler())
// .getCurrentAttemptForContainer(containerId2);
// @Override // finish this application
// public void nodeUpdate(RMNode nodeInfo, MockRM.finishApplicationMaster(app1, rm1, nm1, am2);
// Map<String, List<Container>> containers) {
// } // the 2nd attempt released the 1st attempt's running container, when the
// // 2nd attempt finishes.
// @Override Assert.assertFalse(schedulerNewAttempt.getLiveContainers().contains(
// public Resource getMaximumResourceCapability() { containerId2));
// // TODO Auto-generated method stub // all 4 normal containers finished.
// return null; Assert.assertEquals(5, newAttempt.getJustFinishedContainers().size());
// }
// rm1.stop();
// @Override }
// public Resource getMinimumResourceCapability() {
// // TODO Auto-generated method stub
// return null;
// }
// }
//
// @Before
// public void setUp() {
//
// asmContext.getDispatcher().register(ApplicationEventType.class,
// new ResourceManager.ApplicationEventDispatcher(asmContext));
//
// appID = recordFactory.newRecordInstance(ApplicationId.class);
// appID.setClusterTimestamp(System.currentTimeMillis());
// appID.setId(1);
// Configuration conf = new Configuration();
// scheduler = new DummyResourceScheduler();
// asmContext.getDispatcher().init(conf);
// asmContext.getDispatcher().start();
// asmContext.getDispatcher().register(ApplicationTrackerEventType.class, scheduler);
// appImpl = new ExtApplicationsManagerImpl(appTokenSecretManager, scheduler, asmContext);
//
// conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 1000L);
// conf.setInt(RMConfig.AM_MAX_RETRIES, maxFailures);
// appImpl.init(conf);
// appImpl.start();
//
// this.clientRMService = new ClientRMService(asmContext, appImpl
// .getAmLivelinessMonitor(), appImpl.getClientToAMSecretManager(),
// scheduler);
// this.clientRMService.init(conf);
// }
//
// @After
// public void tearDown() {
// }
//
// private void waitForFailed(AppAttempt application, ApplicationState
// finalState) throws Exception {
// int count = 0;
// while(application.getState() != finalState && count < 10) {
// Thread.sleep(500);
// count++;
// }
// Assert.assertEquals(finalState, application.getState());
// }
//
// @Test
// public void testAMRestart() throws Exception {
// ApplicationSubmissionContext subContext = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
// subContext.setApplicationId(appID);
// subContext.setApplicationName("dummyApp");
//// subContext.command = new ArrayList<String>();
//// subContext.environment = new HashMap<String, String>();
//// subContext.fsTokens = new ArrayList<String>();
// subContext.setFsTokensTodo(ByteBuffer.wrap(new byte[0]));
// SubmitApplicationRequest request = recordFactory
// .newRecordInstance(SubmitApplicationRequest.class);
// request.setApplicationSubmissionContext(subContext);
// clientRMService.submitApplication(request);
// AppAttempt application = asmContext.getApplications().get(appID);
// synchronized (schedulerNotify) {
// while(schedulerNotify.get() == 0) {
// schedulerNotify.wait();
// }
// }
// Assert.assertEquals(maxFailures, launcherCleanupCalled);
// Assert.assertEquals(maxFailures, launcherLaunchCalled);
// Assert.assertEquals(maxFailures, schedulerAddApplication);
// Assert.assertEquals(maxFailures, schedulerRemoveApplication);
// Assert.assertEquals(maxFailures, application.getFailedCount());
// waitForFailed(application, ApplicationState.FAILED);
// stop = true;
// }
} }

View File

@ -460,7 +460,7 @@ public class TestRMAppTransitions {
LOG.info("--- START: testUnmanagedAppFailPath ---"); LOG.info("--- START: testUnmanagedAppFailPath ---");
application = testCreateAppRunning(subContext); application = testCreateAppRunning(subContext);
RMAppEvent event = new RMAppFailedAttemptEvent( RMAppEvent event = new RMAppFailedAttemptEvent(
application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, ""); application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
RMAppAttempt appAttempt = application.getCurrentAppAttempt(); RMAppAttempt appAttempt = application.getCurrentAppAttempt();
@ -582,7 +582,7 @@ public class TestRMAppTransitions {
for (int i=1; i < maxAppAttempts; i++) { for (int i=1; i < maxAppAttempts; i++) {
RMAppEvent event = RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(), new RMAppFailedAttemptEvent(application.getApplicationId(),
RMAppEventType.ATTEMPT_FAILED, ""); RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event); application.handle(event);
assertAppState(RMAppState.ACCEPTED, application); assertAppState(RMAppState.ACCEPTED, application);
event = event =
@ -598,7 +598,7 @@ public class TestRMAppTransitions {
String message = "Test fail"; String message = "Test fail";
RMAppEvent event = RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(), new RMAppFailedAttemptEvent(application.getApplicationId(),
RMAppEventType.ATTEMPT_FAILED, message); RMAppEventType.ATTEMPT_FAILED, message, false);
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
sendAppUpdateSavedEvent(application); sendAppUpdateSavedEvent(application);
@ -655,7 +655,7 @@ public class TestRMAppTransitions {
for (int i=1; i<maxAppAttempts; i++) { for (int i=1; i<maxAppAttempts; i++) {
RMAppEvent event = RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(), new RMAppFailedAttemptEvent(application.getApplicationId(),
RMAppEventType.ATTEMPT_FAILED, ""); RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
assertAppState(RMAppState.ACCEPTED, application); assertAppState(RMAppState.ACCEPTED, application);
@ -680,7 +680,7 @@ public class TestRMAppTransitions {
// after max application attempts // after max application attempts
RMAppEvent event = RMAppEvent event =
new RMAppFailedAttemptEvent(application.getApplicationId(), new RMAppFailedAttemptEvent(application.getApplicationId(),
RMAppEventType.ATTEMPT_FAILED, ""); RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
sendAppUpdateSavedEvent(application); sendAppUpdateSavedEvent(application);
@ -804,7 +804,7 @@ public class TestRMAppTransitions {
// KILLED => KILLED event RMAppEventType.ATTEMPT_FAILED // KILLED => KILLED event RMAppEventType.ATTEMPT_FAILED
event = event =
new RMAppFailedAttemptEvent(application.getApplicationId(), new RMAppFailedAttemptEvent(application.getApplicationId(),
RMAppEventType.ATTEMPT_FAILED, ""); RMAppEventType.ATTEMPT_FAILED, "", false);
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
assertTimesAtFinish(application); assertTimesAtFinish(application);

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
@ -68,10 +69,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventT
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
@ -120,13 +121,14 @@ public class TestRMAppAttemptTransitions {
private AMLivelinessMonitor amFinishingMonitor; private AMLivelinessMonitor amFinishingMonitor;
private RMStateStore store; private RMStateStore store;
private RMApp application; private RMAppImpl application;
private RMAppAttempt applicationAttempt; private RMAppAttempt applicationAttempt;
private Configuration conf = new Configuration(); private Configuration conf = new Configuration();
private AMRMTokenSecretManager amRMTokenManager = spy(new AMRMTokenSecretManager(conf)); private AMRMTokenSecretManager amRMTokenManager = spy(new AMRMTokenSecretManager(conf));
private ClientToAMTokenSecretManagerInRM clientToAMTokenManager = private ClientToAMTokenSecretManagerInRM clientToAMTokenManager =
spy(new ClientToAMTokenSecretManagerInRM()); spy(new ClientToAMTokenSecretManagerInRM());
private boolean transferStateFromPreviousAttempt = false;
private final class TestApplicationAttemptEventDispatcher implements private final class TestApplicationAttemptEventDispatcher implements
EventHandler<RMAppAttemptEvent> { EventHandler<RMAppAttemptEvent> {
@ -150,6 +152,11 @@ public class TestRMAppAttemptTransitions {
@Override @Override
public void handle(RMAppEvent event) { public void handle(RMAppEvent event) {
assertEquals(application.getApplicationId(), event.getApplicationId()); assertEquals(application.getApplicationId(), event.getApplicationId());
if (event instanceof RMAppFailedAttemptEvent) {
transferStateFromPreviousAttempt =
((RMAppFailedAttemptEvent) event)
.getTransferStateFromPreviousAttempt();
}
try { try {
application.handle(event); application.handle(event);
} catch (Throwable t) { } catch (Throwable t) {
@ -254,10 +261,10 @@ public class TestRMAppAttemptTransitions {
unmanagedAM = false; unmanagedAM = false;
application = mock(RMApp.class); application = mock(RMAppImpl.class);
applicationAttempt = applicationAttempt =
new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler, new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler,
masterService, submissionContext, new Configuration()); masterService, submissionContext, new Configuration(), false);
when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt); when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
when(application.getApplicationId()).thenReturn(applicationId); when(application.getApplicationId()).thenReturn(applicationId);
@ -371,6 +378,7 @@ public class TestRMAppAttemptTransitions {
assertNull(applicationAttempt.getFinalApplicationStatus()); assertNull(applicationAttempt.getFinalApplicationStatus());
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
verifyAttemptFinalStateSaved(); verifyAttemptFinalStateSaved();
assertFalse(transferStateFromPreviousAttempt);
} }
/** /**
@ -525,6 +533,7 @@ public class TestRMAppAttemptTransitions {
assertEquals(container, applicationAttempt.getMasterContainer()); assertEquals(container, applicationAttempt.getMasterContainer());
assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus()); assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
assertFalse(transferStateFromPreviousAttempt);
} }
@ -654,6 +663,7 @@ public class TestRMAppAttemptTransitions {
diagnostics)); diagnostics));
testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1, testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1,
true); true);
assertFalse(transferStateFromPreviousAttempt);
} }
private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) { private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) {
@ -681,6 +691,21 @@ public class TestRMAppAttemptTransitions {
"Unmanaged AM must register after AM attempt reaches LAUNCHED state."); "Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
} }
@Test
public void testUnmanagedAMContainersCleanup() {
unmanagedAM = true;
when(submissionContext.getUnmanagedAM()).thenReturn(true);
when(submissionContext.getKeepContainersAcrossApplicationAttempts())
.thenReturn(true);
// submit AM and check it goes to SUBMITTED state
submitApplicationAttempt();
// launch AM and verify attempt failed
applicationAttempt.handle(new RMAppAttemptRegistrationEvent(
applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl"));
sendAttemptUpdateSavedEvent(applicationAttempt);
assertFalse(transferStateFromPreviousAttempt);
}
@Test @Test
public void testNewToKilled() { public void testNewToKilled() {
applicationAttempt.handle( applicationAttempt.handle(
@ -1092,6 +1117,64 @@ public class TestRMAppAttemptTransitions {
Assert.assertNull(token); Assert.assertNull(token);
} }
@Test
public void testFailedToFailed() {
// create a failed attempt.
when(submissionContext.getKeepContainersAcrossApplicationAttempts())
.thenReturn(true);
Container amContainer = allocateApplicationAttempt();
launchApplicationAttempt(amContainer);
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
ContainerStatus cs1 =
ContainerStatus.newInstance(amContainer.getId(),
ContainerState.COMPLETE, "some error", 123);
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
appAttemptId, cs1));
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
// should not kill containers when attempt fails.
assertTrue(transferStateFromPreviousAttempt);
// failed attempt captured the container finished event.
assertEquals(0, applicationAttempt.getJustFinishedContainers().size());
ContainerStatus cs2 =
ContainerStatus.newInstance(ContainerId.newInstance(appAttemptId, 2),
ContainerState.COMPLETE, "", 0);
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
appAttemptId, cs2));
assertEquals(1, applicationAttempt.getJustFinishedContainers().size());
assertEquals(cs2.getContainerId(), applicationAttempt
.getJustFinishedContainers().get(0).getContainerId());
}
@Test
public void testContainersCleanupForLastAttempt() {
// create a failed attempt.
applicationAttempt =
new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), rmContext,
scheduler, masterService, submissionContext, new Configuration(),
true);
when(submissionContext.getKeepContainersAcrossApplicationAttempts())
.thenReturn(true);
when(submissionContext.getMaxAppAttempts()).thenReturn(1);
Container amContainer = allocateApplicationAttempt();
launchApplicationAttempt(amContainer);
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
ContainerStatus cs1 =
ContainerStatus.newInstance(amContainer.getId(),
ContainerState.COMPLETE, "some error", 123);
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
appAttemptId, cs1));
sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState());
assertFalse(transferStateFromPreviousAttempt);
}
private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) { private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId); verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId);
if (UserGroupInformation.isSecurityEnabled()) { if (UserGroupInformation.isSecurityEnabled()) {

View File

@ -562,18 +562,18 @@ public class TestCapacityScheduler {
new AppAddedSchedulerEvent(appId, "default", "user"); new AppAddedSchedulerEvent(appId, "default", "user");
cs.handle(addAppEvent); cs.handle(addAppEvent);
SchedulerEvent addAttemptEvent = SchedulerEvent addAttemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
cs.handle(addAttemptEvent); cs.handle(addAttemptEvent);
// Verify the blacklist can be updated independent of requesting containers // Verify the blacklist can be updated independent of requesting containers
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), Collections.<ContainerId>emptyList(),
Collections.singletonList(host), null); Collections.singletonList(host), null);
Assert.assertTrue(cs.getApplication(appAttemptId).isBlacklisted(host)); Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), null, Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host)); Collections.singletonList(host));
Assert.assertFalse(cs.getApplication(appAttemptId).isBlacklisted(host)); Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
rm.stop(); rm.stop();
} }
@ -598,66 +598,6 @@ public class TestCapacityScheduler {
assertTrue(appComparator.compare(app2, app3) < 0); assertTrue(appComparator.compare(app2, app3) < 0);
} }
@Test
public void testConcurrentAccessOnApplications() throws Exception {
CapacityScheduler cs = new CapacityScheduler();
verifyConcurrentAccessOnApplications(
cs.appAttempts, FiCaSchedulerApp.class, Queue.class);
}
public static <T extends SchedulerApplicationAttempt, Q extends Queue>
void verifyConcurrentAccessOnApplications(
final Map<ApplicationAttemptId, T> applications, Class<T> appClazz,
final Class<Q> queueClazz)
throws Exception {
final int size = 10000;
final ApplicationId appId = ApplicationId.newInstance(0, 0);
final Constructor<T> ctor = appClazz.getDeclaredConstructor(
ApplicationAttemptId.class, String.class, queueClazz,
ActiveUsersManager.class, RMContext.class);
ApplicationAttemptId appAttemptId0
= ApplicationAttemptId.newInstance(appId, 0);
applications.put(appAttemptId0, ctor.newInstance(
appAttemptId0, null, mock(queueClazz), null, null));
assertNotNull(applications.get(appAttemptId0));
// Imitating the thread of scheduler that will add and remove apps
final AtomicBoolean finished = new AtomicBoolean(false);
final AtomicBoolean failed = new AtomicBoolean(false);
Thread t = new Thread() {
@Override
public void run() {
for (int i = 1; i <= size; ++i) {
ApplicationAttemptId appAttemptId
= ApplicationAttemptId.newInstance(appId, i);
try {
applications.put(appAttemptId, ctor.newInstance(
appAttemptId, null, mock(queueClazz), null, null));
} catch (Exception e) {
failed.set(true);
finished.set(true);
return;
}
}
for (int i = 1; i <= size; ++i) {
ApplicationAttemptId appAttemptId
= ApplicationAttemptId.newInstance(appId, i);
applications.remove(appAttemptId);
}
finished.set(true);
}
};
t.start();
// Imitating the thread of rmappattempt that will get the app
while (!finished.get()) {
assertNotNull(applications.get(appAttemptId0));
}
assertFalse(failed.get());
}
@Test @Test
public void testGetAppsInQueue() throws Exception { public void testGetAppsInQueue() throws Exception {
Application application_0 = new Application("user_0", "a1", resourceManager); Application application_0 = new Application("user_0", "a1", resourceManager);

View File

@ -63,6 +63,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@ -345,11 +347,16 @@ public class TestLeafQueue {
.getMockApplicationAttemptId(0, 1); .getMockApplicationAttemptId(0, 1);
FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a, null, FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a, null,
rmContext); rmContext);
a.submitApplicationAttempt(app_0, user_0); AppAddedSchedulerEvent addAppEvent =
new AppAddedSchedulerEvent(appAttemptId_0.getApplicationId(),
a.getQueueName(), user_0);
cs.handle(addAppEvent);
AppAttemptAddedSchedulerEvent addAttemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);
cs.handle(addAttemptEvent);
when(cs.getApplication(appAttemptId_0)).thenReturn(app_0);
AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent( AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent(
appAttemptId_0, RMAppAttemptState.FAILED); appAttemptId_0, RMAppAttemptState.FAILED, false);
cs.handle(event); cs.handle(event);
assertEquals(0, a.getMetrics().getAppsPending()); assertEquals(0, a.getMetrics().getAppsPending());
@ -365,9 +372,8 @@ public class TestLeafQueue {
assertEquals(1, a.getMetrics().getAppsSubmitted()); assertEquals(1, a.getMetrics().getAppsSubmitted());
assertEquals(1, a.getMetrics().getAppsPending()); assertEquals(1, a.getMetrics().getAppsPending());
when(cs.getApplication(appAttemptId_1)).thenReturn(app_0);
event = new AppAttemptRemovedSchedulerEvent(appAttemptId_0, event = new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
RMAppAttemptState.FINISHED); RMAppAttemptState.FINISHED, false);
cs.handle(event); cs.handle(event);
assertEquals(1, a.getMetrics().getAppsSubmitted()); assertEquals(1, a.getMetrics().getAppsSubmitted());

View File

@ -261,7 +261,7 @@ public class TestFairScheduler {
// This conditional is for testAclSubmitApplication where app is rejected // This conditional is for testAclSubmitApplication where app is rejected
// and no app is added. // and no app is added.
if (scheduler.applications.containsKey(id.getApplicationId())) { if (scheduler.applications.containsKey(id.getApplicationId())) {
scheduler.addApplicationAttempt(id); scheduler.addApplicationAttempt(id, false);
} }
List<ResourceRequest> ask = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY, ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY,
@ -590,7 +590,7 @@ public class TestFairScheduler {
// Make sure queue 2 is waiting with a reservation // Make sure queue 2 is waiting with a reservation
assertEquals(0, scheduler.getQueueManager().getQueue("queue2"). assertEquals(0, scheduler.getQueueManager().getQueue("queue2").
getResourceUsage().getMemory()); getResourceUsage().getMemory());
assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory()); assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
// Now another node checks in with capacity // Now another node checks in with capacity
RMNode node2 = RMNode node2 =
@ -606,10 +606,10 @@ public class TestFairScheduler {
getResourceUsage().getMemory()); getResourceUsage().getMemory());
// The old reservation should still be there... // The old reservation should still be there...
assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory()); assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
// ... but it should disappear when we update the first node. // ... but it should disappear when we update the first node.
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
assertEquals(0, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory()); assertEquals(0, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
} }
@ -630,7 +630,7 @@ public class TestFairScheduler {
"user1"); "user1");
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attempAddedEvent = AppAttemptAddedSchedulerEvent attempAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
scheduler.handle(attempAddedEvent); scheduler.handle(attempAddedEvent);
assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
.getRunnableAppSchedulables().size()); .getRunnableAppSchedulables().size());
@ -656,7 +656,7 @@ public class TestFairScheduler {
"user2"); "user2");
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attempAddedEvent = AppAttemptAddedSchedulerEvent attempAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
scheduler.handle(attempAddedEvent); scheduler.handle(attempAddedEvent);
assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
.getRunnableAppSchedulables().size()); .getRunnableAppSchedulables().size());
@ -710,7 +710,6 @@ public class TestFairScheduler {
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId appId; ApplicationAttemptId appId;
Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.appAttempts;
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>(); List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
rules.add(new QueuePlacementRule.Specified().initialize(true, null)); rules.add(new QueuePlacementRule.Specified().initialize(true, null));
@ -723,17 +722,17 @@ public class TestFairScheduler {
scheduler.getAllocationConfiguration().placementPolicy = scheduler.getAllocationConfiguration().placementPolicy =
new QueuePlacementPolicy(rules, queues, conf); new QueuePlacementPolicy(rules, queues, conf);
appId = createSchedulingRequest(1024, "somequeue", "user1"); appId = createSchedulingRequest(1024, "somequeue", "user1");
assertEquals("root.somequeue", apps.get(appId).getQueueName()); assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user1"); appId = createSchedulingRequest(1024, "default", "user1");
assertEquals("root.user1", apps.get(appId).getQueueName()); assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user3"); appId = createSchedulingRequest(1024, "default", "user3");
assertEquals("root.user3group", apps.get(appId).getQueueName()); assertEquals("root.user3group", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user4"); appId = createSchedulingRequest(1024, "default", "user4");
assertEquals("root.user4subgroup1", apps.get(appId).getQueueName()); assertEquals("root.user4subgroup1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user5"); appId = createSchedulingRequest(1024, "default", "user5");
assertEquals("root.user5subgroup2", apps.get(appId).getQueueName()); assertEquals("root.user5subgroup2", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "otheruser"); appId = createSchedulingRequest(1024, "default", "otheruser");
assertEquals("root.default", apps.get(appId).getQueueName()); assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
// test without specified as first rule // test without specified as first rule
rules = new ArrayList<QueuePlacementRule>(); rules = new ArrayList<QueuePlacementRule>();
@ -743,11 +742,11 @@ public class TestFairScheduler {
scheduler.getAllocationConfiguration().placementPolicy = scheduler.getAllocationConfiguration().placementPolicy =
new QueuePlacementPolicy(rules, queues, conf); new QueuePlacementPolicy(rules, queues, conf);
appId = createSchedulingRequest(1024, "somequeue", "user1"); appId = createSchedulingRequest(1024, "somequeue", "user1");
assertEquals("root.user1", apps.get(appId).getQueueName()); assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "somequeue", "otheruser"); appId = createSchedulingRequest(1024, "somequeue", "otheruser");
assertEquals("root.somequeue", apps.get(appId).getQueueName()); assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "otheruser"); appId = createSchedulingRequest(1024, "default", "otheruser");
assertEquals("root.default", apps.get(appId).getQueueName()); assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
} }
@Test @Test
@ -802,13 +801,13 @@ public class TestFairScheduler {
ApplicationAttemptId id11 = createAppAttemptId(1, 1); ApplicationAttemptId id11 = createAppAttemptId(1, 1);
scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1"); scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1");
scheduler.addApplicationAttempt(id11); scheduler.addApplicationAttempt(id11, false);
ApplicationAttemptId id21 = createAppAttemptId(2, 1); ApplicationAttemptId id21 = createAppAttemptId(2, 1);
scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1"); scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1");
scheduler.addApplicationAttempt(id21); scheduler.addApplicationAttempt(id21, false);
ApplicationAttemptId id22 = createAppAttemptId(2, 2); ApplicationAttemptId id22 = createAppAttemptId(2, 2);
scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1"); scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1");
scheduler.addApplicationAttempt(id22); scheduler.addApplicationAttempt(id22, false);
int minReqSize = int minReqSize =
FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB; FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
@ -854,7 +853,7 @@ public class TestFairScheduler {
"user1"); "user1");
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attemptAddedEvent = AppAttemptAddedSchedulerEvent attemptAddedEvent =
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1)); new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
scheduler.handle(attemptAddedEvent); scheduler.handle(attemptAddedEvent);
// Scheduler should have two queues (the default and the one created for user1) // Scheduler should have two queues (the default and the one created for user1)
@ -865,7 +864,7 @@ public class TestFairScheduler {
.getRunnableAppSchedulables().size()); .getRunnableAppSchedulables().size());
AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent( AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent(
createAppAttemptId(1, 1), RMAppAttemptState.FINISHED); createAppAttemptId(1, 1), RMAppAttemptState.FINISHED, false);
// Now remove app // Now remove app
scheduler.handle(appRemovedEvent1); scheduler.handle(appRemovedEvent1);
@ -1138,12 +1137,12 @@ public class TestFairScheduler {
scheduler.handle(nodeUpdate3); scheduler.handle(nodeUpdate3);
} }
assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app3).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app3).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app6).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app6).getLiveContainers().size());
// Now new requests arrive from queues C and D // Now new requests arrive from queues C and D
ApplicationAttemptId app7 = ApplicationAttemptId app7 =
@ -1166,16 +1165,16 @@ public class TestFairScheduler {
// Make sure it is lowest priority container. // Make sure it is lowest priority container.
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size());
// First verify we are adding containers to preemption list for the application // First verify we are adding containers to preemption list for the application
assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app3).getLiveContainers(), assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app3).getLiveContainers(),
scheduler.appAttempts.get(app3).getPreemptionContainers())); scheduler.getSchedulerApp(app3).getPreemptionContainers()));
assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app6).getLiveContainers(), assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app6).getLiveContainers(),
scheduler.appAttempts.get(app6).getPreemptionContainers())); scheduler.getSchedulerApp(app6).getPreemptionContainers()));
// Pretend 15 seconds have passed // Pretend 15 seconds have passed
clock.tick(15); clock.tick(15);
@ -1185,8 +1184,8 @@ public class TestFairScheduler {
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
// At this point the containers should have been killed (since we are not simulating AM) // At this point the containers should have been killed (since we are not simulating AM)
assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
// Trigger a kill by insisting we want containers back // Trigger a kill by insisting we want containers back
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
@ -1200,22 +1199,22 @@ public class TestFairScheduler {
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
// Now A and B are below fair share, so preemption shouldn't do anything // Now A and B are below fair share, so preemption shouldn't do anything
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1374,9 +1373,9 @@ public class TestFairScheduler {
// One container should get reservation and the other should get nothing // One container should get reservation and the other should get nothing
assertEquals(1024, assertEquals(1024,
scheduler.appAttempts.get(attId1).getCurrentReservation().getMemory()); scheduler.getSchedulerApp(attId1).getCurrentReservation().getMemory());
assertEquals(0, assertEquals(0,
scheduler.appAttempts.get(attId2).getCurrentReservation().getMemory()); scheduler.getSchedulerApp(attId2).getCurrentReservation().getMemory());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1411,7 +1410,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
// App 1 should be running // App 1 should be running
assertEquals(1, scheduler.appAttempts.get(attId1).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"user1", 1); "user1", 1);
@ -1420,7 +1419,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
// App 2 should not be running // App 2 should not be running
assertEquals(0, scheduler.appAttempts.get(attId2).getLiveContainers().size()); assertEquals(0, scheduler.getSchedulerApp(attId2).getLiveContainers().size());
// Request another container for app 1 // Request another container for app 1
createSchedulingRequestExistingApplication(1024, 1, attId1); createSchedulingRequestExistingApplication(1024, 1, attId1);
@ -1429,7 +1428,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
// Request should be fulfilled // Request should be fulfilled
assertEquals(2, scheduler.appAttempts.get(attId1).getLiveContainers().size()); assertEquals(2, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1449,10 +1448,10 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
FSSchedulerApp app = scheduler.appAttempts.get(attId); FSSchedulerApp app = scheduler.getSchedulerApp(attId);
assertEquals(1, app.getLiveContainers().size()); assertEquals(1, app.getLiveContainers().size());
ContainerId containerId = scheduler.appAttempts.get(attId) ContainerId containerId = scheduler.getSchedulerApp(attId)
.getLiveContainers().iterator().next().getContainerId(); .getLiveContainers().iterator().next().getContainerId();
// Cause reservation to be created // Cause reservation to be created
@ -1521,9 +1520,9 @@ public class TestFairScheduler {
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"norealuserhasthisname2", 1); "norealuserhasthisname2", 1);
FSSchedulerApp app1 = scheduler.appAttempts.get(attId1); FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
assertNotNull("The application was not allowed", app1); assertNotNull("The application was not allowed", app1);
FSSchedulerApp app2 = scheduler.appAttempts.get(attId2); FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
assertNull("The application was allowed", app2); assertNull("The application was allowed", app2);
} }
@ -1547,7 +1546,7 @@ public class TestFairScheduler {
ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
scheduler.addApplication(appId.getApplicationId(), "queue1", "user1"); scheduler.addApplication(appId.getApplicationId(), "queue1", "user1");
scheduler.addApplicationAttempt(appId); scheduler.addApplicationAttempt(appId, false);
// 1 request with 2 nodes on the same rack. another request with 1 node on // 1 request with 2 nodes on the same rack. another request with 1 node on
// a different rack // a different rack
@ -1566,14 +1565,14 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent1); scheduler.handle(updateEvent1);
// should assign node local // should assign node local
assertEquals(1, scheduler.appAttempts.get(appId).getLiveContainers().size()); assertEquals(1, scheduler.getSchedulerApp(appId).getLiveContainers().size());
// node 2 checks in // node 2 checks in
scheduler.update(); scheduler.update();
NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
scheduler.handle(updateEvent2); scheduler.handle(updateEvent2);
// should assign rack local // should assign rack local
assertEquals(2, scheduler.appAttempts.get(appId).getLiveContainers().size()); assertEquals(2, scheduler.getSchedulerApp(appId).getLiveContainers().size());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1592,8 +1591,8 @@ public class TestFairScheduler {
"user1", 2); "user1", 2);
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app1 = scheduler.appAttempts.get(attId1); FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
FSSchedulerApp app2 = scheduler.appAttempts.get(attId2); FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true); FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true);
queue1.setPolicy(new FifoPolicy()); queue1.setPolicy(new FifoPolicy());
@ -1633,7 +1632,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = ApplicationAttemptId attId =
createSchedulingRequest(1024, "root.default", "user", 8); createSchedulingRequest(1024, "root.default", "user", 8);
FSSchedulerApp app = scheduler.appAttempts.get(attId); FSSchedulerApp app = scheduler.getSchedulerApp(attId);
// set maxAssign to 2: only 2 containers should be allocated // set maxAssign to 2: only 2 containers should be allocated
scheduler.maxAssign = 2; scheduler.maxAssign = 2;
@ -1695,10 +1694,10 @@ public class TestFairScheduler {
ApplicationAttemptId attId4 = ApplicationAttemptId attId4 =
createSchedulingRequest(1024, fifoQueue, user, 4); createSchedulingRequest(1024, fifoQueue, user, 4);
FSSchedulerApp app1 = scheduler.appAttempts.get(attId1); FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
FSSchedulerApp app2 = scheduler.appAttempts.get(attId2); FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
FSSchedulerApp app3 = scheduler.appAttempts.get(attId3); FSSchedulerApp app3 = scheduler.getSchedulerApp(attId3);
FSSchedulerApp app4 = scheduler.appAttempts.get(attId4); FSSchedulerApp app4 = scheduler.getSchedulerApp(attId4);
scheduler.getQueueManager().getLeafQueue(fifoQueue, true) scheduler.getQueueManager().getLeafQueue(fifoQueue, true)
.setPolicy(SchedulingPolicy.parse("fifo")); .setPolicy(SchedulingPolicy.parse("fifo"));
@ -1813,7 +1812,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
FSSchedulerApp app = scheduler.appAttempts.get(attId); FSSchedulerApp app = scheduler.getSchedulerApp(attId);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
assertEquals(0, app.getReservedContainers().size()); assertEquals(0, app.getReservedContainers().size());
@ -1882,7 +1881,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container // no matter how many heartbeats, node2 should never get a container
FSSchedulerApp app = scheduler.appAttempts.get(attId1); FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent); scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
@ -1921,7 +1920,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container // no matter how many heartbeats, node2 should never get a container
FSSchedulerApp app = scheduler.appAttempts.get(attId1); FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent); scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
@ -1954,7 +1953,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
"user1", 0); "user1", 0);
FSSchedulerApp app = scheduler.appAttempts.get(attId); FSSchedulerApp app = scheduler.getSchedulerApp(attId);
ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true); ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true);
ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true); ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true);
@ -1994,7 +1993,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default", ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default",
"user1", 2); "user1", 2);
FSSchedulerApp app = scheduler.appAttempts.get(attId); FSSchedulerApp app = scheduler.getSchedulerApp(attId);
scheduler.update(); scheduler.update();
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
@ -2014,10 +2013,10 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1", ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1); FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1", ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2); FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity()); drfPolicy.initialize(scheduler.getClusterCapacity());
@ -2055,13 +2054,13 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1", ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1); FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1", ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2); FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2", ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2",
"user1", 2); "user1", 2);
FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3); FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity()); drfPolicy.initialize(scheduler.getClusterCapacity());
@ -2092,19 +2091,19 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(3074, 1, "queue1.subqueue1", ApplicationAttemptId appAttId1 = createSchedulingRequest(3074, 1, "queue1.subqueue1",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1); FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 3, "queue1.subqueue1", ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 3, "queue1.subqueue1",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2); FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
ApplicationAttemptId appAttId3 = createSchedulingRequest(2048, 2, "queue1.subqueue2", ApplicationAttemptId appAttId3 = createSchedulingRequest(2048, 2, "queue1.subqueue2",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3); FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3);
ApplicationAttemptId appAttId4 = createSchedulingRequest(1024, 2, "queue2", ApplicationAttemptId appAttId4 = createSchedulingRequest(1024, 2, "queue2",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app4 = scheduler.appAttempts.get(appAttId4); FSSchedulerApp app4 = scheduler.getSchedulerApp(appAttId4);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity()); drfPolicy.initialize(scheduler.getClusterCapacity());
@ -2184,7 +2183,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container // no matter how many heartbeats, node2 should never get a container
FSSchedulerApp app = scheduler.appAttempts.get(attId1); FSSchedulerApp app = scheduler.getSchedulerApp(attId1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent); scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
@ -2195,16 +2194,8 @@ public class TestFairScheduler {
assertEquals(1, app.getLiveContainers().size()); assertEquals(1, app.getLiveContainers().size());
} }
@Test
public void testConcurrentAccessOnApplications() throws Exception {
FairScheduler fs = new FairScheduler();
TestCapacityScheduler.verifyConcurrentAccessOnApplications(
fs.appAttempts, FSSchedulerApp.class, FSLeafQueue.class);
}
private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) { private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) {
FSSchedulerApp app = scheduler.appAttempts.get(attId); FSSchedulerApp app = scheduler.getSchedulerApp(attId);
FSLeafQueue queue = app.getQueue(); FSLeafQueue queue = app.getQueue();
Collection<AppSchedulable> runnableApps = Collection<AppSchedulable> runnableApps =
queue.getRunnableAppSchedulables(); queue.getRunnableAppSchedulables();
@ -2260,7 +2251,7 @@ public class TestFairScheduler {
// Remove app 1 and both app 2 and app 4 should becomes runnable in its place // Remove app 1 and both app 2 and app 4 should becomes runnable in its place
AppAttemptRemovedSchedulerEvent appRemovedEvent1 = AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED); new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED, false);
scheduler.handle(appRemovedEvent1); scheduler.handle(appRemovedEvent1);
verifyAppRunnable(attId2, true); verifyAppRunnable(attId2, true);
verifyQueueNumRunnable("queue2", 1, 0); verifyQueueNumRunnable("queue2", 1, 0);
@ -2324,7 +2315,7 @@ public class TestFairScheduler {
// Even though the app was removed from sub3, the app from sub2 gets to go // Even though the app was removed from sub3, the app from sub2 gets to go
// because it came in first // because it came in first
AppAttemptRemovedSchedulerEvent appRemovedEvent1 = AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED); new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED, false);
scheduler.handle(appRemovedEvent1); scheduler.handle(appRemovedEvent1);
verifyAppRunnable(attId4, true); verifyAppRunnable(attId4, true);
verifyQueueNumRunnable("queue1.sub2", 2, 0); verifyQueueNumRunnable("queue1.sub2", 2, 0);
@ -2333,7 +2324,7 @@ public class TestFairScheduler {
// Now test removal of a non-runnable app // Now test removal of a non-runnable app
AppAttemptRemovedSchedulerEvent appRemovedEvent2 = AppAttemptRemovedSchedulerEvent appRemovedEvent2 =
new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED); new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED, true);
scheduler.handle(appRemovedEvent2); scheduler.handle(appRemovedEvent2);
assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps
.get("user1").size()); .get("user1").size());
@ -2341,7 +2332,7 @@ public class TestFairScheduler {
verifyQueueNumRunnable("queue1.sub3", 0, 0); verifyQueueNumRunnable("queue1.sub3", 0, 0);
// verify it doesn't become runnable when there would be space for it // verify it doesn't become runnable when there would be space for it
AppAttemptRemovedSchedulerEvent appRemovedEvent3 = AppAttemptRemovedSchedulerEvent appRemovedEvent3 =
new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED); new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED, true);
scheduler.handle(appRemovedEvent3); scheduler.handle(appRemovedEvent3);
verifyQueueNumRunnable("queue1.sub2", 1, 0); verifyQueueNumRunnable("queue1.sub2", 1, 0);
verifyQueueNumRunnable("queue1.sub3", 0, 0); verifyQueueNumRunnable("queue1.sub3", 0, 0);
@ -2378,7 +2369,7 @@ public class TestFairScheduler {
ApplicationAttemptId appAttemptId = ApplicationAttemptId appAttemptId =
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11"); fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11");
fs.addApplicationAttempt(appAttemptId); fs.addApplicationAttempt(appAttemptId, false);
List<ResourceRequest> ask = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
ResourceRequest request = ResourceRequest request =
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true); createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
@ -2389,7 +2380,7 @@ public class TestFairScheduler {
// at least one pass // at least one pass
Thread.sleep(fs.getConf().getContinuousSchedulingSleepMs() + 500); Thread.sleep(fs.getConf().getContinuousSchedulingSleepMs() + 500);
FSSchedulerApp app = fs.appAttempts.get(appAttemptId); FSSchedulerApp app = fs.getSchedulerApp(appAttemptId);
// Wait until app gets resources. // Wait until app gets resources.
while (app.getCurrentConsumption().equals(Resources.none())) { } while (app.getCurrentConsumption().equals(Resources.none())) { }
@ -2477,7 +2468,7 @@ public class TestFairScheduler {
ApplicationAttemptId appAttemptId = ApplicationAttemptId appAttemptId =
createSchedulingRequest(GB, "root.default", "user", 1); createSchedulingRequest(GB, "root.default", "user", 1);
FSSchedulerApp app = scheduler.appAttempts.get(appAttemptId); FSSchedulerApp app = scheduler.getSchedulerApp(appAttemptId);
// Verify the blacklist can be updated independent of requesting containers // Verify the blacklist can be updated independent of requesting containers
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
@ -2487,7 +2478,7 @@ public class TestFairScheduler {
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), null, Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host)); Collections.singletonList(host));
assertFalse(scheduler.appAttempts.get(appAttemptId).isBlacklisted(host)); assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
List<ResourceRequest> update = Arrays.asList( List<ResourceRequest> update = Arrays.asList(
createResourceRequest(GB, node.getHostName(), 1, 0, true)); createResourceRequest(GB, node.getHostName(), 1, 0, true));

View File

@ -156,7 +156,7 @@ public class TestFifoScheduler {
SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user"); SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user");
schedular.handle(appEvent); schedular.handle(appEvent);
SchedulerEvent attemptEvent = SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
schedular.handle(attemptEvent); schedular.handle(attemptEvent);
appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2); appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2);
@ -166,7 +166,7 @@ public class TestFifoScheduler {
"user"); "user");
schedular.handle(appEvent2); schedular.handle(appEvent2);
SchedulerEvent attemptEvent2 = SchedulerEvent attemptEvent2 =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
schedular.handle(attemptEvent2); schedular.handle(attemptEvent2);
int afterAppsSubmitted = metrics.getAppsSubmitted(); int afterAppsSubmitted = metrics.getAppsSubmitted();
@ -203,7 +203,7 @@ public class TestFifoScheduler {
"user1"); "user1");
scheduler.handle(appEvent); scheduler.handle(appEvent);
AppAttemptAddedSchedulerEvent attemptEvent = AppAttemptAddedSchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
scheduler.handle(attemptEvent); scheduler.handle(attemptEvent);
int memory = 64; int memory = 64;
@ -293,7 +293,7 @@ public class TestFifoScheduler {
"user1"); "user1");
scheduler.handle(appEvent); scheduler.handle(appEvent);
AppAttemptAddedSchedulerEvent attemptEvent = AppAttemptAddedSchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
scheduler.handle(attemptEvent); scheduler.handle(attemptEvent);
int memory = 1024; int memory = 1024;
@ -534,13 +534,6 @@ public class TestFifoScheduler {
LOG.info("--- END: testFifoScheduler ---"); LOG.info("--- END: testFifoScheduler ---");
} }
@Test
public void testConcurrentAccessOnApplications() throws Exception {
FifoScheduler fs = new FifoScheduler();
TestCapacityScheduler.verifyConcurrentAccessOnApplications(
fs.appAttempts, FiCaSchedulerApp.class, Queue.class);
}
@SuppressWarnings("resource") @SuppressWarnings("resource")
@Test @Test
public void testBlackListNodes() throws Exception { public void testBlackListNodes() throws Exception {
@ -564,18 +557,18 @@ public class TestFifoScheduler {
"user"); "user");
fs.handle(appEvent); fs.handle(appEvent);
SchedulerEvent attemptEvent = SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId); new AppAttemptAddedSchedulerEvent(appAttemptId, false);
fs.handle(attemptEvent); fs.handle(attemptEvent);
// Verify the blacklist can be updated independent of requesting containers // Verify the blacklist can be updated independent of requesting containers
fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), Collections.<ContainerId>emptyList(),
Collections.singletonList(host), null); Collections.singletonList(host), null);
Assert.assertTrue(fs.getApplication(appAttemptId).isBlacklisted(host)); Assert.assertTrue(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), null, Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host)); Collections.singletonList(host));
Assert.assertFalse(fs.getApplication(appAttemptId).isBlacklisted(host)); Assert.assertFalse(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
rm.stop(); rm.stop();
} }

View File

@ -29,6 +29,7 @@ import javax.xml.parsers.DocumentBuilder;
import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.DocumentBuilderFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.service.Service.STATE; import org.apache.hadoop.service.Service.STATE;
import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.QueueState;
@ -267,6 +268,7 @@ public class TestRMWebServices extends JerseyTest {
verifyClusterGeneric(WebServicesTestUtils.getXmlLong(element, "id"), verifyClusterGeneric(WebServicesTestUtils.getXmlLong(element, "id"),
WebServicesTestUtils.getXmlLong(element, "startedOn"), WebServicesTestUtils.getXmlLong(element, "startedOn"),
WebServicesTestUtils.getXmlString(element, "state"), WebServicesTestUtils.getXmlString(element, "state"),
WebServicesTestUtils.getXmlString(element, "haState"),
WebServicesTestUtils.getXmlString(element, "hadoopVersionBuiltOn"), WebServicesTestUtils.getXmlString(element, "hadoopVersionBuiltOn"),
WebServicesTestUtils.getXmlString(element, "hadoopBuildVersion"), WebServicesTestUtils.getXmlString(element, "hadoopBuildVersion"),
WebServicesTestUtils.getXmlString(element, "hadoopVersion"), WebServicesTestUtils.getXmlString(element, "hadoopVersion"),
@ -282,9 +284,10 @@ public class TestRMWebServices extends JerseyTest {
Exception { Exception {
assertEquals("incorrect number of elements", 1, json.length()); assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("clusterInfo"); JSONObject info = json.getJSONObject("clusterInfo");
assertEquals("incorrect number of elements", 9, info.length()); assertEquals("incorrect number of elements", 10, info.length());
verifyClusterGeneric(info.getLong("id"), info.getLong("startedOn"), verifyClusterGeneric(info.getLong("id"), info.getLong("startedOn"),
info.getString("state"), info.getString("hadoopVersionBuiltOn"), info.getString("state"), info.getString("haState"),
info.getString("hadoopVersionBuiltOn"),
info.getString("hadoopBuildVersion"), info.getString("hadoopVersion"), info.getString("hadoopBuildVersion"), info.getString("hadoopVersion"),
info.getString("resourceManagerVersionBuiltOn"), info.getString("resourceManagerVersionBuiltOn"),
info.getString("resourceManagerBuildVersion"), info.getString("resourceManagerBuildVersion"),
@ -293,9 +296,10 @@ public class TestRMWebServices extends JerseyTest {
} }
public void verifyClusterGeneric(long clusterid, long startedon, public void verifyClusterGeneric(long clusterid, long startedon,
String state, String hadoopVersionBuiltOn, String hadoopBuildVersion, String state, String haState, String hadoopVersionBuiltOn,
String hadoopVersion, String resourceManagerVersionBuiltOn, String hadoopBuildVersion, String hadoopVersion,
String resourceManagerBuildVersion, String resourceManagerVersion) { String resourceManagerVersionBuiltOn, String resourceManagerBuildVersion,
String resourceManagerVersion) {
assertEquals("clusterId doesn't match: ", assertEquals("clusterId doesn't match: ",
ResourceManager.getClusterTimeStamp(), clusterid); ResourceManager.getClusterTimeStamp(), clusterid);
@ -303,6 +307,8 @@ public class TestRMWebServices extends JerseyTest {
ResourceManager.getClusterTimeStamp(), startedon); ResourceManager.getClusterTimeStamp(), startedon);
assertTrue("stated doesn't match: " + state, assertTrue("stated doesn't match: " + state,
state.matches(STATE.INITED.toString())); state.matches(STATE.INITED.toString()));
assertTrue("HA state doesn't match: " + haState,
haState.matches("INITIALIZING"));
WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn", WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
VersionInfo.getDate(), hadoopVersionBuiltOn); VersionInfo.getDate(), hadoopVersionBuiltOn);

View File

@ -1390,7 +1390,7 @@ public class TestRMWebServicesApps extends JerseyTest {
@Test @Test
public void testMultipleAppAttempts() throws JSONException, Exception { public void testMultipleAppAttempts() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 8192);
RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(), rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
@ -1403,11 +1403,13 @@ public class TestRMWebServicesApps extends JerseyTest {
while (--retriesLeft > 0) { while (--retriesLeft > 0) {
RMAppEvent event = RMAppEvent event =
new RMAppFailedAttemptEvent(app1.getApplicationId(), new RMAppFailedAttemptEvent(app1.getApplicationId(),
RMAppEventType.ATTEMPT_FAILED, ""); RMAppEventType.ATTEMPT_FAILED, "", false);
app1.handle(event); app1.handle(event);
rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
} }
// kick the scheduler to allocate the am container.
amNodeManager.nodeHeartbeat(true);
rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(), rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
RMAppAttemptState.ALLOCATED); RMAppAttemptState.ALLOCATED);
assertEquals("incorrect number of attempts", maxAppAttempts, assertEquals("incorrect number of attempts", maxAppAttempts,

View File

@ -62,6 +62,8 @@ ResourceManager REST API's.
*---------------+--------------+-------------------------------+ *---------------+--------------+-------------------------------+
| state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED| | state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED|
*---------------+--------------+-------------------------------+ *---------------+--------------+-------------------------------+
| haState | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED|
*---------------+--------------+-------------------------------+
| resourceManagerVersion | string | Version of the ResourceManager | | resourceManagerVersion | string | Version of the ResourceManager |
*---------------+--------------+-------------------------------+ *---------------+--------------+-------------------------------+
| resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum | | resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum |