소스 검색

AMBARI-9102. Refactor rolling upgrades prerequisite checks to expose and repackage utility classes (rlevas)

Robert Levas 10 년 전
부모
커밋
7e691292ad
18개의 변경된 파일876개의 추가작업 그리고 583개의 파일을 삭제
  1. 86 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/AbstractCheckDescriptor.java
  2. 60 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/HostsHeartbeatCheck.java
  3. 73 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/HostsMasterMaintenanceCheck.java
  4. 72 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/HostsRepositoryVersionCheck.java
  5. 62 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesDecommissionCheck.java
  6. 72 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesJobsDistributedCacheCheck.java
  7. 59 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesMaintenanceModeCheck.java
  8. 63 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesNamenodeHighAvailabilityCheck.java
  9. 62 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesUpCheck.java
  10. 70 0
      ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesYarnWorkPreservingCheck.java
  11. 4 4
      ambari-server/src/main/java/org/apache/ambari/server/controller/PrereqCheckRequest.java
  12. 54 14
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/PreUpgradeCheckResourceProvider.java
  13. 72 0
      ambari-server/src/main/java/org/apache/ambari/server/state/CheckHelper.java
  14. 0 497
      ambari-server/src/main/java/org/apache/ambari/server/state/UpgradeCheckHelper.java
  15. 4 4
      ambari-server/src/main/java/org/apache/ambari/server/state/stack/PrereqCheckStatus.java
  16. 4 4
      ambari-server/src/main/java/org/apache/ambari/server/state/stack/PrereqCheckType.java
  17. 10 10
      ambari-server/src/main/java/org/apache/ambari/server/state/stack/PrerequisiteCheck.java
  18. 49 50
      ambari-server/src/test/java/org/apache/ambari/server/state/CheckHelperTest.java

+ 86 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/AbstractCheckDescriptor.java

@@ -0,0 +1,86 @@
+/*
+ * 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.checks;
+
+import com.google.inject.Inject;
+import com.google.inject.Provider;
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.configuration.Configuration;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.orm.dao.HostVersionDAO;
+import org.apache.ambari.server.orm.dao.RepositoryVersionDAO;
+import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+
+/**
+ * Describes prerequisite check.
+ */
+public abstract class AbstractCheckDescriptor {
+
+  public final String id;
+  public final String description;
+  public final PrereqCheckType type;
+
+  @Inject
+  Provider<Clusters> clustersProvider;
+
+  @Inject
+  Provider<Configuration> configurationProvider;
+
+  @Inject
+  Provider<HostVersionDAO> hostVersionDaoProvider;
+
+  @Inject
+  Provider<RepositoryVersionDAO> repositoryVersionDaoProvider;
+
+  /**
+   * Constructor.
+   *
+   * @param id unique identifier
+   * @param type type
+   * @param description description
+   */
+  public AbstractCheckDescriptor(String id, PrereqCheckType type, String description) {
+    this.id = id;
+    this.type = type;
+    this.description = description;
+  }
+
+  /**
+   * Tests if the prerequisite check is applicable to given cluster. By default returns true.
+   *
+   * @param request prerequisite check request
+   * @return true if check should be performed
+   *
+   * @throws org.apache.ambari.server.AmbariException if server error happens
+   */
+  public boolean isApplicable(PrereqCheckRequest request) throws AmbariException {
+    return true;
+  }
+
+  /**
+   * Executes check against given cluster.
+   *
+   * @param prerequisiteCheck dto for upgrade check results
+   * @param request pre upgrade check request
+   *
+   * @throws AmbariException if server error happens
+   */
+  public abstract void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException;
+}

+ 60 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/HostsHeartbeatCheck.java

@@ -0,0 +1,60 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Host;
+import org.apache.ambari.server.state.HostHealthStatus;
+import org.apache.ambari.server.state.MaintenanceState;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+
+import java.util.Map;
+
+/**
+ * Checks that all hosts are either in maintenance mode or heartbeating with the server.
+ */
+public class HostsHeartbeatCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public HostsHeartbeatCheck() {
+    super("HOSTS_HEARTBEAT", PrereqCheckType.HOST, "All hosts must be heartbeating with the server unless they are in Maintenance Mode");
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    final Map<String, Host> clusterHosts = clustersProvider.get().getHostsForCluster(clusterName);
+    for (Map.Entry<String, Host> hostEntry : clusterHosts.entrySet()) {
+      final Host host = hostEntry.getValue();
+      if (host.getHealthStatus().getHealthStatus() == HostHealthStatus.HealthStatus.UNKNOWN && host.getMaintenanceState(cluster.getClusterId()) == MaintenanceState.OFF) {
+        prerequisiteCheck.getFailedOn().add(host.getHostName());
+      }
+    }
+    if (!prerequisiteCheck.getFailedOn().isEmpty()) {
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("Some hosts are not heartbeating with the server");
+    }
+  }
+}

+ 73 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/HostsMasterMaintenanceCheck.java

@@ -0,0 +1,73 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.stack.HostsType;
+import org.apache.ambari.server.stack.MasterHostResolver;
+import org.apache.ambari.server.state.*;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Checks that all hosts in maintenance state do not have master components.
+ */
+public class HostsMasterMaintenanceCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public HostsMasterMaintenanceCheck() {
+    super("HOSTS_MASTER_MAINTENANCE", PrereqCheckType.HOST, "Hosts in Maintenance Mode must not have any master components");
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    final MasterHostResolver masterHostResolver = new MasterHostResolver(cluster);
+    final Set<String> hostsWithMasterComponent = new HashSet<String>();
+    for (Map.Entry<String, Service> serviceEntry: cluster.getServices().entrySet()) {
+      final Service service = serviceEntry.getValue();
+      for (Map.Entry<String, ServiceComponent> serviceComponentEntry: service.getServiceComponents().entrySet()) {
+        final ServiceComponent serviceComponent = serviceComponentEntry.getValue();
+        final HostsType hostsType = masterHostResolver.getMasterAndHosts(service.getName(), serviceComponent.getName());
+        if (hostsType != null && hostsType.master != null) {
+          hostsWithMasterComponent.add(hostsType.master);
+        }
+      }
+    }
+    final Map<String, Host> clusterHosts = clustersProvider.get().getHostsForCluster(clusterName);
+    for (Map.Entry<String, Host> hostEntry : clusterHosts.entrySet()) {
+      final Host host = hostEntry.getValue();
+      if (host.getMaintenanceState(cluster.getClusterId()) == MaintenanceState.ON && hostsWithMasterComponent.contains(host.getHostName())) {
+        prerequisiteCheck.getFailedOn().add(host.getHostName());
+      }
+    }
+    if (!prerequisiteCheck.getFailedOn().isEmpty()) {
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("Some hosts with master components are in Maintenance Mode");
+    }
+  }
+}

+ 72 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/HostsRepositoryVersionCheck.java

