You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by jo...@apache.org on 2016/02/23 14:38:21 UTC

ambari git commit: AMBARI-15131 - Create Component/Service Upgrade History On Finalize (jonathanhurley)

Repository: ambari
Updated Branches:
  refs/heads/branch-dev-patch-upgrade ec91f74a4 -> 55342fc62


AMBARI-15131 - Create Component/Service Upgrade History On Finalize (jonathanhurley)


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

Branch: refs/heads/branch-dev-patch-upgrade
Commit: 55342fc621aed290173cc09f137af97afa18dc0d
Parents: ec91f74
Author: Jonathan Hurley <jh...@hortonworks.com>
Authored: Mon Feb 22 11:01:20 2016 -0500
Committer: Jonathan Hurley <jh...@hortonworks.com>
Committed: Mon Feb 22 11:50:00 2016 -0500

----------------------------------------------------------------------
 .../internal/UpgradeResourceProvider.java       |  16 ++-
 .../upgrades/FinalizeUpgradeAction.java         |  91 +++++++++++++--
 .../upgrades/UpgradeActionTest.java             | 110 ++++++++++++++++++-
 3 files changed, 198 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/55342fc6/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
index 860ba88..0190014 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/UpgradeResourceProvider.java
@@ -173,6 +173,8 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
   private static final String COMMAND_PARAM_CLUSTER_NAME = "clusterName";
   private static final String COMMAND_PARAM_DIRECTION = "upgrade_direction";
   private static final String COMMAND_PARAM_UPGRADE_PACK = "upgrade_pack";
+  private static final String COMMAND_PARAM_REQUEST_ID = "request_id";
+
   // TODO AMBARI-12698, change this variable name since it is no longer always a restart. Possible values are rolling_upgrade or nonrolling_upgrade
   // This will involve changing Script.py
   private static final String COMMAND_PARAM_RESTART_TYPE = "restart_type";
@@ -1189,7 +1191,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         new ArrayList<String>(wrapper.getHosts()));
 
     LOG.debug("Analyzing upgrade item {} with tasks: {}.", entity.getText(), entity.getTasks());
-    Map<String, String> params = getNewParameterMap();
+    Map<String, String> params = getNewParameterMap(request);
     params.put(COMMAND_PARAM_TASKS, entity.getTasks());
     params.put(COMMAND_PARAM_VERSION, context.getVersion());
     params.put(COMMAND_PARAM_DIRECTION, context.getDirection().name().toLowerCase());
@@ -1297,7 +1299,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
         break;
     }
 
