You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ma...@apache.org on 2016/06/16 12:07:48 UTC

ambari git commit: AMBARI-17251. Client components that are dependencies of services in the stack definitions are always added to blueprint deployments. (Laszlo Puskas via magyari_sandor)

Repository: ambari
Updated Branches:
  refs/heads/branch-2.4 3f28b93ef -> 27a01bbfc


AMBARI-17251. Client components that are dependencies of services in the stack definitions are always added to blueprint deployments. (Laszlo Puskas via magyari_sandor)


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

Branch: refs/heads/branch-2.4
Commit: 27a01bbfcb8d91db21ba097baf6ead718f2437d9
Parents: 3f28b93
Author: Laszlo Puskas <lp...@hortonworks.com>
Authored: Thu Jun 16 13:57:35 2016 +0200
Committer: Sandor Magyari <sm...@hortonworks.com>
Committed: Thu Jun 16 14:05:01 2016 +0200

----------------------------------------------------------------------
 .../server/topology/BlueprintValidatorImpl.java |  68 ++++++++-----
 .../topology/BlueprintValidatorImplTest.java    | 100 +++++++++++++++----
 2 files changed, 122 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/27a01bbf/ambari-server/src/main/java/org/apache/ambari/server/topology/BlueprintValidatorImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/topology/BlueprintValidatorImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/topology/BlueprintValidatorImpl.java
index 432c6f8..de5e2b3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/topology/BlueprintValidatorImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/topology/BlueprintValidatorImpl.java
@@ -18,23 +18,26 @@
 
 package org.apache.ambari.server.topology;
 
-import org.apache.ambari.server.controller.internal.Stack;
-import org.apache.ambari.server.state.AutoDeployInfo;
-import org.apache.ambari.server.state.DependencyInfo;
-import org.apache.ambari.server.utils.SecretReference;
-import org.apache.ambari.server.utils.VersionUtils;
-
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 
+import org.apache.ambari.server.controller.internal.Stack;
+import org.apache.ambari.server.state.AutoDeployInfo;
+import org.apache.ambari.server.state.DependencyInfo;
+import org.apache.ambari.server.utils.SecretReference;
+import org.apache.ambari.server.utils.VersionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * Default blueprint validator.
  */
 public class BlueprintValidatorImpl implements BlueprintValidator {
 
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlueprintValidatorImpl.class);
   private final Blueprint blueprint;
   private final Stack stack;
 
