Ver código fonte

YARN-5513. Move Java only tests from slider develop to yarn-native-services. Contributed by Gour Saha

Jian He 8 anos atrás
pai
commit
02e2740bdf
39 arquivos alterados com 5368 adições e 0 exclusões
  1. 20 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/dev-support/findbugs-exclude.xml
  2. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
  3. 159 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
  4. 157 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncher.java
  5. 92 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncherWithAmReset.java
  6. 222 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
  7. 77 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
  8. 76 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java
  9. 94 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java
  10. 264 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java
  11. 115 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java
  12. 33 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java
  13. 107 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java
  14. 177 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
  15. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/TestServiceRecordAttributes.java
  16. 60 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
  17. 40 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java
  18. 37 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/servicemonitor/TestPortProbe.java
  19. 540 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java
  20. 156 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java
  21. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java
  22. 70 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java
  23. 96 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java
  24. 46 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java
  25. 116 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java
  26. 113 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java
  27. 66 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java
  28. 107 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java
  29. 151 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java
  30. 64 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java
  31. 139 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java
  32. 901 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/ContractTestUtils.java
  33. 395 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/MiniZooKeeperCluster.java
  34. 181 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/tools/TestUtility.java
  35. 16 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt
  36. 98 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml
  37. 16 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt
  38. 17 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml
  39. 180 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml

+ 20 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/dev-support/findbugs-exclude.xml

@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml

@@ -127,6 +127,13 @@
       <scope>compile</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
@@ -267,6 +274,20 @@
       <scope>test</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <version>3.1</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-easymock</artifactId>
+      <version>1.5</version>
+      <scope>test</scope>
+    </dependency>
+
     <dependency>
       <groupId>org.mortbay.jetty</groupId>
       <artifactId>jetty</artifactId>
@@ -292,6 +313,7 @@
       <groupId>org.codehaus.jettison</groupId>
       <artifactId>jettison</artifactId>
     </dependency>
+
     <dependency>
       <groupId>org.mortbay.jetty</groupId>
       <artifactId>jetty-sslengine</artifactId>

+ 159 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java

@@ -0,0 +1,159 @@
+/*
+ * 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.slider.common.tools;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl;
+import org.apache.slider.tools.TestUtility;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+/** Test slider util methods. */
+public class TestSliderUtils {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestSliderUtils.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Test
+  public void testGetMetaInfoStreamFromZip() throws Exception {
+    String zipFileName = TestUtility.createAppPackage(
+        folder,
+        "testpkg",
+        "test.zip",
+        "target/test-classes/org/apache/slider/common/tools/test");
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    InputStream stream = SliderUtils.getApplicationResourceInputStream(
+        sliderFileSystem.getFileSystem(),
+        new Path(zipFileName),
+        "metainfo.xml");
+    Assert.assertTrue(stream != null);
+    Assert.assertTrue(stream.available() > 0);
+  }
+
+  @Test
+  public void testTruncate() {
+    Assert.assertEquals(SliderUtils.truncate(null, 5), null);
+    Assert.assertEquals(SliderUtils.truncate("323", -1), "323");
+    Assert.assertEquals(SliderUtils.truncate("3232", 5), "3232");
+    Assert.assertEquals(SliderUtils.truncate("1234567890", 0), "1234567890");
+    Assert.assertEquals(SliderUtils.truncate("123456789012345", 15), "123456789012345");
+    Assert.assertEquals(SliderUtils.truncate("123456789012345", 14), "12345678901...");
+    Assert.assertEquals(SliderUtils.truncate("1234567890", 1), "1");
+    Assert.assertEquals(SliderUtils.truncate("1234567890", 10), "1234567890");
+    Assert.assertEquals(SliderUtils.truncate("", 10), "");
+  }
+
+  @Test
+  public void testApplicationReportComparison() {
+    List<ApplicationReport> instances = getApplicationReports();
+
+    SliderUtils.sortApplicationsByMostRecent(instances);
+
+    Assert.assertEquals(1000, instances.get(0).getStartTime());
+    Assert.assertEquals(1000, instances.get(1).getStartTime());
+    Assert.assertEquals(1000, instances.get(2).getStartTime());
+    Assert.assertEquals(1000, instances.get(3).getStartTime());
+
+    instances = getApplicationReports();
+
+    SliderUtils.sortApplicationReport(instances);
+    Assert.assertEquals(1000, instances.get(0).getStartTime());
+    Assert.assertEquals(1000, instances.get(1).getStartTime());
+    Assert.assertEquals(1000, instances.get(2).getStartTime());
+    Assert.assertEquals(1000, instances.get(3).getStartTime());
+
+    Assert.assertTrue(instances.get(0).getYarnApplicationState() == YarnApplicationState.ACCEPTED ||
+                      instances.get(0).getYarnApplicationState() == YarnApplicationState.RUNNING);
+    Assert.assertTrue(instances.get(1).getYarnApplicationState() == YarnApplicationState.ACCEPTED ||
+                      instances.get(1).getYarnApplicationState() == YarnApplicationState.RUNNING);
+    Assert.assertTrue(instances.get(2).getYarnApplicationState() == YarnApplicationState.ACCEPTED ||
+                      instances.get(2).getYarnApplicationState() == YarnApplicationState.RUNNING);
+    Assert.assertTrue(instances.get(3).getYarnApplicationState() == YarnApplicationState.KILLED);
+  }
+
+  private List<ApplicationReport> getApplicationReports() {
+    List<ApplicationReport> instances = new ArrayList<ApplicationReport>();
+    instances.add(getApplicationReport(1000, 0, "app1", YarnApplicationState.ACCEPTED));
+    instances.add(getApplicationReport(900, 998, "app1", YarnApplicationState.KILLED));
+    instances.add(getApplicationReport(900, 998, "app2", YarnApplicationState.FAILED));
+    instances.add(getApplicationReport(1000, 0, "app2", YarnApplicationState.RUNNING));
+    instances.add(getApplicationReport(800, 837, "app3", YarnApplicationState.FINISHED));
+    instances.add(getApplicationReport(1000, 0, "app3", YarnApplicationState.RUNNING));
+    instances.add(getApplicationReport(900, 998, "app3", YarnApplicationState.KILLED));
+    instances.add(getApplicationReport(800, 837, "app4", YarnApplicationState.FINISHED));
+    instances.add(getApplicationReport(1000, 1050, "app4", YarnApplicationState.KILLED));
+    instances.add(getApplicationReport(900, 998, "app4", YarnApplicationState.FINISHED));
+
+    Assert.assertEquals("app1", instances.get(0).getApplicationType());
+    Assert.assertEquals("app1", instances.get(1).getApplicationType());
+    Assert.assertEquals("app2", instances.get(2).getApplicationType());
+    Assert.assertEquals("app2", instances.get(3).getApplicationType());
+    return instances;
+  }
+
+  private ApplicationReportPBImpl getApplicationReport(long startTime,
+                                                       long finishTime,
+                                                       String name,
+                                                       YarnApplicationState state) {
+    ApplicationReportPBImpl ar = new ApplicationReportPBImpl();
+    ar.setFinishTime(finishTime);
+    ar.setStartTime(startTime);
+    ar.setApplicationType(name);
+    ar.setYarnApplicationState(state);
+    return ar;
+  }
+
+
+  @Test
+  public void testGetHdpVersion() {
+    String hdpVersion = "2.3.2.0-2766";
+    Assert.assertEquals("Version should be empty", null,
+        SliderUtils.getHdpVersion());
+  }
+
+  @Test
+  public void testIsHdp() {
+    Assert.assertFalse("Should be false", SliderUtils.isHdp());
+  }
+
+  @Test
+  public void testWrite() throws IOException {
+    File testWriteFile = folder.newFile("testWrite");
+    SliderUtils.write(testWriteFile, "test".getBytes("UTF-8"), true);
+    Assert.assertTrue(FileUtils.readFileToString(testWriteFile, "UTF-8").equals("test"));
+  }
+}

+ 157 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncher.java

@@ -0,0 +1,157 @@
+/**
+ * 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.slider.core.launch;
+
+import java.lang.reflect.Method;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.SliderKeys;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestAppMasterLauncher {
+  SliderYarnClientImpl mockYarnClient;
+  YarnClientApplication yarnClientApp;
+  ApplicationSubmissionContext appSubmissionContext;
+  Set<String> tags = Collections.emptySet();
+  AppMasterLauncher appMasterLauncher = null;
+  boolean isOldApi = true;
+  Method rolledLogsIncludeMethod = null;
+  Method rolledLogsExcludeMethod = null;
+
+  @Before
+  public void initialize() throws Exception {
+    mockYarnClient = EasyMock.createNiceMock(SliderYarnClientImpl.class);
+    yarnClientApp = EasyMock.createNiceMock(YarnClientApplication.class);
+    appSubmissionContext = EasyMock
+        .createNiceMock(ApplicationSubmissionContext.class);
+    EasyMock.expect(yarnClientApp.getApplicationSubmissionContext())
+        .andReturn(appSubmissionContext).once();
+    EasyMock.expect(mockYarnClient.createApplication())
+        .andReturn(yarnClientApp).once();
+
+    try {
+      LogAggregationContext.class.getMethod("newInstance", String.class,
+          String.class, String.class, String.class);
+      isOldApi = false;
+      rolledLogsIncludeMethod = LogAggregationContext.class
+          .getMethod("getRolledLogsIncludePattern");
+      rolledLogsExcludeMethod = LogAggregationContext.class
+          .getMethod("getRolledLogsExcludePattern");
+    } catch (Exception e) {
+      isOldApi = true;
+    }
+  }
+
+  /**
+   * These tests will probably fail when compiled against hadoop 2.7+. Please
+   * refer to SLIDER-810. It has been purposely not modified so that it fails
+   * and that someone needs to modify the code in
+   * {@code AbstractLauncher#extractLogAggregationContext(Map)}. Comments are
+   * provided in that method as to what needs to be done.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testExtractLogAggregationContext() throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ResourceKeys.YARN_LOG_INCLUDE_PATTERNS,
+        " | slider*.txt  |agent.out| |");
+    options.put(ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS,
+        "command*.json|  agent.log*        |     ");
+
+    EasyMock.replay(mockYarnClient, appSubmissionContext, yarnClientApp);
+    appMasterLauncher = new AppMasterLauncher("cl1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    // Verify the include/exclude patterns
+    String expectedInclude = "slider*.txt|agent.out";
+    String expectedExclude = "command*.json|agent.log*";
+    assertPatterns(expectedInclude, expectedExclude);
+
+    EasyMock.verify(mockYarnClient, appSubmissionContext, yarnClientApp);
+
+  }
+
+  @Test
+  public void testExtractLogAggregationContextEmptyIncludePattern()
+      throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ResourceKeys.YARN_LOG_INCLUDE_PATTERNS, " ");
+    options.put(ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS,
+        "command*.json|  agent.log*        |     ");
+
+    EasyMock.replay(mockYarnClient, appSubmissionContext, yarnClientApp);
+    appMasterLauncher = new AppMasterLauncher("cl1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    // Verify the include/exclude patterns
+    String expectedInclude = isOldApi ? "" : ".*";
+    String expectedExclude = "command*.json|agent.log*";
+    assertPatterns(expectedInclude, expectedExclude);
+
+    EasyMock.verify(mockYarnClient, appSubmissionContext, yarnClientApp);
+  }
+
+  @Test
+  public void testExtractLogAggregationContextEmptyIncludeAndExcludePattern()
+      throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ResourceKeys.YARN_LOG_INCLUDE_PATTERNS, "");
+    options.put(ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS, "  ");
+
+    EasyMock.replay(mockYarnClient, appSubmissionContext, yarnClientApp);
+    appMasterLauncher = new AppMasterLauncher("cl1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    // Verify the include/exclude patterns
+    String expectedInclude = isOldApi ? "" : ".*";
+    String expectedExclude = "";
+    assertPatterns(expectedInclude, expectedExclude);
+
+    EasyMock.verify(mockYarnClient, appSubmissionContext, yarnClientApp);
+  }
+
+  private void assertPatterns(String expectedIncludePattern,
+      String expectedExcludePattern) throws Exception {
+    if (isOldApi) {
+      Assert.assertEquals(expectedIncludePattern,
+          appMasterLauncher.logAggregationContext.getIncludePattern());
+      Assert.assertEquals(expectedExcludePattern,
+          appMasterLauncher.logAggregationContext.getExcludePattern());
+    } else {
+      Assert.assertEquals(expectedIncludePattern,
+          (String) rolledLogsIncludeMethod
+              .invoke(appMasterLauncher.logAggregationContext));
+      Assert.assertEquals(expectedExcludePattern,
+          (String) rolledLogsExcludeMethod
+              .invoke(appMasterLauncher.logAggregationContext));
+    }
+  }
+}

+ 92 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncherWithAmReset.java

@@ -0,0 +1,92 @@
+/**
+ * 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.slider.core.launch;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.SliderKeys;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestAppMasterLauncherWithAmReset {
+  SliderYarnClientImpl mockYarnClient;
+  YarnClientApplication yarnClientApp;
+  ApplicationSubmissionContext appSubmissionContext;
+  GetNewApplicationResponse newApp;
+  Set<String> tags = Collections.emptySet();
+  AppMasterLauncher appMasterLauncher = null;
+  boolean isOldApi = true;
+
+  @Before
+  public void initialize() throws Exception {
+    mockYarnClient = EasyMock.createNiceMock(SliderYarnClientImpl.class);
+    yarnClientApp = EasyMock.createNiceMock(YarnClientApplication.class);
+    newApp = EasyMock.createNiceMock(GetNewApplicationResponse.class);
+    EasyMock.expect(mockYarnClient.createApplication())
+        .andReturn(new YarnClientApplication(newApp,
+        Records.newRecord(ApplicationSubmissionContext.class)));
+  }
+
+  @Test
+  public void testExtractYarnResourceManagerAmRetryCountWindowMs() throws
+      Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    final String expectedInterval = Integer.toString (120000);
+    options.put(ResourceKeys.YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS,
+        expectedInterval);
+    EasyMock.replay(mockYarnClient, yarnClientApp);
+
+    appMasterLauncher = new AppMasterLauncher("am1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    ApplicationSubmissionContext ctx = appMasterLauncher.application
+        .getApplicationSubmissionContext();
+    String retryIntervalWindow = Long.toString(ctx
+        .getAttemptFailuresValidityInterval());
+    Assert.assertEquals(expectedInterval, retryIntervalWindow);
+  }
+
+  @Test
+  public void testExtractYarnResourceManagerAmRetryCountWindowMsDefaultValue()
+      throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    EasyMock.replay(mockYarnClient, yarnClientApp);
+
+    appMasterLauncher = new AppMasterLauncher("am1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    ApplicationSubmissionContext ctx = appMasterLauncher.application
+        .getApplicationSubmissionContext();
+    long retryIntervalWindow = ctx.getAttemptFailuresValidityInterval();
+    Assert.assertEquals(ResourceKeys.DEFAULT_AM_RETRY_COUNT_WINDOW_MS,
+        retryIntervalWindow);
+  }
+
+}

+ 222 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java

@@ -0,0 +1,222 @@
+/*
+ * 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.slider.core.registry.docstore;
+
+import com.google.common.base.Charsets;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.powermock.api.easymock.PowerMock;
+import org.yaml.snakeyaml.Yaml;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.expect;
+import static org.mockito.Matchers.anyString;
+import static org.powermock.api.easymock.PowerMock.createNiceMock;
+
+public class TestPublishedConfigurationOutputter {
+  private static HashMap<String, String> config = new HashMap<>();
+
+  @Rule
+  public TemporaryFolder tmpDir = new TemporaryFolder();
+
+  @Before
+  public void setup() {
+    config.put("key1", "val1");
+  }
+
+  @Test
+  public void testJson() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.JSON,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
+        "");
+    assert "{\"key1\":\"val1\"}".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    ObjectMapper mapper = new ObjectMapper();
+    @SuppressWarnings("unchecked")
+    Map<String, String> read = mapper.readValue(file, Map.class);
+    assert 1 == read.size();
+    assert "val1".equals(read.get("key1"));
+  }
+
+  @Test
+  public void testXml() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.XML,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
+        "");
+    assert output.contains(
+        "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    assert FileUtils.readFileToString(file, Charsets.UTF_8)
+        .replaceAll("( |\\r|\\n)", "")
+        .contains(
+            "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+  }
+
+  @Test
+  public void testHadoopXml() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.HADOOP_XML,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
+        "");
+    assert output.contains("<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    assert FileUtils.readFileToString(file, Charsets.UTF_8)
+        .replaceAll("( |\\r|\\n)", "")
+        .contains( "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+  }
+
+  @Test
+  public void testProperties() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.PROPERTIES,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString();
+    assert output.contains("key1=val1");
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    Properties properties = new Properties();
+    FileInputStream fis = null;
+    try {
+      fis = new FileInputStream(file);
+      properties.load(fis);
+    } finally {
+      if (fis != null) {
+        fis.close();
+      }
+    }
+    assert 1 == properties.size();
+    assert "val1".equals(properties.getProperty("key1"));
+  }
+
+  @Test
+  public void testYaml() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.YAML,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("(\\r|\\n)",
+        "");
+    assert "key1: val1".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    Yaml yaml = new Yaml();
+    FileInputStream fis = null;
+    Map<String, String> read;
+    try {
+      fis = new FileInputStream(file);
+      read = (Map<String, String>) yaml.load(fis);
+    } finally {
+      if (fis != null) {
+        fis.close();
+      }
+    }
+    assert 1 == read.size();
+    assert "val1".equals(read.get("key1"));
+  }
+
+  @Test
+  public void testEnv() throws IOException {
+    HashMap<String, String> envConfig = new HashMap<>(config);
+    envConfig.put("content", "content {{key1}} ");
+
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.ENV,
+            new PublishedConfiguration("description",
+                envConfig.entrySet()));
+
+    String output = configurationOutputter.asString();
+    assert "content val1 ".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    assert "content val1 ".equals(FileUtils.readFileToString(file,
+        Charsets.UTF_8));
+  }
+
+  @Test
+  public void testTemplate1() throws IOException {
+    HashMap<String, String> templateConfig = new HashMap<>(config);
+    templateConfig.put(ConfigUtils.TEMPLATE_FILE, "templateFileName");
+
+    SliderFileSystem fileSystem = createNiceMock(SliderFileSystem.class);
+    expect(fileSystem.buildResourcePath(anyString())).andReturn(new Path("path")).anyTimes();
+    expect(fileSystem.isFile(anyObject(Path.class))).andReturn(true).anyTimes();
+    expect(fileSystem.cat(anyObject(Path.class))).andReturn("content {{key1}}\n more ${key1} content").anyTimes();
+
+    PowerMock.replay(fileSystem);
+
+    ConfigUtils.prepConfigForTemplateOutputter(ConfigFormat.TEMPLATE,
+        templateConfig, fileSystem, "clusterName", null);
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.TEMPLATE,
+            new PublishedConfiguration("description",
+                templateConfig.entrySet()));
+
+    String output = configurationOutputter.asString();
+    assert "content val1\n more val1 content".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    PowerMock.verify(fileSystem);
+
+    assert "content val1\n more val1 content".equals(
+        FileUtils.readFileToString(file, Charsets.UTF_8));
+  }
+}

+ 77 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java

@@ -0,0 +1,77 @@
+/*
+ * 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.slider.providers.agent;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.tools.TestUtility;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+
+/**
+ *
+ */
+public class TestAgentClientProvider {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentClientProvider.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Test
+  public void testGetApplicationTags() throws Exception {
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    AgentClientProvider provider = new AgentClientProvider(null);
+    String zipFileName = TestUtility.createAppPackage(
+        folder,
+        "testpkg",
+        "test.zip",
+        "target/test-classes/org/apache/slider/common/tools/test");
+    Set<String> tags = provider.getApplicationTags(sliderFileSystem, zipFileName);
+    assert tags != null;
+    assert !tags.isEmpty();
+    assert tags.contains("Name: STORM");
+    assert tags.contains("Description: Apache Hadoop Stream processing framework");
+    assert tags.contains("Version: 0.9.1.2.1");
+
+  }
+
+  @Test
+  public void testValidateInstanceDefinition() throws Exception {
+    AgentClientProvider provider = new AgentClientProvider(null);
+    AggregateConf instanceDefinition = new AggregateConf();
+
+    try {
+      provider.validateInstanceDefinition(instanceDefinition, null);
+      Assert.assertFalse("Should fail with BadConfigException", true);
+    } catch (BadConfigException e) {
+      log.info(e.toString());
+      Assert.assertTrue(e.getMessage().contains("Application definition must be provided"));
+    }
+  }
+}

