You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by mp...@apache.org on 2017/01/03 18:24:14 UTC

[4/6] ambari git commit: AMBARI-19242. Ambari Server hangs generating tasks for install+start when adding 600 hosts at a time. (mpapirkovskyy)

AMBARI-19242. Ambari Server hangs generating tasks for install+start when adding 600 hosts at a time. (mpapirkovskyy)


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

Branch: refs/heads/branch-2.5
Commit: 1689df7f48113e1c6a9c0aac1148c0afde4c1408
Parents: 6607d11
Author: Myroslav Papirkovskyi <mp...@hortonworks.com>
Authored: Mon Dec 19 19:09:20 2016 +0200
Committer: Myroslav Papirkovskyi <mp...@hortonworks.com>
Committed: Tue Jan 3 20:23:20 2017 +0200

----------------------------------------------------------------------
 .../actionmanager/ActionDBAccessorImpl.java     |  4 +-
 .../AmbariManagementControllerImpl.java         |  4 +-
 .../internal/HostResourceProvider.java          | 20 +++--
 .../ambari/server/events/HostAddedEvent.java    | 51 ------------
 .../ambari/server/events/HostRemovedEvent.java  | 69 ---------------
 .../ambari/server/events/HostsAddedEvent.java   | 50 +++++++++++
 .../ambari/server/events/HostsRemovedEvent.java | 88 ++++++++++++++++++++
 .../listeners/alerts/AlertHostListener.java     | 32 +++----
 .../upgrade/HostVersionOutOfSyncListener.java   | 23 ++---
 .../apache/ambari/server/orm/dao/CrudDAO.java   | 14 ++++
 .../ambari/server/orm/dao/HostVersionDAO.java   |  4 +-
 .../apache/ambari/server/state/Clusters.java    | 11 ++-
 .../server/state/cluster/ClustersImpl.java      | 87 +++++++++----------
 .../ambari/server/state/host/HostImpl.java      |  2 +-
 .../ambari/server/topology/TopologyManager.java | 32 +++----
 .../server/agent/HeartbeatTestHelper.java       |  2 +-
 .../server/agent/TestHeartbeatMonitor.java      | 10 +--
 .../AmbariManagementControllerTest.java         |  8 +-
 .../internal/HostResourceProviderTest.java      |  5 +-
 .../HostVersionOutOfSyncListenerTest.java       |  4 +-
 .../apache/ambari/server/orm/OrmTestHelper.java |  5 +-
 .../upgrades/UpgradeActionTest.java             |  2 +-
 .../server/state/cluster/ClusterImplTest.java   |  8 +-
 .../server/state/cluster/ClusterTest.java       |  4 +-
 .../server/state/cluster/ClustersTest.java      |  6 +-
 .../svccomphost/ServiceComponentHostTest.java   |  2 +-
 26 files changed, 302 insertions(+), 245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java
index 2c87583..7837a7b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java
@@ -43,7 +43,7 @@ import org.apache.ambari.server.audit.event.OperationStatusAuditEvent;
 import org.apache.ambari.server.audit.event.TaskStatusAuditEvent;
 import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.controller.internal.CalculatedStatus;
-import org.apache.ambari.server.events.HostRemovedEvent;
+import org.apache.ambari.server.events.HostsRemovedEvent;
 import org.apache.ambari.server.events.RequestFinishedEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.orm.dao.ClusterDAO;
