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/01/31 01:36:15 UTC

ambari git commit: AMBARI-9365. RU - FinalizeUpgradeAction needs to ignore hosts without a host_version record (alejandro)

Repository: ambari
Updated Branches:
  refs/heads/trunk d8997295d -> 082beafcb


AMBARI-9365. RU - FinalizeUpgradeAction needs to ignore hosts without a host_version record (alejandro)


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

Branch: refs/heads/trunk
Commit: 082beafcb28d2e2b2c34349805ee97061a8eb103
Parents: d899729
Author: Alejandro Fernandez <af...@hortonworks.com>
Authored: Tue Jan 27 16:33:03 2015 -0800
Committer: Alejandro Fernandez <af...@hortonworks.com>
Committed: Fri Jan 30 16:36:05 2015 -0800

----------------------------------------------------------------------
 .../server/orm/dao/HostComponentStateDAO.java   |  19 ++-
 .../orm/entities/HostComponentStateEntity.java  |   5 +
 .../upgrades/FinalizeUpgradeAction.java         |  95 +++++++++++----
 .../server/state/cluster/ClusterImpl.java       |  34 ++----
 .../svccomphost/ServiceComponentHostImpl.java   |  27 +----
 .../ServiceComponentHostSummary.java            | 118 +++++++++++++++++++
 .../stacks/HDP/2.2/upgrades/upgrade-2.2.xml     |   2 +-
 7 files changed, 220 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/082beafc/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostComponentStateDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostComponentStateDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostComponentStateDAO.java
