Browse Source

AMBARI-9340. RU - Cluster install did not transition cluster_version to CURRENT because host has only AMS (alejandro)

Alejandro Fernandez 10 năm trước cách đây
mục cha
commit
c242beeac7
15 tập tin đã thay đổi với 351 bổ sung119 xóa
  1. 6 3
      ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
  2. 7 7
      ambari-server/src/main/java/org/apache/ambari/server/controller/StackServiceComponentResponse.java
  3. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackServiceComponentResourceProvider.java
  4. 5 1
      ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java
  5. 65 0
      ambari-server/src/main/java/org/apache/ambari/server/events/HostComponentVersionEvent.java
  6. 89 0
      ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/StackVersionListener.java
  7. 1 1
      ambari-server/src/main/java/org/apache/ambari/server/stack/ComponentModule.java
  8. 17 1
      ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java
  9. 9 8
      ambari-server/src/main/java/org/apache/ambari/server/state/ComponentInfo.java
  10. 119 10
      ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java
  11. 27 82
      ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
  12. 2 2
      ambari-server/src/main/resources/common-services/AMS/0.1.0/metainfo.xml
  13. 1 1
      ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/metainfo.xml
  14. 1 1
      ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
  15. 1 1
      ambari-server/src/main/resources/common-services/KERBEROS/1.10.3-10/metainfo.xml

+ 6 - 3
ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java

@@ -47,6 +47,7 @@ import org.apache.ambari.server.controller.MaintenanceStateHelper;
 import org.apache.ambari.server.events.ActionFinalReportReceivedEvent;
 import org.apache.ambari.server.events.AlertEvent;
 import org.apache.ambari.server.events.AlertReceivedEvent;
+import org.apache.ambari.server.events.HostComponentVersionEvent;
 import org.apache.ambari.server.events.publishers.AlertEventPublisher;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.metadata.ActionMetadata;
@@ -481,13 +482,15 @@ public class HeartBeatHandler {
                 final String previousVersion = scHost.getVersion();
                 if (!StringUtils.equals(previousVersion, structuredOutput.getVersion())) {
                   scHost.setVersion(structuredOutput.getVersion());
-                  if (previousVersion != null && !previousVersion.equals("UNKNOWN")) {
+                  if (previousVersion != null && !previousVersion.equalsIgnoreCase(State.UNKNOWN.toString())) {
                     scHost.setUpgradeState(UpgradeState.COMPLETE);
                   }
-                  String repoVersion = scHost.recalculateHostVersionState();
-                  cl.recalculateClusterVersionState(repoVersion);
                 }
               }
+              // Safer to recalculate the version even if we don't detect a difference in the value.
+              // This is useful in case that a manual database edit is done while ambari-server is stopped.
+              HostComponentVersionEvent event = new HostComponentVersionEvent(cl, scHost);
+              ambariEventPublisher.publish(event);
             }
 
             // Updating stack version, if needed

+ 7 - 7
ambari-server/src/main/java/org/apache/ambari/server/controller/StackServiceComponentResponse.java

