You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by al...@apache.org on 2015/04/24 01:16:35 UTC

[1/3] ambari git commit: AMBARI-10679. Full Delete of Host : Switch config related tables to use host_id instead of host_name column (alejandro)

Repository: ambari
Updated Branches:
  refs/heads/trunk 909677516 -> 290276c62


http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostTest.java
index fde1945..f6ab0ec 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostTest.java
@@ -18,10 +18,13 @@
 
 package org.apache.ambari.server.state.svccomphost;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import javax.persistence.EntityManager;
 
@@ -33,9 +36,11 @@ import org.apache.ambari.server.controller.ServiceComponentHostResponse;
 import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
 import org.apache.ambari.server.orm.OrmTestHelper;
+import org.apache.ambari.server.orm.dao.ClusterDAO;
 import org.apache.ambari.server.orm.dao.HostComponentDesiredStateDAO;
 import org.apache.ambari.server.orm.dao.HostComponentStateDAO;
 import org.apache.ambari.server.orm.dao.HostDAO;
+import org.apache.ambari.server.orm.entities.ClusterEntity;
 import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
@@ -102,32 +107,34 @@ public class ServiceComponentHostTest {
   @Inject
   private OrmTestHelper helper;
   @Inject
-  HostDAO hostDAO;
+  private ClusterDAO clusterDAO;
+  @Inject
+  private HostDAO hostDAO;
+
+  private String clusterName = "c1";
+  private String hostName1 = "h1";
+  private Map<String, String> hostAttributes = new HashMap<String, String>();
+
 
   @Before
   public void setup() throws Exception {
     injector = Guice.createInjector(new InMemoryDefaultTestModule());
     injector.getInstance(GuiceJpaInitializer.class);
     injector.injectMembers(this);
+
     StackId stackId = new StackId("HDP-0.1");
-    clusters.addCluster("C1", stackId);
-    clusters.addHost("h1");
-    setOsFamily(clusters.getHost("h1"), "redhat", "5.9");
-    clusters.getHost("h1").persist();
+    createCluster(stackId, clusterName);
+    hostAttributes.put("os_family", "redhat");
+    hostAttributes.put("os_release_version", "5.9");
 
-    Cluster c1 = clusters.getCluster("C1");
+    Set<String> hostNames = new HashSet<String>();
+    hostNames.add(hostName1);
+    addHostsToCluster(clusterName, hostAttributes, hostNames);
+
+    Cluster c1 = clusters.getCluster(clusterName);
     helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
     c1.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
         RepositoryVersionState.UPGRADING);
-    clusters.mapHostToCluster("h1","C1");
-  }
-
-  private void setOsFamily(Host host, String osFamily, String osVersion) {
-    Map<String, String> hostAttributes = new HashMap<String, String>(2);
-    hostAttributes.put("os_family", osFamily);
-    hostAttributes.put("os_release_version", osVersion);
-
-    host.setHostAttributes(hostAttributes);
   }
 
   @After
@@ -135,11 +142,35 @@ public class ServiceComponentHostTest {
     injector.getInstance(PersistService.class).stop();
   }
 
