You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ds...@apache.org on 2015/04/01 23:08:54 UTC

ambari git commit: AMBARI-10316 Stack-advisor mechanism when recommending configuration-dependencies has incorrect services.json content (dsen)

Repository: ambari
Updated Branches:
  refs/heads/trunk 5da97da85 -> e508a5fbf


AMBARI-10316 Stack-advisor mechanism when recommending configuration-dependencies has incorrect services.json content (dsen)


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

Branch: refs/heads/trunk
Commit: e508a5fbf8209ca119e56167c6269b1b6cb0f898
Parents: 5da97da
Author: Dmytro Sen <ds...@apache.org>
Authored: Thu Apr 2 00:07:08 2015 +0300
Committer: Dmytro Sen <ds...@apache.org>
Committed: Thu Apr 2 00:07:33 2015 +0300

----------------------------------------------------------------------
 .../server/api/services/AmbariMetaInfo.java     | 38 -----------
 .../commands/StackAdvisorCommand.java           | 12 ++--
 ...StackLevelConfigurationResourceProvider.java |  6 +-
 .../src/main/resources/scripts/stack_advisor.py | 44 +-----------
 .../src/main/resources/stacks/stack_advisor.py  | 71 +++++++++++++++++++-
 .../server/api/services/AmbariMetaInfoTest.java | 12 ----
 .../stacks/2.2/common/test_stack_advisor.py     | 38 +----------
 7 files changed, 80 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/e508a5fb/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
index 3a7fb6a..3e37c3c 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
@@ -1224,42 +1224,4 @@ public class AmbariMetaInfo {
     return kerberosServiceDescriptors;
   }
 
-  /**
-   * Get set of all the depended-by properties down to directed acyclic graph(DAG)
-   * of dependencies between configuration properties
-   * @param stackName the stack name
-   * @param stackVersion the stack version
-   * @param changedConfigs the list of changed configurations
-   * @return set of all depended-by properties including all changedConfigs
-   */
-  public Set<PropertyDependencyInfo> getDependedByProperties(String stackName,
-                                                             String stackVersion,
-                                                             List<PropertyDependencyInfo> changedConfigs) {
-    StackInfo stack = getStackManager().getStack(stackName, stackVersion);
-
-    if (changedConfigs == null) {
-      return Collections.emptySet();
-    }
-    int size = 0;
-    Set<PropertyDependencyInfo> configs =
-      new HashSet<PropertyDependencyInfo>();
-
-    configs.addAll(changedConfigs);
-
-    while (size != configs.size()) {
-      size = configs.size();
-      for (ServiceInfo service: stack.getServices()) {
-        for (PropertyInfo pi: service.getProperties()) {
-          String type = ConfigHelper.fileNameToConfigType(pi.getFilename());
-          String name = pi.getName();
-          PropertyDependencyInfo dep =
-            new PropertyDependencyInfo(type, name);
-          if (configs.contains(dep) && !configs.containsAll(pi.getDependedByProperties())) {
-            configs.addAll(pi.getDependedByProperties());
-          }
-        }
-      }
-    }
-    return configs;
-  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/e508a5fb/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/commands/StackAdvisorCommand.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/commands/StackAdvisorCommand.java b/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/commands/StackAdvisorCommand.java
index a7646a0..ce67002 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/commands/StackAdvisorCommand.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/commands/StackAdvisorCommand.java
@@ -71,10 +71,13 @@ public abstract class StackAdvisorCommand<T extends StackAdvisorResponse> extend
 
   private static final String GET_HOSTS_INFO_URI = "/api/v1/hosts"
       + "?fields=Hosts&Hosts/host_name.in(%s)";
-  private static final String GET_SERVICES_INFO_URI = "/api/v1/stacks/%s/versions/%s"
+  private static final String GET_SERVICES_INFO_URI = "/api/v1/stacks/%s/versions/%s/"
       + "?fields=Versions/stack_name,Versions/stack_version,Versions/parent_stack_version"
       + ",services/StackServices/service_name,services/StackServices/service_version"
       + ",services/components/StackServiceComponents,services/components/dependencies,services/components/auto_deploy"
+      + ",services/configurations/StackConfigurations/property_depends_on"
+      + ",services/configurations/StackConfigurations/property_depended_by"
+      + ",services/configurations/StackConfigurations/type"
       + "&services/StackServices/service_name.in(%s)";
   private static final String SERVICES_PROPERTY = "services";
   private static final String SERVICES_COMPONENTS_PROPERTY = "components";
@@ -83,7 +86,6 @@ public abstract class StackAdvisorCommand<T extends StackAdvisorResponse> extend
   private static final String COMPONENT_HOSTNAMES_PROPERTY = "hostnames";
   private static final String CONFIGURATIONS_PROPERTY = "configurations";
   private static final String CHANGED_CONFIGURATIONS_PROPERTY = "changed-configurations";
-  private static final String DEPENDED_CONFIGURATIONS_PROPERTY = "depended-configurations";
 
   private File recommendationsDir;
   private String stackAdvisorScript;
@@ -174,15 +176,9 @@ public abstract class StackAdvisorCommand<T extends StackAdvisorResponse> extend
         }
       }
     }
