AMBARI-21719. Result of install_packages uses version string instead of 
repo_version_id (ncole)


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

Branch: refs/heads/feature-improved-rack-awareness-handling
Commit: f206e95711ab3d117cfc26587c9d09c52c86c16f
Parents: f134cee
Author: Nate Cole <nc...@hortonworks.com>
Authored: Mon Aug 14 15:25:11 2017 -0400
Committer: Nate Cole <nc...@hortonworks.com>
Committed: Mon Aug 14 15:25:11 2017 -0400

----------------------------------------------------------------------
 .../libraries/functions/repository_util.py      |   1 +
 .../DistributeRepositoriesStructuredOutput.java |  70 -------------
 .../DistributeRepositoriesActionListener.java   | 104 +++++++++----------
 .../stack/upgrade/RepositoryVersionHelper.java  |  15 ---
 .../custom_actions/scripts/install_packages.py  |  66 +++---------
 .../server/agent/HeartbeatProcessorTest.java    |  65 +++++++++++-
 ...ClusterStackVersionResourceProviderTest.java |   4 +-
 .../custom_actions/TestInstallPackages.py       |  86 ++++++---------
 .../configs/install_packages_config.json        |   2 +-
 9 files changed, 163 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-common/src/main/python/resource_management/libraries/functions/repository_util.py
----------------------------------------------------------------------
diff --git 
a/ambari-common/src/main/python/resource_management/libraries/functions/repository_util.py
 
b/ambari-common/src/main/python/resource_management/libraries/functions/repository_util.py
index c9920e2..6ad1aee 100644
--- 
a/ambari-common/src/main/python/resource_management/libraries/functions/repository_util.py
+++ 
b/ambari-common/src/main/python/resource_management/libraries/functions/repository_util.py
@@ -94,6 +94,7 @@ class CommandRepository(object):
     # version_id is the primary id of the repo_version table in the database
     self.version_id = _find_value(json_dict, 'repoVersionId')
     self.stack_name = _find_value(json_dict, 'stackName')
+    self.version_string = _find_value(json_dict, 'repoVersion')
     self.repositories = []
 
     repos_def = _find_value(json_dict, 'repositories')

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/DistributeRepositoriesStructuredOutput.java
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/DistributeRepositoriesStructuredOutput.java
 
