Explorar o código

AMBARI-24651. Add cluster and stack settings properties to agent STOMP updates. (mpapirkovskyy)

Myroslav Papirkovskyi %!s(int64=7) %!d(string=hai) anos
pai
achega
a520f0e831
Modificáronse 26 ficheiros con 285 adicións e 504 borrados
  1. 5 1
      ambari-agent/src/main/python/ambari_agent/ConfigurationBuilder.py
  2. 0 23
      ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
  3. 0 178
      ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java
  4. 101 4
      ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/HostLevelParamsHolder.java
  5. 3 0
      ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/MetadataHolder.java
  6. 2 0
      ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/TopologyHolder.java
  7. 11 1
      ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/dto/HostLevelParamsCluster.java
  8. 38 6
      ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/dto/MetadataCluster.java
  9. 15 0
      ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/dto/TopologyComponent.java
  10. 0 6
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
  11. 0 10
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementController.java
  12. 1 97
      ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
  13. 2 1
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java
  14. 0 9
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradePlanInstallResourceProvider.java
  15. 2 1
      ambari-server/src/main/java/org/apache/ambari/server/events/MetadataUpdateEvent.java
  16. 31 3
      ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/MpackInstallStateListener.java
  17. 14 59
      ambari-server/src/main/java/org/apache/ambari/server/metadata/ClusterMetadataGenerator.java
  18. 3 3
      ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
  19. 7 0
      ambari-server/src/main/java/org/apache/ambari/server/state/ServiceComponentHost.java
  20. 1 0
      ambari-server/src/main/java/org/apache/ambari/server/state/ServiceGroupImpl.java
  21. 26 3
      ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
  22. 5 0
      ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
  23. 1 3
      ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java
  24. 6 6
      ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatMonitor.java
  25. 11 10
      ambari-server/src/test/java/org/apache/ambari/server/agent/stomp/HostLevelParamsHolderTest.java
  26. 0 80
      ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java

+ 5 - 1
ambari-agent/src/main/python/ambari_agent/ConfigurationBuilder.py