+ 76 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java

@@ -0,0 +1,76 @@
+/*
+ * 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.slider.providers.agent;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ *
+ */
+public class TestAgentLaunchParameter {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentLaunchParameter.class);
+
+  @Test
+  public void testTestAgentLaunchParameter() throws Exception {
+    AgentLaunchParameter alp = new AgentLaunchParameter("");
+    Assert.assertEquals("", alp.getNextLaunchParameter("abc"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("HBASE_MASTER"));
+
+    alp = new AgentLaunchParameter("a:1:2:3|b:5:6:NONE");
+    Assert.assertEquals("1", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("2", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+
+    Assert.assertEquals("5", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("6", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("c"));
+
+    alp = new AgentLaunchParameter("|a:1:3|b::5:NONE:");
+    Assert.assertEquals("1", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("5", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+
+    alp = new AgentLaunchParameter("|:");
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("a"));
+
+    alp = new AgentLaunchParameter("HBASE_MASTER:a,b:DO_NOT_REGISTER:");
+    Assert.assertEquals("a,b", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
+
+    alp = new AgentLaunchParameter("HBASE_MASTER:a,b:DO_NOT_REGISTER::c:::");
+    Assert.assertEquals("a,b", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("c", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("c", alp.getNextLaunchParameter("HBASE_MASTER"));
+  }
+}

+ 94 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java

@@ -0,0 +1,94 @@
+/**
+ * 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.slider.providers.agent;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.providers.agent.application.metadata.Metainfo;
+import org.apache.slider.tools.TestUtility;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestAgentUtils {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentUtils.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private static final String metainfo_str = "<metainfo>\n"
+      + "  <schemaVersion>2.0</schemaVersion>\n"
+      + "  <application>\n"
+      + "      <name>MYTESTAPPLICATION</name>\n"
+      + "      <comment>\n"
+      + "        My Test Application\n"
+      + "      </comment>\n"
+      + "      <version>1.0</version>\n"
+      + "      <type>YARN-APP</type>\n"
+      + "      <components>\n"
+      + "        <component>\n"
+      + "          <name>REST</name>\n"
+      + "          <category>MASTER</category>\n"
+      + "          <commandScript>\n"
+      + "            <script>scripts/rest.py</script>\n"
+      + "            <scriptType>PYTHON</scriptType>\n"
+      + "            <timeout>600</timeout>\n"
+      + "          </commandScript>\n"
+      + "        </component>\n"
+      + "      </components>\n"
+      + "  </application>\n"
+      + "</metainfo>";
+
+  @Test
+  public void testGetApplicationMetainfo() throws Exception {
+    String zipFileName = TestUtility.createAppPackage(
+        folder,
+        "testpkg",
+        "test.zip",
+        "target/test-classes/org/apache/slider/common/tools/test");
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    // Without accompany metainfo file, read metainfo from the zip file
+    Metainfo metainfo = AgentUtils.getApplicationMetainfo(
+        sliderFileSystem, zipFileName, false);
+    Assert.assertNotNull(metainfo.getApplication());
+    Assert.assertEquals("STORM", metainfo.getApplication().getName());
+
+    // With accompany metainfo file, read metainfo from the accompany file
+    String acompanyFileName = zipFileName + ".metainfo.xml";
+    File f = new File(acompanyFileName);
+    try (BufferedWriter writer = new BufferedWriter(new FileWriter(f))) {
+      writer.write(metainfo_str);
+    }
+    metainfo = AgentUtils.getApplicationMetainfo(
+        sliderFileSystem, zipFileName, false);
+    Assert.assertNotNull(metainfo.getApplication());
+    Assert.assertEquals("MYTESTAPPLICATION", metainfo.getApplication().getName());
+  }
+}

+ 264 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java

@@ -0,0 +1,264 @@
+/*
+ * 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.slider.providers.agent;
+
+import com.google.common.io.Files;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.slider.common.params.ActionCreateArgs;
+import org.apache.slider.common.params.AddonArgsDelegate;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.persist.AppDefinitionPersister;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class TestAppDefinitionPersister {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAppDefinitionPersister.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  /**
+   * @BeforeClass public static void initialize() { BasicConfigurator.resetConfiguration();
+   * BasicConfigurator.configure(); }*
+   */
+
+
+  @Test
+  public void testAppDefinitionPersister() throws Exception {
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    AppDefinitionPersister adp = new AppDefinitionPersister(sliderFileSystem);
+    String clustername = "c1";
+    ActionCreateArgs buildInfo = new ActionCreateArgs();
+    buildInfo.appMetaInfo = null;
+    buildInfo.appDef = null;
+    buildInfo.addonDelegate = new AddonArgsDelegate();
+
+    // nothing to do
+    adp.processSuppliedDefinitions(clustername, buildInfo, null);
+    adp.persistPackages();
+    List<AppDefinitionPersister.AppDefinition> appDefinitions = adp.getAppDefinitions();
+    Assert.assertTrue(appDefinitions.size() == 0);
+
+    ConfTree ct = new ConfTree();
+    ConfTreeOperations appConf = new ConfTreeOperations(ct);
+    final File tempDir = Files.createTempDir();
+    final File metainfo = new File(tempDir, "metainfo.json");
+
+    // unreadable metainfo
+    buildInfo.appMetaInfo = metainfo;
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Path specified with "
+              + "--metainfo either cannot be read or is not a file"));
+    }
+
+    try (PrintWriter writer = new PrintWriter(metainfo.getAbsolutePath(), "UTF-8")) {
+      writer.println("{");
+      writer.println("}");
+    }
+    buildInfo.appDef = metainfo;
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Both --metainfo and --appdef cannot be specified"));
+    }
+
+    // both --metainfojson and --appdef cannot be specified
+    buildInfo.appMetaInfo = null;
+    buildInfo.appMetaInfoJson = "{}";
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Both --metainfojson and --appdef cannot be specified"));
+    }
+
+    buildInfo.appDef = null;
+
+    buildInfo.appMetaInfoJson = "";
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Empty string specified with --metainfojson"));
+    }
+    buildInfo.appMetaInfo = metainfo;
+
+    // both --metainfo and --metainfojson cannot be specified
+    buildInfo.appMetaInfoJson = "{}";
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Both --metainfo and --metainfojson cannot be specified"));
+    }
+    buildInfo.appMetaInfoJson = null;
+
+    appConf.getGlobalOptions().set(AgentKeys.APP_DEF, metainfo.getAbsolutePath());
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "application.def cannot "
+              + "not be set if --metainfo is specified in the cmd line"));
+    }
+
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+
+    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    appDefinitions = adp.getAppDefinitions();
+    Assert.assertTrue(appDefinitions.size() == 1);
+    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
+    log.info(appDefinitions.get(0).toString());
+    Assert.assertTrue(appDefinitions.get(0).appDefPkgOrFolder.toString().endsWith("default"));
+    Assert.assertTrue(appDefinitions.get(0).targetFolderInFs.toString().contains("cluster/c1/appdef"));
+    Assert.assertEquals("appPkg.zip", appDefinitions.get(0).pkgName);
+
+    buildInfo.appDef = tempDir;
+    buildInfo.appMetaInfo = null;
+
+    appConf.getGlobalOptions().set(AgentKeys.APP_DEF, metainfo.getAbsolutePath());
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains("application.def must not be set if --appdef is provided"));
+    }
+
+    adp.getAppDefinitions().clear();
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    appDefinitions = adp.getAppDefinitions();
+    Assert.assertTrue(appDefinitions.size() == 1);
+    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
+    log.info(appDefinitions.get(0).toString());
+    Assert.assertTrue(appDefinitions.get(0).appDefPkgOrFolder.toString().endsWith(tempDir.toString()));
+    Assert.assertTrue(appDefinitions.get(0).targetFolderInFs.toString().contains("cluster/c1/appdef"));
+    Assert.assertEquals("appPkg.zip", appDefinitions.get(0).pkgName);
+
+    adp.getAppDefinitions().clear();
+    buildInfo.appDef = null;
+    buildInfo.appMetaInfo = null;
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+
+    ArrayList<String> list = new ArrayList<String>() {{
+      add("addon1");
+      add("");
+      add("addon2");
+      add(metainfo.getAbsolutePath());
+    }};
+
+    buildInfo.addonDelegate.addonTuples = list;
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains("addon package can only be specified if main app package is specified"));
+    }
+
+    buildInfo.appMetaInfo = metainfo;
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains("Invalid path for addon package addon1"));
+    }
+
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+
+    list = new ArrayList<String>() {{
+      add("addon1");
+      add(tempDir.getAbsolutePath());
+      add("addon2");
+      add(metainfo.getAbsolutePath());
+    }};
+
+    buildInfo.addonDelegate.addonTuples = list;
+    adp.getAppDefinitions().clear();
+
+    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    appDefinitions = adp.getAppDefinitions();
+
+    Assert.assertTrue(appDefinitions.size() == 3);
+    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
+    Assert.assertTrue(appConf.getGlobalOptions().get("application.addon.addon1").contains(
+        "addons/addon1/addon_addon1.zip"));
+    Assert.assertTrue(appConf.getGlobalOptions().get("application.addon.addon2").contains(
+        "addons/addon2/addon_addon2.zip"));
+    log.info(appConf.getGlobalOptions().get("application.addons"));
+    Assert.assertTrue(appConf.getGlobalOptions().get("application.addons").contains(
+        "application.addon.addon2,application.addon.addon1")
+                      || appConf.getGlobalOptions().get("application.addons").contains(
+        "application.addon.addon1,application.addon.addon2"));
+    int seen = 0;
+    for (AppDefinitionPersister.AppDefinition adp_ad : appDefinitions) {
+      if (adp_ad.pkgName.equals("appPkg.zip")) {
+        log.info(adp_ad.toString());
+        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith("default"));
+        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("cluster/c1/appdef"));
+        seen++;
+      }
+      if (adp_ad.pkgName.equals("addon_addon1.zip")) {
+        log.info(adp_ad.toString());
+        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith(tempDir.toString()));
+        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("addons/addon1"));
+        seen++;
+      }
+      if (adp_ad.pkgName.equals("addon_addon2.zip")) {
+        log.info(adp_ad.toString());
+        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith("metainfo.json"));
+        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("addons/addon2"));
+        seen++;
+      }
+    }
+    Assert.assertEquals(3, seen);
+  }
+}

+ 115 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java

