You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by js...@apache.org on 2014/11/10 22:32:10 UTC

[11/11] ambari git commit: AMBARI-7175. Add explicit stack service inheritance

AMBARI-7175.  Add explicit stack service inheritance


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/2fc7adec
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/2fc7adec
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/2fc7adec

Branch: refs/heads/trunk
Commit: 2fc7adecea03ec70141f41e98f1aa2d089c3b364
Parents: b71407f
Author: John Speidel <js...@hortonworks.com>
Authored: Wed Nov 5 23:12:11 2014 -0500
Committer: John Speidel <js...@hortonworks.com>
Committed: Mon Nov 10 16:31:43 2014 -0500

----------------------------------------------------------------------
 .../ambari/server/ObjectNotFoundException.java  |   18 +-
 .../server/ParentObjectNotFoundException.java   |    2 +-
 .../ambari/server/agent/HeartBeatHandler.java   |    2 +-
 .../ambari/server/agent/HeartbeatMonitor.java   |    8 +-
 .../query/render/ClusterBlueprintRenderer.java  |   20 +-
 .../server/api/services/AmbariMetaInfo.java     |  629 +-
 .../server/api/util/StackExtensionHelper.java   |  975 ---
 .../controller/AmbariActionExecutionHelper.java |    9 +-
 .../AmbariCustomCommandExecutionHelper.java     |   14 +-
 .../AmbariManagementControllerImpl.java         |   49 +-
 .../server/controller/StackServiceResponse.java |   10 +-
 .../internal/AbstractResourceProvider.java      |   12 +-
 .../internal/BlueprintResourceProvider.java     |    1 -
 .../internal/ClientConfigResourceProvider.java  |    4 +-
 .../internal/ClusterResourceProvider.java       |   21 +-
 .../internal/ComponentResourceProvider.java     |   29 +-
 .../internal/ServiceResourceProvider.java       |   36 +-
 .../server/controller/internal/Stack.java       |   29 -
 .../server/metadata/RoleCommandOrder.java       |    2 +-
 .../server/orm/entities/BlueprintEntity.java    |   17 +-
 .../apache/ambari/server/stack/BaseModule.java  |   93 +
 .../ambari/server/stack/ComponentModule.java    |  150 +
 .../server/stack/ConfigurationDirectory.java    |  129 +
 .../ambari/server/stack/ConfigurationInfo.java  |  164 +
 .../server/stack/ConfigurationModule.java       |  170 +
 .../server/stack/ModuleFileUnmarshaller.java    |   77 +
 .../ambari/server/stack/ServiceDirectory.java   |  181 +
 .../ambari/server/stack/ServiceModule.java      |  289 +
 .../ambari/server/stack/StackContext.java       |  196 +
 .../server/stack/StackDefinitionDirectory.java  |  106 +
 .../server/stack/StackDefinitionModule.java     |   69 +
 .../ambari/server/stack/StackDirectory.java     |  365 +
 .../ambari/server/stack/StackManager.java       |  175 +
 .../apache/ambari/server/stack/StackModule.java |  498 ++
 .../ambari/server/state/AutoDeployInfo.java     |   20 +
 .../ambari/server/state/ComponentInfo.java      |   48 +
 .../ambari/server/state/ConfigHelper.java       |   54 +-
 .../ambari/server/state/DependencyInfo.java     |   26 +
 .../server/state/ServiceComponentImpl.java      |   27 +-
 .../apache/ambari/server/state/ServiceImpl.java |    4 +-
 .../apache/ambari/server/state/ServiceInfo.java |  145 +-
 .../ambari/server/state/ServiceOsSpecific.java  |   60 +
 .../org/apache/ambari/server/state/Stack.java   |   66 -
 .../apache/ambari/server/state/StackInfo.java   |  112 +-
 .../server/state/cluster/ClusterImpl.java       |    8 +-
 .../render/ClusterBlueprintRendererTest.java    |   10 +-
 .../server/api/services/AmbariMetaInfoTest.java |  405 +-
 .../api/util/StackExtensionHelperTest.java      |  792 --
 .../AmbariManagementControllerImplTest.java     |    6 +-
 .../AmbariManagementControllerTest.java         |   31 +-
 .../internal/BlueprintResourceProviderTest.java |   33 +-
 .../ClientConfigResourceProviderTest.java       |    6 +-
 .../internal/ClusterResourceProviderTest.java   |    8 -
 .../internal/ComponentResourceProviderTest.java |   75 +-
 .../internal/HostResourceProviderTest.java      |    8 +-
 .../internal/ServiceResourceProviderTest.java   |   32 +-
 .../orm/entities/BlueprintEntityTest.java       |   61 +-
 .../server/stack/ComponentModuleTest.java       |  409 +
 .../ambari/server/stack/ServiceModuleTest.java  |  983 +++
 .../ambari/server/stack/StackManagerTest.java   |  594 ++
 .../ambari/server/state/PropertyInfoTest.java   |   85 +-
 .../bad-stacks/HDP/0.1/repos/repoinfo.xml       |   37 +-
 .../stacks/HDP/2.0.7/services/HIVE/metainfo.xml |    2 -
 .../stacks/HDP/2.0.7/services/YARN/metainfo.xml |    4 +
 .../services/HBASE/configuration/hbase-site.xml |  356 +
 .../HDP/2.0.8/services/HBASE/metainfo.xml       |    3 +
 .../stacks/HDP/2.1.1/services/PIG/metainfo.xml  |    8 +-
 .../resources/stacks/OTHER/1.0/metainfo.xml     |   23 +
 .../resources/stacks/OTHER/1.0/repos/hdp.json   |   10 +
 .../stacks/OTHER/1.0/repos/repoinfo.xml         |   62 +
 .../stacks/OTHER/1.0/role_command_order.json    |  104 +
 .../services/HDFS/configuration/hdfs-site.xml   |   45 +
 .../stacks/OTHER/1.0/services/HDFS/metainfo.xml |  146 +
 .../stacks/OTHER/1.0/services/HDFS/metrics.json | 7840 ++++++++++++++++++
 .../1.0/services/HDFS/package/dummy-script.py   |   20 +
 .../OTHER/1.0/services/SQOOP2/metainfo.xml      |   30 +
 .../STORM/configuration/placeholder.txt         |   17 +
 .../OTHER/1.0/services/STORM/metainfo.xml       |   29 +
 .../OTHER/1.0/services/STORM/metrics.json       |   99 +
 .../1.0/services/STORM/package/placeholder.txt  |   17 +
 .../resources/stacks/OTHER/2.0/metainfo.xml     |   24 +
 .../resources/stacks/OTHER/2.0/repos/hdp.json   |   10 +
 .../stacks/OTHER/2.0/repos/repoinfo.xml         |   62 +
 .../OTHER/2.0/services/HBASE/metainfo.xml       |   32 +
 .../OTHER/2.0/services/SQOOP2/metainfo.xml      |   28 +
 .../stacks_with_cycle/OTHER/1.0/metainfo.xml    |   23 +
 .../stacks_with_cycle/OTHER/1.0/repos/hdp.json  |   10 +
 .../OTHER/1.0/repos/repoinfo.xml                |   62 +
 .../OTHER/1.0/role_command_order.json           |  104 +
 .../services/HDFS/configuration/hdfs-site.xml   |   45 +
 .../OTHER/1.0/services/HDFS/metainfo.xml        |  146 +
 .../OTHER/1.0/services/HDFS/metrics.json        | 7840 ++++++++++++++++++
 .../1.0/services/HDFS/package/dummy-script.py   |   20 +
 .../OTHER/1.0/services/SQOOP2/metainfo.xml      |   30 +
 .../stacks_with_cycle/OTHER/2.0/metainfo.xml    |   24 +
 .../stacks_with_cycle/OTHER/2.0/repos/hdp.json  |   10 +
 .../OTHER/2.0/repos/repoinfo.xml                |   62 +
 .../OTHER/2.0/services/SQOOP2/metainfo.xml      |   29 +
 .../stacks_with_cycle2/stack1/1.0/metainfo.xml  |   23 +
 .../stack1/1.0/repos/hdp.json                   |   10 +
 .../stack1/1.0/repos/repoinfo.xml               |   62 +
 .../stack1/1.0/services/HDFS/metainfo.xml       |   28 +
 .../stacks_with_cycle2/stack2/1.0/metainfo.xml  |   23 +
 .../stack2/1.0/repos/hdp.json                   |   10 +
 .../stack2/1.0/repos/repoinfo.xml               |   62 +
 .../stack2/1.0/services/HDFS/metainfo.xml       |   28 +
 .../stacks_with_cycle2/stack3/1.0/metainfo.xml  |   23 +
 .../stack3/1.0/repos/hdp.json                   |   10 +
 .../stack3/1.0/repos/repoinfo.xml               |   62 +
 .../stack3/1.0/services/HDFS/metainfo.xml       |   28 +
 110 files changed, 23438 insertions(+), 3001 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/2fc7adec/ambari-server/src/main/java/org/apache/ambari/server/ObjectNotFoundException.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/ObjectNotFoundException.java b/ambari-server/src/main/java/org/apache/ambari/server/ObjectNotFoundException.java