@@ -39,6 +39,7 @@ class ConfigurationBuilder:
 
       command_dict = {
         'clusterLevelParams': metadata_cache.clusterLevelParams,
+        'clusterSettings': metadata_cache.clusterSettings,
         'hostLevelParams': host_level_params_cache,
         'clusterHostInfo': self.topology_cache.get_cluster_host_info(cluster_id),
         'localComponents': self.topology_cache.get_cluster_local_components(cluster_id),
@@ -57,9 +58,12 @@ class ConfigurationBuilder:
 
       component_dict = self.topology_cache.get_component_info_by_key(cluster_id, service_name, component_name)
       if component_dict is not None:
+        mpack_id = component_dict.desiredMpackId
+        stack_settings = host_level_params_cache.stacksSettings[str(mpack_id)]
         command_dict.update({
           'componentLevelParams': component_dict.componentLevelParams,
-          'commandParams': component_dict.commandParams
+          'commandParams': component_dict.commandParams,
+          'stackSettings': stack_settings
         })
 
       command_dict.update(configurations_cache)

+ 0 - 23
ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java

@@ -61,12 +61,6 @@ public class ExecutionCommand extends AgentCommand {
   @JsonProperty("clusterName")
   private String clusterName;
 
-  @SerializedName("clusterSettings")
-  private Map<String, String> clusterSettings;
-
-  @SerializedName("stackSettings")
-  private Map<String, String> stackSettings;
-
   @SerializedName("requestId")
   @JsonProperty("requestId")
   private long requestId;
@@ -328,23 +322,6 @@ public class ExecutionCommand extends AgentCommand {
     hostLevelParams = params;
   }
 
-  public Map<String, String> getClusterSettings() {
-    return clusterSettings;
-  }
-
-  public void setClusterSettings(Map<String, String> clusterSettings) {
-    this.clusterSettings = clusterSettings;
-  }
-
-
-  public Map<String, String> getStackSettings() {
-    return stackSettings;
-  }
-
-  public void setStackSettings(Map<String, String> stackSettings) {
-    this.stackSettings = stackSettings;
-  }
-
   public Map<String, Set<String>> getClusterHostInfo() {
     return clusterHostInfo;
   }

+ 0 - 178
ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java

@@ -17,25 +17,9 @@
  */
 package org.apache.ambari.server.agent;
 
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMMAND_TIMEOUT;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.HOOKS_FOLDER;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JDK_LOCATION;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SCRIPT;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SCRIPT_TYPE;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SERVICE_PACKAGE_FOLDER;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.STACK_NAME;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.STACK_VERSION;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
 
 import org.apache.ambari.server.AmbariException;
-import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.configuration.Configuration;
@@ -44,19 +28,12 @@ import org.apache.ambari.server.events.MessageNotDelivered;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
-import org.apache.ambari.server.state.CommandScriptDefinition;
-import org.apache.ambari.server.state.ComponentInfo;
-import org.apache.ambari.server.state.Config;
 import org.apache.ambari.server.state.ConfigHelper;
-import org.apache.ambari.server.state.DesiredConfig;
 import org.apache.ambari.server.state.Host;
 import org.apache.ambari.server.state.HostState;
 import org.apache.ambari.server.state.Service;
 import org.apache.ambari.server.state.ServiceComponent;
 import org.apache.ambari.server.state.ServiceComponentHost;
-import org.apache.ambari.server.state.ServiceInfo;
-import org.apache.ambari.server.state.StackId;
-import org.apache.ambari.server.state.StackInfo;
 import org.apache.ambari.server.state.State;
 import org.apache.ambari.server.state.fsm.InvalidStateTransitionException;
 import org.apache.ambari.server.state.host.HostHeartbeatLostEvent;
@@ -171,161 +148,6 @@ public class HeartbeatMonitor implements Runnable {
     }
   }
 
-  /**
-   * @param hostname
-   * @return list of commands to get status of service components on a concrete host
-   */
-  public List<StatusCommand> generateStatusCommands(String hostname) throws AmbariException {
-    List<StatusCommand> cmds = new ArrayList<>();
-
-    for (Cluster cl : clusters.getClustersForHost(hostname)) {
-      Map<String, DesiredConfig> desiredConfigs = cl.getDesiredConfigs();
-      for (ServiceComponentHost sch : cl.getServiceComponentHosts(hostname)) {
-        switch (sch.getState()) {
-          case INIT:
-          case INSTALLING:
-          case STARTING:
-          case STOPPING:
-            //don't send commands until component is installed at least
-            continue;
-          default:
-            StatusCommand statusCmd = createStatusCommand(hostname, cl, sch, desiredConfigs);
-            cmds.add(statusCmd);
-        }
-
-      }
-    }
-    return cmds;
-  }
-
-  /**
-   * Generates status command and fills all appropriate fields.
-   * @throws AmbariException
-   */
-  private StatusCommand createStatusCommand(String hostname, Cluster cluster,
-      ServiceComponentHost sch, Map<String, DesiredConfig> desiredConfigs) throws AmbariException {
-    String serviceName = sch.getServiceName();
-    String serviceType = sch.getServiceType();
-    String componentName = sch.getServiceComponentName();
-
-    StackId stackId = sch.getDesiredStackId();
-
-    ServiceInfo serviceInfo = ambariMetaInfo.getService(stackId.getStackName(),
-        stackId.getStackVersion(), serviceType);
-    ComponentInfo componentInfo = ambariMetaInfo.getComponent(
-            stackId.getStackName(), stackId.getStackVersion(),
-            serviceType, componentName);
-    StackInfo stackInfo = ambariMetaInfo.getStack(stackId.getStackName(),
-        stackId.getStackVersion());
-
-    Map<String, Map<String, String>> configurations = new TreeMap<>();
-    Map<String, Map<String,  Map<String, String>>> configurationAttributes = new TreeMap<>();
-
-    // get the cluster config for type '*-env'
-    // apply config group overrides
-    //Config clusterConfig = cluster.getDesiredConfigByType(GLOBAL);
-    Collection<Config> clusterConfigs = cluster.getAllConfigs();
-
-    // creating list with desired config types to validate if cluster config actual
-    Set<String> desiredConfigTypes = desiredConfigs.keySet();
-
-    // Apply global properties for this host from all config groups
-    Map<String, Map<String, String>> allConfigTags = configHelper
-        .getEffectiveDesiredTags(cluster, hostname);
-
-    for(Config clusterConfig: clusterConfigs) {
-      String configType = clusterConfig.getType();
-      if(!configType.endsWith("-env") || !desiredConfigTypes.contains(configType)) {
-        continue;
-      }
-
-      // cluster config for 'global'
-      Map<String, String> props = new HashMap<>(clusterConfig.getProperties());
-
-      Map<String, Map<String, String>> configTags = new HashMap<>();
-
-      for (Map.Entry<String, Map<String, String>> entry : allConfigTags.entrySet()) {
-        if (entry.getKey().equals(clusterConfig.getType())) {
-          configTags.put(clusterConfig.getType(), entry.getValue());
-        }
-      }
-
-      Map<String, Map<String, String>> properties = configHelper
-              .getEffectiveConfigProperties(cluster, configTags);
-
-      if (!properties.isEmpty()) {
-        for (Map<String, String> propertyMap : properties.values()) {
-          props.putAll(propertyMap);
-        }
-      }
-
-      configurations.put(clusterConfig.getType(), props);
-
-      Map<String, Map<String, String>> attrs = new TreeMap<>();
-      configHelper.cloneAttributesMap(clusterConfig.getPropertiesAttributes(), attrs);
-
-      Map<String, Map<String, Map<String, String>>> attributes = configHelper
-          .getEffectiveConfigAttributes(cluster, configTags);
-      for (Map<String, Map<String, String>> attributesMap : attributes.values()) {
-        configHelper.cloneAttributesMap(attributesMap, attrs);
-      }
-      configurationAttributes.put(clusterConfig.getType(), attrs);
-    }
-
-    StatusCommand statusCmd = new StatusCommand();
-    statusCmd.setClusterName(cluster.getClusterName());
-    statusCmd.setServiceName(serviceName);
-    statusCmd.setServiceType(serviceType);
-    statusCmd.setComponentName(componentName);
-    statusCmd.setConfigurations(configurations);
-    statusCmd.setConfigurationAttributes(configurationAttributes);
-    statusCmd.setHostname(hostname);
-
-    // If Agent wants the command and the States differ
-    statusCmd.setDesiredState(sch.getDesiredState());
-    statusCmd.setHasStaleConfigs(configHelper.isStaleConfigs(sch, desiredConfigs));
-    if (getAgentRequests().shouldSendExecutionDetails(hostname, componentName)) {
-      LOG.info(componentName + " is at " + sch.getState() + " adding more payload per agent ask");
-      statusCmd.setPayloadLevel(StatusCommand.StatusCommandPayload.EXECUTION_COMMAND);
-    }
-
-    // Fill command params
-    Map<String, String> commandParams = statusCmd.getCommandParams();
-
-    String commandTimeout = configuration.getDefaultAgentTaskTimeout(false);
-    CommandScriptDefinition script = componentInfo.getCommandScript();
-    if (serviceInfo.getSchemaVersion().equals(AmbariMetaInfo.SCHEMA_VERSION_2)) {
-      if (script != null) {
-        commandParams.put(SCRIPT, script.getScript());
-        commandParams.put(SCRIPT_TYPE, script.getScriptType().toString());
-        if (script.getTimeout() > 0) {
-          commandTimeout = String.valueOf(script.getTimeout());
-        }
-      } else {
-        String message = String.format("Component %s of service %s has not " +
-                "command script defined", componentName, serviceName);
-        throw new AmbariException(message);
-      }
-    }
-    commandParams.put(COMMAND_TIMEOUT, commandTimeout);
-    commandParams.put(SERVICE_PACKAGE_FOLDER,
-       serviceInfo.getServicePackageFolder());
-    commandParams.put(HOOKS_FOLDER, configuration.getProperty(Configuration.HOOKS_FOLDER));
-    // Fill host level params
-    Map<String, String> hostLevelParams = statusCmd.getHostLevelParams();
-    hostLevelParams.put(JDK_LOCATION, ambariManagementController.getJdkResourceUrl());
-    hostLevelParams.put(STACK_NAME, stackId.getStackName());
-    hostLevelParams.put(STACK_VERSION, stackId.getStackVersion());
-
-    if (statusCmd.getPayloadLevel() == StatusCommand.StatusCommandPayload.EXECUTION_COMMAND) {
-      ExecutionCommand ec = ambariManagementController.getExecutionCommand(cluster, sch, RoleCommand.START);
-      statusCmd.setExecutionCommand(ec);
-      LOG.debug("{} has more payload for execution command", componentName);
-    }
-
-    return statusCmd;
-  }
-
   private void handleHeartbeatLost(Long hostId) throws AmbariException, InvalidStateTransitionException {
     Host hostObj = clusters.getHostById(hostId);
     String host = hostObj.getHostName();

+ 101 - 4
ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/HostLevelParamsHolder.java

@@ -17,10 +17,19 @@
  */
 package org.apache.ambari.server.agent.stomp;
 
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.DFS_TYPE;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.GROUP_LIST;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.NOT_MANAGED_HDFS_PATH_LIST;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.STACK_VERSION;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.USER_GROUPS;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.USER_LIST;
+
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.TreeSet;
 
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.agent.CommandRepository;
@@ -28,20 +37,30 @@ import org.apache.ambari.server.agent.RecoveryConfig;
 import org.apache.ambari.server.agent.RecoveryConfigHelper;
 import org.apache.ambari.server.agent.stomp.dto.HostLevelParamsCluster;
 import org.apache.ambari.server.agent.stomp.dto.HostRepositories;
+import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.events.HostLevelParamsUpdateEvent;
 import org.apache.ambari.server.events.MaintenanceModeEvent;
 import org.apache.ambari.server.events.ServiceComponentRecoveryChangedEvent;
 import org.apache.ambari.server.events.ServiceGroupMpackChangedEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
+import org.apache.ambari.server.mpack.MpackManager;
+import org.apache.ambari.server.orm.entities.MpackHostStateEntity;
 import org.apache.ambari.server.state.BlueprintProvisioningState;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.state.ConfigHelper;
+import org.apache.ambari.server.state.DesiredConfig;
 import org.apache.ambari.server.state.Host;
+import org.apache.ambari.server.state.Mpack;
+import org.apache.ambari.server.state.PropertyInfo;
 import org.apache.ambari.server.state.ServiceComponentHost;
+import org.apache.ambari.server.state.ServiceInfo;
+import org.apache.ambari.server.state.StackId;
 import org.apache.commons.collections.MapUtils;
 
 import com.google.common.eventbus.Subscribe;
+import com.google.gson.Gson;
 import com.google.inject.Inject;
 import com.google.inject.Provider;
 import com.google.inject.Singleton;
@@ -59,8 +78,20 @@ public class HostLevelParamsHolder extends AgentHostDataHolder<HostLevelParamsUp
   private Provider<AmbariManagementController> m_ambariManagementController;
 
   @Inject
-  public HostLevelParamsHolder(AmbariEventPublisher ambariEventPublisher) {
+  private Gson gson;
+
+  @Inject
+  private ConfigHelper configHelper;
+
+  private final AmbariMetaInfo ambariMetaInfo;
+
+  private final MpackManager mpackManager;
+
+  @Inject
+  public HostLevelParamsHolder(AmbariEventPublisher ambariEventPublisher, AmbariMetaInfo ambariMetaInfo) {
     ambariEventPublisher.register(this);
+    this.ambariMetaInfo = ambariMetaInfo;
+    this.mpackManager = ambariMetaInfo.getMpackManager();
   }
 
   @Override
@@ -78,7 +109,8 @@ public class HostLevelParamsHolder extends AgentHostDataHolder<HostLevelParamsUp
       HostLevelParamsCluster hostLevelParamsCluster = new HostLevelParamsCluster(
           m_ambariManagementController.get().retrieveHostRepositories(cl, host),
           recoveryConfigHelper.getRecoveryConfig(cl.getClusterName(), host.getHostName()),
-          m_ambariManagementController.get().getBlueprintProvisioningStates(cl.getClusterId(), host.getHostId()));
+          m_ambariManagementController.get().getBlueprintProvisioningStates(cl.getClusterId(), host.getHostId()),
+          getHostStacksSettings(cl, host));
 
       hostLevelParamsClusters.put(Long.toString(cl.getClusterId()),
           hostLevelParamsCluster);
@@ -117,6 +149,7 @@ public class HostLevelParamsHolder extends AgentHostDataHolder<HostLevelParamsUp
           SortedMap<Long, CommandRepository> mergedRepositories;
           Map<String, BlueprintProvisioningState> mergedBlueprintProvisioningStates;
           SortedMap<String, Long> mergedComponentRepos;
+          SortedMap<Long, SortedMap<String, String>> mergedStacksSettings;
           if (!currentCluster.getRecoveryConfig().equals(updatedCluster.getRecoveryConfig())) {
             mergedRecoveryConfig = updatedCluster.getRecoveryConfig();
             clusterChanged = true;
@@ -144,11 +177,19 @@ public class HostLevelParamsHolder extends AgentHostDataHolder<HostLevelParamsUp
           } else {
             mergedComponentRepos = currentCluster.getHostRepositories().getComponentRepos();
           }
+          if (!currentCluster.getStacksSettings()
+              .equals(updatedCluster.getStacksSettings())) {
+            mergedStacksSettings = updatedCluster.getStacksSettings();
+            clusterChanged = true;
+          } else {
+            mergedStacksSettings = currentCluster.getStacksSettings();
+          }
           if (clusterChanged) {
             HostLevelParamsCluster mergedCluster = new HostLevelParamsCluster(
                 new HostRepositories(mergedRepositories, mergedComponentRepos),
                 mergedRecoveryConfig,
-                mergedBlueprintProvisioningStates);
+                mergedBlueprintProvisioningStates,
+                mergedStacksSettings);
             mergedClusters.put(clusterId, mergedCluster);
             changed = true;
           } else {
@@ -196,7 +237,8 @@ public class HostLevelParamsHolder extends AgentHostDataHolder<HostLevelParamsUp
             new HostLevelParamsCluster(
                     m_ambariManagementController.get().retrieveHostRepositories(cluster, host),
                     recoveryConfigHelper.getRecoveryConfig(cluster.getClusterName(), host.getHostName()),
-                    m_ambariManagementController.get().getBlueprintProvisioningStates(clusterId, host.getHostId())));
+                    m_ambariManagementController.get().getBlueprintProvisioningStates(clusterId, host.getHostId()),
+                    getHostStacksSettings(cluster, host)));
     updateData(hostLevelParamsUpdateEvent);
   }
 
@@ -214,4 +256,59 @@ public class HostLevelParamsHolder extends AgentHostDataHolder<HostLevelParamsUp
       }
     }
   }