@@ -0,0 +1,72 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.orm.entities.HostVersionEntity;
+import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Host;
+import org.apache.ambari.server.state.MaintenanceState;
+import org.apache.ambari.server.state.RepositoryVersionState;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+
+import java.util.Map;
+
+/**
+ * Checks that all hosts have particular repository version.
+ */
+public class HostsRepositoryVersionCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public HostsRepositoryVersionCheck() {
+    super("HOSTS_REPOSITORY_VERSION", PrereqCheckType.HOST, "Hosts should have the new repository version installed");
+  }
+
+  @Override
+  public boolean isApplicable(PrereqCheckRequest request) throws AmbariException {
+    return request.getRepositoryVersion() != null;
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    final Map<String, Host> clusterHosts = clustersProvider.get().getHostsForCluster(clusterName);
+    for (Map.Entry<String, Host> hostEntry : clusterHosts.entrySet()) {
+      final Host host = hostEntry.getValue();
+      if (host.getMaintenanceState(cluster.getClusterId()) == MaintenanceState.OFF) {
+        final RepositoryVersionEntity repositoryVersion = repositoryVersionDaoProvider.get().findByDisplayName(request.getRepositoryVersion());
+        final HostVersionEntity hostVersion = hostVersionDaoProvider.get().findByClusterStackVersionAndHost(clusterName, repositoryVersion.getStack(), repositoryVersion.getVersion(), host.getHostName());
+        if (hostVersion == null || hostVersion.getState() != RepositoryVersionState.INSTALLED) {
+          prerequisiteCheck.getFailedOn().add(host.getHostName());
+        }
+      }
+    }
+    if (!prerequisiteCheck.getFailedOn().isEmpty()) {
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("Some hosts do not have repository version " + request.getRepositoryVersion() + " installed");
+    }
+
+  }
+}

+ 62 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesDecommissionCheck.java

@@ -0,0 +1,62 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.*;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+
+import java.util.Map;
+
+/**
+ * Checks that there are no services in decommission state.
+ */
+public class ServicesDecommissionCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public ServicesDecommissionCheck() {
+    super("SERVICES_DECOMMISSION", PrereqCheckType.SERVICE, "Services should not be in Decommission state");
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    for (Map.Entry<String, Service> serviceEntry: cluster.getServices().entrySet()) {
+      final Service service = serviceEntry.getValue();
+      for (Map.Entry<String, ServiceComponent> serviceComponentEntry: service.getServiceComponents().entrySet()) {
+        final ServiceComponent serviceComponent = serviceComponentEntry.getValue();
+        for (String hostName : serviceComponent.getServiceComponentHosts().keySet()) {
+          final ServiceComponentHost scHost = serviceComponent.getServiceComponentHost(hostName);
+          if (scHost.getComponentAdminState() == HostComponentAdminState.DECOMMISSIONED || scHost.getComponentAdminState() == HostComponentAdminState.DECOMMISSIONING) {
+            prerequisiteCheck.getFailedOn().add(serviceComponent.getName());
+          }
+        }
+      }
+    }
+    if (!prerequisiteCheck.getFailedOn().isEmpty()) {
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("There are services in decommission or decommissioning state");
+    }
+  }
+}

+ 72 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesJobsDistributedCacheCheck.java

@@ -0,0 +1,72 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.ServiceNotFoundException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Config;
+import org.apache.ambari.server.state.DesiredConfig;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+
+import java.util.Map;
+
+/**
+ * Checks that MR, Oozie and Tez jobs reference hadoop libraries from the distributed cache.
+ */
+public class ServicesJobsDistributedCacheCheck extends AbstractCheckDescriptor {
+
+  @Override
+  public boolean isApplicable(PrereqCheckRequest request)
+    throws AmbariException {
+    final Cluster cluster = clustersProvider.get().getCluster(request.getClusterName());
+    try {
+      cluster.getService("YARN");
+    } catch (ServiceNotFoundException ex) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Constructor.
+   */
+  public ServicesJobsDistributedCacheCheck() {
+    super("SERVICES_JOBS_DISTRIBUTED_CACHE", PrereqCheckType.SERVICE, "Jobs should reference hadoop libraries from the distributed cache");
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    final String configType = "mapred-site";
+    final Map<String, DesiredConfig> desiredConfigs = cluster.getDesiredConfigs();
+    final DesiredConfig desiredConfig = desiredConfigs.get(configType);
+    final Config config = cluster.getConfig(configType, desiredConfig.getTag());
+    if (!config.getProperties().containsKey("mapreduce.application.framework.path") || !config.getProperties().containsKey("mapreduce.application.classpath")) {
+      // TODO actually it is needed to validate that these properties contain proper values but the tickets for these changes are still open, so it will cause
+      // prerequisite checks to fail
+      prerequisiteCheck.getFailedOn().add("MR");
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("mapreduce.application.framework.path and mapreduce.application.classpath should reference distributed cache");
+    }
+  }
+}

+ 59 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesMaintenanceModeCheck.java

@@ -0,0 +1,59 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.MaintenanceState;
+import org.apache.ambari.server.state.Service;
+import org.apache.ambari.server.state.State;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+
+import java.util.Map;
+
+/**
+ * Checks that services are in the maintenance mode.
+ */
+public class ServicesMaintenanceModeCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public ServicesMaintenanceModeCheck() {
+    super("SERVICES_MAINTENANCE_MODE", PrereqCheckType.SERVICE, "No service can be in maintenance mode");
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    for (Map.Entry<String, Service> serviceEntry : cluster.getServices().entrySet()) {
+      final Service service = serviceEntry.getValue();
+      if (service.getDesiredState() != State.STARTED || service.getMaintenanceState() == MaintenanceState.ON) {
+        prerequisiteCheck.getFailedOn().add(service.getName());
+      }
+    }
+    if (!prerequisiteCheck.getFailedOn().isEmpty()) {
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("Some services are in Maintenance Mode");
+    }
+  }
+}

+ 63 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesNamenodeHighAvailabilityCheck.java

@@ -0,0 +1,63 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Config;
+import org.apache.ambari.server.state.DesiredConfig;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+
+import java.util.Map;
+
+/**
+ * Checks that namenode high availability is enabled.
+ */
+public class ServicesNamenodeHighAvailabilityCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public ServicesNamenodeHighAvailabilityCheck() {
+    super("SERVICES_NAMENODE_HA", PrereqCheckType.SERVICE, "Namenode high availability should be enabled");
+  }
+
+  @Override
+  public boolean isApplicable(PrereqCheckRequest request) throws AmbariException {
+    final Cluster cluster = clustersProvider.get().getCluster(request.getClusterName());
+    return cluster.getService("HDFS") != null;
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    final String configType = "hdfs-site";
+    final Map<String, DesiredConfig> desiredConfigs = cluster.getDesiredConfigs();
+    final DesiredConfig desiredConfig = desiredConfigs.get(configType);
+    final Config config = cluster.getConfig(configType, desiredConfig.getTag());
+    if (!config.getProperties().containsKey("dfs.nameservices")) {
+      prerequisiteCheck.getFailedOn().add("HDFS");
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("Namenode high availability is disabled");
+    }
+  }
+}