-    Map<String, String> commandParams = getNewParameterMap();
+    Map<String, String> commandParams = getNewParameterMap(request);
 
     // TODO AMBARI-12698, change COMMAND_PARAM_RESTART_TYPE to something that isn't "RESTART" specific.
     if (context.getType() == UpgradeType.ROLLING) {
@@ -1365,7 +1367,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
 
     Cluster cluster = context.getCluster();
 
-    Map<String, String> commandParams = getNewParameterMap();
+    Map<String, String> commandParams = getNewParameterMap(request);
     commandParams.put(COMMAND_PARAM_VERSION, context.getVersion());
     commandParams.put(COMMAND_PARAM_DIRECTION, context.getDirection().name().toLowerCase());
     commandParams.put(COMMAND_PARAM_ORIGINAL_STACK, context.getOriginalStackId().getStackId());
@@ -1405,7 +1407,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
     stage.setStageId(stageId);
     entity.setStageId(Long.valueOf(stageId));
 
-    Map<String, String> requestParams = getNewParameterMap();
+    Map<String, String> requestParams = getNewParameterMap(request);
     s_commandExecutionHelper.get().addExecutionCommandsToStage(actionContext, stage, requestParams);
 
     request.addStages(Collections.singletonList(stage));
@@ -1432,7 +1434,7 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
 
     Cluster cluster = context.getCluster();
 
-    Map<String, String> commandParams = getNewParameterMap();
+    Map<String, String> commandParams = getNewParameterMap(request);
     commandParams.put(COMMAND_PARAM_CLUSTER_NAME, cluster.getClusterName());
     commandParams.put(COMMAND_PARAM_VERSION, context.getVersion());
     commandParams.put(COMMAND_PARAM_DIRECTION, context.getDirection().name().toLowerCase());
@@ -1543,13 +1545,15 @@ public class UpgradeResourceProvider extends AbstractControllerResourceProvider
    * command was created. For upgrades, this is problematic since the commands
    * are all created ahead of time, but the upgrade may change configs as part
    * of the upgrade pack.</li>
+   * <li>{@link #COMMAND_PARAM_REQUEST_ID}</li> the ID of the request.
    * <ul>
    *
    * @return
    */
-  private Map<String, String> getNewParameterMap() {
+  private Map<String, String> getNewParameterMap(RequestStageContainer requestStageContainer) {
     Map<String, String> parameters = new HashMap<String, String>();
     parameters.put(KeyNames.REFRESH_CONFIG_TAGS_BEFORE_EXECUTION, "*");
+    parameters.put(COMMAND_PARAM_REQUEST_ID, String.valueOf(requestStageContainer.getId()));
     return parameters;
   }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/55342fc6/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
index 03d407a..0c8df78 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/serveraction/upgrades/FinalizeUpgradeAction.java
@@ -35,11 +35,18 @@ import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.orm.dao.ClusterVersionDAO;
 import org.apache.ambari.server.orm.dao.HostComponentStateDAO;
 import org.apache.ambari.server.orm.dao.HostVersionDAO;
+import org.apache.ambari.server.orm.dao.ServiceComponentDesiredStateDAO;
+import org.apache.ambari.server.orm.dao.StackDAO;
+import org.apache.ambari.server.orm.dao.UpgradeDAO;
 import org.apache.ambari.server.orm.entities.ClusterVersionEntity;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
 import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.HostVersionEntity;
 import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
+import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
+import org.apache.ambari.server.orm.entities.ServiceComponentHistoryEntity;
+import org.apache.ambari.server.orm.entities.StackEntity;
+import org.apache.ambari.server.orm.entities.UpgradeEntity;
 import org.apache.ambari.server.serveraction.AbstractServerAction;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
@@ -65,6 +72,7 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
   public static final String CLUSTER_NAME_KEY = "cluster_name";
   public static final String UPGRADE_DIRECTION_KEY = "upgrade_direction";
   public static final String VERSION_KEY = "version";
+  public static final String REQUEST_ID = "request_id";
   public static final String PREVIOUS_UPGRADE_NOT_COMPLETED_MSG = "It is possible that a previous upgrade was not finalized. " +
       "For this reason, Ambari will not remove any configs. Please ensure that all database records are correct.";
 
@@ -97,6 +105,24 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
   @Inject
   private HostComponentStateDAO hostComponentStateDAO;
 
+  /**
+   * Gets {@link StackEntity} instances from {@link StackId}.
+   */
+  @Inject
+  private StackDAO stackDAO;
+
+  /**
+   * Gets desired state entities for service components.
+   */
+  @Inject
+  private ServiceComponentDesiredStateDAO serviceComponentDesiredStateDAO;
+
+  /**
+   * Gets {@link UpgradeEntity} instances.
+   */
+  @Inject
+  private UpgradeDAO upgradeDAO;
+
   @Inject
   private AmbariMetaInfo ambariMetaInfo;
 
@@ -116,10 +142,9 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
     String clusterName = getExecutionCommand().getClusterName();
 
     if (isDowngrade) {
-      return finalizeDowngrade(clusterName, originalStackId, targetStackId,
-          version);
+      return finalizeDowngrade(clusterName, originalStackId, targetStackId, version);
     } else {
-      return finalizeUpgrade(clusterName, version);
+      return finalizeUpgrade(clusterName, version, commandParams);
     }
   }
 
@@ -129,7 +154,8 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
    * @param version     the target version of the upgrade
    * @return the command report
    */
-  private CommandReport finalizeUpgrade(String clusterName, String version)
+  private CommandReport finalizeUpgrade(String clusterName, String version,
+      Map<String, String> commandParams)
     throws AmbariException, InterruptedException {
 
     StringBuilder outSB = new StringBuilder();
@@ -140,6 +166,7 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
 
       Cluster cluster = clusters.getCluster(clusterName);
       StackId clusterDesiredStackId = cluster.getDesiredStackVersion();
+      StackId clusterCurrentStackId = cluster.getCurrentStackVersion();
 
       ClusterVersionEntity upgradingClusterVersion = clusterVersionDAO.findByClusterAndStackAndVersion(
           clusterName, clusterDesiredStackId, version);
@@ -258,7 +285,9 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
             upgradingClusterVersion.getState(), RepositoryVersionState.CURRENT.toString()));
       }
 