-    // Populating changed configs info and depended by configs info
-    Set<PropertyDependencyInfo> dependedProperties =
-      metaInfo.getDependedByProperties(request.getStackName(),
-      request.getStackVersion(), request.getChangedConfigurations());
 
     JsonNode changedConfigs = mapper.valueToTree(request.getChangedConfigurations());
-    JsonNode dependendConfigs = mapper.valueToTree(dependedProperties);
     root.put(CHANGED_CONFIGURATIONS_PROPERTY, changedConfigs);
-    root.put(DEPENDED_CONFIGURATIONS_PROPERTY, dependendConfigs);
   }
 
   protected void populateStackHierarchy(ObjectNode root) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/e508a5fb/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackLevelConfigurationResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackLevelConfigurationResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackLevelConfigurationResourceProvider.java
index f010b37..1e5109c 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackLevelConfigurationResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackLevelConfigurationResourceProvider.java
@@ -55,13 +55,13 @@ public class StackLevelConfigurationResourceProvider extends
       .getPropertyId("StackLevelConfigurations", "property_value");
 
   public static final String PROPERTY_VALUE_ATTRIBUTES_PROPERTY_ID = PropertyHelper
-      .getPropertyId("StackConfigurations", "property_value_attributes");
+      .getPropertyId("StackLevelConfigurations", "property_value_attributes");
 
   public static final String DEPENDS_ON_PROPERTY_ID = PropertyHelper
-      .getPropertyId("StackConfigurations", "property_depends_on");
+      .getPropertyId("StackLevelConfigurations", "property_depends_on");
 
   public static final String DEPENDED_BY_PROPERTY_ID = PropertyHelper
-      .getPropertyId("StackConfigurations", "property_depended_by");
+      .getPropertyId("StackLevelConfigurations", "property_depended_by");
 
   public static final String PROPERTY_DESCRIPTION_PROPERTY_ID = PropertyHelper
       .getPropertyId("StackLevelConfigurations", "property_description");

http://git-wip-us.apache.org/repos/asf/ambari/blob/e508a5fb/ambari-server/src/main/resources/scripts/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/scripts/stack_advisor.py b/ambari-server/src/main/resources/scripts/stack_advisor.py
index 1d67872..446086c 100755
--- a/ambari-server/src/main/resources/scripts/stack_advisor.py
+++ b/ambari-server/src/main/resources/scripts/stack_advisor.py
@@ -51,7 +51,7 @@ def loadJson(path):
     with open(path, 'r') as f:
       return json.load(f)
   except Exception, err:
-    raise StackAdvisorException("File not found at: {0}".format(hostsFile))
+    raise StackAdvisorException("File not found at: {0}".format(path))
 
 
 def dumpJson(json_object, dump_file):
@@ -107,9 +107,8 @@ def main(argv=None):
     result = stackAdvisor.recommendConfigurations(services, hosts)
     result_file = os.path.join(actionDir, "configurations.json")
   elif action == RECOMMEND_CONFIGURATION_DEPENDENCIES:
-    result = stackAdvisor.recommendConfigurations(services, hosts)
+    result = stackAdvisor.recommendConfigurationDependencies(services, hosts)
     result_file = os.path.join(actionDir, "configurations.json")
-    result = filterResult(result, services)
   else: # action == VALIDATE_CONFIGURATIONS
     result = stackAdvisor.validateConfigurations(services, hosts)
     result_file = os.path.join(actionDir, "configurations-validation.json")
@@ -117,45 +116,6 @@ def main(argv=None):
   dumpJson(result, result_file)
   pass
 
