You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2014/06/30 17:37:30 UTC

[41/50] [abbrv] git commit: SLIDER-96. Ensure documents and metainfo.xml use/refer to consistent XML structure - rooted at application

SLIDER-96. Ensure documents and metainfo.xml use/refer to consistent XML structure - rooted at application


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/4eb8ea99
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/4eb8ea99
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/4eb8ea99

Branch: refs/heads/feature/SLIDER-151_Implement_full_slider_API_in_REST_and_switch_client_to_it
Commit: 4eb8ea99970b8b7dd5d284809c43ab71dc2be9fc
Parents: e4c855a
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jun 24 18:12:11 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jun 26 13:41:31 2014 -0700

----------------------------------------------------------------------
 .../providers/agent/AgentClientProvider.java    |  14 +-
 .../slider/providers/agent/AgentKeys.java       |   1 -
 .../providers/agent/AgentProviderService.java   |  52 +++----
 .../agent/application/metadata/Application.java | 121 +++++++++++++++++
 .../agent/application/metadata/Metainfo.java    |  17 +--
 .../application/metadata/MetainfoParser.java    |  10 +-
 .../agent/application/metadata/Service.java     | 121 -----------------
 .../slider/common/tools/TestSliderUtils.java    |  23 ++--
 .../agent/TestAgentClientProvider.java          |  40 +++++-
 .../agent/TestAgentProviderService.java         |  61 ++++-----
 .../metadata/MetainfoParserTest.java            |  15 +-
 .../org/apache/slider/tools/TestUtility.java    |  71 ++++++++++
 .../org/apache/slider/common/tools/test.zip     | Bin 1273 -> 0 bytes
 .../slider/common/tools/test/metainfo.txt       |  16 +++
 .../slider/common/tools/test/metainfo.xml       |  95 +++++++++++++
 .../slider/common/tools/test/someOtherFile.txt  |  16 +++
 .../slider/common/tools/test/someOtherFile.xml  |  16 +++
 .../agent/application/metadata/metainfo.xml     | 136 +++++++++----------
 18 files changed, 532 insertions(+), 293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
index 1946ebd..ea1c1b7 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
@@ -34,8 +34,8 @@ import org.apache.slider.core.launch.AbstractLauncher;
 import org.apache.slider.providers.AbstractClientProvider;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderUtils;
+import org.apache.slider.providers.agent.application.metadata.Application;
 import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.apache.slider.providers.agent.application.metadata.Service;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -152,7 +152,7 @@ public class AgentClientProvider extends AbstractClientProvider
       instanceDefinition.getAppConfOperations().
           getGlobalOptions().getMandatoryOption(AgentKeys.APP_DEF);
     } catch (BadConfigException bce) {
-      throw new BadConfigException("Application definition must be provided." + bce.getMessage());
+      throw new BadConfigException("Application definition must be provided. " + bce.getMessage());
     }
     String appDef = instanceDefinition.getAppConfOperations().
         getGlobalOptions().getMandatoryOption(AgentKeys.APP_DEF);