+
+  public SortedMap<Long, SortedMap<String, String>> getHostStacksSettings(Cluster cl, Host host) throws AmbariException {
+    SortedMap<Long, SortedMap<String, String>> stacksSettings = new TreeMap<>();
+    for (MpackHostStateEntity mpackHostStateEntity : host.getMPackInstallStates()) {
+      Long mpackId = mpackHostStateEntity.getMpack().getId();
+      Mpack mpack = mpackManager.getMpackMap().get(mpackId);
+      if (mpack == null) {
+        throw new AmbariException(String.format("No mpack with id %s found", mpackId));
+      }
+      stacksSettings.put(mpackId, new TreeMap<>(getStackSettings(cl, mpack.getStackId())));
+    }
+    return stacksSettings;
+  }
+
+
+  private SortedMap<String, String> getStackSettings(Cluster cluster, StackId stackId) throws AmbariException {
+    SortedMap<String, String> stackLevelParams = new TreeMap<>(ambariMetaInfo.getStackSettingsNameValueMap(stackId));
+
+    // STACK_NAME is part of stack settings, but STACK_VERSION is not
+    stackLevelParams.put(STACK_VERSION, stackId.getStackVersion());
+
+    Map<String, DesiredConfig> clusterDesiredConfigs = cluster.getDesiredConfigs(false);
+    Set<PropertyInfo> stackProperties = ambariMetaInfo.getStackProperties(stackId.getStackName(), stackId.getStackVersion());
+    Map<String, ServiceInfo> servicesMap = ambariMetaInfo.getServices(stackId.getStackName(), stackId.getStackVersion());
+    Set<PropertyInfo> clusterProperties = ambariMetaInfo.getClusterProperties();
+
+    Map<PropertyInfo, String> users = configHelper.getPropertiesWithPropertyType(PropertyInfo.PropertyType.USER, cluster, clusterDesiredConfigs, servicesMap, stackProperties, clusterProperties);
+    Set<String> userSet = new TreeSet<>(users.values());
+    String userList = gson.toJson(userSet);
+    stackLevelParams.put(USER_LIST, userList);
+
+    Map<PropertyInfo, String> groups = configHelper.getPropertiesWithPropertyType(PropertyInfo.PropertyType.GROUP, cluster, clusterDesiredConfigs, servicesMap, stackProperties, clusterProperties);
+    Set<String> groupSet = new TreeSet<>(groups.values());
+    String groupList = gson.toJson(groupSet);
+    stackLevelParams.put(GROUP_LIST, groupList);
+
+    Map<String, Set<String>> userGroupsMap = configHelper.createUserGroupsMap(users, groups);
+    String userGroups = gson.toJson(userGroupsMap);
+    stackLevelParams.put(USER_GROUPS, userGroups);
+
+    Map<PropertyInfo, String> notManagedHdfsPathMap = configHelper.getPropertiesWithPropertyType(PropertyInfo.PropertyType.NOT_MANAGED_HDFS_PATH, cluster, clusterDesiredConfigs, servicesMap, stackProperties, clusterProperties);
+    Set<String> notManagedHdfsPathSet = configHelper.filterInvalidPropertyValues(notManagedHdfsPathMap, NOT_MANAGED_HDFS_PATH_LIST);
+    String notManagedHdfsPathList = gson.toJson(notManagedHdfsPathSet);
+    stackLevelParams.put(NOT_MANAGED_HDFS_PATH_LIST, notManagedHdfsPathList);
+
+    Map<String, ServiceInfo> serviceInfos = ambariMetaInfo.getServices(stackId.getStackName(), stackId.getStackVersion());
+    for (ServiceInfo serviceInfoInstance : serviceInfos.values()) {
+      if (serviceInfoInstance.getServiceType() != null) {
+        stackLevelParams.put(DFS_TYPE, serviceInfoInstance.getServiceType());
+        break;
+      }
+    }
+
+    return stackLevelParams;
+  }
 }

+ 3 - 0
ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/MetadataHolder.java

@@ -92,6 +92,9 @@ public class MetadataHolder extends AgentClusterDataHolder<MetadataUpdateEvent>
             if (cluster.updateServiceLevelParams(updatedCluster.getServiceLevelParams(), updatedCluster.isFullServiceLevelMetadata())) {
               changed = true;
             }