index fd1ce9e..66e91d3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostComponentStateDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostComponentStateDAO.java
@@ -35,6 +35,8 @@ import java.util.List;
 public class HostComponentStateDAO {
   @Inject
   Provider<EntityManager> entityManagerProvider;
+  @Inject
+  DaoUtils daoUtils;
 
   @RequiresSession
   public HostComponentStateEntity findByPK(HostComponentStateEntityPK primaryKey) {
@@ -43,8 +45,7 @@ public class HostComponentStateDAO {
 
   @RequiresSession
   public List<HostComponentStateEntity> findAll() {
-    TypedQuery<HostComponentStateEntity> query = entityManagerProvider.get()
-      .createQuery("SELECT hsc from HostComponentStateEntity hsc", HostComponentStateEntity.class);
+    final TypedQuery<HostComponentStateEntity> query = entityManagerProvider.get().createNamedQuery("HostComponentStateEntity.findAll", HostComponentStateEntity.class);
     try {
       return query.getResultList();
     } catch (NoResultException ignored) {
@@ -52,6 +53,20 @@ public class HostComponentStateDAO {
     return null;
   }
 
+  /**
+   * Retrieve all of the Host Component States for the given host.
+   *
+   * @param hostName HOst name
+   * @return Return all of the Host Component States that match the criteria.
+   */
+  @RequiresSession
+  public List<HostComponentStateEntity> findByHost(String hostName) {
+    final TypedQuery<HostComponentStateEntity> query = entityManagerProvider.get().createNamedQuery("HostComponentStateEntity.findByHost", HostComponentStateEntity.class);
+    query.setParameter("hostName", hostName);
+
+    return daoUtils.selectList(query);
+  }
+
   @Transactional
   public void refresh(HostComponentStateEntity hostComponentStateEntity) {
     entityManagerProvider.get().refresh(hostComponentStateEntity);

http://git-wip-us.apache.org/repos/asf/ambari/blob/082beafc/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
index 1e48950..0a31b90 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
@@ -29,6 +29,11 @@ import static org.apache.commons.lang.StringUtils.defaultString;
 @IdClass(HostComponentStateEntityPK.class)
 @Table(name = "hostcomponentstate")
 @Entity
+@NamedQueries({
+    @NamedQuery(name = "HostComponentStateEntity.findAll", query = "SELECT hcs from HostComponentStateEntity hcs"),
+    @NamedQuery(name = "HostComponentStateEntity.findByHost", query =
+        "SELECT hcs from HostComponentStateEntity hcs WHERE hcs.hostName=:hostName"),
+})
 public class HostComponentStateEntity {
 
   @Id

http://git-wip-us.apache.org/repos/asf/ambari/blob/082beafc/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
index 56281a2..fceb44d 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
@@ -22,24 +22,27 @@ import com.google.inject.Inject;
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
 import org.apache.ambari.server.agent.CommandReport;
+import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.orm.dao.ClusterVersionDAO;
 import org.apache.ambari.server.orm.dao.HostComponentStateDAO;
 import org.apache.ambari.server.orm.dao.HostVersionDAO;
 import org.apache.ambari.server.orm.entities.ClusterVersionEntity;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
+import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.HostVersionEntity;
 import org.apache.ambari.server.serveraction.AbstractServerAction;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
-import org.apache.ambari.server.state.Host;
 import org.apache.ambari.server.state.RepositoryVersionState;
+import  org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.UpgradeState;
+import org.apache.ambari.server.state.svccomphost.ServiceComponentHostSummary;
 import org.apache.commons.lang.StringUtils;
 
 import java.text.MessageFormat;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
@@ -64,6 +67,9 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
   @Inject
   private HostComponentStateDAO hostComponentStateDAO;
 
+  @Inject
+  private AmbariMetaInfo ambariMetaInfo;
+
   @Override
   public CommandReport execute(
       ConcurrentMap<String, Object> requestSharedDataContext)
@@ -82,7 +88,8 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
 
       Cluster cluster = clusters.getCluster(clusterName);
 
-      String stackId = cluster.getCurrentStackVersion().getStackId();
+      StackId stack = cluster.getCurrentStackVersion();
+      String stackId = stack.getStackId();
       ClusterVersionEntity upgradingClusterVersion = clusterVersionDAO.findByClusterAndStackAndVersion(clusterName,
           stackId, version);
 
@@ -90,44 +97,80 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
         throw new AmbariException(String.format("Cluster stack version %s not found", version));
       }
 
-      if (! allowedStates.contains(upgradingClusterVersion.getState())) {
-        throw new AmbariException(String.format("The cluster stack version state %s is not allowed to transition directly into %s",
-            upgradingClusterVersion.getState(), RepositoryVersionState.CURRENT.toString()));
-      }
-
-      // Validate that all of the hosts in the cluster have the version being upgraded to, and it is in an allowed state.
-      Map<String, Host> hosts = clusters.getHostsForCluster(clusterName);
+      // Validate that all of the hosts with a version in the cluster have the version being upgraded to, and it is in an allowed state.
       List<HostVersionEntity> hostVersions = hostVersionDAO.findByClusterStackAndVersion(clusterName, stackId, version);
 
-      Set<String> hostWithAllowedVersion = new HashSet<String>();
+      // Will include hosts whose state is UPGRADED, and potentially INSTALLED
+      Set<HostVersionEntity> hostsWithAllowedVersion = new HashSet<HostVersionEntity>();
+      Set<HostVersionEntity> hostsWithoutCorrectVersionState = new HashSet<HostVersionEntity>();
+      Set<String> hostsToUpdate = new HashSet<String>();
+      // If true, then the cluster version is still in UPGRADING and allowed to transition to UPGRADED, and then CURRENT
+      boolean atLeastOneHostInInstalledState = false;
+
+      // It is important to only iterate over the hosts with a version, as opposed to all hosts, since some hosts
+      // may only have components that do not advertise a version, such as AMS.
       for (HostVersionEntity hostVersion : hostVersions) {
+        boolean isStateCorrect = false;
+
         if (allowedStates.contains(hostVersion.getState())) {
-          hostWithAllowedVersion.add(hostVersion.getHostName());
+          isStateCorrect = true;
+        } else {
+          if (hostVersion.getState() == RepositoryVersionState.INSTALLED) {
+            // It is possible that the host version has a state of INSTALLED and it never changed if the host only has
+            // components that do not advertise a version.
+            HostEntity host = hostVersion.getHostEntity();
+            ServiceComponentHostSummary hostSummary = new ServiceComponentHostSummary(ambariMetaInfo, host, stack);
+            if (hostSummary.haveAllComponentsFinishedAdvertisingVersion()){
+              isStateCorrect = true;
+              atLeastOneHostInInstalledState = true;
+            }
+          }
         }
-      }
 
-      Set<String> hostsWithoutCorrectVersionState = new HashSet<String>();
-      for (String host : hosts.keySet()) {
-        if (!hostWithAllowedVersion.contains(host)) {
-          hostsWithoutCorrectVersionState.add(host);
+        if (isStateCorrect) {
+          hostsWithAllowedVersion.add(hostVersion);
+          hostsToUpdate.add(hostVersion.getHostName());
+        } else {
+          hostsWithoutCorrectVersionState.add(hostVersion);
         }
       }
 
       if (hostsWithoutCorrectVersionState.size() > 0) {
-        throw new AmbariException(String.format("The following host(s) have not been upgraded to version %s. " +
-                "Please install and upgrade the Stack Version on those hosts and try again.",
+        String message = String.format("The following %d host(s) have not been upgraded to version %s. " +
+                "Please install and upgrade the Stack Version on those hosts and try again.\nHosts: %s\n",
+            hostsWithoutCorrectVersionState.size(),
             version,
-            StringUtils.join(hostsWithoutCorrectVersionState, ", ")));
+            StringUtils.join(hostsWithoutCorrectVersionState, ", "));
+        outSB.append(message);
+        throw new AmbariException(message);
       }
 
-      outSB.append(String.format("Will finalize the upgraded state of all host components.\n"));
-      for (HostComponentStateEntity hostComponentStateEntity: hostComponentStateDAO.findAll()) {
-        hostComponentStateEntity.setUpgradeState(UpgradeState.NONE);
-        hostComponentStateDAO.merge(hostComponentStateEntity);
+      // Allow the cluster version to transition from UPGRADING to CURRENT
+      if (atLeastOneHostInInstalledState) {
+        cluster.transitionClusterVersion(stackId, version, RepositoryVersionState.UPGRADED);
+        upgradingClusterVersion = clusterVersionDAO.findByClusterAndStackAndVersion(clusterName,
+            stackId, version);
       }
 
-      outSB.append(String.format("Will finalize the version for %d host(s).\n", hosts.keySet().size()));
-      cluster.mapHostVersions(hosts.keySet(), upgradingClusterVersion, RepositoryVersionState.CURRENT);
+      if (!allowedStates.contains(upgradingClusterVersion.getState())) {
+        throw new AmbariException(String.format("The cluster stack version state %s is not allowed to transition directly into %s",
+            upgradingClusterVersion.getState(), RepositoryVersionState.CURRENT.toString()));
+      }
+
+      outSB.append(String.format("Will finalize the upgraded state of host components in %d host(s).\n", hostsWithAllowedVersion.size()));
+
+      for (HostVersionEntity hostVersion : hostsWithAllowedVersion) {
+        Collection<HostComponentStateEntity> hostComponentStates = hostComponentStateDAO.findByHost(hostVersion.getHostName());
+        for (HostComponentStateEntity hostComponentStateEntity: hostComponentStates) {
+          hostComponentStateEntity.setUpgradeState(UpgradeState.NONE);
+          hostComponentStateDAO.merge(hostComponentStateEntity);
+        }
+      }
+
+      outSB.append(String.format("Will finalize the version for %d host(s).\n", hostsWithAllowedVersion.size()));
+
+      // Impacts all hosts that have a version
+      cluster.mapHostVersions(hostsToUpdate, upgradingClusterVersion, RepositoryVersionState.CURRENT);
 
       outSB.append(String.format("Will finalize the version for cluster %s.\n", clusterName));
       cluster.transitionClusterVersion(stackId, version, RepositoryVersionState.CURRENT);

http://git-wip-us.apache.org/repos/asf/ambari/blob/082beafc/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 2328bed..0197c24 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
@@ -112,6 +112,7 @@ import org.apache.ambari.server.state.fsm.InvalidStateTransitionException;
 import org.apache.ambari.server.state.scheduler.RequestExecution;
 import org.apache.ambari.server.state.scheduler.RequestExecutionFactory;
 import org.apache.ambari.server.state.svccomphost.ServiceComponentHostImpl;
+import org.apache.ambari.server.state.svccomphost.ServiceComponentHostSummary;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -1330,37 +1331,20 @@ public class ClusterImpl implements Cluster {
       }
     }
 
-    final Collection<HostComponentStateEntity> allHostComponents = host.getHostComponentStateEntities();
-    final Collection<HostComponentStateEntity> versionedHostComponents = new HashSet<HostComponentStateEntity>();
-    final Collection<HostComponentStateEntity> noVersionNeededComponents = new HashSet<HostComponentStateEntity>();
-
-    for (HostComponentStateEntity hostComponentStateEntity: allHostComponents) {
-      if (!hostComponentStateEntity.getVersion().equalsIgnoreCase(State.UNKNOWN.toString())) {
-        versionedHostComponents.add(hostComponentStateEntity);
-      } else {
-        // Some Components cannot advertise a version. E.g., ZKF, AMS, Kerberos
-        ComponentInfo compInfo = ambariMetaInfo.getComponent(
-            stack.getStackName(), stack.getStackVersion(), hostComponentStateEntity.getServiceName(),
-            hostComponentStateEntity.getComponentName());
-
-        if (!compInfo.isVersionAdvertised()) {
-          noVersionNeededComponents.add(hostComponentStateEntity);
-        }
-      }
-    }
+    final ServiceComponentHostSummary hostSummary = new ServiceComponentHostSummary(ambariMetaInfo, host, stack);
+    final Collection<HostComponentStateEntity> versionedHostComponents = hostSummary.getVersionedHostComponents();
 
     // If 0 or 1 cluster version exists, then a brand new cluster permits the host to transition from UPGRADING->CURRENT
     // If multiple cluster versions exist, then it means that the change in versions is happening due to an Upgrade,
     // so should only allow transitioning to UPGRADED or UPGRADING, depending on further circumstances.
     List<ClusterVersionEntity> clusterVersions = clusterVersionDAO.findByCluster(getClusterName());
-    final int versionedPlusNoVersionNeededSize = versionedHostComponents.size() + noVersionNeededComponents.size();
     if (clusterVersions.size() <= 1) {
       // Transition from UPGRADING -> CURRENT. This is allowed because Host Version Entity is bootstrapped in an UPGRADING state.
       // This also covers hosts that do not advertise a version when the cluster was created, and then have another component added
       // that does advertise a version.
-      if (allHostComponents.size() == versionedPlusNoVersionNeededSize &&
+      if (hostSummary.haveAllComponentsFinishedAdvertisingVersion() &&
           (hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADING) || hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADED)) &&
-          ServiceComponentHostImpl.haveSameVersion(versionedHostComponents)) {
+          ServiceComponentHostSummary.haveSameVersion(versionedHostComponents)) {
         hostVersionEntity.setState(RepositoryVersionState.CURRENT);
         hostVersionDAO.merge(hostVersionEntity);
       }
@@ -1368,16 +1352,16 @@ public class ClusterImpl implements Cluster {
       // Transition from UPGRADING -> UPGRADED.
       // We should never transition directly from INSTALLED -> UPGRADED without first going to UPGRADING because
       // they belong in different phases (1. distribute bits 2. perform upgrade).
-      if (allHostComponents.size() == versionedPlusNoVersionNeededSize &&
+      if (hostSummary.haveAllComponentsFinishedAdvertisingVersion() &&
           hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADING) &&
-          ServiceComponentHostImpl.haveSameVersion(versionedHostComponents)) {
+          ServiceComponentHostSummary.haveSameVersion(versionedHostComponents)) {
         hostVersionEntity.setState(RepositoryVersionState.UPGRADED);
         hostVersionDAO.merge(hostVersionEntity);
       } else{
         // HostVersion is INSTALLED and an upgrade is in-progress because at least 2 components have different versions,
         // Or the host has no components that advertise a version, so still consider it as UPGRADING.
-        if (hostVersionEntity.getState().equals(RepositoryVersionState.INSTALLED) && versionedHostComponents.size() > 0 &&
-          !ServiceComponentHostImpl.haveSameVersion(versionedHostComponents)) {
+        if (hostVersionEntity.getState().equals(RepositoryVersionState.INSTALLED) && versionedHostComponents.size() > 0 && 
+          !ServiceComponentHostSummary.haveSameVersion(versionedHostComponents)) {
           hostVersionEntity.setState(RepositoryVersionState.UPGRADING);
           hostVersionDAO.merge(hostVersionEntity);
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/082beafc/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
index 55a6ce7..c43044c 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
@@ -1528,30 +1528,5 @@ public class ServiceComponentHostImpl implements ServiceComponentHost {
     return version;
   }
 
-  /**
-   * Checks that every component has the same version
-   *
-   * @param hostComponents host components
-   * @return true if components have the same version
-   */
-  public static boolean haveSameVersion(Collection<HostComponentStateEntity> hostComponents) {
-    if (hostComponents.isEmpty()) {
-      // should never happen
-      // but just in case: no components passed -> do not change host version
-      return false;
-    }
-    String firstVersion = null;
-    for (HostComponentStateEntity hostComponent : hostComponents) {
-      if (!hostComponent.getVersion().isEmpty()) {
-        if (firstVersion == null) {
-          firstVersion = hostComponent.getVersion();
-        } else {
-          if (!StringUtils.equals(firstVersion, hostComponent.getVersion())) {
-            return false;
-          }
-        }
-      }
-    }
-    return true;
-  }
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/082beafc/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostSummary.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostSummary.java b/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostSummary.java
new file mode 100644
index 0000000..1676b35
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostSummary.java
@@ -0,0 +1,118 @@
+/**
+ * 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.state.svccomphost;
+
+
+import com.google.inject.Inject;
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.api.services.AmbariMetaInfo;
+import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
+import org.apache.ambari.server.orm.entities.HostEntity;
+import org.apache.ambari.server.state.ComponentInfo;
+import org.apache.ambari.server.state.StackId;
+import org.apache.ambari.server.state.State;
+import org.apache.commons.lang.StringUtils;
+
+import java.util.Collection;
+import java.util.HashSet;
+
+
+/**
+ * Represents a summary of the versions of the components installed on a host.
+ */
+public class ServiceComponentHostSummary  {
+
+  private Collection<HostComponentStateEntity> allHostComponents;
+  private Collection<HostComponentStateEntity> versionedHostComponents;
+  private Collection<HostComponentStateEntity> noVersionNeededComponents;
+
+
+  public ServiceComponentHostSummary(AmbariMetaInfo ambariMetaInfo, HostEntity host, String stackName, String stackVersion) throws AmbariException {
+    allHostComponents = host.getHostComponentStateEntities();
+    versionedHostComponents = new HashSet<HostComponentStateEntity>();
+    noVersionNeededComponents = new HashSet<HostComponentStateEntity>();
+
+    for (HostComponentStateEntity hostComponentStateEntity: allHostComponents) {
+      if (!hostComponentStateEntity.getVersion().equalsIgnoreCase(State.UNKNOWN.toString())) {
+        versionedHostComponents.add(hostComponentStateEntity);
+      } else {
+        // Some Components cannot advertise a version. E.g., ZKF, AMS, Kerberos
+        ComponentInfo compInfo = ambariMetaInfo.getComponent(
+            stackName, stackVersion, hostComponentStateEntity.getServiceName(),
+            hostComponentStateEntity.getComponentName());
+
+        if (!compInfo.isVersionAdvertised()) {
+          noVersionNeededComponents.add(hostComponentStateEntity);
+        }
+      }
+    }
+  }
+
+  public ServiceComponentHostSummary(AmbariMetaInfo ambariMetaInfo, HostEntity host, StackId stackId) throws AmbariException {
+    this(ambariMetaInfo, host, stackId.getStackName(), stackId.getStackVersion());
+  }
+
+  public Collection<HostComponentStateEntity> getAllHostComponents() {
+    return allHostComponents;
+  }
+
+  public Collection<HostComponentStateEntity> getVersionedHostComponents() {
+    return versionedHostComponents;
+  }
+
+  public Collection<HostComponentStateEntity> getNoVersionNeededComponents() {
+    return noVersionNeededComponents;
+  }
+
+  /**
+   * Determine if all of the components on this host have finished advertising a version, which occurs when all of the
+   * components that advertise a version, plus the components that do not advertise a version, equal the total number
+   * of components.
+   * @return Return a bool indicating if all components that can report a version have done so.
+   */
+  public boolean haveAllComponentsFinishedAdvertisingVersion() {
+    return allHostComponents.size() == versionedHostComponents.size() + noVersionNeededComponents.size();
+  }
+
+  /**
+   * Checks that every component has the same version
+   *
+   * @param hostComponents host components
+   * @return true if components have the same version, or collection is empty, false otherwise.
+   */
+  public static boolean haveSameVersion(Collection<HostComponentStateEntity> hostComponents) {
+    // It is important to return true even if the collection is empty because technically, there are no conflicts.
+    if (hostComponents.isEmpty()) {
+      return true;
+    }
+    String firstVersion = null;
+    for (HostComponentStateEntity hostComponent : hostComponents) {
+      if (!hostComponent.getVersion().isEmpty()) {
+        if (firstVersion == null) {
+          firstVersion = hostComponent.getVersion();
+        } else {
+          if (!StringUtils.equals(firstVersion, hostComponent.getVersion())) {
+            return false;
+          }
+        }
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/082beafc/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml b/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
index 139dbdb..6388bc9 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/upgrades/upgrade-2.2.xml
@@ -75,7 +75,7 @@
 
       <batch>
         <percent>33</percent>
-        <summary>Verfication Required</summary>
+        <summary>Verification Required</summary>
         <message>Core Services have been upgraded. You are advised to perform tests against your cluster to ensure proper operation before proceeding with upgrade of remaining services.</message>
       </batch>
     </group>