-      outSB.append(String.format("Will finalize the upgraded state of host components in %d host(s).\n", hostVersionsAllowed.size()));
+      outSB.append(
+          String.format("Finalizing the upgraded state of host components in %d host(s).\n",
+              hostVersionsAllowed.size()));
 
       // Reset the upgrade state
       for (HostVersionEntity hostVersion : hostVersionsAllowed) {
@@ -269,18 +298,34 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
         }
       }
 
-      outSB.append(String.format("Will finalize the version for %d host(s).\n", hostVersionsAllowed.size()));
-
       // Impacts all hosts that have a version
+      outSB.append(
+          String.format("Finalizing the version for %d host(s).\n", hostVersionsAllowed.size()));
       cluster.mapHostVersions(hostsToUpdate, upgradingClusterVersion, RepositoryVersionState.CURRENT);
 
-      outSB.append(String.format("Will finalize the version for cluster %s.\n", clusterName));
-
       // transitioning the cluster into CURRENT will update the current/desired
       // stack values
+      outSB.append(String.format("Finalizing the version for cluster %s.\n", clusterName));
       cluster.transitionClusterVersion(clusterDesiredStackId, version,
           RepositoryVersionState.CURRENT);
 
+      if (commandParams.containsKey(REQUEST_ID)) {
+        String requestId = commandParams.get(REQUEST_ID);
+        UpgradeEntity upgradeEntity = upgradeDAO.findUpgradeByRequestId(Long.valueOf(requestId));
+
+        if (null != upgradeEntity) {
+          outSB.append("Creating upgrade history.\n");
+          writeComponentHistory(cluster, upgradeEntity, clusterCurrentStackId,
+              clusterDesiredStackId);
+        } else {
+          String warning = String.format(
+              "Unable to create upgrade history because no upgrade could be found for request with ID %s\n",
+              requestId);
+
+          outSB.append(warning);
+        }
+      }
+
       outSB.append("Upgrade was successful!\n");
       return createCommandReport(0, HostRoleStatus.COMPLETED, "{}", outSB.toString(), errSB.toString());
     } catch (Exception e) {
@@ -455,6 +500,33 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
     return errors;
   }
 
