MAPREDUCE-5240. Fix a bug in MRAppMaster because of which OutputCommitter could not access credentials set by the user. Contributed by Vinod Kumar Vavilapalli.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1482175 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-05-14 02:48:53 +00:00
parent 6f236b28ff
commit 14d856be57
3 changed files with 35 additions and 15 deletions

View File

@ -410,6 +410,9 @@ Release 2.0.5-beta - UNRELEASED
MAPREDUCE-5220. Setter methods in TaskCompletionEvent are public in MR1 and
protected in MR2. (sandyr via tucu)
MAPREDUCE-5240. Fix a bug in MRAppMaster because of which OutputCommitter
could not access credentials set by the user. (vinodkv)
Release 2.0.4-alpha - 2013-04-25
INCOMPATIBLE CHANGES

View File

@ -1304,7 +1304,7 @@ public class MRAppMaster extends CompositeService {
Integer.parseInt(maxAppAttempts));
ShutdownHookManager.get().addShutdownHook(
new MRAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY);
YarnConfiguration conf = new YarnConfiguration(new JobConf());
JobConf conf = new JobConf(new YarnConfiguration());
conf.addResource(new Path(MRJobConfig.JOB_CONF_FILE));
String jobUserName = System
.getenv(ApplicationConstants.Environment.USER.name());
@ -1357,7 +1357,7 @@ public class MRAppMaster extends CompositeService {
}
protected static void initAndStartAppMaster(final MRAppMaster appMaster,
final YarnConfiguration conf, String jobUserName) throws IOException,
final JobConf conf, String jobUserName) throws IOException,
InterruptedException {
UserGroupInformation.setConfiguration(conf);
// Security framework already loaded the tokens into current UGI, just use
@ -1367,6 +1367,7 @@ public class MRAppMaster extends CompositeService {
UserGroupInformation appMasterUgi = UserGroupInformation
.createRemoteUser(jobUserName);
appMasterUgi.addCredentials(credentials);
conf.getCredentials().addAll(credentials);
appMasterUgi.doAs(new PrivilegedExceptionAction<Object>() {
@Override
public Object run() throws Exception {

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TypeConverter;
@ -61,7 +62,6 @@ import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.ConverterUtils;
@ -113,7 +113,7 @@ public class TestMRAppMaster {
MRAppMasterTest appMaster =
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
System.currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
YarnConfiguration conf = new YarnConfiguration();
JobConf conf = new JobConf();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
Path userPath = new Path(stagingDir, userName);
@ -128,7 +128,7 @@ public class TestMRAppMaster {
String applicationAttemptIdStr = "appattempt_1317529182569_0004_000002";
String containerIdStr = "container_1317529182569_0004_000002_1";
String userName = "TestAppMasterUser";
YarnConfiguration conf = new YarnConfiguration();
JobConf conf = new JobConf();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
ApplicationAttemptId applicationAttemptId = ConverterUtils
.toApplicationAttemptId(applicationAttemptIdStr);
@ -163,7 +163,7 @@ public class TestMRAppMaster {
String applicationAttemptIdStr = "appattempt_1317529182569_0004_000002";
String containerIdStr = "container_1317529182569_0004_000002_1";
String userName = "TestAppMasterUser";
YarnConfiguration conf = new YarnConfiguration();
JobConf conf = new JobConf();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
ApplicationAttemptId applicationAttemptId = ConverterUtils
.toApplicationAttemptId(applicationAttemptIdStr);
@ -199,7 +199,7 @@ public class TestMRAppMaster {
String applicationAttemptIdStr = "appattempt_1317529182569_0004_000002";
String containerIdStr = "container_1317529182569_0004_000002_1";
String userName = "TestAppMasterUser";
YarnConfiguration conf = new YarnConfiguration();
JobConf conf = new JobConf();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
ApplicationAttemptId applicationAttemptId = ConverterUtils
.toApplicationAttemptId(applicationAttemptIdStr);
@ -235,7 +235,7 @@ public class TestMRAppMaster {
String applicationAttemptIdStr = "appattempt_1317529182569_0004_000002";
String containerIdStr = "container_1317529182569_0004_000002_1";
String userName = "TestAppMasterUser";
YarnConfiguration conf = new YarnConfiguration();
JobConf conf = new JobConf();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
ApplicationAttemptId applicationAttemptId = ConverterUtils
.toApplicationAttemptId(applicationAttemptIdStr);
@ -280,7 +280,7 @@ public class TestMRAppMaster {
ApplicationAttemptId applicationAttemptId = ConverterUtils
.toApplicationAttemptId(applicationAttemptIdStr);
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
YarnConfiguration conf = new YarnConfiguration();
JobConf conf = new JobConf();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
File stagingDir =
@ -345,10 +345,12 @@ public class TestMRAppMaster {
new Token<TokenIdentifier>(identifier, password, kind, service);
Text tokenAlias = new Text("myToken");
credentials.addToken(tokenAlias, myToken);
Text keyAlias = new Text("mySecretKeyAlias");
credentials.addSecretKey(keyAlias, "mySecretKey".getBytes());
Token<? extends TokenIdentifier> storedToken =
credentials.getToken(tokenAlias);
YarnConfiguration conf = new YarnConfiguration();
JobConf conf = new JobConf();
Path tokenFilePath = new Path(testDir.getAbsolutePath(), "tokens-file");
Map<String, String> newEnv = new HashMap<String, String>();
@ -381,14 +383,28 @@ public class TestMRAppMaster {
// Now validate the credentials
Credentials appMasterCreds = appMaster.credentials;
Assert.assertNotNull(appMasterCreds);
Assert.assertEquals(1, appMasterCreds.numberOfSecretKeys());
Assert.assertEquals(1, appMasterCreds.numberOfTokens());
// Validate the tokens
Token<? extends TokenIdentifier> usedToken =
appMasterCreds.getToken(tokenAlias);
Assert.assertNotNull(usedToken);
Assert
.assertEquals("MyIdentifier", new String(storedToken.getIdentifier()));
Assert.assertEquals("MyPassword", new String(storedToken.getPassword()));
Assert.assertEquals("MyTokenKind", storedToken.getKind().toString());
Assert.assertEquals("host:port", storedToken.getService().toString());
Assert.assertEquals(storedToken, usedToken);
// Validate the keys
byte[] usedKey = appMasterCreds.getSecretKey(keyAlias);
Assert.assertNotNull(usedKey);
Assert.assertEquals("mySecretKey", new String(usedKey));
// The credentials should also be added to conf so that OuputCommitter can
// access it
Credentials confCredentials = conf.getCredentials();
Assert.assertEquals(1, confCredentials.numberOfSecretKeys());
Assert.assertEquals(1, confCredentials.numberOfTokens());
Assert.assertEquals(storedToken, confCredentials.getToken(tokenAlias));
Assert.assertEquals("mySecretKey",
new String(confCredentials.getSecretKey(keyAlias)));
}
}