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

[34/45] ambari git commit: AMBARI-20802. StackAdvisor should not recommend configuration types for services that are not present in the cluster.(vbrodetskyi)

AMBARI-20802. StackAdvisor should not recommend configuration types for services that are not present in the cluster.(vbrodetskyi)


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

Branch: refs/heads/branch-feature-AMBARI-12556
Commit: 8d3cc471e2ac460bc64bbbb22b079f873d9f5e66
Parents: b299641
Author: Vitaly Brodetskyi <vb...@hortonworks.com>
Authored: Fri Apr 21 18:15:30 2017 +0300
Committer: Vitaly Brodetskyi <vb...@hortonworks.com>
Committed: Fri Apr 21 18:15:30 2017 +0300

----------------------------------------------------------------------
 .../ambari/server/topology/AmbariContext.java   |  3 +
 .../HIVE/0.12.0.2.0/configuration/hive-env.xml  |  1 +
 .../HIVE/0.12.0.2.0/package/scripts/hive.py     | 25 +++++---
 .../stacks/HDP/2.1/services/stack_advisor.py    | 11 ++--
 .../stacks/HDP/2.2/services/stack_advisor.py    | 61 ++++++++++----------
 .../stacks/2.1/HIVE/test_hive_metastore.py      |  7 +++
 .../stacks/2.1/common/test_stack_advisor.py     |  7 ++-
 .../stacks/2.6/common/test_stack_advisor.py     |  6 +-
 8 files changed, 76 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/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 6d12402..5e0d707 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