+  private void writeComponentHistory(Cluster cluster, UpgradeEntity upgradeEntity,
+      StackId fromStackId, StackId toStackId) {
+
+    StackEntity fromStack = stackDAO.find(fromStackId.getStackName(), fromStackId.getStackVersion());
+    StackEntity toStack = stackDAO.find(toStackId.getStackName(), toStackId.getStackVersion());
+
+    // for every service component, if it was included in the upgrade then
+    // create a historical entry
+    for (Service service : cluster.getServices().values()) {
+      for (ServiceComponent serviceComponent : service.getServiceComponents().values()) {
+        if (serviceComponent.isVersionAdvertised()) {
+          ServiceComponentHistoryEntity historyEntity = new ServiceComponentHistoryEntity();
+          historyEntity.setUpgrade(upgradeEntity);
+          historyEntity.setFromStack(fromStack);
+          historyEntity.setToStack(toStack);
+
+          ServiceComponentDesiredStateEntity desiredStateEntity = serviceComponentDesiredStateDAO.findByName(
+              cluster.getClusterId(), serviceComponent.getServiceName(),
+              serviceComponent.getName());
+
+          historyEntity.setServiceComponentDesiredState(desiredStateEntity);
+          serviceComponentDesiredStateDAO.create(historyEntity);
+        }
+      }
+    }
+  }
+
   protected static class InfoTuple {
     protected final String serviceName;
     protected final String componentName;
@@ -467,7 +539,6 @@ public class FinalizeUpgradeAction extends AbstractServerAction {
       hostName = host;
       currentVersion = version;
     }
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/55342fc6/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
index 989eba2..f43642c 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/serveraction/upgrades/UpgradeActionTest.java
@@ -50,11 +50,17 @@ import org.apache.ambari.server.orm.dao.ClusterVersionDAO;
 import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.orm.dao.HostVersionDAO;
 import org.apache.ambari.server.orm.dao.RepositoryVersionDAO;
+import org.apache.ambari.server.orm.dao.RequestDAO;
+import org.apache.ambari.server.orm.dao.ServiceComponentDesiredStateDAO;
 import org.apache.ambari.server.orm.dao.StackDAO;
+import org.apache.ambari.server.orm.dao.UpgradeDAO;
 import org.apache.ambari.server.orm.entities.ClusterVersionEntity;
 import org.apache.ambari.server.orm.entities.HostVersionEntity;
 import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
+import org.apache.ambari.server.orm.entities.RequestEntity;
+import org.apache.ambari.server.orm.entities.ServiceComponentHistoryEntity;
 import org.apache.ambari.server.orm.entities.StackEntity;
+import org.apache.ambari.server.orm.entities.UpgradeEntity;
 import org.apache.ambari.server.serveraction.ServerAction;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
@@ -73,6 +79,7 @@ import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.State;
 import org.apache.ambari.server.state.stack.UpgradePack;
 import org.apache.ambari.server.state.stack.upgrade.Direction;
+import org.apache.ambari.server.state.stack.upgrade.UpgradeType;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -109,8 +116,6 @@ public class UpgradeActionTest {
 
   private AmbariManagementController amc;
 
-  private AmbariMetaInfo ambariMetaInfo;
-
   @Inject
   private OrmTestHelper m_helper;
 
@@ -141,6 +146,15 @@ public class UpgradeActionTest {
   @Inject
   private ServiceComponentHostFactory serviceComponentHostFactory;
 
+  @Inject
+  private RequestDAO requestDAO;
+
+  @Inject
+  private UpgradeDAO upgradeDAO;
+
+  @Inject
+  private ServiceComponentDesiredStateDAO serviceComponentDesiredStateDAO;
+
   @Before
   public void setup() throws Exception {
     m_injector = Guice.createInjector(new InMemoryDefaultTestModule());
@@ -150,7 +164,6 @@ public class UpgradeActionTest {
 
     // Initialize AmbariManagementController
     amc = m_injector.getInstance(AmbariManagementController.class);
-    ambariMetaInfo = m_injector.getInstance(AmbariMetaInfo.class);
 
     Field field = AmbariServer.class.getDeclaredField("clusterController");
     field.setAccessible(true);
@@ -907,6 +920,97 @@ public class UpgradeActionTest {
     assertEquals(targetStack, desiredStackId);
   }
 
+  @Test
+  public void testUpgradeHistory() throws Exception {
+    StackId sourceStack = HDP_21_STACK;
+    StackId targetStack = HDP_21_STACK;
+    String sourceRepo = HDP_2_1_1_0;
+    String targetRepo = HDP_2_1_1_1;
+
+    makeUpgradeCluster(sourceStack, sourceRepo, targetStack, targetRepo);
+
+    // Verify the repo before calling Finalize
+    AmbariMetaInfo metaInfo = m_injector.getInstance(AmbariMetaInfo.class);
+    AmbariCustomCommandExecutionHelper helper = m_injector.getInstance(AmbariCustomCommandExecutionHelper.class);
+    Host host = clusters.getHost("h1");
+    Cluster cluster = clusters.getCluster(clusterName);
+
+    // install HDFS with some components
+    Service service = installService(cluster, "HDFS");
+    addServiceComponent(cluster, service, "NAMENODE");
+    addServiceComponent(cluster, service, "DATANODE");
+    ServiceComponentHost nnSCH = createNewServiceComponentHost(cluster, "HDFS", "NAMENODE", "h1");
+    ServiceComponentHost dnSCH = createNewServiceComponentHost(cluster, "HDFS", "DATANODE", "h1");
+
+    // fake their upgrade
+    nnSCH.setStackVersion(nnSCH.getDesiredStackVersion());
+    nnSCH.setVersion(targetRepo);
+    dnSCH.setStackVersion(nnSCH.getDesiredStackVersion());
+    dnSCH.setVersion(targetRepo);
+
+    // create some entities for the finalize action to work with for patch
+    // history
+    RequestEntity requestEntity = new RequestEntity();
+    requestEntity.setClusterId(cluster.getClusterId());
+    requestEntity.setRequestId(1L);
+    requestEntity.setStartTime(System.currentTimeMillis());
+    requestEntity.setCreateTime(System.currentTimeMillis());
+    requestDAO.create(requestEntity);
+
+    UpgradeEntity upgradeEntity = new UpgradeEntity();
+    upgradeEntity.setId(1L);
+    upgradeEntity.setClusterId(cluster.getClusterId());
+    upgradeEntity.setRequestId(requestEntity.getRequestId());
+    upgradeEntity.setUpgradePackage("");
+    upgradeEntity.setFromVersion(sourceRepo);
+    upgradeEntity.setToVersion(targetRepo);
+    upgradeEntity.setUpgradeType(UpgradeType.NON_ROLLING);
+    upgradeDAO.create(upgradeEntity);
+
+    // verify that no history exist exists yet
+    List<ServiceComponentHistoryEntity> historyEntites = serviceComponentDesiredStateDAO.findHistory(
+        cluster.getClusterId(), nnSCH.getServiceName(),
+        nnSCH.getServiceComponentName());
+
+    assertEquals(0, historyEntites.size());
+
+    RepositoryInfo repo = metaInfo.getRepository(sourceStack.getStackName(), sourceStack.getStackVersion(), "redhat6", sourceStack.getStackId());
+    assertEquals(HDP_211_CENTOS6_REPO_URL, repo.getBaseUrl());
+    verifyBaseRepoURL(helper, cluster, host, HDP_211_CENTOS6_REPO_URL);
+
+    // Finalize the upgrade, passing in the request ID so that history is
+    // created
+    Map<String, String> commandParams = new HashMap<String, String>();
+    commandParams.put(FinalizeUpgradeAction.REQUEST_ID, String.valueOf(requestEntity.getRequestId()));
+    commandParams.put(FinalizeUpgradeAction.UPGRADE_DIRECTION_KEY, "upgrade");
+    commandParams.put(FinalizeUpgradeAction.VERSION_KEY, targetRepo);
+
+    ExecutionCommand executionCommand = new ExecutionCommand();
+    executionCommand.setCommandParams(commandParams);
+    executionCommand.setClusterName(clusterName);
+
+    HostRoleCommand hostRoleCommand = hostRoleCommandFactory.create(null, null, null, null);
+    hostRoleCommand.setExecutionCommandWrapper(new ExecutionCommandWrapper(executionCommand));
+
+    FinalizeUpgradeAction action = m_injector.getInstance(FinalizeUpgradeAction.class);
+    action.setExecutionCommand(executionCommand);
+    action.setHostRoleCommand(hostRoleCommand);
+
+    CommandReport report = action.execute(null);
+    assertNotNull(report);
+    assertEquals(HostRoleStatus.COMPLETED.name(), report.getStatus());
+
+    // Verify the metainfo url
+    verifyBaseRepoURL(helper, cluster, host, "http://foo1");
+
+    // ensure that history now exists
+    historyEntites = serviceComponentDesiredStateDAO.findHistory(cluster.getClusterId(),
+        nnSCH.getServiceName(), nnSCH.getServiceComponentName());
+
+    assertEquals(1, historyEntites.size());
+  }
+
+
   private ServiceComponentHost createNewServiceComponentHost(Cluster cluster, String svc,
                                                              String svcComponent, String hostName) throws AmbariException {
     Assert.assertNotNull(cluster.getConfigGroups());