@@ -0,0 +1,115 @@
+/**
+ * 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.slider.providers.agent;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class TestComponentTagProvider {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestComponentTagProvider.class);
+
+  @Test
+  public void testTagProvider() throws Exception {
+    ComponentTagProvider ctp = new ComponentTagProvider();
+    Assert.assertEquals("", ctp.getTag(null, null));
+    Assert.assertEquals("", ctp.getTag(null, "cid"));
+    Assert.assertEquals("", ctp.getTag("comp1", null));
+
+    Assert.assertEquals("1", ctp.getTag("comp1", "cid1"));
+    Assert.assertEquals("2", ctp.getTag("comp1", "cid2"));
+    Assert.assertEquals("3", ctp.getTag("comp1", "cid3"));
+    ctp.releaseTag("comp1", "cid2");
+    Assert.assertEquals("2", ctp.getTag("comp1", "cid22"));
+
+    ctp.releaseTag("comp1", "cid4");
+    ctp.recordAssignedTag("comp1", "cid5", "5");
+    Assert.assertEquals("4", ctp.getTag("comp1", "cid4"));
+    Assert.assertEquals("4", ctp.getTag("comp1", "cid4"));
+    Assert.assertEquals("6", ctp.getTag("comp1", "cid6"));
+
+    ctp.recordAssignedTag("comp1", "cid55", "5");
+    Assert.assertEquals("5", ctp.getTag("comp1", "cid55"));
+
+    ctp.recordAssignedTag("comp2", "cidb3", "3");
+    Assert.assertEquals("1", ctp.getTag("comp2", "cidb1"));
+    Assert.assertEquals("2", ctp.getTag("comp2", "cidb2"));
+    Assert.assertEquals("4", ctp.getTag("comp2", "cidb4"));
+
+    ctp.recordAssignedTag("comp2", "cidb5", "six");
+    ctp.recordAssignedTag("comp2", "cidb5", "-55");
+    ctp.recordAssignedTag("comp2", "cidb5", "tags");
+    ctp.recordAssignedTag("comp2", "cidb5", null);
+    ctp.recordAssignedTag("comp2", "cidb5", "");
+    ctp.recordAssignedTag("comp2", "cidb5", "5");
+    Assert.assertEquals("6", ctp.getTag("comp2", "cidb6"));
+
+    ctp.recordAssignedTag("comp2", null, "5");
+    ctp.recordAssignedTag(null, null, "5");
+    ctp.releaseTag("comp1", null);
+    ctp.releaseTag(null, "cid4");
+    ctp.releaseTag(null, null);
+  }
+
+  @Test
+  public void testTagProviderWithThread() throws Exception {
+    ComponentTagProvider ctp = new ComponentTagProvider();
+    Thread thread = new Thread(new Taggged(ctp));
+    Thread thread2 = new Thread(new Taggged(ctp));
+    Thread thread3 = new Thread(new Taggged(ctp));
+    thread.start();
+    thread2.start();
+    thread3.start();
+    ctp.getTag("comp1", "cid50");
+    thread.join();
+    thread2.join();
+    thread3.join();
+    Assert.assertEquals("101", ctp.getTag("comp1", "cid101"));
+  }
+
+  public class Taggged implements Runnable {
+    private final ComponentTagProvider ctp;
+
+    public Taggged(ComponentTagProvider ctp) {
+      this.ctp = ctp;
+    }
+
+    public void run() {
+      for (int i = 0; i < 100; i++) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.getTag("comp1", containerId);
+      }
+      for (int i = 0; i < 100; i++) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.getTag("comp1", containerId);
+      }
+      for (int i = 0; i < 100; i += 2) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.releaseTag("comp1", containerId);
+      }
+      for (int i = 0; i < 100; i += 2) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.getTag("comp1", containerId);
+      }
+    }
+  }
+}

+ 33 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java

@@ -0,0 +1,33 @@
+/**
+ * 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.slider.providers.agent;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestState {
+  protected static final Logger log = LoggerFactory.getLogger(TestState.class);
+
+  @Test
+  public void testState() throws Exception {
+    State state = State.STARTED;
+    Assert.assertEquals(Command.STOP, state.getSupportedCommand(false, true));
+  }
+}

+ 107 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java

@@ -0,0 +1,107 @@
+/*
+ * 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.slider.providers.agent.application.metadata;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ *
+ */
+public class TestConfigParser {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestConfigParser.class);
+  private static final String config_1_str = "<configuration>\n"
+                                             + "  <property>\n"
+                                             + "    <name>security.client.protocol.acl</name>\n"
+                                             + "    <value>*</value>\n"
+                                             + "    <description>ACL for HRegionInterface protocol implementations (ie. \n"
+                                             + "    clients talking to HRegionServers)\n"
+                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
+                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
+                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
+                                             + "  </property>\n"
+                                             + "\n"
+                                             + "  <property>\n"
+                                             + "    <name>security.admin.protocol.acl</name>\n"
+                                             + "    <value>*</value>\n"
+                                             + "    <description>ACL for HMasterInterface protocol implementation (ie. \n"
+                                             + "    clients talking to HMaster for admin operations).\n"
+                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
+                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
+                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
+                                             + "  </property>\n"
+                                             + "\n"
+                                             + "  <property>\n"
+                                             + "    <name>security.masterregion.protocol.acl</name>\n"
+                                             + "    <value>*</value>\n"
+                                             + "    <description>ACL for HMasterRegionInterface protocol implementations\n"
+                                             + "    (for HRegionServers communicating with HMaster)\n"
+                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
+                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
+                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
+                                             + "  </property>\n"
+                                             + "  <property>\n"
+                                             + "    <name>emptyVal</name>\n"
+                                             + "    <value></value>\n"
+                                             + "    <description>non-empty-desc</description>\n"
+                                             + "  </property>\n"
+                                             + "  <property>\n"
+                                             + "    <name>emptyDesc</name>\n"
+                                             + "    <value></value>\n"
+                                             + "    <description></description>\n"
+                                             + "  </property>\n"
+                                             + "  <property>\n"
+                                             + "    <name>noDesc</name>\n"
+                                             + "    <value></value>\n"
+                                             + "  </property>\n"
+                                             + "</configuration>";
+
+  @Test
+  public void testParse() throws IOException {
+
+    InputStream config_1 = new ByteArrayInputStream(config_1_str.getBytes());
+    DefaultConfig config = new DefaultConfigParser().parse(config_1);
+    Assert.assertNotNull(config);
+    Assert.assertNotNull(config.getPropertyInfos());
+    Assert.assertEquals(6, config.getPropertyInfos().size());
+    for (PropertyInfo pInfo : config.getPropertyInfos()) {
+      if (pInfo.getName().equals("security.client.protocol.acl")) {
+        Assert.assertEquals("*", pInfo.getValue());
+        Assert.assertTrue(pInfo.getDescription().startsWith("ACL for HRegionInterface "));
+      }
+      if (pInfo.getName().equals("emptyVal")) {
+        Assert.assertEquals("", pInfo.getValue());
+        Assert.assertEquals("non-empty-desc", pInfo.getDescription());
+      }
+      if (pInfo.getName().equals("emptyDesc")) {
+        Assert.assertEquals("", pInfo.getValue());
+        Assert.assertEquals("", pInfo.getDescription());
+      }
+      if (pInfo.getName().equals("noDesc")) {
+        Assert.assertEquals("", pInfo.getValue());
+        Assert.assertNull(pInfo.getDescription());
+      }
+    }
+  }
+}

+ 177 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java

@@ -0,0 +1,177 @@
+/*
+ * 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.slider.providers.agent.application.metadata;
+
+import org.apache.slider.providers.agent.AgentProviderService;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import static org.mockito.Mockito.doReturn;
+
+/**
+ *
+ */
+public class TestMetainfoParser {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestMetainfoParser.class);
+  public static final String METAINFO_XML =
+      "/org/apache/slider/providers/agent/application/metadata/metainfo.xml";
+
+  @Test
+  public void testParse() throws IOException {
+
+    InputStream resStream = this.getClass().getResourceAsStream(
+        METAINFO_XML);
+    MetainfoParser parser = new MetainfoParser();
+    Metainfo metainfo = parser.fromXmlStream(resStream);
+    Assert.assertNotNull(metainfo);
+    Assert.assertNotNull(metainfo.getApplication());
+    Application application = metainfo.getApplication();
+    assert "STORM".equals(application.getName());
+    assert 6 == application.getComponents().size();
+    OSPackage pkg = application.getOSSpecifics().get(0).getPackages().get(0);
+    assert "tarball".equals(pkg.getType());
+    assert "files/apache-storm-0.9.1.2.1.1.0-237.tar.gz".equals(pkg.getName());
+    boolean found = false;
+    for (Component comp : application.getComponents()) {
+      if (comp != null && comp.getName().equals("NIMBUS")) {
+        found = true;
+        Assert.assertEquals(0, comp.getComponentExports().size());
+      }
+      if (comp != null && comp.getName().equals("SUPERVISOR")) {
+        Assert.assertEquals(1, comp.getComponentExports().size());
+      }
+      if (comp != null && comp.getName().equals("ANOTHER_COMPONENT")) {
+        assert 2 == comp.getCommands().size();
+        assert "start command".equals(comp.getCommands().get(0).getExec());
+        assert "START".equals(comp.getCommands().get(0).getName());
+        assert "stop command".equals(comp.getCommands().get(1).getExec());
+        assert "STOP".equals(comp.getCommands().get(1).getName());
+      }
+    }
+    assert found;
+    Assert.assertEquals(0, application.getConfigFiles().size());
+    assert 1 == application.getPackages().size();
+    Package p = application.getPackages().get(0);
+    assert "tarball".equals(p.getType());
+    assert "test-tarball-name.tgz".equals(p.getName());
+  }
+
+  @Test
+  public void testJsonParse() throws IOException {
+    String metaInfo1_json = "{\n"
+                            + "\"schemaVersion\":\"2.2\",\n"
+                            + "\"application\":{\n"
+                            +     "\"name\": \"MEMCACHED\","
+                            +     "\"exportGroups\": ["
+                            +        "{"
+                            +          "\"name\": \"Servers\","
+                            +          "\"exports\": ["
+                            +            "{"
+                            +               "\"name\": \"host_port\","
+                            +               "\"value\": \"${MEMCACHED_HOST}:${site.global.port}\""
+                            +            "}"
+                            +          "]"
+                            +        "}"
+                            +      "],"
+                            +     "\"components\": ["
+                            +        "{"
+                            +          "\"name\": \"MEMCACHED\","
+                            +          "\"compExports\": \"Servers-host_port\","
+                            +          "\"commands\": ["
+                            +            "{"
+                            +               "\"exec\": \"java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main\""
+                            +            "}"
+                            +          "]"
+                            +        "},"
+                            +        "{"
+                            +          "\"name\": \"MEMCACHED2\","
+                            +          "\"commands\": ["
+                            +            "{"
+                            +               "\"exec\": \"scripts/config.py\","
+                            +               "\"type\": \"PYTHON\","
+                            +               "\"name\": \"CONFIGURE\""
+                            +            "}"
+                            +          "],"
+                            +          "\"dockerContainers\": ["
+                            +            "{"
+                            +               "\"name\": \"redis\","
+                            +               "\"image\": \"dockerhub/redis\","
+                            +               "\"options\": \"-net=bridge\","
+                            +               "\"mounts\": ["
+                            +                 "{"
+                            +                   "\"containerMount\": \"/tmp/conf\","
+                            +                   "\"hostMount\": \"{$conf:@//site/global/app_root}/conf\""
+                            +                 "}"
+                            +               "]"
+                            +            "}"
+                            +          "]"
+                            +        "}"
+                            +      "]"
+                            +   "}"
+                            + "}";
+
+    MetainfoParser parser = new MetainfoParser();
+    Metainfo mInfo = parser.fromJsonString(metaInfo1_json);
+    Assert.assertEquals("2.2", mInfo.getSchemaVersion());
+
+    Application app = mInfo.getApplication();
+    Assert.assertNotNull(app);
+
+    Assert.assertEquals("MEMCACHED", app.getName());
+    List<ExportGroup> egs = app.getExportGroups();
+    Assert.assertEquals(1, egs.size());
+    ExportGroup eg = egs.get(0);
+    Assert.assertEquals("Servers", eg.getName());
+    List<Export> exports = eg.getExports();
+    Assert.assertEquals(1, exports.size());
+    Export export = exports.get(0);
+    Assert.assertEquals("host_port", export.getName());
+    Assert.assertEquals("${MEMCACHED_HOST}:${site.global.port}", export.getValue());
+
+    List<Component> components = app.getComponents();
+    Assert.assertEquals(2, components.size());
+
+    Component c1 = mInfo.getApplicationComponent("MEMCACHED");
+    Assert.assertNotNull(c1);
+    Assert.assertEquals("MEMCACHED", c1.getName());
+    Assert.assertEquals("Servers-host_port", c1.getCompExports());
+    Assert.assertEquals(1, c1.getCommands().size());
+    ComponentCommand cmd = c1.getCommands().get(0);
+    Assert.assertEquals("START", cmd.getName());
+    Assert.assertEquals("SHELL", cmd.getType());
+    Assert.assertEquals("java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main",
+                        cmd.getExec());
+
+    Component c2 = mInfo.getApplicationComponent("MEMCACHED2");
+    Assert.assertNotNull(c2);
+    Assert.assertEquals("MEMCACHED2", c2.getName());
+    Assert.assertEquals(1, c2.getCommands().size());
+    cmd = c2.getCommands().get(0);
+    Assert.assertEquals("CONFIGURE", cmd.getName());
+    Assert.assertEquals("PYTHON", cmd.getType());
+    Assert.assertEquals("scripts/config.py", cmd.getExec());
+  }
+}

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/TestServiceRecordAttributes.java

@@ -0,0 +1,68 @@
+/*
+ * 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.slider.server.appmaster;
+
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ *
+ */
+public class TestServiceRecordAttributes extends Assert {
+
+  @Test
+  public void testAppConfigProvidedServiceRecordAttributes() throws Exception {
+    Map<String, String> options = new HashMap<>();
+    options.put("slider.some.arbitrary.option", "arbitrary value");
+    options.put("service.record.attribute.one_attribute", "one_attribute_value");
+    options.put("service.record.attribute.second_attribute", "second_attribute_value");
+    MapOperations serviceProps = new MapOperations(SliderKeys.COMPONENT_AM, options);
+    options = new HashMap<>();
+    options.put("some.component.attribute", "component_attribute_value");
+    options.put("service.record.attribute.component_attribute", "component_attribute_value");
+    MapOperations compProps = new MapOperations("TEST_COMP", options);
+
+    SliderAppMaster appMaster = new SliderAppMaster();
+
+    ServiceRecord appServiceRecord = new ServiceRecord();
+
+    appMaster.setProvidedServiceRecordAttributes(serviceProps, appServiceRecord);
+
+    assertNull("property should not be attribute",
+               appServiceRecord.get("slider.some.arbitrary.option"));
+    assertEquals("wrong value", "one_attribute_value",
+                 appServiceRecord.get("one_attribute"));
+    assertEquals("wrong value", "second_attribute_value",
+                 appServiceRecord.get("second_attribute"));
+
+    ServiceRecord compServiceRecord = new ServiceRecord();
+
+    appMaster.setProvidedServiceRecordAttributes(compProps, compServiceRecord);
+
+    assertNull("should not be attribute",
+               compServiceRecord.get("some.component.attribute"));
+    assertEquals("wrong value", "component_attribute_value",
+                 compServiceRecord.get("component_attribute"));
+
+  }
+}

+ 60 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java

@@ -0,0 +1,60 @@
+/*
+ * 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.slider.server.appmaster.web.rest.publisher;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.providers.agent.AgentProviderService;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ */
+public class TestAgentProviderService extends AgentProviderService {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentProviderService.class);
+
+  public TestAgentProviderService() {
+    super();
+    log.info("TestAgentProviderService created");
+  }
+
+  @Override
+  public void bind(StateAccessForProviders stateAccessor,
+      QueueAccess queueAccess,
+      List<Container> liveContainers) {
+    super.bind(stateAccessor, queueAccess, liveContainers);
+    Map<String,String> dummyProps = new HashMap<String, String>();
+    dummyProps.put("prop1", "val1");
+    dummyProps.put("prop2", "val2");
+    log.info("publishing dummy-site.xml with values {}", dummyProps);
+    publishApplicationInstanceData("dummy-site", "dummy configuration",
+                                   dummyProps.entrySet());
+    // publishing global config for testing purposes
+    publishApplicationInstanceData("global", "global configuration",
+                                   stateAccessor.getAppConfSnapshot()
+                                       .getGlobalOptions().entrySet());
+  }
+
+}

+ 40 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java

@@ -0,0 +1,40 @@
+/*
+ * 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.slider.server.appmaster.web.rest.publisher;
+
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.providers.agent.AgentProviderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class TestSliderProviderFactory extends AgentProviderFactory{
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestSliderProviderFactory.class);
+
+  public TestSliderProviderFactory() {
+    log.info("Created TestSliderProviderFactory");
+  }
+
+  @Override
+  public ProviderService createServerProvider() {
+    log.info("Creating TestAgentProviderService");
+    return new TestAgentProviderService();
+  }
+}

+ 37 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/servicemonitor/TestPortProbe.java

@@ -0,0 +1,37 @@
+/*
+ * 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.slider.server.servicemonitor;
+
+import org.junit.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class TestPortProbe extends Assert {
+  /**
+   * Assert that a port probe failed if the port is closed
+   * @throws Throwable
+   */
+  @Test
+  public void testPortProbeFailsClosedPort() throws Throwable {
+    PortProbe probe = new PortProbe("127.0.0.1", 65500, 100, "", new Configuration());
+    probe.init();
+    ProbeStatus status = probe.ping(true);
+    assertFalse("Expected a failure but got successful result: " + status,
+      status.isSuccess());
+  }
+}

+ 540 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java