+            if (cluster.updateClusterSettings(updatedCluster.getClusterSettings())) {
+              changed = true;
+            }
             if (CollectionUtils.isNotEmpty(updatedCluster.getStatusCommandsToRun())
                 && !cluster.getStatusCommandsToRun().containsAll(updatedCluster.getStatusCommandsToRun())) {
               cluster.getStatusCommandsToRun().addAll(updatedCluster.getStatusCommandsToRun());

+ 2 - 0
ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/TopologyHolder.java

@@ -108,6 +108,7 @@ public class TopologyHolder extends AgentClusterDataHolder<TopologyUpdateEvent>
                 .setServiceGroupName(serviceGroupName)
                 .setVersion(sch.getVersion())
                 .setMpackVersion(sch.getMpackVersion())
+                .setDesiredMpackId(sch.getDesiredMpackId())
                 .setHostIds(hostOrderIds)
                 .setComponentLevelParams(ambariManagementController.getTopologyComponentLevelParams(sch))
                 .setCommandParams(ambariManagementController.getTopologyCommandParams(sch))
@@ -228,6 +229,7 @@ public class TopologyHolder extends AgentClusterDataHolder<TopologyUpdateEvent>
               .setServiceGroupName(sch.getServiceGroupName())
               .setVersion(sch.getVersion())
               .setMpackVersion(sch.getMpackVersion())
+              .setDesiredMpackId(sch.getDesiredMpackId())
               .setCommandParams(ambariManagementController.getTopologyCommandParams(sch))
               .setComponentLevelParams(ambariManagementController.getTopologyComponentLevelParams(sch))
               .build();

+ 11 - 1
ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/dto/HostLevelParamsCluster.java

@@ -19,6 +19,7 @@ package org.apache.ambari.server.agent.stomp.dto;
 
 
 import java.util.Map;
+import java.util.SortedMap;
 
 import org.apache.ambari.server.agent.RecoveryConfig;
 import org.apache.ambari.server.state.BlueprintProvisioningState;
@@ -38,11 +39,16 @@ public class HostLevelParamsCluster {
   @JsonProperty("blueprint_provisioning_state")
   private Map<String, BlueprintProvisioningState> blueprintProvisioningState;
 
+  @JsonProperty("stacksSettings")
+  private SortedMap<Long, SortedMap<String, String>> stacksSettings;
+
   public HostLevelParamsCluster(HostRepositories hostRepositories, RecoveryConfig recoveryConfig,
-                                Map<String, BlueprintProvisioningState> blueprintProvisioningState) {
+        Map<String, BlueprintProvisioningState> blueprintProvisioningState,
+                                SortedMap<Long, SortedMap<String, String>> stacksSettings) {
     this.hostRepositories = hostRepositories;
     this.recoveryConfig = recoveryConfig;
     this.blueprintProvisioningState = blueprintProvisioningState;
+    this.stacksSettings = stacksSettings;
   }
 
   public HostRepositories getHostRepositories() {
@@ -53,6 +59,10 @@ public class HostLevelParamsCluster {
     return recoveryConfig;
   }
 
+  public SortedMap<Long, SortedMap<String, String>> getStacksSettings() {
+    return stacksSettings;
+  }
+
   public Map<String, BlueprintProvisioningState> getBlueprintProvisioningState() {
     return blueprintProvisioningState;
   }

+ 38 - 6
ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/dto/MetadataCluster.java

@@ -40,10 +40,12 @@ public class MetadataCluster {
   private final boolean fullServiceLevelMetadata; //this is true in case serviceLevelParams has all parameters for all services
   private SortedMap<String, MetadataServiceInfo> serviceLevelParams;
   private SortedMap<String, String> clusterLevelParams;
+  private SortedMap<String, String> clusterSettings;
   private SortedMap<String, SortedMap<String,String>> agentConfigs;
 
   public MetadataCluster(SecurityType securityType, SortedMap<String,MetadataServiceInfo> serviceLevelParams, boolean fullServiceLevelMetadata,
-                         SortedMap<String, String> clusterLevelParams, SortedMap<String, SortedMap<String,String>> agentConfigs) {
+                         SortedMap<String, String> clusterLevelParams, SortedMap<String, SortedMap<String,String>> agentConfigs,
+                         SortedMap<String, String> clusterSettings) {
     this.statusCommandsToRun  = new HashSet<>();
     if (securityType != null) {
       this.statusCommandsToRun.add("STATUS");
@@ -52,19 +54,28 @@ public class MetadataCluster {
     this.serviceLevelParams = serviceLevelParams;
     this.clusterLevelParams = clusterLevelParams;
     this.agentConfigs = agentConfigs;
+    this.clusterSettings = clusterSettings;
   }
 
   public static MetadataCluster emptyMetadataCluster() {
-    return new MetadataCluster(null, null, false, null, null);
+    return new MetadataCluster(null, null,
+        false, null, null, null);
   }
 
   public static MetadataCluster serviceLevelParamsMetadataCluster(SecurityType securityType, SortedMap<String, MetadataServiceInfo> serviceLevelParams,
       boolean fullServiceLevelMetadata) {
-    return new MetadataCluster(securityType, serviceLevelParams, fullServiceLevelMetadata, null, null);
+    return new MetadataCluster(securityType, serviceLevelParams, fullServiceLevelMetadata, null,
+        null, null);
   }
 
   public static MetadataCluster clusterLevelParamsMetadataCluster(SecurityType securityType, SortedMap<String, String> clusterLevelParams) {
-    return new MetadataCluster(securityType, null, false, clusterLevelParams, null);
+    return new MetadataCluster(securityType, null, false, clusterLevelParams,
+        null, null);
+  }
+
+  public static MetadataCluster clusterSettingsMetadataCluster(SecurityType securityType, SortedMap<String, String> clusterSettings) {
+    return new MetadataCluster(securityType, null, false, null,
+        null, clusterSettings);
   }
 
   public Set<String> getStatusCommandsToRun() {
@@ -83,6 +94,10 @@ public class MetadataCluster {
     return agentConfigs;
   }
 
+  public SortedMap<String, String> getClusterSettings() {
+    return clusterSettings;
+  }
+
   public boolean isFullServiceLevelMetadata() {
     return fullServiceLevelMetadata;
   }
@@ -119,6 +134,22 @@ public class MetadataCluster {
     return false;
   }
 
+  public boolean updateClusterSettings(SortedMap<String, String> update) {
+    if (update != null) {
+      try {
+        lock.lock();
+        if (this.clusterSettings == null) {
+          this.clusterSettings = new TreeMap<>();
+        }
+        return updateMapIfNeeded(this.clusterSettings, update, true);
+      } finally {
+        lock.unlock();
+      }
+    }
+
+    return false;
+  }
+
   private <T> boolean updateMapIfNeeded(Map<String, T> currentMap, Map<String, T> updatedMap, boolean fullMetadataInUpdatedMap) {
     boolean changed = false;
     if (fullMetadataInUpdatedMap) { // we have full metadata in updatedMap (i.e. in case of service removal we have full metadata in updatedMap)
@@ -148,11 +179,12 @@ public class MetadataCluster {
 
     return Objects.equals(statusCommandsToRun, that.statusCommandsToRun) &&
       Objects.equals(serviceLevelParams, that.serviceLevelParams) &&
-      Objects.equals(clusterLevelParams, that.clusterLevelParams);
+      Objects.equals(clusterLevelParams, that.clusterLevelParams) &&
+      Objects.equals(clusterSettings, that.clusterSettings);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(statusCommandsToRun, serviceLevelParams, clusterLevelParams);
+    return Objects.hash(statusCommandsToRun, serviceLevelParams, clusterLevelParams, clusterSettings);
   }
 }

+ 15 - 0
ambari-server/src/main/java/org/apache/ambari/server/agent/stomp/dto/TopologyComponent.java

@@ -39,6 +39,7 @@ public class TopologyComponent {
   private String serviceType;
   private String version;
   private String mpackVersion;
+  private Long desiredMpackId;
   private Set<Long> hostIds = new HashSet<>();
   private Set<String> hostNames = new HashSet<>();
   private Set<String> publicHostNames = new HashSet<>();
@@ -88,6 +89,11 @@ public class TopologyComponent {
       return this;
     }
 
+    public Builder setDesiredMpackId(Long desiredMpackId) {
+      TopologyComponent.this.setDesiredMpackId(desiredMpackId);
+      return this;
+    }
+
     public Builder setHostIds(Set<Long> hostIds) {
       TopologyComponent.this.setHostIds(hostIds);
       return this;
@@ -200,6 +206,7 @@ public class TopologyComponent {
     return TopologyComponent.newBuilder().setComponentName(getComponentName())
         .setServiceType(getServiceType())
         .setServiceName(getServiceName())
+        .setDesiredMpackId(getDesiredMpackId())
         .setServiceGroupName(getServiceGroupName())
         .setComponentLevelParams(getComponentLevelParams() == null ? null : new TreeMap<>(getComponentLevelParams()))
         .setHostIds(getHostIds() == null ? null : new HashSet<>(getHostIds()))
@@ -249,6 +256,14 @@ public class TopologyComponent {
     this.mpackVersion = mpackVersion;
   }
 
+  public Long getDesiredMpackId() {
+    return desiredMpackId;
+  }
+
+  public void setDesiredMpackId(Long desiredMpackId) {
+    this.desiredMpackId = desiredMpackId;
+  }
+
   public Set<Long> getHostIds() {
     return hostIds;
   }

+ 0 - 6
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java

@@ -429,12 +429,6 @@ public class AmbariCustomCommandExecutionHelper {
       String userGroups = gson.toJson(userGroupsMap);
       hostLevelParams.put(USER_GROUPS, userGroups);
 
-      // Set exec command with 'ClusterSettings' map
-      execCmd.setClusterSettings(cluster.getClusterSettingsNameValueMap());
-
-      // Set exec command with 'StackSettings' map
-      execCmd.setStackSettings(ambariMetaInfo.getStackSettingsNameValueMap(stackId));
-
       Set<String> groupSet = configHelper.getPropertyValuesWithPropertyType(stackId, PropertyType.GROUP, cluster, desiredConfigs);
       String groupList = gson.toJson(groupSet);
       hostLevelParams.put(GROUP_LIST, groupList);

+ 0 - 10
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementController.java

@@ -25,9 +25,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.ambari.server.AmbariException;
-import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.actionmanager.ActionManager;
-import org.apache.ambari.server.agent.ExecutionCommand;
 import org.apache.ambari.server.agent.stomp.dto.HostRepositories;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.api.services.LoggingService;
@@ -925,14 +923,6 @@ public interface AmbariManagementController {
    */
   void initializeWidgetsAndLayouts(Cluster cluster, Service service) throws AmbariException;
 
-  /**
-   * Gets an execution command for host component life cycle command
-   * @return
-   */
-  ExecutionCommand getExecutionCommand(Cluster cluster,
-                                              ServiceComponentHost scHost,
-                                              RoleCommand roleCommand) throws AmbariException;
-
   /**
    * Get configuration dependencies which are specific for a specific service configuration property
    * @param requests

+ 1 - 97
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java

@@ -884,6 +884,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
           .setPublicHostNames(publicHostNames)
           .setComponentLevelParams(getTopologyComponentLevelParams(sch))
           .setCommandParams(getTopologyCommandParams(sch))
+          .setDesiredMpackId(sch.getDesiredMpackId())
           .build();
 
       String clusterId = Long.toString(cluster.getClusterId());
@@ -2720,13 +2721,6 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
       commandParams.put(PACKAGE_LIST, packageList);
     }
 
-    // Set exec command with 'ClusterSettings' map
-    execCmd.setClusterSettings(cluster.getClusterSettingsNameValueMap());
-
-    // Set exec command with 'StackSettings' map
-    // TODO avoid sending in each command, send in "async" metadata
-    execCmd.setStackSettings(metadataGenerator.getMetadataStackLevelParams(cluster, stackId));
-
     if (databaseType == DatabaseType.ORACLE) {
       hostParams.put(DB_DRIVER_FILENAME, configs.getOjdbcJarName());
     } else if (databaseType == DatabaseType.MYSQL) {
@@ -3401,67 +3395,6 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
 
   }
 
-  @Override
-  public ExecutionCommand getExecutionCommand(Cluster cluster,
-                                              ServiceComponentHost scHost,
-                                              RoleCommand roleCommand) throws AmbariException {
-    Map<String, Set<String>> clusterHostInfo = StageUtils.getClusterHostInfo(cluster);
-    String clusterHostInfoJson = StageUtils.getGson().toJson(clusterHostInfo);
-
-
-    Map<String, String> hostParamsCmd = customCommandExecutionHelper.createDefaultHostParams(
-        cluster, scHost.getServiceComponent().getStackId());
-
-    Stage stage = createNewStage(0, cluster, 1, "", "{}", "");
-
-    Map<String, Map<String, String>> configTags = configHelper.getEffectiveDesiredTags(cluster, scHost.getHostName());
-    Map<String, Map<String, String>> configurations = configHelper.getEffectiveConfigProperties(cluster, configTags);
-
-    Map<String, Map<String, Map<String, String>>>
-        configurationAttributes =
-        new TreeMap<>();
-
-    boolean isUpgradeSuspended = cluster.isUpgradeSuspended();
-    DatabaseType databaseType = configs.getDatabaseType();
-    Map<String, DesiredConfig> clusterDesiredConfigs = cluster.getDesiredConfigs();
-    createHostAction(cluster, stage, scHost, configurations, configurationAttributes, configTags,
-                     roleCommand, null, null, false, isUpgradeSuspended, databaseType,
-                     clusterDesiredConfigs, false);
-    ExecutionCommand ec = stage.getExecutionCommands().get(scHost.getHostName()).get(0).getExecutionCommand();
-
-    // createHostAction does not take a hostLevelParams but creates one
-    hostParamsCmd.putAll(ec.getHostLevelParams());
-    ec.getHostLevelParams().putAll(hostParamsCmd);
-
-    if (null != cluster) {
-      // Generate localComponents
-      for (ServiceComponentHost sch : cluster.getServiceComponentHosts(scHost.getHostName())) {
-        ec.getLocalComponents().add(sch.getServiceComponentName());
-      }
-    }
-
-    ConfigHelper.processHiddenAttribute(ec.getConfigurations(), ec.getConfigurationAttributes(), ec.getRole(), false);
-
-    // Add attributes
-    Map<String, Map<String, Map<String, String>>> configAttributes =
-        configHelper.getEffectiveConfigAttributes(cluster,
-          ec.getConfigurationTags());
-
-    for (Map.Entry<String, Map<String, Map<String, String>>> attributesOccurrence : configAttributes.entrySet()) {
-      String type = attributesOccurrence.getKey();
-      Map<String, Map<String, String>> attributes = attributesOccurrence.getValue();
-
-      if (ec.getConfigurationAttributes() != null) {
-        if (!ec.getConfigurationAttributes().containsKey(type)) {
-          ec.getConfigurationAttributes().put(type, new TreeMap<>());
-        }
-        configHelper.cloneAttributesMap(attributes, ec.getConfigurationAttributes().get(type));
-      }
-    }
-
-    return ec;
-  }
-
   @Override
   public Set<StackConfigurationDependencyResponse> getStackConfigurationDependencies(
           Set<StackConfigurationDependencyRequest> requests) throws AmbariException {
@@ -5502,35 +5435,6 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
     return QuickLinkVisibilityControllerFactory.get(quickLinkProfileJson);
   }
 
-  /**
-   * Collects full metadata info about clusters for agent.
-   * @return metadata info about clusters
-   * @throws AmbariException
-   */
-  public MetadataUpdateEvent getClustersMetadata() throws AmbariException {
-    TreeMap<String, MetadataCluster> metadataClusters = new TreeMap<>();
-
-    for (Cluster cl : clusters.getClusters().values()) {
-      StackId stackId = cl.getDesiredStackVersion();
-
-      SecurityType securityType = cl.getSecurityType();
-
-      MetadataCluster metadataCluster = new MetadataCluster(securityType, getMetadataServiceLevelParams(cl), true, getMetadataClusterLevelParams(cl, stackId), null);
-      metadataClusters.put(Long.toString(cl.getClusterId()), metadataCluster);
-    }
-
-    MetadataUpdateEvent metadataUpdateEvent = new MetadataUpdateEvent(metadataClusters,
-        getMetadataAmbariLevelParams(), getMetadataAgentConfigs(), UpdateEventType.CREATE);
-    return metadataUpdateEvent;
-  }
-
-  public MetadataUpdateEvent getClusterMetadata(Cluster cl) throws AmbariException {
-    final TreeMap<String, MetadataCluster> metadataClusters = new TreeMap<>();
-    MetadataCluster metadataCluster = new MetadataCluster(cl.getSecurityType(), getMetadataServiceLevelParams(cl), true, getMetadataClusterLevelParams(cl, cl.getDesiredStackVersion()), null);
-    metadataClusters.put(Long.toString(cl.getClusterId()), metadataCluster);
-    return new MetadataUpdateEvent(metadataClusters, null, getMetadataAgentConfigs(), UpdateEventType.UPDATE);
-  }
-
   @Override
   public MetadataUpdateEvent getClusterMetadataOnConfigsUpdate(Cluster cl) throws AmbariException {
     final TreeMap<String, MetadataCluster> metadataClusters = new TreeMap<>();

+ 2 - 1
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java

@@ -577,7 +577,8 @@ public class HostResourceProvider extends AbstractControllerResourceProvider {
             getManagementController().retrieveHostRepositories(cl, addedHost),
             recoveryConfigHelper.getRecoveryConfig(cl.getClusterName(),
                 addedHost.getHostName()),
-            getManagementController().getBlueprintProvisioningStates(cl.getClusterId(), addedHost.getHostId())
+            getManagementController().getBlueprintProvisioningStates(cl.getClusterId(), addedHost.getHostId()),
+            hostLevelParamsHolder.getHostStacksSettings(cl, addedHost)
         ));
         hostLevelParamsUpdateEvents.add(hostLevelParamsUpdateEvent);
       }

+ 0 - 9
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradePlanInstallResourceProvider.java

@@ -407,15 +407,6 @@ public class UpgradePlanInstallResourceProvider extends AbstractControllerResour
         Mpack mpack = getManagementController().getAmbariMetaInfo().getMpack(installDetail.mpackId);
         RepoOsEntity repoOsEntity = s_repoHelper.getOSEntityForHost(installDetail.mpackEntity, h);
 
-        // this isn't being placed correctly elsewhere
-        actionContext.addVisitor(command -> {
-          try {
-            command.setClusterSettings(cluster.getClusterSettingsNameValueMap());
-          } catch (AmbariException e) {
-            LOG.warn("Could not set cluster settings on the command", e);
-          }
-        });
-
         s_repoHelper.addCommandRepositoryToContext(actionContext, mpack, repoOsEntity);
 
         s_actionExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, null);

+ 2 - 1
ambari-server/src/main/java/org/apache/ambari/server/events/MetadataUpdateEvent.java

@@ -63,7 +63,8 @@ public class MetadataUpdateEvent extends STOMPEvent implements Hashable {
     super(Type.METADATA);
     this.metadataClusters = metadataClusters;
     if (ambariLevelParams != null) {
-      this.metadataClusters.put(AMBARI_LEVEL_CLUSTER_ID, new MetadataCluster(null, null, false, ambariLevelParams, metadataAgentConfigs));
+      this.metadataClusters.put(AMBARI_LEVEL_CLUSTER_ID, new MetadataCluster(null, null,
+          false, ambariLevelParams, metadataAgentConfigs, null));
     }
     this.eventType = eventType;
   }

+ 31 - 3
ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/MpackInstallStateListener.java

@@ -24,11 +24,13 @@ import javax.annotation.Nullable;
 
 import org.apache.ambari.annotations.Experimental;
 import org.apache.ambari.annotations.ExperimentalFeature;
+import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.EagerSingleton;
 import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
 import org.apache.ambari.server.agent.CommandReport;
 import org.apache.ambari.server.agent.StructuredOutputType;
+import org.apache.ambari.server.agent.stomp.HostLevelParamsHolder;
 import org.apache.ambari.server.controller.internal.UpgradePlanInstallResourceProvider;
 import org.apache.ambari.server.events.CommandReportReceivedEvent;
 import org.apache.ambari.server.events.HostsAddedEvent;
@@ -89,6 +91,9 @@ public class MpackInstallStateListener {
   @Inject
   private Gson m_gson;
 
+  @Inject
+  private Provider<HostLevelParamsHolder> hostLevelParamsHolderProvider;
+
   /**
    * Used for ensuring that the concurrent nature of the event handler methods
    * don't collide when attempting to register hosts and mpacks at the same
@@ -127,6 +132,7 @@ public class MpackInstallStateListener {
     MpackHostStateDAO mpackHostStateDAO = m_mpackHostStateDAOProvider.get();
     List<MpackEntity> mpackEntities = m_mpackDAOProvider.get().findAll();
 
+    HostLevelParamsHolder hostLevelParamsHolder = hostLevelParamsHolderProvider.get();
     for (String hostName : event.getHostNames()) {
       Lock lock = m_locksByHost.get(hostName);
       lock.lock();
@@ -145,6 +151,12 @@ public class MpackInstallStateListener {
           hostEntity.getMpackInstallStates().add(mpackHostStateEntity);
           hostEntity = hostDAO.merge(hostEntity);
         }
+        try {
+          hostLevelParamsHolder.updateData(hostLevelParamsHolder.getCurrentData(hostEntity.getHostId()));
+        } catch (AmbariException e) {
+          LOG.warn("Exception during host level params update on mpack assigning during host adding for host with id = " +
+              hostEntity.getHostId(), e);
+        }
       } catch (Throwable throwable) {
         LOG.error(throwable.getMessage(), throwable);
       } finally {
@@ -158,16 +170,32 @@ public class MpackInstallStateListener {
    * every host.
    */
   @Subscribe
-  @Transactional
   public void onMpackEvent(MpackRegisteredEvent event) {
+    HostDAO hostDAO = m_hostDAOProvider.get();
+    List<HostEntity> hosts = hostDAO.findAll();
+    processMpackRegister(event, hosts);
+
+    //update all host level params
+    HostLevelParamsHolder hostLevelParamsHolder = hostLevelParamsHolderProvider.get();
+    for (HostEntity host : hosts) {
+      try {
+        hostLevelParamsHolder.updateData(hostLevelParamsHolder.getCurrentData(host.getHostId()));
+      } catch (AmbariException e) {
+        LOG.warn("Exception during host level params update on mpack registering for host with id = " + host.getHostId(), e);
+      }
+    }
+  }
+
+  @Transactional
+  protected void processMpackRegister(MpackRegisteredEvent event, List<HostEntity> hosts) {
     if (LOG.isDebugEnabled()) {
       LOG.debug(event.toString());
     }
 
-    HostDAO hostDAO = m_hostDAOProvider.get();
     MpackHostStateDAO mpackHostStateDAO = m_mpackHostStateDAOProvider.get();
 
-    List<HostEntity> hosts = hostDAO.findAll();
+    HostDAO hostDAO = m_hostDAOProvider.get();
+
     MpackEntity mpackEntity = m_mpackDAOProvider.get().findById(event.getMpackIdId());
 
     for (HostEntity hostEntity : hosts) {

+ 14 - 59
ambari-server/src/main/java/org/apache/ambari/server/metadata/ClusterMetadataGenerator.java

@@ -25,9 +25,7 @@ import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.AMBARI_SE
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.CLUSTER_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.DB_DRIVER_FILENAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.DB_NAME;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.DFS_TYPE;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.GPL_LICENSE_ACCEPTED;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.GROUP_LIST;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.HOOKS_FOLDER;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.HOST_SYS_PREPPED;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JAVA_HOME;
@@ -36,18 +34,12 @@ import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JCE_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JDK_LOCATION;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JDK_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.MYSQL_JDBC_URL;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.NOT_MANAGED_HDFS_PATH_LIST;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.ORACLE_JDBC_URL;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.STACK_VERSION;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.USER_GROUPS;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.USER_LIST;
 
 import java.net.UnknownHostException;
 import java.util.Map;
-import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.TreeSet;
 
 import javax.inject.Inject;
 
@@ -63,31 +55,24 @@ import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
 import org.apache.ambari.server.state.CommandScriptDefinition;
 import org.apache.ambari.server.state.ConfigHelper;
-import org.apache.ambari.server.state.DesiredConfig;
-import org.apache.ambari.server.state.PropertyInfo;
-import org.apache.ambari.server.state.PropertyInfo.PropertyType;
 import org.apache.ambari.server.state.SecurityType;
 import org.apache.ambari.server.state.Service;
 import org.apache.ambari.server.state.ServiceInfo;
 import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.utils.StageUtils;
 
-import com.google.gson.Gson;
-
 public class ClusterMetadataGenerator {
 
   private final Configuration configs;
   private final ConfigHelper configHelper;
   private final AmbariMetaInfo ambariMetaInfo;
-  private final Gson gson;
   private final AmbariConfig ambariConfig;
 
   @Inject
-  public ClusterMetadataGenerator(AmbariMetaInfo metaInfo, Configuration configs, ConfigHelper configHelper, Gson gson) throws UnknownHostException {
+  public ClusterMetadataGenerator(AmbariMetaInfo metaInfo, Configuration configs, ConfigHelper configHelper) throws UnknownHostException {
     this.ambariMetaInfo = metaInfo;
     this.configs = configs;
     this.configHelper = configHelper;
-    this.gson = gson;
 
     ambariConfig = new AmbariConfig(configs);
   }
@@ -96,47 +81,6 @@ public class ClusterMetadataGenerator {
     return ambariConfig;
   }
 
-  public SortedMap<String, String> getMetadataStackLevelParams(Cluster cluster, StackId stackId) throws AmbariException {
-    SortedMap<String, String> stackLevelParams = new TreeMap<>(ambariMetaInfo.getStackSettingsNameValueMap(stackId));
-
-    // STACK_NAME is part of stack settings, but STACK_VERSION is not
-    stackLevelParams.put(STACK_VERSION, stackId.getStackVersion());
-
-    Map<String, DesiredConfig> clusterDesiredConfigs = cluster.getDesiredConfigs(false);
-    Set<PropertyInfo> stackProperties = ambariMetaInfo.getStackProperties(stackId.getStackName(), stackId.getStackVersion());
-    Map<String, ServiceInfo> servicesMap = ambariMetaInfo.getServices(stackId.getStackName(), stackId.getStackVersion());
-    Set<PropertyInfo> clusterProperties = ambariMetaInfo.getClusterProperties();
-
-    Map<PropertyInfo, String> users = configHelper.getPropertiesWithPropertyType(PropertyType.USER, cluster, clusterDesiredConfigs, servicesMap, stackProperties, clusterProperties);
-    Set<String> userSet = new TreeSet<>(users.values());
-    String userList = gson.toJson(userSet);
-    stackLevelParams.put(USER_LIST, userList);
-
-    Map<PropertyInfo, String> groups = configHelper.getPropertiesWithPropertyType(PropertyType.GROUP, cluster, clusterDesiredConfigs, servicesMap, stackProperties, clusterProperties);
-    Set<String> groupSet = new TreeSet<>(groups.values());
-    String groupList = gson.toJson(groupSet);
-    stackLevelParams.put(GROUP_LIST, groupList);
-
-    Map<String, Set<String>> userGroupsMap = configHelper.createUserGroupsMap(users, groups);
-    String userGroups = gson.toJson(userGroupsMap);
-    stackLevelParams.put(USER_GROUPS, userGroups);
-
-    Map<PropertyInfo, String> notManagedHdfsPathMap = configHelper.getPropertiesWithPropertyType(PropertyType.NOT_MANAGED_HDFS_PATH, cluster, clusterDesiredConfigs, servicesMap, stackProperties, clusterProperties);
-    Set<String> notManagedHdfsPathSet = configHelper.filterInvalidPropertyValues(notManagedHdfsPathMap, NOT_MANAGED_HDFS_PATH_LIST);
-    String notManagedHdfsPathList = gson.toJson(notManagedHdfsPathSet);
-    stackLevelParams.put(NOT_MANAGED_HDFS_PATH_LIST, notManagedHdfsPathList);
-
-    Map<String, ServiceInfo> serviceInfos = ambariMetaInfo.getServices(stackId.getStackName(), stackId.getStackVersion());
-    for (ServiceInfo serviceInfoInstance : serviceInfos.values()) {
-      if (serviceInfoInstance.getServiceType() != null) {
-        stackLevelParams.put(DFS_TYPE, serviceInfoInstance.getServiceType());
-        break;
-      }
-    }
-
-    return stackLevelParams;
-  }
-
   /**
    * Collects metadata info about clusters for agent.
    */
@@ -150,7 +94,8 @@ public class ClusterMetadataGenerator {
         getMetadataServiceLevelParams(cl),
         true,
         getMetadataClusterLevelParams(cl),
-        null);
+        null,
+        getClusterSettings(cl));
       metadataClusters.put(Long.toString(cl.getClusterId()), metadataCluster);
     }
 
@@ -159,7 +104,8 @@ public class ClusterMetadataGenerator {
 
   public MetadataUpdateEvent getClusterMetadata(Cluster cl) throws AmbariException {
     SortedMap<String, MetadataCluster> metadataClusters = new TreeMap<>();
-    MetadataCluster metadataCluster = new MetadataCluster(cl.getSecurityType(), getMetadataServiceLevelParams(cl), true, getMetadataClusterLevelParams(cl), null);
+    MetadataCluster metadataCluster = new MetadataCluster(cl.getSecurityType(), getMetadataServiceLevelParams(cl),
+        true, getMetadataClusterLevelParams(cl), null, getClusterSettings(cl));
     metadataClusters.put(Long.toString(cl.getClusterId()), metadataCluster);
     return new MetadataUpdateEvent(metadataClusters, null, getMetadataAgentConfigs(), UpdateEventType.UPDATE);
   }
@@ -180,6 +126,12 @@ public class ClusterMetadataGenerator {
     return getClusterMetadataOnServiceCredentialStoreUpdate(cl, serviceGroupName, serviceName);
   }
 
+  public MetadataUpdateEvent getClusterMetadataOnClusterSettingsUpdate(Cluster cl) {
+    SortedMap<String, MetadataCluster> metadataClusters = new TreeMap<>();
+    metadataClusters.put(Long.toString(cl.getClusterId()), MetadataCluster.clusterSettingsMetadataCluster(null, getClusterSettings(cl)));
+    return new MetadataUpdateEvent(metadataClusters, null, getMetadataAgentConfigs(), UpdateEventType.UPDATE);
+  }
+
   public MetadataUpdateEvent getClusterMetadataOnServiceCredentialStoreUpdate(Cluster cl, String serviceGroupName, String serviceName) throws AmbariException {
     final SortedMap<String, MetadataCluster> metadataClusters = new TreeMap<>();
     Service service = cl.getService(serviceGroupName, serviceName);
@@ -291,4 +243,7 @@ public class ClusterMetadataGenerator {
     return agentConfigs;
   }
 
+  private SortedMap<String, String> getClusterSettings(Cluster cluster) {
+    return new TreeMap<>(cluster.getClusterSettingsNameValueMap());
+  }
 }

+ 3 - 3
ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java

@@ -211,7 +211,7 @@ public interface Cluster {
    *
    * @return
    */
-  Map<String, ServiceGroup> getServiceGroups() throws AmbariException;
+  Map<String, ServiceGroup> getServiceGroups();
 
   /**
    * Get a cluster setting
@@ -234,7 +234,7 @@ public interface Cluster {
    *
    * @return
    */
-  Map<String, ClusterSetting> getClusterSettings() throws AmbariException;
+  Map<String, ClusterSetting> getClusterSettings();
 
   /**
    * Get all cluster settings name and value as Map.
@@ -242,7 +242,7 @@ public interface Cluster {
    *
    * @return
    */
-  Map<String, String> getClusterSettingsNameValueMap() throws AmbariException;
+  Map<String, String> getClusterSettingsNameValueMap();
 
   /**
    * Get all ServiceComponentHosts on a given host

+ 7 - 0
ambari-server/src/main/java/org/apache/ambari/server/state/ServiceComponentHost.java

@@ -147,6 +147,13 @@ public interface ServiceComponentHost {
    */
   String getMpackVersion();
 
+  /**
+   * Gets the mpack id associated with parent dservice.
+   *
+   * @return mpack id for parent service.
+   */
+  Long getDesiredMpackId() throws AmbariException;
+
   /**
    * Sets the versions reported for a component which include the mpack it
    * belongs to and the specific version of that component within the mpack.

+ 1 - 0
ambari-server/src/main/java/org/apache/ambari/server/state/ServiceGroupImpl.java

@@ -370,6 +370,7 @@ public class ServiceGroupImpl implements ServiceGroup {
     serviceGroupEntity.setStack(stackEntity);
     serviceGroupEntity = serviceGroupDAO.merge(serviceGroupEntity);
     stackId = new StackId(stackEntity.getStackName(), stackEntity.getStackVersion());
+    // TODO should mpack id be also changed?
   }
 
   @Override

+ 26 - 3
ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java

@@ -63,6 +63,7 @@ import org.apache.ambari.server.ServiceGroupNotFoundException;
 import org.apache.ambari.server.ServiceNotFoundException;
 import org.apache.ambari.server.agent.ExecutionCommand.KeyNames;
 import org.apache.ambari.server.agent.stomp.HostLevelParamsHolder;
+import org.apache.ambari.server.agent.stomp.MetadataHolder;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.api.services.ServiceGroupKey;
 import org.apache.ambari.server.controller.AmbariManagementController;
@@ -86,6 +87,7 @@ import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.events.publishers.JPAEventPublisher;
 import org.apache.ambari.server.events.publishers.STOMPUpdatePublisher;
 import org.apache.ambari.server.logging.LockFactory;
+import org.apache.ambari.server.metadata.ClusterMetadataGenerator;
 import org.apache.ambari.server.metadata.RoleCommandOrder;
 import org.apache.ambari.server.metadata.RoleCommandOrderProvider;
 import org.apache.ambari.server.orm.RequiresSession;
@@ -370,6 +372,12 @@ public class ClusterImpl implements Cluster {
   @Inject
   private HostComponentDesiredStateDAO hostComponentDesiredStateDAO;
 
+  @Inject
+  private MetadataHolder metadataHolder;
+
+  @Inject
+  private ClusterMetadataGenerator clusterMetadataGenerator;
+
   /**
    * A simple cache for looking up {@code cluster-env} properties for a cluster.
    * This map is changed whenever {{cluster-env}} is changed and we receive a
@@ -1088,6 +1096,11 @@ public class ClusterImpl implements Cluster {
 
     clusterSettings.put(clusterSetting.getClusterSettingName(), clusterSetting);
     clusterSettingsById.put(clusterSetting.getClusterSettingId(), clusterSetting);
+    try {
+      metadataHolder.updateData(clusterMetadataGenerator.getClusterMetadataOnClusterSettingsUpdate(this));
+    } catch (AmbariException e) {
+      LOG.warn("Exception on cluster settings metadata update", e);
+    }
   }
 
   @Override
@@ -1111,6 +1124,11 @@ public class ClusterImpl implements Cluster {
     // Update the changed 'clusterSetting' in the below maps, to reflect object with newest changes.
     clusterSettings.put(clusterSetting.getClusterSettingName(), clusterSetting);
     clusterSettingsById.put(clusterSetting.getClusterSettingId(), clusterSetting);
+    try {
+      metadataHolder.updateData(clusterMetadataGenerator.getClusterMetadataOnClusterSettingsUpdate(this));
+    } catch (AmbariException e) {
+      LOG.warn("Exception on cluster settings metadata update", e);
+    }
   }
 
   @Override
@@ -1316,7 +1334,7 @@ public class ClusterImpl implements Cluster {
   }
 
   @Override
-  public Map<String, ServiceGroup> getServiceGroups() throws AmbariException {
+  public Map<String, ServiceGroup> getServiceGroups() {
     return new HashMap<>(serviceGroups);
   }
 
@@ -1340,12 +1358,12 @@ public class ClusterImpl implements Cluster {
   }
 
   @Override
-  public Map<String, ClusterSetting> getClusterSettings() throws AmbariException {
+  public Map<String, ClusterSetting> getClusterSettings() {
     return new HashMap<>(clusterSettings);
   }
 
   @Override
-  public Map<String, String> getClusterSettingsNameValueMap() throws AmbariException {
+  public Map<String, String> getClusterSettingsNameValueMap() {
     Map<String, ClusterSetting> clusterSettings = getClusterSettings();
     if (clusterSettings != null) {
       return clusterSettings.values().stream().collect(
@@ -1817,6 +1835,11 @@ public class ClusterImpl implements Cluster {
       deleteClusterSetting(clusterSetting);
       clusterSettings.remove(clusterSettingName);
       clusterSettingsById.remove(clusterSettingId);
+      try {
+        metadataHolder.updateData(clusterMetadataGenerator.getClusterMetadataOnClusterSettingsUpdate(this));
+      } catch (AmbariException e) {
+        LOG.warn("Exception on cluster settings metadata update", e);
+      }
     } finally {
       clusterGlobalLock.writeLock().unlock();
     }

+ 5 - 0
ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java

@@ -976,6 +976,11 @@ public class ServiceComponentHostImpl implements ServiceComponentHost {
     return null;
   }
 
+  @Override
+  public Long getDesiredMpackId() throws AmbariException {
+    return clusters.getClusterById(getClusterId()).getServiceGroup(getServiceGroupId()).getMpackId();
+  }
+
   @Override
   @Transactional
   public void setVersions(String mpackVersion, String version) throws AmbariException {

+ 1 - 3
ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java

@@ -43,9 +43,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.BufferedWriter;
 import java.io.File;
@@ -754,7 +752,7 @@ public class TestHeartbeatHandler {
     statusCmd1.setServiceName(HDFS);
     dummyCmds.add(statusCmd1);
     HeartbeatMonitor hm = mock(HeartbeatMonitor.class);
-    when(hm.generateStatusCommands(anyString())).thenReturn(dummyCmds);
+    //when(hm.generateStatusCommands(anyString())).thenReturn(dummyCmds);
 
     ActionManager am = actionManagerTestHelper.getMockActionManager();
     replay(am);

+ 6 - 6
ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatMonitor.java

@@ -205,7 +205,7 @@ public class TestHeartbeatMonitor {
 
     hm.getAgentRequests().setExecutionDetailsRequest(hostname1, "DATANODE", Boolean.TRUE.toString());
 
-    List<StatusCommand> cmds = hm.generateStatusCommands(hostname1);
+    List<StatusCommand> cmds = null;//hm.generateStatusCommands(hostname1);
     assertTrue("HeartbeatMonitor should generate StatusCommands for host1", cmds.size() == 3);
     assertEquals("HDFS", cmds.get(0).getServiceName());
     boolean  containsDATANODEStatus = false;
@@ -231,7 +231,7 @@ public class TestHeartbeatMonitor {
     assertEquals(true, containsNAMENODEStatus);
     assertEquals(true, containsSECONDARY_NAMENODEStatus);
 
-    cmds = hm.generateStatusCommands(hostname2);
+    cmds = null;//hm.generateStatusCommands(hostname2);
     assertTrue("HeartbeatMonitor should not generate StatusCommands for host2 because it has no services", cmds.isEmpty());
   }
 
@@ -326,7 +326,7 @@ public class TestHeartbeatMonitor {
 
     // HeartbeatMonitor should generate StatusCommands for
     // MASTER, SLAVE or CLIENT components
-    List<StatusCommand> cmds = hm.generateStatusCommands(hostname1);
+    List<StatusCommand> cmds = null;//hm.generateStatusCommands(hostname1);
     assertTrue("HeartbeatMonitor should generate StatusCommands for host1",
       cmds.size() == 4);
     assertEquals("HDFS", cmds.get(0).getServiceName());
@@ -350,7 +350,7 @@ public class TestHeartbeatMonitor {
     assertTrue(containsSECONDARY_NAMENODEStatus);
     assertTrue(containsHDFS_CLIENTStatus);
 
-    cmds = hm.generateStatusCommands(hostname2);
+    cmds = null;//hm.generateStatusCommands(hostname2);
     assertTrue("HeartbeatMonitor should generate StatusCommands for host2, " +
       "even if it has only client components", cmds.size() == 1);
     assertTrue(cmds.get(0).getComponentName().equals(Role.HDFS_CLIENT.name()));
@@ -608,12 +608,12 @@ public class TestHeartbeatMonitor {
     hb.setResponseId(12);
     handler.handleHeartBeat(hb);
 
-    List<StatusCommand> cmds = hm.generateStatusCommands(hostname1);
+    List<StatusCommand> cmds = null;//hm.generateStatusCommands(hostname1);
     assertEquals("HeartbeatMonitor should generate StatusCommands for host1",
         3, cmds.size());
     assertEquals("HDFS", cmds.get(0).getServiceName());
 
-    cmds = hm.generateStatusCommands(hostname2);
+    cmds = null;//hm.generateStatusCommands(hostname2);
     assertTrue("HeartbeatMonitor should not generate StatusCommands for host2 because it has no services", cmds.isEmpty());
   }
 }

+ 11 - 10
ambari-server/src/test/java/org/apache/ambari/server/agent/stomp/HostLevelParamsHolderTest.java

@@ -29,6 +29,7 @@ import java.util.Map;
 import org.apache.ambari.server.agent.RecoveryConfig;
 import org.apache.ambari.server.agent.stomp.dto.HostLevelParamsCluster;
 import org.apache.ambari.server.agent.stomp.dto.HostRepositories;
+import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.events.HostLevelParamsUpdateEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.junit.Test;
@@ -42,11 +43,11 @@ public class HostLevelParamsHolderTest {
     Map<String, HostLevelParamsCluster> clusters = new HashMap<>();
     HostRepositories hostRepositories = new HostRepositories(Collections.emptySortedMap(), Collections.emptySortedMap());
     HostLevelParamsCluster cluster = new HostLevelParamsCluster(hostRepositories,
-        new RecoveryConfig(null), Collections.emptyMap());
+        new RecoveryConfig(null), Collections.emptyMap(), Collections.emptySortedMap());
     clusters.put("1", cluster);
     HostLevelParamsUpdateEvent update = new HostLevelParamsUpdateEvent(HOST_ID, clusters);
 
-    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class));
+    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class), createNiceMock(AmbariMetaInfo.class));
     HostLevelParamsUpdateEvent result = levelParamsHolder.handleUpdate(current, update);
 
     assertFalse(result == update);
@@ -59,12 +60,12 @@ public class HostLevelParamsHolderTest {
     Map<String, HostLevelParamsCluster> clusters = new HashMap<>();
     HostRepositories hostRepositories = new HostRepositories(Collections.emptySortedMap(), Collections.emptySortedMap());
     HostLevelParamsCluster cluster = new HostLevelParamsCluster(hostRepositories,
-        new RecoveryConfig(null), Collections.emptyMap());
+        new RecoveryConfig(null), Collections.emptyMap(), Collections.emptySortedMap());
     clusters.put("1", cluster);
     HostLevelParamsUpdateEvent current = new HostLevelParamsUpdateEvent(HOST_ID, clusters);
     HostLevelParamsUpdateEvent update = new HostLevelParamsUpdateEvent(HOST_ID, Collections.emptyMap());
 
-    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class));
+    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class), createNiceMock(AmbariMetaInfo.class));
     HostLevelParamsUpdateEvent result = levelParamsHolder.handleUpdate(current, update);
 
     assertFalse(result == update);
@@ -77,18 +78,18 @@ public class HostLevelParamsHolderTest {
     Map<String, HostLevelParamsCluster> currentClusters = new HashMap<>();
     HostRepositories currentHostRepositories = new HostRepositories(Collections.emptySortedMap(), Collections.emptySortedMap());
     HostLevelParamsCluster currentCluster = new HostLevelParamsCluster(currentHostRepositories,
-        new RecoveryConfig(null), Collections.emptyMap());
+        new RecoveryConfig(null), Collections.emptyMap(), Collections.emptySortedMap());
     currentClusters.put("1", currentCluster);
     HostLevelParamsUpdateEvent current = new HostLevelParamsUpdateEvent(HOST_ID, currentClusters);
 
     Map<String, HostLevelParamsCluster> updateClusters = new HashMap<>();
     HostRepositories updateHostRepositories = new HostRepositories(Collections.emptySortedMap(), Collections.emptySortedMap());
     HostLevelParamsCluster updateCluster = new HostLevelParamsCluster(updateHostRepositories,
-        new RecoveryConfig(null), Collections.emptyMap());
+        new RecoveryConfig(null), Collections.emptyMap(), Collections.emptySortedMap());
     updateClusters.put("1", updateCluster);
     HostLevelParamsUpdateEvent update = new HostLevelParamsUpdateEvent(HOST_ID, updateClusters);
 
-    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class));
+    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class), createNiceMock(AmbariMetaInfo.class));
     HostLevelParamsUpdateEvent result = levelParamsHolder.handleUpdate(current, update);
 
     assertFalse(result == update);
@@ -101,18 +102,18 @@ public class HostLevelParamsHolderTest {
     Map<String, HostLevelParamsCluster> currentClusters = new HashMap<>();
     HostRepositories currentHostRepositories = new HostRepositories(Collections.emptySortedMap(), Collections.emptySortedMap());
     HostLevelParamsCluster currentCluster = new HostLevelParamsCluster(currentHostRepositories,
-        new RecoveryConfig(null), Collections.emptyMap());
+        new RecoveryConfig(null), Collections.emptyMap(), Collections.emptySortedMap());
     currentClusters.put("1", currentCluster);
     HostLevelParamsUpdateEvent current = new HostLevelParamsUpdateEvent(HOST_ID, currentClusters);
 
     Map<String, HostLevelParamsCluster> updateClusters = new HashMap<>();
     HostRepositories updateHostRepositories = new HostRepositories(Collections.emptySortedMap(), Collections.emptySortedMap());
     HostLevelParamsCluster updateCluster = new HostLevelParamsCluster(updateHostRepositories,
-        new RecoveryConfig(null), Collections.emptyMap());
+        new RecoveryConfig(null), Collections.emptyMap(), Collections.emptySortedMap());
     updateClusters.put("2", updateCluster);
     HostLevelParamsUpdateEvent update = new HostLevelParamsUpdateEvent(HOST_ID, updateClusters);
 
-    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class));
+    HostLevelParamsHolder levelParamsHolder = new HostLevelParamsHolder(createNiceMock(AmbariEventPublisher.class), createNiceMock(AmbariMetaInfo.class));
     HostLevelParamsUpdateEvent result = levelParamsHolder.handleUpdate(current, update);
 
     assertFalse(result == update);

+ 0 - 80
ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java

@@ -1255,86 +1255,6 @@ public class AmbariManagementControllerTest {
     assertEquals("false", ec.getCommandParams().get("command_retry_enabled"));
   }
 
-
-  @Test
-  public void testGetExecutionCommand() throws Exception {
-    String cluster1 = getUniqueName();
-    final String host1 = getUniqueName();
-
-    String serviceGroupName = "CORE";
-    String serviceName = "HDFS";
-    createServiceComponentHostSimple(cluster1, host1, getUniqueName(), serviceGroupName, serviceName);
-
-    Cluster cluster = clusters.getCluster(cluster1);
-    Service s1 = cluster.getService(serviceGroupName, serviceName);
-
-    // Create and attach config
-    Map<String, String> configs = new HashMap<>();
-    configs.put("a", "b");
-
-    Map<String, String> hadoopEnvConfigs = new HashMap<>();
-    hadoopEnvConfigs.put("hdfs_user", "myhdfsuser");
-    hadoopEnvConfigs.put("hdfs_group", "myhdfsgroup");
-
-    ConfigurationRequest cr1,cr2, cr3;
-
-    cr1 = new ConfigurationRequest(cluster1, "core-site","version1",
-                                   configs, null, s1.getServiceId(), 1L);
-    cr2 = new ConfigurationRequest(cluster1, "hdfs-site","version1",
-                                   configs, null, s1.getServiceId(), 1L);
-    cr3 = new ConfigurationRequest(cluster1, "hadoop-env","version1",
-      hadoopEnvConfigs, null, s1.getServiceId(), 1L);
-
-    ClusterRequest crReq = new ClusterRequest(cluster.getClusterId(), cluster1, null, null);
-    crReq.setDesiredConfig(Collections.singletonList(cr1));
-    controller.updateClusters(Collections.singleton(crReq), null);
-    crReq = new ClusterRequest(cluster.getClusterId(), cluster1, null, null);
-    crReq.setDesiredConfig(Collections.singletonList(cr2));
-    controller.updateClusters(Collections.singleton(crReq), null);
-    crReq = new ClusterRequest(cluster.getClusterId(), cluster1, null, null);
-    crReq.setDesiredConfig(Collections.singletonList(cr3));
-    controller.updateClusters(Collections.singleton(crReq), null);
-
-
-
-    // Install
-    installService(cluster1, serviceGroupName, serviceName, false, false);
-
-    ExecutionCommand ec =
-        controller.getExecutionCommand(cluster,
-                                       s1.getServiceComponent("NAMENODE").getServiceComponentHost(host1),
-                                       RoleCommand.START);
-    assertEquals("1-0", ec.getCommandId());
-    assertEquals(cluster1, ec.getClusterName());
-    Map<String, Map<String, String>> configurations = ec.getConfigurations();
-    assertNotNull(configurations);
-    assertEquals(3, configurations.size());
-    assertTrue(configurations.containsKey("hdfs-site"));
-    assertTrue(configurations.containsKey("core-site"));
-    assertTrue(configurations.containsKey("hadoop-env"));
-    assertTrue(ec.getConfigurationAttributes().containsKey("hdfs-site"));
-    assertTrue(ec.getConfigurationAttributes().containsKey("core-site"));
-    assertTrue(ec.getConfigurationAttributes().containsKey("hadoop-env"));
-    assertTrue(ec.getCommandParams().containsKey("max_duration_for_retries"));
-    assertEquals("0", ec.getCommandParams().get("max_duration_for_retries"));
-    assertTrue(ec.getCommandParams().containsKey("command_retry_enabled"));
-    assertEquals("false", ec.getCommandParams().get("command_retry_enabled"));
-    assertFalse(ec.getCommandParams().containsKey("custom_folder"));
-
-    ec = controller.getExecutionCommand(cluster,
-                                        s1.getServiceComponent("DATANODE").getServiceComponentHost(host1),
-                                        RoleCommand.START);
-    assertEquals(cluster1, ec.getClusterName());
-    assertNotNull(ec.getCommandParams());
-    assertNotNull(ec.getHostLevelParams());
-    assertTrue(ec.getStackSettings().containsKey(ExecutionCommand.KeyNames.USER_LIST));
-    assertEquals("[\"ambari-qa\",\"myhdfsuser\"]", ec.getStackSettings().get(ExecutionCommand.KeyNames.USER_LIST));
-    assertTrue(ec.getStackSettings().containsKey(ExecutionCommand.KeyNames.GROUP_LIST));
-    assertEquals("[\"hadoop\",\"myhdfsgroup\"]", ec.getStackSettings().get(ExecutionCommand.KeyNames.GROUP_LIST));
-    assertTrue(ec.getStackSettings().containsKey(ExecutionCommand.KeyNames.USER_GROUPS));
-    assertEquals("{\"myhdfsuser\":[\"myhdfsgroup\"]}", ec.getStackSettings().get(ExecutionCommand.KeyNames.USER_GROUPS));
-  }
-
   @Test
   public void testCreateServiceComponentMultiple() throws Exception {
     String cluster1 = getUniqueName();