+ 62 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesUpCheck.java

@@ -0,0 +1,62 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Service;
+import org.apache.ambari.server.state.State;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+
+import java.util.Map;
+
+/**
+ * Checks that services are up.
+ */
+public class ServicesUpCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public ServicesUpCheck() {
+    super("SERVICES_UP", PrereqCheckType.SERVICE, "All services must be up");
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    for (Map.Entry<String, Service> serviceEntry : cluster.getServices().entrySet()) {
+
+      final Service service = serviceEntry.getValue();
+
+      if (!service.isClientOnlyService() && service.getDesiredState() != State.STARTED) {
+        prerequisiteCheck.getFailedOn().add(service.getName());
+      }
+    }
+
+    if (!prerequisiteCheck.getFailedOn().isEmpty()) {
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("Some services are down");
+    }
+  }
+}

+ 70 - 0
ambari-server/src/main/java/org/apache/ambari/server/checks/ServicesYarnWorkPreservingCheck.java

@@ -0,0 +1,70 @@
+/*
+ * 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.checks;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.ServiceNotFoundException;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Config;
+import org.apache.ambari.server.state.DesiredConfig;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.apache.ambari.server.state.stack.PrereqCheckType;
+
+import java.util.Map;
+
+/**
+ * Checks that YARN has work-preserving restart enabled.
+ */
+public class ServicesYarnWorkPreservingCheck extends AbstractCheckDescriptor {
+
+  /**
+   * Constructor.
+   */
+  public ServicesYarnWorkPreservingCheck() {
+    super("SERVICES_YARN_WP", PrereqCheckType.SERVICE, "YARN work preserving restart should be enabled");
+  }
+
+  @Override
+  public boolean isApplicable(PrereqCheckRequest request) throws AmbariException {
+    final Cluster cluster = clustersProvider.get().getCluster(request.getClusterName());
+    try {
+      cluster.getService("YARN");
+    } catch (ServiceNotFoundException ex) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void perform(PrerequisiteCheck prerequisiteCheck, PrereqCheckRequest request) throws AmbariException {
+    final String clusterName = request.getClusterName();
+    final Cluster cluster = clustersProvider.get().getCluster(clusterName);
+    final String configType = "yarn-site";
+    final Map<String, DesiredConfig> desiredConfigs = cluster.getDesiredConfigs();
+    final DesiredConfig desiredConfig = desiredConfigs.get(configType);
+    final Config config = cluster.getConfig(configType, desiredConfig.getTag());
+    if (!config.getProperties().containsKey("yarn.resourcemanager.work-preserving-recovery.enabled") ||
+      !Boolean.getBoolean(config.getProperties().get("yarn.resourcemanager.work-preserving-recovery.enabled"))) {
+      prerequisiteCheck.getFailedOn().add("YARN");
+      prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+      prerequisiteCheck.setFailReason("YARN doesn't have work preserving restart, yarn.resourcemanager.work-preserving-recovery.enabled property is missing");
+    }
+  }
+}

+ 4 - 4
ambari-server/src/main/java/org/apache/ambari/server/controller/PreUpgradeCheckRequest.java → ambari-server/src/main/java/org/apache/ambari/server/controller/PrereqCheckRequest.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,14 +18,14 @@
 package org.apache.ambari.server.controller;
 
 /**
- * Represents a pre upgrade request.
+ * Represents a prerequisite check request.
  */
-public class PreUpgradeCheckRequest {
+public class PrereqCheckRequest {
   private final String clusterName;
   private String repositoryVersion;
 
   //TODO make repositoryVersionName also final as soon as UI will be changed to always provide it to API
-  public PreUpgradeCheckRequest(String clusterName) {
+  public PrereqCheckRequest(String clusterName) {
     this.clusterName = clusterName;
   }
 

+ 54 - 14
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/PreUpgradeCheckResourceProvider.java

@@ -21,10 +21,14 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.List;
+import java.util.ArrayList;
 
 import org.apache.ambari.server.StaticallyInject;
+import org.apache.ambari.server.checks.*;
+import org.apache.ambari.server.state.CheckHelper;
 import org.apache.ambari.server.controller.AmbariManagementController;
-import org.apache.ambari.server.controller.PreUpgradeCheckRequest;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
 import org.apache.ambari.server.controller.spi.NoSuchParentResourceException;
 import org.apache.ambari.server.controller.spi.NoSuchResourceException;
 import org.apache.ambari.server.controller.spi.Predicate;
@@ -34,8 +38,7 @@ import org.apache.ambari.server.controller.spi.Resource.Type;
 import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
-import org.apache.ambari.server.state.UpgradeCheckHelper;
-import org.apache.ambari.server.state.stack.upgrade.UpgradeCheck;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
 
 import com.google.inject.Inject;
 
@@ -56,6 +59,44 @@ public class PreUpgradeCheckResourceProvider extends ReadOnlyResourceProvider {
   public static final String UPGRADE_CHECK_CLUSTER_NAME_PROPERTY_ID       = PropertyHelper.getPropertyId("UpgradeChecks", "cluster_name");
   public static final String UPGRADE_CHECK_REPOSITORY_VERSION_PROPERTY_ID = PropertyHelper.getPropertyId("UpgradeChecks", "repository_version");
 
+  @Inject
+  private static ServicesMaintenanceModeCheck servicesMaintenanceModeCheck;
+  @Inject
+  private static HostsMasterMaintenanceCheck hostsMasterMaintenanceCheck;
+  @Inject
+  private static HostsRepositoryVersionCheck hostsRepositoryVersionCheck;
+  @Inject
+  private static ServicesNamenodeHighAvailabilityCheck servicesNamenodeHighAvailabilityCheck;
+  @Inject
+  private static ServicesYarnWorkPreservingCheck servicesYarnWorkPreservingCheck;
+  @Inject
+  private static ServicesDecommissionCheck servicesDecommissionCheck;
+  @Inject
+  private static ServicesJobsDistributedCacheCheck servicesJobsDistributedCacheCheck;
+  @Inject
+  private static HostsHeartbeatCheck heartbeatCheck;
+  @Inject
+  private static ServicesUpCheck servicesUpCheck;
+
+
+  /**
+   * List of the registered upgrade checks
+   */
+  @SuppressWarnings("serial")
+  private final List<AbstractCheckDescriptor> updateChecksRegistry = new ArrayList<AbstractCheckDescriptor>() {
+    {
+      add(hostsMasterMaintenanceCheck);
+      add(hostsRepositoryVersionCheck);
+      add(servicesMaintenanceModeCheck);
+      add(servicesNamenodeHighAvailabilityCheck);
+      add(servicesYarnWorkPreservingCheck);
+      add(servicesDecommissionCheck);
+      add(servicesJobsDistributedCacheCheck);
+      add(heartbeatCheck);
+      add(servicesUpCheck);
+    }
+  };
+
   @SuppressWarnings("serial")
   private static Set<String> pkPropertyIds = new HashSet<String>() {
     {
@@ -86,7 +127,7 @@ public class PreUpgradeCheckResourceProvider extends ReadOnlyResourceProvider {
   };
 
   @Inject
-  private static UpgradeCheckHelper upgradeChecks;
+  private static CheckHelper checkHelper;
 
   /**
    * Constructor.
@@ -107,26 +148,25 @@ public class PreUpgradeCheckResourceProvider extends ReadOnlyResourceProvider {
 
     for (Map<String, Object> propertyMap: propertyMaps) {
       final String clusterName = propertyMap.get(UPGRADE_CHECK_CLUSTER_NAME_PROPERTY_ID).toString();
-      final PreUpgradeCheckRequest upgradeCheckRequest = new PreUpgradeCheckRequest(clusterName);
+      final PrereqCheckRequest upgradeCheckRequest = new PrereqCheckRequest(clusterName);
       if (propertyMap.containsKey(UPGRADE_CHECK_REPOSITORY_VERSION_PROPERTY_ID)) {
         upgradeCheckRequest.setRepositoryVersion(propertyMap.get(UPGRADE_CHECK_REPOSITORY_VERSION_PROPERTY_ID).toString());
       }
-      for (UpgradeCheck upgradeCheck: upgradeChecks.performPreUpgradeChecks(upgradeCheckRequest)) {
+      for (PrerequisiteCheck prerequisiteCheck : checkHelper.performChecks(upgradeCheckRequest, updateChecksRegistry)) {
         final Resource resource = new ResourceImpl(Resource.Type.PreUpgradeCheck);
-        setResourceProperty(resource, UPGRADE_CHECK_ID_PROPERTY_ID, upgradeCheck.getId(), requestedIds);
-        setResourceProperty(resource, UPGRADE_CHECK_CHECK_PROPERTY_ID, upgradeCheck.getDescription(), requestedIds);
-        setResourceProperty(resource, UPGRADE_CHECK_STATUS_PROPERTY_ID, upgradeCheck.getStatus(), requestedIds);
-        setResourceProperty(resource, UPGRADE_CHECK_REASON_PROPERTY_ID, upgradeCheck.getFailReason(), requestedIds);
-        setResourceProperty(resource, UPGRADE_CHECK_FAILED_ON_PROPERTY_ID, upgradeCheck.getFailedOn(), requestedIds);
-        setResourceProperty(resource, UPGRADE_CHECK_CHECK_TYPE_PROPERTY_ID, upgradeCheck.getType(), requestedIds);
-        setResourceProperty(resource, UPGRADE_CHECK_CLUSTER_NAME_PROPERTY_ID, upgradeCheck.getClusterName(), requestedIds);
+        setResourceProperty(resource, UPGRADE_CHECK_ID_PROPERTY_ID, prerequisiteCheck.getId(), requestedIds);
+        setResourceProperty(resource, UPGRADE_CHECK_CHECK_PROPERTY_ID, prerequisiteCheck.getDescription(), requestedIds);
+        setResourceProperty(resource, UPGRADE_CHECK_STATUS_PROPERTY_ID, prerequisiteCheck.getStatus(), requestedIds);
+        setResourceProperty(resource, UPGRADE_CHECK_REASON_PROPERTY_ID, prerequisiteCheck.getFailReason(), requestedIds);
+        setResourceProperty(resource, UPGRADE_CHECK_FAILED_ON_PROPERTY_ID, prerequisiteCheck.getFailedOn(), requestedIds);
+        setResourceProperty(resource, UPGRADE_CHECK_CHECK_TYPE_PROPERTY_ID, prerequisiteCheck.getType(), requestedIds);
+        setResourceProperty(resource, UPGRADE_CHECK_CLUSTER_NAME_PROPERTY_ID, prerequisiteCheck.getClusterName(), requestedIds);
         if (upgradeCheckRequest.getRepositoryVersion() != null) {
           setResourceProperty(resource, UPGRADE_CHECK_REPOSITORY_VERSION_PROPERTY_ID, upgradeCheckRequest.getRepositoryVersion(), requestedIds);
         }
         resources.add(resource);
       }
     }
-
     return resources;
   }
 

+ 72 - 0
ambari-server/src/main/java/org/apache/ambari/server/state/CheckHelper.java

@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import com.google.inject.Singleton;
+import org.apache.ambari.server.ClusterNotFoundException;
+import org.apache.ambari.server.checks.AbstractCheckDescriptor;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@Singleton
+public class CheckHelper {
+  /**
+   * Log.
+   */
+  private static Logger LOG = LoggerFactory.getLogger(CheckHelper.class);
+
+
+  /**
+   * Executes all registered pre-requisite checks.
+   *
+   * @param request pre-requisite check request
+   * @return list of pre-requisite check results
+   */
+  public List<PrerequisiteCheck> performChecks(PrereqCheckRequest request, List<AbstractCheckDescriptor> checksRegistry) {
+
+    final String clusterName = request.getClusterName();
+    final List<PrerequisiteCheck> prerequisiteCheckResults = new ArrayList<PrerequisiteCheck>();
+    for (AbstractCheckDescriptor checkDescriptor : checksRegistry) {
+      final PrerequisiteCheck prerequisiteCheck = new PrerequisiteCheck(
+        checkDescriptor.id, checkDescriptor.description,
+        checkDescriptor.type, clusterName);
+      try {
+        if (checkDescriptor.isApplicable(request)) {
+          checkDescriptor.perform(prerequisiteCheck, request);
+          prerequisiteCheckResults.add(prerequisiteCheck);
+        }
+      } catch (ClusterNotFoundException ex) {
+        prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+        prerequisiteCheck.setFailReason("Cluster with name " + clusterName + " doesn't exists");
+        prerequisiteCheckResults.add(prerequisiteCheck);
+      } catch (Exception ex) {
+        LOG.error("Check " + checkDescriptor.id + " failed", ex);
+        prerequisiteCheck.setStatus(PrereqCheckStatus.FAIL);
+        prerequisiteCheck.setFailReason("Unexpected server error happened");
+        prerequisiteCheckResults.add(prerequisiteCheck);
+      }
+    }
+    return prerequisiteCheckResults;
+  }
+}

+ 0 - 497
ambari-server/src/main/java/org/apache/ambari/server/state/UpgradeCheckHelper.java

@@ -1,497 +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.state;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.ambari.server.AmbariException;
-import org.apache.ambari.server.ClusterNotFoundException;
-import org.apache.ambari.server.ServiceNotFoundException;
-import org.apache.ambari.server.configuration.Configuration;
-import org.apache.ambari.server.controller.PreUpgradeCheckRequest;
-import org.apache.ambari.server.orm.dao.HostVersionDAO;
-import org.apache.ambari.server.orm.dao.RepositoryVersionDAO;
-import org.apache.ambari.server.orm.entities.HostVersionEntity;
-import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
-import org.apache.ambari.server.stack.HostsType;
-import org.apache.ambari.server.stack.MasterHostResolver;
-import org.apache.ambari.server.state.HostHealthStatus.HealthStatus;
-import org.apache.ambari.server.state.stack.upgrade.UpgradeCheck;
-import org.apache.ambari.server.state.stack.upgrade.UpgradeCheckStatus;
-import org.apache.ambari.server.state.stack.upgrade.UpgradeCheckType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.inject.Inject;
-import com.google.inject.Provider;
-import com.google.inject.Singleton;
-
-/**
- * Manages pre-upgrade checks.
- */
-@Singleton
-public class UpgradeCheckHelper {
-
-  /**
-   * Log.
-   */
-  private static Logger LOG = LoggerFactory.getLogger(UpgradeCheckHelper.class);
-
-  /**
-   * List of all possible upgrade checks.
-   */
-  final List<UpgradeCheckDescriptor> registry = new ArrayList<UpgradeCheckDescriptor>();
-
-  @Inject
-  Provider<Clusters> clustersProvider;
-
-  @Inject
-  Provider<Configuration> configurationProvider;
-
-  @Inject
-  Provider<HostVersionDAO> hostVersionDaoProvider;
-
-  @Inject
-  Provider<RepositoryVersionDAO> repositoryVersionDaoProvider;
-
-  /**
-   * Constructor. Fills upgrade check registry upon creation.
-   */
-  public UpgradeCheckHelper() {
-    registry.add(new HostsHeartbeatCheck());
-    registry.add(new HostsMasterMaintenanceCheck());
-    registry.add(new HostsRepositoryVersionCheck());
-    registry.add(new ServicesUpCheck());
-    registry.add(new ServicesMaintenanceModeCheck());
-    registry.add(new ServicesNamenodeHighAvailabilityCheck());
-    registry.add(new ServicesYarnWorkPreservingCheck());
-    registry.add(new ServicesDecommissionCheck());
-    registry.add(new ServicesJobsDistributedCacheCheck());
-  }
-
-  /**
-   * Executes all registered pre upgrade checks.
-   *
-   * @param request pre upgrade check request
-   * @return list of upgrade check results
-   */
-  public List<UpgradeCheck> performPreUpgradeChecks(PreUpgradeCheckRequest request) {
-    final String clusterName = request.getClusterName();
-    final List<UpgradeCheck> upgradeCheckResults = new ArrayList<UpgradeCheck>();
-    for (UpgradeCheckDescriptor upgradeCheckDescriptor: registry) {
-      final UpgradeCheck upgradeCheck = new UpgradeCheck(
-          upgradeCheckDescriptor.id, upgradeCheckDescriptor.description,
-          upgradeCheckDescriptor.type, clusterName);
-      try {
-        if (upgradeCheckDescriptor.isApplicable(request)) {
-          upgradeCheckDescriptor.perform(upgradeCheck, request);
-          upgradeCheckResults.add(upgradeCheck);
-        }
-      } catch (ClusterNotFoundException ex) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Cluster with name " + clusterName + " doesn't exists");
-        upgradeCheckResults.add(upgradeCheck);
-      } catch (Exception ex) {
-        LOG.error("Pre-upgrade check " + upgradeCheckDescriptor.id + " failed", ex);
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Unexpected server error happened");
-        upgradeCheckResults.add(upgradeCheck);
-      }
-    }
-    return upgradeCheckResults;
-  }
-
-  /**
-   * Describes upgrade check.
-   */
-  protected abstract class UpgradeCheckDescriptor {
-
-    private final String id;
-    private final String description;
-    private final UpgradeCheckType type;
-
-    /**
-     * Constructor.
-     *
-     * @param id unique identifier
-     * @param type type
-     * @param description description
-     */
-    public UpgradeCheckDescriptor(String id, UpgradeCheckType type, String description) {
-      this.id = id;
-      this.type = type;
-      this.description = description;
-    }
-
-    /**
-     * Tests if the upgrade check is applicable to given cluster. By default returns true.
-     *
-     * @param request pre upgrade check request
-     * @return true if check should be performed
-     *
-     * @throws AmbariException if server error happens
-     */
-    public boolean isApplicable(PreUpgradeCheckRequest request) throws AmbariException {
-      return true;
-    }
-
-    /**
-     * Executes check against given cluster.
-     *
-     * @param upgradeCheck dto for upgrade check results
-     * @param request pre upgrade check request
-     *
-     * @throws AmbariException if server error happens
-     */
-    public abstract void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException;
-  }
-
-  /**
-   * Checks that services are up.
-   */
-  protected class ServicesUpCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public ServicesUpCheck() {
-      super("SERVICES_UP", UpgradeCheckType.SERVICE, "All services must be up");
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      for (Map.Entry<String, Service> serviceEntry : cluster.getServices().entrySet()) {
-        final Service service = serviceEntry.getValue();
-        if (!service.isClientOnlyService() && service.getDesiredState() != State.STARTED) {
-          upgradeCheck.getFailedOn().add(service.getName());
-        }
-      }
-      if (!upgradeCheck.getFailedOn().isEmpty()) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Some services are down");
-      }
-    }
-  }
-
-  /**
-   * Checks that services are in the maintenance mode.
-   */
-  protected class ServicesMaintenanceModeCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public ServicesMaintenanceModeCheck() {
-      super("SERVICES_MAINTENANCE_MODE", UpgradeCheckType.SERVICE, "No service can be in maintenance mode");
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      for (Map.Entry<String, Service> serviceEntry : cluster.getServices().entrySet()) {
-        final Service service = serviceEntry.getValue();
-        if (service.getMaintenanceState() == MaintenanceState.ON) {
-          upgradeCheck.getFailedOn().add(service.getName());
-        }
-      }
-      if (!upgradeCheck.getFailedOn().isEmpty()) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Some services are in Maintenance Mode");
-      }
-    }
-  }
-
-  /**
-   * Checks that all hosts are either in maintenance mode or heartbeating with the server.
-   */
-  protected class HostsHeartbeatCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public HostsHeartbeatCheck() {
-      super("HOSTS_HEARTBEAT", UpgradeCheckType.HOST, "All hosts must be heartbeating with the server unless they are in Maintenance Mode");
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      final Map<String, Host> clusterHosts = clustersProvider.get().getHostsForCluster(clusterName);
-      for (Map.Entry<String, Host> hostEntry : clusterHosts.entrySet()) {
-        final Host host = hostEntry.getValue();
-        if (host.getHealthStatus().getHealthStatus() == HealthStatus.UNKNOWN && host.getMaintenanceState(cluster.getClusterId()) == MaintenanceState.OFF) {
-          upgradeCheck.getFailedOn().add(host.getHostName());
-        }
-      }
-      if (!upgradeCheck.getFailedOn().isEmpty()) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Some hosts are not heartbeating with the server");
-      }
-    }
-  }
-
-  /**
-   * Checks that all hosts in maintenance state do not have master components.
-   */
-  protected class HostsMasterMaintenanceCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public HostsMasterMaintenanceCheck() {
-      super("HOSTS_MASTER_MAINTENANCE", UpgradeCheckType.HOST, "Hosts in Maintenance Mode must not have any master components");
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      final MasterHostResolver masterHostResolver = new MasterHostResolver(cluster,
-          request.getRepositoryVersion());
-      final Set<String> hostsWithMasterComponent = new HashSet<String>();
-      for (Entry<String, Service> serviceEntry: cluster.getServices().entrySet()) {
-        final Service service = serviceEntry.getValue();
-        for (Entry<String, ServiceComponent> serviceComponentEntry: service.getServiceComponents().entrySet()) {
-          final ServiceComponent serviceComponent = serviceComponentEntry.getValue();
-          final HostsType hostsType = masterHostResolver.getMasterAndHosts(
-              service.getName(), serviceComponent.getName());
-          if (hostsType != null && hostsType.master != null) {
-            hostsWithMasterComponent.add(hostsType.master);
-          }
-        }
-      }
-      final Map<String, Host> clusterHosts = clustersProvider.get().getHostsForCluster(clusterName);
-      for (Map.Entry<String, Host> hostEntry : clusterHosts.entrySet()) {
-        final Host host = hostEntry.getValue();
-        if (host.getMaintenanceState(cluster.getClusterId()) == MaintenanceState.ON && hostsWithMasterComponent.contains(host.getHostName())) {
-          upgradeCheck.getFailedOn().add(host.getHostName());
-        }
-      }
-      if (!upgradeCheck.getFailedOn().isEmpty()) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Some hosts with master components are in Maintenance Mode");
-      }
-    }
-  }
-
-  /**
-   * Checks that all hosts have particular repository version.
-   */
-  protected class HostsRepositoryVersionCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public HostsRepositoryVersionCheck() {
-      super("HOSTS_REPOSITORY_VERSION", UpgradeCheckType.HOST, "Hosts should have the new repository version installed");
-    }
-
-    @Override
-    public boolean isApplicable(PreUpgradeCheckRequest request) throws AmbariException {
-      return request.getRepositoryVersion() != null;
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      final Map<String, Host> clusterHosts = clustersProvider.get().getHostsForCluster(clusterName);
-      final StackId stackId = cluster.getDesiredStackVersion();
-      final RepositoryVersionEntity repositoryVersion = repositoryVersionDaoProvider.get().findByStackAndVersion(stackId.getStackId(), request.getRepositoryVersion());
-      if (repositoryVersion == null) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Repository version " + request.getRepositoryVersion() + " doesn't exist");
-        upgradeCheck.getFailedOn().addAll(clusterHosts.keySet());
-        return;
-      }
-      for (Map.Entry<String, Host> hostEntry : clusterHosts.entrySet()) {
-        final Host host = hostEntry.getValue();
-        if (host.getMaintenanceState(cluster.getClusterId()) == MaintenanceState.OFF) {
-          final HostVersionEntity hostVersion = hostVersionDaoProvider.get().findByClusterStackVersionAndHost(clusterName, repositoryVersion.getStack(), repositoryVersion.getVersion(), host.getHostName());
-          if (hostVersion == null || hostVersion.getState() != RepositoryVersionState.INSTALLED) {
-            upgradeCheck.getFailedOn().add(host.getHostName());
-          }
-        }
-      }
-      if (!upgradeCheck.getFailedOn().isEmpty()) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Some hosts do not have repository version " + request.getRepositoryVersion() + " installed");
-      }
-    }
-  }
-
-  /**
-   * Checks that namenode high availability is enabled.
-   */
-  protected class ServicesNamenodeHighAvailabilityCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public ServicesNamenodeHighAvailabilityCheck() {
-      super("SERVICES_NAMENODE_HA", UpgradeCheckType.SERVICE, "Namenode high availability should be enabled");
-    }
-
-    @Override
-    public boolean isApplicable(PreUpgradeCheckRequest request) throws AmbariException {
-      final Cluster cluster = clustersProvider.get().getCluster(request.getClusterName());
-      return cluster.getService("HDFS") != null;
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      final String configType = "hdfs-site";
-      final Map<String, DesiredConfig> desiredConfigs = cluster.getDesiredConfigs();
-      final DesiredConfig desiredConfig = desiredConfigs.get(configType);
-      final Config config = cluster.getConfig(configType, desiredConfig.getTag());
-      if (!config.getProperties().containsKey("dfs.nameservices")) {
-        upgradeCheck.getFailedOn().add("HDFS");
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("Namenode high availability is disabled");
-      }
-    }
-  }
-
-  /**
-   * Checks that YARN has work-preserving restart enabled.
-   */
-  protected class ServicesYarnWorkPreservingCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public ServicesYarnWorkPreservingCheck() {
-      super("SERVICES_YARN_WP", UpgradeCheckType.SERVICE, "YARN work preserving restart should be enabled");
-    }
-
-    @Override
-    public boolean isApplicable(PreUpgradeCheckRequest request) throws AmbariException {
-      final Cluster cluster = clustersProvider.get().getCluster(request.getClusterName());
-      try {
-        cluster.getService("YARN");
-      } catch (ServiceNotFoundException ex) {
-        return false;
-      }
-      return true;
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      final String configType = "yarn-site";
-      final Map<String, DesiredConfig> desiredConfigs = cluster.getDesiredConfigs();
-      final DesiredConfig desiredConfig = desiredConfigs.get(configType);
-      final Config config = cluster.getConfig(configType, desiredConfig.getTag());
-      if (!config.getProperties().containsKey("yarn.resourcemanager.work-preserving-recovery.enabled") ||
-          !Boolean.getBoolean(config.getProperties().get("yarn.resourcemanager.work-preserving-recovery.enabled"))) {
-        upgradeCheck.getFailedOn().add("YARN");
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("YARN doesn't have work preserving restart, yarn.resourcemanager.work-preserving-recovery.enabled property is missing");
-      }
-    }
-  }
-
-  /**
-   * Checks that there are no services in decommission state.
-   */
-  protected class ServicesDecommissionCheck extends UpgradeCheckDescriptor {
-
-    /**
-     * Constructor.
-     */
-    public ServicesDecommissionCheck() {
-      super("SERVICES_DECOMMISSION", UpgradeCheckType.SERVICE, "Services should not be in Decommission state");
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      for (Entry<String, Service> serviceEntry: cluster.getServices().entrySet()) {
-        final Service service = serviceEntry.getValue();
-        for (Entry<String, ServiceComponent> serviceComponentEntry: service.getServiceComponents().entrySet()) {
-          final ServiceComponent serviceComponent = serviceComponentEntry.getValue();
-          for (String hostName : serviceComponent.getServiceComponentHosts().keySet()) {
-            final ServiceComponentHost scHost = serviceComponent.getServiceComponentHost(hostName);
-            if (scHost.getComponentAdminState() == HostComponentAdminState.DECOMMISSIONED || scHost.getComponentAdminState() == HostComponentAdminState.DECOMMISSIONING) {
-              upgradeCheck.getFailedOn().add(serviceComponent.getName());
-            }
-          }
-        }
-      }
-      if (!upgradeCheck.getFailedOn().isEmpty()) {
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("There are services in decommission or decommissioning state");
-      }
-    }
-  }
-
-  /**
-   * Checks that MR, Oozie and Tez jobs reference hadoop libraries from the distributed cache.
-   */
-  protected class ServicesJobsDistributedCacheCheck extends UpgradeCheckDescriptor {
-
-    @Override
-    public boolean isApplicable(PreUpgradeCheckRequest request)
-        throws AmbariException {
-      final Cluster cluster = clustersProvider.get().getCluster(request.getClusterName());
-      try {
-        cluster.getService("YARN");
-      } catch (ServiceNotFoundException ex) {
-        return false;
-      }
-      return true;
-    }
-
-    /**
-     * Constructor.
-     */
-    public ServicesJobsDistributedCacheCheck() {
-      super("SERVICES_JOBS_DISTRIBUTED_CACHE", UpgradeCheckType.SERVICE, "Jobs should reference hadoop libraries from the distributed cache");
-    }
-
-    @Override
-    public void perform(UpgradeCheck upgradeCheck, PreUpgradeCheckRequest request) throws AmbariException {
-      final String clusterName = request.getClusterName();
-      final Cluster cluster = clustersProvider.get().getCluster(clusterName);
-      final String configType = "mapred-site";
-      final Map<String, DesiredConfig> desiredConfigs = cluster.getDesiredConfigs();
-      final DesiredConfig desiredConfig = desiredConfigs.get(configType);
-      final Config config = cluster.getConfig(configType, desiredConfig.getTag());
-      if (!config.getProperties().containsKey("mapreduce.application.framework.path") || !config.getProperties().containsKey("mapreduce.application.classpath")) {
-        // TODO actually it is needed to validate that these properties contain proper values but the tickets for these changes are still open, so it will cause
-        // preupgrade checks to fail
-        upgradeCheck.getFailedOn().add("MR");
-        upgradeCheck.setStatus(UpgradeCheckStatus.FAIL);
-        upgradeCheck.setFailReason("mapreduce.application.framework.path and mapreduce.application.classpath should reference distributed cache");
-      }
-    }
-  }
-}

