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

ambari git commit: AMBARI-21161 Able to safe guard Ambari server against circular dependencies in role command order (dili)

Repository: ambari
Updated Branches:
  refs/heads/trunk 1586a1a92 -> acb2f9896


AMBARI-21161 Able to safe guard Ambari server against circular dependencies in role command order (dili)


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

Branch: refs/heads/trunk
Commit: acb2f9896e55db202f5b054565d09c0e598011ed
Parents: 1586a1a
Author: Di Li <di...@apache.org>
Authored: Wed Jun 7 12:26:47 2017 -0400
Committer: Di Li <di...@apache.org>
Committed: Wed Jun 7 12:26:47 2017 -0400

----------------------------------------------------------------------
 .../ambari/server/stageplanner/RoleGraph.java   |  49 ++++-
 .../server/stageplanner/TestStagePlanner.java   | 197 +++++++++++++++++--
 .../stacks/HDP/2.0.6.1/role_command_order.json  | 101 ++--------
 3 files changed, 241 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/acb2f989/ambari-server/src/main/java/org/apache/ambari/server/stageplanner/RoleGraph.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/stageplanner/RoleGraph.java b/ambari-server/src/main/java/org/apache/ambari/server/stageplanner/RoleGraph.java
index 024a5e1..d6d26c9 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/stageplanner/RoleGraph.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/stageplanner/RoleGraph.java
@@ -19,10 +19,12 @@ package org.apache.ambari.server.stageplanner;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.Role;
 import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.actionmanager.CommandExecutionType;