b/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/DistributeRepositoriesStructuredOutput.java
deleted file mode 100644
index d8ae32f..0000000
--- 
a/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/DistributeRepositoriesStructuredOutput.java
+++ /dev/null
@@ -1,70 +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.bootstrap;
-
-import com.google.gson.annotations.SerializedName;
-
-/**
- * This class is used for mapping json of structured output for
- * "Distribute repositories/install packages" action.
- */
-public class DistributeRepositoriesStructuredOutput {
-
-  /**
-   * Repository version that has been (re)installed as a result of current 
custom action
-   */
-  @SerializedName("installed_repository_version")
-  private String installedRepositoryVersion;
-
-  /**
-   * Either SUCCESS or FAIL
-   */
-  @SerializedName("package_installation_result")
-  private String packageInstallationResult;
-
-  /**
-   * The actual version returned, even when a failure during install occurs.
-   */
-  @SerializedName("actual_version")
-  private String actualVersion;
-
-  /**
-   * The stack id used to look up version
-   */
-  @SerializedName("stack_id")
-  private String stackId;
-
-  public String getInstalledRepositoryVersion() {
-
-    return installedRepositoryVersion;
-  }
-
-  public String getPackageInstallationResult() {
-    return packageInstallationResult;
-  }
-
-  public String getActualVersion() {
-    return actualVersion;
-  }
-
-  public String getStackId() {
-    return stackId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/DistributeRepositoriesActionListener.java
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/DistributeRepositoriesActionListener.java
 
b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/DistributeRepositoriesActionListener.java
index c3c8f67..aa29894 100644
--- 
a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/DistributeRepositoriesActionListener.java
+++ 
b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/upgrade/DistributeRepositoriesActionListener.java
@@ -21,23 +21,21 @@ import java.util.List;
 
 import org.apache.ambari.server.EagerSingleton;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
-import 
org.apache.ambari.server.bootstrap.DistributeRepositoriesStructuredOutput;
 import org.apache.ambari.server.events.ActionFinalReportReceivedEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 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.state.Clusters;
 import org.apache.ambari.server.state.RepositoryVersionState;
-import org.apache.ambari.server.state.StackId;
-import org.apache.ambari.server.utils.StageUtils;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.eventbus.Subscribe;
+import com.google.gson.Gson;
 import com.google.gson.JsonSyntaxException;
+import com.google.gson.annotations.SerializedName;
 import com.google.inject.Inject;
 import com.google.inject.Provider;
 import com.google.inject.Singleton;
@@ -61,11 +59,10 @@ public class DistributeRepositoriesActionListener {
   private Provider<HostVersionDAO> hostVersionDAO;
 
   @Inject
-  private Provider<Clusters> clusters;
-
-  @Inject
   private RepositoryVersionDAO repoVersionDAO;
 
+  @Inject
+  private Gson gson;
 
   /**
    * Constructor.
@@ -88,7 +85,6 @@ public class DistributeRepositoriesActionListener {
       LOG.debug(event.toString());
     }
 
-    RepositoryVersionState newHostState = 
RepositoryVersionState.INSTALL_FAILED;
     Long clusterId = event.getClusterId();
     if (clusterId == null) {
       LOG.error("Distribute Repositories expected a cluster Id for host " + 
event.getHostname());
@@ -96,6 +92,7 @@ public class DistributeRepositoriesActionListener {
     }
 
     String repositoryVersion = null;
+    RepositoryVersionState newHostState = 
RepositoryVersionState.INSTALL_FAILED;
 
     if (event.getCommandReport() == null) {
       LOG.error(
@@ -106,56 +103,31 @@ public class DistributeRepositoriesActionListener {
           "Distribute repositories did not complete, will set all INSTALLING 
versions for host {} to INSTALL_FAILED.",
           event.getHostname());
     } else {
-      // Parse structured output
+
+      DistributeRepositoriesStructuredOutput structuredOutput = null;
       try {
+        structuredOutput = 
gson.fromJson(event.getCommandReport().getStructuredOut(),
+            DistributeRepositoriesStructuredOutput.class);
+      } catch (JsonSyntaxException e) {
+        LOG.error("Cannot parse structured output %s", e);
+      }
+
+      if (null == structuredOutput || null == 
structuredOutput.repositoryVersionId) {
+        LOG.error("Received an installation reponse, but it did not contain a 
repository version id");
+      } else {
         newHostState = RepositoryVersionState.INSTALLED;
-        DistributeRepositoriesStructuredOutput structuredOutput = 
StageUtils.getGson().fromJson(
-                event.getCommandReport().getStructuredOut(),
-                DistributeRepositoriesStructuredOutput.class);
-
-        repositoryVersion = structuredOutput.getInstalledRepositoryVersion();
-
-        // Handle the case in which the version to install did not contain the 
build number,
-        // but the structured output does contain the build number.
-        if (!StringUtils.isEmpty(structuredOutput.getActualVersion())
-            && 
!StringUtils.isEmpty(structuredOutput.getInstalledRepositoryVersion())
-            && !StringUtils.isEmpty(structuredOutput.getStackId())
-            && !StringUtils.equals(structuredOutput.getActualVersion(),
-                structuredOutput.getInstalledRepositoryVersion())) {
-
-          // !!! getInstalledRepositoryVersion() from the agent is the one
-          // entered in the UI.  getActualVersion() is computed.
-
-          StackId stackId = new StackId(structuredOutput.getStackId());
-          RepositoryVersionEntity version = 
repoVersionDAO.findByStackAndVersion(
-              stackId, structuredOutput.getInstalledRepositoryVersion());
-
-          if (null != version) {
-            LOG.info("Repository version {} was found, but {} is the actual 
value",
-                structuredOutput.getInstalledRepositoryVersion(),
-                structuredOutput.getActualVersion());
-            // !!! the entered version is not correct
-            version.setVersion(structuredOutput.getActualVersion());
-            repoVersionDAO.merge(version);
-            repositoryVersion = structuredOutput.getActualVersion();
-          } else {
-            // !!! extra check that the actual version is correct
-            stackId = new StackId(structuredOutput.getStackId());
-            version = repoVersionDAO.findByStackAndVersion(stackId,
-                structuredOutput.getActualVersion());
-
-            LOG.debug("Repository version {} was not found, check for {}.  
Found={}",
-                structuredOutput.getInstalledRepositoryVersion(),
-                structuredOutput.getActualVersion(),
-                Boolean.valueOf(null != version));
-
-            if (null != version) {
-              repositoryVersion = structuredOutput.getActualVersion();
-            }
+
+        String actualVersion = structuredOutput.actualVersion;
+
+        RepositoryVersionEntity repoVersion = 
repoVersionDAO.findByPK(structuredOutput.repositoryVersionId);
+
+        if (null != repoVersion && StringUtils.isNotBlank(actualVersion)) {
+          if (!StringUtils.equals(repoVersion.getVersion(), actualVersion)) {
+            repoVersion.setVersion(actualVersion);
+            repoVersionDAO.merge(repoVersion);
+            repositoryVersion = actualVersion;
           }
         }
-      } catch (JsonSyntaxException e) {
-        LOG.error("Cannot parse structured output %s", e);
       }
     }
 
@@ -179,4 +151,28 @@ public class DistributeRepositoriesActionListener {
       }
     }
   }
+
+  /**
+   * Used only to parse the structured output of a distribute versions call
+   */
+  private static class DistributeRepositoriesStructuredOutput {
+    /**
+     * Either SUCCESS or FAIL
+     */
+    @SerializedName("package_installation_result")
+    private String packageInstallationResult;
+
+    /**
+     * The actual version returned, even when a failure during install occurs.
+     */
+    @SerializedName("actual_version")
+    private String actualVersion;
+
+    /**
+     * The repository id that is returned in structured output.
+     */
+    @SerializedName("repository_version_id")
+    private Long repositoryVersionId = null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/RepositoryVersionHelper.java
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/RepositoryVersionHelper.java
 
b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/RepositoryVersionHelper.java
index 471e30f..bf47af0 100644
--- 
a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/RepositoryVersionHelper.java
+++ 
b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/upgrade/RepositoryVersionHelper.java
@@ -45,7 +45,6 @@ import org.apache.ambari.server.state.RepositoryInfo;
 import org.apache.ambari.server.state.ServiceInfo;
 import org.apache.ambari.server.state.ServiceOsSpecific;
 import org.apache.ambari.server.state.StackId;
-import org.apache.ambari.server.state.repository.VersionDefinitionXml;
 import org.apache.ambari.server.state.stack.UpgradePack;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
@@ -291,22 +290,8 @@ public class RepositoryVersionHelper {
 
     Map<String, String> roleParams = new HashMap<>();
     roleParams.put("stack_id", stackId.getStackId());
-    roleParams.put("repository_version", repoVersion.getVersion());
     // !!! TODO make roleParams <String, Object> so we don't have to do this 
awfulness.
     roleParams.put(KeyNames.PACKAGE_LIST, gson.toJson(packages));
-    roleParams.put(KeyNames.REPO_VERSION_ID, repoVersion.getId().toString());
-
-    VersionDefinitionXml xml = null;
-    try {
-      xml = repoVersion.getRepositoryXml();
-    } catch (Exception e) {
-      throw new SystemException(String.format("Could not load xml from repo 
version %s",
-          repoVersion.getVersion()));
-    }
-
-    if (null != xml && 
StringUtils.isNotBlank(xml.getPackageVersion(osFamily))) {
-      roleParams.put(KeyNames.PACKAGE_VERSION, 
xml.getPackageVersion(osFamily));
-    }
 
     return roleParams;
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/main/resources/custom_actions/scripts/install_packages.py
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/resources/custom_actions/scripts/install_packages.py 
b/ambari-server/src/main/resources/custom_actions/scripts/install_packages.py
index ef5607e..f24b811 100644
--- 
a/ambari-server/src/main/resources/custom_actions/scripts/install_packages.py
+++ 
b/ambari-server/src/main/resources/custom_actions/scripts/install_packages.py
@@ -18,11 +18,10 @@ limitations under the License.
 
 """
 import signal
-
+import os
 import re
 
 import ambari_simplejson as json
-import sys, traceback
 
 from ambari_commons.os_check import OSCheck
 from ambari_commons.str_utils import cbool, cint
@@ -60,6 +59,11 @@ class InstallPackages(Script):
     # Parse parameters
     config = Script.get_config()
 
+    try:
+      command_repository = CommandRepository(config['repositoryFile'])
+    except KeyError:
+      raise Fail("The command repository indicated by 'repositoryFile' was not 
found")
+
     repo_rhel_suse = 
config['configurations']['cluster-env']['repo_suse_rhel_template']
     repo_ubuntu = 
config['configurations']['cluster-env']['repo_ubuntu_template']
     template = repo_rhel_suse if OSCheck.is_redhat_family() or 
OSCheck.is_suse_family() else repo_ubuntu
@@ -68,30 +72,15 @@ class InstallPackages(Script):
     signal.signal(signal.SIGTERM, self.abort_handler)
     signal.signal(signal.SIGINT, self.abort_handler)
 
-    self.repository_version_id = None
+    self.repository_version = command_repository.version_string
 
-    base_urls = []
     # Select dict that contains parameters
     try:
-      if 'base_urls' in config['roleParams']:
-        base_urls = json.loads(config['roleParams']['base_urls'])
-
-      self.repository_version = config['roleParams']['repository_version']
       package_list = json.loads(config['roleParams']['package_list'])
       stack_id = config['roleParams']['stack_id']
-
-      if 'repository_version_id' in config['roleParams']:
-        self.repository_version_id = 
config['roleParams']['repository_version_id']
     except KeyError:
       pass
 
-    # current stack information
-    self.current_stack_version_formatted = None
-    if 'stack_version' in config['hostLevelParams']:
-      current_stack_version_unformatted = 
str(config['hostLevelParams']['stack_version'])
-      self.current_stack_version_formatted = 
format_stack_version(current_stack_version_unformatted)
-
-
     self.stack_name = Script.get_stack_name()
     if self.stack_name is None:
       raise Fail("Cannot determine the stack name")
@@ -105,48 +94,25 @@ class InstallPackages(Script):
 
     self.repository_version = self.repository_version.strip()
 
-    # Install/update repositories
-    self.current_repositories = []
-    self.current_repo_files = set()
-
-    # Enable base system repositories
-    # We don't need that for RHEL family, because we leave all repos enabled
-    # except disabled HDP* ones
-    if OSCheck.is_suse_family():
-      self.current_repositories.append('base')
-    elif OSCheck.is_ubuntu_family():
-      self.current_repo_files.add('base')
-
-    Logger.info("Will install packages for repository version 
{0}".format(self.repository_version))
-
-    if 0 == len(base_urls):
-      Logger.warning("Repository list is empty. Ambari may not be managing the 
repositories for {0}.".format(self.repository_version))
-
     try:
-      if 'repositoryFile' in config:
-        create_repo_files(template, 
CommandRepository(config['repositoryFile']))
+      if 0 == len(command_repository.repositories):
+        Logger.warning(
+          "Repository list is empty. Ambari may not be managing the 
repositories for {0}.".format(
+            self.repository_version))
       else:
-        append_to_file = False
-        for url_info in base_urls:
-          repo_name, repo_file = self.install_repository(url_info, 
append_to_file, template)
-          self.current_repositories.append(repo_name)
-          self.current_repo_files.add(repo_file)
-          append_to_file = True
-
+        Logger.info(
+          "Will install packages for repository version 
{0}".format(self.repository_version))
+        create_repo_files(template, command_repository)
     except Exception, err:
       Logger.logger.exception("Cannot install repository files. Error: 
{0}".format(str(err)))
       num_errors += 1
 
     # Build structured output with initial values
     self.structured_output = {
-      'installed_repository_version': self.repository_version,
-      'stack_id': stack_id,
-      'package_installation_result': 'FAIL'
+      'package_installation_result': 'FAIL',
+      'repository_version_id': command_repository.version_id
     }
 
-    if self.repository_version_id is not None:
-      self.structured_output['repository_version_id'] = 
self.repository_version_id
-
     self.put_structured_out(self.structured_output)
 
     if num_errors > 0:

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatProcessorTest.java
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatProcessorTest.java
 
b/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatProcessorTest.java
index db611fa..e199724 100644
--- 
a/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatProcessorTest.java
+++ 
b/ambari-server/src/test/java/org/apache/ambari/server/agent/HeartbeatProcessorTest.java
@@ -1133,7 +1133,6 @@ public class HeartbeatProcessorTest {
     json.addProperty("installed_repository_version", "0.1-1234");
     json.addProperty("stack_id", 
cluster.getDesiredStackVersion().getStackId());
 
-
     CommandReport cmdReport = new CommandReport();
     cmdReport.setActionId(StageUtils.getActionId(requestId, stageId));
     cmdReport.setTaskId(1);
@@ -1161,7 +1160,69 @@ public class HeartbeatProcessorTest {
 
     heartbeatProcessor.processHeartbeat(hb);
 
-    entity = dao.findByStackAndVersion(stackId, "0.1-1234");
+    entity = dao.findByStackAndVersion(stackId, "2.2.1.0-2222");
+    Assert.assertNull(entity);
+
+    entity = dao.findByStackAndVersion(stackId, "0.1.1");
+    Assert.assertNotNull(entity);
+  }
+
+  @Test
+  public void testInstallPackagesWithId() throws Exception {
+    // required since this test method checks the DAO result of handling a
+    // heartbeat which performs some async tasks
+    EventBusSynchronizer.synchronizeAmbariEventPublisher(injector);
+
+    final HostRoleCommand command = 
hostRoleCommandFactory.create(DummyHostname1,
+        Role.DATANODE, null, null);
+
+    ActionManager am = actionManagerTestHelper.getMockActionManager();
+    expect(am.getTasks(EasyMock.<List<Long>>anyObject())).andReturn(
+        Collections.singletonList(command)).anyTimes();
+    replay(am);
+
+    Cluster cluster = heartbeatTestHelper.getDummyCluster();
+
+    RepositoryVersionDAO dao = 
injector.getInstance(RepositoryVersionDAO.class);
+    RepositoryVersionEntity entity = 
helper.getOrCreateRepositoryVersion(cluster);
+    Assert.assertNotNull(entity);
+
+    HeartBeatHandler handler = heartbeatTestHelper.getHeartBeatHandler(am, new 
ActionQueue());
+    HeartbeatProcessor heartbeatProcessor = handler.getHeartbeatProcessor();
+    HeartBeat hb = new HeartBeat();
+
+    JsonObject json = new JsonObject();
+    json.addProperty("actual_version", "2.2.1.0-2222");
+    json.addProperty("package_installation_result", "SUCCESS");
+    json.addProperty("repository_version_id", entity.getId());
+
+    CommandReport cmdReport = new CommandReport();
+    cmdReport.setActionId(StageUtils.getActionId(requestId, stageId));
+    cmdReport.setTaskId(1);
+    cmdReport.setCustomCommand("install_packages");
+    cmdReport.setStructuredOut(json.toString());
+    cmdReport.setRoleCommand(RoleCommand.ACTIONEXECUTE.name());
+    cmdReport.setStatus(HostRoleStatus.COMPLETED.name());
+    cmdReport.setRole("install_packages");
+    cmdReport.setClusterName(DummyCluster);
+
+    List<CommandReport> reports = new ArrayList<>();
+    reports.add(cmdReport);
+    hb.setReports(reports);
+    hb.setTimestamp(0L);
+    hb.setResponseId(0);
+    hb.setNodeStatus(new HostStatus(HostStatus.Status.HEALTHY, 
DummyHostStatus));
+    hb.setHostname(DummyHostname1);
+    hb.setComponentStatus(new ArrayList<ComponentStatus>());
+
+    StackId stackId = new StackId("HDP", "0.1");
+
+    heartbeatProcessor.processHeartbeat(hb);
+
+    entity = dao.findByStackAndVersion(stackId, "2.2.1.0-2222");
+    Assert.assertNotNull(entity);
+
+    entity = dao.findByStackAndVersion(stackId, "0.1.1");
     Assert.assertNull(entity);
   }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterStackVersionResourceProviderTest.java
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterStackVersionResourceProviderTest.java
 
b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterStackVersionResourceProviderTest.java
index 39aee82..7d67726 100644
--- 
a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterStackVersionResourceProviderTest.java
+++ 
b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/ClusterStackVersionResourceProviderTest.java
@@ -819,7 +819,8 @@ public class ClusterStackVersionResourceProviderTest {
     Float successFactor = successFactors.get(Role.INSTALL_PACKAGES);
     Assert.assertEquals(Float.valueOf(0.85f), successFactor);
 
-    
Assert.assertTrue(executionCommand.getRoleParams().containsKey(KeyNames.PACKAGE_VERSION));
+    
Assert.assertTrue(executionCommand.getRoleParams().containsKey(KeyNames.PACKAGE_LIST));
+    
Assert.assertTrue(executionCommand.getRoleParams().containsKey("stack_id"));
   }
 
    @Test
@@ -1057,7 +1058,6 @@ public class ClusterStackVersionResourceProviderTest {
     Float successFactor = successFactors.get(Role.INSTALL_PACKAGES);
     Assert.assertEquals(Float.valueOf(0.85f), successFactor);
 
-    
Assert.assertTrue(executionCommand.getRoleParams().containsKey(KeyNames.PACKAGE_VERSION));
     Assert.assertNotNull(executionCommand.getRepositoryFile());
     Assert.assertEquals(0, 
executionCommand.getRepositoryFile().getRepositories().size());
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/test/python/custom_actions/TestInstallPackages.py
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/test/python/custom_actions/TestInstallPackages.py 
b/ambari-server/src/test/python/custom_actions/TestInstallPackages.py
index 3e90c64..56a9fed 100644
--- a/ambari-server/src/test/python/custom_actions/TestInstallPackages.py
+++ b/ambari-server/src/test/python/custom_actions/TestInstallPackages.py
@@ -99,8 +99,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertResourceCalled('Repository', 'HDP-UTILS-1.1.0.20',
                               
base_url=u'http://repo1/HDP/centos5/2.x/updates/2.2.0.0',
@@ -150,9 +149,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['base_urls'] = "[]"
-    del command_json['repositoryFile']
-
+    command_json['repositoryFile']['repositories'] = []
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value=[]
@@ -166,8 +163,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     
     self.assertResourceCalled('Package', 'hdp-select', action=["upgrade"], 
retry_count=5, retry_on_repo_unavailability=False)
@@ -209,8 +205,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertResourceCalled('Repository', 'HDP-UTILS-1.1.0.20',
                               
base_url=u'http://repo1/HDP/centos5/2.x/updates/2.2.0.0',
@@ -271,8 +266,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertResourceCalled('Repository', 'HDP-UTILS-1.1.0.20',
                               
base_url=u'http://repo1/HDP/centos5/2.x/updates/2.2.0.0',
@@ -347,8 +341,7 @@ class TestInstallPackages(RMFTestCase):
 
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
-                      {'stack_id': 'HDP-2.2',
-                      'installed_repository_version': VERSION_STUB,
+                      {'repository_version_id': 1,
                       'package_installation_result': 'FAIL'})
     self.assertResourceCalled('Repository', 'HDP-UTILS-1.1.0.20',
                               
base_url=u'http://repo1/HDP/centos5/2.x/updates/2.2.0.0',
@@ -403,8 +396,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertResourceCalled('Repository', 'HDP-UTILS-1.1.0.20',
                               
base_url=u'http://repo1/HDP/centos5/2.x/updates/2.2.0.0',
@@ -474,8 +466,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 4,
                        'actual_version': VERSION_STUB})
     self.assertResourceCalled('Repository', 'HDP-UTILS-1.1.0.20-repo-4',
                               
base_url=u'http://repo1/HDP-UTILS/centos5/2.x/updates/2.2.0.0',
@@ -529,7 +520,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['repository_version'] = VERSION_STUB
+    command_json['repositoryFile']['repoVersion'] = VERSION_STUB
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value = []
@@ -543,8 +534,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertTrue(write_actual_version_to_history_file_mock.called)
     self.assertEquals(write_actual_version_to_history_file_mock.call_args[0], 
(VERSION_STUB_WITHOUT_BUILD_NUMBER, VERSION_STUB))
@@ -564,7 +554,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['repository_version'] = VERSION_STUB
+    command_json['repositoryFile']['repoVersion'] = VERSION_STUB
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value = []
@@ -578,14 +568,12 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
 
 
-
   
@patch("resource_management.libraries.functions.list_ambari_managed_repos.list_ambari_managed_repos")
   
@patch("resource_management.libraries.functions.packages_analyzer.allInstalledPackages")
   @patch("resource_management.libraries.script.Script.put_structured_out")
@@ -630,10 +618,9 @@ class TestInstallPackages(RMFTestCase):
 
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args_list[-1][0][0],
-                      {
+                      { 'actual_version': '2.2.0.1-885',
                         'package_installation_result': 'FAIL',
-                        'installed_repository_version': '2.2.0.1',
-                        'stack_id': u'HDP-2.2'})
+                        'repository_version_id': 1})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
 
@@ -666,7 +653,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['repository_version'] = 
VERSION_STUB_WITHOUT_BUILD_NUMBER
+    command_json['repositoryFile']['repoVersion'] = 
VERSION_STUB_WITHOUT_BUILD_NUMBER
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value = []
@@ -685,8 +672,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args_list[-1][0][0],
                       {'package_installation_result': 'FAIL',
-                       'stack_id': u'HDP-2.2',
-                       'installed_repository_version': '2.2.0.1'})
+                       'repository_version_id': 1})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
 
@@ -706,7 +692,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['repository_version'] = VERSION_STUB
+    command_json['repositoryFile']['repoVersion'] = VERSION_STUB
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value = []
@@ -725,8 +711,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'FAIL',
-                       'stack_id': u'HDP-2.2',
-                       'installed_repository_version': VERSION_STUB,
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
@@ -766,8 +751,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': 
VERSION_STUB_WITHOUT_BUILD_NUMBER,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertTrue(write_actual_version_to_history_file_mock.called)
     self.assertEquals(write_actual_version_to_history_file_mock.call_args[0], 
(VERSION_STUB_WITHOUT_BUILD_NUMBER, VERSION_STUB))
@@ -787,7 +771,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['repository_version'] = 
VERSION_STUB_WITHOUT_BUILD_NUMBER
+    command_json['repositoryFile']['repoVersion'] = 
VERSION_STUB_WITHOUT_BUILD_NUMBER
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value = []
@@ -801,8 +785,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': 
VERSION_STUB_WITHOUT_BUILD_NUMBER,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
@@ -828,7 +811,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['repository_version'] = '2.2.0.1-500'  # User 
specified wrong build number
+    command_json['repositoryFile']['repoVersion'] = '2.2.0.1-500'  # User 
specified wrong build number
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value = []
@@ -842,8 +825,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': '2.2.0.1-500',
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertTrue(write_actual_version_to_history_file_mock.called)
     self.assertEquals(write_actual_version_to_history_file_mock.call_args[0], 
('2.2.0.1', VERSION_STUB))
@@ -877,8 +859,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': '2.2.0.1-500',
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
@@ -908,7 +889,7 @@ class TestInstallPackages(RMFTestCase):
     with open(config_file, "r") as f:
       command_json = json.load(f)
 
-    command_json['roleParams']['repository_version'] = 
VERSION_STUB_WITHOUT_BUILD_NUMBER
+    command_json['repositoryFile']['repoVersion'] = 
VERSION_STUB_WITHOUT_BUILD_NUMBER
 
     allInstalledPackages_mock.side_effect = TestInstallPackages._add_packages
     list_ambari_managed_repos_mock.return_value = []
@@ -927,8 +908,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args_list[-1][0][0],
                       {'package_installation_result': 'FAIL',
-                       'stack_id': u'HDP-2.2',
-                       'installed_repository_version': '2.2.0.1'})
+                       'repository_version_id': 1})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
 
@@ -967,8 +947,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'FAIL',
-                       'stack_id': u'HDP-2.2',
-                       'installed_repository_version': VERSION_STUB,
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
@@ -1008,9 +987,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
-                       'repository_version_id': '2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
     self.assertTrue(write_actual_version_to_history_file_mock.called)
     self.assertEquals(write_actual_version_to_history_file_mock.call_args[0], 
(VERSION_STUB_WITHOUT_BUILD_NUMBER, VERSION_STUB))
@@ -1045,9 +1022,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
-                       'repository_version_id': '2',
+                       'repository_version_id': 1,
                        'actual_version': VERSION_STUB})
 
     self.assertFalse(write_actual_version_to_history_file_mock.called)
@@ -1081,8 +1056,7 @@ class TestInstallPackages(RMFTestCase):
     self.assertTrue(put_structured_out_mock.called)
     self.assertEquals(put_structured_out_mock.call_args[0][0],
                       {'package_installation_result': 'SUCCESS',
-                       'installed_repository_version': VERSION_STUB,
-                       'stack_id': 'HDP-2.2',
+                       'repository_version_id': 4,
                        'actual_version': VERSION_STUB})
 
     self.assertResourceCalled('Repository', 'HDP-UTILS-1.1.0.20-repo-4',

http://git-wip-us.apache.org/repos/asf/ambari/blob/f206e957/ambari-server/src/test/python/custom_actions/configs/install_packages_config.json
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/test/python/custom_actions/configs/install_packages_config.json
 
b/ambari-server/src/test/python/custom_actions/configs/install_packages_config.json
index be010b3..4bebe99 100644
--- 
a/ambari-server/src/test/python/custom_actions/configs/install_packages_config.json
+++ 
b/ambari-server/src/test/python/custom_actions/configs/install_packages_config.json
@@ -53,7 +53,7 @@
     "repositoryFile": {
         "stackName": "HDP",
         "repoVersionId": 1,
-        "repoVersion": "2",
+        "repoVersion": "2.2.0.1-885",
         "repositories": [
             {
                 "repoName": "HDP-UTILS",

Reply via email to