|
@@ -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)));
|
|
|
}
|
|
|
}
|
|
|
|