@@ -165,10 +167,15 @@ public class RoleGraph {
    * Returns a list of stages that need to be executed one after another
    * to execute the DAG generated in the last {@link #build(Stage)} call.
    */
-  public List<Stage> getStages() {
+  public List<Stage> getStages() throws AmbariException {
     long initialStageId = initialStage.getStageId();
     List<Stage> stageList = new ArrayList<>();
     List<RoleGraphNode> firstStageNodes = new ArrayList<>();
+    if(!graph.isEmpty()){
+      LOG.info("Detecting cycle graphs");
+      LOG.info(stringifyGraph());
+      breakCycleGraph();
+    }
     while (!graph.isEmpty()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug(stringifyGraph());
@@ -313,4 +320,44 @@ public class RoleGraph {
     }
     return builder.toString();
   }
+
+  /**
+   * Cycle graphs indicate circular dependencies such as the following example
+   * that can cause Ambari enter an infinite loop while building stages.
+   *   (DATANODE, START, 2) --> (NAMENODE, START, 2) --> (SECONDARY_NAMENODE, START, 3)
+   *   (HDFS_CLIENT, INSTALL, 0) --> (DATANODE, START, 2) --> (NAMENODE, START, 2) --> (SECONDARY_NAMENODE, START, 3)
+   *   (NAMENODE, START, 2) --> (DATANODE, START, 2) --> (SECONDARY_NAMENODE, START, 3)
+   *   (SECONDARY_NAMENODE, START, 3)
+   * It is important to safe guard against cycle graphs,
+   * when Ambari supports mpacks, custom services and service level role command order.
+   * */
+  public void breakCycleGraph() throws AmbariException{
+    List<String> edges = new ArrayList<String>();
+    for (String role : graph.keySet()){
+      RoleGraphNode fromNode = graph.get(role);
+      String fnRole = fromNode.getRole().name();
+      String fnCommand = fromNode.getCommand().name();
+
+      Iterator<RoleGraphNode> it = fromNode.getEdges().iterator();
+      while(it.hasNext()){
+        RoleGraphNode toNode = it.next();
+        String tnRole = toNode.getRole().name();
+        String tnCommand = toNode.getCommand().name();
+        //Check if the reversed edge exists in the list already
+        //If the edit exists, print an error message and break the edge
+        String format = "%s:%s --> %s:%s";
+        String edge = String.format(format, fnRole, fnCommand, tnRole, tnCommand);
+        String reversedEdge = String.format(format, tnRole, tnCommand, fnRole, fnCommand);
+        if (edges.contains(reversedEdge)){
+          String msg = String.format(
+              "Circular dependencies detected between %s and %s for %s. "
+              + "%s already exists in the role command order.", fnRole, tnRole, edge, reversedEdge);
+          LOG.error(msg);
+          throw new AmbariException(msg);
+        } else {
+          edges.add(edge);
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/acb2f989/ambari-server/src/test/java/org/apache/ambari/server/stageplanner/TestStagePlanner.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/stageplanner/TestStagePlanner.java b/ambari-server/src/test/java/org/apache/ambari/server/stageplanner/TestStagePlanner.java
index 271ee89..ae3b316 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/stageplanner/TestStagePlanner.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/stageplanner/TestStagePlanner.java
@@ -44,13 +44,18 @@ import org.apache.ambari.server.utils.StageUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Guice;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
 
+import junit.framework.Assert;
+
 public class TestStagePlanner {
+  private static final Logger log = LoggerFactory.getLogger(TestStagePlanner.class);
 
   private Injector injector;
 
@@ -88,7 +93,7 @@ public class TestStagePlanner {
     rg.build(stage);
     List<Stage> outStages = rg.getStages();
     for (Stage s: outStages) {
-      System.out.println(s.toString());
+      log.info(s.toString());
     }
     assertEquals(1, outStages.size());
     assertEquals(stage.getExecutionCommands(hostname), outStages.get(0)
@@ -96,7 +101,165 @@ public class TestStagePlanner {
   }
 
   @Test
-  public void testMultiStagePlan() {
+  public void testSCCInGraphDetectedShort() {
+    ClusterImpl cluster = mock(ClusterImpl.class);
+    when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6.1"));
+
+    Service hbaseService = mock(Service.class);
+    when(hbaseService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6.1"));
+    Service zkService = mock(Service.class);
+    when(zkService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6.1"));
+
+    when(cluster.getServices()).thenReturn(ImmutableMap.<String, Service>builder()
+        .put("HBASE", hbaseService)
+        .put("ZOOKEEPER", zkService)
+        .build());
+
+    RoleCommandOrder rco = roleCommandOrderProvider.getRoleCommandOrder(cluster);
+    RoleGraph rg = roleGraphFactory.createNew(rco);
+    long now = System.currentTimeMillis();
+    Stage stage = StageUtils.getATestStage(1, 1, "host1", "", "");
+    stage.addHostRoleExecutionCommand("host2", Role.HBASE_MASTER,
+        RoleCommand.START, new ServiceComponentHostStartEvent("HBASE_MASTER",
+            "host2", now), "cluster1", "HBASE", false, false);
+    stage.addHostRoleExecutionCommand("host3", Role.ZOOKEEPER_SERVER,
+        RoleCommand.START, new ServiceComponentHostStartEvent("ZOOKEEPER_SERVER",
+            "host3", now), "cluster1", "ZOOKEEPER", false, false);
+    log.info("Build and ready to detect circular dependencies - short chain");
+    rg.build(stage);
+    boolean exceptionThrown = false;
+    try {
+      List<Stage> outStages = rg.getStages();
+    } catch (AmbariException e) {
+      exceptionThrown = true;
+    }
+    Assert.assertTrue(exceptionThrown);
+  }
+
+  @Test
+  public void testSCCInGraphDetectedLong() {
+    ClusterImpl cluster = mock(ClusterImpl.class);
+    when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6.1"));
+
+    Service hbaseService = mock(Service.class);
+    when(hbaseService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6.1"));
+    Service zkService = mock(Service.class);
+    when(zkService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6.1"));
+    Service yarnService = mock(Service.class);
+    when(yarnService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6.1"));
+
+    when(cluster.getServices()).thenReturn(ImmutableMap.<String, Service>builder()
+        .put("HBASE", hbaseService)
+        .put("ZOOKEEPER", zkService)
+        .put("YARN", yarnService)
+        .build());
+
+    RoleCommandOrder rco = roleCommandOrderProvider.getRoleCommandOrder(cluster);
+    RoleGraph rg = roleGraphFactory.createNew(rco);
+    long now = System.currentTimeMillis();
+    Stage stage = StageUtils.getATestStage(1, 1, "host1", "", "");
+    stage.addHostRoleExecutionCommand("host2", Role.HBASE_MASTER,
+        RoleCommand.STOP, new ServiceComponentHostStartEvent("HBASE_MASTER",
+            "host2", now), "cluster1", "HBASE", false, false);
+    stage.addHostRoleExecutionCommand("host3", Role.ZOOKEEPER_SERVER,
+        RoleCommand.STOP, new ServiceComponentHostStartEvent("ZOOKEEPER_SERVER",
+            "host3", now), "cluster1", "ZOOKEEPER", false, false);
+    stage.addHostRoleExecutionCommand("host2", Role.RESOURCEMANAGER,
+        RoleCommand.STOP, new ServiceComponentHostStartEvent("RESOURCEMANAGER",
+            "host4", now), "cluster1", "YARN", false, false);
+
+    log.info("Build and ready to detect circular dependencies - long chain");
+    rg.build(stage);
+    boolean exceptionThrown = false;
+    try {
+      List<Stage> outStages = rg.getStages();
+    } catch (AmbariException e) {
+      exceptionThrown = true;
+    }
+    Assert.assertTrue(exceptionThrown);
+  }
+
+  @Test
+  public void testSCCInGraphDetectedLongTwo() {
+    ClusterImpl cluster = mock(ClusterImpl.class);
+    when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6.1"));
+
+    Service hbaseService = mock(Service.class);
+    when(hbaseService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6.1"));
+    Service zkService = mock(Service.class);
+    when(zkService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6.1"));
+
+    when(cluster.getServices()).thenReturn(ImmutableMap.<String, Service>builder()
+        .put("HBASE", hbaseService)
+        .put("ZOOKEEPER", zkService)
+        .build());
+
+    RoleCommandOrder rco = roleCommandOrderProvider.getRoleCommandOrder(cluster);
+    RoleGraph rg = roleGraphFactory.createNew(rco);
+    long now = System.currentTimeMillis();
+    Stage stage = StageUtils.getATestStage(1, 1, "host1", "", "");
+    stage.addHostRoleExecutionCommand("host2", Role.HBASE_MASTER,
+        RoleCommand.UPGRADE, new ServiceComponentHostStartEvent("HBASE_MASTER",
+            "host2", now), "cluster1", "HBASE", false, false);
+    stage.addHostRoleExecutionCommand("host3", Role.ZOOKEEPER_SERVER,
+        RoleCommand.UPGRADE, new ServiceComponentHostStartEvent("ZOOKEEPER_SERVER",
+            "host3", now), "cluster1", "ZOOKEEPER", false, false);
+    stage.addHostRoleExecutionCommand("host2", Role.HBASE_REGIONSERVER,
+        RoleCommand.UPGRADE, new ServiceComponentHostStartEvent("HBASE_REGIONSERVER",
+            "host4", now), "cluster1", "HBASE", false, false);
+
+    log.info("Build and ready to detect circular dependencies - long chain");
+    rg.build(stage);
+    boolean exceptionThrown = false;
+    try {
+      List<Stage> outStages = rg.getStages();
+    } catch (AmbariException e) {
+      exceptionThrown = true;
+    }
+    Assert.assertTrue(exceptionThrown);
+  }
+
+  @Test
+  public void testNoSCCInGraphDetected() {
+    ClusterImpl cluster = mock(ClusterImpl.class);
+    when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6"));
+
+    Service hbaseService = mock(Service.class);
+    when(hbaseService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6"));
+    Service zkService = mock(Service.class);
+    when(zkService.getDesiredStackId()).thenReturn(new StackId("HDP-2.0.6"));
+
+    when(cluster.getServices()).thenReturn(ImmutableMap.<String, Service>builder()
+        .put("HBASE", hbaseService)
+        .put("ZOOKEEPER", zkService)
+        .build());
+
+    RoleCommandOrder rco = roleCommandOrderProvider.getRoleCommandOrder(cluster);
+    RoleGraph rg = roleGraphFactory.createNew(rco);
+    long now = System.currentTimeMillis();
+    Stage stage = StageUtils.getATestStage(1, 1, "host1", "", "");
+    stage.addHostRoleExecutionCommand("host2", Role.HBASE_MASTER,
+        RoleCommand.STOP, new ServiceComponentHostStartEvent("HBASE_MASTER",
+            "host2", now), "cluster1", "HBASE", false, false);
+    stage.addHostRoleExecutionCommand("host3", Role.ZOOKEEPER_SERVER,
+        RoleCommand.STOP, new ServiceComponentHostStartEvent("ZOOKEEPER_SERVER",
+            "host3", now), "cluster1", "ZOOKEEPER", false, false);
+    stage.addHostRoleExecutionCommand("host2", Role.HBASE_REGIONSERVER,
+        RoleCommand.STOP, new ServiceComponentHostStartEvent("HBASE_REGIONSERVER",
+            "host4", now), "cluster1", "HBASE", false, false);
+    log.info("Build and ready to detect circular dependencies");
+    rg.build(stage);
+    boolean exceptionThrown = false;
+    try {
+      List<Stage> outStages = rg.getStages();
+    } catch (AmbariException e) {
+      exceptionThrown = true;
+    }
+    Assert.assertFalse(exceptionThrown);
+  }
+
+  @Test
+  public void testMultiStagePlan() throws Throwable {
     ClusterImpl cluster = mock(ClusterImpl.class);
     when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6"));
 
@@ -121,19 +284,19 @@ public class TestStagePlanner {
     stage.addHostRoleExecutionCommand("host3", Role.ZOOKEEPER_SERVER,
         RoleCommand.START, new ServiceComponentHostStartEvent("ZOOKEEPER_SERVER",
             "host3", now), "cluster1", "ZOOKEEPER", false, false);
-    System.out.println(stage.toString());
+    log.info(stage.toString());
 
     rg.build(stage);
-    System.out.println(rg.stringifyGraph());
+    log.info(rg.stringifyGraph());
     List<Stage> outStages = rg.getStages();
     for (Stage s: outStages) {
-      System.out.println(s.toString());
+      log.info(s.toString());
     }
     assertEquals(3, outStages.size());
   }
 
   @Test
-  public void testRestartStagePlan() {
+  public void testRestartStagePlan() throws Throwable {
     ClusterImpl cluster = mock(ClusterImpl.class);
     when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6"));
 
@@ -158,19 +321,19 @@ public class TestStagePlanner {
       RoleCommand.CUSTOM_COMMAND, "cluster1",
       new ServiceComponentHostServerActionEvent("host2", System.currentTimeMillis()),
       null, "command detail", null, null, false, false);
-    System.out.println(stage.toString());
+    log.info(stage.toString());
 
     rg.build(stage);
-    System.out.println(rg.stringifyGraph());
+    log.info(rg.stringifyGraph());
     List<Stage> outStages = rg.getStages();
     for (Stage s: outStages) {
-      System.out.println(s.toString());
+      log.info(s.toString());
     }
     assertEquals(2, outStages.size());
   }
 
   @Test
-  public void testManyStages() {
+  public void testManyStages() throws Throwable {
     ClusterImpl cluster = mock(ClusterImpl.class);
     when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6"));
 
@@ -244,18 +407,18 @@ public class TestStagePlanner {
       RoleCommand.START, new ServiceComponentHostStartEvent("GANGLIA_SERVER",
         "host9", now), "cluster1", "GANGLIA", false, false);
 
-    System.out.println(stage.toString());
+    log.info(stage.toString());
     rg.build(stage);
-    System.out.println(rg.stringifyGraph());
+    log.info(rg.stringifyGraph());
     List<Stage> outStages = rg.getStages();
     for (Stage s : outStages) {
-      System.out.println(s.toString());
+      log.info(s.toString());
     }
     assertEquals(4, outStages.size());
   }
 
   @Test
-  public void testDependencyOrderedStageCreate() {
+  public void testDependencyOrderedStageCreate() throws Throwable {
     ClusterImpl cluster = mock(ClusterImpl.class);
     when(cluster.getCurrentStackVersion()).thenReturn(new StackId("HDP-2.0.6"));
     RoleCommandOrder rco = roleCommandOrderProvider.getRoleCommandOrder(cluster);
@@ -296,12 +459,12 @@ public class TestStagePlanner {
     stage.addHostRoleExecutionCommand("host9", Role.GANGLIA_SERVER,
       RoleCommand.START, new ServiceComponentHostStartEvent("GANGLIA_SERVER",
         "host9", now), "cluster1", "GANGLIA", false, false);
-    System.out.println(stage.toString());
+    log.info(stage.toString());
     rg.build(stage);
-    System.out.println(rg.stringifyGraph());
+    log.info(rg.stringifyGraph());
     List<Stage> outStages = rg.getStages();
     for (Stage s : outStages) {
-      System.out.println(s.toString());
+      log.info(s.toString());
     }
     assertEquals(1, outStages.size());
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/acb2f989/ambari-server/src/test/resources/stacks/HDP/2.0.6.1/role_command_order.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/resources/stacks/HDP/2.0.6.1/role_command_order.json b/ambari-server/src/test/resources/stacks/HDP/2.0.6.1/role_command_order.json
index 6154004..4d63fe3 100644
--- a/ambari-server/src/test/resources/stacks/HDP/2.0.6.1/role_command_order.json
+++ b/ambari-server/src/test/resources/stacks/HDP/2.0.6.1/role_command_order.json
@@ -1,92 +1,17 @@
 {
-  "_comment" : "Record format:",
-  "_comment" : "blockedRole-blockedCommand: [blockerRole1-blockerCommand1, blockerRole2-blockerCommand2, ...]",
   "general_deps" : {
-    "_comment" : "dependencies for all cases",
+    "_comment" : "circular dependencies, short link",
     "HBASE_MASTER-START": ["ZOOKEEPER_SERVER-START"],
-    "HBASE_REGIONSERVER-START": ["HBASE_MASTER-START"],
-    "OOZIE_SERVER-START": ["JOBTRACKER-START", "TASKTRACKER-START"],
-    "WEBHCAT_SERVER-START": ["TASKTRACKER-START", "HIVE_SERVER-START"],
-    "HIVE_METASTORE-START": ["MYSQL_SERVER-START"],
-    "HIVE_SERVER-START": ["TASKTRACKER-START", "MYSQL_SERVER-START"],
-    "HUE_SERVER-START": ["HIVE_SERVER-START", "HCAT-START", "OOZIE_SERVER-START"],
-    "FLUME_HANDLER-START": ["OOZIE_SERVER-START"],
-    "MAPREDUCE_SERVICE_CHECK-SERVICE_CHECK": ["JOBTRACKER-START", "TASKTRACKER-START"],
-    "OOZIE_SERVICE_CHECK-SERVICE_CHECK": ["OOZIE_SERVER-START", "MAPREDUCE2_SERVICE_CHECK-SERVICE_CHECK"],
-    "WEBHCAT_SERVICE_CHECK-SERVICE_CHECK": ["WEBHCAT_SERVER-START"],
-    "HBASE_SERVICE_CHECK-SERVICE_CHECK": ["HBASE_MASTER-START", "HBASE_REGIONSERVER-START"],
-    "HIVE_SERVICE_CHECK-SERVICE_CHECK": ["HIVE_SERVER-START", "HIVE_METASTORE-START"],
-    "HCAT_SERVICE_CHECK-SERVICE_CHECK": ["HIVE_SERVER-START"],
-    "PIG_SERVICE_CHECK-SERVICE_CHECK": ["JOBTRACKER-START", "TASKTRACKER-START"],
-    "SQOOP_SERVICE_CHECK-SERVICE_CHECK": ["JOBTRACKER-START", "TASKTRACKER-START"],
-    "ZOOKEEPER_SERVICE_CHECK-SERVICE_CHECK": ["ZOOKEEPER_SERVER-START"],
-    "ZOOKEEPER_QUORUM_SERVICE_CHECK-SERVICE_CHECK": ["ZOOKEEPER_SERVER-START"],
-    "ZOOKEEPER_SERVER-STOP" : ["HBASE_MASTER-STOP", "HBASE_REGIONSERVER-STOP"],
-    "HBASE_MASTER-STOP": ["HBASE_REGIONSERVER-STOP"],
-    "TASKTRACKER-UPGRADE": ["JOBTRACKER-UPGRADE"],
-    "MAPREDUCE_CLIENT-UPGRADE": ["TASKTRACKER-UPGRADE", "JOBTRACKER-UPGRADE"],
-    "ZOOKEEPER_SERVER-UPGRADE": ["MAPREDUCE_CLIENT-UPGRADE"],
-    "ZOOKEEPER_CLIENT-UPGRADE": ["ZOOKEEPER_SERVER-UPGRADE"],
-    "HBASE_MASTER-UPGRADE": ["ZOOKEEPER_CLIENT-UPGRADE"],
-    "HBASE_REGIONSERVER-UPGRADE": ["HBASE_MASTER-UPGRADE"],
-    "HBASE_CLIENT-UPGRADE": ["HBASE_REGIONSERVER-UPGRADE"],
-    "HIVE_SERVER-UPGRADE" : ["HBASE_CLIENT-UPGRADE"],
-    "HIVE_METASTORE-UPGRADE" : ["HIVE_SERVER-UPGRADE"],
-    "MYSQL_SERVER-UPGRADE": ["HIVE_METASTORE-UPGRADE"],
-    "HIVE_CLIENT-UPGRADE": ["MYSQL_SERVER-UPGRADE"],
-    "HCAT-UPGRADE": ["HIVE_CLIENT-UPGRADE"],
-    "OOZIE_SERVER-UPGRADE" : ["HCAT-UPGRADE"],
-    "OOZIE_CLIENT-UPGRADE" : ["OOZIE_SERVER-UPGRADE"],
-    "WEBHCAT_SERVER-UPGRADE" : ["OOZIE_CLIENT-UPGRADE"],
-    "PIG-UPGRADE" : ["WEBHCAT_SERVER-UPGRADE"],
-    "SQOOP-UPGRADE" : ["PIG-UPGRADE"],
-    "GANGLIA_SERVER-UPGRADE" : ["SQOOP-UPGRADE"],
-    "GANGLIA_MONITOR-UPGRADE" : ["GANGLIA_SERVER-UPGRADE"]
-  },
-  "_comment" : "GLUSTERFS-specific dependencies",
-  "optional_glusterfs": {
-    "HBASE_MASTER-START": ["PEERSTATUS-START"],
-    "JOBTRACKER-START": ["PEERSTATUS-START"],
-    "TASKTRACKER-START": ["PEERSTATUS-START"],
-    "GLUSTERFS_SERVICE_CHECK-SERVICE_CHECK": ["PEERSTATUS-START"],
-    "JOBTRACKER-UPGRADE": ["GLUSTERFS_CLIENT-UPGRADE"]
-  },
-  "_comment" : "Dependencies that are used when GLUSTERFS is not present in cluster",
-  "optional_no_glusterfs": {
-    "SECONDARY_NAMENODE-START": ["NAMENODE-START"],
-    "RESOURCEMANAGER-START": ["NAMENODE-START", "DATANODE-START"],
-    "NODEMANAGER-START": ["NAMENODE-START", "DATANODE-START", "RESOURCEMANAGER-START"],
-    "HISTORYSERVER-START": ["NAMENODE-START", "DATANODE-START"],
-    "HBASE_MASTER-START": ["NAMENODE-START", "DATANODE-START"],
-    "JOBTRACKER-START": ["NAMENODE-START", "DATANODE-START"],
-    "TASKTRACKER-START": ["NAMENODE-START", "DATANODE-START"],
-    "HIVE_SERVER-START": ["DATANODE-START"],
-    "WEBHCAT_SERVER-START": ["DATANODE-START"],
-    "HDFS_SERVICE_CHECK-SERVICE_CHECK": ["NAMENODE-START", "DATANODE-START",
-        "SECONDARY_NAMENODE-START"],
-    "MAPREDUCE2_SERVICE_CHECK-SERVICE_CHECK": ["NODEMANAGER-START",
-        "RESOURCEMANAGER-START", "HISTORYSERVER-START", "YARN_SERVICE_CHECK-SERVICE_CHECK"],
-    "YARN_SERVICE_CHECK-SERVICE_CHECK": ["NODEMANAGER-START", "RESOURCEMANAGER-START"],
-    "RESOURCEMANAGER_SERVICE_CHECK-SERVICE_CHECK": ["RESOURCEMANAGER-START"],
-    "PIG_SERVICE_CHECK-SERVICE_CHECK": ["RESOURCEMANAGER-START", "NODEMANAGER-START"],
-    "NAMENODE-STOP": ["JOBTRACKER-STOP", "TASKTRACKER-STOP", "RESOURCEMANAGER-STOP",
-        "NODEMANAGER-STOP", "HISTORYSERVER-STOP", "HBASE_MASTER-STOP"],
-    "DATANODE-STOP": ["JOBTRACKER-STOP", "TASKTRACKER-STOP", "RESOURCEMANAGER-STOP",
-        "NODEMANAGER-STOP", "HISTORYSERVER-STOP", "HBASE_MASTER-STOP"],
-    "SECONDARY_NAMENODE-UPGRADE": ["NAMENODE-UPGRADE"],
-    "DATANODE-UPGRADE": ["SECONDARY_NAMENODE-UPGRADE"],
-    "HDFS_CLIENT-UPGRADE": ["DATANODE-UPGRADE"],
-    "JOBTRACKER-UPGRADE": ["HDFS_CLIENT-UPGRADE"]
-  },
-  "_comment" : "Dependencies that are used in HA NameNode cluster",
-  "namenode_optional_ha": {
-    "NAMENODE-START": ["JOURNALNODE-START", "ZOOKEEPER_SERVER-START"],
-    "ZKFC-START": ["NAMENODE-START"],
-    "HDFS_SERVICE_CHECK-SERVICE_CHECK": ["ZKFC-START"]
-  },
-  "_comment" : "Dependencies that are used in ResourceManager HA cluster",
-  "resourcemanager_optional_ha" : {
-    "RESOURCEMANAGER-START": ["ZOOKEEPER_SERVER-START"]
-  }
-}
+    "ZOOKEEPER_SERVER-START": ["HBASE_MASTER-START"],
+
+    "_comment" : "circular dependencies, long link",
+    "ZOOKEEPER_SERVER-STOP" : ["HBASE_MASTER-STOP"],
+    "HBASE_MASTER-STOP": ["RESOURCEMANAGER-STOP"],
+    "RESOURCEMANAGER-STOP": ["ZOOKEEPER_SERVER-STOP"],
 
+    "_comment" : "circular dependencies, long link",
+    "ZOOKEEPER_SERVER-UPGRADE" : ["HBASE_MASTER-UPGRADE"],
+    "HBASE_MASTER-UPGRADE": ["HBASE_REGIONSERVER-UPGRADE"],
+    "HBASE_REGIONSERVER-UPGRADE": ["ZOOKEEPER_SERVER-UPGRADE"]
+  }
+}
\ No newline at end of file