You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by js...@apache.org on 2014/04/15 20:36:09 UTC

git commit: AMBARI-5462. Allow host group scoped configuration to be specified in blueprint. This configuration is specified inline within host groups.

Repository: ambari
Updated Branches:
  refs/heads/trunk 5ef841a0e -> dd5f864f0


AMBARI-5462.  Allow host group scoped configuration to be specified in blueprint.
This configuration is specified inline within host groups.


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

Branch: refs/heads/trunk
Commit: dd5f864f05564e36fcc5c40b2fad9dd4fa45a51e
Parents: 5ef841a
Author: John Speidel <js...@hortonworks.com>
Authored: Mon Apr 14 12:18:31 2014 -0400
Committer: John Speidel <js...@hortonworks.com>
Committed: Tue Apr 15 12:42:15 2014 -0400

----------------------------------------------------------------------
 .../internal/BlueprintResourceProvider.java     | 131 +++++++++++-----
 .../internal/ClusterResourceProvider.java       | 121 ++++++++++++++-
 .../internal/ConfigGroupResourceProvider.java   |  55 ++++---
 .../orm/entities/BlueprintConfigEntity.java     |   2 +-
 .../orm/entities/BlueprintConfiguration.java    |  68 +++++++++
 .../orm/entities/HostGroupConfigEntity.java     | 152 +++++++++++++++++++
 .../orm/entities/HostGroupConfigEntityPK.java   | 113 ++++++++++++++
 .../server/orm/entities/HostGroupEntity.java    |  21 +++
 .../server/upgrade/SchemaUpgradeHelper.java     |   1 +
 .../server/upgrade/UpgradeCatalog151.java       |   4 +-
 .../server/upgrade/UpgradeCatalog160.java       |  78 ++++++++++
 .../main/resources/Ambari-DDL-MySQL-CREATE.sql  |   2 +
 .../main/resources/Ambari-DDL-Oracle-CREATE.sql |   2 +
 .../resources/Ambari-DDL-Postgres-CREATE.sql    |   3 +
 .../src/main/resources/META-INF/persistence.xml |   1 +
 .../internal/BlueprintResourceProviderTest.java |  35 ++++-
 .../internal/ClusterResourceProviderTest.java   |  81 +++++++---
 .../entities/HostGroupConfigEntityPKTest.java   |  93 ++++++++++++
 .../orm/entities/HostGroupConfigEntityTest.java |  66 ++++++++
 .../server/upgrade/UpgradeCatalog160Test.java   | 143 +++++++++++++++++
 20 files changed, 1085 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintResourceProvider.java
index ee3ff3b..ea2fdec 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintResourceProvider.java
@@ -34,8 +34,10 @@ import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
 import org.apache.ambari.server.orm.dao.BlueprintDAO;
 import org.apache.ambari.server.orm.entities.BlueprintConfigEntity;
+import org.apache.ambari.server.orm.entities.BlueprintConfiguration;
 import org.apache.ambari.server.orm.entities.BlueprintEntity;
 import org.apache.ambari.server.orm.entities.HostGroupComponentEntity;
+import org.apache.ambari.server.orm.entities.HostGroupConfigEntity;
 import org.apache.ambari.server.orm.entities.HostGroupEntity;
 
 import java.util.ArrayList;
@@ -85,6 +87,9 @@ public class BlueprintResourceProvider extends AbstractResourceProvider {
    */
   private static BlueprintDAO dao;
 
+  /**
+   * Used to serialize to/from json.
+   */
   private static Gson jsonSerializer;
 
 
@@ -246,24 +251,15 @@ public class BlueprintResourceProvider extends AbstractResourceProvider {
         listComponentProps.add(mapComponentProps);
       }
       mapGroupProps.put(COMPONENT_PROPERTY_ID, listComponentProps);
+      mapGroupProps.put(CONFIGURATION_PROPERTY_ID, populateConfigurationList(
+          hostGroup.getConfigurations()));
     }
     setResourceProperty(resource, HOST_GROUP_PROPERTY_ID, listGroupProps, requestedIds);
-
-    List<Map<String, Object>> listConfigurations = new ArrayList<Map<String, Object>>();
-    Collection<BlueprintConfigEntity> configurations = entity.getConfigurations();
-    for (BlueprintConfigEntity config : configurations) {
-      Map<String, Object> mapConfigurations = new HashMap<String, Object>();
-      String type = config.getType();
-      Map<String, String> properties = jsonSerializer.<Map<String, String>>fromJson(
-          config.getConfigData(), Map.class);
-      mapConfigurations.put(type, properties);
-      listConfigurations.add(mapConfigurations);
-    }
-    setResourceProperty(resource, CONFIGURATION_PROPERTY_ID, listConfigurations, requestedIds);
+    setResourceProperty(resource, CONFIGURATION_PROPERTY_ID,
+        populateConfigurationList(entity.getConfigurations()), requestedIds);
 
     return resource;
   }
-
   /**
    * Convert a resource to a blueprint entity.
    *
@@ -292,6 +288,8 @@ public class BlueprintResourceProvider extends AbstractResourceProvider {
 
       Collection<HostGroupComponentEntity> hostGroupComponents = new ArrayList<HostGroupComponentEntity>();
       group.setComponents(hostGroupComponents);
+      createHostGroupConfigEntities((Collection<Map<String,
+          String>>) properties.get(CONFIGURATION_PROPERTY_ID), group);
 
       List<Map<String, String>> listComponents = (List<Map<String, String>>)
           properties.get(BlueprintResourceProvider.COMPONENT_PROPERTY_ID);
@@ -308,8 +306,8 @@ public class BlueprintResourceProvider extends AbstractResourceProvider {
       blueprintHostGroups.add(group);
     }
 
-    entity.setConfigurations(createConfigEntities(
-        (Collection<Map<String, String>>) resource.getPropertyValue(CONFIGURATION_PROPERTY_ID), entity));
+    createBlueprintConfigEntities((Collection<Map<String,
+        String>>) resource.getPropertyValue(CONFIGURATION_PROPERTY_ID), entity);
 
     return entity;
   }
@@ -344,6 +342,8 @@ public class BlueprintResourceProvider extends AbstractResourceProvider {
       group.setBlueprintEntity(blueprint);
       group.setBlueprintName(name);
       group.setCardinality((String) hostGroupProperties.get(HOST_GROUP_CARDINALITY_PROPERTY_ID));
+      createHostGroupConfigEntities((Collection<Map<String,
+          String>>) hostGroupProperties.get(CONFIGURATION_PROPERTY_ID), group);
 
       Collection<HostGroupComponentEntity> components = new ArrayList<HostGroupComponentEntity>();
       group.setComponents(components);
@@ -362,46 +362,101 @@ public class BlueprintResourceProvider extends AbstractResourceProvider {
       blueprintHostGroups.add(group);
     }
 
-    blueprint.setConfigurations(createConfigEntities(
-        (Collection<Map<String, String>>) properties.get(CONFIGURATION_PROPERTY_ID), blueprint));
+    createBlueprintConfigEntities((Collection<Map<String,
+        String>>) properties.get(CONFIGURATION_PROPERTY_ID), blueprint);
 
     return blueprint;
   }
 
   /**
-   * Create blueprint configuration entities from properties.
+   * Populate a list of configuration property maps from a collection of configuration entities.
    *
-   * @param setConfigurations  set of property maps
-   * @param blueprint          blueprint entity
+   * @param configurations  collection of configuration entities
    *
-   * @return collection of blueprint config entities
+   * @return list of configuration property maps
    */