+ 4 - 4
ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/UpgradeCheckStatus.java → ambari-server/src/main/java/org/apache/ambari/server/state/stack/PrereqCheckStatus.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ambari.server.state.stack.upgrade;
+package org.apache.ambari.server.state.stack;
 
 /**
- * Indicates status of upgrade check.
+ * Indicates status of prerequisite check.
  */
-public enum UpgradeCheckStatus {
+public enum PrereqCheckStatus {
   PASS,
   FAIL
 }

+ 4 - 4
ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/UpgradeCheckType.java → ambari-server/src/main/java/org/apache/ambari/server/state/stack/PrereqCheckType.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ambari.server.state.stack.upgrade;
+package org.apache.ambari.server.state.stack;
 
 /**
- * Type of upgrade check.
+ * Type of prerequisite check.
  */
-public enum UpgradeCheckType {
+public enum PrereqCheckType {
   SERVICE,
   HOST,
   CLUSTER

+ 10 - 10
ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/UpgradeCheck.java → ambari-server/src/main/java/org/apache/ambari/server/state/stack/PrerequisiteCheck.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,25 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ambari.server.state.stack.upgrade;
+package org.apache.ambari.server.state.stack;
 
 import java.util.ArrayList;
 import java.util.List;
 
 /**
- * Contains information about performed upgrade check.
+ * Contains information about performed prerequisite check.
  *
  */
-public class UpgradeCheck {
+public class PrerequisiteCheck {
   private final String id;
   private final String description;
-  private final UpgradeCheckType type;
+  private final PrereqCheckType type;
   private final String clusterName;
-  private UpgradeCheckStatus status = UpgradeCheckStatus.PASS;
+  private PrereqCheckStatus status = PrereqCheckStatus.PASS;
   private String failReason = "";
   private List<String> failedOn = new ArrayList<String>();
 
-  public UpgradeCheck(String id, String description, UpgradeCheckType type, String clusterName) {
+  public PrerequisiteCheck(String id, String description, PrereqCheckType type, String clusterName) {
     this.id = id;
     this.description = description;
     this.type = type;
@@ -48,11 +48,11 @@ public class UpgradeCheck {
     return description;
   }
 
-  public UpgradeCheckStatus getStatus() {
+  public PrereqCheckStatus getStatus() {
     return status;
   }
 
-  public void setStatus(UpgradeCheckStatus status) {
+  public void setStatus(PrereqCheckStatus status) {
     this.status = status;
   }
 
@@ -72,7 +72,7 @@ public class UpgradeCheck {
     this.failedOn = failedOn;
   }
 
-  public UpgradeCheckType getType() {
+  public PrereqCheckType getType() {
     return type;
   }
 

+ 49 - 50
ambari-server/src/test/java/org/apache/ambari/server/state/UpgradeCheckHelperTest.java → ambari-server/src/test/java/org/apache/ambari/server/state/CheckHelperTest.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,86 +15,85 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.ambari.server.state;
 
+import java.util.ArrayList;
 import java.util.List;
 
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.util.Providers;
 import junit.framework.Assert;
 
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.ClusterNotFoundException;
+import org.apache.ambari.server.checks.AbstractCheckDescriptor;
+import org.apache.ambari.server.checks.ServicesUpCheck;
 import org.apache.ambari.server.configuration.Configuration;
-import org.apache.ambari.server.controller.PreUpgradeCheckRequest;
+import org.apache.ambari.server.controller.PrereqCheckRequest;
 import org.apache.ambari.server.orm.dao.HostVersionDAO;
 import org.apache.ambari.server.orm.dao.RepositoryVersionDAO;
-import org.apache.ambari.server.state.UpgradeCheckHelper.UpgradeCheckDescriptor;
-import org.apache.ambari.server.state.stack.upgrade.UpgradeCheck;
-import org.apache.ambari.server.state.stack.upgrade.UpgradeCheckStatus;
+import org.apache.ambari.server.state.stack.PrerequisiteCheck;
+import org.apache.ambari.server.state.stack.PrereqCheckStatus;
 import org.easymock.EasyMock;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import com.google.inject.AbstractModule;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.util.Providers;
 
 /**
- * Tests the {@link UpgradeCheckHelper} class
+ * Tests the {@link CheckHelper} class
  */
-public class UpgradeCheckHelperTest {
 
-  /**
+public class CheckHelperTest {
+
+/**
    * Makes sure that people don't forget to add new checks to registry.
    */
-  @Test
-  public void defaultConstructorTest() throws Exception {
-    final UpgradeCheckHelper helper = new UpgradeCheckHelper();
-
-    Assert.assertEquals(UpgradeCheckHelper.class.getDeclaredClasses().length - 1, helper.registry.size());
-  }
 
   @Test
   public void performPreUpgradeChecksTest_ok() throws Exception {
-    final UpgradeCheckHelper helper = new UpgradeCheckHelper();
-    helper.registry.clear();
-    UpgradeCheckDescriptor descriptor = EasyMock.createNiceMock(UpgradeCheckDescriptor.class);
-    descriptor.perform(EasyMock.<UpgradeCheck> anyObject(), EasyMock.<PreUpgradeCheckRequest> anyObject());
+    final CheckHelper helper = new CheckHelper();
+    List<AbstractCheckDescriptor> updateChecksRegistry = new ArrayList<AbstractCheckDescriptor>();
+    AbstractCheckDescriptor descriptor = EasyMock.createNiceMock(AbstractCheckDescriptor.class);
+    descriptor.perform(EasyMock.<PrerequisiteCheck> anyObject(), EasyMock.<PrereqCheckRequest> anyObject());
     EasyMock.expectLastCall().times(1);
-    EasyMock.expect(descriptor.isApplicable(EasyMock.<PreUpgradeCheckRequest> anyObject())).andReturn(true);
+    EasyMock.expect(descriptor.isApplicable(EasyMock.<PrereqCheckRequest> anyObject())).andReturn(true);
     EasyMock.replay(descriptor);
-    helper.registry.add(descriptor);
-    helper.performPreUpgradeChecks(new PreUpgradeCheckRequest("cluster"));
+    updateChecksRegistry.add(descriptor);
+
+    helper.performChecks(new PrereqCheckRequest("cluster"), updateChecksRegistry);
     EasyMock.verify(descriptor);
   }
 
   @Test
   public void performPreUpgradeChecksTest_notApplicable() throws Exception {
-    final UpgradeCheckHelper helper = new UpgradeCheckHelper();
-    helper.registry.clear();
-    UpgradeCheckDescriptor descriptor = EasyMock.createNiceMock(UpgradeCheckDescriptor.class);
-    EasyMock.expect(descriptor.isApplicable(EasyMock.<PreUpgradeCheckRequest> anyObject())).andReturn(false);
+    final CheckHelper helper = new CheckHelper();
+    List<AbstractCheckDescriptor> updateChecksRegistry = new ArrayList<AbstractCheckDescriptor>();
+    AbstractCheckDescriptor descriptor = EasyMock.createNiceMock(AbstractCheckDescriptor.class);
+    EasyMock.expect(descriptor.isApplicable(EasyMock.<PrereqCheckRequest> anyObject())).andReturn(false);
     EasyMock.replay(descriptor);
-    helper.registry.add(descriptor);
-    helper.performPreUpgradeChecks(new PreUpgradeCheckRequest("cluster"));
+    updateChecksRegistry.add(descriptor);
+    helper.performChecks(new PrereqCheckRequest("cluster"), updateChecksRegistry);
     EasyMock.verify(descriptor);
   }
 
   @Test
   public void performPreUpgradeChecksTest_throwsException() throws Exception {
-    final UpgradeCheckHelper helper = new UpgradeCheckHelper();
-    helper.registry.clear();
-    UpgradeCheckDescriptor descriptor = EasyMock.createNiceMock(UpgradeCheckDescriptor.class);
-    descriptor.perform(EasyMock.<UpgradeCheck> anyObject(), EasyMock.<PreUpgradeCheckRequest> anyObject());
+    final CheckHelper helper = new CheckHelper();
+    List<AbstractCheckDescriptor> updateChecksRegistry = new ArrayList<AbstractCheckDescriptor>();
+    AbstractCheckDescriptor descriptor = EasyMock.createNiceMock(AbstractCheckDescriptor.class);
+    descriptor.perform(EasyMock.<PrerequisiteCheck> anyObject(), EasyMock.<PrereqCheckRequest> anyObject());
     EasyMock.expectLastCall().andThrow(new AmbariException("error"));
-    EasyMock.expect(descriptor.isApplicable(EasyMock.<PreUpgradeCheckRequest> anyObject())).andReturn(true);
+    EasyMock.expect(descriptor.isApplicable(EasyMock.<PrereqCheckRequest> anyObject())).andReturn(true);
     EasyMock.replay(descriptor);
-    helper.registry.add(descriptor);
-    final List<UpgradeCheck> upgradeChecks = helper.performPreUpgradeChecks(new PreUpgradeCheckRequest("cluster"));
+    updateChecksRegistry.add(descriptor);
+    final List<PrerequisiteCheck> upgradeChecks = helper.performChecks(new PrereqCheckRequest("cluster"), updateChecksRegistry);
     EasyMock.verify(descriptor);
-    Assert.assertEquals(UpgradeCheckStatus.FAIL, upgradeChecks.get(0).getStatus());
+    Assert.assertEquals(PrereqCheckStatus.FAIL, upgradeChecks.get(0).getStatus());
   }
 
   @Test
@@ -116,20 +115,20 @@ public class UpgradeCheckHelperTest {
       @Override
       protected void configure() {
         bind(Clusters.class).toInstance(clusters);
-        bind(Configuration.class).toProvider(Providers.<Configuration> of(null));
-        bind(HostVersionDAO.class).toProvider(Providers.<HostVersionDAO> of(null));
-        bind(RepositoryVersionDAO.class).toProvider(Providers.<RepositoryVersionDAO> of(null));
+        bind(Configuration.class).toProvider(Providers.<Configuration>of(null));
+        bind(HostVersionDAO.class).toProvider(Providers.<HostVersionDAO>of(null));
+        bind(RepositoryVersionDAO.class).toProvider(Providers.<RepositoryVersionDAO>of(null));
       }
     });
-    final UpgradeCheckHelper helper = injector.getInstance(UpgradeCheckHelper.class);
-    helper.registry.clear();
-    helper.registry.add(helper.new ServicesUpCheck()); //mocked Cluster has no services, so the check should always be PASS
-    List<UpgradeCheck> upgradeChecks = helper.performPreUpgradeChecks(new PreUpgradeCheckRequest("existing"));
-    Assert.assertEquals(UpgradeCheckStatus.PASS, upgradeChecks.get(0).getStatus());
-    upgradeChecks = helper.performPreUpgradeChecks(new PreUpgradeCheckRequest("non-existing"));
-    Assert.assertEquals(UpgradeCheckStatus.FAIL, upgradeChecks.get(0).getStatus());
+    final CheckHelper helper = injector.getInstance(CheckHelper.class);
+    List<AbstractCheckDescriptor> updateChecksRegistry = new ArrayList<AbstractCheckDescriptor>();
+    updateChecksRegistry.add(injector.getInstance(ServicesUpCheck.class)); //mocked Cluster has no services, so the check should always be PASS
+    List<PrerequisiteCheck> upgradeChecks = helper.performChecks(new PrereqCheckRequest("existing"), updateChecksRegistry);
+    Assert.assertEquals(PrereqCheckStatus.PASS, upgradeChecks.get(0).getStatus());
+    upgradeChecks = helper.performChecks(new PrereqCheckRequest("non-existing"), updateChecksRegistry);
+    Assert.assertEquals(PrereqCheckStatus.FAIL, upgradeChecks.get(0).getStatus());
     //non existing cluster is an expected error
     Assert.assertTrue(!upgradeChecks.get(0).getFailReason().equals("Unexpected server error happened"));
   }
-
 }
+