@@ -44,11 +47,10 @@ public class BlueprintValidatorImpl implements BlueprintValidator {
   }
   @Override
   public void validateTopology() throws InvalidTopologyException {
+    LOGGER.info("Validating topology for blueprint: [{}]", blueprint.getName());
     Collection<HostGroup> hostGroups = blueprint.getHostGroups().values();
-    Map<String, Map<String, Collection<DependencyInfo>>> missingDependencies =
-        new HashMap<String, Map<String, Collection<DependencyInfo>>>();
+    Map<String, Map<String, Collection<DependencyInfo>>> missingDependencies = new HashMap<String, Map<String, Collection<DependencyInfo>>>();
 
-    Collection<String> services = blueprint.getServices();
     for (HostGroup group : hostGroups) {
       Map<String, Collection<DependencyInfo>> missingGroupDependencies = validateHostGroup(group);
       if (! missingGroupDependencies.isEmpty()) {
@@ -57,6 +59,8 @@ public class BlueprintValidatorImpl implements BlueprintValidator {
     }
 
     Collection<String> cardinalityFailures = new HashSet<String>();
+    Collection<String> services = blueprint.getServices();
+
     for (String service : services) {
       for (String component : stack.getComponents(service)) {
         Cardinality cardinality = stack.getCardinality(component);
@@ -130,22 +134,22 @@ public class BlueprintValidatorImpl implements BlueprintValidator {
         if (component.equals("HIVE_METASTORE")) {
           Map<String, String> hiveEnvConfig = clusterConfigurations.get("hive-env");
           if (hiveEnvConfig != null && !hiveEnvConfig.isEmpty() && hiveEnvConfig.get("hive_database") !=null
-                  && hiveEnvConfig.get("hive_database").equals("Existing SQL Anywhere Database")
-                  && VersionUtils.compareVersions(stack.getVersion(), "2.3.0.0") < 0
-                  && stack.getName().equalsIgnoreCase("HDP")) {
+              && hiveEnvConfig.get("hive_database").equals("Existing SQL Anywhere Database")
+              && VersionUtils.compareVersions(stack.getVersion(), "2.3.0.0") < 0
+              && stack.getName().equalsIgnoreCase("HDP")) {
             throw new InvalidTopologyException("Incorrect configuration: SQL Anywhere db is available only for stack HDP-2.3+ " +
-                    "and repo version 2.3.2+!");
+                "and repo version 2.3.2+!");
           }
         }
 
         if (component.equals("OOZIE_SERVER")) {
           Map<String, String> oozieEnvConfig = clusterConfigurations.get("oozie-env");
           if (oozieEnvConfig != null && !oozieEnvConfig.isEmpty() && oozieEnvConfig.get("oozie_database") !=null
-                  && oozieEnvConfig.get("oozie_database").equals("Existing SQL Anywhere Database")
-                  && VersionUtils.compareVersions(stack.getVersion(), "2.3.0.0") < 0
-                  && stack.getName().equalsIgnoreCase("HDP")) {
+              && oozieEnvConfig.get("oozie_database").equals("Existing SQL Anywhere Database")
+              && VersionUtils.compareVersions(stack.getVersion(), "2.3.0.0") < 0
+              && stack.getName().equalsIgnoreCase("HDP")) {
             throw new InvalidTopologyException("Incorrect configuration: SQL Anywhere db is available only for stack HDP-2.3+ " +
-                    "and repo version 2.3.2+!");
+                "and repo version 2.3.2+!");
           }
         }
 
@@ -224,16 +228,26 @@ public class BlueprintValidatorImpl implements BlueprintValidator {
   }
 
   private Map<String, Collection<DependencyInfo>> validateHostGroup(HostGroup group) {
-    Map<String, Collection<DependencyInfo>> missingDependencies =
-        new HashMap<String, Collection<DependencyInfo>>();
-
-    Collection<String> blueprintServices = blueprint.getServices();
-    Collection<String> groupComponents = group.getComponentNames();
-    for (String component : new HashSet<String>(groupComponents)) {
-      Collection<DependencyInfo> dependenciesForComponent = stack.getDependenciesForComponent(component);
-      for (DependencyInfo dependency : dependenciesForComponent) {
+    LOGGER.info("Validating hostgroup: {}", group.getName());
+    Map<String, Collection<DependencyInfo>> missingDependencies = new HashMap<String, Collection<DependencyInfo>>();
+
+    for (String component : new HashSet<String>(group.getComponentNames())) {
+      LOGGER.debug("Processing component: {}", component);
+
+      for (DependencyInfo dependency : stack.getDependenciesForComponent(component)) {
+        LOGGER.debug("Processing dependency [{}] for component [{}]", dependency.getName(), component);
+
         String conditionalService = stack.getConditionalServiceForDependency(dependency);
-        if (conditionalService != null && ! blueprintServices.contains(conditionalService)) {
+        if (conditionalService != null && !blueprint.getServices().contains(conditionalService)) {
+          LOGGER.debug("Conditional service  [{}] is missing from the blueprint, skipping dependency [{}]",
+              conditionalService, dependency.getName());
+          continue;
+        }
+
+        // dependent components from the stack definitions are only added if related services are explicitly added to the blueprint!
+        if (!blueprint.getServices().contains(dependency.getServiceName())) {
+          LOGGER.debug("The service [{}] for component [{}] is missing from the blueprint [{}], skipping dependency",
+              dependency.getServiceName(), dependency.getComponentName(), blueprint.getName());
           continue;
         }
 
@@ -248,7 +262,7 @@ public class BlueprintValidatorImpl implements BlueprintValidator {
 
           resolved = missingDependencyInfo.isEmpty();
         } else if (dependencyScope.equals("host")) {
-          if (groupComponents.contains(component) || (autoDeployInfo != null && autoDeployInfo.isEnabled())) {
+          if (group.getComponentNames().contains(component) || (autoDeployInfo != null && autoDeployInfo.isEnabled())) {
             resolved = true;
             group.addComponent(componentName);
           }

http://git-wip-us.apache.org/repos/asf/ambari/blob/27a01bbf/ambari-server/src/test/java/org/apache/ambari/server/topology/BlueprintValidatorImplTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/topology/BlueprintValidatorImplTest.java b/ambari-server/src/test/java/org/apache/ambari/server/topology/BlueprintValidatorImplTest.java
index f78d86d..b85d454 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/topology/BlueprintValidatorImplTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/topology/BlueprintValidatorImplTest.java
@@ -18,13 +18,6 @@
 
 package org.apache.ambari.server.topology;
 
-import org.apache.ambari.server.controller.internal.Stack;
-import org.apache.ambari.server.state.AutoDeployInfo;
-import org.apache.ambari.server.state.DependencyInfo;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -33,7 +26,17 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
-import static org.easymock.EasyMock.createNiceMock;
+import org.apache.ambari.server.controller.internal.Stack;
+import org.apache.ambari.server.state.AutoDeployInfo;
+import org.apache.ambari.server.state.DependencyInfo;
+import org.easymock.EasyMockRule;
+import org.easymock.Mock;
+import org.easymock.MockType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.reset;
@@ -44,23 +47,33 @@ import static org.easymock.EasyMock.verify;
  */
 public class BlueprintValidatorImplTest{
 
-  private final Blueprint blueprint = createNiceMock(Blueprint.class);
-  private final Stack stack = createNiceMock(Stack.class);
-  private final HostGroup group1 = createNiceMock(HostGroup.class);
-  private final HostGroup group2 = createNiceMock(HostGroup.class);
   private final Map<String, HostGroup> hostGroups = new LinkedHashMap<>();
+  @Rule
+  public EasyMockRule mocks = new EasyMockRule(this);
+
+  @Mock(type = MockType.NICE)
+  private Blueprint blueprint;
+
+  @Mock(type = MockType.NICE)
+  private Stack stack;
+
+  @Mock(type = MockType.NICE)
+  private HostGroup group1;
+
+  @Mock(type = MockType.NICE)
+  private HostGroup group2;
+
+  @Mock(type = MockType.NICE)
+  private DependencyInfo dependency1;
 
   private final Collection<String> group1Components = new ArrayList<String>();
   private final Collection<String> group2Components = new ArrayList<String>();
   private final Collection<String> services = new ArrayList<String>();
-  private final DependencyInfo dependency1 = createNiceMock(DependencyInfo.class);
-  private Collection<DependencyInfo> dependencies1 = new ArrayList<DependencyInfo>();
 
+  private Collection<DependencyInfo> dependencies1 = new ArrayList<DependencyInfo>();
   private AutoDeployInfo autoDeploy = new AutoDeployInfo();
-
-  Map<String, Map<String, String>> configProperties = new HashMap<String, Map<String, String>>();
-  private Configuration configuration = new Configuration(configProperties,
-      Collections.<String, Map<String, Map<String, String>>>emptyMap());
+  private Map<String, Map<String, String>> configProperties = new HashMap<String, Map<String, String>>();
+  private Configuration configuration = new Configuration(configProperties, Collections.<String, Map<String, Map<String, String>>>emptyMap());
 
 
   @Before
@@ -76,7 +89,9 @@ public class BlueprintValidatorImplTest{
     expect(blueprint.getServices()).andReturn(services).anyTimes();
 
     expect(group1.getComponentNames()).andReturn(group1Components).anyTimes();
+    expect(group1.getName()).andReturn("host-group-1").anyTimes();
     expect(group2.getComponentNames()).andReturn(group2Components).anyTimes();
+    expect(group2.getName()).andReturn("host-group-2").anyTimes();
 
     expect(stack.getDependenciesForComponent("component1")).andReturn(dependencies1).anyTimes();
     expect(stack.getDependenciesForComponent("component2")).andReturn(dependencies1).anyTimes();
@@ -163,17 +178,21 @@ public class BlueprintValidatorImplTest{
     expect(stack.getComponents("service2")).andReturn(Collections.singleton("component3")).anyTimes();
     expect(stack.getAutoDeployInfo("component1")).andReturn(autoDeploy).anyTimes();
 
-    expect(dependency1.getScope()).andReturn("host").anyTimes();
     AutoDeployInfo dependencyAutoDeploy = new AutoDeployInfo();
     dependencyAutoDeploy.setEnabled(true);
     dependencyAutoDeploy.setCoLocate("service1/component1");
+
+    expect(dependency1.getScope()).andReturn("host").anyTimes();
     expect(dependency1.getAutoDeploy()).andReturn(dependencyAutoDeploy).anyTimes();
     expect(dependency1.getComponentName()).andReturn("component3").anyTimes();
+    expect(dependency1.getServiceName()).andReturn("service1").anyTimes();
+    expect(dependency1.getName()).andReturn("dependency1").anyTimes();
 
     expect(group1.addComponent("component1")).andReturn(true).once();
     expect(group1.addComponent("component3")).andReturn(true).once();
 
     replay(blueprint, stack, group1, group2, dependency1);
+
     BlueprintValidator validator = new BlueprintValidatorImpl(blueprint);
     validator.validateTopology();
 
@@ -225,4 +244,47 @@ public class BlueprintValidatorImplTest{
     BlueprintValidator validator = new BlueprintValidatorImpl(blueprint);
     validator.validateRequiredProperties();
   }
+
+  @Test
+  public void testShouldDependencyBeExcludedWenRelatedServiceIsNotInBlueprint() throws Exception {
+    // GIVEN
+    hostGroups.clear();
+    hostGroups.put("group1", group1);
+
+    group1Components.add("component-1");
+    dependencies1.add(dependency1);
+    services.addAll(Collections.singleton("service-1"));
+
+
+    expect(blueprint.getHostGroupsForComponent("component-1")).andReturn(Arrays.asList(group1)).anyTimes();
+    expect(blueprint.getName()).andReturn("blueprint-1").anyTimes();
+
+    Cardinality cardinality = new Cardinality("1");
+
+    expect(stack.getComponents("service-1")).andReturn(Arrays.asList("component-1")).anyTimes();
+    expect(stack.getAutoDeployInfo("component-1")).andReturn(autoDeploy).anyTimes();
+    expect(stack.getDependenciesForComponent("component-1")).andReturn(dependencies1).anyTimes();
+    expect(stack.getCardinality("component-1")).andReturn(cardinality).anyTimes();
+
+
+    AutoDeployInfo dependencyAutoDeploy = new AutoDeployInfo();
+    dependencyAutoDeploy.setEnabled(true);
+    dependencyAutoDeploy.setCoLocate("service1/component1");
+
+    expect(dependency1.getScope()).andReturn("host").anyTimes();
+    expect(dependency1.getAutoDeploy()).andReturn(dependencyAutoDeploy).anyTimes();
+    expect(dependency1.getComponentName()).andReturn("component-d").anyTimes();
+    expect(dependency1.getServiceName()).andReturn("service-d").anyTimes();
+    expect(dependency1.getName()).andReturn("dependency-1").anyTimes();
+
+    replay(blueprint, stack, group1, group2, dependency1);
+
+    // WHEN
+    BlueprintValidator validator = new BlueprintValidatorImpl(blueprint);
+    validator.validateTopology();
+
+    // THEN
+    verify(group1);
+
+  }
 }