-# returns recommendations only for changed and depended properties
-def filterResult(result, services):
-  allRequestedProperties = getAllRequestedProperties(services)
-
-  configs = result['recommendations']['blueprint']['configurations']
-  filteredConfigs = {}
-  for type, names in configs.items():
-    for name in names['properties']:
-      if type in allRequestedProperties.keys() and \
-              name in allRequestedProperties[type]:
-        if type not in filteredConfigs.keys():
-          filteredConfigs[type] = {'properties': {}}
-        filteredConfigs[type]['properties'][name] = \
-          configs[type]['properties'][name]
-    if 'property_attributes' in names.keys():
-      for name in names['property_attributes']:
-        if type in allRequestedProperties.keys() and \
-                name in allRequestedProperties[type]:
-          if type not in filteredConfigs.keys():
-            filteredConfigs[type] = {'property_Attributes': {}}
-          elif 'property_attributes' not in filteredConfigs[type].keys():
-            filteredConfigs[type]['property_attributes'] = {}
-          filteredConfigs[type]['property_attributes'][name] = \
-            configs[type]['property_attributes'][name]
-
-  result['recommendations']['blueprint']['configurations'] = filteredConfigs
-  return result
-
-def getAllRequestedProperties(services):
-  changedConfigs = []
-  changedConfigs.extend(services['changed-configurations'])
-  changedConfigs.extend(services['depended-configurations'])
-  allRequestedProperties = {}
-  for config in changedConfigs:
-    if config['type'] in allRequestedProperties:
-      allRequestedProperties[config['type']].append(config['name'])
-    else:
-      allRequestedProperties[config['type']] = [config['name']]
-  return allRequestedProperties
 
 def instantiateStackAdvisor(stackName, stackVersion, parentVersions):
   """Instantiates StackAdvisor implementation for the specified Stack"""

http://git-wip-us.apache.org/repos/asf/ambari/blob/e508a5fb/ambari-server/src/main/resources/stacks/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/stack_advisor.py b/ambari-server/src/main/resources/stacks/stack_advisor.py
index c19a7ce..4da8abb 100644
--- a/ambari-server/src/main/resources/stacks/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/stack_advisor.py
@@ -18,6 +18,8 @@ limitations under the License.
 """
 
 import socket
+import re
+
 
 class StackAdvisor(object):
   """
@@ -613,4 +615,71 @@ class DefaultStackAdvisor(StackAdvisor):
         for component in service['components']:
           if component["StackServiceComponents"]["component_name"] == componentName:
             return component["StackServiceComponents"]["hostnames"]
-  pass
\ No newline at end of file
+  pass
+
+  def recommendConfigurationDependencies(self, services, hosts):
+    result = self.recommendConfigurations(services, hosts)
+    return self.filterResult(result, services)
+
+  # returns recommendations only for changed and depended properties
+  def filterResult(self, result, services):
+    allRequestedProperties = self.getAllRequestedProperties(services)
+
+    configs = result['recommendations']['blueprint']['configurations']
+    filteredConfigs = {}
+    for type, names in configs.items():
+      for name in names['properties']:
+        if type in allRequestedProperties.keys() and \
+                name in allRequestedProperties[type]:
+          if type not in filteredConfigs.keys():
+            filteredConfigs[type] = {'properties': {}}
+          filteredConfigs[type]['properties'][name] = \
+            configs[type]['properties'][name]
+      if 'property_attributes' in names.keys():
+        for name in names['property_attributes']:
+          if type in allRequestedProperties.keys() and \
+                  name in allRequestedProperties[type]:
+            if type not in filteredConfigs.keys():
+              filteredConfigs[type] = {'property_Attributes': {}}
+            elif 'property_attributes' not in filteredConfigs[type].keys():
+              filteredConfigs[type]['property_attributes'] = {}
+            filteredConfigs[type]['property_attributes'][name] = \
+              configs[type]['property_attributes'][name]
+
+    result['recommendations']['blueprint']['configurations'] = filteredConfigs
+    return result
+
+  def getAllRequestedProperties(self, services):
+    affectedConfigs = self.getAffectedConfigs(services)
+    allRequestedProperties = {}
+    for config in affectedConfigs:
+      if config['type'] in allRequestedProperties:
+        allRequestedProperties[config['type']].append(config['name'])
+      else:
+        allRequestedProperties[config['type']] = [config['name']]
+    return allRequestedProperties
+
+  def getAffectedConfigs(self, services):
+    """returns properties dict including changed-configurations and depended-by configs"""
+    changedConfigs = services['changed-configurations']
+    allDependencies = []
+
+    for item in services['services']:
+      allDependencies.extend(item['configurations'])
+
+    dependencies = []
+    dependencies.extend(changedConfigs)
+
+    size = 0
+    while size != len(dependencies):
+      size = len(dependencies)
+      for config in allDependencies:
+        type = re.sub('\.xml$', '', config['StackConfigurations']['type'])
+        name = config['StackConfigurations']['property_name']
+
+        if {"type": type, "name": name} in dependencies:
+          for dependedConfig in config['StackConfigurations']['property_depended_by']:
+            if dependedConfig not in dependencies:
+              dependencies.append(dependedConfig)
+
+    return  dependencies

