You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by nc...@apache.org on 2017/05/03 19:28:57 UTC

ambari git commit: AMBARI-20923. Repositories must be resolved correctly when installing new components (ncole)

Repository: ambari
Updated Branches:
  refs/heads/branch-feature-AMBARI-12556 d84abbfd3 -> 8782cf691


AMBARI-20923. Repositories must be resolved correctly when installing new components (ncole)


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

Branch: refs/heads/branch-feature-AMBARI-12556
Commit: 8782cf69160dae3d3894d7d93a6eb3f2c384e237
Parents: d84abbf
Author: Nate Cole <nc...@hortonworks.com>
Authored: Wed May 3 13:28:13 2017 -0400
Committer: Nate Cole <nc...@hortonworks.com>
Committed: Wed May 3 15:28:40 2017 -0400

----------------------------------------------------------------------
 .../libraries/script/script.py                  | 20 +++--
 .../ambari/server/agent/CommandRepository.java  | 14 ++++
 .../ambari/server/agent/ExecutionCommand.java   |  6 ++
 .../AmbariCustomCommandExecutionHelper.java     | 83 +++++++++++++-------
 .../AmbariManagementControllerImpl.java         |  6 +-
 .../ServiceComponentDesiredStateEntity.java     | 16 ++--
 .../ambari/server/topology/AmbariContext.java   |  4 +-
 .../AmbariCustomCommandExecutionHelperTest.java | 74 ++++++++++++++++-
 .../upgrades/UpgradeActionTest.java             | 42 +---------
 .../custom_actions/TestInstallPackages.py       | 70 +++++++++++++++++
 10 files changed, 249 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-common/src/main/python/resource_management/libraries/script/script.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/script/script.py b/ambari-common/src/main/python/resource_management/libraries/script/script.py
