|
@@ -36,12 +36,19 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.FileContext;
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.RemoteIterator;
|
|
|
+import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.net.ServerSocketUtil;
|
|
|
import org.apache.hadoop.util.JarFinder;
|
|
|
import org.apache.hadoop.util.Shell;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
|
|
@@ -50,29 +57,50 @@ import org.apache.hadoop.yarn.client.api.YarnClient;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils;
|
|
|
+import org.apache.hadoop.yarn.server.timeline.NameValuePair;
|
|
|
+import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
|
|
|
+import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
|
|
|
+import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.rules.Timeout;
|
|
|
|
|
|
public class TestDistributedShell {
|
|
|
|
|
|
private static final Log LOG =
|
|
|
LogFactory.getLog(TestDistributedShell.class);
|
|
|
|
|
|
- protected MiniYARNCluster yarnCluster = null;
|
|
|
+ protected MiniYARNCluster yarnCluster = null;
|
|
|
+ protected MiniDFSCluster hdfsCluster = null;
|
|
|
+ private FileSystem fs = null;
|
|
|
protected YarnConfiguration conf = null;
|
|
|
private static final int NUM_NMS = 1;
|
|
|
+ private static final float DEFAULT_TIMELINE_VERSION = 1.0f;
|
|
|
|
|
|
protected final static String APPMASTER_JAR =
|
|
|
JarFinder.getJar(ApplicationMaster.class);
|
|
|
|
|
|
+ @Rule
|
|
|
+ public TimelineVersionWatcher timelineVersionWatcher
|
|
|
+ = new TimelineVersionWatcher();
|
|
|
+ @Rule
|
|
|
+ public Timeout globalTimeout = new Timeout(90000);
|
|
|
+
|
|
|
@Before
|
|
|
public void setup() throws Exception {
|
|
|
- setupInternal(NUM_NMS);
|
|
|
+ setupInternal(NUM_NMS, timelineVersionWatcher.getTimelineVersion());
|
|
|
}
|
|
|
|
|
|
protected void setupInternal(int numNodeManager) throws Exception {
|
|
|
+ setupInternal(numNodeManager, DEFAULT_TIMELINE_VERSION);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setupInternal(int numNodeManager, float timelineVersion)
|
|
|
+ throws Exception {
|
|
|
|
|
|
LOG.info("Starting up YARN cluster");
|
|
|
|
|
@@ -84,6 +112,26 @@ public class TestDistributedShell {
|
|
|
conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
|
|
|
conf.set("mapreduce.jobhistory.address",
|
|
|
"0.0.0.0:" + ServerSocketUtil.getPort(10021, 10));
|
|
|
+
|
|
|
+ // ATS version specific settings
|
|
|
+ if (timelineVersion == 1.0f) {
|
|
|
+ conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.0f);
|
|
|
+ conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
|
|
|
+ CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT);
|
|
|
+ } else if (timelineVersion == 1.5f) {
|
|
|
+ if (hdfsCluster == null) {
|
|
|
+ HdfsConfiguration hdfsConfig = new HdfsConfiguration();
|
|
|
+ hdfsCluster = new MiniDFSCluster.Builder(hdfsConfig)
|
|
|
+ .numDataNodes(1).build();
|
|
|
+ }
|
|
|
+ fs = hdfsCluster.getFileSystem();
|
|
|
+ PluginStoreTestUtils.prepareFileSystemForPluginStore(fs);
|
|
|
+ PluginStoreTestUtils.prepareConfiguration(conf, hdfsCluster);
|
|
|
+ conf.set(YarnConfiguration.TIMELINE_SERVICE_ENTITY_GROUP_PLUGIN_CLASSES,
|
|
|
+ DistributedShellTimelinePlugin.class.getName());
|
|
|
+ } else {
|
|
|
+ Assert.fail("Wrong timeline version number: " + timelineVersion);
|
|
|
+ }
|
|
|
|
|
|
if (yarnCluster == null) {
|
|
|
yarnCluster =
|
|
@@ -138,6 +186,13 @@ public class TestDistributedShell {
|
|
|
yarnCluster = null;
|
|
|
}
|
|
|
}
|
|
|
+ if (hdfsCluster != null) {
|
|
|
+ try {
|
|
|
+ hdfsCluster.shutdown();
|
|
|
+ } finally {
|
|
|
+ hdfsCluster = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
FileContext fsContext = FileContext.getLocalFSFileContext();
|
|
|
fsContext
|
|
|
.delete(
|
|
@@ -146,16 +201,28 @@ public class TestDistributedShell {
|
|
|
true);
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSShellWithDomain() throws Exception {
|
|
|
testDSShell(true);
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSShellWithoutDomain() throws Exception {
|
|
|
testDSShell(false);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ @TimelineVersion(1.5f)
|
|
|
+ public void testDSShellWithoutDomainV1_5() throws Exception {
|
|
|
+ testDSShell(false);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ @TimelineVersion(1.5f)
|
|
|
+ public void testDSShellWithDomainV1_5() throws Exception {
|
|
|
+ testDSShell(true);
|
|
|
+ }
|
|
|
+
|
|
|
public void testDSShell(boolean haveDomain) throws Exception {
|
|
|
String[] args = {
|
|
|
"--jar",
|
|
@@ -239,6 +306,24 @@ public class TestDistributedShell {
|
|
|
LOG.info("Client run completed. Result=" + result);
|
|
|
Assert.assertTrue(result.get());
|
|
|
|
|
|
+ if (timelineVersionWatcher.getTimelineVersion() == 1.5f) {
|
|
|
+ long scanInterval = conf.getLong(
|
|
|
+ YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS,
|
|
|
+ YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS_DEFAULT
|
|
|
+ );
|
|
|
+ Path doneDir = new Path(
|
|
|
+ YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT
|
|
|
+ );
|
|
|
+ // Wait till the data is moved to done dir, or timeout and fail
|
|
|
+ while (true) {
|
|
|
+ RemoteIterator<FileStatus> iterApps = fs.listStatusIterator(doneDir);
|
|
|
+ if (iterApps.hasNext()) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ Thread.sleep(scanInterval * 2);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
TimelineDomain domain = null;
|
|
|
if (haveDomain) {
|
|
|
domain = yarnCluster.getApplicationHistoryServer()
|
|
@@ -265,11 +350,18 @@ public class TestDistributedShell {
|
|
|
Assert.assertEquals("DEFAULT",
|
|
|
entitiesAttempts.getEntities().get(0).getDomainId());
|
|
|
}
|
|
|
+ String currAttemptEntityId
|
|
|
+ = entitiesAttempts.getEntities().get(0).getEntityId();
|
|
|
+ ApplicationAttemptId attemptId
|
|
|
+ = ConverterUtils.toApplicationAttemptId(currAttemptEntityId);
|
|
|
+ NameValuePair primaryFilter = new NameValuePair(
|
|
|
+ ApplicationMaster.APPID_TIMELINE_FILTER_NAME,
|
|
|
+ attemptId.getApplicationId().toString());
|
|
|
TimelineEntities entities = yarnCluster
|
|
|
.getApplicationHistoryServer()
|
|
|
.getTimelineStore()
|
|
|
.getEntities(ApplicationMaster.DSEntity.DS_CONTAINER.toString(), null,
|
|
|
- null, null, null, null, null, null, null, null);
|
|
|
+ null, null, null, null, primaryFilter, null, null, null);
|
|
|
Assert.assertNotNull(entities);
|
|
|
Assert.assertEquals(2, entities.getEntities().size());
|
|
|
Assert.assertEquals(entities.getEntities().get(0).getEntityType()
|
|
@@ -341,7 +433,7 @@ public class TestDistributedShell {
|
|
|
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSRestartWithPreviousRunningContainers() throws Exception {
|
|
|
String[] args = {
|
|
|
"--jar",
|
|
@@ -376,7 +468,7 @@ public class TestDistributedShell {
|
|
|
* how many attempt failures for previous 2.5 seconds.
|
|
|
* The application is expected to be successful.
|
|
|
*/
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSAttemptFailuresValidityIntervalSucess() throws Exception {
|
|
|
String[] args = {
|
|
|
"--jar",
|
|
@@ -414,7 +506,7 @@ public class TestDistributedShell {
|
|
|
* how many attempt failure for previous 15 seconds.
|
|
|
* The application is expected to be fail.
|
|
|
*/
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSAttemptFailuresValidityIntervalFailed() throws Exception {
|
|
|
String[] args = {
|
|
|
"--jar",
|
|
@@ -446,7 +538,7 @@ public class TestDistributedShell {
|
|
|
Assert.assertFalse(result);
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSShellWithCustomLogPropertyFile() throws Exception {
|
|
|
final File basedir =
|
|
|
new File("target", TestDistributedShell.class.getName());
|
|
@@ -541,7 +633,7 @@ public class TestDistributedShell {
|
|
|
verifyContainerLog(2, expectedContent, false, "");
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSShellWithMultipleArgs() throws Exception {
|
|
|
String[] args = {
|
|
|
"--jar",
|
|
@@ -575,7 +667,7 @@ public class TestDistributedShell {
|
|
|
verifyContainerLog(4, expectedContent, false, "");
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSShellWithShellScript() throws Exception {
|
|
|
final File basedir =
|
|
|
new File("target", TestDistributedShell.class.getName());
|
|
@@ -623,7 +715,7 @@ public class TestDistributedShell {
|
|
|
verifyContainerLog(1, expectedContent, false, "");
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDSShellWithInvalidArgs() throws Exception {
|
|
|
Client client = new Client(new Configuration(yarnCluster.getConfig()));
|
|
|
|
|
@@ -785,7 +877,7 @@ public class TestDistributedShell {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testContainerLaunchFailureHandling() throws Exception {
|
|
|
String[] args = {
|
|
|
"--jar",
|
|
@@ -813,7 +905,7 @@ public class TestDistributedShell {
|
|
|
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=90000)
|
|
|
+ @Test
|
|
|
public void testDebugFlag() throws Exception {
|
|
|
String[] args = {
|
|
|
"--jar",
|