@@ -78,7 +78,7 @@ public class StackServiceComponentResponse {
   /**
    * does the component need to advertise a version
    */
-  private boolean advertiseVersion;
+  private boolean versionAdvertised;
 
   /**
    * auto deploy information
@@ -104,7 +104,7 @@ public class StackServiceComponentResponse {
     isClient = component.isClient();
     isMaster = component.isMaster();
     cardinality = component.getCardinality();
-    advertiseVersion = component.isAdvertiseVersion();
+    versionAdvertised = component.isVersionAdvertised();
     autoDeploy = component.getAutoDeploy();
 
     // the custom command names defined for this component
@@ -288,17 +288,17 @@ public class StackServiceComponentResponse {
    *
    * @return Whether the components needs to advertise a version
    */
-  public boolean isAdvertiseVersion() {
-    return advertiseVersion;
+  public boolean isVersionAdvertised() {
+    return versionAdvertised;
   }
 
   /**
    * Set whether the component needs to advertise a version.
    *
-   * @param advertiseVersion whether the component needs to advertise a version
+   * @param versionAdvertised whether the component needs to advertise a version
    */
-  public void setAdvertiseVersion(boolean advertiseVersion) {
-    this.advertiseVersion = advertiseVersion;
+  public void setVersionAdvertised(boolean versionAdvertised) {
+    this.versionAdvertised = versionAdvertised;
   }
 
 

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackServiceComponentResourceProvider.java

@@ -139,7 +139,7 @@ public class StackServiceComponentResourceProvider extends
           response.getCardinality(), requestedIds);
 
       setResourceProperty(resource, ADVERTISE_VERSION_ID,
-          response.isAdvertiseVersion(), requestedIds);
+          response.isVersionAdvertised(), requestedIds);
 
       setResourceProperty(resource, CUSTOM_COMMANDS_PROPERTY_ID,
           response.getCustomCommands(), requestedIds);

+ 5 - 1
ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java

@@ -77,6 +77,11 @@ public abstract class AmbariEvent {
      */
     HOST_REGISTERED,
 
+    /**
+     * A host was registered with the server.
+     */
+    HOST_COMPONENT_VERSION_ADVERTISED,
+
     /**
      * A host was added to the cluster.
      */
@@ -108,7 +113,6 @@ public abstract class AmbariEvent {
    *
    * @param eventType
    *          the type of event (not {@code null}).
-   * @param clusterId
    */
   public AmbariEvent(AmbariEventType eventType) {
     m_eventType = eventType;

+ 65 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/HostComponentVersionEvent.java

@@ -0,0 +1,65 @@
+/**
+ * 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 org.apache.ambari.server.state.ServiceComponentHost;
+
+/**
+ * The {@link org.apache.ambari.server.events.HostComponentVersionEvent} represents all events in Ambari that relate
+ * to a Host Component advertising a version.
+ */
+public class HostComponentVersionEvent extends ClusterEvent {
+
+  protected Cluster cluster;
+  protected ServiceComponentHost sch;
+
+  /**
+   * Constructor.
+   *
+   * @param cluster: cluster.
+   * @param sch: the service component host
+   */
+  public HostComponentVersionEvent(Cluster cluster, ServiceComponentHost sch) {
+    super(AmbariEventType.HOST_COMPONENT_VERSION_ADVERTISED, cluster.getClusterId());
+    this.cluster = cluster;
+    this.sch = sch;
+  }
+
+  public ServiceComponentHost getServiceComponentHost() {
+    return sch;
+  }
+
+  public Cluster getCluster() {
+    return cluster;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public String toString() {
+    StringBuilder buffer = new StringBuilder("HostComponentVersionEvent{ ");
+    buffer.append("cluserId=").append(m_clusterId);
+    buffer.append(", serviceName=").append(this.sch.getServiceName());
+    buffer.append(", componentName=").append(this.sch.getServiceComponentName());
+    buffer.append(", hostName=").append(this.sch.getHostName());
+    buffer.append("}");
+    return buffer.toString();
+  }
+}

+ 89 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/StackVersionListener.java

@@ -0,0 +1,89 @@
+/**
+ * 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.listeners.upgrade;
+
+import com.google.common.eventbus.AllowConcurrentEvents;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Inject;
+import com.google.inject.Provider;
+import com.google.inject.Singleton;
+import org.apache.ambari.server.EagerSingleton;
+import org.apache.ambari.server.events.HostComponentVersionEvent;
+import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.state.ServiceComponentHost;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * The {@link StackVersionListener} class handles the propagation of versions advertised by the {@link org.apache.ambari.server.state.ServiceComponentHost}
+ * that bubble up to the {@link org.apache.ambari.server.orm.entities.HostVersionEntity} and eventually the
+ * {@link org.apache.ambari.server.orm.entities.ClusterVersionEntity}
+ */
+@Singleton
+@EagerSingleton
+public class StackVersionListener {
+  /**
+   * Logger.
+   */
+  private final static Logger LOG = LoggerFactory.getLogger(StackVersionListener.class);
+
+  private AmbariEventPublisher ambariEventPublisher;
+
+  /**
+   * Used to prevent multiple threads from trying to create host alerts
+   * simultaneously.
+   */
+  private Lock m_stackVersionLock = new ReentrantLock();
+
+  /**
+   * Constructor.
+   *
+   * @param ambariEventPublisher
+   */
+  @Inject
+  public StackVersionListener(AmbariEventPublisher ambariEventPublisher) {
+    this.ambariEventPublisher = ambariEventPublisher;
+    ambariEventPublisher.register(this);
+  }
+
+  @Subscribe
+  @AllowConcurrentEvents
+  public void onAmbariEvent(HostComponentVersionEvent event) {
+    LOG.debug("Received event {}", event);
+
+    ServiceComponentHost sch = event.getServiceComponentHost();
+
+    m_stackVersionLock.lock();
+
+    try {
+      Cluster cluster = event.getCluster();
+      String repoVersion = sch.recalculateHostVersionState();
+      cluster.recalculateClusterVersionState(repoVersion);
+    } catch (Exception e) {
+      LOG.error("Unable to propagate version for ServiceHostComponent on component: " + sch.getServiceComponentName() +
+          ", host: " + sch.getHostName() + ". Error: " + e.getMessage());
+    } finally {
+      m_stackVersionLock.unlock();
+    }
+  }
+}

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/stack/ComponentModule.java

@@ -71,7 +71,7 @@ public class ComponentModule extends BaseModule<ComponentModule, ComponentInfo>
     if (componentInfo.getCardinality() == null) {
       componentInfo.setCardinality(parentInfo.getCardinality());
     }
-    componentInfo.setAdvertiseVersion(parentInfo.isAdvertiseVersion());
+    componentInfo.setVersionAdvertised(parentInfo.isVersionAdvertised());
     if (componentInfo.getAutoDeploy() == null) {
       componentInfo.setAutoDeploy(parentInfo.getAutoDeploy());
     }

+ 17 - 1
ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java

@@ -27,8 +27,11 @@ import java.util.concurrent.locks.ReadWriteLock;
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.controller.ClusterResponse;
 import org.apache.ambari.server.controller.ServiceConfigVersionResponse;
-import org.apache.ambari.server.orm.entities.PrivilegeEntity;
 import org.apache.ambari.server.orm.entities.ClusterVersionEntity;
+import org.apache.ambari.server.orm.entities.HostEntity;
+import org.apache.ambari.server.orm.entities.HostVersionEntity;
+import org.apache.ambari.server.orm.entities.PrivilegeEntity;
+import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
 import org.apache.ambari.server.state.configgroup.ConfigGroup;
 import org.apache.ambari.server.state.scheduler.RequestExecution;
 
@@ -162,6 +165,19 @@ public interface Cluster {
    */
   void recalculateClusterVersionState(String repositoryVersion) throws AmbariException;
 
+  /**
+   * For a given host, will either either update an existing Host Version Entity for the given version, or create
+   * one if it doesn't exist. The object will be created with a state of
+   * {@link org.apache.ambari.server.state.RepositoryVersionState#UPGRADING}
+   *
+   * @param host Host Entity object
+   * @param repositoryVersion Repository Version that the host is transitioning to
+   * @param stack Stack information with the version
+   * @return Returns either the newly created or the updated Host Version Entity.
+   * @throws AmbariException
+   */
+  public HostVersionEntity transitionHostVersionState(HostEntity host, final RepositoryVersionEntity repositoryVersion, final StackId stack) throws AmbariException;
+
   /**
    * Update state of all cluster stack versions for cluster based on states of host versions.
    * May be called multiple times.

+ 9 - 8
ambari-server/src/main/java/org/apache/ambari/server/state/ComponentInfo.java

@@ -42,7 +42,8 @@ public class ComponentInfo {
    * Some components do not need to advertise a version because it is either redundant, or they don't have a mechanism
    * at the moment. For instance, ZKFC has the same version as NameNode, while AMS and KERBEROS do not have a mechanism.
    */
-  private boolean advertiseVersion = true;
+  @XmlElements(@XmlElement(name = "versionAdvertised"))
+  private boolean versionAdvertised = true;
 
   /**
   * Added at schema ver 2
@@ -99,7 +100,7 @@ public class ComponentInfo {
     category = prototype.category;
     deleted = prototype.deleted;
     cardinality = prototype.cardinality;
-    advertiseVersion = prototype.advertiseVersion;
+    versionAdvertised = prototype.versionAdvertised;
     clientsToUpdateConfigs = prototype.clientsToUpdateConfigs;
     commandScript = prototype.commandScript;
     customCommands = prototype.customCommands;
@@ -234,12 +235,12 @@ public class ComponentInfo {
     return cardinality;
   }
 
-  public void setAdvertiseVersion(boolean advertiseVersion) {
-    this.advertiseVersion = advertiseVersion;
+  public void setVersionAdvertised(boolean versionAdvertised) {
+    this.versionAdvertised = versionAdvertised;
   }
 
-  public boolean isAdvertiseVersion() {
-    return advertiseVersion;
+  public boolean isVersionAdvertised() {
+    return versionAdvertised;
   }
 
   public List<String> getClientsToUpdateConfigs() {
@@ -260,7 +261,7 @@ public class ComponentInfo {
     if (deleted != that.deleted) return false;
     if (autoDeploy != null ? !autoDeploy.equals(that.autoDeploy) : that.autoDeploy != null) return false;
     if (cardinality != null ? !cardinality.equals(that.cardinality) : that.cardinality != null) return false;
-    if (advertiseVersion != that.advertiseVersion) return false;
+    if (versionAdvertised != that.versionAdvertised) return false;
     if (category != null ? !category.equals(that.category) : that.category != null) return false;
     if (clientConfigFiles != null ? !clientConfigFiles.equals(that.clientConfigFiles) : that.clientConfigFiles != null)
       return false;
@@ -285,7 +286,7 @@ public class ComponentInfo {
     result = 31 * result + (category != null ? category.hashCode() : 0);
     result = 31 * result + (deleted ? 1 : 0);
     result = 31 * result + (cardinality != null ? cardinality.hashCode() : 0);
-    result = 31 * result + (advertiseVersion ? 1 : 0);
+    result = 31 * result + (versionAdvertised ? 1 : 0);
     result = 31 * result + (commandScript != null ? commandScript.hashCode() : 0);
     result = 31 * result + (clientConfigFiles != null ? clientConfigFiles.hashCode() : 0);
     result = 31 * result + (customCommands != null ? customCommands.hashCode() : 0);

+ 119 - 10
ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java

@@ -68,10 +68,11 @@ import org.apache.ambari.server.orm.dao.UpgradeDAO;
 import org.apache.ambari.server.orm.entities.ClusterConfigEntity;
 import org.apache.ambari.server.orm.entities.ClusterConfigMappingEntity;
 import org.apache.ambari.server.orm.entities.ClusterEntity;
-import org.apache.ambari.server.orm.entities.ClusterVersionEntity;
 import org.apache.ambari.server.orm.entities.ClusterServiceEntity;
 import org.apache.ambari.server.orm.entities.ClusterStateEntity;
+import org.apache.ambari.server.orm.entities.ClusterVersionEntity;
 import org.apache.ambari.server.orm.entities.ConfigGroupEntity;
+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.orm.entities.PermissionEntity;
@@ -85,8 +86,7 @@ import org.apache.ambari.server.state.Alert;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.ClusterHealthReport;
 import org.apache.ambari.server.state.Clusters;
-import org.apache.ambari.server.state.HostState;
-import org.apache.ambari.server.state.RepositoryVersionState;
+import org.apache.ambari.server.state.ComponentInfo;
 import org.apache.ambari.server.state.Config;
 import org.apache.ambari.server.state.ConfigFactory;
 import org.apache.ambari.server.state.ConfigHelper;
@@ -94,8 +94,10 @@ import org.apache.ambari.server.state.ConfigVersionHelper;
 import org.apache.ambari.server.state.DesiredConfig;
 import org.apache.ambari.server.state.Host;
 import org.apache.ambari.server.state.HostHealthStatus;
+import org.apache.ambari.server.state.HostState;
 import org.apache.ambari.server.state.MaintenanceState;
 import org.apache.ambari.server.state.PropertyInfo;
+import org.apache.ambari.server.state.RepositoryVersionState;
 import org.apache.ambari.server.state.SecurityType;
 import org.apache.ambari.server.state.Service;
 import org.apache.ambari.server.state.ServiceComponent;
@@ -111,6 +113,7 @@ import org.apache.ambari.server.state.configgroup.ConfigGroupFactory;
 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.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -1214,6 +1217,7 @@ public class ClusterImpl implements Cluster {
           // Notice that if any hosts already have the desired stack and version, regardless of the state, we try
           // to be robust and only insert records for the missing hosts.
           if (!intersection.contains(hostname)) {
+            // According to the business logic, we don't create objects in a CURRENT state.
             HostEntity hostEntity = hostDAO.findByName(hostname);
             HostVersionEntity hostVersionEntity = new HostVersionEntity(hostname, currentClusterVersion.getRepositoryVersion(), desiredState);
             hostVersionEntity.setHostEntity(hostEntity);
@@ -1242,7 +1246,7 @@ public class ClusterImpl implements Cluster {
 
   /**
    * Because this is a top-down approach, it should only be called for the purposes of bootstrapping data, such as
-   * installing a brand new cluster or initiating an Upgrade.
+   * installing a brand new cluster (through Blueprints).
    * @param sourceClusterVersion cluster version to be queried for a stack name/version info and desired RepositoryVersionState.
    * The only valid state of this cluster version is {@link RepositoryVersionState#INSTALLING}
    * @throws AmbariException
@@ -1385,7 +1389,6 @@ public class ClusterImpl implements Cluster {
         }
 
         // Ignore if cluster version is CURRENT or UPGRADE_FAILED
-        RepositoryVersionState lowestPriorityState;
         if (clusterVersion.getState() != RepositoryVersionState.INSTALL_FAILED &&
                 clusterVersion.getState() != RepositoryVersionState.OUT_OF_SYNC &&
                 clusterVersion.getState() != RepositoryVersionState.INSTALLING &&
@@ -1399,16 +1402,16 @@ public class ClusterImpl implements Cluster {
         // Part 2, check for transitions.
         Map<String, Host> hosts = clusters.getHostsForCluster(this.getClusterName());
 
+        Set<Host> hostsWithoutHostVersion = new HashSet<Host>();
         Map<RepositoryVersionState, Set<String>> stateToHosts = new HashMap<RepositoryVersionState, Set<String>>();
         for (Host host : hosts.values()) {
           String hostName = host.getHostName();
           HostVersionEntity hostVersion = hostVersionDAO.findByClusterStackVersionAndHost(this.getClusterName(), stackId.getStackId(), repositoryVersion, hostName);
           if (hostVersion == null) {
-            // It is likely that the services are still being installed on the cluster, so not all agents
-            // have had a chance to heartbeat and insert their own Host Version as part of recalculateHostVersionState().
-            // Therefore, we should not allow any transitions yet.
-            LOG.debug("Skipping transitioning the cluster version because host " + hostName + " does not have a version yet.");
-            return;
+            // This host either has not had a chance to heartbeat yet with its installed component, or it has components
+            // that do not advertise a version.
+            hostsWithoutHostVersion.add(host);
+            continue;
           }
 
           RepositoryVersionState hostState = hostVersion.getState();
@@ -1426,6 +1429,27 @@ public class ClusterImpl implements Cluster {
           }
         }
 
+        // Ensure that all of the hosts without a Host Version only have Components that do not advertise a version.
+        // Otherwise, operations are still in progress.
+        for (Host host : hostsWithoutHostVersion) {
+          HostEntity hostEntity = hostDAO.findByName(host.getHostName());
+          final Collection<HostComponentStateEntity> allHostComponents = hostEntity.getHostComponentStateEntities();
+
+          for (HostComponentStateEntity hostComponentStateEntity: allHostComponents) {
+            if (hostComponentStateEntity.getVersion().equalsIgnoreCase(State.UNKNOWN.toString())) {
+              // Some Components cannot advertise a version. E.g., ZKF, AMS, Kerberos
+              ComponentInfo compInfo = ambariMetaInfo.getComponent(
+                  stackId.getStackName(), stackId.getStackVersion(), hostComponentStateEntity.getServiceName(),
+                  hostComponentStateEntity.getComponentName());
+
+              if (compInfo.isVersionAdvertised()) {
+                LOG.debug("Skipping transitioning the cluster version because host " + host.getHostName() + " does not have a version yet.");
+                return;
+              }
+            }
+          }
+        }
+        
         RepositoryVersionState effectiveClusterVersionState = getEffectiveState(stateToHosts);
         if (effectiveClusterVersionState != null && effectiveClusterVersionState != clusterVersion.getState()) {
           // Any mismatch will be caught while transitioning, and raise an exception.
@@ -1443,6 +1467,91 @@ public class ClusterImpl implements Cluster {
     }
   }
 
+  /**
+   * Transition the Host Version across states.
+   * If a Host Component has a valid version, then create a Host Version if it does not already exist.
+   * Pre-req is for a Repository Version to exist.
+   * If no no cluster version exists, or exactly one, then potentially transition to CURRENT when no more work is needed.
+   * If in the middle of an upgrade and no more work exists, transition to UPGRADED.
+   * Otherwise, if a mismatch of versions exist, transition to UPGRADING since in the middle of an upgrade.
+   * @param host
+   * @param repositoryVersion
+   * @param stack
+   * @throws AmbariException
+   */
+  @Transactional
+  public HostVersionEntity transitionHostVersionState(HostEntity host, final RepositoryVersionEntity repositoryVersion, final StackId stack) throws AmbariException {
+    HostVersionEntity hostVersionEntity = null;
+    List<HostVersionEntity> hostVersions = hostVersionDAO.findByHost(host.getHostName());
+    if (hostVersions == null || hostVersions.isEmpty()) {
+      // Since the host has no versions, allow bootstrapping a version for it.
+      hostVersionEntity = new HostVersionEntity(host.getHostName(), repositoryVersion, RepositoryVersionState.UPGRADING);
+      hostVersionEntity.setHostEntity(host);
+      hostVersionDAO.create(hostVersionEntity);
+    } else {
+      hostVersionEntity = hostVersionDAO.findByClusterStackVersionAndHost(this.getClusterName(), repositoryVersion.getStack(), repositoryVersion.getVersion(), host.getHostName());
+      if (hostVersionEntity == null) {
+        throw new AmbariException("Host " + host.getHostName() + " is expected to have a Host Version for stack " + repositoryVersion.getStackVersion());
+      }
+    }
+
+    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);
+        }
+      }
+    }
+
+    // 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(this.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 &&
+          (hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADING) || hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADED)) &&
+          ServiceComponentHostImpl.haveSameVersion(versionedHostComponents)) {
+        hostVersionEntity.setState(RepositoryVersionState.CURRENT);
+        hostVersionDAO.merge(hostVersionEntity);
+      }
+    } else {
+      // 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 &&
+          hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADING) &&
+          ServiceComponentHostImpl.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)) {
+          hostVersionEntity.setState(RepositoryVersionState.UPGRADING);
+          hostVersionDAO.merge(hostVersionEntity);
+        }
+      }
+    }
+
+    return hostVersionEntity;
+  }
+
   @Override
   public void recalculateAllClusterVersionStates() throws AmbariException {
     clusterGlobalLock.readLock().lock();

+ 27 - 82
ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java

@@ -711,7 +711,7 @@ public class ServiceComponentHostImpl implements ServiceComponentHost {
     stateEntity.setClusterId(serviceComponent.getClusterId());
     stateEntity.setComponentName(serviceComponent.getName());
     stateEntity.setServiceName(serviceComponent.getServiceName());
-    stateEntity.setVersion("UNKNOWN");
+    stateEntity.setVersion(State.UNKNOWN.toString());
     stateEntity.setHostName(hostName);
     stateEntity.setCurrentState(stateMachine.getCurrentState());
     stateEntity.setUpgradeState(UpgradeState.NONE);
@@ -1718,101 +1718,46 @@ public class ServiceComponentHostImpl implements ServiceComponentHost {
   }
 
   @Transactional
+  private RepositoryVersionEntity createRepositoryVersion(String version, final StackId stackId, final StackInfo stackInfo) throws AmbariException {
+    // During an Ambari Upgrade from 1.7.0 -> 2.0.0, the Repo Version will not exist, so bootstrap it.
+    LOG.info("Creating new repository version " + stackId.getStackName() + "-" + version);
+    RepositoryVersionEntity repositoryVersion = repositoryVersionDAO.create(stackId.getStackId(), version, stackId.getStackName() + "-" + version,
+        repositoryVersionHelper.getUpgradePackageNameSafe(stackId.getStackName(), stackId.getStackVersion(), version),
+        repositoryVersionHelper.serializeOperatingSystems(stackInfo.getRepositories()));
+    return repositoryVersion;
+  }
+
+  /**
+   * Bootstrap any Repo Version, and potentially transition the Host Version across states.
+   * If a Host Component has a valid version, then create a Host Version if it does not already exist.
+   * If a Host Component does not have a version, return right away because no information is known.
+   * @return Return the version
+   * @throws AmbariException
+   */
   @Override
   public String recalculateHostVersionState() throws AmbariException {
-    final String version = getVersion();
-    if (version.equals("UNKNOWN")) {
-      // recalculate only if some particular version is set
+    String version = getVersion();
+    if (version == null || version.isEmpty() || version.equalsIgnoreCase(State.UNKNOWN.toString())) {
+      // Recalculate only if some particular version is set
       return null;
     }
 
     final String hostName = getHostName();
-    final HostEntity host = hostDAO.findByName(hostName);
     final Set<Cluster> clustersForHost = clusters.getClustersForHost(hostName);
     if (clustersForHost.size() != 1) {
       throw new AmbariException("Host " + hostName + " should be assigned only to one cluster");
     }
     final Cluster cluster = clustersForHost.iterator().next();
-    final StackId stack = cluster.getDesiredStackVersion();
-    final StackInfo stackInfo = ambariMetaInfo.getStack(stack.getStackName(), stack.getStackVersion());
+    final StackId stackId = cluster.getDesiredStackVersion();
+    final StackInfo stackInfo = ambariMetaInfo.getStack(stackId.getStackName(), stackId.getStackVersion());
 
-    RepositoryVersionEntity repositoryVersion = repositoryVersionDAO.findByStackAndVersion(stack.getStackId(), version);
-
-    // During an Ambari Upgrade from 1.7.0 -> 2.0.0, the Repo Version will not exist, so bootstrap it.
+    RepositoryVersionEntity repositoryVersion = repositoryVersionDAO.findByStackAndVersion(stackId.getStackId(), version);
     if (repositoryVersion == null) {
-      LOG.info("Creating new repository version " + stack.getStackName() + "-" + version);
-      repositoryVersion = repositoryVersionDAO.create(stack.getStackId(), version, stack.getStackName() + "-" + version,
-          repositoryVersionHelper.getUpgradePackageNameSafe(stack.getStackName(), stack.getStackVersion(), version),
-          repositoryVersionHelper.serializeOperatingSystems(stackInfo.getRepositories()));
-    }
-
-    HostVersionEntity hostVersionEntity = null;
-    List<HostVersionEntity> hostVersions = hostVersionDAO.findByHost(hostName);
-    if (hostVersions == null || hostVersions.isEmpty()) {
-      // Since the host has no versions, allow bootstrapping a version for it.
-      hostVersionEntity = new HostVersionEntity(hostName, repositoryVersion, RepositoryVersionState.UPGRADING);
-      hostVersionEntity.setHostEntity(host);
-      hostVersionDAO.create(hostVersionEntity);
-    } else {
-      hostVersionEntity = hostVersionDAO.findByClusterStackVersionAndHost(cluster.getClusterName(), repositoryVersion.getStack(), repositoryVersion.getVersion(), hostName);
-      if (hostVersionEntity == null) {
-        throw new AmbariException("Host " + hostName + " is expected to have a Host Version for stack " + stack.getStackId());
-      }
-    }
-
-    final Collection<HostComponentStateEntity> allHostComponents = host.getHostComponentStateEntities();
-    final Collection<HostComponentStateEntity> versionedHostComponents = new HashSet<HostComponentStateEntity>();
-
-    for (HostComponentStateEntity hostComponentStateEntity: allHostComponents) {
-      if (!hostComponentStateEntity.getVersion().equalsIgnoreCase("UNKNOWN")) {
-        versionedHostComponents.add(hostComponentStateEntity);
-      }
-    }
-
-    final Collection<HostComponentStateEntity> noVersionNeededComponents = new HashSet<HostComponentStateEntity>();
-    final Collection<HostComponentStateEntity> nonVersionedHostComponents = CollectionUtils.subtract(allHostComponents, versionedHostComponents);
-    for (HostComponentStateEntity hostComponentStateEntity: nonVersionedHostComponents) {
-      // 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.isAdvertiseVersion()) {
-        noVersionNeededComponents.add(hostComponentStateEntity);
-      }
+      repositoryVersion = createRepositoryVersion(version, stackId, stackInfo);
     }
 
-    // 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, dependending on further circumstances.
-    List<ClusterVersionEntity> clusterVersions = clusterVersionDAO.findByCluster(cluster.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.
-      if (allHostComponents.size() == versionedPlusNoVersionNeededSize &&
-          (hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADING) || hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADED)) &&
-          haveSameVersion(versionedHostComponents)) {
-        hostVersionEntity.setState(RepositoryVersionState.CURRENT);
-        hostVersionDAO.merge(hostVersionEntity);
-      }
-    } else {
-      // 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 &&
-          hostVersionEntity.getState().equals(RepositoryVersionState.UPGRADING) &&
-          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
-        if (hostVersionEntity.getState().equals(RepositoryVersionState.INSTALLED) && versionedHostComponents.size() > 0 &&
-            !haveSameVersion(versionedHostComponents)) {
-          hostVersionEntity.setState(RepositoryVersionState.UPGRADING);
-          hostVersionDAO.merge(hostVersionEntity);
-        }
-      }
-    }
+    final HostEntity host = hostDAO.findByName(hostName);
+    cluster.transitionHostVersionState(host, repositoryVersion, stackId);
     return version;
   }
 