@@ -0,0 +1,540 @@
+/*
+ * 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.slider.server.services.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.slider.Slider;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.Principal;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ *
+ */
+public class TestCertificateManager {
+  @Rule
+  public TemporaryFolder workDir = new TemporaryFolder();
+  private File secDir;
+  private CertificateManager certMan;
+
+  @Before
+  public void setup() throws Exception {
+    certMan = new CertificateManager();
+    MapOperations compOperations = new MapOperations();
+    secDir = new File(workDir.getRoot(), SliderKeys.SECURITY_DIR);
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    compOperations.put(SliderXmlConfKeys.KEY_KEYSTORE_LOCATION,
+                       keystoreFile.getAbsolutePath());
+    certMan.initialize(compOperations, "cahost", null, null);
+  }
+
+  @Test
+  public void testServerCertificateGenerated() throws Exception {
+    File serverCrt = new File(secDir, SliderKeys.CRT_FILE_NAME);
+    Assert.assertTrue("Server CRD does not exist:" + serverCrt,
+                      serverCrt.exists());
+  }
+
+  @Test
+  public void testAMKeystoreGenerated() throws Exception {
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    Assert.assertTrue("Keystore does not exist: " + keystoreFile,
+                      keystoreFile.exists());
+    InputStream is = null;
+    try {
+
+      is = new FileInputStream(keystoreFile);
+      KeyStore keystore = KeyStore.getInstance("pkcs12");
+      String password = SecurityUtils.getKeystorePass();
+      keystore.load(is, password.toCharArray());
+
+      Certificate certificate = keystore.getCertificate(
+          keystore.aliases().nextElement());
+      Assert.assertNotNull(certificate);
+
+      if (certificate instanceof X509Certificate) {
+        X509Certificate x509cert = (X509Certificate) certificate;
+
+        // Get subject
+        Principal principal = x509cert.getSubjectDN();
+        String subjectDn = principal.getName();
+        Assert.assertEquals("wrong DN",
+                            "CN=cahost",
+                            subjectDn);
+
+        // Get issuer
+        principal = x509cert.getIssuerDN();
+        String issuerDn = principal.getName();
+        Assert.assertEquals("wrong Issuer DN",
+                            "CN=cahost",
+                            issuerDn);
+      }
+    } finally {
+      if(null != is) {
+        is.close();
+      }
+    }
+  }
+
+  @Test
+  public void testContainerCertificateGeneration() throws Exception {
+    certMan.generateContainerCertificate("testhost", "container1");
+    Assert.assertTrue("container certificate not generated",
+                      new File(secDir, "container1.crt").exists());
+  }
+
+  @Test
+  public void testContainerKeystoreGeneration() throws Exception {
+    SecurityStore keystoreFile = certMan.generateContainerKeystore("testhost",
+                                                                   "container1",
+                                                                   "component1",
+                                                                   "password");
+    validateKeystore(keystoreFile.getFile(), "testhost", "cahost");
+  }
+
+  private void validateKeystore(File keystoreFile, String certHostname,
+                                String issuerHostname)
+      throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+
+    InputStream is = null;
+    try {
+
+      is = new FileInputStream(keystoreFile);
+      KeyStore keystore = KeyStore.getInstance("pkcs12");
+      String password = "password";
+      keystore.load(is, password.toCharArray());
+
+      Certificate certificate = keystore.getCertificate(
+          keystore.aliases().nextElement());
+      Assert.assertNotNull(certificate);
+
+      if (certificate instanceof X509Certificate) {
+        X509Certificate x509cert = (X509Certificate) certificate;
+
+        // Get subject
+        Principal principal = x509cert.getSubjectDN();
+        String subjectDn = principal.getName();
+        Assert.assertEquals("wrong DN", "CN=" + certHostname + ", OU=container1",
+                            subjectDn);
+
+        // Get issuer
+        principal = x509cert.getIssuerDN();
+        String issuerDn = principal.getName();
+        Assert.assertEquals("wrong Issuer DN",
+                            "CN=" + issuerHostname,
+                            issuerDn);
+      }
+    } finally {
+      if(null != is) {
+        is.close();
+      }
+    }
+  }
+
+  @Test
+  public void testContainerKeystoreGenerationViaStoresGenerator() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    instanceDefinition.getAppConf().components.put("component1", compOps);
+    compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_PROPERTY_KEY,
+                "app1.component1.password.property");
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+    instanceDefinition.getAppConf().global.put(
+        "app1.component1.password.property", "password");
+    instanceDefinition.resolve();
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    validateKeystore(files[0].getFile(), "testhost", "cahost");
+  }
+
+  @Test
+  public void testContainerKeystoreGenerationViaStoresGeneratorUsingGlobalProps() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    instanceDefinition.getAppConf().components.put("component1", compOps);
+    compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_PROPERTY_KEY,
+                "app1.component1.password.property");
+    instanceDefinition.getAppConf().global.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+    compOps.put(
+        "app1.component1.password.property", "password");
+    instanceDefinition.resolve();
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    validateKeystore(files[0].getFile(), "testhost", "cahost");
+  }
+
+  @Test
+  public void testContainerKeystoreGenerationViaStoresGeneratorOverrideGlobalSetting() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = setupComponentOptions(true, null,
+                                                  "app1.component1.password.property",
+                                                  null, null);
+    instanceDefinition.getAppConf().components.put("component1", compOps);
+    instanceDefinition.getAppConf().global.put(
+        "app1.component1.password.property", "password");
+    instanceDefinition.getAppConf().global.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "false");
+    instanceDefinition.resolve();
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    validateKeystore(files[0].getFile(), "testhost", "cahost");
+  }
+
+  @Test
+  public void testContainerTrusttoreGeneration() throws Exception {
+    SecurityStore keystoreFile =
+        certMan.generateContainerKeystore("testhost",
+                                          "container1",
+                                          "component1",
+                                          "keypass");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.getFile().exists());
+    SecurityStore truststoreFile =
+        certMan.generateContainerTruststore("container1",
+                                            "component1", "trustpass"
+        );
+    Assert.assertTrue("container truststore not generated",
+                      truststoreFile.getFile().exists());
+
+    validateTruststore(keystoreFile.getFile(), truststoreFile.getFile());
+  }
+
+  @Test
+  public void testContainerGenerationUsingStoresGeneratorNoTruststore() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+    compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_KEY,
+                "test.keystore.password");
+
+    setupCredentials(instanceDefinition, "test.keystore.password", null);
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+
+    Assert.assertTrue("keystore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(keystoreFile,
+                                                    SecurityStore.StoreType.keystore)));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertFalse("container truststore generated",
+                      truststoreFile.exists());
+    Assert.assertFalse("truststore in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                    SecurityStore.StoreType.truststore)));
+
+  }
+
+  @Test
+  public void testContainerGenerationUsingStoresGeneratorJustTruststoreWithDefaultAlias() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = setupComponentOptions(true);
+
+    setupCredentials(instanceDefinition, null,
+                     SliderKeys.COMP_TRUSTSTORE_PASSWORD_ALIAS_DEFAULT);
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertFalse("container keystore generated",
+                       keystoreFile.exists());
+    Assert.assertFalse("keystore in returned list",
+                       Arrays.asList(files).contains(keystoreFile));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertTrue("container truststore not generated",
+                      truststoreFile.exists());
+    Assert.assertTrue("truststore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                                      SecurityStore.StoreType.truststore)));
+
+  }
+
+  @Test
+  public void testContainerTrusttoreGenerationUsingStoresGenerator() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = setupComponentOptions(true,
+                                                  "test.keystore.password",
+                                                  null,
+                                                  "test.truststore.password",
+                                                  null);
+
+    setupCredentials(instanceDefinition, "test.keystore.password",
+                     "test.truststore.password");
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 2, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+    Assert.assertTrue("keystore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(keystoreFile,
+                                                                      SecurityStore.StoreType.keystore)));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertTrue("container truststore not generated",
+                      truststoreFile.exists());
+    Assert.assertTrue("truststore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                                      SecurityStore.StoreType.truststore)));
+
+    validateTruststore(keystoreFile, truststoreFile);
+  }
+
+  private void setupCredentials(AggregateConf instanceDefinition,
+                                String keyAlias, String trustAlias)
+      throws Exception {
+    Configuration conf = new Configuration();
+    final Path jksPath = new Path(SecurityUtils.getSecurityDir(), "test.jks");
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+    File file = new File(SecurityUtils.getSecurityDir(), "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    instanceDefinition.getAppConf().credentials.put(ourUrl, new ArrayList<String>());
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+
+    // create new aliases
+    try {
+
+      if (keyAlias != null) {
+        char[] storepass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            keyAlias, storepass);
+      }
+
+      if (trustAlias != null) {
+        char[] trustpass = {'t', 'r', 'u', 's', 't', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            trustAlias, trustpass);
+      }
+
+      // write out so that it can be found in checks
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  private MapOperations setupComponentOptions(boolean storesRequired) {
+    return this.setupComponentOptions(storesRequired, null, null, null, null);
+  }
+
+  private MapOperations setupComponentOptions(boolean storesRequired,
+                                              String keyAlias,
+                                              String keyPwd,
+                                              String trustAlias,
+                                              String trustPwd) {
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY,
+                Boolean.toString(storesRequired));
+    if (keyAlias != null) {
+      compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_KEY,
+                  "test.keystore.password");
+    }
+    if (trustAlias != null) {
+      compOps.put(SliderKeys.COMP_TRUSTSTORE_PASSWORD_ALIAS_KEY,
+                  "test.truststore.password");
+    }
+    if (keyPwd != null) {
+      compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_PROPERTY_KEY,
+                  keyPwd);
+    }
+    if (trustPwd != null) {
+      compOps.put(SliderKeys.COMP_TRUSTSTORE_PASSWORD_PROPERTY_KEY,
+                  trustPwd);
+    }
+    return compOps;
+  }
+
+  @Test
+  public void testContainerStoresGenerationKeystoreOnly() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+
+    setupCredentials(instanceDefinition,
+                     SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_DEFAULT, null);
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+    Assert.assertTrue("keystore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(keystoreFile,
+                                                                      SecurityStore.StoreType.keystore)));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertFalse("container truststore generated",
+                       truststoreFile.exists());
+    Assert.assertFalse("truststore in returned list",
+                       Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                                       SecurityStore.StoreType.truststore)));
+
+  }
+
+  @Test
+  public void testContainerStoresGenerationMisconfiguration() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+
+    setupCredentials(instanceDefinition, "cant.be.found", null);
+
+    try {
+      StoresGenerator.generateSecurityStores("testhost", "container1",
+                                                            "component1", instanceDefinition,
+                                                            compOps);
+      Assert.fail("SliderException should have been generated");
+    } catch (SliderException e) {
+      // ignore - should be thrown
+    }
+  }
+
+  private void validateTruststore(File keystoreFile, File truststoreFile)
+      throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+    InputStream keyis = null;
+    InputStream trustis = null;
+    try {
+
+      // create keystore
+      keyis = new FileInputStream(keystoreFile);
+      KeyStore keystore = KeyStore.getInstance("pkcs12");
+      String password = "keypass";
+      keystore.load(keyis, password.toCharArray());
+
+      // obtain server cert
+      Certificate certificate = keystore.getCertificate(
+          keystore.aliases().nextElement());
+      Assert.assertNotNull(certificate);
+
+      // create trust store from generated trust store file
+      trustis = new FileInputStream(truststoreFile);
+      KeyStore truststore = KeyStore.getInstance("pkcs12");
+      password = "trustpass";
+      truststore.load(trustis, password.toCharArray());
+
+      // validate keystore cert using trust store
+      TrustManagerFactory
+          trustManagerFactory =
+          TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+      trustManagerFactory.init(truststore);
+
+      for (TrustManager trustManager: trustManagerFactory.getTrustManagers()) {
+        if (trustManager instanceof X509TrustManager) {
+          X509TrustManager x509TrustManager = (X509TrustManager)trustManager;
+          x509TrustManager.checkServerTrusted(
+              new X509Certificate[] {(X509Certificate) certificate},
+              "RSA_EXPORT");
+        }
+      }
+
+    } finally {
+      if(null != keyis) {
+        keyis.close();
+      }
+      if(null != trustis) {
+        trustis.close();
+      }
+    }
+  }
+
+}

+ 156 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java

@@ -0,0 +1,156 @@
+/*
+ * 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.slider.server.services.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ *
+ */
+public class TestMultiThreadedStoreGeneration {
+
+  public static final int NUM_THREADS = 30;
+  @Rule
+  public TemporaryFolder workDir = new TemporaryFolder();;
+
+  private void setupCredentials(AggregateConf instanceDefinition,
+                                String keyAlias, String trustAlias)
+      throws Exception {
+    Configuration conf = new Configuration();
+    final Path jksPath = new Path(SecurityUtils.getSecurityDir(), "test.jks");
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+    File file = new File(SecurityUtils.getSecurityDir(), "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    instanceDefinition.getAppConf().credentials.put(ourUrl, new ArrayList<String>());
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+
+    // create new aliases
+    try {
+
+      if (keyAlias != null) {
+        char[] storepass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            keyAlias, storepass);
+      }
+
+      if (trustAlias != null) {
+        char[] trustpass = {'t', 'r', 'u', 's', 't', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            trustAlias, trustpass);
+      }
+
+      // write out so that it can be found in checks
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+
+  @Test
+  public void testMultiThreadedStoreGeneration() throws Exception {
+
+    CertificateManager certMan = new CertificateManager();
+    MapOperations compOperations = new MapOperations();
+    File secDir = new File(workDir.getRoot(), SliderKeys.SECURITY_DIR);
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    compOperations.put(SliderXmlConfKeys.KEY_KEYSTORE_LOCATION,
+                       keystoreFile.getAbsolutePath());
+    certMan.initialize(compOperations, "cahost", null, null);
+
+    final CountDownLatch latch = new CountDownLatch(1);
+    final List<SecurityStore> stores = new ArrayList<>();
+    List<Thread> threads = new ArrayList<>();
+    final AggregateConf instanceDefinition = new AggregateConf();
+
+    setupCredentials(instanceDefinition,
+                     SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_DEFAULT, null);
+    final MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+
+    for (int i=0; i<NUM_THREADS; ++i) {
+      final int finalI = i;
+      Runnable runner = new Runnable() {
+        public void run() {
+          System.out.println ("----> In run");
+          try {
+            latch.await();
+            SecurityStore[] stores1 = StoresGenerator.generateSecurityStores(
+                "testhost",
+                "container" + finalI,
+                "component" + finalI,
+                instanceDefinition,
+                compOps);
+            System.out.println ("----> stores1" + stores1);
+            List<SecurityStore>
+                securityStores =
+                Arrays.asList(stores1);
+            stores.addAll(securityStores);
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          } catch (SliderException e) {
+            e.printStackTrace();
+          } catch (IOException e) {
+            e.printStackTrace();
+          } catch (Exception e) {
+            e.printStackTrace();
+          }
+        }
+      };
+      Thread thread = new Thread(runner, "TestThread" + i);
+      threads.add(thread);
+      thread.start();
+    }
+    latch.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+
+    for (int i=0; i < NUM_THREADS; i++) {
+      assertTrue("keystore " + i + " not generated", stores.get(i).getFile().exists());
+    }
+  }
+
+}

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java

@@ -0,0 +1,80 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.ServiceStateException;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+public class MockService extends AbstractService {
+  private final boolean fail;
+  private final int lifespan;
+  private final ExecutorService executorService =
+      Executors.newSingleThreadExecutor();
+
+  MockService() {
+    this("mock", false, -1);
+  }
+
+  MockService(String name, boolean fail, int lifespan) {
+    super(name);
+    this.fail = fail;
+    this.lifespan = lifespan;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    //act on the lifespan here
+    if (lifespan > 0) {
+      executorService.submit(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            Thread.sleep(lifespan);
+          } catch (InterruptedException ignored) {
+
+          }
+          finish();
+        }
+      });
+    } else {
+      if (lifespan == 0) {
+        finish();
+      } else {
+        //continue until told not to
+      }
+    }
+  }
+
+  void finish() {
+    if (fail) {
+      ServiceStateException e =
+          new ServiceStateException(getName() + " failed");
+
+      noteFailure(e);
+      stop();
+      throw e;
+    } else {
+      stop();
+    }
+  }
+
+}

+ 70 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java

