You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by nc...@apache.org on 2013/03/18 15:06:19 UTC

svn commit: r1457772 - in /incubator/ambari/trunk: ./ ambari-agent/src/main/python/ambari_agent/ ambari-agent/src/test/python/ ambari-server/src/main/java/org/apache/ambari/server/agent/ ambari-server/src/main/java/org/apache/ambari/server/controller/ ...

Author: ncole
Date: Mon Mar 18 14:06:19 2013
New Revision: 1457772

URL: http://svn.apache.org/r1457772
Log:
AMBARI-1599. Add ability to report actual config on a host

Modified:
    incubator/ambari/trunk/CHANGES.txt
    incubator/ambari/trunk/ambari-agent/src/main/python/ambari_agent/ActionQueue.py
    incubator/ambari/trunk/ambari-agent/src/test/python/TestActionQueue.py
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandReport.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/ClusterResponse.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/DesiredConfig.java
    incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
    incubator/ambari/trunk/ambari-server/src/main/resources/properties.json
    incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java
    incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/controller/ClusterResponseTest.java
    incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/DesiredConfigTest.java
    incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java
    incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/utils/TestStageUtils.java

Modified: incubator/ambari/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/CHANGES.txt?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/CHANGES.txt (original)
+++ incubator/ambari/trunk/CHANGES.txt Mon Mar 18 14:06:19 2013
@@ -12,6 +12,8 @@ Trunk (unreleased changes):
 
  NEW FEATURES
 
+ AMBARI-1599. Add ability to report actual configuration applied to a host. (ncole)
+
  AMBARI-1647. Integrate server and agent changes for upgrade on cluster. 
  (Sumit Mohanty via swagle)
 

Modified: incubator/ambari/trunk/ambari-agent/src/main/python/ambari_agent/ActionQueue.py
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-agent/src/main/python/ambari_agent/ActionQueue.py?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-agent/src/main/python/ambari_agent/ActionQueue.py (original)
+++ incubator/ambari/trunk/ambari-agent/src/main/python/ambari_agent/ActionQueue.py Mon Mar 18 14:06:19 2013
@@ -220,6 +220,11 @@ class ActionQueue(threading.Thread):
       roleResult['stdout'] = 'None'
     if roleResult['stderr'] == '':
       roleResult['stderr'] = 'None'
+
+    # let ambari know that configuration tags were applied
+    if status == "COMPLETED" and command.has_key('configurationTags'):
+      roleResult['configurationTags'] = command['configurationTags']
+
     result.append(roleResult)
     return result
 

Modified: incubator/ambari/trunk/ambari-agent/src/test/python/TestActionQueue.py
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-agent/src/test/python/TestActionQueue.py?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-agent/src/test/python/TestActionQueue.py (original)
+++ incubator/ambari/trunk/ambari-agent/src/test/python/TestActionQueue.py Mon Mar 18 14:06:19 2013
@@ -102,6 +102,52 @@ class TestActionQueue(TestCase):
     #print("in_progress: " + pprint.pformat(in_progress_result))
     #print("after: " + pprint.pformat(after_start_result))
 
+  def test_configtags(self):
+    config = AmbariConfig().getConfig()
+    tmpfile = tempfile.gettempdir()
+    config.set('agent', 'prefix', tmpfile)
+    actionQueue = ActionQueue(config)
+    actionQueue.IDLE_SLEEP_TIME = 0.01
+    executor_started_event = threading.Event()
+    end_executor_event = threading.Event()
+    actionQueue.puppetExecutor = FakeExecutor(executor_started_event, end_executor_event)
+    # before_start_result = actionQueue.result()
+
+    command = {
+      'commandId': 17,
+      'role' : "role",
+      'taskId' : "taskId",
+      'clusterName' : "clusterName",
+      'serviceName' : "serviceName",
+      'status' : 'IN_PROGRESS',
+      'hostname' : "localhost.localdomain",
+      'hostLevelParams': "hostLevelParams",
+      'clusterHostInfo': "clusterHostInfo",
+      'roleCommand': "roleCommand",
+      'configurations': "configurations",
+      'commandType': "EXECUTION_COMMAND",
+      'configurations':{'global' : {}},
+      'configurationTags':{'global' : { 'tag': 'v1' }}
+    }
+    actionQueue.put(command)
+
+    actionQueue.start()
+    executor_started_event.wait()
+
+
+    end_executor_event.set()
+    actionQueue.stop()
+    actionQueue.join()
+    after_start_result = actionQueue.result()
+
+    self.assertEquals(len(after_start_result['componentStatus']), 0)
+    self.assertEquals(len(after_start_result['reports']), 1)
+    self.assertEquals(after_start_result['reports'][0]['status'], "COMPLETED")
+    self.assertEquals(after_start_result['reports'][0]['stdout'], "returned stdout")
+    self.assertEquals(after_start_result['reports'][0]['exitCode'], 0)
+    self.assertEquals(after_start_result['reports'][0]['stderr'], 'returned stderr')
+    self.assertEquals(len(after_start_result['reports'][0]['configurationTags']), 1)
+
   @patch.object(ActionQueue, "executeCommand")
   @patch.object(ActionQueue, "stopped")
   def test_upgradeCommand_dispatching(self, stopped_method, executeCommand_method):

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandReport.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandReport.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandReport.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandReport.java Mon Mar 18 14:06:19 2013
@@ -17,6 +17,8 @@
  */
 package org.apache.ambari.server.agent;
 