@@ -826,7 +826,7 @@ public class ActionDBAccessorImpl implements ActionDBAccessor {
    * @param event @HostRemovedEvent
    */
   @Subscribe
-  public void invalidateCommandCacheOnHostRemove(HostRemovedEvent event) {
+  public void invalidateCommandCacheOnHostRemove(HostsRemovedEvent event) {
     LOG.info("Invalidating HRC cache after receiveing {}", event);
     hostRoleCommandCache.invalidateAll();
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
index 22eabdd..c67c603 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
@@ -483,7 +483,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
     Cluster c = clusters.getCluster(request.getClusterName());
 
     if (request.getHostNames() != null) {
-      clusters.mapHostsToCluster(request.getHostNames(),
+      clusters.mapAndPublishHostsToCluster(request.getHostNames(),
           request.getClusterName());
     }
     // Create cluster widgets and layouts
@@ -1721,7 +1721,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
         request.getHostNames() != null && !request.getHostNames().isEmpty();
 
     if (requiresHostListUpdate) {
-      clusters.mapHostsToCluster(
+      clusters.mapAndPublishHostsToCluster(
           request.getHostNames(), request.getClusterName());
     }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java
index 8142afc..62a5bd4 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java
@@ -721,7 +721,7 @@ public class HostResourceProvider extends AbstractControllerResourceProvider {
       try {
         // The below method call throws an exception when trying to create a duplicate mapping in the clusterhostmapping
         // table. This is done to detect duplicates during host create. In order to be robust, handle these gracefully.
-        clusters.mapHostToCluster(request.getHostname(), clusterName);
+        clusters.mapAndPublishHostsToCluster(new HashSet<>(Arrays.asList(request.getHostname())), clusterName);
       } catch (DuplicateResourceException e) {
         // do nothing
       }
@@ -871,10 +871,17 @@ public class HostResourceProvider extends AbstractControllerResourceProvider {
   }
 
   private void processDeleteHostRequests(List<HostRequest> requests,  Clusters clusters, DeleteStatusMetaData deleteStatusMetaData) throws AmbariException {
+    Set<String> hostsClusters = new HashSet<>();
+    Set<String> hostNames = new HashSet<>();
+    Set<Cluster> allClustersWithHosts = new HashSet<>();
     for (HostRequest hostRequest : requests) {
       // Assume the user also wants to delete it entirely, including all clusters.
       String hostname = hostRequest.getHostname();
+      hostNames.add(hostname);
 
+      if (hostRequest.getClusterName() != null) {
+        hostsClusters.add(hostRequest.getClusterName());
+      }
       // delete all host components
       Set<ServiceComponentHostRequest> schrs = new HashSet<>();
       for(Cluster cluster : clusters.getClustersForHost(hostname)) {
@@ -906,6 +913,9 @@ public class HostResourceProvider extends AbstractControllerResourceProvider {
         }
       }
 
+      if (hostRequest.getClusterName() != null) {
+        hostsClusters.add(hostRequest.getClusterName());
+      }
       try {
         clusters.deleteHost(hostname);
         deleteStatusMetaData.addDeletedKey(hostname);
@@ -916,10 +926,10 @@ public class HostResourceProvider extends AbstractControllerResourceProvider {
       for (LogicalRequest logicalRequest: topologyManager.getRequests(Collections.<Long>emptyList())) {
         logicalRequest.removeHostRequestByHostName(hostname);
       }
-
-      if (null != hostRequest.getClusterName()) {
-        clusters.getCluster(hostRequest.getClusterName()).recalculateAllClusterVersionStates();
-      }
+    }
+    clusters.publishHostsDeletion(allClustersWithHosts, hostNames);
+    for (String clustername : hostsClusters) {
+      clusters.getCluster(clustername).recalculateAllClusterVersionStates();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/events/HostAddedEvent.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/HostAddedEvent.java b/ambari-server/src/main/java/org/apache/ambari/server/events/HostAddedEvent.java
deleted file mode 100644
index 7832c16..0000000
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/HostAddedEvent.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ambari.server.events;
-
-/**
- * The {@link HostAddedEvent} is fired when a host is added to a cluster.
- */
-public class HostAddedEvent extends ClusterEvent {
-
-  /**
-   * The host's name.
-   */
-  protected final String m_hostName;
-
-  /**
-   * Constructor.
-   *
-   * @param clusterId
-   *          the ID of the cluster.
-   * @param hostName
-   *          the name of the host.
-   */
-  public HostAddedEvent(long clusterId, String hostName) {
-    super(AmbariEventType.HOST_ADDED, clusterId);
-    m_hostName = hostName;
-  }
-
-  /**
-   * Gets the host's name that the event belongs to.
-   *
-   * @return the hostName
-   */
-  public String getHostName() {
-    return m_hostName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/events/HostRemovedEvent.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/HostRemovedEvent.java b/ambari-server/src/main/java/org/apache/ambari/server/events/HostRemovedEvent.java
deleted file mode 100644
index 286e861..0000000
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/HostRemovedEvent.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ambari.server.events;
-
-import java.util.Collections;
-import java.util.Set;
-
-import org.apache.ambari.server.state.Cluster;
-
-/**
- * The {@link HostRemovedEvent} class is fired when a host is removed from the
- * cluster.
- */
-public class HostRemovedEvent extends HostEvent {
-
-  /**
-   * The clusters that the removed host belonged to.
-   */
-  private final Set<Cluster> m_clusters;
-
-  /**
-   * Constructor.
-   *
-   * @param hostName
-   */
-  public HostRemovedEvent(String hostName, Set<Cluster> clusters) {
-    super(AmbariEventType.HOST_REMOVED, hostName);
-    m_clusters = clusters;
-  }
-
-  /**
-   * The clusters that the host belonged to.
-   *
-   * @return the clusters, or an empty set.
-   */
-  public Set<Cluster> getClusters() {
-    if (null == m_clusters) {
-      return Collections.emptySet();
-    }
-
-    return m_clusters;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder("HostRemovedEvent{");
-    buffer.append("hostName=").append(m_hostName);
-    buffer.append("}");
-    return buffer.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/events/HostsAddedEvent.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/HostsAddedEvent.java b/ambari-server/src/main/java/org/apache/ambari/server/events/HostsAddedEvent.java
new file mode 100644
index 0000000..069673c
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/HostsAddedEvent.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ambari.server.events;
+
+import java.util.Set;
+
+/**
+ * The {@link HostsAddedEvent} is fired when the hosts are added to a cluster.
+ */
+public class HostsAddedEvent extends ClusterEvent {
+
+  /**
+   * The hosts' names.
+   */
+  protected final Set<String> m_hostNames;
+
+  /**
+   * Constructor.
+   * @param clusterId
+   * @param hostNames
+   */
+  public HostsAddedEvent(long clusterId, Set<String> hostNames) {
+    super(AmbariEventType.HOST_ADDED, clusterId);
+    m_hostNames = hostNames;
+  }
+
+  /**
+   * Gets the hosts' names that the event belongs to.
+   *
+   * @return the hostName
+   */
+  public Set<String> getHostNames() {
+    return m_hostNames;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/events/HostsRemovedEvent.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/HostsRemovedEvent.java b/ambari-server/src/main/java/org/apache/ambari/server/events/HostsRemovedEvent.java
new file mode 100644
index 0000000..ca28246
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/HostsRemovedEvent.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ambari.server.events;
+
+import org.apache.ambari.server.state.Cluster;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * The {@link HostsRemovedEvent} class is fired when the hosts are removed from the
+ * cluster.
+ */
+public class HostsRemovedEvent extends AmbariEvent {
+
+  /**
+   * The clusters that the removed hosts belonged to.
+   */
+  private final Set<Cluster> m_clusters;
+
+  /**
+   * Removed hosts.
+   */
+  private final Set<String> m_hosts;
+
+  /**
+   * Constructor.
+   * @param hosts
+   * @param clusters
+   */
+  public HostsRemovedEvent(Set<String> hosts, Set<Cluster> clusters) {
+    super(AmbariEventType.HOST_REMOVED);
+    m_clusters = clusters;
+    m_hosts = hosts;
+  }
+
+  /**
+   * The clusters that the hosts belonged to.
+   *
+   * @return the clusters, or an empty set.
+   */
+  public Set<Cluster> getClusters() {
+    if (null == m_clusters) {
+      return Collections.emptySet();
+    }
+
+    return m_clusters;
+  }
+
+  /**
+   * Removed hosts.
+   * @return
+   */
+  public Set<String> getHostNames() {
+    if (null == m_hosts) {
+      return Collections.emptySet();
+    }
+
+    return m_hosts;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public String toString() {
+    final StringBuffer sb = new StringBuffer("HostsRemovedEvent{");
+    sb.append("m_clusters=").append(m_clusters);
+    sb.append(", m_hosts=").append(m_hosts);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertHostListener.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertHostListener.java b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertHostListener.java
index 50bf77a..79ba4b4 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertHostListener.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertHostListener.java
@@ -25,8 +25,8 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.ambari.server.EagerSingleton;
 import org.apache.ambari.server.events.AlertHashInvalidationEvent;
-import org.apache.ambari.server.events.HostAddedEvent;
-import org.apache.ambari.server.events.HostRemovedEvent;
+import org.apache.ambari.server.events.HostsAddedEvent;
+import org.apache.ambari.server.events.HostsRemovedEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.metadata.AmbariServiceAlertDefinitions;
 import org.apache.ambari.server.orm.dao.AlertDefinitionDAO;
@@ -44,8 +44,8 @@ import com.google.inject.Inject;
 import com.google.inject.Singleton;
 
 /**
- * The {@link AlertHostListener} class handles {@link HostAddedEvent} and
- * {@link HostRemovedEvent} and ensures that {@link AlertCurrentEntity}
+ * The {@link AlertHostListener} class handles {@link HostsAddedEvent} and
+ * {@link HostsRemovedEvent} and ensures that {@link AlertCurrentEntity}
  * instances are properly cleaned up
  */
 @Singleton
@@ -104,7 +104,7 @@ public class AlertHostListener {
   }
 
   /**
-   * Handles the {@link HostAddedEvent} by performing the following actions:
+   * Handles the {@link HostsAddedEvent} by performing the following actions:
    * <ul>
    * <li>Ensures that all host-level alerts are loaded for the cluster. This is
    * especially useful when creating a cluster and no alerts were loaded on
@@ -115,7 +115,7 @@ public class AlertHostListener {
    */
   @Subscribe
   @AllowConcurrentEvents
-  public void onAmbariEvent(HostAddedEvent event) {
+  public void onAmbariEvent(HostsAddedEvent event) {
     LOG.debug("Received event {}", event);
 
     long clusterId = event.getClusterId();
@@ -155,24 +155,28 @@ public class AlertHostListener {
       m_hostAlertLock.unlock();
     }
 
-    AlertHashInvalidationEvent invalidationEvent = new AlertHashInvalidationEvent(
-        event.getClusterId(), Collections.singletonList(event.getHostName()));
+    for (String hostName : event.getHostNames()) {
+      AlertHashInvalidationEvent invalidationEvent = new AlertHashInvalidationEvent(
+        event.getClusterId(), Collections.singletonList(hostName));
 
-    m_eventPublisher.publish(invalidationEvent);
+      m_eventPublisher.publish(invalidationEvent);
+    }
   }
 
   /**
-   * Handles the {@link HostRemovedEvent} by performing the following actions:
+   * Handles the {@link HostsRemovedEvent} by performing the following actions:
    * <ul>
-   * <li>Removes all {@link AlertCurrentEntity} for the removed host</li>
+   * <li>Removes all {@link AlertCurrentEntity} for the removed hosts</li>
    * </ul>
    */
   @Subscribe
   @AllowConcurrentEvents
-  public void onAmbariEvent(HostRemovedEvent event) {
+  public void onAmbariEvent(HostsRemovedEvent event) {
     LOG.debug("Received event {}", event);
 
-    // remove any current alerts for the removed host
-    m_alertsDao.removeCurrentByHost(event.getHostName());
+    // remove any current alerts for the removed hosts
+    for (String hostName : event.getHostNames()) {
+      m_alertsDao.removeCurrentByHost(hostName);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListener.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListener.java b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListener.java
index 1b298f6..dabd348 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListener.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListener.java
@@ -28,8 +28,8 @@ import java.util.Set;
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.EagerSingleton;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
-import org.apache.ambari.server.events.HostAddedEvent;
-import org.apache.ambari.server.events.HostRemovedEvent;
+import org.apache.ambari.server.events.HostsAddedEvent;
+import org.apache.ambari.server.events.HostsRemovedEvent;
 import org.apache.ambari.server.events.ServiceComponentInstalledEvent;
 import org.apache.ambari.server.events.ServiceInstalledEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
@@ -61,7 +61,7 @@ import com.google.inject.persist.Transactional;
  * {@link org.apache.ambari.server.events.ServiceComponentInstalledEvent}
  * to update {@link org.apache.ambari.server.state.RepositoryVersionState}
  *
- * @see org.apache.ambari.server.state.Cluster#recalculateClusterVersionState(StackId, String)
+ * @see org.apache.ambari.server.state.Cluster#recalculateClusterVersionState(RepositoryVersionEntity)
  */
 @Singleton
 @EagerSingleton
@@ -187,7 +187,7 @@ public class HostVersionOutOfSyncListener {
 
   @Subscribe
   @Transactional
-  public void onHostEvent(HostAddedEvent event) {
+  public void onHostEvent(HostsAddedEvent event) {
     if (LOG.isDebugEnabled()) {
       LOG.debug(event.toString());
     }
@@ -198,16 +198,17 @@ public class HostVersionOutOfSyncListener {
       Collection<ClusterVersionEntity> allClusterVersions = cluster.getAllClusterVersions();
       for (ClusterVersionEntity clusterVersion : allClusterVersions) {
         if (clusterVersion.getState() != RepositoryVersionState.CURRENT) { // Current version is taken care of automatically
-          String hostName = event.getHostName();
-          HostEntity hostEntity = hostDAO.get().findByName(hostName);
           RepositoryVersionEntity repositoryVersion = clusterVersion.getRepositoryVersion();
-          HostVersionEntity missingHostVersion = new HostVersionEntity(hostEntity,
-                  repositoryVersion, RepositoryVersionState.OUT_OF_SYNC);
+          for (String hostName : event.getHostNames()) {
+            HostEntity hostEntity = hostDAO.get().findByName(hostName);
+            HostVersionEntity missingHostVersion = new HostVersionEntity(hostEntity,
+              repositoryVersion, RepositoryVersionState.OUT_OF_SYNC);
 
-          LOG.info("Creating host version for {}, state={}, repo={} (repo_id={})",
+            LOG.info("Creating host version for {}, state={}, repo={} (repo_id={})",
               missingHostVersion.getHostName(), missingHostVersion.getState(),
               missingHostVersion.getRepositoryVersion().getVersion(), missingHostVersion.getRepositoryVersion().getId());
-          hostVersionDAO.get().create(missingHostVersion);
+            hostVersionDAO.get().create(missingHostVersion);
+          }
           cluster.recalculateClusterVersionState(repositoryVersion);
         }
       }
@@ -228,7 +229,7 @@ public class HostVersionOutOfSyncListener {
    */
   @Subscribe
   @Transactional
-  public void onHostEvent(HostRemovedEvent event) {
+  public void onHostEvent(HostsRemovedEvent event) {
     if (LOG.isDebugEnabled()) {
       LOG.debug(event.toString());
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/CrudDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/CrudDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/CrudDAO.java
index 473c558..7150dcc 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/CrudDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/CrudDAO.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ambari.server.orm.dao;
 
+import java.util.Collection;
 import java.util.List;
 
 import javax.persistence.EntityManager;
@@ -130,6 +131,19 @@ public class CrudDAO<E, K> {
   }
 
   /**
+   * Deletes entities.
+   *
+   * @param entities entities to delete
+   */
+  @Transactional
+  public void remove(Collection<E> entities) {
+    for (E entity : entities) {
+      entityManagerProvider.get().remove(merge(entity));
+    }
+    entityManagerProvider.get().getEntityManagerFactory().getCache().evictAll();
+  }
+
+  /**
    * Deletes entity by PK.
    *
    * @param pk primary key

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostVersionDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostVersionDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostVersionDAO.java
index e28f9ef..930a535 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostVersionDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostVersionDAO.java
@@ -278,9 +278,7 @@ public class HostVersionDAO extends CrudDAO<HostVersionEntity, Long> {
   @Transactional
   public void removeByHostName(String hostName) {
     Collection<HostVersionEntity> hostVersions = this.findByHost(hostName);
-    for (HostVersionEntity hostVersion : hostVersions) {
-      this.remove(hostVersion);
-    }
+    this.remove(hostVersions);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/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 e2164c0..850e1b3 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
@@ -157,7 +157,7 @@ public interface Clusters {
    * @param clusterName
    * @throws AmbariException
    */
-  void mapHostsToCluster(Set<String> hostnames, String clusterName)
+  void mapAndPublishHostsToCluster(Set<String> hostnames, String clusterName)
       throws AmbariException;
 
   /**
@@ -243,6 +243,15 @@ public interface Clusters {
       throws AmbariException;
 
   /**
+   * Publish event set of hosts were removed
+   * @param clusters
+   * @param hostNames
+   * @throws AmbariException
+   */
+  void publishHostsDeletion(Set<Cluster> clusters, Set<String> hostNames)
+      throws AmbariException;
+
+  /**
    * Determine whether or not access to the cluster resource identified
    * by the given cluster name should be allowed based on the permissions
    * granted to the current user.

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/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 1de10f9..3c2b582 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
@@ -38,9 +38,9 @@ import org.apache.ambari.server.DuplicateResourceException;
 import org.apache.ambari.server.HostNotFoundException;
 import org.apache.ambari.server.agent.DiskInfo;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
-import org.apache.ambari.server.events.HostAddedEvent;
 import org.apache.ambari.server.events.HostRegisteredEvent;
-import org.apache.ambari.server.events.HostRemovedEvent;
+import org.apache.ambari.server.events.HostsAddedEvent;
+import org.apache.ambari.server.events.HostsRemovedEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.orm.dao.ClusterDAO;
 import org.apache.ambari.server.orm.dao.ClusterVersionDAO;
@@ -453,25 +453,35 @@ public class ClustersImpl implements Clusters {
     }
 
     Map<String, Host> hostMap = getHostsMap(hostClusters.keySet());
-    Set<String> clusterNames = new HashSet<String>();
-    for (Set<String> cSet : hostClusters.values()) {
-      clusterNames.addAll(cSet);
-    }
 
-    for (String hostname : hostClusters.keySet()) {
+    Map<String, Set<String>> clusterHosts = new HashMap<>();
+    for (Map.Entry<String, Set<String>> hostClustersEntry : hostClusters.entrySet()) {
+      Set<String> hostClusterNames = hostClustersEntry.getValue();
+      String hostname = hostClustersEntry.getKey();
+
+      // populate attributes
       Host host = hostMap.get(hostname);
       Map<String, String> attributes = hostAttributes.get(hostname);
       if (attributes != null && !attributes.isEmpty()) {
         host.setHostAttributes(attributes);
       }
 
-      Set<String> hostClusterNames = hostClusters.get(hostname);
+      // create cluster to hosts map
       for (String clusterName : hostClusterNames) {
         if (clusterName != null && !clusterName.isEmpty()) {
-          mapHostToCluster(hostname, clusterName);
+          if (!clusterHosts.containsKey(clusterName)) {
+            clusterHosts.put(clusterName, new HashSet<String>());
+          }
+          clusterHosts.get(clusterName).add(hostname);
         }
       }
     }
+
+    for (Map.Entry<String, Set<String>> clusterHostsEntry : clusterHosts.entrySet()) {
+      Set<String> clusterHostsNames = clusterHostsEntry.getValue();
+      String clusterName = clusterHostsEntry.getKey();
+      mapAndPublishHostsToCluster(clusterHostsNames, clusterName);
+    }
   }
 
   private Map<String, Host> getHostsMap(Collection<String> hostSet) throws
@@ -502,11 +512,17 @@ public class ClustersImpl implements Clusters {
    * @throws AmbariException
    */
   @Override
-  public void mapHostsToCluster(Set<String> hostnames, String clusterName) throws AmbariException {
-    ClusterVersionEntity clusterVersionEntity = clusterVersionDAO.findByClusterAndStateCurrent(clusterName);
+  public void mapAndPublishHostsToCluster(Set<String> hostnames, String clusterName) throws AmbariException {
     for (String hostname : hostnames) {
-      mapHostToCluster(hostname, clusterName, clusterVersionEntity);
+      mapHostToCluster(hostname, clusterName);
     }
+    publishAddingHostsToCluster(hostnames, clusterName);
+    getCluster(clusterName).refresh();
+  }
+
+  private void publishAddingHostsToCluster(Set<String> hostnames, String clusterName) throws AmbariException {
+    HostsAddedEvent event = new HostsAddedEvent(getCluster(clusterName).getClusterId(), hostnames);
+    eventPublisher.publish(event);
   }
 
   /**
@@ -514,11 +530,11 @@ public class ClustersImpl implements Clusters {
    * record for the cluster's currently applied (stack, version) if not already present.
    * @param hostname Host name
    * @param clusterName Cluster name
-   * @param currentClusterVersion Cluster's current stack version
    * @throws AmbariException May throw a DuplicateResourceException.
    */
-  public void mapHostToCluster(String hostname, String clusterName,
-      ClusterVersionEntity currentClusterVersion) throws AmbariException {
+  @Override
+  public void mapHostToCluster(String hostname, String clusterName)
+      throws AmbariException {
     Host host = null;
     Cluster cluster = null;
 
@@ -529,15 +545,15 @@ public class ClustersImpl implements Clusters {
     for (Cluster c : hostClusterMap.get(hostname)) {
       if (c.getClusterName().equals(clusterName)) {
         throw new DuplicateResourceException("Attempted to create a host which already exists: clusterName=" +
-            clusterName + ", hostName=" + hostname);
+          clusterName + ", hostName=" + hostname);
       }
     }
 
     if (!isOsSupportedByClusterStack(cluster, host)) {
       String message = "Trying to map host to cluster where stack does not"
-          + " support host's os type" + ", clusterName=" + clusterName
-          + ", clusterStackId=" + cluster.getDesiredStackVersion().getStackId()
-          + ", hostname=" + hostname + ", hostOsFamily=" + host.getOsFamily();
+        + " support host's os type" + ", clusterName=" + clusterName
+        + ", clusterStackId=" + cluster.getDesiredStackVersion().getStackId()
+        + ", hostname=" + hostname + ", hostOsFamily=" + host.getOsFamily();
       LOG.error(message);
       throw new AmbariException(message);
     }
@@ -545,28 +561,12 @@ public class ClustersImpl implements Clusters {
     long clusterId = cluster.getClusterId();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Mapping host {} to cluster {} (id={})", hostname, clusterName,
-          clusterId);
+        clusterId);
     }
 
     mapHostClusterEntities(hostname, clusterId);
     hostClusterMap.get(hostname).add(cluster);
     clusterHostMap.get(clusterName).add(host);
-
-    cluster.refresh();
-  }
-
-  /**
-   * Attempts to map the host to the cluster via clusterhostmapping table if not already present, and add a host_version
-   * record for the cluster's currently applied (stack, version) if not already present. This function is overloaded.
-   * @param hostname Host name
-   * @param clusterName Cluster name
-   * @throws AmbariException May throw a DuplicateResourceException.
-   */
-  @Override
-  public void mapHostToCluster(String hostname, String clusterName)
-      throws AmbariException {
-    ClusterVersionEntity clusterVersionEntity = clusterVersionDAO.findByClusterAndStateCurrent(clusterName);
-    mapHostToCluster(hostname, clusterName, clusterVersionEntity);
   }
 
   @Transactional
@@ -579,10 +579,6 @@ public class ClustersImpl implements Clusters {
 
     clusterDAO.merge(clusterEntity);
     hostDAO.merge(hostEntity);
-
-    // publish the event for adding a host to a cluster
-    HostAddedEvent event = new HostAddedEvent(clusterId, hostName);
-    eventPublisher.publish(event);
   }
 
   @Override
@@ -729,8 +725,6 @@ public class ClustersImpl implements Clusters {
    * Delete a host entirely from the cluster and all database tables, except
    * AlertHistory. If the host is not found, throws
    * {@link org.apache.ambari.server.HostNotFoundException}.
-   * <p/>
-   * This method will trigger a {@link HostRemovedEvent} when completed.
    *
    * @param hostname
    * @throws AmbariException
@@ -745,19 +739,20 @@ public class ClustersImpl implements Clusters {
       throw new HostNotFoundException(hostname);
     }
 
-    Set<Cluster> hostsClusters = new HashSet<>(clusters);
-
     deleteHostEntityRelationships(hostname);
+  }
 
+  @Override
+  public void publishHostsDeletion(Set<Cluster> clusters, Set<String> hostNames) throws AmbariException {
     // Publish the event, using the original list of clusters that the host
     // belonged to
-    HostRemovedEvent event = new HostRemovedEvent(hostname, hostsClusters);
+    HostsRemovedEvent event = new HostsRemovedEvent(hostNames, clusters);
     eventPublisher.publish(event);
   }
 
   /***
    * Deletes all of the JPA relationships between a host and other entities.
-   * This method will not fire {@link HostRemovedEvent} since it is performed
+   * This method will not fire {@link HostsRemovedEvent} since it is performed
    * within an {@link Transactional} and the event must fire after the
    * transaction is successfully committed.
    *

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/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 a444f5d..3c7b040 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
@@ -696,7 +696,7 @@ public class HostImpl implements Host {
   @Override
   public long getAvailableMemBytes() {
     HostStateEntity hostStateEntity = getHostStateEntity();
-    return hostStateEntity != null ? hostStateEntity.getAvailableMem() : null;
+    return hostStateEntity != null ? hostStateEntity.getAvailableMem() : 0;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/main/java/org/apache/ambari/server/topology/TopologyManager.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/topology/TopologyManager.java b/ambari-server/src/main/java/org/apache/ambari/server/topology/TopologyManager.java
index d527b2d..7db07a0 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/topology/TopologyManager.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/topology/TopologyManager.java
@@ -59,7 +59,7 @@ import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.events.AmbariEvent;
 import org.apache.ambari.server.events.ClusterConfigFinishedEvent;
-import org.apache.ambari.server.events.HostRemovedEvent;
+import org.apache.ambari.server.events.HostsRemovedEvent;
 import org.apache.ambari.server.events.RequestFinishedEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.orm.dao.HostRoleCommandStatusSummaryDTO;
@@ -1020,33 +1020,37 @@ public class TopologyManager {
   /**
    *
    * Removes a host from the available hosts when the host gets deleted.
-   * @param hostRemovedEvent the event containing the hostname
+   * @param hostsRemovedEvent the event containing the hostname
    */
   @Subscribe
-  public void processHostRemovedEvent(HostRemovedEvent hostRemovedEvent) {
+  public void processHostRemovedEvent(HostsRemovedEvent hostsRemovedEvent) {
 
-    if (null == hostRemovedEvent.getHostName()) {
-      LOG.warn("Missing host name from host removed event [{}] !", hostRemovedEvent);
+    if (hostsRemovedEvent.getHostNames().isEmpty()) {
+      LOG.warn("Missing host name from host removed event [{}] !", hostsRemovedEvent);
       return;
     }
 
-    LOG.info("Removing host [{}] from available hosts on host removed event.", hostRemovedEvent.getHostName());
-    HostImpl toBeRemoved = null;
+    LOG.info("Removing hosts [{}] from available hosts on hosts removed event.", hostsRemovedEvent.getHostNames());
+    Set<HostImpl> toBeRemoved = new HashSet<>();
 
     // synchronization is required here as the list may be modified concurrently. See comments in this whole class.
     synchronized (availableHosts) {
       for (HostImpl hostImpl : availableHosts) {
-        if (hostRemovedEvent.getHostName().equals(hostImpl.getHostName())) {
-          toBeRemoved = hostImpl;
-          break;
+        for (String hostName : hostsRemovedEvent.getHostNames()) {
+          if (hostName.equals(hostImpl.getHostName())) {
+            toBeRemoved.add(hostImpl);
+            break;
+          }
         }
       }
 
-      if (null != toBeRemoved) {
-        availableHosts.remove(toBeRemoved);
-        LOG.info("Removed host: [{}] from available hosts", toBeRemoved.getHostName());
+      if (!toBeRemoved.isEmpty()) {
+        for (HostImpl host : toBeRemoved) {
+          availableHosts.remove(host);
+          LOG.info("Removed host: [{}] from available hosts", host.getHostName());
+        }
       } else {
-        LOG.debug("Host [{}] not found in available hosts", hostRemovedEvent.getHostName());
+        LOG.debug("No any host [{}] found in available hosts", hostsRemovedEvent.getHostNames());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatTestHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatTestHelper.java b/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatTestHelper.java
index fc2bca5..32715de 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatTestHelper.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatTestHelper.java
@@ -216,7 +216,7 @@ public class HeartbeatTestHelper {
     }
 
     clusterEntity.setHostEntities(hostEntities);
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
 
     return cluster;
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatMonitor.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatMonitor.java b/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatMonitor.java
index 68e9993..4d129a9 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatMonitor.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/agent/TestHeartbeatMonitor.java
@@ -164,7 +164,7 @@ public class TestHeartbeatMonitor {
     cluster.addDesiredConfig("_test", Collections.singleton(config));
 
 
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
     Service hdfs = cluster.addService(serviceName);
     hdfs.addServiceComponent(Role.DATANODE.name());
     hdfs.getServiceComponent(Role.DATANODE.name()).addServiceComponentHost(hostname1);
@@ -253,7 +253,7 @@ public class TestHeartbeatMonitor {
     cluster.addDesiredConfig("_test", Collections.singleton(hadoopEnvConfig));
 
 
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
     Service hdfs = cluster.addService(serviceName);
     hdfs.addServiceComponent(Role.DATANODE.name());
     hdfs.getServiceComponent(Role.DATANODE.name()).addServiceComponentHost
@@ -360,7 +360,7 @@ public class TestHeartbeatMonitor {
       add(hostname1);
      }};
 
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
 
     Service hdfs = cluster.addService(serviceName);
     hdfs.addServiceComponent(Role.DATANODE.name());
@@ -442,7 +442,7 @@ public class TestHeartbeatMonitor {
       add(hostname1);
      }};
 
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
 
     Service hdfs = cluster.addService(serviceName);
     hdfs.addServiceComponent(Role.DATANODE.name());
@@ -561,7 +561,7 @@ public class TestHeartbeatMonitor {
       add(hostname2);
     }};
 
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
 
     Service hdfs = cluster.addService(serviceName);
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/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 fb18472..24016a8 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
@@ -1690,9 +1690,9 @@ public class AmbariManagementControllerTest {
     hostnames.add(host1);
     hostnames.add(host2);
     hostnames.add(host3);
-    clusters.mapHostsToCluster(hostnames, clusterFoo);
-    clusters.mapHostsToCluster(hostnames, cluster1);
-    clusters.mapHostsToCluster(hostnames, cluster2);
+    clusters.mapAndPublishHostsToCluster(hostnames, clusterFoo);
+    clusters.mapAndPublishHostsToCluster(hostnames, cluster1);
+    clusters.mapAndPublishHostsToCluster(hostnames, cluster2);
 
     set1.clear();
     ServiceComponentHostRequest valid =
@@ -8239,7 +8239,7 @@ public class AmbariManagementControllerTest {
     clusters.addHost(hostName1);
     setOsFamily(clusters.getHost(hostName1), "redhat", "5.9");
 
-    clusters.mapHostsToCluster(new HashSet<String>(){
+    clusters.mapAndPublishHostsToCluster(new HashSet<String>(){
       {add(hostName1);}}, cluster1);
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/HostResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/HostResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/HostResourceProviderTest.java
index c71b054..b9b8d2a 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/HostResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/HostResourceProviderTest.java
@@ -905,7 +905,7 @@ public class HostResourceProviderTest extends EasyMockSupport {
     expect(clusters.getHosts()).andReturn(Arrays.asList(host100)).anyTimes();
     expect(clusters.getHostsForCluster("Cluster100")).andReturn(Collections.singletonMap("Host100", host100)).anyTimes();
     expect(clusters.getHost("Host100")).andReturn(host100).anyTimes();
-    clusters.mapHostToCluster("Host100", "Cluster100");
+    clusters.mapAndPublishHostsToCluster(Collections.singleton("Host100"), "Cluster100");
     expectLastCall().anyTimes();
     cluster.recalculateAllClusterVersionStates();
     expectLastCall().anyTimes();
@@ -998,7 +998,7 @@ public class HostResourceProviderTest extends EasyMockSupport {
     expect(clusters.getClustersForHost("Host100")).andReturn(clusterSet).anyTimes();
     expect(clusters.getHost("Host100")).andReturn(host100).anyTimes();
     expect(clusters.getHostsForCluster("Cluster100")).andReturn(Collections.singletonMap("Host100", host100)).anyTimes();
-    clusters.mapHostToCluster("Host100", "Cluster100");
+    clusters.mapAndPublishHostsToCluster(Collections.singleton("Host100"), "Cluster100");
     expectLastCall().anyTimes();
     cluster.recalculateAllClusterVersionStates();
     expectLastCall().anyTimes();
@@ -1079,6 +1079,7 @@ public class HostResourceProviderTest extends EasyMockSupport {
     expect(cluster.getClusterId()).andReturn(100L).anyTimes();
     expect(cluster.getDesiredConfigs()).andReturn(new HashMap<String, DesiredConfig>()).anyTimes();
     clusters.deleteHost("Host100");
+    clusters.publishHostsDeletion(Collections.EMPTY_SET, Collections.singleton("Host100"));
     cluster.recalculateAllClusterVersionStates();
     expect(host1.getHostName()).andReturn("Host100").anyTimes();
     expect(healthStatus.getHealthStatus()).andReturn(HostHealthStatus.HealthStatus.HEALTHY).anyTimes();

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListenerTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListenerTest.java b/ambari-server/src/test/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListenerTest.java
index 9a5802d..3592668 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListenerTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/events/listeners/upgrade/HostVersionOutOfSyncListenerTest.java
@@ -31,7 +31,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.ambari.server.AmbariException;
-import org.apache.ambari.server.events.HostRemovedEvent;
 import org.apache.ambari.server.events.ServiceComponentInstalledEvent;
 import org.apache.ambari.server.events.ServiceInstalledEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
@@ -378,7 +377,7 @@ public class HostVersionOutOfSyncListenerTest {
   }
 
   /**
-   * Tests that when a host is removed, the {@link HostRemovedEvent} fires and
+   * Tests that when a host is removed, the {@link org.apache.ambari.server.events.HostsRemovedEvent} fires and
    * eventually calls to recalculate the cluster state.
    */
   @Test
@@ -428,6 +427,7 @@ public class HostVersionOutOfSyncListenerTest {
     // event handle it
     injector.getInstance(UnitOfWork.class).begin();
     clusters.deleteHost("h2");
+    clusters.publishHostsDeletion(Collections.singleton(c1), Collections.singleton("h2"));
     injector.getInstance(UnitOfWork.class).end();
     assertRepoVersionState(stackId.getStackId(), "2.2.0", RepositoryVersionState.CURRENT);
     assertRepoVersionState(stackId.getStackId(), "2.2.9-9999", RepositoryVersionState.INSTALLED);

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/orm/OrmTestHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/orm/OrmTestHelper.java b/ambari-server/src/test/java/org/apache/ambari/server/orm/OrmTestHelper.java
index 15e81c5..ead0a2d 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/orm/OrmTestHelper.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/orm/OrmTestHelper.java
@@ -359,6 +359,9 @@ public class OrmTestHelper {
     assertNotNull(clusterEntity);
     assertTrue(clusterEntity.getClusterId() > 0);
 
+    clusterEntity.setClusterStateEntity(clusterStateEntity);
+    clusterDAO.merge(clusterEntity);
+
     // because this test method goes around the Clusters business object, we
     // forcefully will refresh the internal state so that any tests which
     // incorrect use Clusters after calling this won't be affected
@@ -411,7 +414,7 @@ public class OrmTestHelper {
     host.setHostAttributes(hostAttributes);
     host.setState(HostState.HEALTHY);
 
-    clusters.mapHostToCluster(hostName, cluster.getClusterName());
+    clusters.mapAndPublishHostsToCluster(Collections.singleton(hostName), cluster.getClusterName());
   }
 
   public void addHostComponent(Cluster cluster, String hostName, String serviceName, String componentName) throws AmbariException {

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
index 262b10a..72d0f13 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
@@ -297,7 +297,7 @@ public class UpgradeActionTest {
     host.setHostAttributes(hostAttributes);
 
     // without this, HostEntity will not have a relation to ClusterEntity
-    clusters.mapHostsToCluster(Collections.singleton(hostName), clusterName);
+    clusters.mapHostToCluster(hostName, clusterName);
 
     // Create the starting repo version
     RepositoryVersionEntity repoEntity = m_helper.getOrCreateRepositoryVersion(sourceStack, sourceRepo);

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterImplTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterImplTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterImplTest.java
index aad074e..401527a 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterImplTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterImplTest.java
@@ -211,7 +211,7 @@ public class ClusterImplTest {
     Host host2 = clusters.getHost(hostName2);
     host2.setHostAttributes(ImmutableMap.of("os_family", "centos", "os_release_version", "6.0"));
 
-    clusters.mapHostsToCluster(Sets.newHashSet(hostName1, hostName2), clusterName);
+    clusters.mapAndPublishHostsToCluster(Sets.newHashSet(hostName1, hostName2), clusterName);
 
     Service hdfs = cluster.addService("HDFS");
 
@@ -269,7 +269,7 @@ public class ClusterImplTest {
     Host host2 = clusters.getHost(hostName2);
     host2.setHostAttributes(ImmutableMap.of("os_family", "centos", "os_release_version", "6.0"));
 
-    clusters.mapHostsToCluster(Sets.newHashSet(hostName1, hostName2), clusterName);
+    clusters.mapAndPublishHostsToCluster(Sets.newHashSet(hostName1, hostName2), clusterName);
 
     // When
     clusters.deleteHost(hostToDelete);
@@ -308,7 +308,7 @@ public class ClusterImplTest {
     Host host2 = clusters.getHost(hostName2);
     host2.setHostAttributes(ImmutableMap.of("os_family", "centos", "os_release_version", "6.0"));
 
-    clusters.mapHostsToCluster(Sets.newHashSet(hostName1, hostName2), clusterName);
+    clusters.mapAndPublishHostsToCluster(Sets.newHashSet(hostName1, hostName2), clusterName);
 
     // When
     int clusterSize = cluster.getClusterSize();
@@ -317,4 +317,4 @@ public class ClusterImplTest {
     assertEquals(2, clusterSize);
 
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/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 daa3abc..bbf1478 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
@@ -243,7 +243,7 @@ public class ClusterTest {
       hostDAO.merge(hostEntity);
     }
 
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
     c1 = clusters.getCluster(clusterName);
 
     helper.getOrCreateRepositoryVersion(stackId, stackId.getStackVersion());
@@ -2035,7 +2035,7 @@ public class ClusterTest {
 
     // Add one more Host, with only Ganglia on it. It should have a HostVersion in OUT_OF_SYNC for v2
     addHost("h-5", hostAttributes);
-    clusters.mapHostToCluster("h-5", clusterName);
+    clusters.mapAndPublishHostsToCluster(Collections.singleton("h-5"), clusterName);
     ServiceComponentHost schHost5Serv3CompB = serviceComponentHostFactory.createNew(sc3CompB, "h-5");
     sc3CompB.addServiceComponentHost(schHost5Serv3CompB);
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClustersTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClustersTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClustersTest.java
index b190704..974b49a 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClustersTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClustersTest.java
@@ -326,7 +326,7 @@ public class ClustersTest {
     hostnames.add(h1);
     hostnames.add(h2);
 
-    clusters.mapHostsToCluster(hostnames, c2);
+    clusters.mapAndPublishHostsToCluster(hostnames, c2);
 
     c = clusters.getClustersForHost(h1);
     Assert.assertEquals(2, c.size());
@@ -425,7 +425,7 @@ public class ClustersTest {
     setOsFamily(clusters.getHost(h1), "centos", "5.9");
     setOsFamily(clusters.getHost(h2), "centos", "5.9");
 
-    clusters.mapHostsToCluster(new HashSet<String>() {
+    clusters.mapAndPublishHostsToCluster(new HashSet<String>() {
       {
         addAll(Arrays.asList(h1, h2));
       }
@@ -688,7 +688,7 @@ public class ClustersTest {
 
     Set<String> hostnames = new HashSet<>();
     hostnames.add(hostName);
-    clusters.mapHostsToCluster(hostnames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostnames, clusterName);
   }
 
   private Cluster createCluster(String clusterName) throws AmbariException {

http://git-wip-us.apache.org/repos/asf/ambari/blob/1689df7f/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 de2b59d..6bd9a09 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
@@ -154,7 +154,7 @@ public class ServiceComponentHostTest {
     clusterEntity.setHostEntities(hostEntities);
     clusterDAO.merge(clusterEntity);
 
-    clusters.mapHostsToCluster(hostNames, clusterName);
+    clusters.mapAndPublishHostsToCluster(hostNames, clusterName);
   }
 
   private ServiceComponentHost createNewServiceComponentHost(String clusterName,