@@ -0,0 +1,70 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.apache.hadoop.service.Service;
+
+/**
+ * Extends {@link WorkflowServiceTestBase} with parent-specific operations
+ * and logic to build up and run the parent service
+ */
+public abstract class ParentWorkflowTestBase extends WorkflowServiceTestBase {
+
+  /**
+   * Wait a second for the service parent to stop
+   * @param parent the service to wait for
+   */
+  protected void waitForParentToStop(ServiceParent parent) {
+    waitForParentToStop(parent, 1000);
+  }
+
+  /**
+   * Wait for the service parent to stop
+   * @param parent the service to wait for
+   * @param timeout time in milliseconds
+   */
+  protected void waitForParentToStop(ServiceParent parent, int timeout) {
+    boolean stop = parent.waitForServiceToStop(timeout);
+    if (!stop) {
+      logState(parent);
+      fail("Service failed to stop : after " + timeout + " millis " + parent);
+    }
+  }
+
+  /**
+   * Subclasses are require to implement this and return an instance of a
+   * ServiceParent
+   * @param services a possibly empty list of services
+   * @return an inited -but -not-started- service parent instance
+   */
+  protected abstract ServiceParent buildService(Service... services);
+
+  /**
+   * Use {@link #buildService(Service...)} to create service and then start it
+   * @param services
+   * @return
+   */
+  protected ServiceParent startService(Service... services) {
+    ServiceParent parent = buildService(services);
+    //expect service to start and stay started
+    parent.start();
+    return parent;
+  }
+
+}

+ 96 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java

@@ -0,0 +1,96 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.apache.hadoop.util.Shell;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A source of commands, with the goal being to allow for adding different
+ * implementations for different platforms
+ */
+public class ProcessCommandFactory {
+
+  protected ProcessCommandFactory() {
+  }
+
+  /**
+   * The command to list a directory
+   * @param dir directory
+   * @return commands
+   */
+  public List<String> ls(File dir) {
+    List<String> commands;
+    if (!Shell.WINDOWS) {
+      commands = Arrays.asList("ls","-1", dir.getAbsolutePath());
+    } else {
+      commands = Arrays.asList("cmd", "/c", "dir", dir.getAbsolutePath());
+    }
+    return commands;
+  }
+
+  /**
+   * Echo some text to stdout
+   * @param text text
+   * @return commands
+   */
+  public List<String> echo(String text) {
+    List<String> commands = new ArrayList<String>(5);
+    commands.add("echo");
+    commands.add(text);
+    return commands;
+  }
+
+  /**
+   * print env variables
+   * @return commands
+   */
+  public List<String> env() {
+    List<String> commands;
+    if (!Shell.WINDOWS) {
+      commands = Arrays.asList("env");
+    } else {
+      commands = Arrays.asList("cmd", "/c", "set");
+    }
+    return commands;
+  }
+
+  /**
+   * execute a command that returns with an error code that will
+   * be converted into a number
+   * @return commands
+   */
+  public List<String> exitFalse() {
+    List<String> commands = new ArrayList<String>(2);
+    commands.add("false");
+    return commands;
+  }
+
+  /**
+   * Create a process command factory for this OS
+   * @return
+   */
+  public static ProcessCommandFactory createProcessCommandFactory() {
+    return new ProcessCommandFactory();
+  }
+}

+ 46 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java

@@ -0,0 +1,46 @@
+/*
+ * 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.slider.server.services.workflow;
+
+/**
+ * Test runnable that can be made to exit, or throw an exception
+ * during its run
+ */
+class SimpleRunnable implements Runnable {
+  boolean throwException = false;
+
+
+  SimpleRunnable() {
+  }
+
+  SimpleRunnable(boolean throwException) {
+    this.throwException = throwException;
+  }
+
+  @Override
+  public synchronized void run() {
+    try {
+      if (throwException) {
+        throw new RuntimeException("SimpleRunnable");
+      }
+    } finally {
+      this.notify();
+    }
+  }
+}

+ 116 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java

@@ -0,0 +1,116 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public class TestWorkflowClosingService extends WorkflowServiceTestBase {
+
+  @Test
+  public void testSimpleClose() throws Throwable {
+    ClosingService<OpenClose> svc = instance(false);
+    OpenClose openClose = svc.getCloseable();
+    assertFalse(openClose.closed);
+    svc.stop();
+    assertTrue(openClose.closed);
+  }
+
+  @Test
+  public void testNullClose() throws Throwable {
+    ClosingService<OpenClose> svc = new ClosingService<OpenClose>("", null);
+    svc.init(new Configuration());
+    svc.start();
+    assertNull(svc.getCloseable());
+    svc.stop();
+  }
+
+  @Test
+  public void testFailingClose() throws Throwable {
+    ClosingService<OpenClose> svc = instance(false);
+    OpenClose openClose = svc.getCloseable();
+    openClose.raiseExceptionOnClose = true;
+    svc.stop();
+    assertTrue(openClose.closed);
+    Throwable cause = svc.getFailureCause();
+    assertNotNull(cause);
+
+    //retry should be a no-op
+    svc.close();
+  }
+
+  @Test
+  public void testDoubleClose() throws Throwable {
+    ClosingService<OpenClose> svc = instance(false);
+    OpenClose openClose = svc.getCloseable();
+    openClose.raiseExceptionOnClose = true;
+    svc.stop();
+    assertTrue(openClose.closed);
+    Throwable cause = svc.getFailureCause();
+    assertNotNull(cause);
+    openClose.closed = false;
+    svc.stop();
+    assertEquals(cause, svc.getFailureCause());
+  }
+
+  /**
+   * This does not recurse forever, as the service has already entered the
+   * STOPPED state before the inner close tries to stop it -that operation
+   * is a no-op
+   * @throws Throwable
+   */
+  @Test
+  public void testCloseSelf() throws Throwable {
+    ClosingService<ClosingService> svc =
+        new ClosingService<ClosingService>("");
+    svc.setCloseable(svc);
+    svc.stop();
+  }
+
+
+  private ClosingService<OpenClose> instance(boolean raiseExceptionOnClose) {
+    ClosingService<OpenClose> svc = new ClosingService<OpenClose>(new OpenClose(
+        raiseExceptionOnClose));
+    svc.init(new Configuration());
+    svc.start();
+    return svc;
+  }
+
+  private static class OpenClose implements Closeable {
+    public boolean closed = false;
+    public boolean raiseExceptionOnClose;
+
+    private OpenClose(boolean raiseExceptionOnClose) {
+      this.raiseExceptionOnClose = raiseExceptionOnClose;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        closed = true;
+        if (raiseExceptionOnClose) {
+          throw new IOException("OpenClose");
+        }
+      }
+    }
+  }
+}

+ 113 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java

@@ -0,0 +1,113 @@
+/*
+ * 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.slider.server.services.workflow;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestWorkflowCompositeService extends ParentWorkflowTestBase {
+  private static final Logger
+      log = LoggerFactory.getLogger(TestWorkflowCompositeService.class);
+
+  @Test
+  public void testSingleChild() throws Throwable {
+    Service parent = startService(new MockService());
+    parent.stop();
+  }
+
+  @Test
+  public void testSingleChildTerminating() throws Throwable {
+    ServiceParent parent =
+        startService(new MockService("1", false, 100));
+    waitForParentToStop(parent);
+  }
+
+  @Test
+  public void testSingleChildFailing() throws Throwable {
+    ServiceParent parent =
+        startService(new MockService("1", true, 100));
+    waitForParentToStop(parent);
+    assert parent.getFailureCause() != null;
+  }
+
+  @Test
+  public void testTwoChildren() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+  @Test
+  public void testCallableChild() throws Throwable {
+
+    MockService one = new MockService("one", false, 100);
+    CallableHandler handler = new CallableHandler("hello");
+    WorkflowCallbackService<String> ens =
+        new WorkflowCallbackService<String>("handler", handler, 100, true);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, ens, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(ens);
+    assertStopped(two);
+    assertTrue(handler.notified);
+    String s = ens.getScheduledFuture().get();
+    assertEquals("hello", s);
+  }
+
+  @Test
+  public void testNestedComposite() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = buildService(one, two);
+    ServiceParent outer = startService(parent);
+    assertTrue(outer.waitForServiceToStop(1000));
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+  @Test
+  public void testFailingComposite() throws Throwable {
+    MockService one = new MockService("one", true, 10);
+    MockService two = new MockService("two", false, 1000);
+    ServiceParent parent = startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+    assertNotNull(one.getFailureCause());
+    assertNotNull(parent.getFailureCause());
+    assertEquals(one.getFailureCause(), parent.getFailureCause());
+  }
+
+  @Override
+  public ServiceParent buildService(Service... services) {
+    ServiceParent parent =
+        new WorkflowCompositeService("test", services);
+    parent.init(new Configuration());
+    return parent;
+  }
+
+}

+ 66 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java

@@ -0,0 +1,66 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.junit.Test;
+
+import java.util.concurrent.ExecutorService;
+
+
+/**
+ * Basic tests for executor service
+ */
+public class TestWorkflowExecutorService extends WorkflowServiceTestBase {
+
+  @Test
+  public void testAsyncRun() throws Throwable {
+
+    ExecutorSvc svc = run(new ExecutorSvc());
+    ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc,
+        new SimpleRunnable());
+
+    // synchronous in-thread execution
+    svc.execute(runnable);
+    Thread.sleep(1000);
+    assertStopped(svc);
+  }
+
+  @Test
+  public void testFailureRun() throws Throwable {
+
+    ExecutorSvc svc = run(new ExecutorSvc());
+    ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc,
+        new SimpleRunnable(true));
+
+    // synchronous in-thread execution
+    svc.execute(runnable);
+    Thread.sleep(1000);
+    assertStopped(svc);
+    assertNotNull(runnable.getException());
+  }
+
+  private static class ExecutorSvc
+      extends WorkflowExecutorService<ExecutorService> {
+    private ExecutorSvc() {
+      super("ExecutorService",
+          ServiceThreadFactory.singleThreadExecutor("test", true));
+    }
+
+  }
+}

+ 107 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java

@@ -0,0 +1,107 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public class TestWorkflowRpcService extends WorkflowServiceTestBase {
+
+  @Test
+  public void testCreateMockRPCService() throws Throwable {
+    MockRPC rpc = new MockRPC();
+    rpc.start();
+    assertTrue(rpc.started);
+    rpc.getListenerAddress();
+    rpc.stop();
+    assertTrue(rpc.stopped);
+  }
+
+  @Test
+  public void testLifecycle() throws Throwable {
+    MockRPC rpc = new MockRPC();
+    WorkflowRpcService svc = new WorkflowRpcService("test", rpc);
+    run(svc);
+    assertTrue(rpc.started);
+    svc.getConnectAddress();
+    svc.stop();
+    assertTrue(rpc.stopped);
+  }
+
+  @Test
+  public void testStartFailure() throws Throwable {
+    MockRPC rpc = new MockRPC();
+    rpc.failOnStart = true;
+    WorkflowRpcService svc = new WorkflowRpcService("test", rpc);
+    svc.init(new Configuration());
+    try {
+      svc.start();
+      fail("expected an exception");
+    } catch (RuntimeException e) {
+      assertEquals("failOnStart", e.getMessage());
+    }
+    svc.stop();
+    assertTrue(rpc.stopped);
+  }
+
+  private static class MockRPC extends Server {
+
+    public boolean stopped;
+    public boolean started;
+    public boolean failOnStart;
+
+    private MockRPC() throws IOException {
+      super("localhost", 0, null, 1, new Configuration());
+    }
+
+    @Override
+    public synchronized void start() {
+      if (failOnStart) {
+        throw new RuntimeException("failOnStart");
+      }
+      started = true;
+      super.start();
+    }
+
+    @Override
+    public synchronized void stop() {
+      stopped = true;
+      super.stop();
+    }
+
+    @Override
+    public synchronized InetSocketAddress getListenerAddress() {
+      return super.getListenerAddress();
+    }
+
+    @Override
+    public Writable call(RPC.RpcKind rpcKind,
+        String protocol,
+        Writable param,
+        long receiveTime) throws Exception {
+      return null;
+    }
+  }
+}

+ 151 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java

@@ -0,0 +1,151 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestWorkflowSequenceService extends ParentWorkflowTestBase {
+  private static final Logger
+      log = LoggerFactory.getLogger(TestWorkflowSequenceService.class);
+
+  @Test
+  public void testSingleSequence() throws Throwable {
+    ServiceParent parent = startService(new MockService());
+    parent.stop();
+  }
+
+  @Test
+  public void testEmptySequence() throws Throwable {
+    ServiceParent parent = startService();
+    waitForParentToStop(parent);
+  }
+
+  @Test
+  public void testSequence() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+    assert ((WorkflowSequenceService) parent).getPreviousService().equals(two);
+  }
+
+  @Test
+  public void testCallableChild() throws Throwable {
+
+    MockService one = new MockService("one", false, 100);
+    CallableHandler handler = new CallableHandler("hello");
+    WorkflowCallbackService<String> ens =
+        new WorkflowCallbackService<String>("handler", handler, 100, true);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, ens, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(ens);
+    assertStopped(two);
+    assertTrue(handler.notified);
+    String s = ens.getScheduledFuture().get();
+    assertEquals("hello", s);
+  }
+
+
+  @Test
+  public void testFailingSequence() throws Throwable {
+    MockService one = new MockService("one", true, 100);
+    MockService two = new MockService("two", false, 100);
+    WorkflowSequenceService parent =
+        (WorkflowSequenceService) startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertInState(two, Service.STATE.NOTINITED);
+    assertEquals(one, parent.getPreviousService());
+  }
+
+
+  @Test
+  public void testFailInStartNext() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", true, 0);
+    MockService three = new MockService("3", false, 0);
+    ServiceParent parent = startService(one, two, three);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+    Throwable failureCause = two.getFailureCause();
+    assertNotNull(failureCause);
+    Throwable parentFailureCause = parent.getFailureCause();
+    assertNotNull(parentFailureCause);
+    assertEquals(parentFailureCause, failureCause);
+    assertInState(three, Service.STATE.NOTINITED);
+  }
+
+  @Test
+  public void testSequenceInSequence() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = buildService(one, two);
+    ServiceParent outer = startService(parent);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+  @Test
+  public void testVarargsConstructor() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = new WorkflowSequenceService("test", one, two);
+    parent.init(new Configuration());
+    parent.start();
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+
+  @Test
+  public void testAddChild() throws Throwable {
+    MockService one = new MockService("one", false, 5000);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, two);
+    CallableHandler handler = new CallableHandler("hello");
+    WorkflowCallbackService<String> ens =
+        new WorkflowCallbackService<String>("handler", handler, 100, true);
+    parent.addService(ens);
+    waitForParentToStop(parent, 10000);
+    assertStopped(one);
+    assertStopped(two);
+    assertStopped(ens);
+    assertStopped(two);
+    assertEquals("hello", ens.getScheduledFuture().get());
+  }
+
+  public WorkflowSequenceService buildService(Service... services) {
+    WorkflowSequenceService parent =
+        new WorkflowSequenceService("test", services);
+    parent.init(new Configuration());
+    return parent;
+  }
+
+}

+ 64 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java

@@ -0,0 +1,64 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.junit.Test;
+
+
+public class TestWorkflowServiceTerminatingRunnable extends WorkflowServiceTestBase {
+
+  @Test
+  public void testNoservice() throws Throwable {
+
+    try {
+      new ServiceTerminatingRunnable(null, new SimpleRunnable());
+      fail("unexpected ");
+    } catch (IllegalArgumentException e) {
+
+      // expected
+    }
+  }
+
+
+  @Test
+  public void testBasicRun() throws Throwable {
+
+    WorkflowCompositeService svc = run(new WorkflowCompositeService());
+    ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc,
+        new SimpleRunnable());
+
+    // synchronous in-thread execution
+    runnable.run();
+    assertStopped(svc);
+  }
+
+  @Test
+  public void testFailureRun() throws Throwable {
+
+    WorkflowCompositeService svc = run(new WorkflowCompositeService());
+    ServiceTerminatingRunnable runnable =
+        new ServiceTerminatingRunnable(svc, new SimpleRunnable(true));
+
+    // synchronous in-thread execution
+    runnable.run();
+    assertStopped(svc);
+    assertNotNull(runnable.getException());
+  }
+
+}

+ 139 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java