index 138774a..59063cb 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/ObjectNotFoundException.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/ObjectNotFoundException.java
@@ -21,8 +21,22 @@ package org.apache.ambari.server;
 @SuppressWarnings("serial")
 public class ObjectNotFoundException extends AmbariException {
 
-  public ObjectNotFoundException(String message) {
-    super(message);
+  /**
+   * Constructor.
+   *
+   * @param msg    message
+   * @param cause  the root cause
+   */
+  public ObjectNotFoundException(String msg, ObjectNotFoundException cause) {
+    super(msg + ".  " + cause.getMessage(), cause);
   }
 
+  /**
+   * Constructor.
+   *
+   * @param msg  message
+   */
+  public ObjectNotFoundException(String msg) {
+    super(msg);
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/2fc7adec/ambari-server/src/main/java/org/apache/ambari/server/ParentObjectNotFoundException.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/ParentObjectNotFoundException.java b/ambari-server/src/main/java/org/apache/ambari/server/ParentObjectNotFoundException.java
index 98d60d5..0674dda 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/ParentObjectNotFoundException.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/ParentObjectNotFoundException.java
@@ -21,7 +21,7 @@ package org.apache.ambari.server;
 /**
  * Indicates that a parent of a resource doesn't exist.
  */
-public class ParentObjectNotFoundException extends AmbariException {
+public class ParentObjectNotFoundException extends ObjectNotFoundException {
 
   /**
    * Constructor.

http://git-wip-us.apache.org/repos/asf/ambari/blob/2fc7adec/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
index e99e39f..82b642b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
@@ -786,7 +786,7 @@ public class HeartBeatHandler {
       throw new AmbariException("Cannot provide stack components map. " +
         "Stack hasn't been selected yet.");
     }
-    StackInfo stack = ambariMetaInfo.getStackInfo(stackId.getStackName(),
+    StackInfo stack = ambariMetaInfo.getStack(stackId.getStackName(),
         stackId.getStackVersion());
 
     response.setClusterName(clusterName);

http://git-wip-us.apache.org/repos/asf/ambari/blob/2fc7adec/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java
index c39ba29..ed4d39c 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java
@@ -228,13 +228,13 @@ public class HeartbeatMonitor implements Runnable {
     String componentName = sch.getServiceComponentName();
     Service service = cluster.getService(sch.getServiceName());
     StackId stackId = cluster.getDesiredStackVersion();
-    ServiceInfo serviceInfo = ambariMetaInfo.getServiceInfo(stackId.getStackName(),
-            stackId.getStackVersion(), serviceName);
+    ServiceInfo serviceInfo = ambariMetaInfo.getService(stackId.getStackName(),
+        stackId.getStackVersion(), serviceName);
     ComponentInfo componentInfo = ambariMetaInfo.getComponent(
             stackId.getStackName(), stackId.getStackVersion(),
             serviceName, componentName);
-    StackInfo stackInfo = ambariMetaInfo.getStackInfo(stackId.getStackName(),
-            stackId.getStackVersion());
+    StackInfo stackInfo = ambariMetaInfo.getStack(stackId.getStackName(),
+        stackId.getStackVersion());
 
     Map<String, Map<String, String>> configurations = new TreeMap<String, Map<String, String>>();
     Map<String, Map<String,  Map<String, String>>> configurationAttributes = new TreeMap<String, Map<String, Map<String, String>>>();

http://git-wip-us.apache.org/repos/asf/ambari/blob/2fc7adec/ambari-server/src/main/java/org/apache/ambari/server/api/query/render/ClusterBlueprintRenderer.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/query/render/ClusterBlueprintRenderer.java b/ambari-server/src/main/java/org/apache/ambari/server/api/query/render/ClusterBlueprintRenderer.java
index 847735d..9967fc6 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/query/render/ClusterBlueprintRenderer.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/query/render/ClusterBlueprintRenderer.java
@@ -18,6 +18,7 @@
 
 package org.apache.ambari.server.api.query.render;
 
+import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.api.query.QueryInfo;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.api.services.Request;
@@ -39,7 +40,7 @@ import org.apache.ambari.server.state.HostConfig;
 import org.apache.ambari.server.state.PropertyInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
+import org.apache.ambari.server.state.ServiceInfo;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
@@ -175,9 +176,20 @@ public class ClusterBlueprintRenderer extends BaseRenderer implements Renderer {
    */
   private void determinePropertiesToStrip(TreeNode<Resource> servicesNode, String stackName, String stackVersion) {
     AmbariMetaInfo stackInfo = getController().getAmbariMetaInfo();
-    for (TreeNode<Resource> service : servicesNode.getChildren()) {
-      String name = (String) service.getObject().getPropertyValue("ServiceInfo/service_name");
-      Map<String, PropertyInfo> requiredProperties = stackInfo.getRequiredProperties(stackName, stackVersion, name);
+    for (TreeNode<Resource> serviceNode : servicesNode.getChildren()) {
+      String name = (String) serviceNode.getObject().getPropertyValue("ServiceInfo/service_name");
+      ServiceInfo service;
+      try {
+        service = stackInfo.getService(stackName, stackVersion, name);
+      } catch (AmbariException e) {
+        // shouldn't ever happen.
+        // Exception indicates that service is not in the stack
+        // but we are getting the name from a running cluster.
+        throw new RuntimeException("Unexpected exception occurred while generating a blueprint.  The service '" +
+            name + "' was not found in the stack: '" + stackName + ":" + stackVersion);
+      }
+
+      Map<String, PropertyInfo> requiredProperties = service.getRequiredProperties();
       for (Map.Entry<String, PropertyInfo> entry : requiredProperties.entrySet()) {
         String propertyName = entry.getKey();
         PropertyInfo propertyInfo = entry.getValue();

http://git-wip-us.apache.org/repos/asf/ambari/blob/2fc7adec/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
index ba365f5..2a137c9 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
@@ -21,9 +21,9 @@ package org.apache.ambari.server.api.services;
 import java.io.File;
 import java.io.FileReader;
 import java.io.FilenameFilter;
-import java.io.IOException;
 import java.lang.reflect.Type;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -32,17 +32,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Scanner;
 import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
 
 import javax.xml.bind.JAXBException;
 
 import org.apache.ambari.server.AmbariException;
-import org.apache.ambari.server.ObjectNotFoundException;
 import org.apache.ambari.server.ParentObjectNotFoundException;
 import org.apache.ambari.server.StackAccessException;
-import org.apache.ambari.server.api.util.StackExtensionHelper;
 import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.customactions.ActionDefinition;
 import org.apache.ambari.server.customactions.ActionDefinitionManager;
@@ -50,11 +45,15 @@ import org.apache.ambari.server.events.AlertDefinitionRegistrationEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.metadata.AgentAlertDefinitions;
 import org.apache.ambari.server.orm.dao.AlertDefinitionDAO;
+import org.apache.ambari.server.metadata.ActionMetadata;
 import org.apache.ambari.server.orm.dao.MetainfoDAO;
 import org.apache.ambari.server.orm.entities.AlertDefinitionEntity;
 import org.apache.ambari.server.orm.entities.MetainfoEntity;
+import org.apache.ambari.server.stack.StackContext;
+import org.apache.ambari.server.stack.StackDirectory;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.stack.StackManager;
 import org.apache.ambari.server.state.ComponentInfo;
 import org.apache.ambari.server.state.DependencyInfo;
 import org.apache.ambari.server.state.OperatingSystemInfo;
@@ -62,26 +61,19 @@ import org.apache.ambari.server.state.PropertyInfo;
 import org.apache.ambari.server.state.RepositoryInfo;
 import org.apache.ambari.server.state.Service;
 import org.apache.ambari.server.state.ServiceInfo;
-import org.apache.ambari.server.state.Stack;
 import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.StackInfo;
 import org.apache.ambari.server.state.alert.AlertDefinition;
 import org.apache.ambari.server.state.alert.AlertDefinitionFactory;
-import org.apache.ambari.server.state.stack.LatestRepoCallable;
 import org.apache.ambari.server.state.stack.MetricDefinition;
 import org.apache.ambari.server.state.stack.OsFamily;
-import org.apache.ambari.server.state.stack.RepositoryXml;
-import org.apache.ambari.server.state.stack.RepositoryXml.Os;
-import org.apache.ambari.server.state.stack.RepositoryXml.Repo;
 import org.apache.ambari.server.state.stack.UpgradePack;
-import org.apache.commons.io.FilenameUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.gson.Gson;
 import com.google.gson.reflect.TypeToken;
 import com.google.inject.Inject;
-import com.google.inject.Injector;
 import com.google.inject.Singleton;
 
 
@@ -91,9 +83,7 @@ import com.google.inject.Singleton;
 @Singleton
 public class AmbariMetaInfo {
 
-  public static final String STACK_METAINFO_FILE_NAME = "metainfo.xml";
-  public static final String SERVICES_FOLDER_NAME = "services";
-  public static final String SERVICE_METAINFO_FILE_NAME = "metainfo.xml";
+
   public static final String SERVICE_CONFIG_FOLDER_NAME = "configuration";
   public static final String SERVICE_CONFIG_FILE_NAME_POSTFIX = ".xml";
   public static final String RCO_FILE_NAME = "role_command_order.json";
@@ -112,19 +102,12 @@ public class AmbariMetaInfo {
   public static final FilenameFilter FILENAME_FILTER = new FilenameFilter() {
     @Override
     public boolean accept(File dir, String s) {
-      if (s.equals(".svn") || s.equals(".git") ||
-          s.equals(StackExtensionHelper.HOOKS_FOLDER_NAME)) // Hooks dir is not a service
-      {
-        return false;
-      }
-      return true;
+      return !(s.equals(".svn") || s.equals(".git") ||
+          s.equals(StackDirectory.HOOKS_FOLDER_NAME));
     }
   };
   private final static Logger LOG = LoggerFactory.getLogger(AmbariMetaInfo.class);
-  private static final String REPOSITORY_FILE_NAME = "repoinfo.xml";
-  private static final String REPOSITORY_FOLDER_NAME = "repos";
   public static final String REPOSITORY_XML_PROPERTY_BASEURL = "baseurl";
-  private static final String UPGRADE_PACK_FOLDER_NAME = "upgrades";
 
   // all the supported OS'es
   @Inject
@@ -138,28 +121,26 @@ public class AmbariMetaInfo {
 
   private final ActionDefinitionManager adManager = new ActionDefinitionManager();
   private String serverVersion = "undefined";
-  private List<StackInfo> stacksResult = new ArrayList<StackInfo>();
+  private StackManager stackManager;
+
   private File stackRoot;
   private File serverVersionFile;
   private File customActionRoot;
 
   @Inject
-  private MetainfoDAO metainfoDAO;
+  private MetainfoDAO metaInfoDAO;
 
-  @Inject
-  Injector injector;
-
-  @Inject
-  Configuration cfg;
   /**
    * Alert Definition DAO used to merge stack definitions into the database.
    */
+  @Inject
   AlertDefinitionDAO alertDefinitionDao;
 
   /**
    * A factory that assists in the creation of {@link AlertDefinition} and
    * {@link AlertDefinitionEntity}.
    */
+  @Inject
   private AlertDefinitionFactory alertDefinitionFactory;
 
   /**
@@ -178,9 +159,10 @@ public class AmbariMetaInfo {
   @Inject
   private AmbariEventPublisher eventPublisher;
 
-  // Required properties by stack version
-  private final Map<StackId, Map<String, Map<String, PropertyInfo>>> requiredProperties =
-    new HashMap<StackId, Map<String, Map<String, PropertyInfo>>>();
+  //todo: only used by StackManager
+  @Inject
+  ActionMetadata actionMetadata;
+
 
   /**
    * Ambari Meta Info Object
@@ -212,43 +194,19 @@ public class AmbariMetaInfo {
   @Inject
   public void init() throws Exception {
     // Need to be initialized before all actions
-    os_family = injector.getInstance(OsFamily.class);
     ALL_SUPPORTED_OS = new ArrayList<String>(os_family.os_list());
-
-    stacksResult = new ArrayList<StackInfo>();
     readServerVersion();
-    getConfigurationInformation(stackRoot);
+    stackManager = new StackManager(stackRoot,
+        new StackContext(metaInfoDAO, actionMetadata, os_family));
     getCustomActionDefinitions(customActionRoot);
-
-    alertDefinitionFactory = injector.getInstance(AlertDefinitionFactory.class);
-    alertDefinitionDao = injector.getInstance(AlertDefinitionDAO.class);
-    eventPublisher = injector.getInstance(AmbariEventPublisher.class);
-    agentAlertDefinitions = injector.getInstance(AgentAlertDefinitions.class);
   }
 
   /**
-   * Get component category
-   *
-   * @param stackName     stack name
-   * @param version       stack version
-   * @param serviceName   service name
-   * @param componentName component name
-   * @return component component Info
-   * @throws AmbariException
+   * Obtain the underlying stack manager.
+   * @return stack manager
    */
-  public ComponentInfo getComponentCategory(String stackName, String version,
-                                            String serviceName, String componentName) throws AmbariException {
-    ComponentInfo component = null;
-    List<ComponentInfo> components = getComponentsByService(stackName, version, serviceName);
-    if (components != null) {
-      for (ComponentInfo cmp : components) {
-        if (cmp.getName().equals(componentName)) {
-          component = cmp;
-          break;
-        }
-      }
-    }
-    return component;
+  public StackManager getStackManager() {
+    return stackManager;
   }
 
   /**
@@ -263,44 +221,28 @@ public class AmbariMetaInfo {
   public List<ComponentInfo> getComponentsByService(String stackName, String version, String serviceName)
       throws AmbariException {
 
-    ServiceInfo service = getServiceInfo(stackName, version, serviceName);
-    if (service == null) {
+    ServiceInfo service;
+    try {
+      service = getService(stackName, version, serviceName);
+    } catch (StackAccessException e) {
       throw new ParentObjectNotFoundException("Parent Service resource doesn't exist. stackName=" +
           stackName + ", stackVersion=" + version + ", serviceName=" + serviceName);
     }
-
     return service.getComponents();
   }
 
-
   public ComponentInfo getComponent(String stackName, String version, String serviceName,
                                     String componentName) throws AmbariException {
 
-    List<ComponentInfo> componentsByService = getComponentsByService(stackName, version, serviceName);
+    ComponentInfo component = getService(stackName, version, serviceName).getComponentByName(componentName);
 
-    if (componentsByService.size() == 0) {
+    if (component == null) {
       throw new StackAccessException("stackName=" + stackName
           + ", stackVersion=" + version
           + ", serviceName=" + serviceName
           + ", componentName=" + componentName);
     }
-
-    ComponentInfo componentResult = null;
-
-    for (ComponentInfo component : componentsByService) {
-      if (component.getName().equals(componentName)) {
-        componentResult = component;
-      }
-    }
-
-    if (componentResult == null) {
-      throw new StackAccessException("stackName=" + stackName
-          + ", stackVersion=" + version
-          + ", serviceName=" + serviceName
-          + ", componentName=" + componentName);
-    }
-
-    return componentResult;
+    return component;
   }
 
   /**
@@ -364,18 +306,16 @@ public class AmbariMetaInfo {
 
   public Map<String, List<RepositoryInfo>> getRepository(String stackName,
                                                          String version) throws AmbariException {
-    Map<String, List<RepositoryInfo>> reposResult = null;
-    StackInfo stack = getStackInfo(stackName, version);
-    if (stack != null) {
-      List<RepositoryInfo> repository = stack.getRepositories();
-      reposResult = new HashMap<String, List<RepositoryInfo>>();
-      for (RepositoryInfo repo : repository) {
-        if (!reposResult.containsKey(repo.getOsType())) {
-          reposResult.put(repo.getOsType(),
-              new ArrayList<RepositoryInfo>());
-        }
-        reposResult.get(repo.getOsType()).add(repo);
+    StackInfo stack = getStack(stackName, version);
+    List<RepositoryInfo> repository = stack.getRepositories();
+
+    Map<String, List<RepositoryInfo>> reposResult = new HashMap<String, List<RepositoryInfo>>();
+    for (RepositoryInfo repo : repository) {
+      if (!reposResult.containsKey(repo.getOsType())) {
+        reposResult.put(repo.getOsType(),
+            new ArrayList<RepositoryInfo>());
       }
+      reposResult.get(repo.getOsType()).add(repo);
     }
     return reposResult;
   }
@@ -383,7 +323,7 @@ public class AmbariMetaInfo {
   public List<RepositoryInfo> getRepositories(String stackName,
                                               String version, String osType) throws AmbariException {
 
-    StackInfo stack = getStackInfo(stackName, version);
+    StackInfo stack = getStack(stackName, version);
     List<RepositoryInfo> repositories = stack.getRepositories();
 
     List<RepositoryInfo> repositoriesResult = new ArrayList<RepositoryInfo>();
@@ -425,42 +365,40 @@ public class AmbariMetaInfo {
   /*
    * function for given a stack name and version, is it a supported stack
    */
-  public boolean isSupportedStack(String stackName, String version) throws AmbariException {
-    boolean exist = false;
+  public boolean isSupportedStack(String stackName, String version) {
     try {
-      StackInfo stackInfo = getStackInfo(stackName, version);
-      if (stackInfo != null) {
-        exist = true;
-      }
-    } catch (ObjectNotFoundException e) {
+      // thrown an exception if the stack doesn't exist
+      getStack(stackName, version);
+      return true;
+    } catch (AmbariException e) {
+      return false;
     }
-    return exist;
   }
 
   /*
    * support isValidService(), isValidComponent for a given stack/version
    */
-  public boolean isValidService(String stackName, String version,
-                                String serviceName) throws AmbariException {
+  public boolean isValidService(String stackName, String version, String serviceName){
 
-    boolean exist = false;
     try {
-      ServiceInfo info= getServiceInfo(stackName, version, serviceName);
-      if (info != null) {
-        exist = true;
-      }
-    } catch (ObjectNotFoundException e) {
+      getService(stackName, version, serviceName);
+      return true;
+    } catch (AmbariException e) {
+      return false;
     }
-    return exist;
   }
 
   /*
    * support isValidService(), isValidComponent for a given stack/version
    */
   public boolean isValidServiceComponent(String stackName, String version,
-                                         String serviceName, String componentName) throws AmbariException {
-    ServiceInfo service = getServiceInfo(stackName, version, serviceName);
-    return service != null && service.getComponentByName(componentName) != null;
+                                         String serviceName, String componentName) {
+    try {
+      getService(stackName, version, serviceName).getComponentByName(componentName);
+      return true;
+    } catch (AmbariException e) {
+      return false;
+    }
   }
 
   /**
@@ -482,8 +420,7 @@ public class AmbariMetaInfo {
     }
     Map<String, ServiceInfo> services = getServices(stackName, version);
     String retService = null;
-    if (services == null
-        || services.isEmpty()) {
+    if (services == null || services.isEmpty()) {
       return retService;
     }
     for (Map.Entry<String, ServiceInfo> entry : services.entrySet()) {
@@ -497,46 +434,6 @@ public class AmbariMetaInfo {
   }
 
   /**
-   * Get the service configs supported for a service in a particular stack
-   *
-   * @param stackName   the stack name
-   * @param version     the version of the stack
-   * @param serviceName the name of the service in the stack
-   * @return the config knobs supported for the service
-   * @throws AmbariException
-   */
-  public Map<String, Map<String, String>> getSupportedConfigs(String stackName,
-                                                              String version, String serviceName) throws AmbariException {
-    Map<String, Map<String, String>> propertiesResult = new HashMap<String, Map<String, String>>();
-
-    ServiceInfo service = getServiceInfo(stackName, version, serviceName);
-    if (service != null) {
-      if (serviceName.equals(service.getName())) {
-        List<PropertyInfo> properties = service.getProperties();
-        if (properties != null) {
-          for (PropertyInfo propertyInfo : properties) {
-            Map<String, String> fileProperties = propertiesResult
-                .get(propertyInfo.getFilename());
-            if (fileProperties == null) {
-              fileProperties = new HashMap<String, String>();
-              fileProperties.put(propertyInfo.getName(),
-                  propertyInfo.getValue());
-              propertiesResult.put(propertyInfo.getFilename(), fileProperties);
-
-            } else {
-              fileProperties.put(propertyInfo.getName(),
-                  propertyInfo.getValue());
-            }
-
-          }
-        }
-      }
-    }
-
-    return propertiesResult;
-  }
-
-  /**
    * Given a stack name and version return all the services with info
    *
    * @param stackName the stack name
@@ -548,10 +445,10 @@ public class AmbariMetaInfo {
 
     Map<String, ServiceInfo> servicesInfoResult = new HashMap<String, ServiceInfo>();
 
-    List<ServiceInfo> services;
+    Collection<ServiceInfo> services;
     StackInfo stack;
     try {
-      stack = getStackInfo(stackName, version);
+      stack = getStack(stackName, version);
     } catch (StackAccessException e) {
       throw new ParentObjectNotFoundException("Parent Stack Version resource doesn't exist", e);
     }
@@ -566,63 +463,22 @@ public class AmbariMetaInfo {
   }
 
   public ServiceInfo getService(String stackName, String version, String serviceName) throws AmbariException {
+    ServiceInfo service = getStack(stackName, version).getService(serviceName);
 
-    Map<String, ServiceInfo> services = getServices(stackName, version);
-
-    if (services.size() == 0) {
-      throw new StackAccessException("stackName=" + stackName + ", stackVersion=" + version + ", serviceName=" + serviceName);
-    }
-
-    ServiceInfo serviceInfo = services.get(serviceName);
-
-    if (serviceInfo == null) {
-      throw new StackAccessException("stackName=" + stackName + ", stackVersion=" + version + ", serviceName=" + serviceName);
-    }
-
-    return serviceInfo;
-
-  }
-
-  public ServiceInfo getServiceInfo(String stackName, String version,
-                                    String serviceName) throws AmbariException {
-    ServiceInfo serviceInfoResult = null;
-    List<ServiceInfo> services;
-    StackInfo stack;
-    try {
-      stack = getStackInfo(stackName, version);
-    } catch (StackAccessException e) {
-      throw new ParentObjectNotFoundException("Parent Stack Version resource doesn't exist", e);
-    }
-
-    services = stack.getServices();
-    if (services != null) {
-      for (ServiceInfo service : services) {
-        if (serviceName.equals(service.getName())) {
-          serviceInfoResult = service;
-          break;
-        }
-      }
+    if (service == null) {
+      throw new StackAccessException("stackName=" + stackName + ", stackVersion=" +
+                                     version + ", serviceName=" + serviceName);
     }
-    return serviceInfoResult;
-  }
 
-  public List<ServiceInfo> getSupportedServices(String stackName, String version)
-    throws AmbariException {
-    List<ServiceInfo> servicesResult = null;
-    StackInfo stack = getStackInfo(stackName, version);
-    if (stack != null) {
-      servicesResult = stack.getServices();
-    }
-    return servicesResult;
+    return service;
   }
 
-  public List<String> getMonitoringServiceNames(String stackName, String version)
+  public Collection<String> getMonitoringServiceNames(String stackName, String version)
     throws AmbariException{
 
     List<String> monitoringServices = new ArrayList<String>();
-    for (ServiceInfo service : getSupportedServices(stackName, version)) {
-      if ((service.isMonitoringService() != null) &&
-        service.isMonitoringService()) {
+    for (ServiceInfo service : getServices(stackName, version).values()) {
+      if ((service.isMonitoringService() != null) && service.isMonitoringService()) {
         monitoringServices.add(service.getName());
       }
     }
@@ -633,9 +489,7 @@ public class AmbariMetaInfo {
     throws AmbariException{
 
     HashSet<String> needRestartServices = new HashSet<String>();
-
-    List<ServiceInfo> serviceInfos = getSupportedServices(stackName, version);
-
+    Collection<ServiceInfo> serviceInfos = getServices(stackName, version).values();
 
     for (ServiceInfo service : serviceInfos) {
       if (service.isRestartRequiredAfterChange() != null && service.isRestartRequiredAfterChange()) {
@@ -644,68 +498,23 @@ public class AmbariMetaInfo {
     }
     return needRestartServices;
   }
-
-  public List<StackInfo> getSupportedStacks() {
-    return stacksResult;
-  }
-
-  public Set<Stack> getStackNames() {
-
-    Set<Stack> stacks = new HashSet<Stack>();
-    List<StackInfo> supportedStacks = getSupportedStacks();
-
-    for (StackInfo stackInfo : supportedStacks) {
-      Stack stack = new Stack(stackInfo.getName());
-      stacks.add(stack);
-    }
-
-    return stacks;
+  
+  public Collection<StackInfo> getStacks() {
+    return stackManager.getStacks();
   }
 
-  public Stack getStack(String stackName) throws AmbariException {
-
-    Set<Stack> supportedStackNames = getStackNames();
-
-    if (supportedStackNames.size() == 0) {
-      throw new StackAccessException("stackName=" + stackName);
-    }
-
-    Stack stackResult = null;
-
-    for (Stack stack : supportedStackNames) {
-      if (stack.getStackName().equals(stackName)) {
-        stackResult = stack;
-      }
-    }
+  public Collection<StackInfo> getStacks(String stackName) throws AmbariException {
+    Collection<StackInfo> stacks = stackManager.getStacks(stackName);
 
-    if (stackResult == null) {
+    if (stacks.isEmpty()) {
       throw new StackAccessException("stackName=" + stackName);
     }
 
-    return stackResult;
-  }
-
-  public Set<StackInfo> getStackInfos(String stackName) {
-
-    Set<StackInfo> stackVersions = new HashSet<StackInfo>();
-    for (StackInfo stackInfo : stacksResult) {
-      if (stackName.equals(stackInfo.getName())) {
-        stackVersions.add(stackInfo);
-      }
-    }
-    return stackVersions;
+    return stacks;
   }
 
-  public StackInfo getStackInfo(String stackName, String version) throws AmbariException {
-    StackInfo stackInfoResult = null;
-
-    for (StackInfo stack : stacksResult) {
-      if (stackName.equals(stack.getName())
-          && version.equals(stack.getVersion())) {
-        stackInfoResult = stack;
-        break;
-      }
-    }
+  public StackInfo getStack(String stackName, String version) throws AmbariException {
+    StackInfo stackInfoResult = stackManager.getStack(stackName, version);
 
     if (stackInfoResult == null) {
       throw new StackAccessException("stackName=" + stackName
@@ -718,42 +527,33 @@ public class AmbariMetaInfo {
   public List<String> getStackParentVersions(String stackName, String version) {
     List<String> parents = new ArrayList<String>();
     try {
-      StackInfo stackInfo = getStackInfo(stackName, version);
-      if (stackInfo != null) {
-        String parentVersion = stackInfo.getParentStackVersion();
-        if (parentVersion != null) {
-          parents.add(parentVersion);
-          parents.addAll(getStackParentVersions(stackName, parentVersion));
-        }
+      StackInfo stackInfo = getStack(stackName, version);
+      String parentVersion = stackInfo.getParentStackVersion();
+      if (parentVersion != null) {
+        parents.add(parentVersion);
+        parents.addAll(getStackParentVersions(stackName, parentVersion));
       }
     } catch (AmbariException e) {
-      // parent was not found. just returning empty list
-    } finally {
-      return parents;
+      // parent was not found.
     }
+    return parents;
   }
 
-  public Set<PropertyInfo> getProperties(String stackName, String version, String serviceName)
+  public Set<PropertyInfo> getServiceProperties(String stackName, String version, String serviceName)
       throws AmbariException {
-    ServiceInfo serviceInfo = getServiceInfo(stackName, version, serviceName);
-    List<PropertyInfo> properties = serviceInfo.getProperties();
-    Set<PropertyInfo> propertiesResult = new HashSet<PropertyInfo>(properties);
 
-    return propertiesResult;
+    return new HashSet<PropertyInfo>(getService(stackName, version, serviceName).getProperties());
   }
 
   public Set<PropertyInfo> getStackProperties(String stackName, String version)
       throws AmbariException {
-    StackInfo stackInfo = getStackInfo(stackName, version);
-    List<PropertyInfo> properties = stackInfo.getProperties();
-    Set<PropertyInfo> propertiesResult = new HashSet<PropertyInfo>(properties);
 
-    return propertiesResult;
+    return new HashSet<PropertyInfo>(getStack(stackName, version).getProperties());
   }
 
   public Set<PropertyInfo> getPropertiesByName(String stackName, String version, String serviceName, String propertyName)
       throws AmbariException {
-    Set<PropertyInfo> properties = getProperties(stackName, version, serviceName);
+    Set<PropertyInfo> properties = getServiceProperties(stackName, version, serviceName);
 
     if (properties.size() == 0) {
       throw new StackAccessException("stackName=" + stackName
@@ -815,7 +615,7 @@ public class AmbariMetaInfo {
       throws AmbariException {
 
     Set<OperatingSystemInfo> operatingSystems = new HashSet<OperatingSystemInfo>();
-    StackInfo stack = getStackInfo(stackName, version);
+    StackInfo stack = getStack(stackName, version);
     List<RepositoryInfo> repositories = stack.getRepositories();
     for (RepositoryInfo repository : repositories) {
       operatingSystems.add(new OperatingSystemInfo(repository.getOsType()));
@@ -840,6 +640,7 @@ public class AmbariMetaInfo {
     for (OperatingSystemInfo operatingSystem : operatingSystems) {
       if (operatingSystem.getOsType().equals(osType)) {
         resultOperatingSystem = operatingSystem;
+        break;
       }
     }
 
@@ -895,168 +696,10 @@ public class AmbariMetaInfo {
     adManager.addActionDefinition(ad);
   }
 
-  private void getConfigurationInformation(File stackRoot) throws Exception {
-    String stackRootAbsPath = stackRoot.getAbsolutePath();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Loading stack information"
-        + ", stackRoot = " + stackRootAbsPath);
-    }
-
-    if (!stackRoot.isDirectory() && !stackRoot.exists()) {
-      throw new IOException("" + Configuration.METADETA_DIR_PATH
-        + " should be a directory with stack"
-        + ", stackRoot = " + stackRootAbsPath);
-    }
-
-    StackExtensionHelper stackExtensionHelper = new StackExtensionHelper(injector, stackRoot);
-    stackExtensionHelper.fillInfo();
-
-    List<StackInfo> stacks = stackExtensionHelper.getAllAvailableStacks();
-    if (stacks.isEmpty()) {
-      throw new AmbariException("Unable to find stack definitions under " +
-        "stackRoot = " + stackRootAbsPath);
-    }
-
-    ExecutorService es = Executors.newSingleThreadExecutor(new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        return new Thread(r, "Stack Version Loading Thread");
-      }
-    });
-
-    List<LatestRepoCallable> lookupList = new ArrayList<LatestRepoCallable>();
-
-    for (StackInfo stack : stacks) {
-      LOG.debug("Adding new stack to known stacks"
-        + ", stackName = " + stack.getName()
-        + ", stackVersion = " + stack.getVersion());
-
-      stacksResult.add(stack);
-
-      String stackPath = stackRootAbsPath + File.separator +
-              stack.getName() + File.separator + stack.getVersion();
-
-      // get repository data for current stack of techs
-      File repositoryFolder = new File(stackPath
-        + File.separator + REPOSITORY_FOLDER_NAME + File.separator
-        + REPOSITORY_FILE_NAME);
-
-      if (repositoryFolder.exists()) {
-        LOG.debug("Adding repositories to stack"
-          + ", stackName=" + stack.getName()
-          + ", stackVersion=" + stack.getVersion()
-          + ", repoFolder=" + repositoryFolder.getPath());
-
-        List<RepositoryInfo> repositoryInfoList = getRepository(repositoryFolder,
-            stack, lookupList);
-
-        stack.getRepositories().addAll(repositoryInfoList);
-      } else {
-        LOG.warn("No repository information defined for "
-          + ", stackName=" + stack.getName()
-          + ", stackVersion=" + stack.getVersion()
-          + ", repoFolder=" + repositoryFolder.getPath());
-      }
-
-      // Populate services
-      List<ServiceInfo> services = stackExtensionHelper.getAllApplicableServices(stack);
-      stack.setServices(services);
-
-      Map<String, Map<String, PropertyInfo>> stackRequiredProps = new HashMap<String, Map<String, PropertyInfo>>();
-      requiredProperties.put(new StackId(stack.getName(), stack.getVersion()), stackRequiredProps);
-      for (ServiceInfo service : services) {
-        // Set required config properties
-        stackRequiredProps.put(service.getName(), getAllRequiredProperties(service));
-      }
-
-      // Resolve hooks folder
-      String stackHooksToUse = stackExtensionHelper.resolveHooksFolder(stack);
-      stack.setStackHooksFolder(stackHooksToUse);
-
-      File upgradesFolder = new File(stackPath + File.separator + UPGRADE_PACK_FOLDER_NAME);
-      if (upgradesFolder.exists() && upgradesFolder.isDirectory()) {
-        stack.setUpgradesFolder(upgradesFolder.getAbsolutePath());
-      }
-    }
-
-    es.invokeAll(lookupList);
-
-    es.shutdown();
-  }
-
-  /**
-   * Get properties with require_input attribute set to true.
-   *
-   * @param stackName     name of the stack, e.g.: HDP
-   * @param stackVersion  version of the stack
-   * @return Map of property name to PropertyInfo
-   */
-  public Map<String, PropertyInfo> getRequiredProperties(String stackName, String stackVersion, String service) {
-
-    Map<String, Map<String, PropertyInfo>> requiredStackProps =
-        requiredProperties.get(new StackId(stackName, stackVersion));
-
-    if (requiredStackProps != null) {
-      Map<String, PropertyInfo> requiredServiceProperties = requiredStackProps.get(service);
-      return requiredServiceProperties == null ? Collections.<String, PropertyInfo>emptyMap() :
-                                                 requiredServiceProperties;
-    }
-    return Collections.emptyMap();
-  }
-
   public String getServerVersion() {
     return serverVersion;
   }
 
-  private List<RepositoryInfo> getRepository(File repositoryFile, StackInfo stack,
-      List<LatestRepoCallable> lookupList)
-      throws JAXBException {
-
-    RepositoryXml rxml = StackExtensionHelper.unmarshal(RepositoryXml.class, repositoryFile);
-
-    List<RepositoryInfo> list = new ArrayList<RepositoryInfo>();
-
-    for (Os o : rxml.getOses()) {
-      for (String os : o.getFamily().split(",")) {
-        for (Repo r : o.getRepos()) {
-          RepositoryInfo ri = new RepositoryInfo();
-          ri.setBaseUrl(r.getBaseUrl());
-          ri.setDefaultBaseUrl(r.getBaseUrl());
-          ri.setMirrorsList(r.getMirrorsList());
-          ri.setOsType(os.trim());
-          ri.setRepoId(r.getRepoId());
-          ri.setRepoName(r.getRepoName());
-          ri.setLatestBaseUrl(r.getBaseUrl());
-
-          if (null != metainfoDAO) {
-            LOG.debug("Checking for override for base_url");
-            String key = generateRepoMetaKey(r.getRepoName(), stack.getVersion(),
-                o.getFamily(), r.getRepoId(), REPOSITORY_XML_PROPERTY_BASEURL);
-            MetainfoEntity entity = metainfoDAO.findByKey(key);
-            if (null != entity) {
-              ri.setBaseUrl(entity.getMetainfoValue());
-            }
-          }
-
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Adding repo to stack"
-                + ", repoInfo=" + ri.toString());
-          }
-
-          list.add(ri);
-        }
-      }
-    }
-
-    if (null != rxml.getLatestURI() && list.size() > 0) {
-      lookupList.add(new LatestRepoCallable(rxml.getLatestURI(),
-          repositoryFile.getParentFile(), stack, os_family));
-    }
-
-    return list;
-
-  }
-
   public boolean isOsSupported(String osType) {
     return ALL_SUPPORTED_OS.contains(osType);
   }
@@ -1102,7 +745,7 @@ public class AmbariMetaInfo {
 
     ri.setBaseUrl(newBaseUrl);
 
-    if (null != metainfoDAO) {
+    if (null != metaInfoDAO) {
       String metaKey = generateRepoMetaKey(stackName, stackVersion, osType,
           repoId, REPOSITORY_XML_PROPERTY_BASEURL);
 
@@ -1111,9 +754,9 @@ public class AmbariMetaInfo {
       entity.setMetainfoValue(newBaseUrl);
 
       if (null != ri.getDefaultBaseUrl() && newBaseUrl.equals(ri.getDefaultBaseUrl())) {
-        metainfoDAO.remove(entity);
+        metaInfoDAO.remove(entity);
       } else {
-        metainfoDAO.merge(entity);
+        metaInfoDAO.merge(entity);
       }
     }
   }
@@ -1169,23 +812,6 @@ public class AmbariMetaInfo {
   }
 
   /**
-   * Get all required properties for the given service.
-   *
-   * @param service  associated service
-   * @return map of property name to PropertyInfo containing all required properties for service
-   */
-  private Map<String, PropertyInfo> getAllRequiredProperties(ServiceInfo service) {
-    Map<String, PropertyInfo> requiredProperties = new HashMap<String, PropertyInfo>();
-    List<PropertyInfo> properties = service.getProperties();
-    for (PropertyInfo propertyInfo : properties) {
-      if (propertyInfo.isRequireInput()) {
-        requiredProperties.put(propertyInfo.getName(), propertyInfo);
-      }
-    }
-    return requiredProperties;
-  }
-
-  /**
    * Gets the alert definitions for the specified stack and service.
    *
    * @param stackName
@@ -1199,20 +825,15 @@ public class AmbariMetaInfo {
    * @throws AmbariException
    */
   public Set<AlertDefinition> getAlertDefinitions(String stackName, String stackVersion,
-      String serviceName) throws AmbariException {
+                                                  String serviceName) throws AmbariException {
 
     ServiceInfo svc = getService(stackName, stackVersion, serviceName);
     return getAlertDefinitions(svc);
   }
-
   /**
    * Gets the alert definitions for the specified stack and service.
    *
-   * @param stackName
-   *          the stack name
-   * @param stackVersion
-   *          the stack version
-   * @param serviceName
+   * @param service
    *          the service name
    * @return the alert definitions for a stack or an empty list if none (never
    *         {@code null}).
@@ -1243,7 +864,7 @@ public class AmbariMetaInfo {
    * definitions that should be run on agent hosts but are not associated with a
    * service.
    *
-   * @param clusters
+   * @param clusters all clusters
    * @throws AmbariException
    */
   public void reconcileAlertDefinitions(Clusters clusters)
@@ -1258,11 +879,11 @@ public class AmbariMetaInfo {
     for (Cluster cluster : clusterMap.values()) {
       long clusterId = cluster.getClusterId();
       StackId stackId = cluster.getDesiredStackVersion();
-      StackInfo stackInfo = getStackInfo(stackId.getStackName(),
+      StackInfo stackInfo = getStack(stackId.getStackName(),
           stackId.getStackVersion());
 
       // creating a mapping between service name and service for fast lookups
-      List<ServiceInfo> stackServices = stackInfo.getServices();
+      Collection<ServiceInfo> stackServices = stackInfo.getServices();
       Map<String, ServiceInfo> stackServiceMap = new HashMap<String, ServiceInfo>();
       for (ServiceInfo stackService : stackServices) {
         stackServiceMap.put(stackService.getName(), stackService);
@@ -1318,8 +939,6 @@ public class AmbariMetaInfo {
           LOG.debug(
               "The alert named {} has been modified from the stack definition and will not be merged",
               stackDefinition.getName());
-
-          continue;
         }
       }
 
@@ -1357,54 +976,22 @@ public class AmbariMetaInfo {
   }
 
   /**
-   * Gets upgrade packs available for a stack.
+   * Get all upgrade packs available for a stack.
+   *
    * @param stackName the stack name
    * @param stackVersion the stack version
    * @return a map of upgrade packs, keyed by the name of the upgrade pack
    */
   public Map<String, UpgradePack> getUpgradePacks(String stackName, String stackVersion) {
-    StackInfo stack = null;
     try {
-      stack = getStackInfo(stackName, stackVersion);
-    } catch (AmbariException e) {
-      LOG.debug("Cannot load upgrade packs for non-existent stack {}-{}",
-          stackName, stackVersion, e);
-      return Collections.emptyMap();
-    }
-
-    File folder = new File(stack.getUpgradesFolder());
-    if (!folder.exists() || !folder.isDirectory()) {
-      LOG.error("Upgrades folder {} no longer exists", stack.getUpgradesFolder());
-      return Collections.emptyMap();
-    }
+      StackInfo stack = getStack(stackName, stackVersion);
+      return stack.getUpgradePacks() == null ?
+          Collections.<String, UpgradePack>emptyMap() : stack.getUpgradePacks();
 
-    String[] fileNames = folder.list(new FilenameFilter() {
-      @Override
-      public boolean accept(File folder, String fileName) {
-        if (fileName.toLowerCase().endsWith(".xml")) {
-          return true;
-        } else {
-          return false;
-        }
-      }
-    });
-
-
-    Map<String, UpgradePack> packs = new HashMap<String, UpgradePack>();
-
-    for (String fileName : fileNames) {
-      File f = new File(folder, fileName);
-
-      String packName = FilenameUtils.removeExtension(fileName);
-
-      try {
-        UpgradePack up = StackExtensionHelper.unmarshal(UpgradePack.class, f);
-        packs.put(packName, up);
-      } catch (Exception e) {
-        LOG.error("Could not parse {} into an upgrade pack", f.getAbsolutePath());
-      }
+    } catch (AmbariException e) {
+      LOG.debug("Cannot load upgrade packs for non-existent stack {}-{}", stackName, stackVersion, e);
     }
 
-    return packs;
+    return Collections.emptyMap();
   }
 }