-  private ServiceComponentHost createNewServiceComponentHost(
+  private ClusterEntity createCluster(StackId stackId, String clusterName) throws AmbariException {
+    clusters.addCluster(clusterName, stackId);
+    ClusterEntity clusterEntity = clusterDAO.findByName(clusterName);
+    Assert.assertNotNull(clusterEntity);
+    return clusterEntity;
+  }
+
+  private void addHostsToCluster(String clusterName, Map<String, String> hostAttributes, Set<String> hostNames) throws AmbariException {
+    ClusterEntity clusterEntity = clusterDAO.findByName(clusterName);
+
+    List<HostEntity> hostEntities = new ArrayList<HostEntity>();
+    for (String hostName : hostNames) {
+      clusters.addHost(hostName);
+      Host host = clusters.getHost(hostName);
+      host.setHostAttributes(hostAttributes);
+      host.persist();
+    }
+
+    clusterEntity.setHostEntities(hostEntities);
+    clusterDAO.merge(clusterEntity);
+
+    clusters.mapHostsToCluster(hostNames, clusterName);
+  }
+
+  private ServiceComponentHost createNewServiceComponentHost(String clusterName,
       String svc,
       String svcComponent,
       String hostName, boolean isClient) throws AmbariException{
-    Cluster c = clusters.getCluster("C1");
+    Cluster c = clusters.getCluster(clusterName);
     Assert.assertNotNull(c.getConfigGroups());
     return createNewServiceComponentHost(c, svc, svcComponent, hostName);
   }
@@ -198,15 +229,15 @@ public class ServiceComponentHostTest {
 
   @Test
   public void testNewServiceComponentHost() throws AmbariException{
-    createNewServiceComponentHost("HDFS", "NAMENODE", "h1", false);
-    createNewServiceComponentHost("HDFS", "HDFS_CLIENT", "h1", true);
+    createNewServiceComponentHost(clusterName, "HDFS", "NAMENODE", hostName1, false);
+    createNewServiceComponentHost(clusterName, "HDFS", "HDFS_CLIENT", hostName1, true);
   }
 
   private ServiceComponentHostEvent createEvent(ServiceComponentHostImpl impl,
       long timestamp, ServiceComponentHostEventType eventType)
       throws AmbariException {
 
-    Cluster c = clusters.getCluster("C1");
+    Cluster c = clusters.getCluster(clusterName);
     if (c.getConfig("time", String.valueOf(timestamp)) == null) {
       Config config = configFactory.createNew (c, "time",
           new HashMap<String, String>(), new HashMap<String, Map<String,String>>());
@@ -421,7 +452,7 @@ public class ServiceComponentHostTest {
   @Test
   public void testClientStateFlow() throws Exception {
     ServiceComponentHostImpl impl = (ServiceComponentHostImpl)
-        createNewServiceComponentHost("HDFS", "HDFS_CLIENT", "h1", true);
+        createNewServiceComponentHost(clusterName, "HDFS", "HDFS_CLIENT", hostName1, true);
 
     runStateChanges(impl, ServiceComponentHostEventType.HOST_SVCCOMP_INSTALL,
         State.INIT,
@@ -459,7 +490,7 @@ public class ServiceComponentHostTest {
   @Test
   public void testDaemonStateFlow() throws Exception {
     ServiceComponentHostImpl impl = (ServiceComponentHostImpl)
-        createNewServiceComponentHost("HDFS", "DATANODE", "h1", false);
+        createNewServiceComponentHost(clusterName, "HDFS", "DATANODE", hostName1, false);
 
     runStateChanges(impl, ServiceComponentHostEventType.HOST_SVCCOMP_INSTALL,
         State.INIT,
@@ -511,8 +542,7 @@ public class ServiceComponentHostTest {
 
   @Test
   public void testGetAndSetBasicInfo() throws AmbariException {
-    ServiceComponentHost sch =
-        createNewServiceComponentHost("HDFS", "NAMENODE", "h1", false);
+    ServiceComponentHost sch = createNewServiceComponentHost(clusterName, "HDFS", "NAMENODE", hostName1, false);
     sch.setDesiredState(State.INSTALLED);
     sch.setState(State.INSTALLING);
     sch.setStackVersion(new StackId("HDP-1.2.0"));
@@ -528,17 +558,16 @@ public class ServiceComponentHostTest {
 
   @Test
   public void testActualConfigs() throws Exception {
-    ServiceComponentHost sch =
-        createNewServiceComponentHost("HDFS", "NAMENODE", "h1", false);
+    ServiceComponentHost sch = createNewServiceComponentHost(clusterName, "HDFS", "NAMENODE", hostName1, false);
     sch.setDesiredState(State.INSTALLED);
     sch.setState(State.INSTALLING);
     sch.setStackVersion(new StackId("HDP-1.2.0"));
     sch.setDesiredStackVersion(new StackId("HDP-1.2.0"));
 
-    Cluster cluster = clusters.getCluster("C1");
+    Cluster cluster = clusters.getCluster(clusterName);
 
     final ConfigGroup configGroup = configGroupFactory.createNew(cluster,
-      "cg1", "t1", "", new HashMap<String, Config>(), new HashMap<String, Host>());
+      "cg1", "t1", "", new HashMap<String, Config>(), new HashMap<Long, Host>());
 
     configGroup.persist();
     cluster.addConfigGroup(configGroup);
@@ -558,22 +587,19 @@ public class ServiceComponentHostTest {
     Assert.assertTrue(confirm.containsKey("global"));
     Assert.assertTrue(confirm.containsKey("core-site"));
     Assert.assertEquals(1, confirm.get("core-site").getConfigGroupOverrides().size());
-
   }
 
   @Test
   public void testConvertToResponse() throws AmbariException {
-    ServiceComponentHost sch =
-        createNewServiceComponentHost("HDFS", "DATANODE", "h1", false);
+    ServiceComponentHost sch = createNewServiceComponentHost(clusterName, "HDFS", "DATANODE", hostName1, false);
     sch.setDesiredState(State.INSTALLED);
     sch.setState(State.INSTALLING);
     sch.setStackVersion(new StackId("HDP-1.2.0"));
-    ServiceComponentHostResponse r =
-        sch.convertToResponse();
+    ServiceComponentHostResponse r = sch.convertToResponse();
     Assert.assertEquals("HDFS", r.getServiceName());
     Assert.assertEquals("DATANODE", r.getComponentName());
-    Assert.assertEquals("h1", r.getHostname());
-    Assert.assertEquals("C1", r.getClusterName());
+    Assert.assertEquals(hostName1, r.getHostname());
+    Assert.assertEquals(clusterName, r.getClusterName());
     Assert.assertEquals(State.INSTALLED.toString(), r.getDesiredState());
     Assert.assertEquals(State.INSTALLING.toString(), r.getLiveState());
     Assert.assertEquals("HDP-1.2.0", r.getStackVersion());
@@ -588,10 +614,8 @@ public class ServiceComponentHostTest {
   }
 
   @Test
-  public void testStopInVariousStates() throws AmbariException,
-      InvalidStateTransitionException {
-    ServiceComponentHost sch =
-        createNewServiceComponentHost("HDFS", "DATANODE", "h1", false);
+  public void testStopInVariousStates() throws AmbariException, InvalidStateTransitionException {
+    ServiceComponentHost sch = createNewServiceComponentHost(clusterName, "HDFS", "DATANODE", hostName1, false);
     ServiceComponentHostImpl impl =  (ServiceComponentHostImpl) sch;
 
     sch.setDesiredState(State.STARTED);
@@ -638,10 +662,9 @@ public class ServiceComponentHostTest {
   }
 
   @Test
-  public void testDisableInVariousStates() throws AmbariException,
-      InvalidStateTransitionException {
-    ServiceComponentHost sch = createNewServiceComponentHost("HDFS",
-        "DATANODE", "h1", false);
+  public void testDisableInVariousStates() throws AmbariException, InvalidStateTransitionException {
+    ServiceComponentHost sch = createNewServiceComponentHost(clusterName, "HDFS",
+        "DATANODE", hostName1, false);
     ServiceComponentHostImpl impl = (ServiceComponentHostImpl) sch;
 
     // Test valid states in which host component can be disabled
@@ -696,7 +719,7 @@ public class ServiceComponentHostTest {
   @Test
   public void testCanBeRemoved() throws Exception{
     ServiceComponentHostImpl impl = (ServiceComponentHostImpl)
-        createNewServiceComponentHost("HDFS", "HDFS_CLIENT", "h1", true);
+        createNewServiceComponentHost(clusterName, "HDFS", "HDFS_CLIENT", hostName1, true);
 
     for (State state : State.values()) {
       impl.setState(state);
@@ -712,23 +735,25 @@ public class ServiceComponentHostTest {
 
   @Test
   public void testStaleConfigs() throws Exception {
-    String stackVersion="HDP-2.0.6";
-    String clusterName = "c2";
-    String hostName = "h3";
+    String stackVersion = "HDP-2.0.6";
     StackId stackId = new StackId(stackVersion);
+    String clusterName = "c2";
+    createCluster(stackId, clusterName);
 
-    clusters.addCluster(clusterName, stackId);
-    clusters.addHost(hostName);
-    setOsFamily(clusters.getHost(hostName), "redhat", "5.9");
-    clusters.getHost(hostName).persist();
-    Cluster c2 = clusters.getCluster(clusterName);
+    final String hostName = "h3";
+    Set<String> hostNames = new HashSet<String>();
+    hostNames.add(hostName);
+    addHostsToCluster(clusterName, hostAttributes, hostNames);
 
-    helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
-    c2.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
-        RepositoryVersionState.UPGRADING);
-    clusters.mapHostToCluster(hostName, clusterName);
+    final HostEntity hostEntity = hostDAO.findByName(hostName);
+    Assert.assertNotNull(hostEntity.getHostId());
 
     Cluster cluster = clusters.getCluster(clusterName);
+    Assert.assertNotNull(cluster);
+
+    helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
+    cluster.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
+        RepositoryVersionState.UPGRADING);
 
     ServiceComponentHost sch1 = createNewServiceComponentHost(cluster, "HDFS", "NAMENODE", hostName);
     ServiceComponentHost sch2 = createNewServiceComponentHost(cluster, "HDFS", "DATANODE", hostName);
@@ -820,7 +845,7 @@ public class ServiceComponentHostTest {
     Assert.assertFalse(sch2.convertToResponse().isStaleConfig());
 
     // make a host override
-    final Host host = clusters.getHostsForCluster(cluster.getClusterName()).get(hostName);
+    final Host host = clusters.getHostsForCluster(clusterName).get(hostName);
     Assert.assertNotNull(host);
 
     final Config c = configFactory.createNew(cluster, "hdfs-site",
@@ -829,10 +854,10 @@ public class ServiceComponentHostTest {
     c.setTag("version3");
     c.persist();
     cluster.addConfig(c);
-    //host.addDesiredConfig(cluster.getClusterId(), true, "user", c);
+    host.addDesiredConfig(cluster.getClusterId(), true, "user", c);
     ConfigGroup configGroup = configGroupFactory.createNew(cluster, "g1",
       "t1", "", new HashMap<String, Config>() {{ put("hdfs-site", c); }},
-      new HashMap<String, Host>() {{ put("h3", host); }});
+      new HashMap<Long, Host>() {{ put(hostEntity.getHostId(), host); }});
     configGroup.persist();
     cluster.addConfigGroup(configGroup);
 
@@ -898,7 +923,7 @@ public class ServiceComponentHostTest {
     cluster.addConfig(c1);
     configGroup = configGroupFactory.createNew(cluster, "g2",
       "t2", "", new HashMap<String, Config>() {{ put("core-site", c1); }},
-      new HashMap<String, Host>() {{ put("h3", host); }});
+      new HashMap<Long, Host>() {{ put(hostEntity.getHostId(), host); }});
     configGroup.persist();
     cluster.addConfigGroup(configGroup);
 
@@ -939,26 +964,22 @@ public class ServiceComponentHostTest {
 
   @Test
   public void testStaleConfigsAttributes() throws Exception {
-    String stackVersion="HDP-2.0.6";
+    String stackVersion = "HDP-2.0.6";
+    StackId stackId = new StackId(stackVersion);
     String clusterName = "c2";
-    String hostName = "h3";
+    createCluster(stackId, clusterName);
 
-    StackId stackId = new StackId(stackVersion);
+    final String hostName = "h3";
+    Set<String> hostNames = new HashSet<String>();
+    hostNames.add(hostName);
+    addHostsToCluster(clusterName, hostAttributes, hostNames);
 
-    clusters.addCluster(clusterName, stackId);
-    clusters.addHost(hostName);
-    setOsFamily(clusters.getHost(hostName), "redhat", "5.9");
-    clusters.getHost(hostName).persist();
-    Cluster c2 = clusters.getCluster(clusterName);
+    Cluster cluster = clusters.getCluster(clusterName);
 
     helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
-    c2.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
+    cluster.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
         RepositoryVersionState.UPGRADING);
 
-    clusters.mapHostToCluster(hostName, clusterName);
-
-    Cluster cluster = clusters.getCluster(clusterName);
-
     ServiceComponentHost sch1 = createNewServiceComponentHost(cluster, "HDFS", "NAMENODE", hostName);
     ServiceComponentHost sch2 = createNewServiceComponentHost(cluster, "HDFS", "DATANODE", hostName);
     ServiceComponentHost sch3 = createNewServiceComponentHost(cluster, "MAPREDUCE2", "HISTORYSERVER", hostName);
@@ -1075,27 +1096,25 @@ public class ServiceComponentHostTest {
 
   @Test
   public void testMaintenance() throws Exception {
-    String stackVersion="HDP-2.0.6";
+    String stackVersion = "HDP-2.0.6";
+    StackId stackId = new StackId(stackVersion);
     String clusterName = "c2";
-    String hostName = "h3";
+    createCluster(stackId, clusterName);
 
-    StackId stackId = new StackId(stackVersion);
+    final String hostName = "h3";
+    Set<String> hostNames = new HashSet<String>();
+    hostNames.add(hostName);
+    addHostsToCluster(clusterName, hostAttributes, hostNames);
 
-    clusters.addCluster(clusterName, stackId);
-    clusters.addHost(hostName);
-    setOsFamily(clusters.getHost(hostName), "redhat", "5.9");
-    clusters.getHost(hostName).persist();
-    Cluster c2 = clusters.getCluster(clusterName);
+    Cluster cluster = clusters.getCluster(clusterName);
 
     helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
-    c2.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
+    cluster.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
         RepositoryVersionState.UPGRADING);
-    clusters.mapHostToCluster(hostName, clusterName);
+
     HostEntity hostEntity = hostDAO.findByName(hostName);
     Assert.assertNotNull(hostEntity);
 
-    Cluster cluster = clusters.getCluster(clusterName);
-
     ServiceComponentHost sch1 = createNewServiceComponentHost(cluster, "HDFS", "NAMENODE", hostName);
     ServiceComponentHost sch2 = createNewServiceComponentHost(cluster, "HDFS", "DATANODE", hostName);
     ServiceComponentHost sch3 = createNewServiceComponentHost(cluster, "MAPREDUCE2", "HISTORYSERVER", hostName);
@@ -1121,28 +1140,23 @@ public class ServiceComponentHostTest {
 
   @Test
   public void testSecurityState() throws Exception {
-    String stackVersion="HDP-2.0.6";
-    String clusterName = "c2";
-    String hostName = "h3";
+    String stackVersion = "HDP-2.0.6";
     StackId stackId = new StackId(stackVersion);
+    String clusterName = "c2";
+    createCluster(stackId, clusterName);
 
-    clusters.addCluster(clusterName, stackId);
-    clusters.addHost(hostName);
-    setOsFamily(clusters.getHost(hostName), "redhat", "5.9");
-    clusters.getHost(hostName).persist();
-    HostEntity hostEntity = hostDAO.findByName(hostName);
-    Assert.assertNotNull(hostEntity);
+    final String hostName = "h3";
+    Set<String> hostNames = new HashSet<String>();
+    hostNames.add(hostName);
+    addHostsToCluster(clusterName, hostAttributes, hostNames);
 
-    Cluster c2 = clusters.getCluster(clusterName);
+    Cluster cluster = clusters.getCluster(clusterName);
 
-    c2.setDesiredStackVersion(stackId);
     helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
-    c2.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
+    cluster.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
         RepositoryVersionState.UPGRADING);
-    clusters.mapHostToCluster(hostName, clusterName);
-
-    Cluster cluster = clusters.getCluster(clusterName);
 
+    HostEntity hostEntity = hostDAO.findByName(hostName);
     ServiceComponentHost sch1 = createNewServiceComponentHost(cluster, "HDFS", "NAMENODE", hostName);
 
     HostComponentDesiredStateDAO daoHostComponentDesiredState = injector.getInstance(HostComponentDesiredStateDAO.class);

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeTest.java b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeTest.java
index ddfab75..b9b01f8 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeTest.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLSyntaxErrorException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -198,7 +199,17 @@ public class UpgradeTest {
     List<UpgradeCatalog> upgradeCatalogs =
       schemaUpgradeHelper.getUpgradePath(sourceVersion, targetVersion);
 
-    schemaUpgradeHelper.executeUpgrade(upgradeCatalogs);
+    try {
+      schemaUpgradeHelper.executeUpgrade(upgradeCatalogs);
+    } catch (Exception e) {
+      // In UpgradeCatalog210, a lot of the classes had host_name removed, but the catalog makes raw SQL queries from Ambari 2.0.0
+      // which still had that column, in order to populate the host_id. Therfore, ignore this exception type.
+      if (e.getMessage().contains("Column 'T.HOST_NAME' is either not in any table in the FROM list") || e.getMessage().contains("Column 'T.HOSTNAME' is either not in any table in the FROM list")) {
+        System.out.println("Ignoring on purpose, " + e.getMessage());
+      } else {
+        throw e;
+      }
+     }
 
     schemaUpgradeHelper.startPersistenceService();
 


[3/3] ambari git commit: AMBARI-10679. Full Delete of Host : Switch config related tables to use host_id instead of host_name column (alejandro)

Posted by al...@apache.org.
AMBARI-10679. Full Delete of Host : Switch config related tables to use host_id instead of host_name column (alejandro)


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

Branch: refs/heads/trunk
Commit: 290276c62152f331e0859c1d81534cc160471327
Parents: 9096775
Author: Alejandro Fernandez <af...@hortonworks.com>
Authored: Wed Apr 22 19:06:43 2015 -0700
Committer: Alejandro Fernandez <af...@hortonworks.com>
Committed: Thu Apr 23 16:16:30 2015 -0700

----------------------------------------------------------------------
 .../ServiceConfigVersionResponse.java           |  17 +-
 .../internal/ConfigGroupResourceProvider.java   |  35 ++-
 .../internal/UpgradeResourceProvider.java       |  25 ++-
 .../orm/cache/ConfigGroupHostMapping.java       |   4 +-
 .../orm/cache/ConfigGroupHostMappingImpl.java   |  43 ++--
 .../server/orm/cache/HostConfigMapping.java     |   8 +-
 .../server/orm/cache/HostConfigMappingImpl.java |  54 ++---
 .../orm/dao/ConfigGroupHostMappingDAO.java      |  46 ++--
 .../server/orm/dao/HostConfigMappingDAO.java    | 193 +++++++++--------
 .../apache/ambari/server/orm/dao/HostDAO.java   |  11 +
 .../entities/ConfigGroupHostMappingEntity.java  |  24 ++-
 .../ConfigGroupHostMappingEntityPK.java         |  16 +-
 .../orm/entities/HostConfigMappingEntity.java   |  18 +-
 .../orm/entities/HostConfigMappingEntityPK.java |  20 +-
 .../orm/entities/ServiceConfigEntity.java       |  12 +-
 .../org/apache/ambari/server/state/Cluster.java |   6 +-
 .../apache/ambari/server/state/Clusters.java    |  17 ++
 .../server/state/cluster/ClusterImpl.java       |  77 ++++---
 .../server/state/cluster/ClustersImpl.java      |  54 ++++-
 .../server/state/configgroup/ConfigGroup.java   |   6 +-
 .../state/configgroup/ConfigGroupFactory.java   |   2 +-
 .../state/configgroup/ConfigGroupImpl.java      |  38 ++--
 .../ambari/server/state/host/HostImpl.java      |  31 ++-
 .../server/upgrade/UpgradeCatalog210.java       |  51 ++++-
 .../main/resources/Ambari-DDL-MySQL-CREATE.sql  |  27 +--
 .../main/resources/Ambari-DDL-Oracle-CREATE.sql |  25 +--
 .../resources/Ambari-DDL-Postgres-CREATE.sql    |  25 +--
 .../Ambari-DDL-Postgres-EMBEDDED-CREATE.sql     |  25 +--
 .../resources/Ambari-DDL-SQLServer-CREATE.sql   |  21 +-
 .../AmbariManagementControllerTest.java         |  10 +-
 .../ConfigGroupResourceProviderTest.java        |  98 +++++++--
 .../server/orm/dao/ConfigGroupDAOTest.java      |  22 +-
 .../orm/dao/HostConfigMappingDAOTest.java       |  87 ++++----
 .../ambari/server/state/ConfigGroupTest.java    |   9 +-
 .../ambari/server/state/ConfigHelperTest.java   |   6 +-
 .../ambari/server/state/UpgradeHelperTest.java  |   5 +-
 .../server/state/cluster/ClusterTest.java       |  97 ++++++---
 .../svccomphost/ServiceComponentHostTest.java   | 214 ++++++++++---------
 .../ambari/server/upgrade/UpgradeTest.java      |  13 +-
 39 files changed, 867 insertions(+), 625 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/controller/ServiceConfigVersionResponse.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/ServiceConfigVersionResponse.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/ServiceConfigVersionResponse.java
index 9a07a29..c02bb6e 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/ServiceConfigVersionResponse.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/ServiceConfigVersionResponse.java
@@ -21,6 +21,9 @@ package org.apache.ambari.server.controller;
 
 import java.util.List;
 
+import com.google.inject.Inject;
+import org.apache.ambari.server.StaticallyInject;
+import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.orm.entities.ClusterEntity;
 import org.apache.ambari.server.orm.entities.ServiceConfigEntity;
 import org.apache.ambari.server.orm.entities.StackEntity;
@@ -28,6 +31,7 @@ import org.apache.ambari.server.state.StackId;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
+@StaticallyInject
 public class ServiceConfigVersionResponse {
   @JsonProperty("cluster_name")
   @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
@@ -72,15 +76,14 @@ public class ServiceConfigVersionResponse {
   @JsonProperty("hosts")
   private final List<String> hosts;
 
+  @Inject
+  private static HostDAO hostDAO;
+
   /**
    * Constructor.
    *
-   * @param clusterName
-   * @param serviceName
-   * @param version
-   * @param isCurrent
-   * @param isCompatibleWithCurrentStack
-   * @param configurations
+   * @param serviceConfigEntity
+   * @param configGroupName
    */
   public ServiceConfigVersionResponse(ServiceConfigEntity serviceConfigEntity,
       String configGroupName) {
@@ -95,7 +98,7 @@ public class ServiceConfigVersionResponse {
     note = serviceConfigEntity.getNote();
     groupId = (null != serviceConfigEntity.getGroupId() ? serviceConfigEntity.getGroupId(): -1L);
     groupName = configGroupName;
-    hosts = serviceConfigEntity.getHostNames();
+    hosts = hostDAO.getHostNamesByHostIds(serviceConfigEntity.getHostIds());
 
     StackEntity serviceConfigStackEntity = serviceConfigEntity.getStack();
     StackEntity clusterStackEntity = clusterEntity.getClusterStateEntity().getCurrentStack();

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/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 3fcb84b..68bad38 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
@@ -17,12 +17,14 @@
  */
 package org.apache.ambari.server.controller.internal;
 
+import com.google.inject.Inject;
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.ClusterNotFoundException;
 import org.apache.ambari.server.ConfigGroupNotFoundException;
 import org.apache.ambari.server.DuplicateResourceException;
 import org.apache.ambari.server.HostNotFoundException;
 import org.apache.ambari.server.ParentObjectNotFoundException;
+import org.apache.ambari.server.StaticallyInject;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.ConfigGroupRequest;
 import org.apache.ambari.server.controller.ConfigGroupResponse;
@@ -37,6 +39,8 @@ import org.apache.ambari.server.controller.spi.ResourcePredicateEvaluator;
 import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
+import org.apache.ambari.server.orm.dao.HostDAO;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
 import org.apache.ambari.server.state.Config;
@@ -48,13 +52,16 @@ import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+@StaticallyInject
 public class ConfigGroupResourceProvider extends
   AbstractControllerResourceProvider implements ResourcePredicateEvaluator {
 
@@ -86,6 +93,9 @@ public class ConfigGroupResourceProvider extends
   private static Set<String> pkPropertyIds = new HashSet<String>(Arrays
     .asList(new String[] { CONFIGGROUP_ID_PROPERTY_ID }));
 
+  @Inject
+  private static HostDAO hostDAO;
+
   /**
    * Create a  new resource provider for the given management controller.
    *
@@ -343,10 +353,11 @@ public class ConfigGroupResourceProvider extends
             // Has tag
             if (configGroup.getTag().equals(request.getTag())) {
               // Has a match with hosts
-              Set<String> groupHosts = new HashSet<String>(configGroup
-                .getHosts().keySet());
-              groupHosts.retainAll(request.getHosts());
-              if (!groupHosts.isEmpty()) {
+              List<Long> groupHostIds = new ArrayList<Long>(configGroup.getHosts().keySet());
+              Set<String> groupHostNames = new HashSet<String>(hostDAO.getHostNamesByHostIds(groupHostIds));
+
+              groupHostNames.retainAll(request.getHosts());
+              if (!groupHostNames.isEmpty()) {
                 responses.add(configGroup.convertToResponse());
               }
             }
@@ -362,7 +373,7 @@ public class ConfigGroupResourceProvider extends
     return responses;
   }
 
-  private void verifyHostList(Cluster cluster, Map<String, Host> hosts,
+  private void verifyHostList(Cluster cluster, Map<Long, Host> hosts,
                               ConfigGroupRequest request) throws AmbariException {
 
     Map<Long, ConfigGroup> configGroupMap = cluster.getConfigGroups();
@@ -472,14 +483,15 @@ public class ConfigGroupResourceProvider extends
       }
 
       // Find hosts
-      Map<String, Host> hosts = new HashMap<String, Host>();
+      Map<Long, Host> hosts = new HashMap<Long, Host>();
       if (request.getHosts() != null && !request.getHosts().isEmpty()) {
         for (String hostname : request.getHosts()) {
           Host host = clusters.getHost(hostname);
-          if (host == null) {
+          HostEntity hostEntity = hostDAO.findByName(hostname);
+          if (host == null || hostEntity == null) {
             throw new HostNotFoundException(hostname);
           }
-          hosts.put(hostname, host);
+          hosts.put(hostEntity.getHostId(), host);
         }
       }
 
@@ -568,14 +580,15 @@ public class ConfigGroupResourceProvider extends
       }
 
       // Update hosts
-      Map<String, Host> hosts = new HashMap<String, Host>();
+      Map<Long, Host> hosts = new HashMap<Long, Host>();
       if (request.getHosts() != null && !request.getHosts().isEmpty()) {
         for (String hostname : request.getHosts()) {
           Host host = clusters.getHost(hostname);
-          if (host == null) {
+          HostEntity hostEntity = hostDAO.findByName(hostname);
+          if (host == null || hostEntity == null) {
             throw new HostNotFoundException(hostname);
           }
-          hosts.put(hostname, host);
+          hosts.put(hostEntity.getHostId(), host);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
index 730fba5..7de7348 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
@@ -23,6 +23,7 @@ import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.VERSION;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -59,11 +60,13 @@ import org.apache.ambari.server.controller.spi.Resource;
 import org.apache.ambari.server.controller.spi.ResourceAlreadyExistsException;
 import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
+import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.orm.dao.HostRoleCommandDAO;
 import org.apache.ambari.server.orm.dao.HostRoleCommandStatusSummaryDTO;
 import org.apache.ambari.server.orm.dao.RepositoryVersionDAO;
 import org.apache.ambari.server.orm.dao.RequestDAO;
 import org.apache.ambari.server.orm.dao.UpgradeDAO;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
 import org.apache.ambari.server.orm.entities.RequestEntity;
 import org.apache.ambari.server.orm.entities.StackEntity;
@@ -170,6 +173,10 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
   @Inject
   private static HostRoleCommandDAO s_hostRoleCommandDAO = null;
 
+  @Inject
+  private static HostDAO s_hostDAO = null;
+
+
   /**
    * Used to generated the correct tasks and stages during an upgrade.
    */
@@ -901,6 +908,19 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     String itemDetail = entity.getText();
     String stageText = StringUtils.abbreviate(entity.getText(), 255);
 
+    String hostName = null;
+    Collection<Long> hostIds = cluster.getAllHostsDesiredConfigs().keySet();
+    if (!hostIds.isEmpty()) {
+      Long hostId = hostIds.iterator().next();
+      HostEntity hostEntity = s_hostDAO.findById(hostId);
+      if (hostEntity != null) {
+        hostName = hostEntity.getHostName();
+      }
+    }
+    if (StringUtils.isBlank(hostName)) {
+      throw new AmbariException("Could not retrieve an arbitrary host name to use for the server-side command.");
+    }
+
     switch (task.getType()) {
       case MANUAL: {
         ManualTask mt = (ManualTask) task;
@@ -980,15 +1000,12 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     stage.setStageId(stageId);
     entity.setStageId(Long.valueOf(stageId));
 
-    // !!! hack hack hack
-    String host = cluster.getAllHostsDesiredConfigs().keySet().iterator().next();
-
     stage.addServerActionCommand(task.getImplementationClass(),
         getManagementController().getAuthName(),
         Role.AMBARI_SERVER_ACTION,
         RoleCommand.EXECUTE,
         cluster.getClusterName(),
-        host,
+        hostName,
         new ServiceComponentHostServerActionEvent(StageUtils.getHostName(), System.currentTimeMillis()),
         commandParams,
         itemDetail,

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMapping.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMapping.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMapping.java
index 5c26a6c..ae3f076 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMapping.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMapping.java
@@ -24,12 +24,12 @@ import org.apache.ambari.server.state.configgroup.ConfigGroup;
 public interface ConfigGroupHostMapping {
   
   public Long getConfigGroupId();
-  public String getHostname();
+  public Long getHostId();
   public Host getHost();
   public ConfigGroup getConfigGroup();
   
   public void setConfigGroupId(Long configGroupId);
-  public void setHostname(String hostname);
+  public void setHostId(Long setHostId);
   public void setHost(Host host);
   public void setConfigGroup(ConfigGroup configGroup);
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMappingImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMappingImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMappingImpl.java
index 54e1ca0..741b02e 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMappingImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/ConfigGroupHostMappingImpl.java
@@ -24,7 +24,7 @@ import org.apache.ambari.server.state.configgroup.ConfigGroup;
 public class ConfigGroupHostMappingImpl implements ConfigGroupHostMapping {
 
   private Long configGroupId;
-  private String hostname;
+  private Long hostId;
   private Host host;
   private ConfigGroup configGroup;
 
@@ -34,8 +34,8 @@ public class ConfigGroupHostMappingImpl implements ConfigGroupHostMapping {
   }
 
   @Override
-  public String getHostname() {
-    return hostname;
+  public Long getHostId() {
+    return hostId;
   }
 
   @Override
@@ -51,25 +51,21 @@ public class ConfigGroupHostMappingImpl implements ConfigGroupHostMapping {
   @Override
   public void setConfigGroupId(Long configGroupId) {
     this.configGroupId = configGroupId;
-
   }
 
   @Override
-  public void setHostname(String hostname) {
-    this.hostname = hostname;
-
+  public void setHostId(Long hostId) {
+    this.hostId = hostId;
   }
 
   @Override
   public void setHost(Host host) {
     this.host = host;
-
   }
 
   @Override
   public void setConfigGroup(ConfigGroup configGroup) {
     this.configGroup = configGroup;
-
   }
 
   @Override
@@ -82,7 +78,7 @@ public class ConfigGroupHostMappingImpl implements ConfigGroupHostMapping {
         prime * result
             + ((configGroupId == null) ? 0 : configGroupId.hashCode());
     result = prime * result + ((host == null) ? 0 : host.hashCode());
-    result = prime * result + ((hostname == null) ? 0 : hostname.hashCode());
+    result = prime * result + ((hostId == null) ? 0 : hostId.hashCode());
     return result;
   }
 
@@ -95,27 +91,12 @@ public class ConfigGroupHostMappingImpl implements ConfigGroupHostMapping {
     if (getClass() != obj.getClass())
       return false;
     ConfigGroupHostMappingImpl other = (ConfigGroupHostMappingImpl) obj;
-    if (configGroup == null) {
-      if (other.configGroup != null)
-        return false;
-    } else if (!configGroup.equals(other.configGroup))
-      return false;
-    if (configGroupId == null) {
-      if (other.configGroupId != null)
-        return false;
-    } else if (!configGroupId.equals(other.configGroupId))
-      return false;
-    if (host == null) {
-      if (other.host != null)
-        return false;
-    } else if (!host.equals(other.host))
-      return false;
-    if (hostname == null) {
-      if (other.hostname != null)
-        return false;
-    } else if (!hostname.equals(other.hostname))
-      return false;
+
+    if (configGroup != null ? !configGroup.equals(other.configGroup) : other.configGroup != null) return false;
+    if (configGroupId != null ? !configGroupId.equals(other.configGroupId) : other.configGroupId != null) return false;
+    if (host != null ? !host.equals(other.host) : other.host != null) return false;
+    if (hostId != null ? !hostId.equals(other.hostId) : other.hostId != null) return false;
+    
     return true;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMapping.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMapping.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMapping.java
index 269daa9..d5afc15 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMapping.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMapping.java
@@ -21,9 +21,9 @@ public interface HostConfigMapping {
   
   public Long getClusterId();
   public void setClusterId(Long clusterId);
-  
-  public String getHostName();
-  public void setHostName(String hostName);
+
+  public Long getHostId();
+  public void setHostId(Long hostId);
   
   public String getType();
   public void setType(String type);
@@ -42,6 +42,4 @@ public interface HostConfigMapping {
   
   public Integer getSelected();
   public void setSelected(Integer selected);
-  
-
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMappingImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMappingImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMappingImpl.java
index 407aeb6..ab86c9b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMappingImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/cache/HostConfigMappingImpl.java
@@ -22,7 +22,7 @@ package org.apache.ambari.server.orm.cache;
 public class HostConfigMappingImpl implements HostConfigMapping {
   
   private Long clusterId;
-  private String hostName;
+  private Long hostId;
   private String type;
   private Long createTimestamp;
   private String version;
@@ -34,7 +34,7 @@ public class HostConfigMappingImpl implements HostConfigMapping {
   
   public HostConfigMappingImpl(HostConfigMapping entry) {
     setClusterId(entry.getClusterId());
-    setHostName(entry.getHostName());
+    setHostId(entry.getHostId());
     setType(entry.getType());
     setCreateTimestamp(entry.getCreateTimestamp());
     setVersion(entry.getVersion());
@@ -57,17 +57,17 @@ public class HostConfigMappingImpl implements HostConfigMapping {
       throw new RuntimeException("ClusterId couldn't be null");
     this.clusterId = clusterId;
   }
-  
+
   @Override
-  public String getHostName() {
-    return hostName;
+  public Long getHostId() {
+    return hostId;
   }
-  
+
   @Override
-  public void setHostName(String hostName) {
-    if (hostName == null)
-      throw new RuntimeException("HostName couldn't be null");
-    this.hostName = hostName;
+  public void setHostId(Long hostId) {
+    if (hostId == null)
+      throw new RuntimeException("HostId couldn't be null");
+    this.hostId = hostId;
   }
   
   @Override
@@ -143,10 +143,8 @@ public class HostConfigMappingImpl implements HostConfigMapping {
     final int prime = 31;
     int result = 1;
     result = prime * result + ((clusterId == null) ? 0 : clusterId.hashCode());
-    result =
-        prime * result
-            + ((createTimestamp == null) ? 0 : createTimestamp.hashCode());
-    result = prime * result + ((hostName == null) ? 0 : hostName.hashCode());
+    result = prime * result + ((createTimestamp == null) ? 0 : createTimestamp.hashCode());
+    result = prime * result + ((hostId == null) ? 0 : hostId.hashCode());
     result = prime * result + ((type == null) ? 0 : type.hashCode());
     return result;
   }
@@ -159,29 +157,13 @@ public class HostConfigMappingImpl implements HostConfigMapping {
       return false;
     if (getClass() != obj.getClass())
       return false;
+
     HostConfigMappingImpl other = (HostConfigMappingImpl) obj;
-    if (clusterId == null) {
-      if (other.clusterId != null)
-        return false;
-    } else if (!clusterId.equals(other.clusterId))
-      return false;
-    if (createTimestamp == null) {
-      if (other.createTimestamp != null)
-        return false;
-    } else if (!createTimestamp.equals(other.createTimestamp))
-      return false;
-    if (hostName == null) {
-      if (other.hostName != null)
-        return false;
-    } else if (!hostName.equals(other.hostName))
-      return false;
-    if (type == null) {
-      if (other.type != null)
-        return false;
-    } else if (!type.equals(other.type))
-      return false;
+    if (clusterId != null ? !clusterId.equals(other.clusterId) : other.clusterId != null) return false;
+    if (createTimestamp != null ? !createTimestamp.equals(other.createTimestamp) : other.createTimestamp != null) return false;
+    if (hostId != null ? !hostId.equals(other.hostId) : other.hostId != null) return false;
+    if (type != null ? !type.equals(other.type) : other.type != null) return false;
+
     return true;
   }
-  
-
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ConfigGroupHostMappingDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ConfigGroupHostMappingDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ConfigGroupHostMappingDAO.java
index 592679e..4628a4c 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ConfigGroupHostMappingDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ConfigGroupHostMappingDAO.java
@@ -64,7 +64,7 @@ public class ConfigGroupHostMappingDAO {
   
   private final ReadWriteLock gl = new ReentrantReadWriteLock();
   
-  private Map<String, Set<ConfigGroupHostMapping>> configGroupHostMappingByHost;
+  private Map<Long, Set<ConfigGroupHostMapping>> configGroupHostMappingByHost;
   
   private volatile boolean cacheLoaded;
 
@@ -75,7 +75,7 @@ public class ConfigGroupHostMappingDAO {
       gl.writeLock().lock();
       try {
         if (configGroupHostMappingByHost == null) {
-          configGroupHostMappingByHost = new WeakHashMap<String, Set<ConfigGroupHostMapping>>();
+          configGroupHostMappingByHost = new WeakHashMap<Long, Set<ConfigGroupHostMapping>>();
           
           TypedQuery<ConfigGroupHostMappingEntity> query = entityManagerProvider.get().createQuery(
               "SELECT entity FROM ConfigGroupHostMappingEntity entity",
@@ -85,11 +85,11 @@ public class ConfigGroupHostMappingDAO {
           
           for (ConfigGroupHostMappingEntity configGroupHostMappingEntity : configGroupHostMappingEntities) {
 
-            Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost.get((configGroupHostMappingEntity.getHostname()));
+            Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost.get((configGroupHostMappingEntity.getHostId()));
               
             if (setByHost == null) {
               setByHost = new HashSet<ConfigGroupHostMapping>();
-              configGroupHostMappingByHost.put(configGroupHostMappingEntity.getHostname(), setByHost);
+              configGroupHostMappingByHost.put(configGroupHostMappingEntity.getHostId(), setByHost);
             }
        
             ConfigGroupHostMapping configGroupHostMapping = buildConfigGroupHostMapping(configGroupHostMappingEntity);
@@ -121,14 +121,14 @@ public class ConfigGroupHostMappingDAO {
   }
 
   @RequiresSession
-  public Set<ConfigGroupHostMapping> findByHost(String hostname) {
+  public Set<ConfigGroupHostMapping> findByHostId(Long hostId) {
     
     populateCache();
     
-    if (!configGroupHostMappingByHost.containsKey(hostname))
+    if (!configGroupHostMappingByHost.containsKey(hostId))
       return null;
     
-    Set<ConfigGroupHostMapping> set = new HashSet<ConfigGroupHostMapping>(configGroupHostMappingByHost.get(hostname));
+    Set<ConfigGroupHostMapping> set = new HashSet<ConfigGroupHostMapping>(configGroupHostMappingByHost.get(hostId));
     
     return set;
     
@@ -169,33 +169,29 @@ public class ConfigGroupHostMappingDAO {
   @Transactional
   public void create(ConfigGroupHostMappingEntity
                          configGroupHostMappingEntity) {
-    
     populateCache();
 
     entityManagerProvider.get().persist(configGroupHostMappingEntity);
     
     //create in cache
-    Set<ConfigGroupHostMapping> set = configGroupHostMappingByHost.get(configGroupHostMappingEntity.getHostname());
+    Set<ConfigGroupHostMapping> set = configGroupHostMappingByHost.get(configGroupHostMappingEntity.getHostId());
     if (set == null){
       set = new HashSet<ConfigGroupHostMapping>();
-      configGroupHostMappingByHost.put(configGroupHostMappingEntity.getHostname(), set);
+      configGroupHostMappingByHost.put(configGroupHostMappingEntity.getHostId(), set);
     }
     
     set.add(buildConfigGroupHostMapping(configGroupHostMappingEntity));
-    
   }
 
-
-
   @Transactional
   public ConfigGroupHostMappingEntity merge(ConfigGroupHostMappingEntity configGroupHostMappingEntity) {
     
     populateCache();
     
-    Set<ConfigGroupHostMapping> set = configGroupHostMappingByHost.get(configGroupHostMappingEntity.getHostname());
+    Set<ConfigGroupHostMapping> set = configGroupHostMappingByHost.get(configGroupHostMappingEntity.getHostId());
     if (set == null){
       set = new HashSet<ConfigGroupHostMapping>();
-      configGroupHostMappingByHost.put(configGroupHostMappingEntity.getHostname(), set);
+      configGroupHostMappingByHost.put(configGroupHostMappingEntity.getHostId(), set);
     }
     
     //Update object in set
@@ -223,7 +219,7 @@ public class ConfigGroupHostMappingDAO {
     
     entityManagerProvider.get().remove(merge(configGroupHostMappingEntity));
     
-    Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost.get(configGroupHostMappingEntity.getHostname());
+    Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost.get(configGroupHostMappingEntity.getHostId());
     
     if (setByHost != null) {
       CollectionUtils.filter(setByHost, new Predicate() {
@@ -244,8 +240,7 @@ public class ConfigGroupHostMappingDAO {
     
     entityManagerProvider.get().remove(findByPK(configGroupHostMappingEntityPK));
     
-    Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost
-      .get(configGroupHostMappingEntityPK.getHostname());
+    Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost.get(configGroupHostMappingEntityPK.getHostId());
     
     if (setByHost != null) {
       CollectionUtils.filter(setByHost, new Predicate() {
@@ -287,19 +282,17 @@ public class ConfigGroupHostMappingDAO {
   }
 
   @Transactional
-  public void removeAllByHost(String hostname) {
+  public void removeAllByHost(Long hostId) {
     TypedQuery<String> query = entityManagerProvider.get().createQuery
       ("DELETE FROM ConfigGroupHostMappingEntity confighosts WHERE " +
-        "confighosts.hostname = ?1", String.class);
+        "confighosts.hostId = ?1", String.class);
 
-    daoUtils.executeUpdate(query, hostname);
+    daoUtils.executeUpdate(query, hostId);
     
     
-    Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost.get(hostname);
+    Set<ConfigGroupHostMapping> setByHost = configGroupHostMappingByHost.get(hostId);
     
     setByHost.clear();
-    
-    
   }
   
   private ConfigGroupHostMapping buildConfigGroupHostMapping(
@@ -309,13 +302,12 @@ public class ConfigGroupHostMappingDAO {
     configGroupHostMapping.setConfigGroup(buildConfigGroup(configGroupHostMappingEntity.getConfigGroupEntity()));
     configGroupHostMapping.setConfigGroupId(configGroupHostMappingEntity.getConfigGroupId());
     configGroupHostMapping.setHost(buildHost(configGroupHostMappingEntity.getHostEntity()));
-    configGroupHostMapping.setHostname(configGroupHostMappingEntity.getHostname());
+    configGroupHostMapping.setHostId(configGroupHostMappingEntity.getHostId());
     
     return configGroupHostMapping;
   }
 
   private ConfigGroup buildConfigGroup(ConfigGroupEntity configGroupEntity) {
-    
     Cluster cluster = clusterFactory.create(configGroupEntity.getClusterEntity());
     ConfigGroup configGroup = configGroupFactory.createExisting(cluster, configGroupEntity);
     
@@ -323,9 +315,7 @@ public class ConfigGroupHostMappingDAO {
   }
 
   private Host buildHost(HostEntity hostEntity) {
-    
     Host host = hostFactory.create(hostEntity, false);
-    
     return host;
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAO.java
index 9bc1235..34d0e3c 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAO.java
@@ -17,7 +17,15 @@
  */
 package org.apache.ambari.server.orm.dao;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.WeakHashMap;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -30,6 +38,7 @@ import org.apache.ambari.server.orm.RequiresSession;
 import org.apache.ambari.server.orm.cache.HostConfigMapping;
 import org.apache.ambari.server.orm.cache.HostConfigMappingImpl;
 import org.apache.ambari.server.orm.entities.HostConfigMappingEntity;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.Predicate;
 
@@ -43,13 +52,16 @@ import com.google.inject.persist.Transactional;
 @Singleton
 public class HostConfigMappingDAO {
   @Inject
-  Provider<EntityManager> entityManagerProvider;
+  private Provider<EntityManager> entityManagerProvider;
+
   @Inject
-  DaoUtils daoUtils;
-  
+  private DaoUtils daoUtils;
+
+  @Inject
+  private HostDAO hostDAO;
 
   private final ReadWriteLock gl = new ReentrantReadWriteLock();
-  private Map<String, Set<HostConfigMapping>> hostConfigMappingByHost;
+  private Map<Long, Set<HostConfigMapping>> hostConfigMappingByHost;
   
   private volatile boolean cacheLoaded;
   
@@ -61,7 +73,7 @@ public class HostConfigMappingDAO {
         gl.writeLock().lock();
         try {
           if (hostConfigMappingByHost == null) {
-            hostConfigMappingByHost = new WeakHashMap<String, Set<HostConfigMapping>>();
+            hostConfigMappingByHost = new WeakHashMap<Long, Set<HostConfigMapping>>();
             
             TypedQuery<HostConfigMappingEntity> query = entityManagerProvider.get().createQuery(
                 "SELECT entity FROM HostConfigMappingEntity entity",
@@ -70,12 +82,18 @@ public class HostConfigMappingDAO {
             List<HostConfigMappingEntity> hostConfigMappingEntities = daoUtils.selectList(query);
             
             for (HostConfigMappingEntity hostConfigMappingEntity : hostConfigMappingEntities) {
+              Long hostId = hostConfigMappingEntity.getHostId();
 
-              Set<HostConfigMapping> setByHost = hostConfigMappingByHost.get((hostConfigMappingEntity.getHostName()));
-              
-              if (setByHost == null) {
+              if (hostId == null) {
+                continue;
+              }
+
+              Set<HostConfigMapping> setByHost;
+              if (hostConfigMappingByHost.containsKey(hostId)) {
+                setByHost = hostConfigMappingByHost.get(hostId);
+              } else {
                 setByHost = new HashSet<HostConfigMapping>();
-                hostConfigMappingByHost.put(hostConfigMappingEntity.getHostName(), setByHost);
+                hostConfigMappingByHost.put(hostId, setByHost);
               }
        
               HostConfigMapping hostConfigMapping = buildHostConfigMapping(hostConfigMappingEntity);
@@ -90,65 +108,70 @@ public class HostConfigMappingDAO {
       }
       
       cacheLoaded = true;
-
     }
-    
   }
   
 
   @Transactional
   public void create(HostConfigMapping hostConfigMapping) {
-    
     populateCache();
     
     //create in db
     entityManagerProvider.get().persist(buildHostConfigMappingEntity(hostConfigMapping));
     
     //create in cache
-    Set<HostConfigMapping> set = hostConfigMappingByHost.get(hostConfigMapping.getHostName());
-    if (set == null){
-      set = new HashSet<HostConfigMapping>();
-      hostConfigMappingByHost.put(hostConfigMapping.getHostName(), set);
+    Long hostId = hostConfigMapping.getHostId();
+
+    if (hostId != null) {
+      Set<HostConfigMapping> set;
+      if (hostConfigMappingByHost.containsKey(hostId)) {
+        set = hostConfigMappingByHost.get(hostId);
+      } else {
+        set = new HashSet<HostConfigMapping>();
+        hostConfigMappingByHost.put(hostId, set);
+      }
+
+      set.add(hostConfigMapping);
     }
-    
-    set.add(hostConfigMapping);
   }
 
   @Transactional
   public HostConfigMapping merge(HostConfigMapping hostConfigMapping) {
-    
     populateCache();
-    
-    Set<HostConfigMapping> set = hostConfigMappingByHost.get(hostConfigMapping.getHostName());
-    if (set == null){
-      set = new HashSet<HostConfigMapping>();
-      hostConfigMappingByHost.put(hostConfigMapping.getHostName(), set);
+
+    Long hostId = hostConfigMapping.getHostId();
+    if (hostId != null) {
+      Set<HostConfigMapping> set;
+      if (hostConfigMappingByHost.containsKey(hostId)) {
+        set = hostConfigMappingByHost.get(hostId);
+      } else {
+        set = new HashSet<HostConfigMapping>();
+        hostConfigMappingByHost.put(hostId, set);
+      }
+
+      //Update object in set
+      set.remove(hostConfigMapping);
+      set.add(hostConfigMapping);
+
+      entityManagerProvider.get().merge(buildHostConfigMappingEntity(hostConfigMapping));
     }
-    
-    //Update object in set
-    set.remove(hostConfigMapping);
-    set.add(hostConfigMapping);
-    
-    entityManagerProvider.get().merge(buildHostConfigMappingEntity(hostConfigMapping));
-    
+
     return hostConfigMapping;
   }
 
   @RequiresSession
-  public Set<HostConfigMapping> findByType(final long clusterId, String hostName, final String type) {
-    
+  public Set<HostConfigMapping> findByType(final long clusterId, Long hostId, final String type) {
     populateCache();
     
-    if (!hostConfigMappingByHost.containsKey(hostName))
+    if (!hostConfigMappingByHost.containsKey(hostId))
       return Collections.emptySet();
       
-    Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostName));
+    Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostId));
      
     CollectionUtils.filter(set, new Predicate() {
         
       @Override
       public boolean evaluate(Object arg0) {
-                  
         return ((HostConfigMapping) arg0).getClusterId().equals(clusterId) 
             && ((HostConfigMapping) arg0).getType().equals(type);
       }
@@ -159,41 +182,35 @@ public class HostConfigMappingDAO {
 
   @RequiresSession
   public HostConfigMapping findSelectedByType(final long clusterId,
-      String hostName, final String type) {
-    
+      Long hostId, final String type) {
     populateCache();
     
-    if (!hostConfigMappingByHost.containsKey(hostName))
+    if (!hostConfigMappingByHost.containsKey(hostId))
       return null;
     
-    Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostName));
+    Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostId));
     
     HostConfigMapping result = (HostConfigMapping) CollectionUtils.find(set, new Predicate() {
       
       @Override
       public boolean evaluate(Object arg0) {
-        
         return ((HostConfigMapping) arg0).getClusterId().equals(clusterId) 
             && ((HostConfigMapping) arg0).getType().equals(type)
             && ((HostConfigMapping) arg0).getSelected() > 0;
       }
     });
     
-    
     return result;
-    
   }
 
   @RequiresSession
-  public Set<HostConfigMapping> findSelected(final long clusterId, String hostName) {
-    
+  public Set<HostConfigMapping> findSelected(final long clusterId, Long hostId) {
     populateCache();
     
-    if (!hostConfigMappingByHost.containsKey(hostName))
+    if (!hostConfigMappingByHost.containsKey(hostId))
       return Collections.emptySet();
-      
     
-    Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostName));
+    Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostId));
     
     CollectionUtils.filter(set, new Predicate() {
       
@@ -208,31 +225,26 @@ public class HostConfigMappingDAO {
   }
 
   @RequiresSession
-  public Set<HostConfigMapping> findSelectedByHosts(long clusterId, Collection<String> hostNames) {
-    
+  public Set<HostConfigMapping> findSelectedByHosts(Collection<Long> hostIds) {
     populateCache();
 
-    if (hostNames == null || hostNames.isEmpty()) {
+    if (hostIds == null || hostIds.isEmpty()) {
       return Collections.emptySet();
     }
     
-    
     HashSet<HostConfigMapping> result = new HashSet<HostConfigMapping>();
-    
-
 
-    for (final String hostName : hostNames) {
-      
-      if (!hostConfigMappingByHost.containsKey(hostName))
+    for (final Long hostId : hostIds) {
+      if (!hostConfigMappingByHost.containsKey(hostId))
         continue;
       
-      Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostName));
+      Set<HostConfigMapping> set = new HashSet<HostConfigMapping>(hostConfigMappingByHost.get(hostId));
       
       CollectionUtils.filter(set, new Predicate() {
         
         @Override
         public boolean evaluate(Object arg0) {
-          return ((HostConfigMapping) arg0).getHostName().equals(hostName) && 
+          return ((HostConfigMapping) arg0).getHostId().equals(hostId) &&
               ((HostConfigMapping) arg0).getSelected() > 0;
         }
       });
@@ -247,9 +259,7 @@ public class HostConfigMappingDAO {
   @RequiresSession
   public Map<String, List<HostConfigMapping>> findSelectedHostsByTypes(final long clusterId,
                                                                              Collection<String> types) {
-    
     populateCache();
-
     
     Map<String, List<HostConfigMapping>> mappingsByType = new HashMap<String, List<HostConfigMapping>>();
     
@@ -268,7 +278,6 @@ public class HostConfigMappingDAO {
           
           if (types.contains(entry.getType()) && entry.getClusterId().equals(clusterId))
             mappings.add(new HostConfigMappingImpl(entry));
-          
         }
       }
 
@@ -276,8 +285,6 @@ public class HostConfigMappingDAO {
         mappingsByType.get(mapping.getType()).add(mapping);
       }
     }
-    
-    
 
     return mappingsByType;
   }
@@ -293,42 +300,40 @@ public class HostConfigMappingDAO {
    */
   @Transactional
   public void removeHost(final long clusterId, String hostName) {
-    
     populateCache();
-    
-    Set<HostConfigMapping> set = hostConfigMappingByHost.get(hostName);
-    
-    //Remove from cache items with clusterId
-    CollectionUtils.filter(set, new Predicate() {
-      
-      @Override
-      public boolean evaluate(Object arg0) {
-        return !((HostConfigMapping) arg0).getClusterId().equals(clusterId);
+
+    if (hostConfigMappingByHost.containsKey(hostName)) {
+      Set<HostConfigMapping> set = hostConfigMappingByHost.get(hostName);
+
+      //Remove from cache items with clusterId
+      CollectionUtils.filter(set, new Predicate() {
+        @Override
+        public boolean evaluate(Object arg0) {
+          return !((HostConfigMapping) arg0).getClusterId().equals(clusterId);
+        }
+      });
+
+      //delete from db
+      TypedQuery<HostConfigMappingEntity> query = entityManagerProvider.get().createQuery(
+          "SELECT entity FROM HostConfigMappingEntity entity " +
+              "WHERE entity.clusterId = ?1 AND entity.hostEntity.hostName = ?2",
+          HostConfigMappingEntity.class);
+
+      List<HostConfigMappingEntity> list = daoUtils.selectList(query, Long.valueOf(clusterId), hostName);
+
+      for (HostConfigMappingEntity entity : list) {
+        entityManagerProvider.get().remove(entity);
       }
-    });
-    
-    //delete from db
-    TypedQuery<HostConfigMappingEntity> query = entityManagerProvider.get().createQuery(
-      "SELECT entity FROM HostConfigMappingEntity entity " +
-      "WHERE entity.clusterId = ?1 AND entity.hostName = ?2",
-      HostConfigMappingEntity.class);
-             
-    List<HostConfigMappingEntity> list = daoUtils.selectList(query, Long.valueOf(clusterId), hostName);
-        
-    for (HostConfigMappingEntity entity : list) {
-      entityManagerProvider.get().remove(entity);
     }
-    
-    
   }
 
   public HostConfigMappingEntity buildHostConfigMappingEntity(HostConfigMapping hostConfigMapping) {
-    
+    HostEntity hostEntity = hostDAO.findById(hostConfigMapping.getHostId());
     HostConfigMappingEntity hostConfigMappingEntity = new HostConfigMappingEntity();
-    
+
     hostConfigMappingEntity.setClusterId(hostConfigMapping.getClusterId());
     hostConfigMappingEntity.setCreateTimestamp(hostConfigMapping.getCreateTimestamp());
-    hostConfigMappingEntity.setHostName(hostConfigMapping.getHostName());
+    hostConfigMappingEntity.setHostId(hostEntity.getHostId());
     hostConfigMappingEntity.setSelected(hostConfigMapping.getSelected());
     hostConfigMappingEntity.setServiceName(hostConfigMapping.getServiceName());
     hostConfigMappingEntity.setType(hostConfigMapping.getType());
@@ -340,12 +345,11 @@ public class HostConfigMappingDAO {
   
   public HostConfigMapping buildHostConfigMapping(
       HostConfigMappingEntity hostConfigMappingEntity) {
-    
     HostConfigMapping hostConfigMapping = new HostConfigMappingImpl();
     
     hostConfigMapping.setClusterId(hostConfigMappingEntity.getClusterId());
     hostConfigMapping.setCreateTimestamp(hostConfigMappingEntity.getCreateTimestamp());
-    hostConfigMapping.setHostName(hostConfigMappingEntity.getHostName());
+    hostConfigMapping.setHostId(hostConfigMappingEntity.getHostId());
     hostConfigMapping.setServiceName(hostConfigMappingEntity.getServiceName());
     hostConfigMapping.setType(hostConfigMappingEntity.getType());
     hostConfigMapping.setUser(hostConfigMappingEntity.getUser());
@@ -354,5 +358,4 @@ public class HostConfigMappingDAO {
     
     return hostConfigMapping;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostDAO.java
index 6442bf5..ebd29e3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostDAO.java
@@ -29,6 +29,7 @@ import org.apache.ambari.server.orm.entities.StageEntity;
 import javax.persistence.EntityManager;
 import javax.persistence.NoResultException;
 import javax.persistence.TypedQuery;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
@@ -117,4 +118,14 @@ public class HostDAO {
     remove(findByName(hostName));
   }
 
+  public List<String> getHostNamesByHostIds(List<Long> hostIds) {
+    List<String> hostNames = new ArrayList<String>();
+    if (hostIds != null) {
+      for (Long hostId : hostIds) {
+        HostEntity hostEntity = findById(hostId);
+        hostNames.add(hostEntity.getHostName());
+      }
+    }
+    return hostNames;
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntity.java
index 261bbe8..a9a2d40 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntity.java
@@ -34,7 +34,7 @@ import javax.persistence.Table;
 @NamedQueries({
   @NamedQuery(name = "groupsByHost", query =
   "SELECT confighosts FROM ConfigGroupHostMappingEntity confighosts " +
-    "WHERE confighosts.hostname=:hostname"),
+    "WHERE confighosts.hostEntity.hostName=:hostname"),
   @NamedQuery(name = "hostsByGroup", query =
   "SELECT confighosts FROM ConfigGroupHostMappingEntity confighosts " +
     "WHERE confighosts.configGroupId=:groupId")
@@ -46,12 +46,12 @@ public class ConfigGroupHostMappingEntity {
   private Long configGroupId;
 
   @Id
-  @Column(name = "host_name", nullable = false, insertable = true, updatable = true)
-  private String hostname;
+  @Column(name = "host_id", nullable = false, insertable = true, updatable = true)
+  private Long hostId;
 
   @ManyToOne
   @JoinColumns({
-    @JoinColumn(name = "host_name", referencedColumnName = "host_name", nullable = false, insertable = false, updatable = false) })
+    @JoinColumn(name = "host_id", referencedColumnName = "host_id", nullable = false, insertable = false, updatable = false) })
   private HostEntity hostEntity;
 
   @ManyToOne
@@ -67,12 +67,16 @@ public class ConfigGroupHostMappingEntity {
     this.configGroupId = configGroupId;
   }
 
-  public String getHostname() {
-    return hostname;
+  public Long getHostId() {
+    return hostId;
+  }
+
+  public void setHostId(Long hostId) {
+    this.hostId = hostId;
   }
 
-  public void setHostname(String hostname) {
-    this.hostname = hostname;
+  public String getHostname() {
+    return hostEntity != null ? hostEntity.getHostName() : null;
   }
 
   public HostEntity getHostEntity() {
@@ -99,7 +103,7 @@ public class ConfigGroupHostMappingEntity {
     ConfigGroupHostMappingEntity that = (ConfigGroupHostMappingEntity) o;
 
     if (!configGroupId.equals(that.configGroupId)) return false;
-    if (!hostname.equals(that.hostname)) return false;
+    if (!hostEntity.equals(that.hostEntity)) return false;
 
     return true;
   }
@@ -107,7 +111,7 @@ public class ConfigGroupHostMappingEntity {
   @Override
   public int hashCode() {
     int result = configGroupId.hashCode();
-    result = 31 * result + hostname.hashCode();
+    result = 31 * result + hostEntity.hashCode();
     return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntityPK.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntityPK.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntityPK.java
index 3ee2b6c..e7ed99e 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntityPK.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ConfigGroupHostMappingEntityPK.java
@@ -23,7 +23,7 @@ import java.io.Serializable;
 
 public class ConfigGroupHostMappingEntityPK implements Serializable {
   private Long configGroupId;
-  private String hostname;
+  private Long hostId;
 
   @Id
   @Column(name = "config_group_id", nullable = false, insertable = true, updatable = true)
@@ -36,13 +36,13 @@ public class ConfigGroupHostMappingEntityPK implements Serializable {
   }
 
   @Id
-  @Column(name = "host_name", nullable = false, insertable = true, updatable = true)
-  public String getHostname() {
-    return hostname;
+  @Column(name = "host_id", nullable = false, insertable = true, updatable = true)
+  public Long getHostId() {
+    return hostId;
   }
 
-  public void setHostname(String hostname) {
-    this.hostname = hostname;
+  public void setHostId(Long hostId) {
+    this.hostId = hostId;
   }
 
   @Override
@@ -53,7 +53,7 @@ public class ConfigGroupHostMappingEntityPK implements Serializable {
     ConfigGroupHostMappingEntityPK that = (ConfigGroupHostMappingEntityPK) o;
 
     if (!configGroupId.equals(that.configGroupId)) return false;
-    if (!hostname.equals(that.hostname)) return false;
+    if (!hostId.equals(that.hostId)) return false;
 
     return true;
   }
@@ -61,7 +61,7 @@ public class ConfigGroupHostMappingEntityPK implements Serializable {
   @Override
   public int hashCode() {
     int result = configGroupId.hashCode();
-    result = 31 * result + hostname.hashCode();
+    result = 31 * result + hostId.hashCode();
     return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntity.java
index 1411a67..915b05f 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntity.java
@@ -36,8 +36,8 @@ public class HostConfigMappingEntity {
   private Long clusterId;
 
   @Id
-  @Column(name = "host_name", insertable = true, updatable = false, nullable = false)
-  private String hostName;
+  @Column(name = "host_id", insertable = true, updatable = false, nullable=false)
+  private Long hostId;
 
   @Id
   @Column(name = "type_name", insertable = true, updatable = false, nullable = false)
@@ -58,7 +58,7 @@ public class HostConfigMappingEntity {
   
   @Column(name = "user_name", insertable = true, updatable = true, nullable = false)
   private String user = null;
-  
+
   public Long getClusterId() {
     return clusterId;
   }
@@ -67,12 +67,12 @@ public class HostConfigMappingEntity {
     clusterId = id;
   }
 
-  public String getHostName() {
-    return hostName;
+  public Long getHostId() {
+    return hostId;
   }
 
-  public void setHostName(String name) {
-    hostName = name;
+  public void setHostId(Long hostId) {
+    this.hostId = hostId;
   }
 
   public String getType() {
@@ -140,7 +140,7 @@ public class HostConfigMappingEntity {
     if (clusterId != null ? !clusterId.equals(that.clusterId) : that.clusterId != null) return false;
     if (createTimestamp != null ? !createTimestamp.equals(that.createTimestamp) : that.createTimestamp != null)
       return false;
-    if (hostName != null ? !hostName.equals(that.hostName) : that.hostName != null) return false;
+    if (hostId != null ? !hostId.equals(that.hostId) : that.hostId != null) return false;
     if (serviceName != null ? !serviceName.equals(that.serviceName) : that.serviceName != null) return false;
     if (type != null ? !type.equals(that.type) : that.type != null) return false;
     if (user != null ? !user.equals(that.user) : that.user != null) return false;
@@ -152,7 +152,7 @@ public class HostConfigMappingEntity {
   @Override
   public int hashCode() {
     int result = clusterId != null ? clusterId.hashCode() : 0;
-    result = 31 * result + (hostName != null ? hostName.hashCode() : 0);
+    result = 31 * result + (hostId != null ? hostId.hashCode() : 0);
     result = 31 * result + (type != null ? type.hashCode() : 0);
     result = 31 * result + (createTimestamp != null ? createTimestamp.hashCode() : 0);
     result = 31 * result + (versionTag != null ? versionTag.hashCode() : 0);

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntityPK.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntityPK.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntityPK.java
index 16111fb..61039bb 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntityPK.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostConfigMappingEntityPK.java
@@ -27,7 +27,7 @@ import java.io.Serializable;
  */
 public class HostConfigMappingEntityPK implements Serializable {
   private Long clusterId;
-  private String hostName;
+  private Long hostId;
   private String type;
   private Long createTimestamp;
 
@@ -41,14 +41,14 @@ public class HostConfigMappingEntityPK implements Serializable {
     clusterId = id;
   }
   
-  @Column(name = "host_name", insertable = true, updatable = true, nullable = false)
+  @Column(name = "host_id", insertable = true, updatable = true, nullable = false)
   @Id
-  public String getHostName() {
-    return hostName;
+  public Long getHostId() {
+    return hostId;
   }
   
-  public void setHostName(String name) {
-    hostName = name;
+  public void setHostId(Long hostId) {
+    this.hostId = hostId;
   }
   
   @Column(name = "type_name", insertable = true, updatable = true, nullable = false)
@@ -81,7 +81,7 @@ public class HostConfigMappingEntityPK implements Serializable {
     HostConfigMappingEntityPK that = (HostConfigMappingEntityPK) o;
 
     if (clusterId != null ? !clusterId.equals(that.clusterId) : that.clusterId != null) return false;
-    if (hostName != null ? !hostName.equals(that.hostName) : that.hostName != null) return false;
+    if (hostId != null ? !hostId.equals(that.hostId) : that.hostId != null) return false;
     if (type != null ? !type.equals(that.type) : that.type != null) return false;
     if (createTimestamp != null ? !createTimestamp.equals (that.createTimestamp) : that.createTimestamp != null) return false;
 
@@ -92,10 +92,8 @@ public class HostConfigMappingEntityPK implements Serializable {
   public int hashCode() {
     int result = clusterId !=null ? clusterId.intValue() : 0;
     result = 31 * result + (type != null ? type.hashCode() : 0);
-    result = 31 * result + (hostName != null ? hostName.hashCode() : 0);
+    result = 31 * result + (hostId != null ? hostId.hashCode() : 0);
     result = 31 * result + createTimestamp.intValue();
     return result;
-  }  
-  
-  
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceConfigEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceConfigEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceConfigEntity.java
index 1a31252..cd3bb54 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceConfigEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceConfigEntity.java
@@ -83,8 +83,8 @@ public class ServiceConfigEntity {
 
   @ElementCollection()
   @CollectionTable(name = "serviceconfighosts", joinColumns = {@JoinColumn(name = "service_config_id")})
-  @Column(name = "hostname")
-  private List<String> hostNames;
+  @Column(name = "host_id")
+  private List<Long> hostIds;
 
   @ManyToMany
   @JoinTable(
@@ -185,12 +185,12 @@ public class ServiceConfigEntity {
     this.groupId = groupId;
   }
 
-  public List<String> getHostNames() {
-    return hostNames;
+  public List<Long> getHostIds() {
+    return hostIds;
   }
 
-  public void setHostNames(List<String> hostNames) {
-    this.hostNames = hostNames;
+  public void setHostIds(List<Long> hostIds) {
+    this.hostIds = hostIds;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java b/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
index 855bb3f..209293f 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
@@ -406,16 +406,16 @@ public interface Cluster {
 
   /**
    * Fetch desired configs for list of hosts in cluster
-   * @param hostnames
+   * @param hostIds
    * @return
    */
-  Map<String, Map<String, DesiredConfig>> getHostsDesiredConfigs(Collection<String> hostnames);
+  Map<Long, Map<String, DesiredConfig>> getHostsDesiredConfigs(Collection<Long> hostIds);
 
   /**
    * Fetch desired configs for all hosts in cluster
    * @return
    */
-  Map<String, Map<String, DesiredConfig>> getAllHostsDesiredConfigs();
+  Map<Long, Map<String, DesiredConfig>> getAllHostsDesiredConfigs();
 
   /**
    * Add a new config group to the set of Config groups associated with this

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/Clusters.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/Clusters.java b/ambari-server/src/main/java/org/apache/ambari/server/state/Clusters.java
index 80ac6a7..6edf7d9 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/Clusters.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/Clusters.java
@@ -80,6 +80,14 @@ public interface Clusters {
   public Host getHost(String hostname) throws AmbariException;
 
   /**
+   * Get a Host object managed by this server
+   * @param hostId Host Id from the {@link org.apache.ambari.server.orm.entities.HostEntity} objecty
+   * @return Host object
+   * @throws AmbariException
+   */
+  public Host getHostById(Long hostId) throws AmbariException;
+
+  /**
    * Add a Host object to be managed by this server
    * @param hostname Host to be added
    * @throws AmbariException
@@ -137,6 +145,15 @@ public interface Clusters {
       throws AmbariException;
 
   /**
+   * Gets all the host Ids associated with the cluster
+   * @param clusterName The name of the cluster
+   * @return <code>Map</code> containing host id and <code>Host</code>
+   * @throws AmbariException
+   */
+  public Map<Long, Host> getHostIdsForCluster(String clusterName)
+      throws AmbariException;
+
+  /**
    * Deletes the cluster identified by the name
    * @param clusterName The name of the cluster
    * @throws AmbariException

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
index 2558de8..bb6f6c1 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
@@ -290,7 +290,6 @@ public class ClusterImpl implements Cluster {
     return clusterGlobalLock;
   }
 
-
   private void loadServiceConfigTypes() throws AmbariException {
     try {
       serviceConfigTypes = collectServiceConfigTypesMapping();
@@ -501,22 +500,24 @@ public class ClusterImpl implements Cluster {
 
     clusterGlobalLock.readLock().lock();
     try {
-      Set<ConfigGroupHostMapping> hostMappingEntities = configGroupHostMappingDAO.findByHost(hostname);
-
-      if (hostMappingEntities != null && !hostMappingEntities.isEmpty()) {
-        for (ConfigGroupHostMapping entity : hostMappingEntities) {
-          ConfigGroup configGroup = configGroupMap.get(entity.getConfigGroupId());
-          if (configGroup != null
-              && !configGroups.containsKey(configGroup.getId())) {
-            configGroups.put(configGroup.getId(), configGroup);
+      HostEntity hostEntity = hostDAO.findByName(hostname);
+      if (hostEntity != null) {
+        Set<ConfigGroupHostMapping> hostMappingEntities = configGroupHostMappingDAO.findByHostId(hostEntity.getHostId());
+
+        if (hostMappingEntities != null && !hostMappingEntities.isEmpty()) {
+          for (ConfigGroupHostMapping entity : hostMappingEntities) {
+            ConfigGroup configGroup = configGroupMap.get(entity.getConfigGroupId());
+            if (configGroup != null
+                && !configGroups.containsKey(configGroup.getId())) {
+              configGroups.put(configGroup.getId(), configGroup);
+            }
           }
         }
       }
-      return configGroups;
-
     } finally {
       clusterGlobalLock.readLock().unlock();
     }
+    return configGroups;
   }
 
   @Override
@@ -1964,6 +1965,9 @@ public class ClusterImpl implements Cluster {
         }
       }
 
+      // TODO AMBARI-10679, need efficient caching from hostId to hostName
+      Map<Long, String> hostIdToName = new HashMap<Long, String>();
+
       if (!map.isEmpty()) {
         Map<String, List<HostConfigMapping>> hostMappingsByType = hostConfigMappingDAO.findSelectedHostsByTypes(
             clusterEntity.getClusterId(), types);
@@ -1971,8 +1975,14 @@ public class ClusterImpl implements Cluster {
         for (Entry<String, DesiredConfig> entry : map.entrySet()) {
           List<DesiredConfig.HostOverride> hostOverrides = new ArrayList<DesiredConfig.HostOverride>();
           for (HostConfigMapping mappingEntity : hostMappingsByType.get(entry.getKey())) {
+
+            if (!hostIdToName.containsKey(mappingEntity.getHostId())) {
+              HostEntity hostEntity = hostDAO.findById(mappingEntity.getHostId());
+              hostIdToName.put(mappingEntity.getHostId(), hostEntity.getHostName());
+            }
+
             hostOverrides.add(new DesiredConfig.HostOverride(
-                mappingEntity.getHostName(), mappingEntity.getVersion()));
+                hostIdToName.get(mappingEntity.getHostId()), mappingEntity.getVersion()));
           }
           entry.getValue().setHostOverrides(hostOverrides);
         }
@@ -2002,7 +2012,7 @@ public class ClusterImpl implements Cluster {
       }
 
       serviceConfigEntity.setClusterConfigEntities(configEntities);
-      serviceConfigEntity.setHostNames(new ArrayList<String>(configGroup.getHosts().keySet()));
+      serviceConfigEntity.setHostIds(new ArrayList<Long>(configGroup.getHosts().keySet()));
 
     } else {
       List<ClusterConfigEntity> configEntities = getClusterConfigEntitiesByService(serviceName);
@@ -2248,13 +2258,15 @@ public class ClusterImpl implements Cluster {
         }
         configGroup.setConfigurations(groupDesiredConfigs);
 
-        Map<String, Host> groupDesiredHosts = new HashMap<String, Host>();
-        for (String hostname : serviceConfigEntity.getHostNames()) {
-          Host host = clusters.getHost(hostname);
-          if (host != null) {
-            groupDesiredHosts.put(hostname, host);
-          } else {
-            LOG.warn("Host {} doesn't exist anymore, skipping", hostname);
+        Map<Long, Host> groupDesiredHosts = new HashMap<Long, Host>();
+        if (serviceConfigEntity.getHostIds() != null) {
+          for (Long hostId : serviceConfigEntity.getHostIds()) {
+            Host host = clusters.getHostById(hostId);
+            if (host != null) {
+              groupDesiredHosts.put(hostId, host);
+            } else {
+              LOG.warn("Host with id {} doesn't exist anymore, skipping", hostId);
+            }
           }
         }
         configGroup.setHosts(groupDesiredHosts);
@@ -2275,7 +2287,7 @@ public class ClusterImpl implements Cluster {
     serviceConfigEntityClone.setStack(serviceConfigEntity.getStack());
     serviceConfigEntityClone.setClusterConfigEntities(serviceConfigEntity.getClusterConfigEntities());
     serviceConfigEntityClone.setClusterId(serviceConfigEntity.getClusterId());
-    serviceConfigEntityClone.setHostNames(serviceConfigEntity.getHostNames());
+    serviceConfigEntityClone.setHostIds(serviceConfigEntity.getHostIds());
     serviceConfigEntityClone.setGroupId(serviceConfigEntity.getGroupId());
     serviceConfigEntityClone.setNote(serviceConfigVersionNote);
     serviceConfigEntityClone.setVersion(nextServiceConfigVersion);
@@ -2388,21 +2400,20 @@ public class ClusterImpl implements Cluster {
     }
   }
 
-
   @Override
-  public Map<String, Map<String, DesiredConfig>> getHostsDesiredConfigs(Collection<String> hostnames) {
+  public Map<Long, Map<String, DesiredConfig>> getHostsDesiredConfigs(Collection<Long> hostIds) {
 
-    if (hostnames == null || hostnames.isEmpty()) {
+    if (hostIds == null || hostIds.isEmpty()) {
       return Collections.emptyMap();
     }
 
     Set<HostConfigMapping> mappingEntities =
-      hostConfigMappingDAO.findSelectedByHosts(clusterEntity.getClusterId(), hostnames);
+        hostConfigMappingDAO.findSelectedByHosts(hostIds);
 
-    Map<String, Map<String, DesiredConfig>> desiredConfigsByHost = new HashMap<String, Map<String, DesiredConfig>>();
+    Map<Long, Map<String, DesiredConfig>> desiredConfigsByHost = new HashMap<Long, Map<String, DesiredConfig>>();
 
-    for (String hostname : hostnames) {
-      desiredConfigsByHost.put(hostname, new HashMap<String, DesiredConfig>());
+    for (Long hostId : hostIds) {
+      desiredConfigsByHost.put(hostId, new HashMap<String, DesiredConfig>());
     }
 
     for (HostConfigMapping mappingEntity : mappingEntities) {
@@ -2411,23 +2422,23 @@ public class ClusterImpl implements Cluster {
       desiredConfig.setServiceName(mappingEntity.getServiceName());
       desiredConfig.setUser(mappingEntity.getUser());
 
-      desiredConfigsByHost.get(mappingEntity.getHostName()).put(mappingEntity.getType(), desiredConfig);
+      desiredConfigsByHost.get(mappingEntity.getHostId()).put(mappingEntity.getType(), desiredConfig);
     }
 
     return desiredConfigsByHost;
   }
 
   @Override
-  public Map<String, Map<String, DesiredConfig>> getAllHostsDesiredConfigs() {
+  public Map<Long, Map<String, DesiredConfig>> getAllHostsDesiredConfigs() {
 
-    Collection<String> hostnames;
+    Collection<Long> hostIds;
     try {
-      hostnames = clusters.getHostsForCluster(clusterEntity.getClusterName()).keySet();
+      hostIds = clusters.getHostIdsForCluster(clusterEntity.getClusterName()).keySet();
     } catch (AmbariException ignored) {
       return Collections.emptyMap();
     }
 
-    return getHostsDesiredConfigs(hostnames);
+    return getHostsDesiredConfigs(hostIds);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClustersImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClustersImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClustersImpl.java
index 70788ff..f8066ca 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClustersImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClustersImpl.java
@@ -89,6 +89,7 @@ public class ClustersImpl implements Clusters {
   private ConcurrentHashMap<String, Cluster> clusters;
   private ConcurrentHashMap<Long, Cluster> clustersById;
   private ConcurrentHashMap<String, Host> hosts;
+  private ConcurrentHashMap<Long, Host> hostsById;
   private ConcurrentHashMap<String, Set<Cluster>> hostClusterMap;
   private ConcurrentHashMap<String, Set<Host>> clusterHostMap;
 
@@ -138,6 +139,7 @@ public class ClustersImpl implements Clusters {
     clusters = new ConcurrentHashMap<String, Cluster>();
     clustersById = new ConcurrentHashMap<Long, Cluster>();
     hosts = new ConcurrentHashMap<String, Host>();
+    hostsById = new ConcurrentHashMap<Long, Host>();
     hostClusterMap = new ConcurrentHashMap<String, Set<Cluster>>();
     clusterHostMap = new ConcurrentHashMap<String, Set<Host>>();
 
@@ -172,6 +174,7 @@ public class ClustersImpl implements Clusters {
     for (HostEntity hostEntity : hostDAO.findAll()) {
       Host host = hostFactory.create(hostEntity, true);
       hosts.put(hostEntity.getHostName(), host);
+      hostsById.put(hostEntity.getHostId(), host);
       Set<Cluster> cSet = Collections.newSetFromMap(new ConcurrentHashMap<Cluster, Boolean>());
       hostClusterMap.put(hostEntity.getHostName(), cSet);
 
@@ -328,6 +331,17 @@ public class ClustersImpl implements Clusters {
     return hosts.get(hostname);
   }
 
+  @Override
+  public Host getHostById(Long hostId) throws AmbariException {
+    checkLoaded();
+
+    if (!hosts.containsKey(hostId)) {
+      throw new HostNotFoundException("Host Id = " + hostId);
+    }
+
+    return hosts.get(hostId);
+  }
+
   /**
    * Register a host by creating a {@link HostEntity} object in the database and setting its state to
    * {@link HostState#INIT}. This does not add the host the cluster.
@@ -614,6 +628,27 @@ public class ClustersImpl implements Clusters {
   }
 
   @Override
+  public Map<Long, Host> getHostIdsForCluster(String clusterName)
+      throws AmbariException {
+
+    checkLoaded();
+    r.lock();
+
+    try {
+      Map<Long, Host> hosts = new HashMap<Long, Host>();
+
+      for (Host h : clusterHostMap.get(clusterName)) {
+        HostEntity hostEntity = hostDAO.findByName(h.getHostName());
+        hosts.put(hostEntity.getHostId(), h);
+      }
+
+      return hosts;
+    } finally {
+      r.unlock();
+    }
+  }
+
+  @Override
   public void deleteCluster(String clusterName)
       throws AmbariException {
     checkLoaded();
@@ -649,6 +684,7 @@ public class ClustersImpl implements Clusters {
       throws AmbariException {
     Host host = null;
     Cluster cluster = null;
+    HostEntity hostEntity = null;
 
     checkLoaded();
 
@@ -656,6 +692,7 @@ public class ClustersImpl implements Clusters {
     try {
       host = getHost(hostname);
       cluster = getCluster(clusterName);
+      hostEntity = hostDAO.findByName(hostname);
     } finally {
       r.unlock();
     }
@@ -677,7 +714,7 @@ public class ClustersImpl implements Clusters {
       host.refresh();
       cluster.refresh();
 
-      deleteConfigGroupHostMapping(hostname);
+      deleteConfigGroupHostMapping(hostEntity.getHostId());
 
       // Remove mapping of principals to the unmapped host
       kerberosPrincipalHostDAO.removeByHost(hostname);
@@ -699,11 +736,11 @@ public class ClustersImpl implements Clusters {
   }
 
   @Transactional
-  private void deleteConfigGroupHostMapping(String hostname) throws AmbariException {
+  private void deleteConfigGroupHostMapping(Long hostId) throws AmbariException {
     // Remove Config group mapping
     for (Cluster cluster : clusters.values()) {
       for (ConfigGroup configGroup : cluster.getConfigGroups().values()) {
-        configGroup.removeHost(hostname);
+        configGroup.removeHost(hostId);
       }
     }
   }
@@ -719,17 +756,23 @@ public class ClustersImpl implements Clusters {
     w.lock();
 
     try {
-      deleteConfigGroupHostMapping(hostname);
+      HostEntity entity = hostDAO.findByName(hostname);
+      
+      if (entity == null) {
+        return;
+      }
+
+      deleteConfigGroupHostMapping(entity.getHostId());
 
       Collection<HostVersionEntity> hostVersions = hosts.get(hostname).getAllHostVersions();
       for (HostVersionEntity hostVersion : hostVersions) {
         hostVersionDAO.remove(hostVersion);
       }
 
-      HostEntity entity = hostDAO.findByName(hostname);
       hostDAO.refresh(entity);
       hostDAO.remove(entity);
       hosts.remove(hostname);
+      hostsById.remove(entity.getHostId());
 
       // Remove mapping of principals to deleted host
       kerberosPrincipalHostDAO.removeByHost(hostname);
@@ -743,7 +786,6 @@ public class ClustersImpl implements Clusters {
     } finally {
       w.unlock();
     }
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroup.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroup.java b/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroup.java
index a4cc6ac..4c806e5 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroup.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroup.java
@@ -84,7 +84,7 @@ public interface ConfigGroup {
    * List of hosts to which configs are applied
    * @return
    */
-  public Map<String, Host> getHosts();
+  public Map<Long, Host> getHosts();
 
   /**
    * List of @Config objects
@@ -137,7 +137,7 @@ public interface ConfigGroup {
    * Reassign the set of hosts associated with this config group
    * @param hosts
    */
-  public void setHosts(Map<String, Host> hosts);
+  public void setHosts(Map<Long, Host> hosts);
 
   /**
    * Reassign the set of configs associated with this config group
@@ -148,7 +148,7 @@ public interface ConfigGroup {
   /**
    * Remove host mapping
    */
-  public void removeHost(String hostname) throws AmbariException;
+  public void removeHost(Long hostId) throws AmbariException;
 
   String getServiceName();
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupFactory.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupFactory.java b/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupFactory.java
index d4597af..9abadf3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupFactory.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupFactory.java
@@ -32,7 +32,7 @@ public interface ConfigGroupFactory {
                        @Assisted("tag") String tag,
                        @Assisted("description") String description,
                        @Assisted("configs") Map<String, Config> configs,
-                       @Assisted("hosts") Map<String, Host> hosts);
+                       @Assisted("hosts") Map<Long, Host> hosts);
 
   ConfigGroup createExisting(Cluster cluster, ConfigGroupEntity entity);
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupImpl.java
index ffa085a..4c5d016 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/configgroup/ConfigGroupImpl.java
@@ -62,7 +62,7 @@ public class ConfigGroupImpl implements ConfigGroup {
 
   private Cluster cluster;
   private ConfigGroupEntity configGroupEntity;
-  private Map<String, Host> hosts;
+  private Map<Long, Host> hosts;
   private Map<String, Config> configurations;
   private volatile boolean isPersisted = false;
 
@@ -89,7 +89,7 @@ public class ConfigGroupImpl implements ConfigGroup {
                          @Assisted("tag") String tag,
                          @Assisted("description") String description,
                          @Assisted("configs") Map<String, Config> configs,
-                         @Assisted("hosts") Map<String, Host> hosts,
+                         @Assisted("hosts") Map<Long, Host> hosts,
                          Injector injector) {
     injector.injectMembers(this);
     this.cluster = cluster;
@@ -103,7 +103,7 @@ public class ConfigGroupImpl implements ConfigGroup {
     if (hosts != null) {
       this.hosts = hosts;
     } else {
-      this.hosts = new HashMap<String, Host>();
+      this.hosts = new HashMap<Long, Host>();
     }
 
     if (configs != null) {
@@ -122,7 +122,7 @@ public class ConfigGroupImpl implements ConfigGroup {
 
     this.configGroupEntity = configGroupEntity;
     configurations = new HashMap<String, Config>();
-    hosts = new HashMap<String, Host>();
+    hosts = new HashMap<Long, Host>();
 
     // Populate configs
     for (ConfigGroupConfigMappingEntity configMappingEntity : configGroupEntity
@@ -147,8 +147,9 @@ public class ConfigGroupImpl implements ConfigGroup {
 
       try {
         Host host = clusters.getHost(hostMappingEntity.getHostname());
-        if (host != null) {
-          hosts.put(host.getHostName(), host);
+        HostEntity hostEntity = hostMappingEntity.getHostEntity();
+        if (host != null && hostEntity != null) {
+          hosts.put(hostEntity.getHostId(), host);
         }
       } catch (AmbariException e) {
         String msg = "Host seems to be deleted but Config group mapping still " +
@@ -235,7 +236,7 @@ public class ConfigGroupImpl implements ConfigGroup {
   }
 
   @Override
-  public Map<String, Host> getHosts() {
+  public Map<Long, Host> getHosts() {
     readWriteLock.readLock().lock();
     try {
       return Collections.unmodifiableMap(hosts);
@@ -260,7 +261,7 @@ public class ConfigGroupImpl implements ConfigGroup {
    * @param hosts
    */
   @Override
-  public void setHosts(Map<String, Host> hosts) {
+  public void setHosts(Map<Long, Host> hosts) {
     readWriteLock.writeLock().lock();
     try {
       this.hosts = hosts;
@@ -287,23 +288,25 @@ public class ConfigGroupImpl implements ConfigGroup {
 
   @Override
   @Transactional
-  public void removeHost(String hostname) throws AmbariException {
+  public void removeHost(Long hostId) throws AmbariException {
     readWriteLock.writeLock().lock();
     try {
-      if (hosts.containsKey(hostname)) {
-        LOG.info("Removing host from config group, hostname = " + hostname);
-        hosts.remove(hostname);
+      if (hosts.containsKey(hostId)) {
+        String hostName = hosts.get(hostId).getHostName();
+        LOG.info("Removing host from config group, hostid = " + hostId + ", hostname = " + hostName);
+        hosts.remove(hostId);
         try {
           ConfigGroupHostMappingEntityPK hostMappingEntityPK = new
             ConfigGroupHostMappingEntityPK();
-          hostMappingEntityPK.setHostname(hostname);
+          hostMappingEntityPK.setHostId(hostId);
           hostMappingEntityPK.setConfigGroupId(configGroupEntity.getGroupId());
           configGroupHostMappingDAO.removeByPK(hostMappingEntityPK);
         } catch (Exception e) {
           LOG.error("Failed to delete config group host mapping"
             + ", clusterName = " + getClusterName()
             + ", id = " + getId()
-            + ", hostname = " + hostname, e);
+            + ", hostid = " + hostId
+            + ", hostname = " + hostName, e);
           throw new AmbariException(e.getMessage());
         }
       }
@@ -366,7 +369,7 @@ public class ConfigGroupImpl implements ConfigGroup {
         if (hostEntity != null) {
           ConfigGroupHostMappingEntity hostMappingEntity = new
             ConfigGroupHostMappingEntity();
-          hostMappingEntity.setHostname(host.getHostName());
+          hostMappingEntity.setHostId(hostEntity.getHostId());
           hostMappingEntity.setHostEntity(hostEntity);
           hostMappingEntity.setConfigGroupEntity(configGroupEntity);
           hostMappingEntity.setConfigGroupId(configGroupEntity.getGroupId());
@@ -487,7 +490,10 @@ public class ConfigGroupImpl implements ConfigGroup {
               configGroupEntity.getGroupName());
           }
         }
-        hosts.put(host.getHostName(), host);
+        HostEntity hostEntity = hostDAO.findByName(host.getHostName());
+        if (hostEntity != null) {
+          hosts.put(hostEntity.getHostId(), host);
+        }
       }
     } finally {
       readWriteLock.writeLock().unlock();

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/state/host/HostImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/host/HostImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/state/host/HostImpl.java
index 41cfee7..50d762e 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/host/HostImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/host/HostImpl.java
@@ -1069,6 +1069,29 @@ public class HostImpl implements Host {
   }
 
   @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    Host that = (Host) o;
+
+    return this.getHostName().equals(that.getHostName());
+  }
+
+  @Override
+  public int hashCode() {
+    return (null == getHostName() ? 0 : getHostName().hashCode());
+  }
+
+  public int compareTo(HostEntity other) {
+    return getHostName().compareTo(other.getHostName());
+  }
+  
+  @Override
   public HostResponse convertToResponse() {
     try {
       readLock.lock();
@@ -1200,7 +1223,7 @@ public class HostImpl implements Host {
     try {
       // set all old mappings for this type to empty
       for (HostConfigMapping e : hostConfigMappingDAO.findByType(clusterId,
-          hostEntity.getHostName(), config.getType())) {
+          hostEntity.getHostId(), config.getType())) {
         e.setSelected(0);
         hostConfigMappingDAO.merge(e);
       }
@@ -1208,7 +1231,7 @@ public class HostImpl implements Host {
       HostConfigMapping hostConfigMapping = new HostConfigMappingImpl();
       hostConfigMapping.setClusterId(clusterId);
       hostConfigMapping.setCreateTimestamp(System.currentTimeMillis());
-      hostConfigMapping.setHostName(hostEntity.getHostName());
+      hostConfigMapping.setHostId(hostEntity.getHostId());
       hostConfigMapping.setSelected(1);
       hostConfigMapping.setUser(user);
       hostConfigMapping.setType(config.getType());
@@ -1230,7 +1253,7 @@ public class HostImpl implements Host {
     Map<String, DesiredConfig> map = new HashMap<String, DesiredConfig>();
 
     for (HostConfigMapping e : hostConfigMappingDAO.findSelected(
-        clusterId, hostEntity.getHostName())) {
+        clusterId, hostEntity.getHostId())) {
 
       DesiredConfig dc = new DesiredConfig();
       dc.setTag(e.getVersion());
@@ -1289,7 +1312,7 @@ public class HostImpl implements Host {
   }
 
   private HostConfigMapping getDesiredConfigEntity(long clusterId, String type) {
-    return hostConfigMappingDAO.findSelectedByType(clusterId, hostEntity.getHostName(), type);
+    return hostConfigMappingDAO.findSelectedByType(clusterId, hostEntity.getHostId(), type);
   }
 
   private void ensureMaintMap() {


[2/3] ambari git commit: AMBARI-10679. Full Delete of Host : Switch config related tables to use host_id instead of host_name column (alejandro)

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
index 0373aac..6249d2a 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
@@ -63,6 +63,7 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
   private static final String HOST_CONFIG_MAPPING_TABLE = "hostconfigmapping";
   private static final String CONFIG_GROUP_HOST_MAPPING_TABLE = "configgrouphostmapping";
   private static final String KERBEROS_PRINCIPAL_HOST_TABLE = "kerberos_principal_host";
+  private static final String SERVICE_CONFIG_HOSTS_TABLE = "serviceconfighosts";
   private static final String CLUSTER_HOST_MAPPING_TABLE = "ClusterHostMapping";
   private static final String WIDGET_TABLE = "widget";
   private static final String WIDGET_LAYOUT_TABLE = "widget_layout";
@@ -254,10 +255,6 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
 
     // TODO, for now, these still point to the host_name and will be fixed one table at a time to point to the host id.
     // Re-add the FKs
-    dbAccessor.addFKConstraint(HOST_CONFIG_MAPPING_TABLE, "FK_hostconfmapping_host_name",
-        "host_name", HOSTS_TABLE, "host_name", false);
-    dbAccessor.addFKConstraint(CONFIG_GROUP_HOST_MAPPING_TABLE, "FK_cghm_hname",
-        "host_name", HOSTS_TABLE, "host_name", false);
     dbAccessor.addFKConstraint(KERBEROS_PRINCIPAL_HOST_TABLE, "FK_krb_pr_host_host_name",
         "host_name", HOSTS_TABLE, "host_name", false);
 
@@ -265,22 +262,31 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
     // Add host_id to the host-related tables, and populate the host_id, one table at a time.
     // TODO, include other tables.
     String[] tablesToAddHostID = new String[] {
+        CONFIG_GROUP_HOST_MAPPING_TABLE,
         CLUSTER_HOST_MAPPING_TABLE,
+        HOST_CONFIG_MAPPING_TABLE,
         HOST_COMPONENT_STATE_TABLE,
         HOST_COMPONENT_DESIRED_STATE_TABLE,
         HOST_ROLE_COMMAND_TABLE,
         HOST_STATE_TABLE,
-        HOST_VERSION_TABLE
+        HOST_VERSION_TABLE,
+        SERVICE_CONFIG_HOSTS_TABLE
     };
 
     for (String tableName : tablesToAddHostID) {
       dbAccessor.addColumn(tableName, new DBColumnInfo(HOST_ID_COL, Long.class, null, null, true));
-      dbAccessor.executeQuery("UPDATE " + tableName + " t SET host_id = (SELECT host_id FROM hosts h WHERE h.host_name = t.host_name) WHERE t.host_id IS NULL AND t.host_name IS NOT NULL");
 
-      // For legacy reasons, the hostrolecommand table will contain "none" for some records where the host_name was not important.
-      // These records were populated during Finalize in Rolling Upgrade, so they must be updated to use a valid host_name.
-      if (tableName == HOST_ROLE_COMMAND_TABLE && StringUtils.isNotBlank(randomHostName)) {
-        dbAccessor.executeQuery("UPDATE " + tableName + " t SET host_id = (SELECT host_id FROM hosts h WHERE h.host_name = '" + randomHostName + "') WHERE t.host_id IS NULL AND t.host_name = 'none'");
+      // The column name is different for one table
+      String hostNameColumnName = tableName == SERVICE_CONFIG_HOSTS_TABLE ? "hostname" : "host_name";
+
+      if (dbAccessor.tableHasData(tableName)) {
+        dbAccessor.executeQuery("UPDATE " + tableName + " t SET host_id = (SELECT host_id FROM hosts h WHERE h.host_name = t." + hostNameColumnName + ") WHERE t.host_id IS NULL AND t." + hostNameColumnName + " IS NOT NULL");
+
+        // For legacy reasons, the hostrolecommand table will contain "none" for some records where the host_name was not important.
+        // These records were populated during Finalize in Rolling Upgrade, so they must be updated to use a valid host_name.
+        if (tableName == HOST_ROLE_COMMAND_TABLE && StringUtils.isNotBlank(randomHostName)) {
+          dbAccessor.executeQuery("UPDATE " + tableName + " t SET host_id = (SELECT host_id FROM hosts h WHERE h.host_name = '" + randomHostName + "') WHERE t.host_id IS NULL AND t.host_name = 'none'");
+        }
       }
 
       if (databaseType == Configuration.DatabaseType.DERBY) {
@@ -293,24 +299,33 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
 
     // These are the FKs that have already been corrected.
     // TODO, include other tables.
+    dbAccessor.addFKConstraint(CONFIG_GROUP_HOST_MAPPING_TABLE, "FK_cghm_host_id",
+        "host_id", HOSTS_TABLE, "host_id", false);
     dbAccessor.addFKConstraint(CLUSTER_HOST_MAPPING_TABLE, "FK_clusterhostmapping_host_id",
         "host_id", HOSTS_TABLE, "host_id", false);
+    dbAccessor.addFKConstraint(HOST_CONFIG_MAPPING_TABLE, "FK_hostconfmapping_host_id",
+        "host_id", HOSTS_TABLE, "host_id", false);
     dbAccessor.addFKConstraint(HOST_COMPONENT_STATE_TABLE, "FK_hostcomponentstate_host_id",
         "host_id", HOSTS_TABLE, "host_id", false);
     dbAccessor.addFKConstraint(HOST_COMPONENT_DESIRED_STATE_TABLE, "FK_hcdesiredstate_host_id",
         "host_id", HOSTS_TABLE, "host_id", false);
     dbAccessor.addFKConstraint(HOST_STATE_TABLE, "FK_hoststate_host_id",
         "host_id", HOSTS_TABLE, "host_id", false);
+    dbAccessor.addFKConstraint(SERVICE_CONFIG_HOSTS_TABLE, "FK_scvhosts_host_id",
+        "host_id", HOSTS_TABLE, "host_id", false);
 
 
 
     // For any tables where the host_name was part of the PK, need to drop the PK, and recreate it with the host_id
     // TODO, include other tables.
     String[] tablesWithHostNameInPK =  new String[] {
+        CONFIG_GROUP_HOST_MAPPING_TABLE,
         CLUSTER_HOST_MAPPING_TABLE,
+        HOST_CONFIG_MAPPING_TABLE,
         HOST_COMPONENT_STATE_TABLE,
         HOST_COMPONENT_DESIRED_STATE_TABLE,
-        HOST_STATE_TABLE
+        HOST_STATE_TABLE,
+        SERVICE_CONFIG_HOSTS_TABLE
     };
 
     if (databaseType == Configuration.DatabaseType.DERBY) {
@@ -321,29 +336,43 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
         }
       }
     } else {
+      dbAccessor.executeQuery("ALTER TABLE " + CONFIG_GROUP_HOST_MAPPING_TABLE + " DROP CONSTRAINT configgrouphostmapping_pkey");
       dbAccessor.executeQuery("ALTER TABLE " + CLUSTER_HOST_MAPPING_TABLE + " DROP CONSTRAINT clusterhostmapping_pkey");
+      dbAccessor.executeQuery("ALTER TABLE " + HOST_CONFIG_MAPPING_TABLE + " DROP CONSTRAINT hostconfigmapping_pkey");
       dbAccessor.executeQuery("ALTER TABLE " + HOST_COMPONENT_STATE_TABLE + " DROP CONSTRAINT hostcomponentstate_pkey");
       dbAccessor.executeQuery("ALTER TABLE " + HOST_COMPONENT_DESIRED_STATE_TABLE + " DROP CONSTRAINT hostcomponentdesiredstate_pkey");
       dbAccessor.executeQuery("ALTER TABLE " + HOST_STATE_TABLE + " DROP CONSTRAINT hoststate_pkey");
+      dbAccessor.executeQuery("ALTER TABLE " + SERVICE_CONFIG_HOSTS_TABLE + " DROP CONSTRAINT serviceconfighosts_pkey");
       // TODO, include other tables.
     }
+    dbAccessor.executeQuery("ALTER TABLE " + CONFIG_GROUP_HOST_MAPPING_TABLE +
+        " ADD CONSTRAINT configgrouphostmapping_pkey PRIMARY KEY (config_group_id, host_id)");
     dbAccessor.executeQuery("ALTER TABLE " + CLUSTER_HOST_MAPPING_TABLE +
         " ADD CONSTRAINT clusterhostmapping_pkey PRIMARY KEY (cluster_id, host_id)");
+    dbAccessor.executeQuery("ALTER TABLE " + HOST_CONFIG_MAPPING_TABLE +
+        " ADD CONSTRAINT hostconfigmapping_pkey PRIMARY KEY (cluster_id, host_id, type_name, create_timestamp)");
     dbAccessor.executeQuery("ALTER TABLE " + HOST_COMPONENT_STATE_TABLE +
         " ADD CONSTRAINT hostcomponentstate_pkey PRIMARY KEY (cluster_id, component_name, host_id, service_name)");
     dbAccessor.executeQuery("ALTER TABLE " + HOST_COMPONENT_DESIRED_STATE_TABLE +
         " ADD CONSTRAINT hostcomponentdesiredstate_pkey PRIMARY KEY (cluster_id, component_name, host_id, service_name)");
     dbAccessor.executeQuery("ALTER TABLE " + HOST_STATE_TABLE +
         " ADD CONSTRAINT hoststate_pkey PRIMARY KEY (host_id)");
+    dbAccessor.executeQuery("ALTER TABLE " + SERVICE_CONFIG_HOSTS_TABLE +
+        " ADD CONSTRAINT serviceconfighosts_pkey PRIMARY KEY (service_config_id, host_id)");
     // TODO, include other tables.
 
     // Finish by deleting the unnecessary host_name columns.
+    dbAccessor.dropColumn(CONFIG_GROUP_HOST_MAPPING_TABLE, "host_name");
     dbAccessor.dropColumn(CLUSTER_HOST_MAPPING_TABLE, "host_name");
+    dbAccessor.dropColumn(HOST_CONFIG_MAPPING_TABLE, "host_name");
     dbAccessor.dropColumn(HOST_COMPONENT_STATE_TABLE, "host_name");
     dbAccessor.dropColumn(HOST_COMPONENT_DESIRED_STATE_TABLE, "host_name");
     dbAccessor.dropColumn(HOST_ROLE_COMMAND_TABLE, "host_name");
     dbAccessor.dropColumn(HOST_STATE_TABLE, "host_name");
     dbAccessor.dropColumn(HOST_VERSION_TABLE, "host_name");
+
+    // Notice that the column name doesn't have an underscore here.
+    dbAccessor.dropColumn(SERVICE_CONFIG_HOSTS_TABLE, "hostname");
     // TODO, include other tables.
 
     // view columns for cluster association

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/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 b6f2aaa..1a146e0 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
@@ -78,10 +78,8 @@ CREATE TABLE serviceconfig (
 
 CREATE TABLE serviceconfighosts (
   service_config_id BIGINT NOT NULL,
-  hostname VARCHAR(255) NOT NULL,
-  -- host_id BIGINT NOT NULL,
-  PRIMARY KEY(service_config_id, hostname));
-  -- PRIMARY KEY(service_config_id, host_id));
+  host_id BIGINT NOT NULL,
+  PRIMARY KEY(service_config_id, host_id));
 
 CREATE TABLE serviceconfigmapping (
   service_config_id BIGINT NOT NULL,
@@ -315,16 +313,14 @@ CREATE TABLE clusterconfigmapping (
 
 CREATE TABLE hostconfigmapping (
   create_timestamp BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
-  -- host_id BIGINT NOT NULL,
+  host_id BIGINT NOT NULL,
   cluster_id BIGINT NOT NULL,
   type_name VARCHAR(255) NOT NULL,
   selected INTEGER NOT NULL DEFAULT 0,
   service_name VARCHAR(255),
   version_tag VARCHAR(255) NOT NULL,
   user_name VARCHAR(255) NOT NULL DEFAULT '_db',
-  PRIMARY KEY (create_timestamp, host_name, cluster_id, type_name));
-  -- PRIMARY KEY (create_timestamp, host_id, cluster_id, type_name));
+  PRIMARY KEY (create_timestamp, host_id, cluster_id, type_name));
 
 CREATE TABLE metainfo (
   `metainfo_key` VARCHAR(255),
@@ -362,10 +358,8 @@ CREATE TABLE configgroup (
 
 CREATE TABLE configgrouphostmapping (
   config_group_id BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
-  -- host_id BIGINT NOT NULL,
-  PRIMARY KEY(config_group_id, host_name));
-  -- PRIMARY KEY(config_group_id, host_id));
+  host_id BIGINT NOT NULL,
+  PRIMARY KEY(config_group_id, host_id));
 
 CREATE TABLE requestschedule (
   schedule_id bigint,
@@ -633,17 +627,16 @@ ALTER TABLE request ADD CONSTRAINT FK_request_schedule_id FOREIGN KEY (request_s
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clhostmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clusterhostmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
-ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_name FOREIGN KEY (host_name) REFERENCES hosts (host_name);
--- ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_config FOREIGN KEY (config_id) REFERENCES clusterconfig(config_id);
-ALTER TABLE serviceconfighosts ADD CONSTRAINT  FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_host_id FOREIGN KEY (host_id) REFERENCES hosts(host_id);
 ALTER TABLE configgroup ADD CONSTRAINT FK_configgroup_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_confg FOREIGN KEY (cluster_id, config_type, version_tag) REFERENCES clusterconfig (cluster_id, type_name, version_tag);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_cgccm_gid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
 ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_cgid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
-ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_hname FOREIGN KEY (host_name) REFERENCES hosts (host_name);
--- ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_schedule_id FOREIGN KEY (schedule_id) REFERENCES requestschedule (schedule_id);
 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);

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/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 25685e5..bc6bd32 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
@@ -69,8 +69,8 @@ CREATE TABLE serviceconfig (
 
 CREATE TABLE serviceconfighosts (
   service_config_id NUMBER(19) NOT NULL,
-  hostname VARCHAR(255) NOT NULL,
-  PRIMARY KEY(service_config_id, hostname));
+  host_id NUMBER(19) NOT NULL,
+  PRIMARY KEY(service_config_id, host_id));
 
 CREATE TABLE serviceconfigmapping (
   service_config_id NUMBER(19) NOT NULL,
@@ -305,16 +305,14 @@ CREATE TABLE clusterconfigmapping (
 
 CREATE TABLE hostconfigmapping (
   create_timestamp NUMBER(19) NOT NULL,
-  host_name VARCHAR2(255) NOT NULL,
-  --host_id NUMBER(19) NOT NULL,
+  host_id NUMBER(19) NOT NULL,
   cluster_id NUMBER(19) NOT NULL,
   type_name VARCHAR2(255) NOT NULL,
   selected NUMBER(10) NOT NULL,
   service_name VARCHAR2(255) NULL,
   version_tag VARCHAR2(255) NOT NULL,
   user_name VARCHAR(255) DEFAULT '_db',
-  PRIMARY KEY (create_timestamp, host_name, cluster_id, type_name));
-  --PRIMARY KEY (create_timestamp, host_id, cluster_id, type_name));
+  PRIMARY KEY (create_timestamp, host_id, cluster_id, type_name));
 
 CREATE TABLE metainfo (
   "metainfo_key" VARCHAR2(255) NOT NULL,
@@ -352,10 +350,8 @@ CREATE TABLE confgroupclusterconfigmapping (
 
 CREATE TABLE configgrouphostmapping (
   config_group_id NUMBER(19) NOT NULL,
-  host_name VARCHAR2(255) NOT NULL,
-  --host_id NUMBER(19) NOT NULL,
-  PRIMARY KEY(config_group_id, host_name));
-  --PRIMARY KEY(config_group_id, host_id));
+  host_id NUMBER(19) NOT NULL,
+  PRIMARY KEY(config_group_id, host_id));
 
 CREATE TABLE requestschedule (
   schedule_id NUMBER(19),
@@ -599,7 +595,8 @@ ALTER TABLE repo_version ADD CONSTRAINT UQ_repo_version_stack_version UNIQUE (st
 ALTER TABLE members ADD CONSTRAINT FK_members_group_id FOREIGN KEY (group_id) REFERENCES groups (group_id);
 ALTER TABLE members ADD CONSTRAINT FK_members_user_id FOREIGN KEY (user_id) REFERENCES users (user_id);
 ALTER TABLE clusterconfig ADD CONSTRAINT FK_clusterconfig_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
-ALTER TABLE serviceconfighosts ADD CONSTRAINT  FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_host_id FOREIGN KEY (host_id) REFERENCES hosts(host_id);
 ALTER TABLE clusterservices ADD CONSTRAINT FK_clusterservices_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE clusterconfigmapping ADD CONSTRAINT clusterconfigmappingcluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE clusterstate ADD CONSTRAINT FK_clusterstate_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
@@ -623,16 +620,14 @@ ALTER TABLE request ADD CONSTRAINT FK_request_schedule_id FOREIGN KEY (request_s
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clhostmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clusterhostmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
-ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_name FOREIGN KEY (host_name) REFERENCES hosts (host_name);
---ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_config FOREIGN KEY (config_id) REFERENCES clusterconfig(config_id);
 ALTER TABLE configgroup ADD CONSTRAINT FK_configgroup_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_confg FOREIGN KEY (version_tag, config_type, cluster_id) REFERENCES clusterconfig (version_tag, type_name, cluster_id);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_cgccm_gid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
 ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_cgid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
-ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_hname FOREIGN KEY (host_name) REFERENCES hosts (host_name);
---ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_schedule_id FOREIGN KEY (schedule_id) REFERENCES requestschedule (schedule_id);
 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);

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/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 9ade56f..f2a016f 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
@@ -78,8 +78,8 @@ CREATE TABLE serviceconfig (
 
 CREATE TABLE serviceconfighosts (
   service_config_id BIGINT NOT NULL,
-  hostname VARCHAR(255) NOT NULL,
-  PRIMARY KEY(service_config_id, hostname));
+  host_id BIGINT NOT NULL,
+  PRIMARY KEY(service_config_id, host_id));
 
 CREATE TABLE serviceconfigmapping (
   service_config_id BIGINT NOT NULL,
@@ -313,16 +313,14 @@ CREATE TABLE key_value_store (
 
 CREATE TABLE hostconfigmapping (
   cluster_id BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
-  --host_id BIGINT NOT NULL,
+  host_id BIGINT NOT NULL,
   type_name VARCHAR(255) NOT NULL,
   version_tag VARCHAR(255) NOT NULL,
   service_name VARCHAR(255),
   create_timestamp BIGINT NOT NULL,
   selected INTEGER NOT NULL DEFAULT 0,
   user_name VARCHAR(255) NOT NULL DEFAULT '_db',
-  PRIMARY KEY (cluster_id, host_name, type_name, create_timestamp));
-  --PRIMARY KEY (cluster_id, host_id, type_name, create_timestamp));
+  PRIMARY KEY (cluster_id, host_id, type_name, create_timestamp));
 
 CREATE TABLE metainfo (
   "metainfo_key" VARCHAR(255),
@@ -354,10 +352,8 @@ CREATE TABLE confgroupclusterconfigmapping (
 
 CREATE TABLE configgrouphostmapping (
   config_group_id BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
-  --host_id BIGINT NOT NULL,
-  PRIMARY KEY(config_group_id, host_name));
-  --PRIMARY KEY(config_group_id, host_id));
+  host_id BIGINT NOT NULL,
+  PRIMARY KEY(config_group_id, host_id));
 
 CREATE TABLE requestschedule (
   schedule_id bigint,
@@ -624,14 +620,12 @@ ALTER TABLE request ADD CONSTRAINT FK_request_schedule_id FOREIGN KEY (request_s
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clhostmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clusterhostmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
-ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_name FOREIGN KEY (host_name) REFERENCES hosts (host_name);
---ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE configgroup ADD CONSTRAINT FK_configgroup_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_confg FOREIGN KEY (version_tag, config_type, cluster_id) REFERENCES clusterconfig (version_tag, type_name, cluster_id);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_cgccm_gid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
 ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_cgid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
-ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_hname FOREIGN KEY (host_name) REFERENCES hosts (host_name);
---ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_schedule_id FOREIGN KEY (schedule_id) REFERENCES requestschedule (schedule_id);
 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);
@@ -657,7 +651,8 @@ ALTER TABLE users ADD CONSTRAINT FK_users_principal_id FOREIGN KEY (principal_id
 ALTER TABLE groups ADD CONSTRAINT FK_groups_principal_id FOREIGN KEY (principal_id) REFERENCES adminprincipal(principal_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_config FOREIGN KEY (config_id) REFERENCES clusterconfig(config_id);
-ALTER TABLE serviceconfighosts ADD CONSTRAINT  FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_host_id FOREIGN KEY (host_id) REFERENCES hosts(host_id);
 ALTER TABLE clusters ADD CONSTRAINT FK_clusters_resource_id FOREIGN KEY (resource_id) REFERENCES adminresource(resource_id);
 ALTER TABLE widget_layout_user_widget ADD CONSTRAINT FK_widget_layout_id FOREIGN KEY (widget_layout_id) REFERENCES widget_layout(id);
 ALTER TABLE widget_layout_user_widget ADD CONSTRAINT FK_widget_id FOREIGN KEY (widget_id) REFERENCES widget(id);

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
index feaeae9..38a241d 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
@@ -94,8 +94,8 @@ GRANT ALL PRIVILEGES ON TABLE ambari.serviceconfig TO :username;
 
 CREATE TABLE ambari.serviceconfighosts (
   service_config_id BIGINT NOT NULL,
-  hostname VARCHAR(255) NOT NULL,
-  PRIMARY KEY(service_config_id, hostname));
+  host_id BIGINT NOT NULL,
+  PRIMARY KEY(service_config_id, host_id));
 GRANT ALL PRIVILEGES ON TABLE ambari.serviceconfighosts TO :username;
 
 CREATE TABLE ambari.serviceconfigmapping (
@@ -353,16 +353,14 @@ GRANT ALL PRIVILEGES ON TABLE ambari.key_value_store TO :username;
 
 CREATE TABLE ambari.hostconfigmapping (
   cluster_id BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
-  --host_id BIGINT NOT NULL,
+  host_id BIGINT NOT NULL,
   type_name VARCHAR(255) NOT NULL,
   version_tag VARCHAR(255) NOT NULL,
   service_name VARCHAR(255),
   create_timestamp BIGINT NOT NULL,
   selected INTEGER NOT NULL DEFAULT 0,
   user_name VARCHAR(255) NOT NULL DEFAULT '_db',
-  PRIMARY KEY (cluster_id, host_name, type_name, create_timestamp));
-  --PRIMARY KEY (cluster_id, host_id, type_name, create_timestamp));
+  PRIMARY KEY (cluster_id, host_id, type_name, create_timestamp));
 GRANT ALL PRIVILEGES ON TABLE ambari.hostconfigmapping TO :username;
 
 CREATE TABLE ambari.metainfo (
@@ -399,10 +397,8 @@ GRANT ALL PRIVILEGES ON TABLE ambari.confgroupclusterconfigmapping TO :username;
 
 CREATE TABLE ambari.configgrouphostmapping (
   config_group_id BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
-  --host_id BIGINT NOT NULL,
-  PRIMARY KEY(config_group_id, host_name));
-  --PRIMARY KEY(config_group_id, host_id));
+  host_id BIGINT NOT NULL,
+  PRIMARY KEY(config_group_id, host_id));
 GRANT ALL PRIVILEGES ON TABLE ambari.configgrouphostmapping TO :username;
 
 CREATE TABLE ambari.requestschedule (
@@ -697,12 +693,10 @@ ALTER TABLE ambari.request ADD CONSTRAINT FK_request_schedule_id FOREIGN KEY (re
 ALTER TABLE ambari.ClusterHostMapping ADD CONSTRAINT FK_clhostmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES ambari.clusters (cluster_id);
 ALTER TABLE ambari.ClusterHostMapping ADD CONSTRAINT FK_clusterhostmapping_host_id FOREIGN KEY (host_id) REFERENCES ambari.hosts (host_id);
 ALTER TABLE ambari.hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES ambari.clusters (cluster_id);
-ALTER TABLE ambari.hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_name FOREIGN KEY (host_name) REFERENCES ambari.hosts (host_name);
---ALTER TABLE ambari.hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES ambari.hosts (host_id);
+ALTER TABLE ambari.hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES ambari.hosts (host_id);
 ALTER TABLE ambari.configgroup ADD CONSTRAINT FK_configgroup_cluster_id FOREIGN KEY (cluster_id) REFERENCES ambari.clusters (cluster_id);
 ALTER TABLE ambari.configgrouphostmapping ADD CONSTRAINT FK_cghm_cgid FOREIGN KEY (config_group_id) REFERENCES ambari.configgroup (group_id);
-ALTER TABLE ambari.configgrouphostmapping ADD CONSTRAINT FK_cghm_hname FOREIGN KEY (host_name) REFERENCES ambari.hosts (host_name);
---ALTER TABLE ambari.configgrouphostmapping ADD CONSTRAINT FK_cghm_hid FOREIGN KEY (host_id) REFERENCES ambari.hosts (id);
+ALTER TABLE ambari.configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES ambari.hosts (id);
 ALTER TABLE ambari.requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_schedule_id FOREIGN KEY (schedule_id) REFERENCES ambari.requestschedule (schedule_id);
 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);
@@ -720,7 +714,8 @@ ALTER TABLE ambari.confgroupclusterconfigmapping ADD CONSTRAINT FK_confg FOREIGN
 ALTER TABLE ambari.confgroupclusterconfigmapping ADD CONSTRAINT FK_cgccm_gid FOREIGN KEY (config_group_id) REFERENCES ambari.configgroup (group_id);
 ALTER TABLE ambari.serviceconfigmapping ADD CONSTRAINT FK_scvm_scv FOREIGN KEY (service_config_id) REFERENCES ambari.serviceconfig(service_config_id);
 ALTER TABLE ambari.serviceconfigmapping ADD CONSTRAINT FK_scvm_config FOREIGN KEY (config_id) REFERENCES ambari.clusterconfig(config_id);
-ALTER TABLE ambari.serviceconfighosts ADD CONSTRAINT  FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES ambari.serviceconfig(service_config_id);
+ALTER TABLE ambari.serviceconfighosts ADD CONSTRAINT FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES ambari.serviceconfig(service_config_id);
+ALTER TABLE ambari.serviceconfighosts ADD CONSTRAINT FK_scvhosts_host_id FOREIGN KEY (host_id) REFERENCES ambari.hosts(host_id);
 ALTER TABLE ambari.adminresource ADD CONSTRAINT FK_resource_resource_type_id FOREIGN KEY (resource_type_id) REFERENCES ambari.adminresourcetype(resource_type_id);
 ALTER TABLE ambari.adminprincipal ADD CONSTRAINT FK_principal_principal_type_id FOREIGN KEY (principal_type_id) REFERENCES ambari.adminprincipaltype(principal_type_id);
 ALTER TABLE ambari.adminpermission ADD CONSTRAINT FK_permission_resource_type_id FOREIGN KEY (resource_type_id) REFERENCES ambari.adminresourcetype(resource_type_id);

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
index 03f1ec8..30959d0 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
@@ -82,8 +82,8 @@ CREATE TABLE serviceconfig (
 
 CREATE TABLE serviceconfighosts (
   service_config_id BIGINT NOT NULL,
-  hostname VARCHAR(255) NOT NULL,
-  PRIMARY KEY CLUSTERED (service_config_id, hostname)
+  host_id BIGINT NOT NULL,
+  PRIMARY KEY CLUSTERED (service_config_id, host_id)
   );
 
 CREATE TABLE serviceconfigmapping (
@@ -345,7 +345,7 @@ CREATE TABLE key_value_store (
 
 CREATE TABLE hostconfigmapping (
   cluster_id BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
+  host_id BIGINT NOT NULL,
   type_name VARCHAR(255) NOT NULL,
   version_tag VARCHAR(255) NOT NULL,
   service_name VARCHAR(255),
@@ -354,7 +354,7 @@ CREATE TABLE hostconfigmapping (
   user_name VARCHAR(255) NOT NULL DEFAULT '_db',
   PRIMARY KEY CLUSTERED (
     cluster_id,
-    host_name,
+    host_id,
     type_name,
     create_timestamp
     )
@@ -398,10 +398,10 @@ CREATE TABLE confgroupclusterconfigmapping (
 
 CREATE TABLE configgrouphostmapping (
   config_group_id BIGINT NOT NULL,
-  host_name VARCHAR(255) NOT NULL,
+  host_id BIGINT NOT NULL,
   PRIMARY KEY CLUSTERED (
     config_group_id,
-    host_name
+    host_id
     )
   );
 
@@ -737,14 +737,12 @@ ALTER TABLE request ADD CONSTRAINT FK_request_schedule_id FOREIGN KEY (request_s
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clhostmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE ClusterHostMapping ADD CONSTRAINT FK_clusterhostmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
-ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_name FOREIGN KEY (host_name) REFERENCES hosts (host_name);
---ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE hostconfigmapping ADD CONSTRAINT FK_hostconfmapping_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE configgroup ADD CONSTRAINT FK_configgroup_cluster_id FOREIGN KEY (cluster_id) REFERENCES clusters (cluster_id);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_confg FOREIGN KEY (cluster_id, config_type, version_tag) REFERENCES clusterconfig (cluster_id, type_name, version_tag);
 ALTER TABLE confgroupclusterconfigmapping ADD CONSTRAINT FK_cgccm_gid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
 ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_cgid FOREIGN KEY (config_group_id) REFERENCES configgroup (group_id);
-ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_hname FOREIGN KEY (host_name) REFERENCES hosts (host_name);
---ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
+ALTER TABLE configgrouphostmapping ADD CONSTRAINT FK_cghm_host_id FOREIGN KEY (host_id) REFERENCES hosts (host_id);
 ALTER TABLE requestschedulebatchrequest ADD CONSTRAINT FK_rsbatchrequest_schedule_id FOREIGN KEY (schedule_id) REFERENCES requestschedule (schedule_id);
 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);
@@ -770,7 +768,8 @@ ALTER TABLE users ADD CONSTRAINT FK_users_principal_id FOREIGN KEY (principal_id
 ALTER TABLE groups ADD CONSTRAINT FK_groups_principal_id FOREIGN KEY (principal_id) REFERENCES adminprincipal(principal_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
 ALTER TABLE serviceconfigmapping ADD CONSTRAINT FK_scvm_config FOREIGN KEY (config_id) REFERENCES clusterconfig(config_id);
-ALTER TABLE serviceconfighosts ADD CONSTRAINT  FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_scv FOREIGN KEY (service_config_id) REFERENCES serviceconfig(service_config_id);
+ALTER TABLE serviceconfighosts ADD CONSTRAINT FK_scvhosts_host_id FOREIGN KEY (host_id) REFERENCES hosts(host_id);
 ALTER TABLE clusters ADD CONSTRAINT FK_clusters_resource_id FOREIGN KEY (resource_id) REFERENCES adminresource(resource_id);
 ALTER TABLE widget_layout_user_widget ADD CONSTRAINT FK_widget_layout_id FOREIGN KEY (widget_layout_id) REFERENCES widget_layout(id);
 ALTER TABLE widget_layout_user_widget ADD CONSTRAINT FK_widget_id FOREIGN KEY (widget_id) REFERENCES widget(id);

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
index 06f9e8a..cba560a 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerTest.java
@@ -96,6 +96,7 @@ import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.orm.dao.WidgetDAO;
 import org.apache.ambari.server.orm.dao.WidgetLayoutDAO;
 import org.apache.ambari.server.orm.entities.ExecutionCommandEntity;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.WidgetEntity;
 import org.apache.ambari.server.orm.entities.WidgetLayoutEntity;
 import org.apache.ambari.server.orm.entities.WidgetLayoutUserWidgetEntity;
@@ -199,6 +200,7 @@ public class AmbariManagementControllerTest {
   private ConfigGroupFactory configGroupFactory;
   private OrmTestHelper helper;
   private StageFactory stageFactory;
+  private HostDAO hostDAO;
 
   @Rule
   public ExpectedException expectedException = ExpectedException.none();
@@ -226,6 +228,7 @@ public class AmbariManagementControllerTest {
     configGroupFactory = injector.getInstance(ConfigGroupFactory.class);
     helper = injector.getInstance(OrmTestHelper.class);
     stageFactory = injector.getInstance(StageFactory.class);
+    hostDAO = injector.getInstance(HostDAO.class);
   }
 
   @After
@@ -332,12 +335,13 @@ public class AmbariManagementControllerTest {
                               List<String> hosts, List<Config> configs)
                               throws AmbariException {
 
-    Map<String, Host> hostMap = new HashMap<String, Host>();
+    Map<Long, Host> hostMap = new HashMap<Long, Host>();
     Map<String, Config> configMap = new HashMap<String, Config>();
 
     for (String hostname : hosts) {
       Host host = clusters.getHost(hostname);
-      hostMap.put(host.getHostName(), host);
+      HostEntity hostEntity = hostDAO.findByName(hostname);
+      hostMap.put(hostEntity.getHostId(), host);
     }
 
     for (Config config : configs) {
@@ -6701,7 +6705,7 @@ public class AmbariManagementControllerTest {
 
     // Associate the right host
     ConfigGroup configGroup = cluster.getConfigGroups().get(groupId);
-    configGroup.setHosts(new HashMap<String, Host>() {{ put("h3",
+    configGroup.setHosts(new HashMap<Long, Host>() {{ put(3L,
       clusters.getHost("h3")); }});
     configGroup.persist();
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProviderTest.java
index db324e5..5e0debf 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ConfigGroupResourceProviderTest.java
@@ -17,6 +17,11 @@
  */
 package org.apache.ambari.server.controller.internal;
 
+import com.google.inject.Binder;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import com.google.inject.util.Modules;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.ConfigGroupRequest;
 import org.apache.ambari.server.controller.ConfigGroupResponse;
@@ -29,6 +34,9 @@ import org.apache.ambari.server.controller.spi.ResourceAlreadyExistsException;
 import org.apache.ambari.server.controller.spi.ResourceProvider;
 import org.apache.ambari.server.controller.utilities.PredicateBuilder;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
+import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
+import org.apache.ambari.server.orm.dao.HostDAO;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
 import org.apache.ambari.server.state.Config;
@@ -39,13 +47,19 @@ import org.apache.ambari.server.state.configgroup.ConfigGroupFactory;
 import org.easymock.Capture;
 import org.easymock.IAnswer;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
+
+
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
+
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertNotNull;
 import static junit.framework.Assert.assertTrue;
@@ -57,19 +71,40 @@ import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
+import static org.easymock.EasyMock.createStrictMock;
 
 public class ConfigGroupResourceProviderTest {
 
-  ConfigGroupResourceProvider getConfigGroupResourceProvider
-    (AmbariManagementController managementController) {
+  private Injector injector;
+
+  private HostDAO hostDAO = null;
 
+  @Before
+  public void setup() throws Exception {
+    hostDAO = createStrictMock(HostDAO.class);
+
+    // Create injector after all mocks have been initialized
+    injector = Guice.createInjector(Modules.override(
+        new InMemoryDefaultTestModule()).with(new MockModule()));
+  }
+
+  ConfigGroupResourceProvider getConfigGroupResourceProvider
+      (AmbariManagementController managementController) {
     Resource.Type type = Resource.Type.ConfigGroup;
 
     return (ConfigGroupResourceProvider) AbstractControllerResourceProvider.getResourceProvider(
-      type,
-      PropertyHelper.getPropertyIds(type),
-      PropertyHelper.getKeyPropertyIds(type),
-      managementController);
+        type,
+        PropertyHelper.getPropertyIds(type),
+        PropertyHelper.getKeyPropertyIds(type),
+        managementController);
+  }
+
+
+  private class MockModule implements Module {
+    @Override
+    public void configure(Binder binder) {
+      binder.bind(HostDAO.class).toInstance(hostDAO);
+    }
   }
 
   @Test
@@ -80,6 +115,8 @@ public class ConfigGroupResourceProviderTest {
     Cluster cluster = createNiceMock(Cluster.class);
     Host h1 = createNiceMock(Host.class);
     Host h2 = createNiceMock(Host.class);
+    HostEntity hostEntity1 = createMock(HostEntity.class);
+    HostEntity hostEntity2 = createMock(HostEntity.class);
     ConfigGroupFactory configGroupFactory = createNiceMock(ConfigGroupFactory.class);
     ConfigGroup configGroup = createNiceMock(ConfigGroup.class);
 
@@ -89,6 +126,10 @@ public class ConfigGroupResourceProviderTest {
     expect(clusters.getHost("h2")).andReturn(h2);
     expect(managementController.getConfigGroupFactory()).andReturn(configGroupFactory);
     expect(managementController.getAuthName()).andReturn("admin").anyTimes();
+    expect(hostDAO.findByName("h1")).andReturn(hostEntity1).atLeastOnce();
+    expect(hostDAO.findByName("h2")).andReturn(hostEntity2).atLeastOnce();
+    expect(hostEntity1.getHostId()).andReturn(1L).atLeastOnce();
+    expect(hostEntity2.getHostId()).andReturn(2L).atLeastOnce();
 
     Capture<Cluster> clusterCapture = new Capture<Cluster>();
     Capture<String> captureName = new Capture<String>();
@@ -96,14 +137,14 @@ public class ConfigGroupResourceProviderTest {
     Capture<String> captureTag = new Capture<String>();
     Capture<Map<String, Config>> captureConfigs = new Capture<Map<String,
       Config>>();
-    Capture<Map<String, Host>> captureHosts = new Capture<Map<String, Host>>();
+    Capture<Map<Long, Host>> captureHosts = new Capture<Map<Long, Host>>();
 
     expect(configGroupFactory.createNew(capture(clusterCapture),
       capture(captureName), capture(captureTag), capture(captureDesc),
       capture(captureConfigs), capture(captureHosts))).andReturn(configGroup);
 
     replay(managementController, clusters, cluster, configGroupFactory,
-      configGroup, response);
+      configGroup, response, hostDAO, hostEntity1, hostEntity2);
 
     ResourceProvider provider = getConfigGroupResourceProvider
       (managementController);
@@ -146,12 +187,12 @@ public class ConfigGroupResourceProviderTest {
     provider.createResources(request);
 
     verify(managementController, clusters, cluster, configGroupFactory,
-      configGroup, response);
+      configGroup, response, hostDAO, hostEntity1, hostEntity2);
 
     assertEquals("version100", captureConfigs.getValue().get("core-site")
       .getTag());
-    assertTrue(captureHosts.getValue().containsKey("h1"));
-    assertTrue(captureHosts.getValue().containsKey("h2"));
+    assertTrue(captureHosts.getValue().containsKey(1L));
+    assertTrue(captureHosts.getValue().containsKey(2L));
   }
 
   @Test
@@ -222,6 +263,9 @@ public class ConfigGroupResourceProviderTest {
     Cluster cluster = createNiceMock(Cluster.class);
     Host h1 = createNiceMock(Host.class);
     Host h2 = createNiceMock(Host.class);
+    HostEntity hostEntity1 = createMock(HostEntity.class);
+    HostEntity hostEntity2 = createMock(HostEntity.class);
+
     final ConfigGroup configGroup = createNiceMock(ConfigGroup.class);
     ConfigGroupResponse configGroupResponse = createNiceMock
       (ConfigGroupResponse.class);
@@ -231,6 +275,10 @@ public class ConfigGroupResourceProviderTest {
     expect(clusters.getCluster("Cluster100")).andReturn(cluster).anyTimes();
     expect(clusters.getHost("h1")).andReturn(h1);
     expect(clusters.getHost("h2")).andReturn(h2);
+    expect(hostDAO.findByName("h1")).andReturn(hostEntity1).atLeastOnce();
+    expect(hostDAO.findByName("h2")).andReturn(hostEntity2).atLeastOnce();
+    expect(hostEntity1.getHostId()).andReturn(1L).atLeastOnce();
+    expect(hostEntity2.getHostId()).andReturn(2L).atLeastOnce();
 
     expect(configGroup.getName()).andReturn("test-1").anyTimes();
     expect(configGroup.getId()).andReturn(25L).anyTimes();
@@ -253,7 +301,7 @@ public class ConfigGroupResourceProviderTest {
     expectLastCall().once();
 
     replay(managementController, clusters, cluster,
-      configGroup, response, configGroupResponse, configHelper);
+      configGroup, response, configGroupResponse, configHelper, hostDAO, hostEntity1, hostEntity2);
 
     ResourceProvider provider = getConfigGroupResourceProvider
       (managementController);
@@ -302,7 +350,7 @@ public class ConfigGroupResourceProviderTest {
     provider.updateResources(request, predicate);
 
     verify(managementController, clusters, cluster,
-      configGroup, response, configGroupResponse, configHelper);
+      configGroup, response, configGroupResponse, configHelper, hostDAO, hostEntity1, hostEntity2);
   }
 
   @SuppressWarnings("unchecked")
@@ -312,6 +360,14 @@ public class ConfigGroupResourceProviderTest {
     Clusters clusters = createNiceMock(Clusters.class);
     Cluster cluster = createNiceMock(Cluster.class);
     Host h1 = createNiceMock(Host.class);
+    final Long host1Id = 1L;
+    List<Long> hostIds = new ArrayList<Long>() {{ add(host1Id); }};
+    List<String> hostNames = new ArrayList<String>() {{ add("h1"); }};
+    HostEntity hostEntity1 = createMock(HostEntity.class);
+
+    expect(hostDAO.getHostNamesByHostIds(hostIds)).andReturn(hostNames).atLeastOnce();
+    expect(hostDAO.findByName("h1")).andReturn(hostEntity1).anyTimes();
+    expect(hostEntity1.getHostId()).andReturn(host1Id).anyTimes();
 
     ConfigGroup configGroup1 = createNiceMock(ConfigGroup.class);
     ConfigGroup configGroup2 = createNiceMock(ConfigGroup.class);
@@ -350,8 +406,8 @@ public class ConfigGroupResourceProviderTest {
     expect(configGroup3.getTag()).andReturn("t3").anyTimes();
     expect(configGroup4.getTag()).andReturn("t4").anyTimes();
 
-    Map<String, Host> hostMap = new HashMap<String, Host>();
-    hostMap.put("h1", h1);
+    Map<Long, Host> hostMap = new HashMap<Long, Host>();
+    hostMap.put(host1Id, h1);
     expect(configGroup4.getHosts()).andReturn(hostMap).anyTimes();
 
 
@@ -373,9 +429,8 @@ public class ConfigGroupResourceProviderTest {
     hostObj.add(hostnames);
     expect(response4.getHosts()).andReturn(hostObj).anyTimes();
 
-    replay(managementController, clusters, cluster, configGroup1,
-      configGroup2, configGroup3, configGroup4, response1, response2,
-      response3, response4);
+    replay(managementController, clusters, cluster, hostDAO, hostEntity1,
+        configGroup1, configGroup2, configGroup3, configGroup4, response1, response2, response3, response4);
 
     ResourceProvider resourceProvider = getConfigGroupResourceProvider
       (managementController);
@@ -470,7 +525,7 @@ public class ConfigGroupResourceProviderTest {
       .CONFIGGROUP_CLUSTER_NAME_PROPERTY_ID).equals("Cluster100").and()
       .property(ConfigGroupResourceProvider.CONFIGGROUP_TAG_PROPERTY_ID)
       .equals("t4").and().property(ConfigGroupResourceProvider
-        .CONFIGGROUP_HOSTS_PROPERTY_ID).equals("h1").toPredicate();
+        .CONFIGGROUP_HOSTS_PROPERTY_ID).equals(host1Id).toPredicate();
 
     resources = resourceProvider.getResources(request, predicate);
 
@@ -513,9 +568,8 @@ public class ConfigGroupResourceProviderTest {
     }
     Assert.assertNotNull(resourceException);
 
-    verify(managementController, clusters, cluster, configGroup1,
-      configGroup2, configGroup3, configGroup4, response1, response2,
-      response3, response4);
+    verify(managementController, clusters, cluster, hostDAO, hostEntity1,
+        configGroup1, configGroup2, configGroup3, configGroup4, response1, response2, response3, response4);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/ConfigGroupDAOTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/ConfigGroupDAOTest.java b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/ConfigGroupDAOTest.java
index 2adbf9d..53af8a5 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/ConfigGroupDAOTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/ConfigGroupDAOTest.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
+import com.google.inject.assistedinject.AssistedInject;
 import junit.framework.Assert;
 
 import org.apache.ambari.server.AmbariException;
@@ -38,6 +39,7 @@ import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.ResourceEntity;
 import org.apache.ambari.server.orm.entities.ResourceTypeEntity;
 import org.apache.ambari.server.orm.entities.StackEntity;
+import org.apache.ambari.server.state.host.HostFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -55,6 +57,7 @@ public class ConfigGroupDAOTest {
   private HostDAO hostDAO;
   private ResourceTypeDAO resourceTypeDAO;
   private StackDAO stackDAO;
+  private HostFactory hostFactory;
 
   @Before
   public void setup() throws Exception {
@@ -73,6 +76,7 @@ public class ConfigGroupDAOTest {
       (ConfigGroupHostMappingDAO.class);
     hostDAO = injector.getInstance(HostDAO.class);
     resourceTypeDAO = injector.getInstance(ResourceTypeDAO.class);
+    hostFactory = injector.getInstance(HostFactory.class);
   }
 
   @After
@@ -115,16 +119,14 @@ public class ConfigGroupDAOTest {
     configGroupDAO.create(configGroupEntity);
 
     if (hosts != null && !hosts.isEmpty()) {
-      List<ConfigGroupHostMappingEntity> hostMappingEntities = new
-        ArrayList<ConfigGroupHostMappingEntity>();
+      List<ConfigGroupHostMappingEntity> hostMappingEntities = new ArrayList<ConfigGroupHostMappingEntity>();
 
       for (HostEntity host : hosts) {
         host.setClusterEntities(Arrays.asList(clusterEntity));
         hostDAO.create(host);
 
-        ConfigGroupHostMappingEntity hostMappingEntity = new
-          ConfigGroupHostMappingEntity();
-        hostMappingEntity.setHostname(host.getHostName());
+        ConfigGroupHostMappingEntity hostMappingEntity = new ConfigGroupHostMappingEntity();
+        hostMappingEntity.setHostId(host.getHostId());
         hostMappingEntity.setHostEntity(host);
         hostMappingEntity.setConfigGroupEntity(configGroupEntity);
         hostMappingEntity.setConfigGroupId(configGroupEntity.getGroupId());
@@ -212,27 +214,29 @@ public class ConfigGroupDAOTest {
   @Test
   public void testFindByHost() throws Exception {
     List<HostEntity> hosts = new ArrayList<HostEntity>();
+    // Partially constructed HostEntity that will persisted in {@link createConfigGroup}
     HostEntity hostEntity = new HostEntity();
     hostEntity.setHostName("h1");
     hostEntity.setOsType("centOS");
+
     hosts.add(hostEntity);
     ConfigGroupEntity configGroupEntity =
       createConfigGroup("c1", "hdfs-1", "HDFS", "some description", hosts, null);
 
+    Assert.assertNotNull(hostEntity.getHostId());
+
     Assert.assertNotNull(configGroupEntity);
     Assert.assertTrue(configGroupEntity.getConfigGroupHostMappingEntities()
       .size() > 0);
     Assert.assertNotNull(configGroupEntity
       .getConfigGroupHostMappingEntities().iterator().next());
 
-    Set<ConfigGroupHostMapping> hostMappingEntities = configGroupHostMappingDAO
-      .findByHost("h1");
+    Set<ConfigGroupHostMapping> hostMappingEntities = configGroupHostMappingDAO.findByHostId(hostEntity.getHostId());
 
     Assert.assertNotNull(hostMappingEntities);
 
     for (ConfigGroupHostMapping hostMappingEntity : hostMappingEntities) {
-
-      Assert.assertEquals("h1", hostMappingEntity.getHostname());
+      Assert.assertEquals(hostEntity.getHostId(), hostMappingEntity.getHostId());
       Assert.assertEquals("centOS", hostMappingEntity.getHost().getOsType());
     }
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAOTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAOTest.java b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAOTest.java
index ec1289a..0dcc471 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAOTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/HostConfigMappingDAOTest.java
@@ -19,6 +19,7 @@ package org.apache.ambari.server.orm.dao;
 
 import java.util.Set;
 
+import com.google.inject.Inject;
 import junit.framework.Assert;
 
 import org.apache.ambari.server.AmbariException;
@@ -26,6 +27,7 @@ import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
 import org.apache.ambari.server.orm.cache.HostConfigMapping;
 import org.apache.ambari.server.orm.cache.HostConfigMappingImpl;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -40,7 +42,12 @@ import com.google.inject.persist.PersistService;
 public class HostConfigMappingDAOTest {
 
   private Injector injector;
+
+  @Inject
   private HostConfigMappingDAO hostConfigMappingDAO;
+
+  @Inject
+  private HostDAO hostDAO;
   
   @Before
    public void setup() throws AmbariException{
@@ -48,6 +55,7 @@ public class HostConfigMappingDAOTest {
     injector.getInstance(GuiceJpaInitializer.class);
     
     hostConfigMappingDAO = injector.getInstance(HostConfigMappingDAO.class);
+    hostDAO = injector.getInstance(HostDAO.class);
   }
 
   @After
@@ -55,19 +63,27 @@ public class HostConfigMappingDAOTest {
     injector.getInstance(PersistService.class).stop();
   }
   
-  private HostConfigMapping createEntity(long clusterId, String host, String type, String version) throws Exception {
-    HostConfigMapping entity = new HostConfigMappingImpl();
-    entity.setClusterId(Long.valueOf(clusterId));
-    entity.setCreateTimestamp(Long.valueOf(System.currentTimeMillis()));
-    entity.setHostName(host);
-    entity.setSelected(1);
-    entity.setType(type);
-    entity.setVersion(version);
-    entity.setUser("_test");
-    
-    hostConfigMappingDAO.create(entity);
-    
-    return entity;
+  private HostConfigMapping createEntity(long clusterId, String hostName, String type, String version) throws Exception {
+    HostConfigMapping hostConfigMappingEntity = new HostConfigMappingImpl();
+    hostConfigMappingEntity.setClusterId(Long.valueOf(clusterId));
+    hostConfigMappingEntity.setCreateTimestamp(Long.valueOf(System.currentTimeMillis()));
+
+    HostEntity hostEntity = hostDAO.findByName(hostName);
+    if (hostEntity == null) {
+      hostEntity = new HostEntity();
+      hostEntity.setHostName(hostName);
+      hostDAO.create(hostEntity);
+    }
+
+    hostConfigMappingEntity.setHostId(hostEntity.getHostId());
+    hostConfigMappingEntity.setSelected(1);
+    hostConfigMappingEntity.setType(type);
+    hostConfigMappingEntity.setVersion(version);
+    hostConfigMappingEntity.setUser("_test");
+
+    hostConfigMappingDAO.create(hostConfigMappingEntity);
+    
+    return hostConfigMappingEntity;
   }
   
   @Test
@@ -79,8 +95,9 @@ public class HostConfigMappingDAOTest {
   @Test
   public void testFindByType() throws Exception {
     HostConfigMapping source = createEntity(1L, "h1", "global", "v1");
+    HostEntity hostEntity = hostDAO.findByName("h1");
     
-    Set<HostConfigMapping> target = hostConfigMappingDAO.findByType(1L, "h1", "global");
+    Set<HostConfigMapping> target = hostConfigMappingDAO.findByType(1L, hostEntity.getHostId(), "global");
 
     Assert.assertEquals("Expected one result", 1, target.size());
     
@@ -91,66 +108,58 @@ public class HostConfigMappingDAOTest {
   @Test
   public void testMerge() throws Exception {
     HostConfigMapping source = createEntity(1L, "h1", "global", "v1");
-    
-    Set<HostConfigMapping> target = hostConfigMappingDAO.findByType(1L, "h1", "global");
+    HostEntity hostEntity = hostDAO.findByName("h1");
 
+    Set<HostConfigMapping> target = hostConfigMappingDAO.findByType(1L, hostEntity.getHostId(), "global");
     Assert.assertEquals("Expected one result", 1, target.size());
     
     HostConfigMapping toChange = null;
     
     for (HostConfigMapping item: target) {
-      
       Assert.assertEquals("Expected version 'v1'", source.getVersion(), item.getVersion());
       Assert.assertEquals("Expected selected flag 1", 1, (int)item.getSelected());
-      
       toChange = item;
-      
       toChange.setSelected(0);
-      
     }
     
-
-    
     hostConfigMappingDAO.merge(toChange);
     
-    target = hostConfigMappingDAO.findByType(1L, "h1", "global");
-
+    target = hostConfigMappingDAO.findByType(1L, hostEntity.getHostId(), "global");
     Assert.assertEquals("Expected one result", 1, target.size());
     
-    
     for (HostConfigMapping item: target) {
-      
       Assert.assertEquals("Expected version 'v1'", source.getVersion(), item.getVersion());
       Assert.assertEquals("Expected selected flag 0", 0, (int)item.getSelected());
-      
     }
   }
   
   @Test
   public void testFindSelected() throws Exception {
     createEntity(1L, "h1", "global", "version1");
-    HostConfigMapping entity2 = createEntity(1L, "h1", "core-site", "version1");
+    HostConfigMapping coreSiteConfigV1 = createEntity(1L, "h1", "core-site", "version1");
+    HostEntity hostEntity = hostDAO.findByName("h1");
     
-    Set<HostConfigMapping> targets = hostConfigMappingDAO.findSelected(1L, "h1");
+    Set<HostConfigMapping> targets = hostConfigMappingDAO.findSelected(1L, hostEntity.getHostId());
     Assert.assertEquals("Expected two entities", 2, targets.size());
-    
-    entity2.setSelected(0);
-    hostConfigMappingDAO.merge(entity2);
+
+    coreSiteConfigV1.setSelected(0);
+    hostConfigMappingDAO.merge(coreSiteConfigV1);
     
     createEntity(1L, "h1", "core-site", "version2");
 
-    targets = hostConfigMappingDAO.findSelected(1L, "h1");
+    targets = hostConfigMappingDAO.findSelected(1L, hostEntity.getHostId());
     Assert.assertEquals("Expected two entities", 2, targets.size());
   }
   
   @Test
   public void testFindSelectedByType() throws Exception {
     HostConfigMapping entity1 = createEntity(1L, "h1", "global", "version1");
+    HostEntity hostEntity = hostDAO.findByName("h1");
     
-    HostConfigMapping target = hostConfigMappingDAO.findSelectedByType(1L, "h1", "core-site");
+    HostConfigMapping target = hostConfigMappingDAO.findSelectedByType(1L, hostEntity.getHostId(), "core-site");
     Assert.assertNull("Expected null entity for type 'core-site'", target);
     
-    target = hostConfigMappingDAO.findSelectedByType(1L, "h1", "global");
+    target = hostConfigMappingDAO.findSelectedByType(1L, hostEntity.getHostId(), "global");
     Assert.assertNotNull("Expected non-null entity for type 'global'", target);
     Assert.assertEquals("Expected version to be '" + entity1.getVersion() + "'", entity1.getVersion(), target.getVersion());
     
@@ -159,7 +168,7 @@ public class HostConfigMappingDAOTest {
     
     HostConfigMapping entity2 = createEntity(1L, "h1", "global", "version2");
     
-    target = hostConfigMappingDAO.findSelectedByType(1L, "h1", "global");
+    target = hostConfigMappingDAO.findSelectedByType(1L, hostEntity.getHostId(), "global");
     Assert.assertNotNull("Expected non-null entity for type 'global'", target);
     
     Assert.assertEquals("Expected version to be '" + entity2.getVersion() + "'", entity2.getVersion(), target.getVersion());
@@ -169,9 +178,11 @@ public class HostConfigMappingDAOTest {
   
   @Test
   public void testEmptyTable() throws Exception {
-    
+    createEntity(1L, "h1", "global", "version1");
+
+    HostEntity hostEntity = hostDAO.findByName("h1");
     hostConfigMappingDAO.removeHost(1L, "h1");
-    HostConfigMapping target = hostConfigMappingDAO.findSelectedByType(1L, "h1", "core-site");
+    HostConfigMapping target = hostConfigMappingDAO.findSelectedByType(1L, hostEntity.getHostId(), "core-site");
     
     Assert.assertEquals(null, target);
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigGroupTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigGroupTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigGroupTest.java
index 28059c0..9bf969c 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigGroupTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigGroupTest.java
@@ -103,10 +103,10 @@ public class ConfigGroupTest {
     Host host = clusters.getHost("h1");
 
     Map<String, Config> configs = new HashMap<String, Config>();
-    Map<String, Host> hosts = new HashMap<String, Host>();
+    Map<Long, Host> hosts = new HashMap<Long, Host>();
 
     configs.put(config.getType(), config);
-    hosts.put(host.getHostName(), host);
+    hosts.put(1L, host);
 
     ConfigGroup configGroup = configGroupFactory.createNew(cluster, "cg-test",
       "HDFS", "New HDFS configs for h1", configs, hosts);
@@ -225,8 +225,9 @@ public class ConfigGroupTest {
     clusters.unmapHostFromCluster("h1", clusterName);
 
     Assert.assertNull(clusters.getHostsForCluster(clusterName).get("h1"));
-    Assert.assertTrue(configGroupHostMappingDAO.findByHost("h1").isEmpty());
-    Assert.assertNull(configGroup.getHosts().get("h1"));
+    // Assumes that 1L is the id of host h1, as specified in createConfigGroup
+    Assert.assertTrue(configGroupHostMappingDAO.findByHostId(1L).isEmpty());
+    Assert.assertFalse(configGroup.getHosts().containsKey(1L));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigHelperTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigHelperTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigHelperTest.java
index 930e45f..7fb8f66 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigHelperTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/ConfigHelperTest.java
@@ -179,12 +179,14 @@ public class ConfigHelperTest {
     private Long addConfigGroup(String name, String tag, List<String> hosts,
                                 List<Config> configs) throws AmbariException {
 
-      Map<String, Host> hostMap = new HashMap<String, Host>();
+      Map<Long, Host> hostMap = new HashMap<Long, Host>();
       Map<String, Config> configMap = new HashMap<String, Config>();
 
+      Long hostId = 1L;
       for (String hostname : hosts) {
         Host host = clusters.getHost(hostname);
-        hostMap.put(host.getHostName(), host);
+        hostMap.put(hostId, host);
+        hostId++;
       }
 
       for (Config config : configs) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
index 9c129e8..319c2ee 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeHelperTest.java
@@ -44,6 +44,7 @@ import org.apache.ambari.server.controller.ConfigurationRequest;
 import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
 import org.apache.ambari.server.orm.OrmTestHelper;
+import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.stack.HostsType;
 import org.apache.ambari.server.stack.MasterHostResolver;
 import org.apache.ambari.server.state.UpgradeHelper.UpgradeGroupHolder;
@@ -80,7 +81,8 @@ public class UpgradeHelperTest {
   private MasterHostResolver m_masterHostResolver;
   private UpgradeHelper m_upgradeHelper;
   private ConfigHelper m_configHelper;
-  AmbariManagementController m_managementController;
+  private AmbariManagementController m_managementController;
+  private HostDAO m_hostDAO;
 
   @Before
   public void before() throws Exception {
@@ -106,6 +108,7 @@ public class UpgradeHelperTest {
     m_upgradeHelper = injector.getInstance(UpgradeHelper.class);
     m_masterHostResolver = EasyMock.createMock(MasterHostResolver.class);
     m_managementController = injector.getInstance(AmbariManagementController.class);
+    m_hostDAO = injector.getInstance(HostDAO.class);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/ambari/blob/290276c6/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java
index bc17c38..63c5440 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java
@@ -42,6 +42,7 @@ import java.util.Set;
 import javax.persistence.EntityManager;
 import javax.persistence.RollbackException;
 
+import com.google.gson.Gson;
 import junit.framework.Assert;
 
 import org.apache.ambari.server.AmbariException;
@@ -55,11 +56,14 @@ import org.apache.ambari.server.controller.ServiceConfigVersionResponse;
 import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
 import org.apache.ambari.server.orm.OrmTestHelper;
+import org.apache.ambari.server.orm.dao.ClusterDAO;
 import org.apache.ambari.server.orm.dao.ClusterVersionDAO;
 import org.apache.ambari.server.orm.dao.HostComponentStateDAO;
 import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.orm.dao.HostVersionDAO;
 import org.apache.ambari.server.orm.dao.RepositoryVersionDAO;
+import org.apache.ambari.server.orm.dao.ResourceTypeDAO;
+import org.apache.ambari.server.orm.dao.StackDAO;
 import org.apache.ambari.server.orm.entities.ClusterEntity;
 import org.apache.ambari.server.orm.entities.ClusterServiceEntity;
 import org.apache.ambari.server.orm.entities.ClusterStateEntity;
@@ -69,6 +73,8 @@ import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.HostStateEntity;
 import org.apache.ambari.server.orm.entities.HostVersionEntity;
 import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
+import org.apache.ambari.server.orm.entities.ResourceEntity;
+import org.apache.ambari.server.orm.entities.ResourceTypeEntity;
 import org.apache.ambari.server.orm.entities.ServiceDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.state.AgentVersion;
@@ -121,11 +127,15 @@ public class ClusterTest {
   private ConfigFactory configFactory;
   private ConfigGroupFactory configGroupFactory;
   private OrmTestHelper helper;
+  private StackDAO stackDAO;
+  private ResourceTypeDAO resourceTypeDAO;
+  private ClusterDAO clusterDAO;
   private HostDAO hostDAO;
   private ClusterVersionDAO clusterVersionDAO;
   private HostVersionDAO hostVersionDAO;
   private HostComponentStateDAO hostComponentStateDAO;
   private RepositoryVersionDAO repositoryVersionDAO;
+  private Gson gson;
 
   @Singleton
   static class ClusterVersionDAOMock extends ClusterVersionDAO {
@@ -174,11 +184,15 @@ public class ClusterTest {
     configFactory = injector.getInstance(ConfigFactory.class);
     metaInfo = injector.getInstance(AmbariMetaInfo.class);
     helper = injector.getInstance(OrmTestHelper.class);
+    stackDAO = injector.getInstance(StackDAO.class);
+    resourceTypeDAO = injector.getInstance(ResourceTypeDAO.class);
+    clusterDAO = injector.getInstance(ClusterDAO.class);
     hostDAO = injector.getInstance(HostDAO.class);
     clusterVersionDAO = injector.getInstance(ClusterVersionDAO.class);
     hostVersionDAO = injector.getInstance(HostVersionDAO.class);
     hostComponentStateDAO = injector.getInstance(HostComponentStateDAO.class);
     repositoryVersionDAO = injector.getInstance(RepositoryVersionDAO.class);
+    gson = injector.getInstance(Gson.class);
   }
 
   @After
@@ -187,38 +201,56 @@ public class ClusterTest {
   }
 
   private void createDefaultCluster() throws Exception {
-    StackId stackId = new StackId("HDP-0.1");
-    clusters.addCluster("c1", stackId);
-    c1 = clusters.getCluster("c1");
-    Assert.assertEquals("c1", c1.getClusterName());
-    Assert.assertEquals(1, c1.getClusterId());
+    // TODO, use common function
+    StackId stackId = new StackId("HDP", "0.1");
+    StackEntity stackEntity = stackDAO.find(stackId.getStackName(), stackId.getStackVersion());
+    org.junit.Assert.assertNotNull(stackEntity);
 
-    clusters.addHost("h1");
-    clusters.addHost("h2");
-    Host host1 = clusters.getHost("h1");
-    host1.setIPv4("ipv4");
-    host1.setIPv6("ipv6");
+    String clusterName = "c1";
 
-    Host host2 = clusters.getHost("h2");
-    host2.setIPv4("ipv4");
-    host2.setIPv6("ipv6");
+    ResourceTypeEntity resourceTypeEntity = resourceTypeDAO.findById(ResourceTypeEntity.CLUSTER_RESOURCE_TYPE);
+    if (resourceTypeEntity == null) {
+      resourceTypeEntity = new ResourceTypeEntity();
+      resourceTypeEntity.setId(ResourceTypeEntity.CLUSTER_RESOURCE_TYPE);
+      resourceTypeEntity.setName(ResourceTypeEntity.CLUSTER_RESOURCE_TYPE_NAME);
+      resourceTypeEntity = resourceTypeDAO.merge(resourceTypeEntity);
+    }
+    ResourceEntity resourceEntity = new ResourceEntity();
+    resourceEntity.setResourceType(resourceTypeEntity);
+
+    ClusterEntity clusterEntity = new ClusterEntity();
+    clusterEntity.setClusterName(clusterName);
+    clusterEntity.setResource(resourceEntity);
+    clusterEntity.setDesiredStack(stackEntity);
+    clusterDAO.create(clusterEntity);
 
     Map<String, String> hostAttributes = new HashMap<String, String>();
     hostAttributes.put("os_family", "redhat");
     hostAttributes.put("os_release_version", "5.9");
-    host1.setHostAttributes(hostAttributes);
-    host2.setHostAttributes(hostAttributes);
 
-    host1.persist();
-    host2.persist();
+    List<HostEntity> hostEntities = new ArrayList<HostEntity>();
+    Set<String> hostNames = new HashSet<String>() {{ add("h1"); add("h2"); }};
+    for (String hostName : hostNames) {
+      HostEntity hostEntity = new HostEntity();
+      hostEntity.setHostName(hostName);
+      hostEntity.setIpv4("ipv4");
+      hostEntity.setIpv6("ipv6");
+      hostEntity.setHostAttributes(gson.toJson(hostAttributes));
+      hostEntity.setClusterEntities(Arrays.asList(clusterEntity));
+      hostEntities.add(hostEntity);
+      hostDAO.create(hostEntity);
+    }
+
+    clusterEntity.setHostEntities(hostEntities);
+    clusterDAO.merge(clusterEntity);
+    c1 = clusters.getCluster(clusterName);
 
     helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
     c1.createClusterVersion(stackId, stackId.getStackVersion(), "admin",
         RepositoryVersionState.UPGRADING);
     c1.transitionClusterVersion(stackId, stackId.getStackVersion(),
         RepositoryVersionState.CURRENT);
-    clusters.mapHostToCluster("h1", "c1");
-    clusters.mapHostToCluster("h2", "c1");
+
     ClusterVersionDAOMock.failOnCurrentVersionState = false;
   }
 
@@ -898,6 +930,7 @@ public class ClusterTest {
     createDefaultCluster();
 
     Host host1 = clusters.getHost("h1");
+    HostEntity hostEntity1 = hostDAO.findByName("h1");
 
     Map<String, Map<String, String>> propAttributes = new HashMap<String, Map<String,String>>();
     propAttributes.put("final", new HashMap<String, String>());
@@ -909,18 +942,18 @@ public class ClusterTest {
 
     host1.addDesiredConfig(c1.getClusterId(), true, "test", config);
 
-    Map<String, Map<String, DesiredConfig>> configs = c1.getAllHostsDesiredConfigs();
+    Map<Long, Map<String, DesiredConfig>> configs = c1.getAllHostsDesiredConfigs();
 
-    assertTrue(configs.containsKey("h1"));
-    assertEquals(1, configs.get("h1").size());
+    assertTrue(configs.containsKey(hostEntity1.getHostId()));
+    assertEquals(1, configs.get(hostEntity1.getHostId()).size());
 
-    List<String> hostnames = new ArrayList<String>();
-    hostnames.add("h1");
+    List<Long> hostIds = new ArrayList<Long>();
+    hostIds.add(hostEntity1.getHostId());
 
-    configs = c1.getHostsDesiredConfigs(hostnames);
+    configs = c1.getHostsDesiredConfigs(hostIds);
 
-    assertTrue(configs.containsKey("h1"));
-    assertEquals(1, configs.get("h1").size());
+    assertTrue(configs.containsKey(hostEntity1.getHostId()));
+    assertEquals(1, configs.get(hostEntity1.getHostId()).size());
   }
 
   @Test
@@ -1060,7 +1093,7 @@ public class ClusterTest {
 
     ConfigGroup configGroup =
       configGroupFactory.createNew(c1, "test group", "HDFS", "descr", Collections.singletonMap("hdfs-site", config2),
-        Collections.<String, Host>emptyMap());
+        Collections.<Long, Host>emptyMap());
 
     configGroup.persist();
 
@@ -1117,7 +1150,7 @@ public class ClusterTest {
 
     ConfigGroup configGroup2 =
         configGroupFactory.createNew(c1, "test group 2", "HDFS", "descr", Collections.singletonMap("hdfs-site", config4),
-            Collections.<String, Host>emptyMap());
+            Collections.<Long, Host>emptyMap());
 
     configGroup2.persist();
     c1.addConfigGroup(configGroup2);
@@ -1328,12 +1361,12 @@ public class ClusterTest {
     assertNotNull(entityHDP2);
 
     List<HostVersionEntity> hostVersionsH1Before = hostVersionDAO.findByClusterAndHost("c1", "h1");
-    assertEquals(0, hostVersionsH1Before.size());
+    assertEquals(1, hostVersionsH1Before.size());
 
     c1.inferHostVersions(entityHDP2);
 
     List<HostVersionEntity> hostVersionsH1After = hostVersionDAO.findByClusterAndHost("c1", "h1");
-    assertEquals(1, hostVersionsH1After.size());
+    assertEquals(2, hostVersionsH1After.size());
 
     boolean checked = false;
     for (HostVersionEntity entity : hostVersionsH1After) {
@@ -1351,7 +1384,7 @@ public class ClusterTest {
     c1.inferHostVersions(entityHDP2);
 
     hostVersionsH1After = hostVersionDAO.findByClusterAndHost("c1", "h1");
-    assertEquals(1, hostVersionsH1After.size());
+    assertEquals(2, hostVersionsH1After.size());
 
     checked = false;
     for (HostVersionEntity entity : hostVersionsH1After) {