+import java.util.Map;
+
 import org.codehaus.jackson.annotate.JsonProperty;
 
 
@@ -32,6 +34,8 @@ public class CommandReport {
   private String serviceName;
   private long taskId;
   
+  private Map<String, Map<String, String>> configurationTags;
+  
   @JsonProperty("taskId")
   public long getTaskId() {
     return taskId;
@@ -122,6 +126,21 @@ public class CommandReport {
     this.serviceName = serviceName;
   }
 
+  /**
+  * @param tags the config tags that match this command
+   */
+  public void setConfigTags(Map<String, Map<String,String>> tags) {
+    configurationTags = tags;
+  }
+  
+  /**
+   * @return the config tags that match this command, or <code>null</code>
+   * if none are present
+   */
+  public Map<String, Map<String,String>> getConfigTags() {
+    return configurationTags;
+  }
+
   @Override
   public String toString() {
     return "CommandReport{" +

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java Mon Mar 18 14:06:19 2013
@@ -51,6 +51,7 @@ public class ExecutionCommand extends Ag
   private Map<String, List<String>> clusterHostInfo = 
       new HashMap<String, List<String>>();
   private Map<String, Map<String, String>> configurations;
+  private Map<String, Map<String, String>> configurationTags;
   private Map<String, String> commandParams;
   private String serviceName;
   
@@ -201,4 +202,18 @@ public class ExecutionCommand extends Ag
     this.serviceName = serviceName;
   }
 
+  /**
+   * @param configTags the config tag map
+   */
+  public void setConfigurationTags(Map<String, Map<String, String>> configTags) {
+    configurationTags = configTags;
+  }  
+
+  /**
+   * @return the configuration tags 
+   */
+  public Map<String, Map<String, String>> getConfigurationTags() {
+    return configurationTags;
+  }
+
 }

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java Mon Mar 18 14:06:19 2013
@@ -189,6 +189,10 @@ public class HeartBeatHandler {
         LOG.info(report.getRole() + " is an action - skip component lookup");
       } else {
         try {
+          if (null != report.getConfigTags() && !report.getConfigTags().isEmpty()) {
+            cl.updateActualConfigs(hostname, report.getConfigTags());
+          }
+          
           Service svc = cl.getService(service);
           ServiceComponent svcComp = svc.getServiceComponent(report.getRole());
           ServiceComponentHost scHost = svcComp.getServiceComponentHost(hostname);

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java Mon Mar 18 14:06:19 2013
@@ -901,6 +901,7 @@ public class AmbariManagementControllerI
   private void createHostAction(Cluster cluster,
       Stage stage, ServiceComponentHost scHost,
       Map<String, Map<String, String>> configurations,
+      Map<String, Map<String, String>> configTags,
       RoleCommand command,
       Map<String, String> commandParams,
       ServiceComponentHostEvent event) throws AmbariException {
@@ -919,7 +920,7 @@ public class AmbariManagementControllerI
     Host host = clusters.getHost(scHost.getHostName());
 
     execCmd.setConfigurations(configurations);
-
+    execCmd.setConfigurationTags(configTags);
     execCmd.setCommandParams(commandParams);
 
     // send stack info to agent
@@ -972,12 +973,14 @@ public class AmbariManagementControllerI
       Cluster c = clusters.getCluster(request.getClusterName());
       ClusterResponse cr = c.convertToResponse();
       cr.setDesiredConfigs(c.getDesiredConfigs());
+      cr.setActualConfigs(c.getActualConfigs());
       response.add(cr);
       return response;
     } else if (request.getClusterId() != null) {
       Cluster c = clusters.getClusterById(request.getClusterId());
       ClusterResponse cr = c.convertToResponse();
       cr.setDesiredConfigs(c.getDesiredConfigs());
+      cr.setActualConfigs(c.getActualConfigs());
       response.add(cr);
       return response;
     }
@@ -1931,6 +1934,7 @@ public class AmbariManagementControllerI
 
             // [ type -> [ key, value ] ]
             Map<String, Map<String, String>> configurations = new TreeMap<String, Map<String,String>>();
+            Map<String, Map<String, String>> configTags = new HashMap<String, Map<String,String>>();
 
             // Do not use host component config mappings.  Instead, the rules are:
             // 1) Use the cluster desired config
@@ -1950,6 +1954,8 @@ public class AmbariManagementControllerI
                 continue;
 
               Map<String, String> props = new HashMap<String, String>(config.getProperties());
+              Map<String, String> tags = new HashMap<String, String>();
+              tags.put("tag", config.getVersionTag());
 
               // 2) apply the service overrides, if any are defined with different tags
               Service service = cluster.getService(scHost.getServiceName());
@@ -1964,9 +1970,11 @@ public class AmbariManagementControllerI
               if (null != dc) {
                 Config hostConfig = cluster.getConfig(svcConfig.getType(), dc.getVersion());
                 props.putAll(hostConfig.getProperties());
+                tags.put("host_override_tag", hostConfig.getVersionTag());
               }
 
               configurations.put(type, props);
+              configTags.put(type, tags);
             }
 
             // HACK HACK HACK
@@ -1979,7 +1987,7 @@ public class AmbariManagementControllerI
               configurations.get("global").put("rca_enabled", "false");
             }
 
-            createHostAction(cluster, stage, scHost, configurations,
+            createHostAction(cluster, stage, scHost, configurations, configTags,
                 roleCommand, requestParameters, event);
           }
         }

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/ClusterResponse.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/ClusterResponse.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/ClusterResponse.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/ClusterResponse.java Mon Mar 18 14:06:19 2013
@@ -34,6 +34,8 @@ public class ClusterResponse {
   private final String desiredStackVersion;
 
   private Map<String, DesiredConfig> desiredConfigs;
+  
+  private Map<String, DesiredConfig> actualConfigs;
 
   public ClusterResponse(Long clusterId, String clusterName,
       Set<String> hostNames, String desiredStackVersion) {
@@ -134,4 +136,18 @@ public class ClusterResponse {
     return desiredConfigs;
   }
 
+  /**
+   * @param actualConfigs the actual configs
+   */
+  public void setActualConfigs(Map<String, DesiredConfig> configs) {
+    actualConfigs = configs;
+  }
+  
+  /**
+   * @return the actual configs
+   */
+  public Map<String, DesiredConfig> getActualConfigs() {
+    return actualConfigs;
+  }
+
 }

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java Mon Mar 18 14:06:19 2013
@@ -53,6 +53,7 @@ class ClusterResourceProvider extends Ab
   protected static final String CLUSTER_NAME_PROPERTY_ID    = PropertyHelper.getPropertyId("Clusters", "cluster_name");
   protected static final String CLUSTER_VERSION_PROPERTY_ID = PropertyHelper.getPropertyId("Clusters", "version");
   protected static final String CLUSTER_DESIRED_CONFIGS_PROPERTY_ID = PropertyHelper.getPropertyId("Clusters", "desired_configs");
+  protected static final String CLUSTER_ACTUAL_CONFIGS_PROPERTY_ID = PropertyHelper.getPropertyId("Clusters", "actual_configs");  
 
 
   private static Set<String> pkPropertyIds =
@@ -122,6 +123,7 @@ class ClusterResourceProvider extends Ab
       setResourceProperty(resource, CLUSTER_ID_PROPERTY_ID, response.getClusterId(), requestedIds);
       setResourceProperty(resource, CLUSTER_NAME_PROPERTY_ID, response.getClusterName(), requestedIds);
       setResourceProperty(resource, CLUSTER_DESIRED_CONFIGS_PROPERTY_ID, response.getDesiredConfigs(), requestedIds);
+      setResourceProperty(resource, CLUSTER_ACTUAL_CONFIGS_PROPERTY_ID, response.getActualConfigs(), requestedIds);
 
       resource.setProperty(CLUSTER_VERSION_PROPERTY_ID,
           response.getDesiredStackVersion());

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java Mon Mar 18 14:06:19 2013
@@ -196,4 +196,17 @@ public interface Cluster {
    */
   Service addService(String serviceName) throws AmbariException;
 
+  /**
+   * Gets the mapping of actual configurations as reported by a host.
+   * @return the map of actual configs
+   */
+  public Map<String, DesiredConfig> getActualConfigs();
+
+  /**
+   * Updates the actual configs as reported by the host
+   * @param hostName the host that reported a successful configuration change
+   * @param configTags the tags from the host as applied to the cluster
+   */
+  public void updateActualConfigs(String hostName, Map<String, Map<String,String>> configTags);
+
 }

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/DesiredConfig.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/DesiredConfig.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/DesiredConfig.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/DesiredConfig.java Mon Mar 18 14:06:19 2013
@@ -17,6 +17,7 @@
  */
 package org.apache.ambari.server.state;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.codehaus.jackson.annotate.JsonProperty;
@@ -31,7 +32,7 @@ public class DesiredConfig {
 
   private String versionTag;
   private String serviceName;
-  private List<HostOverride> hostOverrides = null;
+  private List<HostOverride> hostOverrides = new ArrayList<HostOverride>();
 
   /**
    * Sets the version tag
@@ -134,7 +135,7 @@ public class DesiredConfig {
       {
         if (i++ != 0)
           sb.append(",");
-        sb.append(h.getName());
+        sb.append(h.getName()).append(':').append(h.getVersionTag());
       }
 
       sb.append(']');

Modified: incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java (original)
+++ incubator/ambari/trunk/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java Mon Mar 18 14:06:19 2013
@@ -19,6 +19,7 @@
 package org.apache.ambari.server.state.cluster;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -52,6 +53,7 @@ import org.apache.ambari.server.state.Cl
 import org.apache.ambari.server.state.Config;
 import org.apache.ambari.server.state.ConfigFactory;
 import org.apache.ambari.server.state.DesiredConfig;
+import org.apache.ambari.server.state.DesiredConfig.HostOverride;
 import org.apache.ambari.server.state.Service;
 import org.apache.ambari.server.state.ServiceComponent;
 import org.apache.ambari.server.state.ServiceComponentHost;
@@ -82,6 +84,11 @@ public class ClusterImpl implements Clus
    * [ Config Type -> [ Config Version Tag -> Config ] ]
    */
   private Map<String, Map<String, Config>> allConfigs;
+  
+  /**
+   * [ type -> DesiredConfig ]
+   */
+  private Map<String, DesiredConfig> actualConfig = new HashMap<String, DesiredConfig>();
 
   /**
    * [ ServiceName -> [ ServiceComponentName -> [ HostName -> [ ... ] ] ] ]
@@ -743,32 +750,93 @@ public class ClusterImpl implements Clus
     clusterDAO.merge(clusterEntity);
 
   }
-
+  
+  
   @Override
-  public Map<String, DesiredConfig> getDesiredConfigs() {
-    Map<String, DesiredConfig> map = new HashMap<String, DesiredConfig>();
-
-    for (ClusterConfigMappingEntity e : clusterEntity.getConfigMappingEntities()) {
-      if (e.isSelected() > 0) {
-        DesiredConfig c = new DesiredConfig();
-        c.setServiceName(null);
-        c.setVersion(e.getVersion());
-
-        List<HostConfigMappingEntity> hostMappings =
-            hostConfigMappingDAO.findSelectedHostsByType(clusterEntity.getClusterId().longValue(),
-                e.getType());
+  public void updateActualConfigs(String hostName, Map<String, Map<String,String>> configTags) {
+    readWriteLock.writeLock().lock();
+    try {
 
-        List<DesiredConfig.HostOverride> hosts = new ArrayList<DesiredConfig.HostOverride>();
-        for (HostConfigMappingEntity mappingEntity : hostMappings) {
-          hosts.add (new DesiredConfig.HostOverride(mappingEntity.getHostName(),
-              mappingEntity.getVersion()));
+      for (Entry<String, Map<String,String>> entry : configTags.entrySet()) {
+        String type = entry.getKey();
+        Map<String, String> values = entry.getValue();
+        
+        String tag = values.get("tag");
+        String hostTag = values.get("host_override_tag");
+  
+        if (actualConfig.containsKey(type)) {
+          DesiredConfig dc = actualConfig.get(type);
+          dc.setVersion(tag);
+          
+          boolean needNew = false;
+          Iterator<HostOverride> it = dc.getHostOverrides().iterator();
+          while (it.hasNext()) {
+            HostOverride override = it.next();
+            if (null != hostName && override.getName().equals(hostName)) {
+              needNew = true;
+              it.remove();
+            }
+          }
+          
+          if (null != hostTag && null != hostName) {
+            dc.getHostOverrides().add(new HostOverride(hostName, hostTag));
+          }
         }
+        else {
+          DesiredConfig dc = new DesiredConfig();
+          dc.setVersion(tag);
+          actualConfig.put(type, dc);
+          if (null != hostTag && null != hostName) {
+            List<HostOverride> list = new ArrayList<HostOverride>();
+            list.add (new HostOverride(hostName, hostTag));
+            dc.setHostOverrides(list);
+          }
+        }
+        
+        DesiredConfig dc = actualConfig.get(type);
+        if (null == dc) {
+          dc = new DesiredConfig();
+          dc.setVersion(tag);
+          actualConfig.put(type, dc);
+        }
+      }
+    }
+    finally {
+      readWriteLock.writeLock().unlock();
+    }
+  }
+  
+  @Override
+  public Map<String, DesiredConfig> getActualConfigs() {
+    return actualConfig;
+  }
 
-        c.setHostOverrides(hosts);
-
-        map.put(e.getType(), c);
+  @Override
+  public Map<String, DesiredConfig> getDesiredConfigs() {
 
-      }
+      Map<String, DesiredConfig> map = new HashMap<String, DesiredConfig>();
+  
+      for (ClusterConfigMappingEntity e : clusterEntity.getConfigMappingEntities()) {
+        if (e.isSelected() > 0) {
+          DesiredConfig c = new DesiredConfig();
+          c.setServiceName(null);
+          c.setVersion(e.getVersion());
+  
+          List<HostConfigMappingEntity> hostMappings =
+              hostConfigMappingDAO.findSelectedHostsByType(clusterEntity.getClusterId().longValue(),
+                  e.getType());
+  
+          List<DesiredConfig.HostOverride> hosts = new ArrayList<DesiredConfig.HostOverride>();
+          for (HostConfigMappingEntity mappingEntity : hostMappings) {
+            hosts.add (new DesiredConfig.HostOverride(mappingEntity.getHostName(),
+                mappingEntity.getVersion()));
+          }
+  
+          c.setHostOverrides(hosts);
+  
+          map.put(e.getType(), c);
+  
+        }
     }
 
     return map;

Modified: incubator/ambari/trunk/ambari-server/src/main/resources/properties.json
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/main/resources/properties.json?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/main/resources/properties.json (original)
+++ incubator/ambari/trunk/ambari-server/src/main/resources/properties.json Mon Mar 18 14:06:19 2013
@@ -5,6 +5,7 @@
         "Clusters/version",
         "Clusters/state",
         "Clusters/desired_configs",
+        "Clusters/actual_configs",
         "_"
     ],
     "Service":[

Modified: incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java (original)
+++ incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatHandler.java Mon Mar 18 14:06:19 2013
@@ -31,7 +31,6 @@ import static org.apache.ambari.server.a
 import static org.apache.ambari.server.agent.DummyHeartbeatConstants.HDFS_CLIENT;
 import static org.apache.ambari.server.agent.DummyHeartbeatConstants.NAMENODE;
 import static org.apache.ambari.server.agent.DummyHeartbeatConstants.SECONDARY_NAMENODE;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -47,6 +46,7 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import javax.xml.bind.JAXBException;
@@ -62,21 +62,7 @@ import org.apache.ambari.server.actionma
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
 import org.apache.ambari.server.actionmanager.Stage;
-import org.apache.ambari.server.agent.ActionQueue;
-import org.apache.ambari.server.agent.CommandReport;
-import org.apache.ambari.server.agent.ComponentStatus;
-import org.apache.ambari.server.agent.ExecutionCommand;
-import org.apache.ambari.server.agent.HeartBeat;
-import org.apache.ambari.server.agent.HeartBeatHandler;
-import org.apache.ambari.server.agent.HeartBeatResponse;
-import org.apache.ambari.server.agent.HeartbeatMonitor;
-import org.apache.ambari.server.agent.HostInfo;
-import org.apache.ambari.server.agent.HostStatus;
 import org.apache.ambari.server.agent.HostStatus.Status;
-import org.apache.ambari.server.agent.Register;
-import org.apache.ambari.server.agent.RegistrationResponse;
-import org.apache.ambari.server.agent.RegistrationStatus;
-import org.apache.ambari.server.agent.StatusCommand;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.controller.HostsMap;
@@ -170,6 +156,68 @@ public class TestHeartbeatHandler {
     assertEquals(HostState.HEALTHY, hostObject.getState());
     assertEquals(0, aq.dequeueAll(DummyHostname1).size());
   }
+  
+  @Test
+  public void testHeartbeatWithConfigs() throws Exception {
+    ActionManager am = getMockActionManager();
+
+    Cluster cluster = getDummyCluster();
+    
+    @SuppressWarnings("serial")
+    Set<String> hostNames = new HashSet<String>(){{
+      add(DummyHostname1);
+    }};
+    clusters.mapHostsToCluster(hostNames, DummyCluster);
+    Service hdfs = cluster.addService(HDFS);
+    hdfs.persist();
+    hdfs.addServiceComponent(DATANODE).persist();
+    hdfs.getServiceComponent(DATANODE).addServiceComponentHost(DummyHostname1).persist();
+    hdfs.addServiceComponent(NAMENODE).persist();
+    hdfs.getServiceComponent(NAMENODE).addServiceComponentHost(DummyHostname1).persist();
+    hdfs.addServiceComponent(SECONDARY_NAMENODE).persist();
+    hdfs.getServiceComponent(SECONDARY_NAMENODE).addServiceComponentHost(DummyHostname1).persist();
+    
+    ActionQueue aq = new ActionQueue();
+    HeartBeatHandler handler = getHeartBeatHandler(am, aq);
+    
+    ServiceComponentHost serviceComponentHost1 = clusters.getCluster(DummyCluster).getService(HDFS).
+            getServiceComponent(DATANODE).getServiceComponentHost(DummyHostname1);
+    ServiceComponentHost serviceComponentHost2 = clusters.getCluster(DummyCluster).getService(HDFS).
+            getServiceComponent(NAMENODE).getServiceComponentHost(DummyHostname1);
+    serviceComponentHost1.setState(State.INSTALLED);
+    serviceComponentHost2.setState(State.INSTALLED);
+
+
+    HeartBeat hb = new HeartBeat();
+    hb.setResponseId(0);
+    hb.setNodeStatus(new HostStatus(Status.HEALTHY, DummyHostStatus));
+    hb.setHostname(DummyHostname1);
+    
+    List<CommandReport> reports = new ArrayList<CommandReport>();
+    CommandReport cr = new CommandReport();
+    cr.setActionId(StageUtils.getActionId(requestId, stageId));
+    cr.setServiceName(HDFS);
+    cr.setTaskId(1);
+    cr.setRole(HDFS);
+    cr.setStatus("COMPLETED");
+    cr.setStdErr("");
+    cr.setStdOut("");
+    cr.setExitCode(215);
+    cr.setClusterName(DummyCluster);
+    
+    cr.setConfigTags(new HashMap<String, Map<String,String>>() {{
+      put("global", new HashMap<String,String>() {{ put("tag", "version1"); }});
+    }});
+    
+    reports.add(cr);
+    hb.setReports(reports);
+    
+    handler.handleHeartBeat(hb);
+
+    // the heartbeat test passed if actual configs is populated
+    Assert.assertNotNull(cluster.getActualConfigs());
+    Assert.assertEquals(cluster.getActualConfigs().size(), 1);
+  }  
 
   @Test
   public void testStatusHeartbeat() throws Exception {
@@ -301,7 +349,7 @@ public class TestHeartbeatHandler {
     assertEquals(State.STARTED, componentState1);
     assertEquals(State.INSTALLED, componentState2);
   }
-
+  
   @Test
   public void testCommandReport() throws AmbariException {
     injector.injectMembers(this);

Modified: incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/controller/ClusterResponseTest.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/controller/ClusterResponseTest.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/controller/ClusterResponseTest.java (original)
+++ incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/controller/ClusterResponseTest.java Mon Mar 18 14:06:19 2013
@@ -18,9 +18,12 @@ package org.apache.ambari.server.control
  * limitations under the License.
  */
 
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 
+import org.apache.ambari.server.state.DesiredConfig;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -35,11 +38,19 @@ public class ClusterResponseTest {
 
     ClusterResponse r1 =
         new ClusterResponse(clusterId, clusterName, hostNames, "bar");
-
+    
+    Map<String, DesiredConfig> actual = new HashMap<String, DesiredConfig>();
+    DesiredConfig dc = new DesiredConfig();
+    dc.setVersion("version1");
+    actual.put("global", dc);
+    
+    r1.setActualConfigs(actual);
+    
     Assert.assertEquals(clusterId, r1.getClusterId());
     Assert.assertEquals(clusterName, r1.getClusterName());
     Assert.assertArrayEquals(hostNames.toArray(), r1.getHostNames().toArray());
     Assert.assertEquals("bar", r1.getDesiredStackVersion());
+    Assert.assertNotNull(r1.getActualConfigs());
 
   }
 

Modified: incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/DesiredConfigTest.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/DesiredConfigTest.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/DesiredConfigTest.java (original)
+++ incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/DesiredConfigTest.java Mon Mar 18 14:06:19 2013
@@ -36,7 +36,8 @@ public class DesiredConfigTest {
 
     Assert.assertEquals("Expected service 'service'", "service", dc.getServiceName());
     Assert.assertEquals("Expected version 'global'", "global", dc.getVersion());
-    Assert.assertNull("Expected null host overrides", dc.getHostOverrides());
+    Assert.assertEquals("Expected no host overrides", 0, dc.getHostOverrides().size());
+    
 
     List<DesiredConfig.HostOverride> hosts = Arrays.asList(
         new DesiredConfig.HostOverride("h1", "v2"),

Modified: incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java (original)
+++ incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java Mon Mar 18 14:06:19 2013
@@ -54,6 +54,7 @@ import org.apache.ambari.server.state.Cl
 import org.apache.ambari.server.state.Config;
 import org.apache.ambari.server.state.ConfigFactory;
 import org.apache.ambari.server.state.DesiredConfig;
+import org.apache.ambari.server.state.DesiredConfig.HostOverride;
 import org.apache.ambari.server.state.Host;
 import org.apache.ambari.server.state.HostState;
 import org.apache.ambari.server.state.Service;
@@ -364,6 +365,127 @@ public class ClusterTest {
     Assert.assertEquals("Expect one host-level override", 1, dc.getHostOverrides().size());
   }
   
+  @Test
+  public void testActualConfigs() throws Exception {
+    
+    Assert.assertEquals (0, c1.getActualConfigs().size());
+    
+    c1.updateActualConfigs("h1",
+        new HashMap<String, Map<String,String>>() {{
+          put("global", new HashMap<String,String>() {{ put("tag", "version1"); }});
+        }});
+    Map<String, DesiredConfig> actual = c1.getActualConfigs();
+    Assert.assertEquals(1, actual.size());
+    Assert.assertNotNull(actual.get("global"));
+    Assert.assertEquals("version1", actual.get("global").getVersion());
+
+    // change global version
+    c1.updateActualConfigs("h1",
+        new HashMap<String, Map<String,String>>() {{
+          put("global", new HashMap<String,String>() {{ put("tag", "version2"); }});
+        }});
+    actual = c1.getActualConfigs();
+    Assert.assertEquals(1, actual.size());
+    Assert.assertNotNull(actual.get("global"));
+    Assert.assertEquals("version2", actual.get("global").getVersion());
+
+
+   // add a host override
+   c1.updateActualConfigs("h1",
+       new HashMap<String, Map<String,String>>() {{
+         put("global", new HashMap<String,String>() {{
+           put("tag", "version2");
+           put("host_override_tag", "xxyyzz");
+         }});
+       }});
+   actual = c1.getActualConfigs();
+   Assert.assertEquals(1, actual.size());
+   Assert.assertNotNull(actual.get("global"));
+   Assert.assertEquals("version2", actual.get("global").getVersion());
+   Assert.assertEquals(1, actual.get("global").getHostOverrides().size());
+
+   // add another host
+   c1.updateActualConfigs("h2",
+       new HashMap<String, Map<String,String>>() {{
+         put("global", new HashMap<String,String>() {{
+           put("tag", "version2");
+           put("host_override_tag", "aabbcc");
+         }});
+       }});
+   actual = c1.getActualConfigs();
+   Assert.assertEquals(1, actual.size());
+   Assert.assertNotNull(actual.get("global"));
+   Assert.assertEquals("version2", actual.get("global").getVersion());
+   Assert.assertEquals(2, actual.get("global").getHostOverrides().size());
+   for (HostOverride o : actual.get("global").getHostOverrides()) {
+     if (o.getName().equals("h1")) {
+       Assert.assertEquals(o.getVersionTag(), "xxyyzz");
+     } else {
+       Assert.assertEquals(o.getVersionTag(), "aabbcc");
+     }
+   }
+   
+   // remove h2 override
+   c1.updateActualConfigs("h2",
+       new HashMap<String, Map<String,String>>() {{
+         put("global", new HashMap<String,String>() {{
+           put("tag", "version3");
+         }});
+       }});
+   actual = c1.getActualConfigs();
+   Assert.assertEquals(1, actual.size());
+   Assert.assertNotNull(actual.get("global"));
+   Assert.assertEquals("version3", actual.get("global").getVersion());
+   Assert.assertEquals(1, actual.get("global").getHostOverrides().size());
+   Assert.assertEquals("h1", actual.get("global").getHostOverrides().get(0).getName());
+
+   
+   // change h1 override
+   c1.updateActualConfigs("h1",
+       new HashMap<String, Map<String,String>>() {{
+         put("global", new HashMap<String,String>() {{
+           put("tag", "version2");
+           put("host_override_tag", "mmnnoo");
+         }});
+       }});
+   actual = c1.getActualConfigs();
+   Assert.assertEquals(1, actual.size());
+   Assert.assertNotNull(actual.get("global"));
+   Assert.assertEquals("version2", actual.get("global").getVersion());
+   Assert.assertEquals(1, actual.get("global").getHostOverrides().size());
+   Assert.assertEquals("h1", actual.get("global").getHostOverrides().get(0).getName());
+   Assert.assertEquals("mmnnoo", actual.get("global").getHostOverrides().get(0).getVersionTag());
+   
+   // remove h1 override
+   c1.updateActualConfigs("h1",
+       new HashMap<String, Map<String,String>>() {{
+         put("global", new HashMap<String,String>() {{
+           put("tag", "version2");
+         }});
+       }});
+   Assert.assertEquals(1, actual.size());
+   Assert.assertNotNull(actual.get("global"));
+   Assert.assertEquals("version2", actual.get("global").getVersion());
+   Assert.assertEquals(0, actual.get("global").getHostOverrides().size());
+
+   
+   // create new one with override, not as an update
+   // remove h1 override
+   c1.updateActualConfigs("h1",
+       new HashMap<String, Map<String,String>>() {{
+         put("core-site", new HashMap<String,String>() {{
+           put("tag", "version4");
+           put("host_override_tag", "qqrrss");
+         }});
+       }});
+   Assert.assertEquals(2, actual.size());
+   Assert.assertNotNull(actual.get("global"));
+   Assert.assertNotNull(actual.get("core-site"));
+   Assert.assertEquals("version4", actual.get("core-site").getVersion());
+   Assert.assertEquals("qqrrss", actual.get("core-site").getHostOverrides().get(0).getVersionTag());
+   
+  }  
+  
   public ClusterEntity createDummyData() {
     ClusterEntity clusterEntity = new ClusterEntity();
     clusterEntity.setClusterName("test_cluster1");

Modified: incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/utils/TestStageUtils.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/utils/TestStageUtils.java?rev=1457772&r1=1457771&r2=1457772&view=diff
==============================================================================
--- incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/utils/TestStageUtils.java (original)
+++ incubator/ambari/trunk/ambari-server/src/test/java/org/apache/ambari/server/utils/TestStageUtils.java Mon Mar 18 14:06:19 2013
@@ -20,6 +20,7 @@ package org.apache.ambari.server.utils;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -34,9 +35,10 @@ import org.apache.ambari.server.configur
 import org.apache.ambari.server.controller.HostsMap;
 import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
-import org.apache.ambari.server.state.*;
-import org.apache.ambari.server.state.cluster.ClustersImpl;
-import org.apache.ambari.server.state.svccomphost.ServiceComponentHostImpl;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.state.ServiceComponentHostFactory;
+import org.apache.ambari.server.state.StackId;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.codehaus.jackson.JsonGenerationException;
@@ -143,6 +145,11 @@ public class TestStageUtils {
       JsonMappingException, JAXBException, IOException {
     Stage s = StageUtils.getATestStage(1, 2, "host1");
     ExecutionCommand cmd = s.getExecutionCommands("host1").get(0).getExecutionCommand();
+    cmd.setConfigurationTags(new HashMap<String, Map<String,String>>() {{
+      put("global", new HashMap<String, String>() {{ put("tag", "version1"); }});
+    }});
+    
+    
     String json = StageUtils.jaxbToString(cmd);
     ExecutionCommand cmdDes = StageUtils.stringToExecutionCommand(json);
     assertEquals(cmd.toString(), cmdDes.toString());