index 0dd9c02..75a1dd4 100644
--- a/ambari-common/src/main/python/resource_management/libraries/script/script.py
+++ b/ambari-common/src/main/python/resource_management/libraries/script/script.py
@@ -463,14 +463,22 @@ class Script(object):
     version is passed from the server, use that as an absolute truth.
     """
 
-    # two different command types put things in different objects.  WHY.
-    # package_version is the form W_X_Y_Z_nnnn
-    package_version = default("roleParams/package_version", None)
-    if not package_version:
-      package_version = default("hostLevelParams/package_version", None)
-
     package_delimiter = '-' if OSCheck.is_ubuntu_family() else '_'
 
+    # repositoryFile is the truth
+    # package_version should be made to the form W_X_Y_Z_nnnn
+    package_version = default("repositoryFile/repoVersion", None)
+    if package_version is not None:
+      package_version = package_version.replace('.', package_delimiter).replace('-', package_delimiter)
+
+    # TODO remove legacy checks
+    if package_version is None:
+      package_version = default("roleParams/package_version", None)
+
+    # TODO remove legacy checks
+    if package_version is None:
+      package_version = default("hostLevelParams/package_version", None)
+
     # The cluster effective version comes down when the version is known after the initial
     # install.  In that case we should not be guessing which version when invoking INSTALL, but
     # use the supplied version to build the package_version

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandRepository.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandRepository.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandRepository.java
index 849d6fb..3d96122 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandRepository.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/CommandRepository.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.ambari.server.orm.entities.RepositoryEntity;
 import org.apache.ambari.server.state.RepositoryInfo;
+import org.apache.commons.lang.builder.ToStringBuilder;
 
 import com.google.gson.annotations.SerializedName;
 
@@ -165,6 +166,19 @@ public class CommandRepository {
     public String getBaseUrl() {
       return m_baseUrl;
     }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String toString() {
+      return new ToStringBuilder(null)
+          .append("os", m_osType)
+          .append("name", m_repoName)
+          .append("id", m_repoId)
+          .append("baseUrl", m_baseUrl)
+          .toString();
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
index bd62cbb..63eb660 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
@@ -431,6 +431,8 @@ public class ExecutionCommand extends AgentCommand {
     String STACK_NAME = "stack_name";
     String SERVICE_TYPE = "service_type";
     String STACK_VERSION = "stack_version";
+    @Deprecated
+    @Experimental(feature=ExperimentalFeature.PATCH_UPGRADES)
     String SERVICE_REPO_INFO = "service_repo_info";
     String PACKAGE_LIST = "package_list";
     String JDK_LOCATION = "jdk_location";
@@ -481,6 +483,8 @@ public class ExecutionCommand extends AgentCommand {
     /**
      * The key indicating that the package_version string is available
      */
+    @Deprecated
+    @Experimental(feature=ExperimentalFeature.PATCH_UPGRADES)
     String PACKAGE_VERSION = "package_version";
 
     /**
@@ -495,6 +499,8 @@ public class ExecutionCommand extends AgentCommand {
      * The agent will return this value back in its response so the repository
      * can be looked up and possibly have its version updated.
      */
+    @Deprecated
+    @Experimental(feature=ExperimentalFeature.PATCH_UPGRADES)
     String REPO_VERSION_ID = "repository_version_id";
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
index d6905fb..1d43093 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
@@ -78,6 +78,7 @@ import org.apache.ambari.server.controller.spi.Resource;
 import org.apache.ambari.server.metadata.ActionMetadata;
 import org.apache.ambari.server.orm.dao.ClusterVersionDAO;
 import org.apache.ambari.server.orm.dao.HostRoleCommandDAO;
+import org.apache.ambari.server.orm.dao.ServiceComponentDesiredStateDAO;
 import org.apache.ambari.server.orm.entities.ClusterVersionEntity;
 import org.apache.ambari.server.orm.entities.OperatingSystemEntity;
 import org.apache.ambari.server.orm.entities.RepositoryEntity;
@@ -182,6 +183,10 @@ public class AmbariCustomCommandExecutionHelper {
   @Inject
   private HostRoleCommandDAO hostRoleCommandDAO;
 
+  @Inject
+  private ServiceComponentDesiredStateDAO serviceComponentDAO;
+
+
   private Map<String, Map<String, Map<String, String>>> configCredentialsForService = new HashMap<>();
 
   protected static final String SERVICE_CHECK_COMMAND_NAME = "SERVICE_CHECK";
@@ -399,6 +404,11 @@ public class AmbariCustomCommandExecutionHelper {
       Service clusterService = cluster.getService(serviceName);
       execCmd.setCredentialStoreEnabled(String.valueOf(clusterService.isCredentialStoreEnabled()));
 
+      ServiceComponent component = null;
+      if (StringUtils.isNotBlank(componentName)) {
+        component = clusterService.getServiceComponent(componentName);
+      }
+
       // Get the map of service config type to password properties for the service
       Map<String, Map<String, String>> configCredentials;
       configCredentials = configCredentialsForService.get(clusterService.getName());
@@ -414,7 +424,7 @@ public class AmbariCustomCommandExecutionHelper {
       hostLevelParams.put(CUSTOM_COMMAND, commandName);
 
       // Set parameters required for re-installing clients on restart
-      hostLevelParams.put(REPO_INFO, getRepoInfo(cluster, host));
+      hostLevelParams.put(REPO_INFO, getRepoInfo(cluster, component, host));
       hostLevelParams.put(STACK_NAME, stackId.getStackName());
       hostLevelParams.put(STACK_VERSION, stackId.getStackVersion());
 
@@ -504,7 +514,7 @@ public class AmbariCustomCommandExecutionHelper {
       execCmd.setCommandParams(commandParams);
       execCmd.setRoleParams(roleParams);
 
-      execCmd.setRepositoryFile(getCommandRepository(cluster, host));
+      execCmd.setRepositoryFile(getCommandRepository(cluster, component, host));
 
       // perform any server side command related logic - eg - set desired states on restart
       applyCustomCommandBackendLogic(cluster, serviceName, componentName, commandName, hostName);
@@ -1179,7 +1189,7 @@ public class AmbariCustomCommandExecutionHelper {
    * @throws AmbariException if the repository information can not be obtained
    */
   @Deprecated
-  public String getRepoInfo(Cluster cluster, Host host) throws AmbariException {
+  public String getRepoInfo(Cluster cluster, ServiceComponent component, Host host) throws AmbariException {
 
     Function<List<RepositoryInfo>, JsonArray> function = new Function<List<RepositoryInfo>, JsonArray>() {
       @Override
@@ -1188,7 +1198,7 @@ public class AmbariCustomCommandExecutionHelper {
       }
     };
 
-    final JsonArray gsonList = getBaseUrls(cluster, host, function);
+    final JsonArray gsonList = getBaseUrls(cluster, component, host, function);
 
     if (null == gsonList) {
       return "";
@@ -1216,7 +1226,6 @@ public class AmbariCustomCommandExecutionHelper {
             if (ose.getOsType().equals(osType) && ose.isAmbariManagedRepos()) {
               for (RepositoryEntity re : ose.getRepositories()) {
                 if (re.getName().equals(repoName) &&
-                    re.getRepositoryId().equals(repoId) &&
                     !re.getBaseUrl().equals(baseUrl)) {
                   obj.addProperty("baseUrl", re.getBaseUrl());
                 }
@@ -1230,7 +1239,7 @@ public class AmbariCustomCommandExecutionHelper {
       }
     };
 
-    return updateBaseUrls(cluster, updater).toString();
+    return updateBaseUrls(cluster, component, updater).toString();
   }
 
   /**
@@ -1243,7 +1252,7 @@ public class AmbariCustomCommandExecutionHelper {
    * @throws AmbariException
    */
   @Experimental(feature=ExperimentalFeature.PATCH_UPGRADES)
-  public CommandRepository getCommandRepository(final Cluster cluster, Host host) throws AmbariException {
+  public CommandRepository getCommandRepository(final Cluster cluster, ServiceComponent component, Host host) throws AmbariException {
 
     Function<List<RepositoryInfo>, List<RepositoryInfo>> function = new Function<List<RepositoryInfo>, List<RepositoryInfo>>() {
       @Override
@@ -1253,7 +1262,7 @@ public class AmbariCustomCommandExecutionHelper {
       }
     };
 
-    final List<RepositoryInfo> repoInfos = getBaseUrls(cluster, host, function);
+    final List<RepositoryInfo> repoInfos = getBaseUrls(cluster, component, host, function);
 
     if (null == repoInfos) {
       return null;
@@ -1275,7 +1284,6 @@ public class AmbariCustomCommandExecutionHelper {
 
         for (CommandRepository.Repository commandRepo : command.getRepositories()) {
           String osType = commandRepo.getOsType();
-          String repoId = commandRepo.getRepoId();
           String repoName = commandRepo.getRepoName();
           String baseUrl = commandRepo.getBaseUrl();
 
@@ -1283,7 +1291,6 @@ public class AmbariCustomCommandExecutionHelper {
             if (ose.getOsType().equals(osType) && ose.isAmbariManagedRepos()) {
               for (RepositoryEntity re : ose.getRepositories()) {
                 if (re.getName().equals(repoName) &&
-                    re.getRepositoryId().equals(repoId) &&
                     !re.getBaseUrl().equals(baseUrl)) {
                   commandRepo.setBaseUrl(re.getBaseUrl());
                 }
@@ -1296,7 +1303,7 @@ public class AmbariCustomCommandExecutionHelper {
       }
     };
 
-    updateBaseUrls(cluster, updater);
+    updateBaseUrls(cluster, component, updater);
 
     return command;
   }
@@ -1306,13 +1313,15 @@ public class AmbariCustomCommandExecutionHelper {
    * implemenation, this may be removed and called inline in {@link #getCommandRepository(Cluster, Host)}
    *
    * @param cluster   the cluster to isolate the stack
+   * @param component the component
    * @param host      used to resolve the family for the repositories
    * @param function  function that will transform the supplied repositories for specific use.
    * @return <T> the type as defined by the supplied {@code function}.
    * @throws AmbariException
    */
   @Experimental(feature = ExperimentalFeature.PATCH_UPGRADES)
-  private <T> T getBaseUrls(Cluster cluster, Host host, Function<List<RepositoryInfo>, T> function) throws AmbariException {
+  private <T> T getBaseUrls(Cluster cluster, ServiceComponent component, Host host,
+      Function<List<RepositoryInfo>, T> function) throws AmbariException {
 
     String hostOsType = host.getOsType();
     String hostOsFamily = host.getOsFamily();
@@ -1354,31 +1363,51 @@ public class AmbariCustomCommandExecutionHelper {
    * @param <T> the result after appling the repository version, if found.
    */
   @Experimental(feature = ExperimentalFeature.PATCH_UPGRADES)
-  private <T> T updateBaseUrls(Cluster cluster, BaseUrlUpdater<T> function) throws AmbariException {
-    ClusterVersionEntity cve = cluster.getCurrentClusterVersion();
+  private <T> T updateBaseUrls(Cluster cluster, ServiceComponent component, BaseUrlUpdater<T> function) throws AmbariException {
 
-    if (null == cve) {
-      List<ClusterVersionEntity> list = clusterVersionDAO.findByClusterAndState(cluster.getClusterName(),
-          RepositoryVersionState.INIT);
+    RepositoryVersionEntity repositoryEntity = null;
 
-      if (!list.isEmpty()) {
-        if (list.size() > 1) {
-          throw new AmbariException(String.format("The cluster can only be initialized by one version: %s found",
-              list.size()));
-        } else {
-          cve = list.get(0);
+    // !!! try to find the component repo first
+    if (null != component) {
+      repositoryEntity = component.getDesiredRepositoryVersion();
+    }
+
+    if (null == component) {
+      LOG.info("Service component not passed in, attempt to resolve the repository for cluster {}",
+          cluster.getClusterName());
+    }
+
+    if (null == repositoryEntity) {
+
+      ClusterVersionEntity cve = cluster.getCurrentClusterVersion();
+
+      if (null == cve) {
+        List<ClusterVersionEntity> list = clusterVersionDAO.findByClusterAndState(cluster.getClusterName(),
+            RepositoryVersionState.INIT);
+
+        if (!list.isEmpty()) {
+          if (list.size() > 1) {
+            throw new AmbariException(String.format("The cluster can only be initialized by one version: %s found",
+                list.size()));
+          } else {
+            cve = list.get(0);
+          }
         }
       }
+
+      if (null != cve && null != cve.getRepositoryVersion()) {
+        repositoryEntity = cve.getRepositoryVersion();
+      } else {
+        LOG.info("Cluster {} has no specific Repository Versions.  Using stack-defined values", cluster.getClusterName());
+      }
     }
 
-    if (null == cve || null == cve.getRepositoryVersion()) {
+    if (null == repositoryEntity) {
       LOG.info("Cluster {} has no specific Repository Versions.  Using stack-defined values", cluster.getClusterName());
       return function.getDefault();
     }
 
-    RepositoryVersionEntity rve = cve.getRepositoryVersion();
-
-    return function.apply(rve);
+    return function.apply(repositoryEntity);
   }
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
index 8995e51..67ae5d5 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
@@ -2232,6 +2232,8 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
     Service clusterService = cluster.getService(serviceName);
     execCmd.setCredentialStoreEnabled(String.valueOf(clusterService.isCredentialStoreEnabled()));
 
+    ServiceComponent component = clusterService.getServiceComponent(componentName);
+
     // Get the map of service config type to password properties for the service
     Map<String, Map<String, String>> configCredentials;
     configCredentials = configCredentialsForService.get(clusterService.getName());
@@ -2354,7 +2356,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
       commandParams.put(ExecutionCommand.KeyNames.REFRESH_TOPOLOGY, "True");
     }
 
-    String repoInfo = customCommandExecutionHelper.getRepoInfo(cluster, host);
+    String repoInfo = customCommandExecutionHelper.getRepoInfo(cluster, component, host);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Sending repo information to agent"
         + ", hostname=" + scHost.getHostName()
@@ -2470,7 +2472,7 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
     execCmd.setCommandParams(commandParams);
 
     execCmd.setAvailableServicesFromServiceInfoMap(ambariMetaInfo.getServices(stackId.getStackName(), stackId.getStackVersion()));
-    execCmd.setRepositoryFile(customCommandExecutionHelper.getCommandRepository(cluster, host));
+    execCmd.setRepositoryFile(customCommandExecutionHelper.getCommandRepository(cluster, component, host));
 
     if ((execCmd != null) && (execCmd.getConfigurationTags().containsKey("cluster-env"))) {
       LOG.debug("AmbariManagementControllerImpl.createHostAction: created ExecutionCommand for host {}, role {}, roleCommand {}, and command ID {}, with cluster-env tags {}",

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
index eb1b187..6b89c02 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
@@ -123,7 +123,7 @@ public class ServiceComponentDesiredStateEntity {
   private Collection<ServiceComponentHistoryEntity> serviceComponentHistory;
 
   @OneToMany(mappedBy = "m_serviceComponentDesiredStateEntity", cascade = { CascadeType.ALL })
-  private Collection<ServiceComponentVersionEntity> serviceComponentVersion;
+  private Collection<ServiceComponentVersionEntity> serviceComponentVersions;
 
   public Long getId() {
     return id;
@@ -205,22 +205,22 @@ public class ServiceComponentDesiredStateEntity {
 
 
   /**
-   * @param versionEntry the version to add
+   * @param versionEntity the version to add
    */
-  public void addVersion(ServiceComponentVersionEntity versionEntry) {
-    if (null == serviceComponentVersion) {
-      serviceComponentVersion = new ArrayList<>();
+  public void addVersion(ServiceComponentVersionEntity versionEntity) {
+    if (null == serviceComponentVersions) {
+      serviceComponentVersions = new ArrayList<>();
     }
 
-    serviceComponentVersion.add(versionEntry);
-    versionEntry.setServiceComponentDesiredState(this);
+    serviceComponentVersions.add(versionEntity);
+    versionEntity.setServiceComponentDesiredState(this);
   }
 
   /**
    * @return the collection of versions for the component
    */
   public Collection<ServiceComponentVersionEntity> getVersions() {
-    return serviceComponentVersion;
+    return serviceComponentVersions;
   }
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-server/src/main/java/org/apache/ambari/server/topology/AmbariContext.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/topology/AmbariContext.java b/ambari-server/src/main/java/org/apache/ambari/server/topology/AmbariContext.java
index 4a1e61f..8ae8e54 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/topology/AmbariContext.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/topology/AmbariContext.java
@@ -212,8 +212,8 @@ public class AmbariContext {
     Set<ServiceComponentRequest> componentRequests = new HashSet<>();
     for (String service : services) {
       String credentialStoreEnabled = topology.getBlueprint().getCredentialStoreEnabled(service);
-      serviceRequests.add(new ServiceRequest(clusterName, service, null, stackId.getStackId(),
-          repositoryVersion, credentialStoreEnabled));
+      serviceRequests.add(new ServiceRequest(clusterName, service, stackId.getStackId(),
+          repositoryVersion, null, credentialStoreEnabled));
 
       for (String component : topology.getBlueprint().getComponents(service)) {
         String recoveryEnabled = topology.getBlueprint().getRecoveryEnabled(service, component);

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelperTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelperTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelperTest.java
index b0d085b..d558c15 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelperTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelperTest.java
@@ -37,6 +37,7 @@ import org.apache.ambari.server.actionmanager.HostRoleCommand;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
 import org.apache.ambari.server.actionmanager.Request;
 import org.apache.ambari.server.actionmanager.Stage;
+import org.apache.ambari.server.agent.CommandRepository;
 import org.apache.ambari.server.agent.ExecutionCommand;
 import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.controller.internal.ComponentResourceProviderTest;
@@ -49,7 +50,12 @@ import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
 import org.apache.ambari.server.orm.OrmTestHelper;
 import org.apache.ambari.server.orm.dao.RepositoryVersionDAO;
+import org.apache.ambari.server.orm.dao.ServiceComponentDesiredStateDAO;
+import org.apache.ambari.server.orm.dao.StackDAO;
 import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
+import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
+import org.apache.ambari.server.orm.entities.ServiceComponentVersionEntity;
+import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.security.TestAuthenticationFactory;
 import org.apache.ambari.server.security.authorization.AuthorizationException;
 import org.apache.ambari.server.state.Cluster;
@@ -60,6 +66,7 @@ import org.apache.ambari.server.state.Host;
 import org.apache.ambari.server.state.HostState;
 import org.apache.ambari.server.state.MaintenanceState;
 import org.apache.ambari.server.state.PropertyInfo;
+import org.apache.ambari.server.state.RepositoryInfo;
 import org.apache.ambari.server.state.SecurityType;
 import org.apache.ambari.server.state.Service;
 import org.apache.ambari.server.state.ServiceComponent;
@@ -69,6 +76,7 @@ import org.apache.ambari.server.state.StackInfo;
 import org.apache.ambari.server.state.State;
 import org.apache.ambari.server.state.UserGroupInfo;
 import org.apache.ambari.server.state.ValueAttributesInfo;
+import org.apache.ambari.server.state.stack.upgrade.RepositoryVersionHelper;
 import org.apache.ambari.server.topology.TopologyManager;
 import org.apache.ambari.server.utils.StageUtils;
 import org.easymock.Capture;
@@ -555,8 +563,72 @@ public class AmbariCustomCommandExecutionHelperTest {
     }
   }
 
+  @Test
+  public void testCommandRepository() throws Exception {
+    Cluster cluster = clusters.getCluster("c1");
+    Service serviceYARN = cluster.getService("YARN");
+    Service serviceZK = cluster.getService("ZOOKEEPER");
+    ServiceComponent componentRM = serviceYARN.getServiceComponent("RESOURCEMANAGER");
+    ServiceComponent componentZKC = serviceZK.getServiceComponent("ZOOKEEPER_CLIENT");
+    Host host = clusters.getHost("c1-c6401");
+
+    AmbariCustomCommandExecutionHelper helper = injector.getInstance(AmbariCustomCommandExecutionHelper.class);
+    StackDAO stackDAO = injector.getInstance(StackDAO.class);
+    RepositoryVersionDAO repoVersionDAO = injector.getInstance(RepositoryVersionDAO.class);
+    ServiceComponentDesiredStateDAO componentDAO = injector.getInstance(ServiceComponentDesiredStateDAO.class);
+    RepositoryVersionHelper repoVersionHelper = injector.getInstance(RepositoryVersionHelper.class);
+
+    CommandRepository commandRepo = helper.getCommandRepository(cluster, componentRM, host);
+
+    Assert.assertEquals(1, commandRepo.getRepositories().size());
+    CommandRepository.Repository repo = commandRepo.getRepositories().iterator().next();
+    Assert.assertEquals("http://public-repo-1.hortonworks.com/HDP/centos6/2.x/updates/2.0.6.0", repo.getBaseUrl());
+
+    RepositoryInfo ri = new RepositoryInfo();
+    ri.setBaseUrl("http://foo");
+    ri.setRepoName("HDP");
+    ri.setRepoId("new-id");
+    ri.setOsType("redhat6");
+    String operatingSystems = repoVersionHelper.serializeOperatingSystems(Collections.singletonList(ri));
+
+
+    StackEntity stackEntity = stackDAO.find(cluster.getDesiredStackVersion().getStackName(),
+        cluster.getDesiredStackVersion().getStackVersion());
+
+    RepositoryVersionEntity repositoryVersion = new RepositoryVersionEntity(stackEntity,
+        "2.1.1.1-1234", "2.1.1.1-1234", operatingSystems);
+    repositoryVersion = repoVersionDAO.merge(repositoryVersion);
+
+    // add a repo version associated with a component
+    ServiceComponentDesiredStateEntity componentEntity = componentDAO.findByName(cluster.getClusterId(),
+        serviceYARN.getName(), componentRM.getName());
+
+    ServiceComponentVersionEntity componentVersionEntity = new ServiceComponentVersionEntity();
+    componentVersionEntity.setRepositoryVersion(repositoryVersion);
+    componentVersionEntity.setUserName("admin");
+
+    componentEntity.setDesiredRepositoryVersion(repositoryVersion);
+    componentEntity.addVersion(componentVersionEntity);
+    componentEntity = componentDAO.merge(componentEntity);
+
+    // !!! make sure the override is set
+    commandRepo = helper.getCommandRepository(cluster, componentRM, host);
+
+    Assert.assertEquals(1, commandRepo.getRepositories().size());
+    repo = commandRepo.getRepositories().iterator().next();
+    Assert.assertEquals("http://foo", repo.getBaseUrl());
+
+    // verify that ZK is NOT overwritten
+    commandRepo = helper.getCommandRepository(cluster, componentZKC, host);
+
+    Assert.assertEquals(1, commandRepo.getRepositories().size());
+    repo = commandRepo.getRepositories().iterator().next();
+    Assert.assertEquals("http://public-repo-1.hortonworks.com/HDP/centos6/2.x/updates/2.0.6.0", repo.getBaseUrl());
+  }
+
   private void createClusterFixture(String clusterName, StackId stackId,
-      String respositoryVersion, String hostPrefix) throws AmbariException, AuthorizationException {
+    String respositoryVersion, String hostPrefix) throws AmbariException, AuthorizationException {
+
     String hostC6401 = hostPrefix + "-c6401";
     String hostC6402 = hostPrefix + "-c6402";
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/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 0a583ae..18eef56 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
@@ -39,10 +39,8 @@ import org.apache.ambari.server.actionmanager.HostRoleCommand;
 import org.apache.ambari.server.actionmanager.HostRoleCommandFactory;
 import org.apache.ambari.server.actionmanager.HostRoleStatus;
 import org.apache.ambari.server.agent.CommandReport;
-import org.apache.ambari.server.agent.CommandRepository;
 import org.apache.ambari.server.agent.ExecutionCommand;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
-import org.apache.ambari.server.controller.AmbariCustomCommandExecutionHelper;
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.AmbariServer;
 import org.apache.ambari.server.controller.ServiceConfigVersionResponse;
@@ -89,10 +87,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.gson.Gson;
-import com.google.gson.JsonArray;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
 import com.google.inject.Guice;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
@@ -320,7 +314,7 @@ public class UpgradeActionTest {
 
     // Create the new repo version
     String urlInfo = "[{'repositories':["
-            + "{'Repositories/base_url':'http://foo1','Repositories/repo_name':'HDP','Repositories/repo_id':'" + targetStack.getStackId() + "'}"
+            + "{'Repositories/base_url':'http://foo1','Repositories/repo_name':'HDP','Repositories/repo_id':'" + targetStack.getStackId() + "-1'}"
             + "], 'OperatingSystems/os_type':'redhat6'}]";
 
     repoVersionDAO.create(stackEntityTarget, targetRepo, String.valueOf(System.currentTimeMillis()), urlInfo);
@@ -604,13 +598,8 @@ public class UpgradeActionTest {
 
     createUpgrade(cluster, sourceStack, sourceRepo, targetRepo);
 
-    // Verify the repo before calling Finalize
-    AmbariCustomCommandExecutionHelper helper = m_injector.getInstance(AmbariCustomCommandExecutionHelper.class);
-    Host host = clusters.getHost("h1");
-
     RepositoryInfo repo = ambariMetaInfo.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
     Map<String, String> commandParams = new HashMap<>();
@@ -664,8 +653,6 @@ public class UpgradeActionTest {
     }
 
     // Verify the repo before calling Finalize
-    AmbariCustomCommandExecutionHelper helper = m_injector.getInstance(AmbariCustomCommandExecutionHelper.class);
-    Host host = clusters.getHost("h1");
     Cluster cluster = clusters.getCluster(clusterName);
 
     createUpgrade(cluster, sourceStack, sourceRepo, targetRepo);
@@ -673,7 +660,6 @@ public class UpgradeActionTest {
     RepositoryInfo repo = ambariMetaInfo.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
     Map<String, String> commandParams = new HashMap<>();
@@ -695,27 +681,6 @@ public class UpgradeActionTest {
     assertEquals(HostRoleStatus.COMPLETED.name(), report.getStatus());
   }
 
-  private void verifyBaseRepoURL(AmbariCustomCommandExecutionHelper helper, Cluster cluster, Host host, String expectedRepoBaseURL) throws AmbariException {
-
-    String repoInfo = helper.getRepoInfo(cluster, host);
-    Gson gson = new Gson();
-    JsonElement element = gson.fromJson(repoInfo, JsonElement.class);
-    assertTrue(element.isJsonArray());
-    JsonArray list = JsonArray.class.cast(element);
-    assertEquals(1, list.size());
-
-    JsonObject o = list.get(0).getAsJsonObject();
-    assertTrue(o.has("baseUrl"));
-    assertEquals(expectedRepoBaseURL, o.get("baseUrl").getAsString());
-
-    CommandRepository commandRepo = helper.getCommandRepository(cluster, host);
-
-    assertNotNull(commandRepo);
-    assertNotNull(commandRepo.getRepositories());
-    assertEquals(1, commandRepo.getRepositories().size());
-    assertEquals(expectedRepoBaseURL, commandRepo.getRepositories().iterator().next().getBaseUrl());
-  }
-
   @Test
   public void testFinalizeUpgradeAcrossStacks() throws Exception {
     StackId sourceStack = HDP_21_STACK;
@@ -958,9 +923,6 @@ public class UpgradeActionTest {
 
     createUpgradeClusterAndSourceRepo(sourceStack, sourceRepo, hostName);
 
-    // Verify the repo before calling Finalize
-    AmbariCustomCommandExecutionHelper helper = m_injector.getInstance(AmbariCustomCommandExecutionHelper.class);
-    Host host = clusters.getHost("h1");
     Cluster cluster = clusters.getCluster(clusterName);
 
     // install HDFS with some components
@@ -989,7 +951,7 @@ public class UpgradeActionTest {
 
     RepositoryInfo repo = ambariMetaInfo.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);
+//    verifyBaseRepoURL(helper, cluster, null, host, HDP_211_CENTOS6_REPO_URL);
 
     // Finalize the upgrade, passing in the request ID so that history is
     // created

http://git-wip-us.apache.org/repos/asf/ambari/blob/8782cf69/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 5206ea3..5f5576e 100644
--- a/ambari-server/src/test/python/custom_actions/TestInstallPackages.py
+++ b/ambari-server/src/test/python/custom_actions/TestInstallPackages.py
@@ -492,6 +492,76 @@ class TestInstallPackages(RMFTestCase):
     self.assertNoMoreResources()
 
 
+  @patch("ambari_commons.os_check.OSCheck.is_suse_family")
+  @patch("resource_management.core.resources.packaging.Package")
+  @patch("resource_management.libraries.script.Script.put_structured_out")
+  @patch("resource_management.libraries.functions.packages_analyzer.allInstalledPackages")
+  @patch("resource_management.libraries.functions.stack_select.get_stack_versions")
+  @patch("resource_management.libraries.functions.repo_version_history.read_actual_version_from_history_file")
+  @patch("resource_management.libraries.functions.repo_version_history.write_actual_version_to_history_file")
+  def test_format_package_name_via_repositoryFile(self, write_actual_version_to_history_file_mock,
+                               read_actual_version_from_history_file_mock,
+                               stack_versions_mock,
+                               allInstalledPackages_mock, put_structured_out_mock,
+                               package_mock, is_suse_family_mock):
+    Script.stack_version_from_distro_select = VERSION_STUB
+    stack_versions_mock.side_effect = [
+      [],  # before installation attempt
+      [VERSION_STUB]
+    ]
+    read_actual_version_from_history_file_mock.return_value = VERSION_STUB
+    allInstalledPackages_mock = MagicMock(side_effect = TestInstallPackages._add_packages)
+    is_suse_family_mock.return_value = True
+
+    
+    config_file = self.get_src_folder() + "/test/python/custom_actions/configs/install_packages_repository_file.json"
+    with open(config_file, "r") as f:
+      command_json = json.load(f)
+
+    command_json['repositoryFile']['repoVersion'] = '2.2.0.1-990'
+
+    self.executeScript("scripts/install_packages.py",
+                       classname="InstallPackages",
+                       command="actionexecute",
+                       config_dict=command_json,
+                       target=RMFTestCase.TARGET_CUSTOM_ACTIONS,
+                       os_type=('Suse', '11', 'Final'),
+                       )
+    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',
+                       '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',
+                              action=['create'],
+                              components=[u'HDP-UTILS', 'main'],
+                              repo_template=u'[{{repo_id}}]\nname={{repo_id}}\n{% if mirror_list %}mirrorlist={{mirror_list}}{% else %}baseurl={{base_url}}{% endif %}\n\npath=/\nenabled=1\ngpgcheck=0',
+                              repo_file_name=u'ambari-hdp-4',
+                              mirror_list=None,
+                              append_to_file=False,
+                              )
+    self.assertResourceCalled('Repository', 'HDP-2.2-repo-4',
+                              base_url=u'http://repo1/HDP/centos5/2.x/updates/2.2.0.0',
+                              action=['create'],
+                              components=[u'HDP', 'main'],
+                              repo_template=u'[{{repo_id}}]\nname={{repo_id}}\n{% if mirror_list %}mirrorlist={{mirror_list}}{% else %}baseurl={{base_url}}{% endif %}\n\npath=/\nenabled=1\ngpgcheck=0',
+                              repo_file_name=u'ambari-hdp-4',
+                              mirror_list=None,
+                              append_to_file=True,
+                              )
+    self.assertResourceCalled('Package', 'hdp-select', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertResourceCalled('Package', 'hadoop_2_2_0_1_990', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertResourceCalled('Package', 'snappy', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertResourceCalled('Package', 'snappy-devel', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertResourceCalled('Package', 'lzo', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertResourceCalled('Package', 'hadooplzo_2_2_0_1_990', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertResourceCalled('Package', 'hadoop_2_2_0_1_990-libhdfs', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertResourceCalled('Package', 'ambari-log4j', action=["upgrade"], retry_count=5, retry_on_repo_unavailability=False)
+    self.assertNoMoreResources()
+
+
   @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")