@@ -0,0 +1,139 @@
+/*
+ * 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.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.Callable;
+
+/**
+ * Test base for workflow service tests.
+ */
+public abstract class WorkflowServiceTestBase extends Assert {
+  private static final Logger
+      log = LoggerFactory.getLogger(WorkflowServiceTestBase.class);
+
+  /**
+   * Set the timeout for every test
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(15000);
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+
+  protected void assertInState(Service service, Service.STATE expected) {
+    Service.STATE actual = service.getServiceState();
+    if (actual != expected) {
+      fail("Service " + service.getName() + " in state " + actual
+           + " -expected " + expected);
+    }
+  }
+
+  protected void assertStopped(Service service) {
+    assertInState(service, Service.STATE.STOPPED);
+  }
+
+  protected void logState(ServiceParent p) {
+    logService(p);
+    for (Service s : p.getServices()) {
+      logService(s);
+    }
+  }
+
+  protected void logService(Service s) {
+    log.info(s.toString());
+    Throwable failureCause = s.getFailureCause();
+    if (failureCause != null) {
+      log.info("Failed in state {} with {}", s.getFailureState(),
+          failureCause);
+    }
+  }
+
+  /**
+   * Init and start a service
+   * @param svc the service
+   * @return the service
+   */
+  protected <S extends Service> S run(S svc) {
+    svc.init(new Configuration());
+    svc.start();
+    return svc;
+  }
+
+  /**
+   * Handler for callable events
+   */
+  public static class CallableHandler implements Callable<String> {
+    public volatile boolean notified = false;
+    public final String result;
+
+    public CallableHandler(String result) {
+      this.result = result;
+    }
+
+    @Override
+    public String call() throws Exception {
+      log.info("CallableHandler::call");
+      notified = true;
+      return result;
+    }
+  }
+
+  /**
+   * Assert that a string is in an output list. Fails fast if the output
+   * list is empty
+   * @param text text to scan for
+   * @param output list of output lines.
+   */
+  public void assertStringInOutput(String text, List<String> output) {
+    assertTrue("Empty output list", !output.isEmpty());
+    boolean found = false;
+    StringBuilder builder = new StringBuilder();
+    for (String s : output) {
+      builder.append(s.toLowerCase(Locale.ENGLISH)).append('\n');
+      if (s.contains(text)) {
+        found = true;
+        break;
+      }
+    }
+
+    if (!found) {
+      String message =
+          "Text \"" + text + "\" not found in " + output.size() + " lines\n";
+      fail(message + builder.toString());
+    }
+  }
+}

+ 901 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/ContractTestUtils.java

@@ -0,0 +1,901 @@
+/*
+ * 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.slider.test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.internal.AssumptionViolatedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * Utilities used across test cases to make assertions about filesystems
+ * -assertions which fail with useful information.
+ * This is lifted from Hadoop common Test; that JAR isn't published, so
+ * we have to make do.
+ */
+public class ContractTestUtils extends Assert {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContractTestUtils.class);
+
+  public static final String IO_FILE_BUFFER_SIZE = "io.file.buffer.size";
+
+  // For scale testing, we can repeatedly write small chunk data to generate
+  // a large file.
+  public static final String IO_CHUNK_BUFFER_SIZE = "io.chunk.buffer.size";
+  public static final int DEFAULT_IO_CHUNK_BUFFER_SIZE = 128;
+  public static final String IO_CHUNK_MODULUS_SIZE = "io.chunk.modulus.size";
+  public static final int DEFAULT_IO_CHUNK_MODULUS_SIZE = 128;
+
+  /**
+   * Assert that a property in the property set matches the expected value
+   * @param props property set
+   * @param key property name
+   * @param expected expected value. If null, the property must not be in the set
+   */
+  public static void assertPropertyEquals(Properties props,
+                                          String key,
+                                          String expected) {
+    String val = props.getProperty(key);
+    if (expected == null) {
+      assertNull("Non null property " + key + " = " + val, val);
+    } else {
+      assertEquals("property " + key + " = " + val,
+                          expected,
+                          val);
+    }
+  }
+
+  /**
+   *
+   * Write a file and read it in, validating the result. Optional flags control
+   * whether file overwrite operations should be enabled, and whether the
+   * file should be deleted afterwards.
+   *
+   * If there is a mismatch between what was written and what was expected,
+   * a small range of bytes either side of the first error are logged to aid
+   * diagnosing what problem occurred -whether it was a previous file
+   * or a corrupting of the current file. This assumes that two
+   * sequential runs to the same path use datasets with different character
+   * moduli.
+   *
+   * @param fs filesystem
+   * @param path path to write to
+   * @param len length of data
+   * @param overwrite should the create option allow overwrites?
+   * @param delete should the file be deleted afterwards? -with a verification
+   * that it worked. Deletion is not attempted if an assertion has failed
+   * earlier -it is not in a <code>finally{}</code> block.
+   * @throws IOException IO problems
+   */
+  public static void writeAndRead(FileSystem fs,
+                                  Path path,
+                                  byte[] src,
+                                  int len,
+                                  int blocksize,
+                                  boolean overwrite,
+                                  boolean delete) throws IOException {
+    fs.mkdirs(path.getParent());
+
+    writeDataset(fs, path, src, len, blocksize, overwrite);
+
+    byte[] dest = readDataset(fs, path, len);
+
+    compareByteArrays(src, dest, len);
+
+    if (delete) {
+      rejectRootOperation(path);
+      boolean deleted = fs.delete(path, false);
+      assertTrue("Deleted", deleted);
+      assertPathDoesNotExist(fs, "Cleanup failed", path);
+    }
+  }
+
+  /**
+   * Write a file.
+   * Optional flags control
+   * whether file overwrite operations should be enabled
+   * @param fs filesystem
+   * @param path path to write to
+   * @param len length of data
+   * @param overwrite should the create option allow overwrites?
+   * @throws IOException IO problems
+   */
+  public static void writeDataset(FileSystem fs,
+                                   Path path,
+                                   byte[] src,
+                                   int len,
+                                   int buffersize,
+                                   boolean overwrite) throws IOException {
+    assertTrue(
+      "Not enough data in source array to write " + len + " bytes",
+      src.length >= len);
+    FSDataOutputStream out = fs.create(path,
+                                       overwrite,
+                                       fs.getConf()
+                                         .getInt(IO_FILE_BUFFER_SIZE,
+                                                 4096),
+                                       (short) 1,
+                                       buffersize);
+    out.write(src, 0, len);
+    out.close();
+    assertFileHasLength(fs, path, len);
+  }
+
+  /**
+   * Read the file and convert to a byte dataset.
+   * This implements readfully internally, so that it will read
+   * in the file without ever having to seek()
+   * @param fs filesystem
+   * @param path path to read from
+   * @param len length of data to read
+   * @return the bytes
+   * @throws IOException IO problems
+   */
+  public static byte[] readDataset(FileSystem fs, Path path, int len)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] dest = new byte[len];
+    int offset =0;
+    int nread = 0;
+    try {
+      while (nread < len) {
+        int nbytes = in.read(dest, offset + nread, len - nread);
+        if (nbytes < 0) {
+          throw new EOFException("End of file reached before reading fully.");
+        }
+        nread += nbytes;
+      }
+    } finally {
+      in.close();
+    }
+    return dest;
+  }
+
+  /**
+   * Read a file, verify its length and contents match the expected array
+   * @param fs filesystem
+   * @param path path to file
+   * @param original original dataset
+   * @throws IOException IO Problems
+   */
+  public static void verifyFileContents(FileSystem fs,
+                                        Path path,
+                                        byte[] original) throws IOException {
+    FileStatus stat = fs.getFileStatus(path);
+    String statText = stat.toString();
+    assertTrue("not a file " + statText, stat.isFile());
+    assertEquals("wrong length " + statText, original.length, stat.getLen());
+    byte[] bytes = readDataset(fs, path, original.length);
+    compareByteArrays(original,bytes,original.length);
+  }
+
+  /**
+   * Verify that the read at a specific offset in a stream
+   * matches that expected
+   * @param stm stream
+   * @param fileContents original file contents
+   * @param seekOff seek offset
+   * @param toRead number of bytes to read
+   * @throws IOException IO problems
+   */
+  public static void verifyRead(FSDataInputStream stm, byte[] fileContents,
+                                int seekOff, int toRead) throws IOException {
+    byte[] out = new byte[toRead];
+    stm.seek(seekOff);
+    stm.readFully(out);
+    byte[] expected = Arrays.copyOfRange(fileContents, seekOff,
+                                         seekOff + toRead);
+    compareByteArrays(expected, out,toRead);
+  }
+
+  /**
+   * Assert that tthe array original[0..len] and received[] are equal.
+   * A failure triggers the logging of the bytes near where the first
+   * difference surfaces.
+   * @param original source data
+   * @param received actual
+   * @param len length of bytes to compare
+   */
+  public static void compareByteArrays(byte[] original,
+                                       byte[] received,
+                                       int len) {
+    assertEquals("Number of bytes read != number written",
+                        len, received.length);
+    int errors = 0;
+    int first_error_byte = -1;
+    for (int i = 0; i < len; i++) {
+      if (original[i] != received[i]) {
+        if (errors == 0) {
+          first_error_byte = i;
+        }
+        errors++;
+      }
+    }
+
+    if (errors > 0) {
+      String message = String.format(" %d errors in file of length %d",
+                                     errors, len);
+      LOG.warn(message);
+      // the range either side of the first error to print
+      // this is a purely arbitrary number, to aid user debugging
+      final int overlap = 10;
+      for (int i = Math.max(0, first_error_byte - overlap);
+           i < Math.min(first_error_byte + overlap, len);
+           i++) {
+        byte actual = received[i];
+        byte expected = original[i];
+        String letter = toChar(actual);
+        String line = String.format("[%04d] %2x %s\n", i, actual, letter);
+        if (expected != actual) {
+          line = String.format("[%04d] %2x %s -expected %2x %s\n",
+                               i,
+                               actual,
+                               letter,
+                               expected,
+                               toChar(expected));
+        }
+        LOG.warn(line);
+      }
+      fail(message);
+    }
+  }
+
+  /**
+   * Convert a byte to a character for printing. If the
+   * byte value is < 32 -and hence unprintable- the byte is
+   * returned as a two digit hex value
+   * @param b byte
+   * @return the printable character string
+   */
+  public static String toChar(byte b) {
+    if (b >= 0x20) {
+      return Character.toString((char) b);
+    } else {
+      return String.format("%02x", b);
+    }
+  }
+
+  /**
+   * Convert a buffer to a string, character by character
+   * @param buffer input bytes
+   * @return a string conversion
+   */
+  public static String toChar(byte[] buffer) {
+    StringBuilder builder = new StringBuilder(buffer.length);
+    for (byte b : buffer) {
+      builder.append(toChar(b));
+    }
+    return builder.toString();
+  }
+
+  public static byte[] toAsciiByteArray(String s) {
+    char[] chars = s.toCharArray();
+    int len = chars.length;
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < len; i++) {
+      buffer[i] = (byte) (chars[i] & 0xff);
+    }
+    return buffer;
+  }
+
+  /**
+   * Cleanup at the end of a test run
+   * @param action action triggering the operation (for use in logging)
+   * @param fileSystem filesystem to work with. May be null
+   * @param cleanupPath path to delete as a string
+   */
+  public static void cleanup(String action,
+                             FileSystem fileSystem,
+                             String cleanupPath) {
+    if (fileSystem == null) {
+      return;
+    }
+    Path path = new Path(cleanupPath).makeQualified(fileSystem.getUri(),
+        fileSystem.getWorkingDirectory());
+    cleanup(action, fileSystem, path);
+  }
+
+  /**
+   * Cleanup at the end of a test run
+   * @param action action triggering the operation (for use in logging)
+   * @param fileSystem filesystem to work with. May be null
+   * @param path path to delete
+   */
+  public static void cleanup(String action, FileSystem fileSystem, Path path) {
+    noteAction(action);
+    try {
+      rm(fileSystem, path, true, false);
+    } catch (Exception e) {
+      LOG.error("Error deleting in "+ action + " - "  + path + ": " + e, e);
+    }
+  }
+
+  /**
+   * Delete a directory. There's a safety check for operations against the
+   * root directory -these are intercepted and rejected with an IOException
+   * unless the allowRootDelete flag is true
+   * @param fileSystem filesystem to work with. May be null
+   * @param path path to delete
+   * @param recursive flag to enable recursive delete
+   * @param allowRootDelete can the root directory be deleted?
+   * @throws IOException on any problem.
+   */
+  public static boolean rm(FileSystem fileSystem,
+      Path path,
+      boolean recursive,
+      boolean allowRootDelete) throws
+      IOException {
+    if (fileSystem != null) {
+      rejectRootOperation(path, allowRootDelete);
+      if (fileSystem.exists(path)) {
+        return fileSystem.delete(path, recursive);
+      }
+    }
+    return false;
+
+  }
+
+  /**
+   * Block any operation on the root path. This is a safety check
+   * @param path path in the filesystem
+   * @param allowRootOperation can the root directory be manipulated?
+   * @throws IOException if the operation was rejected
+   */
+  public static void rejectRootOperation(Path path,
+      boolean allowRootOperation) throws IOException {
+    if (path.isRoot() && !allowRootOperation) {
+      throw new IOException("Root directory operation rejected: " + path);
+    }
+  }
+
+  /**
+   * Block any operation on the root path. This is a safety check
+   * @param path path in the filesystem
+   * @throws IOException if the operation was rejected
+   */
+  public static void rejectRootOperation(Path path) throws IOException {
+    rejectRootOperation(path, false);
+  }
+
+
+  public static void noteAction(String action) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("==============  "+ action +" =============");
+    }
+  }
+
+  /**
+   * downgrade a failure to a message and a warning, then an
+   * exception for the Junit test runner to mark as failed
+   * @param message text message
+   * @param failure what failed
+   * @throws AssumptionViolatedException always
+   */
+  public static void downgrade(String message, Throwable failure) {
+    LOG.warn("Downgrading test " + message, failure);
+    AssumptionViolatedException ave =
+      new AssumptionViolatedException(failure, null);
+    throw ave;
+  }
+
+  /**
+   * report an overridden test as unsupported
+   * @param message message to use in the text
+   * @throws AssumptionViolatedException always
+   */
+  public static void unsupported(String message) {
+    skip(message);
+  }
+
+  /**
+   * report a test has been skipped for some reason
+   * @param message message to use in the text
+   * @throws AssumptionViolatedException always
+   */
+  public static void skip(String message) {
+    LOG.info("Skipping: {}", message);
+    throw new AssumptionViolatedException(message);
+  }
+
+  /**
+   * Fail with an exception that was received
+   * @param text text to use in the exception
+   * @param thrown a (possibly null) throwable to init the cause with
+   * @throws AssertionError with the text and throwable -always
+   */
+  public static void fail(String text, Throwable thrown) {
+    AssertionError e = new AssertionError(text);
+    e.initCause(thrown);
+    throw e;
+  }
+
+  /**
+   * Make an assertion about the length of a file
+   * @param fs filesystem
+   * @param path path of the file
+   * @param expected expected length
+   * @throws IOException on File IO problems
+   */
+  public static void assertFileHasLength(FileSystem fs, Path path,
+                                         int expected) throws IOException {
+    FileStatus status = fs.getFileStatus(path);
+    assertEquals(
+      "Wrong file length of file " + path + " status: " + status,
+      expected,
+      status.getLen());
+  }
+
+  /**
+   * Assert that a path refers to a directory
+   * @param fs filesystem
+   * @param path path of the directory
+   * @throws IOException on File IO problems
+   */
+  public static void assertIsDirectory(FileSystem fs,
+                                       Path path) throws IOException {
+    FileStatus fileStatus = fs.getFileStatus(path);
+    assertIsDirectory(fileStatus);
+  }
+
+  /**
+   * Assert that a path refers to a directory
+   * @param fileStatus stats to check
+   */
+  public static void assertIsDirectory(FileStatus fileStatus) {
+    assertTrue("Should be a directory -but isn't: " + fileStatus,
+               fileStatus.isDirectory());
+  }
+
+  /**
+   * Write the text to a file, returning the converted byte array
+   * for use in validating the round trip
+   * @param fs filesystem
+   * @param path path of file
+   * @param text text to write
+   * @param overwrite should the operation overwrite any existing file?
+   * @return the read bytes
+   * @throws IOException on IO problems
+   */
+  public static byte[] writeTextFile(FileSystem fs,
+                                   Path path,
+                                   String text,
+                                   boolean overwrite) throws IOException {
+    byte[] bytes = new byte[0];
+    if (text != null) {
+      bytes = toAsciiByteArray(text);
+    }
+    createFile(fs, path, overwrite, bytes);
+    return bytes;
+  }
+
+  /**
+   * Create a file
+   * @param fs filesystem
+   * @param path       path to write
+   * @param overwrite overwrite flag
+   * @param data source dataset. Can be null
+   * @throws IOException on any problem
+   */
+  public static void createFile(FileSystem fs,
+                                 Path path,
+                                 boolean overwrite,
+                                 byte[] data) throws IOException {
+    FSDataOutputStream stream = fs.create(path, overwrite);
+    if (data != null && data.length > 0) {
+      stream.write(data);
+    }
+    stream.close();
+  }
+
+  /**
+   * Touch a file
+   * @param fs filesystem
+   * @param path path
+   * @throws IOException IO problems
+   */
+  public static void touch(FileSystem fs,
+                           Path path) throws IOException {
+    createFile(fs, path, true, null);
+  }
+
+  /**
+   * Delete a file/dir and assert that delete() returned true
+   * <i>and</i> that the path no longer exists. This variant rejects
+   * all operations on root directories
+   * @param fs filesystem
+   * @param file path to delete
+   * @param recursive flag to enable recursive delete
+   * @throws IOException IO problems
+   */
+  public static void assertDeleted(FileSystem fs,
+                                   Path file,
+                                   boolean recursive) throws IOException {
+    assertDeleted(fs, file, recursive, false);
+  }
+
+  /**
+   * Delete a file/dir and assert that delete() returned true
+   * <i>and</i> that the path no longer exists. This variant rejects
+   * all operations on root directories
+   * @param fs filesystem
+   * @param file path to delete
+   * @param recursive flag to enable recursive delete
+   * @param allowRootOperations can the root dir be deleted?
+   * @throws IOException IO problems
+   */
+  public static void assertDeleted(FileSystem fs,
+      Path file,
+      boolean recursive,
+      boolean allowRootOperations) throws IOException {
+    rejectRootOperation(file, allowRootOperations);
+    assertPathExists(fs, "about to be deleted file", file);
+    boolean deleted = fs.delete(file, recursive);
+    String dir = ls(fs, file.getParent());
+    assertTrue("Delete failed on " + file + ": " + dir, deleted);
+    assertPathDoesNotExist(fs, "Deleted file", file);
+  }
+
+  /**
+   * Read in "length" bytes, convert to an ascii string
+   * @param fs filesystem
+   * @param path path to read
+   * @param length #of bytes to read.
+   * @return the bytes read and converted to a string
+   * @throws IOException IO problems
+   */
+  public static String readBytesToString(FileSystem fs,
+                                  Path path,
+                                  int length) throws IOException {
+    FSDataInputStream in = fs.open(path);
+    try {
+      byte[] buf = new byte[length];
+      in.readFully(0, buf);
+      return toChar(buf);
+    } finally {
+      in.close();
+    }
+  }
+
+  /**
+   * Take an array of filestats and convert to a string (prefixed w/ a [01] counter
+   * @param stats array of stats
+   * @param separator separator after every entry
+   * @return a stringified set
+   */
+  public static String fileStatsToString(FileStatus[] stats, String separator) {
+    StringBuilder buf = new StringBuilder(stats.length * 128);
+    for (int i = 0; i < stats.length; i++) {
+      buf.append(String.format("[%02d] %s", i, stats[i])).append(separator);
+    }
+    return buf.toString();
+  }
+
+  /**
+   * List a directory
+   * @param fileSystem FS
+   * @param path path
+   * @return a directory listing or failure message
+   * @throws IOException
+   */
+  public static String ls(FileSystem fileSystem, Path path) throws IOException {
+    if (path == null) {
+      //surfaces when someone calls getParent() on something at the top of the path
+      return "/";
+    }
+    FileStatus[] stats;
+    String pathtext = "ls " + path;
+    try {
+      stats = fileSystem.listStatus(path);
+    } catch (FileNotFoundException e) {
+      return pathtext + " -file not found";
+    } catch (IOException e) {
+      return pathtext + " -failed: " + e;
+    }
+    return dumpStats(pathtext, stats);
+  }
+
+  public static String dumpStats(String pathname, FileStatus[] stats) {
+    return pathname + fileStatsToString(stats, "\n");
+  }
+
+   /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   * @param fileSystem filesystem to resolve path against
+   * @param filename name of the file
+   * @throws IOException IO problems during file operations
+   */
+  public static void assertIsFile(FileSystem fileSystem, Path filename) throws
+                                                                 IOException {
+    assertPathExists(fileSystem, "Expected file", filename);
+    FileStatus status = fileSystem.getFileStatus(filename);
+    assertIsFile(filename, status);
+  }
+
+  /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   * @param filename name of the file
+   * @param status file status
+   */
+  public static void assertIsFile(Path filename, FileStatus status) {
+    String fileInfo = filename + "  " + status;
+    assertFalse("File claims to be a directory " + fileInfo,
+                status.isDirectory());
+    assertFalse("File claims to be a symlink " + fileInfo,
+                       status.isSymlink());
+  }
+
+  /**
+   * Create a dataset for use in the tests; all data is in the range
+   * base to (base+modulo-1) inclusive
+   * @param len length of data
+   * @param base base of the data
+   * @param modulo the modulo
+   * @return the newly generated dataset
+   */
+  public static byte[] dataset(int len, int base, int modulo) {
+    byte[] dataset = new byte[len];
+    for (int i = 0; i < len; i++) {
+      dataset[i] = (byte) (base + (i % modulo));
+    }
+    return dataset;
+  }
+
+  /**
+   * Assert that a path exists -but make no assertions as to the
+   * type of that entry
+   *
+   * @param fileSystem filesystem to examine
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws FileNotFoundException raised if the path is missing
+   * @throws IOException IO problems
+   */
+  public static void assertPathExists(FileSystem fileSystem, String message,
+                               Path path) throws IOException {
+    if (!fileSystem.exists(path)) {
+      //failure, report it
+      String listing = ls(fileSystem, path.getParent());
+      throw new FileNotFoundException(message + ": not found " + path
+        + " in \"" + path.getParent() + "\" :\n" + listing);
+    }
+  }
+
+  /**
+   * Assert that a path does not exist
+   *
+   * @param fileSystem filesystem to examine
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws IOException IO problems
+   */
+  public static void assertPathDoesNotExist(FileSystem fileSystem,
+                                            String message,
+                                            Path path) throws IOException {
+    try {
+      FileStatus status = fileSystem.getFileStatus(path);
+      fail(message + ": unexpectedly found " + path + " as  " + status);
+    } catch (FileNotFoundException expected) {
+      //this is expected
+
+    }
+  }
+
+  /**
+   * Assert that a FileSystem.listStatus on a dir finds the subdir/child entry
+   * @param fs filesystem
+   * @param dir directory to scan
+   * @param subdir full path to look for
+   * @throws IOException IO probles
+   */
+  public static void assertListStatusFinds(FileSystem fs,
+                                           Path dir,
+                                           Path subdir) throws IOException {
+    FileStatus[] stats = fs.listStatus(dir);
+    boolean found = false;
+    StringBuilder builder = new StringBuilder();
+    for (FileStatus stat : stats) {
+      builder.append(stat.toString()).append('\n');
+      if (stat.getPath().equals(subdir)) {
+        found = true;
+      }
+    }
+    assertTrue("Path " + subdir
+                      + " not found in directory " + dir + ":" + builder,
+                      found);
+  }
+
+  /**
+   * Test for the host being an OSX machine
+   * @return true if the JVM thinks that is running on OSX
+   */
+  public static boolean isOSX() {
+    return System.getProperty("os.name").contains("OS X");
+  }
+
+  /**
+   * compare content of file operations using a double byte array
+   * @param concat concatenated files
+   * @param bytes bytes
+   */
+  public static void validateFileContent(byte[] concat, byte[][] bytes) {
+    int idx = 0;
+    boolean mismatch = false;
+
+    for (byte[] bb : bytes) {
+      for (byte b : bb) {
+        if (b != concat[idx++]) {
+          mismatch = true;
+          break;
+        }
+      }
+      if (mismatch)
+        break;
+    }
+    assertFalse("File content of file is not as expected at offset " + idx,
+                mismatch);
+  }
+
+  /**
+   * Receives test data from the given input file and checks the size of the
+   * data as well as the pattern inside the received data.
+   *
+   * @param fs FileSystem
+   * @param path Input file to be checked
+   * @param expectedSize the expected size of the data to be read from the
+   *        input file in bytes
+   * @param bufferLen Pattern length
+   * @param modulus   Pattern modulus
+   * @throws IOException
+   *         thrown if an error occurs while reading the data
+   */
+  public static void verifyReceivedData(FileSystem fs, Path path,
+                                      final long expectedSize,
+                                      final int bufferLen,
+                                      final int modulus) throws IOException {
+    final byte[] testBuffer = new byte[bufferLen];
+
+    long totalBytesRead = 0;
+    int nextExpectedNumber = 0;
+    final InputStream inputStream = fs.open(path);
+    try {
+      while (true) {
+        final int bytesRead = inputStream.read(testBuffer);
+        if (bytesRead < 0) {
+          break;
+        }
+
+        totalBytesRead += bytesRead;
+
+        for (int i = 0; i < bytesRead; ++i) {
+          if (testBuffer[i] != nextExpectedNumber) {
+            throw new IOException("Read number " + testBuffer[i]
+                + " but expected " + nextExpectedNumber);
+          }
+
+          ++nextExpectedNumber;
+
+          if (nextExpectedNumber == modulus) {
+            nextExpectedNumber = 0;
+          }
+        }
+      }
+
+      if (totalBytesRead != expectedSize) {
+        throw new IOException("Expected to read " + expectedSize +
+            " bytes but only received " + totalBytesRead);
+      }
+    } finally {
+      inputStream.close();
+    }
+  }
+
+  /**
+   * Generates test data of the given size according to some specific pattern
+   * and writes it to the provided output file.
+   *
+   * @param fs FileSystem
+   * @param path Test file to be generated
+   * @param size The size of the test data to be generated in bytes
+   * @param bufferLen Pattern length
+   * @param modulus   Pattern modulus
+   * @throws IOException
+   *         thrown if an error occurs while writing the data
+   */
+  public static long generateTestFile(FileSystem fs, Path path,
+                                      final long size,
+                                      final int bufferLen,
+                                      final int modulus) throws IOException {
+    final byte[] testBuffer = new byte[bufferLen];
+    for (int i = 0; i < testBuffer.length; ++i) {
+      testBuffer[i] = (byte) (i % modulus);
+    }
+
+    final OutputStream outputStream = fs.create(path, false);
+    long bytesWritten = 0;
+    try {
+      while (bytesWritten < size) {
+        final long diff = size - bytesWritten;
+        if (diff < testBuffer.length) {
+          outputStream.write(testBuffer, 0, (int) diff);
+          bytesWritten += diff;
+        } else {
+          outputStream.write(testBuffer);
+          bytesWritten += testBuffer.length;
+        }
+      }
+
+      return bytesWritten;
+    } finally {
+      outputStream.close();
+    }
+  }
+
+  /**
+   * Creates and reads a file with the given size. The test file is generated
+   * according to a specific pattern so it can be easily verified even if it's
+   * a multi-GB one.
+   * During the read phase the incoming data stream is also checked against
+   * this pattern.
+   *
+   * @param fs FileSystem
+   * @param parent Test file parent dir path
+   * @throws IOException
+   *    thrown if an I/O error occurs while writing or reading the test file
+   */
+  public static void createAndVerifyFile(FileSystem fs, Path parent, final long fileSize)
+      throws IOException {
+    int testBufferSize = fs.getConf()
+        .getInt(IO_CHUNK_BUFFER_SIZE, DEFAULT_IO_CHUNK_BUFFER_SIZE);
+    int modulus = fs.getConf()
+        .getInt(IO_CHUNK_MODULUS_SIZE, DEFAULT_IO_CHUNK_MODULUS_SIZE);
+
+    final String objectName = UUID.randomUUID().toString();
+    final Path objectPath = new Path(parent, objectName);
+
+    // Write test file in a specific pattern
+    assertEquals(fileSize,
+        generateTestFile(fs, objectPath, fileSize, testBufferSize, modulus));
+    assertPathExists(fs, "not created successful", objectPath);
+
+    // Now read the same file back and verify its content
+    try {
+      verifyReceivedData(fs, objectPath, fileSize, testBufferSize, modulus);
+    } finally {
+      // Delete test file
+      fs.delete(objectPath, false);
+    }
+  }
+}