@@ -1822,7 +1767,7 @@ public class ServiceComponentHostImpl implements ServiceComponentHost {
    * @param hostComponents host components
    * @return true if components have the same version
    */
-  private boolean haveSameVersion(Collection<HostComponentStateEntity> hostComponents) {
+  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

+ 2 - 2
ambari-server/src/main/resources/common-services/AMS/0.1.0/metainfo.xml

@@ -32,7 +32,7 @@
           <displayName>Metric Collector</displayName>
           <category>MASTER</category>
           <cardinality>1</cardinality>
-          <advertiseVersion>false</advertiseVersion>
+          <versionAdvertised>false</versionAdvertised>
           <dependencies>
             <dependency>
               <name>ZOOKEEPER/ZOOKEEPER_SERVER</name>
@@ -53,7 +53,7 @@
           <displayName>Metric Monitor</displayName>
           <category>SLAVE</category>
           <cardinality>ALL</cardinality>
-          <advertiseVersion>false</advertiseVersion>
+          <versionAdvertised>false</versionAdvertised>
           <auto-deploy>
             <enabled>true</enabled>
           </auto-deploy>

+ 1 - 1
ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/metainfo.xml

@@ -141,7 +141,7 @@
           <category>SLAVE</category>
           <!-- TODO: cardinality is conditional on HA topology -->
           <cardinality>0+</cardinality>
-          <advertiseVersion>false</advertiseVersion>
+          <versionAdvertised>false</versionAdvertised>
           <commandScript>
             <script>scripts/zkfc_slave.py</script>
             <scriptType>PYTHON</scriptType>

+ 1 - 1
ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml

@@ -150,7 +150,7 @@
           <displayName>MySQL Server</displayName>
           <category>MASTER</category>
           <cardinality>0-1</cardinality>
-          <advertiseVersion>false</advertiseVersion>
+          <versionAdvertised>false</versionAdvertised>
           <clientsToUpdateConfigs></clientsToUpdateConfigs>
           <commandScript>
             <script>scripts/mysql_server.py</script>

+ 1 - 1
ambari-server/src/main/resources/common-services/KERBEROS/1.10.3-10/metainfo.xml

@@ -34,7 +34,7 @@
           <displayName>Kerberos Client</displayName>
           <category>CLIENT</category>
           <cardinality>ALL</cardinality>
-          <advertiseVersion>false</advertiseVersion>
+          <versionAdvertised>false</versionAdvertised>
           <auto-deploy>
             <enabled>true</enabled>
           </auto-deploy>