You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/05/30 22:10:36 UTC

[GitHub] [helix] xyuanlu commented on a change in pull request #1043: Code clean up Topology.java

xyuanlu commented on a change in pull request #1043:
URL: https://github.com/apache/helix/pull/1043#discussion_r432891174



##########
File path: helix-core/src/main/java/org/apache/helix/controller/rebalancer/topology/Topology.java
##########
@@ -212,123 +185,104 @@ private static Node cloneTree(Node root, Map<Node, Integer> newNodeWeight, Set<N
     return newRoot;
   }
 
-  /**
-   * Creates a tree representing the cluster structure using default cluster topology definition
-   * (i,e no topology definition given and no domain id set).
-   */
-  private Node createClusterTreeWithDefaultTopologyDef() {
+  private Node createClusterTree() {
     // root
     Node root = new Node();
     root.setName("root");
     root.setId(computeId("root"));
     root.setType(Types.ROOT.name());
 
-    for (String ins : _allInstances) {
-      InstanceConfig config = _instanceConfigMap.get(ins);
-      Map<String, String> pathValueMap = new HashMap<>();
-      if (_topologyAwareEnabled) {
-        String zone = config.getZoneId();
+    for (String instance : _allInstances) {
+      InstanceConfig insConfig = _instanceConfigMap.get(instance);
+      LinkedHashMap<String, String> instanceTopologyMap = new LinkedHashMap<>();
+      if (computeInstanceTopologyMap(_clusterConfig, instance, insConfig, instanceTopologyMap)) {
+        int weight = insConfig.getWeight();
+        if (weight < 0 || weight == InstanceConfig.WEIGHT_NOT_SET) {
+          weight = DEFAULT_NODE_WEIGHT;
+        }
+        addEndNode(root, instance, instanceTopologyMap, weight, _liveInstances);
+      }
+    }
+    return root;
+  }
+
+  private boolean computeInstanceTopologyMap(ClusterConfig clusterConfig, String instance,
+      InstanceConfig instanceConfig, LinkedHashMap<String, String> instanceTopologyMap) {
+    if (clusterConfig.isTopologyAwareEnabled()) {
+      String topologyDef = clusterConfig.getTopology();
+      if (topologyDef == null) {
+        // Return a map using default cluster topology definition, i,e. /root/zone/instance
+        String zone = instanceConfig.getZoneId();
         if (zone == null) {
           // we have the hierarchy style of domain id for instance.
-          if (config.getInstanceEnabled() && (_clusterConfig.getDisabledInstances() == null
-              || !_clusterConfig.getDisabledInstances().containsKey(ins))) {
+          if (instanceConfig.getInstanceEnabled() && (clusterConfig.getDisabledInstances() == null
+              || !clusterConfig.getDisabledInstances().containsKey(instance))) {
             // if enabled instance missing ZONE_ID information, fails the rebalance.
             throw new HelixException(String
-                .format("ZONE_ID for instance %s is not set, failed the topology-aware placement!",
-                    ins));
+                .format("ZONE_ID for instance %s is not set, fail the topology-aware placement!",
+                    instance));
           } else {
             // if the disabled instance missing ZONE setting, ignore it should be fine.
             logger.warn(String
-                .format("ZONE_ID for instance %s is not set, failed the topology-aware placement!",
-                    ins));
-            continue;
+                .format("ZONE_ID for instance %s is not set, ignore the instance!", instance));
+            return false;
           }
-
         }
-        pathValueMap.put(Types.ZONE.name(), zone);
-      }
-      pathValueMap.put(Types.INSTANCE.name(), ins);
-      int weight = config.getWeight();
-      if (weight < 0 || weight == InstanceConfig.WEIGHT_NOT_SET) {
-        weight = DEFAULT_NODE_WEIGHT;
-      }
-      root = addEndNode(root, ins, pathValueMap, weight, _liveInstances);
-    }
-    return root;
-  }
-
-  /**
-   * Creates a tree representing the cluster structure using default cluster topology definition
-   * (i,e no topology definition given and no domain id set).
-   */
-  private Node createClusterTreeWithCustomizedTopology() {
-    // root
-    Node root = new Node();
-    root.setName("root");
-    root.setId(computeId("root"));
-    root.setType(Types.ROOT.name());
-
-    for (String ins : _allInstances) {
-      InstanceConfig insConfig = _instanceConfigMap.get(ins);
-      String domain = insConfig.getDomainAsString();
-      if (domain == null) {
-        if (insConfig.getInstanceEnabled() && (_clusterConfig.getDisabledInstances() == null
-            || !_clusterConfig.getDisabledInstances().containsKey(ins))) {
-          // if enabled instance missing domain information, fails the rebalance.
-          throw new HelixException(String
-              .format("Domain for instance %s is not set, failed the topology-aware placement!",
-                  ins));
-        } else {
-          // if the disabled instance missing domain setting, ignore it should be fine.
-          logger
-              .warn(String.format("Domain for instance %s is not set, ignore the instance!", ins));
-          continue;
+        instanceTopologyMap.put(Types.ZONE.name(), zone);
+        instanceTopologyMap.put(Types.INSTANCE.name(), instance);
+      } else {
+        /*
+         * Return a map representing the cluster structure using cluster topology defined in
+         * TOPOLOGY in ClusterConfig.
+         */
+        String domain = instanceConfig.getDomainAsString();
+        if (domain == null || domain.isEmpty()) {
+          if (instanceConfig.getInstanceEnabled() && (clusterConfig.getDisabledInstances() == null
+              || !clusterConfig.getDisabledInstances().containsKey(instance))) {
+            // if enabled instance missing domain information, fails the rebalance.
+            throw new HelixException(String
+                .format("Domain for instance %s is not set, fail the topology-aware placement!",
+                    instance));
+          } else {
+            // if the disabled instance missing domain setting, ignore it should be fine.
+            logger.warn(
+                String.format("Domain for instance %s is not set, ignore the instance!", instance));
+            return false;
+          }
         }
-      }
-
-      String[] pathPairs = domain.trim().split(",");
-      Map<String, String> pathValueMap = new HashMap<>();
-      for (String pair : pathPairs) {
-        String[] values = pair.trim().split("=");
-        if (values.length != 2 || values[0].isEmpty() || values[1].isEmpty()) {
+        Map<String, String> domainAsMap;
+        try {
+          domainAsMap = instanceConfig.getDomainAsMap();
+        } catch (IllegalArgumentException e) {
           throw new HelixException(String.format(
-              "Domain-Value pair %s for instance %s is not valid, failed the topology-aware placement!",
-              pair, ins));
+              "Domain %s for instance %s is not valid, fail the topology-aware placement!",
+              domain, instance));
         }
-        String type = values[0];
-        String value = values[1];
-
-        if (!_types.contains(type)) {
-          logger.warn(String
-              .format("Path %s defined in domain of instance %s not recognized, ignored!", pair,
-                  ins));
-          continue;
+        String[] topologyKeys = topologyDef.trim().split("/");
+        for (String key : topologyKeys) {
+          if (!key.isEmpty()) {
+            // if a key does not exist in the instance domain config, apply the default domain value.
+            instanceTopologyMap.put(key, domainAsMap.getOrDefault(key, "Helix_default_" + key));
+          }
         }
-        pathValueMap.put(type, value);
-      }
-
-      int weight = insConfig.getWeight();
-      if (weight < 0 || weight == InstanceConfig.WEIGHT_NOT_SET) {
-        weight = DEFAULT_NODE_WEIGHT;
       }
-      root = addEndNode(root, ins, pathValueMap, weight, _liveInstances);
+    } else {
+      instanceTopologyMap.put(Types.INSTANCE.name(), instance);
     }
-    return root;
+    return true;
   }
 
-
   /**
    * Add an end node to the tree, create all the paths to the leaf node if not present.
    */
-  private Node addEndNode(Node root, String instanceName, Map<String, String> pathNameMap,
+  private void addEndNode(Node root, String instanceName, LinkedHashMap<String, String> pathNameMap,

Review comment:
       Please correct me if I am wrong. I think Map does not guarantee the order in which pairs were inserted into the map. When constructing the topology tree, it is important to maintain the original order for the path. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org