http://git-wip-us.apache.org/repos/asf/ambari/blob/e508a5fb/ambari-server/src/test/java/org/apache/ambari/server/api/services/AmbariMetaInfoTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/api/services/AmbariMetaInfoTest.java b/ambari-server/src/test/java/org/apache/ambari/server/api/services/AmbariMetaInfoTest.java
index 22bece8..ec61af8 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/api/services/AmbariMetaInfoTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/api/services/AmbariMetaInfoTest.java
@@ -818,18 +818,6 @@ public class AmbariMetaInfoTest {
       originalProperty.getDescription());
     Assert.assertEquals(6, redefinedService.getConfigDependencies().size());
     Assert.assertEquals(7, redefinedService.getConfigDependenciesWithComponents().size());
-
-    // Test directed-acyclic-graph (DAG) of dependencies between configurations
-    List<PropertyDependencyInfo> changedConfigs = new LinkedList<PropertyDependencyInfo>();
-    String type = ConfigHelper.fileNameToConfigType(newProperty.getFilename());
-    String name = newProperty.getName();
-    changedConfigs.add(new PropertyDependencyInfo(type, name));
-    Set<PropertyDependencyInfo> dependedByProperties = metaInfo.getDependedByProperties(stackInfo.getName(), stackInfo.getVersion(), changedConfigs);
-    Assert.assertEquals(3, dependedByProperties.size());
-    Assert.assertTrue(dependedByProperties.contains(new PropertyDependencyInfo("yarn-site", "new-enhanced-yarn-property2")));
-    Assert.assertTrue(dependedByProperties.contains(new PropertyDependencyInfo("yarn-site", "new-enhanced-yarn-property")));
-    Assert.assertTrue(dependedByProperties.contains(new PropertyDependencyInfo("yarn-site", "new-yarn-property")));
-
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/ambari/blob/e508a5fb/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py b/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
index 76ecc96..d12c067 100644
--- a/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
+++ b/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
@@ -778,45 +778,9 @@ class TestHDP22StackAdvisor(TestCase):
           "type": "yarn-env",
           "name": "min_user_id"
         },
-      ],
-      "depended-configurations": [
-        {
-          "type" : "mapred-site",
-          "name" : "yarn.app.mapreduce.am.admin-command-opts"
-        }, {
-          "type" : "yarn-site",
-          "name" : "yarn.scheduler.maximum-allocation-mb"
-        }, {
-          "type" : "yarn-site",
-          "name" : "yarn.scheduler.minimum-allocation-mb"
-        }, {
-          "type" : "mapred-site",
-          "name" : "mapreduce.reduce.java.opts"
-        }, {
-          "type" : "mapred-site",
-          "name" : "mapreduce.map.java.opts"
-        }, {
-          "type" : "mapred-site",
-          "name" : "yarn.app.mapreduce.am.command-opts"
-        }, {
-          "type" : "mapred-site",
-          "name" : "yarn.app.mapreduce.am.resource.mb"
-        }, {
-          "type" : "yarn-site",
-          "name" : "yarn.nodemanager.resource.memory-mb"
-        }, {
-          "type" : "mapred-site",
-          "name" : "mapreduce.task.io.sort.mb"
-        }, {
-          "type" : "mapred-site",
-          "name" : "mapreduce.reduce.memory.mb"
-        }, {
-          "type" : "mapred-site",
-          "name" : "mapreduce.map.memory.mb"
-        }
       ]
 
-      }
+    }
     hosts = {
       "items" : [
         {