-  private Collection<BlueprintConfigEntity> createConfigEntities(Collection<Map<String, String>> setConfigurations,
-                                                                 BlueprintEntity blueprint) {
+  private List<Map<String, Object>> populateConfigurationList(
+      Collection<? extends BlueprintConfiguration> configurations) {
+
+    List<Map<String, Object>> listConfigurations = new ArrayList<Map<String, Object>>();
+    for (BlueprintConfiguration config : configurations) {
+      Map<String, Object> mapConfigurations = new HashMap<String, Object>();
+      String type = config.getType();
+      Map<String, String> properties = jsonSerializer.<Map<String, String>>fromJson(
+          config.getConfigData(), Map.class);
+      mapConfigurations.put(type, properties);
+      listConfigurations.add(mapConfigurations);
+    }
+
+    return listConfigurations;
+  }
+
+  /**
+   * Populate blueprint configurations.
+   *
+   * @param propertyMaps  collection of configuration property maps
+   * @param blueprint     blueprint entity to set configurations on
+   */
+  private void createBlueprintConfigEntities(Collection<Map<String, String>> propertyMaps,
+                                             BlueprintEntity blueprint) {
 
     Collection<BlueprintConfigEntity> configurations = new ArrayList<BlueprintConfigEntity>();
-    if (setConfigurations != null) {
-      for (Map<String, String> configuration : setConfigurations) {
+    if (propertyMaps != null) {
+      for (Map<String, String> configuration : propertyMaps) {
         BlueprintConfigEntity configEntity = new BlueprintConfigEntity();
         configEntity.setBlueprintEntity(blueprint);
-        configEntity.setBlueprintName(blueprint.getBlueprintName());
-        Map<String, String> configData = new HashMap<String, String>();
-
-        for (Map.Entry<String, String> entry : configuration.entrySet()) {
-          String absolutePropName = entry.getKey();
+        populateConfigurationEntity(blueprint.getBlueprintName(), configuration, configEntity);
+        configurations.add(configEntity);
+      }
+    }
+    blueprint.setConfigurations(configurations);
+  }
 
-          int idx = absolutePropName.indexOf('/');
-          if (configEntity.getType() == null) {
-            configEntity.setType(absolutePropName.substring(0, idx));
-          }
-          configData.put(absolutePropName.substring(idx + 1), entry.getValue());
-        }
-        configEntity.setConfigData(jsonSerializer.toJson(configData));
+  /**
+   * Populate host group configurations.
+   *
+   * @param propertyMaps  collection of configuration property maps
+   * @param hostGroup     host group entity to set configurations on
+   */
+  private void createHostGroupConfigEntities(Collection<Map<String, String>> propertyMaps,
+                                             HostGroupEntity hostGroup) {
+
+    Collection<HostGroupConfigEntity> configurations = new ArrayList<HostGroupConfigEntity>();
+    if (propertyMaps != null) {
+      for (Map<String, String> configuration : propertyMaps) {
+        HostGroupConfigEntity configEntity = new HostGroupConfigEntity();
+        configEntity.setHostGroupEntity(hostGroup);
+        configEntity.setHostGroupName(hostGroup.getName());
+        populateConfigurationEntity(hostGroup.getBlueprintName(), configuration, configEntity);
         configurations.add(configEntity);
       }
     }
+    hostGroup.setConfigurations(configurations);
+  }
 
-    return configurations;
+  /**
+   * Populate a configuration entity from properties.
+   *
+   * @param blueprintName  name of blueprint
+   * @param configuration  property map
+   * @param configEntity   config entity to populate
+   */
+  private void  populateConfigurationEntity(String blueprintName, Map<String, String> configuration,
+                                            BlueprintConfiguration configEntity) {
+
+    configEntity.setBlueprintName(blueprintName);
+    Map<String, String> configData = new HashMap<String, String>();
+
+    for (Map.Entry<String, String> entry : configuration.entrySet()) {
+      String absolutePropName = entry.getKey();
+
+      int idx = absolutePropName.indexOf('/');
+      if (configEntity.getType() == null) {
+        configEntity.setType(absolutePropName.substring(0, idx));
+      }
+      configData.put(absolutePropName.substring(idx + 1), entry.getValue());
+    }
+    configEntity.setConfigData(jsonSerializer.toJson(configData));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java
index 8d26cfa..18031d0 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClusterResourceProvider.java
@@ -32,6 +32,7 @@ import org.apache.ambari.server.api.services.PersistKeyValueService;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.ClusterRequest;
 import org.apache.ambari.server.controller.ClusterResponse;
+import org.apache.ambari.server.controller.ConfigGroupRequest;
 import org.apache.ambari.server.controller.ConfigurationRequest;
 import org.apache.ambari.server.controller.RequestStatusResponse;
 import org.apache.ambari.server.controller.StackConfigurationRequest;
@@ -56,7 +57,10 @@ import org.apache.ambari.server.orm.dao.BlueprintDAO;
 import org.apache.ambari.server.orm.entities.BlueprintConfigEntity;
 import org.apache.ambari.server.orm.entities.BlueprintEntity;
 import org.apache.ambari.server.orm.entities.HostGroupComponentEntity;
+import org.apache.ambari.server.orm.entities.HostGroupConfigEntity;
 import org.apache.ambari.server.orm.entities.HostGroupEntity;
+import org.apache.ambari.server.state.Config;
+import org.apache.ambari.server.state.ConfigImpl;
 
 /**
  * Resource provider for cluster resources.
@@ -266,12 +270,12 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
         (String) properties.get(CLUSTER_VERSION_PROPERTY_ID),
         null);
 
-    
+
     ConfigurationRequest configRequest = getConfigurationRequest("Clusters", properties);
-    
+
     if (null != configRequest)
       cr.setDesiredConfig(configRequest);
-    
+
     return cr;
   }
 
@@ -326,6 +330,8 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
     createServiceAndComponentResources(blueprintHostGroups, clusterName, services);
     createHostAndComponentResources(blueprintHostGroups, clusterName);
 
+    registerConfigGroups(clusterName, blueprintHostGroups, stack);
+
     persistInstallStateForUI();
     return ((ServiceResourceProvider) getResourceProviderByType(Resource.Type.Service)).
         installAndStart(clusterName);
@@ -868,6 +874,54 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
     return resultGroups;
   }
 
+  /**
+   * Register config groups for host group scoped configuration.
+   * For each host group with configuration specified in the blueprint, a config group is created
+   * and the hosts associated with the host group are assigned to the config group.
+   *
+   * @param clusterName  name of cluster
+   * @param hostGroups   map of host group name to host group
+   * @param stack        associated stack information
+   *
+   * @throws ResourceAlreadyExistsException attempt to create a config group that already exists
+   * @throws SystemException                an unexpected exception occurs
+   * @throws UnsupportedPropertyException   an invalid property is provided when creating a config group
+   * @throws NoSuchParentResourceException  attempt to create a config group for a non-existing cluster
+   */
+  private void registerConfigGroups(String clusterName, Map<String, HostGroup> hostGroups, Stack stack) throws
+      ResourceAlreadyExistsException, SystemException,
+      UnsupportedPropertyException, NoSuchParentResourceException {
+
+    for (HostGroup group : hostGroups.values()) {
+      HostGroupEntity entity = group.getEntity();
+      Map<String, Map<String, Config>> groupConfigs = new HashMap<String, Map<String, Config>>();
+      for (Map.Entry<String, Map<String, String>> entry: group.getConfigurations().entrySet()) {
+        String type = entry.getKey();
+        String service = stack.getServiceForConfigType(type);
+        Config config = new ConfigImpl(type);
+        config.setVersionTag(entity.getName());
+        config.setProperties(entry.getValue());
+        Map<String, Config> serviceConfigs = groupConfigs.get(service);
+        if (serviceConfigs == null) {
+          serviceConfigs = new HashMap<String, Config>();
+          groupConfigs.put(service, serviceConfigs);
+        }
+        serviceConfigs.put(type, config);
+      }
+
+      for (Map.Entry<String, Map<String, Config>> entry : groupConfigs.entrySet()) {
+        String service = entry.getKey();
+        Map<String, Config> serviceConfigs = entry.getValue();
+        ConfigGroupRequest request = new ConfigGroupRequest(
+            null, clusterName, entity.getName(), service, "Host Group Configuration",
+            new HashSet<String>(group.getHostInfo()), serviceConfigs);
+
+        ((ConfigGroupResourceProvider) getResourceProviderByType(Resource.Type.ConfigGroup)).
+            createResources(Collections.singleton(request));
+      }
+    }
+  }
+
 
   // ----- Inner Classes -----------------------------------------------------
 
@@ -995,6 +1049,24 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
     }
 
     /**
+     * Obtain the service name which corresponds to the specified configuration.
+     *
+     * @param config  configuration type
+     *
+     * @return name of service which corresponds to the specified configuration type
+     */
+    public String getServiceForConfigType(String config) {
+      for (Map.Entry<String, Map<String, Map<String, String>>> entry : serviceConfigurations.entrySet()) {
+        Map<String, Map<String, String>> typeMap = entry.getValue();
+        if (typeMap.containsKey(config)) {
+          return entry.getKey();
+        }
+      }
+      throw new IllegalArgumentException(
+          "Specified configuration type is not associated with any service: " + config);
+    }
+
+    /**
      * Parse components for the specified service from the stack definition.
      *
      * @param service  service name
@@ -1075,6 +1147,13 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
     private Map<String, Set<String>> componentsForService = new HashMap<String, Set<String>>();
 
     /**
+     * Map of host group configurations.
+     * Type -> Map<Key, Val>
+     */
+    private Map<String, Map<String, String>> configurations =
+        new HashMap<String, Map<String, String>>();
+
+    /**
      * Associated stack
      */
     private Stack stack;
@@ -1089,6 +1168,7 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
       this.hostGroup = hostGroup;
       this.stack = stack;
       parseComponents();
+      parseConfigurations();
     }
 
     /**
@@ -1130,6 +1210,24 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
     }
 
     /**
+     * Get the configurations associated with the host group.
+     *
+     * @return map of configuration type to a map of properties
+     */
+    public Map<String, Map<String, String>> getConfigurations() {
+      return configurations;
+    }
+
+    /**
+     * Get the associated entity.
+     *
+     * @return  associated host group entity
+     */
+    public HostGroupEntity getEntity() {
+      return hostGroup;
+    }
+
+    /**
      * Parse component information.
      */
     private void parseComponents() {
@@ -1145,6 +1243,23 @@ public class ClusterResourceProvider extends AbstractControllerResourceProvider
         serviceComponents.add(name);
       }
     }
+
+    /**
+     * Parse host group configurations.
+     */
+    private void parseConfigurations() {
+      Gson jsonSerializer = new Gson();
+      for (HostGroupConfigEntity configEntity : hostGroup.getConfigurations()) {
+        String type = configEntity.getType();
+        Map<String, String> typeProperties = configurations.get(type);
+        if ( typeProperties == null) {
+          typeProperties = new HashMap<String, String>();
+          configurations.put(type, typeProperties);
+        }
+        configurations.put(type, jsonSerializer.<Map<String, String>>fromJson(
+            configEntity.getConfigData(), Map.class));
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProvider.java
index e817473..7f50b00 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProvider.java
@@ -108,24 +108,9 @@ public class ConfigGroupResourceProvider extends
     for (Map<String, Object> propertyMap : request.getProperties()) {
       requests.add(getConfigGroupRequest(propertyMap));
     }
-    Set<ConfigGroupResponse> responses =
-      createResources(new Command<Set<ConfigGroupResponse>>() {
-        @Override
-        public Set<ConfigGroupResponse> invoke() throws AmbariException {
-          return createConfigGroups(requests);
-        }
-      });
-
+    RequestStatus status = createResources(requests);
     notifyCreate(Resource.Type.ConfigGroup, request);
-
-    Set<Resource> associatedResources = new HashSet<Resource>();
-    for (ConfigGroupResponse response : responses) {
-      Resource resource = new ResourceImpl(Resource.Type.ConfigGroup);
-      resource.setProperty(CONFIGGROUP_ID_PROPERTY_ID, response.getId());
-      associatedResources.add(resource);
-    }
-
-    return getRequestStatus(null, associatedResources);
+    return status;
   }
 
   @Override
@@ -221,6 +206,40 @@ public class ConfigGroupResourceProvider extends
     return getRequestStatus(null);
   }
 
+  /**
+   * Create configuration group resources based on set of config group requests.
+   *
+   * @param requests  set of config group requests
+   *
+   * @return a request status
+   *
+   * @throws SystemException                an internal system exception occurred
+   * @throws UnsupportedPropertyException   the request contains unsupported property ids
+   * @throws ResourceAlreadyExistsException attempted to create a resource which already exists
+   * @throws NoSuchParentResourceException  a parent resource of the resource to create doesn't exist
+   */
+  public RequestStatus createResources(final Set<ConfigGroupRequest> requests)throws
+      SystemException, UnsupportedPropertyException,
+      ResourceAlreadyExistsException, NoSuchParentResourceException{
+
+    Set<ConfigGroupResponse> responses =
+        createResources(new Command<Set<ConfigGroupResponse>>() {
+          @Override
+          public Set<ConfigGroupResponse> invoke() throws AmbariException {
+            return createConfigGroups(requests);
+          }
+        });
+
+    Set<Resource> associatedResources = new HashSet<Resource>();
+    for (ConfigGroupResponse response : responses) {
+      Resource resource = new ResourceImpl(Resource.Type.ConfigGroup);
+      resource.setProperty(CONFIGGROUP_ID_PROPERTY_ID, response.getId());
+      associatedResources.add(resource);
+    }
+
+    return getRequestStatus(null, associatedResources);
+  }
+
   private synchronized  Set<ConfigGroupResponse> getConfigGroups
     (Set<ConfigGroupRequest> requests) throws AmbariException {
     Set<ConfigGroupResponse> responses = new HashSet<ConfigGroupResponse>();
@@ -613,7 +632,7 @@ public class ConfigGroupResourceProvider extends
    * @param predicate  the predicate
    * @param resource   the resource
    *
-   * @return
+   * @return always returns true
    */
   @Override
   public boolean evaluate(Predicate predicate, Resource resource) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfigEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfigEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfigEntity.java
index 7572930..bc2e039 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfigEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfigEntity.java
@@ -32,7 +32,7 @@ import javax.persistence.Table;
 @javax.persistence.IdClass(BlueprintConfigEntityPK.class)
 @Table(name = "blueprint_configuration")
 @Entity
-public class BlueprintConfigEntity {
+public class BlueprintConfigEntity implements BlueprintConfiguration {
 
   @Id
   @Column(name = "blueprint_name", nullable = false, insertable = false, updatable = false)

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfiguration.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfiguration.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfiguration.java
new file mode 100644
index 0000000..761c7bb
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/BlueprintConfiguration.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.orm.entities;
+
+/**
+ * Blueprint configuration.
+ */
+public interface BlueprintConfiguration {
+  /**
+   * Set the configuration type.
+   *
+   * @param type configuration type
+   */
+  public void setType(String type);
+
+  /**
+   * Get the configuration type.
+   *
+   * @return configuration type
+   */
+  public String getType();
+
+  /**
+   * Set the blueprint name.
+   *
+   * @param blueprintName  blueprint name
+   */
+  public void setBlueprintName(String blueprintName);
+
+  /**
+   * Get the blueprint name.
+   *
+   * @return blueprint name
+   */
+  public String getBlueprintName();
+
+  /**
+   * Set the configuration properties.
+   * Data must be a map of configuration properties in
+   * json format.
+   *
+   * @param configData json representation of property map
+   */
+  public void setConfigData(String configData);
+
+  /**
+   * Get the configuration properties.
+   *
+   * @return json representation of property map
+   */
+  public String getConfigData();
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntity.java
new file mode 100644
index 0000000..db44eef
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntity.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.orm.entities;
+
+import javax.persistence.Basic;
+import javax.persistence.Column;
+import javax.persistence.Entity;
+import javax.persistence.Id;
+import javax.persistence.IdClass;
+import javax.persistence.JoinColumn;
+import javax.persistence.JoinColumns;
+import javax.persistence.ManyToOne;
+import javax.persistence.Table;
+
+/**
+ * Represents a blueprint host group configuration.
+ */
+@IdClass(HostGroupConfigEntityPK.class)
+@Table(name = "hostgroup_configuration")
+@Entity
+public class HostGroupConfigEntity implements BlueprintConfiguration {
+
+  @Id
+  @Column(name = "blueprint_name", nullable = false, insertable = false, updatable = false)
+  private String blueprintName;
+
+  @Id
+  @Column(name = "hostgroup_name", nullable = false, insertable = false, updatable = false)
+  private String hostGroupName;
+
+  @Id
+  @Column(name = "type_name", nullable = false, insertable = true, updatable = false)
+  private String type;
+
+  @Column(name = "config_data", nullable = false, insertable = true, updatable = false)
+  @Basic
+  private String configData;
+
+  @ManyToOne
+  @JoinColumns({
+      @JoinColumn(name = "hostgroup_name", referencedColumnName = "name", nullable = false),
+      @JoinColumn(name = "blueprint_name", referencedColumnName = "blueprint_name", nullable = false)
+  })
+  private HostGroupEntity hostGroup;
+
+
+  /**
+   * Get the configuration type.
+   *
+   * @return configuration type
+   */
+  public String getType() {
+    return type;
+  }
+
+  /**
+   * Set the configuration type.
+   *
+   * @param type  configuration type
+   */
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  /**
+   * Get the host group entity instance.
+   *
+   * @return host group entity
+   */
+  public HostGroupEntity getHostGroupEntity() {
+    return hostGroup;
+  }
+
+  /**
+   * Set the host group entity instance.
+   *
+   * @param entity  host group entity
+   */
+  public void setHostGroupEntity(HostGroupEntity entity) {
+    this.hostGroup = entity;
+  }
+
+  /**
+   * Get the name of the associated blueprint.
+   *
+   * @return blueprint name
+   */
+  public String getBlueprintName() {
+    return blueprintName;
+  }
+
+  /**
+   * Set the name of the associated blueprint.
+   * '
+   * @param blueprintName  blueprint name
+   */
+  public void setBlueprintName(String blueprintName) {
+    this.blueprintName = blueprintName;
+  }
+
+  /**
+   * Get the name of the associated host group.
+   *
+   * @return host group name
+   */
+  public String getHostGroupName() {
+    return hostGroupName;
+  }
+
+  /**
+   * Set the name of the associated host group.
+   * '
+   * @param hostGroupName  host group name
+   */
+  public void setHostGroupName(String hostGroupName) {
+    this.hostGroupName = hostGroupName;
+  }
+
+  /**
+   * Get the config data.
+   *
+   * @return config data in json format
+   */
+  public String getConfigData() {
+    return configData;
+  }
+
+  /**
+   * Set the config data.
+   *
+   * @param configData  all config data in json format
+   */
+  public void setConfigData(String configData) {
+    this.configData = configData;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPK.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPK.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPK.java
new file mode 100644
index 0000000..c342701
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPK.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.orm.entities;
+
+import javax.persistence.Column;
+import javax.persistence.Id;
+
+/**
+ * Composite primary key for HostGroupConfigEntity.
+ */
+public class HostGroupConfigEntityPK {
+
+  @Id
+  @Column(name = "blueprint_name", nullable = false, insertable = true, updatable = false, length = 100)
+  private String blueprintName;
+
+  @Id
+  @Column(name = "hostgroup_name", nullable = false, insertable = false, updatable = false)
+  private String hostGroupName;
+
+  @Id
+  @Column(name = "type_name", nullable = false, insertable = true, updatable = false, length = 100)
+  private String type;
+
+  /**
+   * Get the name of the associated blueprint.
+   *
+   * @return blueprint name
+   */
+  public String getBlueprintName() {
+    return blueprintName;
+  }
+
+  /**
+   * Set the name of the associated blueprint.
+   *
+   * @param blueprintName  blueprint name
+   */
+  public void setBlueprintName(String blueprintName) {
+    this.blueprintName = blueprintName;
+  }
+
+  /**
+   * Get the name of the associated host group.
+   *
+   * @return host group name
+   */
+  public String getHostGroupName() {
+    return hostGroupName;
+  }
+
+  /**
+   * Set the name of the associated host group.
+   *
+   * @param hostGroupName  host group name
+   */
+  public void setHostGroupName(String hostGroupName) {
+    this.hostGroupName = hostGroupName;
+  }
+
+  /**
+   * Get the configuration type.
+   *
+   * @return configuration type
+   */
+  public String getType() {
+    return type;
+  }
+
+  /**
+   * Set the configuration type.
+   *
+   * @param type  configuration type
+   */
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    HostGroupConfigEntityPK that = (HostGroupConfigEntityPK) o;
+
+    return this.blueprintName.equals(that.blueprintName) &&
+           this.hostGroupName.equals(that.hostGroupName) &&
+           this.type.equals(that.type);
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * blueprintName.hashCode() +
+           hostGroupName.hashCode() +
+           type.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupEntity.java
index fbc6ef4..863632a 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostGroupEntity.java
@@ -52,6 +52,9 @@ public class HostGroupEntity {
   @OneToMany(cascade = CascadeType.ALL, mappedBy = "hostGroup")
   private Collection<HostGroupComponentEntity> components;
 
+  @OneToMany(cascade = CascadeType.ALL, mappedBy = "hostGroup")
+  private Collection<HostGroupConfigEntity> configurations;
+
   @ManyToOne
   @JoinColumn(name = "blueprint_name", referencedColumnName = "blueprint_name", nullable = false)
   private BlueprintEntity blueprint;
@@ -130,6 +133,24 @@ public class HostGroupEntity {
   }
 
   /**
+   * Get the collection of associated configuration entities.
+   *
+   * @return collection of configurations
+   */
+  public Collection<HostGroupConfigEntity> getConfigurations() {
+    return configurations;
+  }
+
+  /**
+   * Set the collection of associated configuration entities.
+   *
+   * @param configurations  collection of configurations
+   */
+  public void setConfigurations(Collection<HostGroupConfigEntity> configurations) {
+    this.configurations = configurations;
+  }
+
+  /**
    * Get the cardinality for this host group.
    *
    * @return cardinality

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/upgrade/SchemaUpgradeHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/SchemaUpgradeHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/SchemaUpgradeHelper.java
index a3f253f..f58d8d2 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/SchemaUpgradeHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/SchemaUpgradeHelper.java
@@ -163,6 +163,7 @@ public class SchemaUpgradeHelper {
         Multibinder.newSetBinder(binder(), UpgradeCatalog.class);
       catalogBinder.addBinding().to(UpgradeCatalog150.class);
       catalogBinder.addBinding().to(UpgradeCatalog151.class);
+      catalogBinder.addBinding().to(UpgradeCatalog160.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog151.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog151.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog151.java
index 3566e23..e7907ba 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog151.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog151.java
@@ -112,7 +112,6 @@ public class UpgradeCatalog151 extends AbstractUpgradeCatalog {
     }
     dbAccessor.createTable("viewresource", columns, "view_name", "name");
 
-
     // ========================================================================
     // Add constraints
     dbAccessor.addFKConstraint("viewparameter", "FK_viewparam_view_name", "view_name", "viewmain", "view_name", true);
@@ -122,6 +121,9 @@ public class UpgradeCatalog151 extends AbstractUpgradeCatalog {
         new String[]{"view_name", "view_instance_name"}, "viewinstance", new String[]{"view_name", "name"}, true);
     dbAccessor.addFKConstraint("viewinstancedata", "FK_viewinstdata_view_name",
         new String[]{"view_name", "view_instance_name"}, "viewinstance", new String[]{"view_name", "name"}, true);
+
+    dbAccessor.addFKConstraint("hostgroup_configuration", "FK_hg_config_blueprint_name", "blueprint_name", "hostgroup", "blueprint_name", true);
+    dbAccessor.addFKConstraint("hostgroup_configuration", "FK_hg_config_hostgroup_name", "hostgroup_name", "hostgroup", "name", true);
   }
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog160.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog160.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog160.java
new file mode 100644
index 0000000..f66fe1c
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog160.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.upgrade;
+
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.orm.DBAccessor;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Upgrade catalog for version 1.6.0.
+ */
+public class UpgradeCatalog160 extends AbstractUpgradeCatalog {
+
+  // ----- Constructors ------------------------------------------------------
+
+  @Inject
+  public UpgradeCatalog160(Injector injector) {
+    super(injector);
+  }
+
+
+  // ----- AbstractUpgradeCatalog --------------------------------------------
+
+  @Override
+  protected void executeDDLUpdates() throws AmbariException, SQLException {
+    List<DBAccessor.DBColumnInfo> columns = new ArrayList<DBAccessor.DBColumnInfo>();
+
+    // BP host group configuration
+    columns.clear();
+    columns.add(new DBAccessor.DBColumnInfo("blueprint_name", String.class, 255, null, false));
+    columns.add(new DBAccessor.DBColumnInfo("hostgroup_name", String.class, 255, null, false));
+    columns.add(new DBAccessor.DBColumnInfo("type_name", String.class, 255, null, false));
+    columns.add(new DBAccessor.DBColumnInfo("config_data", byte[].class, null, null, false));
+
+    dbAccessor.createTable("hostgroup_configuration", columns, "blueprint_name",
+        "hostgroup_name", "type_name");
+
+    // ========================================================================
+    // Add constraints
+    dbAccessor.addFKConstraint("hostgroup_configuration", "FK_hg_config_blueprint_name",
+        "blueprint_name", "hostgroup", "blueprint_name", true);
+    dbAccessor.addFKConstraint("hostgroup_configuration", "FK_hg_config_hostgroup_name",
+        "hostgroup_name", "hostgroup", "name", true);
+  }
+
+
+  // ----- UpgradeCatalog ----------------------------------------------------
+
+  @Override
+  protected void executeDMLUpdates() throws AmbariException, SQLException {
+  }
+
+  @Override
+  public String getTargetVersion() {
+    return "1.6.0";
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
index 41e3127..1d04d0b 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
@@ -60,6 +60,7 @@ CREATE TABLE blueprint (blueprint_name VARCHAR(255) NOT NULL, stack_name VARCHAR
 CREATE TABLE hostgroup (blueprint_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, cardinality VARCHAR(255) NOT NULL, PRIMARY KEY(blueprint_name, name));
 CREATE TABLE hostgroup_component (blueprint_name VARCHAR(255) NOT NULL, hostgroup_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, PRIMARY KEY(blueprint_name, hostgroup_name, name));
 CREATE TABLE blueprint_configuration (blueprint_name VARCHAR(255) NOT NULL, type_name VARCHAR(255) NOT NULL, config_data VARCHAR(32000) NOT NULL , PRIMARY KEY(blueprint_name, type_name));
+CREATE TABLE hostgroup_configuration (blueprint_name VARCHAR(255) NOT NULL, hostgroup_name VARCHAR(255) NOT NULL, type_name VARCHAR(255) NOT NULL, config_data TEXT NOT NULL, PRIMARY KEY(blueprint_name, hostgroup_name, type_name));
 CREATE TABLE viewmain (view_name VARCHAR(255) NOT NULL, label VARCHAR(255), version VARCHAR(255), archive VARCHAR(255), PRIMARY KEY(view_name));
 CREATE TABLE viewinstancedata (view_name VARCHAR(255) NOT NULL, view_instance_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, value VARCHAR(2000) NOT NULL, PRIMARY KEY(view_name, view_instance_name, name));
 CREATE TABLE viewinstance (view_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, PRIMARY KEY(view_name, name));
@@ -102,6 +103,7 @@ ALTER TABLE requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_schedul
 ALTER TABLE hostgroup ADD CONSTRAINT FK_hg_blueprint_name FOREIGN KEY (blueprint_name) REFERENCES blueprint(blueprint_name);
 ALTER TABLE hostgroup_component ADD CONSTRAINT FK_hgc_blueprint_name FOREIGN KEY (blueprint_name, hostgroup_name) REFERENCES hostgroup(blueprint_name, name);
 ALTER TABLE blueprint_configuration ADD CONSTRAINT FK_cfg_blueprint_name FOREIGN KEY (blueprint_name) REFERENCES blueprint(blueprint_name);
+ALTER TABLE hostgroup_configuration ADD CONSTRAINT FK_hg_cfg_bp_hg_name FOREIGN KEY (blueprint_name, hostgroup_name) REFERENCES hostgroup (blueprint_name, name);
 ALTER TABLE requestresourcefilter ADD CONSTRAINT FK_reqresfilter_req_id FOREIGN KEY (request_id) REFERENCES request (request_id);
 ALTER TABLE viewparameter ADD CONSTRAINT FK_viewparam_view_name FOREIGN KEY (view_name) REFERENCES viewmain(view_name);
 ALTER TABLE viewresource ADD CONSTRAINT FK_viewres_view_name FOREIGN KEY (view_name) REFERENCES viewmain(view_name);

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
index 5418cac..ae18892 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
@@ -50,6 +50,7 @@ CREATE TABLE blueprint (blueprint_name VARCHAR2(255) NOT NULL, stack_name VARCHA
 CREATE TABLE hostgroup (blueprint_name VARCHAR2(255) NOT NULL, name VARCHAR2(255) NOT NULL, cardinality VARCHAR2(255) NOT NULL, PRIMARY KEY(blueprint_name, name));
 CREATE TABLE hostgroup_component (blueprint_name VARCHAR2(255) NOT NULL, hostgroup_name VARCHAR2(255) NOT NULL, name VARCHAR2(255) NOT NULL, PRIMARY KEY(blueprint_name, hostgroup_name, name));
 CREATE TABLE blueprint_configuration (blueprint_name VARCHAR2(255) NOT NULL, type_name VARCHAR2(255) NOT NULL, config_data CLOB NOT NULL , PRIMARY KEY(blueprint_name, type_name));
+CREATE TABLE hostgroup_configuration (blueprint_name VARCHAR2(255) NOT NULL, hostgroup_name VARCHAR2(255) NOT NULL, type_name VARCHAR2(255) NOT NULL, config_data CLOB NOT NULL, PRIMARY KEY(blueprint_name, hostgroup_name, type_name));
 CREATE TABLE viewmain (view_name VARCHAR(255) NOT NULL, label VARCHAR(255), version VARCHAR(255), archive VARCHAR(255), PRIMARY KEY(view_name));
 CREATE TABLE viewinstancedata (view_name VARCHAR(255) NOT NULL, view_instance_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, value VARCHAR(2000) NOT NULL, PRIMARY KEY(view_name, view_instance_name, name));
 CREATE TABLE viewinstance (view_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, PRIMARY KEY(view_name, name));
@@ -92,6 +93,7 @@ ALTER TABLE requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_schedul
 ALTER TABLE hostgroup ADD CONSTRAINT FK_hg_blueprint_name FOREIGN KEY (blueprint_name) REFERENCES blueprint(blueprint_name);
 ALTER TABLE hostgroup_component ADD CONSTRAINT FK_hgc_blueprint_name FOREIGN KEY (blueprint_name, hostgroup_name) REFERENCES hostgroup(blueprint_name, name);
 ALTER TABLE blueprint_configuration ADD CONSTRAINT FK_cfg_blueprint_name FOREIGN KEY (blueprint_name) REFERENCES blueprint(blueprint_name);
+ALTER TABLE hostgroup_configuration ADD CONSTRAINT FK_hg_cfg_bp_hg_name FOREIGN KEY (blueprint_name, hostgroup_name) REFERENCES hostgroup(blueprint_name, name);
 ALTER TABLE requestresourcefilter ADD CONSTRAINT FK_reqresfilter_req_id FOREIGN KEY (request_id) REFERENCES request (request_id);
 ALTER TABLE viewparameter ADD CONSTRAINT FK_viewparam_view_name FOREIGN KEY (view_name) REFERENCES viewmain(view_name);
 ALTER TABLE viewresource ADD CONSTRAINT FK_viewres_view_name FOREIGN KEY (view_name) REFERENCES viewmain(view_name);

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
index c2b0574..1be43f1 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
@@ -122,10 +122,12 @@ CREATE TABLE ambari.blueprint (blueprint_name VARCHAR(255) NOT NULL, stack_name
 CREATE TABLE ambari.hostgroup (blueprint_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, cardinality VARCHAR(255) NOT NULL, PRIMARY KEY(blueprint_name, name));
 CREATE TABLE ambari.hostgroup_component (blueprint_name VARCHAR(255) NOT NULL, hostgroup_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, PRIMARY KEY(blueprint_name, hostgroup_name, name));
 CREATE TABLE ambari.blueprint_configuration (blueprint_name varchar(255) NOT NULL, type_name varchar(255) NOT NULL, config_data varchar(32000) NOT NULL , PRIMARY KEY(blueprint_name, type_name));
+CREATE TABLE ambari.hostgroup_configuration (blueprint_name VARCHAR(255) NOT NULL, hostgroup_name VARCHAR(255) NOT NULL, type_name VARCHAR(255) NOT NULL, config_data TEXT NOT NULL, PRIMARY KEY(blueprint_name, hostgroup_name, type_name));
 GRANT ALL PRIVILEGES ON TABLE ambari.blueprint TO :username;
 GRANT ALL PRIVILEGES ON TABLE ambari.hostgroup TO :username;
 GRANT ALL PRIVILEGES ON TABLE ambari.hostgroup_component TO :username;
 GRANT ALL PRIVILEGES ON TABLE ambari.blueprint_configuration TO :username;
+GRANT ALL PRIVILEGES ON TABLE ambari.hostgroup_configuration TO :username;
 
 CREATE TABLE ambari.viewmain (view_name VARCHAR(255) NOT NULL, label VARCHAR(255), version VARCHAR(255), archive VARCHAR(255), PRIMARY KEY(view_name));
 CREATE TABLE ambari.viewinstancedata (view_name VARCHAR(255) NOT NULL, view_instance_name VARCHAR(255) NOT NULL, name VARCHAR(255) NOT NULL, value VARCHAR(2000) NOT NULL, PRIMARY KEY(view_name, view_instance_name, name));
@@ -175,6 +177,7 @@ ALTER TABLE ambari.requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_
 ALTER TABLE ambari.hostgroup ADD CONSTRAINT FK_hg_blueprint_name FOREIGN KEY (blueprint_name) REFERENCES ambari.blueprint(blueprint_name);
 ALTER TABLE ambari.hostgroup_component ADD CONSTRAINT FK_hgc_blueprint_name FOREIGN KEY (blueprint_name, hostgroup_name) REFERENCES ambari.hostgroup (blueprint_name, name);
 ALTER TABLE ambari.blueprint_configuration ADD CONSTRAINT FK_cfg_blueprint_name FOREIGN KEY (blueprint_name) REFERENCES ambari.blueprint(blueprint_name);
+ALTER TABLE ambari.hostgroup_configuration ADD CONSTRAINT FK_hg_cfg_bp_hg_name FOREIGN KEY (blueprint_name, hostgroup_name) REFERENCES ambari.hostgroup (blueprint_name, name);
 ALTER TABLE ambari.requestresourcefilter ADD CONSTRAINT FK_reqresfilter_req_id FOREIGN KEY (request_id) REFERENCES ambari.request (request_id);
 ALTER TABLE viewparameter ADD CONSTRAINT FK_viewparam_view_name FOREIGN KEY (view_name) REFERENCES viewmain(view_name);
 ALTER TABLE viewresource ADD CONSTRAINT FK_viewres_view_name FOREIGN KEY (view_name) REFERENCES viewmain(view_name);

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/main/resources/META-INF/persistence.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/META-INF/persistence.xml b/ambari-server/src/main/resources/META-INF/persistence.xml
index 50f1fe0..aa85db1 100644
--- a/ambari-server/src/main/resources/META-INF/persistence.xml
+++ b/ambari-server/src/main/resources/META-INF/persistence.xml
@@ -44,6 +44,7 @@
     <class>org.apache.ambari.server.orm.entities.BlueprintConfigEntity</class>
     <class>org.apache.ambari.server.orm.entities.HostGroupEntity</class>
     <class>org.apache.ambari.server.orm.entities.HostGroupComponentEntity</class>
+    <class>org.apache.ambari.server.orm.entities.HostGroupConfigEntity</class>
     <class>org.apache.ambari.server.orm.entities.RequestResourceFilterEntity</class>
     <class>org.apache.ambari.server.orm.entities.ViewEntity</class>
     <class>org.apache.ambari.server.orm.entities.ViewInstanceDataEntity</class>

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintResourceProviderTest.java
index fbeecbd..9e12c63 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintResourceProviderTest.java
@@ -34,6 +34,7 @@ import org.apache.ambari.server.orm.dao.BlueprintDAO;
 import org.apache.ambari.server.orm.entities.BlueprintConfigEntity;
 import org.apache.ambari.server.orm.entities.BlueprintEntity;
 import org.apache.ambari.server.orm.entities.HostGroupComponentEntity;
+import org.apache.ambari.server.orm.entities.HostGroupConfigEntity;
 import org.apache.ambari.server.orm.entities.HostGroupEntity;
 import org.easymock.Capture;
 
@@ -278,13 +279,26 @@ public class BlueprintResourceProviderTest {
   }
 
   private void setConfigurationProperties(Set<Map<String, Object>> properties ) {
-    Map<String, String> mapConfigsProps = new HashMap<String, String>();
-    mapConfigsProps.put("core-site/fs.trash.interval", "480");
-    mapConfigsProps.put("core-site/ipc.client.idlethreshold", "8500");
+    Map<String, String> clusterProperties = new HashMap<String, String>();
+    clusterProperties.put("core-site/fs.trash.interval", "480");
+    clusterProperties.put("core-site/ipc.client.idlethreshold", "8500");
 
     // single entry in set which was created in getTestProperties
     Map<String, Object> mapProperties = properties.iterator().next();
-    mapProperties.put("configurations", Collections.singleton(mapConfigsProps));
+    mapProperties.put("configurations", Collections.singleton(clusterProperties));
+
+    Map<String, Object> hostGroupProperties = new HashMap<String, Object>();
+    hostGroupProperties.put("core-site/my.custom.hg.property", "anything");
+
+    Collection<Map<String, Object>> hostGroups = (Collection<Map<String, Object>>) mapProperties.get
+        (BlueprintResourceProvider.HOST_GROUP_PROPERTY_ID);
+
+    for (Map<String, Object> hostGroupProps : hostGroups) {
+      if (hostGroupProps.get(BlueprintResourceProvider.HOST_GROUP_NAME_PROPERTY_ID).equals("group2")) {
+        hostGroupProps.put("configurations", Collections.singleton(hostGroupProperties));
+        break;
+      }
+    }
   }
 
   private void validateEntity(BlueprintEntity entity, boolean containsConfig) {
@@ -313,6 +327,19 @@ public class BlueprintResourceProviderTest {
         assertEquals(1, componentEntities.size());
         HostGroupComponentEntity componentEntity = componentEntities.iterator().next();
         assertEquals("component1", componentEntity.getName());
+
+        if (containsConfig) {
+          Collection<HostGroupConfigEntity> configurations = hostGroup.getConfigurations();
+          assertEquals(1, configurations.size());
+          HostGroupConfigEntity hostGroupConfigEntity = configurations.iterator().next();
+          assertEquals(BLUEPRINT_NAME, hostGroupConfigEntity.getBlueprintName());
+          assertSame(hostGroup, hostGroupConfigEntity.getHostGroupEntity());
+          assertEquals("core-site", hostGroupConfigEntity.getType());
+          Map<String, String> properties = gson.<Map<String, String>>fromJson(
+              hostGroupConfigEntity.getConfigData(), Map.class);
+          assertEquals(1, properties.size());
+          assertEquals("anything", properties.get("my.custom.hg.property"));
+        }
       } else {
         fail("Unexpected host group name");
       }

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterResourceProviderTest.java
index 922df27..c9ac437 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterResourceProviderTest.java
@@ -39,6 +39,7 @@ import org.apache.ambari.server.api.services.PersistKeyValueService;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.ClusterRequest;
 import org.apache.ambari.server.controller.ClusterResponse;
+import org.apache.ambari.server.controller.ConfigGroupRequest;
 import org.apache.ambari.server.controller.ConfigurationRequest;
 import org.apache.ambari.server.controller.RequestStatusResponse;
 import org.apache.ambari.server.controller.StackConfigurationRequest;
@@ -58,6 +59,7 @@ import org.apache.ambari.server.orm.dao.BlueprintDAO;
 import org.apache.ambari.server.orm.entities.BlueprintConfigEntity;
 import org.apache.ambari.server.orm.entities.BlueprintEntity;
 import org.apache.ambari.server.orm.entities.HostGroupComponentEntity;
+import org.apache.ambari.server.orm.entities.HostGroupConfigEntity;
 import org.apache.ambari.server.orm.entities.HostGroupEntity;
 import org.easymock.Capture;
 import org.easymock.EasyMock;
@@ -139,6 +141,7 @@ public class ClusterResourceProviderTest {
     String blueprintName = "test-blueprint";
     String stackName = "test";
     String stackVersion = "1.23";
+    String clusterName = "c1";
 
     BlueprintDAO blueprintDAO = createStrictMock(BlueprintDAO.class);
     AmbariManagementController managementController = createStrictMock(AmbariManagementController.class);
@@ -169,10 +172,13 @@ public class ClusterResourceProviderTest {
     HostGroupComponentEntity hostGroupComponent2 = createNiceMock(HostGroupComponentEntity.class);
     HostGroupComponentEntity hostGroupComponent3 = createNiceMock(HostGroupComponentEntity.class);
 
+    HostGroupConfigEntity hostGroupConfig = createNiceMock(HostGroupConfigEntity.class);
+
     ServiceResourceProvider serviceResourceProvider = createStrictMock(ServiceResourceProvider.class);
     ResourceProvider componentResourceProvider = createStrictMock(ResourceProvider.class);
     ResourceProvider hostResourceProvider = createStrictMock(ResourceProvider.class);
     ResourceProvider hostComponentResourceProvider = createStrictMock(ResourceProvider.class);
+    ConfigGroupResourceProvider configGroupResourceProvider = createStrictMock(ConfigGroupResourceProvider.class);
     PersistKeyValueImpl persistKeyValue = createNiceMock(PersistKeyValueImpl.class);
 
     Capture<ClusterRequest> createClusterRequestCapture = new Capture<ClusterRequest>();
@@ -188,6 +194,7 @@ public class ClusterResourceProviderTest {
     Capture<Request> componentRequestCapture2 = new Capture<Request>();
     Capture<Request> hostRequestCapture = new Capture<Request>();
     Capture<Request> hostComponentRequestCapture = new Capture<Request>();
+    Capture<Set<ConfigGroupRequest>> configGroupRequestCapture = new Capture<Set<ConfigGroupRequest>>();
 
     Set<StackServiceResponse> stackServiceResponses = new LinkedHashSet<StackServiceResponse>();
     stackServiceResponses.add(stackServiceResponse1);
@@ -221,7 +228,7 @@ public class ClusterResourceProviderTest {
     Set<Map<String, Object>> propertySet = new LinkedHashSet<Map<String, Object>>();
     Map<String, Object> properties = new LinkedHashMap<String, Object>();
 
-    properties.put(ClusterResourceProvider.CLUSTER_NAME_PROPERTY_ID, "c1");
+    properties.put(ClusterResourceProvider.CLUSTER_NAME_PROPERTY_ID, clusterName);
     properties.put(ClusterResourceProvider.BLUEPRINT_PROPERTY_ID, blueprintName);
     propertySet.add(properties);
 
@@ -236,6 +243,9 @@ public class ClusterResourceProviderTest {
     hostGroupHosts.add(hostGroupHostProperties);
     properties.put("host-groups", hostGroups);
 
+    Map<String, String> mapGroupConfigProperties = new HashMap<String, String>();
+    mapGroupConfigProperties.put("myGroupProp", "awesomeValue");
+
     // blueprint cluster configuration properties
     Map<String, String> blueprintConfigProperties = new HashMap<String, String>();
     blueprintConfigProperties.put("property1", "value2");
@@ -285,12 +295,17 @@ public class ClusterResourceProviderTest {
     expect(blueprintConfig.getType()).andReturn("core-site").anyTimes();
     expect(blueprintConfig.getConfigData()).andReturn(new Gson().toJson(blueprintConfigProperties));
 
-    expect(blueprint.getHostGroups()).andReturn(Collections.singleton(hostGroup));
-    expect(hostGroup.getName()).andReturn("group1");
-    expect(hostGroup.getComponents()).andReturn(hostGroupComponents);
-    expect(hostGroupComponent1.getName()).andReturn("component1");
-    expect(hostGroupComponent2.getName()).andReturn("component2");
-    expect(hostGroupComponent3.getName()).andReturn("component3");
+    expect(blueprint.getHostGroups()).andReturn(Collections.singleton(hostGroup)).anyTimes();
+    expect(hostGroup.getName()).andReturn("group1").anyTimes();
+    expect(hostGroup.getComponents()).andReturn(hostGroupComponents).anyTimes();
+    expect(hostGroupComponent1.getName()).andReturn("component1").anyTimes();
+    expect(hostGroupComponent2.getName()).andReturn("component2").anyTimes();
+    expect(hostGroupComponent3.getName()).andReturn("component3").anyTimes();
+    expect(hostGroup.getConfigurations()).andReturn(
+        Collections.<HostGroupConfigEntity>singleton(hostGroupConfig)).anyTimes();
+
+    expect(hostGroupConfig.getType()).andReturn("core-site").anyTimes();
+    expect(hostGroupConfig.getConfigData()).andReturn(new Gson().toJson(mapGroupConfigProperties)).anyTimes();
 
     managementController.createCluster(capture(createClusterRequestCapture));
     expect(managementController.updateClusters(capture(updateClusterRequestCapture),
@@ -306,22 +321,26 @@ public class ClusterResourceProviderTest {
     expect(hostResourceProvider.createResources(capture(hostRequestCapture))).andReturn(null);
     expect(hostComponentResourceProvider.createResources(capture(hostComponentRequestCapture))).andReturn(null);
 
-    expect(serviceResourceProvider.installAndStart("c1")).andReturn(response);
+    expect(serviceResourceProvider.installAndStart(clusterName)).andReturn(response);
+
+    expect(configGroupResourceProvider.createResources(
+        capture(configGroupRequestCapture))).andReturn(null);
 
     persistKeyValue.put("CLUSTER_CURRENT_STATUS", "{\"clusterState\":\"CLUSTER_STARTED_5\"}");
 
     replay(blueprintDAO, managementController, request, response, blueprint, stackServiceResponse1, stackServiceResponse2,
            stackServiceComponentResponse1, stackServiceComponentResponse2, stackServiceComponentResponse3,
            stackConfigurationResponse1, stackConfigurationResponse2, stackConfigurationResponse3, stackConfigurationResponse4,
-           blueprintConfig, hostGroup, hostGroupComponent1, hostGroupComponent2, hostGroupComponent3, serviceResourceProvider,
-           componentResourceProvider, hostResourceProvider, hostComponentResourceProvider, persistKeyValue);
+           blueprintConfig, hostGroup, hostGroupComponent1, hostGroupComponent2, hostGroupComponent3, hostGroupConfig,
+           serviceResourceProvider, componentResourceProvider, hostResourceProvider, hostComponentResourceProvider,
+           configGroupResourceProvider, persistKeyValue);
 
     // test
     ClusterResourceProvider.injectBlueprintDAO(blueprintDAO);
     PersistKeyValueService.init(persistKeyValue);
     ResourceProvider provider = new TestClusterResourceProvider(
         managementController, serviceResourceProvider, componentResourceProvider,
-        hostResourceProvider, hostComponentResourceProvider);
+        hostResourceProvider, hostComponentResourceProvider, configGroupResourceProvider);
 
     RequestStatus requestStatus = provider.createResources(request);
 
@@ -367,7 +386,7 @@ public class ClusterResourceProviderTest {
         ! configReq2.getServiceName().equals(configReq1.getServiceName()));
 
     ClusterRequest clusterRequest = createClusterRequestCapture.getValue();
-    assertEquals("c1", clusterRequest.getClusterName());
+    assertEquals(clusterName, clusterRequest.getClusterName());
     assertEquals("test-1.23", clusterRequest.getStackVersion());
 
     Set<ClusterRequest> updateClusterRequest1 = updateClusterRequestCapture.getValue();
@@ -379,9 +398,9 @@ public class ClusterResourceProviderTest {
     ClusterRequest ucr1 = updateClusterRequest1.iterator().next();
     ClusterRequest ucr2 = updateClusterRequest2.iterator().next();
     ClusterRequest ucr3 = updateClusterRequest3.iterator().next();
-    assertEquals("c1", ucr1.getClusterName());
-    assertEquals("c1", ucr2.getClusterName());
-    assertEquals("c1", ucr3.getClusterName());
+    assertEquals(clusterName, ucr1.getClusterName());
+    assertEquals(clusterName, ucr2.getClusterName());
+    assertEquals(clusterName, ucr3.getClusterName());
     ConfigurationRequest cr1 = ucr1.getDesiredConfig();
     ConfigurationRequest cr2 = ucr2.getDesiredConfig();
     ConfigurationRequest cr3 = ucr3.getDesiredConfig();
@@ -422,37 +441,50 @@ public class ClusterResourceProviderTest {
     Set<String> componentRequest1Names = new HashSet<String>();
     for (Map<String, Object> componentRequest1Properties : componentRequest.getProperties()) {
       assertEquals(3, componentRequest1Properties.size());
-      assertEquals("c1", componentRequest1Properties.get("ServiceComponentInfo/cluster_name"));
+      assertEquals(clusterName, componentRequest1Properties.get("ServiceComponentInfo/cluster_name"));
       assertEquals("service1", componentRequest1Properties.get("ServiceComponentInfo/service_name"));
       componentRequest1Names.add((String) componentRequest1Properties.get("ServiceComponentInfo/component_name"));
     }
     assertTrue(componentRequest1Names.contains("component1") && componentRequest1Names.contains("component2"));
     assertEquals(1, componentRequest2.getProperties().size());
     Map<String, Object> componentRequest2Properties = componentRequest2.getProperties().iterator().next();
-    assertEquals("c1", componentRequest2Properties.get("ServiceComponentInfo/cluster_name"));
+    assertEquals(clusterName, componentRequest2Properties.get("ServiceComponentInfo/cluster_name"));
     assertEquals("service2", componentRequest2Properties.get("ServiceComponentInfo/service_name"));
     assertEquals("component3", componentRequest2Properties.get("ServiceComponentInfo/component_name"));
     Request hostRequest = hostRequestCapture.getValue();
     assertEquals(1, hostRequest.getProperties().size());
-    assertEquals("c1", hostRequest.getProperties().iterator().next().get("Hosts/cluster_name"));
+    assertEquals(clusterName, hostRequest.getProperties().iterator().next().get("Hosts/cluster_name"));
     assertEquals("host.domain", hostRequest.getProperties().iterator().next().get("Hosts/host_name"));
     Request hostComponentRequest = hostComponentRequestCapture.getValue();
     assertEquals(3, hostComponentRequest.getProperties().size());
     Set<String> componentNames = new HashSet<String>();
     for (Map<String, Object> hostComponentProperties : hostComponentRequest.getProperties()) {
       assertEquals(3, hostComponentProperties.size());
-      assertEquals("c1", hostComponentProperties.get("HostRoles/cluster_name"));
+      assertEquals(clusterName, hostComponentProperties.get("HostRoles/cluster_name"));
       assertEquals("host.domain", hostComponentProperties.get("HostRoles/host_name"));
       componentNames.add((String) hostComponentProperties.get("HostRoles/component_name"));
     }
     assertTrue(componentNames.contains("component1") && componentNames.contains("component2") &&
         componentNames.contains("component3"));
 
+    Set<ConfigGroupRequest> configGroupRequests = configGroupRequestCapture.getValue();
+    assertEquals(1, configGroupRequests.size());
+    ConfigGroupRequest configGroupRequest = configGroupRequests.iterator().next();
+    assertEquals(clusterName, configGroupRequest.getClusterName());
+    assertEquals("group1", configGroupRequest.getGroupName());
+    assertEquals("service1", configGroupRequest.getTag());
+    assertEquals("Host Group Configuration", configGroupRequest.getDescription());
+    Set<String> hosts = configGroupRequest.getHosts();
+    assertEquals(1, hosts.size());
+    assertEquals("host.domain", hosts.iterator().next());
+    assertEquals(1, configGroupRequest.getConfigs().size());
+
     verify(blueprintDAO, managementController, request, response, blueprint, stackServiceResponse1, stackServiceResponse2,
         stackServiceComponentResponse1, stackServiceComponentResponse2, stackServiceComponentResponse3,
         stackConfigurationResponse1, stackConfigurationResponse2, stackConfigurationResponse3, stackConfigurationResponse4,
-        blueprintConfig, hostGroup, hostGroupComponent1, hostGroupComponent2, hostGroupComponent3, serviceResourceProvider,
-        componentResourceProvider, hostResourceProvider, hostComponentResourceProvider, persistKeyValue);
+        blueprintConfig, hostGroup, hostGroupComponent1, hostGroupComponent2, hostGroupComponent3, hostGroupConfig,
+        serviceResourceProvider, componentResourceProvider, hostResourceProvider, hostComponentResourceProvider,
+        configGroupResourceProvider, persistKeyValue);
   }
 
   @Test
@@ -719,12 +751,14 @@ public class ClusterResourceProviderTest {
     private ResourceProvider componentResourceProvider;
     private ResourceProvider hostResourceProvider;
     private ResourceProvider hostComponentResourceProvider;
+    private ResourceProvider configGroupResourceProvider;
 
     TestClusterResourceProvider(AmbariManagementController managementController,
                                 ResourceProvider serviceResourceProvider,
                                 ResourceProvider componentResourceProvider,
                                 ResourceProvider hostResourceProvider,
-                                ResourceProvider hostComponentResourceProvider) {
+                                ResourceProvider hostComponentResourceProvider,
+                                ResourceProvider configGroupResourceProvider) {
 
       super(PropertyHelper.getPropertyIds(Resource.Type.Cluster),
             PropertyHelper.getKeyPropertyIds(Resource.Type.Cluster),
@@ -734,6 +768,7 @@ public class ClusterResourceProviderTest {
       this.componentResourceProvider = componentResourceProvider;
       this.hostResourceProvider = hostResourceProvider;
       this.hostComponentResourceProvider = hostComponentResourceProvider;
+      this.configGroupResourceProvider = configGroupResourceProvider;
     }
 
     @Override
@@ -746,6 +781,8 @@ public class ClusterResourceProviderTest {
         return this.hostResourceProvider;
       } else if (type == Resource.Type.HostComponent) {
         return this.hostComponentResourceProvider;
+      } else if (type == Resource.Type.ConfigGroup) {
+        return this.configGroupResourceProvider;
       } else {
         fail("Unexpected resource provider type requested");
       }

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPKTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPKTest.java b/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPKTest.java
new file mode 100644
index 0000000..277b519
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityPKTest.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.orm.entities;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * HostGroupConfigEntityPK unit tests.
+ */
+public class HostGroupConfigEntityPKTest {
+  @Test
+  public void testSetGetBlueprintName() {
+    HostGroupConfigEntityPK pk = new HostGroupConfigEntityPK();
+    pk.setBlueprintName("foo");
+    assertEquals("foo", pk.getBlueprintName());
+  }
+
+  @Test
+  public void testSetGetHostGroupName() {
+    HostGroupConfigEntityPK pk = new HostGroupConfigEntityPK();
+    pk.setHostGroupName("foo");
+    assertEquals("foo", pk.getHostGroupName());
+  }
+
+  @Test
+  public void testSetGetType() {
+    HostGroupConfigEntityPK pk = new HostGroupConfigEntityPK();
+    pk.setType("testType");
+    assertEquals("testType", pk.getType());
+  }
+
+  @Test
+  public void testHashcode() {
+    HostGroupConfigEntityPK pk1 = new HostGroupConfigEntityPK();
+    HostGroupConfigEntityPK pk2 = new HostGroupConfigEntityPK();
+
+    pk1.setType("foo");
+    pk2.setType("foo");
+    pk1.setBlueprintName("bp");
+    pk2.setBlueprintName("bp");
+    pk1.setHostGroupName("hg");
+    pk2.setHostGroupName("hg");
+
+    assertEquals(pk1.hashCode(), pk2.hashCode());
+  }
+
+  @Test
+  public void testEquals() {
+    HostGroupConfigEntityPK pk1 = new HostGroupConfigEntityPK();
+    HostGroupConfigEntityPK pk2 = new HostGroupConfigEntityPK();
+
+    pk1.setType("foo");
+    pk2.setType("foo");
+    pk1.setBlueprintName("bp");
+    pk2.setBlueprintName("bp");
+    pk1.setHostGroupName("hg");
+    pk2.setHostGroupName("hg");
+
+    assertEquals(pk1, pk2);
+
+    pk1.setType("something_else");
+    assertFalse(pk1.equals(pk2));
+
+    pk2.setType("something_else");
+    assertEquals(pk1, pk2);
+    pk1.setType("other_type");
+    assertFalse(pk1.equals(pk2));
+
+    pk2.setType("other_type");
+    assertEquals(pk1, pk2);
+    pk1.setHostGroupName("hg2");
+    assertFalse(pk1.equals(pk2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityTest.java b/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityTest.java
new file mode 100644
index 0000000..98d8e55
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/orm/entities/HostGroupConfigEntityTest.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.orm.entities;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+
+/**
+ * Unit tests for HostGroupConfigEntity.
+ */
+public class HostGroupConfigEntityTest {
+  @Test
+  public void testSetGetHostGroupName() {
+    HostGroupConfigEntity entity = new HostGroupConfigEntity();
+    entity.setHostGroupName("foo");
+    assertEquals("foo", entity.getHostGroupName());
+  }
+
+  @Test
+  public void testSetGetType() {
+    HostGroupConfigEntity entity = new HostGroupConfigEntity();
+    entity.setType("testType");
+    assertEquals("testType", entity.getType());
+  }
+
+  @Test
+  public void testSetGetHostGroupEntity() {
+    HostGroupEntity group = new HostGroupEntity();
+    HostGroupConfigEntity entity = new HostGroupConfigEntity();
+    entity.setHostGroupEntity(group);
+    assertSame(group, entity.getHostGroupEntity());
+  }
+
+  @Test
+  public void testSetGetBlueprintName() {
+    HostGroupConfigEntity entity = new HostGroupConfigEntity();
+    entity.setBlueprintName("foo");
+    assertEquals("foo", entity.getBlueprintName());
+  }
+
+  @Test
+  public void testSetGetConfigData() {
+    HostGroupConfigEntity entity = new HostGroupConfigEntity();
+    String configData = "{ \"prop_name\" : \"value\" }";
+    entity.setConfigData(configData);
+    assertEquals(configData, entity.getConfigData());
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/dd5f864f/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog160Test.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog160Test.java b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog160Test.java
new file mode 100644
index 0000000..e2234b2
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog160Test.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ambari.server.upgrade;
+
+import com.google.inject.Binder;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import org.apache.ambari.server.configuration.Configuration;
+import org.apache.ambari.server.orm.DBAccessor;
+import org.easymock.Capture;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.sql.SQLException;
+import java.util.List;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertNull;
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.createNiceMock;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+/**
+ * UpgradeCatalog160 unit tests.
+ */
+public class UpgradeCatalog160Test {
+
+  @Test
+  public void testExecuteDDLUpdates() throws Exception {
+
+    final DBAccessor dbAccessor = createNiceMock(DBAccessor.class);
+    Configuration configuration = createNiceMock(Configuration.class);
+    Capture<List<DBAccessor.DBColumnInfo>> hgConfigcolumnCapture = new Capture<List<DBAccessor.DBColumnInfo>>();
+
+    expect(configuration.getDatabaseUrl()).andReturn(Configuration.JDBC_IN_MEMORY_URL).anyTimes();
+
+    setBPHostGroupConfigExpectations(dbAccessor, hgConfigcolumnCapture);
+
+    replay(dbAccessor, configuration);
+    AbstractUpgradeCatalog upgradeCatalog = getUpgradeCatalog(dbAccessor);
+    Class<?> c = AbstractUpgradeCatalog.class;
+    Field f = c.getDeclaredField("configuration");
+    f.setAccessible(true);
+    f.set(upgradeCatalog, configuration);
+
+    upgradeCatalog.executeDDLUpdates();
+    verify(dbAccessor, configuration);
+
+    assertHGConfigColumns(hgConfigcolumnCapture);
+  }
+
+  @Test
+  public void testExecuteDMLUpdates() throws Exception {
+    final DBAccessor dbAccessor     = createNiceMock(DBAccessor.class);
+    UpgradeCatalog160 upgradeCatalog = (UpgradeCatalog160) getUpgradeCatalog(dbAccessor);
+
+    upgradeCatalog.executeDMLUpdates();
+  }
+
+  @Test
+  public void testGetTargetVersion() throws Exception {
+    final DBAccessor dbAccessor     = createNiceMock(DBAccessor.class);
+    UpgradeCatalog   upgradeCatalog = getUpgradeCatalog(dbAccessor);
+
+    Assert.assertEquals("1.6.0", upgradeCatalog.getTargetVersion());
+  }
+
+  private AbstractUpgradeCatalog getUpgradeCatalog(final DBAccessor dbAccessor) {
+    Module module = new Module() {
+      @Override
+      public void configure(Binder binder) {
+        binder.bind(DBAccessor.class).toInstance(dbAccessor);
+      }
+    };
+    Injector injector = Guice.createInjector(module);
+    return injector.getInstance(UpgradeCatalog160.class);
+  }
+
+  private void setBPHostGroupConfigExpectations(DBAccessor dbAccessor, Capture<List<DBAccessor.DBColumnInfo>> columnCapture) throws SQLException {
+    dbAccessor.createTable(eq("hostgroup_configuration"), capture(columnCapture),
+        eq("blueprint_name"), eq("hostgroup_name"), eq("type_name"));
+
+
+    dbAccessor.addFKConstraint("hostgroup_configuration", "FK_hg_config_blueprint_name",
+        "blueprint_name", "hostgroup", "blueprint_name", true);
+    dbAccessor.addFKConstraint("hostgroup_configuration", "FK_hg_config_hostgroup_name",
+        "hostgroup_name", "hostgroup", "name", true);
+  }
+
+  private void assertHGConfigColumns(Capture<List<DBAccessor.DBColumnInfo>> hgConfigcolumnCapture) {
+    List<DBAccessor.DBColumnInfo> columns = hgConfigcolumnCapture.getValue();
+    assertEquals(4, columns.size());
+    DBAccessor.DBColumnInfo column = columns.get(0);
+    assertEquals("blueprint_name", column.getName());
+    assertEquals(255, (int) column.getLength());
+    assertEquals(String.class, column.getType());
+    assertNull(column.getDefaultValue());
+    assertFalse(column.isNullable());
+
+    column = columns.get(1);
+    assertEquals("hostgroup_name", column.getName());
+    assertEquals(255, (int) column.getLength());
+    assertEquals(String.class, column.getType());
+    assertNull(column.getDefaultValue());
+    assertFalse(column.isNullable());
+
+    column = columns.get(2);
+    assertEquals("type_name", column.getName());
+    assertEquals(255, (int) column.getLength());
+    assertEquals(String.class, column.getType());
+    assertNull(column.getDefaultValue());
+    assertFalse(column.isNullable());
+
+    column = columns.get(3);
+    assertEquals("config_data", column.getName());
+    assertEquals(null, column.getLength());
+    assertEquals(byte[].class, column.getType());
+    assertNull(column.getDefaultValue());
+    assertFalse(column.isNullable());
+  }
+}