+ 395 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/MiniZooKeeperCluster.java

@@ -0,0 +1,395 @@
+/*
+ * 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.slider.test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+
+/**
+ * This is a version of the HBase ZK cluster cut out to be standalone
+ */
+public class MiniZooKeeperCluster {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      MiniZooKeeperCluster.class);
+
+  private static final int TICK_TIME = 2000;
+  private static final int CONNECTION_TIMEOUT = 30000;
+  public static final int MAX_CLIENT_CONNECTIONS = 1000;
+
+  private boolean started;
+
+  /** The default port. If zero, we use a random port. */
+  private int defaultClientPort = 0;
+
+  private int clientPort;
+
+  private List<NIOServerCnxnFactory> standaloneServerFactoryList;
+  private List<ZooKeeperServer> zooKeeperServers;
+  private List<Integer> clientPortList;
+
+  private int activeZKServerIndex;
+  private int tickTime = 0;
+
+  private Configuration configuration;
+
+  public MiniZooKeeperCluster() {
+    this(new Configuration());
+  }
+
+  public MiniZooKeeperCluster(Configuration configuration) {
+    this.started = false;
+    this.configuration = configuration;
+    activeZKServerIndex = -1;
+    zooKeeperServers = new ArrayList<ZooKeeperServer>();
+    clientPortList = new ArrayList<Integer>();
+    standaloneServerFactoryList = new ArrayList<NIOServerCnxnFactory>();
+  }
+
+  public void setDefaultClientPort(int clientPort) {
+    if (clientPort <= 0) {
+      throw new IllegalArgumentException("Invalid default ZK client port: "
+                                         + clientPort);
+    }
+    this.defaultClientPort = clientPort;
+  }
+
+  /**
+   * Selects a ZK client port. Returns the default port if specified.
+   * Otherwise, returns a random port. The random port is selected from the
+   * range between 49152 to 65535. These ports cannot be registered with IANA
+   * and are intended for dynamic allocation (see http://bit.ly/dynports).
+   */
+  private int selectClientPort() {
+    if (defaultClientPort > 0) {
+      return defaultClientPort;
+    }
+    return 0xc000 + new Random().nextInt(0x3f00);
+  }
+
+  public void setTickTime(int tickTime) {
+    this.tickTime = tickTime;
+  }
+
+  public int getBackupZooKeeperServerNum() {
+    return zooKeeperServers.size() - 1;
+  }
+
+  public int getZooKeeperServerNum() {
+    return zooKeeperServers.size();
+  }
+
+  // / XXX: From o.a.zk.t.ClientBase
+  private static void setupTestEnv() {
+    // during the tests we run with 100K prealloc in the logs.
+    // on windows systems prealloc of 64M was seen to take ~15seconds
+    // resulting in test failure (client timeout on first session).
+    // set env and directly in order to handle static init/gc issues
+    System.setProperty("zookeeper.preAllocSize", "100");
+    FileTxnLog.setPreallocSize(100 * 1024);
+  }
+
+  public int startup(File baseDir) throws IOException, InterruptedException {
+    return startup(baseDir, 1);
+  }
+
+  /**
+   * @param baseDir
+   * @param numZooKeeperServers
+   * @return ClientPort server bound to, -1 if there was a
+   *         binding problem and we couldn't pick another port.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public int startup(File baseDir, int numZooKeeperServers) throws IOException,
+      InterruptedException {
+    if (numZooKeeperServers <= 0)
+      return -1;
+
+    setupTestEnv();
+    shutdown();
+
+    int tentativePort = selectClientPort();
+
+    // running all the ZK servers
+    for (int i = 0; i < numZooKeeperServers; i++) {
+      File dir = new File(baseDir, "zookeeper_" + i).getAbsoluteFile();
+      recreateDir(dir);
+      int tickTimeToUse;
+      if (this.tickTime > 0) {
+        tickTimeToUse = this.tickTime;
+      } else {
+        tickTimeToUse = TICK_TIME;
+      }
+      ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
+      NIOServerCnxnFactory standaloneServerFactory;
+      while (true) {
+        try {
+          standaloneServerFactory = new NIOServerCnxnFactory();
+          standaloneServerFactory.configure(
+              new InetSocketAddress(tentativePort),
+              MAX_CLIENT_CONNECTIONS
+          );
+        } catch (BindException e) {
+          LOG.debug("Failed binding ZK Server to client port: " +
+                    tentativePort, e);
+          // We're told to use some port but it's occupied, fail
+          if (defaultClientPort > 0) return -1;
+          // This port is already in use, try to use another.
+          tentativePort = selectClientPort();
+          continue;
+        }
+        break;
+      }
+
+      // Start up this ZK server
+      standaloneServerFactory.startup(server);
+      if (!waitForServerUp(tentativePort, CONNECTION_TIMEOUT)) {
+        throw new IOException("Waiting for startup of standalone server");
+      }
+
+      // We have selected this port as a client port.
+      clientPortList.add(tentativePort);
+      standaloneServerFactoryList.add(standaloneServerFactory);
+      zooKeeperServers.add(server);
+      tentativePort++; //for the next server
+    }
+
+    // set the first one to be active ZK; Others are backups
+    activeZKServerIndex = 0;
+    started = true;
+    clientPort = clientPortList.get(activeZKServerIndex);
+    LOG.info("Started MiniZK Cluster and connect 1 ZK server " +
+             "on client port: " + clientPort);
+    return clientPort;
+  }
+
+  private void recreateDir(File dir) throws IOException {
+    if (dir.exists()) {
+      if (!FileUtil.fullyDelete(dir)) {
+        throw new IOException("Could not delete zk base directory: " + dir);
+      }
+    }
+    try {
+      dir.mkdirs();
+    } catch (SecurityException e) {
+      throw new IOException("creating dir: " + dir, e);
+    }
+  }
+
+  /**
+   * @throws IOException
+   */
+  public void shutdown() throws IOException {
+    if (!started) {
+      return;
+    }
+
+    // shut down all the zk servers
+    for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
+      NIOServerCnxnFactory standaloneServerFactory =
+          standaloneServerFactoryList.get(i);
+      int clientPort = clientPortList.get(i);
+
+      standaloneServerFactory.shutdown();
+      if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+        throw new IOException("Waiting for shutdown of standalone server");
+      }
+    }
+    for (ZooKeeperServer zkServer : zooKeeperServers) {
+      //explicitly close ZKDatabase since ZookeeperServer does not close them
+      zkServer.getZKDatabase().close();
+    }
+
+    // clear everything
+    started = false;
+    activeZKServerIndex = 0;
+    standaloneServerFactoryList.clear();
+    clientPortList.clear();
+    zooKeeperServers.clear();
+
+    LOG.info("Shutdown MiniZK cluster with all ZK servers");
+  }
+
+  /**@return clientPort return clientPort if there is another ZK backup can run
+   *         when killing the current active; return -1, if there is no backups.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public int killCurrentActiveZooKeeperServer() throws IOException,
+      InterruptedException {
+    if (!started || activeZKServerIndex < 0) {
+      return -1;
+    }
+
+    // Shutdown the current active one
+    NIOServerCnxnFactory standaloneServerFactory =
+        standaloneServerFactoryList.get(activeZKServerIndex);
+    int clientPort = clientPortList.get(activeZKServerIndex);
+
+    standaloneServerFactory.shutdown();
+    if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+      throw new IOException("Waiting for shutdown of standalone server");
+    }
+
+    zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
+
+    // remove the current active zk server
+    standaloneServerFactoryList.remove(activeZKServerIndex);
+    clientPortList.remove(activeZKServerIndex);
+    zooKeeperServers.remove(activeZKServerIndex);
+    LOG.info("Kill the current active ZK servers in the cluster " +
+             "on client port: " + clientPort);
+
+    if (standaloneServerFactoryList.size() == 0) {
+      // there is no backup servers;
+      return -1;
+    }
+    clientPort = clientPortList.get(activeZKServerIndex);
+    LOG.info("Activate a backup zk server in the cluster " +
+             "on client port: " + clientPort);
+    // return the next back zk server's port
+    return clientPort;
+  }
+
+  /**
+   * Kill one back up ZK servers
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void killOneBackupZooKeeperServer() throws IOException,
+      InterruptedException {
+    if (!started || activeZKServerIndex < 0 ||
+        standaloneServerFactoryList.size() <= 1) {
+      return;
+    }
+
+    int backupZKServerIndex = activeZKServerIndex + 1;
+    // Shutdown the current active one
+    NIOServerCnxnFactory standaloneServerFactory =
+        standaloneServerFactoryList.get(backupZKServerIndex);
+    int clientPort = clientPortList.get(backupZKServerIndex);
+
+    standaloneServerFactory.shutdown();
+    if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+      throw new IOException("Waiting for shutdown of standalone server");
+    }
+
+    zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
+
+    // remove this backup zk server
+    standaloneServerFactoryList.remove(backupZKServerIndex);
+    clientPortList.remove(backupZKServerIndex);
+    zooKeeperServers.remove(backupZKServerIndex);
+    LOG.info("Kill one backup ZK servers in the cluster " +
+             "on client port: " + clientPort);
+  }
+
+  // XXX: From o.a.zk.t.ClientBase
+  private static boolean waitForServerDown(int port, long timeout) {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        Socket sock = new Socket("localhost", port);
+        try {
+          OutputStream outstream = sock.getOutputStream();
+          outstream.write("stat".getBytes());
+          outstream.flush();
+        } finally {
+          sock.close();
+        }
+      } catch (IOException e) {
+        return true;
+      }
+
+      if (System.currentTimeMillis() > start + timeout) {
+        break;
+      }
+      try {
+        Thread.sleep(250);
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+    return false;
+  }
+
+  // XXX: From o.a.zk.t.ClientBase
+  private static boolean waitForServerUp(int port, long timeout) {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        Socket sock = new Socket("localhost", port);
+        BufferedReader reader = null;
+        try {
+          OutputStream outstream = sock.getOutputStream();
+          outstream.write("stat".getBytes());
+          outstream.flush();
+
+          Reader isr = new InputStreamReader(sock.getInputStream());
+          reader = new BufferedReader(isr);
+          String line = reader.readLine();
+          if (line != null && line.startsWith("Zookeeper version:")) {
+            return true;
+          }
+        } finally {
+          sock.close();
+          if (reader != null) {
+            reader.close();
+          }
+        }
+      } catch (IOException e) {
+        // ignore as this is expected
+        LOG.info("server localhost:" + port + " not up " + e);
+      }
+
+      if (System.currentTimeMillis() > start + timeout) {
+        break;
+      }
+      try {
+        Thread.sleep(250);
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+    return false;
+  }
+
+  public int getClientPort() {
+    return clientPort;
+  }
+}

+ 181 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/tools/TestUtility.java

@@ -0,0 +1,181 @@
+/*
+ * 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.slider.tools;
+
+import org.apache.commons.compress.archivers.zip.ZipArchiveEntry;
+import org.apache.commons.compress.archivers.zip.ZipArchiveOutputStream;
+import org.apache.commons.compress.utils.IOUtils;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/**
+ *  Various utility methods
+ *  Byte comparison methods are from
+ *  <code>org.apache.hadoop.fs.contract.ContractTestUtils</code>
+ */
+public class TestUtility {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestUtility.class);
+
+  public static void addDir(File dirObj, ZipArchiveOutputStream zipFile, String prefix) throws IOException {
+    for (File file : dirObj.listFiles()) {
+      if (file.isDirectory()) {
+        addDir(file, zipFile, prefix + file.getName() + File.separator);
+      } else {
+        log.info("Adding to zip - " + prefix + file.getName());
+        zipFile.putArchiveEntry(new ZipArchiveEntry(prefix + file.getName()));
+        IOUtils.copy(new FileInputStream(file), zipFile);
+        zipFile.closeArchiveEntry();
+      }
+    }
+  }
+
+  public static void zipDir(String zipFile, String dir) throws IOException {
+    File dirObj = new File(dir);
+    ZipArchiveOutputStream out = new ZipArchiveOutputStream(new FileOutputStream(zipFile));
+    log.info("Creating : {}", zipFile);
+    try {
+      addDir(dirObj, out, "");
+    } finally {
+      out.close();
+    }
+  }
+
+  public static String createAppPackage(
+      TemporaryFolder folder, String subDir, String pkgName, String srcPath) throws IOException {
+    String zipFileName;
+    File pkgPath = folder.newFolder(subDir);
+    File zipFile = new File(pkgPath, pkgName).getAbsoluteFile();
+    zipFileName = zipFile.getAbsolutePath();
+    TestUtility.zipDir(zipFileName, srcPath);
+    log.info("Created temporary zip file at {}", zipFileName);
+    return zipFileName;
+  }
+
+
+  /**
+   * Assert that tthe array original[0..len] and received[] are equal.
+   * A failure triggers the logging of the bytes near where the first
+   * difference surfaces.
+   * @param original source data
+   * @param received actual
+   * @param len length of bytes to compare
+   */
+  public static void compareByteArrays(byte[] original,
+      byte[] received,
+      int len) {
+    Assert.assertEquals("Number of bytes read != number written",
+        len, received.length);
+    int errors = 0;
+    int first_error_byte = -1;
+    for (int i = 0; i < len; i++) {
+      if (original[i] != received[i]) {
+        if (errors == 0) {
+          first_error_byte = i;
+        }
+        errors++;
+      }
+    }
+
+    if (errors > 0) {
+      String message = String.format(" %d errors in file of length %d",
+          errors, len);
+      log.warn(message);
+      // the range either side of the first error to print
+      // this is a purely arbitrary number, to aid user debugging
+      final int overlap = 10;
+      for (int i = Math.max(0, first_error_byte - overlap);
+           i < Math.min(first_error_byte + overlap, len);
+           i++) {
+        byte actual = received[i];
+        byte expected = original[i];
+        String letter = toChar(actual);
+        String line = String.format("[%04d] %2x %s\n", i, actual, letter);
+        if (expected != actual) {
+          line = String.format("[%04d] %2x %s -expected %2x %s\n",
+              i,
+              actual,
+              letter,
+              expected,
+              toChar(expected));
+        }
+        log.warn(line);
+      }
+      Assert.fail(message);
+    }
+  }
+  /**
+   * Convert a byte to a character for printing. If the
+   * byte value is < 32 -and hence unprintable- the byte is
+   * returned as a two digit hex value
+   * @param b byte
+   * @return the printable character string
+   */
+  public static String toChar(byte b) {
+    if (b >= 0x20) {
+      return Character.toString((char) b);
+    } else {
+      return String.format("%02x", b);
+    }
+  }
+
+  /**
+   * Convert a buffer to a string, character by character
+   * @param buffer input bytes
+   * @return a string conversion
+   */
+  public static String toChar(byte[] buffer) {
+    StringBuilder builder = new StringBuilder(buffer.length);
+    for (byte b : buffer) {
+      builder.append(toChar(b));
+    }
+    return builder.toString();
+  }
+
+  public static byte[] toAsciiByteArray(String s) {
+    char[] chars = s.toCharArray();
+    int len = chars.length;
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < len; i++) {
+      buffer[i] = (byte) (chars[i] & 0xff);
+    }
+    return buffer;
+  }
+
+  /**
+   * Create a dataset for use in the tests; all data is in the range
+   * base to (base+modulo-1) inclusive
+   * @param len length of data
+   * @param base base of the data
+   * @param modulo the modulo
+   * @return the newly generated dataset
+   */
+  public static byte[] dataset(int len, int base, int modulo) {
+    byte[] dataset = new byte[len];
+    for (int i = 0; i < len; i++) {
+      dataset[i] = (byte) (base + (i % modulo));
+    }
+    return dataset;
+  }
+}

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt

@@ -0,0 +1,16 @@
+<!--
+   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.
+-->

+ 98 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml

@@ -0,0 +1,98 @@
+<?xml version="1.0"?>
+<!--
+   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.
+-->
+
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <application>
+    <name>STORM</name>
+    <comment>Apache Hadoop Stream processing framework</comment>
+    <version>0.9.1.2.1</version>
+    <components>
+
+      <component>
+        <name>NIMBUS</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/nimbus.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_REST_API</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/rest_api.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>SUPERVISOR</name>
+        <category>SLAVE</category>
+        <commandScript>
+          <script>scripts/supervisor.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_UI_SERVER</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/ui_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>DRPC_SERVER</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/drpc_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/apache-storm-0.9.1.2.1.1.0-237.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+    <configFiles>
+      <configFile>
+        <type>xml</type>
+        <fileName>storm-site.xml</fileName>
+        <dictionaryName>storm-site</dictionaryName>
+      </configFile>
+    </configFiles>
+  </application>
+</metainfo>

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt

@@ -0,0 +1,16 @@
+<!--
+   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.
+-->

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml

@@ -0,0 +1,17 @@
+<!--
+   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.
+-->
+<metainfo></metainfo>

+ 180 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml

@@ -0,0 +1,180 @@
+<?xml version="1.0"?>
+<!--
+   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.
+-->
+
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <application>
+    <name>STORM</name>
+    <comment>Apache Hadoop Stream processing framework</comment>
+    <version>0.9.1.2.1</version>
+    <exportedConfigs>storm-site</exportedConfigs>
+
+    <exportGroups>
+      <exportGroup>
+        <name>QuickLinks</name>
+        <exports>
+          <export>
+            <name>app.jmx</name>
+            <value>http://${STORM_REST_API_HOST}:${site.global.rest_api_port}/api/cluster/summary</value>
+          </export>
+          <export>
+            <name>app.monitor</name>
+            <value>http://${STORM_UI_SERVER_HOST}:${site.storm-site.ui.port}</value>
+          </export>
+          <export>
+            <name>app.metrics</name>
+            <value>http://${site.global.ganglia_server_host}/cgi-bin/rrd.py?c=${site.global.ganglia_server_id}</value>
+          </export>
+          <export>
+            <name>ganglia.ui</name>
+            <value>http://${site.global.ganglia_server_host}/ganglia?c=${site.global.ganglia_server_id}</value>
+          </export>
+          <export>
+            <name>nimbus.url</name>
+            <value>http://${NIMBUS_HOST}:${site.storm-site.nimbus.thrift.port}</value>
+          </export>
+        </exports>
+      </exportGroup>
+    </exportGroups>
+
+    <commandOrders>
+      <commandOrder>
+        <command>NIMBUS-START</command>
+        <requires>SUPERVISOR-INSTALLED,STORM_UI_SERVER-INSTALLED,DRPC_SERVER-INSTALLED,STORM_REST_API-INSTALLED
+        </requires>
+      </commandOrder>
+      <commandOrder>
+        <command>SUPERVISOR-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>DRPC_SERVER-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>STORM_REST_API-START</command>
+        <requires>NIMBUS-STARTED,DRPC_SERVER-STARTED,STORM_UI_SERVER-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>STORM_UI_SERVER-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+    </commandOrders>
+
+    <components>
+
+      <component>
+        <name>NIMBUS</name>
+        <category>MASTER</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <appExports>QuickLinks-nimbus.url,QuickLinks-ganglia.ui,QuickLinks-app.metrics</appExports>
+        <commandScript>
+          <script>scripts/nimbus.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_REST_API</name>
+        <category>MASTER</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <appExports>QuickLinks-app.jmx</appExports>
+        <commandScript>
+          <script>scripts/rest_api.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>SUPERVISOR</name>
+        <category>SLAVE</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <componentExports>
+          <componentExport>
+            <name>log_viewer_port</name>
+            <value>${THIS_HOST}:${site.storm-site.logviewer.port}</value>
+          </componentExport>
+        </componentExports>
+        <commandScript>
+          <script>scripts/supervisor.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_UI_SERVER</name>
+        <category>MASTER</category>
+        <publishConfig>true</publishConfig>
+        <appExports>QuickLinks-app.monitor</appExports>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <commandScript>
+          <script>scripts/ui_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>DRPC_SERVER</name>
+        <category>MASTER</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <commandScript>
+          <script>scripts/drpc_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>ANOTHER_COMPONENT</name>
+        <category>MASTER</category>
+        <commands>
+          <command>
+            <exec>start command</exec>
+          </command>
+          <command>
+            <exec>stop command</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/apache-storm-0.9.1.2.1.1.0-237.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+    <packages>
+      <package>
+        <type>tarball</type>
+        <name>test-tarball-name.tgz</name>
+      </package>
+    </packages>
+  </application>
+</metainfo>