@@ -427,6 +427,9 @@ public class AmbariContext {
       for (String actualConfigType : updatedConfigTypes) {
         // get the actual cluster config for comparison
         DesiredConfig actualConfig = cluster.getDesiredConfigs().get(actualConfigType);
+        if (actualConfig == null && actualConfigType.equals("core-site")) {
+          continue;
+        }
         if (!actualConfig.getTag().equals(TopologyManager.TOPOLOGY_RESOLVED_TAG)) {
           // if any expected config is not resolved, deployment must wait
           LOG.info("Config type " + actualConfigType + " not resolved yet, Blueprint deployment will wait until configuration update is completed");

http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-env.xml b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-env.xml
index 5f56617..caa598a 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-env.xml
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-env.xml
@@ -220,6 +220,7 @@
     <display-name>hive-env template</display-name>
     <description>This is the jinja template for hive-env.sh file</description>
     <value>
+ export JAVA_HOME={{java64_home}}
  if [ "$SERVICE" = "cli" ]; then
    if [ -z "$DEBUG" ]; then
      export HADOOP_OPTS="$HADOOP_OPTS -XX:NewRatio=12 -XX:MaxHeapFreeRatio=40 -XX:MinHeapFreeRatio=15 -XX:+UseParNewGC -XX:-UseGCOverheadLimit"

http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
index b7b04a2..959e111 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
@@ -300,6 +300,15 @@ def setup_metastore():
        content=StaticFile('startMetastore.sh')
   )
 
+  if not is_empty(params.hive_exec_scratchdir):
+    dirPathStr = urlparse(params.hive_exec_scratchdir).path
+    pathComponents = dirPathStr.split("/")
+    if dirPathStr.startswith("/tmp") and len(pathComponents) > 2:
+      Directory (params.hive_exec_scratchdir,
+                 owner = params.hive_user,
+                 create_parents = True,
+                 mode=0777)
+
 def create_metastore_schema():
   import params
 
@@ -345,14 +354,14 @@ def fill_conf_dir(component_conf_dir):
             mode=mode_identified_for_dir
   )
 
-  XmlConfig("mapred-site.xml",
-            conf_dir=component_conf_dir,
-            configurations=params.config['configurations']['mapred-site'],
-            configuration_attributes=params.config['configuration_attributes']['mapred-site'],
-            owner=params.hive_user,
-            group=params.user_group,
-            mode=mode_identified_for_file)
-
+  if 'mapred-site' in params.config['configurations']:
+    XmlConfig("mapred-site.xml",
+              conf_dir=component_conf_dir,
+              configurations=params.config['configurations']['mapred-site'],
+              configuration_attributes=params.config['configuration_attributes']['mapred-site'],
+              owner=params.hive_user,
+              group=params.user_group,
+              mode=mode_identified_for_file)
 
   File(format("{component_conf_dir}/hive-default.xml.template"),
        owner=params.hive_user,

http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
index 866d4cb..ddd795f 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
@@ -210,12 +210,13 @@ class HDP21StackAdvisor(HDP206StackAdvisor):
             webHcatSitePropertyAttributes("webhcat.proxyuser.{0}.hosts".format(old_ambari_user), 'delete', 'true')
             webHcatSitePropertyAttributes("webhcat.proxyuser.{0}.groups".format(old_ambari_user), 'delete', 'true')
 
-    if self.is_secured_cluster(services):
-      putCoreSiteProperty = self.putProperty(configurations, "core-site", services)
+    if "HDFS" in servicesList or "YARN" in servicesList:
+      if self.is_secured_cluster(services):
+        putCoreSiteProperty = self.putProperty(configurations, "core-site", services)
 
-      meta = self.get_service_component_meta("HIVE", "WEBHCAT_SERVER", services)
-      if "hostnames" in meta:
-        self.put_proxyuser_value("HTTP", meta["hostnames"], services=services, configurations=configurations, put_function=putCoreSiteProperty)
+        meta = self.get_service_component_meta("HIVE", "WEBHCAT_SERVER", services)
+        if "hostnames" in meta:
+          self.put_proxyuser_value("HTTP", meta["hostnames"], services=services, configurations=configurations, put_function=putCoreSiteProperty)
 
   def recommendTezConfigurations(self, configurations, clusterData, services, hosts):
     putTezProperty = self.putProperty(configurations, "tez-site")

http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
index 4d672d2..726514b 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
@@ -422,36 +422,37 @@ class HDP22StackAdvisor(HDP21StackAdvisor):
 
     container_size = "512"
 
-    if not "yarn-site" in configurations:
-      self.recommendYARNConfigurations(configurations, clusterData, services, hosts)
-    #properties below should be always present as they are provided in HDP206 stack advisor at least
-    yarnMaxAllocationSize = min(30 * int(configurations["yarn-site"]["properties"]["yarn.scheduler.minimum-allocation-mb"]), int(configurations["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]))
-    #duplicate tez task resource calc logic, direct dependency doesn't look good here (in case of Hive without Tez)
-    container_size = clusterData['mapMemory'] if clusterData['mapMemory'] > 2048 else int(clusterData['reduceMemory'])
-    container_size = min(clusterData['containers'] * clusterData['ramPerContainer'], container_size, yarnMaxAllocationSize)
-
-    putHiveSiteProperty("hive.tez.container.size", min(int(configurations["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]), container_size))
-
-    putHiveSitePropertyAttribute("hive.tez.container.size", "minimum", int(configurations["yarn-site"]["properties"]["yarn.scheduler.minimum-allocation-mb"]))
-    putHiveSitePropertyAttribute("hive.tez.container.size", "maximum", int(configurations["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]))
-
-    if "yarn-site" in services["configurations"]:
-      if "yarn.scheduler.minimum-allocation-mb" in services["configurations"]["yarn-site"]["properties"]:
-        putHiveSitePropertyAttribute("hive.tez.container.size", "minimum", int(services["configurations"]["yarn-site"]["properties"]["yarn.scheduler.minimum-allocation-mb"]))
-      if "yarn.scheduler.maximum-allocation-mb" in services["configurations"]["yarn-site"]["properties"]:
-        putHiveSitePropertyAttribute("hive.tez.container.size", "maximum", int(services["configurations"]["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]))
-
-    putHiveSiteProperty("hive.prewarm.enabled", "false")
-    putHiveSiteProperty("hive.prewarm.numcontainers", "3")
-    putHiveSiteProperty("hive.tez.auto.reducer.parallelism", "true")
-    putHiveSiteProperty("hive.tez.dynamic.partition.pruning", "true")
-
-    container_size = configurations["hive-site"]["properties"]["hive.tez.container.size"]
-    container_size_bytes = int(int(container_size)*0.8*1024*1024) # Xmx == 80% of container
-    # Memory
-    putHiveSiteProperty("hive.auto.convert.join.noconditionaltask.size", int(round(container_size_bytes/3)))
-    putHiveSitePropertyAttribute("hive.auto.convert.join.noconditionaltask.size", "maximum", container_size_bytes)
-    putHiveSiteProperty("hive.exec.reducers.bytes.per.reducer", "67108864")
+    if "YARN" in servicesList:
+      if not "yarn-site" in configurations:
+        self.recommendYARNConfigurations(configurations, clusterData, services, hosts)
+      #properties below should be always present as they are provided in HDP206 stack advisor at least
+      yarnMaxAllocationSize = min(30 * int(configurations["yarn-site"]["properties"]["yarn.scheduler.minimum-allocation-mb"]), int(configurations["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]))
+      #duplicate tez task resource calc logic, direct dependency doesn't look good here (in case of Hive without Tez)
+      container_size = clusterData['mapMemory'] if clusterData['mapMemory'] > 2048 else int(clusterData['reduceMemory'])
+      container_size = min(clusterData['containers'] * clusterData['ramPerContainer'], container_size, yarnMaxAllocationSize)
+
+      putHiveSiteProperty("hive.tez.container.size", min(int(configurations["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]), container_size))
+
+      putHiveSitePropertyAttribute("hive.tez.container.size", "minimum", int(configurations["yarn-site"]["properties"]["yarn.scheduler.minimum-allocation-mb"]))
+      putHiveSitePropertyAttribute("hive.tez.container.size", "maximum", int(configurations["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]))
+
+      if "yarn-site" in services["configurations"]:
+        if "yarn.scheduler.minimum-allocation-mb" in services["configurations"]["yarn-site"]["properties"]:
+          putHiveSitePropertyAttribute("hive.tez.container.size", "minimum", int(services["configurations"]["yarn-site"]["properties"]["yarn.scheduler.minimum-allocation-mb"]))
+        if "yarn.scheduler.maximum-allocation-mb" in services["configurations"]["yarn-site"]["properties"]:
+          putHiveSitePropertyAttribute("hive.tez.container.size", "maximum", int(services["configurations"]["yarn-site"]["properties"]["yarn.scheduler.maximum-allocation-mb"]))
+
+      putHiveSiteProperty("hive.prewarm.enabled", "false")
+      putHiveSiteProperty("hive.prewarm.numcontainers", "3")
+      putHiveSiteProperty("hive.tez.auto.reducer.parallelism", "true")
+      putHiveSiteProperty("hive.tez.dynamic.partition.pruning", "true")
+
+      container_size = configurations["hive-site"]["properties"]["hive.tez.container.size"]
+      container_size_bytes = int(int(container_size)*0.8*1024*1024) # Xmx == 80% of container
+      # Memory
+      putHiveSiteProperty("hive.auto.convert.join.noconditionaltask.size", int(round(container_size_bytes/3)))
+      putHiveSitePropertyAttribute("hive.auto.convert.join.noconditionaltask.size", "maximum", container_size_bytes)
+      putHiveSiteProperty("hive.exec.reducers.bytes.per.reducer", "67108864")
 
     # CBO
     if "hive-site" in services["configurations"] and "hive.cbo.enable" in services["configurations"]["hive-site"]["properties"]:

http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/ambari-server/src/test/python/stacks/2.1/HIVE/test_hive_metastore.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.1/HIVE/test_hive_metastore.py b/ambari-server/src/test/python/stacks/2.1/HIVE/test_hive_metastore.py
index 94f639a..e34734c 100644
--- a/ambari-server/src/test/python/stacks/2.1/HIVE/test_hive_metastore.py
+++ b/ambari-server/src/test/python/stacks/2.1/HIVE/test_hive_metastore.py
@@ -688,6 +688,13 @@ class TestHiveMetastore(RMFTestCase):
         content = StaticFile('startMetastore.sh'),
         mode = 0755,
     )
+
+    self.assertResourceCalled('Directory', '/tmp/hive',
+                              owner = 'hive',
+                              create_parents = True,
+                              mode=0777
+                              )
+
     self.assertResourceCalled('Execute', ('cp',
      '--remove-destination',
      '/usr/share/java/mysql-connector-java.jar',

http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py b/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
index e8bd5d0..bdd34fc 100644
--- a/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
+++ b/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
@@ -272,6 +272,11 @@ class TestHDP21StackAdvisor(TestCase):
       "services": [
         {
           "StackServices": {
+            "service_name": "YARN"
+          }, "components": []
+        },
+        {
+          "StackServices": {
             "service_name": "HIVE",
           },
           "components": [
@@ -338,7 +343,7 @@ class TestHDP21StackAdvisor(TestCase):
     self.assertEqual(configurations["core-site"]["properties"]["hadoop.proxyuser.HTTP.hosts"] == "example.com", True)
 
     newhost_list = ["example.com", "example.org"]
-    services["services"][0]["components"][0]["StackServiceComponents"]["hostnames"] = newhost_list
+    services["services"][1]["components"][0]["StackServiceComponents"]["hostnames"] = newhost_list
     configurations["core-site"]["properties"]["hadoop.proxyuser.HTTP.hosts"] = ""
 
     self.stackAdvisor.recommendHiveConfigurations(configurations, clusterData, services, hosts)

http://git-wip-us.apache.org/repos/asf/ambari/blob/8d3cc471/ambari-server/src/test/python/stacks/2.6/common/test_stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.6/common/test_stack_advisor.py b/ambari-server/src/test/python/stacks/2.6/common/test_stack_advisor.py
index 2d7322d..114dd3a 100644
--- a/ambari-server/src/test/python/stacks/2.6/common/test_stack_advisor.py
+++ b/ambari-server/src/test/python/stacks/2.6/common/test_stack_advisor.py
@@ -1012,7 +1012,11 @@ class TestHDP26StackAdvisor(TestCase):
 
     services = {
       "services":
-        [
+        [{
+           "StackServices": {
+             "service_name": "YARN"
+           }, "components": []
+         },
          {
             "StackServices": {
               "service_name" : "HIVE",