@@ -171,7 +171,7 @@ public class AgentClientProvider extends AbstractClientProvider
       throw new BadConfigException("Either agent package path " +
                                    AgentKeys.PACKAGE_PATH + " or image root " +
                                    OptionKeys.INTERNAL_APPLICATION_IMAGE_PATH
-                                   + " must be provided");
+                                   + " must be provided.");
     }
 
     try {
@@ -204,11 +204,11 @@ public class AgentClientProvider extends AbstractClientProvider
     Set<String> tags;
     try {
       Metainfo metainfo = AgentUtils.getApplicationMetainfo(fileSystem, appDef);
-      Service service = metainfo.getServices().get(0);
+      Application application = metainfo.getApplication();
       tags = new HashSet<>();
-      tags.add("Name: " + service.getName());
-      tags.add("Version: " + service.getVersion());
-      tags.add("Description: " + SliderUtils.truncate(service.getComment(), 80));
+      tags.add("Name: " + application.getName());
+      tags.add("Version: " + application.getVersion());
+      tags.add("Description: " + SliderUtils.truncate(application.getComment(), 80));
     } catch (IOException e) {
       log.error("error retrieving metainfo from {}", appDef, e);
       throw new SliderException("error retrieving metainfo", e);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
index dea39ea..333058d 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
@@ -86,7 +86,6 @@ public interface AgentKeys {
 
   String JAVA_HOME = "java_home";
   String PACKAGE_LIST = "package_list";
-  String COMPONENT_SCRIPT = "role.script";
   String WAIT_HEARTBEAT = "wait.heartbeat";
   String PYTHON_EXE = "python";
   String HEARTBEAT_MONITOR_INTERVAL = "heartbeat.monitor.interval";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
index 85945d8..a685a3e 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
@@ -50,11 +50,11 @@ import org.apache.slider.providers.ProviderCompleted;
 import org.apache.slider.providers.ProviderCore;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderUtils;
+import org.apache.slider.providers.agent.application.metadata.Application;
 import org.apache.slider.providers.agent.application.metadata.Component;
 import org.apache.slider.providers.agent.application.metadata.Export;
 import org.apache.slider.providers.agent.application.metadata.ExportGroup;
 import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.apache.slider.providers.agent.application.metadata.Service;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentCommandType;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
@@ -184,12 +184,12 @@ public class AgentProviderService extends AbstractProviderService implements
           initializeAgentDebugCommands(instanceDefinition);
 
           metainfo = getApplicationMetainfo(fileSystem, appDef);
-          if (metainfo == null || metainfo.getServices() == null || metainfo.getServices().size() == 0) {
+          if (metainfo == null || metainfo.getApplication() == null) {
             log.error("metainfo.xml is unavailable or malformed at {}.", appDef);
             throw new SliderException("metainfo.xml is required in app package.");
           }
 
-          commandOrder = new ComponentCommandOrder(metainfo.getServices().get(0).getCommandOrder());
+          commandOrder = new ComponentCommandOrder(metainfo.getApplication().getCommandOrder());
           monitor = new HeartbeatMonitor(this, getHeartbeatMonitorInterval());
           monitor.start();
         }
@@ -574,8 +574,8 @@ public class AgentProviderService extends AbstractProviderService implements
             publishComponentConfiguration(key, key, configs.entrySet());
           }
 
-          Service service = getMetainfo().getServices().get(0);
-          List<ExportGroup> exportGroups = service.getExportGroups();
+          Application application = getMetainfo().getApplication();
+          List<ExportGroup> exportGroups = application.getExportGroups();
           if (exportGroups != null && !exportGroups.isEmpty()) {
 
             String configKeyFormat = "${site.%s.%s}";
@@ -624,17 +624,18 @@ public class AgentProviderService extends AbstractProviderService implements
 
   /**
    * Extract script path from the application metainfo
+   *
    * @param roleName
+   *
    * @return
    */
   protected String getScriptPathFromMetainfo(String roleName) {
     String scriptPath = null;
-    List<Service> services = getMetainfo().getServices();
-    if (services.size() != 1) {
-      log.error("Malformed app definition: Expect only one service in the metainfo.xml");
+    Application application = getMetainfo().getApplication();
+    if (application == null) {
+      log.error("Malformed app definition: Expect application as the top level element for metainfo.xml");
     }
-    Service service = services.get(0);
-    for (Component component : service.getComponents()) {
+    for (Component component : application.getComponents()) {
       if (component.getName().equals(roleName)) {
         scriptPath = component.getCommandScript().getScript();
         break;
@@ -645,16 +646,17 @@ public class AgentProviderService extends AbstractProviderService implements
 
   /**
    * Is the role of type MASTER
+   *
    * @param roleName
+   *
    * @return
    */
   protected boolean isMaster(String roleName) {
-    List<Service> services = getMetainfo().getServices();
-    if (services.size() != 1) {
-      log.error("Malformed app definition: Expect only one service in the metainfo.xml");
+    Application application = getMetainfo().getApplication();
+    if (application == null) {
+      log.error("Malformed app definition: Expect application as the top level element for metainfo.xml");
     } else {
-      Service service = services.get(0);
-      for (Component component : service.getComponents()) {
+      for (Component component : application.getComponents()) {
         if (component.getName().equals(roleName)) {
           if (component.getCategory().equals("MASTER")) {
             return true;
@@ -669,16 +671,17 @@ public class AgentProviderService extends AbstractProviderService implements
 
   /**
    * Can the role publish configuration
+   *
    * @param roleName
+   *
    * @return
    */
   protected boolean canPublishConfig(String roleName) {
-    List<Service> services = getMetainfo().getServices();
-    if (services.size() != 1) {
-      log.error("Malformed app definition: Expect only one service in the metainfo.xml");
+    Application application = getMetainfo().getApplication();
+    if (application == null) {
+      log.error("Malformed app definition: Expect application as the top level element for metainfo.xml");
     } else {
-      Service service = services.get(0);
-      for (Component component : service.getComponents()) {
+      for (Component component : application.getComponents()) {
         if (component.getName().equals(roleName)) {
           return Boolean.TRUE.toString().equals(component.getPublishConfig());
         }
@@ -693,12 +696,11 @@ public class AgentProviderService extends AbstractProviderService implements
    */
   protected boolean canAnyMasterPublishConfig() {
     if (canAnyMasterPublish == null) {
-      List<Service> services = getMetainfo().getServices();
-      if (services.size() != 1) {
-        log.error("Malformed app definition: Expect only one service in the metainfo.xml");
+      Application application = getMetainfo().getApplication();
+      if (application == null) {
+        log.error("Malformed app definition: Expect application as root element in the metainfo.xml");
       } else {
-        Service service = services.get(0);
-        for (Component component : service.getComponents()) {
+        for (Component component : application.getComponents()) {
           if (Boolean.TRUE.toString().equals(component.getPublishConfig()) &&
               component.getCategory().equals("MASTER")) {
             canAnyMasterPublish = true;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
new file mode 100644
index 0000000..b007313
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
@@ -0,0 +1,121 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Application type defined in the metainfo
+ */
+public class Application {
+  String name;
+  String comment;
+  String version;
+  List<Component> components;
+  List<ExportGroup> exportGroups;
+  List<OSSpecific> osSpecifics;
+  List<CommandOrder> commandOrders;
+  ConfigurationDependencies configDependencies;
+
+  public Application() {
+    exportGroups = new ArrayList<>();
+    components = new ArrayList<>();
+    osSpecifics = new ArrayList<>();
+    commandOrders = new ArrayList<>();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getComment() {
+    return comment;
+  }
+
+  public void setComment(String comment) {
+    this.comment = comment;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public void setVersion(String version) {
+    this.version = version;
+  }
+
+  public ConfigurationDependencies getConfigDependencies() {
+    return configDependencies;
+  }
+
+  public void setConfigDependencies(ConfigurationDependencies configDependencies) {
+    this.configDependencies = configDependencies;
+  }
+
+  public void addComponent(Component component) {
+    components.add(component);
+  }
+
+  public List<Component> getComponents() {
+    return components;
+  }
+
+  public void addExportGroup(ExportGroup exportGroup) {
+    exportGroups.add(exportGroup);
+  }
+
+  public List<ExportGroup> getExportGroups() {
+    return exportGroups;
+  }
+
+  public void addOSSpecific(OSSpecific osSpecific) {
+    osSpecifics.add(osSpecific);
+  }
+
+  public List<OSSpecific> getOSSpecifics() {
+    return osSpecifics;
+  }
+
+  public void addCommandOrder(CommandOrder commandOrder) {
+    commandOrders.add(commandOrder);
+  }
+
+  public List<CommandOrder> getCommandOrder() {
+    return commandOrders;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append(",\n\"name\": ").append(name);
+    sb.append(",\n\"comment\": ").append(comment);
+    sb.append(",\n\"version\" :").append(version);
+    sb.append(",\n\"components\" : {");
+    for (Component component : components) {
+      sb.append("\n").append(component);
+    }
+    sb.append("\n},");
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
index 21e8b24..2455e8e 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
@@ -16,19 +16,14 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
 /**
- *
+ * Application metainfo uber class
  */
 public class Metainfo {
   String schemaVersion;
-  List<Service> services;
+  Application application;
 
   public Metainfo() {
-    services = new ArrayList<Service>();
   }
 
   public String getSchemaVersion() {
@@ -39,11 +34,11 @@ public class Metainfo {
     this.schemaVersion = schemaVersion;
   }
 
-  public void addService(Service service) {
-    services.add(service);
+  public Application getApplication() {
+    return application;
   }
 
-  public List<Service> getServices() {
-    return Collections.unmodifiableList(services);
+  public void setApplication(Application application) {
+    this.application = application;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
index a97c879..c7922a7 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
@@ -34,10 +34,10 @@ public class MetainfoParser {
     digester.addObjectCreate("metainfo", Metainfo.class);
     digester.addBeanPropertySetter("metainfo/schemaVersion");
 
-    digester.addObjectCreate("*/service", Service.class);
-    digester.addBeanPropertySetter("*/service/name");
-    digester.addBeanPropertySetter("*/service/comment");
-    digester.addBeanPropertySetter("*/service/version");
+    digester.addObjectCreate("*/application", Application.class);
+    digester.addBeanPropertySetter("*/application/name");
+    digester.addBeanPropertySetter("*/application/comment");
+    digester.addBeanPropertySetter("*/application/version");
 
     digester.addObjectCreate("*/commandOrder", CommandOrder.class);
     digester.addBeanPropertySetter("*/commandOrder/command");
@@ -79,7 +79,7 @@ public class MetainfoParser {
     digester.addBeanPropertySetter("*/config-type", "configType");
     digester.addSetNext("*/configuration-dependencies", "setConfigDependencies");
 
-    digester.addSetNext("*/service", "addService");
+    digester.addSetRoot("*/application", "setApplication");
 
     try {
       return (Metainfo) digester.parse(metainfoStream);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Service.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Service.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Service.java
deleted file mode 100644
index 0fc009f..0000000
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Service.java
+++ /dev/null
@@ -1,121 +0,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.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- *
- */
-public class Service {
-  String name;
-  String comment;
-  String version;
-  List<Component> components;
-  List<ExportGroup> exportGroups;
-  List<OSSpecific> osSpecifics;
-  List<CommandOrder> commandOrders;
-  ConfigurationDependencies configDependencies;
-
-  public Service() {
-    exportGroups = new ArrayList<>();
-    components = new ArrayList<>();
-    osSpecifics = new ArrayList<>();
-    commandOrders = new ArrayList<>();
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getComment() {
-    return comment;
-  }
-
-  public void setComment(String comment) {
-    this.comment = comment;
-  }
-
-  public String getVersion() {
-    return version;
-  }
-
-  public void setVersion(String version) {
-    this.version = version;
-  }
-
-  public ConfigurationDependencies getConfigDependencies() {
-    return configDependencies;
-  }
-
-  public void setConfigDependencies(ConfigurationDependencies configDependencies) {
-    this.configDependencies = configDependencies;
-  }
-
-  public void addComponent(Component component) {
-    components.add(component);
-  }
-
-  public List<Component> getComponents() {
-    return components;
-  }
-
-  public void addExportGroup(ExportGroup exportGroup) {
-    exportGroups.add(exportGroup);
-  }
-
-  public List<ExportGroup> getExportGroups() {
-    return exportGroups;
-  }
-
-  public void addOSSpecific(OSSpecific osSpecific) {
-    osSpecifics.add(osSpecific);
-  }
-
-  public List<OSSpecific> getOSSpecifics() {
-    return osSpecifics;
-  }
-
-  public void addCommandOrder(CommandOrder commandOrder) {
-    commandOrders.add(commandOrder);
-  }
-
-  public List<CommandOrder> getCommandOrder() {
-    return commandOrders;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append(",\n\"name\": ").append(name);
-    sb.append(",\n\"comment\": ").append(comment);
-    sb.append(",\n\"version\" :").append(version);
-    sb.append(",\n\"components\" : {");
-    for (Component component : components) {
-      sb.append("\n").append(component);
-    }
-    sb.append("\n},");
-    sb.append('}');
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java b/slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
index 7cac439..be850da 100644
--- a/slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
+++ b/slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
@@ -19,23 +19,30 @@ package org.apache.slider.common.tools;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
+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.InputStream;
-import java.net.URI;
 
-/**
- *
- */
+/** 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 {
+  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());
@@ -43,14 +50,14 @@ public class TestSliderUtils {
 
     InputStream stream = SliderUtils.getApplicationResourceInputStream(
         sliderFileSystem.getFileSystem(),
-        new Path("target/test-classes/org/apache/slider/common/tools/test.zip"),
+        new Path(zipFileName),
         "metainfo.xml");
     Assert.assertTrue(stream != null);
     Assert.assertTrue(stream.available() > 0);
   }
 
   @Test
-  public void testTruncate () {
+  public void testTruncate() {
     Assert.assertEquals(SliderUtils.truncate(null, 5), null);
     Assert.assertEquals(SliderUtils.truncate("323", -1), "323");
     Assert.assertEquals(SliderUtils.truncate("3232", 5), "3232");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
index 69f5a1c..4cb35aa 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
@@ -18,27 +18,41 @@ package org.apache.slider.providers.agent;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.common.tools.SliderUtils;
+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.io.InputStream;
 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 {
+  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);
-    Set<String> tags = provider.getApplicationTags(sliderFileSystem,
-      "target/test-classes/org/apache/slider/common/tools/test.zip");
+    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");
@@ -46,4 +60,18 @@ public class TestAgentClientProvider {
     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);
+      Assert.assertFalse("Should fail with BadConfigException", true);
+    } catch (BadConfigException e) {
+      log.info(e.toString());
+      Assert.assertTrue(e.getMessage().contains("Application definition must be provided"));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
index 918fda3..9e9e0db 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
@@ -43,13 +43,13 @@ import org.apache.slider.core.conf.ConfTreeOperations;
 import org.apache.slider.core.conf.MapOperations;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.launch.ContainerLauncher;
+import org.apache.slider.providers.agent.application.metadata.Application;
 import org.apache.slider.providers.agent.application.metadata.CommandOrder;
 import org.apache.slider.providers.agent.application.metadata.Component;
 import org.apache.slider.providers.agent.application.metadata.Export;
 import org.apache.slider.providers.agent.application.metadata.ExportGroup;
 import org.apache.slider.providers.agent.application.metadata.Metainfo;
 import org.apache.slider.providers.agent.application.metadata.MetainfoParser;
-import org.apache.slider.providers.agent.application.metadata.Service;
 import org.apache.slider.server.appmaster.model.mock.MockContainerId;
 import org.apache.slider.server.appmaster.model.mock.MockFileSystem;
 import org.apache.slider.server.appmaster.model.mock.MockNodeId;
@@ -97,8 +97,7 @@ public class TestAgentProviderService {
       LoggerFactory.getLogger(TestAgentProviderService.class);
   private static final String metainfo_1_str = "<metainfo>\n"
                                                + "  <schemaVersion>2.0</schemaVersion>\n"
-                                               + "  <services>\n"
-                                               + "    <service>\n"
+                                               + "  <application>\n"
                                                + "      <name>HBASE</name>\n"
                                                + "      <comment>\n"
                                                + "        Apache HBase\n"
@@ -164,15 +163,11 @@ public class TestAgentProviderService {
                                                + "          </packages>\n"
                                                + "        </osSpecific>\n"
                                                + "      </osSpecifics>\n"
-                                               + "    </service>\n"
-                                               + "  </services>\n"
+                                               + "  </application>\n"
                                                + "</metainfo>";
-
-
   private static final String metainfo_2_str = "<metainfo>\n"
                                                + "  <schemaVersion>2.0</schemaVersion>\n"
-                                               + "  <services>\n"
-                                               + "    <service>\n"
+                                               + "  <application>\n"
                                                + "      <name>HBASE</name>\n"
                                                + "      <comment>\n"
                                                + "        Apache HBase\n"
@@ -203,9 +198,9 @@ public class TestAgentProviderService {
                                                + "          </commandScript>\n"
                                                + "        </component>\n"
                                                + "      </components>\n"
-                                               + "    </service>\n"
-                                               + "  </services>\n"
+                                               + "  </application>\n"
                                                + "</metainfo>";
+
   @Test
   public void testRegistration() throws IOException {
 
@@ -244,7 +239,7 @@ public class TestAgentProviderService {
     doReturn(access).when(mockAps).getAmState();
     doReturn("scripts/hbase_master.py").when(mockAps).getScriptPathFromMetainfo(anyString());
     Metainfo metainfo = new Metainfo();
-    metainfo.addService(new Service());
+    metainfo.setApplication(new Application());
     doReturn(metainfo).when(mockAps).getApplicationMetainfo(any(SliderFileSystem.class), anyString());
 
     try {
@@ -262,7 +257,6 @@ public class TestAgentProviderService {
     desc.setInfo(StatusKeys.INFO_AM_HOSTNAME, "host1");
     desc.setInfo(StatusKeys.INFO_AM_WEB_PORT, "8088");
     desc.setInfo(OptionKeys.APPLICATION_NAME, "HBASE");
-    desc.getOrAddRole("HBASE_MASTER").put(AgentKeys.COMPONENT_SCRIPT, "scripts/hbase_master.py");
     expect(access.getClusterStatus()).andReturn(desc).anyTimes();
 
     AggregateConf aggConf = new AggregateConf();
@@ -352,7 +346,7 @@ public class TestAgentProviderService {
   public void testProcessConfig() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
     Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
-    assert metainfo.getServices().size() == 1;
+    Assert.assertNotNull(metainfo.getApplication());
     AgentProviderService aps = new AgentProviderService();
     HeartBeat hb = new HeartBeat();
     ComponentStatus status = new ComponentStatus();
@@ -405,12 +399,12 @@ public class TestAgentProviderService {
   public void testMetainfoParsing() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
     Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
-    Assert.assertEquals(metainfo.getServices().size(), 1);
-    Service service = metainfo.getServices().get(0);
-    log.info("Service: " + service.toString());
-    Assert.assertEquals(service.getName(), "HBASE");
-    Assert.assertEquals(service.getComponents().size(), 2);
-    List<Component> components = service.getComponents();
+    Assert.assertNotNull(metainfo.getApplication());
+    Application application = metainfo.getApplication();
+    log.info("Service: " + application.toString());
+    Assert.assertEquals(application.getName(), "HBASE");
+    Assert.assertEquals(application.getComponents().size(), 2);
+    List<Component> components = application.getComponents();
     int found = 0;
     for (Component component : components) {
       if (component.getName().equals("HBASE_MASTER")) {
@@ -430,8 +424,8 @@ public class TestAgentProviderService {
     }
     Assert.assertEquals(found, 2);
 
-    assert service.getExportGroups().size() == 1;
-    List<ExportGroup> egs = service.getExportGroups();
+    assert application.getExportGroups().size() == 1;
+    List<ExportGroup> egs = application.getExportGroups();
     ExportGroup eg = egs.get(0);
     assert eg.getName().equals("QuickLinks");
     assert eg.getExports().size() == 2;
@@ -441,20 +435,20 @@ public class TestAgentProviderService {
       if (export.getName().equals("JMX_Endpoint")) {
         found++;
         Assert.assertEquals(export.getValue(),
-            "http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/jmx");
+                            "http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/jmx");
       }
       if (export.getName().equals("Master_Status")) {
         found++;
         Assert.assertEquals(export.getValue(),
-            "http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/master-status");
+                            "http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/master-status");
       }
     }
     Assert.assertEquals(found, 2);
 
-    List<CommandOrder> cmdOrders = service.getCommandOrder();
+    List<CommandOrder> cmdOrders = application.getCommandOrder();
     Assert.assertEquals(cmdOrders.size(), 2);
     found = 0;
-    for (CommandOrder co : service.getCommandOrder()) {
+    for (CommandOrder co : application.getCommandOrder()) {
       if (co.getCommand().equals("HBASE_REGIONSERVER-START")) {
         Assert.assertTrue(co.getRequires().equals("HBASE_MASTER-STARTED"));
         found++;
@@ -660,14 +654,16 @@ public class TestAgentProviderService {
       cr.setRole("HBASE_REGIONSERVER");
       cr.setRoleCommand("INSTALL");
       cr.setStatus("COMPLETED");
-      cr.setFolders(new HashMap<String, String>() {{put("a", "b");}});
+      cr.setFolders(new HashMap<String, String>() {{
+        put("a", "b");
+      }});
       hb.setReports(Arrays.asList(cr));
       hbr = mockAps.handleHeartBeat(hb);
       Assert.assertEquals(3, hbr.getResponseId());
       Mockito.verify(mockAps, Mockito.times(0)).addStartCommand(anyString(),
-                                                                  anyString(),
-                                                                  any(HeartBeatResponse.class),
-                                                                  anyString());
+                                                                anyString(),
+                                                                any(HeartBeatResponse.class),
+                                                                anyString());
       // RS still does not start
       hb = new HeartBeat();
       hb.setResponseId(3);
@@ -723,8 +719,8 @@ public class TestAgentProviderService {
       hb.setReports(Arrays.asList(cr));
       mockAps.handleHeartBeat(hb);
       Mockito.verify(mockAps, Mockito.times(1)).addGetConfigCommand(anyString(),
-                                                                anyString(),
-                                                                any(HeartBeatResponse.class));
+                                                                    anyString(),
+                                                                    any(HeartBeatResponse.class));
 
       // RS starts now
       hb = new HeartBeat();
@@ -746,7 +742,6 @@ public class TestAgentProviderService {
         anyCollection());
   }
 
-
   @Test
   public void testAddStartCommand() throws Exception {
     AgentProviderService aps = new AgentProviderService();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/MetainfoParserTest.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/MetainfoParserTest.java b/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/MetainfoParserTest.java
index ac1bd81..61c53df 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/MetainfoParserTest.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/MetainfoParserTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,16 +40,16 @@ public class MetainfoParserTest {
         METAINFO_XML);
     MetainfoParser parser = new MetainfoParser();
     Metainfo metainfo = parser.parse(resStream);
-    assert metainfo != null;
-    assert metainfo.services.size() == 1;
-    Service service = metainfo.getServices().get(0);
-    assert "STORM".equals(service.getName());
-    assert 5 == service.getComponents().size();
-    OSPackage pkg = service.getOSSpecifics().get(0).getPackages().get(0);
+    Assert.assertNotNull(metainfo);
+    Assert.assertNotNull(metainfo.getApplication());
+    Application application = metainfo.getApplication();
+    assert "STORM".equals(application.getName());
+    assert 5 == 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 : service.getComponents()) {
+    for (Component comp : application.getComponents()) {
       if (comp != null && comp.getName().equals("NIMBUS")) {
         found = true;
       }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/java/org/apache/slider/tools/TestUtility.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/tools/TestUtility.java b/slider-core/src/test/java/org/apache/slider/tools/TestUtility.java
new file mode 100644
index 0000000..a8b14ac
--- /dev/null
+++ b/slider-core/src/test/java/org/apache/slider/tools/TestUtility.java
@@ -0,0 +1,71 @@
+/*
+ * 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.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 */
+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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/resources/org/apache/slider/common/tools/test.zip
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/common/tools/test.zip b/slider-core/src/test/resources/org/apache/slider/common/tools/test.zip
deleted file mode 100644
index 18acf1c..0000000
Binary files a/slider-core/src/test/resources/org/apache/slider/common/tools/test.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt b/slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt
new file mode 100644
index 0000000..a1d7780
--- /dev/null
+++ b/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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml b/slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml
new file mode 100644
index 0000000..3d24f96
--- /dev/null
+++ b/slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml
@@ -0,0 +1,95 @@
+<?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>
+
+    <configuration-dependencies>
+      <config-type>storm-site</config-type>
+      <config-type>global</config-type>
+    </configuration-dependencies>
+  </application>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt b/slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt
new file mode 100644
index 0000000..a1d7780
--- /dev/null
+++ b/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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml b/slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml
new file mode 100644
index 0000000..a1d7780
--- /dev/null
+++ b/slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml
@@ -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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4eb8ea99/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml b/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
index 2fcf4cd..3d24f96 100644
--- a/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
+++ b/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
@@ -18,80 +18,78 @@
 
 <metainfo>
   <schemaVersion>2.0</schemaVersion>
-  <services>
-    <service>
-      <name>STORM</name>
-      <comment>Apache Hadoop Stream processing framework</comment>
-      <version>0.9.1.2.1</version>
-      <components>
+  <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>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>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>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>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>
+      <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>
+    <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>
 
-      <configuration-dependencies>
-        <config-type>storm-site</config-type>
-        <config-type>global</config-type>
-      </configuration-dependencies>
-    </service>
-  </services>
+    <configuration-dependencies>
+      <config-type>storm-site</config-type>
+      <config-type>global</config-type>
+    </configuration-dependencies>
+  </application>
 </metainfo>